2006-03-26 23:42:36

by Esben Nielsen

[permalink] [raw]
Subject: PI patch against 2.6.16-rt9

Hi,
I got the patch I mentioned earlier to run. It passes my userspace
testscripts as well as all the scripts for Thomas's rtmutex-tester on a UP
machine.

The idea is to avoid the deadlocks by releasing all locks before going to
the next lock in the chain. I use get_/put_task_struct to avoid the task
disappearing during the iteration.

Why change it:
1) Simpler code.
2) Shorter latencies: We don't traverse the lock chain with spinlocks
held. They are released along the way opening for preemption.

You can say that preemption while going through the chain is bad as it
opens up a lot of issues. But I can not think of any bad cases. There are
a few corner cases where the PI boosting/unboosting can be preempted. But
that is not a problem for the determinism which is the very aim of having PI
in the first place.

Esben


diff -upr linux-2.6.16-rt9/include/linux/rtmutex.h linux-2.6.16-rt9.pipatch/include/linux/rtmutex.h
--- linux-2.6.16-rt9/include/linux/rtmutex.h 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/include/linux/rtmutex.h 2006-03-27 01:10:00.000000000 +0200
@@ -107,8 +107,7 @@ extern void rt_mutex_unlock(struct rt_mu
#ifdef CONFIG_RT_MUTEXES
# define INIT_RT_MUTEXES(tsk) \
.pi_waiters = PLIST_HEAD_INIT(tsk.pi_waiters, tsk.pi_lock), \
- .pi_lock = RAW_SPIN_LOCK_UNLOCKED, \
- .pi_lock_chain = LIST_HEAD_INIT(tsk.pi_lock_chain),
+ .pi_lock = RAW_SPIN_LOCK_UNLOCKED,
#else
# define INIT_RT_MUTEXES(tsk)
#endif
diff -upr linux-2.6.16-rt9/include/linux/sched.h linux-2.6.16-rt9.pipatch/include/linux/sched.h
--- linux-2.6.16-rt9/include/linux/sched.h 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/include/linux/sched.h 2006-03-27 01:02:41.000000000 +0200
@@ -985,9 +985,6 @@ struct task_struct {
struct plist_head pi_waiters;
/* Deadlock detection and priority inheritance handling */
struct rt_mutex_waiter *pi_blocked_on;
- /* PI locking helpers */
- struct task_struct *pi_locked_by;
- struct list_head pi_lock_chain;
#endif

#ifdef CONFIG_DEBUG_MUTEXES
diff -upr linux-2.6.16-rt9/include/linux/spinlock_api_smp.h linux-2.6.16-rt9.pipatch/include/linux/spinlock_api_smp.h
--- linux-2.6.16-rt9/include/linux/spinlock_api_smp.h 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/include/linux/spinlock_api_smp.h 2006-03-27 01:02:41.000000000 +0200
@@ -39,6 +39,7 @@ int __lockfunc _raw_read_trylock(raw_rwl
int __lockfunc _raw_write_trylock(raw_rwlock_t *lock);
int __lockfunc _raw_spin_trylock_irqsave(raw_spinlock_t *lock,
unsigned long *flags);
+int __lockfunc _raw_spin_trylock_irq(raw_spinlock_t *lock);
int __lockfunc _raw_spin_trylock_bh(raw_spinlock_t *lock);
void __lockfunc _raw_spin_unlock(raw_spinlock_t *lock) __releases(raw_spinlock_t);
void __lockfunc _raw_spin_unlock_no_resched(raw_spinlock_t *lock) __releases(raw_spinlock_t);
diff -upr linux-2.6.16-rt9/include/linux/spinlock_api_up.h linux-2.6.16-rt9.pipatch/include/linux/spinlock_api_up.h
--- linux-2.6.16-rt9/include/linux/spinlock_api_up.h 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/include/linux/spinlock_api_up.h 2006-03-27 01:02:41.000000000 +0200
@@ -39,6 +39,9 @@
#define __TRYLOCK_IRQSAVE(lock, flags) \
({ local_irq_save(*(flags)); __LOCK(lock); 1; })

+#define __TRYLOCK_IRQ(lock) \
+ ({ raw_local_irq_disable(); __LOCK(lock); 1; })
+
#define _raw_spin_trylock_irqsave(lock, flags) __TRYLOCK_IRQSAVE(lock, flags)

#define __UNLOCK(lock) \
@@ -75,6 +78,7 @@
#define _raw_read_trylock_bh(lock) ({ __LOCK_BH(lock); 1; })
#define _raw_write_trylock_bh(lock) ({ __LOCK_BH(lock); 1; })
#define _raw_spin_trylock_irqsave(lock, flags) __TRYLOCK_IRQSAVE(lock, flags)
+#define _raw_spin_trylock_irq(lock) __TRYLOCK_IRQ(lock)
#define _raw_read_trylock_irqsave(lock, flags) __TRYLOCK_IRQSAVE(lock, flags)
#define _raw_read_trylock_irqsave(lock, flags) __TRYLOCK_IRQSAVE(lock, flags)
#define _raw_spin_unlock(lock) __UNLOCK(lock)
diff -upr linux-2.6.16-rt9/include/linux/sysctl.h linux-2.6.16-rt9.pipatch/include/linux/sysctl.h
--- linux-2.6.16-rt9/include/linux/sysctl.h 2006-03-27 01:04:20.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/include/linux/sysctl.h 2006-03-27 01:02:41.000000000 +0200
@@ -148,6 +148,7 @@ enum
KERN_SPIN_RETRY=70, /* int: number of spinlock retries */
KERN_ACPI_VIDEO_FLAGS=71, /* int: flags for setting up video after ACPI sleep */
KERN_IA64_UNALIGNED=72, /* int: ia64 unaligned userland trap enable */
+ KERN_MAX_LOCK_DEPTH=73
};


diff -upr linux-2.6.16-rt9/kernel/fork.c linux-2.6.16-rt9.pipatch/kernel/fork.c
--- linux-2.6.16-rt9/kernel/fork.c 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/kernel/fork.c 2006-03-27 01:13:09.000000000 +0200
@@ -947,8 +947,6 @@ static inline void rt_mutex_init_task(st
spin_lock_init(&p->pi_lock);
plist_head_init(&p->pi_waiters, &p->pi_lock);
p->pi_blocked_on = NULL;
- p->pi_locked_by = NULL;
- INIT_LIST_HEAD(&p->pi_lock_chain);
#endif
}

diff -upr linux-2.6.16-rt9/kernel/rtmutex-debug.c linux-2.6.16-rt9.pipatch/kernel/rtmutex-debug.c
--- linux-2.6.16-rt9/kernel/rtmutex-debug.c 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/kernel/rtmutex-debug.c 2006-03-27 01:02:41.000000000 +0200
@@ -398,9 +398,7 @@ restart:
void notrace rt_mutex_debug_task_free(struct task_struct *tsk)
{
WARN_ON(!plist_head_empty(&tsk->pi_waiters));
- WARN_ON(!list_empty(&tsk->pi_lock_chain));
WARN_ON(tsk->pi_blocked_on);
- WARN_ON(tsk->pi_locked_by);
}

/*
diff -upr linux-2.6.16-rt9/kernel/rtmutex.c linux-2.6.16-rt9.pipatch/kernel/rtmutex.c
--- linux-2.6.16-rt9/kernel/rtmutex.c 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/kernel/rtmutex.c 2006-03-27 01:05:48.000000000 +0200
@@ -161,301 +161,176 @@ static void rt_mutex_adjust_prio(struct
}

/*
- * PI-locking: we lock PI-dependencies opportunistically via trylock.
+ * Task blocks on lock.
*
- * In the overwhelming majority of cases the 'PI chain' is empty or at
- * most 1-2 entries long, for which the trylock is sufficient,
- * scalability-wise. The locking might look a bit scary, for which we
- * apologize in advance :-)
- *
- * If any of the trylocks fails then we back out, task the global
- * pi_conflicts_lock and take the locks again. This ensures deadlock-free
- * but still scalable locking in the dependency graph, combined with
- * the ability to reliably (and cheaply) detect user-space deadlocks.
- */
-static DEFINE_RAW_SPINLOCK(pi_conflicts_lock);
-
-/*
- * Lock the full boosting chain.
+ * Prepare waiter and potentially propagate our priority into the pi chain.
*
- * If 'try' is set, we have to backout if we hit a owner who is
- * running its own pi chain operation. We go back and take the slow
- * path via the pi_conflicts_lock.
- *
- * We put all held locks into a list, via ->pi_lock_chain, and walk
- * this list at unlock_pi_chain() time.
- */
-static int lock_pi_chain(struct rt_mutex *act_lock,
- struct rt_mutex_waiter *waiter,
- struct list_head *lock_chain,
- int try, int detect_deadlock)
-{
- struct task_struct *owner;
- struct rt_mutex *nextlock, *lock = act_lock;
- struct rt_mutex_waiter *nextwaiter;
-
- /*
- * Debugging might turn deadlock detection on, unconditionally:
- */
- detect_deadlock = debug_rt_mutex_detect_deadlock(detect_deadlock);
+ * This must be called with lock->wait_lock held.
+ */

- for (;;) {
- owner = rt_mutex_owner(lock);

- /* Check for circular dependencies */
- if (unlikely(owner->pi_locked_by == current)) {
- debug_rt_mutex_deadlock(detect_deadlock, waiter, lock);
- return detect_deadlock ? -EDEADLK : 0;
- }
+int max_lock_depth = 100;

- while (!spin_trylock(&owner->pi_lock)) {
- /*
- * Owner runs its own chain. Go back and take
- * the slow path
- */
- if (try && owner->pi_locked_by == owner)
- return -EBUSY;
- cpu_relax();
- }
+/*
+ * Adjust the priority chain. Also used for deadlock detection.
+ * Decreases task's usage by one - may thus free the task.
+ * Returns 0 or -EDEADLK.
+ */
+static int rt_mutex_adjust_prio_chain(task_t *task,
+ int deadlock_detect,
+ struct rt_mutex_waiter *orig_waiter
+ __IP_DECL__)
+{
+ struct rt_mutex *lock = orig_waiter->lock;
+ struct rt_mutex_waiter *waiter, *top_waiter;
+ task_t *owner;
+ unsigned long flags;

- BUG_ON(owner->pi_locked_by);
- owner->pi_locked_by = current;
- BUG_ON(!list_empty(&owner->pi_lock_chain));
- list_add(&owner->pi_lock_chain, lock_chain);
+ int detect_deadlock, ret = 0, depth = 0;

- /*
- * When the owner is blocked on a lock, try to take
- * the lock:
- */
- nextwaiter = owner->pi_blocked_on;
+ detect_deadlock = debug_rt_mutex_detect_deadlock(deadlock_detect);

- /* End of chain? */
- if (!nextwaiter)
- return 0;
-
- nextlock = nextwaiter->lock;
-
- /* Check for circular dependencies: */
- if (unlikely(nextlock == act_lock ||
- rt_mutex_owner(nextlock) == current)) {
- debug_rt_mutex_deadlock(detect_deadlock, waiter,
- nextlock);
- list_del_init(&owner->pi_lock_chain);
- owner->pi_locked_by = NULL;
- spin_unlock(&owner->pi_lock);
- return detect_deadlock ? -EDEADLK : 0;
+ for (;;) {
+ depth++;
+ if(task==current || depth>max_lock_depth) {
+ debug_rt_mutex_deadlock(deadlock_detect,
+ orig_waiter, lock);
+
+ put_task_struct(task);
+
+ return deadlock_detect ? -EDEADLK : 0;
}
+ retry:
+ _raw_spin_lock(&task->pi_lock);
+ __rt_mutex_adjust_prio(task);
+ waiter = task->pi_blocked_on;
+ if( !waiter )
+ break;

- /* Try to get nextlock->wait_lock: */
- if (unlikely(!spin_trylock(&nextlock->wait_lock))) {
- list_del_init(&owner->pi_lock_chain);
- owner->pi_locked_by = NULL;
- spin_unlock(&owner->pi_lock);
+ if( !detect_deadlock &&
+ waiter->list_entry.prio == task->prio &&
+ waiter->pi_list_entry.prio == waiter->list_entry.prio )
+ break;
+
+
+ lock = waiter->lock;
+ if( !spin_trylock_irqsave(&lock->wait_lock, flags) ) {
+ _raw_spin_unlock(&task->pi_lock);
cpu_relax();
- continue;
+ goto retry;
}

- lock = nextlock;
-
- /*
- * If deadlock detection is done (or has to be done, as
- * for userspace locks), we have to walk the full chain
- * unconditionally.
- */
- if (detect_deadlock)
- continue;
+ top_waiter = rt_mutex_top_waiter(lock);

- /*
- * Optimization: we only have to continue up to the point
- * where boosting/unboosting still has to be done:
- */
+ plist_del(&waiter->list_entry, &lock->wait_list);
+ waiter->list_entry.prio = task->prio;
+ plist_add(&waiter->list_entry, &lock->wait_list);
+
+ _raw_spin_unlock(&task->pi_lock);

- /* Boost or unboost? */
- if (waiter) {
- /* If the top waiter has higher priority, stop: */
- if (rt_mutex_top_waiter(lock)->list_entry.prio <=
- waiter->list_entry.prio)
- return 0;
- } else {
- /* If nextwaiter is not the top waiter, stop: */
- if (rt_mutex_top_waiter(lock) != nextwaiter)
- return 0;
+ owner = rt_mutex_owner(lock);
+ BUG_ON(!owner);
+ BUG_ON(owner==task);
+ if(waiter == rt_mutex_top_waiter(lock)) {
+ _raw_spin_lock(&owner->pi_lock);
+ plist_del(&top_waiter->pi_list_entry,
+ &owner->pi_waiters);
+
+ waiter->pi_list_entry.prio = waiter->list_entry.prio;
+ plist_add(&waiter->pi_list_entry, &owner->pi_waiters);
+ _raw_spin_unlock(&owner->pi_lock);
}
- }
-}
-
-/*
- * Unlock the pi_chain:
- */
-static void unlock_pi_chain(struct list_head *lock_chain)
-{
- struct task_struct *owner, *tmp;
-
- list_for_each_entry_safe(owner, tmp, lock_chain, pi_lock_chain) {
- struct rt_mutex_waiter *waiter = owner->pi_blocked_on;
-
- list_del_init(&owner->pi_lock_chain);
- BUG_ON(!owner->pi_locked_by);
- owner->pi_locked_by = NULL;
- if (waiter)
- spin_unlock(&waiter->lock->wait_lock);
- spin_unlock(&owner->pi_lock);
- }
-}
-
-/*
- * Do the priority (un)boosting along the chain:
- */
-static void adjust_pi_chain(struct rt_mutex *lock,
- struct rt_mutex_waiter *waiter,
- struct rt_mutex_waiter *top_waiter,
- struct list_head *lock_chain)
-{
- struct task_struct *owner = rt_mutex_owner(lock);
- struct list_head *curr = lock_chain->prev;
-
- for (;;) {
- if (top_waiter)
- plist_del(&top_waiter->pi_list_entry,
+ else if(top_waiter == waiter) {
+ /* waiter is no longer the frontmost waiter */
+ _raw_spin_lock(&owner->pi_lock);
+ plist_del(&waiter->pi_list_entry,
&owner->pi_waiters);
-
- if (waiter)
- waiter->pi_list_entry.prio = waiter->task->prio;
-
- if (rt_mutex_has_waiters(lock)) {
top_waiter = rt_mutex_top_waiter(lock);
- plist_add(&top_waiter->pi_list_entry,
+ top_waiter->pi_list_entry.prio =
+ top_waiter->list_entry.prio;
+ plist_add(&top_waiter->pi_list_entry,
&owner->pi_waiters);
+ _raw_spin_unlock(&owner->pi_lock);
}

- __rt_mutex_adjust_prio(owner);

- waiter = owner->pi_blocked_on;
- if (!waiter || curr->prev == lock_chain)
- return;
+ get_task_struct(owner);

- curr = curr->prev;
- lock = waiter->lock;
- owner = rt_mutex_owner(lock);
- top_waiter = rt_mutex_top_waiter(lock);
+ spin_unlock_irqrestore(&lock->wait_lock,flags);

- plist_del(&waiter->list_entry, &lock->wait_list);
- waiter->list_entry.prio = waiter->task->prio;
- plist_add(&waiter->list_entry, &lock->wait_list);
-
- /*
- * We can stop here, if the waiter is/was not the top
- * priority waiter:
- */
- if (top_waiter != waiter &&
- waiter != rt_mutex_top_waiter(lock))
- return;
-
- /*
- * Note: waiter is not necessarily the new top
- * waiter!
- */
- waiter = rt_mutex_top_waiter(lock);
+ put_task_struct(task);
+ task = owner;
}
+
+ _raw_spin_unlock(&task->pi_lock);
+ put_task_struct(task);
+ return ret;
}

/*
* Task blocks on lock.
*
- * Prepare waiter and potentially propagate our priority into the pi chain.
+ * Prepare waiter and propagate pi chain
*
* This must be called with lock->wait_lock held.
+ *
+ * Returns owner if it is needed to be boosted with adjust_prio_chain() if
+ * it is in itself blocked on a lock.
*/
-static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
- struct rt_mutex_waiter *waiter,
- int detect_deadlock __IP_DECL__)
+static task_t *task_blocks_on_rt_mutex(struct rt_mutex *lock,
+ struct rt_mutex_waiter *waiter,
+ int detect_deadlock
+ __IP_DECL__)
{
- int res = 0;
struct rt_mutex_waiter *top_waiter = waiter;
- LIST_HEAD(lock_chain);
+ task_t *owner = rt_mutex_owner(lock);
+ task_t *res = NULL;

+ _raw_spin_lock(&current->pi_lock);
+ __rt_mutex_adjust_prio(current);
waiter->task = current;
waiter->lock = lock;
- debug_rt_mutex_reset_waiter(waiter);
-
- spin_lock(&current->pi_lock);
- current->pi_locked_by = current;
plist_node_init(&waiter->list_entry, current->prio);
plist_node_init(&waiter->pi_list_entry, current->prio);

- /* Get the top priority waiter of the lock: */
+ /* Get the top priority waiter on the lock */
if (rt_mutex_has_waiters(lock))
top_waiter = rt_mutex_top_waiter(lock);
plist_add(&waiter->list_entry, &lock->wait_list);

current->pi_blocked_on = waiter;

- /*
- * Call adjust_prio_chain, when waiter is the new top waiter
- * or when deadlock detection is requested:
- */
- if (waiter != rt_mutex_top_waiter(lock) &&
- !debug_rt_mutex_detect_deadlock(detect_deadlock))
- goto out;
-
- /* Try to lock the full chain: */
- res = lock_pi_chain(lock, waiter, &lock_chain, 1, detect_deadlock);
-
- if (likely(!res))
- adjust_pi_chain(lock, waiter, top_waiter, &lock_chain);
-
- /* Common case: we managed to lock it: */
- if (res != -EBUSY)
- goto out_unlock;
-
- /* Rare case: we hit some other task running a pi chain operation: */
- unlock_pi_chain(&lock_chain);
-
- plist_del(&waiter->list_entry, &lock->wait_list);
- current->pi_blocked_on = NULL;
- current->pi_locked_by = NULL;
- spin_unlock(&current->pi_lock);
- fixup_rt_mutex_waiters(lock);
+ _raw_spin_unlock(&current->pi_lock);

- spin_unlock(&lock->wait_lock);
-
- spin_lock(&pi_conflicts_lock);
+ if (waiter == rt_mutex_top_waiter(lock)) {
+ _raw_spin_lock(&owner->pi_lock);
+ plist_del(&top_waiter->pi_list_entry, &owner->pi_waiters);
+ plist_add(&waiter->pi_list_entry, &owner->pi_waiters);
+
+ __rt_mutex_adjust_prio(owner);
+ if(owner->pi_blocked_on) {
+ /* Nested locks. We do the boosting of the next
+ tasks just before going to sleep in schedule */
+ res = owner;
+ get_task_struct(owner);
+ }

- spin_lock(&current->pi_lock);
- current->pi_locked_by = current;
- spin_lock(&lock->wait_lock);
- if (!rt_mutex_owner(lock)) {
- waiter->task = NULL;
- spin_unlock(&pi_conflicts_lock);
- goto out;
+ _raw_spin_unlock(&owner->pi_lock);
+ }
+ else if( debug_rt_mutex_detect_deadlock(detect_deadlock) ) {
+ _raw_spin_lock(&owner->pi_lock);
+ if(owner->pi_blocked_on) {
+ res = owner;
+ get_task_struct(owner);
+ }
+ _raw_spin_unlock(&owner->pi_lock);
}
- plist_node_init(&waiter->list_entry, current->prio);
- plist_node_init(&waiter->pi_list_entry, current->prio);
-
- /* Get the top priority waiter of the lock: */
- if (rt_mutex_has_waiters(lock))
- top_waiter = rt_mutex_top_waiter(lock);
- plist_add(&waiter->list_entry, &lock->wait_list);
-
- current->pi_blocked_on = waiter;
-
- /* Lock the full chain: */
- res = lock_pi_chain(lock, waiter, &lock_chain, 0, detect_deadlock);
-
- /* Drop the conflicts lock before adjusting: */
- spin_unlock(&pi_conflicts_lock);

- if (likely(!res))
- adjust_pi_chain(lock, waiter, top_waiter, &lock_chain);

- out_unlock:
- unlock_pi_chain(&lock_chain);
- out:
- current->pi_locked_by = NULL;
- spin_unlock(&current->pi_lock);
return res;
}

+
/*
* Optimization: check if we can steal the lock from the
* assigned pending owner [which might not have taken the
@@ -621,90 +496,53 @@ static void wakeup_next_waiter(struct rt
wake_up_process(pendowner);
}

+
/*
* Remove a waiter from a lock
*
- * Must be called with lock->wait_lock held.
+ * Must be called with lock->wait_lock held
*/
-static int remove_waiter(struct rt_mutex *lock,
- struct rt_mutex_waiter *waiter __IP_DECL__)
+static task_t *remove_waiter(struct rt_mutex *lock,
+ struct rt_mutex_waiter *waiter __IP_DECL__)
{
- struct rt_mutex_waiter *next_waiter = NULL,
- *top_waiter = rt_mutex_top_waiter(lock);
- LIST_HEAD(lock_chain);
- int res;
+ int first = (waiter == rt_mutex_top_waiter(lock));
+ task_t *res = NULL;

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

- spin_lock(&current->pi_lock);
-
- if (waiter != top_waiter || rt_mutex_owner(lock) == current)
- goto out;
-
- current->pi_locked_by = current;

- if (rt_mutex_has_waiters(lock))
- next_waiter = rt_mutex_top_waiter(lock);
+ if (first && rt_mutex_owner(lock) != current) {
+ task_t *owner = rt_mutex_owner(lock);

- /* Try to lock the full chain: */
- res = lock_pi_chain(lock, next_waiter, &lock_chain, 1, 0);
+ _raw_spin_lock(&owner->pi_lock);

- if (likely(res != -EBUSY)) {
- adjust_pi_chain(lock, next_waiter, waiter, &lock_chain);
- goto out_unlock;
- }
+ plist_del(&waiter->pi_list_entry, &owner->pi_waiters);

- /* We hit some other task running a pi chain operation: */
- unlock_pi_chain(&lock_chain);
- plist_add(&waiter->list_entry, &lock->wait_list);
- current->pi_blocked_on = waiter;
- current->pi_locked_by = NULL;
- spin_unlock(&current->pi_lock);
- spin_unlock(&lock->wait_lock);
-
- spin_lock(&pi_conflicts_lock);
+ if (rt_mutex_has_waiters(lock)) {
+ struct rt_mutex_waiter *next;

- spin_lock(&current->pi_lock);
- current->pi_locked_by = current;
+ next = rt_mutex_top_waiter(lock);
+ plist_add(&next->pi_list_entry, &owner->pi_waiters);
+ }

- spin_lock(&lock->wait_lock);

- /* We might have been woken up: */
- if (!waiter->task) {
- spin_unlock(&pi_conflicts_lock);
- goto out;
+ __rt_mutex_adjust_prio(owner);
+ if(owner->pi_blocked_on) {
+ /* Owner is blocked on something - we have
+ to (un)boost throughout the lock chain but
+ we have to wait until we have dropped all locks */
+ res = owner;
+ get_task_struct(owner);
+ }
+ _raw_spin_unlock(&owner->pi_lock);
}

- top_waiter = rt_mutex_top_waiter(lock);
-
- plist_del(&waiter->list_entry, &lock->wait_list);
-
- if (waiter != top_waiter || rt_mutex_owner(lock) == current)
- goto out;
-
- /* Get the top priority waiter of the lock: */
- if (rt_mutex_has_waiters(lock))
- next_waiter = rt_mutex_top_waiter(lock);
-
- /* Lock the full chain: */
- lock_pi_chain(lock, next_waiter, &lock_chain, 0, 0);
-
- /* Drop the conflicts lock: */
- spin_unlock(&pi_conflicts_lock);
-
- adjust_pi_chain(lock, next_waiter, waiter, &lock_chain);
+ WARN_ON(!plist_node_empty(&waiter->pi_list_entry));

- out_unlock:
- unlock_pi_chain(&lock_chain);
- out:
current->pi_blocked_on = NULL;
waiter->task = NULL;
- current->pi_locked_by = NULL;
- spin_unlock(&current->pi_lock);
-
- WARN_ON(!plist_node_empty(&waiter->pi_list_entry));

- return 0;
+ return res;
}

#ifdef CONFIG_PREEMPT_RT
@@ -744,6 +582,7 @@ rt_lock_slowlock(struct rt_mutex *lock _
{
struct rt_mutex_waiter waiter;
unsigned long saved_state, state, flags;
+ task_t *owner;

debug_rt_mutex_init_waiter(&waiter);
waiter.task = NULL;
@@ -784,8 +623,24 @@ rt_lock_slowlock(struct rt_mutex *lock _
* but the lock got stolen by an higher prio task.
*/
if (unlikely(!waiter.task))
- task_blocks_on_rt_mutex(lock, &waiter, 0 __IP__);
-
+ owner = task_blocks_on_rt_mutex(lock, &waiter,
+ 0 __IP__);
+ else
+ owner = NULL;
+
+ if (unlikely(owner)) {
+ spin_unlock_irqrestore(&lock->wait_lock,flags);
+ rt_mutex_adjust_prio_chain(owner, 0, &waiter __IP__);
+ owner = NULL;
+ spin_lock_irqsave(&lock->wait_lock,flags);
+ if(unlikely(!waiter.task))
+ continue; /* We got woken up by the owner
+ * Start loop all over without
+ * going into schedule to try
+ * to get the lock now
+ */
+ }
+
/*
* Prevent schedule() to drop BKL, while waiting for
* the lock ! We restore lock_depth when we come back.
@@ -795,6 +650,8 @@ rt_lock_slowlock(struct rt_mutex *lock _
current->flags &= ~PF_NOSCHED;
spin_unlock_irqrestore(&lock->wait_lock, flags);

+ debug_rt_mutex_print_deadlock(&waiter);
+
if (waiter.task)
schedule_rt_mutex(lock);

@@ -817,7 +674,9 @@ rt_lock_slowlock(struct rt_mutex *lock _
* can end up with a non-NULL waiter.task:
*/
if (unlikely(waiter.task))
- remove_waiter(lock, &waiter __IP__);
+ owner = remove_waiter(lock, &waiter __IP__);
+ else
+ owner = NULL;
/*
* try_to_take_rt_mutex() sets the waiter bit
* unconditionally. We might have to fix that up:
@@ -827,6 +686,8 @@ rt_lock_slowlock(struct rt_mutex *lock _
spin_unlock_irqrestore(&lock->wait_lock, flags);

debug_rt_mutex_free_waiter(&waiter);
+ if (unlikely(owner))
+ rt_mutex_adjust_prio_chain(owner, 0, &waiter __IP__);
}

/*
@@ -915,6 +776,7 @@ rt_mutex_slowlock(struct rt_mutex *lock,
struct rt_mutex_waiter waiter;
int ret = 0, saved_lock_depth = -1;
unsigned long flags;
+ task_t *owner;

debug_rt_mutex_init_waiter(&waiter);
waiter.task = NULL;
@@ -929,8 +791,6 @@ rt_mutex_slowlock(struct rt_mutex *lock,
return 0;
}

- BUG_ON(rt_mutex_owner(lock) == current);
-
/*
* We drop BKL here before we go into the wait loop to avoid a
* possible deadlock in the scheduler.
@@ -971,13 +831,25 @@ rt_mutex_slowlock(struct rt_mutex *lock,
* when we have been woken up by the previous owner
* but the lock got stolen by an higher prio task.
*/
- if (!waiter.task) {
- ret = task_blocks_on_rt_mutex(lock, &waiter,
- detect_deadlock __IP__);
- if (ret == -EDEADLK)
- break;
- if (ret == -EBUSY)
- continue;
+ if (!waiter.task)
+ owner = task_blocks_on_rt_mutex(lock, &waiter,
+ detect_deadlock
+ __IP__);
+ else
+ owner = NULL;
+
+ if (unlikely(owner)) {
+ spin_unlock_irqrestore(&lock->wait_lock,flags);
+ rt_mutex_adjust_prio_chain(owner, detect_deadlock,
+ &waiter __IP__);
+ owner = NULL;
+ spin_lock_irqsave(&lock->wait_lock,flags);
+ if(unlikely(!waiter.task))
+ continue; /* We got woken up by the owner
+ * Start loop all over without
+ * going into schedule to try
+ * to get the lock now
+ */
}

saved_flags = current->flags & PF_NOSCHED;
@@ -998,8 +870,10 @@ rt_mutex_slowlock(struct rt_mutex *lock,

set_current_state(TASK_RUNNING);

- if (unlikely(waiter.task))
- remove_waiter(lock, &waiter __IP__);
+ if (unlikely(waiter.task))
+ owner = remove_waiter(lock, &waiter __IP__);
+ else
+ owner = NULL;

/*
* try_to_take_rt_mutex() sets the waiter bit
@@ -1009,6 +883,9 @@ rt_mutex_slowlock(struct rt_mutex *lock,

spin_unlock_irqrestore(&lock->wait_lock, flags);

+ if (unlikely(owner))
+ rt_mutex_adjust_prio_chain(owner, 0, &waiter __IP__);
+
/* Remove pending timer: */
if (unlikely(timeout && timeout->task))
hrtimer_cancel(&timeout->timer);
Only in linux-2.6.16-rt9.pipatch/kernel: rtmutex.c.orig
Only in linux-2.6.16-rt9.pipatch/kernel: rtmutex.c.rej
Only in linux-2.6.16-rt9.pipatch/kernel: rtmutex.c~
diff -upr linux-2.6.16-rt9/kernel/sysctl.c linux-2.6.16-rt9.pipatch/kernel/sysctl.c
--- linux-2.6.16-rt9/kernel/sysctl.c 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/kernel/sysctl.c 2006-03-27 01:02:41.000000000 +0200
@@ -132,6 +132,10 @@ extern int acct_parm[];
extern int no_unaligned_warning;
#endif

+#ifdef CONFIG_RT_MUTEXES
+extern int max_lock_depth;
+#endif
+
static int parse_table(int __user *, int, void __user *, size_t __user *, void __user *, size_t,
ctl_table *, void **);
static int proc_doutsstring(ctl_table *table, int write, struct file *filp,
@@ -827,6 +831,17 @@ static ctl_table kern_table[] = {
.proc_handler = &proc_dointvec,
},
#endif
+#ifdef CONFIG_RT_MUTEXES
+ {
+ .ctl_name = KERN_MAX_LOCK_DEPTH,
+ .procname = "max_lock_depth",
+ .data = &max_lock_depth,
+ .maxlen = sizeof(int),
+ .mode = 0644,
+ .proc_handler = &proc_dointvec,
+ },
+#endif
+
{ .ctl_name = 0 }
};

diff -upr linux-2.6.16-rt9/scripts/rt-tester/t3-l1-pi-signal.tst linux-2.6.16-rt9.pipatch/scripts/rt-tester/t3-l1-pi-signal.tst
--- linux-2.6.16-rt9/scripts/rt-tester/t3-l1-pi-signal.tst 2006-03-27 01:04:28.000000000 +0200
+++ linux-2.6.16-rt9.pipatch/scripts/rt-tester/t3-l1-pi-signal.tst 2006-03-27 01:02:41.000000000 +0200
@@ -69,15 +69,18 @@ W: locked: 0: 0
C: locknowait: 1: 0
W: blocked: 1: 0
T: prioeq: 0: 80
+T: prioeq: 1: 80

# T2 lock L0 interruptible, no wait in the wakeup path
C: lockintnowait: 2: 0
W: blocked: 2: 0
T: prioeq: 0: 81
+T: prioeq: 1: 80

# Interrupt T2
C: signal: 2: 2
W: unlocked: 2: 0
+T: prioeq: 1: 80
T: prioeq: 0: 80

T: locked: 0: 0


2006-03-26 23:49:57

by Ingo Molnar

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


* Esben Nielsen <[email protected]> wrote:

> Hi,
> I got the patch I mentioned earlier to run. It passes my userspace
> testscripts as well as all the scripts for Thomas's rtmutex-tester on a UP
> machine.
>
> The idea is to avoid the deadlocks by releasing all locks before going
> to the next lock in the chain. I use get_/put_task_struct to avoid the
> task disappearing during the iteration.

but we lose reliable deadlock detection ...

how do you guarantee that some other CPU doesnt send us on some
goose-chase?

Ingo

2006-03-27 00:07:17

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Mon, 27 Mar 2006, Ingo Molnar wrote:

>
> * Esben Nielsen <[email protected]> wrote:
>
> > Hi,
> > I got the patch I mentioned earlier to run. It passes my userspace
> > testscripts as well as all the scripts for Thomas's rtmutex-tester on a UP
> > machine.
> >
> > The idea is to avoid the deadlocks by releasing all locks before going
> > to the next lock in the chain. I use get_/put_task_struct to avoid the
> > task disappearing during the iteration.
>
> but we lose reliable deadlock detection ...
>
> how do you guarantee that some other CPU doesnt send us on some
> goose-chase?
>

How should another CPU suddenly be able to insert stuff into a lock chain?
Only the tasks themselves can do that and they are blocked on some lock -
at least when we tested in some previous iteration. Ofcourse, they can
have been signalled or timed out since, such they are already unblocked
when the deadlock is reported. But that is not an error since the locks at
some point actually were in a deadlock situation.

I do put in a limit of 100 (can be changed with sysctl) iterations. But
that is to avoid looping forever when a new task blocks on a lock already
part of a deadlock.

> Ingo
>

Esben

2006-03-27 00:11:29

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


On Mon, 27 Mar 2006, Esben Nielsen wrote:

> On Mon, 27 Mar 2006, Ingo Molnar wrote:
>
> >
> > * Esben Nielsen <[email protected]> wrote:
> >
> > > Hi,
> > > I got the patch I mentioned earlier to run. It passes my userspace
> > > testscripts as well as all the scripts for Thomas's rtmutex-tester on a UP
> > > machine.
> > >
> > > The idea is to avoid the deadlocks by releasing all locks before going
> > > to the next lock in the chain. I use get_/put_task_struct to avoid the
> > > task disappearing during the iteration.
> >
> > but we lose reliable deadlock detection ...
> >
> > how do you guarantee that some other CPU doesnt send us on some
> > goose-chase?
> >
>
> How should another CPU suddenly be able to insert stuff into a lock chain?
> Only the tasks themselves can do that and they are blocked on some lock -
> at least when we tested in some previous iteration. Ofcourse, they can
> have been signalled or timed out since, such they are already unblocked
> when the deadlock is reported. But that is not an error since the locks at
> some point actually were in a deadlock situation.

I might add: That can in principle happen for any deadlock detector:

1) Your task detects a deadlock.
2) Your task releases the last spinlock.
3) It gets preempted.
4) Some of the deadlocked tasks is signalled.
5) Your tasks returns from the kernel and reports a deadlock which is no
longer there.

Esben

>
> I do put in a limit of 100 (can be changed with sysctl) iterations. But
> that is to avoid looping forever when a new task blocks on a lock already
> part of a deadlock.
>
> > Ingo
> >
>
> Esben
>
>

2006-03-27 00:23:43

by Ingo Molnar

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


* Esben Nielsen <[email protected]> wrote:

> > how do you guarantee that some other CPU doesnt send us on some
> > goose-chase?
>
> How should another CPU suddenly be able to insert stuff into a lock
> chain? Only the tasks themselves can do that and they are blocked on
> some lock - at least when we tested in some previous iteration.
> Ofcourse, they can have been signalled or timed out since, such they
> are already unblocked when the deadlock is reported. But that is not
> an error since the locks at some point actually were in a deadlock
> situation.

we are observing a non-time-coherent snapshot of the locking graph.
There is no guarantee that due to timeouts or signals the chain we
observe isnt artificially long - while if a time-coherent snapshot is
taken it is always fine. E.g. lets take dentry locks as an example:
their locking is ordered by the dentry (kernel-pointer) address. We
could in theory have a 'chain' of subsequent locking dependencies
related to 10,000 dentries, which are nicely ordered and create a
10,000-entry 'chain' if looked at in a non-time-coherent form. I.e. your
code could detect a deadlock where there's none. The more CPUs there
are, the larger the likelyhood is that other CPUs 'lure us' into a long
chain.

In other words: without taking all the locks we have no mathematical
proof that we detected a deadlock!

also, how does the taking of 2 locks only improve latencies? We still
have to hold the ->waiter_lock of this lock during this act, dont we? Or
can we do boosting with totally unlocked (and interrupts-enabled)
rescheduling points? If yes then the same situation could happen on UP
too: if there's lots of rescheduling of this boosting chain.

nevertheless it _might_ work in practice, and it's certainly elegant and
thus tempting. Could you try to port your patch to -rt10? [you can skip
most of the conflicting rt7->rt10 deltas in rtmutex.c i think.]

Ingo

2006-03-27 15:00:30

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Mon, 27 Mar 2006, Ingo Molnar wrote:

>
> * Esben Nielsen <[email protected]> wrote:
>
> > > how do you guarantee that some other CPU doesnt send us on some
> > > goose-chase?
> >
> > How should another CPU suddenly be able to insert stuff into a lock
> > chain? Only the tasks themselves can do that and they are blocked on
> > some lock - at least when we tested in some previous iteration.
> > Ofcourse, they can have been signalled or timed out since, such they
> > are already unblocked when the deadlock is reported. But that is not
> > an error since the locks at some point actually were in a deadlock
> > situation.
>
> we are observing a non-time-coherent snapshot of the locking graph.
> There is no guarantee that due to timeouts or signals the chain we
> observe isnt artificially long - while if a time-coherent snapshot is
> taken it is always fine. E.g. lets take dentry locks as an example:
> their locking is ordered by the dentry (kernel-pointer) address. We
> could in theory have a 'chain' of subsequent locking dependencies
> related to 10,000 dentries, which are nicely ordered and create a
> 10,000-entry 'chain' if looked at in a non-time-coherent form. I.e. your
> code could detect a deadlock where there's none. The more CPUs there
> are, the larger the likelyhood is that other CPUs 'lure us' into a long
> chain.

I don't quite understand you examble: Are all 10,000 held at once?
If no, how are they all going to suddenly put into the lock chain due to
signals or timeouts? Those things unlocks locks and therefore breaks the
chain.

We need to cook up a specific examble we can discuss - also to see whether
it is a really an error or not.

>
> In other words: without taking all the locks we have no mathematical
> proof that we detected a deadlock!

Before you talk about a mathematical proof you need to make a mathematical
definition of a deadlock. Your definition seems to be that at some point
in time there is a circular dependency. (By the way: Time can be hard to
define too :-)

That definition does not make much sense in practice if you are rescued
by timeouts or signals within the time of the lock() call - which can be
very long as preemption is enabled in the prolog and the epilog of the call.

Deadlock detection therefore only makes sense in programs where the
mutex timeouts and signal spacing are long compared to the normal
timescale of the program. If not, the program will not "deadlock", i.e.
get stuck, but keep on running, even though at some point in time there
was a circular locking.

>
> also, how does the taking of 2 locks only improve latencies? We still
> have to hold the ->waiter_lock of this lock during this act, dont we? Or
> can we do boosting with totally unlocked (and interrupts-enabled)
> rescheduling points? If yes then the same situation could happen on UP
> too: if there's lots of rescheduling of this boosting chain.
>

Yes, lock->wait_lock is dropped during the iteration - otherwise it could
deadlock. The boosting is done with preemption and interrupts enabled once
in every iteration. And yes the problems you talk about are therefore
not specific to SMP but can also happen on UP - just with a lower rate.

> nevertheless it _might_ work in practice, and it's certainly elegant and
> thus tempting. Could you try to port your patch to -rt10? [you can skip
> most of the conflicting rt7->rt10 deltas in rtmutex.c i think.]
>

I'll try to see what I can do. I am bit busy right now. We are packing to
go to England for 4 months on Saturday. There are lot of practicalities
we are still missing - but ofcourse PI code much more fun :-) Maybe I can
"steal" some time tonight.

Esben

> Ingo
> -
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at http://www.tux.org/lkml/
>

2006-03-27 23:06:04

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9



On Mon, 27 Mar 2006, Esben Nielsen wrote:

> On Mon, 27 Mar 2006, Ingo Molnar wrote:
> [...]
> > nevertheless it _might_ work in practice, and it's certainly elegant and
> > thus tempting. Could you try to port your patch to -rt10? [you can skip
> > most of the conflicting rt7->rt10 deltas in rtmutex.c i think.]
> >
>
> I'll try to see what I can do. I am bit busy right now. We are packing to
> go to England for 4 months on Saturday. There are lot of practicalities
> we are still missing - but ofcourse PI code much more fun :-) Maybe I can
> "steal" some time tonight.
>

My girl friend will be angry for me not being to bed yet, but
I had to steal time to make this patch. I hope I managed to send it
without white-space damage or anything like it.

Esben


diff -upr linux-2.6.16-rt10/include/linux/rtmutex.h linux-2.6.16-rt10.pipatch/include/linux/rtmutex.h
--- linux-2.6.16-rt10/include/linux/rtmutex.h 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/include/linux/rtmutex.h 2006-03-27 18:40:08.000000000 +0200
@@ -107,8 +107,7 @@ extern void rt_mutex_unlock(struct rt_mu
#ifdef CONFIG_RT_MUTEXES
# define INIT_RT_MUTEXES(tsk) \
.pi_waiters = PLIST_HEAD_INIT(tsk.pi_waiters, tsk.pi_lock), \
- .pi_lock = RAW_SPIN_LOCK_UNLOCKED, \
- .pi_lock_chain = LIST_HEAD_INIT(tsk.pi_lock_chain),
+ .pi_lock = RAW_SPIN_LOCK_UNLOCKED,
#else
# define INIT_RT_MUTEXES(tsk)
#endif
diff -upr linux-2.6.16-rt10/include/linux/sched.h linux-2.6.16-rt10.pipatch/include/linux/sched.h
--- linux-2.6.16-rt10/include/linux/sched.h 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/include/linux/sched.h 2006-03-27 18:40:08.000000000 +0200
@@ -985,9 +985,6 @@ struct task_struct {
struct plist_head pi_waiters;
/* Deadlock detection and priority inheritance handling */
struct rt_mutex_waiter *pi_blocked_on;
- /* PI locking helpers */
- struct task_struct *pi_locked_by;
- struct list_head pi_lock_chain;
#endif

#ifdef CONFIG_DEBUG_MUTEXES
diff -upr linux-2.6.16-rt10/include/linux/spinlock_api_smp.h linux-2.6.16-rt10.pipatch/include/linux/spinlock_api_smp.h
--- linux-2.6.16-rt10/include/linux/spinlock_api_smp.h 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/include/linux/spinlock_api_smp.h 2006-03-27 18:40:08.000000000 +0200
@@ -39,6 +39,7 @@ int __lockfunc _raw_read_trylock(raw_rwl
int __lockfunc _raw_write_trylock(raw_rwlock_t *lock);
int __lockfunc _raw_spin_trylock_irqsave(raw_spinlock_t *lock,
unsigned long *flags);
+int __lockfunc _raw_spin_trylock_irq(raw_spinlock_t *lock);
int __lockfunc _raw_spin_trylock_bh(raw_spinlock_t *lock);
void __lockfunc _raw_spin_unlock(raw_spinlock_t *lock) __releases(raw_spinlock_t);
void __lockfunc _raw_spin_unlock_no_resched(raw_spinlock_t *lock) __releases(raw_spinlock_t);
diff -upr linux-2.6.16-rt10/include/linux/spinlock_api_up.h linux-2.6.16-rt10.pipatch/include/linux/spinlock_api_up.h
--- linux-2.6.16-rt10/include/linux/spinlock_api_up.h 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/include/linux/spinlock_api_up.h 2006-03-27 18:40:08.000000000 +0200
@@ -39,6 +39,9 @@
#define __TRYLOCK_IRQSAVE(lock, flags) \
({ local_irq_save(*(flags)); __LOCK(lock); 1; })

+#define __TRYLOCK_IRQ(lock) \
+ ({ raw_local_irq_disable(); __LOCK(lock); 1; })
+
#define _raw_spin_trylock_irqsave(lock, flags) __TRYLOCK_IRQSAVE(lock, flags)

#define __UNLOCK(lock) \
@@ -75,6 +78,7 @@
#define _raw_read_trylock_bh(lock) ({ __LOCK_BH(lock); 1; })
#define _raw_write_trylock_bh(lock) ({ __LOCK_BH(lock); 1; })
#define _raw_spin_trylock_irqsave(lock, flags) __TRYLOCK_IRQSAVE(lock, flags)
+#define _raw_spin_trylock_irq(lock) __TRYLOCK_IRQ(lock)
#define _raw_read_trylock_irqsave(lock, flags) __TRYLOCK_IRQSAVE(lock, flags)
#define _raw_read_trylock_irqsave(lock, flags) __TRYLOCK_IRQSAVE(lock, flags)
#define _raw_spin_unlock(lock) __UNLOCK(lock)
diff -upr linux-2.6.16-rt10/include/linux/sysctl.h linux-2.6.16-rt10.pipatch/include/linux/sysctl.h
--- linux-2.6.16-rt10/include/linux/sysctl.h 2006-03-20 06:53:29.000000000 +0100
+++ linux-2.6.16-rt10.pipatch/include/linux/sysctl.h 2006-03-27 18:40:08.000000000 +0200
@@ -148,6 +148,7 @@ enum
KERN_SPIN_RETRY=70, /* int: number of spinlock retries */
KERN_ACPI_VIDEO_FLAGS=71, /* int: flags for setting up video after ACPI sleep */
KERN_IA64_UNALIGNED=72, /* int: ia64 unaligned userland trap enable */
+ KERN_MAX_LOCK_DEPTH=73
};


diff -upr linux-2.6.16-rt10/kernel/fork.c linux-2.6.16-rt10.pipatch/kernel/fork.c
--- linux-2.6.16-rt10/kernel/fork.c 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/kernel/fork.c 2006-03-27 18:40:08.000000000 +0200
@@ -947,8 +947,6 @@ static inline void rt_mutex_init_task(st
spin_lock_init(&p->pi_lock);
plist_head_init(&p->pi_waiters, &p->pi_lock);
p->pi_blocked_on = NULL;
- p->pi_locked_by = NULL;
- INIT_LIST_HEAD(&p->pi_lock_chain);
#endif
}

diff -upr linux-2.6.16-rt10/kernel/rtmutex-debug.c linux-2.6.16-rt10.pipatch/kernel/rtmutex-debug.c
--- linux-2.6.16-rt10/kernel/rtmutex-debug.c 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/kernel/rtmutex-debug.c 2006-03-27 18:40:08.000000000 +0200
@@ -398,9 +398,7 @@ restart:
void notrace rt_mutex_debug_task_free(struct task_struct *tsk)
{
WARN_ON(!plist_head_empty(&tsk->pi_waiters));
- WARN_ON(!list_empty(&tsk->pi_lock_chain));
WARN_ON(tsk->pi_blocked_on);
- WARN_ON(tsk->pi_locked_by);
}

/*
diff -upr linux-2.6.16-rt10/kernel/rtmutex.c linux-2.6.16-rt10.pipatch/kernel/rtmutex.c
--- linux-2.6.16-rt10/kernel/rtmutex.c 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/kernel/rtmutex.c 2006-03-28 00:11:24.000000000 +0200
@@ -160,204 +160,103 @@ static void rt_mutex_adjust_prio(struct
spin_unlock_irqrestore(&task->pi_lock, flags);
}

-/*
- * PI-locking: we lock PI-dependencies opportunistically via trylock.
- *
- * In the overwhelming majority of cases the 'PI chain' is empty or at
- * most 1-2 entries long, for which the trylock is sufficient,
- * scalability-wise. The locking might look a bit scary, for which we
- * apologize in advance :-)
- *
- * If any of the trylocks fails then we back out, task the global
- * pi_conflicts_lock and take the locks again. This ensures deadlock-free
- * but still scalable locking in the dependency graph, combined with
- * the ability to reliably (and cheaply) detect user-space deadlocks.
- */
-static DEFINE_RAW_SPINLOCK(pi_conflicts_lock);

-/*
- * Lock the full boosting chain.
- *
- * If 'try' is set, we have to backout if we hit a owner who is
- * running its own pi chain operation. We go back and take the slow
- * path via the pi_conflicts_lock.
- *
- * We put all held locks into a list, via ->pi_lock_chain, and walk
- * this list at unlock_pi_chain() time.
- */
-static int lock_pi_chain(struct rt_mutex *act_lock,
- struct rt_mutex_waiter *waiter,
- struct list_head *lock_chain,
- int try, int detect_deadlock)
-{
- struct task_struct *owner;
- struct rt_mutex *nextlock, *lock = act_lock;
- struct rt_mutex_waiter *nextwaiter;
- int deadlock_detect;
+int max_lock_depth = 100;

- /*
- * Debugging might turn deadlock detection on, unconditionally:
- */
- deadlock_detect = debug_rt_mutex_detect_deadlock(detect_deadlock);
+/*
+ * Adjust the priority chain. Also used for deadlock detection.
+ * Decreases task's usage by one - may thus free the task.
+ * Returns 0 or -EDEADLK.
+ */
+static int rt_mutex_adjust_prio_chain(task_t *task,
+ int deadlock_detect,
+ struct rt_mutex_waiter *orig_waiter
+ __IP_DECL__)
+{
+ struct rt_mutex *lock = orig_waiter->lock;
+ struct rt_mutex_waiter *waiter, *top_waiter;
+ task_t *owner;
+ unsigned long flags;

- for (;;) {
- owner = rt_mutex_owner(lock);
+ int detect_deadlock, ret = 0, depth = 0;

- /* Check for circular dependencies */
- if (unlikely(owner->pi_locked_by == current)) {
- debug_rt_mutex_deadlock(detect_deadlock, waiter, lock);
- return detect_deadlock ? -EDEADLK : 1;
- }
+ detect_deadlock = debug_rt_mutex_detect_deadlock(deadlock_detect);

- while (!spin_trylock(&owner->pi_lock)) {
- /*
- * Owner runs its own chain. Go back and take
- * the slow path
- */
- if (try && owner->pi_locked_by == owner)
- return -EBUSY;
- cpu_relax();
+ for (;;) {
+ depth++;
+ if(task==current || depth>max_lock_depth) {
+ debug_rt_mutex_deadlock(deadlock_detect,
+ orig_waiter, lock);
+
+ put_task_struct(task);
+
+ return deadlock_detect ? -EDEADLK : 0;
}
+ retry:
+ _raw_spin_lock(&task->pi_lock);
+ __rt_mutex_adjust_prio(task);
+ waiter = task->pi_blocked_on;
+ if( !waiter )
+ break;

- BUG_ON(owner->pi_locked_by);
- owner->pi_locked_by = current;
- BUG_ON(!list_empty(&owner->pi_lock_chain));
- list_add(&owner->pi_lock_chain, lock_chain);
-
- /*
- * When the owner is blocked on a lock, try to take
- * the lock:
- */
- nextwaiter = owner->pi_blocked_on;
-
- /* End of chain? */
- if (!nextwaiter)
- return 1;
-
- nextlock = nextwaiter->lock;
-
- /* Check for circular dependencies: */
- if (unlikely(nextlock == act_lock ||
- rt_mutex_owner(nextlock) == current)) {
- debug_rt_mutex_deadlock(detect_deadlock, waiter,
- nextlock);
- list_del_init(&owner->pi_lock_chain);
- owner->pi_locked_by = NULL;
- spin_unlock(&owner->pi_lock);
- return detect_deadlock ? -EDEADLK : 1;
- }
+ if( !detect_deadlock &&
+ waiter->list_entry.prio == task->prio &&
+ waiter->pi_list_entry.prio == waiter->list_entry.prio )
+ break;

- /* Try to get nextlock->wait_lock: */
- if (unlikely(!spin_trylock(&nextlock->wait_lock))) {
- list_del_init(&owner->pi_lock_chain);
- owner->pi_locked_by = NULL;
- spin_unlock(&owner->pi_lock);
+
+ lock = waiter->lock;
+ if( !spin_trylock_irqsave(&lock->wait_lock, flags) ) {
+ _raw_spin_unlock(&task->pi_lock);
cpu_relax();
- continue;
+ goto retry;
}

- lock = nextlock;
-
- /*
- * If deadlock detection is done (or has to be done, as
- * for userspace locks), we have to walk the full chain
- * unconditionally.
- */
- if (deadlock_detect)
- continue;
+ top_waiter = rt_mutex_top_waiter(lock);

- /*
- * Optimization: we only have to continue up to the point
- * where boosting/unboosting still has to be done:
- */
+ plist_del(&waiter->list_entry, &lock->wait_list);
+ waiter->list_entry.prio = task->prio;
+ plist_add(&waiter->list_entry, &lock->wait_list);
+
+ _raw_spin_unlock(&task->pi_lock);

- /* Boost or unboost? */
- if (waiter) {
- /* If the top waiter has higher priority, stop: */
- if (rt_mutex_top_waiter(lock)->list_entry.prio <=
- waiter->list_entry.prio)
- return 1;
- } else {
- /* If nextwaiter is not the top waiter, stop: */
- if (rt_mutex_top_waiter(lock) != nextwaiter)
- return 1;
+ owner = rt_mutex_owner(lock);
+ BUG_ON(!owner);
+ BUG_ON(owner==task);
+ if(waiter == rt_mutex_top_waiter(lock)) {
+ _raw_spin_lock(&owner->pi_lock);
+ plist_del(&top_waiter->pi_list_entry,
+ &owner->pi_waiters);
+
+ waiter->pi_list_entry.prio = waiter->list_entry.prio;
+ plist_add(&waiter->pi_list_entry, &owner->pi_waiters);
+ _raw_spin_unlock(&owner->pi_lock);
}
- }
-}
-
-/*
- * Unlock the pi_chain:
- */
-static void unlock_pi_chain(struct list_head *lock_chain)
-{
- struct task_struct *owner, *tmp;
-
- list_for_each_entry_safe(owner, tmp, lock_chain, pi_lock_chain) {
- struct rt_mutex_waiter *waiter = owner->pi_blocked_on;
-
- list_del_init(&owner->pi_lock_chain);
- BUG_ON(!owner->pi_locked_by);
- owner->pi_locked_by = NULL;
- if (waiter)
- spin_unlock(&waiter->lock->wait_lock);
- spin_unlock(&owner->pi_lock);
- }
-}
-
-/*
- * Do the priority (un)boosting along the chain:
- */
-static void adjust_pi_chain(struct rt_mutex *lock,
- struct rt_mutex_waiter *waiter,
- struct rt_mutex_waiter *top_waiter,
- struct list_head *lock_chain)
-{
- struct task_struct *owner = rt_mutex_owner(lock);
- struct list_head *curr = lock_chain->prev;
-
- for (;;) {
- if (top_waiter)
- plist_del(&top_waiter->pi_list_entry,
+ else if(top_waiter == waiter) {
+ /* waiter is no longer the frontmost waiter */
+ _raw_spin_lock(&owner->pi_lock);
+ plist_del(&waiter->pi_list_entry,
&owner->pi_waiters);
-
- if (waiter)
- waiter->pi_list_entry.prio = waiter->task->prio;
-
- if (rt_mutex_has_waiters(lock)) {
top_waiter = rt_mutex_top_waiter(lock);
- plist_add(&top_waiter->pi_list_entry,
+ top_waiter->pi_list_entry.prio =
+ top_waiter->list_entry.prio;
+ plist_add(&top_waiter->pi_list_entry,
&owner->pi_waiters);
+ _raw_spin_unlock(&owner->pi_lock);
}

- __rt_mutex_adjust_prio(owner);

- waiter = owner->pi_blocked_on;
- if (!waiter || curr->prev == lock_chain)
- return;
+ get_task_struct(owner);

- curr = curr->prev;
- lock = waiter->lock;
- owner = rt_mutex_owner(lock);
- top_waiter = rt_mutex_top_waiter(lock);
+ spin_unlock_irqrestore(&lock->wait_lock,flags);

- plist_del(&waiter->list_entry, &lock->wait_list);
- waiter->list_entry.prio = waiter->task->prio;
- plist_add(&waiter->list_entry, &lock->wait_list);
-
- /*
- * We can stop here, if the waiter is/was not the top
- * priority waiter:
- */
- if (top_waiter != waiter &&
- waiter != rt_mutex_top_waiter(lock))
- return;
-
- /*
- * Note: waiter is not necessarily the new top
- * waiter!
- */
- waiter = rt_mutex_top_waiter(lock);
+ put_task_struct(task);
+ task = owner;
}
+
+ _raw_spin_unlock(&task->pi_lock);
+ put_task_struct(task);
+ return ret;
}

/*
@@ -468,111 +367,63 @@ static int try_to_take_rt_mutex(struct r
/*
* Task blocks on lock.
*
- * Prepare waiter and potentially propagate our priority into the pi chain.
+ * Prepare waiter and propagate pi chain
*
* This must be called with lock->wait_lock held.
- * return values: 1: waiter queued, 0: got the lock,
- * -EDEADLK: deadlock detected.
+ *
+ * Returns owner if it is needed to be boosted with adjust_prio_chain() if
+ * it is in itself blocked on a lock.
*/
-static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
- struct rt_mutex_waiter *waiter,
- int detect_deadlock __IP_DECL__)
+static task_t *task_blocks_on_rt_mutex(struct rt_mutex *lock,
+ struct rt_mutex_waiter *waiter,
+ int detect_deadlock
+ __IP_DECL__)
{
struct rt_mutex_waiter *top_waiter = waiter;
- LIST_HEAD(lock_chain);
- int res = 1;
+ task_t *owner = rt_mutex_owner(lock);
+ task_t *res = NULL;

+ _raw_spin_lock(&current->pi_lock);
+ __rt_mutex_adjust_prio(current);
waiter->task = current;
waiter->lock = lock;
- debug_rt_mutex_reset_waiter(waiter);
-
- spin_lock(&current->pi_lock);
- current->pi_locked_by = current;
plist_node_init(&waiter->list_entry, current->prio);
plist_node_init(&waiter->pi_list_entry, current->prio);

- /* Get the top priority waiter of the lock: */
+ /* Get the top priority waiter on the lock */
if (rt_mutex_has_waiters(lock))
top_waiter = rt_mutex_top_waiter(lock);
plist_add(&waiter->list_entry, &lock->wait_list);

current->pi_blocked_on = waiter;

- /*
- * Call adjust_prio_chain, when waiter is the new top waiter
- * or when deadlock detection is requested:
- */
- if (waiter != rt_mutex_top_waiter(lock) &&
- !debug_rt_mutex_detect_deadlock(detect_deadlock))
- goto out_unlock_pi;
-
- /* Try to lock the full chain: */
- res = lock_pi_chain(lock, waiter, &lock_chain, 1, detect_deadlock);
-
- if (likely(res == 1))
- adjust_pi_chain(lock, waiter, top_waiter, &lock_chain);
-
- /* Common case: we managed to lock it: */
- if (res != -EBUSY)
- goto out_unlock_chain_pi;
+ _raw_spin_unlock(&current->pi_lock);

- /* Rare case: we hit some other task running a pi chain operation: */
- unlock_pi_chain(&lock_chain);
-
- plist_del(&waiter->list_entry, &lock->wait_list);
- current->pi_blocked_on = NULL;
- current->pi_locked_by = NULL;
- spin_unlock(&current->pi_lock);
- fixup_rt_mutex_waiters(lock);
-
- spin_unlock(&lock->wait_lock);
-
- /*
- * Here we have dropped all locks, and take the global
- * pi_conflicts_lock. We have to redo all the work, no
- * previous information about the lock is valid anymore:
- */
- spin_lock(&pi_conflicts_lock);
+ if (waiter == rt_mutex_top_waiter(lock)) {
+ _raw_spin_lock(&owner->pi_lock);
+ plist_del(&top_waiter->pi_list_entry, &owner->pi_waiters);
+ plist_add(&waiter->pi_list_entry, &owner->pi_waiters);
+
+ __rt_mutex_adjust_prio(owner);
+ if(owner->pi_blocked_on) {
+ /* Nested locks. We do the boosting of the next
+ tasks just before going to sleep in schedule */
+ res = owner;
+ get_task_struct(owner);
+ }

- spin_lock(&lock->wait_lock);
- if (try_to_take_rt_mutex(lock __IP__)) {
- /*
- * Rare race: against all odds we got the lock.
- */
- res = 0;
- goto out;
+ _raw_spin_unlock(&owner->pi_lock);
+ }
+ else if( debug_rt_mutex_detect_deadlock(detect_deadlock) ) {
+ _raw_spin_lock(&owner->pi_lock);
+ if(owner->pi_blocked_on) {
+ res = owner;
+ get_task_struct(owner);
+ }
+ _raw_spin_unlock(&owner->pi_lock);
}

- WARN_ON(!rt_mutex_owner(lock) || rt_mutex_owner(lock) == current);
-
- spin_lock(&current->pi_lock);
- current->pi_locked_by = current;
-
- plist_node_init(&waiter->list_entry, current->prio);
- plist_node_init(&waiter->pi_list_entry, current->prio);
-
- /* Get the top priority waiter of the lock: */
- if (rt_mutex_has_waiters(lock))
- top_waiter = rt_mutex_top_waiter(lock);
- plist_add(&waiter->list_entry, &lock->wait_list);
-
- current->pi_blocked_on = waiter;
-
- /* Lock the full chain: */
- res = lock_pi_chain(lock, waiter, &lock_chain, 0, detect_deadlock);
-
- /* Drop the conflicts lock before adjusting: */
- spin_unlock(&pi_conflicts_lock);
-
- if (likely(res == 1))
- adjust_pi_chain(lock, waiter, top_waiter, &lock_chain);

- out_unlock_chain_pi:
- unlock_pi_chain(&lock_chain);
- out_unlock_pi:
- current->pi_locked_by = NULL;
- spin_unlock(&current->pi_lock);
- out:
return res;
}

@@ -639,87 +490,49 @@ static void wakeup_next_waiter(struct rt
/*
* Remove a waiter from a lock
*
- * Must be called with lock->wait_lock held.
+ * Must be called with lock->wait_lock held
*/
-static int remove_waiter(struct rt_mutex *lock,
- struct rt_mutex_waiter *waiter __IP_DECL__)
+static task_t *remove_waiter(struct rt_mutex *lock,
+ struct rt_mutex_waiter *waiter __IP_DECL__)
{
- struct rt_mutex_waiter *next_waiter = NULL,
- *top_waiter = rt_mutex_top_waiter(lock);
- LIST_HEAD(lock_chain);
- int res;
+ int first = (waiter == rt_mutex_top_waiter(lock));
+ task_t *res = NULL;

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

- spin_lock(&current->pi_lock);
-
- if (waiter != top_waiter || rt_mutex_owner(lock) == current)
- goto out;
-
- current->pi_locked_by = current;

- if (rt_mutex_has_waiters(lock))
- next_waiter = rt_mutex_top_waiter(lock);
+ if (first && rt_mutex_owner(lock) != current) {
+ task_t *owner = rt_mutex_owner(lock);

- /* Try to lock the full chain: */
- res = lock_pi_chain(lock, next_waiter, &lock_chain, 1, 0);
+ _raw_spin_lock(&owner->pi_lock);

- if (likely(res != -EBUSY)) {
- adjust_pi_chain(lock, next_waiter, waiter, &lock_chain);
- goto out_unlock;
- }
+ plist_del(&waiter->pi_list_entry, &owner->pi_waiters);

- /* We hit some other task running a pi chain operation: */
- unlock_pi_chain(&lock_chain);
- plist_add(&waiter->list_entry, &lock->wait_list);
- current->pi_blocked_on = waiter;
- current->pi_locked_by = NULL;
- spin_unlock(&current->pi_lock);
- spin_unlock(&lock->wait_lock);
-
- spin_lock(&pi_conflicts_lock);
+ if (rt_mutex_has_waiters(lock)) {
+ struct rt_mutex_waiter *next;

- spin_lock(&lock->wait_lock);
+ next = rt_mutex_top_waiter(lock);
+ plist_add(&next->pi_list_entry, &owner->pi_waiters);
+ }

- spin_lock(&current->pi_lock);
- current->pi_locked_by = current;

- /* We might have been woken up: */
- if (!waiter->task) {
- spin_unlock(&pi_conflicts_lock);
- goto out;
+ __rt_mutex_adjust_prio(owner);
+ if(owner->pi_blocked_on) {
+ /* Owner is blocked on something - we have
+ to (un)boost throughout the lock chain but
+ we have to wait until we have dropped all locks */
+ res = owner;
+ get_task_struct(owner);
+ }
+ _raw_spin_unlock(&owner->pi_lock);
}

- top_waiter = rt_mutex_top_waiter(lock);
-
- plist_del(&waiter->list_entry, &lock->wait_list);
-
- if (waiter != top_waiter || rt_mutex_owner(lock) == current)
- goto out;
-
- /* Get the top priority waiter of the lock: */
- if (rt_mutex_has_waiters(lock))
- next_waiter = rt_mutex_top_waiter(lock);
-
- /* Lock the full chain: */
- lock_pi_chain(lock, next_waiter, &lock_chain, 0, 0);
-
- /* Drop the conflicts lock: */
- spin_unlock(&pi_conflicts_lock);
-
- adjust_pi_chain(lock, next_waiter, waiter, &lock_chain);
+ WARN_ON(!plist_node_empty(&waiter->pi_list_entry));

- out_unlock:
- unlock_pi_chain(&lock_chain);
- out:
current->pi_blocked_on = NULL;
waiter->task = NULL;
- current->pi_locked_by = NULL;
- spin_unlock(&current->pi_lock);

- WARN_ON(!plist_node_empty(&waiter->pi_list_entry));
-
- return 0;
+ return res;
}

#ifdef CONFIG_PREEMPT_RT
@@ -759,6 +572,7 @@ rt_lock_slowlock(struct rt_mutex *lock _
{
struct rt_mutex_waiter waiter;
unsigned long saved_state, state, flags;
+ task_t *owner;

debug_rt_mutex_init_waiter(&waiter);
waiter.task = NULL;
@@ -788,7 +602,7 @@ rt_lock_slowlock(struct rt_mutex *lock _

for (;;) {
unsigned long saved_flags;
- int ret, saved_lock_depth = current->lock_depth;
+ int saved_lock_depth = current->lock_depth;

/* Try to acquire the lock */
if (try_to_take_rt_mutex(lock __IP__))
@@ -798,12 +612,25 @@ rt_lock_slowlock(struct rt_mutex *lock _
* when we have been woken up by the previous owner
* but the lock got stolen by an higher prio task.
*/
- if (!waiter.task) {
- ret = task_blocks_on_rt_mutex(lock, &waiter, 0 __IP__);
- /* got the lock or deadlock: */
- if (ret == 0 || ret == -EDEADLK)
- break;
+ if (!waiter.task)
+ owner = task_blocks_on_rt_mutex(lock, &waiter, 0
+ __IP__);
+ else
+ owner = NULL;
+
+ if (unlikely(owner)) {
+ spin_unlock_irqrestore(&lock->wait_lock,flags);
+ rt_mutex_adjust_prio_chain(owner, 0, &waiter __IP__);
+ owner = NULL;
+ spin_lock_irqsave(&lock->wait_lock,flags);
+ if(unlikely(!waiter.task))
+ continue; /* We got woken up by the owner
+ * Start loop all over without
+ * going into schedule to try
+ * to get the lock now
+ */
}
+

/*
* Prevent schedule() to drop BKL, while waiting for
@@ -838,7 +665,9 @@ rt_lock_slowlock(struct rt_mutex *lock _
* can end up with a non-NULL waiter.task:
*/
if (unlikely(waiter.task))
- remove_waiter(lock, &waiter __IP__);
+ owner = remove_waiter(lock, &waiter __IP__);
+ else
+ owner = NULL;
/*
* try_to_take_rt_mutex() sets the waiter bit
* unconditionally. We might have to fix that up:
@@ -847,6 +676,9 @@ rt_lock_slowlock(struct rt_mutex *lock _

spin_unlock_irqrestore(&lock->wait_lock, flags);

+ if (unlikely(owner))
+ rt_mutex_adjust_prio_chain(owner, 0, &waiter __IP__);
+
debug_rt_mutex_free_waiter(&waiter);
}

@@ -936,6 +768,7 @@ rt_mutex_slowlock(struct rt_mutex *lock,
struct rt_mutex_waiter waiter;
int ret = 0, saved_lock_depth = -1;
unsigned long flags;
+ task_t *owner;

debug_rt_mutex_init_waiter(&waiter);
waiter.task = NULL;
@@ -950,8 +783,6 @@ rt_mutex_slowlock(struct rt_mutex *lock,
return 0;
}

- BUG_ON(rt_mutex_owner(lock) == current);
-
/*
* We drop BKL here before we go into the wait loop to avoid a
* possible deadlock in the scheduler.
@@ -992,15 +823,31 @@ rt_mutex_slowlock(struct rt_mutex *lock,
* when we have been woken up by the previous owner
* but the lock got stolen by an higher prio task.
*/
- if (!waiter.task) {
- ret = task_blocks_on_rt_mutex(lock, &waiter,
+ if (!waiter.task)
+ owner = task_blocks_on_rt_mutex(lock, &waiter,
detect_deadlock __IP__);
- /* got the lock or deadlock: */
- if (ret == 0 || ret == -EDEADLK)
+ else
+ owner = NULL;
+
+ if (unlikely(owner)) {
+ spin_unlock_irqrestore(&lock->wait_lock,flags);
+ ret = rt_mutex_adjust_prio_chain(owner,
+ detect_deadlock,
+ &waiter __IP__);
+ owner = NULL;
+ spin_lock_irqsave(&lock->wait_lock,flags);
+
+ if (unlikely(ret))
break;
- ret = 0;
- }

+ if (unlikely(!waiter.task))
+ continue; /* We got woken up by the owner
+ * Start loop all over without
+ * going into schedule to try
+ * to get the lock now
+ */
+ }
+
saved_flags = current->flags & PF_NOSCHED;
current->flags &= ~PF_NOSCHED;

@@ -1019,8 +866,10 @@ rt_mutex_slowlock(struct rt_mutex *lock,

set_current_state(TASK_RUNNING);

- if (unlikely(waiter.task))
- remove_waiter(lock, &waiter __IP__);
+ if (unlikely(waiter.task))
+ owner = remove_waiter(lock, &waiter __IP__);
+ else
+ owner = NULL;

/*
* try_to_take_rt_mutex() sets the waiter bit
@@ -1030,6 +879,9 @@ rt_mutex_slowlock(struct rt_mutex *lock,

spin_unlock_irqrestore(&lock->wait_lock, flags);

+ if (unlikely(owner))
+ rt_mutex_adjust_prio_chain(owner, 0, &waiter __IP__);
+
/* Remove pending timer: */
if (unlikely(timeout && timeout->task))
hrtimer_cancel(&timeout->timer);
Only in linux-2.6.16-rt10.pipatch/kernel: rtmutex.c.orig
Only in linux-2.6.16-rt10.pipatch/kernel: rtmutex.c.rej
Only in linux-2.6.16-rt10.pipatch/kernel: rtmutex.c~
diff -upr linux-2.6.16-rt10/kernel/sysctl.c linux-2.6.16-rt10.pipatch/kernel/sysctl.c
--- linux-2.6.16-rt10/kernel/sysctl.c 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/kernel/sysctl.c 2006-03-27 18:40:08.000000000 +0200
@@ -132,6 +132,10 @@ extern int acct_parm[];
extern int no_unaligned_warning;
#endif

+#ifdef CONFIG_RT_MUTEXES
+extern int max_lock_depth;
+#endif
+
static int parse_table(int __user *, int, void __user *, size_t __user *, void __user *, size_t,
ctl_table *, void **);
static int proc_doutsstring(ctl_table *table, int write, struct file *filp,
@@ -827,6 +831,17 @@ static ctl_table kern_table[] = {
.proc_handler = &proc_dointvec,
},
#endif
+#ifdef CONFIG_RT_MUTEXES
+ {
+ .ctl_name = KERN_MAX_LOCK_DEPTH,
+ .procname = "max_lock_depth",
+ .data = &max_lock_depth,
+ .maxlen = sizeof(int),
+ .mode = 0644,
+ .proc_handler = &proc_dointvec,
+ },
+#endif
+
{ .ctl_name = 0 }
};

diff -upr linux-2.6.16-rt10/scripts/rt-tester/t3-l1-pi-signal.tst linux-2.6.16-rt10.pipatch/scripts/rt-tester/t3-l1-pi-signal.tst
--- linux-2.6.16-rt10/scripts/rt-tester/t3-l1-pi-signal.tst 2006-03-28 00:47:05.000000000 +0200
+++ linux-2.6.16-rt10.pipatch/scripts/rt-tester/t3-l1-pi-signal.tst 2006-03-27 18:40:08.000000000 +0200
@@ -69,15 +69,18 @@ W: locked: 0: 0
C: locknowait: 1: 0
W: blocked: 1: 0
T: prioeq: 0: 80
+T: prioeq: 1: 80

# T2 lock L0 interruptible, no wait in the wakeup path
C: lockintnowait: 2: 0
W: blocked: 2: 0
T: prioeq: 0: 81
+T: prioeq: 1: 80

# Interrupt T2
C: signal: 2: 2
W: unlocked: 2: 0
+T: prioeq: 1: 80
T: prioeq: 0: 80

T: locked: 0: 0

> Esben
>
> > Ingo
> > -
> > To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> > the body of a message to [email protected]
> > More majordomo info at http://vger.kernel.org/majordomo-info.html
> > Please read the FAQ at http://www.tux.org/lkml/
> >
>
> -
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at http://www.tux.org/lkml/
>

2006-03-28 20:58:08

by Ingo Molnar

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


* Esben Nielsen <[email protected]> wrote:

> > we are observing a non-time-coherent snapshot of the locking graph.
> > There is no guarantee that due to timeouts or signals the chain we
> > observe isnt artificially long - while if a time-coherent snapshot is
> > taken it is always fine. E.g. lets take dentry locks as an example:
> > their locking is ordered by the dentry (kernel-pointer) address. We
> > could in theory have a 'chain' of subsequent locking dependencies
> > related to 10,000 dentries, which are nicely ordered and create a
> > 10,000-entry 'chain' if looked at in a non-time-coherent form. I.e. your
> > code could detect a deadlock where there's none. The more CPUs there
> > are, the larger the likelyhood is that other CPUs 'lure us' into a long
> > chain.
>
> I don't quite understand you examble: Are all 10,000 held at once?

no.

> If no, how are they all going to suddenly put into the lock chain due
> to signals or timeouts? Those things unlocks locks and therefore
> breaks the chain.

the core problem with your approach is that for each step in the
'boosting chain' (which can be quite long in theory), all that we are
holding is a task reference get get_task_struct(), to a task that was
blocked before. We then make ourselves preemptible - and once get get
back and continue the boosting chain, there is no guarantee that the
boosting makes any sense! Normally that task will probably still be
blocked, and we continue with our boosting. But the task could have
gotten unblocked, it could have gotten re-blocked, and we'd continue
doing the boosting.

in short: wow do you ensure that the boosting is still part of the same
dependency chain where it started off?

Ingo

2006-03-28 21:05:22

by Ingo Molnar

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


* Esben Nielsen <[email protected]> wrote:

> My girl friend will be angry for me not being to bed yet, but I had to
> steal time to make this patch. I hope I managed to send it without
> white-space damage or anything like it.

thanks - this looks fine to me, but i'm still worried about the
nonatomic chain boosting side-effects that i outlined in the previous
mail.

the tempting property of your patch is the fundamental reschedulability
of the boosting itself - and the resulting simplicity of locking. On the
con side, i dont see how we can detect deadlocks reliably, nor how we
can avoid 'incorrect boosting' (outlined in the previous mail), if the
chain gets 'broken'.

Ingo

2006-03-28 21:17:31

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Tue, 28 Mar 2006, Ingo Molnar wrote:

>
> * Esben Nielsen <[email protected]> wrote:
>
> > > we are observing a non-time-coherent snapshot of the locking graph.
> > > There is no guarantee that due to timeouts or signals the chain we
> > > observe isnt artificially long - while if a time-coherent snapshot is
> > > taken it is always fine. E.g. lets take dentry locks as an example:
> > > their locking is ordered by the dentry (kernel-pointer) address. We
> > > could in theory have a 'chain' of subsequent locking dependencies
> > > related to 10,000 dentries, which are nicely ordered and create a
> > > 10,000-entry 'chain' if looked at in a non-time-coherent form. I.e. your
> > > code could detect a deadlock where there's none. The more CPUs there
> > > are, the larger the likelyhood is that other CPUs 'lure us' into a long
> > > chain.
> >
> > I don't quite understand you examble: Are all 10,000 held at once?
>
> no.
>
> > If no, how are they all going to suddenly put into the lock chain due
> > to signals or timeouts? Those things unlocks locks and therefore
> > breaks the chain.
>
> the core problem with your approach is that for each step in the
> 'boosting chain' (which can be quite long in theory), all that we are
> holding is a task reference get get_task_struct(), to a task that was
> blocked before. We then make ourselves preemptible - and once get get
> back and continue the boosting chain, there is no guarantee that the
> boosting makes any sense! Normally that task will probably still be
> blocked, and we continue with our boosting. But the task could have
> gotten unblocked, it could have gotten re-blocked, and we'd continue
> doing the boosting.
>
> in short: wow do you ensure that the boosting is still part of the same
> dependency chain where it started off?
>

I don't insure that. But does it matter?!?
If the task is still blocked on a lock and the owner of that lock might
need boosting. The boosting operation itself will always be _correct_ as the
pi_lock is held when it is done. But the task doing the boosting might have
preempted for so long that there is nothing left to do - and then it
simply stops unless deadlock detection is on.

I think we talk about the situation

B locks 1 C locks 2 D locks 3
B locks 2, boosts C and block
A locks 2
A is boost B
A drop it's spinlocks and is preempted
C unlocks 2 and auto unboosts
B is running
B locks 3, boosts C and blocks
A gets a CPU again
A boosts B
A boosts D

Is there anything wrong with that?
And in the case where A==D there indeed is a deadlock which will be
detected.

Esben



> Ingo
> -
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at http://www.tux.org/lkml/
>

2006-03-28 21:27:22

by Ingo Molnar

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


* Esben Nielsen <[email protected]> wrote:

> > in short: wow do you ensure that the boosting is still part of the same
> > dependency chain where it started off?
>
> I don't insure that. But does it matter?!?

yes.

> If the task is still blocked on a lock and the owner of that lock
> might need boosting. The boosting operation itself will always be
> _correct_ as the pi_lock is held when it is done. But the task doing
> the boosting might have preempted for so long that there is nothing
> left to do - and then it simply stops unless deadlock detection is on.

well, another possibility is that the task got blocked again, and we'll
continue boosting _the wrong chain_. I.e. we'll add extra priority to
task(s) that might not deserve it at all (it doesnt own the lock we are
interested in anymore).

i.e. we must observe the boosting chain in a time-coherent form. We must
observe an actual "frozen" (all locks held) state of the system that we
_know_ forms a correct dependency chain at that moment, to be able to
propagate the priority one step forward. The act of 'boosting' must be
atomic.

Ingo

2006-03-28 21:35:47

by Thomas Gleixner

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Tue, 2006-03-28 at 22:17 +0100, Esben Nielsen wrote:
> I think we talk about the situation

No, we talk about existing lock chains L(0) --> L(n).

> B locks 1 C locks 2 D locks 3
> B locks 2, boosts C and block
> A locks 2
> A is boost B
> A drop it's spinlocks and is preempted
> C unlocks 2 and auto unboosts
> B is running
> B locks 3, boosts C and blocks
> A gets a CPU again
> A boosts B
> A boosts D
>
> Is there anything wrong with that?
> And in the case where A==D there indeed is a deadlock which will be
> detected.

If you get to L(x) the underlying dependencies might have changed
already as well as the dependencies x ... n. We might get false
positives in the deadlock detection that way, as a deadlock is an
"atomic" state.

tglx


2006-03-28 22:23:31

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Tue, 28 Mar 2006, Thomas Gleixner wrote:

> On Tue, 2006-03-28 at 22:17 +0100, Esben Nielsen wrote:
> > I think we talk about the situation
>
> No, we talk about existing lock chains L(0) --> L(n).
>
> > B locks 1 C locks 2 D locks 3
> > B locks 2, boosts C and block
> > A locks 2
> > A is boost B
> > A drop it's spinlocks and is preempted
> > C unlocks 2 and auto unboosts
> > B is running
> > B locks 3, boosts C and blocks
> > A gets a CPU again
> > A boosts B
> > A boosts D
> >
> > Is there anything wrong with that?
> > And in the case where A==D there indeed is a deadlock which will be
> > detected.
>
> If you get to L(x) the underlying dependencies might have changed
> already as well as the dependencies x ... n. We might get false
> positives in the deadlock detection that way, as a deadlock is an
> "atomic" state.

As I see it you might detect a circular lock graph "atomically". But is
that a "deadlock"? Yes, if you rule out signals and timeouts, this
situation does indeed deadlock your program.

But if you count in signals and timeouts your algoritm also gives "false
positives": You can detect a circular lock but when you return from
rt_mutex_slowlock(), a signal is delivered and there is no longer a
circular dependency and most important: The program wouldn't be
deadlocked even if you didn't ask for deadlock detection and your task in
that case would block.

I would like to see an examble of a false deadlock. I don't rule them out
in the present code. But they might be simple to fix.

Esben

>
> tglx
>
>
> -
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at http://www.tux.org/lkml/
>

2006-03-28 22:41:09

by Thomas Gleixner

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Tue, 2006-03-28 at 23:23 +0100, Esben Nielsen wrote:
> > If you get to L(x) the underlying dependencies might have changed
> > already as well as the dependencies x ... n. We might get false
> > positives in the deadlock detection that way, as a deadlock is an
> > "atomic" state.
>
> As I see it you might detect a circular lock graph "atomically". But is
> that a "deadlock"? Yes, if you rule out signals and timeouts, this
> situation does indeed deadlock your program.
>
> But if you count in signals and timeouts your algoritm also gives "false
> positives": You can detect a circular lock but when you return from
> rt_mutex_slowlock(), a signal is delivered and there is no longer a
> circular dependency and most important: The program wouldn't be
> deadlocked even if you didn't ask for deadlock detection and your task in
> that case would block.
>
> I would like to see an examble of a false deadlock. I don't rule them out
> in the present code. But they might be simple to fix.

Simply the initial lock chain is L1->L2->L3->L4, which is no deadlock.
Now in the course of your lock dropping L2 gets removed while you are at
L3 and L5 gets added on top of L4. You follow the chain blindly and
detect a dealock vs. L5, but its not longer valid. The L2 cleanup is
blocked by yourself. There is no way to prevent this with your method.

Your method is tempting, but I do not see how it works out right now.

tglx



2006-03-28 22:51:21

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Tue, 28 Mar 2006, Ingo Molnar wrote:

>
> * Esben Nielsen <[email protected]> wrote:
>
> > > in short: wow do you ensure that the boosting is still part of the same
> > > dependency chain where it started off?
> >
> > I don't insure that. But does it matter?!?
>
> yes.

How does it matter?

>
> > If the task is still blocked on a lock and the owner of that lock
> > might need boosting. The boosting operation itself will always be
> > _correct_ as the pi_lock is held when it is done. But the task doing
> > the boosting might have preempted for so long that there is nothing
> > left to do - and then it simply stops unless deadlock detection is on.
>
> well, another possibility is that the task got blocked again, and we'll
> continue boosting _the wrong chain_. I.e. we'll add extra priority to
> task(s) that might not deserve it at all (it doesnt own the lock we are
> interested in anymore).
>

This can't happen. We are always looking at the first waiter on
task->pi_waiter task->pi_lock held when doing the boosting. If task
has released the lock the entry task->pi_waiter is gone and no boosting
will take place!

> i.e. we must observe the boosting chain in a time-coherent form. We must
> observe an actual "frozen" (all locks held) state of the system that we
> _know_ forms a correct dependency chain at that moment, to be able to
> propagate the priority one step forward. The act of 'boosting' must be
> atomic.
>
The i.e. refers to wrong conclusion and therefore the above can't be
concluded.

Esben

> Ingo
> -
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at http://www.tux.org/lkml/
>

2006-03-28 23:35:11

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Wed, 29 Mar 2006, Thomas Gleixner wrote:

> On Tue, 2006-03-28 at 23:23 +0100, Esben Nielsen wrote:
> > > If you get to L(x) the underlying dependencies might have changed
> > > already as well as the dependencies x ... n. We might get false
> > > positives in the deadlock detection that way, as a deadlock is an
> > > "atomic" state.
> >
> > As I see it you might detect a circular lock graph "atomically". But is
> > that a "deadlock"? Yes, if you rule out signals and timeouts, this
> > situation does indeed deadlock your program.
> >
> > But if you count in signals and timeouts your algoritm also gives "false
> > positives": You can detect a circular lock but when you return from
> > rt_mutex_slowlock(), a signal is delivered and there is no longer a
> > circular dependency and most important: The program wouldn't be
> > deadlocked even if you didn't ask for deadlock detection and your task in
> > that case would block.
> >
> > I would like to see an examble of a false deadlock. I don't rule them out
> > in the present code. But they might be simple to fix.
>
> Simply the initial lock chain is L1->L2->L3->L4, which is no deadlock.
> Now in the course of your lock dropping L2 gets removed while you are at
> L3 and L5 gets added on top of L4. You follow the chain blindly and
> detect a dealock vs. L5, but its not longer valid. The L2 cleanup is
> blocked by yourself. There is no way to prevent this with your method.
>

Hmm, let me try to write it out

A B C D
lock L1 lock L2 lock L3 lock L4
lock L2 lock L3 lock L4
traverse to C
is preempted
unlock L4
unlock L4
unlock L3
unlock L3 lock L4
unlock L2 lock L3
lock L3 lock L4

Continue from C

Ok, I see the problem for _deadlock detection_. There still is no problem
for PI.



> Your method is tempting, but I do not see how it works out right now.
>

It works for PI. It might give false positives for deadlock detection even
without signals involved. But that might be solved by simply checking
again. If it is stored on a task when they blocked on a lock it
could be seen if they had released and reobtained the task since the last
traversal.

If I should choose between a 100% certain deadlock detection and
rescheduling while doing PI I would choose that latter as that gives a
deterministic RT system. Are there at all applications depending on
deadlock detection or is it only for debug perposes anyway?

Esben

> tglx
>
>
>
> -
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at http://www.tux.org/lkml/
>


2006-03-28 23:58:25

by Thomas Gleixner

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Wed, 2006-03-29 at 00:34 +0100, Esben Nielsen wrote:
> > Your method is tempting, but I do not see how it works out right now
>
> It works for PI.

Well, works and effective are two things. In the worst case it
introduces scheduler floods.

> It might give false positives for deadlock detection even
> without signals involved. But that might be solved by simply checking
> again.

Which is even more broken. Rechecking is less deterministic as the
global lock fall back solution.

> If it is stored on a task when they blocked on a lock it
> could be seen if they had released and reobtained the task since the last
> traversal.

-ENOPARSE

> If I should choose between a 100% certain deadlock detection and
> rescheduling while doing PI I would choose that latter as that gives a
> deterministic RT system. Are there at all applications depending on
> deadlock detection or is it only for debug perposes anyway?

No, userspace can request deadlock checking and we have to return
-EDEADLK in that case.

[EDEADLK]
A deadlock condition was detected or the current thread already
owns the mutex.

Returning false positives might break well designed applications and
prevent real deadlock detection.

Btw, your get/put_task proposal adds two atomic ops. Atomic ops are
implicit memory barriers and therefor you add two extra slow downs into
the non conflict case.

tglx


2006-03-29 07:17:29

by Ingo Molnar

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


* Esben Nielsen <[email protected]> wrote:

> > well, another possibility is that the task got blocked again, and we'll
> > continue boosting _the wrong chain_. I.e. we'll add extra priority to
> > task(s) that might not deserve it at all (it doesnt own the lock we are
> > interested in anymore).
>
> This can't happen. We are always looking at the first waiter on
> task->pi_waiter task->pi_lock held when doing the boosting. If task
> has released the lock the entry task->pi_waiter is gone and no
> boosting will take place!

no, the task got blocked _again_, as part of a _new_ blocking chain, and
there's a _new_ PI waiter! How does the two-lock preemptible boosting
algorithm ensure that if we are in the middle of boosting a
blocking-dependency chain:

T1 -> T2 -> ... -> TI -> TI+1 -> ... TN-1 -> TN

we are at TI, and we [the task doing the boosting] now get preempted.

What prevents TI from being part of a _totally new_ blocking-chain,
where the only similarity between the two chains is that TI is in the
middle of it:

T1' -> T2' -> ... -> TI -> TI+1' -> ... TM-1 -> TM'

the only match between the two chains is 'TI'. Now the algorithm will
happily walk the wrong boosting chain, and will boost the wrong tasks.

Ingo

2006-03-29 07:59:39

by Esben Nielsen

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9

On Wed, 29 Mar 2006, Ingo Molnar wrote:

>
> * Esben Nielsen <[email protected]> wrote:
>
> > > well, another possibility is that the task got blocked again, and we'll
> > > continue boosting _the wrong chain_. I.e. we'll add extra priority to
> > > task(s) that might not deserve it at all (it doesnt own the lock we are
> > > interested in anymore).
> >
> > This can't happen. We are always looking at the first waiter on
> > task->pi_waiter task->pi_lock held when doing the boosting. If task
> > has released the lock the entry task->pi_waiter is gone and no
> > boosting will take place!
>
> no, the task got blocked _again_, as part of a _new_ blocking chain, and
> there's a _new_ PI waiter! How does the two-lock preemptible boosting
> algorithm ensure that if we are in the middle of boosting a
> blocking-dependency chain:
>
> T1 -> T2 -> ... -> TI -> TI+1 -> ... TN-1 -> TN
>
> we are at TI, and we [the task doing the boosting] now get preempted.
>
> What prevents TI from being part of a _totally new_ blocking-chain,
> where the only similarity between the two chains is that TI is in the
> middle of it:
>
> T1' -> T2' -> ... -> TI -> TI+1' -> ... TM-1 -> TM'
>
> the only match between the two chains is 'TI'. Now the algorithm will
> happily walk the wrong boosting chain, and will boost the wrong tasks.
>

The point is: It does not matter that is another chain!

It will _not_ boost any task which doesn't need boosting, because it is
not boosting according to current->prio but always task->pi_waiters. So
all it does is to fix the priorities on some tasks. There is
absolutely nothing wrong with that. But these task will already have the
right priorities, because the new outermost locker (T1') will have
traversed the list and done the boosting. So current will stop boosting
and break out of the loop (unless it is doing deadlock detection).

But what about the original chain? Well, as the tasks aren't blocked
anymore, they doesn't need boosting anymore, so no boosting missed either.
Or if they are blocked on something else, the those locking operations
have or will take care of it.

Esben



> Ingo
> -
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at http://www.tux.org/lkml/
>

2006-03-29 12:32:36

by Ingo Molnar

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


* Thomas Gleixner <[email protected]> wrote:

> Btw, your get/put_task proposal adds two atomic ops. Atomic ops are
> implicit memory barriers and therefor you add two extra slow downs
> into the non conflict case.

i'm not that worried about this - the atomic ops are for already cached
cachelines, any sane CPU ought to execute them close to full speed.
(x86-ish cpus certainly do)

Ingo

2006-03-29 12:37:59

by Ingo Molnar

[permalink] [raw]
Subject: Re: PI patch against 2.6.16-rt9


* Esben Nielsen <[email protected]> wrote:

> The point is: It does not matter that is another chain!
>
> It will _not_ boost any task which doesn't need boosting, because it
> is not boosting according to current->prio but always
> task->pi_waiters. So all it does is to fix the priorities on some
> tasks. There is absolutely nothing wrong with that. [...]

doh, you are right, i missed that. All the state to do the boosting is
contained in a single entry along the chain, so no prior information is
needed.

the problem with deadlock detection remains though. Can we live with
deadlock detection being a bit statistical? I think we can: deadlock
detection is for _bugs_, no application should rely on it to provide
actual functionality. (if it still does it will still work fine, but we
dont design for them.) Also, if we walk long enough (say 1024 entries)
the probability of a false positive ought to be pretty low. So i think
the following type of deadlock detection ought to be pretty OK:

- check whether we get back to 'current'.

- check whether we exceed a configurable limit of steps

most 'sane' deadlocks will be detected quickly: they'll lead back to
'current' and the kernel returns. On the off chance of the chain-walking
getting lured into a completely unrelated chain the limit will catch it.

Ingo