Skip to content

Commit 54f6a78

Browse files
committed
Add JoinHandle::into_join_future().
This allows spawned threads to be incorporated into `Future`-based concurrency control without needing to add separate result-reporting channels and an additional layer of `catch_unwind()` to the thread functions. I believe this will be useful to async/blocking interop and for various applications which want to manage parallel tasks in a lightweight way. There is a small additional cost which is paid even if the mechanism is unused: the algorithm built into the shutdown of a spawned thread must obtain and invoke a `Waker`, and the `Packet` internal struct is larger by one `Mutex<Waker>`. In the future, this `Mutex` should be replaced by something equivalent to `futures::task::AtomicWaker`, which will be more efficient and eliminate deadlock and blocking hazards, but `std` doesn't contain one of those yet. This is not an `impl IntoFuture for JoinHandle` so that it can avoid being insta-stable; particularly because during the design discussion, concerns were raised that a proper implementation should obey structured concurrency via an `AsyncDrop` that forces waiting for the thread. I personally think that would be a mistake, and structured spawning should be its own thing, but this choice of API permits either option in the future by keeping everything unstable, where a trait implementation would not.
1 parent 1e4f10b commit 54f6a78

File tree

4 files changed

+240
-17
lines changed

4 files changed

+240
-17
lines changed

library/std/src/lib.rs

+1
Original file line numberDiff line numberDiff line change
@@ -349,6 +349,7 @@
349349
#![feature(lazy_get)]
350350
#![feature(maybe_uninit_slice)]
351351
#![feature(maybe_uninit_write_slice)]
352+
#![feature(noop_waker)]
352353
#![feature(panic_can_unwind)]
353354
#![feature(panic_internals)]
354355
#![feature(pin_coerce_unsized_trait)]

library/std/src/thread/mod.rs

+171-15
Original file line numberDiff line numberDiff line change
@@ -164,17 +164,18 @@ use core::mem::MaybeUninit;
164164

165165
use crate::any::Any;
166166
use crate::cell::UnsafeCell;
167+
use crate::future::Future;
167168
use crate::marker::PhantomData;
168169
use crate::mem::{self, ManuallyDrop, forget};
169170
use crate::num::NonZero;
170171
use crate::pin::Pin;
171-
use crate::sync::Arc;
172172
use crate::sync::atomic::{AtomicUsize, Ordering};
173+
use crate::sync::{Arc, Mutex, PoisonError};
173174
use crate::sys::sync::Parker;
174175
use crate::sys::thread as imp;
175176
use crate::sys_common::{AsInner, IntoInner};
176177
use crate::time::{Duration, Instant};
177-
use crate::{env, fmt, io, panic, panicking, str};
178+
use crate::{env, fmt, io, panic, panicking, str, task};
178179

