diff --git a/futures-core/src/task/mod.rs b/futures-core/src/task/mod.rs index 95dc3cb57f..29c8509924 100644 --- a/futures-core/src/task/mod.rs +++ b/futures-core/src/task/mod.rs @@ -5,6 +5,6 @@ mod spawn; pub mod __internal; pub use self::spawn::{Spawn, LocalSpawn, SpawnError}; -pub use core::task::{Poll, Waker, LocalWaker, UnsafeWake}; +pub use core::task::{Poll, Waker, LocalWaker, RawWaker, RawWakerVTable}; #[cfg(feature = "std")] -pub use std::task::{Wake, local_waker, local_waker_from_nonlocal}; +pub use std::task::{ArcWake}; diff --git a/futures-executor/src/local_pool.rs b/futures-executor/src/local_pool.rs index 62f9dabd89..827111e075 100644 --- a/futures-executor/src/local_pool.rs +++ b/futures-executor/src/local_pool.rs @@ -2,9 +2,10 @@ use crate::{enter, ThreadPool}; use futures_core::future::{Future, FutureObj, LocalFutureObj}; use futures_core::stream::{Stream}; use futures_core::task::{ - self, Poll, LocalWaker, Wake, + Poll, LocalWaker, ArcWake, Spawn, LocalSpawn, SpawnError, }; +use futures_util::task::{LocalWakerRef, local_waker_ref}; use futures_util::stream::FuturesUnordered; use futures_util::stream::StreamExt; use lazy_static::lazy_static; @@ -53,7 +54,7 @@ thread_local! { }); } -impl Wake for ThreadNotify { +impl ArcWake for ThreadNotify { fn wake(arc_self: &Arc<Self>) { arc_self.thread.unpark(); } @@ -67,8 +68,7 @@ fn run_executor<T, F: FnMut(&LocalWaker) -> Poll<T>>(mut f: F) -> T { another executor"); CURRENT_THREAD_NOTIFY.with(|thread_notify| { - let local_waker = - task::local_waker_from_nonlocal(thread_notify.clone()); + let local_waker: LocalWakerRef = unsafe { local_waker_ref(thread_notify) }; loop { if let Poll::Ready(t) = f(&local_waker) { return t; diff --git a/futures-executor/src/thread_pool.rs b/futures-executor/src/thread_pool.rs index 798a50df90..d233d2120a 100644 --- a/futures-executor/src/thread_pool.rs +++ b/futures-executor/src/thread_pool.rs @@ -1,7 +1,7 @@ use crate::enter; use crate::unpark_mutex::UnparkMutex; use futures_core::future::{Future, FutureObj}; -use futures_core::task::{Poll, Wake, Spawn, SpawnError}; +use futures_core::task::{Poll, ArcWake, Spawn, SpawnError}; use futures_util::future::FutureExt; use futures_util::task::local_waker_ref_from_nonlocal; use num_cpus; @@ -338,7 +338,7 @@ impl fmt::Debug for Task { } } -impl Wake for WakeHandle { +impl ArcWake for WakeHandle { fn wake(arc_self: &Arc<Self>) { match arc_self.mutex.notify() { Ok(task) => arc_self.exec.state.send(Message::Run(task)), diff --git a/futures-test/src/task/mod.rs b/futures-test/src/task/mod.rs index 43d22320c4..74c57b8e51 100644 --- a/futures-test/src/task/mod.rs +++ b/futures-test/src/task/mod.rs @@ -32,7 +32,7 @@ mod panic_spawner; pub use self::panic_spawner::{panic_spawner_mut, PanicSpawner}; mod panic_waker; -pub use self::panic_waker::{panic_local_waker, panic_local_waker_ref, PanicWake}; +pub use self::panic_waker::{panic_local_waker, panic_local_waker_ref}; mod record_spawner; pub use self::record_spawner::RecordSpawner; diff --git a/futures-test/src/task/panic_waker.rs b/futures-test/src/task/panic_waker.rs index 6dcad3cbb1..b78f69bd1f 100644 --- a/futures-test/src/task/panic_waker.rs +++ b/futures-test/src/task/panic_waker.rs @@ -1,74 +1,56 @@ -use futures_core::task::{LocalWaker, UnsafeWake, Wake, Waker}; -use std::cell::UnsafeCell; -use std::ptr::NonNull; -use std::sync::Arc; +use futures_core::task::{LocalWaker, RawWaker, RawWakerVTable}; +use core::cell::UnsafeCell; +use core::ptr::null; -/// An implementation of [`Wake`](futures_core::task::Wake) that panics when -/// woken. -/// -/// # Examples -/// -/// ```should_panic -/// #![feature(futures_api)] -/// use futures_test::task::panic_local_waker_ref; -/// -/// let lw = panic_local_waker_ref(); -/// lw.wake(); // Will panic -/// ``` -#[derive(Debug)] -pub struct PanicWake { - _reserved: (), +unsafe fn noop_clone(_data: *const()) -> RawWaker { + raw_panic_waker() } -impl PanicWake { - /// Create a new instance - pub fn new() -> Self { - Self { _reserved: () } - } +unsafe fn noop(_data: *const()) { } -impl Default for PanicWake { - fn default() -> Self { - Self::new() - } +unsafe fn wake_panic(_data: *const()) { + panic!("should not be woken"); } -impl Wake for PanicWake { - fn wake(_arc_self: &Arc<Self>) { - panic!("should not be woken") - } +unsafe fn noop_into_waker(_data: *const()) -> Option<RawWaker> { + Some(raw_panic_waker()) } -unsafe impl UnsafeWake for PanicWake { - unsafe fn clone_raw(&self) -> Waker { - panic_waker() - } +const PANIC_WAKER_VTABLE: RawWakerVTable = RawWakerVTable { + clone: noop_clone, + drop_fn: noop, + wake: wake_panic, + into_waker: noop_into_waker, +}; - unsafe fn drop_raw(&self) {} - - unsafe fn wake(&self) { - panic!("should not be woken") +fn raw_panic_waker() -> RawWaker { + RawWaker { + data: null(), + vtable: &PANIC_WAKER_VTABLE, } } -fn panic_unsafe_wake() -> NonNull<dyn UnsafeWake> { - static mut INSTANCE: PanicWake = PanicWake { _reserved: () }; - unsafe { NonNull::new_unchecked(&mut INSTANCE as *mut dyn UnsafeWake) } -} - -fn panic_waker() -> Waker { - unsafe { Waker::new(panic_unsafe_wake()) } -} - -/// Create a new [`LocalWaker`](futures_core::task::LocalWaker) referencing -/// a singleton instance of [`PanicWake`]. +/// Create a new [`LocalWaker`](futures_core::task::LocalWaker) which will +/// panic when `wake()` is called on it. The [`LocalWaker`] can be converted +/// into a [`Waker`] which will behave the same way. +/// +/// # Examples +/// +/// ```should_panic +/// #![feature(futures_api)] +/// use futures_test::task::panic_local_waker; +/// +/// let lw = panic_local_waker(); +/// lw.wake(); // Will panic +/// ``` pub fn panic_local_waker() -> LocalWaker { - unsafe { LocalWaker::new(panic_unsafe_wake()) } + unsafe { LocalWaker::new_unchecked(raw_panic_waker()) } } -/// Get a thread local reference to a +/// Get a global reference to a /// [`LocalWaker`](futures_core::task::LocalWaker) referencing a singleton -/// instance of [`PanicWake`]. +/// instance of a [`LocalWaker`] which panics when woken. /// /// # Examples /// @@ -82,8 +64,8 @@ pub fn panic_local_waker() -> LocalWaker { /// ``` pub fn panic_local_waker_ref() -> &'static LocalWaker { thread_local! { - static LOCAL_WAKER_INSTANCE: UnsafeCell<LocalWaker> = + static PANIC_WAKER_INSTANCE: UnsafeCell<LocalWaker> = UnsafeCell::new(panic_local_waker()); } - LOCAL_WAKER_INSTANCE.with(|l| unsafe { &*l.get() }) + PANIC_WAKER_INSTANCE.with(|l| unsafe { &*l.get() }) } diff --git a/futures-test/src/task/wake_counter.rs b/futures-test/src/task/wake_counter.rs index 6afaa9ef5a..df79820aa3 100644 --- a/futures-test/src/task/wake_counter.rs +++ b/futures-test/src/task/wake_counter.rs @@ -1,4 +1,4 @@ -use futures_core::task::{self, LocalWaker, Wake}; +use futures_core::task::{LocalWaker, ArcWake}; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; @@ -39,7 +39,7 @@ impl WakeCounter { count: AtomicUsize::new(0), }); WakeCounter { - local_waker: task::local_waker_from_nonlocal(inner.clone()), + local_waker: ArcWake::into_local_waker(inner.clone()), inner, } } @@ -63,7 +63,7 @@ impl Default for WakeCounter { } } -impl Wake for Inner { +impl ArcWake for Inner { fn wake(arc_self: &Arc<Self>) { arc_self.count.fetch_add(1, Ordering::SeqCst); } diff --git a/futures-util/src/future/shared.rs b/futures-util/src/future/shared.rs index eaf283ff95..8175a2a705 100644 --- a/futures-util/src/future/shared.rs +++ b/futures-util/src/future/shared.rs @@ -1,6 +1,6 @@ use crate::task::local_waker_ref_from_nonlocal; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{LocalWaker, Poll, Wake, Waker}; +use futures_core::task::{LocalWaker, Poll, ArcWake, Waker}; use slab::Slab; use std::cell::UnsafeCell; use std::fmt; @@ -130,10 +130,12 @@ where self.waker_key = wakers.insert(Some(lw.clone().into_waker())); } else { let waker_slot = &mut wakers[self.waker_key]; - let needs_replacement = if let Some(old_waker) = waker_slot { + let needs_replacement = if let Some(_old_waker) = waker_slot { // If there's still an unwoken waker in the slot, only replace // if the current one wouldn't wake the same task. - !lw.will_wake_nonlocal(old_waker) + // TODO: This API is currently not available, so replace always + // !lw.will_wake_nonlocal(old_waker) + true } else { true }; @@ -315,7 +317,7 @@ where } } -impl Wake for Notifier { +impl ArcWake for Notifier { fn wake(arc_self: &Arc<Self>) { arc_self.state.compare_and_swap(POLLING, REPOLL, SeqCst); diff --git a/futures-util/src/stream/futures_unordered/task.rs b/futures-util/src/stream/futures_unordered/task.rs index 55f3d598f9..1f1c3471c5 100644 --- a/futures-util/src/stream/futures_unordered/task.rs +++ b/futures-util/src/stream/futures_unordered/task.rs @@ -1,14 +1,11 @@ use std::cell::UnsafeCell; -use std::marker::PhantomData; -use std::mem; -use std::ptr::{self, NonNull}; use std::sync::{Arc, Weak}; use std::sync::atomic::{AtomicPtr, AtomicBool}; use std::sync::atomic::Ordering::SeqCst; -use futures_core::task::{UnsafeWake, Waker, LocalWaker}; +use futures_core::task::{ArcWake}; -use crate::task::LocalWakerRef; +use crate::task::{LocalWakerRef, local_waker_ref_from_nonlocal}; use super::ReadyToRunQueue; use super::abort::abort; @@ -32,9 +29,14 @@ pub(super) struct Task<Fut> { pub(super) queued: AtomicBool, } -impl<Fut> Task<Fut> { - pub(super) fn wake(self: &Arc<Task<Fut>>) { - let inner = match self.ready_to_run_queue.upgrade() { +// Task isn't really thread-safe, but we only access thread-safe fields from +// other threads in the `ArcWake` implementation. +unsafe impl<Fut> Send for Task<Fut> {} +unsafe impl<Fut> Sync for Task<Fut> {} + +impl<Fut> ArcWake for Task<Fut> { + fn wake(arc_self: &Arc<Self>) { + let inner = match arc_self.ready_to_run_queue.upgrade() { Some(inner) => inner, None => return, }; @@ -51,59 +53,20 @@ impl<Fut> Task<Fut> { // implementation guarantees that if we set the `queued` flag that // there's a reference count held by the main `FuturesUnordered` queue // still. - let prev = self.queued.swap(true, SeqCst); + let prev = arc_self.queued.swap(true, SeqCst); if !prev { - inner.enqueue(&**self); + inner.enqueue(&**arc_self); inner.waker.wake(); } } +} - /// Returns a waker. - pub(super) fn waker(self: &Arc<Task<Fut>>) -> Waker { - let clone = self.clone(); - - // Safety: This is save because an `Arc` is a struct which contains - // a single field that is a pointer. - let ptr = unsafe { - mem::transmute::<Arc<Task<Fut>>, - NonNull<ArcTask<Fut>>>(clone) - }; - - let ptr = ptr as NonNull<dyn UnsafeWake>; - - // Hide lifetime of `Fut` - // Safety: The waker can safely outlive the future because the - // `UnsafeWake` impl is guaranteed to not touch `Fut`. - let ptr = unsafe { - mem::transmute::<NonNull<dyn UnsafeWake>, - NonNull<dyn UnsafeWake>>(ptr) - }; - - unsafe { Waker::new(ptr) } - } - +impl<Fut> Task<Fut> { /// Returns a local waker for this task without cloning the Arc. pub(super) fn local_waker<'a>(self: &'a Arc<Task<Fut>>) -> LocalWakerRef<'a> { - // Safety: This is safe because an `Arc` is a struct which contains - // a single field that is a pointer. - let ptr = unsafe { - *(self as *const _ as *const NonNull<ArcTaskUnowned<Fut>>) - }; - - let ptr = ptr as NonNull<dyn UnsafeWake>; - - // Hide lifetime of `self` - // Safety: - // - Since the `Arc` has not been cloned, the local waker must - // not outlive it. This is ensured by the lifetime of `LocalWakerRef`. - // - The local waker can safely outlive the future because the - // `UnsafeWake` impl is guaranteed to not touch `Fut`. - unsafe { - let ptr = mem::transmute::<NonNull<dyn UnsafeWake>, - NonNull<dyn UnsafeWake>>(ptr); - LocalWakerRef::new(LocalWaker::new(ptr)) - } + local_waker_ref_from_nonlocal(self) } + } impl<Fut> Drop for Task<Fut> { @@ -123,75 +86,3 @@ impl<Fut> Drop for Task<Fut> { } } } - -// `ArcTask<Fut>` represents conceptually the struct an `Arc<Task<Fut>>` points -// to. `*const ArcTask<Fut>` is equal to `Arc<Task<Fut>>` -// It may only be used through references because its layout obviously doesn't -// match the real inner struct of an `Arc` which (currently) has the form -// `{ strong, weak, data }`. -struct ArcTask<Fut>(PhantomData<Fut>); - -struct ArcTaskUnowned<Fut>(PhantomData<Fut>); // Doesn't drop the `Arc`'s data - -// We should never touch the future `Fut` on any thread other than the one -// owning `FuturesUnordered`, so this should be a safe operation. -unsafe impl<Fut> Send for ArcTask<Fut> {} -unsafe impl<Fut> Sync for ArcTask<Fut> {} - -unsafe impl<Fut> Send for ArcTaskUnowned<Fut> {} -unsafe impl<Fut> Sync for ArcTaskUnowned<Fut> {} - -// We need to implement `UnsafeWake` trait directly and can't implement `Wake` -// for `Task<Fut>` because `Fut`, the future, isn't required to have a static -// lifetime. `UnsafeWake` lets us forget about `Fut` and its lifetime. This is -// safe because neither `drop_raw` nor `wake` touch `Fut`. This is the case even -// though `drop_raw` runs the destructor for `Task<Fut>` because its destructor -// is guaranteed to not touch `Fut`. `Fut` must already have been dropped by the -// time it runs. See `Drop` impl for `Task<Fut>` for more details. -unsafe impl<Fut> UnsafeWake for ArcTask<Fut> { - #[inline] - unsafe fn clone_raw(&self) -> Waker { - let me: *const ArcTask<Fut> = self; - let task = &*(&me as *const *const ArcTask<Fut> - as *const Arc<Task<Fut>>); - task.waker() - } - - #[inline] - unsafe fn drop_raw(&self) { - let mut me: *const ArcTask<Fut> = self; - let task_ptr = &mut me as *mut *const ArcTask<Fut> - as *mut Arc<Task<Fut>>; - ptr::drop_in_place(task_ptr); - } - - #[inline] - unsafe fn wake(&self) { - let me: *const ArcTask<Fut> = self; - let task = &*(&me as *const *const ArcTask<Fut> - as *const Arc<Task<Fut>>); - task.wake(); - } -} - -unsafe impl<Fut> UnsafeWake for ArcTaskUnowned<Fut> { - #[inline] - unsafe fn clone_raw(&self) -> Waker { - let me: *const ArcTaskUnowned<Fut> = self; - let task = &*(&me as *const *const ArcTaskUnowned<Fut> - as *const Arc<Task<Fut>>); - task.waker() // Clones the `Arc` and the returned waker owns the - // clone. (`ArcTask<Fut>` not `ArcTaskUnowned<Fut>`) - } - - #[inline] - unsafe fn drop_raw(&self) {} // Does nothing - - #[inline] - unsafe fn wake(&self) { - let me: *const ArcTaskUnowned<Fut> = self; - let task = &*(&me as *const *const ArcTaskUnowned<Fut> - as *const Arc<Task<Fut>>); - task.wake(); - } -} diff --git a/futures-util/src/task/local_waker_ref.rs b/futures-util/src/task/local_waker_ref.rs index ea0e360d44..8d1d9bf4eb 100644 --- a/futures-util/src/task/local_waker_ref.rs +++ b/futures-util/src/task/local_waker_ref.rs @@ -2,9 +2,10 @@ use std::marker::PhantomData; use std::ops::Deref; -use std::ptr::NonNull; use std::sync::Arc; -use std::task::{LocalWaker, Waker, Wake, UnsafeWake}; +use std::task::{LocalWaker, ArcWake, RawWaker, RawWakerVTable}; + +// TODO: Maybe it's better to rename this type, e.g. to LocalArcWakerRef /// A [`LocalWaker`](::std::task::LocalWaker) that is only valid for a given lifetime. /// @@ -38,32 +39,56 @@ impl<'a> Deref for LocalWakerRef<'a> { } } -// Pointers to this type below are really pointers to `Arc<W>` -struct ReferencedArc<W> { - _marker: PhantomData<W>, +/// Creates the VTable for the LocalWaker in the reference +/// This LocalWaker does not increase the refcount of the Arc, since it +/// assumes it is still alive. Therefore it does not need to do anything on drop. +/// When a clone is created, this will increase the refcount and replace the +/// vtable with one that releases the refcount on drop. +macro_rules! local_ref_vtable { + ($ty:ident) => { + &RawWakerVTable { + clone: clone_arc_local_raw::<$ty>, + drop_fn: noop, + wake: wake_arc_local_raw::<$ty>, + into_waker: into_waker_raw::<$ty>, + } + }; } -unsafe impl<W: Wake + 'static> UnsafeWake for ReferencedArc<W> { - #[inline] - unsafe fn clone_raw(&self) -> Waker { - let me = self as *const ReferencedArc<W> as *const Arc<W>; - Arc::clone(&*me).into() - } - - #[inline] - unsafe fn drop_raw(&self) {} +// Another reference vtable which doesn't do decrement the refcount on drop. +// However on clone it will create a vtable which equals a Waker, and on wake +// it will call the nonlocal wake function. +macro_rules! nonlocal_ref_vtable { + ($ty:ident) => { + &RawWakerVTable { + clone: clone_arc_nonlocal_raw::<$ty>, + drop_fn: noop, + wake: wake_arc_nonlocal_raw::<$ty>, + into_waker: into_waker_raw::<$ty>, + } + }; +} - #[inline] - unsafe fn wake(&self) { - let me = self as *const ReferencedArc<W> as *const Arc<W>; - W::wake(&*me) - } +macro_rules! local_vtable { + ($ty:ident) => { + &RawWakerVTable { + clone: clone_arc_local_raw::<$ty>, + drop_fn: drop_arc_raw::<$ty>, + wake: wake_arc_local_raw::<$ty>, + into_waker: into_waker_raw::<$ty>, + } + }; +} - #[inline] - unsafe fn wake_local(&self) { - let me = self as *const ReferencedArc<W> as *const Arc<W>; - W::wake_local(&*me) - } +macro_rules! nonlocal_vtable { + ($ty:ident) => { + &RawWakerVTable { + clone: clone_arc_nonlocal_raw::<$ty>, + drop_fn: drop_arc_raw::<$ty>, + wake: wake_arc_nonlocal_raw::<$ty>, + into_waker: into_waker_raw::<$ty>, + } + }; } /// Creates a reference to a [`LocalWaker`](::std::task::LocalWaker) @@ -79,45 +104,19 @@ unsafe impl<W: Wake + 'static> UnsafeWake for ReferencedArc<W> { #[inline] pub unsafe fn local_waker_ref<W>(wake: &Arc<W>) -> LocalWakerRef<'_> where - W: Wake + 'static, + W: ArcWake { - let ptr = wake - as *const Arc<W> - as *const ReferencedArc<W> - as *const dyn UnsafeWake - as *mut dyn UnsafeWake; - let local_waker = LocalWaker::new(NonNull::new_unchecked(ptr)); + // This uses the same mechanism as Arc::into_raw, without needing a reference. + // This is potentially not stable + let ptr = &*wake as &W as *const W as *const(); + + let local_waker = LocalWaker::new_unchecked(RawWaker{ + data: ptr, + vtable: local_ref_vtable!(W), + }); LocalWakerRef::new(local_waker) } -// Pointers to this type below are really pointers to `Arc<W>`, -struct NonlocalReferencedArc<W> { - _marker: PhantomData<W>, -} - -unsafe impl<W: Wake + 'static> UnsafeWake for NonlocalReferencedArc<W> { - #[inline] - unsafe fn clone_raw(&self) -> Waker { - let me = self as *const NonlocalReferencedArc<W> as *const Arc<W>; - Arc::clone(&*me).into() - } - - #[inline] - unsafe fn drop_raw(&self) {} - - #[inline] - unsafe fn wake(&self) { - let me = self as *const NonlocalReferencedArc<W> as *const Arc<W>; - W::wake(&*me) - } - - #[inline] - unsafe fn wake_local(&self) { - let me = self as *const NonlocalReferencedArc<W> as *const Arc<W>; - W::wake(&*me) - } -} - /// Creates a reference to a [`LocalWaker`](::std::task::LocalWaker) /// from a non-local [`wake`](::std::task::Wake). /// @@ -128,13 +127,69 @@ unsafe impl<W: Wake + 'static> UnsafeWake for NonlocalReferencedArc<W> { #[inline] pub fn local_waker_ref_from_nonlocal<W>(wake: &Arc<W>) -> LocalWakerRef<'_> where - W: Wake + 'static, + W: ArcWake { - let ptr = wake - as *const Arc<W> - as *const NonlocalReferencedArc<W> - as *const dyn UnsafeWake - as *mut dyn UnsafeWake; - let local_waker = unsafe { LocalWaker::new(NonNull::new_unchecked(ptr)) }; + // This uses the same mechanism as Arc::into_raw, without needing a reference. + // This is potentially not stable + let ptr = &*wake as &W as *const W as *const(); + + let local_waker = unsafe { + LocalWaker::new_unchecked(RawWaker{ + data: ptr, + vtable: nonlocal_ref_vtable!(W), + }) + }; LocalWakerRef::new(local_waker) } + +unsafe fn noop(_data: *const()) { +} + +unsafe fn increase_refcount<T: ArcWake>(data: *const()) { + // Retain Arc by creating a copy + let arc: Arc<T> = Arc::from_raw(data as *const T); + let arc_clone = arc.clone(); + // Forget the Arcs again, so that the refcount isn't decrased + let _ = Arc::into_raw(arc); + let _ = Arc::into_raw(arc_clone); +} + +unsafe fn clone_arc_nonlocal_raw<T: ArcWake>(data: *const()) -> RawWaker { + increase_refcount::<T>(data); + RawWaker { + data: data, + vtable: nonlocal_vtable!(T), + } +} + +unsafe fn clone_arc_local_raw<T: ArcWake>(data: *const()) -> RawWaker { + increase_refcount::<T>(data); + RawWaker { + data: data, + vtable: local_vtable!(T), + } +} + +unsafe fn drop_arc_raw<T: ArcWake>(data: *const()) { + // Drop Arc + let _: Arc<T> = Arc::from_raw(data as *const T); +} + +unsafe fn wake_arc_local_raw<T: ArcWake>(data: *const()) { + let arc: Arc<T> = Arc::from_raw(data as *const T); + ArcWake::wake_local(&arc); // TODO: If this panics, the refcount is too big + let _ = Arc::into_raw(arc); +} + +unsafe fn wake_arc_nonlocal_raw<T: ArcWake>(data: *const()) { + let arc: Arc<T> = Arc::from_raw(data as *const T); + ArcWake::wake(&arc); // TODO: If this panics, the refcount is too big + let _ = Arc::into_raw(arc); +} + +unsafe fn into_waker_raw<T: ArcWake>(data: *const ()) -> Option<RawWaker> { + Some(RawWaker { + data: data, + vtable: nonlocal_vtable!(T), + }) +} diff --git a/futures-util/src/task/noop_waker.rs b/futures-util/src/task/noop_waker.rs index 704b0d041b..89b311dc78 100644 --- a/futures-util/src/task/noop_waker.rs +++ b/futures-util/src/task/noop_waker.rs @@ -1,41 +1,55 @@ //! Utilities for creating zero-cost wakers that don't do anything. -use futures_core::task::{LocalWaker, UnsafeWake, Waker}; -use core::ptr::NonNull; +use futures_core::task::{LocalWaker, RawWaker, RawWakerVTable}; +use core::ptr::null; +use core::cell::UnsafeCell; -#[derive(Debug)] -struct NoopWake { - _reserved: (), +unsafe fn noop_clone(_data: *const()) -> RawWaker { + noop_raw_waker() } -unsafe impl UnsafeWake for NoopWake { - unsafe fn clone_raw(&self) -> Waker { - noop_waker() - } - - unsafe fn drop_raw(&self) {} - - unsafe fn wake(&self) {} +unsafe fn noop(_data: *const()) { } -fn noop_unsafe_wake() -> NonNull<dyn UnsafeWake> { - static mut INSTANCE: NoopWake = NoopWake { _reserved: () }; - unsafe { NonNull::new_unchecked(&mut INSTANCE as *mut dyn UnsafeWake) } +unsafe fn noop_into_waker(_data: *const()) -> Option<RawWaker> { + Some(noop_raw_waker()) } -fn noop_waker() -> Waker { - unsafe { Waker::new(noop_unsafe_wake()) } +const NOOP_WAKER_VTABLE: RawWakerVTable = RawWakerVTable { + clone: noop_clone, + drop_fn: noop, + wake: noop, + into_waker: noop_into_waker, +}; + +fn noop_raw_waker() -> RawWaker { + RawWaker { + data: null(), + vtable: &NOOP_WAKER_VTABLE, + } } -/// Create a new [`LocalWaker`](futures_core::task::LocalWaker) referencing a -/// singleton instance of [`NoopWake`]. +/// Create a new [`LocalWaker`](futures_core::task::LocalWaker) which does +/// nothing when `wake()` is called on it. The [`LocalWaker`] can be converted +/// into a [`Waker`] which will behave the same way. +/// +/// # Examples +/// +/// ``` +/// #![feature(futures_api)] +/// use futures::task::noop_local_waker; +/// let lw = noop_local_waker(); +/// lw.wake(); +/// ``` #[inline] pub fn noop_local_waker() -> LocalWaker { - unsafe { LocalWaker::new(noop_unsafe_wake()) } + unsafe { + LocalWaker::new_unchecked(noop_raw_waker()) + } } /// Get a thread local reference to a /// [`LocalWaker`](futures_core::task::LocalWaker) referencing a singleton -/// instance of [`NoopWake`]. +/// instance of a [`LocalWaker`] which panics when woken. /// /// # Examples /// @@ -47,16 +61,10 @@ pub fn noop_local_waker() -> LocalWaker { /// ``` #[inline] pub fn noop_local_waker_ref() -> &'static LocalWaker { - static NOOP_WAKE_REF: &(dyn UnsafeWake + Sync) = &NoopWake { _reserved: () }; - // Unsafety: `Waker` and `LocalWaker` are `repr(transparent)` wrappers around - // `NonNull<dyn UnsafeWake>`, which has the same repr as `&(dyn UnsafeWake + Sync)` - // So an &'static &(dyn UnsafeWake + Sync) can be unsafely cast to a - // &'static LocalWaker - #[allow(clippy::transmute_ptr_to_ptr)] - unsafe { - core::mem::transmute::< - &&(dyn UnsafeWake + Sync), - &'static LocalWaker, - >(&NOOP_WAKE_REF) + thread_local! { + static NOOP_WAKER_INSTANCE: UnsafeCell<LocalWaker> = + UnsafeCell::new(noop_local_waker()); } + NOOP_WAKER_INSTANCE.with(|l| unsafe { &*l.get() }) } + diff --git a/futures/src/lib.rs b/futures/src/lib.rs index 7309d4d782..38ac511d35 100644 --- a/futures/src/lib.rs +++ b/futures/src/lib.rs @@ -354,12 +354,12 @@ pub mod task { pub use futures_core::task::{ Poll, Spawn, LocalSpawn, SpawnError, - Waker, LocalWaker, UnsafeWake, + Waker, LocalWaker, RawWaker, RawWakerVTable }; #[cfg(feature = "std")] pub use futures_core::task::{ - Wake, local_waker, local_waker_from_nonlocal + ArcWake, }; #[cfg(feature = "std")]