diff --git a/futures-channel/Cargo.toml b/futures-channel/Cargo.toml index 4051ebf979..5e97175d76 100644 --- a/futures-channel/Cargo.toml +++ b/futures-channel/Cargo.toml @@ -23,4 +23,5 @@ futures-core-preview = { path = "../futures-core", version = "0.3.0-alpha.6", de [dev-dependencies] futures-preview = { path = "../futures", version = "0.3.0-alpha.6", default-features = true } +futures-test-preview = { path = "../futures-test", version = "0.3.0-alpha.6", default-features = true } pin-utils = "0.1.0-alpha.2" diff --git a/futures-channel/benches/sync_mpsc.rs b/futures-channel/benches/sync_mpsc.rs index dc0b5c0411..8e801b1f03 100644 --- a/futures-channel/benches/sync_mpsc.rs +++ b/futures-channel/benches/sync_mpsc.rs @@ -1,91 +1,75 @@ #![feature(test, futures_api, pin, arbitrary_self_types)] -use futures::ready; -use futures::channel::mpsc::{self, Sender, UnboundedSender}; -use futures::executor::LocalPool; -use futures::stream::{Stream, StreamExt}; -use futures::sink::Sink; -use futures::task::{self, Poll, Wake, LocalWaker}; -use std::pin::PinMut; -use std::sync::Arc; -use test::Bencher; - -fn notify_noop() -> LocalWaker { - struct Noop; - - impl Wake for Noop { - fn wake(_: &Arc) {} - } - - task::local_waker_from_nonlocal(Arc::new(Noop)) -} - -fn noop_cx(f: impl FnOnce(&mut task::Context)) { - let pool = LocalPool::new(); - let mut spawn = pool.spawner(); - let waker = notify_noop(); - let cx = &mut task::Context::new(&waker, &mut spawn); - f(cx) -} +extern crate test; +use crate::test::Bencher; + +use { + futures::{ + channel::mpsc::{self, Sender, UnboundedSender}, + ready, + stream::{Stream, StreamExt}, + sink::Sink, + task::{LocalWaker, Poll}, + }, + futures_test::task::noop_local_waker_ref, + std::pin::Pin, +}; /// Single producer, single consumer #[bench] fn unbounded_1_tx(b: &mut Bencher) { - noop_cx(|cx| { - b.iter(|| { - let (tx, mut rx) = mpsc::unbounded(); + let lw = noop_local_waker_ref(); + b.iter(|| { + let (tx, mut rx) = mpsc::unbounded(); - // 1000 iterations to avoid measuring overhead of initialization - // Result should be divided by 1000 - for i in 0..1000 { + // 1000 iterations to avoid measuring overhead of initialization + // Result should be divided by 1000 + for i in 0..1000 { - // Poll, not ready, park - assert_eq!(Poll::Pending, rx.poll_next_unpin(cx)); + // Poll, not ready, park + assert_eq!(Poll::Pending, rx.poll_next_unpin(lw)); - UnboundedSender::unbounded_send(&tx, i).unwrap(); + UnboundedSender::unbounded_send(&tx, i).unwrap(); - // Now poll ready - assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(cx)); - } - }) + // Now poll ready + assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(lw)); + } }) } /// 100 producers, single consumer #[bench] fn unbounded_100_tx(b: &mut Bencher) { - noop_cx(|cx| { - b.iter(|| { - let (tx, mut rx) = mpsc::unbounded(); + let lw = noop_local_waker_ref(); + b.iter(|| { + let (tx, mut rx) = mpsc::unbounded(); - let tx: Vec<_> = (0..100).map(|_| tx.clone()).collect(); + let tx: Vec<_> = (0..100).map(|_| tx.clone()).collect(); - // 1000 send/recv operations total, result should be divided by 1000 - for _ in 0..10 { - for i in 0..tx.len() { - assert_eq!(Poll::Pending, rx.poll_next_unpin(cx)); + // 1000 send/recv operations total, result should be divided by 1000 + for _ in 0..10 { + for i in 0..tx.len() { + assert_eq!(Poll::Pending, rx.poll_next_unpin(lw)); - UnboundedSender::unbounded_send(&tx[i], i).unwrap(); + UnboundedSender::unbounded_send(&tx[i], i).unwrap(); - assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(cx)); - } + assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(lw)); } - }) + } }) } #[bench] fn unbounded_uncontended(b: &mut Bencher) { - noop_cx(|cx| { - b.iter(|| { - let (tx, mut rx) = mpsc::unbounded(); - - for i in 0..1000 { - UnboundedSender::unbounded_send(&tx, i).expect("send"); - // No need to create a task, because poll is not going to park. - assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(cx)); - } - }) + let lw = noop_local_waker_ref(); + b.iter(|| { + let (tx, mut rx) = mpsc::unbounded(); + + for i in 0..1000 { + UnboundedSender::unbounded_send(&tx, i).expect("send"); + // No need to create a task, because poll is not going to park. + assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(lw)); + } }) } @@ -100,16 +84,16 @@ struct TestSender { impl Stream for TestSender { type Item = u32; - fn poll_next(mut self: PinMut, cx: &mut task::Context) + fn poll_next(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { let this = &mut *self; - let mut tx = PinMut::new(&mut this.tx); + let mut tx = Pin::new(&mut this.tx); - ready!(tx.reborrow().poll_ready(cx)).unwrap(); - tx.reborrow().start_send(this.last + 1).unwrap(); + ready!(tx.as_mut().poll_ready(lw)).unwrap(); + tx.as_mut().start_send(this.last + 1).unwrap(); this.last += 1; - assert_eq!(Poll::Ready(Ok(())), tx.reborrow().poll_flush(cx)); + assert_eq!(Poll::Ready(Ok(())), tx.as_mut().poll_flush(lw)); Poll::Ready(Some(this.last)) } } @@ -117,46 +101,44 @@ impl Stream for TestSender { /// Single producers, single consumer #[bench] fn bounded_1_tx(b: &mut Bencher) { - noop_cx(|cx| { - b.iter(|| { - let (tx, mut rx) = mpsc::channel(0); + let lw = noop_local_waker_ref(); + b.iter(|| { + let (tx, mut rx) = mpsc::channel(0); - let mut tx = TestSender { tx, last: 0 }; + let mut tx = TestSender { tx, last: 0 }; - for i in 0..1000 { - assert_eq!(Poll::Ready(Some(i + 1)), tx.poll_next_unpin(cx)); - assert_eq!(Poll::Pending, tx.poll_next_unpin(cx)); - assert_eq!(Poll::Ready(Some(i + 1)), rx.poll_next_unpin(cx)); - } - }) + for i in 0..1000 { + assert_eq!(Poll::Ready(Some(i + 1)), tx.poll_next_unpin(lw)); + assert_eq!(Poll::Pending, tx.poll_next_unpin(lw)); + assert_eq!(Poll::Ready(Some(i + 1)), rx.poll_next_unpin(lw)); + } }) } /// 100 producers, single consumer #[bench] fn bounded_100_tx(b: &mut Bencher) { - noop_cx(|cx| { - b.iter(|| { - // Each sender can send one item after specified capacity - let (tx, mut rx) = mpsc::channel(0); - - let mut tx: Vec<_> = (0..100).map(|_| { - TestSender { - tx: tx.clone(), - last: 0 - } - }).collect(); - - for i in 0..10 { - for j in 0..tx.len() { - // Send an item - assert_eq!(Poll::Ready(Some(i + 1)), tx[j].poll_next_unpin(cx)); - // Then block - assert_eq!(Poll::Pending, tx[j].poll_next_unpin(cx)); - // Recv the item - assert_eq!(Poll::Ready(Some(i + 1)), rx.poll_next_unpin(cx)); - } + let lw = noop_local_waker_ref(); + b.iter(|| { + // Each sender can send one item after specified capacity + let (tx, mut rx) = mpsc::channel(0); + + let mut tx: Vec<_> = (0..100).map(|_| { + TestSender { + tx: tx.clone(), + last: 0 + } + }).collect(); + + for i in 0..10 { + for j in 0..tx.len() { + // Send an item + assert_eq!(Poll::Ready(Some(i + 1)), tx[j].poll_next_unpin(lw)); + // Then block + assert_eq!(Poll::Pending, tx[j].poll_next_unpin(lw)); + // Recv the item + assert_eq!(Poll::Ready(Some(i + 1)), rx.poll_next_unpin(lw)); } - }) + } }) } diff --git a/futures-channel/src/mpsc/mod.rs b/futures-channel/src/mpsc/mod.rs index 675aca7eb1..ac730b3578 100644 --- a/futures-channel/src/mpsc/mod.rs +++ b/futures-channel/src/mpsc/mod.rs @@ -79,12 +79,12 @@ // by the queue structure. use futures_core::stream::Stream; -use futures_core::task::{self, Waker, Poll}; +use futures_core::task::{LocalWaker, Waker, Poll}; use std::any::Any; use std::error::Error; use std::fmt; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; use std::sync::{Arc, Mutex}; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; @@ -113,7 +113,7 @@ pub struct Sender { maybe_parked: bool, } -// We never project PinMut to `PinMut` +// We never project Pin<&mut Sender> to `Pin<&mut T>` impl Unpin for Sender {} /// The transmission end of an unbounded mpsc channel. @@ -139,7 +139,7 @@ pub struct Receiver { #[derive(Debug)] pub struct UnboundedReceiver(Receiver); -// `PinMut>` is never projected to `PinMut` +// `Pin<&mut UnboundedReceiver>` is never projected to `Pin<&mut T>` impl Unpin for UnboundedReceiver {} /// The error type for [`Sender`s](Sender) used as `Sink`s. @@ -451,7 +451,7 @@ impl Sender { // Do the send without failing // None means close - fn do_send(&mut self, cx: Option<&mut task::Context>, msg: T) + fn do_send(&mut self, lw: Option<&LocalWaker>, msg: T) -> Result<(), TrySendError> { // Anyone callig do_send *should* make sure there is room first, @@ -483,7 +483,7 @@ impl Sender { // maintain internal consistency, a blank message is pushed onto the // parked task queue. if park_self { - self.park(cx); + self.park(lw); } self.queue_push_and_signal(Some(msg)); @@ -616,10 +616,10 @@ impl Sender { } } - fn park(&mut self, cx: Option<&mut task::Context>) { + fn park(&mut self, lw: Option<&LocalWaker>) { // TODO: clean up internal state if the task::current will fail - let task = cx.map(|cx| cx.waker().clone()); + let task = lw.map(|lw| lw.clone().into_waker()); { let mut sender = self.sender_task.lock().unwrap(); @@ -651,7 +651,7 @@ impl Sender { /// - `Err(SendError)` if the receiver has been dropped. pub fn poll_ready( &mut self, - cx: &mut task::Context + lw: &LocalWaker ) -> Poll> { let state = decode_state(self.inner.state.load(SeqCst)); if !state.is_open { @@ -660,7 +660,7 @@ impl Sender { })); } - self.poll_unparked(Some(cx)).map(Ok) + self.poll_unparked(Some(lw)).map(Ok) } /// Returns whether this channel is closed without needing a context. @@ -677,7 +677,7 @@ impl Sender { let _ = self.do_send_nb(None); } - fn poll_unparked(&mut self, cx: Option<&mut task::Context>) -> Poll<()> { + fn poll_unparked(&mut self, lw: Option<&LocalWaker>) -> Poll<()> { // First check the `maybe_parked` variable. This avoids acquiring the // lock in most cases if self.maybe_parked { @@ -695,7 +695,7 @@ impl Sender { // // Update the task in case the `Sender` has been moved to another // task - task.task = cx.map(|cx| cx.waker().clone()); + task.task = lw.map(|lw| lw.clone().into_waker()); Poll::Pending } else { @@ -708,7 +708,7 @@ impl UnboundedSender { /// Check if the channel is ready to receive a message. pub fn poll_ready( &self, - _: &mut task::Context, + _: &LocalWaker, ) -> Poll> { self.0.poll_ready_nb() } @@ -914,7 +914,7 @@ impl Receiver { } // Try to park the receiver task - fn try_park(&self, cx: &mut task::Context) -> TryPark { + fn try_park(&self, lw: &LocalWaker) -> TryPark { let curr = self.inner.state.load(SeqCst); let state = decode_state(curr); @@ -932,7 +932,7 @@ impl Receiver { return TryPark::NotEmpty; } - recv_task.task = Some(cx.waker().clone()); + recv_task.task = Some(lw.clone().into_waker()); TryPark::Parked } @@ -953,15 +953,15 @@ impl Receiver { } } -// The receiver does not ever take a PinMut to the inner T +// The receiver does not ever take a Pin to the inner T impl Unpin for Receiver {} impl Stream for Receiver { type Item = T; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { loop { // Try to read a message off of the message queue. @@ -971,7 +971,7 @@ impl Stream for Receiver { // There are no messages to read, in this case, attempt to // park. The act of parking will verify that the channel is // still empty after the park operation has completed. - match self.try_park(cx) { + match self.try_park(lw) { TryPark::Parked => { // The task was parked, and the channel is still // empty, return Pending. @@ -1030,10 +1030,10 @@ impl Stream for UnboundedReceiver { type Item = T; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - PinMut::new(&mut self.0).poll_next(cx) + Pin::new(&mut self.0).poll_next(lw) } } diff --git a/futures-channel/src/oneshot.rs b/futures-channel/src/oneshot.rs index 7e261e621c..d233a49a85 100644 --- a/futures-channel/src/oneshot.rs +++ b/futures-channel/src/oneshot.rs @@ -1,9 +1,9 @@ //! A channel for sending a single message between asynchronous tasks. use futures_core::future::Future; -use futures_core::task::{self, Poll, Waker}; +use futures_core::task::{LocalWaker, Poll, Waker}; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; use std::sync::Arc; use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering::SeqCst; @@ -29,7 +29,7 @@ pub struct Sender { inner: Arc>, } -// The channels do not ever project PinMut to the inner T +// The channels do not ever project Pin to the inner T impl Unpin for Receiver {} impl Unpin for Sender {} @@ -155,7 +155,7 @@ impl Inner { } } - fn poll_cancel(&self, cx: &mut task::Context) -> Poll<()> { + fn poll_cancel(&self, lw: &LocalWaker) -> Poll<()> { // Fast path up first, just read the flag and see if our other half is // gone. This flag is set both in our destructor and the oneshot // destructor, but our destructor hasn't run yet so if it's set then the @@ -177,7 +177,7 @@ impl Inner { // `Receiver` may have been dropped. The first thing it does is set the // flag, and if it fails to acquire the lock it assumes that we'll see // the flag later on. So... we then try to see the flag later on! - let handle = cx.waker().clone(); + let handle = lw.clone().into_waker(); match self.tx_task.try_lock() { Some(mut p) => *p = Some(handle), None => return Poll::Ready(()), @@ -250,7 +250,7 @@ impl Inner { } } - fn recv(&self, cx: &mut task::Context) -> Poll> { + fn recv(&self, lw: &LocalWaker) -> Poll> { // Check to see if some data has arrived. If it hasn't then we need to // block our task. // @@ -261,7 +261,7 @@ impl Inner { let done = if self.complete.load(SeqCst) { true } else { - let task = cx.waker().clone(); + let task = lw.clone().into_waker(); match self.rx_task.try_lock() { Some(mut slot) => { *slot = Some(task); false }, None => true, @@ -349,8 +349,8 @@ impl Sender { /// alive and may be able to receive a message if sent. The current task, /// however, is scheduled to receive a notification if the corresponding /// `Receiver` goes away. - pub fn poll_cancel(&mut self, cx: &mut task::Context) -> Poll<()> { - self.inner.poll_cancel(cx) + pub fn poll_cancel(&mut self, lw: &LocalWaker) -> Poll<()> { + self.inner.poll_cancel(lw) } /// Tests to see whether this `Sender`'s corresponding `Receiver` @@ -401,9 +401,6 @@ impl Receiver { /// Attempts to receive a message outside of the context of a task. /// - /// Useful when a [`Context`](task::Context) is not available - /// such as within a `Drop` impl. - /// /// Does not schedule a task wakeup or have any other side effects. /// /// A return value of `None` must be considered immediately stale (out of @@ -419,10 +416,10 @@ impl Future for Receiver { type Output = Result; fn poll( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.inner.recv(cx) + self.inner.recv(lw) } } diff --git a/futures-channel/tests/channel.rs b/futures-channel/tests/channel.rs index a2af8b52f9..d2b3a38fbe 100644 --- a/futures-channel/tests/channel.rs +++ b/futures-channel/tests/channel.rs @@ -36,8 +36,8 @@ async fn send_sequence(n: u32, mut sender: mpsc::Sender) { fn drop_sender() { let (tx, mut rx) = mpsc::channel::(1); drop(tx); - let f = poll_fn(|cx| { - rx.poll_next_unpin(cx) + let f = poll_fn(|lw| { + rx.poll_next_unpin(lw) }); assert_eq!(block_on(f), None) } diff --git a/futures-channel/tests/mpsc.rs b/futures-channel/tests/mpsc.rs index 5356a70bd4..182f20644c 100644 --- a/futures-channel/tests/mpsc.rs +++ b/futures-channel/tests/mpsc.rs @@ -28,34 +28,34 @@ fn send_recv() { #[test] fn send_recv_no_buffer() { // Run on a task context - block_on(poll_fn(move |cx| { + block_on(poll_fn(move |lw| { let (tx, rx) = mpsc::channel::(0); pin_mut!(tx, rx); - assert!(tx.reborrow().poll_flush(cx).is_ready()); - assert!(tx.reborrow().poll_ready(cx).is_ready()); + assert!(tx.as_mut().poll_flush(lw).is_ready()); + assert!(tx.as_mut().poll_ready(lw).is_ready()); // Send first message - assert!(tx.reborrow().start_send(1).is_ok()); - assert!(tx.reborrow().poll_ready(cx).is_pending()); + assert!(tx.as_mut().start_send(1).is_ok()); + assert!(tx.as_mut().poll_ready(lw).is_pending()); // poll_ready said Pending, so no room in buffer, therefore new sends // should get rejected with is_full. - assert!(tx.reborrow().start_send(0).unwrap_err().is_full()); - assert!(tx.reborrow().poll_ready(cx).is_pending()); + assert!(tx.as_mut().start_send(0).unwrap_err().is_full()); + assert!(tx.as_mut().poll_ready(lw).is_pending()); // Take the value - assert_eq!(rx.reborrow().poll_next(cx), Poll::Ready(Some(1))); - assert!(tx.reborrow().poll_ready(cx).is_ready()); + assert_eq!(rx.as_mut().poll_next(lw), Poll::Ready(Some(1))); + assert!(tx.as_mut().poll_ready(lw).is_ready()); // Send second message - assert!(tx.reborrow().poll_ready(cx).is_ready()); - assert!(tx.reborrow().start_send(2).is_ok()); - assert!(tx.reborrow().poll_ready(cx).is_pending()); + assert!(tx.as_mut().poll_ready(lw).is_ready()); + assert!(tx.as_mut().start_send(2).is_ok()); + assert!(tx.as_mut().poll_ready(lw).is_pending()); // Take the value - assert_eq!(rx.reborrow().poll_next(cx), Poll::Ready(Some(2))); - assert!(tx.reborrow().poll_ready(cx).is_ready()); + assert_eq!(rx.as_mut().poll_next(lw), Poll::Ready(Some(2))); + assert!(tx.as_mut().poll_ready(lw).is_ready()); Poll::Ready(()) })); @@ -118,11 +118,11 @@ fn recv_close_gets_none() { let (mut tx, mut rx) = mpsc::channel::(10); // Run on a task context - block_on(poll_fn(move |cx| { + block_on(poll_fn(move |lw| { rx.close(); - assert_eq!(rx.poll_next_unpin(cx), Poll::Ready(None)); - match tx.poll_ready(cx) { + assert_eq!(rx.poll_next_unpin(lw), Poll::Ready(None)); + match tx.poll_ready(lw) { Poll::Pending | Poll::Ready(Ok(_)) => panic!(), Poll::Ready(Err(e)) => assert!(e.is_disconnected()), }; @@ -138,9 +138,9 @@ fn tx_close_gets_none() { let (_, mut rx) = mpsc::channel::(10); // Run on a task context - block_on(poll_fn(move |cx| { - assert_eq!(rx.poll_next_unpin(cx), Poll::Ready(None)); - assert_eq!(rx.poll_next_unpin(cx), Poll::Ready(None)); + block_on(poll_fn(move |lw| { + assert_eq!(rx.poll_next_unpin(lw), Poll::Ready(None)); + assert_eq!(rx.poll_next_unpin(lw), Poll::Ready(None)); Poll::Ready(()) })); @@ -304,8 +304,8 @@ fn stress_receiver_multi_task_bounded_hard() { } else { // Just poll let n = n.clone(); - let f = poll_fn(move |cx| { - let r = match rx.poll_next_unpin(cx) { + let f = poll_fn(move |lw| { + let r = match rx.poll_next_unpin(lw) { Poll::Ready(Some(_)) => { n.fetch_add(1, Ordering::Relaxed); false @@ -475,8 +475,8 @@ fn try_send_2() { let (readytx, readyrx) = oneshot::channel::<()>(); let th = thread::spawn(move || { - block_on(poll_fn(|cx| { - assert!(tx.poll_ready(cx).is_pending()); + block_on(poll_fn(|lw| { + assert!(tx.poll_ready(lw).is_pending()); Poll::Ready(()) })); diff --git a/futures-channel/tests/oneshot.rs b/futures-channel/tests/oneshot.rs index 2b7bcff91b..4b4aecc342 100644 --- a/futures-channel/tests/oneshot.rs +++ b/futures-channel/tests/oneshot.rs @@ -3,8 +3,8 @@ use futures::channel::oneshot::{self, Sender}; use futures::executor::block_on; use futures::future::{Future, FutureExt, poll_fn}; -use futures::task::{self, Poll}; -use std::pin::PinMut; +use futures::task::{LocalWaker, Poll}; +use std::pin::Pin; use std::sync::mpsc; use std::thread; @@ -12,12 +12,12 @@ use std::thread; fn smoke_poll() { let (mut tx, rx) = oneshot::channel::(); let mut rx = Some(rx); - let f = poll_fn(|cx| { - assert!(tx.poll_cancel(cx).is_pending()); - assert!(tx.poll_cancel(cx).is_pending()); + let f = poll_fn(|lw| { + assert!(tx.poll_cancel(lw).is_pending()); + assert!(tx.poll_cancel(lw).is_pending()); drop(rx.take()); - assert!(tx.poll_cancel(cx).is_ready()); - assert!(tx.poll_cancel(cx).is_ready()); + assert!(tx.poll_cancel(lw).is_ready()); + assert!(tx.poll_cancel(lw).is_ready()); Poll::Ready(()) }); @@ -42,8 +42,8 @@ struct WaitForCancel { impl Future for WaitForCancel { type Output = (); - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - self.tx.poll_cancel(cx) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.tx.poll_cancel(lw) } } @@ -73,12 +73,12 @@ fn cancel_lots() { fn close() { let (mut tx, mut rx) = oneshot::channel::(); rx.close(); - block_on(poll_fn(|cx| { - match rx.poll_unpin(cx) { + block_on(poll_fn(|lw| { + match rx.poll_unpin(lw) { Poll::Ready(Err(_)) => {}, _ => panic!(), }; - assert!(tx.poll_cancel(cx).is_ready()); + assert!(tx.poll_cancel(lw).is_ready()); Poll::Ready(()) })); } diff --git a/futures-core/src/future/future_obj.rs b/futures-core/src/future/future_obj.rs new file mode 100644 index 0000000000..90326ac7b0 --- /dev/null +++ b/futures-core/src/future/future_obj.rs @@ -0,0 +1,260 @@ +use core::{ + fmt, mem, + future::Future, + marker::{PhantomData, Unpin}, + pin::Pin, + task::{LocalWaker, Poll}, +}; + +/// A custom trait object for polling futures, roughly akin to +/// `Box + 'a>`. +/// +/// This custom trait object was introduced for two reasons: +/// - Currently it is not possible to take `dyn Trait` by value and +/// `Box` is not available in no_std contexts. +/// - The `Future` trait is currently not object safe: The `Future::poll` +/// method makes uses the arbitrary self types feature and traits in which +/// this feature is used are currently not object safe due to current compiler +/// limitations. (See tracking issue for arbitrary self types for more +/// information #44874) +pub struct LocalFutureObj<'a, T> { + ptr: *mut (), + poll_fn: unsafe fn(*mut (), &LocalWaker) -> Poll, + drop_fn: unsafe fn(*mut ()), + _marker: PhantomData<&'a ()>, +} + +impl<'a, T> Unpin for LocalFutureObj<'a, T> {} + +impl<'a, T> LocalFutureObj<'a, T> { + /// Create a `LocalFutureObj` from a custom trait object representation. + #[inline] + pub fn new + 'a>(f: F) -> LocalFutureObj<'a, T> { + LocalFutureObj { + ptr: f.into_raw(), + poll_fn: F::poll, + drop_fn: F::drop, + _marker: PhantomData, + } + } + + /// Converts the `LocalFutureObj` into a `FutureObj` + /// To make this operation safe one has to ensure that the `UnsafeFutureObj` + /// instance from which this `LocalFutureObj` was created actually + /// implements `Send`. + #[inline] + pub unsafe fn into_future_obj(self) -> FutureObj<'a, T> { + FutureObj(self) + } +} + +impl<'a, T> fmt::Debug for LocalFutureObj<'a, T> { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_struct("LocalFutureObj") + .finish() + } +} + +impl<'a, T> From> for LocalFutureObj<'a, T> { + #[inline] + fn from(f: FutureObj<'a, T>) -> LocalFutureObj<'a, T> { + f.0 + } +} + +impl<'a, T> Future for LocalFutureObj<'a, T> { + type Output = T; + + #[inline] + fn poll(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + unsafe { + ((*self).poll_fn)((*self).ptr, lw) + } + } +} + +impl<'a, T> Drop for LocalFutureObj<'a, T> { + fn drop(&mut self) { + unsafe { + (self.drop_fn)(self.ptr) + } + } +} + +/// A custom trait object for polling futures, roughly akin to +/// `Box + Send + 'a>`. +/// +/// This custom trait object was introduced for two reasons: +/// - Currently it is not possible to take `dyn Trait` by value and +/// `Box` is not available in no_std contexts. +/// - The `Future` trait is currently not object safe: The `Future::poll` +/// method makes uses the arbitrary self types feature and traits in which +/// this feature is used are currently not object safe due to current compiler +/// limitations. (See tracking issue for arbitrary self types for more +/// information #44874) +pub struct FutureObj<'a, T>(LocalFutureObj<'a, T>); + +impl<'a, T> Unpin for FutureObj<'a, T> {} +unsafe impl<'a, T> Send for FutureObj<'a, T> {} + +impl<'a, T> FutureObj<'a, T> { + /// Create a `FutureObj` from a custom trait object representation. + #[inline] + pub fn new + Send>(f: F) -> FutureObj<'a, T> { + FutureObj(LocalFutureObj::new(f)) + } +} + +impl<'a, T> fmt::Debug for FutureObj<'a, T> { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_struct("FutureObj") + .finish() + } +} + +impl<'a, T> Future for FutureObj<'a, T> { + type Output = T; + + #[inline] + fn poll(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + let pinned_field: Pin<&mut LocalFutureObj<'a, T>> = unsafe { + Pin::map_unchecked_mut(self, |x| &mut x.0) + }; + LocalFutureObj::poll(pinned_field, lw) + } +} + +/// A custom implementation of a future trait object for `FutureObj`, providing +/// a hand-rolled vtable. +/// +/// This custom representation is typically used only in `no_std` contexts, +/// where the default `Box`-based implementation is not available. +/// +/// The implementor must guarantee that it is safe to call `poll` repeatedly (in +/// a non-concurrent fashion) with the result of `into_raw` until `drop` is +/// called. +pub unsafe trait UnsafeFutureObj<'a, T>: 'a { + /// Convert an owned instance into a (conceptually owned) void pointer. + fn into_raw(self) -> *mut (); + + /// Poll the future represented by the given void pointer. + /// + /// # Safety + /// + /// The trait implementor must guarantee that it is safe to repeatedly call + /// `poll` with the result of `into_raw` until `drop` is called; such calls + /// are not, however, allowed to race with each other or with calls to + /// `drop`. + unsafe fn poll(ptr: *mut (), lw: &LocalWaker) -> Poll; + + /// Drops the future represented by the given void pointer. + /// + /// # Safety + /// + /// The trait implementor must guarantee that it is safe to call this + /// function once per `into_raw` invocation; that call cannot race with + /// other calls to `drop` or `poll`. + unsafe fn drop(ptr: *mut ()); +} + +unsafe impl<'a, T, F> UnsafeFutureObj<'a, T> for &'a mut F +where + F: Future + Unpin + 'a +{ + fn into_raw(self) -> *mut () { + self as *mut F as *mut () + } + + unsafe fn poll(ptr: *mut (), lw: &LocalWaker) -> Poll { + let p: Pin<&mut F> = Pin::new_unchecked(&mut *(ptr as *mut F)); + F::poll(p, lw) + } + + unsafe fn drop(_ptr: *mut ()) {} +} + +unsafe impl<'a, T, F> UnsafeFutureObj<'a, T> for Pin<&'a mut F> +where + F: Future + 'a +{ + fn into_raw(mut self) -> *mut () { + let mut_ref: &mut F = unsafe { Pin::get_mut_unchecked(Pin::as_mut(&mut self)) }; + mut_ref as *mut F as *mut () + } + + unsafe fn poll(ptr: *mut (), lw: &LocalWaker) -> Poll { + let future: Pin<&mut F> = Pin::new_unchecked(&mut *(ptr as *mut F)); + F::poll(future, lw) + } + + unsafe fn drop(_ptr: *mut ()) {} +} + +if_std! { + use std::boxed::Box; + + unsafe impl<'a, T, F> UnsafeFutureObj<'a, T> for Box + where F: Future + 'a + { + fn into_raw(self) -> *mut () { + Box::into_raw(self) as *mut () + } + + unsafe fn poll(ptr: *mut (), lw: &LocalWaker) -> Poll { + let ptr = ptr as *mut F; + let pin: Pin<&mut F> = Pin::new_unchecked(&mut *ptr); + F::poll(pin, lw) + } + + unsafe fn drop(ptr: *mut ()) { + drop(Box::from_raw(ptr as *mut F)) + } + } + + unsafe impl<'a, T, F> UnsafeFutureObj<'a, T> for Pin> + where + F: Future + 'a + { + fn into_raw(mut self) -> *mut () { + let mut_ref: &mut F = unsafe { Pin::get_mut_unchecked(Pin::as_mut(&mut self)) }; + let ptr = mut_ref as *mut F as *mut (); + mem::forget(self); // Don't drop the box + ptr + } + + unsafe fn poll(ptr: *mut (), lw: &LocalWaker) -> Poll { + let ptr = ptr as *mut F; + let pin: Pin<&mut F> = Pin::new_unchecked(&mut *ptr); + F::poll(pin, lw) + } + + unsafe fn drop(ptr: *mut ()) { + #[allow(clippy::cast_ptr_alignment)] + drop(Box::from_raw(ptr as *mut Pin>)); + } + } + + impl<'a, F: Future + Send + 'a> From>> for FutureObj<'a, ()> { + fn from(boxed: Pin>) -> Self { + FutureObj::new(boxed) + } + } + + impl<'a, F: Future + Send + 'a> From> for FutureObj<'a, ()> { + fn from(boxed: Box) -> Self { + FutureObj::new(boxed) + } + } + + impl<'a, F: Future + 'a> From>> for LocalFutureObj<'a, ()> { + fn from(boxed: Pin>) -> Self { + LocalFutureObj::new(boxed) + } + } + + impl<'a, F: Future + 'a> From> for LocalFutureObj<'a, ()> { + fn from(boxed: Box) -> Self { + LocalFutureObj::new(boxed) + } + } +} diff --git a/futures-core/src/future.rs b/futures-core/src/future/mod.rs similarity index 69% rename from futures-core/src/future.rs rename to futures-core/src/future/mod.rs index fc5a71165a..cc80ef43fb 100644 --- a/futures-core/src/future.rs +++ b/futures-core/src/future/mod.rs @@ -1,9 +1,12 @@ //! Futures. -use crate::task::{self, Poll}; -use core::pin::PinMut; +use crate::task::{LocalWaker, Poll}; +use core::pin::Pin; -pub use core::future::{Future, FutureObj, LocalFutureObj, UnsafeFutureObj}; +pub use core::future::Future; + +mod future_obj; +pub use self::future_obj::{FutureObj, LocalFutureObj, UnsafeFutureObj}; /// A convenience for futures that return `Result` values that includes /// a variety of adapters tailored to such futures. @@ -20,8 +23,8 @@ pub trait TryFuture { /// directly inheriting from the `Future` trait; in the future it won't be /// needed. fn try_poll( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll>; } @@ -32,7 +35,7 @@ impl TryFuture for F type Error = E; #[inline] - fn try_poll(self: PinMut, cx: &mut task::Context) -> Poll { - self.poll(cx) + fn try_poll(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.poll(lw) } } diff --git a/futures-core/src/lib.rs b/futures-core/src/lib.rs index cd58433e87..4e0a067aa8 100644 --- a/futures-core/src/lib.rs +++ b/futures-core/src/lib.rs @@ -1,6 +1,7 @@ //! Core traits and types for asynchronous operations in Rust. #![feature(pin, arbitrary_self_types, futures_api)] +#![feature(tool_lints)] // to allow clippy lints #![no_std] diff --git a/futures-core/src/stream/mod.rs b/futures-core/src/stream/mod.rs index 3d4a6d1438..17b0dff449 100644 --- a/futures-core/src/stream/mod.rs +++ b/futures-core/src/stream/mod.rs @@ -1,8 +1,9 @@ //! Asynchronous streams. -use crate::task::{self, Poll}; +use crate::task::{LocalWaker, Poll}; use core::marker::Unpin; -use core::pin::PinMut; +use core::ops; +use core::pin::Pin; #[cfg(feature = "either")] use either::Either; @@ -52,8 +53,8 @@ pub trait Stream { /// to ensure that `poll_next` always returns `Ready(None)` in subsequent /// calls. fn poll_next( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll>; } @@ -61,21 +62,25 @@ impl<'a, S: ?Sized + Stream + Unpin> Stream for &'a mut S { type Item = S::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - S::poll_next(PinMut::new(&mut **self), cx) + S::poll_next(Pin::new(&mut **self), lw) } } -impl<'a, S: ?Sized + Stream> Stream for PinMut<'a, S> { - type Item = S::Item; +impl

Stream for Pin

+where + P: ops::DerefMut, + P::Target: Stream, +{ + type Item = ::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - S::poll_next((*self).reborrow(), cx) + Pin::get_mut(self).as_mut().poll_next(lw) } } @@ -86,11 +91,11 @@ impl Stream for Either { type Item = A::Item; - fn poll_next(self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { unsafe { - match PinMut::get_mut_unchecked(self) { - Either::Left(a) => PinMut::new_unchecked(a).poll_next(cx), - Either::Right(b) => PinMut::new_unchecked(b).poll_next(cx), + match Pin::get_mut_unchecked(self) { + Either::Left(a) => Pin::new_unchecked(a).poll_next(lw), + Either::Right(b) => Pin::new_unchecked(b).poll_next(lw), } } } @@ -110,7 +115,7 @@ pub trait TryStream { /// This method is a stopgap for a compiler limitation that prevents us from /// directly inheriting from the `Stream` trait; in the future it won't be /// needed. - fn try_poll_next(self: PinMut, cx: &mut task::Context) + fn try_poll_next(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll>>; } @@ -120,36 +125,24 @@ impl TryStream for S type Ok = T; type Error = E; - fn try_poll_next(self: PinMut, cx: &mut task::Context) + fn try_poll_next(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll>> { - self.poll_next(cx) + self.poll_next(lw) } } if_std! { use std::boxed::Box; - use std::pin::PinBox; impl Stream for Box { type Item = S::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, - ) -> Poll> { - PinMut::new(&mut **self).poll_next(cx) - } - } - - impl Stream for PinBox { - type Item = S::Item; - - fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.as_pin_mut().poll_next(cx) + Pin::new(&mut **self).poll_next(lw) } } @@ -157,10 +150,10 @@ if_std! { type Item = S::Item; fn poll_next( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - unsafe { PinMut::map_unchecked(self, |x| &mut x.0) }.poll_next(cx) + unsafe { Pin::map_unchecked_mut(self, |x| &mut x.0) }.poll_next(lw) } } @@ -168,8 +161,8 @@ if_std! { type Item = T; fn poll_next( - mut self: PinMut, - _cx: &mut task::Context, + mut self: Pin<&mut Self>, + _lw: &LocalWaker, ) -> Poll> { Poll::Ready(self.pop_front()) } diff --git a/futures-core/src/stream/stream_obj.rs b/futures-core/src/stream/stream_obj.rs index 9063f953d9..cc0f43d45f 100644 --- a/futures-core/src/stream/stream_obj.rs +++ b/futures-core/src/stream/stream_obj.rs @@ -1,8 +1,8 @@ use super::Stream; -use crate::task::{self, Poll}; +use crate::task::{LocalWaker, Poll}; use core::fmt; use core::marker::{PhantomData, Unpin}; -use core::pin::PinMut; +use core::pin::Pin; /// A custom trait object for polling streams, roughly akin to /// `Box + 'a>`. @@ -17,7 +17,7 @@ use core::pin::PinMut; /// information #44874) pub struct LocalStreamObj<'a, T> { ptr: *mut (), - poll_next_fn: unsafe fn(*mut (), &mut task::Context) -> Poll>, + poll_next_fn: unsafe fn(*mut (), &LocalWaker) -> Poll>, drop_fn: unsafe fn(*mut ()), _marker: PhantomData<&'a ()>, } @@ -64,10 +64,10 @@ impl<'a, T> Stream for LocalStreamObj<'a, T> { #[inline] fn poll_next( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - unsafe { (self.poll_next_fn)(self.ptr, cx) } + unsafe { (self.poll_next_fn)(self.ptr, lw) } } } @@ -112,11 +112,11 @@ impl<'a, T> Stream for StreamObj<'a, T> { #[inline] fn poll_next( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - let pinned_field = unsafe { PinMut::map_unchecked(self, |x| &mut x.0) }; - pinned_field.poll_next(cx) + let pinned_field = unsafe { Pin::map_unchecked_mut(self, |x| &mut x.0) }; + pinned_field.poll_next(lw) } } @@ -143,7 +143,7 @@ pub unsafe trait UnsafeStreamObj<'a, T>: 'a { /// `drop`. unsafe fn poll_next( ptr: *mut (), - cx: &mut task::Context, + lw: &LocalWaker, ) -> Poll>; /// Drops the stream represented by the given void pointer. @@ -166,27 +166,27 @@ where unsafe fn poll_next( ptr: *mut (), - cx: &mut task::Context, + lw: &LocalWaker, ) -> Poll> { - PinMut::new_unchecked(&mut *(ptr as *mut F)).poll_next(cx) + Pin::new_unchecked(&mut *(ptr as *mut F)).poll_next(lw) } unsafe fn drop(_ptr: *mut ()) {} } -unsafe impl<'a, T, F> UnsafeStreamObj<'a, T> for PinMut<'a, F> +unsafe impl<'a, T, F> UnsafeStreamObj<'a, T> for Pin<&'a mut F> where F: Stream + 'a, { fn into_raw(self) -> *mut () { - unsafe { PinMut::get_mut_unchecked(self) as *mut F as *mut () } + unsafe { Pin::get_mut_unchecked(self) as *mut F as *mut () } } unsafe fn poll_next( ptr: *mut (), - cx: &mut task::Context, + lw: &LocalWaker, ) -> Poll> { - PinMut::new_unchecked(&mut *(ptr as *mut F)).poll_next(cx) + Pin::new_unchecked(&mut *(ptr as *mut F)).poll_next(lw) } unsafe fn drop(_ptr: *mut ()) {} @@ -194,7 +194,6 @@ where if_std! { use std::boxed::Box; - use std::pin::PinBox; unsafe impl<'a, T, F> UnsafeStreamObj<'a, T> for Box where F: Stream + 'a @@ -203,10 +202,10 @@ if_std! { Box::into_raw(self) as *mut () } - unsafe fn poll_next(ptr: *mut (), cx: &mut task::Context) -> Poll> { + unsafe fn poll_next(ptr: *mut (), lw: &LocalWaker) -> Poll> { let ptr = ptr as *mut F; - let pin: PinMut = PinMut::new_unchecked(&mut *ptr); - pin.poll_next(cx) + let pin: Pin<&mut F> = Pin::new_unchecked(&mut *ptr); + pin.poll_next(lw) } unsafe fn drop(ptr: *mut ()) { @@ -214,26 +213,26 @@ if_std! { } } - unsafe impl<'a, T, F> UnsafeStreamObj<'a, T> for PinBox + unsafe impl<'a, T, F> UnsafeStreamObj<'a, T> for Pin> where F: Stream + 'a { - fn into_raw(self) -> *mut () { - PinBox::into_raw(self) as *mut () + fn into_raw(mut self) -> *mut () { + unsafe { Pin::get_mut_unchecked(Pin::as_mut(&mut self)) as *mut F as *mut () } } - unsafe fn poll_next(ptr: *mut (), cx: &mut task::Context) -> Poll> { + unsafe fn poll_next(ptr: *mut (), lw: &LocalWaker) -> Poll> { let ptr = ptr as *mut F; - let pin: PinMut = PinMut::new_unchecked(&mut *ptr); - pin.poll_next(cx) + let pin: Pin<&mut F> = Pin::new_unchecked(&mut *ptr); + pin.poll_next(lw) } unsafe fn drop(ptr: *mut ()) { - drop(PinBox::from_raw(ptr as *mut F)) + drop(Box::from_raw(ptr as *mut F)) } } - impl<'a, F: Stream + Send + 'a> From> for StreamObj<'a, ()> { - fn from(boxed: PinBox) -> Self { + impl<'a, F: Stream + Send + 'a> From>> for StreamObj<'a, ()> { + fn from(boxed: Pin>) -> Self { StreamObj::new(boxed) } } @@ -244,8 +243,8 @@ if_std! { } } - impl<'a, F: Stream + 'a> From> for LocalStreamObj<'a, ()> { - fn from(boxed: PinBox) -> Self { + impl<'a, F: Stream + 'a> From>> for LocalStreamObj<'a, ()> { + fn from(boxed: Pin>) -> Self { LocalStreamObj::new(boxed) } } diff --git a/futures-core/src/task.rs b/futures-core/src/task.rs deleted file mode 100644 index f0fbb614a1..0000000000 --- a/futures-core/src/task.rs +++ /dev/null @@ -1,11 +0,0 @@ -//! Task notification. - -pub use core::task::{ - Context, Poll, Spawn, - Waker, LocalWaker, UnsafeWake, - SpawnErrorKind, SpawnObjError, SpawnLocalObjError, -}; - -if_std! { - pub use std::task::{Wake, local_waker, local_waker_from_nonlocal}; -} diff --git a/futures-core/src/task/mod.rs b/futures-core/src/task/mod.rs new file mode 100644 index 0000000000..b01ea1bc93 --- /dev/null +++ b/futures-core/src/task/mod.rs @@ -0,0 +1,9 @@ +//! Task notification. + +mod spawn; +pub use self::spawn::{Spawn, LocalSpawn, SpawnError}; + +pub use core::task::{Poll, Waker, LocalWaker, UnsafeWake}; +if_std! { + pub use std::task::{Wake, local_waker, local_waker_from_nonlocal}; +} diff --git a/futures-core/src/task/spawn.rs b/futures-core/src/task/spawn.rs new file mode 100644 index 0000000000..47f4e195ba --- /dev/null +++ b/futures-core/src/task/spawn.rs @@ -0,0 +1,79 @@ +use crate::future::{FutureObj, LocalFutureObj}; +use core::fmt; + +/// The `Spawn` trait allows for pushing futures onto an executor that will +/// run them to completion. +pub trait Spawn { + /// Spawns a future that will be run to completion. + /// + /// # Errors + /// + /// The executor may be unable to spawn tasks. Spawn errors should + /// represent relatively rare scenarios, such as the executor + /// having been shut down so that it is no longer able to accept + /// tasks. + fn spawn_obj(&mut self, future: FutureObj<'static, ()>) + -> Result<(), SpawnError>; + + /// Determines whether the executor is able to spawn new tasks. + /// + /// This method will return `Ok` when the executor is *likely* + /// (but not guaranteed) to accept a subsequent spawn attempt. + /// Likewise, an `Err` return means that `spawn` is likely, but + /// not guaranteed, to yield an error. + #[inline] + fn status(&self) -> Result<(), SpawnError> { + Ok(()) + } +} + +/// The `LocalSpawn` is similar to `[Spawn]`, but allows spawning futures +/// that don't implement `Send`. +pub trait LocalSpawn { + /// Spawns a future that will be run to completion. + /// + /// # Errors + /// + /// The executor may be unable to spawn tasks. Spawn errors should + /// represent relatively rare scenarios, such as the executor + /// having been shut down so that it is no longer able to accept + /// tasks. + fn spawn_local_obj(&mut self, future: LocalFutureObj<'static, ()>) + -> Result<(), SpawnError>; + + /// Determines whether the executor is able to spawn new tasks. + /// + /// This method will return `Ok` when the executor is *likely* + /// (but not guaranteed) to accept a subsequent spawn attempt. + /// Likewise, an `Err` return means that `spawn` is likely, but + /// not guaranteed, to yield an error. + #[inline] + fn status_local(&self) -> Result<(), SpawnError> { + Ok(()) + } +} + +/// An error that occurred during spawning. +pub struct SpawnError { + _hidden: (), +} + +impl fmt::Debug for SpawnError { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_tuple("SpanError") + .field(&"shutdown") + .finish() + } +} + +impl SpawnError { + /// Spawning failed because the executor has been shut down. + pub fn shutdown() -> Self { + Self { _hidden: () } + } + + /// Check whether spawning failed to the executor being shut down. + pub fn is_shutdown(&self) -> bool { + true + } +} diff --git a/futures-executor/benches/poll.rs b/futures-executor/benches/poll.rs deleted file mode 100644 index e46ff524df..0000000000 --- a/futures-executor/benches/poll.rs +++ /dev/null @@ -1,68 +0,0 @@ -#![feature(test, pin, arbitrary_self_types, futures_api)] - -use futures::executor::LocalPool; -use futures::future::{Future, FutureExt}; -use futures::task::{self, Poll, Waker, LocalWaker, Wake}; -use std::marker::Unpin; -use std::pin::PinMut; -use std::sync::Arc; -use test::Bencher; - -fn notify_noop() -> LocalWaker { - struct Noop; - - impl Wake for Noop { - fn wake(_: &Arc) {} - } - - task::local_waker_from_nonlocal(Arc::new(Noop)) -} - -#[bench] -fn task_init(b: &mut Bencher) { - const NUM: u32 = 100_000; - - struct MyFuture { - num: u32, - task: Option, - }; - impl Unpin for MyFuture {} - - impl Future for MyFuture { - type Output = (); - - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - if self.num == NUM { - Poll::Ready(()) - } else { - self.num += 1; - - if let Some(ref t) = self.task { - t.wake(); - return Poll::Pending; - } - - let t = cx.waker().clone(); - t.wake(); - self.task = Some(t); - - Poll::Pending - } - } - } - - let mut fut = MyFuture { - num: 0, - task: None, - }; - - let pool = LocalPool::new(); - let mut spawn = pool.spawner(); - let waker = notify_noop(); - let mut cx = task::Context::new(&waker, &mut spawn); - - b.iter(|| { - fut.num = 0; - while let Poll::Pending = fut.poll_unpin(&mut cx) {} - }); -} diff --git a/futures-executor/benches/thread_notify.rs b/futures-executor/benches/thread_notify.rs index a4e09a4b73..da2579a075 100644 --- a/futures-executor/benches/thread_notify.rs +++ b/futures-executor/benches/thread_notify.rs @@ -1,11 +1,13 @@ #![feature(test, futures_api, pin, arbitrary_self_types)] +extern crate test; +use crate::test::Bencher; + use futures::executor::block_on; use futures::future::Future; -use futures::task::{self, Poll, Waker}; +use futures::task::{Poll, LocalWaker, Waker}; use std::marker::Unpin; -use std::pin::PinMut; -use test::Bencher; +use std::pin::Pin; #[bench] fn thread_yield_single_thread_one_wait(b: &mut Bencher) { @@ -18,12 +20,12 @@ fn thread_yield_single_thread_one_wait(b: &mut Bencher) { impl Future for Yield { type Output = (); - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { if self.rem == 0 { Poll::Ready(()) } else { self.rem -= 1; - cx.waker().wake(); + lw.wake(); Poll::Pending } } @@ -46,12 +48,12 @@ fn thread_yield_single_thread_many_wait(b: &mut Bencher) { impl Future for Yield { type Output = (); - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { if self.rem == 0 { Poll::Ready(()) } else { self.rem -= 1; - cx.waker().wake(); + lw.wake(); Poll::Pending } } @@ -83,12 +85,12 @@ fn thread_yield_multi_thread(b: &mut Bencher) { impl Future for Yield { type Output = (); - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { if self.rem == 0 { Poll::Ready(()) } else { self.rem -= 1; - self.tx.send(cx.waker().clone()).unwrap(); + self.tx.send(lw.clone().into_waker()).unwrap(); Poll::Pending } } diff --git a/futures-executor/src/lib.rs b/futures-executor/src/lib.rs index 63f4535d86..fe7e0a3e82 100644 --- a/futures-executor/src/lib.rs +++ b/futures-executor/src/lib.rs @@ -18,7 +18,7 @@ macro_rules! if_std { if_std! { mod local_pool; - pub use crate::local_pool::{block_on, block_on_stream, BlockingStream, LocalPool, LocalSpawn}; + pub use crate::local_pool::{block_on, block_on_stream, BlockingStream, LocalPool, LocalSpawner}; mod unpark_mutex; mod thread_pool; diff --git a/futures-executor/src/local_pool.rs b/futures-executor/src/local_pool.rs index b3e95ccddf..b58d30bb6f 100644 --- a/futures-executor/src/local_pool.rs +++ b/futures-executor/src/local_pool.rs @@ -2,8 +2,8 @@ use crate::{enter, ThreadPool}; use futures_core::future::{Future, FutureObj, LocalFutureObj}; use futures_core::stream::{Stream}; use futures_core::task::{ - self, Context, Poll, LocalWaker, Wake, - Spawn, SpawnObjError, SpawnLocalObjError, SpawnErrorKind + self, Poll, LocalWaker, Wake, + Spawn, LocalSpawn, SpawnError, }; use futures_util::stream::FuturesUnordered; use futures_util::stream::StreamExt; @@ -27,7 +27,7 @@ use std::thread::{self, Thread}; /// [`Spawn`](futures_core::task::Spawn), use the /// [`spawner()`](LocalPool::spawner) method. Because the executor is /// single-threaded, it supports a special form of task spawning for non-`Send` -/// futures, via [`spawn_local_obj`](LocalSpawn::spawn_local_obj). +/// futures, via [`spawn_local_obj`](LocalSpawner::spawn_local_obj). #[derive(Debug)] pub struct LocalPool { pool: FuturesUnordered>, @@ -37,7 +37,7 @@ pub struct LocalPool { /// A handle to a [`LocalPool`](LocalPool) that implements /// [`Spawn`](futures_core::task::Spawn). #[derive(Clone, Debug)] -pub struct LocalSpawn { +pub struct LocalSpawner { incoming: Weak, } @@ -88,8 +88,8 @@ impl LocalPool { } /// Get a clonable handle to the pool as a [`Spawn`]. - pub fn spawner(&self) -> LocalSpawn { - LocalSpawn { + pub fn spawner(&self) -> LocalSpawner { + LocalSpawner { incoming: Rc::downgrade(&self.incoming) } } @@ -104,18 +104,17 @@ impl LocalPool { /// use futures::executor::LocalPool; /// /// let mut pool = LocalPool::new(); - /// let mut spawn = pool.spawner(); /// /// // ... spawn some initial tasks using `spawn.spawn()` or `spawn.spawn_local()` /// /// // run *all* tasks in the pool to completion, including any newly-spawned ones. - /// pool.run(&mut spawn); + /// pool.run(); /// ``` /// /// The function will block the calling thread until *all* tasks in the pool /// are complete, including any spawned while running existing tasks. - pub fn run(&mut self, spawn: &mut Sp) where Sp: Spawn + Sized { - run_executor(|local_waker| self.poll_pool(local_waker, spawn)) + pub fn run(&mut self) { + run_executor(|local_waker| self.poll_pool(local_waker)) } /// Runs all the tasks in the pool until the given future completes. @@ -130,12 +129,11 @@ impl LocalPool { /// use futures::future::ready; /// /// let mut pool = LocalPool::new(); - /// let mut spawn = pool.spawner(); /// # let my_app = ready(()); /// /// // run tasks in the pool until `my_app` completes, by default spawning /// // further tasks back onto the pool - /// pool.run_until(my_app, &mut spawn); + /// pool.run_until(my_app); /// ``` /// /// The function will block the calling thread *only* until the future `f` @@ -143,37 +141,27 @@ impl LocalPool { /// be inert after the call completes, but can continue with further use of /// `run` or `run_until`. While the function is running, however, all tasks /// in the pool will try to make progress. - pub fn run_until(&mut self, future: F, spawn: &mut Sp) - -> F::Output - where F: Future, Sp: Spawn + Sized - { + pub fn run_until(&mut self, future: F) -> F::Output { pin_mut!(future); run_executor(|local_waker| { { - let mut main_cx = Context::new(local_waker, spawn); - // if our main task is done, so are we - let result = future.reborrow().poll(&mut main_cx); + let result = future.as_mut().poll(local_waker); if let Poll::Ready(output) = result { return Poll::Ready(output); } } - self.poll_pool(local_waker, spawn); + self.poll_pool(local_waker); Poll::Pending }) } // Make maximal progress on the entire pool of spawned task, returning `Ready` // if the pool is empty and `Pending` if no further progress can be made. - fn poll_pool(&mut self, local_waker: &LocalWaker, spawn: &mut Sp) - -> Poll<()> - where Sp: Spawn + Sized - { + fn poll_pool(&mut self, local_waker: &LocalWaker) -> Poll<()> { // state for the FuturesUnordered, which will never be used - let mut pool_cx = Context::new(local_waker, spawn); - loop { // empty the incoming queue of newly-spawned tasks { @@ -183,7 +171,7 @@ impl LocalPool { } } - let ret = self.pool.poll_next_unpin(&mut pool_cx); + let ret = self.pool.poll_next_unpin(local_waker); // we queued up some new tasks; add them and poll again if !self.incoming.borrow().is_empty() { continue; @@ -221,7 +209,7 @@ lazy_static! { /// spawned tasks. pub fn block_on(f: F) -> F::Output { let mut pool = LocalPool::new(); - pool.run_until(f, &mut GLOBAL_POOL.clone()) + pool.run_until(f) } /// Turn a stream into a blocking iterator. @@ -260,43 +248,50 @@ impl BlockingStream { impl Iterator for BlockingStream { type Item = S::Item; fn next(&mut self) -> Option { - LocalPool::new().run_until(self.stream.next(), &mut GLOBAL_POOL.clone()) + LocalPool::new().run_until(self.stream.next()) } } -impl Spawn for LocalSpawn { +impl Spawn for LocalSpawner { fn spawn_obj( &mut self, future: FutureObj<'static, ()>, - ) -> Result<(), SpawnObjError> { + ) -> Result<(), SpawnError> { if let Some(incoming) = self.incoming.upgrade() { incoming.borrow_mut().push(future.into()); Ok(()) } else { - Err(SpawnObjError{ future, kind: SpawnErrorKind::shutdown() }) + Err(SpawnError::shutdown()) } } - fn status(&self) -> Result<(), SpawnErrorKind> { + fn status(&self) -> Result<(), SpawnError> { if self.incoming.upgrade().is_some() { Ok(()) } else { - Err(SpawnErrorKind::shutdown()) + Err(SpawnError::shutdown()) } } } -impl LocalSpawn { - /// Spawn a non-`Send` future onto the associated [`LocalPool`](LocalPool). - pub fn spawn_local_obj( +impl LocalSpawn for LocalSpawner { + fn spawn_local_obj( &mut self, future: LocalFutureObj<'static, ()>, - ) -> Result<(), SpawnLocalObjError> { + ) -> Result<(), SpawnError> { if let Some(incoming) = self.incoming.upgrade() { incoming.borrow_mut().push(future); Ok(()) } else { - Err(SpawnLocalObjError{ future, kind: SpawnErrorKind::shutdown() }) + Err(SpawnError::shutdown()) + } + } + + fn status_local(&self) -> Result<(), SpawnError> { + if self.incoming.upgrade().is_some() { + Ok(()) + } else { + Err(SpawnError::shutdown()) } } } diff --git a/futures-executor/src/thread_pool.rs b/futures-executor/src/thread_pool.rs index 19f247c64c..125c26e773 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::{self, Poll, Wake, Spawn, SpawnObjError}; +use futures_core::task::{Poll, Wake, Spawn, SpawnError}; use futures_util::future::FutureExt; use futures_util::task::local_waker_ref_from_nonlocal; use num_cpus; @@ -95,7 +95,7 @@ impl ThreadPool { /// Note that the function will return when the provided future completes, /// even if some of the tasks it spawned are still running. pub fn run(&mut self, f: F) -> F::Output { - crate::LocalPool::new().run_until(f, self) + crate::LocalPool::new().run_until(f) } } @@ -103,7 +103,7 @@ impl Spawn for ThreadPool { fn spawn_obj( &mut self, future: FutureObj<'static, ()>, - ) -> Result<(), SpawnObjError> { + ) -> Result<(), SpawnError> { let task = Task { future, wake_handle: Arc::new(WakeHandle { @@ -298,10 +298,7 @@ impl Task { wake_handle.mutex.start_poll(); loop { - let res = { - let mut cx = task::Context::new(&local_waker, &mut exec); - future.poll_unpin(&mut cx) - }; + let res = future.poll_unpin(&local_waker); match res { Poll::Pending => {} Poll::Ready(()) => return wake_handle.mutex.complete(), diff --git a/futures-executor/tests/local_pool.rs b/futures-executor/tests/local_pool.rs index a4fcee6921..f06c782fb1 100644 --- a/futures-executor/tests/local_pool.rs +++ b/futures-executor/tests/local_pool.rs @@ -3,9 +3,9 @@ use futures::channel::oneshot; use futures::executor::LocalPool; use futures::future::{Future, lazy}; -use futures::task::{self, Poll, Spawn}; +use futures::task::{LocalWaker, Poll, Spawn, LocalSpawn}; use std::cell::{Cell, RefCell}; -use std::pin::{PinBox, PinMut}; +use std::pin::Pin; use std::rc::Rc; struct Pending(Rc<()>); @@ -13,7 +13,7 @@ struct Pending(Rc<()>); impl Future for Pending { type Output = (); - fn poll(self: PinMut, _cx: &mut task::Context) -> Poll<()> { + fn poll(self: Pin<&mut Self>, _lw: &LocalWaker) -> Poll<()> { Poll::Pending } } @@ -28,12 +28,11 @@ fn run_until_single_future() { { let mut pool = LocalPool::new(); - let mut spawn = pool.spawner(); let fut = lazy(|_| { cnt += 1; () }); - assert_eq!(pool.run_until(fut, &mut spawn), ()); + assert_eq!(pool.run_until(fut), ()); } assert_eq!(cnt, 1); @@ -43,8 +42,8 @@ fn run_until_single_future() { fn run_until_ignores_spawned() { let mut pool = LocalPool::new(); let mut spawn = pool.spawner(); - spawn.spawn_local_obj(PinBox::new(pending()).into()).unwrap(); - assert_eq!(pool.run_until(lazy(|_| ()), &mut spawn), ()); + spawn.spawn_local_obj(Box::pinned(pending()).into()).unwrap(); + assert_eq!(pool.run_until(lazy(|_| ())), ()); } #[test] @@ -52,11 +51,11 @@ fn run_until_executes_spawned() { let (tx, rx) = oneshot::channel(); let mut pool = LocalPool::new(); let mut spawn = pool.spawner(); - spawn.spawn_local_obj(PinBox::new(lazy(move |_| { + spawn.spawn_local_obj(Box::pinned(lazy(move |_| { tx.send(()).unwrap(); () })).into()).unwrap(); - pool.run_until(rx, &mut spawn).unwrap(); + pool.run_until(rx).unwrap(); } #[test] @@ -68,15 +67,15 @@ fn run_executes_spawned() { let mut spawn = pool.spawner(); let mut spawn2 = pool.spawner(); - spawn.spawn_local_obj(PinBox::new(lazy(move |_| { - spawn2.spawn_local_obj(PinBox::new(lazy(move |_| { + spawn.spawn_local_obj(Box::pinned(lazy(move |_| { + spawn2.spawn_local_obj(Box::pinned(lazy(move |_| { cnt2.set(cnt2.get() + 1); () })).into()).unwrap(); () })).into()).unwrap(); - pool.run(&mut spawn); + pool.run(); assert_eq!(cnt.get(), 1); } @@ -93,13 +92,13 @@ fn run_spawn_many() { for _ in 0..ITER { let cnt = cnt.clone(); - spawn.spawn_local_obj(PinBox::new(lazy(move |_| { + spawn.spawn_local_obj(Box::pinned(lazy(move |_| { cnt.set(cnt.get() + 1); () })).into()).unwrap(); } - pool.run(&mut spawn); + pool.run(); assert_eq!(cnt.get(), ITER); } @@ -110,12 +109,11 @@ fn nesting_run() { let mut pool = LocalPool::new(); let mut spawn = pool.spawner(); - spawn.spawn_obj(PinBox::new(lazy(|_| { + spawn.spawn_obj(Box::pinned(lazy(|_| { let mut pool = LocalPool::new(); - let mut spawn = pool.spawner(); - pool.run(&mut spawn); + pool.run(); })).into()).unwrap(); - pool.run(&mut spawn); + pool.run(); } #[test] @@ -130,7 +128,7 @@ fn tasks_are_scheduled_fairly() { impl Future for Spin { type Output = (); - fn poll(self: PinMut, cx: &mut task::Context) -> Poll<()> { + fn poll(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll<()> { let mut state = self.state.borrow_mut(); if self.idx == 0 { @@ -149,7 +147,7 @@ fn tasks_are_scheduled_fairly() { return Poll::Ready(()); } - cx.waker().wake(); + lw.wake(); Poll::Pending } } @@ -157,16 +155,16 @@ fn tasks_are_scheduled_fairly() { let mut pool = LocalPool::new(); let mut spawn = pool.spawner(); - spawn.spawn_local_obj(PinBox::new(Spin { + spawn.spawn_local_obj(Box::pinned(Spin { state: state.clone(), idx: 0, }).into()).unwrap(); - spawn.spawn_local_obj(PinBox::new(Spin { + spawn.spawn_local_obj(Box::pinned(Spin { state: state, idx: 1, }).into()).unwrap(); - pool.run(&mut spawn); + pool.run(); } diff --git a/futures-io/src/lib.rs b/futures-io/src/lib.rs index 7837bd2aee..1cc4c4e10d 100644 --- a/futures-io/src/lib.rs +++ b/futures-io/src/lib.rs @@ -21,7 +21,7 @@ macro_rules! if_std { } if_std! { - use futures_core::task::{self, Poll}; + use futures_core::task::{LocalWaker, Poll}; use std::boxed::Box; use std::cmp; use std::io as StdIo; @@ -106,7 +106,7 @@ if_std! { /// /// If no data is available for reading, the method returns /// `Ok(Async::Pending)` and arranges for the current task (via - /// `cx.waker()`) to receive a notification when the object becomes + /// `lw.waker()`) to receive a notification when the object becomes /// readable or is closed. /// /// # Implementation @@ -115,7 +115,7 @@ if_std! { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Async::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_read(&mut self, cx: &mut task::Context, buf: &mut [u8]) + fn poll_read(&mut self, lw: &LocalWaker, buf: &mut [u8]) -> Poll>; /// Attempt to read from the `AsyncRead` into `vec` using vectored @@ -128,7 +128,7 @@ if_std! { /// /// If no data is available for reading, the method returns /// `Ok(Async::Pending)` and arranges for the current task (via - /// `cx.waker()`) to receive a notification when the object becomes + /// `lw.waker()`) to receive a notification when the object becomes /// readable or is closed. /// By default, this method delegates to using `poll_read` on the first /// buffer in `vec`. Objects which support vectored IO should override @@ -140,11 +140,11 @@ if_std! { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Async::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_vectored_read(&mut self, cx: &mut task::Context, vec: &mut [&mut IoVec]) + fn poll_vectored_read(&mut self, lw: &LocalWaker, vec: &mut [&mut IoVec]) -> Poll> { if let Some(ref mut first_iovec) = vec.get_mut(0) { - self.poll_read(cx, first_iovec) + self.poll_read(lw, first_iovec) } else { // `vec` is empty. Poll::Ready(Ok(0)) @@ -166,7 +166,7 @@ if_std! { /// /// If the object is not ready for writing, the method returns /// `Ok(Async::Pending)` and arranges for the current task (via - /// `cx.waker()`) to receive a notification when the object becomes + /// `lw.waker()`) to receive a notification when the object becomes /// readable or is closed. /// /// # Implementation @@ -175,7 +175,7 @@ if_std! { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Async::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_write(&mut self, cx: &mut task::Context, buf: &[u8]) + fn poll_write(&mut self, lw: &LocalWaker, buf: &[u8]) -> Poll>; /// Attempt to write bytes from `vec` into the object using vectored @@ -188,7 +188,7 @@ if_std! { /// /// If the object is not ready for writing, the method returns /// `Ok(Async::Pending)` and arranges for the current task (via - /// `cx.waker()`) to receive a notification when the object becomes + /// `lw.waker()`) to receive a notification when the object becomes /// readable or is closed. /// /// By default, this method delegates to using `poll_write` on the first @@ -201,11 +201,11 @@ if_std! { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Async::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_vectored_write(&mut self, cx: &mut task::Context, vec: &[&IoVec]) + fn poll_vectored_write(&mut self, lw: &LocalWaker, vec: &[&IoVec]) -> Poll> { if let Some(ref first_iovec) = vec.get(0) { - self.poll_write(cx, &*first_iovec) + self.poll_write(lw, &*first_iovec) } else { // `vec` is empty. Poll::Ready(Ok(0)) @@ -219,7 +219,7 @@ if_std! { /// /// If flushing cannot immediately complete, this method returns /// `Ok(Async::Pending)` and arranges for the current task (via - /// `cx.waker()`) to receive a notification when the object can make + /// `lw.waker()`) to receive a notification when the object can make /// progress towards flushing. /// /// # Implementation @@ -228,7 +228,7 @@ if_std! { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Async::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_flush(&mut self, cx: &mut task::Context) -> Poll>; + fn poll_flush(&mut self, lw: &LocalWaker) -> Poll>; /// Attempt to close the object. /// @@ -236,7 +236,7 @@ if_std! { /// /// If closing cannot immediately complete, this function returns /// `Ok(Async::Pending)` and arranges for the current task (via - /// `cx.waker()`) to receive a notification when the object can make + /// `lw.waker()`) to receive a notification when the object can make /// progress towards closing. /// /// # Implementation @@ -245,7 +245,7 @@ if_std! { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Async::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_close(&mut self, cx: &mut task::Context) -> Poll>; + fn poll_close(&mut self, lw: &LocalWaker) -> Poll>; } macro_rules! deref_async_read { @@ -254,16 +254,16 @@ if_std! { (**self).initializer() } - fn poll_read(&mut self, cx: &mut task::Context, buf: &mut [u8]) + fn poll_read(&mut self, lw: &LocalWaker, buf: &mut [u8]) -> Poll> { - (**self).poll_read(cx, buf) + (**self).poll_read(lw, buf) } - fn poll_vectored_read(&mut self, cx: &mut task::Context, vec: &mut [&mut IoVec]) + fn poll_vectored_read(&mut self, lw: &LocalWaker, vec: &mut [&mut IoVec]) -> Poll> { - (**self).poll_vectored_read(cx, vec) + (**self).poll_vectored_read(lw, vec) } } } @@ -284,7 +284,7 @@ if_std! { Initializer::nop() } - fn poll_read(&mut self, _: &mut task::Context, buf: &mut [u8]) + fn poll_read(&mut self, _: &LocalWaker, buf: &mut [u8]) -> Poll> { Poll::Ready(StdIo::Read::read(self, buf)) @@ -306,24 +306,24 @@ if_std! { macro_rules! deref_async_write { () => { - fn poll_write(&mut self, cx: &mut task::Context, buf: &[u8]) + fn poll_write(&mut self, lw: &LocalWaker, buf: &[u8]) -> Poll> { - (**self).poll_write(cx, buf) + (**self).poll_write(lw, buf) } - fn poll_vectored_write(&mut self, cx: &mut task::Context, vec: &[&IoVec]) + fn poll_vectored_write(&mut self, lw: &LocalWaker, vec: &[&IoVec]) -> Poll> { - (**self).poll_vectored_write(cx, vec) + (**self).poll_vectored_write(lw, vec) } - fn poll_flush(&mut self, cx: &mut task::Context) -> Poll> { - (**self).poll_flush(cx) + fn poll_flush(&mut self, lw: &LocalWaker) -> Poll> { + (**self).poll_flush(lw) } - fn poll_close(&mut self, cx: &mut task::Context) -> Poll> { - (**self).poll_close(cx) + fn poll_close(&mut self, lw: &LocalWaker) -> Poll> { + (**self).poll_close(lw) } } } @@ -338,18 +338,18 @@ if_std! { macro_rules! delegate_async_write_to_stdio { () => { - fn poll_write(&mut self, _: &mut task::Context, buf: &[u8]) + fn poll_write(&mut self, _: &LocalWaker, buf: &[u8]) -> Poll> { Poll::Ready(StdIo::Write::write(self, buf)) } - fn poll_flush(&mut self, _: &mut task::Context) -> Poll> { + fn poll_flush(&mut self, _: &LocalWaker) -> Poll> { Poll::Ready(StdIo::Write::flush(self)) } - fn poll_close(&mut self, cx: &mut task::Context) -> Poll> { - self.poll_flush(cx) + fn poll_close(&mut self, lw: &LocalWaker) -> Poll> { + self.poll_flush(lw) } } } @@ -357,7 +357,7 @@ if_std! { impl> AsyncWrite for StdIo::Cursor { fn poll_write( &mut self, - _: &mut task::Context, + _: &LocalWaker, buf: &[u8], ) -> Poll> { let position = self.position(); @@ -372,12 +372,12 @@ if_std! { Poll::Ready(result) } - fn poll_flush(&mut self, _: &mut task::Context) -> Poll> { + fn poll_flush(&mut self, _: &LocalWaker) -> Poll> { Poll::Ready(StdIo::Write::flush(&mut self.get_mut().as_mut())) } - fn poll_close(&mut self, cx: &mut task::Context) -> Poll> { - self.poll_flush(cx) + fn poll_close(&mut self, lw: &LocalWaker) -> Poll> { + self.poll_flush(lw) } } diff --git a/futures-sink/src/channel_impls.rs b/futures-sink/src/channel_impls.rs index d0138f335e..edd3f6fc82 100644 --- a/futures-sink/src/channel_impls.rs +++ b/futures-sink/src/channel_impls.rs @@ -1,25 +1,25 @@ use crate::{Sink, Poll}; -use futures_core::task; +use futures_core::task::LocalWaker; use futures_channel::mpsc::{Sender, SendError, UnboundedSender}; -use std::pin::PinMut; +use std::pin::Pin; impl Sink for Sender { type SinkItem = T; type SinkError = SendError; - fn poll_ready(mut self: PinMut, cx: &mut task::Context) -> Poll> { - (*self).poll_ready(cx) + fn poll_ready(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + (*self).poll_ready(lw) } - fn start_send(mut self: PinMut, msg: T) -> Result<(), Self::SinkError> { + fn start_send(mut self: Pin<&mut Self>, msg: T) -> Result<(), Self::SinkError> { (*self).start_send(msg) } - fn poll_flush(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(mut self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { self.close_channel(); Poll::Ready(Ok(())) } @@ -29,19 +29,19 @@ impl Sink for UnboundedSender { type SinkItem = T; type SinkError = SendError; - fn poll_ready(self: PinMut, cx: &mut task::Context) -> Poll> { - UnboundedSender::poll_ready(&*self, cx) + fn poll_ready(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + UnboundedSender::poll_ready(&*self, lw) } - fn start_send(mut self: PinMut, msg: T) -> Result<(), Self::SinkError> { + fn start_send(mut self: Pin<&mut Self>, msg: T) -> Result<(), Self::SinkError> { UnboundedSender::start_send(&mut *self, msg) } - fn poll_flush(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_close(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { self.close_channel(); Poll::Ready(Ok(())) } @@ -51,20 +51,20 @@ impl<'a, T> Sink for &'a UnboundedSender { type SinkItem = T; type SinkError = SendError; - fn poll_ready(self: PinMut, cx: &mut task::Context) -> Poll> { - UnboundedSender::poll_ready(*self, cx) + fn poll_ready(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + UnboundedSender::poll_ready(*self, lw) } - fn start_send(self: PinMut, msg: T) -> Result<(), Self::SinkError> { + fn start_send(self: Pin<&mut Self>, msg: T) -> Result<(), Self::SinkError> { self.unbounded_send(msg) .map_err(|err| err.into_send_error()) } - fn poll_flush(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_close(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { self.close_channel(); Poll::Ready(Ok(())) } diff --git a/futures-sink/src/lib.rs b/futures-sink/src/lib.rs index 6141d36a00..0c69e9ff05 100644 --- a/futures-sink/src/lib.rs +++ b/futures-sink/src/lib.rs @@ -16,9 +16,9 @@ macro_rules! if_std { )*) } -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; /// A `Sink` is a value into which other values can be sent, asynchronously. /// @@ -60,12 +60,12 @@ pub trait Sink { /// /// This method returns `Poll::Ready` once the underlying sink is ready to /// receive data. If this method returns `Async::Pending`, the current task - /// is registered to be notified (via `cx.waker()`) when `poll_ready` + /// is registered to be notified (via `lw.waker()`) when `poll_ready` /// should be called again. /// /// In most cases, if the sink encounters an error, the sink will /// permanently be unable to receive items. - fn poll_ready(self: PinMut, cx: &mut task::Context) -> Poll>; + fn poll_ready(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll>; /// Begin the process of sending a value to the sink. /// Each call to this function must be proceeded by a successful call to @@ -86,7 +86,7 @@ pub trait Sink { /// /// In most cases, if the sink encounters an error, the sink will /// permanently be unable to receive items. - fn start_send(self: PinMut, item: Self::SinkItem) + fn start_send(self: Pin<&mut Self>, item: Self::SinkItem) -> Result<(), Self::SinkError>; /// Flush any remaining output from this sink. @@ -96,12 +96,12 @@ pub trait Sink { /// via `start_send` have been flushed. /// /// Returns `Ok(Async::Pending)` if there is more work left to do, in which - /// case the current task is scheduled (via `cx.waker()`) to wake up when + /// case the current task is scheduled (via `lw.waker()`) to wake up when /// `poll_flush` should be called again. /// /// In most cases, if the sink encounters an error, the sink will /// permanently be unable to receive items. - fn poll_flush(self: PinMut, cx: &mut task::Context) -> Poll>; + fn poll_flush(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll>; /// Flush any remaining output and close this sink, if necessary. /// @@ -109,53 +109,53 @@ pub trait Sink { /// has been successfully closed. /// /// Returns `Ok(Async::Pending)` if there is more work left to do, in which - /// case the current task is scheduled (via `cx.waker()`) to wake up when + /// case the current task is scheduled (via `lw.waker()`) to wake up when /// `poll_close` should be called again. /// /// If this function encounters an error, the sink should be considered to /// have failed permanently, and no more `Sink` methods should be called. - fn poll_close(self: PinMut, cx: &mut task::Context) -> Poll>; + fn poll_close(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll>; } impl<'a, S: ?Sized + Sink + Unpin> Sink for &'a mut S { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn poll_ready(mut self: PinMut, cx: &mut task::Context) -> Poll> { - PinMut::new(&mut **self).poll_ready(cx) + fn poll_ready(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + Pin::new(&mut **self).poll_ready(lw) } - fn start_send(mut self: PinMut, item: Self::SinkItem) -> Result<(), Self::SinkError> { - PinMut::new(&mut **self).start_send(item) + fn start_send(mut self: Pin<&mut Self>, item: Self::SinkItem) -> Result<(), Self::SinkError> { + Pin::new(&mut **self).start_send(item) } - fn poll_flush(mut self: PinMut, cx: &mut task::Context) -> Poll> { - PinMut::new(&mut **self).poll_flush(cx) + fn poll_flush(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + Pin::new(&mut **self).poll_flush(lw) } - fn poll_close(mut self: PinMut, cx: &mut task::Context) -> Poll> { - PinMut::new(&mut **self).poll_close(cx) + fn poll_close(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + Pin::new(&mut **self).poll_close(lw) } } -impl<'a, S: ?Sized + Sink> Sink for PinMut<'a, S> { +impl<'a, S: ?Sized + Sink> Sink for Pin<&'a mut S> { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn poll_ready(mut self: PinMut, cx: &mut task::Context) -> Poll> { - S::poll_ready((*self).reborrow(), cx) + fn poll_ready(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + S::poll_ready((*self).as_mut(), lw) } - fn start_send(mut self: PinMut, item: Self::SinkItem) -> Result<(), Self::SinkError> { - S::start_send((*self).reborrow(), item) + fn start_send(mut self: Pin<&mut Self>, item: Self::SinkItem) -> Result<(), Self::SinkError> { + S::start_send((*self).as_mut(), item) } - fn poll_flush(mut self: PinMut, cx: &mut task::Context) -> Poll> { - S::poll_flush((*self).reborrow(), cx) + fn poll_flush(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + S::poll_flush((*self).as_mut(), lw) } - fn poll_close(mut self: PinMut, cx: &mut task::Context) -> Poll> { - S::poll_close((*self).reborrow(), cx) + fn poll_close(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + S::poll_close((*self).as_mut(), lw) } } @@ -171,21 +171,21 @@ if_std! { type SinkItem = T; type SinkError = VecSinkError; - fn poll_ready(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } - fn start_send(self: PinMut, item: Self::SinkItem) -> Result<(), Self::SinkError> { + fn start_send(self: Pin<&mut Self>, item: Self::SinkItem) -> Result<(), Self::SinkError> { // TODO: impl Unpin for Vec {} - unsafe { PinMut::get_mut_unchecked(self) }.push(item); + unsafe { Pin::get_mut_unchecked(self) }.push(item); Ok(()) } - fn poll_flush(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_close(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } } @@ -194,21 +194,21 @@ if_std! { type SinkItem = T; type SinkError = VecSinkError; - fn poll_ready(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } - fn start_send(self: PinMut, item: Self::SinkItem) -> Result<(), Self::SinkError> { + fn start_send(self: Pin<&mut Self>, item: Self::SinkItem) -> Result<(), Self::SinkError> { // TODO: impl Unpin for Vec {} - unsafe { PinMut::get_mut_unchecked(self) }.push_back(item); + unsafe { Pin::get_mut_unchecked(self) }.push_back(item); Ok(()) } - fn poll_flush(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_close(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Ok(())) } } @@ -217,20 +217,20 @@ if_std! { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn poll_ready(mut self: PinMut, cx: &mut task::Context) -> Poll> { - PinMut::new(&mut **self).poll_ready(cx) + fn poll_ready(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + Pin::new(&mut **self).poll_ready(lw) } - fn start_send(mut self: PinMut, item: Self::SinkItem) -> Result<(), Self::SinkError> { - PinMut::new(&mut **self).start_send(item) + fn start_send(mut self: Pin<&mut Self>, item: Self::SinkItem) -> Result<(), Self::SinkError> { + Pin::new(&mut **self).start_send(item) } - fn poll_flush(mut self: PinMut, cx: &mut task::Context) -> Poll> { - PinMut::new(&mut **self).poll_flush(cx) + fn poll_flush(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + Pin::new(&mut **self).poll_flush(lw) } - fn poll_close(mut self: PinMut, cx: &mut task::Context) -> Poll> { - PinMut::new(&mut **self).poll_close(cx) + fn poll_close(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + Pin::new(&mut **self).poll_close(lw) } } } @@ -246,38 +246,38 @@ impl Sink for Either type SinkItem = ::SinkItem; type SinkError = ::SinkError; - fn poll_ready(self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { unsafe { - match PinMut::get_mut_unchecked(self) { - Either::Left(x) => PinMut::new_unchecked(x).poll_ready(cx), - Either::Right(x) => PinMut::new_unchecked(x).poll_ready(cx), + match Pin::get_mut_unchecked(self) { + Either::Left(x) => Pin::new_unchecked(x).poll_ready(lw), + Either::Right(x) => Pin::new_unchecked(x).poll_ready(lw), } } } - fn start_send(self: PinMut, item: Self::SinkItem) -> Result<(), Self::SinkError> { + fn start_send(self: Pin<&mut Self>, item: Self::SinkItem) -> Result<(), Self::SinkError> { unsafe { - match PinMut::get_mut_unchecked(self) { - Either::Left(x) => PinMut::new_unchecked(x).start_send(item), - Either::Right(x) => PinMut::new_unchecked(x).start_send(item), + match Pin::get_mut_unchecked(self) { + Either::Left(x) => Pin::new_unchecked(x).start_send(item), + Either::Right(x) => Pin::new_unchecked(x).start_send(item), } } } - fn poll_flush(self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { unsafe { - match PinMut::get_mut_unchecked(self) { - Either::Left(x) => PinMut::new_unchecked(x).poll_flush(cx), - Either::Right(x) => PinMut::new_unchecked(x).poll_flush(cx), + match Pin::get_mut_unchecked(self) { + Either::Left(x) => Pin::new_unchecked(x).poll_flush(lw), + Either::Right(x) => Pin::new_unchecked(x).poll_flush(lw), } } } - fn poll_close(self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_close(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { unsafe { - match PinMut::get_mut_unchecked(self) { - Either::Left(x) => PinMut::new_unchecked(x).poll_close(cx), - Either::Right(x) => PinMut::new_unchecked(x).poll_close(cx), + match Pin::get_mut_unchecked(self) { + Either::Left(x) => Pin::new_unchecked(x).poll_close(lw), + Either::Right(x) => Pin::new_unchecked(x).poll_close(lw), } } } diff --git a/futures-test/src/assert.rs b/futures-test/src/assert.rs index 93e4b0b4a1..723318221a 100644 --- a/futures-test/src/assert.rs +++ b/futures-test/src/assert.rs @@ -30,10 +30,10 @@ macro_rules! assert_stream_pending { ($stream:expr) => {{ let mut stream = &mut $stream; $crate::assert::assert_is_unpin_stream(stream); - let stream = $crate::std_reexport::pin::PinMut::new(stream); - let cx = &mut $crate::task::no_spawn_context(); + let stream = $crate::std_reexport::pin::Pin::new(stream); + let lw = &$crate::task::noop_local_waker_ref(); let poll = $crate::futures_core_reexport::stream::Stream::poll_next( - stream, cx, + stream, lw, ); if poll.is_ready() { panic!("assertion failed: stream is not pending"); @@ -67,9 +67,9 @@ macro_rules! assert_stream_next { ($stream:expr, $item:expr) => {{ let mut stream = &mut $stream; $crate::assert::assert_is_unpin_stream(stream); - let stream = $crate::std_reexport::pin::PinMut::new(stream); - let cx = &mut $crate::task::no_spawn_context(); - match $crate::futures_core_reexport::stream::Stream::poll_next(stream, cx) { + let stream = $crate::std_reexport::pin::Pin::new(stream); + let lw = &$crate::task::noop_local_waker_ref(); + match $crate::futures_core_reexport::stream::Stream::poll_next(stream, lw) { $crate::futures_core_reexport::task::Poll::Ready(Some(x)) => { assert_eq!(x, $item); } @@ -110,9 +110,9 @@ macro_rules! assert_stream_done { ($stream:expr) => {{ let mut stream = &mut $stream; $crate::assert::assert_is_unpin_stream(stream); - let stream = $crate::std_reexport::pin::PinMut::new(stream); - let cx = &mut $crate::task::no_spawn_context(); - match $crate::futures_core_reexport::stream::Stream::poll_next(stream, cx) { + let stream = $crate::std_reexport::pin::Pin::new(stream); + let lw = &$crate::task::noop_local_waker_ref(); + match $crate::futures_core_reexport::stream::Stream::poll_next(stream, lw) { $crate::futures_core_reexport::task::Poll::Ready(Some(_)) => { panic!("assertion failed: expected stream to be done but had more elements"); } diff --git a/futures-test/src/future/assert_unmoved.rs b/futures-test/src/future/assert_unmoved.rs index 225fbf78f6..77c4bdb7a4 100644 --- a/futures-test/src/future/assert_unmoved.rs +++ b/futures-test/src/future/assert_unmoved.rs @@ -1,8 +1,8 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::marker::Pinned; -use std::pin::PinMut; +use std::pin::Pin; use std::ptr; /// Combinator for the @@ -33,8 +33,8 @@ impl Future for AssertUnmoved { type Output = Fut::Output; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { let cur_this = &*self as *const Self; if self.this_ptr.is_null() { @@ -43,7 +43,7 @@ impl Future for AssertUnmoved { } else { assert_eq!(self.this_ptr, cur_this, "Future moved between poll calls"); } - self.future().poll(cx) + self.future().poll(lw) } } diff --git a/futures-test/src/future/mod.rs b/futures-test/src/future/mod.rs index caf4de5837..f340fdd860 100644 --- a/futures-test/src/future/mod.rs +++ b/futures-test/src/future/mod.rs @@ -43,10 +43,10 @@ pub trait FutureTestExt: Future { /// let future = (async { 5 }).pending_once(); /// pin_mut!(future); /// - /// let cx = &mut task::no_spawn_context(); + /// let lw = &task::noop_local_waker_ref(); /// - /// assert_eq!(future.poll_unpin(cx), Poll::Pending); - /// assert_eq!(future.poll_unpin(cx), Poll::Ready(5)); + /// assert_eq!(future.poll_unpin(lw), Poll::Pending); + /// assert_eq!(future.poll_unpin(lw), Poll::Ready(5)); /// ``` fn pending_once(self) -> PendingOnce where diff --git a/futures-test/src/future/pending_once.rs b/futures-test/src/future/pending_once.rs index d98e53242f..228c66a6f0 100644 --- a/futures-test/src/future/pending_once.rs +++ b/futures-test/src/future/pending_once.rs @@ -1,6 +1,6 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll}; -use std::pin::PinMut; +use futures_core::task::{LocalWaker, Poll}; +use std::pin::Pin; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Combinator that guarantees one [`Poll::Pending`] before polling its inner @@ -32,14 +32,14 @@ impl Future for PendingOnce { type Output = Fut::Output; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { if *self.polled_before() { - self.future().poll(cx) + self.future().poll(lw) } else { *self.polled_before() = true; - cx.waker().wake(); + lw.wake(); Poll::Pending } } diff --git a/futures-test/src/task/context.rs b/futures-test/src/task/context.rs deleted file mode 100644 index 3dd4ebb34d..0000000000 --- a/futures-test/src/task/context.rs +++ /dev/null @@ -1,66 +0,0 @@ -use crate::task::{panic_local_waker_ref, panic_spawner_mut}; -use crate::task::{noop_local_waker_ref, noop_spawner_mut}; -use futures_core::task::Context; - -/// Create a new [`task::Context`](futures_core::task::Context) where both -/// the [waker](futures_core::task::Context::waker) and -/// [spawner](futures_core::task::Context::spawner) will panic if used. -/// -/// # Examples -/// -/// ```should_panic -/// #![feature(futures_api)] -/// use futures_test::task; -/// -/// let cx = task::panic_context(); -/// cx.waker().wake(); // Will panic -/// ``` -pub fn panic_context() -> Context<'static> { - Context::new(panic_local_waker_ref(), panic_spawner_mut()) -} - -/// Create a new [`task::Context`](futures_core::task::Context) where the -/// [waker](futures_core::task::Context::waker) will ignore any calls to -/// `wake` while the [spawner](futures_core::task::Context::spawner) will -/// panic if used. -/// -/// # Examples -/// -/// ``` -/// #![feature(async_await, futures_api, pin)] -/// use futures::future::Future; -/// use futures::task::Poll; -/// use futures_test::task::no_spawn_context; -/// use pin_utils::pin_mut; -/// -/// let mut future = async { 5 }; -/// pin_mut!(future); -/// -/// assert_eq!(future.poll(&mut no_spawn_context()), Poll::Ready(5)); -/// ``` -pub fn no_spawn_context() -> Context<'static> { - Context::new(noop_local_waker_ref(), panic_spawner_mut()) -} - -/// Create a new [`task::Context`](futures_core::task::Context) where the -/// [waker](futures_core::task::Context::waker) and -/// [spawner](futures_core::task::Context::spawner) will both ignore any -/// uses. -/// -/// # Examples -/// -/// ``` -/// #![feature(async_await, futures_api, pin)] -/// use futures::future::Future; -/// use futures::task::Poll; -/// use futures_test::task::noop_context; -/// use pin_utils::pin_mut; -/// -/// let mut future = async { 5 }; -/// pin_mut!(future); -/// -/// assert_eq!(future.poll(&mut noop_context()), Poll::Ready(5)); -/// ``` -pub fn noop_context() -> Context<'static> { - Context::new(noop_local_waker_ref(), noop_spawner_mut()) -} diff --git a/futures-test/src/task/mod.rs b/futures-test/src/task/mod.rs index cf06056565..5219d34d95 100644 --- a/futures-test/src/task/mod.rs +++ b/futures-test/src/task/mod.rs @@ -1,17 +1,9 @@ //! Task related testing utilities. //! //! This module provides utilities for creating test -//! [`task::Context`](futures_core::task::Context)s, //! [`LocalWaker`](futures_core::task::LocalWaker)s and //! [`Spawn`](futures_core::task::Spawn) implementations. //! -//! Commonly needed [`task::Context`](futures_core::task::Context)s can be -//! created via the functions [`no_spawn_context`], -//! [`noop_context`] and [`panic_context`]. For more advanced use cases, you -//! can create your own task context via -//! [`task::Context::new`](futures_core::task::Context::new) and make use -//! of the various provided test wakers and spawners: -//! //! Test wakers: //! - [`noop_local_waker`] creates a waker that ignores calls to //! [`wake`](futures_core::task::LocalWaker). @@ -31,9 +23,6 @@ //! return waker/spawner references: [`noop_local_waker_ref`], //! [`panic_local_waker_ref`], [`noop_spawner_mut`] and [`panic_spawner_mut`]. -mod context; -pub use self::context::{no_spawn_context, noop_context, panic_context}; - mod noop_spawner; pub use self::noop_spawner::{noop_spawner_mut, NoopSpawner}; diff --git a/futures-test/src/task/noop_spawner.rs b/futures-test/src/task/noop_spawner.rs index a46e9b3abe..8882933a1a 100644 --- a/futures-test/src/task/noop_spawner.rs +++ b/futures-test/src/task/noop_spawner.rs @@ -1,5 +1,5 @@ use futures_core::future::FutureObj; -use futures_core::task::{Spawn, SpawnObjError}; +use futures_core::task::{Spawn, SpawnError}; /// An implementation of [`Spawn`](futures_core::task::Spawn) that /// discards spawned futures when used. @@ -9,13 +9,10 @@ use futures_core::task::{Spawn, SpawnObjError}; /// ``` /// #![feature(async_await, futures_api)] /// use futures::task::SpawnExt; -/// use futures_test::task::{panic_context, NoopSpawner}; +/// use futures_test::task::NoopSpawner; /// -/// let mut cx = panic_context(); -/// let mut spawn = NoopSpawner::new(); -/// let cx = &mut cx.with_spawner(&mut spawn); -/// -/// cx.spawner().spawn(async { }); +/// let mut spawner = NoopSpawner::new(); +/// spawner.spawn(async { }); /// ``` #[derive(Debug)] pub struct NoopSpawner { @@ -33,7 +30,7 @@ impl Spawn for NoopSpawner { fn spawn_obj( &mut self, _future: FutureObj<'static, ()>, - ) -> Result<(), SpawnObjError> { + ) -> Result<(), SpawnError> { Ok(()) } } @@ -50,15 +47,11 @@ impl Default for NoopSpawner { /// /// ``` /// #![feature(async_await, futures_api)] -/// use futures::task::{self, SpawnExt}; -/// use futures_test::task::{noop_local_waker_ref, noop_spawner_mut}; -/// -/// let mut cx = task::Context::new( -/// noop_local_waker_ref(), -/// noop_spawner_mut(), -/// ); +/// use futures::task::SpawnExt; +/// use futures_test::task::noop_spawner_mut; /// -/// cx.spawner().spawn(async { }); +/// let spawner = noop_spawner_mut(); +/// spawner.spawn(async { }); /// ``` pub fn noop_spawner_mut() -> &'static mut NoopSpawner { Box::leak(Box::new(NoopSpawner::new())) diff --git a/futures-test/src/task/noop_waker.rs b/futures-test/src/task/noop_waker.rs index 63d5934c12..3a7dc38248 100644 --- a/futures-test/src/task/noop_waker.rs +++ b/futures-test/src/task/noop_waker.rs @@ -10,12 +10,9 @@ use std::sync::Arc; /// /// ``` /// #![feature(futures_api)] -/// use futures_test::task::{panic_context, noop_local_waker_ref}; -/// -/// let mut cx = panic_context(); -/// let cx = &mut cx.with_waker(noop_local_waker_ref()); -/// -/// cx.waker().wake(); +/// use futures_test::task::noop_local_waker_ref; +/// let lw = noop_local_waker_ref(); +/// lw.wake(); /// ``` #[derive(Debug)] pub struct NoopWake { @@ -71,16 +68,10 @@ pub fn noop_local_waker() -> LocalWaker { /// # Examples /// /// ``` -/// #![feature(async_await, futures_api)] -/// use futures::task; -/// use futures_test::task::{noop_local_waker_ref, noop_spawner_mut}; -/// -/// let mut cx = task::Context::new( -/// noop_local_waker_ref(), -/// noop_spawner_mut(), -/// ); -/// -/// cx.waker().wake(); +/// #![feature(futures_api)] +/// use futures_test::task::noop_local_waker_ref; +/// let lw = noop_local_waker_ref(); +/// lw.wake(); /// ``` pub fn noop_local_waker_ref() -> &'static LocalWaker { thread_local! { diff --git a/futures-test/src/task/panic_spawner.rs b/futures-test/src/task/panic_spawner.rs index 6c587367f2..3e14431c57 100644 --- a/futures-test/src/task/panic_spawner.rs +++ b/futures-test/src/task/panic_spawner.rs @@ -1,5 +1,5 @@ use futures_core::future::FutureObj; -use futures_core::task::{Spawn, SpawnObjError}; +use futures_core::task::{Spawn, SpawnError}; /// An implementation of [`Spawn`](futures_core::task::Spawn) that panics /// when used. @@ -9,13 +9,10 @@ use futures_core::task::{Spawn, SpawnObjError}; /// ```should_panic /// #![feature(async_await, futures_api)] /// use futures::task::SpawnExt; -/// use futures_test::task::{noop_context, PanicSpawner}; +/// use futures_test::task::PanicSpawner; /// -/// let mut cx = noop_context(); /// let mut spawn = PanicSpawner::new(); -/// let cx = &mut cx.with_spawner(&mut spawn); -/// -/// cx.spawner().spawn(async { }); // Will panic +/// spawn.spawn(async { }); // Will panic /// ``` #[derive(Debug)] pub struct PanicSpawner { @@ -33,7 +30,7 @@ impl Spawn for PanicSpawner { fn spawn_obj( &mut self, _future: FutureObj<'static, ()>, - ) -> Result<(), SpawnObjError> { + ) -> Result<(), SpawnError> { panic!("should not spawn") } } @@ -50,15 +47,11 @@ impl Default for PanicSpawner { /// /// ```should_panic /// #![feature(async_await, futures_api)] -/// use futures::task::{self, SpawnExt}; -/// use futures_test::task::{panic_local_waker_ref, panic_spawner_mut}; -/// -/// let mut cx = task::Context::new( -/// panic_local_waker_ref(), -/// panic_spawner_mut(), -/// ); +/// use futures::task::SpawnExt; +/// use futures_test::task::panic_spawner_mut; /// -/// cx.spawner().spawn(async { }); // Will panic +/// let spawner = panic_spawner_mut(); +/// spawner.spawn(async { }); // Will panic /// ``` pub fn panic_spawner_mut() -> &'static mut PanicSpawner { Box::leak(Box::new(PanicSpawner::new())) diff --git a/futures-test/src/task/panic_waker.rs b/futures-test/src/task/panic_waker.rs index aaada5ec0a..6dcad3cbb1 100644 --- a/futures-test/src/task/panic_waker.rs +++ b/futures-test/src/task/panic_waker.rs @@ -10,12 +10,10 @@ use std::sync::Arc; /// /// ```should_panic /// #![feature(futures_api)] -/// use futures_test::task::{noop_context, panic_local_waker_ref}; +/// use futures_test::task::panic_local_waker_ref; /// -/// let mut cx = noop_context(); -/// let cx = &mut cx.with_waker(panic_local_waker_ref()); -/// -/// cx.waker().wake(); // Will panic +/// let lw = panic_local_waker_ref(); +/// lw.wake(); // Will panic /// ``` #[derive(Debug)] pub struct PanicWake { @@ -77,14 +75,10 @@ pub fn panic_local_waker() -> LocalWaker { /// ```should_panic /// #![feature(async_await, futures_api)] /// use futures::task; -/// use futures_test::task::{panic_local_waker_ref, panic_spawner_mut}; -/// -/// let mut cx = task::Context::new( -/// panic_local_waker_ref(), -/// panic_spawner_mut(), -/// ); +/// use futures_test::task::panic_local_waker_ref; /// -/// cx.waker().wake(); // Will panic +/// let lw = panic_local_waker_ref(); +/// lw.wake(); // Will panic /// ``` pub fn panic_local_waker_ref() -> &'static LocalWaker { thread_local! { diff --git a/futures-test/src/task/record_spawner.rs b/futures-test/src/task/record_spawner.rs index 147359c22d..5ac9aed0ff 100644 --- a/futures-test/src/task/record_spawner.rs +++ b/futures-test/src/task/record_spawner.rs @@ -1,5 +1,5 @@ use futures_core::future::FutureObj; -use futures_core::task::{Spawn, SpawnObjError}; +use futures_core::task::{Spawn, SpawnError}; /// An implementation of [`Spawn`](futures_core::task::Spawn) that records /// any [`Future`](futures_core::future::Future)s spawned on it. @@ -9,16 +9,10 @@ use futures_core::task::{Spawn, SpawnObjError}; /// ``` /// #![feature(async_await, futures_api)] /// use futures::task::SpawnExt; -/// use futures_test::task::{panic_context, RecordSpawner}; +/// use futures_test::task::RecordSpawner; /// /// let mut recorder = RecordSpawner::new(); -/// -/// { -/// let mut cx = panic_context(); -/// let cx = &mut cx.with_spawner(&mut recorder); -/// cx.spawner().spawn(async { }); -/// } -/// +/// recorder.spawn(async { }); /// assert_eq!(recorder.spawned().len(), 1); /// ``` #[derive(Debug)] @@ -44,7 +38,7 @@ impl Spawn for RecordSpawner { fn spawn_obj( &mut self, future: FutureObj<'static, ()>, - ) -> Result<(), SpawnObjError> { + ) -> Result<(), SpawnError> { self.spawned.push(future); Ok(()) } diff --git a/futures-test/src/task/wake_counter.rs b/futures-test/src/task/wake_counter.rs index 4dde15acf8..6afaa9ef5a 100644 --- a/futures-test/src/task/wake_counter.rs +++ b/futures-test/src/task/wake_counter.rs @@ -9,16 +9,15 @@ use std::sync::Arc; /// /// ``` /// #![feature(futures_api)] -/// use futures_test::task::{panic_context, WakeCounter}; +/// use futures_test::task::WakeCounter; /// /// let wake_counter = WakeCounter::new(); -/// let mut cx = panic_context(); -/// let cx = &mut cx.with_waker(wake_counter.local_waker()); +/// let lw = wake_counter.local_waker(); /// /// assert_eq!(wake_counter.count(), 0); /// -/// cx.waker().wake(); -/// cx.waker().wake(); +/// lw.wake(); +/// lw.wake(); /// /// assert_eq!(wake_counter.count(), 2); /// ``` diff --git a/futures-util/benches/futures_unordered.rs b/futures-util/benches/futures_unordered.rs index 759021741a..80b87d0aba 100644 --- a/futures-util/benches/futures_unordered.rs +++ b/futures-util/benches/futures_unordered.rs @@ -1,5 +1,8 @@ #![feature(test, futures_api)] +extern crate test; +use crate::test::Bencher; + use futures::channel::oneshot; use futures::executor::block_on; use futures::future; @@ -7,7 +10,6 @@ use futures::stream::{StreamExt, FuturesUnordered}; use futures::task::Poll; use std::collections::VecDeque; use std::thread; -use test::Bencher; #[bench] fn oneshots(b: &mut Bencher) { @@ -29,9 +31,9 @@ fn oneshots(b: &mut Bencher) { } }); - block_on(future::poll_fn(move |cx| { + block_on(future::poll_fn(move |lw| { loop { - if let Poll::Ready(None) = rxs.poll_next_unpin(cx) { + if let Poll::Ready(None) = rxs.poll_next_unpin(lw) { break } } diff --git a/futures-util/benches_disabled/bilock.rs b/futures-util/benches_disabled/bilock.rs index f6963ca49a..b8fd377c12 100644 --- a/futures-util/benches_disabled/bilock.rs +++ b/futures-util/benches_disabled/bilock.rs @@ -2,7 +2,7 @@ #[cfg(feature = "bench")] mod bench { -use futures::task::{self, Wake, Waker}; +use futures::task::{LocalWaker, Wake, Waker}; use futures::executor::LocalPool; use futures_util::lock::BiLock; use futures_util::lock::BiLockAcquire; @@ -45,8 +45,8 @@ impl Stream for LockStream { type Item = BiLockAcquired; type Error = (); - fn poll_next(&mut self, cx: &mut task::Context) -> Poll, Self::Error> { - self.lock.poll(cx).map(|a| match a { + fn poll_next(&mut self, lw: &LocalWaker) -> Poll, Self::Error> { + self.lock.poll(lw).map(|a| match a { Async::Ready(a) => Async::Ready(Some(a)), Async::Pending => Async::Pending, }) @@ -60,7 +60,7 @@ fn contended(b: &mut Bencher) { let mut exec = pool.executor(); let waker = notify_noop(); let mut map = task::LocalMap::new(); - let mut cx = task::Context::new(&mut map, &waker, &mut exec); + let mut lw = task::Context::new(&mut map, &waker, &mut exec); b.iter(|| { let (x, y) = BiLock::new(1); @@ -69,20 +69,20 @@ fn contended(b: &mut Bencher) { let mut y = LockStream::new(y); for _ in 0..1000 { - let x_guard = match x.poll_next(&mut cx) { + let x_guard = match x.poll_next(&mut lw) { Ok(Async::Ready(Some(guard))) => guard, _ => panic!(), }; // Try poll second lock while first lock still holds the lock - match y.poll_next(&mut cx) { + match y.poll_next(&mut lw) { Ok(Async::Pending) => (), _ => panic!(), }; x.release_lock(x_guard); - let y_guard = match y.poll_next(&mut cx) { + let y_guard = match y.poll_next(&mut lw) { Ok(Async::Ready(Some(guard))) => guard, _ => panic!(), }; @@ -99,7 +99,7 @@ fn lock_unlock(b: &mut Bencher) { let mut exec = pool.executor(); let waker = notify_noop(); let mut map = task::LocalMap::new(); - let mut cx = task::Context::new(&mut map, &waker, &mut exec); + let mut lw = task::Context::new(&mut map, &waker, &mut exec); b.iter(|| { let (x, y) = BiLock::new(1); @@ -108,14 +108,14 @@ fn lock_unlock(b: &mut Bencher) { let mut y = LockStream::new(y); for _ in 0..1000 { - let x_guard = match x.poll_next(&mut cx) { + let x_guard = match x.poll_next(&mut lw) { Ok(Async::Ready(Some(guard))) => guard, _ => panic!(), }; x.release_lock(x_guard); - let y_guard = match y.poll_next(&mut cx) { + let y_guard = match y.poll_next(&mut lw) { Ok(Async::Ready(Some(guard))) => guard, _ => panic!(), }; diff --git a/futures-util/src/async_await/join.rs b/futures-util/src/async_await/join.rs index c3619c889d..da77c30cf6 100644 --- a/futures-util/src/async_await/join.rs +++ b/futures-util/src/async_await/join.rs @@ -29,18 +29,18 @@ macro_rules! join { // is no longer accessible by the end user. let mut $fut = $crate::future::maybe_done($fut); )* - await!($crate::future::poll_fn(move |cx| { + await!($crate::future::poll_fn(move |lw| { let mut all_done = true; $( if $crate::core_reexport::future::Future::poll( - unsafe { $crate::core_reexport::pin::PinMut::new_unchecked(&mut $fut) }, cx).is_pending() + unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }, lw).is_pending() { all_done = false; } )* if all_done { $crate::core_reexport::task::Poll::Ready(($( - unsafe { $crate::core_reexport::pin::PinMut::new_unchecked(&mut $fut) }.take_output().unwrap(), + unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }.take_output().unwrap(), )*)) } else { $crate::core_reexport::task::Poll::Pending @@ -97,19 +97,19 @@ macro_rules! try_join { let mut $fut = $crate::future::maybe_done($fut); )* - let res: $crate::core_reexport::result::Result<_, _> = await!($crate::future::poll_fn(move |cx| { + let res: $crate::core_reexport::result::Result<_, _> = await!($crate::future::poll_fn(move |lw| { let mut all_done = true; $( if $crate::core_reexport::future::Future::poll( - unsafe { $crate::core_reexport::pin::PinMut::new_unchecked(&mut $fut) }, cx).is_pending() + unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }, lw).is_pending() { all_done = false; - } else if unsafe { $crate::core_reexport::pin::PinMut::new_unchecked(&mut $fut) }.output_mut().unwrap().is_err() { + } else if unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }.output_mut().unwrap().is_err() { // `.err().unwrap()` rather than `.unwrap_err()` so that we don't introduce // a `T: Debug` bound. return $crate::core_reexport::task::Poll::Ready( $crate::core_reexport::result::Result::Err( - unsafe { $crate::core_reexport::pin::PinMut::new_unchecked(&mut $fut) }.take_output().unwrap().err().unwrap() + unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }.take_output().unwrap().err().unwrap() ) ); } @@ -119,7 +119,7 @@ macro_rules! try_join { $crate::core_reexport::result::Result::Ok(($( // `.ok().unwrap()` rather than `.unwrap()` so that we don't introduce // an `E: Debug` bound. - unsafe { $crate::core_reexport::pin::PinMut::new_unchecked(&mut $fut) }.take_output().unwrap().ok().unwrap(), + unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }.take_output().unwrap().ok().unwrap(), )*)) ) } else { diff --git a/futures-util/src/async_await/mod.rs b/futures-util/src/async_await/mod.rs index 1707107ef9..df98ae10c0 100644 --- a/futures-util/src/async_await/mod.rs +++ b/futures-util/src/async_await/mod.rs @@ -22,10 +22,6 @@ mod join; #[macro_use] mod select; -// Primary export is a macro -#[macro_use] -mod spawn; - #[doc(hidden)] #[inline(always)] pub fn assert_unpin(_: &T) {} diff --git a/futures-util/src/async_await/pending.rs b/futures-util/src/async_await/pending.rs index 75bec84617..e84489ae9f 100644 --- a/futures-util/src/async_await/pending.rs +++ b/futures-util/src/async_await/pending.rs @@ -1,12 +1,12 @@ -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A macro which yields to the event loop once. -/// -/// This is equivalent to returning [`Poll::Pending`](futures_core::task::Poll) -/// from a [`Future::poll`](futures_core::future::Future::poll) implementation. -/// Similarly, when using this macro, it must be ensured that [`wake`](std::task::Waker::wake) +/// +/// This is equivalent to returning [`Poll::Pending`](futures_core::task::Poll) +/// from a [`Future::poll`](futures_core::future::Future::poll) implementation. +/// Similarly, when using this macro, it must be ensured that [`wake`](std::task::Waker::wake) /// is called somewhere when further progress can be made. /// /// This macro is only usable inside of async functions, closures, and blocks. @@ -30,7 +30,7 @@ pub struct PendingOnce { impl Future for PendingOnce { type Output = (); - fn poll(mut self: PinMut, _: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, _: &LocalWaker) -> Poll { if self.is_ready { Poll::Ready(()) } else { diff --git a/futures-util/src/async_await/poll.rs b/futures-util/src/async_await/poll.rs index 3c12ca0081..2e8efd7728 100644 --- a/futures-util/src/async_await/poll.rs +++ b/futures-util/src/async_await/poll.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A macro which returns the result of polling a future once within the /// current `async` context. @@ -27,7 +27,7 @@ pub struct PollOnce { impl Future for PollOnce { type Output = Poll; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - Poll::Ready(PinMut::new(&mut self.future).poll(cx)) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + Poll::Ready(Pin::new(&mut self.future).poll(lw)) } } diff --git a/futures-util/src/async_await/select.rs b/futures-util/src/async_await/select.rs index 40887da6b9..7357cc76ba 100644 --- a/futures-util/src/async_await/select.rs +++ b/futures-util/src/async_await/select.rs @@ -47,10 +47,10 @@ macro_rules! select { )* } - let __priv_res = await!($crate::future::poll_fn(|cx| { + let __priv_res = await!($crate::future::poll_fn(|lw| { $( match $crate::core_reexport::future::Future::poll( - $crate::core_reexport::pin::PinMut::new(&mut $name), cx) + $crate::core_reexport::pin::Pin::new(&mut $name), lw) { $crate::core_reexport::task::Poll::Ready(x) => return $crate::core_reexport::task::Poll::Ready(__PrivResult::$name(x)), diff --git a/futures-util/src/async_await/spawn.rs b/futures-util/src/async_await/spawn.rs deleted file mode 100644 index 30374d3ed2..0000000000 --- a/futures-util/src/async_await/spawn.rs +++ /dev/null @@ -1,58 +0,0 @@ -/// Spawns a task onto the context's executor that polls the given future with -/// output `()` to completion. -/// -/// This macro returns a [`Result`] that contains a -/// [`SpawnError`](crate::task::SpawnError) if spawning fails. -/// -/// You can use [`spawn_with_handle!`] if you want to spawn a future -/// with output other than `()` or if you want to be able to await its -/// completion. -/// -/// ``` -/// #![feature(async_await, await_macro, futures_api)] -/// # futures::executor::block_on(async { -/// use futures::spawn; -/// -/// let future = async { /* ... */ }; -/// spawn!(future).unwrap(); -/// # }); -/// ``` -#[macro_export] -macro_rules! spawn { - ($future:expr) => { - await!($crate::future::lazy(|cx| { - $crate::task::SpawnExt::spawn(cx.spawner(), $future) - })) - } -} - -/// Spawns a task onto the context's executor that polls the given future to -/// completion and returns a future that resolves to the spawned future's -/// output. -/// -/// This macro returns a [`Result`] that contains a -/// [`JoinHandle`](crate::task::JoinHandle), or, if spawning fails, a -/// [`SpawnError`](crate::task::SpawnError). -/// [`JoinHandle`](crate::task::JoinHandle) is a future that resolves -/// to the output of the spawned future -/// -/// # Examples -/// -/// ``` -/// #![feature(async_await, await_macro, futures_api)] -/// # futures::executor::block_on(async { -/// use futures::{future, spawn_with_handle}; -/// -/// let future = future::ready(1); -/// let join_handle = spawn_with_handle!(future).unwrap(); -/// assert_eq!(await!(join_handle), 1); -/// # }); -/// ``` -#[macro_export] -macro_rules! spawn_with_handle { - ($future:expr) => { - await!($crate::future::lazy(|cx| { - $crate::task::SpawnExt::spawn_with_handle(cx.spawner(), $future) - })) - } -} diff --git a/futures-util/src/compat/compat.rs b/futures-util/src/compat/compat.rs index 3bfc75931a..396909cdd9 100644 --- a/futures-util/src/compat/compat.rs +++ b/futures-util/src/compat/compat.rs @@ -6,19 +6,18 @@ /// [`Sink`](futures::sink::Sink) and vice versa. #[derive(Debug)] #[must_use = "futures do nothing unless polled"] -pub struct Compat { +pub struct Compat { pub(crate) inner: T, - pub(crate) spawn: Option, } -impl Compat { +impl Compat { /// Returns the inner item. pub fn into_inner(self) -> T { self.inner } /// Creates a new [`Compat`]. - pub(crate) fn new(inner: T, spawn: Option) -> Compat { - Compat { inner, spawn } + pub(crate) fn new(inner: T) -> Compat { + Compat { inner } } } diff --git a/futures-util/src/compat/compat01to03.rs b/futures-util/src/compat/compat01to03.rs index 7cf34e1b80..481508a97a 100644 --- a/futures-util/src/compat/compat01to03.rs +++ b/futures-util/src/compat/compat01to03.rs @@ -5,19 +5,25 @@ use futures::executor::{ }; use futures::{Async as Async01, Future as Future01, Stream as Stream01}; use futures_core::{task as task03, Future as Future03, Stream as Stream03}; -use std::pin::PinMut; +use std::mem; +use std::task::{LocalWaker, Waker}; +use std::pin::Pin; -impl Future03 for Compat { +// TODO(cramertj) use as_waker from std when it lands +fn local_as_waker(lw: &LocalWaker) -> &Waker { + unsafe { mem::transmute(lw) } +} + +impl Future03 for Compat { type Output = Result; fn poll( - self: PinMut, - cx: &mut task03::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> task03::Poll { - let notify = &WakerToHandle(cx.waker()); - + let notify = &WakerToHandle(local_as_waker(lw)); executor01::with_notify(notify, 0, move || { - match unsafe { PinMut::get_mut_unchecked(self) }.inner.poll() { + match unsafe { Pin::get_mut_unchecked(self) }.inner.poll() { Ok(Async01::Ready(t)) => task03::Poll::Ready(Ok(t)), Ok(Async01::NotReady) => task03::Poll::Pending, Err(e) => task03::Poll::Ready(Err(e)), @@ -26,17 +32,16 @@ impl Future03 for Compat { } } -impl Stream03 for Compat { +impl Stream03 for Compat { type Item = Result; fn poll_next( - self: PinMut, - cx: &mut task03::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> task03::Poll> { - let notify = &WakerToHandle(cx.waker()); - + let notify = &WakerToHandle(local_as_waker(lw)); executor01::with_notify(notify, 0, move || { - match unsafe { PinMut::get_mut_unchecked(self) }.inner.poll() { + match unsafe { Pin::get_mut_unchecked(self) }.inner.poll() { Ok(Async01::Ready(Some(t))) => task03::Poll::Ready(Some(Ok(t))), Ok(Async01::Ready(None)) => task03::Poll::Ready(None), Ok(Async01::NotReady) => task03::Poll::Pending, diff --git a/futures-util/src/compat/compat03to01.rs b/futures-util/src/compat/compat03to01.rs index 105c66ce96..d19fbe2b5f 100644 --- a/futures-util/src/compat/compat03to01.rs +++ b/futures-util/src/compat/compat03to01.rs @@ -10,16 +10,15 @@ use futures_core::{ use futures_sink::Sink as Sink03; use std::{marker::Unpin, pin::PinMut, ptr::NonNull, sync::Arc}; -impl Future01 for Compat +impl Future01 for Compat where Fut: TryFuture03 + Unpin, - Sp: task03::Spawn, { type Item = Fut::Ok; type Error = Fut::Error; fn poll(&mut self) -> Poll01 { - with_context(self, |inner, cx| match inner.try_poll(cx) { + with_context(self, |inner, lw| match inner.try_poll(lw) { task03::Poll::Ready(Ok(t)) => Ok(Async01::Ready(t)), task03::Poll::Pending => Ok(Async01::NotReady), task03::Poll::Ready(Err(e)) => Err(e), @@ -27,16 +26,15 @@ where } } -impl Stream01 for Compat +impl Stream01 for Compat where St: TryStream03 + Unpin, - Sp: task03::Spawn, { type Item = St::Ok; type Error = St::Error; fn poll(&mut self) -> Poll01, Self::Error> { - with_context(self, |inner, cx| match inner.try_poll_next(cx) { + with_context(self, |inner, lw| match inner.try_poll_next(lw) { task03::Poll::Ready(None) => Ok(Async01::Ready(None)), task03::Poll::Ready(Some(Ok(t))) => Ok(Async01::Ready(Some(t))), task03::Poll::Pending => Ok(Async01::NotReady), @@ -45,10 +43,9 @@ where } } -impl Sink01 for Compat +impl Sink01 for Compat where T: Sink03 + Unpin, - E: task03::Spawn, { type SinkItem = T::SinkItem; type SinkError = T::SinkError; @@ -57,8 +54,8 @@ where &mut self, item: Self::SinkItem, ) -> StartSend01 { - with_context(self, |mut inner, cx| { - match inner.reborrow().poll_ready(cx) { + with_context(self, |mut inner, lw| { + match inner.as_mut().poll_ready(lw) { task03::Poll::Ready(Ok(())) => { inner.start_send(item).map(|()| AsyncSink01::Ready) } @@ -69,7 +66,7 @@ where } fn poll_complete(&mut self) -> Poll01<(), Self::SinkError> { - with_context(self, |inner, cx| match inner.poll_flush(cx) { + with_context(self, |inner, lw| match inner.poll_flush(lw) { task03::Poll::Ready(Ok(())) => Ok(Async01::Ready(())), task03::Poll::Pending => Ok(Async01::NotReady), task03::Poll::Ready(Err(e)) => Err(e), @@ -77,7 +74,7 @@ where } fn close(&mut self) -> Poll01<(), Self::SinkError> { - with_context(self, |inner, cx| match inner.poll_close(cx) { + with_context(self, |inner, lw| match inner.poll_close(lw) { task03::Poll::Ready(Ok(())) => Ok(Async01::Ready(())), task03::Poll::Pending => Ok(Async01::NotReady), task03::Poll::Ready(Err(e)) => Err(e), @@ -116,14 +113,11 @@ impl task03::Wake for CurrentOwned { } } -fn with_context(compat: &mut Compat, f: F) -> R +fn with_context(compat: &mut Compat, f: F) -> R where T: Unpin, - E: task03::Spawn, - F: FnOnce(PinMut, &mut task03::Context) -> R, + F: FnOnce(Pin<&mut T>, &task03::LocalWaker) -> R, { - let waker = current_ref_as_waker(); - let spawn = compat.spawn.as_mut().unwrap(); - let mut cx = task03::Context::new(&waker, spawn); - f(PinMut::new(&mut compat.inner), &mut cx) + let lw = current_ref_as_waker(); + f(Pin::new(&mut compat.inner), &lw) } diff --git a/futures-util/src/compat/executor.rs b/futures-util/src/compat/executor.rs index 95cfc91820..61a568e8b4 100644 --- a/futures-util/src/compat/executor.rs +++ b/futures-util/src/compat/executor.rs @@ -2,13 +2,12 @@ use super::Compat; use crate::{TryFutureExt, FutureExt, future::UnitError}; use futures::future::Executor as Executor01; -use futures_core::task::Spawn as Spawn03; -use futures_core::task as task03; +use futures_core::task::{Spawn as Spawn03, SpawnError as SpawnError03}; use futures_core::future::FutureObj; /// A future that can run on a futures 0.1 /// [`Executor`](futures::future::Executor). -pub type Executor01Future = Compat>, Box>; +pub type Executor01Future = Compat>>; /// Extension trait for futures 0.1 [`Executor`](futures::future::Executor). pub trait Executor01CompatExt: Executor01 + @@ -26,7 +25,6 @@ pub trait Executor01CompatExt: Executor01 + /// use tokio_threadpool::ThreadPool; /// /// let pool01 = ThreadPool::new(); - /// let spawner03 = pool01.sender().clone().compat(); /// # let (tx, rx) = futures::channel::oneshot::channel(); /// /// let future03 = async { @@ -37,7 +35,7 @@ pub trait Executor01CompatExt: Executor01 + /// }).unwrap(); /// }; /// - /// let future01 = future03.unit_error().boxed().compat(spawner03); + /// let future01 = future03.unit_error().boxed().compat(); /// /// pool01.spawn(future01); /// pool01.shutdown_on_idle().wait().unwrap(); @@ -71,21 +69,11 @@ where Ex: Executor01, fn spawn_obj( &mut self, future: FutureObj<'static, ()>, - ) -> Result<(), task03::SpawnObjError> { - let exec: Box = Box::new(self.clone()); - let future = future.unit_error().compat(exec); + ) -> Result<(), SpawnError03> { + let future = future.unit_error().compat(); - match self.executor01.execute(future) { - Ok(()) => Ok(()), - Err(err) => { - use futures_core::task::{SpawnObjError, SpawnErrorKind}; - - let fut = err.into_future().into_inner().unwrap_or_else(|_| ()); - Err(SpawnObjError { - kind: SpawnErrorKind::shutdown(), - future: Box::new(fut).into(), - }) - } - } + self.executor01.execute(future).map_err(|_| + SpawnError03::shutdown() + ) } } diff --git a/futures-util/src/compat/future01ext.rs b/futures-util/src/compat/future01ext.rs index 605f6a546c..1286f10cc6 100644 --- a/futures-util/src/compat/future01ext.rs +++ b/futures-util/src/compat/future01ext.rs @@ -9,7 +9,7 @@ pub trait Future01CompatExt: Future01 { /// [`Future`](futures::future::Future) /// into a futures 0.3 /// [`Future>`](futures_core::future::Future). - fn compat(self) -> Compat where Self: Sized { - Compat::new(self, None) + fn compat(self) -> Compat where Self: Sized { + Compat::new(self) } } diff --git a/futures-util/src/compat/mod.rs b/futures-util/src/compat/mod.rs index 702de3aa50..52db15bb91 100644 --- a/futures-util/src/compat/mod.rs +++ b/futures-util/src/compat/mod.rs @@ -5,7 +5,7 @@ mod executor; pub use self::executor::{Executor01CompatExt, Executor01Future, Executor01As03}; -#[allow(module_inception)] mod compat; +mod compat; pub use self::compat::Compat; mod compat01to03; diff --git a/futures-util/src/compat/stream01ext.rs b/futures-util/src/compat/stream01ext.rs index 097894cbfe..a09a43395c 100644 --- a/futures-util/src/compat/stream01ext.rs +++ b/futures-util/src/compat/stream01ext.rs @@ -9,7 +9,7 @@ pub trait Stream01CompatExt: Stream01 { /// [`Stream`](futures::stream::Stream) /// into a futures 0.3 /// [`Stream>`](futures_core::stream::Stream). - fn compat(self) -> Compat where Self: Sized { - Compat::new(self, None) + fn compat(self) -> Compat where Self: Sized { + Compat::new(self) } } diff --git a/futures-util/src/compat/tokio.rs b/futures-util/src/compat/tokio.rs index b7e4ccc0ed..13ae2dc732 100644 --- a/futures-util/src/compat/tokio.rs +++ b/futures-util/src/compat/tokio.rs @@ -1,6 +1,6 @@ use crate::{future::FutureExt, try_future::TryFutureExt}; use futures_core::future::FutureObj; -use futures_core::task::{Spawn, SpawnErrorKind, SpawnObjError}; +use futures_core::task::{Spawn, SpawnError}; use tokio_executor::{DefaultExecutor, Executor as TokioExecutor}; /// A spawner that delegates to `tokio`'s @@ -43,8 +43,8 @@ impl Spawn for TokioDefaultSpawner { fn spawn_obj( &mut self, future: FutureObj<'static, ()>, - ) -> Result<(), SpawnObjError> { - let fut = Box::new(future.unit_error().compat(*self)); + ) -> Result<(), SpawnError> { + let fut = Box::new(future.unit_error().compat()); DefaultExecutor::current().spawn(fut).map_err(|err| { panic!( "tokio failed to spawn and doesn't return the future: {:?}", @@ -53,10 +53,10 @@ impl Spawn for TokioDefaultSpawner { }) } - fn status(&self) -> Result<(), SpawnErrorKind> { + fn status(&self) -> Result<(), SpawnError> { DefaultExecutor::current().status().map_err(|err| { if err.is_shutdown() { - SpawnErrorKind::shutdown() + SpawnError::shutdown() } else { panic!( "tokio executor failed for non-shutdown reason: {:?}", diff --git a/futures-util/src/future/abortable.rs b/futures-util/src/future/abortable.rs index 35ba071d79..f5f124725b 100644 --- a/futures-util/src/future/abortable.rs +++ b/futures-util/src/future/abortable.rs @@ -1,9 +1,9 @@ use crate::task::AtomicWaker; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; use std::sync::Arc; use std::sync::atomic::{AtomicBool, Ordering}; @@ -122,19 +122,19 @@ pub struct Aborted; impl Future for Abortable where Fut: Future { type Output = Result; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { // Check if the future has been aborted if self.inner.cancel.load(Ordering::Relaxed) { return Poll::Ready(Err(Aborted)) } // attempt to complete the future - if let Poll::Ready(x) = self.future().poll(cx) { + if let Poll::Ready(x) = self.future().poll(lw) { return Poll::Ready(Ok(x)) } // Register to receive a wakeup if the future is aborted in the... future - self.inner.waker.register(cx.waker()); + self.inner.waker.register(lw); // Check to see if the future was aborted between the first check and // registration. diff --git a/futures-util/src/future/catch_unwind.rs b/futures-util/src/future/catch_unwind.rs index 0b26793d50..2eb7c6524d 100644 --- a/futures-util/src/future/catch_unwind.rs +++ b/futures-util/src/future/catch_unwind.rs @@ -1,8 +1,8 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; use std::any::Any; -use std::pin::PinMut; +use std::pin::Pin; use std::panic::{catch_unwind, UnwindSafe, AssertUnwindSafe}; use std::prelude::v1::*; @@ -28,8 +28,8 @@ impl Future for CatchUnwind { type Output = Result>; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - match catch_unwind(AssertUnwindSafe(|| self.future().poll(cx))) { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + match catch_unwind(AssertUnwindSafe(|| self.future().poll(lw))) { Ok(res) => res.map(Ok), Err(e) => Poll::Ready(Err(e)) } diff --git a/futures-util/src/future/chain.rs b/futures-util/src/future/chain.rs index 204db279b7..4eb6308d65 100644 --- a/futures-util/src/future/chain.rs +++ b/futures-util/src/future/chain.rs @@ -1,6 +1,6 @@ -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; #[must_use = "futures do nothing unless polled"] #[derive(Debug)] @@ -19,8 +19,8 @@ impl Chain } pub(crate) fn poll( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, f: F, ) -> Poll where F: FnOnce(Fut1::Output, Data) -> Fut2, @@ -28,18 +28,18 @@ impl Chain let mut f = Some(f); // Safe to call `get_mut_unchecked` because we won't move the futures. - let this = unsafe { PinMut::get_mut_unchecked(self) }; + let this = unsafe { Pin::get_mut_unchecked(self) }; loop { let (output, data) = match this { Chain::First(fut1, data) => { - match unsafe { PinMut::new_unchecked(fut1) }.poll(cx) { + match unsafe { Pin::new_unchecked(fut1) }.poll(lw) { Poll::Pending => return Poll::Pending, Poll::Ready(output) => (output, data.take().unwrap()), } } Chain::Second(fut2) => { - return unsafe { PinMut::new_unchecked(fut2) }.poll(cx); + return unsafe { Pin::new_unchecked(fut2) }.poll(lw); } Chain::Empty => unreachable!() }; diff --git a/futures-util/src/future/disabled/join_all.rs b/futures-util/src/future/disabled/join_all.rs index d9ae0cd7dc..0846ae24a3 100644 --- a/futures-util/src/future/disabled/join_all.rs +++ b/futures-util/src/future/disabled/join_all.rs @@ -87,13 +87,13 @@ impl Future for JoinAll type Error = F::Error; - fn poll(&mut self, cx: &mut task::Context) -> Poll { + fn poll(&mut self, lw: &LocalWaker) -> Poll { let mut all_done = true; for idx in 0 .. self.elems.len() { let done_val = match self.elems[idx] { ElemState::Pending(ref mut t) => { - match t.poll(cx) { + match t.poll(lw) { Ok(Async::Ready(v)) => Ok(v), Ok(Async::Pending) => { all_done = false; diff --git a/futures-util/src/future/disabled/select.rs b/futures-util/src/future/disabled/select.rs index cf994d4ce3..be267d6677 100644 --- a/futures-util/src/future/disabled/select.rs +++ b/futures-util/src/future/disabled/select.rs @@ -23,12 +23,12 @@ impl Future for Select where A: Future, B: Future { type Item = Either<(A::Item, B), (B::Item, A)>; type Error = Either<(A::Error, B), (B::Error, A)>; - fn poll(&mut self, cx: &mut task::Context) -> Poll { + fn poll(&mut self, lw: &LocalWaker) -> Poll { let (mut a, mut b) = self.inner.take().expect("cannot poll Select twice"); - match a.poll(cx) { + match a.poll(lw) { Err(e) => Err(Either::Left((e, b))), Ok(Async::Ready(x)) => Ok(Async::Ready(Either::Left((x, b)))), - Ok(Async::Pending) => match b.poll(cx) { + Ok(Async::Pending) => match b.poll(lw) { Err(e) => Err(Either::Right((e, a))), Ok(Async::Ready(x)) => Ok(Async::Ready(Either::Right((x, a)))), Ok(Async::Pending) => { diff --git a/futures-util/src/future/disabled/select_all.rs b/futures-util/src/future/disabled/select_all.rs index 38c2594a32..713ff28e08 100644 --- a/futures-util/src/future/disabled/select_all.rs +++ b/futures-util/src/future/disabled/select_all.rs @@ -49,9 +49,9 @@ impl Future for SelectAll type Item = (A::Item, usize, Vec); type Error = (A::Error, usize, Vec); - fn poll(&mut self, cx: &mut task::Context) -> Poll { + fn poll(&mut self, lw: &LocalWaker) -> Poll { let item = self.inner.iter_mut().enumerate().filter_map(|(i, f)| { - match f.poll(cx) { + match f.poll(lw) { Ok(Async::Pending) => None, Ok(Async::Ready(e)) => Some((i, Ok(e))), Err(e) => Some((i, Err(e))), diff --git a/futures-util/src/future/disabled/select_ok.rs b/futures-util/src/future/disabled/select_ok.rs index 24a7200f96..faaa92d8e7 100644 --- a/futures-util/src/future/disabled/select_ok.rs +++ b/futures-util/src/future/disabled/select_ok.rs @@ -45,11 +45,11 @@ impl Future for SelectOk where A: Future { type Item = (A::Item, Vec); type Error = A::Error; - fn poll(&mut self, cx: &mut task::Context) -> Poll { + fn poll(&mut self, lw: &LocalWaker) -> Poll { // loop until we've either exhausted all errors, a success was hit, or nothing is ready loop { let item = self.inner.iter_mut().enumerate().filter_map(|(i, f)| { - match f.poll(cx) { + match f.poll(lw) { Ok(Async::Pending) => None, Ok(Async::Ready(e)) => Some((i, Ok(e))), Err(e) => Some((i, Err(e))), diff --git a/futures-util/src/future/empty.rs b/futures-util/src/future/empty.rs index a5b89b6feb..9e3f20ae76 100644 --- a/futures-util/src/future/empty.rs +++ b/futures-util/src/future/empty.rs @@ -1,7 +1,7 @@ use core::marker; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A future which is never resolved. /// @@ -36,7 +36,7 @@ pub fn empty() -> Empty { impl Future for Empty { type Output = T; - fn poll(self: PinMut, _: &mut task::Context) -> Poll { + fn poll(self: Pin<&mut Self>, _: &LocalWaker) -> Poll { Poll::Pending } } diff --git a/futures-util/src/future/flatten.rs b/futures-util/src/future/flatten.rs index 66a131acad..471e2f26ca 100644 --- a/futures-util/src/future/flatten.rs +++ b/futures-util/src/future/flatten.rs @@ -1,8 +1,8 @@ use super::chain::Chain; use core::fmt; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Future for the `flatten` combinator. @@ -48,7 +48,7 @@ impl Future for Flatten { type Output = ::Output; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - self.state().poll(cx, |a, ()| a) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.state().poll(lw, |a, ()| a) } } diff --git a/futures-util/src/future/flatten_stream.rs b/futures-util/src/future/flatten_stream.rs index ace1bc7d3d..814597147b 100644 --- a/futures-util/src/future/flatten_stream.rs +++ b/futures-util/src/future/flatten_stream.rs @@ -1,8 +1,8 @@ use core::fmt; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// Future for the `flatten_stream` combinator, flattening a /// future-of-a-stream to get just the result of the final stream as a stream. @@ -46,14 +46,14 @@ impl Stream for FlattenStream { type Item = ::Item; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { loop { // safety: data is never moved via the resulting &mut reference - let stream = match &mut unsafe { PinMut::get_mut_unchecked(self.reborrow()) }.state { + let stream = match &mut unsafe { Pin::get_mut_unchecked(self.as_mut()) }.state { State::Future(f) => { // safety: the future we're re-pinning here will never be moved; // it will just be polled, then dropped in place - match unsafe { PinMut::new_unchecked(f) }.poll(cx) { + match unsafe { Pin::new_unchecked(f) }.poll(lw) { Poll::Pending => { // State is not changed, early return. return Poll::Pending @@ -69,14 +69,14 @@ impl Stream for FlattenStream State::Stream(s) => { // safety: the stream we're repinning here will never be moved; // it will just be polled, then dropped in place - return unsafe { PinMut::new_unchecked(s) }.poll_next(cx); + return unsafe { Pin::new_unchecked(s) }.poll_next(lw); } }; unsafe { // safety: we use the &mut only for an assignment, which causes // only an in-place drop - PinMut::get_mut_unchecked(self.reborrow()).state = State::Stream(stream); + Pin::get_mut_unchecked(self.as_mut()).state = State::Stream(stream); } } } diff --git a/futures-util/src/future/fuse.rs b/futures-util/src/future/fuse.rs index b421063a3f..d9a35d6cd5 100644 --- a/futures-util/src/future/fuse.rs +++ b/futures-util/src/future/fuse.rs @@ -1,6 +1,6 @@ -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// A future which "fuses" a future once it's been resolved. @@ -29,12 +29,12 @@ impl Fuse { impl Future for Fuse { type Output = Fut::Output; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { // safety: we use this &mut only for matching, not for movement let v = match self.future().as_pin_mut() { Some(fut) => { // safety: this re-pinned future will never move before being dropped - match fut.poll(cx) { + match fut.poll(lw) { Poll::Pending => return Poll::Pending, Poll::Ready(v) => v } @@ -42,7 +42,7 @@ impl Future for Fuse { None => return Poll::Pending, }; - PinMut::set(self.future(), None); + Pin::set(self.future(), None); Poll::Ready(v) } } diff --git a/futures-util/src/future/inspect.rs b/futures-util/src/future/inspect.rs index fd8307d419..62a51b6546 100644 --- a/futures-util/src/future/inspect.rs +++ b/futures-util/src/future/inspect.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Do something with the item of a future, passing it on. @@ -34,8 +34,8 @@ impl Future for Inspect { type Output = Fut::Output; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - let e = match self.future().poll(cx) { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + let e = match self.future().poll(lw) { Poll::Pending => return Poll::Pending, Poll::Ready(e) => e, }; diff --git a/futures-util/src/future/into_stream.rs b/futures-util/src/future/into_stream.rs index 57e25a00b9..001b9ebea9 100644 --- a/futures-util/src/future/into_stream.rs +++ b/futures-util/src/future/into_stream.rs @@ -1,7 +1,7 @@ -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// A type which converts a `Future` into a `Stream` @@ -25,10 +25,10 @@ impl IntoStream { impl Stream for IntoStream { type Item = Fut::Output; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { let v = match self.future().as_pin_mut() { Some(fut) => { - match fut.poll(cx) { + match fut.poll(lw) { Poll::Pending => return Poll::Pending, Poll::Ready(v) => v } @@ -36,7 +36,7 @@ impl Stream for IntoStream { None => return Poll::Ready(None), }; - PinMut::set(self.future(), None); + Pin::set(self.future(), None); Poll::Ready(Some(v)) } } diff --git a/futures-util/src/future/join.rs b/futures-util/src/future/join.rs index 1ace2bdac4..633beaf904 100644 --- a/futures-util/src/future/join.rs +++ b/futures-util/src/future/join.rs @@ -2,9 +2,9 @@ use crate::future::{MaybeDone, maybe_done}; use core::fmt; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; macro_rules! generate { @@ -48,11 +48,11 @@ macro_rules! generate { #[allow(clippy::useless_let_if_seq)] fn poll( - mut self: PinMut, cx: &mut task::Context + mut self: Pin<&mut Self>, lw: &LocalWaker ) -> Poll { let mut all_done = true; $( - if self.$Fut().poll(cx).is_pending() { + if self.$Fut().poll(lw).is_pending() { all_done = false; } )* diff --git a/futures-util/src/future/lazy.rs b/futures-util/src/future/lazy.rs index 90b8aee960..492d3b36ea 100644 --- a/futures-util/src/future/lazy.rs +++ b/futures-util/src/future/lazy.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A future which, when polled, invokes a closure and yields its result. /// @@ -12,7 +12,7 @@ pub struct Lazy { f: Option } -// safe because we never generate `PinMut` +// safe because we never generate `Pin<&mut F>` impl Unpin for Lazy {} /// Creates a new future that allows delayed execution of a closure. @@ -36,17 +36,17 @@ impl Unpin for Lazy {} /// # }); /// ``` pub fn lazy(f: F) -> Lazy - where F: FnOnce(&mut task::Context) -> R, + where F: FnOnce(&LocalWaker) -> R, { Lazy { f: Some(f) } } impl Future for Lazy - where F: FnOnce(&mut task::Context) -> R, + where F: FnOnce(&LocalWaker) -> R, { type Output = R; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - Poll::Ready((self.f.take().unwrap())(cx)) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + Poll::Ready((self.f.take().unwrap())(lw)) } } diff --git a/futures-util/src/future/map.rs b/futures-util/src/future/map.rs index b77be1575c..11ea79d047 100644 --- a/futures-util/src/future/map.rs +++ b/futures-util/src/future/map.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the `map` combinator, changing the type of a future. @@ -32,8 +32,8 @@ impl Future for Map { type Output = T; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - match self.future().poll(cx) { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + match self.future().poll(lw) { Poll::Pending => Poll::Pending, Poll::Ready(output) => { let f = self.f().take() diff --git a/futures-util/src/future/maybe_done.rs b/futures-util/src/future/maybe_done.rs index 3ab729c6d4..43c4732121 100644 --- a/futures-util/src/future/maybe_done.rs +++ b/futures-util/src/future/maybe_done.rs @@ -2,9 +2,9 @@ use core::marker::Unpin; use core::mem; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A future that may have completed. /// @@ -20,7 +20,7 @@ pub enum MaybeDone { Gone, } -// Safe because we never generate `PinMut` +// Safe because we never generate `Pin<&mut Fut::Output>` impl Unpin for MaybeDone {} /// Wraps a future into a `MaybeDone` @@ -35,10 +35,10 @@ impl Unpin for MaybeDone {} /// /// let future = future::maybe_done(future::ready(5)); /// pin_mut!(future); -/// assert_eq!(future.reborrow().take_output(), None); -/// let () = await!(future.reborrow()); -/// assert_eq!(future.reborrow().take_output(), Some(5)); -/// assert_eq!(future.reborrow().take_output(), None); +/// assert_eq!(future.as_mut().take_output(), None); +/// let () = await!(future.as_mut()); +/// assert_eq!(future.as_mut().take_output(), Some(5)); +/// assert_eq!(future.as_mut().take_output(), None); /// # }); /// ``` pub fn maybe_done(future: Fut) -> MaybeDone { @@ -52,9 +52,9 @@ impl MaybeDone { /// has not yet been called. #[inline] #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 - pub fn output_mut<'a>(self: PinMut<'a, Self>) -> Option<&'a mut Fut::Output> { + pub fn output_mut<'a>(self: Pin<&'a mut Self>) -> Option<&'a mut Fut::Output> { unsafe { - let this = PinMut::get_mut_unchecked(self); + let this = Pin::get_mut_unchecked(self); match this { MaybeDone::Done(res) => Some(res), _ => None, @@ -65,9 +65,9 @@ impl MaybeDone { /// Attempt to take the output of a `MaybeDone` without driving it /// towards completion. #[inline] - pub fn take_output(self: PinMut) -> Option { + pub fn take_output(self: Pin<&mut Self>) -> Option { unsafe { - let this = PinMut::get_mut_unchecked(self); + let this = Pin::get_mut_unchecked(self); match this { MaybeDone::Done(_) => {}, MaybeDone::Future(_) | MaybeDone::Gone => return None, @@ -84,11 +84,11 @@ impl MaybeDone { impl Future for MaybeDone { type Output = (); - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { let res = unsafe { - match PinMut::get_mut_unchecked(self.reborrow()) { + match Pin::get_mut_unchecked(self.as_mut()) { MaybeDone::Future(a) => { - if let Poll::Ready(res) = PinMut::new_unchecked(a).poll(cx) { + if let Poll::Ready(res) = Pin::new_unchecked(a).poll(lw) { res } else { return Poll::Pending @@ -98,7 +98,7 @@ impl Future for MaybeDone { MaybeDone::Gone => panic!("MaybeDone polled after value taken"), } }; - PinMut::set(self, MaybeDone::Done(res)); + Pin::set(self, MaybeDone::Done(res)); Poll::Ready(()) } } diff --git a/futures-util/src/future/mod.rs b/futures-util/src/future/mod.rs index 8597b00dc3..7badfc8058 100644 --- a/futures-util/src/future/mod.rs +++ b/futures-util/src/future/mod.rs @@ -4,10 +4,10 @@ //! including the `FutureExt` trait which adds methods to `Future` types. use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll, Spawn}; +use futures_core::task::{LocalWaker, Poll}; // Primitive futures mod empty; @@ -60,22 +60,20 @@ pub use self::inspect::Inspect; mod unit_error; pub use self::unit_error::UnitError; -mod with_spawner; -pub use self::with_spawner::WithSpawner; - // Implementation details mod chain; pub(crate) use self::chain::Chain; if_std! { - use std::pin::PinBox; - mod abortable; pub use self::abortable::{abortable, Abortable, AbortHandle, AbortRegistration, Aborted}; mod catch_unwind; pub use self::catch_unwind::CatchUnwind; + mod remote_handle; + pub use self::remote_handle::{Remote, RemoteHandle}; + // ToDo // mod join_all; // pub use self::join_all::{join_all, JoinAll}; @@ -623,12 +621,25 @@ pub trait FutureExt: Future { Shared::new(self) } + /// Turn this future into a future that yields `()` on completion and sends + /// its output to another future on a separate task. + /// + /// This can be used with spawning executors to easily retrieve the result + /// of a future executing on a separate task or thread. + #[cfg(feature = "std")] + fn remote_handle(self) -> (Remote, RemoteHandle) + where + Self: Sized, + { + remote_handle::remote_handle(self) + } + /// Wrap the future in a Box, pinning it. #[cfg(feature = "std")] - fn boxed(self) -> PinBox + fn boxed(self) -> Pin> where Self: Sized { - PinBox::new(self) + Box::pinned(self) } /// Turns a `Future` into a `TryFuture` with `Error = ()`. @@ -638,50 +649,11 @@ pub trait FutureExt: Future { UnitError::new(self) } - /// Assigns the provided `Spawn` to be used when spawning tasks - /// from within the future. - /// - /// # Examples - /// - /// ``` - /// #![feature(async_await, await_macro, futures_api)] - /// # futures::executor::block_on(async { - /// use futures::spawn; - /// use futures::executor::ThreadPool; - /// use futures::future::FutureExt; - /// use std::thread; - /// # let (tx, rx) = futures::channel::oneshot::channel(); - /// - /// let pool = ThreadPool::builder() - /// .name_prefix("my-pool-") - /// .pool_size(1) - /// .create().unwrap(); - /// - /// let val = await!((async { - /// assert_ne!(thread::current().name(), Some("my-pool-0")); - /// - /// // Spawned task runs on the executor specified via `with_spawner` - /// spawn!(async { - /// assert_eq!(thread::current().name(), Some("my-pool-0")); - /// # tx.send("ran").unwrap(); - /// }).unwrap(); - /// }).with_spawner(pool)); - /// - /// # assert_eq!(await!(rx), Ok("ran")) - /// # }) - /// ``` - fn with_spawner(self, spawner: Sp) -> WithSpawner - where Self: Sized, - Sp: Spawn - { - WithSpawner::new(self, spawner) - } - /// A convenience for calling `Future::poll` on `Unpin` future types. - fn poll_unpin(&mut self, cx: &mut task::Context) -> Poll + fn poll_unpin(&mut self, lw: &LocalWaker) -> Poll where Self: Unpin + Sized { - PinMut::new(self).poll(cx) + Pin::new(self).poll(lw) } } diff --git a/futures-util/src/future/option.rs b/futures-util/src/future/option.rs index 787dee5d60..11afa06e23 100644 --- a/futures-util/src/future/option.rs +++ b/futures-util/src/future/option.rs @@ -1,8 +1,8 @@ //! Definition of the `Option` (optional step) combinator -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// A future representing a value which may or may not be present. @@ -37,11 +37,11 @@ impl Future for OptionFuture { type Output = Option; fn poll( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll { match self.option().as_pin_mut() { - Some(x) => x.poll(cx).map(Some), + Some(x) => x.poll(lw).map(Some), None => Poll::Ready(None), } } diff --git a/futures-util/src/future/poll_fn.rs b/futures-util/src/future/poll_fn.rs index 80d2a55047..2a41baff77 100644 --- a/futures-util/src/future/poll_fn.rs +++ b/futures-util/src/future/poll_fn.rs @@ -1,9 +1,9 @@ //! Definition of the `PollFn` adapter combinator use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A future which wraps a function returning [`Poll`]. /// @@ -26,9 +26,9 @@ impl Unpin for PollFn {} /// #![feature(async_await, await_macro, futures_api)] /// # futures::executor::block_on(async { /// use futures::future::poll_fn; -/// use futures::task::{self, Poll}; +/// use futures::task::{LocalWaker, Poll}; /// -/// fn read_line(cx: &mut task::Context) -> Poll { +/// fn read_line(lw: &LocalWaker) -> Poll { /// Poll::Ready("Hello, World!".into()) /// } /// @@ -38,17 +38,17 @@ impl Unpin for PollFn {} /// ``` pub fn poll_fn(f: F) -> PollFn where - F: FnMut(&mut task::Context) -> Poll + F: FnMut(&LocalWaker) -> Poll { PollFn { f } } impl Future for PollFn - where F: FnMut(&mut task::Context) -> Poll, + where F: FnMut(&LocalWaker) -> Poll, { type Output = T; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - (&mut self.f)(cx) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + (&mut self.f)(lw) } } diff --git a/futures-util/src/future/ready.rs b/futures-util/src/future/ready.rs index 01b062c34b..bc15e589b9 100644 --- a/futures-util/src/future/ready.rs +++ b/futures-util/src/future/ready.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A future that is immediately ready with a value /// @@ -16,7 +16,7 @@ impl Future for Ready { type Output = T; #[inline] - fn poll(mut self: PinMut, _cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, _lw: &LocalWaker) -> Poll { Poll::Ready(self.0.take().unwrap()) } } diff --git a/futures-util/src/future/remote_handle.rs b/futures-util/src/future/remote_handle.rs new file mode 100644 index 0000000000..d0e819d4f5 --- /dev/null +++ b/futures-util/src/future/remote_handle.rs @@ -0,0 +1,119 @@ +use { + crate::future::{CatchUnwind, FutureExt}, + futures_channel::oneshot::{self, Sender, Receiver}, + futures_core::{ + future::Future, + task::{LocalWaker, Poll}, + }, + pin_utils::{unsafe_pinned, unsafe_unpinned}, + std::{ + any::Any, + fmt, + marker::Unpin, + panic::{self, AssertUnwindSafe}, + pin::Pin, + sync::{ + Arc, + atomic::{AtomicBool, Ordering}, + }, + thread, + }, +}; + +/// The handle to a remote future returned by +/// [`remote_handle`](crate::future::FutureExt::remote_handle). +#[must_use = "futures do nothing unless polled"] +#[derive(Debug)] +pub struct RemoteHandle { + rx: Receiver>, + keep_running: Arc, +} + +impl RemoteHandle { + /// Drops this handle *without* canceling the underlying future. + /// + /// This method can be used if you want to drop the handle, but let the + /// execution continue. + pub fn forget(self) { + self.keep_running.store(true, Ordering::SeqCst); + } +} + +impl Future for RemoteHandle { + type Output = T; + + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + match self.rx.poll_unpin(lw) { + Poll::Ready(Ok(Ok(output))) => Poll::Ready(output), + Poll::Ready(Ok(Err(e))) => panic::resume_unwind(e), + Poll::Ready(Err(e)) => panic::resume_unwind(Box::new(e)), + Poll::Pending => Poll::Pending, + } + } +} + +type SendMsg = Result<::Output, Box<(dyn Any + Send + 'static)>>; + +/// A future which sends its output to the corresponding `RemoteHandle`. +/// Created by [`remote_handle`](crate::future::FutureExt::remote_handle). +#[must_use = "futures do nothing unless polled"] +pub struct Remote { + tx: Option>>, + keep_running: Arc, + future: CatchUnwind>, +} + +impl fmt::Debug for Remote { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_tuple("Remote") + .field(&self.future) + .finish() + } +} + +impl Unpin for Remote {} + +impl Remote { + unsafe_pinned!(future: CatchUnwind>); + unsafe_unpinned!(tx: Option>>); + unsafe_unpinned!(keep_running: Arc); +} + +impl Future for Remote { + type Output = (); + + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll<()> { + if let Poll::Ready(_) = self.tx().as_mut().unwrap().poll_cancel(lw) { + if !self.keep_running().load(Ordering::SeqCst) { + // Cancelled, bail out + return Poll::Ready(()) + } + } + + let output = match self.future().poll(lw) { + Poll::Ready(output) => output, + Poll::Pending => return Poll::Pending, + }; + + // if the receiving end has gone away then that's ok, we just ignore the + // send error here. + drop(self.tx().take().unwrap().send(output)); + Poll::Ready(()) + } +} + +pub(super) fn remote_handle(future: Fut) -> (Remote, RemoteHandle) { + let (tx, rx) = oneshot::channel(); + let keep_running = Arc::new(AtomicBool::new(false)); + + // AssertUnwindSafe is used here because `Send + 'static` is basically + // an alias for an implementation of the `UnwindSafe` trait but we can't + // express that in the standard library right now. + let wrapped = Remote { + future: AssertUnwindSafe(future).catch_unwind(), + tx: Some(tx), + keep_running: keep_running.clone(), + }; + + (wrapped, RemoteHandle { rx, keep_running }) +} diff --git a/futures-util/src/future/shared.rs b/futures-util/src/future/shared.rs index 7ce1179aa6..b6d64107e9 100644 --- a/futures-util/src/future/shared.rs +++ b/futures-util/src/future/shared.rs @@ -1,10 +1,10 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll, Wake, Waker}; +use futures_core::task::{LocalWaker, Poll, Wake, Waker}; use slab::Slab; use std::fmt; use std::cell::UnsafeCell; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; use std::sync::{Arc, Mutex}; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; @@ -105,7 +105,7 @@ where } /// Registers the current task to receive a wakeup when `Inner` is awoken. - fn set_waker(&mut self, cx: &mut task::Context) { + fn set_waker(&mut self, lw: &LocalWaker) { // Acquire the lock first before checking COMPLETE to ensure there // isn't a race. let mut wakers = self.inner.notifier.wakers.lock().unwrap(); @@ -116,18 +116,18 @@ where return }; if self.waker_key == NULL_WAKER_KEY { - self.waker_key = wakers.insert(Some(cx.waker().clone())); + 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 { // If there's still an unwoken waker in the slot, only replace // if the current one wouldn't wake the same task. - !old_waker.will_wake(cx.waker()) + !lw.will_wake_nonlocal(old_waker) } else { true }; if needs_replacement { - *waker_slot = Some(cx.waker().clone()); + *waker_slot = Some(lw.clone().into_waker()); } } debug_assert!(self.waker_key != NULL_WAKER_KEY); @@ -150,10 +150,10 @@ where { type Output = Fut::Output; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { let this = &mut *self; - this.set_waker(cx); + this.set_waker(lw); match this.inner.notifier.state.compare_and_swap(IDLE, POLLING, SeqCst) { IDLE => { @@ -173,7 +173,7 @@ where } let waker = local_waker_from_nonlocal(this.inner.notifier.clone()); - let mut cx = cx.with_waker(&waker); + let lw = &waker; loop { struct Reset<'a>(&'a AtomicUsize); @@ -193,7 +193,7 @@ where // Poll the future let res = unsafe { if let FutureOrOutput::Future(future) = &mut *this.inner.future_or_output.get() { - PinMut::new_unchecked(future).poll(&mut cx) + Pin::new_unchecked(future).poll(lw) } else { unreachable!() } diff --git a/futures-util/src/future/then.rs b/futures-util/src/future/then.rs index 8f735126bf..5eedef57d5 100644 --- a/futures-util/src/future/then.rs +++ b/futures-util/src/future/then.rs @@ -1,7 +1,7 @@ use super::Chain; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Future for the `then` combinator, chaining computations on the end of @@ -35,7 +35,7 @@ impl Future for Then { type Output = Fut2::Output; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - self.chain().poll(cx, |output, f| f(output)) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.chain().poll(lw, |output, f| f(output)) } } diff --git a/futures-util/src/future/unit_error.rs b/futures-util/src/future/unit_error.rs index 3b57e9a8f4..478a2ca645 100644 --- a/futures-util/src/future/unit_error.rs +++ b/futures-util/src/future/unit_error.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Future for the `unit_error` combinator, turning a `Future` into a `TryFuture`. @@ -29,7 +29,7 @@ impl Future for UnitError { type Output = Result; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll> { - self.future().poll(cx).map(Ok) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + self.future().poll(lw).map(Ok) } } diff --git a/futures-util/src/future/with_spawner.rs b/futures-util/src/future/with_spawner.rs deleted file mode 100644 index c6d23d6234..0000000000 --- a/futures-util/src/future/with_spawner.rs +++ /dev/null @@ -1,37 +0,0 @@ -use core::marker::Unpin; -use core::pin::PinMut; -use futures_core::future::Future; -use futures_core::task::{self, Poll, Spawn}; - -/// Future for the `with_spawner` combinator, assigning a [`Spawn`] -/// to be used when spawning other futures. -/// -/// This is created by the `Future::with_spawner` method. -#[derive(Debug)] -#[must_use = "futures do nothing unless polled"] -pub struct WithSpawner where Fut: Future, Sp: Spawn { - spawner: Sp, - future: Fut -} - -impl WithSpawner { - pub(super) fn new(future: Fut, spawner: Sp) -> WithSpawner { - WithSpawner { spawner, future } - } -} - -impl Unpin for WithSpawner {} - -impl Future for WithSpawner - where Fut: Future, - Sp: Spawn, -{ - type Output = Fut::Output; - - fn poll(self: PinMut, cx: &mut task::Context) -> Poll { - let this = unsafe { PinMut::get_mut_unchecked(self) }; - let fut = unsafe { PinMut::new_unchecked(&mut this.future) }; - let spawner = &mut this.spawner; - fut.poll(&mut cx.with_spawner(spawner)) - } -} diff --git a/futures-util/src/io/allow_std.rs b/futures-util/src/io/allow_std.rs index 2d4f1c723c..2dbb9f4405 100644 --- a/futures-util/src/io/allow_std.rs +++ b/futures-util/src/io/allow_std.rs @@ -1,4 +1,4 @@ -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_io::{AsyncRead, AsyncWrite}; use std::{fmt, io}; use std::string::String; @@ -73,19 +73,19 @@ impl io::Write for AllowStdIo where T: io::Write { } impl AsyncWrite for AllowStdIo where T: io::Write { - fn poll_write(&mut self, _: &mut task::Context, buf: &[u8]) + fn poll_write(&mut self, _: &LocalWaker, buf: &[u8]) -> Poll> { Poll::Ready(Ok(try_with_interrupt!(io::Write::write(&mut self.0, buf)))) } - fn poll_flush(&mut self, _: &mut task::Context) -> Poll> { + fn poll_flush(&mut self, _: &LocalWaker) -> Poll> { try_with_interrupt!(io::Write::flush(self)); Poll::Ready(Ok(())) } - fn poll_close(&mut self, cx: &mut task::Context) -> Poll> { - self.poll_flush(cx) + fn poll_close(&mut self, lw: &LocalWaker) -> Poll> { + self.poll_flush(lw) } } @@ -107,7 +107,7 @@ impl io::Read for AllowStdIo where T: io::Read { } impl AsyncRead for AllowStdIo where T: io::Read { - fn poll_read(&mut self, _: &mut task::Context, buf: &mut [u8]) + fn poll_read(&mut self, _: &LocalWaker, buf: &mut [u8]) -> Poll> { Poll::Ready(Ok(try_with_interrupt!(io::Read::read(&mut self.0, buf)))) diff --git a/futures-util/src/io/close.rs b/futures-util/src/io/close.rs index 3d1010f275..965f05f22a 100644 --- a/futures-util/src/io/close.rs +++ b/futures-util/src/io/close.rs @@ -1,9 +1,9 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_io::AsyncWrite; use std::io; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; /// A future used to fully close an I/O object. /// @@ -15,7 +15,7 @@ pub struct Close<'a, W: ?Sized + 'a> { writer: &'a mut W, } -// PinMut is never projected to fields +// Pin is never projected to fields impl Unpin for Close<'_, W> {} impl<'a, W: AsyncWrite + ?Sized> Close<'a, W> { @@ -27,7 +27,7 @@ impl<'a, W: AsyncWrite + ?Sized> Close<'a, W> { impl Future for Close<'_, W> { type Output = io::Result<()>; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - self.writer.poll_close(cx) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.writer.poll_close(lw) } } diff --git a/futures-util/src/io/copy_into.rs b/futures-util/src/io/copy_into.rs index 7a3f27e317..49f24713a5 100644 --- a/futures-util/src/io/copy_into.rs +++ b/futures-util/src/io/copy_into.rs @@ -1,10 +1,10 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_io::{AsyncRead, AsyncWrite}; use std::boxed::Box; use std::io; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; /// A future which will copy all data from a reader into a writer. /// @@ -23,7 +23,7 @@ pub struct CopyInto<'a, R: ?Sized + 'a, W: ?Sized + 'a> { buf: Box<[u8]>, } -// No projections of PinMut into PinMut are ever done. +// No projections of Pin<&mut CopyInto> into Pin<&mut Field> are ever done. impl Unpin for CopyInto<'_, R, W> {} impl<'a, R: ?Sized, W: ?Sized> CopyInto<'a, R, W> { @@ -46,13 +46,13 @@ impl Future for CopyInto<'_, R, W> { type Output = io::Result; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { let this = &mut *self; loop { // If our buffer is empty, then we need to read some data to // continue. if this.pos == this.cap && !this.read_done { - let n = try_ready!(this.reader.poll_read(cx, &mut this.buf)); + let n = try_ready!(this.reader.poll_read(lw, &mut this.buf)); if n == 0 { this.read_done = true; } else { @@ -63,7 +63,7 @@ impl Future for CopyInto<'_, R, W> // If our buffer has some data, let's write it out! while this.pos < this.cap { - let i = try_ready!(this.writer.poll_write(cx, &this.buf[this.pos..this.cap])); + let i = try_ready!(this.writer.poll_write(lw, &this.buf[this.pos..this.cap])); if i == 0 { return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) } else { @@ -76,7 +76,7 @@ impl Future for CopyInto<'_, R, W> // data and finish the transfer. // done with the entire transfer. if this.pos == this.cap && this.read_done { - try_ready!(this.writer.poll_flush(cx)); + try_ready!(this.writer.poll_flush(lw)); return Poll::Ready(Ok(this.amt)); } } diff --git a/futures-util/src/io/disabled/lines.rs b/futures-util/src/io/disabled/lines.rs index d21a6322a8..933ba277e1 100644 --- a/futures-util/src/io/disabled/lines.rs +++ b/futures-util/src/io/disabled/lines.rs @@ -45,7 +45,7 @@ impl Stream for Lines type Item = String; type Error = io::Error; - fn poll(&mut self, cx: &mut task::Context) -> Poll, io::Error> { + fn poll(&mut self, lw: &LocalWaker) -> Poll, io::Error> { let n = ready!(self.io.read_line(&mut self.line)); if n == 0 && self.line.len() == 0 { return Ok(None.into()) diff --git a/futures-util/src/io/flush.rs b/futures-util/src/io/flush.rs index aba347599a..ea88943976 100644 --- a/futures-util/src/io/flush.rs +++ b/futures-util/src/io/flush.rs @@ -1,8 +1,8 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use std::io; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; use futures_io::AsyncWrite; @@ -32,7 +32,7 @@ impl Future for Flush<'_, W> { type Output = io::Result<()>; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - self.writer.poll_flush(cx) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.writer.poll_flush(lw) } } diff --git a/futures-util/src/io/read.rs b/futures-util/src/io/read.rs index 5a30cd7395..d1c986298c 100644 --- a/futures-util/src/io/read.rs +++ b/futures-util/src/io/read.rs @@ -1,9 +1,9 @@ use crate::io::AsyncRead; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use std::io; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; /// A future which can be used to easily read available number of bytes to fill /// a buffer. @@ -25,8 +25,8 @@ impl<'a, R: AsyncRead + ?Sized> Read<'a, R> { impl Future for Read<'_, R> { type Output = io::Result; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { let this = &mut *self; - this.reader.poll_read(cx, this.buf) + this.reader.poll_read(lw, this.buf) } } diff --git a/futures-util/src/io/read_exact.rs b/futures-util/src/io/read_exact.rs index fd1f732b40..ec762b9a5e 100644 --- a/futures-util/src/io/read_exact.rs +++ b/futures-util/src/io/read_exact.rs @@ -1,10 +1,10 @@ use crate::io::AsyncRead; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use std::io; use std::marker::Unpin; use std::mem; -use std::pin::PinMut; +use std::pin::Pin; /// A future which can be used to easily read exactly enough bytes to fill /// a buffer. @@ -29,10 +29,10 @@ impl<'a, R: AsyncRead + ?Sized> ReadExact<'a, R> { impl Future for ReadExact<'_, R> { type Output = io::Result<()>; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { let this = &mut *self; while !this.buf.is_empty() { - let n = try_ready!(this.reader.poll_read(cx, this.buf)); + let n = try_ready!(this.reader.poll_read(lw, this.buf)); { let (_, rest) = mem::replace(&mut this.buf, &mut []).split_at_mut(n); this.buf = rest; diff --git a/futures-util/src/io/read_to_end.rs b/futures-util/src/io/read_to_end.rs index 1294e7be08..44970dba81 100644 --- a/futures-util/src/io/read_to_end.rs +++ b/futures-util/src/io/read_to_end.rs @@ -1,9 +1,9 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_io::AsyncRead; use std::io; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; use std::vec::Vec; /// A future which can be used to easily read the entire contents of a stream @@ -46,7 +46,7 @@ impl Drop for Guard<'_> { // readers, we need to make sure to truncate that if any of this panics. fn read_to_end_internal( rd: &mut R, - cx: &mut task::Context, + lw: &LocalWaker, buf: &mut Vec, ) -> Poll> { let mut g = Guard { len: buf.len(), buf }; @@ -61,7 +61,7 @@ fn read_to_end_internal( } } - match rd.poll_read(cx, &mut g.buf[g.len..]) { + match rd.poll_read(lw, &mut g.buf[g.len..]) { Poll::Ready(Ok(0)) => { ret = Poll::Ready(Ok(())); break; @@ -83,8 +83,8 @@ impl Future for ReadToEnd<'_, A> { type Output = io::Result<()>; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { let this = &mut *self; - read_to_end_internal(this.reader, cx, this.buf) + read_to_end_internal(this.reader, lw, this.buf) } } diff --git a/futures-util/src/io/split.rs b/futures-util/src/io/split.rs index 4f1e84b9b9..18c4330106 100644 --- a/futures-util/src/io/split.rs +++ b/futures-util/src/io/split.rs @@ -1,8 +1,8 @@ use crate::lock::BiLock; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_io::{AsyncRead, AsyncWrite, IoVec}; use std::io; -use std::pin::PinMut; +use std::pin::Pin; /// The readable half of an object returned from `AsyncRead::split`. #[derive(Debug)] @@ -18,15 +18,15 @@ pub struct WriteHalf { fn lock_and_then( lock: &BiLock, - cx: &mut task::Context, + lw: &LocalWaker, f: F ) -> Poll> - where F: FnOnce(&mut T, &mut task::Context) -> Poll> + where F: FnOnce(&mut T, &LocalWaker) -> Poll> { - match lock.poll_lock(cx) { + match lock.poll_lock(lw) { // Safety: the value behind the bilock used by `ReadHalf` and `WriteHalf` is never exposed - // as a `PinMut` anywhere other than here as a way to get to `&mut`. - Poll::Ready(mut l) => f(unsafe { PinMut::get_mut_unchecked(l.as_pin_mut()) }, cx), + // as a `Pin<&mut T>` anywhere other than here as a way to get to `&mut T`. + Poll::Ready(mut l) => f(unsafe { Pin::get_mut_unchecked(l.as_pin_mut()) }, lw), Poll::Pending => Poll::Pending, } } @@ -37,37 +37,37 @@ pub fn split(t: T) -> (ReadHalf, WriteHalf) { } impl AsyncRead for ReadHalf { - fn poll_read(&mut self, cx: &mut task::Context, buf: &mut [u8]) + fn poll_read(&mut self, lw: &LocalWaker, buf: &mut [u8]) -> Poll> { - lock_and_then(&self.handle, cx, |l, cx| l.poll_read(cx, buf)) + lock_and_then(&self.handle, lw, |l, lw| l.poll_read(lw, buf)) } - fn poll_vectored_read(&mut self, cx: &mut task::Context, vec: &mut [&mut IoVec]) + fn poll_vectored_read(&mut self, lw: &LocalWaker, vec: &mut [&mut IoVec]) -> Poll> { - lock_and_then(&self.handle, cx, |l, cx| l.poll_vectored_read(cx, vec)) + lock_and_then(&self.handle, lw, |l, lw| l.poll_vectored_read(lw, vec)) } } impl AsyncWrite for WriteHalf { - fn poll_write(&mut self, cx: &mut task::Context, buf: &[u8]) + fn poll_write(&mut self, lw: &LocalWaker, buf: &[u8]) -> Poll> { - lock_and_then(&self.handle, cx, |l, cx| l.poll_write(cx, buf)) + lock_and_then(&self.handle, lw, |l, lw| l.poll_write(lw, buf)) } - fn poll_vectored_write(&mut self, cx: &mut task::Context, vec: &[&IoVec]) + fn poll_vectored_write(&mut self, lw: &LocalWaker, vec: &[&IoVec]) -> Poll> { - lock_and_then(&self.handle, cx, |l, cx| l.poll_vectored_write(cx, vec)) + lock_and_then(&self.handle, lw, |l, lw| l.poll_vectored_write(lw, vec)) } - fn poll_flush(&mut self, cx: &mut task::Context) -> Poll> { - lock_and_then(&self.handle, cx, |l, cx| l.poll_flush(cx)) + fn poll_flush(&mut self, lw: &LocalWaker) -> Poll> { + lock_and_then(&self.handle, lw, |l, lw| l.poll_flush(lw)) } - fn poll_close(&mut self, cx: &mut task::Context) -> Poll> { - lock_and_then(&self.handle, cx, |l, cx| l.poll_close(cx)) + fn poll_close(&mut self, lw: &LocalWaker) -> Poll> { + lock_and_then(&self.handle, lw, |l, lw| l.poll_close(lw)) } } diff --git a/futures-util/src/io/write_all.rs b/futures-util/src/io/write_all.rs index 0ffbcc4406..43290a1a34 100644 --- a/futures-util/src/io/write_all.rs +++ b/futures-util/src/io/write_all.rs @@ -1,10 +1,10 @@ use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_io::AsyncWrite; use std::io; use std::marker::Unpin; use std::mem; -use std::pin::PinMut; +use std::pin::Pin; /// A future used to write the entire contents of some data to a stream. /// @@ -29,10 +29,10 @@ impl<'a, W: AsyncWrite + ?Sized> WriteAll<'a, W> { impl Future for WriteAll<'_, W> { type Output = io::Result<()>; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { let this = &mut *self; while !this.buf.is_empty() { - let n = try_ready!(this.writer.poll_write(cx, this.buf)); + let n = try_ready!(this.writer.poll_write(lw, this.buf)); { let (_, rest) = mem::replace(&mut this.buf, &[]).split_at(n); this.buf = rest; diff --git a/futures-util/src/lib.rs b/futures-util/src/lib.rs index 56ac3c65ad..e3d2f74c71 100644 --- a/futures-util/src/lib.rs +++ b/futures-util/src/lib.rs @@ -36,31 +36,31 @@ pub mod core_reexport { macro_rules! delegate_sink { ($field:ident) => { fn poll_ready( - mut self: PinMut, - cx: &mut $crate::core_reexport::task::Context, + mut self: Pin<&mut Self>, + lw: &$crate::core_reexport::task::LocalWaker, ) -> $crate::core_reexport::task::Poll> { - self.$field().poll_ready(cx) + self.$field().poll_ready(lw) } fn start_send( - mut self: PinMut, + mut self: Pin<&mut Self>, item: Self::SinkItem ) -> Result<(), Self::SinkError> { self.$field().start_send(item) } fn poll_flush( - mut self: PinMut, - cx: &mut $crate::core_reexport::task::Context + mut self: Pin<&mut Self>, + lw: &$crate::core_reexport::task::LocalWaker ) -> $crate::core_reexport::task::Poll> { - self.$field().poll_flush(cx) + self.$field().poll_flush(lw) } fn poll_close( - mut self: PinMut, - cx: &mut $crate::core_reexport::task::Context + mut self: Pin<&mut Self>, + lw: &$crate::core_reexport::task::LocalWaker ) -> $crate::core_reexport::task::Poll> { - self.$field().poll_close(cx) + self.$field().poll_close(lw) } } } diff --git a/futures-util/src/lock.rs b/futures-util/src/lock.rs index 773b583517..a61b1c7fae 100644 --- a/futures-util/src/lock.rs +++ b/futures-util/src/lock.rs @@ -2,7 +2,7 @@ #![allow(unused)] use futures_core::future::Future; -use futures_core::task::{self, Poll, Waker}; +use futures_core::task::{LocalWaker, Poll, Waker}; use std::any::Any; use std::boxed::Box; use std::cell::UnsafeCell; @@ -11,7 +11,7 @@ use std::fmt; use std::marker::Unpin; use std::mem; use std::ops::{Deref, DerefMut}; -use std::pin::PinMut; +use std::pin::Pin; use std::sync::Arc; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; @@ -55,9 +55,9 @@ impl BiLock { /// that will ever be available to the lock. These can then be sent to separate /// tasks to be managed there. /// - /// The data behind the bilock is considered to be pinned, which allows `PinMut` + /// The data behind the bilock is considered to be pinned, which allows `Pin` /// references to locked data. However, this means that the locked value - /// will only be available through `PinMut` (not `&mut`) unless `T` is `Unpin`. + /// will only be available through `Pin<&mut T>` (not `&mut T`) unless `T` is `Unpin`. /// Similarly, reuniting the lock and extracting the inner value is only /// possible when `T` is `Unpin`. pub fn new(t: T) -> (BiLock, BiLock) { @@ -87,7 +87,7 @@ impl BiLock { /// /// This function will panic if called outside the context of a future's /// task. - pub fn poll_lock(&self, cx: &mut task::Context) -> Poll> { + pub fn poll_lock(&self, lw: &LocalWaker) -> Poll> { loop { match self.arc.state.swap(1, SeqCst) { // Woohoo, we grabbed the lock! @@ -104,7 +104,7 @@ impl BiLock { } // type ascription for safety's sake! - let me: Box = Box::new(cx.waker().clone()); + let me: Box = Box::new(lw.clone().into_waker()); let me = Box::into_raw(me) as usize; match self.arc.state.compare_exchange(1, me, SeqCst, SeqCst) { @@ -242,10 +242,10 @@ impl<'a, T: Unpin> DerefMut for BiLockGuard<'a, T> { impl<'a, T> BiLockGuard<'a, T> { /// Get a mutable pinned reference to the locked value. - pub fn as_pin_mut(&mut self) -> PinMut<'_, T> { + pub fn as_pin_mut(&mut self) -> Pin<&mut T> { // Safety: we never allow moving a !Unpin value out of a bilock, nor // allow mutable access to it - unsafe { PinMut::new_unchecked(&mut *self.bilock.arc.value.as_ref().unwrap().get()) } + unsafe { Pin::new_unchecked(&mut *self.bilock.arc.value.as_ref().unwrap().get()) } } } @@ -269,7 +269,7 @@ impl<'a, T> Unpin for BiLockAcquire<'a, T> {} impl<'a, T> Future for BiLockAcquire<'a, T> { type Output = BiLockGuard<'a, T>; - fn poll(self: PinMut, cx: &mut task::Context) -> Poll { - self.bilock.poll_lock(cx) + fn poll(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.bilock.poll_lock(lw) } } diff --git a/futures-util/src/sink/buffer.rs b/futures-util/src/sink/buffer.rs index 44d7736326..fced850026 100644 --- a/futures-util/src/sink/buffer.rs +++ b/futures-util/src/sink/buffer.rs @@ -1,10 +1,10 @@ use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::collections::VecDeque; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; /// Sink for the `Sink::buffer` combinator, which buffers up to some fixed /// number of values when the underlying sink is unable to accept them. @@ -40,16 +40,16 @@ impl Buffer { } fn try_empty_buffer( - self: &mut PinMut, - cx: &mut task::Context + self: &mut Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { - try_ready!(self.sink().poll_ready(cx)); + try_ready!(self.sink().poll_ready(lw)); while let Some(item) = self.buf().pop_front() { if let Err(e) = self.sink().start_send(item) { return Poll::Ready(Err(e)); } if !self.buf.is_empty() { - try_ready!(self.sink().poll_ready(cx)); + try_ready!(self.sink().poll_ready(lw)); } } Poll::Ready(Ok(())) @@ -60,8 +60,8 @@ impl Buffer { impl Stream for Buffer where S: Sink + Stream { type Item = S::Item; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { - self.sink().poll_next(cx) + fn poll_next(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + self.sink().poll_next(lw) } } @@ -70,14 +70,14 @@ impl Sink for Buffer { type SinkError = Si::SinkError; fn poll_ready( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { if *self.capacity() == 0 { - return self.sink().poll_ready(cx); + return self.sink().poll_ready(lw); } - if let Poll::Ready(Err(e)) = self.try_empty_buffer(cx) { + if let Poll::Ready(Err(e)) = self.try_empty_buffer(lw) { return Poll::Ready(Err(e)); } @@ -89,7 +89,7 @@ impl Sink for Buffer { } fn start_send( - mut self: PinMut, + mut self: Pin<&mut Self>, item: Self::SinkItem, ) -> Result<(), Self::SinkError> { if *self.capacity() == 0 { @@ -101,20 +101,20 @@ impl Sink for Buffer { } fn poll_flush( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - try_ready!(self.try_empty_buffer(cx)); + try_ready!(self.try_empty_buffer(lw)); debug_assert!(self.buf().is_empty()); - self.sink().poll_flush(cx) + self.sink().poll_flush(lw) } fn poll_close( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - try_ready!(self.try_empty_buffer(cx)); + try_ready!(self.try_empty_buffer(lw)); debug_assert!(self.buf().is_empty()); - self.sink().poll_close(cx) + self.sink().poll_close(lw) } } diff --git a/futures-util/src/sink/close.rs b/futures-util/src/sink/close.rs index e88cf2a0bc..30f582963e 100644 --- a/futures-util/src/sink/close.rs +++ b/futures-util/src/sink/close.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; /// Future for the `close` combinator, which polls the sink until all data has @@ -25,9 +25,9 @@ impl Future for Close<'_, Si> { type Output = Result<(), Si::SinkError>; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - PinMut::new(&mut self.sink).poll_close(cx) + Pin::new(&mut self.sink).poll_close(lw) } } diff --git a/futures-util/src/sink/drain.rs b/futures-util/src/sink/drain.rs index cbc56aa64e..e65ed45ed0 100644 --- a/futures-util/src/sink/drain.rs +++ b/futures-util/src/sink/drain.rs @@ -1,7 +1,7 @@ use core::fmt; use core::marker::PhantomData; -use core::pin::PinMut; -use futures_core::task::{self, Poll}; +use core::pin::Pin; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; /// A sink that will discard all items given to it. @@ -42,29 +42,29 @@ impl Sink for Drain { type SinkError = DrainError; fn poll_ready( - self: PinMut, - _cx: &mut task::Context, + self: Pin<&mut Self>, + _lw: &LocalWaker, ) -> Poll> { Poll::Ready(Ok(())) } fn start_send( - self: PinMut, + self: Pin<&mut Self>, _item: Self::SinkItem, ) -> Result<(), Self::SinkError> { Ok(()) } fn poll_flush( - self: PinMut, - _cx: &mut task::Context, + self: Pin<&mut Self>, + _lw: &LocalWaker, ) -> Poll> { Poll::Ready(Ok(())) } fn poll_close( - self: PinMut, - _cx: &mut task::Context, + self: Pin<&mut Self>, + _lw: &LocalWaker, ) -> Poll> { Poll::Ready(Ok(())) } diff --git a/futures-util/src/sink/err_into.rs b/futures-util/src/sink/err_into.rs index 4d17738f71..8205c72e74 100644 --- a/futures-util/src/sink/err_into.rs +++ b/futures-util/src/sink/err_into.rs @@ -1,7 +1,7 @@ use crate::sink::{SinkExt, SinkMapErr}; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::{Sink}; use pin_utils::unsafe_pinned; @@ -62,9 +62,9 @@ impl Stream for SinkErrInto type Item = S::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.sink().poll_next(cx) + self.sink().poll_next(lw) } } diff --git a/futures-util/src/sink/fanout.rs b/futures-util/src/sink/fanout.rs index 87a39f8f2c..6b9a8443f1 100644 --- a/futures-util/src/sink/fanout.rs +++ b/futures-util/src/sink/fanout.rs @@ -1,6 +1,6 @@ use core::fmt::{Debug, Formatter, Result as FmtResult}; -use core::pin::PinMut; -use futures_core::task::{self, Poll}; +use core::pin::Pin; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use pin_utils::unsafe_pinned; @@ -51,17 +51,17 @@ impl Sink for Fanout type SinkError = Si1::SinkError; fn poll_ready( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - let sink1_ready = try_poll!(self.sink1().poll_ready(cx)).is_ready(); - let sink2_ready = try_poll!(self.sink2().poll_ready(cx)).is_ready(); + let sink1_ready = try_poll!(self.sink1().poll_ready(lw)).is_ready(); + let sink2_ready = try_poll!(self.sink2().poll_ready(lw)).is_ready(); let ready = sink1_ready && sink2_ready; if ready { Poll::Ready(Ok(())) } else { Poll::Pending } } fn start_send( - mut self: PinMut, + mut self: Pin<&mut Self>, item: Self::SinkItem, ) -> Result<(), Self::SinkError> { self.sink1().start_send(item.clone())?; @@ -70,21 +70,21 @@ impl Sink for Fanout } fn poll_flush( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - let sink1_ready = try_poll!(self.sink1().poll_flush(cx)).is_ready(); - let sink2_ready = try_poll!(self.sink2().poll_flush(cx)).is_ready(); + let sink1_ready = try_poll!(self.sink1().poll_flush(lw)).is_ready(); + let sink2_ready = try_poll!(self.sink2().poll_flush(lw)).is_ready(); let ready = sink1_ready && sink2_ready; if ready { Poll::Ready(Ok(())) } else { Poll::Pending } } fn poll_close( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - let sink1_ready = try_poll!(self.sink1().poll_close(cx)).is_ready(); - let sink2_ready = try_poll!(self.sink2().poll_close(cx)).is_ready(); + let sink1_ready = try_poll!(self.sink1().poll_close(lw)).is_ready(); + let sink2_ready = try_poll!(self.sink2().poll_close(lw)).is_ready(); let ready = sink1_ready && sink2_ready; if ready { Poll::Ready(Ok(())) } else { Poll::Pending } } diff --git a/futures-util/src/sink/flush.rs b/futures-util/src/sink/flush.rs index 42ef6ef957..03209e1031 100644 --- a/futures-util/src/sink/flush.rs +++ b/futures-util/src/sink/flush.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; /// Future for the `flush` combinator, which polls the sink until all data @@ -31,9 +31,9 @@ impl Future for Flush<'_, Si> { type Output = Result<(), Si::SinkError>; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - PinMut::new(&mut self.sink).poll_flush(cx) + Pin::new(&mut self.sink).poll_flush(lw) } } diff --git a/futures-util/src/sink/map_err.rs b/futures-util/src/sink/map_err.rs index 15bc044647..7dab01eacb 100644 --- a/futures-util/src/sink/map_err.rs +++ b/futures-util/src/sink/map_err.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::{Sink}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -35,8 +35,8 @@ impl SinkMapErr { /// Get a pinned reference to the inner sink. #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 - pub fn get_pin_mut<'a>(self: PinMut<'a, Self>) -> PinMut<'a, Si> { - unsafe { PinMut::map_unchecked(self, |x| &mut x.sink) } + pub fn get_pin_mut<'a>(self: Pin<&'a mut Self>) -> Pin<&'a mut Si> { + unsafe { Pin::map_unchecked_mut(self, |x| &mut x.sink) } } /// Consumes this combinator, returning the underlying sink. @@ -47,7 +47,7 @@ impl SinkMapErr { self.sink } - fn take_f(mut self: PinMut) -> F { + fn take_f(mut self: Pin<&mut Self>) -> F { self.f().take().expect("polled MapErr after completion") } } @@ -60,15 +60,15 @@ impl Sink for SinkMapErr type SinkError = E; fn poll_ready( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 - self.sink().poll_ready(cx).map_err(|e| self.take_f()(e)) + self.sink().poll_ready(lw).map_err(|e| self.take_f()(e)) } fn start_send( - mut self: PinMut, + mut self: Pin<&mut Self>, item: Self::SinkItem, ) -> Result<(), Self::SinkError> { #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 @@ -76,19 +76,19 @@ impl Sink for SinkMapErr } fn poll_flush( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 - self.sink().poll_flush(cx).map_err(|e| self.take_f()(e)) + self.sink().poll_flush(lw).map_err(|e| self.take_f()(e)) } fn poll_close( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 - self.sink().poll_close(cx).map_err(|e| self.take_f()(e)) + self.sink().poll_close(lw).map_err(|e| self.take_f()(e)) } } @@ -96,9 +96,9 @@ impl Stream for SinkMapErr { type Item = S::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.sink().poll_next(cx) + self.sink().poll_next(lw) } } diff --git a/futures-util/src/sink/mod.rs b/futures-util/src/sink/mod.rs index 4cd1cf6567..5de042bf19 100644 --- a/futures-util/src/sink/mod.rs +++ b/futures-util/src/sink/mod.rs @@ -12,9 +12,6 @@ use futures_sink::Sink; #[cfg(feature = "compat")] use crate::compat::Compat; -#[cfg(feature = "compat")] -use futures_core::task::Spawn; - mod close; pub use self::close::Close; @@ -256,10 +253,9 @@ pub trait SinkExt: Sink { /// Wraps a [`Sink`] into a sink compatible with libraries using /// futures 0.1 `Sink`. Requires the `compat` feature to be enabled. #[cfg(feature = "compat")] - fn compat(self, spawn: Sp) -> Compat + fn compat(self) -> Compat where Self: Sized + Unpin, - Sp: Spawn, { - Compat::new(self, Some(spawn)) + Compat::new(self) } } diff --git a/futures-util/src/sink/send.rs b/futures-util/src/sink/send.rs index 3d09e3d9a4..535e95b5a7 100644 --- a/futures-util/src/sink/send.rs +++ b/futures-util/src/sink/send.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; /// Future for the `Sink::send` combinator, which sends a value to a sink and @@ -29,15 +29,15 @@ impl Future for Send<'_, Si> { type Output = Result<(), Si::SinkError>; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { let this = &mut *self; if let Some(item) = this.item.take() { - let mut sink = PinMut::new(this.sink); - match sink.reborrow().poll_ready(cx) { + let mut sink = Pin::new(&mut this.sink); + match sink.as_mut().poll_ready(lw) { Poll::Ready(Ok(())) => { - if let Err(e) = sink.reborrow().start_send(item) { + if let Err(e) = sink.as_mut().start_send(item) { return Poll::Ready(Err(e)); } } @@ -51,7 +51,7 @@ impl Future for Send<'_, Si> { // we're done sending the item, but want to block on flushing the // sink - try_ready!(PinMut::new(this.sink).poll_flush(cx)); + try_ready!(Pin::new(&mut this.sink).poll_flush(lw)); Poll::Ready(Ok(())) } diff --git a/futures-util/src/sink/send_all.rs b/futures-util/src/sink/send_all.rs index f6f57381c8..d78a4cab9c 100644 --- a/futures-util/src/sink/send_all.rs +++ b/futures-util/src/sink/send_all.rs @@ -1,9 +1,9 @@ use crate::stream::{StreamExt, Fuse}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; /// Future for the `Sink::send_all` combinator, which sends a stream of values @@ -45,13 +45,13 @@ where fn try_start_send( &mut self, - cx: &mut task::Context, + lw: &LocalWaker, item: Si::SinkItem, ) -> Poll> { debug_assert!(self.buffered.is_none()); - match PinMut::new(self.sink).poll_ready(cx) { + match Pin::new(&mut self.sink).poll_ready(lw) { Poll::Ready(Ok(())) => { - Poll::Ready(PinMut::new(self.sink).start_send(item)) + Poll::Ready(Pin::new(&mut self.sink).start_send(item)) } Poll::Ready(Err(e)) => Poll::Ready(Err(e)), Poll::Pending => { @@ -70,27 +70,27 @@ where type Output = Result<(), Si::SinkError>; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { let this = &mut *self; // If we've got an item buffered already, we need to write it to the // sink before we can do anything else if let Some(item) = this.buffered.take() { - try_ready!(this.try_start_send(cx, item)) + try_ready!(this.try_start_send(lw, item)) } loop { - match this.stream.poll_next_unpin(cx) { + match this.stream.poll_next_unpin(lw) { Poll::Ready(Some(item)) => { - try_ready!(this.try_start_send(cx, item)) + try_ready!(this.try_start_send(lw, item)) } Poll::Ready(None) => { - try_ready!(PinMut::new(this.sink).poll_flush(cx)); + try_ready!(Pin::new(&mut this.sink).poll_flush(lw)); return Poll::Ready(Ok(())) } Poll::Pending => { - try_ready!(PinMut::new(this.sink).poll_flush(cx)); + try_ready!(Pin::new(&mut this.sink).poll_flush(lw)); return Poll::Pending } } diff --git a/futures-util/src/sink/with.rs b/futures-util/src/sink/with.rs index dc8f06416e..0b22a4b635 100644 --- a/futures-util/src/sink/with.rs +++ b/futures-util/src/sink/with.rs @@ -1,9 +1,9 @@ use core::marker::{Unpin, PhantomData}; use core::mem; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -63,16 +63,16 @@ impl State { #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 #[allow(clippy::wrong_self_convention)] fn as_pin_mut<'a>( - self: PinMut<'a, Self>, - ) -> State, PinMut<'a, T>> { + self: Pin<&'a mut Self>, + ) -> State, Pin<&'a mut T>> { unsafe { - match PinMut::get_mut_unchecked(self) { + match Pin::get_mut_unchecked(self) { State::Empty => State::Empty, State::Process(fut) => - State::Process(PinMut::new_unchecked(fut)), + State::Process(Pin::new_unchecked(fut)), State::Buffered(item) => - State::Buffered(PinMut::new_unchecked(item)), + State::Buffered(Pin::new_unchecked(item)), } } } @@ -87,10 +87,10 @@ impl Stream for With type Item = S::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.sink().poll_next(cx) + self.sink().poll_next(lw) } } @@ -119,18 +119,18 @@ impl With } fn poll( - self: &mut PinMut, - cx: &mut task::Context, + self: &mut Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { let buffered = match self.state().as_pin_mut() { State::Empty => return Poll::Ready(Ok(())), - State::Process(fut) => Some(try_ready!(fut.poll(cx))), + State::Process(fut) => Some(try_ready!(fut.poll(lw))), State::Buffered(_) => None, }; if let Some(buffered) = buffered { - PinMut::set(self.state(), State::Buffered(buffered)); + Pin::set(self.state(), State::Buffered(buffered)); } - if let State::Buffered(item) = unsafe { mem::replace(PinMut::get_mut_unchecked(self.state()), State::Empty) } { + if let State::Buffered(item) = unsafe { mem::replace(Pin::get_mut_unchecked(self.state()), State::Empty) } { Poll::Ready(self.sink().start_send(item).map_err(Into::into)) } else { unreachable!() @@ -148,36 +148,36 @@ impl Sink for With type SinkError = E; fn poll_ready( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.poll(cx) + self.poll(lw) } fn start_send( - mut self: PinMut, + mut self: Pin<&mut Self>, item: Self::SinkItem, ) -> Result<(), Self::SinkError> { let item = (self.f())(item); - PinMut::set(self.state(), State::Process(item)); + Pin::set(self.state(), State::Process(item)); Ok(()) } fn poll_flush( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - try_ready!(self.poll(cx)); - try_ready!(self.sink().poll_flush(cx)); + try_ready!(self.poll(lw)); + try_ready!(self.sink().poll_flush(lw)); Poll::Ready(Ok(())) } fn poll_close( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - try_ready!(self.poll(cx)); - try_ready!(self.sink().poll_close(cx)); + try_ready!(self.poll(lw)); + try_ready!(self.sink().poll_close(lw)); Poll::Ready(Ok(())) } } diff --git a/futures-util/src/sink/with_flat_map.rs b/futures-util/src/sink/with_flat_map.rs index 15cb902377..b74f0d24ae 100644 --- a/futures-util/src/sink/with_flat_map.rs +++ b/futures-util/src/sink/with_flat_map.rs @@ -1,7 +1,7 @@ use core::marker::{Unpin, PhantomData}; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -62,8 +62,8 @@ where /// Get a pinned mutable reference to the inner sink. #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 - pub fn get_pin_mut<'a>(self: PinMut<'a, Self>) -> PinMut<'a, Si> { - unsafe { PinMut::map_unchecked(self, |x| &mut x.sink) } + pub fn get_pin_mut<'a>(self: Pin<&'a mut Self>) -> Pin<&'a mut Si> { + unsafe { Pin::map_unchecked_mut(self, |x| &mut x.sink) } } /// Consumes this combinator, returning the underlying sink. @@ -75,25 +75,25 @@ where } fn try_empty_stream( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { let WithFlatMap { sink, stream, buffer, .. } = - unsafe { PinMut::get_mut_unchecked(self) }; - let mut sink = unsafe { PinMut::new_unchecked(sink) }; - let mut stream = unsafe { PinMut::new_unchecked(stream) }; + unsafe { Pin::get_mut_unchecked(self) }; + let mut sink = unsafe { Pin::new_unchecked(sink) }; + let mut stream = unsafe { Pin::new_unchecked(stream) }; if buffer.is_some() { - try_ready!(sink.reborrow().poll_ready(cx)); + try_ready!(sink.as_mut().poll_ready(lw)); let item = buffer.take().unwrap(); - try_ready!(Poll::Ready(sink.reborrow().start_send(item))); + try_ready!(Poll::Ready(sink.as_mut().start_send(item))); } - if let Some(mut some_stream) = stream.reborrow().as_pin_mut() { - while let Some(x) = ready!(some_stream.reborrow().poll_next(cx)) { + if let Some(mut some_stream) = stream.as_mut().as_pin_mut() { + while let Some(x) = ready!(some_stream.as_mut().poll_next(lw)) { let item = try_ready!(Poll::Ready(x)); - match try_poll!(sink.reborrow().poll_ready(cx)) { + match try_poll!(sink.as_mut().poll_ready(lw)) { Poll::Ready(()) => { - try_poll!(Poll::Ready(sink.reborrow().start_send(item))) + try_poll!(Poll::Ready(sink.as_mut().start_send(item))) } Poll::Pending => { *buffer = Some(item); @@ -102,7 +102,7 @@ where }; } } - PinMut::set(stream, None); + Pin::set(stream, None); Poll::Ready(Ok(())) } } @@ -115,10 +115,10 @@ where { type Item = S::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.sink().poll_next(cx) + self.sink().poll_next(lw) } } @@ -132,40 +132,40 @@ where type SinkError = Si::SinkError; fn poll_ready( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.try_empty_stream(cx) + self.try_empty_stream(lw) } fn start_send( - mut self: PinMut, + mut self: Pin<&mut Self>, item: Self::SinkItem, ) -> Result<(), Self::SinkError> { assert!(self.stream().is_none()); let stream = (self.f())(item); - PinMut::set(self.stream(), Some(stream)); + Pin::set(self.stream(), Some(stream)); Ok(()) } fn poll_flush( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - match self.reborrow().try_empty_stream(cx) { + match self.as_mut().try_empty_stream(lw) { Poll::Pending => Poll::Pending, - Poll::Ready(Ok(())) => self.sink().poll_flush(cx), + Poll::Ready(Ok(())) => self.sink().poll_flush(lw), Poll::Ready(Err(e)) => Poll::Ready(Err(e)), } } fn poll_close( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - match self.reborrow().try_empty_stream(cx) { + match self.as_mut().try_empty_stream(lw) { Poll::Pending => Poll::Pending, - Poll::Ready(Ok(())) => self.sink().poll_close(cx), + Poll::Ready(Ok(())) => self.sink().poll_close(lw), Poll::Ready(Err(e)) => Poll::Ready(Err(e)), } } diff --git a/futures-util/src/stream/buffer_unordered.rs b/futures-util/src/stream/buffer_unordered.rs index 3455f2030f..76de06c36c 100644 --- a/futures-util/src/stream/buffer_unordered.rs +++ b/futures-util/src/stream/buffer_unordered.rs @@ -1,12 +1,12 @@ use crate::stream::{Fuse, FuturesUnordered}; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::fmt; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; /// An adaptor for a stream of futures to execute the futures concurrently, if /// possible, delivering results as they become available. @@ -86,8 +86,8 @@ where /// Note that care must be taken to avoid tampering with the state of the /// stream which may otherwise confuse this combinator. #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 - pub fn get_pin_mut<'a>(self: PinMut<'a, Self>) -> PinMut<'a, St> { - unsafe { PinMut::map_unchecked(self, |x| x.get_mut()) } + pub fn get_pin_mut<'a>(self: Pin<&'a mut Self>) -> Pin<&'a mut St> { + unsafe { Pin::map_unchecked_mut(self, |x| x.get_mut()) } } /// Consumes this combinator, returning the underlying stream. @@ -107,20 +107,20 @@ where type Item = ::Output; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { // First up, try to spawn off as many futures as possible by filling up // our slab of futures. while self.in_progress_queue.len() < self.max { - match self.stream().poll_next(cx) { + match self.stream().poll_next(lw) { Poll::Ready(Some(fut)) => self.in_progress_queue().push(fut), Poll::Ready(None) | Poll::Pending => break, } } // Attempt to pull the next value from the in_progress_queue - match PinMut::new(self.in_progress_queue()).poll_next(cx) { + match Pin::new(self.in_progress_queue()).poll_next(lw) { x @ Poll::Pending | x @ Poll::Ready(Some(_)) => return x, Poll::Ready(None) => {} } diff --git a/futures-util/src/stream/buffered.rs b/futures-util/src/stream/buffered.rs index a3bf487b4c..735499898c 100644 --- a/futures-util/src/stream/buffered.rs +++ b/futures-util/src/stream/buffered.rs @@ -1,12 +1,12 @@ use crate::stream::{Fuse, FuturesOrdered}; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::fmt; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; /// An adaptor for a stream of futures to execute the futures concurrently, if /// possible. @@ -82,8 +82,8 @@ where /// Note that care must be taken to avoid tampering with the state of the /// stream which may otherwise confuse this combinator. #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 - pub fn get_pin_mut<'a>(self: PinMut<'a, Self>) -> PinMut<'a, St> { - unsafe { PinMut::map_unchecked(self, |x| x.get_mut()) } + pub fn get_pin_mut<'a>(self: Pin<&'a mut Self>) -> Pin<&'a mut St> { + unsafe { Pin::map_unchecked_mut(self, |x| x.get_mut()) } } /// Consumes this combinator, returning the underlying stream. @@ -103,20 +103,20 @@ where type Item = ::Output; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { // Try to spawn off as many futures as possible by filling up // our in_progress_queue of futures. while self.in_progress_queue.len() < self.max { - match self.stream().poll_next(cx) { + match self.stream().poll_next(lw) { Poll::Ready(Some(fut)) => self.in_progress_queue().push(fut), Poll::Ready(None) | Poll::Pending => break, } } // Attempt to pull the next value from the in_progress_queue - let res = PinMut::new(self.in_progress_queue()).poll_next(cx); + let res = Pin::new(self.in_progress_queue()).poll_next(lw); if let Some(val) = ready!(res) { return Poll::Ready(Some(val)) } diff --git a/futures-util/src/stream/catch_unwind.rs b/futures-util/src/stream/catch_unwind.rs index 579fa34478..837c55cfa0 100644 --- a/futures-util/src/stream/catch_unwind.rs +++ b/futures-util/src/stream/catch_unwind.rs @@ -1,8 +1,8 @@ use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::any::Any; -use std::pin::PinMut; +use std::pin::Pin; use std::panic::{catch_unwind, UnwindSafe, AssertUnwindSafe}; use std::prelude::v1::*; @@ -30,14 +30,14 @@ impl Stream for CatchUnwind type Item = Result>; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { if *self.caught_unwind() { Poll::Ready(None) } else { let res = catch_unwind(AssertUnwindSafe(|| { - self.stream().poll_next(cx) + self.stream().poll_next(lw) })); match res { diff --git a/futures-util/src/stream/chain.rs b/futures-util/src/stream/chain.rs index 0da35f4ab4..c451c5eda7 100644 --- a/futures-util/src/stream/chain.rs +++ b/futures-util/src/stream/chain.rs @@ -1,6 +1,6 @@ -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// An adapter for chaining the output of two streams. @@ -14,7 +14,7 @@ pub struct Chain { second: St2, } -// All interactions with `PinMut>` happen through these methods +// All interactions with `Pin<&mut Chain<..>>` happen through these methods impl Chain where St1: Stream, St2: Stream, @@ -37,15 +37,15 @@ where St1: Stream, type Item = St1::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { if let Some(first) = self.first().as_pin_mut() { - if let Some(item) = ready!(first.poll_next(cx)) { + if let Some(item) = ready!(first.poll_next(lw)) { return Poll::Ready(Some(item)) } } - PinMut::set(self.first(), None); - self.second().poll_next(cx) + Pin::set(self.first(), None); + self.second().poll_next(lw) } } diff --git a/futures-util/src/stream/chunks.rs b/futures-util/src/stream/chunks.rs index 9aecc9542e..d79de2179e 100644 --- a/futures-util/src/stream/chunks.rs +++ b/futures-util/src/stream/chunks.rs @@ -1,10 +1,10 @@ use crate::stream::Fuse; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::marker::Unpin; use std::mem; -use std::pin::PinMut; +use std::pin::Pin; use std::prelude::v1::*; /// An adaptor that chunks up elements in a vector. @@ -34,7 +34,7 @@ impl Chunks where St: Stream { } } - fn take(mut self: PinMut) -> Vec { + fn take(mut self: Pin<&mut Self>) -> Vec { let cap = self.items().capacity(); mem::replace(self.items(), Vec::with_capacity(cap)) } @@ -67,12 +67,12 @@ impl Stream for Chunks { type Item = Vec; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { let cap = self.items.capacity(); loop { - match ready!(self.stream().poll_next(cx)) { + match ready!(self.stream().poll_next(lw)) { // Push the item into the buffer and check whether it is full. // If so, replace our buffer with a new and empty one and return // the full one. diff --git a/futures-util/src/stream/collect.rs b/futures-util/src/stream/collect.rs index 991d6fbb68..73be32e162 100644 --- a/futures-util/src/stream/collect.rs +++ b/futures-util/src/stream/collect.rs @@ -1,10 +1,10 @@ use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::marker::Unpin; use std::mem; -use std::pin::PinMut; +use std::pin::Pin; use std::prelude::v1::*; /// A future which collects all of the values of a stream into a vector. @@ -23,7 +23,7 @@ impl Collect { unsafe_pinned!(stream: St); unsafe_unpinned!(collection: C); - fn finish(mut self: PinMut) -> C { + fn finish(mut self: Pin<&mut Self>) -> C { mem::replace(self.collection(), Default::default()) } @@ -41,9 +41,9 @@ where St: Stream, { type Output = C; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { loop { - match ready!(self.stream().poll_next(cx)) { + match ready!(self.stream().poll_next(lw)) { Some(e) => self.collection().extend(Some(e)), None => return Poll::Ready(self.finish()), } diff --git a/futures-util/src/stream/concat.rs b/futures-util/src/stream/concat.rs index 39a7ac8278..202a0f6f30 100644 --- a/futures-util/src/stream/concat.rs +++ b/futures-util/src/stream/concat.rs @@ -1,10 +1,10 @@ use core::fmt::{Debug, Formatter, Result as FmtResult}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use core::default::Default; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator to concatenate the results of a stream into the first @@ -54,10 +54,10 @@ where St: Stream, type Output = St::Item; fn poll( - mut self: PinMut, cx: &mut task::Context + mut self: Pin<&mut Self>, lw: &LocalWaker ) -> Poll { loop { - match self.stream().poll_next(cx) { + match self.stream().poll_next(lw) { Poll::Pending => return Poll::Pending, Poll::Ready(None) => { return Poll::Ready(self.accum().take().unwrap_or_default()) diff --git a/futures-util/src/stream/disabled/select_all.rs b/futures-util/src/stream/disabled/select_all.rs index 5360402494..214aafa242 100644 --- a/futures-util/src/stream/disabled/select_all.rs +++ b/futures-util/src/stream/disabled/select_all.rs @@ -1,7 +1,7 @@ //! An unbounded set of streams use std::fmt::{self, Debug}; -use std::pin::PinMut; +use std::pin::Pin; use futures_core::{Poll, Stream}; use futures_core::task; @@ -69,9 +69,9 @@ impl Stream for SelectAll { fn poll_next( &mut self, - cx: &mut task::Context, + lw: &LocalWaker, ) -> Poll, Self::Error> { - match self.inner.poll_next(cx).map_err(|(err, _)| err)? { + match self.inner.poll_next(lw).map_err(|(err, _)| err)? { Async::Pending => Ok(Async::Pending), Async::Ready(Some((Some(item), remaining))) => { self.push(remaining); diff --git a/futures-util/src/stream/empty.rs b/futures-util/src/stream/empty.rs index 5571c64ff2..4ea9b4a78b 100644 --- a/futures-util/src/stream/empty.rs +++ b/futures-util/src/stream/empty.rs @@ -1,7 +1,7 @@ use core::marker::{Unpin, PhantomData}; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A stream which contains no elements. /// @@ -26,7 +26,7 @@ impl Unpin for Empty {} impl Stream for Empty { type Item = T; - fn poll_next(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_next(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(None) } } diff --git a/futures-util/src/stream/filter.rs b/futures-util/src/stream/filter.rs index 75489fc019..ccefe08b5a 100644 --- a/futures-util/src/stream/filter.rs +++ b/futures-util/src/stream/filter.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator used to filter the results of a stream and only yield @@ -79,22 +79,22 @@ impl Stream for Filter type Item = St::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { loop { if self.pending_fut().as_pin_mut().is_none() { - let item = match ready!(self.stream().poll_next(cx)) { + let item = match ready!(self.stream().poll_next(lw)) { Some(e) => e, None => return Poll::Ready(None), }; let fut = (self.f())(&item); - PinMut::set(self.pending_fut(), Some(fut)); + Pin::set(self.pending_fut(), Some(fut)); *self.pending_item() = Some(item); } - let yield_item = ready!(self.pending_fut().as_pin_mut().unwrap().poll(cx)); - PinMut::set(self.pending_fut(), None); + let yield_item = ready!(self.pending_fut().as_pin_mut().unwrap().poll(lw)); + Pin::set(self.pending_fut(), None); let item = self.pending_item().take().unwrap(); if yield_item { diff --git a/futures-util/src/stream/filter_map.rs b/futures-util/src/stream/filter_map.rs index acb642396d..0b904f4b39 100644 --- a/futures-util/src/stream/filter_map.rs +++ b/futures-util/src/stream/filter_map.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A combinator used to filter the results of a stream and simultaneously map @@ -72,21 +72,21 @@ impl Stream for FilterMap type Item = T; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { loop { if self.pending().as_pin_mut().is_none() { - let item = match ready!(self.stream().poll_next(cx)) { + let item = match ready!(self.stream().poll_next(lw)) { Some(e) => e, None => return Poll::Ready(None), }; let fut = (self.f())(item); - PinMut::set(self.pending(), Some(fut)); + Pin::set(self.pending(), Some(fut)); } - let item = ready!(self.pending().as_pin_mut().unwrap().poll(cx)); - PinMut::set(self.pending(), None); + let item = ready!(self.pending().as_pin_mut().unwrap().poll(lw)); + Pin::set(self.pending(), None); if item.is_some() { return Poll::Ready(item); } diff --git a/futures-util/src/stream/flatten.rs b/futures-util/src/stream/flatten.rs index cd70f72100..144a4f1669 100644 --- a/futures-util/src/stream/flatten.rs +++ b/futures-util/src/stream/flatten.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// A combinator used to flatten a stream-of-streams into one long stream of @@ -64,21 +64,21 @@ impl Stream for Flatten type Item = ::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { loop { if self.next().as_pin_mut().is_none() { - match ready!(self.stream().poll_next(cx)) { - Some(e) => PinMut::set(self.next(), Some(e)), + match ready!(self.stream().poll_next(lw)) { + Some(e) => Pin::set(self.next(), Some(e)), None => return Poll::Ready(None), } } - let item = ready!(self.next().as_pin_mut().unwrap().poll_next(cx)); + let item = ready!(self.next().as_pin_mut().unwrap().poll_next(lw)); if item.is_some() { return Poll::Ready(item); } else { - PinMut::set(self.next(), None); + Pin::set(self.next(), None); } } } diff --git a/futures-util/src/stream/fold.rs b/futures-util/src/stream/fold.rs index bb2e8fb974..cb3135b7ea 100644 --- a/futures-util/src/stream/fold.rs +++ b/futures-util/src/stream/fold.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A future used to collect all the results of a stream into one generic type. @@ -46,22 +46,22 @@ impl Future for Fold { type Output = T; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { loop { // we're currently processing a future to produce a new accum value if self.accum().is_none() { - let accum = ready!(self.future().as_pin_mut().unwrap().poll(cx)); + let accum = ready!(self.future().as_pin_mut().unwrap().poll(lw)); *self.accum() = Some(accum); - PinMut::set(self.future(), None); + Pin::set(self.future(), None); } - let item = ready!(self.stream().poll_next(cx)); + let item = ready!(self.stream().poll_next(lw)); let accum = self.accum().take() .expect("Fold polled after completion"); if let Some(e) = item { let future = (self.f())(accum, e); - PinMut::set(self.future(), Some(future)); + Pin::set(self.future(), Some(future)); } else { return Poll::Ready(accum) } diff --git a/futures-util/src/stream/for_each.rs b/futures-util/src/stream/for_each.rs index a7296b8aa9..efc6425d35 100644 --- a/futures-util/src/stream/for_each.rs +++ b/futures-util/src/stream/for_each.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which executes a unit closure over each item on a @@ -48,17 +48,17 @@ impl Future for ForEach { type Output = (); - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll<()> { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll<()> { loop { if let Some(future) = self.future().as_pin_mut() { - ready!(future.poll(cx)); + ready!(future.poll(lw)); } - PinMut::set(self.future(), None); + Pin::set(self.future(), None); - match ready!(self.stream().poll_next(cx)) { + match ready!(self.stream().poll_next(lw)) { Some(e) => { let future = (self.f())(e); - PinMut::set(self.future(), Some(future)); + Pin::set(self.future(), Some(future)); } None => { return Poll::Ready(()); diff --git a/futures-util/src/stream/for_each_concurrent.rs b/futures-util/src/stream/for_each_concurrent.rs index 5363a9e526..0680643fa0 100644 --- a/futures-util/src/stream/for_each_concurrent.rs +++ b/futures-util/src/stream/for_each_concurrent.rs @@ -1,10 +1,10 @@ use crate::stream::{FuturesUnordered, StreamExt}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use core::num::NonZeroUsize; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which executes a unit closure over each item on a @@ -55,7 +55,7 @@ impl Future for ForEachConcurrent { type Output = (); - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll<()> { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll<()> { loop { let mut made_progress_this_iter = false; @@ -65,7 +65,7 @@ impl Future for ForEachConcurrent if self.limit().map(|limit| limit.get() > current_len).unwrap_or(true) { let mut stream_completed = false; let elem = if let Some(stream) = self.stream().as_pin_mut() { - match stream.poll_next(cx) { + match stream.poll_next(lw) { Poll::Ready(Some(elem)) => { made_progress_this_iter = true; Some(elem) @@ -80,7 +80,7 @@ impl Future for ForEachConcurrent None }; if stream_completed { - PinMut::set(self.stream(), None); + Pin::set(self.stream(), None); } if let Some(elem) = elem { let next_future = (self.f())(elem); @@ -88,7 +88,7 @@ impl Future for ForEachConcurrent } } - match self.futures().poll_next_unpin(cx) { + match self.futures().poll_next_unpin(lw) { Poll::Ready(Some(())) => made_progress_this_iter = true, Poll::Ready(None) => { if self.stream().is_none() { diff --git a/futures-util/src/stream/forward.rs b/futures-util/src/stream/forward.rs index ab6bc32fb9..f4058f756d 100644 --- a/futures-util/src/stream/forward.rs +++ b/futures-util/src/stream/forward.rs @@ -1,9 +1,9 @@ use crate::stream::{StreamExt, Fuse}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -45,14 +45,14 @@ where } fn try_start_send( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, item: Si::SinkItem, ) -> Poll> { debug_assert!(self.buffered_item.is_none()); { let mut sink = self.sink().as_pin_mut().unwrap(); - if try_poll!(sink.reborrow().poll_ready(cx)).is_ready() { + if try_poll!(sink.as_mut().poll_ready(lw)).is_ready() { return Poll::Ready(sink.start_send(item)); } } @@ -69,28 +69,28 @@ where type Output = Result; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { // If we've got an item buffered already, we need to write it to the // sink before we can do anything else if let Some(item) = self.buffered_item().take() { - try_ready!(self.reborrow().try_start_send(cx, item)); + try_ready!(self.as_mut().try_start_send(lw, item)); } loop { - match self.stream().poll_next(cx) { + match self.stream().poll_next(lw) { Poll::Ready(Some(Ok(item))) => - try_ready!(self.reborrow().try_start_send(cx, item)), + try_ready!(self.as_mut().try_start_send(lw, item)), Poll::Ready(Some(Err(e))) => return Poll::Ready(Err(e)), Poll::Ready(None) => { try_ready!(self.sink().as_pin_mut().expect(INVALID_POLL) - .poll_close(cx)); + .poll_close(lw)); return Poll::Ready(Ok(self.sink().take().unwrap())) } Poll::Pending => { try_ready!(self.sink().as_pin_mut().expect(INVALID_POLL) - .poll_flush(cx)); + .poll_flush(lw)); return Poll::Pending } } diff --git a/futures-util/src/stream/fuse.rs b/futures-util/src/stream/fuse.rs index 3550d83b41..aec1f4c901 100644 --- a/futures-util/src/stream/fuse.rs +++ b/futures-util/src/stream/fuse.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -57,8 +57,8 @@ impl Fuse { /// Note that care must be taken to avoid tampering with the state of the /// stream which may otherwise confuse this combinator. #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 - pub fn get_pin_mut<'a>(self: PinMut<'a, Self>) -> PinMut<'a, St> { - unsafe { PinMut::map_unchecked(self, |x| x.get_mut()) } + pub fn get_pin_mut<'a>(self: Pin<&'a mut Self>) -> Pin<&'a mut St> { + unsafe { Pin::map_unchecked_mut(self, |x| x.get_mut()) } } /// Consumes this combinator, returning the underlying stream. @@ -74,14 +74,14 @@ impl Stream for Fuse { type Item = S::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { if *self.done() { return Poll::Ready(None); } - let item = ready!(self.stream().poll_next(cx)); + let item = ready!(self.stream().poll_next(lw)); if item.is_none() { *self.done() = true; } diff --git a/futures-util/src/stream/futures_ordered.rs b/futures-util/src/stream/futures_ordered.rs index 857577e627..2915a9e703 100644 --- a/futures-util/src/stream/futures_ordered.rs +++ b/futures-util/src/stream/futures_ordered.rs @@ -1,14 +1,14 @@ use crate::stream::FuturesUnordered; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; use std::cmp::{Eq, PartialEq, PartialOrd, Ord, Ordering}; use std::collections::binary_heap::{BinaryHeap, PeekMut}; use std::fmt::{self, Debug}; use std::iter::FromIterator; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; #[must_use = "futures do nothing unless polled"] #[derive(Debug)] @@ -48,10 +48,10 @@ impl Future for OrderWrapper type Output = OrderWrapper; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - self.data().poll(cx) + self.data().poll(lw) .map(|output| OrderWrapper { data: output, index: self.index }) } } @@ -171,8 +171,8 @@ impl Stream for FuturesOrdered { type Item = Fut::Output; fn poll_next( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { let this = &mut *self; @@ -185,7 +185,7 @@ impl Stream for FuturesOrdered { } loop { - match PinMut::new(&mut this.in_progress_queue).poll_next(cx) { + match Pin::new(&mut this.in_progress_queue).poll_next(lw) { Poll::Ready(Some(output)) => { if output.index == this.next_outgoing_index { this.next_outgoing_index += 1; diff --git a/futures-util/src/stream/futures_unordered/iter.rs b/futures-util/src/stream/futures_unordered/iter.rs index 53ca3341f2..7f467c062f 100644 --- a/futures-util/src/stream/futures_unordered/iter.rs +++ b/futures-util/src/stream/futures_unordered/iter.rs @@ -1,7 +1,7 @@ use super::FuturesUnordered; use super::task::Task; use std::marker::{PhantomData, Unpin}; -use std::pin::PinMut; +use std::pin::Pin; #[derive(Debug)] /// Mutable iterator over all futures in the unordered set. @@ -16,9 +16,9 @@ pub struct IterPinMut<'a, Fut: 'a> { pub struct IterMut<'a, Fut: Unpin + 'a> (pub(super) IterPinMut<'a, Fut>); impl<'a, Fut> Iterator for IterPinMut<'a, Fut> { - type Item = PinMut<'a, Fut>; + type Item = Pin<&'a mut Fut>; - fn next(&mut self) -> Option> { + fn next(&mut self) -> Option> { if self.task.is_null() { return None; } @@ -27,7 +27,7 @@ impl<'a, Fut> Iterator for IterPinMut<'a, Fut> { let next = *(*self.task).next_all.get(); self.task = next; self.len -= 1; - Some(PinMut::new_unchecked(future)) + Some(Pin::new_unchecked(future)) } } @@ -42,7 +42,7 @@ impl<'a, Fut: Unpin> Iterator for IterMut<'a, Fut> { type Item = &'a mut Fut; fn next(&mut self) -> Option<&'a mut Fut> { - self.0.next().map(PinMut::get_mut) + self.0.next().map(Pin::get_mut) } fn size_hint(&self) -> (usize, Option) { diff --git a/futures-util/src/stream/futures_unordered/mod.rs b/futures-util/src/stream/futures_unordered/mod.rs index b1102ca59f..8330a1a393 100644 --- a/futures-util/src/stream/futures_unordered/mod.rs +++ b/futures-util/src/stream/futures_unordered/mod.rs @@ -1,15 +1,15 @@ //! An unbounded set of futures. use crate::task::AtomicWaker; -use futures_core::future::Future; +use futures_core::future::{Future, FutureObj, LocalFutureObj}; use futures_core::stream::Stream; -use futures_core::task::{self as core_task, Poll}; +use futures_core::task::{LocalWaker, Poll, Spawn, LocalSpawn, SpawnError}; use std::cell::UnsafeCell; use std::fmt::{self, Debug}; use std::iter::FromIterator; use std::marker::{PhantomData, Unpin}; use std::mem; -use std::pin::PinMut; +use std::pin::Pin; use std::ptr; use std::sync::atomic::Ordering::SeqCst; use std::sync::atomic::{AtomicPtr, AtomicBool}; @@ -55,6 +55,24 @@ unsafe impl Send for FuturesUnordered {} unsafe impl Sync for FuturesUnordered {} impl Unpin for FuturesUnordered {} +impl<'a> Spawn for FuturesUnordered> { + fn spawn_obj(&mut self, future_obj: FutureObj<'static, ()>) + -> Result<(), SpawnError> + { + self.push(future_obj); + Ok(()) + } +} + +impl<'a> LocalSpawn for FuturesUnordered> { + fn spawn_local_obj(&mut self, future_obj: LocalFutureObj<'static, ()>) + -> Result<(), SpawnError> + { + self.push(future_obj); + Ok(()) + } +} + // FuturesUnordered is implemented using two linked lists. One which links all // futures managed by a `FuturesUnordered` and one that tracks futures that have // been scheduled for polling. The first linked list is not thread safe and is @@ -158,12 +176,12 @@ impl FuturesUnordered { /// Returns an iterator that allows modifying each future in the set. pub fn iter_mut(&mut self) -> IterMut where Fut: Unpin { - IterMut(PinMut::new(self).iter_pin_mut()) + IterMut(Pin::new(self).iter_pin_mut()) } /// Returns an iterator that allows modifying each future in the set. #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 - pub fn iter_pin_mut<'a>(self: PinMut<'a, Self>) -> IterPinMut<'a, Fut> { + pub fn iter_pin_mut<'a>(self: Pin<&'a mut Self>) -> IterPinMut<'a, Fut> { IterPinMut { task: self.head_all, len: self.len, @@ -254,11 +272,11 @@ impl FuturesUnordered { impl Stream for FuturesUnordered { type Item = Fut::Output; - fn poll_next(mut self: PinMut, cx: &mut core_task::Context) + fn poll_next(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { // Ensure `parent` is correctly set. - self.ready_to_run_queue.waker.register(cx.waker()); + self.ready_to_run_queue.waker.register(lw); loop { // Safety: &mut self guarantees the mutual exclusion `dequeue` @@ -275,7 +293,7 @@ impl Stream for FuturesUnordered { // At this point, it may be worth yielding the thread & // spinning a few times... but for now, just yield using the // task system. - cx.local_waker().wake(); + lw.wake(); return Poll::Pending; } Dequeue::Data(task) => task, @@ -366,13 +384,12 @@ impl Stream for FuturesUnordered { // the internal allocation, appropriately accessing fields and // deallocating the task if need be. let res = { - let local_waker = bomb.task.as_ref().unwrap().local_waker(); - let mut cx = cx.with_waker(&*local_waker); + let lw = bomb.task.as_ref().unwrap().local_waker(); // Safety: We won't move the future ever again - let future = unsafe { PinMut::new_unchecked(future) }; + let future = unsafe { Pin::new_unchecked(future) }; - future.poll(&mut cx) + future.poll(&lw) }; match res { diff --git a/futures-util/src/stream/inspect.rs b/futures-util/src/stream/inspect.rs index aee6bd00b7..13ba1be3ed 100644 --- a/futures-util/src/stream/inspect.rs +++ b/futures-util/src/stream/inspect.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Do something with the items of a stream, passing it on. @@ -58,10 +58,10 @@ impl Stream for Inspect type Item = St::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { - let item = ready!(self.stream().poll_next(cx)); + let item = ready!(self.stream().poll_next(lw)); Poll::Ready(item.map(|e| { (self.f())(&e); e diff --git a/futures-util/src/stream/into_future.rs b/futures-util/src/stream/into_future.rs index a864fc3ab8..9a7bd15756 100644 --- a/futures-util/src/stream/into_future.rs +++ b/futures-util/src/stream/into_future.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A combinator used to temporarily convert a stream into a future. /// @@ -62,12 +62,12 @@ impl Future for StreamFuture { type Output = (Option, St); fn poll( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll { let item = { let s = self.stream.as_mut().expect("polling StreamFuture twice"); - ready!(PinMut::new(s).poll_next(cx)) + ready!(Pin::new(s).poll_next(lw)) }; let stream = self.stream.take().unwrap(); Poll::Ready((item, stream)) diff --git a/futures-util/src/stream/iter.rs b/futures-util/src/stream/iter.rs index 3fe81d59e6..9b293adb62 100644 --- a/futures-util/src/stream/iter.rs +++ b/futures-util/src/stream/iter.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A stream which is just a shim over an underlying instance of `Iterator`. /// @@ -40,7 +40,7 @@ impl Stream for Iter { type Item = I::Item; - fn poll_next(mut self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(self.iter.next()) } } diff --git a/futures-util/src/stream/map.rs b/futures-util/src/stream/map.rs index 9fd2ef2406..7c40e54fa9 100644 --- a/futures-util/src/stream/map.rs +++ b/futures-util/src/stream/map.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which will change the type of a stream from one @@ -59,10 +59,10 @@ impl Stream for Map type Item = T; fn poll_next( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { - let option = ready!(self.stream().poll_next(cx)); + let option = ready!(self.stream().poll_next(lw)); Poll::Ready(option.map(self.f())) } } diff --git a/futures-util/src/stream/mod.rs b/futures-util/src/stream/mod.rs index e524fc0503..53859ab3a0 100644 --- a/futures-util/src/stream/mod.rs +++ b/futures-util/src/stream/mod.rs @@ -4,11 +4,11 @@ //! including the `StreamExt` trait which adds methods to `Stream` types. use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use either::Either; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; mod iter; @@ -95,7 +95,6 @@ pub use self::zip::Zip; if_std! { use std; use std::iter::Extend; - use std::pin::PinBox; mod buffer_unordered; pub use self::buffer_unordered::BufferUnordered; @@ -139,8 +138,9 @@ pub trait StreamExt: Stream { /// Note that because `next` doesn't take ownership over the stream, /// the [`Stream`] type must be [`Unpin`]. If you want to use `next` with a /// [`!Unpin`](Unpin) stream, you'll first have to pin the stream. This can - /// be done by wrapping the stream in a [`PinBox`](std::boxed::PinBox) or - /// pinning it to the stack using the `pin_mut!` macro. + /// be done by boxing the stream using [`Box::pinned`] or + /// pinning it to the stack using the `pin_mut!` macro from the `pin_utils` + /// crate. /// /// # Examples /// @@ -171,8 +171,9 @@ pub trait StreamExt: Stream { /// Note that because `into_future` moves the stream, the [`Stream`] type /// must be [`Unpin`]. If you want to use `into_future` with a /// [`!Unpin`](Unpin) stream, you'll first have to pin the stream. This can - /// be done by wrapping the stream in a [`PinBox`](std::boxed::PinBox) or - /// pinning it to the stack using the `pin_mut!` macro. + /// be done by boxing the stream using [`Box::pinned`] or + /// pinning it to the stack using the `pin_mut!` macro from the `pin_utils` + /// crate. /// /// # Examples /// @@ -779,10 +780,10 @@ pub trait StreamExt: Stream { /// Wrap the stream in a Box, pinning it. #[cfg(feature = "std")] - fn boxed(self) -> PinBox + fn boxed(self) -> Pin> where Self: Sized { - PinBox::new(self) + Box::pinned(self) } /// An adaptor for creating a buffered list of pending futures. @@ -1033,10 +1034,10 @@ pub trait StreamExt: Stream { /// stream types. fn poll_next_unpin( &mut self, - cx: &mut task::Context + lw: &LocalWaker ) -> Poll> where Self: Unpin + Sized { - PinMut::new(self).poll_next(cx) + Pin::new(self).poll_next(lw) } } diff --git a/futures-util/src/stream/next.rs b/futures-util/src/stream/next.rs index 0d9537e8cf..7c2dc5b880 100644 --- a/futures-util/src/stream/next.rs +++ b/futures-util/src/stream/next.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A future of the next element of a stream. #[derive(Debug)] @@ -23,9 +23,9 @@ impl Future for Next<'_, St> { type Output = Option; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - PinMut::new(&mut *self.stream).poll_next(cx) + Pin::new(&mut *self.stream).poll_next(lw) } } diff --git a/futures-util/src/stream/once.rs b/futures-util/src/stream/once.rs index 8139678905..a7a34ea2a5 100644 --- a/futures-util/src/stream/once.rs +++ b/futures-util/src/stream/once.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Creates a stream of single element @@ -39,15 +39,15 @@ impl Stream for Once { type Item = Fut::Output; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { let val = if let Some(f) = self.future().as_pin_mut() { - ready!(f.poll(cx)) + ready!(f.poll(lw)) } else { return Poll::Ready(None) }; - PinMut::set(self.future(), None); + Pin::set(self.future(), None); Poll::Ready(Some(val)) } } diff --git a/futures-util/src/stream/peek.rs b/futures-util/src/stream/peek.rs index 4cc1380ec6..80626af9fc 100644 --- a/futures-util/src/stream/peek.rs +++ b/futures-util/src/stream/peek.rs @@ -1,8 +1,8 @@ use crate::stream::{StreamExt, Fuse}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A `Stream` that implements a `peek` method. @@ -35,13 +35,13 @@ impl Peekable { /// This method polls the underlying stream and return either a reference /// to the next item if the stream is ready or passes through any errors. pub fn peek<'a>( - self: &'a mut PinMut, - cx: &mut task::Context, + self: &'a mut Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { if self.peeked().is_some() { return Poll::Ready(self.peeked().as_ref()) } - match ready!(self.stream().poll_next(cx)) { + match ready!(self.stream().poll_next(lw)) { None => Poll::Ready(None), Some(item) => { *self.peeked() = Some(item); @@ -55,13 +55,13 @@ impl Stream for Peekable { type Item = S::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { if let Some(item) = self.peeked().take() { return Poll::Ready(Some(item)) } - self.stream().poll_next(cx) + self.stream().poll_next(lw) } } diff --git a/futures-util/src/stream/poll_fn.rs b/futures-util/src/stream/poll_fn.rs index e327a3c176..bb4e2231c6 100644 --- a/futures-util/src/stream/poll_fn.rs +++ b/futures-util/src/stream/poll_fn.rs @@ -1,9 +1,9 @@ //! Definition of the `PollFn` combinator use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// A stream which adapts a function returning `Poll`. /// @@ -37,18 +37,18 @@ impl Unpin for PollFn {} /// ``` pub fn poll_fn(f: F) -> PollFn where - F: FnMut(&mut task::Context) -> Poll>, + F: FnMut(&LocalWaker) -> Poll>, { PollFn { f } } impl Stream for PollFn where - F: FnMut(&mut task::Context) -> Poll>, + F: FnMut(&LocalWaker) -> Poll>, { type Item = T; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { - (&mut self.f)(cx) + fn poll_next(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + (&mut self.f)(lw) } } diff --git a/futures-util/src/stream/repeat.rs b/futures-util/src/stream/repeat.rs index ab8245eab5..5d9e00f6d3 100644 --- a/futures-util/src/stream/repeat.rs +++ b/futures-util/src/stream/repeat.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// Stream that produces the same element repeatedly. /// @@ -38,7 +38,7 @@ impl Stream for Repeat { type Item = T; - fn poll_next(self: PinMut, _: &mut task::Context) -> Poll> { + fn poll_next(self: Pin<&mut Self>, _: &LocalWaker) -> Poll> { Poll::Ready(Some(self.item.clone())) } } diff --git a/futures-util/src/stream/select.rs b/futures-util/src/stream/select.rs index 566db8ec15..d98ece3ae9 100644 --- a/futures-util/src/stream/select.rs +++ b/futures-util/src/stream/select.rs @@ -1,8 +1,8 @@ use crate::stream::{StreamExt, Fuse}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; /// An adapter for merging the output of two streams. /// @@ -43,37 +43,37 @@ impl Stream for Select type Item = St1::Item; fn poll_next( - self: PinMut, - cx: &mut task::Context + self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { let Select { flag, stream1, stream2 } = - unsafe { PinMut::get_mut_unchecked(self) }; - let stream1 = unsafe { PinMut::new_unchecked(stream1) }; - let stream2 = unsafe { PinMut::new_unchecked(stream2) }; + unsafe { Pin::get_mut_unchecked(self) }; + let stream1 = unsafe { Pin::new_unchecked(stream1) }; + let stream2 = unsafe { Pin::new_unchecked(stream2) }; if *flag { - poll_inner(flag, stream1, stream2, cx) + poll_inner(flag, stream1, stream2, lw) } else { - poll_inner(flag, stream2, stream1, cx) + poll_inner(flag, stream2, stream1, lw) } } } fn poll_inner( flag: &mut bool, - a: PinMut, - b: PinMut, - cx: &mut task::Context + a: Pin<&mut St1>, + b: Pin<&mut St2>, + lw: &LocalWaker ) -> Poll> where St1: Stream, St2: Stream { - let a_done = match a.poll_next(cx) { + let a_done = match a.poll_next(lw) { Poll::Ready(Some(item)) => return Poll::Ready(Some(item)), Poll::Ready(None) => true, Poll::Pending => false, }; - match b.poll_next(cx) { + match b.poll_next(lw) { Poll::Ready(Some(item)) => { // If the other stream isn't finished yet, give them a chance to // go first next time as we pulled something off `b`. diff --git a/futures-util/src/stream/skip.rs b/futures-util/src/stream/skip.rs index 8ae14036e7..df522783b9 100644 --- a/futures-util/src/stream/skip.rs +++ b/futures-util/src/stream/skip.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which skips a number of elements before continuing. @@ -55,17 +55,17 @@ impl Stream for Skip { type Item = St::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { while *self.remaining() > 0 { - match ready!(self.stream().poll_next(cx)) { + match ready!(self.stream().poll_next(lw)) { Some(_) => *self.remaining() -= 1, None => return Poll::Ready(None), } } - self.stream().poll_next(cx) + self.stream().poll_next(lw) } } diff --git a/futures-util/src/stream/skip_while.rs b/futures-util/src/stream/skip_while.rs index 90ee4ece3f..1c2c61ad62 100644 --- a/futures-util/src/stream/skip_while.rs +++ b/futures-util/src/stream/skip_while.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which skips elements of a stream while a predicate @@ -74,27 +74,27 @@ impl Stream for SkipWhile type Item = St::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { if *self.done_skipping() { - return self.stream().poll_next(cx); + return self.stream().poll_next(lw); } loop { if self.pending_item().is_none() { - let item = match ready!(self.stream().poll_next(cx)) { + let item = match ready!(self.stream().poll_next(lw)) { Some(e) => e, None => return Poll::Ready(None), }; let fut = (self.f())(&item); - PinMut::set(self.pending_fut(), Some(fut)); + Pin::set(self.pending_fut(), Some(fut)); *self.pending_item() = Some(item); } - let skipped = ready!(self.pending_fut().as_pin_mut().unwrap().poll(cx)); + let skipped = ready!(self.pending_fut().as_pin_mut().unwrap().poll(lw)); let item = self.pending_item().take().unwrap(); - PinMut::set(self.pending_fut(), None); + Pin::set(self.pending_fut(), None); if !skipped { *self.done_skipping() = true; diff --git a/futures-util/src/stream/split.rs b/futures-util/src/stream/split.rs index 10d71fc477..ef82a9d8be 100644 --- a/futures-util/src/stream/split.rs +++ b/futures-util/src/stream/split.rs @@ -1,11 +1,11 @@ use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; use std::any::Any; use std::error::Error; use std::fmt; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; use crate::lock::BiLock; @@ -28,9 +28,9 @@ impl SplitStream { impl Stream for SplitStream { type Item = S::Item; - fn poll_next(self: PinMut, cx: &mut task::Context) -> Poll> { - match self.0.poll_lock(cx) { - Poll::Ready(mut inner) => inner.as_pin_mut().poll_next(cx), + fn poll_next(self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { + match self.0.poll_lock(lw) { + Poll::Ready(mut inner) => inner.as_pin_mut().poll_next(lw), Poll::Pending => Poll::Pending, } } @@ -68,47 +68,47 @@ impl Sink for SplitSink { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn poll_ready(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_ready(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { loop { if self.slot.is_none() { return Poll::Ready(Ok(())); } - try_ready!(self.reborrow().poll_flush(cx)); + try_ready!(self.as_mut().poll_flush(lw)); } } - fn start_send(mut self: PinMut, item: S::SinkItem) -> Result<(), S::SinkError> { + fn start_send(mut self: Pin<&mut Self>, item: S::SinkItem) -> Result<(), S::SinkError> { self.slot = Some(item); Ok(()) } - fn poll_flush(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_flush(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { let this = &mut *self; - match this.lock.poll_lock(cx) { + match this.lock.poll_lock(lw) { Poll::Ready(mut inner) => { if this.slot.is_some() { - try_ready!(inner.as_pin_mut().poll_ready(cx)); + try_ready!(inner.as_pin_mut().poll_ready(lw)); if let Err(e) = inner.as_pin_mut().start_send(this.slot.take().unwrap()) { return Poll::Ready(Err(e)); } } - inner.as_pin_mut().poll_flush(cx) + inner.as_pin_mut().poll_flush(lw) } Poll::Pending => Poll::Pending, } } - fn poll_close(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll> { let this = &mut *self; - match this.lock.poll_lock(cx) { + match this.lock.poll_lock(lw) { Poll::Ready(mut inner) => { if this.slot.is_some() { - try_ready!(inner.as_pin_mut().poll_ready(cx)); + try_ready!(inner.as_pin_mut().poll_ready(lw)); if let Err(e) = inner.as_pin_mut().start_send(this.slot.take().unwrap()) { return Poll::Ready(Err(e)); } } - inner.as_pin_mut().poll_close(cx) + inner.as_pin_mut().poll_close(lw) } Poll::Pending => Poll::Pending, } diff --git a/futures-util/src/stream/take.rs b/futures-util/src/stream/take.rs index d0e46ea1af..6ad7f19866 100644 --- a/futures-util/src/stream/take.rs +++ b/futures-util/src/stream/take.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which returns a maximum number of elements. @@ -57,13 +57,13 @@ impl Stream for Take type Item = St::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { if *self.remaining() == 0 { Poll::Ready(None) } else { - let next = ready!(self.stream().poll_next(cx)); + let next = ready!(self.stream().poll_next(lw)); match next { Some(_) => *self.remaining() -= 1, None => *self.remaining() = 0, diff --git a/futures-util/src/stream/take_while.rs b/futures-util/src/stream/take_while.rs index a7939ce652..12e2974ac4 100644 --- a/futures-util/src/stream/take_while.rs +++ b/futures-util/src/stream/take_while.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which takes elements from a stream while a predicate @@ -74,25 +74,25 @@ impl Stream for TakeWhile type Item = St::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { if *self.done_taking() { return Poll::Ready(None); } if self.pending_item().is_none() { - let item = match ready!(self.stream().poll_next(cx)) { + let item = match ready!(self.stream().poll_next(lw)) { Some(e) => e, None => return Poll::Ready(None), }; let fut = (self.f())(&item); - PinMut::set(self.pending_fut(), Some(fut)); + Pin::set(self.pending_fut(), Some(fut)); *self.pending_item() = Some(item); } - let take = ready!(self.pending_fut().as_pin_mut().unwrap().poll(cx)); - PinMut::set(self.pending_fut(), None); + let take = ready!(self.pending_fut().as_pin_mut().unwrap().poll(lw)); + Pin::set(self.pending_fut(), None); let item = self.pending_item().take().unwrap(); if take { diff --git a/futures-util/src/stream/then.rs b/futures-util/src/stream/then.rs index f9438cea5c..edee04ba63 100644 --- a/futures-util/src/stream/then.rs +++ b/futures-util/src/stream/then.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which chains a computation onto each item produced by a @@ -44,20 +44,20 @@ impl Stream for Then type Item = Fut::Output; fn poll_next( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { if self.future().as_pin_mut().is_none() { - let item = match ready!(self.stream().poll_next(cx)) { + let item = match ready!(self.stream().poll_next(lw)) { None => return Poll::Ready(None), Some(e) => e, }; let fut = (self.f())(item); - PinMut::set(self.future(), Some(fut)); + Pin::set(self.future(), Some(fut)); } - let e = ready!(self.future().as_pin_mut().unwrap().poll(cx)); - PinMut::set(self.future(), None); + let e = ready!(self.future().as_pin_mut().unwrap().poll(lw)); + Pin::set(self.future(), None); Poll::Ready(Some(e)) } } diff --git a/futures-util/src/stream/unfold.rs b/futures-util/src/stream/unfold.rs index 6e32a94164..19d343f23b 100644 --- a/futures-util/src/stream/unfold.rs +++ b/futures-util/src/stream/unfold.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Creates a `Stream` from a seed and a closure returning a `Future`. @@ -85,16 +85,16 @@ impl Stream for Unfold type Item = It; fn poll_next( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { if let Some(state) = self.state().take() { let fut = (self.f())(state); - PinMut::set(self.fut(), Some(fut)); + Pin::set(self.fut(), Some(fut)); } - let step = ready!(self.fut().as_pin_mut().unwrap().poll(cx)); - PinMut::set(self.fut(), None); + let step = ready!(self.fut().as_pin_mut().unwrap().poll(lw)); + Pin::set(self.fut(), None); if let Some((item, next_state)) = step { *self.state() = Some(next_state); diff --git a/futures-util/src/stream/zip.rs b/futures-util/src/stream/zip.rs index 42886c5a5d..40a024e40f 100644 --- a/futures-util/src/stream/zip.rs +++ b/futures-util/src/stream/zip.rs @@ -1,8 +1,8 @@ use crate::stream::{StreamExt, Fuse}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// An adapter for merging the output of two streams. @@ -43,17 +43,17 @@ impl Stream for Zip type Item = (St1::Item, St2::Item); fn poll_next( - mut self: PinMut, - cx: &mut task::Context + mut self: Pin<&mut Self>, + lw: &LocalWaker ) -> Poll> { if self.queued1().is_none() { - match self.stream1().poll_next(cx) { + match self.stream1().poll_next(lw) { Poll::Ready(Some(item1)) => *self.queued1() = Some(item1), Poll::Ready(None) | Poll::Pending => {} } } if self.queued2().is_none() { - match self.stream2().poll_next(cx) { + match self.stream2().poll_next(lw) { Poll::Ready(Some(item2)) => *self.queued2() = Some(item2), Poll::Ready(None) | Poll::Pending => {} } diff --git a/futures-util/src/task/atomic_waker.rs b/futures-util/src/task/atomic_waker.rs index 6767ce0b19..88ce2e9177 100644 --- a/futures-util/src/task/atomic_waker.rs +++ b/futures-util/src/task/atomic_waker.rs @@ -2,7 +2,7 @@ use core::fmt; use core::cell::UnsafeCell; use core::sync::atomic::AtomicUsize; use core::sync::atomic::Ordering::{Acquire, Release, AcqRel}; -use futures_core::task::Waker; +use futures_core::task::{LocalWaker, Waker}; /// A synchronization primitive for task wakeup. /// @@ -171,10 +171,10 @@ impl AtomicWaker { /// ``` /// #![feature(pin, arbitrary_self_types, futures_api)] /// use futures::future::Future; - /// use futures::task::{self, Poll, AtomicWaker}; + /// use futures::task::{LocalWaker, Poll, AtomicWaker}; /// use std::sync::atomic::AtomicBool; /// use std::sync::atomic::Ordering::SeqCst; - /// use std::pin::PinMut; + /// use std::pin::Pin; /// /// struct Flag { /// waker: AtomicWaker, @@ -184,10 +184,10 @@ impl AtomicWaker { /// impl Future for Flag { /// type Output = (); /// - /// fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll<()> { + /// fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll<()> { /// // Register **before** checking `set` to avoid a race condition /// // that would result in lost notifications. - /// self.waker.register(cx.waker()); + /// self.waker.register(lw); /// /// if self.set.load(SeqCst) { /// Poll::Ready(()) @@ -197,12 +197,12 @@ impl AtomicWaker { /// } /// } /// ``` - pub fn register(&self, waker: &Waker) { + pub fn register(&self, lw: &LocalWaker) { match self.state.compare_and_swap(WAITING, REGISTERING, Acquire) { WAITING => { unsafe { // Locked acquired, update the waker cell - *self.waker.get() = Some(waker.clone()); + *self.waker.get() = Some(lw.clone().into_waker()); // Release the lock. If the state transitioned to include // the `WAKING` bit, this means that a wake has been @@ -253,7 +253,7 @@ impl AtomicWaker { // Currently in the process of waking the task, i.e., // `wake` is currently being called on the old task handle. // So, we call wake on the new waker - waker.wake(); + lw.wake(); } state => { // In this case, a concurrent thread is holding the diff --git a/futures-util/src/task/mod.rs b/futures-util/src/task/mod.rs index 1ac08222c0..3744296770 100644 --- a/futures-util/src/task/mod.rs +++ b/futures-util/src/task/mod.rs @@ -1,11 +1,9 @@ //! Task notification mod spawn; -pub use self::spawn::{SpawnExt, SpawnError}; +pub use self::spawn::{SpawnExt, LocalSpawnExt}; if_std! { - pub use self::spawn::JoinHandle; - mod local_waker_ref; pub use self::local_waker_ref::{local_waker_ref, local_waker_ref_from_nonlocal, LocalWakerRef}; } diff --git a/futures-util/src/task/spawn.rs b/futures-util/src/task/spawn.rs new file mode 100644 index 0000000000..509556e219 --- /dev/null +++ b/futures-util/src/task/spawn.rs @@ -0,0 +1,149 @@ +use futures_core::task::{LocalSpawn, Spawn}; + +if_std! { + use crate::future::{FutureExt, RemoteHandle}; + use futures_core::future::{Future, FutureObj, LocalFutureObj}; + use futures_core::task::SpawnError; +} + +impl SpawnExt for Sp where Sp: Spawn {} +impl LocalSpawnExt for Sp where Sp: LocalSpawn {} + +/// Extension trait for `Spawn`. +pub trait SpawnExt: Spawn { + /// Spawns a task that polls the given future with output `()` to + /// completion. + /// + /// This method returns a [`Result`] that contains a [`SpawnError`] if + /// spawning fails. + /// + /// You can use [`spawn_with_handle`](SpawnExt::spawn_with_handle) if + /// you want to spawn a future with output other than `()` or if you want + /// to be able to await its completion. + /// + /// Note this method will eventually be replaced with the upcoming + /// `Spawn::spawn` method which will take a `dyn Future` as input. + /// Technical limitations prevent `Spawn::spawn` from being implemented + /// today. Feel free to use this method in the meantime. + /// + /// ``` + /// #![feature(async_await, await_macro, futures_api)] + /// use futures::executor::ThreadPool; + /// use futures::task::SpawnExt; + /// + /// let mut executor = ThreadPool::new().unwrap(); + /// + /// let future = async { /* ... */ }; + /// executor.spawn(future).unwrap(); + /// ``` + #[cfg(feature = "std")] + fn spawn(&mut self, future: Fut) -> Result<(), SpawnError> + where + Fut: Future + Send + 'static, + { + self.spawn_obj(FutureObj::new(Box::new(future))) + } + + /// Spawns a task that polls the given future to completion and returns a + /// future that resolves to the spawned future's output. + /// + /// This method returns a [`Result`] that contains a [`RemoteHandle`], or, if + /// spawning fails, a [`SpawnError`]. [`RemoteHandle`] is a future that + /// resolves to the output of the spawned future. + /// + /// ``` + /// #![feature(async_await, await_macro, futures_api)] + /// use futures::executor::ThreadPool; + /// use futures::future; + /// use futures::task::SpawnExt; + /// + /// let mut executor = ThreadPool::new().unwrap(); + /// + /// let future = future::ready(1); + /// let join_handle_fut = executor.spawn_with_handle(future).unwrap(); + /// assert_eq!(executor.run(join_handle_fut), 1); + /// ``` + #[cfg(feature = "std")] + fn spawn_with_handle( + &mut self, + future: Fut + ) -> Result, SpawnError> + where + Fut: Future + Send + 'static, + Fut::Output: Send, + { + let (future, handle) = future.remote_handle(); + self.spawn(future)?; + Ok(handle) + } +} + +/// Extension trait for `LocalSpawn`. +pub trait LocalSpawnExt: LocalSpawn { + /// Spawns a task that polls the given future with output `()` to + /// completion. + /// + /// This method returns a [`Result`] that contains a [`SpawnError`] if + /// spawning fails. + /// + /// You can use [`spawn_with_handle`](SpawnExt::spawn_with_handle) if + /// you want to spawn a future with output other than `()` or if you want + /// to be able to await its completion. + /// + /// Note this method will eventually be replaced with the upcoming + /// `Spawn::spawn` method which will take a `dyn Future` as input. + /// Technical limitations prevent `Spawn::spawn` from being implemented + /// today. Feel free to use this method in the meantime. + /// + /// ``` + /// #![feature(async_await, await_macro, futures_api)] + /// use futures::executor::LocalPool; + /// use futures::task::LocalSpawnExt; + /// + /// let executor = LocalPool::new(); + /// let mut spawner = executor.spawner(); + /// + /// let future = async { /* ... */ }; + /// spawner.spawn_local(future).unwrap(); + /// ``` + #[cfg(feature = "std")] + fn spawn_local(&mut self, future: Fut) -> Result<(), SpawnError> + where + Fut: Future + 'static, + { + self.spawn_local_obj(LocalFutureObj::new(Box::new(future))) + } + + /// Spawns a task that polls the given future to completion and returns a + /// future that resolves to the spawned future's output. + /// + /// This method returns a [`Result`] that contains a [`RemoteHandle`], or, if + /// spawning fails, a [`SpawnError`]. [`RemoteHandle`] is a future that + /// resolves to the output of the spawned future. + /// + /// ``` + /// #![feature(async_await, await_macro, futures_api)] + /// use futures::executor::LocalPool; + /// use futures::future; + /// use futures::task::LocalSpawnExt; + /// + /// let mut executor = LocalPool::new(); + /// let mut spawner = executor.spawner(); + /// + /// let future = future::ready(1); + /// let join_handle_fut = spawner.spawn_local_with_handle(future).unwrap(); + /// assert_eq!(executor.run_until(join_handle_fut), 1); + /// ``` + #[cfg(feature = "std")] + fn spawn_local_with_handle( + &mut self, + future: Fut + ) -> Result, SpawnError> + where + Fut: Future + 'static, + { + let (future, handle) = future.remote_handle(); + self.spawn_local(future)?; + Ok(handle) + } +} diff --git a/futures-util/src/task/spawn/mod.rs b/futures-util/src/task/spawn/mod.rs deleted file mode 100644 index c8970c8505..0000000000 --- a/futures-util/src/task/spawn/mod.rs +++ /dev/null @@ -1,85 +0,0 @@ -use futures_core::task::Spawn; - -mod spawn_error; -pub use self::spawn_error::SpawnError; - -if_std! { - use futures_core::future::Future; - - mod spawn_with_handle; - use self::spawn_with_handle::spawn_with_handle; - pub use self::spawn_with_handle::JoinHandle; -} - -impl SpawnExt for Sp where Sp: Spawn {} - -/// Extension trait for `Spawn` -pub trait SpawnExt: Spawn { - /// Spawns a task that polls the given future with output `()` to - /// completion. - /// - /// This method returns a [`Result`] that contains a [`SpawnError`] if - /// spawning fails. - /// - /// You can use [`spawn_with_handle`](SpawnExt::spawn_with_handle) if - /// you want to spawn a future with output other than `()` or if you want - /// to be able to await its completion. - /// - /// Note this method will eventually be replaced with the upcoming - /// `Spawn::spawn` method which will take a `dyn Future` as input. - /// Technical limitations prevent `Spawn::spawn` from being implemented - /// today. Feel free to use this method in the meantime. - /// - /// ``` - /// #![feature(async_await, await_macro, futures_api)] - /// # futures::executor::block_on(async { - /// use futures::executor::ThreadPool; - /// use futures::task::SpawnExt; - /// - /// let mut executor = ThreadPool::new().unwrap(); - /// - /// let future = async { /* ... */ }; - /// executor.spawn(future).unwrap(); - /// # }); - /// ``` - #[cfg(feature = "std")] - fn spawn(&mut self, future: Fut) -> Result<(), SpawnError> - where Fut: Future + Send + 'static, - { - let res = self.spawn_obj(Box::new(future).into()); - res.map_err(|err| SpawnError { kind: err.kind }) - } - - /// Spawns a task that polls the given future to completion and returns a - /// future that resolves to the spawned future's output. - /// - /// This method returns a [`Result`] that contains a [`JoinHandle`], or, if - /// spawning fails, a [`SpawnError`]. [`JoinHandle`] is a future that - /// resolves to the output of the spawned future. - /// - /// ``` - /// #![feature(async_await, await_macro, futures_api)] - /// # futures::executor::block_on(async { - /// use futures::executor::ThreadPool; - /// use futures::future; - /// use futures::task::SpawnExt; - /// - /// let mut executor = ThreadPool::new().unwrap(); - /// - /// let future = future::ready(1); - /// let join_handle = executor.spawn_with_handle(future).unwrap(); - /// assert_eq!(await!(join_handle), 1); - /// # }); - /// ``` - #[cfg(feature = "std")] - fn spawn_with_handle( - &mut self, - future: Fut - ) -> Result, SpawnError> - where Fut: Future + Send + 'static, - Fut::Output: Send, - { - spawn_with_handle(self, future) - } -} - diff --git a/futures-util/src/task/spawn/spawn_error.rs b/futures-util/src/task/spawn/spawn_error.rs deleted file mode 100644 index 211bc6bdd4..0000000000 --- a/futures-util/src/task/spawn/spawn_error.rs +++ /dev/null @@ -1,8 +0,0 @@ -use futures_core::task::SpawnErrorKind; - -/// The result of a failed spawn -#[derive(Debug)] -pub struct SpawnError { - /// The kind of error - pub kind: SpawnErrorKind, -} diff --git a/futures-util/src/task/spawn/spawn_with_handle.rs b/futures-util/src/task/spawn/spawn_with_handle.rs deleted file mode 100644 index 99a4df77f1..0000000000 --- a/futures-util/src/task/spawn/spawn_with_handle.rs +++ /dev/null @@ -1,108 +0,0 @@ -use crate::future::FutureExt; -use super::SpawnError; -use futures_channel::oneshot::{self, Sender, Receiver}; -use futures_core::future::Future; -use futures_core::task::{self, Poll, Spawn, SpawnObjError}; -use pin_utils::{unsafe_pinned, unsafe_unpinned}; -use std::marker::Unpin; -use std::pin::PinMut; -use std::panic::{self, AssertUnwindSafe}; -use std::sync::Arc; -use std::sync::atomic::{AtomicBool, Ordering}; -use std::thread; - -/// The join handle returned by -/// [`spawn_with_handle`](crate::task::SpawnExt::spawn_with_handle). -#[must_use = "futures do nothing unless polled"] -#[derive(Debug)] -pub struct JoinHandle { - rx: Receiver>, - keep_running: Arc, -} - -impl JoinHandle { - /// Drops this handle *without* canceling the underlying future. - /// - /// This method can be used if you want to drop the handle, but let the - /// execution continue. - pub fn forget(self) { - self.keep_running.store(true, Ordering::SeqCst); - } -} - -impl Future for JoinHandle { - type Output = T; - - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - match self.rx.poll_unpin(cx) { - Poll::Ready(Ok(Ok(output))) => Poll::Ready(output), - Poll::Ready(Ok(Err(e))) => panic::resume_unwind(e), - Poll::Ready(Err(e)) => panic::resume_unwind(Box::new(e)), - Poll::Pending => Poll::Pending, - } - } -} - -struct Wrapped { - tx: Option>, - keep_running: Arc, - future: Fut, -} - -impl Unpin for Wrapped {} - -impl Wrapped { - unsafe_pinned!(future: Fut); - unsafe_unpinned!(tx: Option>); - unsafe_unpinned!(keep_running: Arc); -} - -impl Future for Wrapped { - type Output = (); - - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll<()> { - if let Poll::Ready(_) = self.tx().as_mut().unwrap().poll_cancel(cx) { - if !self.keep_running().load(Ordering::SeqCst) { - // Cancelled, bail out - return Poll::Ready(()) - } - } - - let output = match self.future().poll(cx) { - Poll::Ready(output) => output, - Poll::Pending => return Poll::Pending, - }; - - // if the receiving end has gone away then that's ok, we just ignore the - // send error here. - drop(self.tx().take().unwrap().send(output)); - Poll::Ready(()) - } -} - -pub(super) fn spawn_with_handle( - executor: &mut Sp, - future: Fut, -) -> Result, SpawnError> -where Sp: Spawn + ?Sized, - Fut: Future + Send + 'static, - Fut::Output: Send, -{ - let (tx, rx) = oneshot::channel(); - let keep_running = Arc::new(AtomicBool::new(false)); - - // AssertUnwindSafe is used here because `Send + 'static` is basically - // an alias for an implementation of the `UnwindSafe` trait but we can't - // express that in the standard library right now. - let wrapped = Wrapped { - future: AssertUnwindSafe(future).catch_unwind(), - tx: Some(tx), - keep_running: keep_running.clone(), - }; - - let res = executor.spawn_obj(Box::new(wrapped).into()); - match res { - Ok(()) => Ok(JoinHandle { rx, keep_running }), - Err(SpawnObjError { kind, .. }) => Err(SpawnError { kind }), - } -} diff --git a/futures-util/src/try_future/and_then.rs b/futures-util/src/try_future/and_then.rs index 49666791d2..e1fa58660d 100644 --- a/futures-util/src/try_future/and_then.rs +++ b/futures-util/src/try_future/and_then.rs @@ -1,7 +1,7 @@ use super::{TryChain, TryChainAction}; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`and_then`](super::TryFutureExt::and_then) combinator. @@ -32,8 +32,8 @@ impl Future for AndThen { type Output = Result; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - self.try_chain().poll(cx, |result, async_op| { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.try_chain().poll(lw, |result, async_op| { match result { Ok(ok) => TryChainAction::Future(async_op(ok)), Err(err) => TryChainAction::Output(Err(err)), diff --git a/futures-util/src/try_future/err_into.rs b/futures-util/src/try_future/err_into.rs index 7dcdd2c244..99a8f19f55 100644 --- a/futures-util/src/try_future/err_into.rs +++ b/futures-util/src/try_future/err_into.rs @@ -1,7 +1,7 @@ use core::marker::{PhantomData, Unpin}; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`err_into`](super::TryFutureExt::err_into) combinator. @@ -32,10 +32,10 @@ impl Future for ErrInto type Output = Result; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - self.future().try_poll(cx) + self.future().try_poll(lw) .map(|res| res.map_err(Into::into)) } } diff --git a/futures-util/src/try_future/flatten_sink.rs b/futures-util/src/try_future/flatten_sink.rs index 2f448845ce..0295e18099 100644 --- a/futures-util/src/try_future/flatten_sink.rs +++ b/futures-util/src/try_future/flatten_sink.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::TryFuture; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use futures_sink::Sink; #[derive(Debug)] @@ -30,12 +30,12 @@ where #[allow(clippy::needless_lifetimes)] // https://github.com/rust-lang/rust/issues/52675 fn project_pin<'a>( - self: PinMut<'a, Self> - ) -> State, PinMut<'a, Si>> { + self: Pin<&'a mut Self> + ) -> State, Pin<&'a mut Si>> { unsafe { - match &mut PinMut::get_mut_unchecked(self).0 { - Waiting(f) => Waiting(PinMut::new_unchecked(f)), - Ready(s) => Ready(PinMut::new_unchecked(s)), + match &mut Pin::get_mut_unchecked(self).0 { + Waiting(f) => Waiting(Pin::new_unchecked(f)), + Ready(s) => Ready(Pin::new_unchecked(s)), Closed => Closed, } } @@ -51,24 +51,24 @@ where type SinkError = Si::SinkError; fn poll_ready( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - let resolved_stream = match self.reborrow().project_pin() { - Ready(s) => return s.poll_ready(cx), - Waiting(f) => try_ready!(f.try_poll(cx)), + let resolved_stream = match self.as_mut().project_pin() { + Ready(s) => return s.poll_ready(lw), + Waiting(f) => try_ready!(f.try_poll(lw)), Closed => panic!("poll_ready called after eof"), }; - PinMut::set(self.reborrow(), FlattenSink(Ready(resolved_stream))); + Pin::set(self.as_mut(), FlattenSink(Ready(resolved_stream))); if let Ready(resolved_stream) = self.project_pin() { - resolved_stream.poll_ready(cx) + resolved_stream.poll_ready(lw) } else { unreachable!() } } fn start_send( - self: PinMut, + self: Pin<&mut Self>, item: Self::SinkItem, ) -> Result<(), Self::SinkError> { match self.project_pin() { @@ -79,11 +79,11 @@ where } fn poll_flush( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { match self.project_pin() { - Ready(s) => s.poll_flush(cx), + Ready(s) => s.poll_flush(lw), // if sink not yet resolved, nothing written ==> everything flushed Waiting(_) => Poll::Ready(Ok(())), Closed => panic!("poll_flush called after eof"), @@ -91,15 +91,15 @@ where } fn poll_close( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - let res = match self.reborrow().project_pin() { - Ready(s) => s.poll_close(cx), + let res = match self.as_mut().project_pin() { + Ready(s) => s.poll_close(lw), Waiting(_) | Closed => Poll::Ready(Ok(())), }; if res.is_ready() { - PinMut::set(self, FlattenSink(Closed)); + Pin::set(self, FlattenSink(Closed)); } res } diff --git a/futures-util/src/try_future/into_future.rs b/futures-util/src/try_future/into_future.rs index 313e84e89b..faa900f888 100644 --- a/futures-util/src/try_future/into_future.rs +++ b/futures-util/src/try_future/into_future.rs @@ -1,6 +1,6 @@ -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`into_future`](super::TryFutureExt::into_future) combinator. @@ -24,9 +24,9 @@ impl Future for IntoFuture { #[inline] fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - self.future().try_poll(cx) + self.future().try_poll(lw) } } diff --git a/futures-util/src/try_future/map_err.rs b/futures-util/src/try_future/map_err.rs index c1bc17d7df..938c7259d5 100644 --- a/futures-util/src/try_future/map_err.rs +++ b/futures-util/src/try_future/map_err.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`map_err`](super::TryFutureExt::map_err) combinator. @@ -31,10 +31,10 @@ impl Future for MapErr type Output = Result; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - match self.future().try_poll(cx) { + match self.future().try_poll(lw) { Poll::Pending => Poll::Pending, Poll::Ready(result) => { let f = self.f().take() diff --git a/futures-util/src/try_future/map_ok.rs b/futures-util/src/try_future/map_ok.rs index 20e6122455..247209928a 100644 --- a/futures-util/src/try_future/map_ok.rs +++ b/futures-util/src/try_future/map_ok.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`map_ok`](super::TryFutureExt::map_ok) combinator. @@ -31,10 +31,10 @@ impl Future for MapOk type Output = Result; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - match self.future().try_poll(cx) { + match self.future().try_poll(lw) { Poll::Pending => Poll::Pending, Poll::Ready(result) => { let op = self.f().take() diff --git a/futures-util/src/try_future/mod.rs b/futures-util/src/try_future/mod.rs index d536c5523a..5b544b4a4f 100644 --- a/futures-util/src/try_future/mod.rs +++ b/futures-util/src/try_future/mod.rs @@ -8,7 +8,6 @@ use futures_sink::Sink; #[cfg(feature = "compat")] use crate::compat::Compat; #[cfg(feature = "compat")] use core::marker::Unpin; -#[cfg(feature = "compat")] use futures_core::task::Spawn; /* TODO mod join; @@ -478,11 +477,10 @@ pub trait TryFutureExt: TryFuture { /// Wraps a [`TryFuture`] into a future compatable with libraries using /// futures 0.1 future definitons. Requires the `compat` feature to enable. #[cfg(feature = "compat")] - fn compat(self, spawn: Sp) -> Compat + fn compat(self) -> Compat where Self: Sized + Unpin, - Sp: Spawn, { - Compat::new(self, Some(spawn)) + Compat::new(self) } /// Wraps a [`TryFuture`] into a type that implements diff --git a/futures-util/src/try_future/or_else.rs b/futures-util/src/try_future/or_else.rs index 6c7e4b6402..b554b7d5e3 100644 --- a/futures-util/src/try_future/or_else.rs +++ b/futures-util/src/try_future/or_else.rs @@ -1,7 +1,7 @@ use super::{TryChain, TryChainAction}; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`or_else`](super::TryFutureExt::or_else) combinator. @@ -33,10 +33,10 @@ impl Future for OrElse type Output = Result; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - self.try_chain().poll(cx, |result, async_op| { + self.try_chain().poll(lw, |result, async_op| { match result { Ok(ok) => TryChainAction::Output(Ok(ok)), Err(err) => TryChainAction::Future(async_op(err)), diff --git a/futures-util/src/try_future/try_chain.rs b/futures-util/src/try_future/try_chain.rs index 88b1dedc8d..025b2be6a9 100644 --- a/futures-util/src/try_future/try_chain.rs +++ b/futures-util/src/try_future/try_chain.rs @@ -1,6 +1,6 @@ -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::TryFuture; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; #[must_use = "futures do nothing unless polled"] #[derive(Debug)] @@ -26,8 +26,8 @@ impl TryChain } pub(crate) fn poll( - self: PinMut, - cx: &mut task::Context, + self: Pin<&mut Self>, + lw: &LocalWaker, f: F, ) -> Poll> where F: FnOnce(Result, Data) -> TryChainAction, @@ -35,20 +35,20 @@ impl TryChain let mut f = Some(f); // Safe to call `get_mut_unchecked` because we won't move the futures. - let this = unsafe { PinMut::get_mut_unchecked(self) }; + let this = unsafe { Pin::get_mut_unchecked(self) }; loop { let (output, data) = match this { TryChain::First(fut1, data) => { // Poll the first future - match unsafe { PinMut::new_unchecked(fut1) }.try_poll(cx) { + match unsafe { Pin::new_unchecked(fut1) }.try_poll(lw) { Poll::Pending => return Poll::Pending, Poll::Ready(output) => (output, data.take().unwrap()), } } TryChain::Second(fut2) => { // Poll the second future - return unsafe { PinMut::new_unchecked(fut2) }.try_poll(cx) + return unsafe { Pin::new_unchecked(fut2) }.try_poll(lw) } TryChain::Empty => { panic!("future must not be polled after it returned `Poll::Ready`"); diff --git a/futures-util/src/try_future/try_join.rs b/futures-util/src/try_future/try_join.rs index 88ea2b9624..4c5bd43562 100644 --- a/futures-util/src/try_future/try_join.rs +++ b/futures-util/src/try_future/try_join.rs @@ -3,9 +3,9 @@ use crate::future::{MaybeDone, maybe_done}; use crate::try_future::{TryFutureExt, IntoFuture}; use core::fmt; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; macro_rules! generate { @@ -70,17 +70,17 @@ macro_rules! generate { #[allow(clippy::useless_let_if_seq)] fn poll( - mut self: PinMut, cx: &mut task::Context + mut self: Pin<&mut Self>, lw: &LocalWaker ) -> Poll { let mut all_done = true; - if self.Fut1().poll(cx).is_pending() { + if self.Fut1().poll(lw).is_pending() { all_done = false; } else if self.Fut1().output_mut().unwrap().is_err() { return Poll::Ready(Err( self.Fut1().take_output().unwrap().err().unwrap())); } $( - if self.$Fut().poll(cx).is_pending() { + if self.$Fut().poll(lw).is_pending() { all_done = false; } else if self.$Fut().output_mut().unwrap().is_err() { return Poll::Ready(Err( diff --git a/futures-util/src/try_future/unwrap_or_else.rs b/futures-util/src/try_future/unwrap_or_else.rs index ed8dde9ff1..61f79c71ec 100644 --- a/futures-util/src/try_future/unwrap_or_else.rs +++ b/futures-util/src/try_future/unwrap_or_else.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`unwrap_or_else`](super::TryFutureExt::unwrap_or_else) @@ -32,10 +32,10 @@ impl Future for UnwrapOrElse type Output = Fut::Ok; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - match self.future().try_poll(cx) { + match self.future().try_poll(lw) { Poll::Pending => Poll::Pending, Poll::Ready(result) => { let op = self.f().take() diff --git a/futures-util/src/try_stream/err_into.rs b/futures-util/src/try_stream/err_into.rs index 0f338fcc23..b9759aaac1 100644 --- a/futures-util/src/try_stream/err_into.rs +++ b/futures-util/src/try_stream/err_into.rs @@ -1,7 +1,7 @@ use core::marker::{PhantomData, Unpin}; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Stream for the [`err_into`](super::TryStreamExt::err_into) combinator. @@ -30,10 +30,10 @@ where type Item = Result; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.stream().try_poll_next(cx) + self.stream().try_poll_next(lw) .map(|res| res.map(|some| some.map_err(Into::into))) } } diff --git a/futures-util/src/try_stream/into_stream.rs b/futures-util/src/try_stream/into_stream.rs index 57e3d0b7f4..54abe6882a 100644 --- a/futures-util/src/try_stream/into_stream.rs +++ b/futures-util/src/try_stream/into_stream.rs @@ -1,6 +1,6 @@ -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; /// Stream for the [`into_stream`](super::TryStreamExt::into_stream) combinator. @@ -41,9 +41,9 @@ impl Stream for IntoStream { #[inline] fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.stream().try_poll_next(cx) + self.stream().try_poll_next(lw) } } diff --git a/futures-util/src/try_stream/map_err.rs b/futures-util/src/try_stream/map_err.rs index f6f735d508..febc052246 100644 --- a/futures-util/src/try_stream/map_err.rs +++ b/futures-util/src/try_stream/map_err.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Stream for the [`map_err`](super::TryStreamExt::map_err) combinator. @@ -33,10 +33,10 @@ where #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - match self.stream().try_poll_next(cx) { + match self.stream().try_poll_next(lw) { Poll::Pending => Poll::Pending, Poll::Ready(opt) => Poll::Ready(opt.map(|res| res.map_err(|e| self.f()(e)))), diff --git a/futures-util/src/try_stream/map_ok.rs b/futures-util/src/try_stream/map_ok.rs index cf8043b91e..3a15fd253e 100644 --- a/futures-util/src/try_stream/map_ok.rs +++ b/futures-util/src/try_stream/map_ok.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Stream for the [`map_ok`](super::TryStreamExt::map_ok) combinator. @@ -33,10 +33,10 @@ where #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - match self.stream().try_poll_next(cx) { + match self.stream().try_poll_next(lw) { Poll::Pending => Poll::Pending, Poll::Ready(opt) => Poll::Ready(opt.map(|res| res.map(|x| self.f()(x)))), diff --git a/futures-util/src/try_stream/mod.rs b/futures-util/src/try_stream/mod.rs index 596037ac66..0917ae2603 100644 --- a/futures-util/src/try_stream/mod.rs +++ b/futures-util/src/try_stream/mod.rs @@ -10,9 +10,6 @@ use futures_core::stream::TryStream; #[cfg(feature = "compat")] use crate::compat::Compat; -#[cfg(feature = "compat")] -use futures_core::task::Spawn; - mod err_into; pub use self::err_into::ErrInto; @@ -510,7 +507,6 @@ pub trait TryStreamExt: TryStream { /// #![feature(async_await, await_macro, futures_api)] /// use futures::future::{FutureExt, TryFutureExt}; /// use futures::spawn; - /// use futures::compat::TokioDefaultSpawner; /// # let (tx, rx) = futures::channel::oneshot::channel(); /// /// let future03 = async { @@ -524,16 +520,15 @@ pub trait TryStreamExt: TryStream { /// let future01 = future03 /// .unit_error() // Make it a TryFuture /// .boxed() // Make it Unpin - /// .compat(TokioDefaultSpawner); + /// .compat(); /// /// tokio::run(future01); /// # futures::executor::block_on(rx).unwrap(); /// ``` #[cfg(feature = "compat")] - fn compat(self, spawn: Sp) -> Compat + fn compat(self) -> Compat where Self: Sized + Unpin, - Sp: Spawn, { - Compat::new(self, Some(spawn)) + Compat::new(self) } } diff --git a/futures-util/src/try_stream/try_buffer_unordered.rs b/futures-util/src/try_stream/try_buffer_unordered.rs index 4128e7cd3a..638962a7b5 100644 --- a/futures-util/src/try_stream/try_buffer_unordered.rs +++ b/futures-util/src/try_stream/try_buffer_unordered.rs @@ -3,10 +3,10 @@ use crate::try_future::{IntoFuture, TryFutureExt}; use crate::try_stream::IntoStream; use futures_core::future::TryFuture; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::marker::Unpin; -use std::pin::PinMut; +use std::pin::Pin; /// A stream returned by the /// [`try_buffer_unordered`](super::TryStreamExt::try_buffer_unordered) method @@ -70,13 +70,13 @@ impl Stream for TryBufferUnordered type Item = Result<::Ok, St::Error>; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { // First up, try to spawn off as many futures as possible by filling up // our slab of futures. Propagate errors from the stream immediately. while self.in_progress_queue.len() < self.max { - match self.stream().poll_next(cx) { + match self.stream().poll_next(lw) { Poll::Ready(Some(Ok(fut))) => self.in_progress_queue().push(fut.into_future()), Poll::Ready(Some(Err(e))) => return Poll::Ready(Some(Err(e))), Poll::Ready(None) | Poll::Pending => break, @@ -84,7 +84,7 @@ impl Stream for TryBufferUnordered } // Attempt to pull the next value from the in_progress_queue - match PinMut::new(self.in_progress_queue()).poll_next(cx) { + match Pin::new(self.in_progress_queue()).poll_next(lw) { x @ Poll::Pending | x @ Poll::Ready(Some(_)) => return x, Poll::Ready(None) => {} } diff --git a/futures-util/src/try_stream/try_collect.rs b/futures-util/src/try_stream/try_collect.rs index eb7cdf575f..a47c4b5e93 100644 --- a/futures-util/src/try_stream/try_collect.rs +++ b/futures-util/src/try_stream/try_collect.rs @@ -1,10 +1,10 @@ use futures_core::future::Future; use futures_core::stream::TryStream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::marker::Unpin; use std::mem; -use std::pin::PinMut; +use std::pin::Pin; use std::prelude::v1::*; /// A future which attempts to collect all of the values of a stream. @@ -28,7 +28,7 @@ impl TryCollect { } } - fn finish(mut self: PinMut) -> C { + fn finish(mut self: Pin<&mut Self>) -> C { mem::replace(self.items(), Default::default()) } } @@ -41,11 +41,11 @@ impl Future for TryCollect type Output = Result; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { loop { - match ready!(self.stream().try_poll_next(cx)) { + match ready!(self.stream().try_poll_next(lw)) { Some(Ok(x)) => self.items().extend(Some(x)), Some(Err(e)) => return Poll::Ready(Err(e)), None => return Poll::Ready(Ok(self.finish())), diff --git a/futures-util/src/try_stream/try_filter_map.rs b/futures-util/src/try_stream/try_filter_map.rs index 534a09e6dc..f515373ae7 100644 --- a/futures-util/src/try_stream/try_filter_map.rs +++ b/futures-util/src/try_stream/try_filter_map.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{TryFuture}; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A combinator that attempts to filter the results of a stream @@ -63,22 +63,22 @@ impl Stream for TryFilterMap type Item = Result; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll>> { loop { if self.pending().as_pin_mut().is_none() { - let item = match ready!(self.stream().try_poll_next(cx)) { + let item = match ready!(self.stream().try_poll_next(lw)) { Some(Ok(x)) => x, Some(Err(e)) => return Poll::Ready(Some(Err(e))), None => return Poll::Ready(None), }; let fut = (self.f())(item); - PinMut::set(self.pending(), Some(fut)); + Pin::set(self.pending(), Some(fut)); } - let result = ready!(self.pending().as_pin_mut().unwrap().try_poll(cx)); - PinMut::set(self.pending(), None); + let result = ready!(self.pending().as_pin_mut().unwrap().try_poll(lw)); + Pin::set(self.pending(), None); match result { Ok(Some(x)) => return Poll::Ready(Some(Ok(x))), Err(e) => return Poll::Ready(Some(Err(e))), diff --git a/futures-util/src/try_stream/try_fold.rs b/futures-util/src/try_stream/try_fold.rs index 6da4a4563b..4519aa61d3 100644 --- a/futures-util/src/try_stream/try_fold.rs +++ b/futures-util/src/try_stream/try_fold.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; use futures_core::stream::TryStream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// The future for the `TryStream::fold` method. @@ -44,22 +44,22 @@ impl Future for TryFold { type Output = Result; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { loop { // we're currently processing a future to produce a new accum value if self.accum().is_none() { - let accum = ready!(self.future().as_pin_mut().unwrap().try_poll(cx)?); + let accum = ready!(self.future().as_pin_mut().unwrap().try_poll(lw)?); *self.accum() = Some(accum); - PinMut::set(self.future(), None); + Pin::set(self.future(), None); } - let item = ready!(self.stream().try_poll_next(cx)?); + let item = ready!(self.stream().try_poll_next(lw)?); let accum = self.accum().take() .expect("TryFold polled after completion"); if let Some(e) = item { let future = (self.f())(accum, e); - PinMut::set(self.future(), Some(future)); + Pin::set(self.future(), Some(future)); } else { return Poll::Ready(Ok(accum)) } diff --git a/futures-util/src/try_stream/try_for_each.rs b/futures-util/src/try_stream/try_for_each.rs index 63f54a1b84..fcc4676165 100644 --- a/futures-util/src/try_stream/try_for_each.rs +++ b/futures-util/src/try_stream/try_for_each.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::{Future, TryFuture}; use futures_core::stream::TryStream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which attempts to execute an async closure over each @@ -44,17 +44,17 @@ impl Future for TryForEach { type Output = Result<(), St::Error>; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { loop { if let Some(future) = self.future().as_pin_mut() { - try_ready!(future.try_poll(cx)); + try_ready!(future.try_poll(lw)); } - PinMut::set(self.future(), None); + Pin::set(self.future(), None); - match ready!(self.stream().try_poll_next(cx)) { + match ready!(self.stream().try_poll_next(lw)) { Some(Ok(e)) => { let future = (self.f())(e); - PinMut::set(self.future(), Some(future)); + Pin::set(self.future(), Some(future)); } Some(Err(e)) => return Poll::Ready(Err(e)), None => return Poll::Ready(Ok(())), diff --git a/futures-util/src/try_stream/try_for_each_concurrent.rs b/futures-util/src/try_stream/try_for_each_concurrent.rs index 229c4c97ba..367de2222b 100644 --- a/futures-util/src/try_stream/try_for_each_concurrent.rs +++ b/futures-util/src/try_stream/try_for_each_concurrent.rs @@ -1,10 +1,10 @@ use crate::stream::{FuturesUnordered, StreamExt}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use core::num::NonZeroUsize; use futures_core::future::Future; use futures_core::stream::TryStream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which executes a unit closure over each item on a @@ -55,7 +55,7 @@ impl Future for TryForEachConcurrent { type Output = Result<(), St::Error>; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { loop { let mut made_progress_this_iter = false; @@ -65,7 +65,7 @@ impl Future for TryForEachConcurrent if self.limit().map(|limit| limit.get() > current_len).unwrap_or(true) { let mut stream_completed = false; let elem = if let Some(stream) = self.stream().as_pin_mut() { - match stream.try_poll_next(cx)? { + match stream.try_poll_next(lw)? { Poll::Ready(Some(elem)) => { made_progress_this_iter = true; Some(elem) @@ -80,7 +80,7 @@ impl Future for TryForEachConcurrent None }; if stream_completed { - PinMut::set(self.stream(), None); + Pin::set(self.stream(), None); } if let Some(elem) = elem { let next_future = (self.f())(elem); @@ -88,7 +88,7 @@ impl Future for TryForEachConcurrent } } - match self.futures().poll_next_unpin(cx)? { + match self.futures().poll_next_unpin(lw)? { Poll::Ready(Some(())) => made_progress_this_iter = true, Poll::Ready(None) => { if self.stream().is_none() { diff --git a/futures-util/src/try_stream/try_next.rs b/futures-util/src/try_stream/try_next.rs index 0a1c978e67..406d35cd2a 100644 --- a/futures-util/src/try_stream/try_next.rs +++ b/futures-util/src/try_stream/try_next.rs @@ -1,8 +1,8 @@ use futures_core::future::Future; use futures_core::stream::TryStream; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; /// A future which attempts to collect all of the values of a stream. /// @@ -25,10 +25,10 @@ impl Future for TryNext<'_, St> { type Output = Result, St::Error>; fn poll( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll { - match PinMut::new(&mut *self.stream).try_poll_next(cx) { + match Pin::new(&mut *self.stream).try_poll_next(lw) { Poll::Ready(Some(Ok(x))) => Poll::Ready(Ok(Some(x))), Poll::Ready(Some(Err(e))) => Poll::Ready(Err(e)), Poll::Ready(None) => Poll::Ready(Ok(None)), diff --git a/futures-util/src/try_stream/try_skip_while.rs b/futures-util/src/try_stream/try_skip_while.rs index 307808940b..15d30e0fb1 100644 --- a/futures-util/src/try_stream/try_skip_while.rs +++ b/futures-util/src/try_stream/try_skip_while.rs @@ -1,8 +1,8 @@ use core::marker::Unpin; -use core::pin::PinMut; +use core::pin::Pin; use futures_core::future::TryFuture; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{self, Poll}; +use futures_core::task::{LocalWaker, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// A stream combinator which skips elements of a stream while a predicate @@ -76,27 +76,27 @@ impl Stream for TrySkipWhile type Item = Result; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { if *self.done_skipping() { - return self.stream().try_poll_next(cx); + return self.stream().try_poll_next(lw); } loop { if self.pending_item().is_none() { - let item = match ready!(self.stream().try_poll_next(cx)?) { + let item = match ready!(self.stream().try_poll_next(lw)?) { Some(e) => e, None => return Poll::Ready(None), }; let fut = (self.f())(&item); - PinMut::set(self.pending_fut(), Some(fut)); + Pin::set(self.pending_fut(), Some(fut)); *self.pending_item() = Some(item); } - let skipped = ready!(self.pending_fut().as_pin_mut().unwrap().try_poll(cx)?); + let skipped = ready!(self.pending_fut().as_pin_mut().unwrap().try_poll(lw)?); let item = self.pending_item().take().unwrap(); - PinMut::set(self.pending_fut(), None); + Pin::set(self.pending_fut(), None); if !skipped { *self.done_skipping() = true; diff --git a/futures/src/lib.rs b/futures/src/lib.rs index cd02b3e19d..32175a9994 100644 --- a/futures/src/lib.rs +++ b/futures/src/lib.rs @@ -57,7 +57,7 @@ pub use futures_util::{ #[cfg(feature = "std")] pub use futures_util::{ // Async-await - join, try_join, select, pending, poll, spawn, spawn_with_handle, + join, try_join, select, pending, poll, }; #[cfg(feature = "std")] @@ -128,16 +128,11 @@ pub mod executor { //! //! # Spawning additional tasks //! - //! There are two ways to spawn a task: - //! - //! - Spawn onto a "default" spawner by calling the top-level - //! [`spawn`](crate::executor::spawn) function or [pulling the spawner - //! from the task context](crate::task::Context::spawner). - //! - Spawn onto a specific spawner by calling its - //! [`spawn_obj`](crate::executor::Spawn::spawn_obj) method directly. - //! - //! Every task always has an associated default spawner, which is usually - //! the executor on which the task is running. + //! Tasks can be spawned onto a spawner by calling its + //! [`spawn_obj`](crate::task::Spawn::spawn_obj) method directly. + //! In the case of `!Send` futures, + //! [`spawn_local_obj`](crate::task::LocalSpawn::spawn_local_obj) + //! can be used instead. //! //! # Single-threaded execution //! @@ -158,7 +153,7 @@ pub mod executor { pub use futures_executor::{ BlockingStream, Enter, EnterError, - LocalSpawn, LocalPool, + LocalSpawner, LocalPool, ThreadPool, ThreadPoolBuilder, block_on, block_on_stream, enter, }; @@ -193,14 +188,15 @@ pub mod future { FutureExt, FlattenStream, Flatten, Fuse, Inspect, IntoStream, Join, Join3, Join4, - Join5, Map, Then, WithSpawner, + Join5, Map, Then, }; #[cfg(feature = "std")] pub use futures_util::future::{ abortable, Abortable, AbortHandle, AbortRegistration, Aborted, + Remote, RemoteHandle, // For FutureExt: - CatchUnwind, Shared + CatchUnwind, Shared, // ToDo: JoinAll, SelectAll, SelectOk, join_all, select_all, select_ok }; @@ -255,7 +251,6 @@ pub mod prelude { pub use crate::future::{self, Future, TryFuture, FutureExt, TryFutureExt}; pub use crate::stream::{self, Stream, TryStream, StreamExt, TryStreamExt}; - pub use crate::task::{self, Poll, SpawnExt}; pub use crate::sink::{self, Sink, SinkExt}; #[cfg(feature = "std")] @@ -350,21 +345,15 @@ pub mod task { //! //! This module contains: //! - //! - [`Context`](crate::task::Context), which provides contextual data - //! present for every task, including a handle for waking up the task. + //! - [`Spawn`](crate::task::Spawn), a trait for spawning new tasks. //! - [`Waker`](crate::task::Waker), a handle for waking up a task. //! - //! Tasks themselves are generally created by spawning a future onto [an - //! executor](crate::executor). However, you can manually construct a task - //! by creating your own `Context` instance, and polling a future with it. - //! //! The remaining types and traits in the module are used for implementing //! executors or dealing with synchronization issues around task wakeup. pub use futures_core::task::{ - Context, Poll, Spawn, + Poll, Spawn, LocalSpawn, SpawnError, Waker, LocalWaker, UnsafeWake, - SpawnErrorKind, SpawnObjError, SpawnLocalObjError, }; #[cfg(feature = "std")] @@ -372,11 +361,10 @@ pub mod task { Wake, local_waker, local_waker_from_nonlocal }; - pub use futures_util::task::{SpawnExt, SpawnError}; - #[cfg(feature = "std")] pub use futures_util::task::{ - LocalWakerRef, local_waker_ref, local_waker_ref_from_nonlocal, JoinHandle + LocalWakerRef, local_waker_ref, local_waker_ref_from_nonlocal, + SpawnExt, LocalSpawnExt, }; #[cfg_attr( diff --git a/futures/tests/abortable.rs b/futures/tests/abortable.rs index 4b1bf3b70a..8738331fdb 100644 --- a/futures/tests/abortable.rs +++ b/futures/tests/abortable.rs @@ -4,7 +4,7 @@ use futures::channel::oneshot; use futures::executor::block_on; use futures::future::{abortable, Aborted, FutureExt}; use futures::task::Poll; -use futures_test::task::{panic_context, WakeCounter}; +use futures_test::task::WakeCounter; #[test] fn abortable_works() { @@ -21,14 +21,13 @@ fn abortable_awakens() { let (mut abortable_rx, abort_handle) = abortable(a_rx); let wake_counter = WakeCounter::new(); - let mut cx = panic_context(); - let cx = &mut cx.with_waker(wake_counter.local_waker()); + let lw = &wake_counter.local_waker(); assert_eq!(0, wake_counter.count()); - assert_eq!(Poll::Pending, abortable_rx.poll_unpin(cx)); + assert_eq!(Poll::Pending, abortable_rx.poll_unpin(lw)); assert_eq!(0, wake_counter.count()); abort_handle.abort(); assert_eq!(1, wake_counter.count()); - assert_eq!(Poll::Ready(Err(Aborted)), abortable_rx.poll_unpin(cx)); + assert_eq!(Poll::Ready(Err(Aborted)), abortable_rx.poll_unpin(lw)); } #[test] diff --git a/futures/tests/eager_drop.rs b/futures/tests/eager_drop.rs index 8ee5473038..13e7cb2e50 100644 --- a/futures/tests/eager_drop.rs +++ b/futures/tests/eager_drop.rs @@ -2,10 +2,10 @@ use futures::channel::oneshot; use futures::future::{self, Future, FutureExt, TryFutureExt}; -use futures::task::{self, Poll}; +use futures::task::{LocalWaker, Poll}; use futures_test::future::FutureTestExt; use pin_utils::unsafe_pinned; -use std::pin::PinMut; +use std::pin::Pin; use std::sync::mpsc; #[test] @@ -56,8 +56,8 @@ impl FutureData { impl Future for FutureData { type Output = F::Output; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - self.future().poll(cx) + fn poll(mut self: Pin<&mut Self>, lw: &LocalWaker) -> Poll { + self.future().poll(lw) } } diff --git a/futures/tests/fuse.rs b/futures/tests/fuse.rs index 1c7d8e5e80..ceacdd877c 100644 --- a/futures/tests/fuse.rs +++ b/futures/tests/fuse.rs @@ -1,12 +1,12 @@ #![feature(pin, arbitrary_self_types, futures_api)] use futures::future::{self, FutureExt}; -use futures_test::task::panic_context; +use futures_test::task::panic_local_waker; #[test] fn fuse() { let mut future = future::ready::(2).fuse(); - let cx = &mut panic_context(); - assert!(future.poll_unpin(cx).is_ready()); - assert!(future.poll_unpin(cx).is_pending()); + let lw = &mut panic_local_waker(); + assert!(future.poll_unpin(lw).is_ready()); + assert!(future.poll_unpin(lw).is_pending()); } diff --git a/futures/tests/futures_ordered.rs b/futures/tests/futures_ordered.rs index 8389febbbc..e5e34d80b3 100644 --- a/futures/tests/futures_ordered.rs +++ b/futures/tests/futures_ordered.rs @@ -4,7 +4,7 @@ use futures::channel::oneshot; use futures::executor::{block_on, block_on_stream}; use futures::future::{self, FutureExt, FutureObj}; use futures::stream::{StreamExt, futures_ordered, FuturesOrdered}; -use futures_test::task::no_spawn_context; +use futures_test::task::noop_local_waker_ref; #[test] fn works_1() { @@ -15,7 +15,7 @@ fn works_1() { let mut stream = futures_ordered(vec![a_rx, b_rx, c_rx]); b_tx.send(99).unwrap(); - assert!(stream.poll_next_unpin(&mut no_spawn_context()).is_pending()); + assert!(stream.poll_next_unpin(&noop_local_waker_ref()).is_pending()); a_tx.send(33).unwrap(); c_tx.send(33).unwrap(); @@ -38,13 +38,13 @@ fn works_2() { FutureObj::new(Box::new(b_rx.join(c_rx).map(|(a, b)| Ok(a? + b?)))), ]); - let cx = &mut no_spawn_context(); + let lw = &noop_local_waker_ref(); a_tx.send(33).unwrap(); b_tx.send(33).unwrap(); - assert!(stream.poll_next_unpin(cx).is_ready()); - assert!(stream.poll_next_unpin(cx).is_pending()); + assert!(stream.poll_next_unpin(lw).is_ready()); + assert!(stream.poll_next_unpin(lw).is_pending()); c_tx.send(33).unwrap(); - assert!(stream.poll_next_unpin(cx).is_ready()); + assert!(stream.poll_next_unpin(lw).is_ready()); } #[test] @@ -70,15 +70,15 @@ fn queue_never_unblocked() { Box::new(b_rx.select(c_rx).then(|res| Ok(Box::new(res) as Box))) as _, ]); - with_no_spawn_context(|cx| { + with_no_spawn_context(|lw| { for _ in 0..10 { - assert!(stream.poll_next(cx).unwrap().is_pending()); + assert!(stream.poll_next(lw).unwrap().is_pending()); } b_tx.send(Box::new(())).unwrap(); - assert!(stream.poll_next(cx).unwrap().is_pending()); + assert!(stream.poll_next(lw).unwrap().is_pending()); c_tx.send(Box::new(())).unwrap(); - assert!(stream.poll_next(cx).unwrap().is_pending()); - assert!(stream.poll_next(cx).unwrap().is_pending()); + assert!(stream.poll_next(lw).unwrap().is_pending()); + assert!(stream.poll_next(lw).unwrap().is_pending()); }) }*/ diff --git a/futures/tests/futures_unordered.rs b/futures/tests/futures_unordered.rs index a61adfa533..cad6f34e94 100644 --- a/futures/tests/futures_unordered.rs +++ b/futures/tests/futures_unordered.rs @@ -7,7 +7,7 @@ use futures::stream::{StreamExt, futures_unordered, FuturesUnordered}; use futures::task::Poll; use futures_test::{assert_stream_done, assert_stream_next}; use futures_test::future::FutureTestExt; -use futures_test::task::no_spawn_context; +use futures_test::task::noop_local_waker_ref; use std::boxed::Box; #[test] @@ -42,11 +42,11 @@ fn works_2() { a_tx.send(9).unwrap(); b_tx.send(10).unwrap(); - let cx = &mut no_spawn_context(); - assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(Some(Ok(9)))); + let lw = &noop_local_waker_ref(); + assert_eq!(stream.poll_next_unpin(lw), Poll::Ready(Some(Ok(9)))); c_tx.send(20).unwrap(); - assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(Some(Ok(30)))); - assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(None)); + assert_eq!(stream.poll_next_unpin(lw), Poll::Ready(Some(Ok(30)))); + assert_eq!(stream.poll_next_unpin(lw), Poll::Ready(None)); } #[test] @@ -72,16 +72,16 @@ fn finished_future() { //FutureObj::new(Box::new(b_rx.select(c_rx))), ]); - support::with_noop_waker_context(f)(|cx| { + support::with_noop_waker_context(f)(|lw| { for _ in 0..10 { - assert!(stream.poll_next_unpin(cx).is_pending()); + assert!(stream.poll_next_unpin(lw).is_pending()); } b_tx.send(12).unwrap(); - assert!(stream.poll_next_unpin(cx).is_ready()); + assert!(stream.poll_next_unpin(lw).is_ready()); c_tx.send(3).unwrap(); - assert!(stream.poll_next_unpin(cx).is_pending()); - assert!(stream.poll_next_unpin(cx).is_pending()); + assert!(stream.poll_next_unpin(lw).is_pending()); + assert!(stream.poll_next_unpin(lw).is_pending()); }) }*/ diff --git a/futures/tests/shared.rs b/futures/tests/shared.rs index 114eead0ab..0fa8dfcd44 100644 --- a/futures/tests/shared.rs +++ b/futures/tests/shared.rs @@ -3,6 +3,7 @@ use futures::channel::oneshot; use futures::executor::{block_on, LocalPool}; use futures::future::{self, FutureExt, LocalFutureObj}; +use futures::task::LocalSpawn; use std::cell::RefCell; use std::rc::Rc; use std::thread; @@ -109,7 +110,7 @@ fn peek() { // Once the Shared has been polled, the value is peekable on the clone. spawn.spawn_local_obj(LocalFutureObj::new(Box::new(f1.map(|_| ())))).unwrap(); - local_pool.run(spawn); + local_pool.run(); for _ in 0..2 { assert_eq!(f2.peek().unwrap(), Ok(42)); } diff --git a/futures/tests/split.rs b/futures/tests/split.rs index 7f4c0f4a72..bcbf492194 100644 --- a/futures/tests/split.rs +++ b/futures/tests/split.rs @@ -3,9 +3,9 @@ use futures::executor::block_on; use futures::sink::{Sink, SinkExt}; use futures::stream::{self, Stream, StreamExt}; -use futures::task::{self, Poll}; +use futures::task::{LocalWaker, Poll}; use pin_utils::unsafe_pinned; -use std::pin::PinMut; +use std::pin::Pin; struct Join { stream: T, @@ -21,10 +21,10 @@ impl Stream for Join { type Item = T::Item; fn poll_next( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.stream().poll_next(cx) + self.stream().poll_next(lw) } } @@ -33,31 +33,31 @@ impl Sink for Join { type SinkError = U::SinkError; fn poll_ready( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.sink().poll_ready(cx) + self.sink().poll_ready(lw) } fn start_send( - mut self: PinMut, + mut self: Pin<&mut Self>, item: Self::SinkItem, ) -> Result<(), Self::SinkError> { self.sink().start_send(item) } fn poll_flush( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.sink().poll_flush(cx) + self.sink().poll_flush(lw) } fn poll_close( - mut self: PinMut, - cx: &mut task::Context, + mut self: Pin<&mut Self>, + lw: &LocalWaker, ) -> Poll> { - self.sink().poll_close(cx) + self.sink().poll_close(lw) } } diff --git a/futures/tests_disabled/all.rs b/futures/tests_disabled/all.rs index 661b9609b7..691073768a 100644 --- a/futures/tests_disabled/all.rs +++ b/futures/tests_disabled/all.rs @@ -124,7 +124,7 @@ fn smoke_oneshot() { let (c, mut p) = oneshot::channel::(); drop(c); - let res = panic_waker_cx(|cx| p.poll(cx)); + let res = panic_waker_lw(|lw| p.poll(lw)); assert!(res.is_err()); let (c, p) = oneshot::channel::(); drop(c); @@ -147,8 +147,8 @@ fn select_cancels() { assert!(brx.try_recv().is_err()); assert!(drx.try_recv().is_err()); a.send(1).unwrap(); - noop_waker_cx(|cx| { - let res = f.poll(cx); + noop_waker_lw(|lw| { + let res = f.poll(lw); assert!(res.ok().unwrap().is_ready()); assert_eq!(brx.recv().unwrap(), 1); drop(c); @@ -160,10 +160,10 @@ fn select_cancels() { let d = d.map(move |d| { dtx.send(d).unwrap(); d }); let mut f = b.select(d).then(unselect); - assert!(f.poll(cx).ok().unwrap().is_pending()); - assert!(f.poll(cx).ok().unwrap().is_pending()); + assert!(f.poll(lw).ok().unwrap().is_pending()); + assert!(f.poll(lw).ok().unwrap().is_pending()); a.send(1).unwrap(); - assert!(f.poll(cx).ok().unwrap().is_ready()); + assert!(f.poll(lw).ok().unwrap().is_ready()); drop((c, f)); assert!(drx.recv().is_err()); }) @@ -178,7 +178,7 @@ fn join_cancels() { let mut f = b.join(d); drop(a); - let res = panic_waker_cx(|cx| f.poll(cx)); + let res = panic_waker_lw(|lw| f.poll(lw)); assert!(res.is_err()); drop(c); assert!(drx.recv().is_err()); @@ -206,39 +206,39 @@ fn join_cancels() { fn join_incomplete() { let (a, b) = oneshot::channel::(); let (tx, rx) = channel(); - noop_waker_cx(|cx| { + noop_waker_lw(|lw| { let mut f = ok(1).join(b).map(move |r| tx.send(r).unwrap()); - assert!(f.poll(cx).ok().unwrap().is_pending()); + assert!(f.poll(lw).ok().unwrap().is_pending()); assert!(rx.try_recv().is_err()); a.send(2).unwrap(); - assert!(f.poll(cx).ok().unwrap().is_ready()); + assert!(f.poll(lw).ok().unwrap().is_ready()); assert_eq!(rx.recv().unwrap(), (1, 2)); let (a, b) = oneshot::channel::(); let (tx, rx) = channel(); let mut f = b.join(Ok(2)).map(move |r| tx.send(r).unwrap()); - assert!(f.poll(cx).ok().unwrap().is_pending()); + assert!(f.poll(lw).ok().unwrap().is_pending()); assert!(rx.try_recv().is_err()); a.send(1).unwrap(); - assert!(f.poll(cx).ok().unwrap().is_ready()); + assert!(f.poll(lw).ok().unwrap().is_ready()); assert_eq!(rx.recv().unwrap(), (1, 2)); let (a, b) = oneshot::channel::(); let (tx, rx) = channel(); let mut f = ok(1).join(b).map_err(move |_r| tx.send(2).unwrap()); - assert!(f.poll(cx).ok().unwrap().is_pending()); + assert!(f.poll(lw).ok().unwrap().is_pending()); assert!(rx.try_recv().is_err()); drop(a); - assert!(f.poll(cx).is_err()); + assert!(f.poll(lw).is_err()); assert_eq!(rx.recv().unwrap(), 2); let (a, b) = oneshot::channel::(); let (tx, rx) = channel(); let mut f = b.join(Ok(2)).map_err(move |_r| tx.send(1).unwrap()); - assert!(f.poll(cx).ok().unwrap().is_pending()); + assert!(f.poll(lw).ok().unwrap().is_pending()); assert!(rx.try_recv().is_err()); drop(a); - assert!(f.poll(cx).is_err()); + assert!(f.poll(lw).is_err()); assert_eq!(rx.recv().unwrap(), 1); }) } @@ -340,7 +340,7 @@ fn select2() { let b = b.map(move |v| { btx.send(v).unwrap(); v }); let d = d.map(move |v| { dtx.send(v).unwrap(); v }); let mut f = b.select(d); - let _res = noop_waker_cx(|cx| f.poll(cx)); + let _res = noop_waker_lw(|lw| f.poll(lw)); drop(f); assert!(drx.recv().is_err()); assert!(brx.recv().is_err()); diff --git a/futures/tests_disabled/future_flatten_stream.rs b/futures/tests_disabled/future_flatten_stream.rs index 4930ff5f23..481b9b7a40 100644 --- a/futures/tests_disabled/future_flatten_stream.rs +++ b/futures/tests_disabled/future_flatten_stream.rs @@ -25,7 +25,7 @@ impl Stream for PanickingStream { type Item = T; type Error = E; - fn poll_next(&mut self, _: &mut task::Context) -> Poll, Self::Error> { + fn poll_next(&mut self, _: &LocalWaker) -> Poll, Self::Error> { panic!() } } diff --git a/futures/tests_disabled/ready_queue.rs b/futures/tests_disabled/ready_queue.rs index 6face1eb14..db0b69a5e4 100644 --- a/futures/tests_disabled/ready_queue.rs +++ b/futures/tests_disabled/ready_queue.rs @@ -12,7 +12,7 @@ impl AssertSendSync for FuturesUnordered<()> {} #[test] fn basic_usage() { - block_on(future::lazy(move |cx| { + block_on(future::lazy(move |lw| { let mut queue = FuturesUnordered::new(); let (tx1, rx1) = oneshot::channel(); let (tx2, rx2) = oneshot::channel(); @@ -22,19 +22,19 @@ fn basic_usage() { queue.push(rx2); queue.push(rx3); - assert!(!queue.poll_next(cx).unwrap().is_ready()); + assert!(!queue.poll_next(lw).unwrap().is_ready()); tx2.send("hello").unwrap(); - assert_eq!(Ready(Some("hello")), queue.poll_next(cx).unwrap()); - assert!(!queue.poll_next(cx).unwrap().is_ready()); + assert_eq!(Ready(Some("hello")), queue.poll_next(lw).unwrap()); + assert!(!queue.poll_next(lw).unwrap().is_ready()); tx1.send("world").unwrap(); tx3.send("world2").unwrap(); - assert_eq!(Ready(Some("world")), queue.poll_next(cx).unwrap()); - assert_eq!(Ready(Some("world2")), queue.poll_next(cx).unwrap()); - assert_eq!(Ready(None), queue.poll_next(cx).unwrap()); + assert_eq!(Ready(Some("world")), queue.poll_next(lw).unwrap()); + assert_eq!(Ready(Some("world2")), queue.poll_next(lw).unwrap()); + assert_eq!(Ready(None), queue.poll_next(lw).unwrap()); Ok::<_, ()>(()) })).unwrap(); @@ -42,7 +42,7 @@ fn basic_usage() { #[test] fn resolving_errors() { - block_on(future::lazy(move |cx| { + block_on(future::lazy(move |lw| { let mut queue = FuturesUnordered::new(); let (tx1, rx1) = oneshot::channel(); let (tx2, rx2) = oneshot::channel(); @@ -52,19 +52,19 @@ fn resolving_errors() { queue.push(rx2); queue.push(rx3); - assert!(!queue.poll_next(cx).unwrap().is_ready()); + assert!(!queue.poll_next(lw).unwrap().is_ready()); drop(tx2); - assert!(queue.poll_next(cx).is_err()); - assert!(!queue.poll_next(cx).unwrap().is_ready()); + assert!(queue.poll_next(lw).is_err()); + assert!(!queue.poll_next(lw).unwrap().is_ready()); drop(tx1); tx3.send("world2").unwrap(); - assert!(queue.poll_next(cx).is_err()); - assert_eq!(Ready(Some("world2")), queue.poll_next(cx).unwrap()); - assert_eq!(Ready(None), queue.poll_next(cx).unwrap()); + assert!(queue.poll_next(lw).is_err()); + assert_eq!(Ready(Some("world2")), queue.poll_next(lw).unwrap()); + assert_eq!(Ready(None), queue.poll_next(lw).unwrap()); Ok::<_, ()>(()) })).unwrap(); @@ -82,16 +82,16 @@ fn dropping_ready_queue() { queue.push(rx2); queue.push(rx3); - support::noop_waker_cx(|cx| { - assert!(!tx1.poll_cancel(cx).unwrap().is_ready()); - assert!(!tx2.poll_cancel(cx).unwrap().is_ready()); - assert!(!tx3.poll_cancel(cx).unwrap().is_ready()); + support::noop_waker_lw(|lw| { + assert!(!tx1.poll_cancel(lw).unwrap().is_ready()); + assert!(!tx2.poll_cancel(lw).unwrap().is_ready()); + assert!(!tx3.poll_cancel(lw).unwrap().is_ready()); drop(queue); - assert!(tx1.poll_cancel(cx).unwrap().is_ready()); - assert!(tx2.poll_cancel(cx).unwrap().is_ready()); - assert!(tx3.poll_cancel(cx).unwrap().is_ready()); + assert!(tx1.poll_cancel(lw).unwrap().is_ready()); + assert!(tx2.poll_cancel(lw).unwrap().is_ready()); + assert!(tx3.poll_cancel(lw).unwrap().is_ready()); }); Ok::<_, ()>(()).into_future() @@ -149,16 +149,16 @@ fn stress() { #[test] fn panicking_future_dropped() { - block_on(future::lazy(move |cx| { + block_on(future::lazy(move |lw| { let mut queue = FuturesUnordered::new(); queue.push(future::poll_fn(|_| -> Poll { panic!() })); - let r = panic::catch_unwind(AssertUnwindSafe(|| queue.poll_next(cx))); + let r = panic::catch_unwind(AssertUnwindSafe(|| queue.poll_next(lw))); assert!(r.is_err()); assert!(queue.is_empty()); - assert_eq!(Ready(None), queue.poll_next(cx).unwrap()); + assert_eq!(Ready(None), queue.poll_next(lw).unwrap()); Ok::<_, ()>(()) })).unwrap(); diff --git a/futures/tests_disabled/sink.rs b/futures/tests_disabled/sink.rs index 44f4e37de5..9c44ddad06 100644 --- a/futures/tests_disabled/sink.rs +++ b/futures/tests_disabled/sink.rs @@ -97,16 +97,16 @@ impl Wake for Flag { } } -fn flag_cx(f: F) -> R - where F: FnOnce(Arc, &mut task::Context) -> R +fn flag_lw(f: F) -> R + where F: FnOnce(Arc, &LocalWaker) -> R { let flag = Flag::new(); let map = &mut task::LocalMap::new(); let waker = Waker::from(flag.clone()); let exec = &mut support::PanicExec; - let cx = &mut task::Context::new(map, &waker, exec); - f(flag, cx) + let lw = &LocalWaker::new(map, &waker, exec); + f(flag, lw) } // Sends a value on an i32 channel sink @@ -122,10 +122,10 @@ impl Future for StartSendFut { type Item = S; type Error = S::SinkError; - fn poll(&mut self, cx: &mut task::Context) -> Poll { + fn poll(&mut self, lw: &LocalWaker) -> Poll { { let inner = self.0.as_mut().unwrap(); - try_ready!(inner.poll_ready(cx)); + try_ready!(inner.poll_ready(lw)); inner.start_send(self.1.take().unwrap())?; } Ok(Async::Ready(self.0.take().unwrap())) @@ -141,15 +141,15 @@ fn mpsc_blocking_start_send() { block_on(futures::future::lazy(|_| { tx.start_send(0).unwrap(); - flag_cx(|flag, cx| { + flag_lw(|flag, lw| { let mut task = StartSendFut::new(tx, 1); - assert!(task.poll(cx).unwrap().is_pending()); + assert!(task.poll(lw).unwrap().is_pending()); assert!(!flag.get()); sassert_next(&mut rx, 0); assert!(flag.get()); flag.set(false); - assert!(task.poll(cx).unwrap().is_ready()); + assert!(task.poll(lw).unwrap().is_ready()); assert!(!flag.get()); sassert_next(&mut rx, 1); @@ -171,13 +171,13 @@ fn with_flush() { assert_eq!(sink.start_send(0), Ok(())); - flag_cx(|flag, cx| { + flag_lw(|flag, lw| { let mut task = sink.flush(); - assert!(task.poll(cx).unwrap().is_pending()); + assert!(task.poll(lw).unwrap().is_pending()); tx.send(()).unwrap(); assert!(flag.get()); - let sink = match task.poll(cx).unwrap() { + let sink = match task.poll(lw).unwrap() { Async::Ready(sink) => sink, _ => panic!() }; @@ -222,7 +222,7 @@ impl Sink for ManualFlush { type SinkItem = Option; // Pass None to flush type SinkError = (); - fn poll_ready(&mut self, _: &mut task::Context) -> Poll<(), Self::SinkError> { + fn poll_ready(&mut self, _: &LocalWaker) -> Poll<(), Self::SinkError> { Ok(Async::Ready(())) } @@ -235,17 +235,17 @@ impl Sink for ManualFlush { Ok(()) } - fn poll_flush(&mut self, cx: &mut task::Context) -> Poll<(), Self::SinkError> { + fn poll_flush(&mut self, lw: &LocalWaker) -> Poll<(), Self::SinkError> { if self.data.is_empty() { Ok(Async::Ready(())) } else { - self.waiting_tasks.push(cx.waker().clone()); + self.waiting_tasks.push(lw.waker().clone()); Ok(Async::Pending) } } - fn poll_close(&mut self, cx: &mut task::Context) -> Poll<(), Self::SinkError> { - self.poll_flush(cx) + fn poll_close(&mut self, lw: &LocalWaker) -> Poll<(), Self::SinkError> { + self.poll_flush(lw) } } @@ -270,18 +270,18 @@ impl ManualFlush { // but doesn't claim to be flushed until the underlying sink is fn with_flush_propagate() { let mut sink = ManualFlush::new().with(|x| -> Result, ()> { Ok(x) }); - flag_cx(|flag, cx| { - assert!(sink.poll_ready(cx).unwrap().is_ready()); + flag_lw(|flag, lw| { + assert!(sink.poll_ready(lw).unwrap().is_ready()); sink.start_send(Some(0)).unwrap(); - assert!(sink.poll_ready(cx).unwrap().is_ready()); + assert!(sink.poll_ready(lw).unwrap().is_ready()); sink.start_send(Some(1)).unwrap(); let mut task = sink.flush(); - assert!(task.poll(cx).unwrap().is_pending()); + assert!(task.poll(lw).unwrap().is_pending()); assert!(!flag.get()); assert_eq!(task.get_mut().unwrap().get_mut().force_flush(), vec![0, 1]); assert!(flag.get()); - assert!(task.poll(cx).unwrap().is_ready()); + assert!(task.poll(lw).unwrap().is_ready()); }) } @@ -317,11 +317,11 @@ impl Allow { } } - fn check(&self, cx: &mut task::Context) -> bool { + fn check(&self, lw: &LocalWaker) -> bool { if self.flag.get() { true } else { - self.tasks.borrow_mut().push(cx.waker().clone()); + self.tasks.borrow_mut().push(lw.waker().clone()); false } } @@ -339,8 +339,8 @@ impl Sink for ManualAllow { type SinkItem = T; type SinkError = Never; - fn poll_ready(&mut self, cx: &mut task::Context) -> Poll<(), Self::SinkError> { - if self.allow.check(cx) { + fn poll_ready(&mut self, lw: &LocalWaker) -> Poll<(), Self::SinkError> { + if self.allow.check(lw) { Ok(Async::Ready(())) } else { Ok(Async::Pending) @@ -352,11 +352,11 @@ impl Sink for ManualAllow { Ok(()) } - fn poll_flush(&mut self, _: &mut task::Context) -> Poll<(), Self::SinkError> { + fn poll_flush(&mut self, _: &LocalWaker) -> Poll<(), Self::SinkError> { Ok(Async::Ready(())) } - fn poll_close(&mut self, _: &mut task::Context) -> Poll<(), Self::SinkError> { + fn poll_close(&mut self, _: &LocalWaker) -> Poll<(), Self::SinkError> { Ok(Async::Ready(())) } } @@ -380,13 +380,13 @@ fn buffer() { let sink = block_on(StartSendFut::new(sink, 0)).unwrap(); let sink = block_on(StartSendFut::new(sink, 1)).unwrap(); - flag_cx(|flag, cx| { + flag_lw(|flag, lw| { let mut task = sink.send(2); - assert!(task.poll(cx).unwrap().is_pending()); + assert!(task.poll(lw).unwrap().is_pending()); assert!(!flag.get()); allow.start(); assert!(flag.get()); - match task.poll(cx).unwrap() { + match task.poll(lw).unwrap() { Async::Ready(sink) => { assert_eq!(sink.get_ref().data, vec![0, 1, 2]); } @@ -415,18 +415,18 @@ fn fanout_backpressure() { let sink = block_on(StartSendFut::new(sink, 0)).unwrap(); - flag_cx(|flag, cx| { + flag_lw(|flag, lw| { let mut task = sink.send(2); assert!(!flag.get()); - assert!(task.poll(cx).unwrap().is_pending()); + assert!(task.poll(lw).unwrap().is_pending()); let (item, left_recv) = block_on(left_recv.next()).unwrap(); assert_eq!(item, Some(0)); assert!(flag.get()); - assert!(task.poll(cx).unwrap().is_pending()); + assert!(task.poll(lw).unwrap().is_pending()); let (item, right_recv) = block_on(right_recv.next()).unwrap(); assert_eq!(item, Some(0)); assert!(flag.get()); - assert!(task.poll(cx).unwrap().is_ready()); + assert!(task.poll(lw).unwrap().is_ready()); // make sure receivers live until end of test to prevent send errors drop(left_recv); drop(right_recv); @@ -435,11 +435,11 @@ fn fanout_backpressure() { #[test] fn map_err() { - panic_waker_cx(|cx| { + panic_waker_lw(|lw| { let (tx, _rx) = mpsc::channel(1); let mut tx = tx.sink_map_err(|_| ()); assert_eq!(tx.start_send(()), Ok(())); - assert_eq!(tx.poll_flush(cx), Ok(Async::Ready(()))); + assert_eq!(tx.poll_flush(lw), Ok(Async::Ready(()))); }); let tx = mpsc::channel(0).0; @@ -457,11 +457,11 @@ impl From for FromErrTest { #[test] fn from_err() { - panic_waker_cx(|cx| { + panic_waker_lw(|lw| { let (tx, _rx) = mpsc::channel(1); let mut tx: SinkErrInto, FromErrTest> = tx.sink_err_into(); assert_eq!(tx.start_send(()), Ok(())); - assert_eq!(tx.poll_flush(cx), Ok(Async::Ready(()))); + assert_eq!(tx.poll_flush(lw), Ok(Async::Ready(()))); }); let tx = mpsc::channel(0).0; diff --git a/futures/tests_disabled/stream.rs b/futures/tests_disabled/stream.rs index 7f604c791b..f4c89ea82e 100644 --- a/futures/tests_disabled/stream.rs +++ b/futures/tests_disabled/stream.rs @@ -25,7 +25,7 @@ impl Stream for Iter type Item = T; type Error = E; - fn poll_next(&mut self, _: &mut task::Context) -> Poll, E> { + fn poll_next(&mut self, _: &LocalWaker) -> Poll, E> { match self.iter.next() { Some(Ok(e)) => Ok(Async::Ready(Some(e))), Some(Err(e)) => Err(e), @@ -288,13 +288,13 @@ fn peek() { type Item = (); type Error = u32; - fn poll(&mut self, cx: &mut task::Context) -> Poll<(), u32> { + fn poll(&mut self, lw: &LocalWaker) -> Poll<(), u32> { { - let res = try_ready!(self.inner.peek(cx)); + let res = try_ready!(self.inner.peek(lw)); assert_eq!(res, Some(&1)); } - assert_eq!(self.inner.peek(cx).unwrap(), Some(&1).into()); - assert_eq!(self.inner.poll_next(cx).unwrap(), Some(1).into()); + assert_eq!(self.inner.peek(lw).unwrap(), Some(&1).into()); + assert_eq!(self.inner.poll_next(lw).unwrap(), Some(1).into()); Ok(Async::Ready(())) } }