179180
#[stable(feature = "scoped_threads", since = "1.63.0")]
180181
mod scoped;
@@ -490,6 +491,7 @@ impl Builder {
490491
let my_packet: Arc<Packet<'scope, T>> = Arc::new(Packet {
491492
scope: scope_data,
492493
result: UnsafeCell::new(None),
494+
waker: Mutex::new(task::Waker::noop().clone()),
493495
_marker: PhantomData,
494496
});
495497
let their_packet = my_packet.clone();
@@ -540,15 +542,35 @@ impl Builder {
540542
let try_result = panic::catch_unwind(panic::AssertUnwindSafe(|| {
541543
crate::sys::backtrace::__rust_begin_short_backtrace(f)
542544
}));
545+
546+
// Store the `Result` of the thread that the `JoinHandle` can retrieve.
547+
//
543548
// SAFETY: `their_packet` as been built just above and moved by the
544549
// closure (it is an Arc<...>) and `my_packet` will be stored in the
545550
// same `JoinInner` as this closure meaning the mutation will be
546551
// safe (not modify it and affect a value far away).
547552
unsafe { *their_packet.result.get() = Some(try_result) };
548-
// Here `their_packet` gets dropped, and if this is the last `Arc` for that packet that
549-
// will call `decrement_num_running_threads` and therefore signal that this thread is
550-
// done.
553+
554+
// Fetch the `Waker` from the packet; this is needed to support `.into_join_future()`.
555+
// If unused, this just returns `Waker::noop()` which will do nothing.
556+
let waker: task::Waker = {
557+
let placeholder = task::Waker::noop().clone();
558+
let mut guard = their_packet.waker.lock().unwrap_or_else(PoisonError::into_inner);
559+
mem::replace(&mut *guard, placeholder)
560+
};
561+
562+
// Here `their_packet` gets dropped, and if this is the last `Arc` for that packet
563+
// (which happens if the `JoinHandle` has been dropped) that will call
564+
// `decrement_num_running_threads` and therefore signal to the scope (if there is one)
565+
// that this thread is done.
551566
drop(their_packet);
567+
568+
// Now that we have become visibly “finished” by dropping the packet
569+
// (`JoinInner::is_finished` will return true), we can use the `Waker` to signal
570+
// any waiting `JoinFuture`. If instead we are being waited for by
571+
// `JoinHandle::join()`, the actual platform thread termination will be the wakeup.
572+
waker.wake();
573+
552574
// Here, the lifetime `'scope` can end. `main` keeps running for a bit
553575
// after that before returning itself.
554576
};
@@ -1192,8 +1214,6 @@ impl ThreadId {
11921214
}
11931215
}
11941216
} else {
1195-
use crate::sync::{Mutex, PoisonError};
1196-
11971217
static COUNTER: Mutex<u64> = Mutex::new(0);
11981218

11991219
let mut counter = COUNTER.lock().unwrap_or_else(PoisonError::into_inner);
@@ -1635,16 +1655,30 @@ impl fmt::Debug for Thread {
16351655
#[stable(feature = "rust1", since = "1.0.0")]
16361656
pub type Result<T> = crate::result::Result<T, Box<dyn Any + Send + 'static>>;
16371657

1638-
// This packet is used to communicate the return value between the spawned
1639-
// thread and the rest of the program. It is shared through an `Arc` and
1640-
// there's no need for a mutex here because synchronization happens with `join()`
1641-
// (the caller will never read this packet until the thread has exited).
1642-
//
1643-
// An Arc to the packet is stored into a `JoinInner` which in turns is placed
1644-
// in `JoinHandle`.
1658+
/// This packet is used to communicate the return value between the spawned
1659+
/// thread and the rest of the program. It is shared through an [`Arc`].
1660+
///
1661+
/// An Arc to the packet is stored into a [`JoinInner`] which in turn is placed
1662+
/// in [`JoinHandle`] or [`ScopedJoinHandle`].
16451663
struct Packet<'scope, T> {
1664+
/// Communication with the enclosing thread scope if there is one.
16461665
scope: Option<Arc<scoped::ScopeData>>,
1666+
1667+
/// Holds the return value.
1668+
///
1669+
/// Synchronization happens via reference counting: as long as the `Arc<Packet>`
1670+
/// has two or more references, this field is never read, and will only be written
1671+
/// once as the thread terminates. After that happens, either the packet is dropped,
1672+
/// or [`JoinInner::join()`] will `take()` the result value from here.
16471673
result: UnsafeCell<Option<Result<T>>>,
1674+
1675+
/// If a [`JoinFuture`] for this thread exists and has been polled,
1676+
/// this is the waker from that poll. If it does not exist or has not
1677+
/// been polled yet, this is [`task::Waker::noop()`].
1678+
// FIXME: This should be an `AtomicWaker` instead of a `Mutex`,
1679+
// to be cheaper and impossible to deadlock.
1680+
waker: Mutex<task::Waker>,
1681+
16481682
_marker: PhantomData<Option<&'scope scoped::ScopeData>>,
16491683
}
16501684

@@ -1698,6 +1732,10 @@ impl<'scope, T> JoinInner<'scope, T> {
16981732
self.native.join();
16991733
Arc::get_mut(&mut self.packet).unwrap().result.get_mut().take().unwrap()
17001734
}
1735+
1736+
fn is_finished(&self) -> bool {
1737+
Arc::strong_count(&self.packet) == 1
1738+
}
17011739
}
17021740

