diff --git a/futures-channel/src/mpsc/mod.rs b/futures-channel/src/mpsc/mod.rs index 5a98f91462..bec1f598bc 100644 --- a/futures-channel/src/mpsc/mod.rs +++ b/futures-channel/src/mpsc/mod.rs @@ -76,7 +76,7 @@ use std::sync::{Arc, Mutex}; use std::thread; use std::usize; -use futures_core::task::{self, Task}; +use futures_core::task; use futures_core::{Async, Poll, Stream}; use mpsc::queue::{Queue, PopResult}; @@ -268,7 +268,7 @@ struct State { #[derive(Debug)] struct ReceiverTask { unparked: bool, - task: Option, + task: Option, } // Returned from Receiver::try_park() @@ -295,7 +295,7 @@ const MAX_BUFFER: usize = MAX_CAPACITY >> 1; // Sent to the consumer to wake up blocked producers #[derive(Debug)] struct SenderTask { - task: Option, + task: Option, is_parked: bool, } @@ -311,7 +311,7 @@ impl SenderTask { self.is_parked = false; if let Some(task) = self.task.take() { - task.notify(); + task.wake(); } } } @@ -397,7 +397,7 @@ impl Sender { /// notified when the channel is no longer full. pub fn try_send(&mut self, msg: T) -> Result<(), TrySendError> { // If the sender is currently blocked, reject the message - if !self.poll_unparked(false).is_ready() { + if !self.poll_unparked(None).is_ready() { return Err(TrySendError { kind: TrySendErrorKind::Full(msg), }); @@ -419,10 +419,10 @@ impl Sender { /// awoken when the channel is ready to receive more messages. /// /// On successful completion, this function will return `Ok(Ok(()))`. - pub fn start_send(&mut self, msg: T) -> Result, SendError> { + pub fn start_send(&mut self, ctx: &mut task::Context, msg: T) -> Result, SendError> { // If the sender is currently blocked, reject the message before doing // any work. - if !self.poll_unparked(true).is_ready() { + if !self.poll_unparked(Some(ctx)).is_ready() { return Ok(Err(msg)); } @@ -569,7 +569,7 @@ impl Sender { }; if let Some(task) = task { - task.notify(); + task.wake(); } } @@ -577,7 +577,7 @@ impl Sender { // TODO: clean up internal state if the task::current will fail let task = if can_park { - Some(task::current()) + None// TODO Some(task::current()) } else { None }; @@ -604,20 +604,16 @@ impl Sender { /// Returns `Ok(Async::Ready(_))` if there is sufficient capacity, or returns /// `Ok(Async::Pending)` if the channel is not guaranteed to have capacity. Returns /// `Err(SendError(_))` if the receiver has been dropped. - /// - /// # Panics - /// - /// This method will panic if called from outside the context of a task or future. - pub fn poll_ready(&mut self) -> Poll<(), SendError<()>> { + pub fn poll_ready(&mut self, ctx: &mut task::Context) -> Poll<(), SendError<()>> { let state = decode_state(self.inner.state.load(SeqCst)); if !state.is_open { return Err(SendError(())); } - Ok(self.poll_unparked(true)) + Ok(self.poll_unparked(Some(ctx))) } - fn poll_unparked(&mut self, do_park: bool) -> Async<()> { + fn poll_unparked(&mut self, ctx: Option<&mut task::Context>) -> Async<()> { // First check the `maybe_parked` variable. This avoids acquiring the // lock in most cases if self.maybe_parked { @@ -635,11 +631,7 @@ impl Sender { // // Update the task in case the `Sender` has been moved to another // task - task.task = if do_park { - Some(task::current()) - } else { - None - }; + task.task = ctx.map(|c| c.waker()); Async::Pending } else { @@ -657,8 +649,8 @@ impl UnboundedSender { /// /// On a successful `start_send`, this function will return /// `Ok(Ok(())`. - pub fn start_send(&mut self, msg: T) -> Result, SendError> { - self.0.start_send(msg) + pub fn start_send(&mut self, ctx: &mut task::Context, msg: T) -> Result, SendError> { + self.0.start_send(ctx, msg) } /// Sends the provided message along this channel. @@ -815,7 +807,7 @@ impl Receiver { } // Try to park the receiver task - fn try_park(&self) -> TryPark { + fn try_park(&self, ctx: &mut task::Context) -> TryPark { let curr = self.inner.state.load(SeqCst); let state = decode_state(curr); @@ -833,7 +825,7 @@ impl Receiver { return TryPark::NotEmpty; } - recv_task.task = Some(task::current()); + recv_task.task = Some(ctx.waker()); TryPark::Parked } @@ -858,7 +850,7 @@ impl Stream for Receiver { type Item = T; type Error = (); - fn poll(&mut self) -> Poll, ()> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, ()> { loop { // Try to read a message off of the message queue. let msg = match self.next_message() { @@ -867,7 +859,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() { + match self.try_park(ctx) { TryPark::Parked => { // The task was parked, and the channel is still // empty, return Pending. @@ -925,8 +917,8 @@ impl Stream for UnboundedReceiver { type Item = T; type Error = (); - fn poll(&mut self) -> Poll, ()> { - self.0.poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, ()> { + self.0.poll(ctx) } } diff --git a/futures-channel/src/oneshot.rs b/futures-channel/src/oneshot.rs index c2b8d76ee9..6b43ddfa26 100644 --- a/futures-channel/src/oneshot.rs +++ b/futures-channel/src/oneshot.rs @@ -7,7 +7,7 @@ use std::error::Error; use std::fmt; use futures_core::{Future, Poll, Async}; -use futures_core::task::{self, Task}; +use futures_core::task; use lock::Lock; @@ -60,11 +60,11 @@ struct Inner { /// the `Lock` here, unlike in `data` above, is important to resolve races. /// Both the `Receiver` and the `Sender` halves understand that if they /// can't acquire the lock then some important interference is happening. - rx_task: Lock>, + rx_task: Lock>, /// Like `rx_task` above, except for the task blocked in /// `Sender::poll_cancel`. Additionally, `Lock` cannot be `UnsafeCell`. - tx_task: Lock>, + tx_task: Lock>, } /// Creates a new futures-aware, one-shot channel. @@ -158,7 +158,7 @@ impl Inner { } } - fn poll_cancel(&self) -> Poll<(), ()> { + fn poll_cancel(&self, ctx: &mut task::Context) -> 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 @@ -180,9 +180,8 @@ impl Inner { // 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 = task::current(); match self.tx_task.try_lock() { - Some(mut p) => *p = Some(handle), + Some(mut p) => *p = Some(ctx.waker()), None => return Ok(Async::Ready(())), } if self.complete.load(SeqCst) { @@ -221,7 +220,7 @@ impl Inner { if let Some(mut slot) = self.rx_task.try_lock() { if let Some(task) = slot.take() { drop(slot); - task.notify(); + task.wake(); } } } @@ -233,12 +232,12 @@ impl Inner { if let Some(mut handle) = self.tx_task.try_lock() { if let Some(task) = handle.take() { drop(handle); - task.notify() + task.wake() } } } - fn recv(&self) -> Poll { + fn recv(&self, ctx: &mut task::Context) -> Poll { let mut done = false; // Check to see if some data has arrived. If it hasn't then we need to @@ -251,9 +250,8 @@ impl Inner { if self.complete.load(SeqCst) { done = true; } else { - let task = task::current(); match self.rx_task.try_lock() { - Some(mut slot) => *slot = Some(task), + Some(mut slot) => *slot = Some(ctx.waker()), None => done = true, } } @@ -306,7 +304,7 @@ impl Inner { if let Some(mut handle) = self.tx_task.try_lock() { if let Some(task) = handle.take() { drop(handle); - task.notify() + task.wake() } } } @@ -351,8 +349,8 @@ impl Sender { /// /// If you're calling this function from a context that does not have a /// task, then you can use the `is_canceled` API instead. - pub fn poll_cancel(&mut self) -> Poll<(), ()> { - self.inner.poll_cancel() + pub fn poll_cancel(&mut self, ctx: &mut task::Context) -> Poll<(), ()> { + self.inner.poll_cancel(ctx) } /// Tests to see whether this `Sender`'s corresponding `Receiver` @@ -412,8 +410,8 @@ impl Future for Receiver { type Item = T; type Error = Canceled; - fn poll(&mut self) -> Poll { - self.inner.recv() + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.inner.recv(ctx) } } diff --git a/futures-channel/tests/channel.rs b/futures-channel/tests/channel.rs index 844c641b38..75936e00e9 100644 --- a/futures-channel/tests/channel.rs +++ b/futures-channel/tests/channel.rs @@ -41,7 +41,7 @@ fn sequence() { fn drop_sender() { let (tx, mut rx) = mpsc::channel::(1); drop(tx); - match rx.poll() { + match rx.poll(&mut TaskContext::panicking()) { Ok(Async::Ready(None)) => {} _ => panic!("channel should be done"), } diff --git a/futures-channel/tests/mpsc.rs b/futures-channel/tests/mpsc.rs index 4aa9d6c07c..63ccf9c566 100644 --- a/futures-channel/tests/mpsc.rs +++ b/futures-channel/tests/mpsc.rs @@ -34,29 +34,29 @@ fn send_recv_no_buffer() { // Run on a task context let f = lazy(move || { - assert!(tx.flush().unwrap().is_ready()); - assert!(tx.poll_ready().unwrap().is_ready()); + assert!(tx.flush(&mut TaskContext::panicking()).unwrap().is_ready()); + assert!(tx.poll_ready(&mut TaskContext::panicking()).unwrap().is_ready()); // Send first message - let res = tx.start_send(1).unwrap(); + let res = tx.start_send(&mut TaskContext::panicking(), 1).unwrap(); assert!(res.is_ok()); - assert!(tx.poll_ready().unwrap().is_not_ready()); + assert!(tx.poll_ready(&mut TaskContext::panicking()).unwrap().is_not_ready()); // Send second message - let res = tx.start_send(2).unwrap(); + let res = tx.start_send(&mut TaskContext::panicking(), 2).unwrap(); assert!(res.is_err()); // Take the value - assert_eq!(rx.poll().unwrap(), Async::Ready(Some(1))); - assert!(tx.poll_ready().unwrap().is_ready()); + assert_eq!(rx.poll(&mut TaskContext::panicking()).unwrap(), Async::Ready(Some(1))); + assert!(tx.poll_ready(&mut TaskContext::panicking()).unwrap().is_ready()); - let res = tx.start_send(2).unwrap(); + let res = tx.start_send(&mut TaskContext::panicking(), 2).unwrap(); assert!(res.is_ok()); - assert!(tx.poll_ready().unwrap().is_not_ready()); + assert!(tx.poll_ready(&mut TaskContext::panicking()).unwrap().is_not_ready()); // Take the value - assert_eq!(rx.poll().unwrap(), Async::Ready(Some(2))); - assert!(tx.poll_ready().unwrap().is_ready()); + assert_eq!(rx.poll(&mut TaskContext::panicking()).unwrap(), Async::Ready(Some(2))); + assert!(tx.poll_ready(&mut TaskContext::panicking()).unwrap().is_ready()); Ok::<(), ()>(()) }); @@ -125,8 +125,8 @@ fn recv_close_gets_none() { let f = lazy(move || { rx.close(); - assert_eq!(rx.poll(), Ok(Async::Ready(None))); - assert!(tx.poll_ready().is_err()); + assert_eq!(rx.poll(&mut TaskContext::panicking()), Ok(Async::Ready(None))); + assert!(tx.poll_ready(&mut TaskContext::panicking()).is_err()); drop(tx); @@ -142,8 +142,8 @@ fn tx_close_gets_none() { // Run on a task context let f = lazy(move || { - assert_eq!(rx.poll(), Ok(Async::Ready(None))); - assert_eq!(rx.poll(), Ok(Async::Ready(None))); + assert_eq!(rx.poll(&mut TaskContext::panicking()), Ok(Async::Ready(None))); + assert_eq!(rx.poll(&mut TaskContext::panicking()), Ok(Async::Ready(None))); Ok::<(), ()>(()) }); @@ -322,7 +322,7 @@ fn stress_receiver_multi_task_bounded_hard() { // Just poll let n = n.clone(); let f = lazy(move || { - let r = match rx.poll().unwrap() { + let r = match rx.poll(&mut TaskContext::panicking()).unwrap() { Async::Ready(Some(_)) => { n.fetch_add(1, Ordering::Relaxed); *lock = Some(rx); @@ -443,12 +443,12 @@ fn stress_poll_ready() { type Item = (); type Error = (); - fn poll(&mut self) -> Poll<(), ()> { + fn poll(&mut self, ctx: &mut TaskContext) -> Poll<(), ()> { // In a loop, check if the channel is ready. If so, push an item onto the channel // (asserting that it doesn't attempt to block). while self.count > 0 { - try_ready!(self.sender.poll_ready().map_err(|_| ())); - assert!(self.sender.start_send(self.count).unwrap().is_ok()); + try_ready!(self.sender.poll_ready(ctx).map_err(|_| ())); + assert!(self.sender.start_send(ctx, self.count).unwrap().is_ok()); self.count -= 1; } Ok(Async::Ready(())) @@ -523,7 +523,7 @@ fn try_send_2() { let th = thread::spawn(|| { run(|c| { c.block_on(lazy(|| { - assert!(tx.start_send("fail").unwrap().is_err()); + assert!(tx.start_send(&mut TaskContext::panicking(), "fail").unwrap().is_err()); Ok::<_, ()>(()) })).unwrap(); diff --git a/futures-channel/tests/oneshot.rs b/futures-channel/tests/oneshot.rs index a64af88e44..5b30d60e60 100644 --- a/futures-channel/tests/oneshot.rs +++ b/futures-channel/tests/oneshot.rs @@ -14,11 +14,11 @@ use futures_executor::current_thread::run; fn smoke_poll() { let (mut tx, rx) = channel::(); let f = lazy(|| { - assert!(tx.poll_cancel().unwrap().is_not_ready()); - assert!(tx.poll_cancel().unwrap().is_not_ready()); + assert!(tx.poll_cancel(&mut TaskContext::panicking()).unwrap().is_not_ready()); + assert!(tx.poll_cancel(&mut TaskContext::panicking()).unwrap().is_not_ready()); drop(rx); - assert!(tx.poll_cancel().unwrap().is_ready()); - assert!(tx.poll_cancel().unwrap().is_ready()); + assert!(tx.poll_cancel(&mut TaskContext::panicking()).unwrap().is_ready()); + assert!(tx.poll_cancel(&mut TaskContext::panicking()).unwrap().is_ready()); ok::<(), ()>(()) }); @@ -44,8 +44,8 @@ impl Future for WaitForCancel { type Item = (); type Error = (); - fn poll(&mut self) -> Poll<(), ()> { - self.tx.poll_cancel() + fn poll(&mut self, ctx: &mut TaskContext) -> Poll<(), ()> { + self.tx.poll_cancel(ctx) } } @@ -78,8 +78,8 @@ fn cancel_lots() { fn close() { let (mut tx, mut rx) = channel::(); rx.close(); - assert!(rx.poll().is_err()); - assert!(tx.poll_cancel().unwrap().is_ready()); + assert!(rx.poll(&mut TaskContext::panicking()).is_err()); + assert!(tx.poll_cancel(&mut TaskContext::panicking()).unwrap().is_ready()); } #[test] @@ -121,7 +121,7 @@ fn cancel_sends() { orx.close(); // Not necessary to wrap in a task because the implementation of oneshot // never calls `task::current()` if the channel has been closed already. - let _ = orx.poll(); + let _ = orx.poll(&mut TaskContext::panicking()); } drop(tx); diff --git a/futures-core/src/future/mod.rs b/futures-core/src/future/mod.rs index 86cf872829..0746f7a6c8 100644 --- a/futures-core/src/future/mod.rs +++ b/futures-core/src/future/mod.rs @@ -5,6 +5,7 @@ //! `Box`, `Option`, and `Result`. use Poll; +use task; mod option; #[path = "result.rs"] @@ -163,15 +164,15 @@ pub trait Future { /// This future may have failed to finish the computation, in which case /// the `Err` variant will be returned with an appropriate payload of an /// error. - fn poll(&mut self) -> Poll; + fn poll(&mut self, ctx: &mut task::Context) -> Poll; } impl<'a, F: ?Sized + Future> Future for &'a mut F { type Item = F::Item; type Error = F::Error; - fn poll(&mut self) -> Poll { - (**self).poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + (**self).poll(ctx) } } @@ -180,8 +181,8 @@ if_std! { type Item = F::Item; type Error = F::Error; - fn poll(&mut self) -> Poll { - (**self).poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + (**self).poll(ctx) } } @@ -190,8 +191,8 @@ if_std! { type Item = F::Item; type Error = F::Error; - fn poll(&mut self) -> Poll { - self.0.poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.0.poll(ctx) } } } diff --git a/futures-core/src/future/option.rs b/futures-core/src/future/option.rs index fb1c726c7f..4f1bf2b492 100644 --- a/futures-core/src/future/option.rs +++ b/futures-core/src/future/option.rs @@ -1,6 +1,7 @@ //! Definition of the `Option` (optional step) combinator use {Future, IntoFuture, Poll, Async}; +use task; use core::option; @@ -27,10 +28,10 @@ impl Future for Option where F: Future { type Item = option::Option; type Error = E; - fn poll(&mut self) -> Poll, E> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, E> { match self.inner { None => Ok(Async::Ready(None)), - Some(ref mut x) => x.poll().map(|x| x.map(Some)), + Some(ref mut x) => x.poll(ctx).map(|x| x.map(Some)), } } } diff --git a/futures-core/src/future/result.rs b/futures-core/src/future/result.rs index a4170ce2f6..bac13ecce1 100644 --- a/futures-core/src/future/result.rs +++ b/futures-core/src/future/result.rs @@ -1,4 +1,5 @@ use {Future, IntoFuture, Poll, Async}; +use task; use core::result; /// A future representing a value that is immediately ready. @@ -75,7 +76,7 @@ impl Future for Result { type Item = T; type Error = E; - fn poll(&mut self) -> Poll { + fn poll(&mut self, _: &mut task::Context) -> Poll { self.inner.take().expect("cannot poll Result twice").map(Async::Ready) } } diff --git a/futures-core/src/never.rs b/futures-core/src/never.rs index 1796b9b4aa..cfd2cd0b9f 100644 --- a/futures-core/src/never.rs +++ b/futures-core/src/never.rs @@ -1,6 +1,7 @@ //! Definition and trait implementations for the `Never` type. use {Future, Stream, Poll}; +use task; /// A type that can never exist. /// This is used to indicate values which can never be created, such as the @@ -21,7 +22,7 @@ impl Future for Never { type Item = Never; type Error = Never; - fn poll(&mut self) -> Poll { + fn poll(&mut self, _: &mut task::Context) -> Poll { match *self {} } } @@ -30,7 +31,7 @@ impl Stream for Never { type Item = Never; type Error = Never; - fn poll(&mut self) -> Poll, Never> { + fn poll(&mut self, _: &mut task::Context) -> Poll, Never> { match *self {} } } diff --git a/futures-core/src/stream/mod.rs b/futures-core/src/stream/mod.rs index 46cdfa78a3..a1a3382693 100644 --- a/futures-core/src/stream/mod.rs +++ b/futures-core/src/stream/mod.rs @@ -16,6 +16,7 @@ //! [online]: https://tokio.rs/docs/getting-started/streams-and-sinks/ use Poll; +use task; /// A stream of values, not all of which may have been produced yet. /// @@ -87,15 +88,15 @@ pub trait Stream { /// further calls to `poll` may result in a panic or other "bad behavior". /// If this is difficult to guard against then the `fuse` adapter can be /// used to ensure that `poll` always has well-defined semantics. - fn poll(&mut self) -> Poll, Self::Error>; + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error>; } impl<'a, S: ?Sized + Stream> Stream for &'a mut S { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, Self::Error> { - (**self).poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { + (**self).poll(ctx) } } @@ -104,8 +105,8 @@ if_std! { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, Self::Error> { - (**self).poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { + (**self).poll(ctx) } } @@ -113,8 +114,8 @@ if_std! { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - self.0.poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + self.0.poll(ctx) } } } diff --git a/futures-core/src/task.rs b/futures-core/src/task.rs index 76ee85e317..8c082571f5 100644 --- a/futures-core/src/task.rs +++ b/futures-core/src/task.rs @@ -28,23 +28,22 @@ //! it's ready to make progress through the `Task::notify` method. pub use task_impl::{ // Types + Context, NotifyHandle, Spawn, - Task, + Waker, // Traits Notify, UnsafeNotify, // Functions - current, - init, spawn, with_notify, }; #[cfg(feature = "std")] pub use task_impl::{ - AtomicTask, + AtomicWaker, LocalKey, }; diff --git a/futures-core/src/task_impl/atomic_task.rs b/futures-core/src/task_impl/atomic_waker.rs similarity index 84% rename from futures-core/src/task_impl/atomic_task.rs rename to futures-core/src/task_impl/atomic_waker.rs index a89e20c70f..f1ad742782 100644 --- a/futures-core/src/task_impl/atomic_task.rs +++ b/futures-core/src/task_impl/atomic_waker.rs @@ -1,6 +1,6 @@ #![allow(dead_code)] -use super::Task; +use super::{Context, Waker}; use core::fmt; use core::cell::UnsafeCell; @@ -9,7 +9,7 @@ use core::sync::atomic::Ordering::{Acquire, Release}; /// A synchronization primitive for task notification. /// -/// `AtomicTask` will coordinate concurrent notifications with the consumer +/// `AtomicWaker` will coordinate concurrent notifications with the consumer /// potentially "updating" the underlying task to notify. This is useful in /// scenarios where a computation completes in another thread and wants to /// notify the consumer, but the consumer is in the process of being migrated to @@ -20,18 +20,18 @@ use core::sync::atomic::Ordering::{Acquire, Release}; /// differs from the usual `thread::park` pattern). It is also permitted for /// `notify` to be called **before** `register`. This results in a no-op. /// -/// A single `AtomicTask` may be reused for any number of calls to `register` or +/// A single `AtomicWaker` may be reused for any number of calls to `register` or /// `notify`. /// -/// `AtomicTask` does not provide any memory ordering guarantees, as such the +/// `AtomicWaker` does not provide any memory ordering guarantees, as such the /// user should use caution and use other synchronization primitives to guard /// the result of the underlying computation. -pub struct AtomicTask { +pub struct AtomicWaker { state: AtomicUsize, - task: UnsafeCell>, + task: UnsafeCell>, } -/// Initial state, the `AtomicTask` is currently not being used. +/// Initial state, the `AtomicWaker` is currently not being used. /// /// The value `2` is picked specifically because it between the write lock & /// read lock values. Since the read lock is represented by an incrementing @@ -40,7 +40,7 @@ pub struct AtomicTask { const WAITING: usize = 2; /// The `register` function has determined that the task is no longer current. -/// This implies that `AtomicTask::register` is being called from a different +/// This implies that `AtomicWaker::register` is being called from a different /// task than is represented by the currently stored task. The write lock is /// obtained to update the task cell. const LOCKED_WRITE: usize = 0; @@ -58,14 +58,14 @@ const LOCKED_WRITE_NOTIFIED: usize = 1; #[allow(dead_code)] const LOCKED_READ: usize = 3; -impl AtomicTask { - /// Create an `AtomicTask` initialized with the given `Task` - pub fn new() -> AtomicTask { +impl AtomicWaker { + /// Create an `AtomicWaker` initialized with the given `Waker` + pub fn new() -> AtomicWaker { // Make sure that task is Sync trait AssertSync: Sync {} - impl AssertSync for Task {} + impl AssertSync for Waker {} - AtomicTask { + AtomicWaker { state: AtomicUsize::new(WAITING), task: UnsafeCell::new(None), } @@ -86,21 +86,18 @@ impl AtomicTask { /// idea. Concurrent calls to `register` will attempt to register different /// tasks to be notified. One of the callers will win and have its task set, /// but there is no guarantee as to which caller will succeed. - pub fn register(&self) { - // Get a new task handle - let task = super::current(); - + pub fn register(&self, ctx: &mut Context) { match self.state.compare_and_swap(WAITING, LOCKED_WRITE, Acquire) { WAITING => { unsafe { // Locked acquired, update the task cell - *self.task.get() = Some(task); + *self.task.get() = Some(ctx.waker()); // Release the lock. If the state transitioned to // `LOCKED_NOTIFIED`, this means that an notify has been // signaled, so notify the task. if LOCKED_WRITE_NOTIFIED == self.state.swap(WAITING, Release) { - (*self.task.get()).as_ref().unwrap().notify(); + (*self.task.get()).as_ref().unwrap().wake(); } } } @@ -118,7 +115,7 @@ impl AtomicTask { // Currently in a read locked state, this implies that `notify` // is currently being called on the old task handle. So, we call // notify on the new task handle - task.notify(); + ctx.waker().wake(); } } } @@ -126,7 +123,7 @@ impl AtomicTask { /// Notifies the task that last called `register`. /// /// If `register` has not been called yet, then this does nothing. - pub fn notify(&self) { + pub fn wake(&self) { let mut curr = WAITING; loop { @@ -156,7 +153,7 @@ impl AtomicTask { // Notify the task unsafe { if let Some(ref task) = *self.task.get() { - task.notify(); + task.wake(); } } @@ -175,17 +172,17 @@ impl AtomicTask { } } -impl Default for AtomicTask { +impl Default for AtomicWaker { fn default() -> Self { - AtomicTask::new() + AtomicWaker::new() } } -impl fmt::Debug for AtomicTask { +impl fmt::Debug for AtomicWaker { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { - write!(fmt, "AtomicTask") + write!(fmt, "AtomicWaker") } } -unsafe impl Send for AtomicTask {} -unsafe impl Sync for AtomicTask {} +unsafe impl Send for AtomicWaker {} +unsafe impl Sync for AtomicWaker {} diff --git a/futures-core/src/task_impl/core.rs b/futures-core/src/task_impl/core.rs index fb559f9379..bc9ebea4f1 100644 --- a/futures-core/src/task_impl/core.rs +++ b/futures-core/src/task_impl/core.rs @@ -1,11 +1,8 @@ #![cfg_attr(feature = "std", allow(dead_code))] use core::marker; -use core::mem; -use core::sync::atomic::{AtomicUsize, ATOMIC_USIZE_INIT}; -use core::sync::atomic::Ordering::{SeqCst, Relaxed}; -use super::{BorrowedTask, NotifyHandle}; +use super::NotifyHandle; pub struct LocalMap; pub fn local_map() -> LocalMap { LocalMap } @@ -61,91 +58,3 @@ impl Drop for TaskUnpark { self.handle.drop_id(self.id); } } - -static GET: AtomicUsize = ATOMIC_USIZE_INIT; -static SET: AtomicUsize = ATOMIC_USIZE_INIT; - -/// Initialize the `futures` task system. -/// -/// This function is an unsafe low-level implementation detail typically only -/// used by crates using `futures` in `no_std` context. Users of this crate -/// who also use the standard library never need to invoke this function. -/// -/// The task system in the `futures` crate relies on some notion of "local -/// storage" for the running thread and/or context. The `task::current` function -/// can get invoked in any context, for example, and needs to be able to return -/// a `Task`. Typically with the standard library this is supported with -/// thread-local-storage, but this is not available in `no_std` contexts! -/// -/// This function is provided to allow `no_std` contexts to continue to be able -/// to use the standard task system in this crate. The functions provided here -/// will be used as-if they were thread-local-storage getters/setters. The `get` -/// function provided is used to retrieve the current thread-local value of the -/// task system's pointer, returning null if not initialized. The `set` function -/// updates the value of the pointer. -/// -/// # Return value -/// -/// This function will return whether initialization succeeded or not. This -/// function can be called concurrently and only the first invocation will -/// succeed. If `false` is returned then the `get` and `set` pointers provided -/// were *not* registered for use with the task system, but if `true` was -/// provided then they will be called when the task system is used. -/// -/// Note that while safe to call concurrently it's recommended to still perform -/// external synchronization when calling this function. This task system is -/// not guaranteed to be ready to go until a call to this function returns -/// `true`. In other words, if you call this function and see `false`, the -/// task system may not be ready to go as another thread may still be calling -/// `init`. -/// -/// # Unsafety -/// -/// This function is unsafe due to the requirements on the behavior of the -/// `get` and `set` functions. The pointers returned from these functions must -/// reflect the semantics specified above and must also be thread-local, -/// depending on the definition of a "thread" in the calling context. -pub unsafe fn init(get: fn() -> *mut u8, set: fn(*mut u8)) -> bool { - if GET.compare_exchange(0, get as usize, SeqCst, SeqCst).is_ok() { - SET.store(set as usize, SeqCst); - true - } else { - false - } -} - -#[inline] -pub fn get_ptr() -> Option<*mut u8> { - match GET.load(Relaxed) { - 0 => None, - n => Some(unsafe { mem::transmute:: *mut u8>(n)() }), - } -} - -#[cfg(feature = "std")] -#[inline] -pub fn is_get_ptr(f: usize) -> bool { - GET.load(Relaxed) == f -} - -pub fn set<'a, F, R>(task: &BorrowedTask<'a>, f: F) -> R - where F: FnOnce() -> R -{ - let set = match SET.load(Relaxed) { - 0 => panic!("not initialized"), - n => unsafe { mem::transmute::(n) }, - }; - - struct Reset(fn(*mut u8), *mut u8); - - impl Drop for Reset { - #[inline] - fn drop(&mut self) { - (self.0)(self.1); - } - } - - let _reset = Reset(set, get_ptr().unwrap()); - set(task as *const _ as *mut u8); - f() -} diff --git a/futures-core/src/task_impl/mod.rs b/futures-core/src/task_impl/mod.rs index 18a48e0178..03f38e05cf 100644 --- a/futures-core/src/task_impl/mod.rs +++ b/futures-core/src/task_impl/mod.rs @@ -3,8 +3,8 @@ use core::marker::PhantomData; use {Poll, Future, Stream}; -mod atomic_task; -pub use self::atomic_task::AtomicTask; +mod atomic_waker; +pub use self::atomic_waker::AtomicWaker; mod core; @@ -15,69 +15,64 @@ pub use self::std::*; #[cfg(not(feature = "std"))] pub use self::core::*; -pub struct BorrowedTask<'a> { - unpark: BorrowedUnpark<'a>, - // Task-local storage +pub struct ContextInner<'a> { + unpark: TaskUnpark, map: &'a LocalMap, } -fn with R, R>(f: F) -> R { - unsafe { - let task = get_ptr().expect("no Task is currently running"); - assert!(!task.is_null(), "no Task is currently running"); - f(&*(task as *const BorrowedTask)) - } -} - -/// A handle to a "task", which represents a single lightweight "thread" of -/// execution driving a future to completion. +/// The context for the currently running task. +/// +/// A task represents a single lightweight "thread" of execution driving a +/// future to completion. /// /// In general, futures are composed into large units of work, which are then /// spawned as tasks onto an *executor*. The executor is responsible for polling /// the future as notifications arrive, until the future terminates. /// -/// This is obtained by the `task::current` function. +/// The poll function accepts a context, which is usually provided by the executor. +pub struct Context<'a>(Option>); + +impl<'a> Context<'a> { + /// Returns a `Context` which will panic when `waker` is called. + /// This is useful when calling `poll` on a `Future` which doesn't + /// access the context. + pub fn panicking() -> Context<'a> { + Context(None) + } + + /// Returns a `Waker` which can be used to wake up the current task. + pub fn waker(&self) -> Waker { + Waker { + unpark: self.get().unpark.clone(), + } + } + + fn get(&self) -> &ContextInner<'a> { + self.0.as_ref().expect("waker should not be called on a panicking context") + } + + fn get_mut(&mut self) -> &mut ContextInner<'a> { + self.0.as_mut().expect("waker should not be called on a panicking context") + } +} + +impl<'a> fmt::Debug for Context<'a> { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_struct("Context") + .finish() + } +} + +/// A handle used to awaken a task. #[derive(Clone)] -pub struct Task { +pub struct Waker { unpark: TaskUnpark, } trait AssertSend: Send {} -impl AssertSend for Task {} - -/// Returns a handle to the current task to call `notify` at a later date. -/// -/// The returned handle implements the `Send` and `'static` bounds and may also -/// be cheaply cloned. This is useful for squirreling away the handle into a -/// location which is then later signaled that a future can make progress. -/// -/// Implementations of the `Future` trait typically use this function if they -/// would otherwise perform a blocking operation. When something isn't ready -/// yet, this `current` function is called to acquire a handle to the current -/// task, and then the future arranges it such that when the blocking operation -/// otherwise finishes (perhaps in the background) it will `notify` the -/// returned handle. -/// -/// It's sometimes necessary to pass extra information to the task when -/// unparking it, so that the task knows something about *why* it was woken. -/// See the `FutureQueue` documentation for details on how to do this. -/// -/// # Panics -/// -/// This function will panic if a task is not currently being executed. That -/// is, this method can be dangerous to call outside of an implementation of -/// `poll`. -pub fn current() -> Task { - with(|borrowed| { - let unpark = borrowed.unpark.to_owned(); - - Task { - unpark: unpark, - } - }) -} +impl AssertSend for Waker {} -impl Task { +impl Waker { /// Indicate that the task should attempt to poll its future in a timely /// fashion. /// @@ -86,39 +81,14 @@ impl Task { /// If the task is currently polling its future when `notify` is called, it /// must poll the future *again* afterwards, ensuring that all relevant /// events are eventually observed by the future. - pub fn notify(&self) { - self.unpark.notify(); - } - - /// This function is intended as a performance optimization for structures - /// which store a `Task` internally. - /// - /// The purpose of this function is to answer the question "if I `notify` - /// this task is it equivalent to `task::current().notify()`". An answer - /// "yes" may mean that you don't actually need to call `task::current()` - /// and store it, but rather you can simply leave a stored task in place. An - /// answer of "no" typically means that you need to call `task::current()` - /// and store it somewhere. - /// - /// As this is purely a performance optimization a valid implementation for - /// this function is to always return `false`. A best effort is done to - /// return `true` where possible, but false negatives may happen. Note that - /// this function will not return a false positive, however. - /// - /// # Panics - /// - /// This function will panic if no current future is being polled. - #[allow(deprecated)] - pub fn will_notify_current(&self) -> bool { - with(|current| { - self.unpark.will_notify(¤t.unpark) - }) + pub fn wake(&self) { + self.unpark.notify() } } -impl fmt::Debug for Task { +impl fmt::Debug for Waker { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("Task") + f.debug_struct("Waker") .finish() } } @@ -206,7 +176,7 @@ impl Spawn { where N: Clone + Into, T: Future, { - self.poll_notify(notify, id, |s| s.poll()) + self.poll_notify(notify, id, |s, ctx| s.poll(ctx)) } /// Like `poll_future_notify`, except polls the underlying stream. @@ -217,42 +187,28 @@ impl Spawn { where N: Clone + Into, T: Stream, { - self.poll_notify(notify, id, |s| s.poll()) + self.poll_notify(notify, id, |s, ctx| s.poll(ctx)) } /// Invokes the function with the provided `notify` in the task context. pub fn poll_notify(&mut self, notify: &N, id: usize, f: F) -> R where N: Clone + Into, - F: FnOnce(&mut T) -> R + F: FnOnce(&mut T, &mut Context) -> R { let mk = || notify.clone().into(); self.enter(BorrowedUnpark::new(&mk, id), f) } - /// TODO: dox - pub fn with_task_data(&mut self, f: F) -> R - where F: FnOnce(&mut T) -> R, - { - let Spawn { ref data, ref mut obj, .. } = *self; - with(|task| { - let new_task = BorrowedTask { - unpark: task.unpark, - map: data, - }; - - set(&new_task, || f(obj)) - }) - } - fn enter(&mut self, unpark: BorrowedUnpark, f: F) -> R - where F: FnOnce(&mut T) -> R + where F: FnOnce(&mut T, &mut Context) -> R { - let borrowed = BorrowedTask { - unpark: unpark, - map: &self.data, - }; let obj = &mut self.obj; - set(&borrowed, || f(obj)) + + let mut ctx = Context(Some(ContextInner { + unpark: unpark.to_owned(), + map: &self.data, + })); + f(obj, &mut ctx) } } @@ -356,19 +312,16 @@ pub trait Notify: Send + Sync { /// This function will panic if it is called outside the context of a future's /// task. This is only valid to call once you've already entered a future via /// `Spawn::poll_*` functions. -pub fn with_notify(notify: &T, id: usize, f: F) -> R - where F: FnOnce() -> R, +pub fn with_notify(ctx: &mut Context, notify: &T, id: usize, f: F) -> R + where F: FnOnce(&mut Context) -> R, T: Clone + Into, { - with(|task| { - let mk = || notify.clone().into(); - let new_task = BorrowedTask { - unpark: BorrowedUnpark::new(&mk, id), - map: task.map, - }; - - set(&new_task, f) - }) + let mk = || notify.clone().into(); + let mut ctx = Context(Some(ContextInner { + unpark: BorrowedUnpark::new(&mk, id).to_owned(), + map: ctx.get().map, + })); + f(&mut ctx) } /// An unsafe trait for implementing custom forms of memory management behind a diff --git a/futures-core/src/task_impl/std/data.rs b/futures-core/src/task_impl/std/data.rs index 770912b219..066f36ec31 100644 --- a/futures-core/src/task_impl/std/data.rs +++ b/futures-core/src/task_impl/std/data.rs @@ -5,8 +5,6 @@ use std::cell::RefCell; use std::hash::{BuildHasherDefault, Hasher}; use std::collections::HashMap; -use task_impl::with; - /// A macro to create a `static` of type `LocalKey` /// /// This macro is intentionally similar to the `thread_local!`, and creates a @@ -90,7 +88,7 @@ impl Hasher for IdHasher { } } -impl LocalKey { +impl<'a> ::task::Context<'a> { /// Access this task-local key, running the provided closure with a /// reference to the value. /// @@ -111,21 +109,19 @@ impl LocalKey { /// * If there is not a current task. /// * If the initialization expression is run and it panics /// * If the closure provided panics - pub fn with(&'static self, f: F) -> R - where F: FnOnce(&T) -> R + pub fn with_local(&mut self, key: LocalKey, f: F) -> R + where F: FnOnce(&mut T) -> R { - let key = (self.__key)(); - with(|task| { - let raw_pointer = { - let mut data = task.map.borrow_mut(); - let entry = data.entry(key).or_insert_with(|| { - Box::new((self.__init)()) - }); - &**entry as *const Opaque as *const T - }; - unsafe { - f(&*raw_pointer) - } - }) + let k = (key.__key)(); + let raw_pointer = { + let mut data = self.get_mut().map.borrow_mut(); + let entry = data.entry(k).or_insert_with(|| { + Box::new((key.__init)()) + }); + &mut **entry as *mut Opaque as *mut T + }; + unsafe { + f(&mut *raw_pointer) + } } } diff --git a/futures-core/src/task_impl/std/mod.rs b/futures-core/src/task_impl/std/mod.rs index aa815372df..a626d78bd5 100644 --- a/futures-core/src/task_impl/std/mod.rs +++ b/futures-core/src/task_impl/std/mod.rs @@ -1,79 +1,18 @@ use std::prelude::v1::*; -use std::cell::Cell; use std::marker::PhantomData; use std::mem; use std::ptr; -use std::sync::{Arc, Mutex, Condvar, Once, ONCE_INIT}; +use std::sync::{Arc, Mutex, Condvar}; use std::sync::atomic::{AtomicUsize, Ordering}; use {Future, Stream, Async}; -use super::core; -use super::{BorrowedTask, NotifyHandle, Spawn, Notify, UnsafeNotify}; +use super::{NotifyHandle, Spawn, Notify, UnsafeNotify}; pub use super::core::{BorrowedUnpark, TaskUnpark}; mod data; pub use self::data::*; -pub use task_impl::core::init; - -thread_local!(static CURRENT_TASK: Cell<*mut u8> = Cell::new(ptr::null_mut())); - -static INIT: Once = ONCE_INIT; - -pub fn get_ptr() -> Option<*mut u8> { - // Since this condition will always return true when TLS task storage is - // used (the default), the branch predictor will be able to optimize the - // branching and a dynamic dispatch will be avoided, which makes the - // compiler happier. - if core::is_get_ptr(0x1) { - Some(CURRENT_TASK.with(|c| c.get())) - } else { - core::get_ptr() - } -} - -fn tls_slot() -> *const Cell<*mut u8> { - CURRENT_TASK.with(|c| c as *const _) -} - -pub fn set<'a, F, R>(task: &BorrowedTask<'a>, f: F) -> R - where F: FnOnce() -> R -{ - // Lazily initialize the get / set ptrs - // - // Note that we won't actually use these functions ever, we'll instead be - // testing the pointer's value elsewhere and calling our own functions. - INIT.call_once(|| unsafe { - let get = mem::transmute::(0x1); - let set = mem::transmute::(0x2); - init(get, set); - }); - - // Same as above. - if core::is_get_ptr(0x1) { - struct Reset(*const Cell<*mut u8>, *mut u8); - - impl Drop for Reset { - #[inline] - fn drop(&mut self) { - unsafe { - (*self.0).set(self.1); - } - } - } - - unsafe { - let slot = tls_slot(); - let _reset = Reset(slot, (*slot).get()); - (*slot).set(task as *const _ as *mut u8); - f() - } - } else { - core::set(task, f) - } -} - impl Spawn { /// Waits for the internal future to complete, blocking this thread's /// execution until it does. diff --git a/futures-cpupool/src/pool.rs b/futures-cpupool/src/pool.rs index 16df3cf3ea..f6fab6cf1a 100644 --- a/futures-cpupool/src/pool.rs +++ b/futures-cpupool/src/pool.rs @@ -267,8 +267,8 @@ impl Future for CpuFuture { type Item = T; type Error = E; - fn poll(&mut self) -> Poll { - match self.inner.poll().expect("cannot poll CpuFuture twice") { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + match self.inner.poll(ctx).expect("cannot poll CpuFuture twice") { Async::Ready(Ok(Ok(e))) => Ok(e.into()), Async::Ready(Ok(Err(e))) => Err(e), Async::Ready(Err(e)) => panic::resume_unwind(e), @@ -281,15 +281,15 @@ impl Future for MySender> { type Item = (); type Error = (); - fn poll(&mut self) -> Poll<(), ()> { - if let Ok(Async::Ready(_)) = self.tx.as_mut().unwrap().poll_cancel() { + fn poll(&mut self, ctx: &mut task::Context) -> Poll<(), ()> { + if let Ok(Async::Ready(_)) = self.tx.as_mut().unwrap().poll_cancel(ctx) { if !self.keep_running_flag.load(Ordering::SeqCst) { // Cancelled, bail out return Ok(().into()) } } - let res = match self.fut.poll() { + let res = match self.fut.poll(ctx) { Ok(Async::Ready(e)) => Ok(e), Ok(Async::Pending) => return Ok(Async::Pending), Err(e) => Err(e), diff --git a/futures-executor/src/task_runner.rs b/futures-executor/src/task_runner.rs index 36898999f9..be579d82b8 100644 --- a/futures-executor/src/task_runner.rs +++ b/futures-executor/src/task_runner.rs @@ -297,7 +297,8 @@ impl Future for SpawnedFuture { type Item = bool; type Error = bool; - fn poll(&mut self) -> Poll { - self.inner.with_task_data(|f| f.poll()) + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + //self.inner.with_task_data(|f| f.poll(ctx)) + self.inner.get_mut().poll(ctx) } } diff --git a/futures-executor/tests/current_thread.rs b/futures-executor/tests/current_thread.rs index c483ff5620..95e8eda4a1 100644 --- a/futures-executor/tests/current_thread.rs +++ b/futures-executor/tests/current_thread.rs @@ -9,7 +9,6 @@ use std::time::Duration; use futures::future::lazy; use futures::prelude::*; -use futures::task; use futures_executor::current_thread::*; use futures_channel::oneshot; @@ -83,7 +82,7 @@ impl Future for Never { type Item = (); type Error = (); - fn poll(&mut self) -> Poll<(), ()> { + fn poll(&mut self, _: &mut TaskContext) -> Poll<(), ()> { Ok(Async::Pending) } } @@ -135,7 +134,7 @@ fn tasks_are_scheduled_fairly() { type Item = (); type Error = (); - fn poll(&mut self) -> Poll<(), ()> { + fn poll(&mut self, ctx: &mut TaskContext) -> Poll<(), ()> { let mut state = self.state.borrow_mut(); if self.idx == 0 { @@ -154,7 +153,7 @@ fn tasks_are_scheduled_fairly() { return Ok(().into()); } - task::current().notify(); + ctx.waker().wake(); Ok(Async::Pending) } } diff --git a/futures-sink/src/channel_impls.rs b/futures-sink/src/channel_impls.rs index 9e08efbc1c..539c94fb97 100644 --- a/futures-sink/src/channel_impls.rs +++ b/futures-sink/src/channel_impls.rs @@ -1,4 +1,5 @@ use {Async, Sink, AsyncSink, StartSend, Poll}; +use futures_core::task; use futures_channel::mpsc::{Sender, SendError, UnboundedSender}; fn res_to_async_sink(res: Result<(), T>) -> AsyncSink { @@ -12,15 +13,15 @@ impl Sink for Sender { type SinkItem = T; type SinkError = SendError; - fn start_send(&mut self, msg: T) -> StartSend> { - self.start_send(msg).map(res_to_async_sink) + fn start_send(&mut self, ctx: &mut task::Context, msg: T) -> StartSend> { + self.start_send(ctx, msg).map(res_to_async_sink) } - fn flush(&mut self) -> Poll<(), SendError> { + fn flush(&mut self, _: &mut task::Context) -> Poll<(), SendError> { Ok(Async::Ready(())) } - fn close(&mut self) -> Poll<(), SendError> { + fn close(&mut self, _: &mut task::Context) -> Poll<(), SendError> { Ok(Async::Ready(())) } } @@ -29,15 +30,15 @@ impl Sink for UnboundedSender { type SinkItem = T; type SinkError = SendError; - fn start_send(&mut self, msg: T) -> StartSend> { - self.start_send(msg).map(res_to_async_sink) + fn start_send(&mut self, ctx: &mut task::Context, msg: T) -> StartSend> { + self.start_send(ctx, msg).map(res_to_async_sink) } - fn flush(&mut self) -> Poll<(), SendError> { + fn flush(&mut self, _: &mut task::Context) -> Poll<(), SendError> { Ok(Async::Ready(())) } - fn close(&mut self) -> Poll<(), SendError> { + fn close(&mut self, _: &mut task::Context) -> Poll<(), SendError> { Ok(Async::Ready(())) } } @@ -46,16 +47,16 @@ impl<'a, T> Sink for &'a UnboundedSender { type SinkItem = T; type SinkError = SendError; - fn start_send(&mut self, msg: T) -> StartSend> { + fn start_send(&mut self, _: &mut task::Context, msg: T) -> StartSend> { self.unbounded_send(msg)?; Ok(AsyncSink::Ready) } - fn flush(&mut self) -> Poll<(), SendError> { + fn flush(&mut self, _: &mut task::Context) -> Poll<(), SendError> { Ok(Async::Ready(())) } - fn close(&mut self) -> Poll<(), SendError> { + fn close(&mut self, _: &mut task::Context) -> Poll<(), SendError> { Ok(Async::Ready(())) } } diff --git a/futures-sink/src/lib.rs b/futures-sink/src/lib.rs index 57cae8e46b..831005b545 100644 --- a/futures-sink/src/lib.rs +++ b/futures-sink/src/lib.rs @@ -22,6 +22,7 @@ macro_rules! if_std { } use futures_core::Poll; +use futures_core::task; /// The result of an asynchronous attempt to send a value to a sink. #[derive(Copy, Clone, Debug, PartialEq)] @@ -76,18 +77,18 @@ if_std! { type SinkItem = T; type SinkError = Never; - fn start_send(&mut self, item: Self::SinkItem) + fn start_send(&mut self, _: &mut task::Context, item: Self::SinkItem) -> StartSend { self.push(item); Ok(::AsyncSink::Ready) } - fn flush(&mut self) -> Poll<(), Self::SinkError> { + fn flush(&mut self, _: &mut task::Context) -> Poll<(), Self::SinkError> { Ok(::Async::Ready(())) } - fn close(&mut self) -> Poll<(), Self::SinkError> { + fn close(&mut self, _: &mut task::Context) -> Poll<(), Self::SinkError> { Ok(::Async::Ready(())) } } @@ -100,17 +101,17 @@ if_std! { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: Self::SinkItem) + fn start_send(&mut self, ctx: &mut task::Context, item: Self::SinkItem) -> StartSend { - (**self).start_send(item) + (**self).start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), Self::SinkError> { - (**self).flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + (**self).flush(ctx) } - fn close(&mut self) -> Poll<(), Self::SinkError> { - (**self).close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + (**self).close(ctx) } } } @@ -191,7 +192,7 @@ pub trait Sink { /// /// - It is called outside of the context of a task. /// - A previous call to `start_send` or `flush` yielded an error. - fn start_send(&mut self, item: Self::SinkItem) + fn start_send(&mut self, ctx: &mut task::Context, item: Self::SinkItem) -> StartSend; /// Flush all output from this sink, if necessary. @@ -242,7 +243,7 @@ pub trait Sink { /// In the 0.2 release series of futures this method will be renamed to /// `flush`. For 0.1, however, the breaking change is not happening /// yet. - fn flush(&mut self) -> Poll<(), Self::SinkError>; + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError>; /// A method to indicate that no more values will ever be pushed into this /// sink. @@ -309,23 +310,23 @@ pub trait Sink { /// It is highly recommended to consider this method a required method and /// to implement it whenever you implement `Sink` locally. It is especially /// crucial to be sure to close inner sinks, if applicable. - fn close(&mut self) -> Poll<(), Self::SinkError>; + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError>; } impl<'a, S: ?Sized + Sink> Sink for &'a mut S { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: Self::SinkItem) + fn start_send(&mut self, ctx: &mut task::Context, item: Self::SinkItem) -> StartSend { - (**self).start_send(item) + (**self).start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), Self::SinkError> { - (**self).flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + (**self).flush(ctx) } - fn close(&mut self) -> Poll<(), Self::SinkError> { - (**self).close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + (**self).close(ctx) } } diff --git a/futures-util/src/future/and_then.rs b/futures-util/src/future/and_then.rs index 0b4dbecd4c..d38840ff5f 100644 --- a/futures-util/src/future/and_then.rs +++ b/futures-util/src/future/and_then.rs @@ -1,4 +1,5 @@ use futures_core::{Future, IntoFuture, Poll}; +use futures_core::task; use super::chain::Chain; @@ -29,8 +30,8 @@ impl Future for AndThen type Item = B::Item; type Error = B::Error; - fn poll(&mut self) -> Poll { - self.state.poll(|result, f| { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.state.poll(ctx, |result, f| { result.map(|e| { Err(f(e).into_future()) }) diff --git a/futures-util/src/future/catch_unwind.rs b/futures-util/src/future/catch_unwind.rs index abb6523fd5..080facb87e 100644 --- a/futures-util/src/future/catch_unwind.rs +++ b/futures-util/src/future/catch_unwind.rs @@ -3,6 +3,7 @@ use std::any::Any; use std::panic::{catch_unwind, UnwindSafe}; use futures_core::{Future, Poll, Async}; +use futures_core::task; /// Future for the `catch_unwind` combinator. /// @@ -27,9 +28,9 @@ impl Future for CatchUnwind type Item = Result; type Error = Box; - fn poll(&mut self) -> Poll { + fn poll(&mut self, _ctx: &mut task::Context) -> Poll { let mut future = self.future.take().expect("cannot poll twice"); - let (res, future) = catch_unwind(|| (future.poll(), future))?; + let (res, future) = catch_unwind(|| (future.poll(&mut task::Context::panicking()), future))?; match res { Ok(Async::Pending) => { self.future = Some(future); diff --git a/futures-util/src/future/chain.rs b/futures-util/src/future/chain.rs index b6886bea8d..dba4d54d91 100644 --- a/futures-util/src/future/chain.rs +++ b/futures-util/src/future/chain.rs @@ -1,7 +1,9 @@ use core::mem; use futures_core::{Future, Poll, Async}; +use futures_core::task; +#[must_use = "futures do nothing unless polled"] #[derive(Debug)] pub enum Chain where A: Future { First(A, C), @@ -17,19 +19,19 @@ impl Chain Chain::First(a, c) } - pub fn poll(&mut self, f: F) -> Poll + pub fn poll(&mut self, ctx: &mut task::Context, f: F) -> Poll where F: FnOnce(Result, C) -> Result, B::Error>, { let a_result = match *self { Chain::First(ref mut a, _) => { - match a.poll() { + match a.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), Ok(Async::Ready(t)) => Ok(t), Err(e) => Err(e), } } - Chain::Second(ref mut b) => return b.poll(), + Chain::Second(ref mut b) => return b.poll(ctx), Chain::Done => panic!("cannot poll a chained future twice"), }; let data = match mem::replace(self, Chain::Done) { @@ -39,7 +41,7 @@ impl Chain match f(a_result, data)? { Ok(e) => Ok(Async::Ready(e)), Err(mut b) => { - let ret = b.poll(); + let ret = b.poll(ctx); *self = Chain::Second(b); ret } diff --git a/futures-util/src/future/either.rs b/futures-util/src/future/either.rs index d0b1627888..36a3f89fc2 100644 --- a/futures-util/src/future/either.rs +++ b/futures-util/src/future/either.rs @@ -1,4 +1,5 @@ use futures_core::{Future, Poll}; +use futures_core::task; /// Combines two different futures yielding the same item and error /// types into a single type. @@ -30,10 +31,10 @@ impl Future for Either type Item = A::Item; type Error = A::Error; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { match *self { - Either::A(ref mut a) => a.poll(), - Either::B(ref mut b) => b.poll(), + Either::A(ref mut a) => a.poll(ctx), + Either::B(ref mut b) => b.poll(ctx), } } } diff --git a/futures-util/src/future/empty.rs b/futures-util/src/future/empty.rs index 7c6a6aeefb..b26c1500f9 100644 --- a/futures-util/src/future/empty.rs +++ b/futures-util/src/future/empty.rs @@ -3,6 +3,7 @@ use core::marker; use futures_core::{Future, Poll, Async}; +use futures_core::task; /// A future which is never resolved. /// @@ -25,7 +26,7 @@ impl Future for Empty { type Item = T; type Error = E; - fn poll(&mut self) -> Poll { + fn poll(&mut self, _: &mut task::Context) -> Poll { Ok(Async::Pending) } } diff --git a/futures-util/src/future/flatten.rs b/futures-util/src/future/flatten.rs index 5562216569..a745ff135f 100644 --- a/futures-util/src/future/flatten.rs +++ b/futures-util/src/future/flatten.rs @@ -1,6 +1,7 @@ use core::fmt; use futures_core::{Future, IntoFuture, Poll}; +use futures_core::task; use super::chain::Chain; @@ -42,8 +43,8 @@ impl Future for Flatten type Item = <::Item as IntoFuture>::Item; type Error = <::Item as IntoFuture>::Error; - fn poll(&mut self) -> Poll { - self.state.poll(|a, ()| { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.state.poll(ctx, |a, ()| { let future = a?.into_future(); Ok(Err(future)) }) diff --git a/futures-util/src/future/flatten_stream.rs b/futures-util/src/future/flatten_stream.rs index 7b0c92970c..8dc63ec797 100644 --- a/futures-util/src/future/flatten_stream.rs +++ b/futures-util/src/future/flatten_stream.rs @@ -1,6 +1,7 @@ use core::fmt; use futures_core::{Async, Future, Poll, Stream}; +use futures_core::task; /// Future for the `flatten_stream` combinator, flattening a /// future-of-a-stream to get just the result of the final stream as a stream. @@ -56,11 +57,11 @@ impl Stream for FlattenStream type Item = ::Item; type Error = ::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { loop { let (next_state, ret_opt) = match self.state { State::Future(ref mut f) => { - match f.poll() { + match f.poll(ctx) { Ok(Async::Pending) => { // State is not changed, early return. return Ok(Async::Pending) @@ -79,7 +80,7 @@ impl Stream for FlattenStream State::Stream(ref mut s) => { // Just forward call to the stream, // do not track its state. - return s.poll(); + return s.poll(ctx); } State::Eof => { (State::Done, Some(Ok(Async::Ready(None)))) diff --git a/futures-util/src/future/from_err.rs b/futures-util/src/future/from_err.rs index f762645443..6ebaa6c4cd 100644 --- a/futures-util/src/future/from_err.rs +++ b/futures-util/src/future/from_err.rs @@ -1,6 +1,7 @@ use core::marker::PhantomData; use futures_core::{Future, Poll, Async}; +use futures_core::task; /// Future for the `from_err` combinator, changing the error type of a future. /// @@ -25,8 +26,8 @@ impl> Future for FromErr { type Item = A::Item; type Error = E; - fn poll(&mut self) -> Poll { - let e = match self.future.poll() { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + let e = match self.future.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), other => other, }; diff --git a/futures-util/src/future/fuse.rs b/futures-util/src/future/fuse.rs index 6496032fa8..1b151f8dcd 100644 --- a/futures-util/src/future/fuse.rs +++ b/futures-util/src/future/fuse.rs @@ -1,4 +1,5 @@ use futures_core::{Future, Poll, Async}; +use futures_core::task; /// A future which "fuses" a future once it's been resolved. /// @@ -23,8 +24,8 @@ impl Future for Fuse { type Item = A::Item; type Error = A::Error; - fn poll(&mut self) -> Poll { - let res = self.future.as_mut().map(|f| f.poll()); + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + let res = self.future.as_mut().map(|f| f.poll(ctx)); match res.unwrap_or(Ok(Async::Pending)) { res @ Ok(Async::Ready(_)) | res @ Err(_) => { diff --git a/futures-util/src/future/inspect.rs b/futures-util/src/future/inspect.rs index 50332aa52f..b2fde137d5 100644 --- a/futures-util/src/future/inspect.rs +++ b/futures-util/src/future/inspect.rs @@ -1,4 +1,5 @@ use futures_core::{Future, Poll, Async}; +use futures_core::task; /// Do something with the item of a future, passing it on. /// @@ -27,8 +28,8 @@ impl Future for Inspect type Item = A::Item; type Error = A::Error; - fn poll(&mut self) -> Poll { - match self.future.poll() { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + match self.future.poll(ctx) { Ok(Async::Pending) => Ok(Async::Pending), Ok(Async::Ready(e)) => { (self.f.take().expect("cannot poll Inspect twice"))(&e); diff --git a/futures-util/src/future/into_stream.rs b/futures-util/src/future/into_stream.rs index 902ea4d00f..ba49567010 100644 --- a/futures-util/src/future/into_stream.rs +++ b/futures-util/src/future/into_stream.rs @@ -1,7 +1,9 @@ use futures_core::{Async, Poll, Future, Stream}; +use futures_core::task; /// Future that forwards one element from the underlying future /// (whether it is success of error) and emits EOF after that. +#[must_use = "futures do nothing unless polled"] #[derive(Debug)] pub struct IntoStream { future: Option @@ -17,11 +19,11 @@ impl Stream for IntoStream { type Item = F::Item; type Error = F::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { let ret = match self.future { None => return Ok(Async::Ready(None)), Some(ref mut future) => { - match future.poll() { + match future.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), Err(e) => Err(e), Ok(Async::Ready(r)) => Ok(r), diff --git a/futures-util/src/future/join.rs b/futures-util/src/future/join.rs index 72337f8e0d..0b3a35f42f 100644 --- a/futures-util/src/future/join.rs +++ b/futures-util/src/future/join.rs @@ -4,6 +4,7 @@ use core::fmt; use core::mem; use futures_core::{Future, Poll, Async}; +use futures_core::task; macro_rules! generate { ($( @@ -63,8 +64,8 @@ macro_rules! generate { type Item = (A::Item, $($B::Item),*); type Error = A::Error; - fn poll(&mut self) -> Poll { - let mut all_done = match self.a.poll() { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + let mut all_done = match self.a.poll(ctx) { Ok(done) => done, Err(e) => { self.erase(); @@ -72,7 +73,7 @@ macro_rules! generate { } }; $( - all_done = match self.$B.poll() { + all_done = match self.$B.poll(ctx) { Ok(done) => all_done && done, Err(e) => { self.erase(); @@ -151,9 +152,9 @@ enum MaybeDone { } impl MaybeDone { - fn poll(&mut self) -> Result { + fn poll(&mut self, ctx: &mut task::Context) -> Result { let res = match *self { - MaybeDone::NotYet(ref mut a) => a.poll()?, + MaybeDone::NotYet(ref mut a) => a.poll(ctx)?, MaybeDone::Done(_) => return Ok(true), MaybeDone::Gone => panic!("cannot poll Join twice"), }; diff --git a/futures-util/src/future/join_all.rs b/futures-util/src/future/join_all.rs index 2622139962..abb8f86684 100644 --- a/futures-util/src/future/join_all.rs +++ b/futures-util/src/future/join_all.rs @@ -7,6 +7,7 @@ use std::fmt; use std::mem; use futures_core::{Future, IntoFuture, Poll, Async}; +use futures_core::task; #[derive(Debug)] enum ElemState where T: Future { @@ -93,13 +94,13 @@ impl Future for JoinAll type Error = ::Error; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> 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() { + match t.poll(ctx) { Ok(Async::Ready(v)) => Ok(v), Ok(Async::Pending) => { all_done = false; diff --git a/futures-util/src/future/lazy.rs b/futures-util/src/future/lazy.rs index f23a623043..e8d7376a8a 100644 --- a/futures-util/src/future/lazy.rs +++ b/futures-util/src/future/lazy.rs @@ -4,6 +4,7 @@ use core::mem; use futures_core::{Future, IntoFuture, Poll}; +use futures_core::task; /// A future which defers creation of the actual future until a callback is /// scheduled. @@ -82,7 +83,7 @@ impl Future for Lazy type Item = R::Item; type Error = R::Error; - fn poll(&mut self) -> Poll { - self.get().poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.get().poll(ctx) } } diff --git a/futures-util/src/future/loop_fn.rs b/futures-util/src/future/loop_fn.rs index 24251374af..30e1945d28 100644 --- a/futures-util/src/future/loop_fn.rs +++ b/futures-util/src/future/loop_fn.rs @@ -1,6 +1,7 @@ //! Definition of the `LoopFn` combinator, implementing `Future` loops. use futures_core::{Async, Future, IntoFuture, Poll}; +use futures_core::task; /// The status of a `loop_fn` loop. #[derive(Debug)] @@ -16,6 +17,7 @@ pub enum Loop { /// A future implementing a tail-recursive loop. /// /// Created by the `loop_fn` function. +#[must_use = "futures do nothing unless polled"] #[derive(Debug)] pub struct LoopFn where A: IntoFuture { future: A::Future, @@ -92,9 +94,9 @@ impl Future for LoopFn type Item = T; type Error = A::Error; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { loop { - match try_ready!(self.future.poll()) { + match try_ready!(self.future.poll(ctx)) { Loop::Break(x) => return Ok(Async::Ready(x)), Loop::Continue(s) => self.future = (self.func)(s).into_future(), } diff --git a/futures-util/src/future/map.rs b/futures-util/src/future/map.rs index 79a3872cc9..9e0fde5519 100644 --- a/futures-util/src/future/map.rs +++ b/futures-util/src/future/map.rs @@ -1,4 +1,5 @@ use futures_core::{Future, Poll, Async}; +use futures_core::task; /// Future for the `map` combinator, changing the type of a future. /// @@ -26,8 +27,8 @@ impl Future for Map type Item = U; type Error = A::Error; - fn poll(&mut self) -> Poll { - let e = match self.future.poll() { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + let e = match self.future.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), Ok(Async::Ready(e)) => Ok(e), Err(e) => Err(e), diff --git a/futures-util/src/future/map_err.rs b/futures-util/src/future/map_err.rs index f545e843c2..4701b75eff 100644 --- a/futures-util/src/future/map_err.rs +++ b/futures-util/src/future/map_err.rs @@ -1,4 +1,5 @@ use futures_core::{Future, Poll, Async}; +use futures_core::task; /// Future for the `map_err` combinator, changing the error type of a future. /// @@ -26,8 +27,8 @@ impl Future for MapErr type Item = A::Item; type Error = U; - fn poll(&mut self) -> Poll { - let e = match self.future.poll() { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + let e = match self.future.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), other => other, }; diff --git a/futures-util/src/future/mod.rs b/futures-util/src/future/mod.rs index ede13e63e1..b55d514b16 100644 --- a/futures-util/src/future/mod.rs +++ b/futures-util/src/future/mod.rs @@ -723,15 +723,15 @@ pub trait FutureExt: Future { /// /// # fn main() { /// let mut future = future::ok::(2); - /// assert_eq!(future.poll(), Ok(Async::Ready(2))); + /// assert_eq!(future.poll(&mut TaskContext), Ok(Async::Ready(2))); /// /// // Normally, a call such as this would panic: /// //future.poll(); /// /// // This, however, is guaranteed to not panic /// let mut future = future::ok::(2).fuse(); - /// assert_eq!(future.poll(), Ok(Async::Ready(2))); - /// assert_eq!(future.poll(), Ok(Async::Pending)); + /// assert_eq!(future.poll(&mut TaskContext), Ok(Async::Ready(2))); + /// assert_eq!(future.poll(&mut TaskContext), Ok(Async::Pending)); /// # } /// ``` fn fuse(self) -> Fuse diff --git a/futures-util/src/future/or_else.rs b/futures-util/src/future/or_else.rs index bcc124855e..9b1e3f81ab 100644 --- a/futures-util/src/future/or_else.rs +++ b/futures-util/src/future/or_else.rs @@ -1,4 +1,5 @@ use futures_core::{Future, IntoFuture, Poll}; +use futures_core::task; use super::chain::Chain; /// Future for the `or_else` combinator, chaining a computation onto the end of @@ -28,8 +29,8 @@ impl Future for OrElse type Item = B::Item; type Error = B::Error; - fn poll(&mut self) -> Poll { - self.state.poll(|a, f| { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.state.poll(ctx, |a, f| { match a { Ok(item) => Ok(Ok(item)), Err(e) => Ok(Err(f(e).into_future())) diff --git a/futures-util/src/future/poll_fn.rs b/futures-util/src/future/poll_fn.rs index e88cc9a286..c9ff5dc77d 100644 --- a/futures-util/src/future/poll_fn.rs +++ b/futures-util/src/future/poll_fn.rs @@ -1,6 +1,7 @@ //! Definition of the `PollFn` adapter combinator use futures_core::{Future, Poll}; +use futures_core::task; /// A future which adapts a function returning `Poll`. /// @@ -23,7 +24,7 @@ pub struct PollFn { /// use futures::future::poll_fn; /// /// # fn main() { -/// fn read_line() -> Poll { +/// fn read_line(ctx: &mut TaskContext) -> Poll { /// Ok(Async::Ready("Hello, World!".into())) /// } /// @@ -31,18 +32,18 @@ pub struct PollFn { /// # } /// ``` pub fn poll_fn(f: F) -> PollFn - where F: FnMut() -> Poll + where F: FnMut(&mut task::Context) -> Poll { PollFn { inner: f } } impl Future for PollFn - where F: FnMut() -> Poll + where F: FnMut(&mut task::Context) -> Poll { type Item = T; type Error = E; - fn poll(&mut self) -> Poll { - (self.inner)() + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + (self.inner)(ctx) } } diff --git a/futures-util/src/future/select.rs b/futures-util/src/future/select.rs index fedc7d9d53..beb0f5dfcd 100644 --- a/futures-util/src/future/select.rs +++ b/futures-util/src/future/select.rs @@ -1,4 +1,5 @@ use futures_core::{Future, Poll, Async}; +use futures_core::task; /// Future for the `select` combinator, waiting for one of two futures to /// complete. @@ -42,14 +43,14 @@ impl Future for Select type Item = (A::Item, SelectNext); type Error = (A::Error, SelectNext); - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { let (ret, is_a) = match self.inner { Some((ref mut a, ref mut b)) => { - match a.poll() { + match a.poll(ctx) { Err(a) => (Err(a), true), Ok(Async::Ready(a)) => (Ok(a), true), Ok(Async::Pending) => { - match b.poll() { + match b.poll(ctx) { Err(a) => (Err(a), false), Ok(Async::Ready(a)) => (Ok(a), false), Ok(Async::Pending) => return Ok(Async::Pending), @@ -77,10 +78,10 @@ impl Future for SelectNext type Item = A::Item; type Error = A::Error; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { match self.inner { - OneOf::A(ref mut a) => a.poll(), - OneOf::B(ref mut b) => b.poll(), + OneOf::A(ref mut a) => a.poll(ctx), + OneOf::B(ref mut b) => b.poll(ctx), } } } diff --git a/futures-util/src/future/select2.rs b/futures-util/src/future/select2.rs index eb20d46993..c7e1b6a5aa 100644 --- a/futures-util/src/future/select2.rs +++ b/futures-util/src/future/select2.rs @@ -1,4 +1,5 @@ use futures_core::{Future, Poll, Async}; +use futures_core::task; use future::Either; @@ -22,12 +23,12 @@ impl Future for Select2 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) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { let (mut a, mut b) = self.inner.take().expect("cannot poll Select2 twice"); - match a.poll() { + match a.poll(ctx) { Err(e) => Err(Either::A((e, b))), Ok(Async::Ready(x)) => Ok(Async::Ready(Either::A((x, b)))), - Ok(Async::Pending) => match b.poll() { + Ok(Async::Pending) => match b.poll(ctx) { Err(e) => Err(Either::B((e, a))), Ok(Async::Ready(x)) => Ok(Async::Ready(Either::B((x, a)))), Ok(Async::Pending) => { diff --git a/futures-util/src/future/select_all.rs b/futures-util/src/future/select_all.rs index 45cd5717b0..160e343a13 100644 --- a/futures-util/src/future/select_all.rs +++ b/futures-util/src/future/select_all.rs @@ -5,6 +5,7 @@ use std::mem; use std::prelude::v1::*; use futures_core::{Future, IntoFuture, Poll, Async}; +use futures_core::task; /// Future for the `select_all` combinator, waiting for one of any of a list of /// futures to complete. @@ -48,9 +49,9 @@ impl Future for SelectAll type Item = (A::Item, usize, Vec); type Error = (A::Error, usize, Vec); - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { let item = self.inner.iter_mut().enumerate().filter_map(|(i, f)| { - match f.poll() { + match f.poll(ctx) { 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/select_ok.rs b/futures-util/src/future/select_ok.rs index 3f1f3941d4..ab31c1c325 100644 --- a/futures-util/src/future/select_ok.rs +++ b/futures-util/src/future/select_ok.rs @@ -5,6 +5,7 @@ use std::mem; use std::prelude::v1::*; use futures_core::{Future, IntoFuture, Poll, Async}; +use futures_core::task; /// Future for the `select_ok` combinator, waiting for one of any of a list of /// futures to successfully complete. Unlike `select_all`, this future ignores all @@ -44,11 +45,11 @@ impl Future for SelectOk where A: Future { type Item = (A::Item, Vec); type Error = A::Error; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> 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() { + match f.poll(ctx) { 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/shared.rs b/futures-util/src/future/shared.rs index 8dbe553a07..d1431edbc5 100644 --- a/futures-util/src/future/shared.rs +++ b/futures-util/src/future/shared.rs @@ -29,7 +29,7 @@ use std::sync::atomic::Ordering::SeqCst; use std::collections::HashMap; use futures_core::{Future, Poll, Async}; -use futures_core::task::{self, Notify, Spawn, Task}; +use futures_core::task::{self, Notify, Spawn}; /// A future that is cloneable and can be polled in multiple threads. /// Use `Future::shared()` method to convert any future into a `Shared` future. @@ -61,7 +61,7 @@ struct Inner { struct Notifier { state: AtomicUsize, - waiters: Mutex>, + waiters: Mutex>, } const IDLE: usize = 0; @@ -99,9 +99,9 @@ impl Shared where F: Future { } } - fn set_waiter(&mut self) { + fn set_waiter(&mut self, ctx: &mut task::Context) { let mut waiters = self.inner.notifier.waiters.lock().unwrap(); - waiters.insert(self.waiter, task::current()); + waiters.insert(self.waiter, ctx.waker()); } unsafe fn clone_result(&self) -> Result, SharedError> { @@ -125,8 +125,8 @@ impl Future for Shared type Item = SharedItem; type Error = SharedError; - fn poll(&mut self) -> Poll { - self.set_waiter(); + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.set_waiter(ctx); match self.inner.notifier.state.compare_and_swap(IDLE, POLLING, SeqCst) { IDLE => { @@ -229,7 +229,7 @@ impl Notify for Notifier { let waiters = mem::replace(&mut *self.waiters.lock().unwrap(), HashMap::new()); for (_, waiter) in waiters { - waiter.notify(); + waiter.wake(); } } } diff --git a/futures-util/src/future/then.rs b/futures-util/src/future/then.rs index d49ab9edd9..abb70310cb 100644 --- a/futures-util/src/future/then.rs +++ b/futures-util/src/future/then.rs @@ -1,4 +1,5 @@ use futures_core::{Future, IntoFuture, Poll}; +use futures_core::task; use super::chain::Chain; /// Future for the `then` combinator, chaining computations on the end of @@ -28,8 +29,8 @@ impl Future for Then type Item = B::Item; type Error = B::Error; - fn poll(&mut self) -> Poll { - self.state.poll(|a, f| { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.state.poll(ctx, |a, f| { Ok(Err(f(a).into_future())) }) } diff --git a/futures-util/src/lock.rs b/futures-util/src/lock.rs index 19dfbe8c61..61153a932f 100644 --- a/futures-util/src/lock.rs +++ b/futures-util/src/lock.rs @@ -12,7 +12,7 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; use futures_core::{Async, Future, Poll}; -use futures_core::task::{self, Task}; +use futures_core::task; /// A type of futures-powered synchronization primitive which is a mutex between /// two possible owners. @@ -74,12 +74,7 @@ impl BiLock { /// `Async::Pending`. In this case the current task will also be scheduled /// to receive a notification when the lock would otherwise become /// available. - /// - /// # Panics - /// - /// This function will panic if called outside the context of a future's - /// task. - pub fn poll_lock(&self) -> Async> { + pub fn poll_lock(&self, ctx: &mut task::Context) -> Async> { loop { match self.inner.state.swap(1, SeqCst) { // Woohoo, we grabbed the lock! @@ -91,11 +86,11 @@ impl BiLock { // A task was previously blocked on this lock, likely our task, // so we need to update that task. n => unsafe { - drop(Box::from_raw(n as *mut Task)); + drop(Box::from_raw(n as *mut task::Waker)); } } - let me = Box::new(task::current()); + let me = Box::new(ctx.waker()); let me = Box::into_raw(me) as usize; match self.inner.state.compare_exchange(1, me, SeqCst, SeqCst) { @@ -107,7 +102,7 @@ impl BiLock { // and before the compare_exchange. Deallocate what we just // allocated and go through the loop again. Err(0) => unsafe { - drop(Box::from_raw(me as *mut Task)); + drop(Box::from_raw(me as *mut task::Waker)); }, // The top of this loop set the previous state to 1, so if we @@ -163,7 +158,7 @@ impl BiLock { // Another task has parked themselves on this lock, let's wake them // up as its now their turn. n => unsafe { - Box::from_raw(n as *mut Task).notify(); + Box::from_raw(n as *mut task::Waker).wake(); } } } @@ -236,6 +231,7 @@ impl<'a, T> Drop for BiLockGuard<'a, T> { /// Future returned by `BiLock::lock` which will resolve when the lock is /// acquired. +#[must_use = "futures do nothing unless polled"] #[derive(Debug)] pub struct BiLockAcquire { inner: Option>, @@ -245,8 +241,8 @@ impl Future for BiLockAcquire { type Item = BiLockAcquired; type Error = (); - fn poll(&mut self) -> Poll, ()> { - match self.inner.as_ref().expect("cannot poll after Ready").poll_lock() { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, ()> { + match self.inner.as_ref().expect("cannot poll after Ready").poll_lock(ctx) { Async::Ready(r) => { mem::forget(r); } diff --git a/futures-util/src/sink/buffer.rs b/futures-util/src/sink/buffer.rs index 0253c5db38..c981656ecc 100644 --- a/futures-util/src/sink/buffer.rs +++ b/futures-util/src/sink/buffer.rs @@ -1,6 +1,7 @@ use std::collections::VecDeque; use futures_core::{Poll, Async, Stream}; +use futures_core::task; use futures_sink::{StartSend, AsyncSink, Sink}; /// Sink for the `Sink::buffer` combinator, which buffers up to some fixed @@ -42,13 +43,13 @@ impl Buffer { self.sink } - fn try_empty_buffer(&mut self) -> Poll<(), S::SinkError> { + fn try_empty_buffer(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { while let Some(item) = self.buf.pop_front() { - if let AsyncSink::Pending(item) = self.sink.start_send(item)? { + if let AsyncSink::Pending(item) = self.sink.start_send(ctx, item)? { self.buf.push_front(item); // ensure that we attempt to complete any pushes we've started - self.sink.flush()?; + self.sink.flush(ctx)?; return Ok(Async::Pending); } @@ -63,8 +64,8 @@ impl Stream for Buffer where S: Sink + Stream { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - self.sink.poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + self.sink.poll(ctx) } } @@ -72,12 +73,12 @@ impl Sink for Buffer { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { + fn start_send(&mut self, ctx: &mut task::Context, item: Self::SinkItem) -> StartSend { if self.cap == 0 { - return self.sink.start_send(item); + return self.sink.start_send(ctx, item); } - self.try_empty_buffer()?; + self.try_empty_buffer(ctx)?; if self.buf.len() == self.cap { return Ok(AsyncSink::Pending(item)); } @@ -85,25 +86,25 @@ impl Sink for Buffer { Ok(AsyncSink::Ready) } - fn flush(&mut self) -> Poll<(), Self::SinkError> { + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { if self.cap == 0 { - return self.sink.flush(); + return self.sink.flush(ctx); } - try_ready!(self.try_empty_buffer()); + try_ready!(self.try_empty_buffer(ctx)); debug_assert!(self.buf.is_empty()); - self.sink.flush() + self.sink.flush(ctx) } - fn close(&mut self) -> Poll<(), Self::SinkError> { + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { if self.cap == 0 { - return self.sink.close(); + return self.sink.close(ctx); } if self.buf.len() > 0 { - try_ready!(self.try_empty_buffer()); + try_ready!(self.try_empty_buffer(ctx)); } assert_eq!(self.buf.len(), 0); - self.sink.close() + self.sink.close(ctx) } } diff --git a/futures-util/src/sink/close.rs b/futures-util/src/sink/close.rs index 3f7f26f48f..a39eba91bf 100644 --- a/futures-util/src/sink/close.rs +++ b/futures-util/src/sink/close.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Async, Future}; +use futures_core::task; use futures_sink::Sink; /// Future for the `close` combinator, which polls the sink until all data has @@ -40,9 +41,9 @@ impl Future for Close { type Item = S; type Error = S::SinkError; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { let mut sink = self.sink.take().expect("Attempted to poll Close after it completed"); - if sink.close()?.is_ready() { + if sink.close(ctx)?.is_ready() { Ok(Async::Ready(sink)) } else { self.sink = Some(sink); diff --git a/futures-util/src/sink/fanout.rs b/futures-util/src/sink/fanout.rs index ae885ad14a..8cfd1d2ffe 100644 --- a/futures-util/src/sink/fanout.rs +++ b/futures-util/src/sink/fanout.rs @@ -2,6 +2,7 @@ use core::fmt::{Debug, Formatter, Result as FmtResult}; use core::mem::replace; use futures_core::{Async, Poll}; +use futures_core::task; use futures_sink::{AsyncSink, Sink, StartSend}; /// Sink that clones incoming items and forwards them to two sinks at the same time. @@ -52,24 +53,25 @@ impl Sink for Fanout fn start_send( &mut self, + ctx: &mut task::Context, item: Self::SinkItem ) -> StartSend { // Attempt to complete processing any outstanding requests. - self.left.keep_flushing()?; - self.right.keep_flushing()?; + self.left.keep_flushing(ctx)?; + self.right.keep_flushing(ctx)?; // Only if both downstream sinks are ready, start sending the next item. if self.left.is_ready() && self.right.is_ready() { - self.left.state = self.left.sink.start_send(item.clone())?; - self.right.state = self.right.sink.start_send(item)?; + self.left.state = self.left.sink.start_send(ctx, item.clone())?; + self.right.state = self.right.sink.start_send(ctx, item)?; Ok(AsyncSink::Ready) } else { Ok(AsyncSink::Pending(item)) } } - fn flush(&mut self) -> Poll<(), Self::SinkError> { - let left_async = self.left.flush()?; - let right_async = self.right.flush()?; + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + let left_async = self.left.flush(ctx)?; + let right_async = self.right.flush(ctx)?; // Only if both downstream sinks are ready, signal readiness. if left_async.is_ready() && right_async.is_ready() { Ok(Async::Ready(())) @@ -78,9 +80,9 @@ impl Sink for Fanout } } - fn close(&mut self) -> Poll<(), Self::SinkError> { - let left_async = self.left.close()?; - let right_async = self.right.close()?; + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + let left_async = self.left.close(ctx)?; + let right_async = self.right.close(ctx)?; // Only if both downstream sinks are ready, signal readiness. if left_async.is_ready() && right_async.is_ready() { Ok(Async::Ready(())) @@ -105,16 +107,16 @@ impl Downstream { self.state.is_ready() } - fn keep_flushing(&mut self) -> Result<(), S::SinkError> { + fn keep_flushing(&mut self, ctx: &mut task::Context) -> Result<(), S::SinkError> { if let AsyncSink::Pending(item) = replace(&mut self.state, AsyncSink::Ready) { - self.state = self.sink.start_send(item)?; + self.state = self.sink.start_send(ctx, item)?; } Ok(()) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.keep_flushing()?; - let async = self.sink.flush()?; + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.keep_flushing(ctx)?; + let async = self.sink.flush(ctx)?; // Only if all values have been sent _and_ the underlying // sink is completely flushed, signal readiness. if self.state.is_ready() && async.is_ready() { @@ -124,11 +126,11 @@ impl Downstream { } } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.keep_flushing()?; + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.keep_flushing(ctx)?; // If all items have been flushed, initiate close. if self.state.is_ready() { - self.sink.close() + self.sink.close(ctx) } else { Ok(Async::Pending) } diff --git a/futures-util/src/sink/flush.rs b/futures-util/src/sink/flush.rs index 01c95baf52..bd42a4baf6 100644 --- a/futures-util/src/sink/flush.rs +++ b/futures-util/src/sink/flush.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Async, Future}; +use futures_core::task; use futures_sink::Sink; /// Future for the `flush` combinator, which polls the sink until all data @@ -43,9 +44,9 @@ impl Future for Flush { type Item = S; type Error = S::SinkError; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { let mut sink = self.sink.take().expect("Attempted to poll Flush after it completed"); - if sink.flush()?.is_ready() { + if sink.flush(ctx)?.is_ready() { Ok(Async::Ready(sink)) } else { self.sink = Some(sink); diff --git a/futures-util/src/sink/from_err.rs b/futures-util/src/sink/from_err.rs index 9467889891..eacf062ac1 100644 --- a/futures-util/src/sink/from_err.rs +++ b/futures-util/src/sink/from_err.rs @@ -1,6 +1,7 @@ use core::marker::PhantomData; use futures_core::{Stream, Poll}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A sink combinator to change the error type of a sink. @@ -49,16 +50,16 @@ impl Sink for SinkFromErr type SinkItem = S::SinkItem; type SinkError = E; - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - self.sink.start_send(item).map_err(|e| e.into()) + fn start_send(&mut self, ctx: &mut task::Context, item: Self::SinkItem) -> StartSend { + self.sink.start_send(ctx, item).map_err(|e| e.into()) } - fn flush(&mut self) -> Poll<(), Self::SinkError> { - self.sink.flush().map_err(|e| e.into()) + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + self.sink.flush(ctx).map_err(|e| e.into()) } - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.sink.close().map_err(|e| e.into()) + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + self.sink.close(ctx).map_err(|e| e.into()) } } @@ -66,7 +67,7 @@ impl Stream for SinkFromErr { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - self.sink.poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + self.sink.poll(ctx) } } diff --git a/futures-util/src/sink/map_err.rs b/futures-util/src/sink/map_err.rs index 89c5729ff6..98a4befc56 100644 --- a/futures-util/src/sink/map_err.rs +++ b/futures-util/src/sink/map_err.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// Sink for the `Sink::sink_map_err` combinator. @@ -40,16 +41,16 @@ impl Sink for SinkMapErr type SinkItem = S::SinkItem; type SinkError = E; - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - self.sink.start_send(item).map_err(|e| self.f.take().expect("cannot use MapErr after an error")(e)) + fn start_send(&mut self, ctx: &mut task::Context, item: Self::SinkItem) -> StartSend { + self.sink.start_send(ctx, item).map_err(|e| self.f.take().expect("cannot use MapErr after an error")(e)) } - fn flush(&mut self) -> Poll<(), Self::SinkError> { - self.sink.flush().map_err(|e| self.f.take().expect("cannot use MapErr after an error")(e)) + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + self.sink.flush(ctx).map_err(|e| self.f.take().expect("cannot use MapErr after an error")(e)) } - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.sink.close().map_err(|e| self.f.take().expect("cannot use MapErr after an error")(e)) + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + self.sink.close(ctx).map_err(|e| self.f.take().expect("cannot use MapErr after an error")(e)) } } @@ -57,7 +58,7 @@ impl Stream for SinkMapErr { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - self.sink.poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + self.sink.poll(ctx) } } diff --git a/futures-util/src/sink/send.rs b/futures-util/src/sink/send.rs index 08c4fc144f..c5cfcee209 100644 --- a/futures-util/src/sink/send.rs +++ b/futures-util/src/sink/send.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Async, Future}; +use futures_core::task; use futures_sink::{Sink, AsyncSink}; /// Future for the `Sink::send` combinator, which sends a value to a sink and @@ -45,9 +46,9 @@ impl Future for Send { type Item = S; type Error = S::SinkError; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { if let Some(item) = self.item.take() { - if let AsyncSink::Pending(item) = self.sink_mut().start_send(item)? { + if let AsyncSink::Pending(item) = self.sink_mut().start_send(ctx, item)? { self.item = Some(item); return Ok(Async::Pending); } @@ -55,7 +56,7 @@ impl Future for Send { // we're done sending the item, but want to block on flushing the // sink - try_ready!(self.sink_mut().flush()); + try_ready!(self.sink_mut().flush(ctx)); // now everything's emptied, so return the sink for further use Ok(Async::Ready(self.take_sink())) diff --git a/futures-util/src/sink/send_all.rs b/futures-util/src/sink/send_all.rs index 09cb4a0e79..1994c7580e 100644 --- a/futures-util/src/sink/send_all.rs +++ b/futures-util/src/sink/send_all.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Async, Future, Stream}; +use futures_core::task; use futures_sink::{Sink, AsyncSink}; use stream::{StreamExt, Fuse}; @@ -47,9 +48,9 @@ impl SendAll (sink, fuse.into_inner()) } - fn try_start_send(&mut self, item: U::Item) -> Poll<(), T::SinkError> { + fn try_start_send(&mut self, ctx: &mut task::Context, item: U::Item) -> Poll<(), T::SinkError> { debug_assert!(self.buffered.is_none()); - if let AsyncSink::Pending(item) = self.sink_mut().start_send(item)? { + if let AsyncSink::Pending(item) = self.sink_mut().start_send(ctx, item)? { self.buffered = Some(item); return Ok(Async::Pending) } @@ -65,22 +66,22 @@ impl Future for SendAll type Item = (T, U); type Error = T::SinkError; - fn poll(&mut self) -> Poll<(T, U), T::SinkError> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll<(T, U), T::SinkError> { // 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.take() { - try_ready!(self.try_start_send(item)) + try_ready!(self.try_start_send(ctx, item)) } loop { - match self.stream_mut().poll()? { - Async::Ready(Some(item)) => try_ready!(self.try_start_send(item)), + match self.stream_mut().poll(ctx)? { + Async::Ready(Some(item)) => try_ready!(self.try_start_send(ctx, item)), Async::Ready(None) => { - try_ready!(self.sink_mut().flush()); + try_ready!(self.sink_mut().flush(ctx)); return Ok(Async::Ready(self.take_result())) } Async::Pending => { - try_ready!(self.sink_mut().flush()); + try_ready!(self.sink_mut().flush(ctx)); return Ok(Async::Pending) } } diff --git a/futures-util/src/sink/with.rs b/futures-util/src/sink/with.rs index ceb333b11b..eaad1d4480 100644 --- a/futures-util/src/sink/with.rs +++ b/futures-util/src/sink/with.rs @@ -2,6 +2,7 @@ use core::mem; use core::marker::PhantomData; use futures_core::{IntoFuture, Future, Poll, Async, Stream}; +use futures_core::task; use futures_sink::{Sink, AsyncSink, StartSend}; /// Sink for the `Sink::with` combinator, chaining a computation to run *prior* @@ -59,8 +60,8 @@ impl Stream for With type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - self.sink.poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + self.sink.poll(ctx) } } @@ -88,12 +89,12 @@ impl With self.sink } - fn poll(&mut self) -> Poll<(), Fut::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll<(), Fut::Error> { loop { match mem::replace(&mut self.state, State::Empty) { State::Empty => break, State::Process(mut fut) => { - match fut.poll()? { + match fut.poll(ctx)? { Async::Ready(item) => { self.state = State::Buffered(item); } @@ -104,7 +105,7 @@ impl With } } State::Buffered(item) => { - if let AsyncSink::Pending(item) = self.sink.start_send(item)? { + if let AsyncSink::Pending(item) = self.sink.start_send(ctx, item)? { self.state = State::Buffered(item); break } @@ -129,24 +130,24 @@ impl Sink for With type SinkItem = U; type SinkError = Fut::Error; - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - if self.poll()?.is_not_ready() { + fn start_send(&mut self, ctx: &mut task::Context, item: Self::SinkItem) -> StartSend { + if self.poll(ctx)?.is_not_ready() { return Ok(AsyncSink::Pending(item)) } self.state = State::Process((self.f)(item).into_future()); Ok(AsyncSink::Ready) } - fn flush(&mut self) -> Poll<(), Fut::Error> { + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Fut::Error> { // poll ourselves first, to push data downward - let me_ready = self.poll()?; + let me_ready = self.poll(ctx)?; // always propagate `flush` downward to attempt to make progress - try_ready!(self.sink.flush()); + try_ready!(self.sink.flush(ctx)); Ok(me_ready) } - fn close(&mut self) -> Poll<(), Fut::Error> { - try_ready!(self.poll()); - Ok(self.sink.close()?) + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Fut::Error> { + try_ready!(self.poll(ctx)); + Ok(self.sink.close(ctx)?) } } diff --git a/futures-util/src/sink/with_flat_map.rs b/futures-util/src/sink/with_flat_map.rs index 25aadba339..1b200afbc5 100644 --- a/futures-util/src/sink/with_flat_map.rs +++ b/futures-util/src/sink/with_flat_map.rs @@ -1,6 +1,7 @@ use core::marker::PhantomData; use futures_core::{Poll, Async, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend, AsyncSink}; /// Sink for the `Sink::with_flat_map` combinator, chaining a computation that returns an iterator @@ -59,16 +60,16 @@ where self.sink } - fn try_empty_stream(&mut self) -> Poll<(), S::SinkError> { + fn try_empty_stream(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { if let Some(x) = self.buffer.take() { - if let AsyncSink::Pending(x) = self.sink.start_send(x)? { + if let AsyncSink::Pending(x) = self.sink.start_send(ctx, x)? { self.buffer = Some(x); return Ok(Async::Pending); } } if let Some(mut stream) = self.stream.take() { - while let Some(x) = try_ready!(stream.poll()) { - if let AsyncSink::Pending(x) = self.sink.start_send(x)? { + while let Some(x) = try_ready!(stream.poll(ctx)) { + if let AsyncSink::Pending(x) = self.sink.start_send(ctx, x)? { self.stream = Some(stream); self.buffer = Some(x); return Ok(Async::Pending); @@ -87,8 +88,8 @@ where { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - self.sink.poll() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + self.sink.poll(ctx) } } @@ -100,26 +101,26 @@ where { type SinkItem = U; type SinkError = S::SinkError; - fn start_send(&mut self, i: Self::SinkItem) -> StartSend { - if self.try_empty_stream()?.is_not_ready() { + fn start_send(&mut self, ctx: &mut task::Context, i: Self::SinkItem) -> StartSend { + if self.try_empty_stream(ctx)?.is_not_ready() { return Ok(AsyncSink::Pending(i)); } assert!(self.stream.is_none()); self.stream = Some((self.f)(i)); - self.try_empty_stream()?; + self.try_empty_stream(ctx)?; Ok(AsyncSink::Ready) } - fn flush(&mut self) -> Poll<(), Self::SinkError> { - if self.try_empty_stream()?.is_not_ready() { + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + if self.try_empty_stream(ctx)?.is_not_ready() { return Ok(Async::Pending); } - self.sink.flush() + self.sink.flush(ctx) } - fn close(&mut self) -> Poll<(), Self::SinkError> { - if self.try_empty_stream()?.is_not_ready() { + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + if self.try_empty_stream(ctx)?.is_not_ready() { return Ok(Async::Pending); } assert!(self.stream.is_none()); - self.sink.close() + self.sink.close(ctx) } } diff --git a/futures-util/src/stream/and_then.rs b/futures-util/src/stream/and_then.rs index b1c32c43f3..30dc11845f 100644 --- a/futures-util/src/stream/and_then.rs +++ b/futures-util/src/stream/and_then.rs @@ -1,4 +1,5 @@ use futures_core::{IntoFuture, Future, Poll, Async, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A stream combinator which chains a computation onto values produced by a @@ -61,16 +62,16 @@ impl Sink for AndThen type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -82,16 +83,16 @@ impl Stream for AndThen type Item = U::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { if self.future.is_none() { - let item = match try_ready!(self.stream.poll()) { + let item = match try_ready!(self.stream.poll(ctx)) { None => return Ok(Async::Ready(None)), Some(e) => e, }; self.future = Some((self.f)(item).into_future()); } assert!(self.future.is_some()); - match self.future.as_mut().unwrap().poll() { + match self.future.as_mut().unwrap().poll(ctx) { Ok(Async::Ready(e)) => { self.future = None; Ok(Async::Ready(Some(e))) diff --git a/futures-util/src/stream/buffer_unordered.rs b/futures-util/src/stream/buffer_unordered.rs index 404211a328..e2c5132721 100644 --- a/futures-util/src/stream/buffer_unordered.rs +++ b/futures-util/src/stream/buffer_unordered.rs @@ -1,6 +1,7 @@ use std::fmt; use futures_core::{Async, IntoFuture, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; use stream::{Fuse, FuturesUnordered}; @@ -81,11 +82,11 @@ impl Stream for BufferUnordered type Item = ::Item; type Error = ::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { // First up, try to spawn off as many futures as possible by filling up // our slab of futures. while self.queue.len() < self.max { - let future = match self.stream.poll()? { + let future = match self.stream.poll(ctx)? { Async::Ready(Some(s)) => s.into_future(), Async::Ready(None) | Async::Pending => break, @@ -95,7 +96,7 @@ impl Stream for BufferUnordered } // Try polling a new future - if let Some(val) = try_ready!(self.queue.poll()) { + if let Some(val) = try_ready!(self.queue.poll(ctx)) { return Ok(Async::Ready(Some(val))); } @@ -118,15 +119,15 @@ impl Sink for BufferUnordered type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } diff --git a/futures-util/src/stream/buffered.rs b/futures-util/src/stream/buffered.rs index c95196e6d9..d10499a8e0 100644 --- a/futures-util/src/stream/buffered.rs +++ b/futures-util/src/stream/buffered.rs @@ -1,6 +1,7 @@ use std::fmt; use futures_core::{Async, IntoFuture, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; use stream::{Fuse, FuturesOrdered}; @@ -84,16 +85,16 @@ impl Sink for Buffered type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -104,11 +105,11 @@ impl Stream for Buffered type Item = ::Item; type Error = ::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { // First up, try to spawn off as many futures as possible by filling up // our slab of futures. while self.queue.len() < self.max { - let future = match self.stream.poll()? { + let future = match self.stream.poll(ctx)? { Async::Ready(Some(s)) => s.into_future(), Async::Ready(None) | Async::Pending => break, @@ -118,7 +119,7 @@ impl Stream for Buffered } // Try polling a new future - if let Some(val) = try_ready!(self.queue.poll()) { + if let Some(val) = try_ready!(self.queue.poll(ctx)) { return Ok(Async::Ready(Some(val))); } diff --git a/futures-util/src/stream/catch_unwind.rs b/futures-util/src/stream/catch_unwind.rs index 37a8d88a95..e345c21d89 100644 --- a/futures-util/src/stream/catch_unwind.rs +++ b/futures-util/src/stream/catch_unwind.rs @@ -4,6 +4,7 @@ use std::panic::{catch_unwind, UnwindSafe}; use std::mem; use futures_core::{Poll, Async, Stream}; +use futures_core::task; /// Stream for the `catch_unwind` combinator. /// @@ -35,7 +36,7 @@ impl Stream for CatchUnwind type Item = Result; type Error = Box; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, _ctx: &mut task::Context) -> Poll, Self::Error> { let mut stream = match mem::replace(&mut self.state, CatchUnwindState::Eof) { CatchUnwindState::Done => panic!("cannot poll after eof"), CatchUnwindState::Eof => { @@ -44,7 +45,7 @@ impl Stream for CatchUnwind } CatchUnwindState::Stream(stream) => stream, }; - let res = catch_unwind(|| (stream.poll(), stream)); + let res = catch_unwind(|| (stream.poll(&mut task::Context::panicking()), stream)); match res { Err(e) => Err(e), // and state is already Eof Ok((poll, stream)) => { diff --git a/futures-util/src/stream/chain.rs b/futures-util/src/stream/chain.rs index 4d13741f30..8b7cbba110 100644 --- a/futures-util/src/stream/chain.rs +++ b/futures-util/src/stream/chain.rs @@ -1,6 +1,7 @@ use core::mem; use futures_core::{Stream, Async, Poll}; +use futures_core::task; /// State of chain stream. #[derive(Debug)] @@ -35,14 +36,14 @@ impl Stream for Chain type Item = S1::Item; type Error = S1::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { loop { match self.state { - State::First(ref mut s1, ref _s2) => match s1.poll() { + State::First(ref mut s1, ref _s2) => match s1.poll(ctx) { Ok(Async::Ready(None)) => (), // roll x => return x, }, - State::Second(ref mut s2) => return s2.poll(), + State::Second(ref mut s2) => return s2.poll(ctx), State::Temp => unreachable!(), } diff --git a/futures-util/src/stream/chunks.rs b/futures-util/src/stream/chunks.rs index 43971435e6..8efa6e5d79 100644 --- a/futures-util/src/stream/chunks.rs +++ b/futures-util/src/stream/chunks.rs @@ -2,6 +2,7 @@ use std::mem; use std::prelude::v1::*; use futures_core::{Async, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; use stream::Fuse; @@ -40,16 +41,16 @@ impl Sink for Chunks type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -90,14 +91,14 @@ impl Stream for Chunks type Item = Vec<::Item>; type Error = ::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { if let Some(err) = self.err.take() { return Err(err) } let cap = self.items.capacity(); loop { - match self.stream.poll() { + match self.stream.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), // Push the item into the buffer and check whether it is full. diff --git a/futures-util/src/stream/collect.rs b/futures-util/src/stream/collect.rs index 83ea598383..8d85090f13 100644 --- a/futures-util/src/stream/collect.rs +++ b/futures-util/src/stream/collect.rs @@ -3,6 +3,7 @@ use std::prelude::v1::*; use std::mem; use futures_core::{Future, Poll, Async, Stream}; +use futures_core::task; /// A future which collects all of the values of a stream into a vector. /// @@ -35,9 +36,9 @@ impl Future for Collect type Item = Vec; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { loop { - match self.stream.poll() { + match self.stream.poll(ctx) { Ok(Async::Ready(Some(e))) => self.items.push(e), Ok(Async::Ready(None)) => return Ok(Async::Ready(self.finish())), Ok(Async::Pending) => return Ok(Async::Pending), diff --git a/futures-util/src/stream/concat.rs b/futures-util/src/stream/concat.rs index 391b1aa1ce..252aeb4bb9 100644 --- a/futures-util/src/stream/concat.rs +++ b/futures-util/src/stream/concat.rs @@ -3,6 +3,7 @@ use core::fmt::{Debug, Formatter, Result as FmtResult}; use core::default::Default; use futures_core::{Async, Future, Poll, Stream}; +use futures_core::task; /// A stream combinator to concatenate the results of a stream into the first /// yielded item. @@ -40,8 +41,8 @@ impl Future for Concat type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll { - self.inner.poll().map(|a| { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + self.inner.poll(ctx).map(|a| { match a { Async::Pending => Async::Pending, Async::Ready(None) => Async::Ready(Default::default()), @@ -78,9 +79,9 @@ impl Future for ConcatSafe type Item = Option; type Error = S::Error; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { loop { - match self.stream.poll() { + match self.stream.poll(ctx) { Ok(Async::Ready(Some(i))) => { match self.extend { Inner::First => { diff --git a/futures-util/src/stream/empty.rs b/futures-util/src/stream/empty.rs index 58cc3f1cf3..b2d0780114 100644 --- a/futures-util/src/stream/empty.rs +++ b/futures-util/src/stream/empty.rs @@ -1,6 +1,7 @@ use core::marker; use futures_core::{Stream, Poll, Async}; +use futures_core::task; /// A stream which contains no elements. /// @@ -22,7 +23,7 @@ impl Stream for Empty { type Item = T; type Error = E; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, _: &mut task::Context) -> Poll, Self::Error> { Ok(Async::Ready(None)) } } diff --git a/futures-util/src/stream/filter.rs b/futures-util/src/stream/filter.rs index 385ad8a223..c575e330bb 100644 --- a/futures-util/src/stream/filter.rs +++ b/futures-util/src/stream/filter.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Future, IntoFuture, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A stream combinator used to filter the results of a stream and only yield @@ -68,16 +69,16 @@ impl Sink for Filter type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -89,10 +90,10 @@ impl Stream for Filter type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { loop { if self.pending.is_none() { - let item = match try_ready!(self.stream.poll()) { + let item = match try_ready!(self.stream.poll(ctx)) { Some(e) => e, None => return Ok(Async::Ready(None)), }; @@ -100,7 +101,7 @@ impl Stream for Filter self.pending = Some((fut, item)); } - match self.pending.as_mut().unwrap().0.poll() { + match self.pending.as_mut().unwrap().0.poll(ctx) { Ok(Async::Ready(true)) => { let (_, item) = self.pending.take().unwrap(); return Ok(Async::Ready(Some(item))); diff --git a/futures-util/src/stream/filter_map.rs b/futures-util/src/stream/filter_map.rs index db47436a2f..97b68c3b31 100644 --- a/futures-util/src/stream/filter_map.rs +++ b/futures-util/src/stream/filter_map.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Future, IntoFuture, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A combinator used to filter the results of a stream and simultaneously map @@ -67,16 +68,16 @@ impl Sink for FilterMap type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -88,10 +89,10 @@ impl Stream for FilterMap type Item = B; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { loop { if self.pending.is_none() { - let item = match try_ready!(self.stream.poll()) { + let item = match try_ready!(self.stream.poll(ctx)) { Some(e) => e, None => return Ok(Async::Ready(None)), }; @@ -99,7 +100,7 @@ impl Stream for FilterMap self.pending = Some(fut); } - match self.pending.as_mut().unwrap().poll() { + match self.pending.as_mut().unwrap().poll(ctx) { x @ Ok(Async::Ready(Some(_))) => { self.pending = None; return x diff --git a/futures-util/src/stream/flatten.rs b/futures-util/src/stream/flatten.rs index 2ab0800f01..93ae982faa 100644 --- a/futures-util/src/stream/flatten.rs +++ b/futures-util/src/stream/flatten.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Async, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A combinator used to flatten a stream-of-streams into one long stream of @@ -56,16 +57,16 @@ impl Sink for Flatten type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -77,16 +78,16 @@ impl Stream for Flatten type Item = ::Item; type Error = ::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { loop { if self.next.is_none() { - match try_ready!(self.stream.poll()) { + match try_ready!(self.stream.poll(ctx)) { Some(e) => self.next = Some(e), None => return Ok(Async::Ready(None)), } } assert!(self.next.is_some()); - match self.next.as_mut().unwrap().poll() { + match self.next.as_mut().unwrap().poll(ctx) { Ok(Async::Ready(None)) => self.next = None, other => return other, } diff --git a/futures-util/src/stream/fold.rs b/futures-util/src/stream/fold.rs index af96b76e5e..4c46167390 100644 --- a/futures-util/src/stream/fold.rs +++ b/futures-util/src/stream/fold.rs @@ -1,6 +1,7 @@ use core::mem; use futures_core::{Future, Poll, IntoFuture, Async, Stream}; +use futures_core::task; /// A future used to collect all the results of a stream into one generic type. /// @@ -45,12 +46,12 @@ impl Future for Fold type Item = T; type Error = S::Error; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { loop { match mem::replace(&mut self.state, State::Empty) { State::Empty => panic!("cannot poll Fold twice"), State::Ready(state) => { - match self.stream.poll()? { + match self.stream.poll(ctx)? { Async::Ready(Some(e)) => { let future = (self.f)(state, e); let future = future.into_future(); @@ -64,7 +65,7 @@ impl Future for Fold } } State::Processing(mut fut) => { - match fut.poll()? { + match fut.poll(ctx)? { Async::Ready(state) => self.state = State::Ready(state), Async::Pending => { self.state = State::Processing(fut); diff --git a/futures-util/src/stream/for_each.rs b/futures-util/src/stream/for_each.rs index f23d4e451f..e2ee29d8ee 100644 --- a/futures-util/src/stream/for_each.rs +++ b/futures-util/src/stream/for_each.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Future, IntoFuture, Poll, Stream}; +use futures_core::task; /// A stream combinator which executes a unit closure over each item on a /// stream. @@ -32,10 +33,10 @@ impl Future for ForEach type Item = S; type Error = S::Error; - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { loop { if let Some(mut fut) = self.fut.take() { - if fut.poll()?.is_not_ready() { + if fut.poll(ctx)?.is_not_ready() { self.fut = Some(fut); return Ok(Async::Pending); } @@ -43,7 +44,7 @@ impl Future for ForEach match self.stream { Some(ref mut stream) => { - match try_ready!(stream.poll()) { + match try_ready!(stream.poll(ctx)) { Some(e) => self.fut = Some((self.f)(e).into_future()), None => break, } diff --git a/futures-util/src/stream/forward.rs b/futures-util/src/stream/forward.rs index 3bb686c837..4090a9245c 100644 --- a/futures-util/src/stream/forward.rs +++ b/futures-util/src/stream/forward.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Future, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, AsyncSink}; use stream::{StreamExt, Fuse}; @@ -63,11 +64,11 @@ impl Forward (fuse.into_inner(), sink) } - fn try_start_send(&mut self, item: T::Item) -> Poll<(), U::SinkError> { + fn try_start_send(&mut self, ctx: &mut task::Context, item: T::Item) -> Poll<(), U::SinkError> { debug_assert!(self.buffered.is_none()); if let AsyncSink::Pending(item) = self.sink_mut() .take().expect("Attempted to poll Forward after completion") - .start_send(item)? + .start_send(ctx, item)? { self.buffered = Some(item); return Ok(Async::Pending) @@ -84,25 +85,25 @@ impl Future for Forward type Item = (T, U); type Error = T::Error; - fn poll(&mut self) -> Poll<(T, U), T::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll<(T, U), T::Error> { // 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.take() { - try_ready!(self.try_start_send(item)) + try_ready!(self.try_start_send(ctx, item)) } loop { match self.stream_mut() .take().expect("Attempted to poll Forward after completion") - .poll()? + .poll(ctx)? { - Async::Ready(Some(item)) => try_ready!(self.try_start_send(item)), + Async::Ready(Some(item)) => try_ready!(self.try_start_send(ctx, item)), Async::Ready(None) => { - try_ready!(self.sink_mut().take().expect("Attempted to poll Forward after completion").flush()); + try_ready!(self.sink_mut().take().expect("Attempted to poll Forward after completion").flush(ctx)); return Ok(Async::Ready(self.take_result())) } Async::Pending => { - try_ready!(self.sink_mut().take().expect("Attempted to poll Forward after completion").flush()); + try_ready!(self.sink_mut().take().expect("Attempted to poll Forward after completion").flush(ctx)); return Ok(Async::Pending) } } diff --git a/futures-util/src/stream/from_err.rs b/futures-util/src/stream/from_err.rs index 61084c6e4a..48287a4b52 100644 --- a/futures-util/src/stream/from_err.rs +++ b/futures-util/src/stream/from_err.rs @@ -1,6 +1,7 @@ use core::marker::PhantomData; use futures_core::{Async, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A stream combinator to change the error type of a stream. @@ -52,8 +53,8 @@ impl> Stream for FromErr { type Item = S::Item; type Error = E; - fn poll(&mut self) -> Poll, E> { - let e = match self.stream.poll() { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, E> { + let e = match self.stream.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), other => other, }; @@ -66,15 +67,15 @@ impl Sink for FromErr { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: Self::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), Self::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), Self::SinkError> { + self.stream.close(ctx) } } diff --git a/futures-util/src/stream/fuse.rs b/futures-util/src/stream/fuse.rs index faa7e2c5ac..a86a00dc5d 100644 --- a/futures-util/src/stream/fuse.rs +++ b/futures-util/src/stream/fuse.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Async, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A stream which "fuse"s a stream once it's terminated. @@ -20,16 +21,16 @@ impl Sink for Fuse type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -41,11 +42,11 @@ impl Stream for Fuse { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { if self.done { Ok(Async::Ready(None)) } else { - let r = self.stream.poll(); + let r = self.stream.poll(ctx); if let Ok(Async::Ready(None)) = r { self.done = true; } diff --git a/futures-util/src/stream/future.rs b/futures-util/src/stream/future.rs index 271434a721..294c0e63d3 100644 --- a/futures-util/src/stream/future.rs +++ b/futures-util/src/stream/future.rs @@ -1,4 +1,5 @@ use futures_core::{Future, Poll, Async, Stream}; +use futures_core::task; /// A combinator used to temporarily convert a stream into a future. /// @@ -57,10 +58,10 @@ impl Future for StreamFuture { type Item = (Option, S); type Error = (S::Error, S); - fn poll(&mut self) -> Poll { + fn poll(&mut self, ctx: &mut task::Context) -> Poll { let item = { let s = self.stream.as_mut().expect("polling StreamFuture twice"); - match s.poll() { + match s.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), Ok(Async::Ready(e)) => Ok(e), Err(e) => Err(e), diff --git a/futures-util/src/stream/futures_ordered.rs b/futures-util/src/stream/futures_ordered.rs index a1f8d10eb7..730a0ae7ae 100644 --- a/futures-util/src/stream/futures_ordered.rs +++ b/futures-util/src/stream/futures_ordered.rs @@ -4,9 +4,11 @@ use std::fmt::{self, Debug}; use std::iter::FromIterator; use futures_core::{Async, Future, IntoFuture, Poll, Stream}; +use futures_core::task; use stream::FuturesUnordered; +#[must_use = "futures do nothing unless polled"] #[derive(Debug)] struct OrderWrapper { item: T, @@ -40,8 +42,8 @@ impl Future for OrderWrapper type Item = OrderWrapper; type Error = T::Error; - fn poll(&mut self) -> Poll { - let result = try_ready!(self.item.poll()); + fn poll(&mut self, ctx: &mut task::Context) -> Poll { + let result = try_ready!(self.item.poll(ctx)); Ok(Async::Ready(OrderWrapper { item: result, index: self.index @@ -167,10 +169,10 @@ impl Stream for FuturesOrdered type Item = T::Item; type Error = T::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { // Get any completed futures from the unordered set. loop { - match self.in_progress.poll()? { + match self.in_progress.poll(ctx)? { Async::Ready(Some(result)) => self.queued_results.push(result), Async::Ready(None) | Async::Pending => break, } diff --git a/futures-util/src/stream/futures_unordered.rs b/futures-util/src/stream/futures_unordered.rs index e14728b633..9a0e634330 100644 --- a/futures-util/src/stream/futures_unordered.rs +++ b/futures-util/src/stream/futures_unordered.rs @@ -12,7 +12,7 @@ use std::sync::{Arc, Weak}; use std::usize; use futures_core::{Stream, Future, Poll, Async}; -use futures_core::task::{self, AtomicTask, Notify, UnsafeNotify, NotifyHandle}; +use futures_core::task::{self, AtomicWaker, Notify, UnsafeNotify, NotifyHandle}; /// An unbounded set of futures. /// @@ -78,7 +78,7 @@ unsafe impl Sync for FuturesUnordered {} #[allow(missing_debug_implementations)] struct Inner { // The task using `FuturesUnordered`. - parent: AtomicTask, + parent: AtomicWaker, // Head/tail of the readiness queue head_readiness: AtomicPtr>, @@ -130,7 +130,7 @@ impl FuturesUnordered }); let stub_ptr = &*stub as *const Node; let inner = Arc::new(Inner { - parent: AtomicTask::new(), + parent: AtomicWaker::new(), head_readiness: AtomicPtr::new(stub_ptr as *mut _), tail_readiness: UnsafeCell::new(stub_ptr), stub: stub, @@ -266,9 +266,9 @@ impl Stream for FuturesUnordered type Item = T::Item; type Error = T::Error; - fn poll(&mut self) -> Poll, T::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, T::Error> { // Ensure `parent` is correctly set. - self.inner.parent.register(); + self.inner.parent.register(ctx); loop { let node = match unsafe { self.inner.dequeue() } { @@ -283,7 +283,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. - task::current().notify(); + ctx.waker().wake(); return Ok(Async::Pending); } Dequeue::Data(node) => node, @@ -352,14 +352,14 @@ impl Stream for FuturesUnordered // queue of ready futures. // // Critically though `Node` won't actually access `T`, the - // future, while it's floating around inside of `Task` + // future, while it's floating around inside of `Waker` // instances. These structs will basically just use `T` to size // the internal allocation, appropriately accessing fields and // deallocating the node if need be. let res = { let notify = NodeToHandle(bomb.node.as_ref().unwrap()); - task::with_notify(¬ify, 0, || { - future.poll() + task::with_notify(ctx, ¬ify, 0, |c| { + future.poll(c) }) }; @@ -388,9 +388,9 @@ impl Debug for FuturesUnordered { impl Drop for FuturesUnordered { fn drop(&mut self) { // When a `FuturesUnordered` is dropped we want to drop all futures associated - // with it. At the same time though there may be tons of `Task` handles + // with it. At the same time though there may be tons of `Waker` handles // flying around which contain `Node` references inside them. We'll - // let those naturally get deallocated when the `Task` itself goes out + // let those naturally get deallocated when the `Waker` itself goes out // of scope or gets notified. unsafe { while !self.head_all.is_null() { @@ -620,7 +620,7 @@ impl Node { let prev = me.queued.swap(true, SeqCst); if !prev { inner.enqueue(&**me); - inner.parent.notify(); + inner.parent.wake(); } } } diff --git a/futures-util/src/stream/inspect.rs b/futures-util/src/stream/inspect.rs index 305091e185..047e424ab7 100644 --- a/futures-util/src/stream/inspect.rs +++ b/futures-util/src/stream/inspect.rs @@ -1,4 +1,5 @@ use futures_core::{Stream, Poll, Async}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// Do something with the items of a stream, passing it on. @@ -53,16 +54,16 @@ impl Sink for Inspect type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -73,8 +74,8 @@ impl Stream for Inspect type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - match try_ready!(self.stream.poll()) { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + match try_ready!(self.stream.poll(ctx)) { Some(e) => { (self.inspect)(&e); Ok(Async::Ready(Some(e))) diff --git a/futures-util/src/stream/inspect_err.rs b/futures-util/src/stream/inspect_err.rs index 7dda6fb993..e0e1847b72 100644 --- a/futures-util/src/stream/inspect_err.rs +++ b/futures-util/src/stream/inspect_err.rs @@ -1,4 +1,5 @@ use futures_core::{Stream, Poll}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// Do something with the error of a stream, passing it on. @@ -53,16 +54,16 @@ impl Sink for InspectErr type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -73,8 +74,8 @@ impl Stream for InspectErr type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - self.stream.poll().map_err(|e| { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + self.stream.poll(ctx).map_err(|e| { (self.inspect)(&e); e }) diff --git a/futures-util/src/stream/iter_ok.rs b/futures-util/src/stream/iter_ok.rs index febcacb62e..b9ddf9c2e1 100644 --- a/futures-util/src/stream/iter_ok.rs +++ b/futures-util/src/stream/iter_ok.rs @@ -1,6 +1,7 @@ use core::marker; use futures_core::{Async, Poll, Stream}; +use futures_core::task; /// A stream which is just a shim over an underlying instance of `Iterator`. /// @@ -47,7 +48,7 @@ impl Stream for IterOk type Item = I::Item; type Error = E; - fn poll(&mut self) -> Poll, E> { + fn poll(&mut self, _: &mut task::Context) -> Poll, E> { Ok(Async::Ready(self.iter.next())) } } diff --git a/futures-util/src/stream/iter_result.rs b/futures-util/src/stream/iter_result.rs index dfd599a3ae..19e1349d84 100644 --- a/futures-util/src/stream/iter_result.rs +++ b/futures-util/src/stream/iter_result.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Poll, Stream}; +use futures_core::task; /// A stream which is just a shim over an underlying instance of `Iterator`. /// @@ -52,7 +53,7 @@ where type Item = T; type Error = E; - fn poll(&mut self) -> Poll, E> { + fn poll(&mut self, _: &mut task::Context) -> Poll, E> { match self.iter.next() { Some(Ok(e)) => Ok(Async::Ready(Some(e))), Some(Err(e)) => Err(e), diff --git a/futures-util/src/stream/map.rs b/futures-util/src/stream/map.rs index b504b652ba..35948ce591 100644 --- a/futures-util/src/stream/map.rs +++ b/futures-util/src/stream/map.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A stream combinator which will change the type of a stream from one @@ -54,16 +55,16 @@ impl Sink for Map type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -74,8 +75,8 @@ impl Stream for Map type Item = U; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - let option = try_ready!(self.stream.poll()); + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + let option = try_ready!(self.stream.poll(ctx)); Ok(Async::Ready(option.map(&mut self.f))) } } diff --git a/futures-util/src/stream/map_err.rs b/futures-util/src/stream/map_err.rs index 5d714dbfef..8583cb21da 100644 --- a/futures-util/src/stream/map_err.rs +++ b/futures-util/src/stream/map_err.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A stream combinator which will change the error type of a stream from one @@ -54,16 +55,16 @@ impl Sink for MapErr type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -74,7 +75,7 @@ impl Stream for MapErr type Item = S::Item; type Error = U; - fn poll(&mut self) -> Poll, U> { - self.stream.poll().map_err(&mut self.f) + fn poll(&mut self, ctx: &mut task::Context) -> Poll, U> { + self.stream.poll(ctx).map_err(&mut self.f) } } diff --git a/futures-util/src/stream/once.rs b/futures-util/src/stream/once.rs index af82efc8df..b79262842d 100644 --- a/futures-util/src/stream/once.rs +++ b/futures-util/src/stream/once.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Async, Stream}; +use futures_core::task; /// A stream which emits single element and then EOF. /// @@ -36,7 +37,7 @@ impl Stream for Once { type Item = T; type Error = E; - fn poll(&mut self) -> Poll, E> { + fn poll(&mut self, _: &mut task::Context) -> Poll, E> { match self.0.take() { Some(Ok(e)) => Ok(Async::Ready(Some(e))), Some(Err(e)) => Err(e), diff --git a/futures-util/src/stream/or_else.rs b/futures-util/src/stream/or_else.rs index fa5a860b3c..ee27aec478 100644 --- a/futures-util/src/stream/or_else.rs +++ b/futures-util/src/stream/or_else.rs @@ -1,4 +1,5 @@ use futures_core::{IntoFuture, Future, Poll, Async, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; /// A stream combinator which chains a computation onto errors produced by a @@ -34,16 +35,16 @@ impl Sink for OrElse type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -55,9 +56,9 @@ impl Stream for OrElse type Item = S::Item; type Error = U::Error; - fn poll(&mut self) -> Poll, U::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, U::Error> { if self.future.is_none() { - let item = match self.stream.poll() { + let item = match self.stream.poll(ctx) { Ok(Async::Ready(e)) => return Ok(Async::Ready(e)), Ok(Async::Pending) => return Ok(Async::Pending), Err(e) => e, @@ -65,7 +66,7 @@ impl Stream for OrElse self.future = Some((self.f)(item).into_future()); } assert!(self.future.is_some()); - match self.future.as_mut().unwrap().poll() { + match self.future.as_mut().unwrap().poll(ctx) { Ok(Async::Ready(e)) => { self.future = None; Ok(Async::Ready(Some(e))) diff --git a/futures-util/src/stream/peek.rs b/futures-util/src/stream/peek.rs index 65afe3caab..46f3043ccc 100644 --- a/futures-util/src/stream/peek.rs +++ b/futures-util/src/stream/peek.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Poll, Stream}; +use futures_core::task; use futures_sink::{Sink, StartSend}; use stream::{StreamExt, Fuse}; @@ -30,16 +31,16 @@ impl Sink for Peekable type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -47,11 +48,11 @@ impl Stream for Peekable { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { if let Some(item) = self.peeked.take() { return Ok(Async::Ready(Some(item))) } - self.stream.poll() + self.stream.poll(ctx) } } @@ -61,11 +62,11 @@ 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(&mut self) -> Poll, S::Error> { + pub fn peek(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { if self.peeked.is_some() { return Ok(Async::Ready(self.peeked.as_ref())) } - match try_ready!(self.poll()) { + match try_ready!(self.poll(ctx)) { None => Ok(Async::Ready(None)), Some(item) => { self.peeked = Some(item); diff --git a/futures-util/src/stream/poll_fn.rs b/futures-util/src/stream/poll_fn.rs index 42425d2211..f7921372c4 100644 --- a/futures-util/src/stream/poll_fn.rs +++ b/futures-util/src/stream/poll_fn.rs @@ -1,6 +1,7 @@ //! Definition of the `PollFn` combinator use futures_core::{Stream, Poll}; +use futures_core::task; /// A stream which adapts a function returning `Poll`. /// @@ -25,7 +26,7 @@ pub struct PollFn { /// # fn main() { /// let mut counter = 1usize; /// -/// let read_stream = poll_fn(move || -> Poll, std::io::Error> { +/// let read_stream = poll_fn(move |_| -> Poll, std::io::Error> { /// if counter == 0 { return Ok(Async::Ready(None)); } /// counter -= 1; /// Ok(Async::Ready(Some("Hello, World!".to_owned()))) @@ -34,19 +35,19 @@ pub struct PollFn { /// ``` pub fn poll_fn(f: F) -> PollFn where - F: FnMut() -> Poll, E>, + F: FnMut(&mut task::Context) -> Poll, E>, { PollFn { inner: f } } impl Stream for PollFn where - F: FnMut() -> Poll, E>, + F: FnMut(&mut task::Context) -> Poll, E>, { type Item = T; type Error = E; - fn poll(&mut self) -> Poll, E> { - (self.inner)() + fn poll(&mut self, ctx: &mut task::Context) -> Poll, E> { + (self.inner)(ctx) } } diff --git a/futures-util/src/stream/repeat.rs b/futures-util/src/stream/repeat.rs index 4cd4ce249f..4ce12a64f8 100644 --- a/futures-util/src/stream/repeat.rs +++ b/futures-util/src/stream/repeat.rs @@ -1,6 +1,7 @@ use core::marker; use futures_core::{Stream, Async, Poll}; +use futures_core::task; /// Stream that produces the same element repeatedly. /// @@ -49,7 +50,7 @@ impl Stream for Repeat type Item = T; type Error = E; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, _: &mut task::Context) -> Poll, Self::Error> { Ok(Async::Ready(Some(self.item.clone()))) } } diff --git a/futures-util/src/stream/select.rs b/futures-util/src/stream/select.rs index abfe95163b..a003aece23 100644 --- a/futures-util/src/stream/select.rs +++ b/futures-util/src/stream/select.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Poll, Stream}; +use futures_core::task; use stream::{StreamExt, Fuse}; @@ -33,7 +34,7 @@ impl Stream for Select type Item = S1::Item; type Error = S1::Error; - fn poll(&mut self) -> Poll, S1::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S1::Error> { let (a, b) = if self.flag { (&mut self.stream2 as &mut Stream, &mut self.stream1 as &mut Stream) @@ -43,13 +44,13 @@ impl Stream for Select }; self.flag = !self.flag; - let a_done = match a.poll()? { + let a_done = match a.poll(ctx)? { Async::Ready(Some(item)) => return Ok(Some(item).into()), Async::Ready(None) => true, Async::Pending => false, }; - match b.poll()? { + match b.poll(ctx)? { Async::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/select_all.rs b/futures-util/src/stream/select_all.rs index 9780b6190d..f4722a1030 100644 --- a/futures-util/src/stream/select_all.rs +++ b/futures-util/src/stream/select_all.rs @@ -3,6 +3,7 @@ use std::fmt::{self, Debug}; use futures_core::{Async, Poll, Stream}; +use futures_core::task; use stream::{StreamExt, StreamFuture, FuturesUnordered}; @@ -65,8 +66,8 @@ impl Stream for SelectAll { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, Self::Error> { - match self.inner.poll().map_err(|(err, _)| err)? { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { + match self.inner.poll(ctx).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/skip.rs b/futures-util/src/stream/skip.rs index d1d1af7207..061f02a088 100644 --- a/futures-util/src/stream/skip.rs +++ b/futures-util/src/stream/skip.rs @@ -1,4 +1,5 @@ use futures_core::{Poll, Async, Stream}; +use futures_core::task; use futures_sink::{StartSend, Sink}; /// A stream combinator which skips a number of elements before continuing. @@ -52,16 +53,16 @@ impl Sink for Skip type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -71,14 +72,14 @@ impl Stream for Skip type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { while self.remaining > 0 { - match try_ready!(self.stream.poll()) { + match try_ready!(self.stream.poll(ctx)) { Some(_) => self.remaining -= 1, None => return Ok(Async::Ready(None)), } } - self.stream.poll() + self.stream.poll(ctx) } } diff --git a/futures-util/src/stream/skip_while.rs b/futures-util/src/stream/skip_while.rs index 6c28b2e9d0..a2c7921113 100644 --- a/futures-util/src/stream/skip_while.rs +++ b/futures-util/src/stream/skip_while.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Poll, IntoFuture, Future, Stream}; +use futures_core::task; use futures_sink::{StartSend, Sink}; /// A stream combinator which skips elements of a stream while a predicate @@ -59,16 +60,16 @@ impl Sink for SkipWhile type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -80,21 +81,21 @@ impl Stream for SkipWhile type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { if self.done_skipping { - return self.stream.poll(); + return self.stream.poll(ctx); } loop { if self.pending.is_none() { - let item = match try_ready!(self.stream.poll()) { + let item = match try_ready!(self.stream.poll(ctx)) { Some(e) => e, None => return Ok(Async::Ready(None)), }; self.pending = Some(((self.pred)(&item).into_future(), item)); } - match self.pending.as_mut().unwrap().0.poll() { + match self.pending.as_mut().unwrap().0.poll(ctx) { Ok(Async::Ready(true)) => self.pending = None, Ok(Async::Ready(false)) => { let (_, item) = self.pending.take().unwrap(); diff --git a/futures-util/src/stream/split.rs b/futures-util/src/stream/split.rs index 65dcd1df38..20a5f8857f 100644 --- a/futures-util/src/stream/split.rs +++ b/futures-util/src/stream/split.rs @@ -3,11 +3,13 @@ use std::error::Error; use std::fmt; use futures_core::{Stream, Poll, Async}; +use futures_core::task; use futures_sink::{StartSend, Sink, AsyncSink}; use lock::BiLock; /// A `Stream` part of the split pair +#[must_use = "streams do nothing unless polled"] #[derive(Debug)] pub struct SplitStream(BiLock); @@ -24,9 +26,9 @@ impl Stream for SplitStream { type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { - match self.0.poll_lock() { - Async::Ready(mut inner) => inner.poll(), + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { + match self.0.poll_lock(ctx) { + Async::Ready(mut inner) => inner.poll(ctx), Async::Pending => Ok(Async::Pending), } } @@ -51,25 +53,25 @@ impl Sink for SplitSink { type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { - match self.0.poll_lock() { - Async::Ready(mut inner) => inner.start_send(item), + match self.0.poll_lock(ctx) { + Async::Ready(mut inner) => inner.start_send(ctx, item), Async::Pending => Ok(AsyncSink::Pending(item)), } } - fn flush(&mut self) -> Poll<(), S::SinkError> { - match self.0.poll_lock() { - Async::Ready(mut inner) => inner.flush(), + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + match self.0.poll_lock(ctx) { + Async::Ready(mut inner) => inner.flush(ctx), Async::Pending => Ok(Async::Pending), } } - fn close(&mut self) -> Poll<(), S::SinkError> { - match self.0.poll_lock() { - Async::Ready(mut inner) => inner.close(), + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + match self.0.poll_lock(ctx) { + Async::Ready(mut inner) => inner.close(ctx), Async::Pending => Ok(Async::Pending), } } diff --git a/futures-util/src/stream/take.rs b/futures-util/src/stream/take.rs index e71cdaa42b..79ccbca60f 100644 --- a/futures-util/src/stream/take.rs +++ b/futures-util/src/stream/take.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Poll, Stream}; +use futures_core::task; use futures_sink::{StartSend, Sink}; /// A stream combinator which returns a maximum number of elements. @@ -52,16 +53,16 @@ impl Sink for Take type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -71,11 +72,11 @@ impl Stream for Take type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { if self.remaining == 0 { Ok(Async::Ready(None)) } else { - let next = try_ready!(self.stream.poll()); + let next = try_ready!(self.stream.poll(ctx)); 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 9c62f4cd28..c2f6d8a8c7 100644 --- a/futures-util/src/stream/take_while.rs +++ b/futures-util/src/stream/take_while.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Poll, IntoFuture, Future, Stream}; +use futures_core::task; use futures_sink::{StartSend, Sink}; /// A stream combinator which takes elements from a stream while a predicate @@ -59,16 +60,16 @@ impl Sink for TakeWhile type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -80,13 +81,13 @@ impl Stream for TakeWhile type Item = S::Item; type Error = S::Error; - fn poll(&mut self) -> Poll, S::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, S::Error> { if self.done_taking { return Ok(Async::Ready(None)); } if self.pending.is_none() { - let item = match try_ready!(self.stream.poll()) { + let item = match try_ready!(self.stream.poll(ctx)) { Some(e) => e, None => return Ok(Async::Ready(None)), }; @@ -94,7 +95,7 @@ impl Stream for TakeWhile } assert!(self.pending.is_some()); - match self.pending.as_mut().unwrap().0.poll() { + match self.pending.as_mut().unwrap().0.poll(ctx) { Ok(Async::Ready(true)) => { let (_, item) = self.pending.take().unwrap(); Ok(Async::Ready(Some(item))) diff --git a/futures-util/src/stream/then.rs b/futures-util/src/stream/then.rs index 7a89f1aed0..de5e4374a0 100644 --- a/futures-util/src/stream/then.rs +++ b/futures-util/src/stream/then.rs @@ -1,4 +1,5 @@ use futures_core::{Async, IntoFuture, Future, Poll, Stream}; +use futures_core::task; use futures_sink::{StartSend, Sink}; /// A stream combinator which chains a computation onto each item produced by a @@ -34,16 +35,16 @@ impl Sink for Then type SinkItem = S::SinkItem; type SinkError = S::SinkError; - fn start_send(&mut self, item: S::SinkItem) -> StartSend { - self.stream.start_send(item) + fn start_send(&mut self, ctx: &mut task::Context, item: S::SinkItem) -> StartSend { + self.stream.start_send(ctx, item) } - fn flush(&mut self) -> Poll<(), S::SinkError> { - self.stream.flush() + fn flush(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.flush(ctx) } - fn close(&mut self) -> Poll<(), S::SinkError> { - self.stream.close() + fn close(&mut self, ctx: &mut task::Context) -> Poll<(), S::SinkError> { + self.stream.close(ctx) } } @@ -55,9 +56,9 @@ impl Stream for Then type Item = U::Item; type Error = U::Error; - fn poll(&mut self) -> Poll, U::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, U::Error> { if self.future.is_none() { - let item = match self.stream.poll() { + let item = match self.stream.poll(ctx) { Ok(Async::Pending) => return Ok(Async::Pending), Ok(Async::Ready(None)) => return Ok(Async::Ready(None)), Ok(Async::Ready(Some(e))) => Ok(e), @@ -66,7 +67,7 @@ impl Stream for Then self.future = Some((self.f)(item).into_future()); } assert!(self.future.is_some()); - match self.future.as_mut().unwrap().poll() { + match self.future.as_mut().unwrap().poll(ctx) { Ok(Async::Ready(e)) => { self.future = None; Ok(Async::Ready(Some(e))) diff --git a/futures-util/src/stream/unfold.rs b/futures-util/src/stream/unfold.rs index 70df4b8a2e..e60237c416 100644 --- a/futures-util/src/stream/unfold.rs +++ b/futures-util/src/stream/unfold.rs @@ -1,6 +1,7 @@ use core::mem; use futures_core::{Future, IntoFuture, Async, Poll, Stream}; +use futures_core::task; /// Creates a `Stream` from a seed and a closure returning a `Future`. /// @@ -79,7 +80,7 @@ impl Stream for Unfold type Item = It; type Error = Fut::Error; - fn poll(&mut self) -> Poll, Fut::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Fut::Error> { loop { match mem::replace(&mut self.state, State::Empty) { // State::Empty may happen if the future returned an error @@ -88,7 +89,7 @@ impl Stream for Unfold self.state = State::Processing((self.f)(state).into_future()); } State::Processing(mut fut) => { - match fut.poll()? { + match fut.poll(ctx)? { Async:: Ready(Some((item, next_state))) => { self.state = State::Ready(next_state); return Ok(Async::Ready(Some(item))); diff --git a/futures-util/src/stream/zip.rs b/futures-util/src/stream/zip.rs index 0586f18d92..3eedf43cfb 100644 --- a/futures-util/src/stream/zip.rs +++ b/futures-util/src/stream/zip.rs @@ -1,4 +1,5 @@ use futures_core::{Async, Poll, Stream}; +use futures_core::task; use stream::{StreamExt, Fuse}; @@ -33,15 +34,15 @@ impl Stream for Zip type Item = (S1::Item, S2::Item); type Error = S1::Error; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, ctx: &mut task::Context) -> Poll, Self::Error> { if self.queued1.is_none() { - match self.stream1.poll()? { + match self.stream1.poll(ctx)? { Async::Ready(Some(item1)) => self.queued1 = Some(item1), Async::Ready(None) | Async::Pending => {} } } if self.queued2.is_none() { - match self.stream2.poll()? { + match self.stream2.poll(ctx)? { Async::Ready(Some(item2)) => self.queued2 = Some(item2), Async::Ready(None) | Async::Pending => {} } diff --git a/futures/src/lib.rs b/futures/src/lib.rs index bd713fdaa2..258530bb63 100644 --- a/futures/src/lib.rs +++ b/futures/src/lib.rs @@ -238,6 +238,8 @@ pub mod prelude { Poll, }; + pub use futures_core::task::Context as TaskContext; + pub use futures_sink::{ Sink, AsyncSink,