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")]