17031741
/// An owned permission to join on a thread (block on its termination).
@@ -1844,6 +1882,45 @@ impl<T> JoinHandle<T> {
18441882
self.0.join()
18451883
}
18461884

1885+
/// Returns a [`Future`] that resolves when the thread has finished.
1886+
///
1887+
/// Its [output](Future::Output) value is identical to that of [`JoinHandle::join()`];
1888+
/// this is the `async` equivalent of that blocking function.
1889+
///
1890+
/// If the returned future is dropped (cancelled), the thread will become *detached*;
1891+
/// there will be no way to observe or wait for the thread’s termination.
1892+
/// This is identical to the behavior of `JoinHandle` itself.
1893+
///
1894+
/// # Example
1895+
///
1896+
// FIXME: ideally we would actually run this example, with the help of a trivial async executor
1897+
/// ```no_run
1898+
/// #![feature(thread_join_future)]
1899+
/// use std::thread;
1900+
///
1901+
/// async fn do_some_heavy_tasks_in_parallel() -> thread::Result<()> {
1902+
/// let future_1 = thread::spawn(|| {
1903+
/// // ... do something ...
1904+
/// }).into_join_future();
1905+
/// let future_2 = thread::spawn(|| {
1906+
/// // ... do something else ...
1907+
/// }).into_join_future();
1908+
///
1909+
/// // Both threads have been started; now await the completion of both.
1910+
/// future_1.await?;
1911+
/// future_2.await?;
1912+
/// Ok(())
1913+
/// }
1914+
/// ```
1915+
#[unstable(feature = "thread_join_future", issue = "none")]
1916+
pub fn into_join_future(self) -> JoinFuture<'static, T> {
1917+
// The method is not named `into_future()` to avoid overlapping with the stable
1918+
// `IntoFuture::into_future()`. We're not implementing `IntoFuture` in order to
1919+
// keep this unstable and preserve the *option* of compatibly making this obey structured
1920+
// concurrency via an async-Drop that waits for the thread to end.
1921+
JoinFuture::new(self.0)
1922+
}
1923+
18471924
/// Checks if the associated thread has finished running its main function.
18481925
///
18491926
/// `is_finished` supports implementing a non-blocking join operation, by checking
@@ -1856,7 +1933,7 @@ impl<T> JoinHandle<T> {
18561933
/// to return quickly, without blocking for any significant amount of time.
18571934
#[stable(feature = "thread_is_running", since = "1.61.0")]
18581935
pub fn is_finished(&self) -> bool {
1859-
Arc::strong_count(&self.0.packet) == 1
1936+
self.0.is_finished()
18601937
}
18611938
}
18621939

