summaryrefslogtreecommitdiffstats
path: root/kernel/futex
diff options
context:
space:
mode:
authorJens Axboe <axboe@kernel.dk>2023-09-28 07:47:07 -0600
committerJens Axboe <axboe@kernel.dk>2023-09-28 07:47:07 -0600
commit52e856c38761bae0cea09b25cfbb4d46cd930d45 (patch)
treee94c5b41e4ea4c8f95b993c6d875e2e7db78bbf8 /kernel/futex
parent73c7e7a91f041f4c2e3c0db1e727163b331c60c9 (diff)
parentcfa92b6d52071aaa8f27d21affdcb14e7448fbc1 (diff)
downloadlinux-stable-52e856c38761bae0cea09b25cfbb4d46cd930d45.tar.gz
linux-stable-52e856c38761bae0cea09b25cfbb4d46cd930d45.tar.bz2
linux-stable-52e856c38761bae0cea09b25cfbb4d46cd930d45.zip
Merge branch 'locking/core' of git://git.kernel.org/pub/scm/linux/kernel/git/tip/tip into io_uring-futex
Pull in locking/core from the tip tree, to get the futex2 dependencies from Peter Zijlstra. * 'locking/core' of git://git.kernel.org/pub/scm/linux/kernel/git/tip/tip: (24 commits) locking/ww_mutex/test: Make sure we bail out instead of livelock locking/ww_mutex/test: Fix potential workqueue corruption locking/ww_mutex/test: Use prng instead of rng to avoid hangs at bootup futex: Add sys_futex_requeue() futex: Add flags2 argument to futex_requeue() futex: Propagate flags into get_futex_key() futex: Add sys_futex_wait() futex: FLAGS_STRICT futex: Add sys_futex_wake() futex: Validate futex value against futex size futex: Flag conversion futex: Extend the FUTEX2 flags futex: Clarify FUTEX2 flags asm-generic: ticket-lock: Optimize arch_spin_value_unlocked() futex/pi: Fix recursive rt_mutex waiter state locking/rtmutex: Add a lockdep assert to catch potential nested blocking locking/rtmutex: Use rt_mutex specific scheduler helpers sched: Provide rt_mutex specific scheduler helpers sched: Extract __schedule_loop() locking/rtmutex: Avoid unconditional slowpath for DEBUG_RT_MUTEXES ...
Diffstat (limited to 'kernel/futex')
-rw-r--r--kernel/futex/core.c74
-rw-r--r--kernel/futex/futex.h86
-rw-r--r--kernel/futex/pi.c91
-rw-r--r--kernel/futex/requeue.c20
-rw-r--r--kernel/futex/syscalls.c221
-rw-r--r--kernel/futex/waitwake.c80
6 files changed, 412 insertions, 160 deletions
diff --git a/kernel/futex/core.c b/kernel/futex/core.c
index f10587d1d481..ade7c731972d 100644
--- a/kernel/futex/core.c
+++ b/kernel/futex/core.c
@@ -193,7 +193,7 @@ static u64 get_inode_sequence_number(struct inode *inode)
/**
* get_futex_key() - Get parameters which are the keys for a futex
* @uaddr: virtual address of the futex
- * @fshared: false for a PROCESS_PRIVATE futex, true for PROCESS_SHARED
+ * @flags: FLAGS_*
* @key: address where result is stored.
* @rw: mapping needs to be read/write (values: FUTEX_READ,
* FUTEX_WRITE)
@@ -217,14 +217,18 @@ static u64 get_inode_sequence_number(struct inode *inode)
*
* lock_page() might sleep, the caller should not hold a spinlock.
*/
-int get_futex_key(u32 __user *uaddr, bool fshared, union futex_key *key,
+int get_futex_key(u32 __user *uaddr, unsigned int flags, union futex_key *key,
enum futex_access rw)
{
unsigned long address = (unsigned long)uaddr;
struct mm_struct *mm = current->mm;
- struct page *page, *tail;
+ struct page *page;
+ struct folio *folio;
struct address_space *mapping;
int err, ro = 0;
+ bool fshared;
+
+ fshared = flags & FLAGS_SHARED;
/*
* The futex address must be "naturally" aligned.
@@ -273,54 +277,52 @@ again:
err = 0;
/*
- * The treatment of mapping from this point on is critical. The page
- * lock protects many things but in this context the page lock
+ * The treatment of mapping from this point on is critical. The folio
+ * lock protects many things but in this context the folio lock
* stabilizes mapping, prevents inode freeing in the shared
* file-backed region case and guards against movement to swap cache.
*
- * Strictly speaking the page lock is not needed in all cases being
- * considered here and page lock forces unnecessarily serialization
+ * Strictly speaking the folio lock is not needed in all cases being
+ * considered here and folio lock forces unnecessarily serialization.
* From this point on, mapping will be re-verified if necessary and
- * page lock will be acquired only if it is unavoidable
+ * folio lock will be acquired only if it is unavoidable
*
- * Mapping checks require the head page for any compound page so the
- * head page and mapping is looked up now. For anonymous pages, it
- * does not matter if the page splits in the future as the key is
- * based on the address. For filesystem-backed pages, the tail is
- * required as the index of the page determines the key. For
- * base pages, there is no tail page and tail == page.
+ * Mapping checks require the folio so it is looked up now. For
+ * anonymous pages, it does not matter if the folio is split
+ * in the future as the key is based on the address. For
+ * filesystem-backed pages, the precise page is required as the
+ * index of the page determines the key.
*/
- tail = page;
- page = compound_head(page);
- mapping = READ_ONCE(page->mapping);
+ folio = page_folio(page);
+ mapping = READ_ONCE(folio->mapping);
/*
- * If page->mapping is NULL, then it cannot be a PageAnon
+ * If folio->mapping is NULL, then it cannot be an anonymous
* page; but it might be the ZERO_PAGE or in the gate area or
* in a special mapping (all cases which we are happy to fail);
* or it may have been a good file page when get_user_pages_fast
* found it, but truncated or holepunched or subjected to
- * invalidate_complete_page2 before we got the page lock (also
+ * invalidate_complete_page2 before we got the folio lock (also
* cases which we are happy to fail). And we hold a reference,
* so refcount care in invalidate_inode_page's remove_mapping
* prevents drop_caches from setting mapping to NULL beneath us.
*
* The case we do have to guard against is when memory pressure made
* shmem_writepage move it from filecache to swapcache beneath us:
- * an unlikely race, but we do need to retry for page->mapping.
+ * an unlikely race, but we do need to retry for folio->mapping.
*/
if (unlikely(!mapping)) {
int shmem_swizzled;
/*
- * Page lock is required to identify which special case above
- * applies. If this is really a shmem page then the page lock
+ * Folio lock is required to identify which special case above
+ * applies. If this is really a shmem page then the folio lock
* will prevent unexpected transitions.
*/
- lock_page(page);
- shmem_swizzled = PageSwapCache(page) || page->mapping;
- unlock_page(page);
- put_page(page);
+ folio_lock(folio);
+ shmem_swizzled = folio_test_swapcache(folio) || folio->mapping;
+ folio_unlock(folio);
+ folio_put(folio);
if (shmem_swizzled)
goto again;
@@ -331,14 +333,14 @@ again:
/*
* Private mappings are handled in a simple way.
*
- * If the futex key is stored on an anonymous page, then the associated
+ * If the futex key is stored in anonymous memory, then the associated
* object is the mm which is implicitly pinned by the calling process.
*
* NOTE: When userspace waits on a MAP_SHARED mapping, even if
* it's a read-only handle, it's expected that futexes attach to
* the object not the particular process.
*/
- if (PageAnon(page)) {
+ if (folio_test_anon(folio)) {
/*
* A RO anonymous page will never change and thus doesn't make
* sense for futex operations.
@@ -357,10 +359,10 @@ again:
/*
* The associated futex object in this case is the inode and
- * the page->mapping must be traversed. Ordinarily this should
- * be stabilised under page lock but it's not strictly
+ * the folio->mapping must be traversed. Ordinarily this should
+ * be stabilised under folio lock but it's not strictly
* necessary in this case as we just want to pin the inode, not
- * update the radix tree or anything like that.
+ * update i_pages or anything like that.
*
* The RCU read lock is taken as the inode is finally freed
* under RCU. If the mapping still matches expectations then the
@@ -368,9 +370,9 @@ again:
*/
rcu_read_lock();
- if (READ_ONCE(page->mapping) != mapping) {
+ if (READ_ONCE(folio->mapping) != mapping) {
rcu_read_unlock();
- put_page(page);
+ folio_put(folio);
goto again;
}
@@ -378,19 +380,19 @@ again:
inode = READ_ONCE(mapping->host);
if (!inode) {
rcu_read_unlock();
- put_page(page);
+ folio_put(folio);
goto again;
}
key->both.offset |= FUT_OFF_INODE; /* inode-based key */
key->shared.i_seq = get_inode_sequence_number(inode);
- key->shared.pgoff = page_to_pgoff(tail);
+ key->shared.pgoff = folio->index + folio_page_idx(folio, page);
rcu_read_unlock();
}
out:
- put_page(page);
+ folio_put(folio);
return err;
}
diff --git a/kernel/futex/futex.h b/kernel/futex/futex.h
index b5379c0e6d6d..a06030a1a27b 100644
--- a/kernel/futex/futex.h
+++ b/kernel/futex/futex.h
@@ -5,6 +5,7 @@
#include <linux/futex.h>
#include <linux/rtmutex.h>
#include <linux/sched/wake_q.h>
+#include <linux/compat.h>
#ifdef CONFIG_PREEMPT_RT
#include <linux/rcuwait.h>
@@ -16,17 +17,84 @@
* Futex flags used to encode options to functions and preserve them across
* restarts.
*/
+#define FLAGS_SIZE_8 0x0000
+#define FLAGS_SIZE_16 0x0001
+#define FLAGS_SIZE_32 0x0002
+#define FLAGS_SIZE_64 0x0003
+
+#define FLAGS_SIZE_MASK 0x0003
+
#ifdef CONFIG_MMU
-# define FLAGS_SHARED 0x01
+# define FLAGS_SHARED 0x0010
#else
/*
* NOMMU does not have per process address space. Let the compiler optimize
* code away.
*/
-# define FLAGS_SHARED 0x00
+# define FLAGS_SHARED 0x0000
#endif
-#define FLAGS_CLOCKRT 0x02
-#define FLAGS_HAS_TIMEOUT 0x04
+#define FLAGS_CLOCKRT 0x0020
+#define FLAGS_HAS_TIMEOUT 0x0040
+#define FLAGS_NUMA 0x0080
+#define FLAGS_STRICT 0x0100
+
+/* FUTEX_ to FLAGS_ */
+static inline unsigned int futex_to_flags(unsigned int op)
+{
+ unsigned int flags = FLAGS_SIZE_32;
+
+ if (!(op & FUTEX_PRIVATE_FLAG))
+ flags |= FLAGS_SHARED;
+
+ if (op & FUTEX_CLOCK_REALTIME)
+ flags |= FLAGS_CLOCKRT;
+
+ return flags;
+}
+
+/* FUTEX2_ to FLAGS_ */
+static inline unsigned int futex2_to_flags(unsigned int flags2)
+{
+ unsigned int flags = flags2 & FUTEX2_SIZE_MASK;
+
+ if (!(flags2 & FUTEX2_PRIVATE))
+ flags |= FLAGS_SHARED;
+
+ if (flags2 & FUTEX2_NUMA)
+ flags |= FLAGS_NUMA;
+
+ return flags;
+}
+
+static inline unsigned int futex_size(unsigned int flags)
+{
+ return 1 << (flags & FLAGS_SIZE_MASK);
+}
+
+static inline bool futex_flags_valid(unsigned int flags)
+{
+ /* Only 64bit futexes for 64bit code */
+ if (!IS_ENABLED(CONFIG_64BIT) || in_compat_syscall()) {
+ if ((flags & FLAGS_SIZE_MASK) == FLAGS_SIZE_64)
+ return false;
+ }
+
+ /* Only 32bit futexes are implemented -- for now */
+ if ((flags & FLAGS_SIZE_MASK) != FLAGS_SIZE_32)
+ return false;
+
+ return true;
+}
+
+static inline bool futex_validate_input(unsigned int flags, u64 val)
+{
+ int bits = 8 * futex_size(flags);
+
+ if (bits < 64 && (val >> bits))
+ return false;
+
+ return true;
+}
#ifdef CONFIG_FAIL_FUTEX
extern bool should_fail_futex(bool fshared);
@@ -116,7 +184,7 @@ enum futex_access {
FUTEX_WRITE
};
-extern int get_futex_key(u32 __user *uaddr, bool fshared, union futex_key *key,
+extern int get_futex_key(u32 __user *uaddr, unsigned int flags, union futex_key *key,
enum futex_access rw);
extern struct hrtimer_sleeper *
@@ -260,10 +328,14 @@ extern int futex_wait_requeue_pi(u32 __user *uaddr, unsigned int flags, u32
val, ktime_t *abs_time, u32 bitset, u32 __user
*uaddr2);
-extern int futex_requeue(u32 __user *uaddr1, unsigned int flags,
- u32 __user *uaddr2, int nr_wake, int nr_requeue,
+extern int futex_requeue(u32 __user *uaddr1, unsigned int flags1,
+ u32 __user *uaddr2, unsigned int flags2,
+ int nr_wake, int nr_requeue,
u32 *cmpval, int requeue_pi);
+extern int __futex_wait(u32 __user *uaddr, unsigned int flags, u32 val,
+ struct hrtimer_sleeper *to, u32 bitset);
+
extern int futex_wait(u32 __user *uaddr, unsigned int flags, u32 val,
ktime_t *abs_time, u32 bitset);
diff --git a/kernel/futex/pi.c b/kernel/futex/pi.c
index ce2889f12375..90e5197f4e56 100644
--- a/kernel/futex/pi.c
+++ b/kernel/futex/pi.c
@@ -1,6 +1,7 @@
// SPDX-License-Identifier: GPL-2.0-or-later
#include <linux/slab.h>
+#include <linux/sched/rt.h>
#include <linux/sched/task.h>
#include "futex.h"
@@ -610,29 +611,16 @@ int futex_lock_pi_atomic(u32 __user *uaddr, struct futex_hash_bucket *hb,
/*
* Caller must hold a reference on @pi_state.
*/
-static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_pi_state *pi_state)
+static int wake_futex_pi(u32 __user *uaddr, u32 uval,
+ struct futex_pi_state *pi_state,
+ struct rt_mutex_waiter *top_waiter)
{
- struct rt_mutex_waiter *top_waiter;
struct task_struct *new_owner;
bool postunlock = false;
DEFINE_RT_WAKE_Q(wqh);
u32 curval, newval;
int ret = 0;
- top_waiter = rt_mutex_top_waiter(&pi_state->pi_mutex);
- if (WARN_ON_ONCE(!top_waiter)) {
- /*
- * As per the comment in futex_unlock_pi() this should not happen.
- *
- * When this happens, give up our locks and try again, giving
- * the futex_lock_pi() instance time to complete, either by
- * waiting on the rtmutex or removing itself from the futex
- * queue.
- */
- ret = -EAGAIN;
- goto out_unlock;
- }
-
new_owner = top_waiter->task;
/*
@@ -945,7 +933,7 @@ int futex_lock_pi(u32 __user *uaddr, unsigned int flags, ktime_t *time, int tryl
to = futex_setup_timer(time, &timeout, flags, 0);
retry:
- ret = get_futex_key(uaddr, flags & FLAGS_SHARED, &q.key, FUTEX_WRITE);
+ ret = get_futex_key(uaddr, flags, &q.key, FUTEX_WRITE);
if (unlikely(ret != 0))
goto out;
@@ -1002,6 +990,12 @@ retry_private:
goto no_block;
}
+ /*
+ * Must be done before we enqueue the waiter, here is unfortunately
+ * under the hb lock, but that *should* work because it does nothing.
+ */
+ rt_mutex_pre_schedule();
+
rt_mutex_init_waiter(&rt_waiter);
/*
@@ -1039,19 +1033,37 @@ retry_private:
ret = rt_mutex_wait_proxy_lock(&q.pi_state->pi_mutex, to, &rt_waiter);
cleanup:
- spin_lock(q.lock_ptr);
/*
* If we failed to acquire the lock (deadlock/signal/timeout), we must
- * first acquire the hb->lock before removing the lock from the
- * rt_mutex waitqueue, such that we can keep the hb and rt_mutex wait
- * lists consistent.
+ * must unwind the above, however we canont lock hb->lock because
+ * rt_mutex already has a waiter enqueued and hb->lock can itself try
+ * and enqueue an rt_waiter through rtlock.
+ *
+ * Doing the cleanup without holding hb->lock can cause inconsistent
+ * state between hb and pi_state, but only in the direction of not
+ * seeing a waiter that is leaving.
+ *
+ * See futex_unlock_pi(), it deals with this inconsistency.
+ *
+ * There be dragons here, since we must deal with the inconsistency on
+ * the way out (here), it is impossible to detect/warn about the race
+ * the other way around (missing an incoming waiter).
*
- * In particular; it is important that futex_unlock_pi() can not
- * observe this inconsistency.
+ * What could possibly go wrong...
*/
if (ret && !rt_mutex_cleanup_proxy_lock(&q.pi_state->pi_mutex, &rt_waiter))
ret = 0;
+ /*
+ * Now that the rt_waiter has been dequeued, it is safe to use
+ * spinlock/rtlock (which might enqueue its own rt_waiter) and fix up
+ * the
+ */
+ spin_lock(q.lock_ptr);
+ /*
+ * Waiter is unqueued.
+ */
+ rt_mutex_post_schedule();
no_block:
/*
* Fixup the pi_state owner and possibly acquire the lock if we
@@ -1117,7 +1129,7 @@ retry:
if ((uval & FUTEX_TID_MASK) != vpid)
return -EPERM;
- ret = get_futex_key(uaddr, flags & FLAGS_SHARED, &key, FUTEX_WRITE);
+ ret = get_futex_key(uaddr, flags, &key, FUTEX_WRITE);
if (ret)
return ret;
@@ -1132,6 +1144,7 @@ retry:
top_waiter = futex_top_waiter(hb, &key);
if (top_waiter) {
struct futex_pi_state *pi_state = top_waiter->pi_state;
+ struct rt_mutex_waiter *rt_waiter;
ret = -EINVAL;
if (!pi_state)
@@ -1144,22 +1157,39 @@ retry:
if (pi_state->owner != current)
goto out_unlock;
- get_pi_state(pi_state);
/*
* By taking wait_lock while still holding hb->lock, we ensure
- * there is no point where we hold neither; and therefore
- * wake_futex_p() must observe a state consistent with what we
- * observed.
+ * there is no point where we hold neither; and thereby
+ * wake_futex_pi() must observe any new waiters.
+ *
+ * Since the cleanup: case in futex_lock_pi() removes the
+ * rt_waiter without holding hb->lock, it is possible for
+ * wake_futex_pi() to not find a waiter while the above does,
+ * in this case the waiter is on the way out and it can be
+ * ignored.
*
* In particular; this forces __rt_mutex_start_proxy() to
* complete such that we're guaranteed to observe the
- * rt_waiter. Also see the WARN in wake_futex_pi().
+ * rt_waiter.
*/
raw_spin_lock_irq(&pi_state->pi_mutex.wait_lock);
+
+ /*
+ * Futex vs rt_mutex waiter state -- if there are no rt_mutex
+ * waiters even though futex thinks there are, then the waiter
+ * is leaving and the uncontended path is safe to take.
+ */
+ rt_waiter = rt_mutex_top_waiter(&pi_state->pi_mutex);
+ if (!rt_waiter) {
+ raw_spin_unlock_irq(&pi_state->pi_mutex.wait_lock);
+ goto do_uncontended;
+ }
+
+ get_pi_state(pi_state);
spin_unlock(&hb->lock);
/* drops pi_state->pi_mutex.wait_lock */
- ret = wake_futex_pi(uaddr, uval, pi_state);
+ ret = wake_futex_pi(uaddr, uval, pi_state, rt_waiter);
put_pi_state(pi_state);
@@ -1187,6 +1217,7 @@ retry:
return ret;
}
+do_uncontended:
/*
* We have no kernel internal state, i.e. no waiters in the
* kernel. Waiters which are about to queue themselves are stuck
diff --git a/kernel/futex/requeue.c b/kernel/futex/requeue.c
index cba8b1a6a4cc..a0a79954f506 100644
--- a/kernel/futex/requeue.c
+++ b/kernel/futex/requeue.c
@@ -346,8 +346,9 @@ futex_proxy_trylock_atomic(u32 __user *pifutex, struct futex_hash_bucket *hb1,
/**
* futex_requeue() - Requeue waiters from uaddr1 to uaddr2
* @uaddr1: source futex user address
- * @flags: futex flags (FLAGS_SHARED, etc.)
+ * @flags1: futex flags (FLAGS_SHARED, etc.)
* @uaddr2: target futex user address
+ * @flags2: futex flags (FLAGS_SHARED, etc.)
* @nr_wake: number of waiters to wake (must be 1 for requeue_pi)
* @nr_requeue: number of waiters to requeue (0-INT_MAX)
* @cmpval: @uaddr1 expected value (or %NULL)
@@ -361,7 +362,8 @@ futex_proxy_trylock_atomic(u32 __user *pifutex, struct futex_hash_bucket *hb1,
* - >=0 - on success, the number of tasks requeued or woken;
* - <0 - on error
*/
-int futex_requeue(u32 __user *uaddr1, unsigned int flags, u32 __user *uaddr2,
+int futex_requeue(u32 __user *uaddr1, unsigned int flags1,
+ u32 __user *uaddr2, unsigned int flags2,
int nr_wake, int nr_requeue, u32 *cmpval, int requeue_pi)
{
union futex_key key1 = FUTEX_KEY_INIT, key2 = FUTEX_KEY_INIT;
@@ -424,10 +426,10 @@ int futex_requeue(u32 __user *uaddr1, unsigned int flags, u32 __user *uaddr2,
}
retry:
- ret = get_futex_key(uaddr1, flags & FLAGS_SHARED, &key1, FUTEX_READ);
+ ret = get_futex_key(uaddr1, flags1, &key1, FUTEX_READ);
if (unlikely(ret != 0))
return ret;
- ret = get_futex_key(uaddr2, flags & FLAGS_SHARED, &key2,
+ ret = get_futex_key(uaddr2, flags2, &key2,
requeue_pi ? FUTEX_WRITE : FUTEX_READ);
if (unlikely(ret != 0))
return ret;
@@ -459,7 +461,7 @@ retry_private:
if (ret)
return ret;
- if (!(flags & FLAGS_SHARED))
+ if (!(flags1 & FLAGS_SHARED))
goto retry_private;
goto retry;
@@ -789,7 +791,7 @@ int futex_wait_requeue_pi(u32 __user *uaddr, unsigned int flags,
*/
rt_mutex_init_waiter(&rt_waiter);
- ret = get_futex_key(uaddr2, flags & FLAGS_SHARED, &key2, FUTEX_WRITE);
+ ret = get_futex_key(uaddr2, flags, &key2, FUTEX_WRITE);
if (unlikely(ret != 0))
goto out;
@@ -850,11 +852,13 @@ int futex_wait_requeue_pi(u32 __user *uaddr, unsigned int flags,
pi_mutex = &q.pi_state->pi_mutex;
ret = rt_mutex_wait_proxy_lock(pi_mutex, to, &rt_waiter);
- /* Current is not longer pi_blocked_on */
- spin_lock(q.lock_ptr);
+ /*
+ * See futex_unlock_pi()'s cleanup: comment.
+ */
if (ret && !rt_mutex_cleanup_proxy_lock(pi_mutex, &rt_waiter))
ret = 0;
+ spin_lock(q.lock_ptr);
debug_rt_mutex_free_waiter(&rt_waiter);
/*
* Fixup the pi_state owner and possibly acquire the lock if we
diff --git a/kernel/futex/syscalls.c b/kernel/futex/syscalls.c
index a8074079b09e..8200d86d30e1 100644
--- a/kernel/futex/syscalls.c
+++ b/kernel/futex/syscalls.c
@@ -1,6 +1,5 @@
// SPDX-License-Identifier: GPL-2.0-or-later
-#include <linux/compat.h>
#include <linux/syscalls.h>
#include <linux/time_namespace.h>
@@ -85,15 +84,12 @@ err_unlock:
long do_futex(u32 __user *uaddr, int op, u32 val, ktime_t *timeout,
u32 __user *uaddr2, u32 val2, u32 val3)
{
+ unsigned int flags = futex_to_flags(op);
int cmd = op & FUTEX_CMD_MASK;
- unsigned int flags = 0;
- if (!(op & FUTEX_PRIVATE_FLAG))
- flags |= FLAGS_SHARED;
-
- if (op & FUTEX_CLOCK_REALTIME) {
- flags |= FLAGS_CLOCKRT;
- if (cmd != FUTEX_WAIT_BITSET && cmd != FUTEX_WAIT_REQUEUE_PI &&
+ if (flags & FLAGS_CLOCKRT) {
+ if (cmd != FUTEX_WAIT_BITSET &&
+ cmd != FUTEX_WAIT_REQUEUE_PI &&
cmd != FUTEX_LOCK_PI2)
return -ENOSYS;
}
@@ -110,9 +106,9 @@ long do_futex(u32 __user *uaddr, int op, u32 val, ktime_t *timeout,
case FUTEX_WAKE_BITSET:
return futex_wake(uaddr, flags, val, val3);
case FUTEX_REQUEUE:
- return futex_requeue(uaddr, flags, uaddr2, val, val2, NULL, 0);
+ return futex_requeue(uaddr, flags, uaddr2, flags, val, val2, NULL, 0);
case FUTEX_CMP_REQUEUE:
- return futex_requeue(uaddr, flags, uaddr2, val, val2, &val3, 0);
+ return futex_requeue(uaddr, flags, uaddr2, flags, val, val2, &val3, 0);
case FUTEX_WAKE_OP:
return futex_wake_op(uaddr, flags, uaddr2, val, val2, val3);
case FUTEX_LOCK_PI:
@@ -129,7 +125,7 @@ long do_futex(u32 __user *uaddr, int op, u32 val, ktime_t *timeout,
return futex_wait_requeue_pi(uaddr, flags, val, timeout, val3,
uaddr2);
case FUTEX_CMP_REQUEUE_PI:
- return futex_requeue(uaddr, flags, uaddr2, val, val2, &val3, 1);
+ return futex_requeue(uaddr, flags, uaddr2, flags, val, val2, &val3, 1);
}
return -ENOSYS;
}
@@ -183,8 +179,7 @@ SYSCALL_DEFINE6(futex, u32 __user *, uaddr, int, op, u32, val,
return do_futex(uaddr, op, val, tp, uaddr2, (unsigned long)utime, val3);
}
-/* Mask of available flags for each futex in futex_waitv list */
-#define FUTEXV_WAITER_MASK (FUTEX_32 | FUTEX_PRIVATE_FLAG)
+#define FUTEX2_VALID_MASK (FUTEX2_SIZE_MASK | FUTEX2_PRIVATE)
/**
* futex_parse_waitv - Parse a waitv array from userspace
@@ -202,16 +197,22 @@ static int futex_parse_waitv(struct futex_vector *futexv,
unsigned int i;
for (i = 0; i < nr_futexes; i++) {
+ unsigned int flags;
+
if (copy_from_user(&aux, &uwaitv[i], sizeof(aux)))
return -EFAULT;
- if ((aux.flags & ~FUTEXV_WAITER_MASK) || aux.__reserved)
+ if ((aux.flags & ~FUTEX2_VALID_MASK) || aux.__reserved)
return -EINVAL;
- if (!(aux.flags & FUTEX_32))
+ flags = futex2_to_flags(aux.flags);
+ if (!futex_flags_valid(flags))
return -EINVAL;
- futexv[i].w.flags = aux.flags;
+ if (!futex_validate_input(flags, aux.val))
+ return -EINVAL;
+
+ futexv[i].w.flags = flags;
futexv[i].w.val = aux.val;
futexv[i].w.uaddr = aux.uaddr;
futexv[i].q = futex_q_init;
@@ -220,6 +221,46 @@ static int futex_parse_waitv(struct futex_vector *futexv,
return 0;
}
+static int futex2_setup_timeout(struct __kernel_timespec __user *timeout,
+ clockid_t clockid, struct hrtimer_sleeper *to)
+{
+ int flag_clkid = 0, flag_init = 0;
+ struct timespec64 ts;
+ ktime_t time;
+ int ret;
+
+ if (!timeout)
+ return 0;
+
+ if (clockid == CLOCK_REALTIME) {
+ flag_clkid = FLAGS_CLOCKRT;
+ flag_init = FUTEX_CLOCK_REALTIME;
+ }
+
+ if (clockid != CLOCK_REALTIME && clockid != CLOCK_MONOTONIC)
+ return -EINVAL;
+
+ if (get_timespec64(&ts, timeout))
+ return -EFAULT;
+
+ /*
+ * Since there's no opcode for futex_waitv, use
+ * FUTEX_WAIT_BITSET that uses absolute timeout as well
+ */
+ ret = futex_init_timeout(FUTEX_WAIT_BITSET, flag_init, &ts, &time);
+ if (ret)
+ return ret;
+
+ futex_setup_timer(&time, to, flag_clkid, 0);
+ return 0;
+}
+
+static inline void futex2_destroy_timeout(struct hrtimer_sleeper *to)
+{
+ hrtimer_cancel(&to->timer);
+ destroy_hrtimer_on_stack(&to->timer);
+}
+
/**
* sys_futex_waitv - Wait on a list of futexes
* @waiters: List of futexes to wait on
@@ -249,8 +290,6 @@ SYSCALL_DEFINE5(futex_waitv, struct futex_waitv __user *, waiters,
{
struct hrtimer_sleeper to;
struct futex_vector *futexv;
- struct timespec64 ts;
- ktime_t time;
int ret;
/* This syscall supports no flags for now */
@@ -260,30 +299,8 @@ SYSCALL_DEFINE5(futex_waitv, struct futex_waitv __user *, waiters,
if (!nr_futexes || nr_futexes > FUTEX_WAITV_MAX || !waiters)
return -EINVAL;
- if (timeout) {
- int flag_clkid = 0, flag_init = 0;
-
- if (clockid == CLOCK_REALTIME) {
- flag_clkid = FLAGS_CLOCKRT;
- flag_init = FUTEX_CLOCK_REALTIME;
- }
-
- if (clockid != CLOCK_REALTIME && clockid != CLOCK_MONOTONIC)
- return -EINVAL;
-
- if (get_timespec64(&ts, timeout))
- return -EFAULT;
-
- /*
- * Since there's no opcode for futex_waitv, use
- * FUTEX_WAIT_BITSET that uses absolute timeout as well
- */
- ret = futex_init_timeout(FUTEX_WAIT_BITSET, flag_init, &ts, &time);
- if (ret)
- return ret;
-
- futex_setup_timer(&time, &to, flag_clkid, 0);
- }
+ if (timeout && (ret = futex2_setup_timeout(timeout, clockid, &to)))
+ return ret;
futexv = kcalloc(nr_futexes, sizeof(*futexv), GFP_KERNEL);
if (!futexv) {
@@ -298,13 +315,125 @@ SYSCALL_DEFINE5(futex_waitv, struct futex_waitv __user *, waiters,
kfree(futexv);
destroy_timer:
- if (timeout) {
- hrtimer_cancel(&to.timer);
- destroy_hrtimer_on_stack(&to.timer);
- }
+ if (timeout)
+ futex2_destroy_timeout(&to);
+ return ret;
+}
+
+/*
+ * sys_futex_wake - Wake a number of futexes
+ * @uaddr: Address of the futex(es) to wake
+ * @mask: bitmask
+ * @nr: Number of the futexes to wake
+ * @flags: FUTEX2 flags
+ *
+ * Identical to the traditional FUTEX_WAKE_BITSET op, except it is part of the
+ * futex2 family of calls.
+ */
+
+SYSCALL_DEFINE4(futex_wake,
+ void __user *, uaddr,
+ unsigned long, mask,
+ int, nr,
+ unsigned int, flags)
+{
+ if (flags & ~FUTEX2_VALID_MASK)
+ return -EINVAL;
+
+ flags = futex2_to_flags(flags);
+ if (!futex_flags_valid(flags))
+ return -EINVAL;
+
+ if (!futex_validate_input(flags, mask))
+ return -EINVAL;
+
+ return futex_wake(uaddr, FLAGS_STRICT | flags, nr, mask);
+}
+
+/*
+ * sys_futex_wait - Wait on a futex
+ * @uaddr: Address of the futex to wait on
+ * @val: Value of @uaddr
+ * @mask: bitmask
+ * @flags: FUTEX2 flags
+ * @timeout: Optional absolute timeout
+ * @clockid: Clock to be used for the timeout, realtime or monotonic
+ *
+ * Identical to the traditional FUTEX_WAIT_BITSET op, except it is part of the
+ * futex2 familiy of calls.
+ */
+
+SYSCALL_DEFINE6(futex_wait,
+ void __user *, uaddr,
+ unsigned long, val,
+ unsigned long, mask,
+ unsigned int, flags,
+ struct __kernel_timespec __user *, timeout,
+ clockid_t, clockid)
+{
+ struct hrtimer_sleeper to;
+ int ret;
+
+ if (flags & ~FUTEX2_VALID_MASK)
+ return -EINVAL;
+
+ flags = futex2_to_flags(flags);
+ if (!futex_flags_valid(flags))
+ return -EINVAL;
+
+ if (!futex_validate_input(flags, val) ||
+ !futex_validate_input(flags, mask))
+ return -EINVAL;
+
+ if (timeout && (ret = futex2_setup_timeout(timeout, clockid, &to)))
+ return ret;
+
+ ret = __futex_wait(uaddr, flags, val, timeout ? &to : NULL, mask);
+
+ if (timeout)
+ futex2_destroy_timeout(&to);
+
return ret;
}
+/*
+ * sys_futex_requeue - Requeue a waiter from one futex to another
+ * @waiters: array describing the source and destination futex
+ * @flags: unused
+ * @nr_wake: number of futexes to wake
+ * @nr_requeue: number of futexes to requeue
+ *
+ * Identical to the traditional FUTEX_CMP_REQUEUE op, except it is part of the
+ * futex2 family of calls.
+ */
+
+SYSCALL_DEFINE4(futex_requeue,
+ struct futex_waitv __user *, waiters,
+ unsigned int, flags,
+ int, nr_wake,
+ int, nr_requeue)
+{
+ struct futex_vector futexes[2];
+ u32 cmpval;
+ int ret;
+
+ if (flags)
+ return -EINVAL;
+
+ if (!waiters)
+ return -EINVAL;
+
+ ret = futex_parse_waitv(futexes, waiters, 2);
+ if (ret)
+ return ret;
+
+ cmpval = futexes[0].w.val;
+
+ return futex_requeue(u64_to_user_ptr(futexes[0].w.uaddr), futexes[0].w.flags,
+ u64_to_user_ptr(futexes[1].w.uaddr), futexes[1].w.flags,
+ nr_wake, nr_requeue, &cmpval, 0);
+}
+
#ifdef CONFIG_COMPAT
COMPAT_SYSCALL_DEFINE2(set_robust_list,
struct compat_robust_list_head __user *, head,
diff --git a/kernel/futex/waitwake.c b/kernel/futex/waitwake.c
index ba01b9408203..37860f794bf7 100644
--- a/kernel/futex/waitwake.c
+++ b/kernel/futex/waitwake.c
@@ -145,16 +145,19 @@ int futex_wake(u32 __user *uaddr, unsigned int flags, int nr_wake, u32 bitset)
struct futex_hash_bucket *hb;
struct futex_q *this, *next;
union futex_key key = FUTEX_KEY_INIT;
- int ret;
DEFINE_WAKE_Q(wake_q);
+ int ret;
if (!bitset)
return -EINVAL;
- ret = get_futex_key(uaddr, flags & FLAGS_SHARED, &key, FUTEX_READ);
+ ret = get_futex_key(uaddr, flags, &key, FUTEX_READ);
if (unlikely(ret != 0))
return ret;
+ if ((flags & FLAGS_STRICT) && !nr_wake)
+ return 0;
+
hb = futex_hash(&key);
/* Make sure we really have tasks to wakeup */
@@ -245,10 +248,10 @@ int futex_wake_op(u32 __user *uaddr1, unsigned int flags, u32 __user *uaddr2,
DEFINE_WAKE_Q(wake_q);
retry:
- ret = get_futex_key(uaddr1, flags & FLAGS_SHARED, &key1, FUTEX_READ);
+ ret = get_futex_key(uaddr1, flags, &key1, FUTEX_READ);
if (unlikely(ret != 0))
return ret;
- ret = get_futex_key(uaddr2, flags & FLAGS_SHARED, &key2, FUTEX_WRITE);
+ ret = get_futex_key(uaddr2, flags, &key2, FUTEX_WRITE);
if (unlikely(ret != 0))
return ret;
@@ -419,11 +422,11 @@ static int futex_wait_multiple_setup(struct futex_vector *vs, int count, int *wo
*/
retry:
for (i = 0; i < count; i++) {
- if ((vs[i].w.flags & FUTEX_PRIVATE_FLAG) && retry)
+ if (!(vs[i].w.flags & FLAGS_SHARED) && retry)
continue;
ret = get_futex_key(u64_to_user_ptr(vs[i].w.uaddr),
- !(vs[i].w.flags & FUTEX_PRIVATE_FLAG),
+ vs[i].w.flags,
&vs[i].q.key, FUTEX_READ);
if (unlikely(ret))
@@ -435,7 +438,7 @@ retry:
for (i = 0; i < count; i++) {
u32 __user *uaddr = (u32 __user *)(unsigned long)vs[i].w.uaddr;
struct futex_q *q = &vs[i].q;
- u32 val = (u32)vs[i].w.val;
+ u32 val = vs[i].w.val;
hb = futex_q_lock(q);
ret = futex_get_value_locked(&uval, uaddr);
@@ -599,7 +602,7 @@ int futex_wait_setup(u32 __user *uaddr, u32 val, unsigned int flags,
* while the syscall executes.
*/
retry:
- ret = get_futex_key(uaddr, flags & FLAGS_SHARED, &q->key, FUTEX_READ);
+ ret = get_futex_key(uaddr, flags, &q->key, FUTEX_READ);
if (unlikely(ret != 0))
return ret;
@@ -629,20 +632,18 @@ retry_private:
return ret;
}
-int futex_wait(u32 __user *uaddr, unsigned int flags, u32 val, ktime_t *abs_time, u32 bitset)
+int __futex_wait(u32 __user *uaddr, unsigned int flags, u32 val,
+ struct hrtimer_sleeper *to, u32 bitset)
{
- struct hrtimer_sleeper timeout, *to;
- struct restart_block *restart;
- struct futex_hash_bucket *hb;
struct futex_q q = futex_q_init;
+ struct futex_hash_bucket *hb;
int ret;
if (!bitset)
return -EINVAL;
+
q.bitset = bitset;
- to = futex_setup_timer(abs_time, &timeout, flags,
- current->timer_slack_ns);
retry:
/*
* Prepare to wait on uaddr. On success, it holds hb->lock and q
@@ -650,18 +651,17 @@ retry:
*/
ret = futex_wait_setup(uaddr, val, flags, &q, &hb);
if (ret)
- goto out;
+ return ret;
/* futex_queue and wait for wakeup, timeout, or a signal. */
futex_wait_queue(hb, &q, to);
/* If we were woken (and unqueued), we succeeded, whatever. */
- ret = 0;
if (!futex_unqueue(&q))
- goto out;
- ret = -ETIMEDOUT;
+ return 0;
+
if (to && !to->task)
- goto out;
+ return -ETIMEDOUT;
/*
* We expect signal_pending(current), but we might be the
@@ -670,24 +670,38 @@ retry:
if (!signal_pending(current))
goto retry;
- ret = -ERESTARTSYS;
- if (!abs_time)
- goto out;
+ return -ERESTARTSYS;
+}
- restart = &current->restart_block;
- restart->futex.uaddr = uaddr;
- restart->futex.val = val;
- restart->futex.time = *abs_time;
- restart->futex.bitset = bitset;
- restart->futex.flags = flags | FLAGS_HAS_TIMEOUT;
+int futex_wait(u32 __user *uaddr, unsigned int flags, u32 val, ktime_t *abs_time, u32 bitset)
+{
+ struct hrtimer_sleeper timeout, *to;
+ struct restart_block *restart;
+ int ret;
- ret = set_restart_fn(restart, futex_wait_restart);
+ to = futex_setup_timer(abs_time, &timeout, flags,
+ current->timer_slack_ns);
-out:
- if (to) {
- hrtimer_cancel(&to->timer);
- destroy_hrtimer_on_stack(&to->timer);
+ ret = __futex_wait(uaddr, flags, val, to, bitset);
+
+ /* No timeout, nothing to clean up. */
+ if (!to)
+ return ret;
+
+ hrtimer_cancel(&to->timer);
+ destroy_hrtimer_on_stack(&to->timer);
+
+ if (ret == -ERESTARTSYS) {
+ restart = &current->restart_block;
+ restart->futex.uaddr = uaddr;
+ restart->futex.val = val;
+ restart->futex.time = *abs_time;
+ restart->futex.bitset = bitset;
+ restart->futex.flags = flags | FLAGS_HAS_TIMEOUT;
+
+ return set_restart_fn(restart, futex_wait_restart);
}
+
return ret;
}