diff --git a/.travis.yml b/.travis.yml index a89cf1d0c5..da4b53d9bb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,10 +6,8 @@ sudo: false matrix: include: - os: osx -# - rust: stable -# - rust: beta -# - rust: nightly -# script: cargo bench --all && cd futures-util && cargo bench --features=bench + - rust: nightly + script: cargo bench --all && cd futures-util && cargo bench --features=bench - rust: nightly before_script: - pip install ghp-import --user && export PATH=$HOME/.local/bin:$PATH @@ -18,28 +16,19 @@ matrix: after_success: - ghp-import -n target/doc - git push -qf https://${GH_TOKEN}@github.com/${TRAVIS_REPO_SLUG}.git gh-pages - # - rust: stable - # script: - # - cargo build --manifest-path futures-core/Cargo.toml --no-default-features - # - cargo build --manifest-path futures/Cargo.toml --no-default-features - # - cargo build --manifest-path futures-channel/Cargo.toml --no-default-features - # - cargo build --manifest-path futures-executor/Cargo.toml --no-default-features - # - cargo build --manifest-path futures-sink/Cargo.toml --no-default-features - # - cargo build --manifest-path futures-util/Cargo.toml --no-default-features - # - rust: nightly - # script: - # - cargo build --manifest-path futures-core/Cargo.toml --features nightly - # - cargo build --manifest-path futures-stable/Cargo.toml --features nightly - # - cargo build --manifest-path futures-async-runtime/Cargo.toml --features nightly - # - cargo build --manifest-path futures-macro-async/Cargo.toml --features nightly - # - cargo build --manifest-path futures/Cargo.toml --features nightly - # - cargo test --manifest-path futures-macro-await/Cargo.toml --features nightly - # - cargo test --manifest-path futures/Cargo.toml --features nightly --test async_await_tests + - rust: nightly + script: + - cargo build --manifest-path futures/Cargo.toml --no-default-features + - cargo build --manifest-path futures-channel/Cargo.toml --no-default-features + - cargo build --manifest-path futures-core/Cargo.toml --no-default-features + - cargo build --manifest-path futures-executor/Cargo.toml --no-default-features + - cargo build --manifest-path futures-sink/Cargo.toml --no-default-features + - cargo build --manifest-path futures-util/Cargo.toml --no-default-features # - rust: nightly # script: # - rustup component add rust-src # - cargo install xargo - # - xargo build --manifest-path futures/Cargo.toml --target thumbv6m-none-eabi --no-default-features --features nightly + # - xargo build --manifest-path futures/Cargo.toml --target thumbv6m-none-eabi --no-default-features # - rust: 1.20.0 # script: cargo test --all # - rust: nightly @@ -47,9 +36,9 @@ matrix: # - cargo test --manifest-path futures/testcrate/Cargo.toml script: - - cargo build --all # for now build only; tests require the full suite of crates -# - cargo test --all -# - cargo test --all --release + - cargo build --all + - cargo test --all + - cargo test --all --release env: global: diff --git a/futures-channel/src/lib.rs b/futures-channel/src/lib.rs index bbcdd72408..e556a7fb28 100644 --- a/futures-channel/src/lib.rs +++ b/futures-channel/src/lib.rs @@ -10,7 +10,7 @@ #![deny(missing_docs, missing_debug_implementations, warnings)] #![deny(bare_trait_objects)] -#![doc(html_root_url = "https://docs.rs/futures-channel/0.2.0")] +#![doc(html_root_url = "https://docs.rs/futures-channel/0.3.0-alpha")] #[cfg(feature = "std")] extern crate std; diff --git a/futures-channel/src/mpsc/mod.rs b/futures-channel/src/mpsc/mod.rs index e6e1de6bf5..de036c4e6a 100644 --- a/futures-channel/src/mpsc/mod.rs +++ b/futures-channel/src/mpsc/mod.rs @@ -1,38 +1,42 @@ //! A multi-producer, single-consumer queue for sending values across //! asynchronous tasks. //! -//! Similarly to the `std`, channel creation provides [`Receiver`](Receiver) and -//! [`Sender`](Sender) handles. [`Receiver`](Receiver) implements -//! [`Stream`](futures_core::Stream) and allows a task to read values out of the -//! channel. If there is no message to read from the channel, the current task -//! will be notified when a new value is sent. [`Sender`](Sender) implements the -//! `Sink` trait and allows a task to send messages into +//! Similarly to the `std`, channel creation provides [`Receiver`] and +//! [`Sender`] handles. [`Receiver`] implements [`Stream`] and allows a task to +//! read values out of the channel. If there is no message to read from the +//! channel, the current task will be notified when a new value is sent. +//! [`Sender`] implements the `Sink` trait and allows a task to send messages into //! the channel. If the channel is at capacity, the send will be rejected and //! the task will be notified when additional capacity is available. In other //! words, the channel provides backpressure. //! -//! Unbounded channels are also available using the [`unbounded`](unbounded) -//! constructor. +//! Unbounded channels are also available using the `unbounded` constructor. //! //! # Disconnection //! -//! When all [`Sender`](Sender) handles have been dropped, it is no longer +//! When all [`Sender`] handles have been dropped, it is no longer //! possible to send values into the channel. This is considered the termination -//! event of the stream. As such, [`Receiver::poll_next`](Receiver::poll_next) +//! event of the stream. As such, [`Receiver::poll_next`] //! will return `Ok(Ready(None))`. //! -//! If the [`Receiver`](Receiver) handle is dropped, then messages can no longer +//! If the [`Receiver`] handle is dropped, then messages can no longer //! be read out of the channel. In this case, all further attempts to send will //! result in an error. //! //! # Clean Shutdown //! -//! If the [`Receiver`](Receiver) is simply dropped, then it is possible for +//! If the [`Receiver`] is simply dropped, then it is possible for //! there to be messages still in the channel that will not be processed. As //! such, it is usually desirable to perform a "clean" shutdown. To do this, the //! receiver will first call `close`, which will prevent any further messages to //! be sent into the channel. Then, the receiver consumes the channel to //! completion, at which point the receiver can be dropped. +//! +//! [`Sender`]: struct.Sender.html +//! [`Receiver`]: struct.Receiver.html +//! [`Stream`]: ../../futures_core/stream/trait.Stream.html +//! [`Receiver::poll_next`]: +//! ../../futures_core/stream/trait.Stream.html#tymethod.poll_next // At the core, the channel uses an atomic FIFO queue for message passing. This // queue is used as the primary coordination primitive. In order to enforce @@ -74,20 +78,19 @@ // happens-before semantics required for the acquire / release semantics used // by the queue structure. -use std::mem::PinMut; -use std::marker::Unpin; -use std::fmt; -use std::error::Error; +use futures_core::stream::Stream; +use futures_core::task::{self, Waker, Poll}; use std::any::Any; +use std::error::Error; +use std::fmt; +use std::marker::Unpin; +use std::mem::PinMut; +use std::sync::{Arc, Mutex}; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; -use std::sync::{Arc, Mutex}; use std::thread; use std::usize; -use futures_core::task::{self, Waker}; -use futures_core::{Poll, Stream}; - use crate::mpsc::queue::{Queue, PopResult}; mod queue; @@ -362,7 +365,7 @@ impl SenderTask { /// `buffer` "first come, first serve" slots available to all senders. /// /// The [`Receiver`](Receiver) returned implements the -/// [`Stream`](futures_core::Stream) trait, while [`Sender`](Sender) implements +/// [`Stream`](futures_core::stream::Stream) trait, while [`Sender`](Sender) implements /// `Sink`. pub fn channel(buffer: usize) -> (Sender, Receiver) { // Check that the requested buffer size does not exceed the maximum buffer @@ -371,7 +374,8 @@ pub fn channel(buffer: usize) -> (Sender, Receiver) { channel2(Some(buffer)) } -/// Creates an unbounded mpsc channel for communicating between asynchronous tasks. +/// Creates an unbounded mpsc channel for communicating between asynchronous +/// tasks. /// /// A `send` on this channel will always succeed as long as the receive half has /// not been closed. If the receiver falls behind, messages will be arbitrarily @@ -556,8 +560,8 @@ impl Sender { // This probably is never hit? Odds are the process will run out of // memory first. It may be worth to return something else in this // case? - assert!(state.num_messages < MAX_CAPACITY, "buffer space exhausted; \ - sending this messages would overflow the state"); + assert!(state.num_messages < MAX_CAPACITY, "buffer space \ + exhausted; sending this messages would overflow the state"); state.num_messages += 1; @@ -642,9 +646,13 @@ impl Sender { /// /// - `Ok(Async::Ready(_))` if there is sufficient capacity; /// - `Ok(Async::Pending)` if the channel may not have - /// capacity, in which case the current task is queued to be notified once capacity is available; + /// capacity, in which case the current task is queued to be notified once + /// capacity is available; /// - `Err(SendError)` if the receiver has been dropped. - pub fn poll_ready(&mut self, cx: &mut task::Context) -> Poll> { + pub fn poll_ready( + &mut self, + cx: &mut task::Context + ) -> Poll> { let state = decode_state(self.inner.state.load(SeqCst)); if !state.is_open { return Poll::Ready(Err(SendError { @@ -698,7 +706,10 @@ impl Sender { impl UnboundedSender { /// Check if the channel is ready to receive a message. - pub fn poll_ready(&self, _: &mut task::Context) -> Poll> { + pub fn poll_ready( + &self, + _: &mut task::Context, + ) -> Poll> { self.0.poll_ready_nb() } @@ -851,8 +862,8 @@ impl Receiver { loop { match unsafe { self.inner.message_queue.pop() } { PopResult::Data(msg) => { - // If there are any parked task handles in the parked queue, pop - // one and unpark it. + // If there are any parked task handles in the parked queue, + // pop one and unpark it. self.unpark_one(); // Decrement number of messages @@ -948,7 +959,10 @@ impl Unpin for Receiver {} impl Stream for Receiver { type Item = T; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { loop { // Try to read a message off of the message queue. let msg = match self.next_message() { @@ -1015,7 +1029,10 @@ impl UnboundedReceiver { impl Stream for UnboundedReceiver { type Item = T; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { PinMut::new(&mut self.0).poll_next(cx) } } diff --git a/futures-channel/src/oneshot.rs b/futures-channel/src/oneshot.rs index d9af801cd7..29392fe907 100644 --- a/futures-channel/src/oneshot.rs +++ b/futures-channel/src/oneshot.rs @@ -1,5 +1,7 @@ //! A channel for sending a single message between asynchronous tasks. +use futures_core::future::Future; +use futures_core::task::{self, Poll, Waker}; use std::marker::Unpin; use std::mem::PinMut; use std::sync::Arc; @@ -8,9 +10,6 @@ use std::sync::atomic::Ordering::SeqCst; use std::error::Error; use std::fmt; -use futures_core::{Future, Poll}; -use futures_core::task::{self, Waker}; - use crate::lock::Lock; /// A future for a value that will be provided by another asynchronous task. @@ -73,37 +72,34 @@ struct Inner { /// Creates a new one-shot channel for sending values across asynchronous tasks. /// -/// This function is similar to Rust's channel constructor found in the standard library. -/// Two halves are returned, the first of which is a `Sender` handle, used to -/// signal the end of a computation and provide its value. The second half is a -/// `Receiver` which implements the `Future` trait, resolving to the value that -/// was given to the `Sender` handle. +/// This function is similar to Rust's channel constructor found in the standard +/// library. Two halves are returned, the first of which is a `Sender` handle, +/// used to signal the end of a computation and provide its value. The second +/// half is a `Receiver` which implements the `Future` trait, resolving to the +/// value that was given to the `Sender` handle. /// /// Each half can be separately owned and sent across tasks. /// /// # Examples /// /// ``` -/// extern crate futures; -/// -/// use std::thread; -/// /// use futures::channel::oneshot; -/// use futures::*; +/// use futures::prelude::*; +/// use std::thread; /// /// fn main() { -/// let (p, c) = oneshot::channel::(); +/// let (sender, receiver) = oneshot::channel::(); /// /// # let t = /// thread::spawn(|| { -/// let future = c.map(|i| { +/// let future = receiver.map(|i| { /// println!("got: {:?}", i); /// }); /// // ... /// # return future; /// }); /// -/// p.send(3).unwrap(); +/// sender.send(3).unwrap(); /// # futures::executor::block_on(t.join().unwrap()); /// } /// ``` @@ -179,10 +175,10 @@ impl Inner { // hence we're canceled. If it succeeds then we just store our handle. // // Crucially we then check `oneshot_gone` *again* before we return. - // While we were storing our handle inside `notify_cancel` the `Receiver` - // may have been dropped. The first thing it does is set the flag, and - // if it fails to acquire the lock it assumes that we'll see the flag - // later on. So... we then try to see the flag later on! + // While we were storing our handle inside `notify_cancel` the + // `Receiver` may have been dropped. The first thing it does is set the + // flag, and if it fails to acquire the lock it assumes that we'll see + // the flag later on. So... we then try to see the flag later on! let handle = cx.waker().clone(); match self.tx_task.try_lock() { Some(mut p) => *p = Some(handle), @@ -284,8 +280,8 @@ impl Inner { // successfully blocked our task and we return `Pending`. if done || self.complete.load(SeqCst) { // If taking the lock fails, the sender will realise that the we're - // `done` when it checks the `complete` flag on the way out, and will - // treat the send as a failure. + // `done` when it checks the `complete` flag on the way out, and + // will treat the send as a failure. if let Some(mut slot) = self.data.try_lock() { if let Some(data) = slot.take() { return Poll::Ready(Ok(data)); @@ -399,21 +395,21 @@ impl Receiver { /// /// Any `send` operation which happens after this method returns is /// guaranteed to fail. After calling this method, you can use - /// [`Receiver::poll`](Receiver::poll) to determine whether a message had - /// previously been sent. + /// [`Receiver::poll`](Future::poll) to determine whether a + /// message had previously been sent. pub fn close(&mut self) { self.inner.close_rx() } /// Attempts to receive a message outside of the context of a task. /// - /// Useful when a [`Context`](Context) is not available such as within a - /// `Drop` impl. + /// Useful when a [`Context`](task::Context) is not available + /// such as within a `Drop` impl. /// /// Does not schedule a task wakeup or have any other side effects. /// /// A return value of `None` must be considered immediately stale (out of - /// date) unless [`::close`](Receiver::close) has been called first. + /// date) unless [`close`](Receiver::close) has been called first. /// /// Returns an error if the sender was dropped. pub fn try_recv(&mut self) -> Result, Canceled> { @@ -424,7 +420,10 @@ impl Receiver { impl Future for Receiver { type Output = Result; - fn poll(self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll( + self: PinMut, + cx: &mut task::Context, + ) -> Poll> { self.inner.recv(cx) } } diff --git a/futures-channel/tests/mpsc.rs b/futures-channel/tests/mpsc.rs index 0b3ba80c34..9067e40b41 100644 --- a/futures-channel/tests/mpsc.rs +++ b/futures-channel/tests/mpsc.rs @@ -408,8 +408,6 @@ fn stress_close_receiver() { async fn stress_poll_ready_sender(mut sender: mpsc::Sender, count: u32) { for i in (1..=count).rev() { - // FIXME: should assert `is_ok()`, but cannot because assigning the result - // of this expression to a variable causes an ICE. See rust-lang/rust/issues/51995. await!(sender.send(i)).unwrap(); } } diff --git a/futures-core/src/future/either.rs b/futures-core/src/future/either.rs index d8876f618c..0104b794fa 100644 --- a/futures-core/src/future/either.rs +++ b/futures-core/src/future/either.rs @@ -1,7 +1,6 @@ -use crate::{task, Stream, Poll}; - +use crate::stream::{Stream}; +use crate::task::{self, Poll}; use core::mem::PinMut; - use either::Either; // impl Future for Either diff --git a/futures-core/src/future/future_obj.rs b/futures-core/src/future/future_obj.rs deleted file mode 100644 index 4b3aa29a4f..0000000000 --- a/futures-core/src/future/future_obj.rs +++ /dev/null @@ -1,247 +0,0 @@ -use core::task::{TaskObj, LocalTaskObj}; -use core::mem; -use std::boxed::{Box, PinBox}; - -use core::fmt; -use core::future::Future; -use core::marker::PhantomData; -use core::mem::PinMut; -use core::task::{Context, Poll}; - -/// A custom trait object for polling futures, roughly akin to -/// `Box>`. -/// Contrary to `FutureObj`, `LocalFutureObj` does not have a `Send` bound. -pub struct LocalFutureObj<'a, T> { - ptr: *mut (), - poll_fn: unsafe fn(*mut (), &mut Context) -> Poll, - drop_fn: unsafe fn(*mut ()), - _marker1: PhantomData, - _marker2: PhantomData<&'a ()>, -} - -impl<'a, T> LocalFutureObj<'a, T> { - /// Create a `LocalFutureObj` from a custom trait object representation. - #[inline] - pub fn new + 'a>(f: F) -> LocalFutureObj<'a, T> { - LocalFutureObj { - ptr: f.into_raw(), - poll_fn: F::poll, - drop_fn: F::drop, - _marker1: PhantomData, - _marker2: PhantomData, - } - } - - /// Converts the `LocalFutureObj` into a `FutureObj` - /// To make this operation safe one has to ensure that the `UnsafeFutureObj` - /// instance from which this `LocalFutureObj` was created actually - /// implements `Send`. - #[inline] - pub unsafe fn into_future_obj(self) -> FutureObj<'a, T> { - FutureObj(self) - } -} - -impl<'a, T> fmt::Debug for LocalFutureObj<'a, T> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("LocalFutureObj") - .finish() - } -} - -impl<'a, T> From> for LocalFutureObj<'a, T> { - #[inline] - fn from(f: FutureObj<'a, T>) -> LocalFutureObj<'a, T> { - f.0 - } -} - -impl<'a, T> Future for LocalFutureObj<'a, T> { - type Output = T; - - #[inline] - fn poll(self: PinMut, cx: &mut Context) -> Poll { - unsafe { - (self.poll_fn)(self.ptr, cx) - } - } -} - -impl<'a, T> Drop for LocalFutureObj<'a, T> { - fn drop(&mut self) { - unsafe { - (self.drop_fn)(self.ptr) - } - } -} - -/// A custom trait object for polling futures, roughly akin to -/// `Box> + Send`. -pub struct FutureObj<'a, T>(LocalFutureObj<'a, T>); - -unsafe impl<'a, T> Send for FutureObj<'a, T> {} - -impl<'a, T> FutureObj<'a, T> { - /// Create a `FutureObj` from a custom trait object representation. - #[inline] - pub fn new + Send>(f: F) -> FutureObj<'a, T> { - FutureObj(LocalFutureObj::new(f)) - } -} - -impl<'a, T> fmt::Debug for FutureObj<'a, T> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("FutureObj") - .finish() - } -} - -impl<'a, T> Future for FutureObj<'a, T> { - type Output = T; - - #[inline] - fn poll(self: PinMut, cx: &mut Context) -> Poll { - let pinned_field = unsafe { PinMut::map_unchecked(self, |x| &mut x.0) }; - pinned_field.poll(cx) - } -} - -/// A custom implementation of a future trait object for `FutureObj`, providing -/// a hand-rolled vtable. -/// -/// This custom representation is typically used only in `no_std` contexts, -/// where the default `Box`-based implementation is not available. -/// -/// The implementor must guarantee that it is safe to call `poll` repeatedly (in -/// a non-concurrent fashion) with the result of `into_raw` until `drop` is -/// called. -pub unsafe trait UnsafeFutureObj<'a, T>: 'a { - /// Convert an owned instance into a (conceptually owned) void pointer. - fn into_raw(self) -> *mut (); - - /// Poll the future represented by the given void pointer. - /// - /// # Safety - /// - /// The trait implementor must guarantee that it is safe to repeatedly call - /// `poll` with the result of `into_raw` until `drop` is called; such calls - /// are not, however, allowed to race with each other or with calls to `drop`. - unsafe fn poll(future: *mut (), cx: &mut Context) -> Poll; - - /// Drops the future represented by the given void pointer. - /// - /// # Safety - /// - /// The trait implementor must guarantee that it is safe to call this - /// function once per `into_raw` invocation; that call cannot race with - /// other calls to `drop` or `poll`. - unsafe fn drop(future: *mut ()); -} - - - - - - - -impl From> for TaskObj { - #[inline] - fn from(f: FutureObj<'static, ()>) -> TaskObj { - unsafe { mem::transmute(f) } - } -} - -impl From> for LocalTaskObj { - #[inline] - fn from(f: LocalFutureObj<'static, ()>) -> LocalTaskObj { - unsafe { mem::transmute(f) } - } -} - -impl From for FutureObj<'static, ()> { - #[inline] - fn from(f: TaskObj) -> FutureObj<'static, ()> { - unsafe { mem::transmute(f) } - } -} - -impl From for LocalFutureObj<'static, ()> { - #[inline] - fn from(f: LocalTaskObj) -> LocalFutureObj<'static, ()> { - unsafe { mem::transmute(f) } - } -} - - - - - - -unsafe impl<'a, T, F: Future + 'a> UnsafeFutureObj<'a, T> for PinBox { - fn into_raw(self) -> *mut () { - PinBox::into_raw(self) as *mut () - } - - unsafe fn poll(task: *mut (), cx: &mut Context) -> Poll { - let ptr = task as *mut F; - let pin: PinMut = PinMut::new_unchecked(&mut *ptr); - pin.poll(cx) - } - - unsafe fn drop(task: *mut ()) { - drop(PinBox::from_raw(task as *mut F)) - } -} - -unsafe impl<'a, T, F: Future + 'a> UnsafeFutureObj<'a, T> for Box { - fn into_raw(self) -> *mut () { - Box::into_raw(self) as *mut () - } - - unsafe fn poll(task: *mut (), cx: &mut Context) -> Poll { - let ptr = task as *mut F; - let pin: PinMut = PinMut::new_unchecked(&mut *ptr); - pin.poll(cx) - } - - unsafe fn drop(task: *mut ()) { - drop(Box::from_raw(task as *mut F)) - } -} - -impl<'a, F: Future + Send + 'a> From> for FutureObj<'a, ()> { - fn from(boxed: PinBox) -> Self { - FutureObj::new(boxed) - } -} - -impl<'a, F: Future + Send + 'a> From> for FutureObj<'a, ()> { - fn from(boxed: Box) -> Self { - FutureObj::new(PinBox::from(boxed)) - } -} - -impl<'a, F: Future + 'a> From> for LocalFutureObj<'a, ()> { - fn from(boxed: PinBox) -> Self { - LocalFutureObj::new(boxed) - } -} - -impl<'a, F: Future + 'a> From> for LocalFutureObj<'a, ()> { - fn from(boxed: Box) -> Self { - LocalFutureObj::new(PinBox::from(boxed)) - } -} - - -unsafe impl<'a, T, F: Future + 'a> UnsafeFutureObj<'a, T> for PinMut<'a, F> { - fn into_raw(self) -> *mut () { - unsafe { PinMut::get_mut_unchecked(self) as *mut F as *mut () } - } - - unsafe fn poll(ptr: *mut (), cx: &mut Context) -> Poll { - PinMut::new_unchecked(&mut *(ptr as *mut F)).poll(cx) - } - - unsafe fn drop(_ptr: *mut ()) {} -} diff --git a/futures-core/src/future/mod.rs b/futures-core/src/future/mod.rs index 932343f8cf..d705a91179 100644 --- a/futures-core/src/future/mod.rs +++ b/futures-core/src/future/mod.rs @@ -1,22 +1,16 @@ //! Futures. -use core::mem::PinMut; +use crate::task::{self, Poll}; use core::marker::Unpin; - -use crate::Poll; -use crate::task; +use core::mem::PinMut; +pub use core::future::{Future, FutureObj, LocalFutureObj, UnsafeFutureObj}; mod option; pub use self::option::FutureOption; -mod future_obj; -pub use self::future_obj::{FutureObj, LocalFutureObj}; - #[cfg(feature = "either")] mod either; -pub use core::future::Future; - /// Will probably merge with futures_util::FutureExt pub trait CoreFutureExt: Future { /// A convenience for calling `Future::poll` on `Unpin` future types. @@ -43,7 +37,10 @@ pub trait TryFuture { /// This method is a stopgap for a compiler limitation that prevents us from /// directly inheriting from the `Future` trait; in the future it won't be /// needed. - fn try_poll(self: PinMut, cx: &mut task::Context) -> Poll>; + fn try_poll( + self: PinMut, + cx: &mut task::Context, + ) -> Poll>; } impl TryFuture for F @@ -52,6 +49,7 @@ impl TryFuture for F type Item = T; type Error = E; + #[inline] fn try_poll(self: PinMut, cx: &mut task::Context) -> Poll { self.poll(cx) } @@ -67,6 +65,7 @@ impl Unpin for ReadyFuture {} impl Future for ReadyFuture { type Output = T; + #[inline] fn poll(mut self: PinMut, _cx: &mut task::Context) -> Poll { Poll::Ready(self.0.take().unwrap()) } diff --git a/futures-core/src/future/option.rs b/futures-core/src/future/option.rs index 6ed2abe461..a961b51089 100644 --- a/futures-core/src/future/option.rs +++ b/futures-core/src/future/option.rs @@ -1,7 +1,7 @@ //! Definition of the `Option` (optional step) combinator -use crate::{Future, Poll}; -use crate::task; +use crate::future::Future; +use crate::task::{self, Poll}; use core::mem::PinMut; /// A future representing a value which may or may not be present. diff --git a/futures-core/src/lib.rs b/futures-core/src/lib.rs index c3d8aa548a..924dcbeb4c 100644 --- a/futures-core/src/lib.rs +++ b/futures-core/src/lib.rs @@ -7,10 +7,9 @@ #![deny(missing_docs, missing_debug_implementations, warnings)] #![deny(bare_trait_objects)] -#![doc(html_root_url = "https://docs.rs/futures-core/0.3.0")] +#![doc(html_root_url = "https://docs.rs/futures-core/0.3.0-alpha")] #![cfg_attr(feature = "nightly", feature(cfg_target_has_atomic))] -#![cfg_attr(feature = "nightly", feature(pin))] #[macro_use] #[cfg(feature = "std")] @@ -20,9 +19,18 @@ extern crate either; #[doc(hidden)] pub mod core_reexport { - pub use core::*; + pub use core::{mem, future, task}; } +#[doc(hidden)] pub use crate::future::Future; +#[doc(hidden)] pub use crate::future::CoreFutureExt; +#[doc(hidden)] pub use crate::future::TryFuture; + +#[doc(hidden)] pub use crate::stream::Stream; +#[doc(hidden)] pub use crate::stream::TryStream; + +#[doc(hidden)] pub use crate::task::Poll; + macro_rules! if_std { ($($i:item)*) => ($( #[cfg(feature = "std")] @@ -30,63 +38,11 @@ macro_rules! if_std { )*) } -#[macro_export] -macro_rules! pinned_deref { - ($e:expr) => ( - $crate::core_reexport::mem::PinMut::new_unchecked( - &mut **$crate::core_reexport::mem::PinMut::get_mut_unchecked($e.reborrow()) - ) - ) -} - -#[macro_export] -macro_rules! pinned_field { - ($e:expr, $f:tt) => ( - $crate::core_reexport::mem::PinMut::new_unchecked( - &mut $crate::core_reexport::mem::PinMut::get_mut_unchecked($e.reborrow()).$f - ) - ) -} - -#[macro_export] -macro_rules! unsafe_pinned { - ($f:tt -> $t:ty) => ( - fn $f<'a>(self: &'a mut PinMut) -> PinMut<'a, $t> { - unsafe { - pinned_field!(self, $f) - } - } - ) -} - -#[macro_export] -macro_rules! unsafe_unpinned { - ($f:tt -> $t:ty) => ( - fn $f<'a>(self: &'a mut PinMut) -> &'a mut $t { - unsafe { - &mut $crate::core_reexport::mem::PinMut::get_mut_unchecked(self.reborrow()).$f - } - } - ) -} - -#[macro_export] -macro_rules! pin_mut { - ($($x:ident),*) => { $( - // Move the value to ensure that it is owned - let mut $x = $x; - // Shadow the original binding so that it can't be directly accessed - // ever again. - #[allow(unused_mut)] - let mut $x = unsafe { $crate::core_reexport::mem::PinMut::new_unchecked(&mut $x) }; - )* } -} +#[macro_use] +mod macros; pub mod future; -pub use crate::future::{Future, CoreFutureExt, TryFuture}; pub mod stream; -pub use crate::stream::{Stream, TryStream}; pub mod task; -pub use crate::task::Poll; diff --git a/futures-core/src/macros/mod.rs b/futures-core/src/macros/mod.rs new file mode 100644 index 0000000000..932898b79c --- /dev/null +++ b/futures-core/src/macros/mod.rs @@ -0,0 +1,5 @@ +#[macro_use] +mod pin; + +#[macro_use] +mod poll; diff --git a/futures-core/src/macros/pin.rs b/futures-core/src/macros/pin.rs new file mode 100644 index 0000000000..bf01c3750f --- /dev/null +++ b/futures-core/src/macros/pin.rs @@ -0,0 +1,96 @@ +/// A pinned projection of a struct field. +/// +/// ``` +/// #![feature(pin, arbitrary_self_types)] +/// # #[macro_use] extern crate futures_core; +/// # struct Bar; +/// # use core::mem::PinMut; +/// struct Foo { +/// field: Bar, +/// } +/// +/// impl Foo { +/// unsafe_pinned!(field -> Bar); +/// +/// fn baz(mut self: PinMut) { +/// let _: PinMut = self.field(); // Pinned reference to the field +/// } +/// } +/// ``` +#[macro_export] +macro_rules! unsafe_pinned { + ($f:tt -> $t:ty) => ( + fn $f<'a>( + self: &'a mut $crate::core_reexport::mem::PinMut + ) -> $crate::core_reexport::mem::PinMut<'a, $t> { + unsafe { + $crate::core_reexport::mem::PinMut::map_unchecked( + self.reborrow(), |x| &mut x.$f + ) + } + } + ) +} + +/// An unpinned projection of a struct field. +/// +/// This macro is unsafe because it returns a normal non-pin reference to +/// the struct field. It is up to the programmer to ensure that the contained +/// value is either not moved at all or only moved when it's safe. +/// +/// ``` +/// #![feature(pin, arbitrary_self_types)] +/// # #[macro_use] extern crate futures_core; +/// # use core::mem::PinMut; +/// # struct Bar; +/// struct Foo { +/// field: Bar, +/// } +/// +/// impl Foo { +/// unsafe_unpinned!(field -> Bar); +/// +/// fn baz(mut self: PinMut) { +/// let _: &mut Bar = self.field(); // Normal reference to the field +/// } +/// } +/// ``` +#[macro_export] +macro_rules! unsafe_unpinned { + ($f:tt -> $t:ty) => ( + fn $f<'a>( + self: &'a mut $crate::core_reexport::mem::PinMut + ) -> &'a mut $t { + unsafe { + &mut $crate::core_reexport::mem::PinMut::get_mut_unchecked( + self.reborrow() + ).$f + } + } + ) +} + +/// Pins a value on the stack. +/// +/// ``` +/// #![feature(pin, arbitrary_self_types)] +/// # #[macro_use] extern crate futures_core; +/// # use core::mem::PinMut; +/// # struct Foo {} +/// let foo = Foo { /* ... */ }; +/// pin_mut!(foo); +/// let _: PinMut = foo; +/// ``` +#[macro_export] +macro_rules! pin_mut { + ($($x:ident),*) => { $( + // Move the value to ensure that it is owned + let mut $x = $x; + // Shadow the original binding so that it can't be directly accessed + // ever again. + #[allow(unused_mut)] + let mut $x = unsafe { + $crate::core_reexport::mem::PinMut::new_unchecked(&mut $x) + }; + )* } +} diff --git a/futures-core/src/macros/poll.rs b/futures-core/src/macros/poll.rs new file mode 100644 index 0000000000..fcc97cd4de --- /dev/null +++ b/futures-core/src/macros/poll.rs @@ -0,0 +1,40 @@ +/// Extracts the successful type of a `Poll>`. +/// +/// This macro bakes in propagation of `Pending` and `Err` signals by returning early. +#[macro_export] +macro_rules! try_ready { + ($x:expr) => { + match $x { + $crate::task::Poll::Ready(Ok(x)) => x, + $crate::task::Poll::Ready(Err(e)) => return $crate::task::Poll::Ready(Err(e.into())), + $crate::task::Poll::Pending => return $crate::task::Poll::Pending, + } + } +} + + +/// Extracts `Poll` from `Poll>`. +/// +/// This macro bakes in propagation of `Err` signals by returning early. +/// This macro bakes in propagation of `Pending` and `Err` signals by returning early. +#[macro_export] +macro_rules! try_poll { + ($x:expr) => { + match $x { + $crate::task::Poll::Ready(Ok(x)) => $crate::task::Poll::Ready(x), + $crate::task::Poll::Ready(Err(e)) => return $crate::task::Poll::Ready(Err(e.into())), + $crate::task::Poll::Pending => $crate::task::Poll::Pending, + } + } +} + +/// Extracts the successful type of a `Poll`. +/// +/// This macro bakes in propagation of `Pending` signals by returning early. +#[macro_export] +macro_rules! ready { + ($e:expr) => (match $e { + $crate::task::Poll::Ready(t) => t, + $crate::task::Poll::Pending => return $crate::task::Poll::Pending, + }) +} diff --git a/futures-core/src/stream/mod.rs b/futures-core/src/stream/mod.rs index 5ed8fbe893..5846e389c9 100644 --- a/futures-core/src/stream/mod.rs +++ b/futures-core/src/stream/mod.rs @@ -1,11 +1,9 @@ //! Asynchronous streams. +use crate::task::{self, Poll}; use core::marker::Unpin; use core::mem::PinMut; -use crate::Poll; -use crate::task; - /// A stream of values produced asynchronously. /// /// If `Future` is an asynchronous version of `T`, then `Stream, cx: &mut task::Context) -> Poll>; + /// calls to `poll_next` 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_next` always returns `Ready(None)` in subsequent + /// calls. + fn poll_next( + self: PinMut, + cx: &mut task::Context, + ) -> Poll>; /// A convenience for calling `Stream::poll_next` on `Unpin` stream types. fn poll_next_unpin(&mut self, cx: &mut task::Context) -> Poll> @@ -59,7 +61,10 @@ pub trait Stream { impl<'a, S: ?Sized + Stream + Unpin> Stream for &'a mut S { type Item = S::Item; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { S::poll_next(PinMut::new(&mut **self), cx) } } @@ -67,7 +72,10 @@ impl<'a, S: ?Sized + Stream + Unpin> Stream for &'a mut S { impl<'a, S: ?Sized + Stream> Stream for PinMut<'a, S> { type Item = S::Item; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { S::poll_next((*self).reborrow(), cx) } } @@ -109,7 +117,10 @@ if_std! { impl Stream for Box { type Item = S::Item; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { PinMut::new(&mut **self).poll_next(cx) } } @@ -117,7 +128,10 @@ if_std! { impl Stream for PinBox { type Item = S::Item; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { self.as_pin_mut().poll_next(cx) } } @@ -125,15 +139,21 @@ if_std! { impl Stream for ::std::panic::AssertUnwindSafe { type Item = S::Item; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { - unsafe { pinned_field!(self, 0).poll_next(cx) } + fn poll_next( + self: PinMut, + cx: &mut task::Context, + ) -> Poll> { + unsafe { PinMut::map_unchecked(self, |x| &mut x.0) }.poll_next(cx) } } impl Stream for ::std::collections::VecDeque { type Item = T; - fn poll_next(mut self: PinMut, _cx: &mut task::Context) -> Poll> { + fn poll_next( + mut self: PinMut, + _cx: &mut task::Context, + ) -> Poll> { Poll::Ready(self.pop_front()) } } diff --git a/futures-core/src/task/atomic_waker.rs b/futures-core/src/task/atomic_waker.rs index 7cdae618fe..11a632a250 100755 --- a/futures-core/src/task/atomic_waker.rs +++ b/futures-core/src/task/atomic_waker.rs @@ -171,8 +171,8 @@ impl AtomicWaker { /// /// ``` /// # #![feature(pin, arbitrary_self_types, futures_api)] - /// # use futures_core::{Future, Poll}; - /// # use futures_core::task::{self, AtomicWaker}; + /// # use futures_core::future::Future; + /// # use futures_core::task::{self, Poll, AtomicWaker}; /// # use std::sync::atomic::AtomicBool; /// # use std::sync::atomic::Ordering::SeqCst; /// # use std::mem::PinMut; diff --git a/futures-core/src/task/mod.rs b/futures-core/src/task/mod.rs index b2a3ddd980..e04f30b7ae 100644 --- a/futures-core/src/task/mod.rs +++ b/futures-core/src/task/mod.rs @@ -1,14 +1,11 @@ //! Task notification. -mod poll; -pub use self::poll::Poll; - pub use core::task::{ - Context, Executor, + Context, Executor, Poll, Waker, LocalWaker, UnsafeWake, - TaskObj, LocalTaskObj, UnsafeTask, SpawnErrorKind, SpawnObjError, SpawnLocalObjError, }; +pub use core::future::FutureObj; #[cfg_attr(feature = "nightly", cfg(target_has_atomic = "ptr"))] mod atomic_waker; @@ -16,7 +13,7 @@ mod atomic_waker; pub use self::atomic_waker::AtomicWaker; if_std! { - use crate::Future; + use crate::future::Future; use std::boxed::PinBox; pub use std::task::{Wake, local_waker, local_waker_from_nonlocal}; @@ -29,15 +26,14 @@ if_std! { /// /// This method will panic if the default executor is unable to spawn. /// - /// To handle executor errors, use [executor()](self::Context::executor) - /// instead. + /// To handle executor errors, use `Context::executor()` on instead. fn spawn(&mut self, f: F) where F: Future + 'static + Send; } impl<'a> ContextExt for Context<'a> { fn spawn(&mut self, f: F) where F: Future + 'static + Send { self.executor() - .spawn_obj(TaskObj::new(PinBox::new(f))).unwrap() + .spawn_obj(FutureObj::new(PinBox::new(f))).unwrap() } } } diff --git a/futures-core/src/task/poll.rs b/futures-core/src/task/poll.rs deleted file mode 100644 index 47bbcee62d..0000000000 --- a/futures-core/src/task/poll.rs +++ /dev/null @@ -1,43 +0,0 @@ - -pub use core::task::Poll; - -/// A macro for extracting the successful type of a `Poll>`. -/// -/// This macro bakes in propagation of `Pending` and `Err` signals by returning early. -#[macro_export] -macro_rules! try_ready { - ($x:expr) => { - match $x { - $crate::Poll::Ready(Ok(x)) => x, - $crate::Poll::Ready(Err(e)) => return $crate::Poll::Ready(Err(e.into())), - $crate::Poll::Pending => return $crate::Poll::Pending, - } - } -} - - -/// A macro for extracting `Poll` from `Poll>`. -/// -/// This macro bakes in propagation of `Err` signals by returning early. -/// This macro bakes in propagation of `Pending` and `Err` signals by returning early. -#[macro_export] -macro_rules! try_poll { - ($x:expr) => { - match $x { - $crate::Poll::Ready(Ok(x)) => $crate::Poll::Ready(x), - $crate::Poll::Ready(Err(e)) => return $crate::Poll::Ready(Err(e.into())), - $crate::Poll::Pending => $crate::Poll::Pending, - } - } -} - -/// A macro for extracting the successful type of a `Poll`. -/// -/// This macro bakes in propagation of `Pending` signals by returning early. -#[macro_export] -macro_rules! ready { - ($e:expr) => (match $e { - $crate::Poll::Ready(t) => t, - $crate::Poll::Pending => return $crate::Poll::Pending, - }) -} diff --git a/futures-executor/src/enter.rs b/futures-executor/src/enter.rs index b51a5ce68b..e46264cfec 100644 --- a/futures-executor/src/enter.rs +++ b/futures-executor/src/enter.rs @@ -6,7 +6,7 @@ thread_local!(static ENTERED: Cell = Cell::new(false)); /// Represents an executor context. /// -/// For more details, see [`enter` documentation](::enter()) +/// For more details, see [`enter` documentation](enter()). pub struct Enter { _a: () } diff --git a/futures-executor/src/lib.rs b/futures-executor/src/lib.rs index f68ac2cefa..f9f4b57b64 100644 --- a/futures-executor/src/lib.rs +++ b/futures-executor/src/lib.rs @@ -7,7 +7,7 @@ #![deny(missing_docs, missing_debug_implementations, warnings)] #![deny(bare_trait_objects)] -#![doc(html_root_url = "https://docs.rs/futures-executor/0.2.0")] +#![doc(html_root_url = "https://docs.rs/futures-executor/0.3.0-alpha")] #[cfg(feature = "std")] #[macro_use] @@ -27,10 +27,6 @@ if_std! { #[macro_use] extern crate lazy_static; - extern crate futures_util; - extern crate futures_channel; - extern crate num_cpus; - mod local_pool; pub use crate::local_pool::{block_on, block_on_stream, BlockingStream, LocalPool, LocalExecutor}; diff --git a/futures-executor/src/local_pool.rs b/futures-executor/src/local_pool.rs index c405631851..b441cf10e7 100644 --- a/futures-executor/src/local_pool.rs +++ b/futures-executor/src/local_pool.rs @@ -1,19 +1,19 @@ -use std::prelude::v1::*; - +use futures_core::future::{Future, FutureObj, LocalFutureObj}; +use futures_core::stream::{Stream}; +use futures_core::task::{ + self, Context, Poll, LocalWaker, Wake, + Executor, SpawnObjError, SpawnLocalObjError, SpawnErrorKind +}; +use futures_util::stream::FuturesUnordered; +use futures_util::stream::StreamExt; use std::cell::{RefCell}; use std::marker::Unpin; use std::ops::{Deref, DerefMut}; +use std::prelude::v1::*; use std::rc::{Rc, Weak}; use std::sync::Arc; use std::thread::{self, Thread}; -use futures_core::{Future, Poll, Stream}; -use futures_core::task::{ - self, Context, LocalWaker, TaskObj, LocalTaskObj, Wake, - Executor, SpawnObjError, SpawnLocalObjError, SpawnErrorKind}; -use futures_util::stream::FuturesUnordered; -use futures_util::stream::StreamExt; - use crate::enter; use crate::ThreadPool; @@ -24,24 +24,24 @@ use crate::ThreadPool; /// work in between I/O actions. /// /// To get a handle to the pool that implements -/// [`Executor`](::futures_core::executor::Executor), use the +/// [`Executor`](futures_core::task::Executor), use the /// [`executor()`](LocalPool::executor) method. Because the executor is /// single-threaded, it supports a special form of task spawning for non-`Send` -/// futures, via [`spawn_local`](LocalExecutor::spawn_local). +/// futures, via [`spawn_local_obj`](LocalExecutor::spawn_local_obj). #[derive(Debug)] pub struct LocalPool { - pool: FuturesUnordered, + pool: FuturesUnordered>, incoming: Rc, } /// A handle to a [`LocalPool`](LocalPool) that implements -/// [`Executor`](::futures_core::executor::Executor). +/// [`Executor`](futures_core::task::Executor). #[derive(Clone, Debug)] pub struct LocalExecutor { incoming: Weak, } -type Incoming = RefCell>; +type Incoming = RefCell>>; pub(crate) struct ThreadNotify { thread: Thread @@ -265,7 +265,10 @@ impl Iterator for BlockingStream { } impl Executor for LocalExecutor { - fn spawn_obj(&mut self, task: TaskObj) -> Result<(), SpawnObjError> { + fn spawn_obj( + &mut self, + task: FutureObj<'static, ()>, + ) -> Result<(), SpawnObjError> { if let Some(incoming) = self.incoming.upgrade() { incoming.borrow_mut().push(task.into()); Ok(()) @@ -285,9 +288,10 @@ impl Executor for LocalExecutor { impl LocalExecutor { /// Spawn a non-`Send` future onto the associated [`LocalPool`](LocalPool). - pub fn spawn_local_obj(&mut self, task: LocalTaskObj) - -> Result<(), SpawnLocalObjError> - { + pub fn spawn_local_obj( + &mut self, + task: LocalFutureObj<'static, ()>, + ) -> Result<(), SpawnLocalObjError> { if let Some(incoming) = self.incoming.upgrade() { incoming.borrow_mut().push(task); Ok(()) diff --git a/futures-executor/src/spawn.rs b/futures-executor/src/spawn.rs index d6f820af7c..9f18a0c641 100644 --- a/futures-executor/src/spawn.rs +++ b/futures-executor/src/spawn.rs @@ -1,5 +1,5 @@ -use futures_core::{Future, Poll, CoreFutureExt}; -use futures_core::task::{Context, ContextExt}; +use futures_core::future::{Future, CoreFutureExt}; +use futures_core::task::{self, Poll, ContextExt}; use futures_channel::oneshot::{channel, Sender, Receiver}; use futures_util::FutureExt; @@ -29,7 +29,7 @@ impl Spawn { /// This function returns a future that will spawn the given future as a task /// onto the default executor. It does *not* provide any way to wait on task /// completion or extract a value from the task. That can either be done through -/// a channel, or by using [`spawn_with_handle`](::spawn_with_handle). +/// a channel, or by using [`spawn_with_handle`](spawn_with_handle). pub fn spawn(future: F) -> Spawn where F: Future + 'static + Send { @@ -39,16 +39,16 @@ pub fn spawn(future: F) -> Spawn impl + Send + 'static> Future for Spawn { type Output = (); - fn poll(mut self: PinMut, cx: &mut Context) -> Poll<()> { + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll<()> { cx.spawn(self.future().take().unwrap()); Poll::Ready(()) } } /// A future representing the completion of task spawning, yielding a -/// [`JoinHandle`](::JoinHandle) to the spawned task. +/// [`JoinHandle`](JoinHandle) to the spawned task. /// -/// See [`spawn_with_handle`](::spawn_with_handle) for details. +/// See [`spawn_with_handle`](spawn_with_handle) for details. #[derive(Debug)] pub struct SpawnWithHandle { future: Option @@ -59,11 +59,11 @@ impl SpawnWithHandle { } /// Spawn a task onto the default executor, yielding a -/// [`JoinHandle`](::JoinHandle) to the spawned task. +/// [`JoinHandle`](JoinHandle) to the spawned task. /// /// This function returns a future that will spawn the given future as a task /// onto the default executor. On completion, that future will yield a -/// [`JoinHandle`](::JoinHandle) that can itself be used as a future +/// [`JoinHandle`](JoinHandle) that can itself be used as a future /// representing the completion of the spawned task. /// /// # Examples @@ -93,7 +93,7 @@ impl Future for SpawnWithHandle { type Output = JoinHandle; - fn poll(mut self: PinMut, cx: &mut Context) -> Poll { + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { let (tx, rx) = channel(); let keep_running_flag = Arc::new(AtomicBool::new(false)); // AssertUnwindSafe is used here because `Send + 'static` is basically @@ -152,7 +152,7 @@ impl JoinHandle { impl Future for JoinHandle { type Output = T; - fn poll(mut self: PinMut, cx: &mut Context) -> Poll { // ToDo: This was weird! Double check! + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { // ToDo: This was weird! Double check! match self.inner.poll_unpin(cx) { Poll::Ready(Ok(Ok(output))) => Poll::Ready(output), Poll::Ready(Ok(Err(e))) => panic::resume_unwind(e), @@ -165,7 +165,7 @@ impl Future for JoinHandle { impl Future for MySender { type Output = (); - fn poll(mut self: PinMut, cx: &mut Context) -> Poll<()> { + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll<()> { if let Poll::Ready(_) = self.tx().as_mut().unwrap().poll_cancel(cx) { if !self.keep_running_flag().load(Ordering::SeqCst) { // Cancelled, bail out diff --git a/futures-executor/src/thread_pool.rs b/futures-executor/src/thread_pool.rs index 89312df1b3..4f9d2a34dd 100644 --- a/futures-executor/src/thread_pool.rs +++ b/futures-executor/src/thread_pool.rs @@ -7,8 +7,8 @@ use std::sync::mpsc; use std::thread; use std::fmt; -use futures_core::*; -use futures_core::task::{self, Wake, TaskObj, Executor, SpawnObjError}; +use futures_core::future::{Future, FutureObj, CoreFutureExt}; +use futures_core::task::{self, Poll, Wake, Executor, SpawnObjError}; use crate::enter; use num_cpus; @@ -70,7 +70,7 @@ impl ThreadPool { /// Creates a new thread pool with the default configuration. /// /// See documentation for the methods in - /// [`ThreadPoolBuilder`](::ThreadPoolBuilder) for details on the default + /// [`ThreadPoolBuilder`](ThreadPoolBuilder) for details on the default /// configuration. pub fn new() -> Result { ThreadPoolBuilder::new().create() @@ -79,7 +79,7 @@ impl ThreadPool { /// Create a default thread pool configuration, which can then be customized. /// /// See documentation for the methods in - /// [`ThreadPoolBuilder`](::ThreadPoolBuilder) for details on the default + /// [`ThreadPoolBuilder`](ThreadPoolBuilder) for details on the default /// configuration. pub fn builder() -> ThreadPoolBuilder { ThreadPoolBuilder::new() @@ -100,7 +100,10 @@ impl ThreadPool { } impl Executor for ThreadPool { - fn spawn_obj(&mut self, task: TaskObj) -> Result<(), SpawnObjError> { + fn spawn_obj( + &mut self, + task: FutureObj<'static, ()>, + ) -> Result<(), SpawnObjError> { let task_container = TaskContainer { task, wake_handle: Arc::new(WakeHandle { @@ -226,7 +229,7 @@ impl ThreadPoolBuilder { self } - /// Create a [`ThreadPool`](::ThreadPool) with the given configuration. + /// Create a [`ThreadPool`](ThreadPool) with the given configuration. /// /// # Panics /// @@ -263,7 +266,7 @@ impl ThreadPoolBuilder { /// Units of work submitted to an `Executor`, currently only created /// internally. struct TaskContainer { - task: TaskObj, + task: FutureObj<'static, ()>, exec: ThreadPool, wake_handle: Arc, } diff --git a/futures-io/src/lib.rs b/futures-io/src/lib.rs index 1b9a15e0be..9ee8734873 100644 --- a/futures-io/src/lib.rs +++ b/futures-io/src/lib.rs @@ -9,7 +9,7 @@ #![deny(missing_docs, missing_debug_implementations, warnings)] #![deny(bare_trait_objects)] -#![doc(html_rnoot_url = "https://docs.rs/futures-io/0.2.0")] +#![doc(html_rnoot_url = "https://docs.rs/futures-io/0.3.0-alpha")] #![feature(futures_api)] @@ -21,11 +21,7 @@ macro_rules! if_std { } if_std! { - extern crate futures_core; - extern crate iovec; - extern crate std; - - use futures_core::{Poll, task}; + use futures_core::task::{self, Poll}; use std::boxed::Box; use std::io as StdIo; use std::ptr; diff --git a/futures-sink/src/lib.rs b/futures-sink/src/lib.rs index f86a77d13d..bec66ff85b 100644 --- a/futures-sink/src/lib.rs +++ b/futures-sink/src/lib.rs @@ -5,7 +5,7 @@ #![no_std] #![deny(missing_docs, missing_debug_implementations)] -#![doc(html_root_url = "https://docs.rs/futures-sink/0.2.0")] +#![doc(html_root_url = "https://docs.rs/futures-sink/0.3.0-alpha")] #![feature(pin, arbitrary_self_types, futures_api)] @@ -23,7 +23,7 @@ macro_rules! if_std { )*) } -use futures_core::{Poll, task}; +use futures_core::task::{self, Poll}; use core::marker::Unpin; use core::mem::PinMut; diff --git a/futures-util/benches/bilock.rs b/futures-util/benches_disabled/bilock.rs similarity index 100% rename from futures-util/benches/bilock.rs rename to futures-util/benches_disabled/bilock.rs diff --git a/futures-util/src/await/join.rs b/futures-util/src/async_await/join.rs similarity index 100% rename from futures-util/src/await/join.rs rename to futures-util/src/async_await/join.rs diff --git a/futures-util/src/async_await/mod.rs b/futures-util/src/async_await/mod.rs new file mode 100644 index 0000000000..1dcc5f01b5 --- /dev/null +++ b/futures-util/src/async_await/mod.rs @@ -0,0 +1,24 @@ +//! Await +//! +//! This module contains a number of functions and combinators for working +//! with `async`/`await` code. + +use core::marker::Unpin; +use futures_core::future::Future; + +#[macro_use] +mod poll; +pub use self::poll::*; + +#[macro_use] +mod pending; +pub use self::pending::*; + +// Primary export is a macro +mod join; + +// Primary export is a macro +mod select; + +#[doc(hidden)] +pub fn assert_unpin(_: &T) {} diff --git a/futures-util/src/async_await/pending.rs b/futures-util/src/async_await/pending.rs new file mode 100644 index 0000000000..b6febf8c3b --- /dev/null +++ b/futures-util/src/async_await/pending.rs @@ -0,0 +1,39 @@ +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; + +/// A macro which yields to the event loop once. +/// This is similar to returning `Poll::Pending` from a `Future::poll` implementation. +/// If `pending!` is used, the current task should be scheduled to receive a wakeup +/// when it is ready to make progress. +/// +/// This macro is only usable inside of `async` functions, closures, and blocks. +#[macro_export] +macro_rules! pending { + () => { + await!($crate::async_await::pending_once()) + } +} + +#[doc(hidden)] +pub fn pending_once() -> PendingOnce { + PendingOnce { is_ready: false } +} + +#[allow(missing_debug_implementations)] +#[doc(hidden)] +pub struct PendingOnce { + is_ready: bool, +} + +impl Future for PendingOnce { + type Output = (); + fn poll(mut self: PinMut, _: &mut task::Context) -> Poll { + if self.is_ready { + Poll::Ready(()) + } else { + self.is_ready = true; + Poll::Pending + } + } +} diff --git a/futures-util/src/async_await/poll.rs b/futures-util/src/async_await/poll.rs new file mode 100644 index 0000000000..d412964414 --- /dev/null +++ b/futures-util/src/async_await/poll.rs @@ -0,0 +1,33 @@ +use core::marker::Unpin; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; + +/// A macro which returns the result of polling a future once within the +/// current `async` context. +/// +/// This macro is only usable inside of `async` functions, closures, and blocks. +#[macro_export] +macro_rules! poll { + ($x:expr) => { + await!($crate::async_await::poll($x)) + } +} + +#[doc(hidden)] +pub fn poll(future: F) -> PollOnce { + PollOnce { future } +} + +#[allow(missing_debug_implementations)] +#[doc(hidden)] +pub struct PollOnce { + future: F, +} + +impl Future for PollOnce { + type Output = Poll; + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + Poll::Ready(PinMut::new(&mut self.future).poll(cx)) + } +} diff --git a/futures-util/src/await/select.rs b/futures-util/src/async_await/select.rs similarity index 95% rename from futures-util/src/await/select.rs rename to futures-util/src/async_await/select.rs index 5e3e9eab1a..d9e0477714 100644 --- a/futures-util/src/await/select.rs +++ b/futures-util/src/async_await/select.rs @@ -9,7 +9,7 @@ macro_rules! select { $name:ident => $body:expr, )*) => { { $( - $crate::await::assert_unpin(&$name); + $crate::async_await::assert_unpin(&$name); let mut $name = $crate::future::maybe_done(&mut $name); let mut $name = $crate::core_reexport::mem::PinMut::new(&mut $name); )* diff --git a/futures-util/src/await/mod.rs b/futures-util/src/await/mod.rs deleted file mode 100644 index d1041db529..0000000000 --- a/futures-util/src/await/mod.rs +++ /dev/null @@ -1,81 +0,0 @@ -//! Await -//! -//! This module contains a number of functions and combinators for working -//! with `async`/`await` code. - -use futures_core::{task, Future, Poll}; -use core::marker::Unpin; -use core::mem::PinMut; - -#[doc(hidden)] -pub fn assert_unpin(_: &T) {} - -/// A macro which returns the result of polling a future once within the -/// current `async` context. -/// -/// This macro is only usable inside of `async` functions, closures, and blocks. -#[macro_export] -macro_rules! poll { - ($x:expr) => { - await!($crate::await::poll($x)) - } -} - -#[doc(hidden)] -pub fn poll(future: F) -> impl Future> { - PollOnce { future } -} - -#[allow(missing_debug_implementations)] -struct PollOnce { - future: F, -} - -impl Future for PollOnce { - type Output = Poll; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - Poll::Ready(PinMut::new(&mut self.future).poll(cx)) - } -} - -/// A macro which yields to the event loop once. -/// This is similar to returning `Poll::Pending` from a `Future::poll` implementation. -/// If `pending!` is used, the current task should be scheduled to receive a wakeup -/// when it is ready to make progress. -/// -/// This macro is only usable inside of `async` functions, closures, and blocks. -#[macro_export] -macro_rules! pending { - () => { - await!($crate::await::pending_once()) - } -} - -#[doc(hidden)] -pub fn pending_once() -> impl Future { - PendingOnce { is_ready: false } -} - -#[allow(missing_debug_implementations)] -struct PendingOnce { - is_ready: bool, -} - -impl Future for PendingOnce { - type Output = (); - fn poll(mut self: PinMut, _: &mut task::Context) -> Poll { - if self.is_ready { - Poll::Ready(()) - } else { - self.is_ready = true; - Poll::Pending - } - } -} - -// Primary export is a macro -mod join; - -// Primary export is a macro -mod select; - diff --git a/futures-util/src/future/catch_unwind.rs b/futures-util/src/future/catch_unwind.rs index e5d2c4186d..f3c840edca 100644 --- a/futures-util/src/future/catch_unwind.rs +++ b/futures-util/src/future/catch_unwind.rs @@ -1,10 +1,9 @@ -use std::mem::PinMut; -use std::prelude::v1::*; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; use std::any::Any; +use std::mem::PinMut; use std::panic::{catch_unwind, UnwindSafe, AssertUnwindSafe}; - -use futures_core::{Future, Poll}; -use futures_core::task; +use std::prelude::v1::*; /// Future for the `catch_unwind` combinator. /// diff --git a/futures-util/src/future/chain.rs b/futures-util/src/future/chain.rs index dea639117d..4ca0ae0409 100644 --- a/futures-util/src/future/chain.rs +++ b/futures-util/src/future/chain.rs @@ -1,54 +1,52 @@ use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; #[must_use = "futures do nothing unless polled"] #[derive(Debug)] -pub enum Chain { +crate enum Chain { First(Fut1, Option), Second(Fut2), + Empty, } impl Chain where Fut1: Future, Fut2: Future, { - pub fn new(fut1: Fut1, data: Data) -> Chain { + crate fn new(fut1: Fut1, data: Data) -> Chain { Chain::First(fut1, Some(data)) } - pub fn poll(mut self: PinMut, cx: &mut task::Context, f: F) -> Poll + crate fn poll( + self: PinMut, + cx: &mut task::Context, + async_op: F, + ) -> Poll where F: FnOnce(Fut1::Output, Data) -> Fut2, { - let mut f = Some(f); + let mut async_op = Some(async_op); + + // Safe to call `get_mut_unchecked` because we won't move the futures. + let this = unsafe { PinMut::get_mut_unchecked(self) }; loop { - // Safe to use `get_mut_unchecked` here because we don't move out - let fut2 = match unsafe { PinMut::get_mut_unchecked(self.reborrow()) } { + let (output, data) = match this { Chain::First(fut1, data) => { - // safe to create a new `PinMut` because `fut1` will never move - // before it's dropped. match unsafe { PinMut::new_unchecked(fut1) }.poll(cx) { Poll::Pending => return Poll::Pending, - Poll::Ready(t) => { - (f.take().unwrap())(t, data.take().unwrap()) - } + Poll::Ready(output) => (output, data.take().unwrap()), } } Chain::Second(fut2) => { - // Safe to create a new `PinMut` because `fut2` will never move - // before it's dropped; once we're in `Chain::Second` we stay - // there forever. - return unsafe { PinMut::new_unchecked(fut2) }.poll(cx) + return unsafe { PinMut::new_unchecked(fut2) }.poll(cx); } + Chain::Empty => unreachable!() }; - // Safe because we're using the `&mut` to do an assignment, not for moving out - unsafe { - // Note: It's safe to move the `fut2` here because we haven't yet polled it - *PinMut::get_mut_unchecked(self.reborrow()) = Chain::Second(fut2); - } + *this = Chain::Empty; // Drop fut1 + let fut2 = (async_op.take().unwrap())(output, data); + *this = Chain::Second(fut2) } } } diff --git a/futures-util/src/future/empty.rs b/futures-util/src/future/empty.rs index 2023d4b1a3..8e8367f579 100644 --- a/futures-util/src/future/empty.rs +++ b/futures-util/src/future/empty.rs @@ -1,10 +1,9 @@ //! Definition of the Empty combinator, a future that's never ready. -use core::mem::PinMut; use core::marker; - -use futures_core::{Future, Poll}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; /// A future which is never resolved. /// diff --git a/futures-util/src/future/flatten.rs b/futures-util/src/future/flatten.rs index 37f2180e97..38c82d0dbd 100644 --- a/futures-util/src/future/flatten.rs +++ b/futures-util/src/future/flatten.rs @@ -1,8 +1,7 @@ use core::fmt; use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; use super::chain::Chain; @@ -12,10 +11,20 @@ use super::chain::Chain; /// /// This is created by the `Future::flatten` method. #[must_use = "futures do nothing unless polled"] -pub struct Flatten where A: Future, A::Output: Future { +pub struct Flatten + where A: Future, + A::Output: Future, +{ state: Chain, } +impl Flatten + where A: Future, + A::Output: Future, +{ + unsafe_pinned!(state -> Chain); +} + impl fmt::Debug for Flatten where A: Future + fmt::Debug, A::Output: Future + fmt::Debug, @@ -43,6 +52,6 @@ impl Future for Flatten type Output = ::Output; fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - unsafe { pinned_field!(self, state) }.poll(cx, |a, ()| a) + self.state().poll(cx, |a, ()| a) } } diff --git a/futures-util/src/future/flatten_stream.rs b/futures-util/src/future/flatten_stream.rs index 319ed27353..0dd930bd2e 100644 --- a/futures-util/src/future/flatten_stream.rs +++ b/futures-util/src/future/flatten_stream.rs @@ -1,8 +1,8 @@ use core::fmt; use core::mem::PinMut; - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// Future for the `flatten_stream` combinator, flattening a /// future-of-a-stream to get just the result of the final stream as a stream. diff --git a/futures-util/src/future/fuse.rs b/futures-util/src/future/fuse.rs index 1323ccbd8b..9e20732125 100644 --- a/futures-util/src/future/fuse.rs +++ b/futures-util/src/future/fuse.rs @@ -1,7 +1,6 @@ use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; /// A future which "fuses" a future once it's been resolved. /// diff --git a/futures-util/src/future/inspect.rs b/futures-util/src/future/inspect.rs index 1e258c122b..fec77d6b98 100644 --- a/futures-util/src/future/inspect.rs +++ b/futures-util/src/future/inspect.rs @@ -1,12 +1,11 @@ use core::marker::Unpin; use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; /// Do something with the item of a future, passing it on. /// -/// This is created by the [`FutureExt::inspect`] method. +/// This is created by the [`super::FutureExt::inspect`] method. #[derive(Debug)] #[must_use = "futures do nothing unless polled"] pub struct Inspect where A: Future { diff --git a/futures-util/src/future/into_stream.rs b/futures-util/src/future/into_stream.rs index ef71da0db3..673bc3bc99 100644 --- a/futures-util/src/future/into_stream.rs +++ b/futures-util/src/future/into_stream.rs @@ -1,7 +1,7 @@ use core::mem::PinMut; - -use futures_core::{Poll, Future, Stream}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A type which converts a `Future` into a `Stream` /// containing a single element. diff --git a/futures-util/src/future/join.rs b/futures-util/src/future/join.rs index 44c1452df1..51f6bc42c3 100644 --- a/futures-util/src/future/join.rs +++ b/futures-util/src/future/join.rs @@ -1,12 +1,10 @@ #![allow(non_snake_case)] +use crate::future::{MaybeDone, maybe_done}; use core::fmt; use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task; - -use crate::future::{MaybeDone, maybe_done}; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; macro_rules! generate { ($( diff --git a/futures-util/src/future/join_all.rs b/futures-util/src/future/join_all.rs index e6c20ffdd6..72cfec76af 100644 --- a/futures-util/src/future/join_all.rs +++ b/futures-util/src/future/join_all.rs @@ -1,14 +1,12 @@ //! Definition of the `JoinAll` combinator, waiting for all of a list of futures //! to finish. -use std::prelude::v1::*; - -use std::fmt; -use std::mem; -use std::iter::FromIterator; - use futures_core::{Future, IntoFuture, Poll, Async}; use futures_core::task; +use std::fmt; +use std::iter::FromIterator; +use std::mem; +use std::prelude::v1::*; #[derive(Debug)] enum ElemState where F: Future { diff --git a/futures-util/src/future/lazy.rs b/futures-util/src/future/lazy.rs index 10f419c95d..450611363c 100644 --- a/futures-util/src/future/lazy.rs +++ b/futures-util/src/future/lazy.rs @@ -1,11 +1,10 @@ //! Definition of the Lazy combinator, deferring execution of a function until //! the future is polled. -use core::mem::PinMut; use core::marker::Unpin; - -use futures_core::{Future, Poll}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; /// A future which, when polled, invokes a closure and yields its result. /// diff --git a/futures-util/src/future/map.rs b/futures-util/src/future/map.rs index 02d37848c5..df1d14dc3e 100644 --- a/futures-util/src/future/map.rs +++ b/futures-util/src/future/map.rs @@ -1,48 +1,44 @@ use core::marker::Unpin; use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; /// Future for the `map` combinator, changing the type of a future. /// /// This is created by the `Future::map` method. #[derive(Debug)] #[must_use = "futures do nothing unless polled"] -pub struct Map where A: Future { - future: A, - f: Option, +pub struct Map { + future: Fut, + op: Option, } -pub fn new(future: A, f: F) -> Map - where A: Future, -{ - Map { - future, - f: Some(f), - } -} +impl Map { + unsafe_pinned!(future -> Fut); + unsafe_unpinned!(op -> Option); -impl Map { - unsafe_pinned!(future -> A); - unsafe_unpinned!(f -> Option); + /// Creates a new Map. + pub(super) fn new(future: Fut, op: F) -> Map { + Map { future, op: Some(op) } + } } -impl Unpin for Map {} +impl Unpin for Map {} -impl Future for Map - where A: Future, - F: FnOnce(A::Output) -> U, +impl Future for Map + where Fut: Future, + F: FnOnce(Fut::Output) -> T, { - type Output = U; - - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - let e = match self.future().poll(cx) { - Poll::Pending => return Poll::Pending, - Poll::Ready(e) => e, - }; - - let f = self.f().take().expect("cannot poll Map twice"); - Poll::Ready(f(e)) + type Output = T; + + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + match self.future().poll(cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(output) => { + let op = self.op().take() + .expect("Map must not be polled after it returned `Poll::Ready`"); + Poll::Ready(op(output)) + } + } } } diff --git a/futures-util/src/future/maybe_done.rs b/futures-util/src/future/maybe_done.rs index 27767a6253..639978cfb5 100644 --- a/futures-util/src/future/maybe_done.rs +++ b/futures-util/src/future/maybe_done.rs @@ -1,10 +1,9 @@ //! Definition of the MaybeDone combinator -use core::mem::{self, PinMut}; use core::marker::Unpin; - -use futures_core::{Future, Poll}; -use futures_core::task; +use core::mem::{self, PinMut}; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; /// `MaybeDone`, a future that may have completed. /// diff --git a/futures-util/src/future/mod.rs b/futures-util/src/future/mod.rs index dcc513466d..3dbb6c4ac2 100644 --- a/futures-util/src/future/mod.rs +++ b/futures-util/src/future/mod.rs @@ -3,64 +3,74 @@ //! This module contains a number of functions for working with `Future`s, //! including the `FutureExt` trait which adds methods to `Future` types. -use futures_core::{Future, Stream}; +use futures_core::future::Future; +use futures_core::stream::Stream; // Primitive futures mod empty; -mod lazy; -mod maybe_done; -mod poll_fn; pub use self::empty::{empty, Empty}; + +mod lazy; pub use self::lazy::{lazy, Lazy}; + +mod maybe_done; pub use self::maybe_done::{maybe_done, MaybeDone}; + +mod poll_fn; pub use self::poll_fn::{poll_fn, PollFn}; -// combinators +// Combinators mod flatten; -mod flatten_stream; -mod fuse; -mod into_stream; -mod join; -mod map; -// mod select; -mod then; -mod inspect; - -// impl details -mod chain; - pub use self::flatten::Flatten; + +mod flatten_stream; pub use self::flatten_stream::FlattenStream; + +mod fuse; pub use self::fuse::Fuse; + +mod into_stream; pub use self::into_stream::IntoStream; + +mod join; pub use self::join::{Join, Join3, Join4, Join5}; + +mod map; pub use self::map::Map; + +// Todo +// mod select; // pub use self::select::Select; + +mod then; pub use self::then::Then; + +mod inspect; pub use self::inspect::Inspect; -pub use either::Either; +mod with_executor; +pub use self::with_executor::WithExecutor; + +// Implementation details +mod chain; +crate use self::chain::Chain; if_std! { mod catch_unwind; + pub use self::catch_unwind::CatchUnwind; - /* TODO - mod join_all; - mod select_all; - mod select_ok; - */ - mod shared; + // ToDo + // mod join_all; + // pub use self::join_all::{join_all, JoinAll}; - /* - pub use self::join_all::{join_all, JoinAll}; - pub use self::select_all::{SelectAll, SelectAllNext, select_all}; - pub use self::select_ok::{SelectOk, select_ok}; - */ - pub use self::shared::Shared; + // mod select_all; + // pub use self::select_all::{SelectAll, SelectAllNext, select_all}; - mod with_executor; - pub use self::catch_unwind::CatchUnwind; - pub use self::with_executor::WithExecutor; + // mod select_ok; + // pub use self::select_ok::{SelectOk, select_ok}; + + mod shared; + pub use self::shared::Shared; } impl FutureExt for T where T: Future {} @@ -97,7 +107,7 @@ pub trait FutureExt: Future { where F: FnOnce(Self::Output) -> U, Self: Sized, { - assert_future::(map::new(self, f)) + assert_future::(Map::new(self, f)) } /// Chain on a computation for when a future finished, passing the result of @@ -125,12 +135,12 @@ pub trait FutureExt: Future { /// let future_of_4 = future_of_1.then(|x| future::ready(x + 3)); /// assert_eq!(block_on(future_of_4), 4); /// ``` - fn then(self, f: F) -> Then - where F: FnOnce(Self::Output) -> B, - B: Future, + fn then(self, async_op: F) -> Then + where F: FnOnce(Self::Output) -> Fut, + Fut: Future, Self: Sized, { - assert_future::(then::new(self, f)) + assert_future::(Then::new(self, async_op)) } /* TODO @@ -572,7 +582,6 @@ pub trait FutureExt: Future { /// let output = block_on(block_on(future)); /// assert_eq!(output, 3); /// ``` - #[cfg(feature = "std")] fn with_executor(self, executor: E) -> WithExecutor where Self: Sized, E: ::futures_core::task::Executor diff --git a/futures-util/src/future/poll_fn.rs b/futures-util/src/future/poll_fn.rs index 361ba8b096..2248ffad3a 100644 --- a/futures-util/src/future/poll_fn.rs +++ b/futures-util/src/future/poll_fn.rs @@ -2,9 +2,8 @@ use core::marker::Unpin; use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; /// A future which adapts a function returning `Poll`. /// @@ -26,7 +25,6 @@ pub struct PollFn { /// # extern crate futures; /// use futures::prelude::*; /// use futures::future::poll_fn; -/// use futures::task; /// /// fn read_line(cx: &mut task::Context) -> Poll { /// Poll::Ready("Hello, World!".into()) diff --git a/futures-util/src/future/shared.rs b/futures-util/src/future/shared.rs index b0010e3286..bc46688329 100644 --- a/futures-util/src/future/shared.rs +++ b/futures-util/src/future/shared.rs @@ -18,8 +18,9 @@ //! # } //! ``` +use futures_core::future::Future; +use futures_core::task::{self, Poll, Wake, Waker}; use slab::Slab; - use std::fmt; use std::cell::UnsafeCell; use std::marker::Unpin; @@ -29,9 +30,6 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; use std::task::local_waker_from_nonlocal; -use futures_core::{Future, Poll}; -use futures_core::task::{self, Wake, Waker}; - /// A future that is cloneable and can be polled in multiple threads. /// Use `Future::shared()` method to convert any future into a `Shared` future. #[must_use = "futures do nothing unless polled"] diff --git a/futures-util/src/future/then.rs b/futures-util/src/future/then.rs index 4d07245a5b..0aa8daa658 100644 --- a/futures-util/src/future/then.rs +++ b/futures-util/src/future/then.rs @@ -1,8 +1,7 @@ +use super::Chain; use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task; -use super::chain::Chain; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; /// Future for the `then` combinator, chaining computations on the end of /// another future regardless of its outcome. @@ -10,27 +9,32 @@ use super::chain::Chain; /// This is created by the `Future::then` method. #[derive(Debug)] #[must_use = "futures do nothing unless polled"] -pub struct Then where A: Future, B: Future { - state: Chain, +pub struct Then { + chain: Chain, } -pub fn new(future: A, f: F) -> Then - where A: Future, - B: Future, +impl Then + where Fut1: Future, + Fut2: Future, { - Then { - state: Chain::new(future, f), + unsafe_pinned!(chain -> Chain); + + /// Creates a new `Then`. + pub(super) fn new(future: Fut1, async_op: F) -> Then { + Then { + chain: Chain::new(future, async_op), + } } } -impl Future for Then - where A: Future, - B: Future, - F: FnOnce(A::Output) -> B, +impl Future for Then + where Fut1: Future, + Fut2: Future, + F: FnOnce(Fut1::Output) -> Fut2, { - type Output = B::Output; + type Output = Fut2::Output; - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - unsafe { pinned_field!(self, state) }.poll(cx, |a, f| f(a)) + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + self.chain().poll(cx, |output, async_op| async_op(output)) } } diff --git a/futures-util/src/future/with_executor.rs b/futures-util/src/future/with_executor.rs index 7f4ab4225e..f6815e7110 100644 --- a/futures-util/src/future/with_executor.rs +++ b/futures-util/src/future/with_executor.rs @@ -1,8 +1,7 @@ use core::marker::Unpin; use core::mem::PinMut; - -use futures_core::{Future, Poll}; -use futures_core::task::{self, Executor}; +use futures_core::future::Future; +use futures_core::task::{self, Poll, Executor}; /// Future for the `with_executor` combinator, assigning an executor /// to be used when spawning other futures. diff --git a/futures-util/src/io/allow_std.rs b/futures-util/src/io/allow_std.rs index 867d0321f5..bc3df36a90 100644 --- a/futures-util/src/io/allow_std.rs +++ b/futures-util/src/io/allow_std.rs @@ -1,4 +1,4 @@ -use futures_core::{Poll, task}; +use futures_core::task::{self, Poll}; use futures_io::{AsyncRead, AsyncWrite}; use std::{fmt, io}; use std::string::String; diff --git a/futures-util/src/io/close.rs b/futures-util/src/io/close.rs index 8c5832dbdf..37d4b2dbf2 100644 --- a/futures-util/src/io/close.rs +++ b/futures-util/src/io/close.rs @@ -1,11 +1,10 @@ +use futures_core::future::Future; +use futures_core::task::{self, Poll}; +use futures_io::AsyncWrite; use std::io; use std::marker::Unpin; use std::mem::PinMut; -use crate::{Poll, Future, task}; - -use futures_io::AsyncWrite; - /// A future used to fully close an I/O object. /// /// Created by the [`close`] function. diff --git a/futures-util/src/io/copy_into.rs b/futures-util/src/io/copy_into.rs index bd5a8defe9..63429b141e 100644 --- a/futures-util/src/io/copy_into.rs +++ b/futures-util/src/io/copy_into.rs @@ -1,12 +1,11 @@ -use std::io; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; +use futures_io::{AsyncRead, AsyncWrite}; use std::boxed::Box; +use std::io; use std::marker::Unpin; use std::mem::PinMut; -use crate::{Future, Poll, task}; - -use futures_io::{AsyncRead, AsyncWrite}; - /// A future which will copy all data from a reader into a writer. /// /// Created by the [`copy_into`] function, this future will resolve to the number of diff --git a/futures-util/src/io/flush.rs b/futures-util/src/io/flush.rs index 715edd664b..8cecc6b105 100644 --- a/futures-util/src/io/flush.rs +++ b/futures-util/src/io/flush.rs @@ -1,9 +1,9 @@ +use futures_core::future::Future; +use futures_core::task::{self, Poll}; use std::io; use std::marker::Unpin; use std::mem::PinMut; -use crate::{Poll, Future, task}; - use futures_io::AsyncWrite; /// A future used to fully flush an I/O object. diff --git a/futures-util/src/io/read.rs b/futures-util/src/io/read.rs index 8de033302c..89ec84e8d7 100644 --- a/futures-util/src/io/read.rs +++ b/futures-util/src/io/read.rs @@ -1,11 +1,10 @@ +use crate::io::AsyncRead; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; use std::io; use std::marker::Unpin; use std::mem::PinMut; -use crate::{Future, Poll, task}; - -use crate::io::AsyncRead; - /// A future which can be used to easily read available number of bytes to fill /// a buffer. /// diff --git a/futures-util/src/io/read_exact.rs b/futures-util/src/io/read_exact.rs index 9df4fa3d15..9dbaefae11 100644 --- a/futures-util/src/io/read_exact.rs +++ b/futures-util/src/io/read_exact.rs @@ -1,11 +1,10 @@ +use crate::io::AsyncRead; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; use std::io; use std::marker::Unpin; use std::mem::{self, PinMut}; -use crate::{Poll, Future, task}; - -use crate::io::AsyncRead; - /// A future which can be used to easily read exactly enough bytes to fill /// a buffer. /// diff --git a/futures-util/src/io/read_to_end.rs b/futures-util/src/io/read_to_end.rs index fc2e5b6af4..73fda0d7e0 100644 --- a/futures-util/src/io/read_to_end.rs +++ b/futures-util/src/io/read_to_end.rs @@ -1,12 +1,11 @@ +use futures_core::future::Future; +use futures_core::task::{self, Poll}; +use futures_io::AsyncRead; use std::io; use std::marker::Unpin; use std::mem::PinMut; use std::vec::Vec; -use crate::{Poll, Future, task}; - -use crate::io::AsyncRead; - /// A future which can be used to easily read the entire contents of a stream /// into a vector. /// diff --git a/futures-util/src/io/split.rs b/futures-util/src/io/split.rs index 26c10dd6c8..f373a54c94 100644 --- a/futures-util/src/io/split.rs +++ b/futures-util/src/io/split.rs @@ -1,10 +1,8 @@ -use std::io; -use std::mem::PinMut; - -use crate::{Poll, task}; use crate::lock::BiLock; - +use futures_core::task::{self, Poll}; use futures_io::{AsyncRead, AsyncWrite, IoVec}; +use std::io; +use std::mem::PinMut; /// The readable half of an object returned from `AsyncRead::split`. #[derive(Debug)] diff --git a/futures-util/src/io/write_all.rs b/futures-util/src/io/write_all.rs index d34fa62f9b..feb536a766 100644 --- a/futures-util/src/io/write_all.rs +++ b/futures-util/src/io/write_all.rs @@ -1,11 +1,10 @@ +use futures_core::future::Future; +use futures_core::task::{self, Poll}; +use futures_io::AsyncWrite; use std::io; use std::marker::Unpin; use std::mem::{self, PinMut}; -use crate::{Poll, Future, task}; - -use futures_io::AsyncWrite; - /// A future used to write the entire contents of some data to a stream. /// /// This is created by the [`write_all`] top-level method. diff --git a/futures-util/src/lib.rs b/futures-util/src/lib.rs index eb4df24fe8..de128c3b17 100644 --- a/futures-util/src/lib.rs +++ b/futures-util/src/lib.rs @@ -7,22 +7,10 @@ #![deny(missing_docs, missing_debug_implementations, warnings)] #![deny(bare_trait_objects)] -#![doc(html_root_url = "https://docs.rs/futures/0.1")] +#![doc(html_root_url = "https://docs.rs/futures-util/0.3.0-alpha")] -#[cfg(test)] -extern crate futures_channel; #[macro_use] extern crate futures_core; -#[cfg(test)] -extern crate futures_executor; - -extern crate futures_io; -extern crate futures_sink; - -extern crate either; - -#[cfg(feature = "std")] -extern crate slab; macro_rules! if_std { ($($i:item)*) => ($( @@ -36,19 +24,31 @@ pub use futures_core::core_reexport; macro_rules! delegate_sink { ($field:ident) => { - fn poll_ready(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_ready( + mut self: PinMut, + cx: &mut $crate::core_reexport::task::Context, + ) -> $crate::core_reexport::task::Poll> { self.$field().poll_ready(cx) } - fn start_send(mut self: PinMut, item: Self::SinkItem) -> Result<(), Self::SinkError> { + fn start_send( + mut self: PinMut, + item: Self::SinkItem + ) -> Result<(), Self::SinkError> { self.$field().start_send(item) } - fn poll_flush(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_flush( + mut self: PinMut, + cx: &mut $crate::core_reexport::task::Context + ) -> $crate::core_reexport::task::Poll> { self.$field().poll_flush(cx) } - fn poll_close(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_close( + mut self: PinMut, + cx: &mut $crate::core_reexport::task::Context + ) -> $crate::core_reexport::task::Poll> { self.$field().poll_close(cx) } } @@ -70,16 +70,11 @@ pub mod sink; pub use crate::sink::SinkExt; if_std! { - extern crate core; - - use futures_core::{Future, Poll, task}; - // FIXME: currently async/await is only available with std - pub mod await; + pub mod async_await; pub mod io; pub use crate::io::{AsyncReadExt, AsyncWriteExt}; - #[cfg(any(test, feature = "bench"))] pub mod lock; #[cfg(not(any(test, feature = "bench")))] diff --git a/futures-util/src/lock.rs b/futures-util/src/lock.rs index 640751be5c..5850a6abfe 100644 --- a/futures-util/src/lock.rs +++ b/futures-util/src/lock.rs @@ -1,6 +1,8 @@ //! Futures-powered synchronization primitives. #![allow(unused)] +use futures_core::future::Future; +use futures_core::task::{self, Poll, Waker}; use std::any::Any; use std::boxed::Box; use std::cell::UnsafeCell; @@ -13,9 +15,6 @@ use std::sync::Arc; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; -use futures_core::{Future, Poll}; -use futures_core::task::{self, Waker}; - /// A type of futures-powered synchronization primitive which is a mutex between /// two possible owners. /// diff --git a/futures-util/src/sink/buffer.rs b/futures-util/src/sink/buffer.rs index f632fb0724..945f463c5b 100644 --- a/futures-util/src/sink/buffer.rs +++ b/futures-util/src/sink/buffer.rs @@ -1,9 +1,7 @@ -use std::collections::VecDeque; - -use futures_core::{Poll, Stream}; -use futures_core::task; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; use futures_sink::Sink; - +use std::collections::VecDeque; use std::marker::Unpin; use std::mem::PinMut; diff --git a/futures-util/src/sink/close.rs b/futures-util/src/sink/close.rs index 45e34ae506..b515babb65 100644 --- a/futures-util/src/sink/close.rs +++ b/futures-util/src/sink/close.rs @@ -1,9 +1,8 @@ -use futures_core::{Poll, Future}; -use futures_core::task; -use futures_sink::Sink; - use core::marker::Unpin; use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; +use futures_sink::Sink; /// Future for the `close` combinator, which polls the sink until all data has /// been closed. diff --git a/futures-util/src/sink/err_into.rs b/futures-util/src/sink/err_into.rs index e212f73fd6..8ea4c6abd6 100644 --- a/futures-util/src/sink/err_into.rs +++ b/futures-util/src/sink/err_into.rs @@ -1,9 +1,8 @@ -use futures_core::{Stream, Poll}; -use futures_core::task; -use futures_sink::{Sink}; use crate::sink::{SinkExt, SinkMapErr}; - use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; +use futures_sink::{Sink}; /// A sink combinator to change the error type of a sink. /// diff --git a/futures-util/src/sink/fanout.rs b/futures-util/src/sink/fanout.rs index 4e3a25c70f..a9c706d1b1 100644 --- a/futures-util/src/sink/fanout.rs +++ b/futures-util/src/sink/fanout.rs @@ -1,8 +1,6 @@ use core::fmt::{Debug, Formatter, Result as FmtResult}; use core::mem::PinMut; - -use futures_core::Poll; -use futures_core::task; +use futures_core::task::{self, Poll}; use futures_sink::Sink; /// Sink that clones incoming items and forwards them to two sinks at the same time. diff --git a/futures-util/src/sink/flush.rs b/futures-util/src/sink/flush.rs index a507759a46..2806e88e1c 100644 --- a/futures-util/src/sink/flush.rs +++ b/futures-util/src/sink/flush.rs @@ -1,9 +1,8 @@ -use futures_core::{Poll, Future}; -use futures_core::task; -use futures_sink::Sink; - use core::marker::Unpin; use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; +use futures_sink::Sink; /// Future for the `flush` combinator, which polls the sink until all data /// has been flushed. diff --git a/futures-util/src/sink/map_err.rs b/futures-util/src/sink/map_err.rs index f9139f7819..5f2050c1fc 100644 --- a/futures-util/src/sink/map_err.rs +++ b/futures-util/src/sink/map_err.rs @@ -1,9 +1,8 @@ -use futures_core::{Poll, Stream}; -use futures_core::task; -use futures_sink::{Sink}; - use core::marker::Unpin; use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; +use futures_sink::{Sink}; /// Sink for the `Sink::sink_map_err` combinator. #[derive(Debug)] diff --git a/futures-util/src/sink/mod.rs b/futures-util/src/sink/mod.rs index 249a42fbc6..825a45fe70 100644 --- a/futures-util/src/sink/mod.rs +++ b/futures-util/src/sink/mod.rs @@ -3,10 +3,11 @@ //! This module contains a number of functions for working with `Sink`s, //! including the `SinkExt` trait which adds methods to `Sink` types. -use futures_core::{Future, Stream}; -use futures_sink::Sink; -use super::future::Either; use core::marker::Unpin; +use either::Either; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_sink::Sink; mod close; mod fanout; @@ -204,8 +205,8 @@ pub trait SinkExt: Sink { /// Doing `sink.send_all(stream)` is roughly equivalent to /// `stream.forward(sink)`. The returned future will exhaust all items from /// `stream` and send them to `self`. - fn send_all<'a, S, E>(&'a mut self, stream: &'a mut S) -> SendAll<'a, Self, S> - where S: Stream> + Unpin, + fn send_all<'a, S>(&'a mut self, stream: &'a mut S) -> SendAll<'a, Self, S> + where S: Stream + Unpin, Self: Unpin, { send_all::new(self, stream) diff --git a/futures-util/src/sink/send.rs b/futures-util/src/sink/send.rs index cef354d2e4..d6ed038d43 100644 --- a/futures-util/src/sink/send.rs +++ b/futures-util/src/sink/send.rs @@ -1,9 +1,8 @@ -use futures_core::{Poll, Future}; -use futures_core::task; -use futures_sink::{Sink}; - use core::marker::Unpin; use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::task::{self, Poll}; +use futures_sink::Sink; /// Future for the `Sink::send` combinator, which sends a value to a sink and /// then waits until the sink has fully flushed. diff --git a/futures-util/src/sink/send_all.rs b/futures-util/src/sink/send_all.rs index f3a75e9b62..927a4be452 100644 --- a/futures-util/src/sink/send_all.rs +++ b/futures-util/src/sink/send_all.rs @@ -1,11 +1,10 @@ -use futures_core::{Poll, Future, Stream}; -use futures_core::task; -use futures_sink::{Sink}; - use crate::stream::{StreamExt, Fuse}; - use core::marker::Unpin; use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; +use futures_sink::Sink; /// Future for the `Sink::send_all` combinator, which sends a stream of values /// to a sink and then waits until the sink has fully flushed those values. @@ -31,7 +30,7 @@ where pub fn new<'a, T, U>(sink: &'a mut T, stream: &'a mut U) -> SendAll<'a, T, U> where T: Sink + Unpin + 'a + ?Sized, - U: Stream> + Unpin + 'a + ?Sized, + U: Stream + Unpin + 'a + ?Sized, { SendAll { sink, @@ -43,7 +42,7 @@ where impl<'a, T, U> SendAll<'a, T, U> where T: Sink + Unpin + 'a + ?Sized, - U: Stream> + Unpin + 'a + ?Sized, + U: Stream + Unpin + 'a + ?Sized, { fn try_start_send(&mut self, cx: &mut task::Context, item: T::SinkItem) -> Poll> @@ -65,7 +64,7 @@ where impl<'a, T, U> Future for SendAll<'a, T, U> where T: Sink + Unpin + 'a + ?Sized, - U: Stream> + Unpin + 'a + ?Sized, + U: Stream + Unpin + 'a + ?Sized, { type Output = Result<(), T::SinkError>; @@ -79,8 +78,9 @@ where loop { match this.stream.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(item))) => try_ready!(this.try_start_send(cx, item)), - Poll::Ready(Some(Err(err))) => return Poll::Ready(Err(err)), + Poll::Ready(Some(item)) => { + try_ready!(this.try_start_send(cx, item)) + } Poll::Ready(None) => { try_ready!(PinMut::new(this.sink).poll_flush(cx)); return Poll::Ready(Ok(())) diff --git a/futures-util/src/sink/with.rs b/futures-util/src/sink/with.rs index c4bfba8157..0a05eb6a15 100644 --- a/futures-util/src/sink/with.rs +++ b/futures-util/src/sink/with.rs @@ -1,10 +1,9 @@ -use core::marker::Unpin; +use core::marker::{Unpin, PhantomData}; use core::mem::{self, PinMut}; -use core::marker::PhantomData; - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; -use futures_sink::{Sink}; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; +use futures_sink::Sink; /// Sink for the `Sink::with` combinator, chaining a computation to run *prior* /// to pushing a value into the underlying sink. diff --git a/futures-util/src/sink/with_flat_map.rs b/futures-util/src/sink/with_flat_map.rs index 599f04f3e5..b5851b17e9 100644 --- a/futures-util/src/sink/with_flat_map.rs +++ b/futures-util/src/sink/with_flat_map.rs @@ -1,7 +1,7 @@ use core::marker::{Unpin, PhantomData}; use core::mem::PinMut; - -use futures_core::{task, Poll, Stream}; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; use futures_sink::Sink; /// Sink for the `Sink::with_flat_map` combinator, chaining a computation that returns an iterator diff --git a/futures-util/src/stream/buffer_unordered.rs b/futures-util/src/stream/buffer_unordered.rs index 0d25190e1d..ea6078b70f 100644 --- a/futures-util/src/stream/buffer_unordered.rs +++ b/futures-util/src/stream/buffer_unordered.rs @@ -1,12 +1,12 @@ +use crate::stream::{Fuse, FuturesUnordered}; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; +use futures_sink::Sink; use std::fmt; use std::marker::Unpin; use std::mem::PinMut; -use futures_core::{task, Future, Poll, Stream}; -use futures_sink::Sink; - -use crate::stream::{Fuse, FuturesUnordered}; - /// An adaptor for a stream of futures to execute the futures concurrently, if /// possible, delivering results as they become available. /// diff --git a/futures-util/src/stream/buffered.rs b/futures-util/src/stream/buffered.rs index 6b0368b83d..f1c6096d0c 100644 --- a/futures-util/src/stream/buffered.rs +++ b/futures-util/src/stream/buffered.rs @@ -1,12 +1,12 @@ +use crate::stream::{Fuse, FuturesOrdered}; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; +use futures_sink::Sink; use std::fmt; use std::marker::Unpin; use std::mem::PinMut; -use futures_core::{task, Future, Poll, Stream}; -use futures_sink::Sink; - -use crate::stream::{Fuse, FuturesOrdered}; - /// An adaptor for a stream of futures to execute the futures concurrently, if /// possible. /// @@ -136,7 +136,7 @@ where if let Some(val) = ready!(PinMut::new(self.in_progress_queue()).poll_next(cx)) { return Poll::Ready(Some(val)) } - + // If more values are still coming from the stream, we're not done yet if self.stream.is_done() { Poll::Pending diff --git a/futures-util/src/stream/catch_unwind.rs b/futures-util/src/stream/catch_unwind.rs index 76185a7516..0da3e13789 100644 --- a/futures-util/src/stream/catch_unwind.rs +++ b/futures-util/src/stream/catch_unwind.rs @@ -1,10 +1,9 @@ -use std::prelude::v1::*; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; use std::any::Any; -use std::panic::{catch_unwind, UnwindSafe, AssertUnwindSafe}; use std::mem::PinMut; - -use futures_core::{Poll, Stream}; -use futures_core::task; +use std::panic::{catch_unwind, UnwindSafe, AssertUnwindSafe}; +use std::prelude::v1::*; /// Stream for the `catch_unwind` combinator. /// diff --git a/futures-util/src/stream/chain.rs b/futures-util/src/stream/chain.rs index 2044f68224..800b0e4833 100644 --- a/futures-util/src/stream/chain.rs +++ b/futures-util/src/stream/chain.rs @@ -1,9 +1,6 @@ use core::mem::PinMut; - - - -use futures_core::{Stream, Poll}; -use futures_core::task; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// An adapter for chaining the output of two streams. /// diff --git a/futures-util/src/stream/chunks.rs b/futures-util/src/stream/chunks.rs index 3bfbfc065a..1983d50917 100644 --- a/futures-util/src/stream/chunks.rs +++ b/futures-util/src/stream/chunks.rs @@ -1,12 +1,10 @@ -use std::mem::{self, PinMut}; +use crate::stream::Fuse; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; use std::marker::Unpin; +use std::mem::{self, PinMut}; use std::prelude::v1::*; -use futures_core::{Poll, Stream}; -use futures_core::task; - -use crate::stream::Fuse; - /// An adaptor that chunks up elements in a vector. /// /// This adaptor will buffer up a list of items in the stream and pass on the diff --git a/futures-util/src/stream/collect.rs b/futures-util/src/stream/collect.rs index 2b75c3fcd4..9cbd3bb57c 100644 --- a/futures-util/src/stream/collect.rs +++ b/futures-util/src/stream/collect.rs @@ -1,9 +1,9 @@ -use std::prelude::v1::*; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; use std::marker::Unpin; use std::mem::{self, PinMut}; - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use std::prelude::v1::*; /// A future which collects all of the values of a stream into a vector. /// diff --git a/futures-util/src/stream/concat.rs b/futures-util/src/stream/concat.rs index c423c3f8e2..b9d3f51210 100644 --- a/futures-util/src/stream/concat.rs +++ b/futures-util/src/stream/concat.rs @@ -1,10 +1,10 @@ -use core::mem::PinMut; use core::fmt::{Debug, Formatter, Result as FmtResult}; -use core::default::Default; use core::marker::Unpin; - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use core::default::Default; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator to concatenate the results of a stream into the first /// yielded item. diff --git a/futures-util/src/stream/empty.rs b/futures-util/src/stream/empty.rs index b0c9f0b6bf..cfb9603b34 100644 --- a/futures-util/src/stream/empty.rs +++ b/futures-util/src/stream/empty.rs @@ -1,8 +1,7 @@ -use core::mem::PinMut; use core::marker::{Unpin, PhantomData}; - -use futures_core::{Stream, Poll}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream which contains no elements. /// diff --git a/futures-util/src/stream/filter.rs b/futures-util/src/stream/filter.rs index 7025bedca4..c163c9740a 100644 --- a/futures-util/src/stream/filter.rs +++ b/futures-util/src/stream/filter.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - - - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator used to filter the results of a stream and only yield /// some values. diff --git a/futures-util/src/stream/filter_map.rs b/futures-util/src/stream/filter_map.rs index 1b76ef26fd..d06c627cb8 100644 --- a/futures-util/src/stream/filter_map.rs +++ b/futures-util/src/stream/filter_map.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - - - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A combinator used to filter the results of a stream and simultaneously map /// them to a different type. diff --git a/futures-util/src/stream/flatten.rs b/futures-util/src/stream/flatten.rs index eb7ebe1642..2889985a70 100644 --- a/futures-util/src/stream/flatten.rs +++ b/futures-util/src/stream/flatten.rs @@ -1,9 +1,6 @@ use core::mem::PinMut; - - - -use futures_core::{Poll, Stream}; -use futures_core::task; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A combinator used to flatten a stream-of-streams into one long stream of /// elements. diff --git a/futures-util/src/stream/fold.rs b/futures-util/src/stream/fold.rs index 92d1976fd7..679e9f42a2 100644 --- a/futures-util/src/stream/fold.rs +++ b/futures-util/src/stream/fold.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - - - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A future used to collect all the results of a stream into one generic type. /// diff --git a/futures-util/src/stream/for_each.rs b/futures-util/src/stream/for_each.rs index dbe6bd6c11..e7ce5747ac 100644 --- a/futures-util/src/stream/for_each.rs +++ b/futures-util/src/stream/for_each.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - - - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator which executes a unit closure over each item on a /// stream. diff --git a/futures-util/src/stream/forward.rs b/futures-util/src/stream/forward.rs index 89702972f1..26b2e8a7bf 100644 --- a/futures-util/src/stream/forward.rs +++ b/futures-util/src/stream/forward.rs @@ -1,11 +1,11 @@ +use crate::stream::{StreamExt, Fuse}; use core::marker::Unpin; use core::mem::PinMut; - -use futures_core::{task, Future, Poll, Stream}; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; use futures_sink::Sink; -use crate::stream::{StreamExt, Fuse}; - const INVALID_POLL: &str = "polled `Forward` after completion"; /// Future for the `Stream::forward` combinator, which sends a stream of values diff --git a/futures-util/src/stream/fuse.rs b/futures-util/src/stream/fuse.rs index cb7f6f3580..e6a519677a 100644 --- a/futures-util/src/stream/fuse.rs +++ b/futures-util/src/stream/fuse.rs @@ -1,6 +1,6 @@ use core::mem::PinMut; - -use futures_core::{task, Poll, Stream}; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; use futures_sink::Sink; /// A stream which "fuse"s a stream once it's terminated. diff --git a/futures-util/src/stream/futures_ordered.rs b/futures-util/src/stream/futures_ordered.rs index 8b8fc85c06..b4d5b67876 100644 --- a/futures-util/src/stream/futures_ordered.rs +++ b/futures-util/src/stream/futures_ordered.rs @@ -1,3 +1,7 @@ +use crate::stream::FuturesUnordered; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; use std::cmp::{Eq, PartialEq, PartialOrd, Ord, Ordering}; use std::collections::binary_heap::{BinaryHeap, PeekMut}; use std::fmt::{self, Debug}; @@ -5,11 +9,6 @@ use std::iter::FromIterator; use std::marker::Unpin; use std::mem::PinMut; -use futures_core::{Future, Poll, Stream}; -use futures_core::task; - -use crate::stream::FuturesUnordered; - #[must_use = "futures do nothing unless polled"] #[derive(Debug)] struct OrderWrapper { diff --git a/futures-util/src/stream/futures_unordered/iter.rs b/futures-util/src/stream/futures_unordered/iter.rs index 8949eba135..eff377427c 100644 --- a/futures-util/src/stream/futures_unordered/iter.rs +++ b/futures-util/src/stream/futures_unordered/iter.rs @@ -1,8 +1,7 @@ -use std::marker::{PhantomData, Unpin}; -use std::mem::PinMut; - use super::FuturesUnordered; use super::node::Node; +use std::marker::{PhantomData, Unpin}; +use std::mem::PinMut; #[derive(Debug)] /// Mutable iterator over all futures in the unordered set. diff --git a/futures-util/src/stream/futures_unordered/mod.rs b/futures-util/src/stream/futures_unordered/mod.rs index 311c90c47f..2e3b55fedb 100644 --- a/futures-util/src/stream/futures_unordered/mod.rs +++ b/futures-util/src/stream/futures_unordered/mod.rs @@ -1,5 +1,8 @@ //! An unbounded set of futures. +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll, AtomicWaker}; use std::cell::UnsafeCell; use std::fmt::{self, Debug}; use std::iter::FromIterator; @@ -11,17 +14,16 @@ use std::sync::atomic::{AtomicPtr, AtomicBool}; use std::sync::{Arc, Weak}; use std::usize; -use futures_core::{Stream, Future, Poll}; -use futures_core::task::{self, AtomicWaker}; - mod abort; -mod ready_to_run_queue; + mod iter; +use self::iter::{IterMut, IterPinMut}; + mod node; +use self::node::Node; +mod ready_to_run_queue; use self::ready_to_run_queue::{ReadyToRunQueue, Dequeue}; -use self::iter::{IterMut, IterPinMut}; -use self::node::Node; /// A set of `Future`s which may complete in any order. /// diff --git a/futures-util/src/stream/futures_unordered/node.rs b/futures-util/src/stream/futures_unordered/node.rs index f24861f8ae..fbd7b21f1a 100644 --- a/futures-util/src/stream/futures_unordered/node.rs +++ b/futures-util/src/stream/futures_unordered/node.rs @@ -1,6 +1,6 @@ use std::cell::UnsafeCell; -use std::mem; use std::marker::PhantomData; +use std::mem; use std::ptr::{self, NonNull}; use std::sync::{Arc, Weak}; use std::sync::atomic::{AtomicPtr, AtomicBool}; diff --git a/futures-util/src/stream/inspect.rs b/futures-util/src/stream/inspect.rs index 9215fe562f..0ecf110a0f 100644 --- a/futures-util/src/stream/inspect.rs +++ b/futures-util/src/stream/inspect.rs @@ -1,8 +1,7 @@ -use core::mem::PinMut; use core::marker::Unpin; - -use futures_core::{Stream, Poll}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// Do something with the items of a stream, passing it on. /// diff --git a/futures-util/src/stream/into_future.rs b/futures-util/src/stream/into_future.rs index 047e3e492f..8a987334e5 100644 --- a/futures-util/src/stream/into_future.rs +++ b/futures-util/src/stream/into_future.rs @@ -1,8 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A combinator used to temporarily convert a stream into a future. /// diff --git a/futures-util/src/stream/iter.rs b/futures-util/src/stream/iter.rs index 5ba1e886e4..6619a6ec2c 100644 --- a/futures-util/src/stream/iter.rs +++ b/futures-util/src/stream/iter.rs @@ -1,8 +1,7 @@ -use core::mem::PinMut; use core::marker::Unpin; - -use futures_core::{Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream which is just a shim over an underlying instance of `Iterator`. /// diff --git a/futures-util/src/stream/map.rs b/futures-util/src/stream/map.rs index 6b9bbad2c4..495bc4b762 100644 --- a/futures-util/src/stream/map.rs +++ b/futures-util/src/stream/map.rs @@ -1,8 +1,7 @@ -use core::mem::PinMut; use core::marker::Unpin; - -use futures_core::{Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator which will change the type of a stream from one /// type to another. diff --git a/futures-util/src/stream/mod.rs b/futures-util/src/stream/mod.rs index 42c6650064..54d2bf1690 100644 --- a/futures-util/src/stream/mod.rs +++ b/futures-util/src/stream/mod.rs @@ -4,9 +4,10 @@ //! including the `StreamExt` trait which adds methods to `Stream` types. use core::marker::Unpin; -use futures_core::{Future, Stream}; +use either::Either; +use futures_core::future::Future; +use futures_core::stream::Stream; use futures_sink::Sink; -use super::future::Either; mod iter; pub use self::iter::{iter, Iter}; @@ -15,78 +16,111 @@ mod repeat; pub use self::repeat::{repeat, Repeat}; mod chain; +pub use self::chain::Chain; + mod concat; +pub use self::concat::Concat; + mod empty; +pub use self::empty::{empty, Empty}; + mod filter; +pub use self::filter::Filter; + mod filter_map; +pub use self::filter_map::FilterMap; + mod flatten; +pub use self::flatten::Flatten; + mod fold; +pub use self::fold::Fold; + +mod forward; +pub use self::forward::Forward; + mod for_each; +pub use self::for_each::ForEach; + mod fuse; +pub use self::fuse::Fuse; + mod into_future; +pub use self::into_future::StreamFuture; + mod inspect; +pub use self::inspect::Inspect; + mod map; +pub use self::map::Map; + mod next; +pub use self::next::Next; + mod once; +pub use self::once::{once, Once}; + mod peek; +pub use self::peek::Peekable; + mod poll_fn; +pub use self::poll_fn::{poll_fn, PollFn}; + mod select; +pub use self::select::Select; + mod skip; +pub use self::skip::Skip; + mod skip_while; +pub use self::skip_while::SkipWhile; + mod take; +pub use self::take::Take; + mod take_while; +pub use self::take_while::TakeWhile; + mod then; +pub use self::then::Then; + mod unfold; +pub use self::unfold::{unfold, Unfold}; + mod zip; -mod forward; -pub use self::chain::Chain; -pub use self::concat::Concat; -pub use self::empty::{Empty, empty}; -pub use self::filter::Filter; -pub use self::filter_map::FilterMap; -pub use self::flatten::Flatten; -pub use self::fold::Fold; -pub use self::for_each::ForEach; -pub use self::fuse::Fuse; -pub use self::into_future::StreamFuture; -pub use self::inspect::Inspect; -pub use self::map::Map; -pub use self::next::Next; -pub use self::once::{Once, once}; -pub use self::peek::Peekable; -pub use self::poll_fn::{poll_fn, PollFn}; -pub use self::select::Select; -pub use self::skip::Skip; -pub use self::skip_while::SkipWhile; -pub use self::take::Take; -pub use self::take_while::TakeWhile; -pub use self::then::Then; -pub use self::unfold::{Unfold, unfold}; pub use self::zip::Zip; -pub use self::forward::Forward; if_std! { use std; use std::iter::Extend; - mod buffered; mod buffer_unordered; - mod catch_unwind; - mod chunks; - mod collect; - //mod select_all; - mod split; - mod futures_unordered; - mod futures_ordered; - pub use self::buffered::Buffered; pub use self::buffer_unordered::BufferUnordered; + + mod buffered; + pub use self::buffered::Buffered; + + mod catch_unwind; pub use self::catch_unwind::CatchUnwind; + + mod chunks; pub use self::chunks::Chunks; + + mod collect; pub use self::collect::Collect; - //pub use self::select_all::{select_all, SelectAll}; - pub use self::split::{SplitStream, SplitSink, ReuniteError}; - pub use self::futures_unordered::{futures_unordered, FuturesUnordered}; + + mod futures_ordered; pub use self::futures_ordered::{futures_ordered, FuturesOrdered}; + + mod futures_unordered; + pub use self::futures_unordered::{futures_unordered, FuturesUnordered}; + + mod split; + pub use self::split::{SplitStream, SplitSink, ReuniteError}; + + // ToDo + // mod select_all; + // pub use self::select_all::{select_all, SelectAll}; } impl StreamExt for T where T: Stream {} diff --git a/futures-util/src/stream/next.rs b/futures-util/src/stream/next.rs index b5ced052cd..1d65559a84 100644 --- a/futures-util/src/stream/next.rs +++ b/futures-util/src/stream/next.rs @@ -1,8 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A future of the next element of a stream. #[derive(Debug)] diff --git a/futures-util/src/stream/once.rs b/futures-util/src/stream/once.rs index 3e6130f666..7dcfbe78f2 100644 --- a/futures-util/src/stream/once.rs +++ b/futures-util/src/stream/once.rs @@ -1,9 +1,7 @@ use core::mem::PinMut; - - - -use futures_core::{Poll, Stream, Future}; -use futures_core::task; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream which emits single element and then EOF. /// diff --git a/futures-util/src/stream/peek.rs b/futures-util/src/stream/peek.rs index e099e2b3c0..1c60beebf4 100644 --- a/futures-util/src/stream/peek.rs +++ b/futures-util/src/stream/peek.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; -use core::marker::Unpin; - -use futures_core::{Poll, Stream}; -use futures_core::task; - use crate::stream::{StreamExt, Fuse}; +use core::marker::Unpin; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A `Stream` that implements a `peek` method. /// @@ -64,7 +62,10 @@ impl Peekable { impl Stream for Peekable { type Item = S::Item; - fn poll_next(mut self: PinMut, cx: &mut task::Context) -> Poll> { + fn poll_next( + mut self: PinMut, + cx: &mut task::Context + ) -> Poll> { if let Some(item) = self.peeked().take() { return Poll::Ready(Some(item)) } diff --git a/futures-util/src/stream/poll_fn.rs b/futures-util/src/stream/poll_fn.rs index 3fd1a5f15b..134bb7d9ff 100644 --- a/futures-util/src/stream/poll_fn.rs +++ b/futures-util/src/stream/poll_fn.rs @@ -1,10 +1,9 @@ //! Definition of the `PollFn` combinator -use core::mem::PinMut; use core::marker::Unpin; - -use futures_core::{Stream, Poll}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream which adapts a function returning `Poll`. /// diff --git a/futures-util/src/stream/repeat.rs b/futures-util/src/stream/repeat.rs index 6dbfc44055..a45fab10af 100644 --- a/futures-util/src/stream/repeat.rs +++ b/futures-util/src/stream/repeat.rs @@ -1,8 +1,7 @@ -use core::mem::PinMut; use core::marker::Unpin; - -use futures_core::{Stream, Poll}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// Stream that produces the same element repeatedly. /// diff --git a/futures-util/src/stream/select.rs b/futures-util/src/stream/select.rs index 20d829c899..096f9331ae 100644 --- a/futures-util/src/stream/select.rs +++ b/futures-util/src/stream/select.rs @@ -1,9 +1,7 @@ -use core::mem::PinMut; - -use futures_core::{Poll, Stream}; -use futures_core::task; - use crate::stream::{StreamExt, Fuse}; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// An adapter for merging the output of two streams. /// diff --git a/futures-util/src/stream/skip.rs b/futures-util/src/stream/skip.rs index 6ef9695b24..3ff3ff865b 100644 --- a/futures-util/src/stream/skip.rs +++ b/futures-util/src/stream/skip.rs @@ -1,7 +1,6 @@ use core::mem::PinMut; - -use futures_core::{Poll, Stream}; -use futures_core::task; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator which skips a number of elements before continuing. /// diff --git a/futures-util/src/stream/skip_while.rs b/futures-util/src/stream/skip_while.rs index 9fc14340d9..8052644201 100644 --- a/futures-util/src/stream/skip_while.rs +++ b/futures-util/src/stream/skip_while.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - - - -use futures_core::{Poll, Future, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator which skips elements of a stream while a predicate /// holds. diff --git a/futures-util/src/stream/split.rs b/futures-util/src/stream/split.rs index 6aa30152ec..5dc0a46197 100644 --- a/futures-util/src/stream/split.rs +++ b/futures-util/src/stream/split.rs @@ -1,12 +1,12 @@ +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; +use futures_sink::Sink; use std::any::Any; use std::error::Error; use std::fmt; use std::marker::Unpin; use std::mem::PinMut; -use futures_core::{task, Stream, Poll}; -use futures_sink::Sink; - use crate::lock::BiLock; /// A `Stream` part of the split pair diff --git a/futures-util/src/stream/take.rs b/futures-util/src/stream/take.rs index d01a133b90..1303dbc5f9 100644 --- a/futures-util/src/stream/take.rs +++ b/futures-util/src/stream/take.rs @@ -1,7 +1,6 @@ use core::mem::PinMut; - -use futures_core::{Poll, Stream}; -use futures_core::task; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator which returns a maximum number of elements. /// diff --git a/futures-util/src/stream/take_while.rs b/futures-util/src/stream/take_while.rs index 41cd6a22c5..b02606b406 100644 --- a/futures-util/src/stream/take_while.rs +++ b/futures-util/src/stream/take_while.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - - - -use futures_core::{Poll, Future, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator which takes elements from a stream while a predicate /// holds. diff --git a/futures-util/src/stream/then.rs b/futures-util/src/stream/then.rs index 7d633fbe76..3086df8244 100644 --- a/futures-util/src/stream/then.rs +++ b/futures-util/src/stream/then.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - - - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// A stream combinator which chains a computation onto each item produced by a /// stream. diff --git a/futures-util/src/stream/unfold.rs b/futures-util/src/stream/unfold.rs index 36d7feb14c..22ed53be08 100644 --- a/futures-util/src/stream/unfold.rs +++ b/futures-util/src/stream/unfold.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; use core::marker::Unpin; - - - -use futures_core::{Future, Poll, Stream}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::Future; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// Creates a `Stream` from a seed and a closure returning a `Future`. /// diff --git a/futures-util/src/stream/zip.rs b/futures-util/src/stream/zip.rs index cb586bb570..344af93b40 100644 --- a/futures-util/src/stream/zip.rs +++ b/futures-util/src/stream/zip.rs @@ -1,10 +1,8 @@ -use core::mem::PinMut; -use core::marker::Unpin; - -use futures_core::{Poll, Stream}; -use futures_core::task; - use crate::stream::{StreamExt, Fuse}; +use core::marker::Unpin; +use core::mem::PinMut; +use futures_core::stream::Stream; +use futures_core::task::{self, Poll}; /// An adapter for merging the output of two streams. /// diff --git a/futures-util/src/try_future/and_then.rs b/futures-util/src/try_future/and_then.rs index 957d553b08..212071a8b7 100644 --- a/futures-util/src/try_future/and_then.rs +++ b/futures-util/src/try_future/and_then.rs @@ -1,7 +1,7 @@ +use super::{TryChain, TryChainAction}; use core::mem::PinMut; - -use futures_core::{Future, Poll, TryFuture}; -use futures_core::task; +use futures_core::future::{Future, TryFuture}; +use futures_core::task::{self, Poll}; /// Future for the `and_then` combinator, chaining a computation onto the end of /// another future which completes successfully. @@ -9,57 +9,37 @@ use futures_core::task; /// This is created by the `Future::and_then` method. #[derive(Debug)] #[must_use = "futures do nothing unless polled"] -pub struct AndThen { - state: State, +pub struct AndThen { + try_chain: TryChain, } -#[derive(Debug)] -enum State { - First(Fut1, Option), - Second(Fut2), -} +impl AndThen + where Fut1: TryFuture, + Fut2: TryFuture, +{ + unsafe_pinned!(try_chain -> TryChain); -pub fn new(future: A, f: F) -> AndThen { - AndThen { - state: State::First(future, Some(f)), + /// Creates a new `Then`. + pub(super) fn new(future: Fut1, async_op: F) -> AndThen { + AndThen { + try_chain: TryChain::new(future, async_op), + } } } -impl Future for AndThen - where A: TryFuture, - B: TryFuture, - F: FnOnce(A::Item) -> B, +impl Future for AndThen + where Fut1: TryFuture, + Fut2: TryFuture, + F: FnOnce(Fut1::Item) -> Fut2, { - type Output = Result; + type Output = Result; fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - loop { - // Safe to use `get_mut_unchecked` here because we don't move out - let fut2 = match &mut unsafe { PinMut::get_mut_unchecked(self.reborrow()) }.state { - State::First(fut1, data) => { - // safe to create a new `PinMut` because `fut1` will never move - // before it's dropped. - match unsafe { PinMut::new_unchecked(fut1) }.try_poll(cx) { - Poll::Pending => return Poll::Pending, - Poll::Ready(Err(e)) => return Poll::Ready(Err(e)), - Poll::Ready(Ok(v)) => { - (data.take().unwrap())(v) - } - } - } - State::Second(fut2) => { - // Safe to create a new `PinMut` because `fut2` will never move - // before it's dropped; once we're in `Chain::Second` we stay - // there forever. - return unsafe { PinMut::new_unchecked(fut2) }.try_poll(cx) - } - }; - - // Safe because we're using the `&mut` to do an assignment, not for moving out - unsafe { - // Note: it's safe to move the `fut2` here because we haven't yet polled it - PinMut::get_mut_unchecked(self.reborrow()).state = State::Second(fut2); + self.try_chain().poll(cx, |result, async_op| { + match result { + Ok(item) => TryChainAction::Future(async_op(item)), + Err(err) => TryChainAction::Output(Err(err)), } - } + }) } } diff --git a/futures-util/src/try_future/err_into.rs b/futures-util/src/try_future/err_into.rs index 339f324658..b280e3d4df 100644 --- a/futures-util/src/try_future/err_into.rs +++ b/futures-util/src/try_future/err_into.rs @@ -1,8 +1,7 @@ -use core::mem::PinMut; use core::marker::PhantomData; - -use futures_core::{Future, Poll, TryFuture}; -use futures_core::task; +use core::mem::PinMut; +use futures_core::future::{Future, TryFuture}; +use futures_core::task::{self, Poll}; /// Future for the `err_into` combinator, changing the error type of a future. /// @@ -11,13 +10,17 @@ use futures_core::task; #[must_use = "futures do nothing unless polled"] pub struct ErrInto { future: A, - f: PhantomData + _marker: PhantomData, +} + +impl ErrInto { + unsafe_pinned!(future -> A); } pub fn new(future: A) -> ErrInto { ErrInto { future, - f: PhantomData + _marker: PhantomData, } } @@ -28,7 +31,7 @@ impl Future for ErrInto type Output = Result; fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - match unsafe { pinned_field!(self, future) }.try_poll(cx) { + match self.future().try_poll(cx) { Poll::Pending => Poll::Pending, Poll::Ready(e) => { Poll::Ready(e.map_err(Into::into)) diff --git a/futures-util/src/try_future/flatten_sink.rs b/futures-util/src/try_future/flatten_sink.rs index 0a7ffb9f31..6dc78a6d4c 100644 --- a/futures-util/src/try_future/flatten_sink.rs +++ b/futures-util/src/try_future/flatten_sink.rs @@ -1,7 +1,7 @@ use core::marker::Unpin; use core::mem::PinMut; - -use futures_core::{task, Poll, TryFuture}; +use futures_core::future::TryFuture; +use futures_core::task::{self, Poll}; use futures_sink::Sink; #[derive(Debug)] diff --git a/futures-util/src/try_future/into_future.rs b/futures-util/src/try_future/into_future.rs new file mode 100644 index 0000000000..78140b4116 --- /dev/null +++ b/futures-util/src/try_future/into_future.rs @@ -0,0 +1,28 @@ +use core::mem::PinMut; +use futures_core::future::{Future, TryFuture}; +use futures_core::task::{self, Poll}; + +/// Converts a `TryFuture` into a normal `Future` +#[derive(Debug)] +#[must_use = "futures do nothing unless polled"] +pub struct IntoFuture { + future: Fut, +} + +impl IntoFuture { + unsafe_pinned!(future -> Fut); + + #[inline] + pub(super) fn new(future: Fut) -> IntoFuture { + IntoFuture { future } + } +} + +impl Future for IntoFuture { + type Output = Result; + + #[inline] + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + self.future().try_poll(cx) + } +} diff --git a/futures-util/src/try_future/map_err.rs b/futures-util/src/try_future/map_err.rs index 205bfc2dcb..5cd74da4af 100644 --- a/futures-util/src/try_future/map_err.rs +++ b/futures-util/src/try_future/map_err.rs @@ -1,39 +1,43 @@ +use core::marker::Unpin; use core::mem::PinMut; - -use futures_core::{Future, Poll, TryFuture}; -use futures_core::task; +use futures_core::future::{Future, TryFuture}; +use futures_core::task::{self, Poll}; /// Future for the `map_err` combinator, changing the type of a future. /// /// This is created by the `Future::map_err` method. #[derive(Debug)] #[must_use = "futures do nothing unless polled"] -pub struct MapErr { - future: A, - f: Option, +pub struct MapErr { + future: Fut, + op: Option, } -pub fn new(future: A, f: F) -> MapErr { - MapErr { - future, - f: Some(f), +impl MapErr { + unsafe_pinned!(future -> Fut); + unsafe_unpinned!(op -> Option); + + /// Creates a new MapErr. + pub(super) fn new(future: Fut, op: F) -> MapErr { + MapErr { future, op: Some(op) } } } -impl Future for MapErr - where A: TryFuture, - F: FnOnce(A::Error) -> U, +impl Unpin for MapErr {} + +impl Future for MapErr + where Fut: TryFuture, + F: FnOnce(Fut::Error) -> E, { - type Output = Result; + type Output = Result; fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - match unsafe { pinned_field!(self, future) }.try_poll(cx) { + match self.future().try_poll(cx) { Poll::Pending => Poll::Pending, - Poll::Ready(e) => { - let f = unsafe { - PinMut::get_mut_unchecked(self).f.take().expect("cannot poll MapErr twice") - }; - Poll::Ready(e.map_err(f)) + Poll::Ready(result) => { + let op = self.op().take() + .expect("MapErr must not be polled after it returned `Poll::Ready`"); + Poll::Ready(result.map_err(op)) } } } diff --git a/futures-util/src/try_future/map_ok.rs b/futures-util/src/try_future/map_ok.rs index f3f18618f8..0417ad60cb 100644 --- a/futures-util/src/try_future/map_ok.rs +++ b/futures-util/src/try_future/map_ok.rs @@ -1,39 +1,43 @@ +use core::marker::Unpin; use core::mem::PinMut; - -use futures_core::{Future, Poll, TryFuture}; -use futures_core::task; +use futures_core::future::{Future, TryFuture}; +use futures_core::task::{self, Poll}; /// Future for the `map_ok` combinator, changing the type of a future. /// /// This is created by the `Future::map_ok` method. #[derive(Debug)] #[must_use = "futures do nothing unless polled"] -pub struct MapOk { - future: A, - f: Option, +pub struct MapOk { + future: Fut, + op: Option, } -pub fn new(future: A, f: F) -> MapOk { - MapOk { - future, - f: Some(f), +impl MapOk { + unsafe_pinned!(future -> Fut); + unsafe_unpinned!(op -> Option); + + /// Creates a new MapOk. + pub(super) fn new(future: Fut, op: F) -> MapOk { + MapOk { future, op: Some(op) } } } -impl Future for MapOk - where A: TryFuture, - F: FnOnce(A::Item) -> U, +impl Unpin for MapOk {} + +impl Future for MapOk + where Fut: TryFuture, + F: FnOnce(Fut::Item) -> T, { - type Output = Result; + type Output = Result; fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - match unsafe { pinned_field!(self, future) }.try_poll(cx) { + match self.future().try_poll(cx) { Poll::Pending => Poll::Pending, - Poll::Ready(e) => { - let f = unsafe { - PinMut::get_mut_unchecked(self).f.take().expect("cannot poll MapOk twice") - }; - Poll::Ready(e.map(f)) + Poll::Ready(result) => { + let op = self.op().take() + .expect("MapOk must not be polled after it returned `Poll::Ready`"); + Poll::Ready(result.map(op)) } } } diff --git a/futures-util/src/try_future/mod.rs b/futures-util/src/try_future/mod.rs index 370cd16337..bfabb941cf 100644 --- a/futures-util/src/try_future/mod.rs +++ b/futures-util/src/try_future/mod.rs @@ -3,18 +3,9 @@ //! This module contains a number of functions for working with `Future`s, //! including the `FutureExt` trait which adds methods to `Future` types. -use futures_core::TryFuture; +use futures_core::future::TryFuture; use futures_sink::Sink; -// combinators -mod and_then; -mod flatten_sink; -mod map_ok; -mod map_err; -mod err_into; -mod or_else; -mod recover; - /* TODO mod join; mod select; @@ -31,13 +22,34 @@ pub use self::select_ok::{SelectOk, select_ok}; } */ +// Combinators +mod and_then; pub use self::and_then::AndThen; + +mod err_into; +pub use self::err_into::ErrInto; + +mod flatten_sink; pub use self::flatten_sink::FlattenSink; -pub use self::map_ok::MapOk; + +mod into_future; +pub use self::into_future::IntoFuture; + +mod map_err; pub use self::map_err::MapErr; -pub use self::err_into::ErrInto; + +mod map_ok; +pub use self::map_ok::MapOk; + +mod or_else; pub use self::or_else::OrElse; -pub use self::recover::Recover; + +mod unwrap_or_else; +pub use self::unwrap_or_else::UnwrapOrElse; + +// Implementation details +mod try_chain; +crate use self::try_chain::{TryChain, TryChainAction}; impl TryFutureExt for F {} @@ -100,11 +112,11 @@ pub trait TryFutureExt: TryFuture { /// let new_future = future.map_ok(|x| x + 3); /// assert_eq!(block_on(new_future), Err(1)); /// ``` - fn map_ok(self, f: F) -> MapOk - where F: FnOnce(Self::Item) -> U, + fn map_ok(self, op: F) -> MapOk + where F: FnOnce(Self::Item) -> T, Self: Sized, { - map_ok::new(self, f) + MapOk::new(self, op) } /// Map this future's error to a different error, returning a new future. @@ -146,11 +158,11 @@ pub trait TryFutureExt: TryFuture { /// let new_future = future.map_err(|x| x + 3); /// assert_eq!(block_on(new_future), Ok(1)); /// ``` - fn map_err(self, f: F) -> MapErr + fn map_err(self, op: F) -> MapErr where F: FnOnce(Self::Error) -> E, Self: Sized, { - map_err::new(self, f) + MapErr::new(self, op) } /// Map this future's error to a new error type using the `Into` trait. @@ -215,12 +227,12 @@ pub trait TryFutureExt: TryFuture { /// panic!("should not be called in case of an error"); /// }); /// ``` - fn and_then(self, f: F) -> AndThen - where F: FnOnce(Self::Item) -> B, - B: TryFuture, + fn and_then(self, async_op: F) -> AndThen + where F: FnOnce(Self::Item) -> Fut, + Fut: TryFuture, Self: Sized, { - and_then::new(self, f) + AndThen::new(self, async_op) } /// Execute another future if this one resolves with an error. @@ -257,12 +269,12 @@ pub trait TryFutureExt: TryFuture { /// panic!("should not be called in case of success"); /// }); /// ``` - fn or_else(self, f: F) -> OrElse - where F: FnOnce(Self::Error) -> B, - B: TryFuture, + fn or_else(self, async_op: F) -> OrElse + where F: FnOnce(Self::Error) -> Fut, + Fut: TryFuture, Self: Sized, { - or_else::new(self, f) + OrElse::new(self, async_op) } /* TODO @@ -400,7 +412,7 @@ pub trait TryFutureExt: TryFuture { /// Handle errors generated by this future by converting them into /// `Self::Item`. /// - /// Because it can never produce an error, the returned `Recover` future can + /// Because it can never produce an error, the returned `UnwrapOrElse` future can /// conform to any specific `Error` type, including `Never`. /// /// # Examples @@ -412,13 +424,22 @@ pub trait TryFutureExt: TryFuture { /// use futures::executor::block_on; /// /// let future = future::ready::>(Err("Boom!")); - /// let new_future = future.recover(|_| ()); + /// let new_future = future.unwrap_or_else(|_| ()); /// assert_eq!(block_on(new_future), ()); /// ``` - fn recover(self, f: F) -> Recover + fn unwrap_or_else(self, op: F) -> UnwrapOrElse where Self: Sized, F: FnOnce(Self::Error) -> Self::Item { - recover::new(self, f) + UnwrapOrElse::new(self, op) + } + + /// Wraps a `TryFuture` so that it implements `Future`. `TryFuture`s + /// currently do not implement the `Future` trait due to limitations of + /// the compiler. + fn into_future(self) -> IntoFuture + where Self: Sized, + { + IntoFuture::new(self) } } diff --git a/futures-util/src/try_future/or_else.rs b/futures-util/src/try_future/or_else.rs index de4c3241b9..03d18f0525 100644 --- a/futures-util/src/try_future/or_else.rs +++ b/futures-util/src/try_future/or_else.rs @@ -1,7 +1,7 @@ +use super::{TryChain, TryChainAction}; use core::mem::PinMut; - -use futures_core::{Future, Poll, TryFuture}; -use futures_core::task; +use futures_core::future::{Future, TryFuture}; +use futures_core::task::{self, Poll}; /// Future for the `or_else` combinator, chaining a computation onto the end of /// a future which fails with an error. @@ -9,57 +9,37 @@ use futures_core::task; /// This is created by the `Future::or_else` method. #[derive(Debug)] #[must_use = "futures do nothing unless polled"] -pub struct OrElse { - state: State, +pub struct OrElse { + try_chain: TryChain, } -#[derive(Debug)] -enum State { - First(Fut1, Option), - Second(Fut2), -} +impl OrElse + where Fut1: TryFuture, + Fut2: TryFuture, +{ + unsafe_pinned!(try_chain -> TryChain); -pub fn new(future: A, f: F) -> OrElse { - OrElse { - state: State::First(future, Some(f)), + /// Creates a new `Then`. + pub(super) fn new(future: Fut1, async_op: F) -> OrElse { + OrElse { + try_chain: TryChain::new(future, async_op), + } } } -impl Future for OrElse - where A: TryFuture, - B: TryFuture, - F: FnOnce(A::Error) -> B, +impl Future for OrElse + where Fut1: TryFuture, + Fut2: TryFuture, + F: FnOnce(Fut1::Error) -> Fut2, { - type Output = Result; + type Output = Result; fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - loop { - // Safe to use `get_mut_unchecked` here because we don't move out - let fut2 = match unsafe { PinMut::get_mut_unchecked(self.reborrow()) }.state { - State::First(ref mut fut1, ref mut data) => { - // Safe to create a new `PinMut` because `fut1` will never move - // before it's dropped. - match unsafe { PinMut::new_unchecked(fut1) }.try_poll(cx) { - Poll::Pending => return Poll::Pending, - Poll::Ready(Ok(v)) => return Poll::Ready(Ok(v)), - Poll::Ready(Err(e)) => { - (data.take().unwrap())(e) - } - } - } - State::Second(ref mut fut2) => { - // Safe to create a new `PinMut` because `fut2` will never move - // before it's dropped; once we're in `Chain::Second` we stay - // there forever. - return unsafe { PinMut::new_unchecked(fut2) }.try_poll(cx) - } - }; - - // Safe because we're using the `&mut` to do an assignment, not for moving out - unsafe { - // Note: It's safe to move the `fut2` here because we haven't yet polled it - PinMut::get_mut_unchecked(self.reborrow()).state = State::Second(fut2); + self.try_chain().poll(cx, |result, async_op| { + match result { + Ok(item) => TryChainAction::Output(Ok(item)), + Err(err) => TryChainAction::Future(async_op(err)), } - } + }) } } diff --git a/futures-util/src/try_future/recover.rs b/futures-util/src/try_future/recover.rs deleted file mode 100644 index 27095e15a2..0000000000 --- a/futures-util/src/try_future/recover.rs +++ /dev/null @@ -1,35 +0,0 @@ -use core::mem::PinMut; - -use futures_core::{Future, Poll, TryFuture}; -use futures_core::task; - -/// Future for the `recover` combinator, handling errors by converting them into -/// an `Item`, compatible with any error type of the caller's choosing. -#[must_use = "futures do nothing unless polled"] -#[derive(Debug)] -pub struct Recover { - inner: A, - f: Option, -} - -pub fn new(future: A, f: F) -> Recover { - Recover { inner: future, f: Some(f) } -} - -impl Future for Recover - where A: TryFuture, - F: FnOnce(A::Error) -> A::Item, -{ - type Output = A::Item; - - fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { - unsafe { pinned_field!(self.reborrow(), inner) }.try_poll(cx) - .map(|res| res.unwrap_or_else(|e| { - let f = unsafe { - PinMut::get_mut_unchecked(self).f.take() - .expect("Polled future::Recover after completion") - }; - f(e) - })) - } -} diff --git a/futures-util/src/try_future/try_chain.rs b/futures-util/src/try_future/try_chain.rs new file mode 100644 index 0000000000..44c6855b02 --- /dev/null +++ b/futures-util/src/try_future/try_chain.rs @@ -0,0 +1,66 @@ +use core::mem::PinMut; +use futures_core::future::TryFuture; +use futures_core::task::{self, Poll}; + +#[must_use = "futures do nothing unless polled"] +#[derive(Debug)] +crate enum TryChain { + First(Fut1, Option), + Second(Fut2), + Empty, +} + +crate enum TryChainAction + where Fut2: TryFuture, +{ + Future(Fut2), + Output(Result), +} + +impl TryChain + where Fut1: TryFuture, + Fut2: TryFuture, +{ + crate fn new(fut1: Fut1, data: Data) -> TryChain { + TryChain::First(fut1, Some(data)) + } + + crate fn poll( + self: PinMut, + cx: &mut task::Context, + op: F, + ) -> Poll> + where F: FnOnce(Result, Data) -> TryChainAction, + { + let mut op = Some(op); + + // Safe to call `get_mut_unchecked` because we won't move the futures. + let this = unsafe { PinMut::get_mut_unchecked(self) }; + + loop { + let (output, data) = match this { + TryChain::First(fut1, data) => { + // Poll the first future + match unsafe { PinMut::new_unchecked(fut1) }.try_poll(cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(output) => (output, data.take().unwrap()), + } + } + TryChain::Second(fut2) => { + // Poll the second future + return unsafe { PinMut::new_unchecked(fut2) }.try_poll(cx) + } + TryChain::Empty => { + panic!("future must not be polled after it returned `Poll::Ready`"); + } + }; + + *this = TryChain::Empty; // Drop fut1 + let op = op.take().unwrap(); + match op(output, data) { + TryChainAction::Future(fut2) => *this = TryChain::Second(fut2), + TryChainAction::Output(output) => return Poll::Ready(output), + } + } + } +} diff --git a/futures-util/src/try_future/unwrap_or_else.rs b/futures-util/src/try_future/unwrap_or_else.rs new file mode 100644 index 0000000000..2d0356cc3e --- /dev/null +++ b/futures-util/src/try_future/unwrap_or_else.rs @@ -0,0 +1,44 @@ +use core::marker::Unpin; +use core::mem::PinMut; +use futures_core::future::{Future, TryFuture}; +use futures_core::task::{self, Poll}; + +/// Future for the `unwrap_or_else` combinator. It unwraps the result, returning +/// the content of the `Ok` as `Output` or if the value is an `Err` then it +/// calls `op` with its value. +#[derive(Debug)] +#[must_use = "futures do nothing unless polled"] +pub struct UnwrapOrElse { + future: Fut, + op: Option, +} + +impl UnwrapOrElse { + unsafe_pinned!(future -> Fut); + unsafe_unpinned!(op -> Option); + + /// Creates a new UnwrapOrElse. + pub(super) fn new(future: Fut, op: F) -> UnwrapOrElse { + UnwrapOrElse { future, op: Some(op) } + } +} + +impl Unpin for UnwrapOrElse {} + +impl Future for UnwrapOrElse + where Fut: TryFuture, + F: FnOnce(Fut::Error) -> Fut::Item, +{ + type Output = Fut::Item; + + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + match self.future().try_poll(cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(result) => { + let op = self.op().take() + .expect("UnwrapOrElse already returned `Poll::Ready` before"); + Poll::Ready(result.unwrap_or_else(op)) + } + } + } +} diff --git a/futures-util/src/try_stream/mod.rs b/futures-util/src/try_stream/mod.rs index 8c495d068c..4a64a13f69 100644 --- a/futures-util/src/try_stream/mod.rs +++ b/futures-util/src/try_stream/mod.rs @@ -3,7 +3,7 @@ //! This module contains a number of functions for working with `Streams`s //! that return `Result`s, allowing for short-circuiting computations. -use futures_core::TryStream; +use futures_core::stream::TryStream; if_std! { // combinators diff --git a/futures-util/src/try_stream/try_collect.rs b/futures-util/src/try_stream/try_collect.rs index 70db39f86c..3b6b27cc21 100644 --- a/futures-util/src/try_stream/try_collect.rs +++ b/futures-util/src/try_stream/try_collect.rs @@ -1,9 +1,9 @@ -use std::prelude::v1::*; +use futures_core::future::Future; +use futures_core::stream::TryStream; +use futures_core::task::{self, Poll}; use std::marker::Unpin; use std::mem::{self, PinMut}; - -use futures_core::{Future, Poll, TryStream}; -use futures_core::task; +use std::prelude::v1::*; /// A future which attempts to collect all of the values of a stream. /// diff --git a/futures/src/lib.rs b/futures/src/lib.rs index 7d8c1ae8b4..b0318057fb 100644 --- a/futures/src/lib.rs +++ b/futures/src/lib.rs @@ -1,148 +1,57 @@ //! Abstractions for asynchronous programming. //! -//! This crate provides a number of core abstractions for writing asynchronous code: +//! This crate provides a number of core abstractions for writing asynchronous +//! code: //! -//! - [Futures](::Future) (sometimes called promises), which represent a single -//! asychronous computation that may result in a final value or an error. +//! - [Futures](crate::future::Future) single eventual values produced by +//! asychronous computations. Some programming languages (e.g. JavaScript) +//! call this concept "promises". +//! - [Streams](crate::stream::Stream) represent a series of values +//! produced asynchronously. +//! - [Sinks](crate::sink::Sink) provide support for asynchronous writing of +//! data. +//! - [Executors](crate::executor) are responsible for running asynchronous +//! tasks. //! -//! - [Streams](::Stream), which represent a series of values or errors produced asynchronously. -//! -//! - [Sinks](::Sink), which support asynchronous writing of data. -//! -//! - [Executors](::executor), which are responsible for running asynchronous tasks. -//! -//! The crate also contains abstractions for [asynchronous I/O](::io) and -//! [cross-task communication](::channel). +//! The crate also contains abstractions for [asynchronous I/O](crate::io) and +//! [cross-task communication](crate::channel). //! //! Underlying all of this is the *task system*, which is a form of lightweight //! threading. Large asynchronous computations are built up using futures, //! streams and sinks, and then spawned as independent tasks that are run to //! completion, but *do not block* the thread running them. -#![feature(futures_api)] +#![feature(pin, arbitrary_self_types, futures_api)] #![no_std] -#![deny(missing_docs, missing_debug_implementations, warnings)] +#![deny(missing_docs, missing_debug_implementations)] #![deny(bare_trait_objects)] -#![doc(html_root_url = "https://docs.rs/futures/0.2.0")] +#![doc(html_root_url = "https://docs.rs/futures-preview/0.3.0-alpha")] #![cfg_attr(feature = "nightly", feature(cfg_target_has_atomic))] #![cfg_attr(feature = "nightly", feature(use_extern_macros))] -// extern crate futures_async_runtime; -extern crate futures_core; -extern crate futures_channel; -extern crate futures_executor; -extern crate futures_io; -extern crate futures_sink; -extern crate futures_util; - -#[doc(hidden)] -pub use futures_core::core_reexport; -pub use futures_core::future::Future; -pub use futures_util::future::FutureExt; -pub use futures_core::stream::Stream; -pub use futures_util::stream::StreamExt; -pub use futures_sink::Sink; -pub use futures_util::sink::SinkExt; - -// Macros redefined here because macro re-exports are unstable. - -/// A macro for extracting the successful type of a `Poll>`. -/// -/// This macro bakes in propagation of `Pending` and `Err` signals by returning early. -#[macro_export] -macro_rules! try_ready { - ($x:expr) => { - match $x { - $crate::Poll::Ready(Ok(x)) => x, - $crate::Poll::Ready(Err(e)) => return $crate::Poll::Ready(Err(e.into())), - $crate::Poll::Pending => return $crate::Poll::Pending, - } - } -} - -/// A macro for extracting `Poll` from `Poll>`. -/// -/// This macro bakes in propagation of `Err` signals by returning early. -/// This macro bakes in propagation of `Pending` and `Err` signals by returning early. -#[macro_export] -macro_rules! try_poll { - ($x:expr) => { - match $x { - $crate::Poll::Ready(Ok(x)) => $crate::Poll::Ready(x), - $crate::Poll::Ready(Err(e)) => return $crate::Poll::Ready(Err(e.into())), - $crate::Poll::Pending => $crate::Poll::Pending, - } - } -} - -/// A macro for extracting the successful type of a `Poll`. -/// -/// This macro bakes in propagation of `Pending` signals by returning early. -#[macro_export] -macro_rules! ready { - ($e:expr) => (match $e { - $crate::Poll::Ready(t) => t, - $crate::Poll::Pending => return $crate::Poll::Pending, - }) -} +#[doc(hidden)] pub use futures_core::core_reexport; +#[doc(hidden)] pub use futures_core::future::Future; +#[doc(hidden)] pub use futures_core::future::CoreFutureExt; +#[doc(hidden)] pub use futures_core::future::TryFuture; +#[doc(hidden)] pub use futures_util::future::FutureExt; +#[doc(hidden)] pub use futures_util::try_future::TryFutureExt; -#[macro_export] -macro_rules! pinned_deref { - ($e:expr) => ( - ::core::mem::PinMut::new_unchecked( - &mut **::core::mem::PinMut::get_mut_unchecked($e.reborrow()) - ) - ) -} +#[doc(hidden)] pub use futures_core::stream::Stream; +#[doc(hidden)] pub use futures_core::stream::TryStream; +#[doc(hidden)] pub use futures_util::stream::StreamExt; +#[doc(hidden)] pub use futures_util::try_stream::TryStreamExt; -#[macro_export] -macro_rules! pinned_field { - ($e:expr, $f:tt) => ( - ::core::mem::PinMut::new_unchecked( - &mut ::core::mem::PinMut::get_mut_unchecked($e.reborrow()).$f - ) - ) -} +#[doc(hidden)] pub use futures_sink::Sink; +#[doc(hidden)] pub use futures_util::sink::SinkExt; -#[macro_export] -macro_rules! unsafe_pinned { - ($f:tt -> $t:ty) => ( - fn $f<'a>(self: &'a mut PinMut) -> PinMut<'a, $t> { - unsafe { - pinned_field!(self, $f) - } - } - ) -} +#[doc(hidden)] pub use futures_core::task::Poll; -#[macro_export] -macro_rules! unsafe_unpinned { - ($f:tt -> $t:ty) => ( - fn $f<'a>(self: &'a mut PinMut) -> &'a mut $t { - unsafe { - &mut ::core::mem::PinMut::get_mut_unchecked(self.reborrow()).$f - } - } - ) -} - -#[macro_export] -macro_rules! pin_mut { - ($($x:ident),*) => { $( - // Move the value to ensure that it is owned - let mut $x = $x; - // Shadow the original binding so that it can't be directly accessed - // ever again. - #[allow(unused_mut)] - let mut $x = unsafe { ::std::mem::PinMut::new_unchecked(&mut $x) }; - )* } -} - -pub use futures_core::Poll; +#[macro_use] +mod macros; #[cfg(feature = "std")] pub mod channel { @@ -151,12 +60,11 @@ pub mod channel { //! Like threads, concurrent tasks sometimes need to communicate with each //! other. This module contains two basic abstractions for doing so: //! - //! - [oneshot](::channel::oneshot), a way of sending a single value from - //! one task to another. - //! - //! - [mpsc](::channel::mpsc), a multi-producer, single-consumer channel for - //! sending values between tasks, analogous to the similarly-named structure - //! in the standard library. + //! - [oneshot](crate::channel::oneshot), a way of sending a single value + //! from one task to another. + //! - [mpsc](crate::channel::mpsc), a multi-producer, single-consumer + //! channel for sending values between tasks, analogous to the + //! similarly-named structure in the standard library. pub use futures_channel::{oneshot, mpsc}; } @@ -172,13 +80,13 @@ pub mod executor { //! # Using a thread pool (M:N task scheduling) //! //! Most of the time tasks should be executed on a [thread - //! pool](::executor::ThreadPool). A small set of worker threads can handle - //! a very large set of spawned tasks (which are much lighter weight than - //! threads). + //! pool](crate::executor::ThreadPool). A small set of worker threads can + //! handle a very large set of spawned tasks (which are much lighter weight + //! than threads). //! //! The simplest way to use a thread pool is to - //! [`run`](::executor::ThreadPool::run) an initial task on it, which can - //! then spawn further tasks back onto the pool to complete its work: + //! [`run`](crate::executor::ThreadPool::run) an initial task on it, which + //! can then spawn further tasks back onto the pool to complete its work: //! //! ``` //! # #![feature(pin, arbitrary_self_types, futures_api)] @@ -190,20 +98,19 @@ pub mod executor { //! ThreadPool::new().expect("Failed to create threadpool").run(my_app); //! ``` //! - //! The call to [`run`](::executor::ThreadPool::run) will block the current - //! thread until the future defined by `my_app` completes, and will return - //! the result of that future. + //! The call to [`run`](crate::executor::ThreadPool::run) will block the + //! current thread until the future defined by `my_app` completes, and will + //! return the result of that future. //! //! # Spawning additional tasks //! //! There are two ways to spawn a task: //! //! - Spawn onto a "default" execuctor by calling the top-level - //! [`spawn`](::executor::spawn) function or [pulling the executor from the - //! task context](::task::Context::executor). - //! + //! [`spawn`](crate::executor::spawn) function or [pulling the executor + //! from the task context](crate::task::Context::executor). //! - Spawn onto a specific executor by calling its - //! [`spawn`](::executor::Executor::spawn) method directly. + //! [`spawn_obj`](crate::executor::Executor::spawn_obj) method directly. //! //! Every task always has an associated default executor, which is usually //! the executor on which the task is running. @@ -212,18 +119,17 @@ pub mod executor { //! //! In addition to thread pools, it's possible to run a task (and the tasks //! it spawns) entirely within a single thread via the - //! [`LocalPool`](::executor::LocalPool) executor. Aside from cutting down - //! on synchronization costs, this executor also makes it possible to + //! [`LocalPool`](crate::executor::LocalPool) executor. Aside from cutting + //! down on synchronization costs, this executor also makes it possible to //! execute non-`Send` tasks, via - //! [`spawn_local`](::executor::LocalExecutor::spawn_local). The `LocalPool` - //! is best suited for running I/O-bound tasks that do relatively little - //! work between I/O operations. + //! [`spawn_local_obj`](crate::executor::LocalExecutor::spawn_local_obj). + //! The `LocalPool` is best suited for running I/O-bound tasks that do + //! relatively little work between I/O operations. //! //! There is also a convenience function, - //! [`block_on`](::executor::block_on), for simply running a future to + //! [`block_on`](crate::executor::block_on), for simply running a future to //! completion on the current thread, while routing any spawned tasks //! to a global thread pool. - // TODO: add docs (or link to apr) for implementing an executor pub use futures_executor::{ BlockingStream, @@ -240,32 +146,43 @@ pub mod future { //! //! This module contains: //! - //! - The [`Future` trait](::Future). - //! - //! - The [`FutureExt`](::future::FutureExt) trait, which provides adapters - //! for chaining and composing futures. - //! - //! - Top-level future combinators like [`lazy`](::future::lazy) which - //! creates a future from a closure that defines its return value, and - //! [`result`](::future::result), which constructs a future with an - //! immediate defined value. + //! - The [`Future` trait](crate::future::Future). + //! - The [`FutureExt`](crate::future::FutureExt) trait, which provides + //! adapters for chaining and composing futures. + //! - Top-level future combinators like [`lazy`](crate::future::lazy) which + //! creates a future from a closure that defines its return value, and + //! [`ready`](crate::future::ready), which constructs a future with an + //! immediate defined value. pub use futures_core::future::{ - FutureOption, Future, TryFuture, ReadyFuture, ready, - FutureObj, LocalFutureObj + FutureOption, Future, TryFuture, + FutureObj, LocalFutureObj, UnsafeFutureObj, + ready, ReadyFuture, }; + pub use futures_util::future::{ - Empty, Flatten, FlattenStream, Fuse, Inspect, IntoStream, Lazy, - Then, Either, PollFn, Map, FutureExt, empty, lazy, poll_fn, - // AndThen, ErrInto, Join, Join3, Join4, Join5, LoopFn, - // MapErr, OrElse, Select, Loop, loop_fn, + empty, Empty, + lazy, Lazy, + maybe_done, MaybeDone, + poll_fn, PollFn, + + FutureExt, + FlattenStream, Flatten, Fuse, Inspect, IntoStream, Join, Join3, Join4, + Join5, Map, Then, WithExecutor, }; #[cfg(feature = "std")] pub use futures_util::future::{ - CatchUnwind - // JoinAll, SelectAll, SelectOk, Shared, SharedError, SharedItem, - // join_all, select_all, select_ok + // For FutureExt: + CatchUnwind, Shared + + // ToDo: JoinAll, SelectAll, SelectOk, join_all, select_all, select_ok + }; + + pub use futures_util::try_future::{ + TryFutureExt, + AndThen, ErrInto, FlattenSink, IntoFuture, MapErr, MapOk, OrElse, + UnwrapOrElse, }; } @@ -274,17 +191,18 @@ pub mod io { //! Asynchronous I/O. //! //! This module is the asynchronous version of `std::io`. It defines two - //! traits, [`AsyncRead`](::io::AsyncRead) and - //! [`AsyncWrite`](::io::AsyncWrite), which mirror the `Read` and `Write` - //! traits of the standard library. However, these traits integrate with the - //! asynchronous task system, so that if an I/O object isn't ready for - //! reading (or writing), the thread is not blocked, and instead the current - //! task is queued to be woken when I/O is ready. - //! - //! In addition, the [`AsyncReadExt`](::io::AsyncReadExt) and - //! [`AsyncWriteExt`](::io::AsyncWriteExt) extension traits offer a variety - //! of useful combinators for operating with asynchronous I/O objects, - //! including ways to work with them using futures, streams and sinks. + //! traits, [`AsyncRead`](crate::io::AsyncRead) and + //! [`AsyncWrite`](crate::io::AsyncWrite), which mirror the `Read` and + //! `Write` traits of the standard library. However, these traits integrate + //! with the asynchronous task system, so that if an I/O object isn't ready + //! for reading (or writing), the thread is not blocked, and instead the + //! current task is queued to be woken when I/O is ready. + //! + //! In addition, the [`AsyncReadExt`](crate::io::AsyncReadExt) and + //! [`AsyncWriteExt`](crate::io::AsyncWriteExt) extension traits offer a + //! variety of useful combinators for operating with asynchronous I/O + //! objects, including ways to work with them using futures, streams and + //! sinks. pub use futures_io::{ Error, Initializer, IoVec, ErrorKind, AsyncRead, AsyncWrite, Result @@ -299,8 +217,8 @@ pub mod prelude { //! A "prelude" for crates using the `futures` crate. //! //! This prelude is similar to the standard library's prelude in that you'll - //! almost always want to import its entire contents, but unlike the standard - //! library's prelude you'll have to do so manually: + //! almost always want to import its entire contents, but unlike the + //! standard library's prelude you'll have to do so manually: //! //! ``` //! use futures::prelude::*; @@ -308,9 +226,9 @@ pub mod prelude { //! //! The prelude may grow over time as additional items see ubiquitous use. - pub use futures_core::{ - Future, CoreFutureExt, TryFuture, Stream, TryStream, Poll, task - }; + pub use futures_core::future::{Future, CoreFutureExt, TryFuture}; + pub use futures_core::stream::{Stream, TryStream}; + pub use futures_core::task::{self, Poll}; pub use futures_sink::Sink; @@ -340,10 +258,10 @@ pub mod sink { //! //! This module contains: //! - //! - The [`Sink` trait](::Sink), which allows you to asynchronously write data. - //! - //! - The [`SinkExt`](::sink::SinkExt) trait, which provides adapters - //! for chaining and composing sinks. + //! - The [`Sink` trait](crate::sink::Sink), which allows you to + //! asynchronously write data. + //! - The [`SinkExt`](crate::sink::SinkExt) trait, which provides adapters + //! for chaining and composing sinks. pub use futures_sink::Sink; @@ -362,41 +280,47 @@ pub mod stream { //! //! This module contains: //! - //! - The [`Stream` trait](::Stream), for objects that can asynchronously - //! produce a sequence of values. - //! - //! - The [`StreamExt`](::StreamExt) trait, which provides adapters - //! for chaining and composing streams. - //! - //! - Top-level stream contructors like [`iter_ok`](::stream::iter_ok) which - //! creates a stream from an iterator, and - //! [`futures_unordered`](::stream::futures_unordered()), which constructs a - //! stream from a collection of futures. + //! - The [`Stream` trait](crate::stream::Stream), for objects that can + //! asynchronously produce a sequence of values. + //! - The [`StreamExt`](crate::stream::StreamExt) trait, which provides + //! adapters for chaining and composing streams. + //! - Top-level stream contructors like [`iter_ok`](crate::stream::iter) + //! which creates a stream from an iterator, and + //! [`futures_unordered`](crate::stream::futures_unordered()), which + //! constructs a stream from a collection of futures. pub use futures_core::stream::{Stream, TryStream}; pub use futures_util::stream::{ - Chain, Concat, Empty, Filter, FilterMap, Flatten, Fold, ForEach, Fuse, - Inspect, Map, Once, Peekable, PollFn, Repeat, Select, Skip, SkipWhile, - StreamFuture, Take, TakeWhile, Then, Unfold, Zip, StreamExt, empty, - once, poll_fn, repeat, unfold, iter, - Forward, - }; + iter, Iter, + repeat, Repeat, + empty, Empty, + once, Once, + poll_fn, PollFn, + unfold, Unfold, - pub use futures_util::try_stream::{ - TryCollect, - // AndThen, ErrInto, InspectErr, MapErr, OrElse + StreamExt, + Chain, Concat, Filter, FilterMap, Flatten, Fold, Forward, ForEach, Fuse, + StreamFuture, Inspect, Map, Next, Peekable, Select, Skip, SkipWhile, + Take, TakeWhile, Then, Zip }; #[cfg(feature = "std")] pub use futures_util::stream::{ - CatchUnwind, Chunks, Collect, - BufferUnordered, Buffered, - FuturesUnordered, FuturesOrdered, - futures_unordered, futures_ordered, - // , select_all, - // ReuniteError, SelectAll, SplitSink, - // SplitStream, + futures_ordered, FuturesOrdered, + futures_unordered, FuturesUnordered, + + // For StreamExt: + BufferUnordered, Buffered, CatchUnwind, Chunks, Collect, SplitStream, + SplitSink, ReuniteError, + + // ToDo: select_all, SelectAll, + }; + + #[cfg(feature = "std")] + pub use futures_util::try_stream::{ + TryCollect, + // ToDo: AndThen, ErrInto, InspectErr, MapErr, OrElse }; } @@ -405,28 +329,23 @@ pub mod task { //! //! This module contains: //! - //! - [`Context`](::task::Context), which provides contextual data present - //! for every task, including a handle for waking up the task. - //! - //! - [`Waker`](::task::Waker), a handle for waking up a task. - //! - //! - [`LocalKey`](::task::LocalKey), a key for task-local data; you should - //! use the [`task_local` macro](../macro.task_local.html) to set up such keys. + //! - [`Context`](crate::task::Context), which provides contextual data + //! present for every task, including a handle for waking up the task. + //! - [`Waker`](crate::task::Waker), a handle for waking up a task. //! //! Tasks themselves are generally created by spawning a future onto [an - //! executor](::executor). However, you can manually construct a task by - //! creating your own `Context` instance, and polling a future with it. + //! executor](crate::executor). However, you can manually construct a task + //! by creating your own `Context` instance, and polling a future with it. //! //! The remaining types and traits in the module are used for implementing //! executors or dealing with synchronization issues around task wakeup. pub use futures_core::task::{ - Context, Waker, LocalWaker, UnsafeWake, UnsafeTask, - Executor, - TaskObj, LocalTaskObj, + Context, Poll, Executor, Waker, LocalWaker, UnsafeWake, SpawnErrorKind, SpawnObjError, SpawnLocalObjError, }; + #[cfg(feature = "std")] #[cfg_attr(feature = "nightly", cfg(target_has_atomic = "ptr"))] pub use futures_core::task::AtomicWaker; diff --git a/futures/src/macros/mod.rs b/futures/src/macros/mod.rs new file mode 100644 index 0000000000..83a87133ae --- /dev/null +++ b/futures/src/macros/mod.rs @@ -0,0 +1,7 @@ +// Macros redefined here because macro re-exports are unstable. + +#[macro_use] +mod pin; + +#[macro_use] +mod poll; diff --git a/futures/src/macros/pin.rs b/futures/src/macros/pin.rs new file mode 100644 index 0000000000..ec27f1683c --- /dev/null +++ b/futures/src/macros/pin.rs @@ -0,0 +1,96 @@ +/// A pinned projection of a struct field. +/// +/// ``` +/// #![feature(pin, arbitrary_self_types)] +/// # #[macro_use] extern crate futures; +/// # struct Bar; +/// # use core::mem::PinMut; +/// struct Foo { +/// field: Bar, +/// } +/// +/// impl Foo { +/// unsafe_pinned!(field -> Bar); +/// +/// fn baz(mut self: PinMut) { +/// let _: PinMut = self.field(); // Pinned reference to the field +/// } +/// } +/// ``` +#[macro_export] +macro_rules! unsafe_pinned { + ($f:tt -> $t:ty) => ( + fn $f<'a>( + self: &'a mut $crate::core_reexport::mem::PinMut + ) -> $crate::core_reexport::mem::PinMut<'a, $t> { + unsafe { + $crate::core_reexport::mem::PinMut::map_unchecked( + self.reborrow(), |x| &mut x.$f + ) + } + } + ) +} + +/// An unpinned projection of a struct field. +/// +/// This macro is unsafe because it returns a normal non-pin reference to +/// the struct field. It is up to the programmer to ensure that the contained +/// value is either not moved at all or only moved when it's safe. +/// +/// ``` +/// #![feature(pin, arbitrary_self_types)] +/// # #[macro_use] extern crate futures; +/// # use core::mem::PinMut; +/// # struct Bar; +/// struct Foo { +/// field: Bar, +/// } +/// +/// impl Foo { +/// unsafe_unpinned!(field -> Bar); +/// +/// fn baz(mut self: PinMut) { +/// let _: &mut Bar = self.field(); // Normal reference to the field +/// } +/// } +/// ``` +#[macro_export] +macro_rules! unsafe_unpinned { + ($f:tt -> $t:ty) => ( + fn $f<'a>( + self: &'a mut $crate::core_reexport::mem::PinMut + ) -> &'a mut $t { + unsafe { + &mut $crate::core_reexport::mem::PinMut::get_mut_unchecked( + self.reborrow() + ).$f + } + } + ) +} + +/// Pins a value on the stack. +/// +/// ``` +/// #![feature(pin, arbitrary_self_types)] +/// # #[macro_use] extern crate futures; +/// # use core::mem::PinMut; +/// # struct Foo {} +/// let foo = Foo { /* ... */ }; +/// pin_mut!(foo); +/// let _: PinMut = foo; +/// ``` +#[macro_export] +macro_rules! pin_mut { + ($($x:ident),*) => { $( + // Move the value to ensure that it is owned + let mut $x = $x; + // Shadow the original binding so that it can't be directly accessed + // ever again. + #[allow(unused_mut)] + let mut $x = unsafe { + $crate::core_reexport::mem::PinMut::new_unchecked(&mut $x) + }; + )* } +} diff --git a/futures/src/macros/poll.rs b/futures/src/macros/poll.rs new file mode 100644 index 0000000000..fcc97cd4de --- /dev/null +++ b/futures/src/macros/poll.rs @@ -0,0 +1,40 @@ +/// Extracts the successful type of a `Poll>`. +/// +/// This macro bakes in propagation of `Pending` and `Err` signals by returning early. +#[macro_export] +macro_rules! try_ready { + ($x:expr) => { + match $x { + $crate::task::Poll::Ready(Ok(x)) => x, + $crate::task::Poll::Ready(Err(e)) => return $crate::task::Poll::Ready(Err(e.into())), + $crate::task::Poll::Pending => return $crate::task::Poll::Pending, + } + } +} + + +/// Extracts `Poll` from `Poll>`. +/// +/// This macro bakes in propagation of `Err` signals by returning early. +/// This macro bakes in propagation of `Pending` and `Err` signals by returning early. +#[macro_export] +macro_rules! try_poll { + ($x:expr) => { + match $x { + $crate::task::Poll::Ready(Ok(x)) => $crate::task::Poll::Ready(x), + $crate::task::Poll::Ready(Err(e)) => return $crate::task::Poll::Ready(Err(e.into())), + $crate::task::Poll::Pending => $crate::task::Poll::Pending, + } + } +} + +/// Extracts the successful type of a `Poll`. +/// +/// This macro bakes in propagation of `Pending` signals by returning early. +#[macro_export] +macro_rules! ready { + ($e:expr) => (match $e { + $crate::task::Poll::Ready(t) => t, + $crate::task::Poll::Pending => return $crate::task::Poll::Pending, + }) +} diff --git a/futures/tests/basic_combinators.rs b/futures/tests/basic_combinators.rs new file mode 100644 index 0000000000..bfd8cf23e6 --- /dev/null +++ b/futures/tests/basic_combinators.rs @@ -0,0 +1,106 @@ +#![feature(pin, arbitrary_self_types, futures_api)] + +#[macro_use] +extern crate futures; + +use futures::future; +use futures::prelude::*; +use std::sync::mpsc; + +mod support; +use self::support::RunInBackgroundExt; + +#[test] +fn basic_future_combinators() { + let (tx1, rx) = mpsc::channel(); + let tx2 = tx1.clone(); + let tx3 = tx1.clone(); + + let fut = future::ready(1) + .then(move |x| { + tx1.send(x).unwrap(); // Send 1 + tx1.send(2).unwrap(); // Send 2 + future::ready(3) + }).map(move |x| { + tx2.send(x).unwrap(); // Send 3 + tx2.send(4).unwrap(); // Send 4 + 5 + }).map(move |x| { + tx3.send(x).unwrap(); // Send 5 + }); + + assert!(rx.try_recv().is_err()); // Not started yet + fut.run_in_background(); // Start it + for i in 1..=5 { assert_eq!(rx.recv(), Ok(i)); } // Check it + assert!(rx.recv().is_err()); // Should be done +} + +#[test] +fn basic_try_future_combinators() { + let (tx1, rx) = mpsc::channel(); + let tx2 = tx1.clone(); + let tx3 = tx1.clone(); + let tx4 = tx1.clone(); + let tx5 = tx1.clone(); + let tx6 = tx1.clone(); + let tx7 = tx1.clone(); + let tx8 = tx1.clone(); + let tx9 = tx1.clone(); + let tx10 = tx1.clone(); + + let fut = future::ready(Ok(1)) + .and_then(move |x: i32| { + tx1.send(x).unwrap(); // Send 1 + tx1.send(2).unwrap(); // Send 2 + future::ready(Ok(3)) + }) + .or_else(move |x: i32| { + tx2.send(x).unwrap(); // Should not run + tx2.send(-1).unwrap(); + future::ready(Ok(-1)) + }) + .map_ok(move |x: i32| { + tx3.send(x).unwrap(); // Send 3 + tx3.send(4).unwrap(); // Send 4 + 5 + }) + .map_err(move |x: i32| { + tx4.send(x).unwrap(); // Should not run + tx4.send(-1).unwrap(); + -1 + }) + .map(move |x: Result| { + tx5.send(x.unwrap()).unwrap(); // Send 5 + tx5.send(6).unwrap(); // Send 6 + Err(7) // Now return errors! + }) + .and_then(move |x: i32| { + tx6.send(x).unwrap(); // Should not run + tx6.send(-1).unwrap(); + future::ready(Err(-1)) + }) + .or_else(move |x: i32| { + tx7.send(x).unwrap(); // Send 7 + tx7.send(8).unwrap(); // Send 8 + future::ready(Err(9)) + }) + .map_ok(move |x: i32| { + tx8.send(x).unwrap(); // Should not run + tx8.send(-1).unwrap(); + -1 + }) + .map_err(move |x: i32| { + tx9.send(x).unwrap(); // Send 9 + tx9.send(10).unwrap(); // Send 10 + 11 + }) + .map(move |x: Result| { + tx10.send(x.err().unwrap()).unwrap(); // Send 11 + tx10.send(12).unwrap(); // Send 12 + }); + + assert!(rx.try_recv().is_err()); // Not started yet + fut.run_in_background(); // Start it + for i in 1..=12 { assert_eq!(rx.recv(), Ok(i)); } // Check it + assert!(rx.recv().is_err()); // Should be done +} diff --git a/futures/tests/eager_drop.rs b/futures/tests/eager_drop.rs new file mode 100644 index 0000000000..68b54aef91 --- /dev/null +++ b/futures/tests/eager_drop.rs @@ -0,0 +1,123 @@ +#![feature(pin, arbitrary_self_types, futures_api)] + +#[macro_use] +extern crate futures; + +use futures::channel::oneshot; +use futures::future; +use futures::prelude::*; +use std::mem::PinMut; +use std::sync::mpsc; + +mod support; +use self::support::RunInBackgroundExt; + +#[test] +fn map_ok() { + // The closure given to `map_ok` should have been dropped by the time `map` + // runs. + let (tx1, rx1) = mpsc::channel::<()>(); + let (tx2, rx2) = mpsc::channel::<()>(); + + future::ready::>(Err(1)) + .map_ok(move |_| { let _tx1 = tx1; panic!("should not run"); }) + .map(move |_| { + assert!(rx1.recv().is_err()); + tx2.send(()).unwrap() + }) + .run_in_background(); + + rx2.recv().unwrap(); +} + +#[test] +fn map_err() { + // The closure given to `map_err` should have been dropped by the time `map` + // runs. + let (tx1, rx1) = mpsc::channel::<()>(); + let (tx2, rx2) = mpsc::channel::<()>(); + + future::ready::>(Ok(1)) + .map_err(move |_| { let _tx1 = tx1; panic!("should not run"); }) + .map(move |_| { + assert!(rx1.recv().is_err()); + tx2.send(()).unwrap() + }) + .run_in_background(); + + rx2.recv().unwrap(); +} + +struct FutureData { + _data: T, + future: F, +} + +impl FutureData { + unsafe_pinned!(future -> F); +} + +impl Future for FutureData { + type Output = F::Output; + + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + self.future().poll(cx) + } +} + +#[test] +fn then_drops_eagerly() { + let (tx0, rx0) = oneshot::channel::<()>(); + let (tx1, rx1) = mpsc::channel::<()>(); + let (tx2, rx2) = mpsc::channel::<()>(); + + FutureData { _data: tx1, future: rx0.unwrap_or_else(|_| { panic!() }) } + .then(move |_| { + assert!(rx1.recv().is_err()); // tx1 should have been dropped + tx2.send(()).unwrap(); + future::ready(()) + }) + .run_in_background(); + + assert_eq!(Err(mpsc::TryRecvError::Empty), rx2.try_recv()); + tx0.send(()).unwrap(); + rx2.recv().unwrap(); +} + +#[test] +fn and_then_drops_eagerly() { + let (tx0, rx0) = oneshot::channel::>(); + let (tx1, rx1) = mpsc::channel::<()>(); + let (tx2, rx2) = mpsc::channel::<()>(); + + FutureData { _data: tx1, future: rx0.unwrap_or_else(|_| { panic!() }) } + .and_then(move |_| { + assert!(rx1.recv().is_err()); // tx1 should have been dropped + tx2.send(()).unwrap(); + future::ready(Ok(())) + }) + .run_in_background(); + + assert_eq!(Err(mpsc::TryRecvError::Empty), rx2.try_recv()); + tx0.send(Ok(())).unwrap(); + rx2.recv().unwrap(); +} + +#[test] +fn or_else_drops_eagerly() { + let (tx0, rx0) = oneshot::channel::>(); + let (tx1, rx1) = mpsc::channel::<()>(); + let (tx2, rx2) = mpsc::channel::<()>(); + + FutureData { _data: tx1, future: rx0.unwrap_or_else(|_| { panic!() }) } + .or_else(move |_| { + assert!(rx1.recv().is_err()); // tx1 should have been dropped + tx2.send(()).unwrap(); + future::ready::>(Ok(())) + }) + .run_in_background(); + + assert_eq!(Err(mpsc::TryRecvError::Empty), rx2.try_recv()); + tx0.send(Err(())).unwrap(); + rx2.recv().unwrap(); +} diff --git a/futures/tests/fuse.rs b/futures/tests/fuse.rs index 8e9b11ede6..5fa9e89efe 100644 --- a/futures/tests/fuse.rs +++ b/futures/tests/fuse.rs @@ -1,16 +1,17 @@ -#![feature(pin, futures_api)] +#![feature(pin, arbitrary_self_types, futures_api)] +#[macro_use] extern crate futures; +use futures::future; use futures::prelude::*; -use futures::future::ready; mod support; #[test] fn fuse() { - let mut future = ready::(2).fuse(); - support::panic_waker_cx(|cx| { + let mut future = future::ready::(2).fuse(); + support::with_panic_waker_context(|cx| { assert!(future.poll_unpin(cx).is_ready()); assert!(future.poll_unpin(cx).is_pending()); }) diff --git a/futures/tests/futures_ordered.rs b/futures/tests/futures_ordered.rs index 1c2355f461..e0f3b824f9 100644 --- a/futures/tests/futures_ordered.rs +++ b/futures/tests/futures_ordered.rs @@ -1,12 +1,11 @@ -#![feature(pin, futures_api)] +#![feature(pin, arbitrary_self_types, futures_api)] +#[macro_use] extern crate futures; -// use std::any::Any; - use futures::channel::oneshot; use futures::executor::{block_on, block_on_stream}; -use futures::future::{ready, FutureObj}; +use futures::future::{self, FutureObj}; use futures::prelude::*; use futures::stream::{futures_ordered, FuturesOrdered}; @@ -21,7 +20,7 @@ fn works_1() { let mut stream = futures_ordered(vec![a_rx, b_rx, c_rx]); b_tx.send(99).unwrap(); - support::noop_waker_cx(|cx| { + support::with_noop_waker_context(|cx| { assert!(stream.poll_next_unpin(cx).is_pending()); }); @@ -46,7 +45,7 @@ fn works_2() { FutureObj::new(Box::new(b_rx.join(c_rx).map(|(a, b)| Ok(a? + b?)))), ]); - support::noop_waker_cx(|cx| { + support::with_noop_waker_context(|cx| { a_tx.send(33).unwrap(); b_tx.send(33).unwrap(); assert!(stream.poll_next_unpin(cx).is_ready()); @@ -59,9 +58,9 @@ fn works_2() { #[test] fn from_iterator() { let stream = vec![ - ready::(1), - ready::(2), - ready::(3) + future::ready::(1), + future::ready::(2), + future::ready::(3) ].into_iter().collect::>(); assert_eq!(stream.len(), 3); assert_eq!(block_on(stream.collect::>()), vec![1,2,3]); @@ -79,7 +78,7 @@ fn queue_never_unblocked() { Box::new(b_rx.select(c_rx).then(|res| Ok(Box::new(res) as Box))) as _, ]); - support::noop_waker_cx(|cx| { + support::with_noop_waker_context(f)(|cx| { for _ in 0..10 { assert!(stream.poll_next(cx).unwrap().is_pending()); } diff --git a/futures/tests/futures_unordered.rs b/futures/tests/futures_unordered.rs index 3c607cdf55..7de102dc76 100644 --- a/futures/tests/futures_unordered.rs +++ b/futures/tests/futures_unordered.rs @@ -1,14 +1,14 @@ -#![feature(pin, futures_api)] +#![feature(pin, arbitrary_self_types, futures_api)] +#[macro_use] extern crate futures; -use std::boxed::Box; - use futures::channel::oneshot; use futures::executor::{block_on, block_on_stream}; -use futures::future::{ready, empty, FutureObj}; +use futures::future::{self, FutureObj}; use futures::stream::{futures_unordered, FuturesUnordered}; use futures::prelude::*; +use std::boxed::Box; mod support; @@ -43,7 +43,7 @@ fn works_2() { a_tx.send(9).unwrap(); b_tx.send(10).unwrap(); - support::noop_waker_cx(|cx| { + support::with_noop_waker_context(|cx| { assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(Some(Ok(9)))); c_tx.send(20).unwrap(); assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(Some(Ok(30)))); @@ -54,9 +54,9 @@ fn works_2() { #[test] fn from_iterator() { let stream = vec![ - ready::(1), - ready::(2), - ready::(3) + future::ready::(1), + future::ready::(2), + future::ready::(3) ].into_iter().collect::>(); assert_eq!(stream.len(), 3); assert_eq!(block_on(stream.collect::>()), vec![1,2,3]); @@ -74,7 +74,7 @@ fn finished_future() { //FutureObj::new(Box::new(b_rx.select(c_rx))), ]); - support::noop_waker_cx(|cx| { + support::with_noop_waker_context(f)(|cx| { for _ in 0..10 { assert!(stream.poll_next_unpin(cx).is_pending()); } @@ -114,9 +114,9 @@ fn iter_mut_cancel() { #[test] fn iter_mut_len() { let mut stream = futures_unordered(vec![ - empty::<()>(), - empty::<()>(), - empty::<()>() + future::empty::<()>(), + future::empty::<()>(), + future::empty::<()>() ]); let mut iter_mut = stream.iter_mut(); diff --git a/futures/tests/inspect.rs b/futures/tests/inspect.rs new file mode 100644 index 0000000000..231c83d081 --- /dev/null +++ b/futures/tests/inspect.rs @@ -0,0 +1,17 @@ +#![feature(pin, arbitrary_self_types, futures_api)] + +use futures::executor::block_on; +use futures::future; +use futures::prelude::*; + +#[test] +fn smoke() { + let mut counter = 0; + + { + let work = future::ready::(40).inspect(|val| { counter += *val; }); + assert_eq!(block_on(work), 40); + } + + assert_eq!(counter, 40); +} diff --git a/futures/tests/oneshot.rs b/futures/tests/oneshot.rs new file mode 100644 index 0000000000..9477d9fa9a --- /dev/null +++ b/futures/tests/oneshot.rs @@ -0,0 +1,73 @@ +#![feature(pin, arbitrary_self_types, futures_api)] + +#[macro_use] +extern crate futures; + +use futures::channel::oneshot; +use futures::prelude::*; +use std::sync::mpsc; +use std::thread; + +mod support; +use self::support::RunInBackgroundExt; + +#[test] +fn oneshot_send1() { + let (tx1, rx1) = oneshot::channel::(); + let (tx2, rx2) = mpsc::channel(); + + let t = thread::spawn(|| tx1.send(1).unwrap()); + rx1.map_ok(move |x| tx2.send(x)).run_in_background(); + assert_eq!(1, rx2.recv().unwrap()); + t.join().unwrap(); +} + +#[test] +fn oneshot_send2() { + let (tx1, rx1) = oneshot::channel::(); + let (tx2, rx2) = mpsc::channel(); + + thread::spawn(|| tx1.send(1).unwrap()).join().unwrap(); + rx1.map_ok(move |x| tx2.send(x).unwrap()).run_in_background(); + assert_eq!(1, rx2.recv().unwrap()); +} + +#[test] +fn oneshot_send3() { + let (tx1, rx1) = oneshot::channel::(); + let (tx2, rx2) = mpsc::channel(); + + rx1.map_ok(move |x| tx2.send(x).unwrap()).run_in_background(); + thread::spawn(|| tx1.send(1).unwrap()).join().unwrap(); + assert_eq!(1, rx2.recv().unwrap()); +} + +#[test] +fn oneshot_drop_tx1() { + let (tx1, rx1) = oneshot::channel::(); + let (tx2, rx2) = mpsc::channel(); + + drop(tx1); + rx1.map(move |result| tx2.send(result).unwrap()).run_in_background(); + + assert_eq!(Err(oneshot::Canceled), rx2.recv().unwrap()); +} + +#[test] +fn oneshot_drop_tx2() { + let (tx1, rx1) = oneshot::channel::(); + let (tx2, rx2) = mpsc::channel(); + + let t = thread::spawn(|| drop(tx1)); + rx1.map(move |result| tx2.send(result).unwrap()).run_in_background(); + t.join().unwrap(); + + assert_eq!(Err(oneshot::Canceled), rx2.recv().unwrap()); +} + +#[test] +fn oneshot_drop_rx() { + let (tx, rx) = oneshot::channel::(); + drop(rx); + assert_eq!(Err(2), tx.send(2)); +} diff --git a/futures/tests/recurse.rs b/futures/tests/recurse.rs new file mode 100644 index 0000000000..ac16fe70ce --- /dev/null +++ b/futures/tests/recurse.rs @@ -0,0 +1,25 @@ +#![feature(pin, arbitrary_self_types, futures_api)] + +use futures::executor::block_on; +use futures::future::{self, FutureObj}; +use futures::prelude::*; +use std::sync::mpsc; +use std::thread; + +#[test] +fn lots() { + fn do_it(input: (i32, i32)) -> FutureObj<'static, i32> { + let (n, x) = input; + if n == 0 { + FutureObj::new(Box::new(future::ready(x))) + } else { + FutureObj::new(Box::new(future::ready((n - 1, x + n)).then(do_it))) + } + } + + let (tx, rx) = mpsc::channel(); + thread::spawn(|| { + block_on(do_it((1_000, 0)).map(move |x| tx.send(x).unwrap())) + }); + assert_eq!(500_500, rx.recv().unwrap()); +} diff --git a/futures/tests/shared.rs b/futures/tests/shared.rs index 3dd88e7488..869f85cb65 100644 --- a/futures/tests/shared.rs +++ b/futures/tests/shared.rs @@ -1,19 +1,12 @@ -#![feature(pin, futures_api)] - -extern crate futures; - -mod support; - -use std::boxed::PinBox; -use std::cell::RefCell; -use std::rc::Rc; -use std::thread; +#![feature(pin, arbitrary_self_types, futures_api)] use futures::channel::oneshot; use futures::executor::{block_on, LocalPool}; -use futures::future::{lazy, LocalFutureObj}; +use futures::future::{self, LocalFutureObj}; use futures::prelude::*; -use futures::task::LocalTaskObj; +use std::cell::RefCell; +use std::rc::Rc; +use std::thread; fn send_shared_oneshot_and_wait_on_multiple_threads(threads_number: u32) { let (tx, rx) = oneshot::channel::(); @@ -82,7 +75,7 @@ fn drop_in_poll() { let slot1 = Rc::new(RefCell::new(None)); let slot2 = slot1.clone(); - let future1 = lazy(move |_| { + let future1 = future::lazy(move |_| { slot2.replace(None); // Drop future 1 }).shared(); @@ -116,7 +109,7 @@ fn peek() { } // Once the Shared has been polled, the value is peekable on the clone. - exec.spawn_local_obj(LocalTaskObj::new(PinBox::new(f1.map(|_| ())))).unwrap(); + exec.spawn_local_obj(LocalFutureObj::new(Box::new(f1.map(|_| ())))).unwrap(); local_pool.run(exec); for _ in 0..2 { assert_eq!(*f2.peek().unwrap(), Ok(42)); diff --git a/futures/tests/split.rs b/futures/tests/split.rs new file mode 100644 index 0000000000..da38c43320 --- /dev/null +++ b/futures/tests/split.rs @@ -0,0 +1,80 @@ +#![feature(pin, arbitrary_self_types, futures_api)] + +#[macro_use] +extern crate futures; + +use futures::executor::block_on; +use futures::prelude::*; +use futures::stream; +use std::mem::PinMut; + +struct Join { + stream: T, + sink: U +} + +impl Join { + unsafe_pinned!(stream -> T); + unsafe_pinned!(sink -> U); +} + +impl Stream for Join { + type Item = T::Item; + + fn poll_next( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { + self.stream().poll_next(cx) + } +} + +impl Sink for Join { + type SinkItem = U::SinkItem; + type SinkError = U::SinkError; + + fn poll_ready( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { + self.sink().poll_ready(cx) + } + + fn start_send( + mut self: PinMut, + item: Self::SinkItem, + ) -> Result<(), Self::SinkError> { + self.sink().start_send(item) + } + + fn poll_flush( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { + self.sink().poll_flush(cx) + } + + fn poll_close( + mut self: PinMut, + cx: &mut task::Context, + ) -> Poll> { + self.sink().poll_close(cx) + } +} + +#[test] +fn test_split() { + let mut dest: Vec = Vec::new(); + { + let join = Join { + stream: stream::iter(vec![10, 20, 30]), + sink: &mut dest + }; + + let (sink, stream) = join.split(); + let join = sink.reunite(stream).expect("test_split: reunite error"); + let (mut sink, mut stream) = join.split(); + block_on(sink.send_all(&mut stream)).unwrap(); + } + assert_eq!(dest, vec![10, 20, 30]); +} diff --git a/futures/tests/support/assert.rs b/futures/tests/support/assert.rs new file mode 100644 index 0000000000..d17aef9a79 --- /dev/null +++ b/futures/tests/support/assert.rs @@ -0,0 +1,37 @@ +use futures::prelude::*; +use std::fmt; +use std::mem::PinMut; + +use super::{with_noop_waker_context, with_panic_waker_context}; + +pub fn assert_stream_pending(stream: PinMut) { + with_noop_waker_context(|cx| { + match stream.poll_next(cx) { + Poll::Ready(_) => panic!("stream is not pending"), + Poll::Pending => {}, + } + }) +} + +pub fn assert_stream_next(stream: PinMut, item: S::Item) + where S::Item: Eq + fmt::Debug +{ + with_panic_waker_context(|cx| { + match stream.poll_next(cx) { + Poll::Ready(Some(x)) => assert_eq!(x, item), + Poll::Ready(None) => panic!("stream is at its end"), + Poll::Pending => panic!("stream wasn't ready"), + } + }) +} + +pub fn assert_stream_done(stream: PinMut) +{ + with_panic_waker_context(|cx| { + match stream.poll_next(cx) { + Poll::Ready(Some(_)) => panic!("stream had more elements"), + Poll::Ready(None) => {}, + Poll::Pending => panic!("stream wasn't ready"), + } + }) +} diff --git a/futures/tests/support/delayed.rs b/futures/tests/support/delayed.rs new file mode 100644 index 0000000000..a6cebd1f51 --- /dev/null +++ b/futures/tests/support/delayed.rs @@ -0,0 +1,33 @@ +use futures::prelude::*; +use std::mem::PinMut; + +pub struct Delayed { + future: F, + polled_before: bool +} + +impl Delayed { + unsafe_pinned!(future -> F); + unsafe_unpinned!(polled_before -> bool); +} + +impl Future for Delayed { + type Output = F::Output; + + fn poll(mut self: PinMut, cx: &mut task::Context) -> Poll { + if *self.polled_before() { + self.future().poll(cx) + } else { + *self.polled_before() = true; + cx.waker().wake(); + Poll::Pending + } + } +} + +/// Introduces one `Poll::Pending` before polling the given future +pub fn delayed(future: F) -> Delayed + where F: Future, +{ + Delayed { future, polled_before: false } +} diff --git a/futures/tests/support/mod.rs b/futures/tests/support/mod.rs index 5926731e20..db52c344d4 100644 --- a/futures/tests/support/mod.rs +++ b/futures/tests/support/mod.rs @@ -1,16 +1,21 @@ #![allow(dead_code)] -// use std::fmt::{self, Debug}; -use std::sync::Arc; -// use std::thread; +pub mod assert; + +mod delayed; +pub use self::delayed::{delayed, Delayed}; + +mod run_in_background; +pub use self::run_in_background::RunInBackgroundExt; + +mod noop_waker_context; +pub use self::noop_waker_context::with_noop_waker_context; + +mod panic_executor; + +mod panic_waker_context; +pub use self::panic_waker_context::with_panic_waker_context; -// use futures::executor::{block_on, Executor, SpawnError}; -// use futures::{Future, IntoFuture, Async, Poll, Never}; -// use futures::future::FutureResult; -// use futures::stream::Stream; -use futures::task::{ - self, Executor, TaskObj, SpawnObjError, Wake -}; // pub fn f_ok(a: i32) -> FutureResult { Ok(a).into_future() } // pub fn f_err(a: u32) -> FutureResult { Err(a).into_future() } @@ -34,132 +39,3 @@ use futures::task::{ // }) // } -// pub fn sassert_done(s: &mut S) { -// panic_waker_cx(|cx| { -// match s.poll_next(cx) { -// Ok(Async::Ready(None)) => {} -// Ok(Async::Ready(Some(_))) => panic!("stream had more elements"), -// Ok(Async::Pending) => panic!("stream wasn't ready"), -// Err(_) => panic!("stream had an error"), -// } -// }) -// } - -// pub fn sassert_empty(s: &mut S) { -// noop_waker_cx(|cx| { -// match s.poll_next(cx) { -// Ok(Async::Ready(None)) => panic!("stream is at its end"), -// Ok(Async::Ready(Some(_))) => panic!("stream had more elements"), -// Ok(Async::Pending) => {} -// Err(_) => panic!("stream had an error"), -// } -// }) -// } - -// pub fn sassert_next(s: &mut S, item: S::Item) -// where S::Item: Eq + fmt::Debug -// { -// panic_waker_cx(|cx| { -// match s.poll_next(cx) { -// Ok(Async::Ready(None)) => panic!("stream is at its end"), -// Ok(Async::Ready(Some(e))) => assert_eq!(e, item), -// Ok(Async::Pending) => panic!("stream wasn't ready"), -// Err(_) => panic!("stream had an error"), -// } -// }) -// } - -// pub fn sassert_err(s: &mut S, err: S::Error) -// where S::Error: Eq + fmt::Debug -// { -// panic_waker_cx(|cx| { -// match s.poll_next(cx) { -// Ok(Async::Ready(None)) => panic!("stream is at its end"), -// Ok(Async::Ready(Some(_))) => panic!("stream had more elements"), -// Ok(Async::Pending) => panic!("stream wasn't ready"), -// Err(e) => assert_eq!(e, err), -// } -// }) -// } - -pub fn panic_waker_cx(f: F) -> R - where F: FnOnce(&mut task::Context) -> R -{ - struct Foo; - - impl Wake for Foo { - fn wake(_: &Arc) { - panic!("should not be woken"); - } - } - - let panic_waker = unsafe { task::local_waker(Arc::new(Foo)) }; - let exec = &mut PanicExec; - - let cx = &mut task::Context::new(&panic_waker, exec); - f(cx) -} - -pub fn noop_waker_cx(f: F) -> R - where F: FnOnce(&mut task::Context) -> R -{ - struct Noop; - - impl Wake for Noop { - fn wake(_: &Arc) {} - } - - let noop_waker = unsafe { task::local_waker(Arc::new(Noop)) }; - let exec = &mut PanicExec; - - let cx = &mut task::Context::new(&noop_waker, exec); - f(cx) -} - -pub struct PanicExec; -impl Executor for PanicExec { - fn spawn_obj( - &mut self, _: TaskObj - ) -> Result<(), SpawnObjError> { - panic!("should not spawn") - } -} - -// pub trait ForgetExt { -// fn forget(self); -// } - -// impl ForgetExt for F -// where F: Future + Sized + Send + 'static, -// F::Item: Send, -// F::Error: Send -// { -// fn forget(self) { -// thread::spawn(|| block_on(self)); -// } -// } - -// pub struct DelayFuture(F,bool); - -// impl Future for DelayFuture { -// type Item = F::Item; -// type Error = F::Error; - -// fn poll(&mut self, cx: &mut task::Context) -> Poll { -// if self.1 { -// self.0.poll(cx) -// } else { -// self.1 = true; -// cx.waker().wake(); -// Ok(Async::Pending) -// } -// } -// } - -// /// Introduces one `Ok(Async::Pending)` before polling the given future -// pub fn delay_future(f: F) -> DelayFuture -// where F: IntoFuture, -// { -// DelayFuture(f.into_future(), false) -// } - diff --git a/futures/tests/support/noop_waker_context.rs b/futures/tests/support/noop_waker_context.rs new file mode 100644 index 0000000000..673e5962a5 --- /dev/null +++ b/futures/tests/support/noop_waker_context.rs @@ -0,0 +1,22 @@ +use std::sync::Arc; + +use futures::task::Wake; +use futures::prelude::*; + +use super::panic_executor::PanicExecutor; + +pub fn with_noop_waker_context(f: F) -> R + where F: FnOnce(&mut task::Context) -> R +{ + struct NoopWake; + + impl Wake for NoopWake { + fn wake(_: &Arc) {} + } + + let noop_waker = unsafe { task::local_waker(Arc::new(NoopWake)) }; + let exec = &mut PanicExecutor; + + let cx = &mut task::Context::new(&noop_waker, exec); + f(cx) +} diff --git a/futures/tests/support/panic_executor.rs b/futures/tests/support/panic_executor.rs new file mode 100644 index 0000000000..1d4b503929 --- /dev/null +++ b/futures/tests/support/panic_executor.rs @@ -0,0 +1,10 @@ +use futures::future::FutureObj; +use futures::task::{Executor, SpawnObjError}; + +pub struct PanicExecutor; + +impl Executor for PanicExecutor { + fn spawn_obj(&mut self, _: FutureObj<'static, ()>) -> Result<(), SpawnObjError> { + panic!("should not spawn") + } +} diff --git a/futures/tests/support/panic_waker_context.rs b/futures/tests/support/panic_waker_context.rs new file mode 100644 index 0000000000..3792a5ccad --- /dev/null +++ b/futures/tests/support/panic_waker_context.rs @@ -0,0 +1,24 @@ +use std::sync::Arc; + +use futures::task::Wake; +use futures::prelude::*; + +use super::panic_executor::PanicExecutor; + +pub fn with_panic_waker_context(f: F) -> R + where F: FnOnce(&mut task::Context) -> R +{ + struct PanicWake; + + impl Wake for PanicWake { + fn wake(_: &Arc) { + panic!("should not be woken"); + } + } + + let panic_waker = unsafe { task::local_waker(Arc::new(PanicWake)) }; + let exec = &mut PanicExecutor; + + let cx = &mut task::Context::new(&panic_waker, exec); + f(cx) +} diff --git a/futures/tests/support/run_in_background.rs b/futures/tests/support/run_in_background.rs new file mode 100644 index 0000000000..f2ed9b7b59 --- /dev/null +++ b/futures/tests/support/run_in_background.rs @@ -0,0 +1,16 @@ +use futures::prelude::*; +use futures::executor::block_on; +use std::thread; + +pub trait RunInBackgroundExt { + fn run_in_background(self); +} + +impl RunInBackgroundExt for F + where F: Future + Sized + Send + 'static, + F::Output: Send, +{ + fn run_in_background(self) { + thread::spawn(|| block_on(self)); + } +} diff --git a/futures/tests/unfold.rs b/futures/tests/unfold.rs new file mode 100644 index 0000000000..1a3f8e3787 --- /dev/null +++ b/futures/tests/unfold.rs @@ -0,0 +1,40 @@ +#![feature(pin, arbitrary_self_types, futures_api)] + +#[macro_use] +extern crate futures; + +use futures::future; +use futures::stream; + +mod support; +use self::support::assert::*; + +#[test] +fn unfold1() { + let stream = stream::unfold(0, |state| { + if state <= 2 { + support::delayed(future::ready(Some((state * 2, state + 1)))) + } else { + support::delayed(future::ready(None)) + } + }); + + pin_mut!(stream); + + // Creates the future with the closure + // Not ready (delayed future) + assert_stream_pending(stream.reborrow()); + // Future is ready, yields the item + assert_stream_next(stream.reborrow(), 0); + + // Repeat + assert_stream_pending(stream.reborrow()); + assert_stream_next(stream.reborrow(), 2); + + assert_stream_pending(stream.reborrow()); + assert_stream_next(stream.reborrow(), 4); + + // No more items + assert_stream_pending(stream.reborrow()); + assert_stream_done(stream.reborrow()); +} diff --git a/futures/tests_disabled/eager_drop.rs b/futures/tests_disabled/eager_drop.rs deleted file mode 100644 index 4602a96d5a..0000000000 --- a/futures/tests_disabled/eager_drop.rs +++ /dev/null @@ -1,65 +0,0 @@ -extern crate futures; - -use std::sync::mpsc::channel; - -use futures::prelude::*; -use futures::channel::oneshot; -use futures::future::{err, ok}; - -mod support; -use support::*; - -#[test] -fn map() { - // Whatever runs after a `map` should have dropped the closure by that - // point. - let (tx, rx) = channel::<()>(); - let (tx2, rx2) = channel(); - err::(1).map(move |a| { drop(tx); a }).map_err(move |_| { - assert!(rx.recv().is_err()); - tx2.send(()).unwrap() - }).forget(); - rx2.recv().unwrap(); -} - -#[test] -fn map_err() { - // Whatever runs after a `map_err` should have dropped the closure by that - // point. - let (tx, rx) = channel::<()>(); - let (tx2, rx2) = channel(); - ok::(1).map_err(move |a| { drop(tx); a }).map(move |_| { - assert!(rx.recv().is_err()); - tx2.send(()).unwrap() - }).forget(); - rx2.recv().unwrap(); -} - -struct FutureData { - _data: T, - future: F, -} - -impl Future for FutureData { - type Item = F::Item; - type Error = F::Error; - - fn poll(&mut self, cx: &mut task::Context) -> Poll { - self.future.poll(cx) - } -} - -#[test] -fn and_then_drops_eagerly() { - let (c, p) = oneshot::channel::<()>(); - let (tx, rx) = channel::<()>(); - let (tx2, rx2) = channel(); - FutureData { _data: tx, future: p }.and_then(move |_| { - assert!(rx.recv().is_err()); - tx2.send(()).unwrap(); - ok(1) - }).forget(); - assert!(rx2.try_recv().is_err()); - c.send(()).unwrap(); - rx2.recv().unwrap(); -} diff --git a/futures/tests_disabled/eventual.rs b/futures/tests_disabled/eventual.rs index c4f89de2bc..e962e93c7c 100644 --- a/futures/tests_disabled/eventual.rs +++ b/futures/tests_disabled/eventual.rs @@ -10,165 +10,6 @@ use futures::prelude::*; use futures::future::{ok, err}; use futures::channel::oneshot; -#[test] -fn and_then1() { - let (tx, rx) = mpsc::channel(); - - let tx2 = tx.clone(); - let p1 = ok::<_, i32>("a").then(move |t| { tx2.send("first").unwrap(); t }); - let tx2 = tx.clone(); - let p2 = ok("b").then(move |t| { tx2.send("second").unwrap(); t }); - let f = p1.and_then(|_| p2); - - assert!(rx.try_recv().is_err()); - f.map(move |s| tx.send(s).unwrap()).forget(); - assert_eq!(rx.recv(), Ok("first")); - assert_eq!(rx.recv(), Ok("second")); - assert_eq!(rx.recv(), Ok("b")); - assert!(rx.recv().is_err()); -} - -#[test] -fn and_then2() { - let (tx, rx) = mpsc::channel(); - - let tx2 = tx.clone(); - let p1 = err::(2).then(move |t| { tx2.send("first").unwrap(); t }); - let tx2 = tx.clone(); - let p2 = ok("b").then(move |t| { tx2.send("second").unwrap(); t }); - let f = p1.and_then(|_| p2); - - assert!(rx.try_recv().is_err()); - f.map_err(|_| drop(tx)).forget(); - assert_eq!(rx.recv(), Ok("first")); - assert!(rx.recv().is_err()); -} - -#[test] -fn oneshot1() { - let (c, p) = oneshot::channel::(); - let t = thread::spawn(|| c.send(1).unwrap()); - - let (tx, rx) = mpsc::channel(); - p.map(move |e| tx.send(e).unwrap()).forget(); - assert_eq!(rx.recv(), Ok(1)); - t.join().unwrap(); -} - -#[test] -fn oneshot2() { - let (c, p) = oneshot::channel::(); - let t = thread::spawn(|| c.send(1).unwrap()); - t.join().unwrap(); - - let (tx, rx) = mpsc::channel(); - p.map(move |e| tx.send(e).unwrap()).forget(); - assert_eq!(rx.recv(), Ok(1)); -} - -#[test] -fn oneshot3() { - let (c, p) = oneshot::channel::(); - let (tx, rx) = mpsc::channel(); - p.map(move |e| tx.send(e).unwrap()).forget(); - - let t = thread::spawn(|| c.send(1).unwrap()); - t.join().unwrap(); - - assert_eq!(rx.recv(), Ok(1)); -} - -#[test] -fn oneshot4() { - let (c, p) = oneshot::channel::(); - drop(c); - - let (tx, rx) = mpsc::channel(); - p.map(move |e| tx.send(e).unwrap()).forget(); - assert!(rx.recv().is_err()); -} - -#[test] -fn oneshot5() { - let (c, p) = oneshot::channel::(); - let t = thread::spawn(|| drop(c)); - let (tx, rx) = mpsc::channel(); - p.map(move |t| tx.send(t).unwrap()).forget(); - t.join().unwrap(); - assert!(rx.recv().is_err()); -} - -#[test] -fn oneshot6() { - let (c, p) = oneshot::channel::(); - drop(p); - c.send(2).unwrap_err(); -} - -#[test] -fn cancel1() { - let (c, p) = oneshot::channel::(); - drop(c); - p.map(|_| panic!()).forget(); -} - -#[test] -fn map_err1() { - ok::(1).map_err(|_| panic!()).forget(); -} - -#[test] -fn map_err2() { - let (tx, rx) = mpsc::channel(); - err::(1).map_err(move |v| tx.send(v).unwrap()).forget(); - assert_eq!(rx.recv(), Ok(1)); - assert!(rx.recv().is_err()); -} - -#[test] -fn map_err3() { - let (c, p) = oneshot::channel::(); - p.map_err(|_| {}).forget(); - drop(c); -} - -#[test] -fn or_else1() { - let (c1, p1) = oneshot::channel::(); - let (c2, p2) = oneshot::channel::(); - - let (tx, rx) = mpsc::channel(); - let tx2 = tx.clone(); - let p1 = p1.map_err(move |i| { tx2.send(2).unwrap(); i }); - let tx2 = tx.clone(); - let p2 = p2.map(move |i| { tx2.send(i).unwrap(); i }); - - assert!(rx.try_recv().is_err()); - drop(c1); - c2.send(3).unwrap(); - p1.or_else(|_| p2).map(move |v| tx.send(v).unwrap()).forget(); - - assert_eq!(rx.recv(), Ok(2)); - assert_eq!(rx.recv(), Ok(3)); - assert_eq!(rx.recv(), Ok(3)); - assert!(rx.recv().is_err()); -} - -#[test] -fn or_else2() { - let (c1, p1) = oneshot::channel::(); - - let (tx, rx) = mpsc::channel(); - - p1.or_else(move |_| { - tx.send(()).unwrap(); - ok::(1) - }).forget(); - - c1.send(2).unwrap(); - assert!(rx.recv().is_err()); -} - #[test] fn join1() { let (tx, rx) = mpsc::channel(); diff --git a/futures/tests_disabled/inspect.rs b/futures/tests_disabled/inspect.rs deleted file mode 100644 index 05978e0d4d..0000000000 --- a/futures/tests_disabled/inspect.rs +++ /dev/null @@ -1,24 +0,0 @@ -extern crate futures; - -use futures::executor::block_on; -use futures::prelude::*; -use futures::future::{ok, err}; - -#[test] -fn smoke() { - let mut counter = 0; - - { - let work = ok::(40).inspect(|val| { counter += *val; }); - assert_eq!(block_on(work), Ok(40)); - } - - assert_eq!(counter, 40); - - { - let work = err::(4).inspect(|val| { counter += *val; }); - assert_eq!(block_on(work), Err(4)); - } - - assert_eq!(counter, 40); -} diff --git a/futures/tests_disabled/recurse.rs b/futures/tests_disabled/recurse.rs deleted file mode 100644 index 5a95a0e1a8..0000000000 --- a/futures/tests_disabled/recurse.rs +++ /dev/null @@ -1,24 +0,0 @@ -extern crate futures; - -use std::sync::mpsc::channel; - -use futures::executor::block_on; -use futures::future::ok; -use futures::prelude::*; - -#[test] -fn lots() { - fn doit(n: usize) -> Box + Send> { - if n == 0 { - Box::new(ok(())) - } else { - Box::new(ok(n - 1).and_then(doit)) - } - } - - let (tx, rx) = channel(); - ::std::thread::spawn(|| { - block_on(doit(1_000).map(move |_| tx.send(()).unwrap())) - }); - rx.recv().unwrap(); -} diff --git a/futures/tests_disabled/split.rs b/futures/tests_disabled/split.rs deleted file mode 100644 index 8d347015e1..0000000000 --- a/futures/tests_disabled/split.rs +++ /dev/null @@ -1,50 +0,0 @@ -extern crate futures; - -use futures::executor::block_on; -use futures::prelude::*; -use futures::stream::iter_ok; - -struct Join(T, U); - -impl Stream for Join { - type Item = T::Item; - type Error = T::Error; - - fn poll_next(&mut self, cx: &mut task::Context) -> Poll, T::Error> { - self.0.poll_next(cx) - } -} - -impl Sink for Join { - type SinkItem = U::SinkItem; - type SinkError = U::SinkError; - - fn poll_ready(&mut self, cx: &mut task::Context) -> Poll<(), Self::SinkError> { - self.1.poll_ready(cx) - } - - fn start_send(&mut self, item: Self::SinkItem) -> Result<(), Self::SinkError> { - self.1.start_send(item) - } - - fn poll_flush(&mut self, cx: &mut task::Context) -> Poll<(), Self::SinkError> { - self.1.poll_flush(cx) - } - - fn poll_close(&mut self, cx: &mut task::Context) -> Poll<(), Self::SinkError> { - self.1.poll_close(cx) - } -} - -#[test] -fn test_split() { - let mut dest = Vec::new(); - { - let j = Join(iter_ok(vec![10, 20, 30]), &mut dest); - let (sink, stream) = j.split(); - let j = sink.reunite(stream).expect("test_split: reunite error"); - let (sink, stream) = j.split(); - block_on(sink.send_all(stream)).unwrap(); - } - assert_eq!(dest, vec![10, 20, 30]); -} diff --git a/futures/tests_disabled/unfold.rs b/futures/tests_disabled/unfold.rs deleted file mode 100644 index 1669a18aa5..0000000000 --- a/futures/tests_disabled/unfold.rs +++ /dev/null @@ -1,52 +0,0 @@ -extern crate futures; - -mod support; - -use futures::stream; - -use support::*; - -#[test] -fn unfold1() { - let mut stream = stream::unfold(0, |state| { - if state <= 2 { - let res: Result<_,()> = Ok((state * 2, state + 1)); - Some(delay_future(res)) - } else { - None - } - }); - // Creates the future with the closure - // Not ready (delayed future) - sassert_empty(&mut stream); - // future is ready, yields the item - sassert_next(&mut stream, 0); - - // Repeat - sassert_empty(&mut stream); - sassert_next(&mut stream, 2); - - sassert_empty(&mut stream); - sassert_next(&mut stream, 4); - - // no more items - sassert_done(&mut stream); -} - -#[test] -fn unfold_err1() { - let mut stream = stream::unfold(0, |state| { - if state <= 2 { - Some(Ok((state * 2, state + 1))) - } else { - Some(Err(-1)) - } - }); - sassert_next(&mut stream, 0); - sassert_next(&mut stream, 2); - sassert_next(&mut stream, 4); - sassert_err(&mut stream, -1); - - // An error was generated by the stream, it will then finish - sassert_done(&mut stream); -}