@@ -1882,9 +1959,88 @@ impl<T> fmt::Debug for JoinHandle<T> {
18821959
fn _assert_sync_and_send() {
18831960
fn _assert_both<T: Send + Sync>() {}
18841961
_assert_both::<JoinHandle<()>>();
1962+
_assert_both::<JoinFuture<'static, ()>>();
18851963
_assert_both::<Thread>();
18861964
}
18871965

1966+
/// A [`Future`] that resolves when a thread has finished.
1967+
///
1968+
/// Its [output](Future::Output) value is identical to that of [`JoinHandle::join()`];
1969+
/// this is the `async` equivalent of that blocking function.
1970+
/// Obtain it by calling [`JoinHandle::into_join_future()`] or
1971+
/// [`ScopedJoinHandle::into_join_future()`].
1972+
///
1973+
/// If a `JoinFuture` is dropped (cancelled), and the thread does not belong to a [scope],
1974+
/// the associated thread will become *detached*;
1975+
/// there will be no way to observe or wait for the thread’s termination.
1976+
#[unstable(feature = "thread_join_future", issue = "none")]
1977+
pub struct JoinFuture<'scope, T>(Option<JoinInner<'scope, T>>);
1978+
1979+
impl<'scope, T> JoinFuture<'scope, T> {
1980+
fn new(inner: JoinInner<'scope, T>) -> Self {
1981+
Self(Some(inner))
1982+
}
1983+
1984+
/// Implements the “getting a result” part of joining/polling, without blocking or changing
1985+
/// the `Waker`. Part of the implementation of `poll()`.
1986+
///
1987+
/// If this returns `Some`, then `self.0` is now `None` and the future will panic
1988+
/// if polled again.
1989+
fn take_result(&mut self) -> Option<Result<T>> {
1990+
self.0.take_if(|i| i.is_finished()).map(JoinInner::join)
1991+
}
1992+
}
1993+
1994+
#[unstable(feature = "thread_join_future", issue = "none")]
1995+
impl<T> Future for JoinFuture<'_, T> {
1996+
type Output = Result<T>;
1997+
fn poll(mut self: Pin<&mut Self>, cx: &mut task::Context<'_>) -> task::Poll<Self::Output> {
1998+
if let Some(result) = self.take_result() {
1999+
return task::Poll::Ready(result);
2000+
}
2001+
2002+
// Update the `Waker` the thread should wake when it completes.
2003+
{
2004+
let Some(inner) = &mut self.0 else {
2005+
panic!("polled after complete");
2006+
};
2007+
2008+
let new_waker = cx.waker();
2009+
2010+
// Lock the mutex, and ignore the poison state because there are no meaningful ways
2011+
// the existing contents can be corrupted; they will be overwritten completely and the
2012+
// overwrite is atomic-in-the-database-sense.
2013+
let mut current_waker_guard =
2014+
inner.packet.waker.lock().unwrap_or_else(PoisonError::into_inner);
2015+
2016+
// Overwrite the waker. Note that we are executing the new waker’s clone and the old
2017+
// waker’s destructor; these could panic (which will merely poison the lock) or hang,
2018+
// which will hold the lock, but the most that can do is prevent the thread from
2019+
// exiting because it's trying to acquire `packet.waker`, which it won't do while
2020+
// holding any *other* locks (...unless the thread’s data includes a lock guard that
2021+
// the waker also wants).
2022+
if !new_waker.will_wake(&*current_waker_guard) {
2023+
*current_waker_guard = new_waker.clone();
2024+
}
2025+
}
2026+
2027+
// Check for completion again in case the thread finished while we were busy
2028+
// setting the waker, to prevent a lost wakeup in that case.
2029+
if let Some(result) = self.take_result() {
2030+
task::Poll::Ready(result)
2031+
} else {
2032+
task::Poll::Pending
2033+
}
2034+
}
2035+
}
2036+
2037+
#[unstable(feature = "thread_join_future", issue = "none")]
2038+
impl<T> fmt::Debug for JoinFuture<'_, T> {
2039+
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2040+
f.debug_struct("JoinHandle").finish_non_exhaustive()
2041+
}
2042+
}
2043+
18882044
/// Returns an estimate of the default amount of parallelism a program should use.
18892045
///
18902046
/// Parallelism is a resource. A given machine provides a certain capacity for

library/std/src/thread/scoped.rs

+21-1
Original file line numberDiff line numberDiff line change
@@ -313,6 +313,26 @@ impl<'scope, T> ScopedJoinHandle<'scope, T> {
313313
self.0.join()
314314
}
315315

316+
/// Returns a [`Future`] that resolves when the thread has finished.
317+
///
318+
/// Its [output] value is identical to that of [`ScopedJoinHandle::join()`];
319+
/// this is the `async` equivalent of that blocking function.
320+
///
321+
/// Note that while this function allows waiting for a scoped thread from `async`
322+
/// functions, the original [`scope()`] is still a blocking function which should
323+
/// not be used in `async` functions.
324+
///
325+
/// [`Future`]: crate::future::Future
326+
/// [output]: crate::future::Future::Output
327+
#[unstable(feature = "thread_join_future", issue = "none")]
328+
pub fn into_join_future(self) -> super::JoinFuture<'scope, T> {
329+
// There is no `ScopedJoinFuture` because the only difference between `JoinHandle`
330+
// and `ScopedJoinHandle` is that `JoinHandle` has no lifetime parameter, because
331+
// it was introduced before scoped threads. `JoinFuture` is new enough that we don’t
332+
// need to make two versions of it.
333+
super::JoinFuture::new(self.0)
334+
}
335+
316336
/// Checks if the associated thread has finished running its main function.
317337
///
318338
/// `is_finished` supports implementing a non-blocking join operation, by checking
@@ -325,7 +345,7 @@ impl<'scope, T> ScopedJoinHandle<'scope, T> {
325345
/// to return quickly, without blocking for any significant amount of time.
326346
#[stable(feature = "scoped_threads", since = "1.63.0")]
327347
pub fn is_finished(&self) -> bool {
328-
Arc::strong_count(&self.0.packet) == 1
348+
self.0.is_finished()
329349
}
330350
}
331351

library/std/src/thread/tests.rs

+47-1
Original file line numberDiff line numberDiff line change
@@ -1,12 +1,14 @@
11
use super::Builder;
22
use crate::any::Any;
3+
use crate::assert_matches::assert_matches;
4+
use crate::future::Future as _;
35
use crate::panic::panic_any;
46
use crate::sync::atomic::{AtomicBool, Ordering};
57
use crate::sync::mpsc::{Sender, channel};
68
use crate::sync::{Arc, Barrier};
79
use crate::thread::{self, Scope, ThreadId};
810
use crate::time::{Duration, Instant};
9-
use crate::{mem, result};
11+
use crate::{mem, result, task};
1012

1113
// !!! These tests are dangerous. If something is buggy, they will hang, !!!
1214
// !!! instead of exiting cleanly. This might wedge the buildbots. !!!
@@ -410,3 +412,47 @@ fn test_minimal_thread_stack() {
410412
assert_eq!(before, 0);
411413
assert_eq!(COUNT.load(Ordering::Relaxed), 1);
412414
}
415+
416+
fn join_future_test(scoped: bool) {
417+
/// Simple `Waker` implementation.
418+
/// If `std` ever gains a `block_on()`, we can consider replacing this with that.
419+
struct MyWaker(Sender<()>);
420+
impl task::Wake for MyWaker {
421+
fn wake(self: Arc<Self>) {
422+
_ = self.0.send(());
423+
}
424+
}
425+
426+
// Communication setup.
427+
let (thread_delay_tx, thread_delay_rx) = channel();
428+
let (waker_tx, waker_rx) = channel();
429+
let waker = task::Waker::from(Arc::new(MyWaker(waker_tx)));
430+
let ctx = &mut task::Context::from_waker(&waker);
431+
432+
thread::scope(|s| {
433+
// Create the thread and the future under test
434+
let thread_body = move || {
435+
thread_delay_rx.recv().unwrap();
436+
"hello"
437+
};
438+
let mut future = crate::pin::pin!(if scoped {
439+
s.spawn(thread_body).into_join_future()
440+
} else {
441+
thread::spawn(thread_body).into_join_future()
442+
});
443+
444+
// Actual test
445+
assert_matches!(future.as_mut().poll(ctx), task::Poll::Pending);
446+
thread_delay_tx.send(()).unwrap(); // Unblock the thread
447+
waker_rx.recv().unwrap(); // Wait for waking (as an executor would)
448+
assert_matches!(future.as_mut().poll(ctx), task::Poll::Ready(Ok("hello")));
449+
});
450+
}
451+
#[test]
452+
fn join_future_unscoped() {
453+
join_future_test(false)
454+
}
455+
#[test]
456+
fn join_future_scoped() {
457+
join_future_test(true)
458+
}

0 commit comments

Comments
 (0)