From 3e306c21c76fa109faa6b1d2281a508a1b625987 Mon Sep 17 00:00:00 2001 From: Zura Benashvili Date: Wed, 18 Nov 2020 20:56:46 +0400 Subject: [PATCH] feat(futures-util/stream): add StreamExt::unzip (#2263) --- examples/functional/src/main.rs | 6 +- examples/imperative/src/main.rs | 4 +- futures-channel/benches/sync_mpsc.rs | 15 +- futures-channel/src/lib.rs | 5 - futures-channel/tests/channel.rs | 10 +- futures-channel/tests/mpsc-close.rs | 4 +- futures-channel/tests/mpsc.rs | 33 ++- futures-channel/tests/oneshot.rs | 4 +- futures-core/src/future.rs | 10 +- futures-core/src/lib.rs | 11 +- futures-core/src/stream.rs | 43 ++-- .../src/task/__internal/atomic_waker.rs | 19 +- futures-core/src/task/mod.rs | 2 +- futures-core/src/task/poll.rs | 11 +- futures-executor/benches/thread_notify.rs | 5 +- futures-executor/src/lib.rs | 11 +- futures-executor/src/local_pool.rs | 14 +- futures-executor/src/thread_pool.rs | 49 ++--- futures-executor/src/unpark_mutex.rs | 27 +-- futures-executor/tests/local_pool.rs | 187 ++++++++++------ futures-io/src/lib.rs | 195 +++++++++-------- futures-macro/src/join.rs | 5 +- futures-macro/src/lib.rs | 3 - futures-macro/src/select.rs | 42 ++-- futures-sink/src/lib.rs | 18 +- futures-task/src/future_obj.rs | 45 ++-- futures-task/src/lib.rs | 9 +- futures-task/src/noop_waker.rs | 6 +- futures-task/src/spawn.rs | 2 +- futures-test/src/assert.rs | 12 +- futures-test/src/assert_unmoved.rs | 10 +- futures-test/src/future/pending_once.rs | 14 +- futures-test/src/interleave_pending.rs | 5 +- futures-test/src/io/limited.rs | 15 +- futures-test/src/lib.rs | 11 +- futures-test/src/task/context.rs | 2 +- futures-test/src/task/mod.rs | 2 +- futures-test/src/task/panic_waker.rs | 12 +- futures-test/src/task/wake_counter.rs | 2 +- futures-test/src/track_closed.rs | 5 +- futures-util/benches/futures_unordered.rs | 4 +- futures-util/src/async_await/mod.rs | 4 +- futures-util/src/async_await/pending.rs | 2 +- futures-util/src/async_await/poll.rs | 2 +- futures-util/src/compat/compat01as03.rs | 75 +++---- futures-util/src/compat/compat03as01.rs | 56 ++--- futures-util/src/compat/executor.rs | 4 +- futures-util/src/compat/mod.rs | 8 +- futures-util/src/fns.rs | 61 ++++-- .../src/future/future/catch_unwind.rs | 10 +- futures-util/src/future/future/flatten.rs | 45 ++-- futures-util/src/future/future/fuse.rs | 4 +- futures-util/src/future/future/map.rs | 12 +- futures-util/src/future/future/mod.rs | 4 +- .../src/future/future/remote_handle.rs | 12 +- futures-util/src/future/future/shared.rs | 23 +- futures-util/src/future/join.rs | 4 +- futures-util/src/future/join_all.rs | 16 +- futures-util/src/future/lazy.rs | 15 +- futures-util/src/future/mod.rs | 6 +- futures-util/src/future/option.rs | 7 +- futures-util/src/future/pending.rs | 7 +- futures-util/src/future/poll_fn.rs | 5 +- futures-util/src/future/select.rs | 10 +- futures-util/src/future/select_all.rs | 19 +- futures-util/src/future/select_ok.rs | 24 +-- .../src/future/try_future/into_future.rs | 9 +- .../src/future/try_future/try_flatten.rs | 77 +++---- .../src/future/try_future/try_flatten_err.rs | 24 +-- futures-util/src/future/try_join.rs | 2 +- futures-util/src/future/try_join_all.rs | 29 +-- futures-util/src/future/try_maybe_done.rs | 18 +- futures-util/src/future/try_select.rs | 17 +- futures-util/src/io/allow_std.rs | 90 +++++--- futures-util/src/io/buf_reader.rs | 17 +- futures-util/src/io/buf_writer.rs | 8 +- futures-util/src/io/chain.rs | 6 +- futures-util/src/io/copy.rs | 8 +- futures-util/src/io/copy_buf.rs | 15 +- futures-util/src/io/cursor.rs | 22 +- futures-util/src/io/fill_buf.rs | 3 +- futures-util/src/io/flush.rs | 3 +- futures-util/src/io/into_sink.rs | 30 +-- futures-util/src/io/lines.rs | 13 +- futures-util/src/io/mod.rs | 96 +++++---- futures-util/src/io/read_exact.rs | 2 +- futures-util/src/io/read_line.rs | 9 +- futures-util/src/io/read_to_end.rs | 18 +- futures-util/src/io/read_to_string.rs | 12 +- futures-util/src/io/split.rs | 53 ++--- futures-util/src/io/take.rs | 4 +- futures-util/src/io/window.rs | 5 +- futures-util/src/io/write_all_vectored.rs | 19 +- futures-util/src/lib.rs | 83 +++---- futures-util/src/sink/buffer.rs | 47 ++-- futures-util/src/sink/close.rs | 10 +- futures-util/src/sink/drain.rs | 20 +- futures-util/src/sink/err_into.rs | 28 +-- futures-util/src/sink/fanout.rs | 52 +++-- futures-util/src/sink/flush.rs | 10 +- futures-util/src/sink/map_err.rs | 29 +-- futures-util/src/sink/mod.rs | 85 ++++---- futures-util/src/sink/send.rs | 10 +- futures-util/src/sink/send_all.rs | 35 +-- futures-util/src/sink/with.rs | 75 +++---- futures-util/src/sink/with_flat_map.rs | 35 +-- futures-util/src/stream/empty.rs | 6 +- futures-util/src/stream/iter.rs | 10 +- futures-util/src/stream/mod.rs | 6 +- futures-util/src/stream/once.rs | 4 +- futures-util/src/stream/repeat.rs | 11 +- futures-util/src/stream/select.rs | 42 ++-- .../src/stream/stream/catch_unwind.rs | 15 +- futures-util/src/stream/stream/chain.rs | 29 ++- futures-util/src/stream/stream/chunks.rs | 20 +- futures-util/src/stream/stream/collect.rs | 15 +- futures-util/src/stream/stream/concat.rs | 35 ++- futures-util/src/stream/stream/cycle.rs | 5 +- futures-util/src/stream/stream/enumerate.rs | 10 +- futures-util/src/stream/stream/filter.rs | 45 ++-- futures-util/src/stream/stream/filter_map.rs | 35 +-- futures-util/src/stream/stream/fold.rs | 28 ++- futures-util/src/stream/stream/for_each.rs | 27 ++- futures-util/src/stream/stream/forward.rs | 15 +- futures-util/src/stream/stream/fuse.rs | 5 +- futures-util/src/stream/stream/into_future.rs | 5 +- futures-util/src/stream/stream/map.rs | 24 +-- futures-util/src/stream/stream/mod.rs | 49 ++++- futures-util/src/stream/stream/next.rs | 5 +- futures-util/src/stream/stream/peek.rs | 14 +- .../src/stream/stream/ready_chunks.rs | 23 +- futures-util/src/stream/stream/scan.rs | 9 +- .../src/stream/stream/select_next_some.rs | 4 +- futures-util/src/stream/stream/skip.rs | 10 +- futures-util/src/stream/stream/skip_while.rs | 46 ++-- futures-util/src/stream/stream/take.rs | 23 +- futures-util/src/stream/stream/take_until.rs | 12 +- futures-util/src/stream/stream/take_while.rs | 39 ++-- futures-util/src/stream/stream/then.rs | 38 ++-- futures-util/src/stream/stream/unzip.rs | 61 ++++++ futures-util/src/stream/stream/zip.rs | 24 +-- .../src/stream/try_stream/and_then.rs | 31 +-- .../src/stream/try_stream/into_async_read.rs | 81 +++---- .../src/stream/try_stream/into_stream.rs | 5 +- futures-util/src/stream/try_stream/mod.rs | 9 +- futures-util/src/stream/try_stream/or_else.rs | 33 +-- .../src/stream/try_stream/try_collect.rs | 10 +- .../src/stream/try_stream/try_concat.rs | 5 +- .../src/stream/try_stream/try_filter.rs | 37 ++-- .../src/stream/try_stream/try_filter_map.rs | 26 +-- .../src/stream/try_stream/try_fold.rs | 28 ++- .../src/stream/try_stream/try_for_each.rs | 20 +- .../src/stream/try_stream/try_next.rs | 5 +- .../src/stream/try_stream/try_skip_while.rs | 44 ++-- .../src/stream/try_stream/try_take_while.rs | 8 +- .../src/stream/try_stream/try_unfold.rs | 16 +- futures-util/src/stream/unfold.rs | 39 ++-- futures-util/src/task/mod.rs | 9 +- futures/src/lib.rs | 202 +++++++----------- futures/tests/_require_features.rs | 13 +- futures/tests/abortable.rs | 4 +- futures/tests/arc_wake.rs | 9 +- futures/tests/async_await_macros.rs | 59 ++--- futures/tests/basic_combinators.rs | 14 +- futures/tests/buffer_unordered.rs | 2 +- futures/tests/compat.rs | 12 +- futures/tests/eager_drop.rs | 18 +- futures/tests/future_obj.rs | 4 +- futures/tests/future_try_flatten_stream.rs | 4 +- futures/tests/futures_ordered.rs | 37 ++-- futures/tests/futures_unordered.rs | 53 ++--- futures/tests/inspect.rs | 4 +- futures/tests/io_buf_reader.rs | 71 +++--- futures/tests/io_buf_writer.rs | 12 +- futures/tests/io_lines.rs | 14 +- futures/tests/io_read.rs | 2 +- futures/tests/io_read_exact.rs | 2 +- futures/tests/io_read_line.rs | 8 +- futures/tests/io_read_to_string.rs | 4 +- futures/tests/io_read_until.rs | 4 +- futures/tests/io_write.rs | 6 +- futures/tests/join_all.rs | 10 +- futures/tests/macro-reexport/src/lib.rs | 5 +- futures/tests/macro-tests/src/main.rs | 1 - futures/tests/macro_comma_support.rs | 24 +-- futures/tests/mutex.rs | 5 +- futures/tests/recurse.rs | 6 +- futures/tests/select_all.rs | 6 +- futures/tests/select_ok.rs | 13 +- futures/tests/shared.rs | 9 +- futures/tests/sink.rs | 49 ++--- futures/tests/split.rs | 30 +-- futures/tests/stream.rs | 17 +- futures/tests/stream_into_async_read.rs | 4 +- futures/tests/stream_select_next_some.rs | 2 +- futures/tests/try_join.rs | 3 +- futures/tests/try_join_all.rs | 4 +- futures/tests/unfold.rs | 4 +- 198 files changed, 1965 insertions(+), 2392 deletions(-) create mode 100644 futures-util/src/stream/stream/unzip.rs diff --git a/examples/functional/src/main.rs b/examples/functional/src/main.rs index 3ce65de66a..7dea28d674 100644 --- a/examples/functional/src/main.rs +++ b/examples/functional/src/main.rs @@ -30,9 +30,7 @@ fn main() { // responsible for transmission pool.spawn_ok(fut_tx_result); - let fut_values = rx - .map(|v| v * 2) - .collect(); + let fut_values = rx.map(|v| v * 2).collect(); // Use the executor provided to this async block to wait for the // future to complete. @@ -45,4 +43,4 @@ fn main() { let values: Vec = executor::block_on(fut_values); println!("Values={:?}", values); -} \ No newline at end of file +} diff --git a/examples/imperative/src/main.rs b/examples/imperative/src/main.rs index ff1afffe27..10ef7f4df5 100644 --- a/examples/imperative/src/main.rs +++ b/examples/imperative/src/main.rs @@ -34,7 +34,7 @@ fn main() { // of the stream to be available. while let Some(v) = rx.next().await { pending.push(v * 2); - }; + } pending }; @@ -45,4 +45,4 @@ fn main() { let values: Vec = executor::block_on(fut_values); println!("Values={:?}", values); -} \ No newline at end of file +} diff --git a/futures-channel/benches/sync_mpsc.rs b/futures-channel/benches/sync_mpsc.rs index e22fe60666..7c3c3d3a80 100644 --- a/futures-channel/benches/sync_mpsc.rs +++ b/futures-channel/benches/sync_mpsc.rs @@ -7,8 +7,8 @@ use { futures::{ channel::mpsc::{self, Sender, UnboundedSender}, ready, - stream::{Stream, StreamExt}, sink::Sink, + stream::{Stream, StreamExt}, task::{Context, Poll}, }, futures_test::task::noop_context, @@ -25,7 +25,6 @@ fn unbounded_1_tx(b: &mut Bencher) { // 1000 iterations to avoid measuring overhead of initialization // Result should be divided by 1000 for i in 0..1000 { - // Poll, not ready, park assert_eq!(Poll::Pending, rx.poll_next_unpin(&mut cx)); @@ -73,7 +72,6 @@ fn unbounded_uncontended(b: &mut Bencher) { }) } - /// A Stream that continuously sends incrementing number of the queue struct TestSender { tx: Sender, @@ -84,9 +82,7 @@ struct TestSender { impl Stream for TestSender { type Item = u32; - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll> - { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = &mut *self; let mut tx = Pin::new(&mut this.tx); @@ -123,12 +119,7 @@ fn bounded_100_tx(b: &mut Bencher) { // Each sender can send one item after specified capacity let (tx, mut rx) = mpsc::channel(0); - let mut tx: Vec<_> = (0..100).map(|_| { - TestSender { - tx: tx.clone(), - last: 0 - } - }).collect(); + let mut tx: Vec<_> = (0..100).map(|_| TestSender { tx: tx.clone(), last: 0 }).collect(); for i in 0..10 { for x in &mut tx { diff --git a/futures-channel/src/lib.rs b/futures-channel/src/lib.rs index 97909e6f7a..62c84803e6 100644 --- a/futures-channel/src/lib.rs +++ b/futures-channel/src/lib.rs @@ -7,21 +7,16 @@ //! library is activated, and it is activated by default. #![cfg_attr(feature = "cfg-target-has-atomic", feature(cfg_target_has_atomic))] - #![cfg_attr(not(feature = "std"), no_std)] - #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms, unreachable_pub)] // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-channel/0.3.8")] #[cfg(all(feature = "cfg-target-has-atomic", not(feature = "unstable")))] diff --git a/futures-channel/tests/channel.rs b/futures-channel/tests/channel.rs index 73dac645b1..5f01a8ef4c 100644 --- a/futures-channel/tests/channel.rs +++ b/futures-channel/tests/channel.rs @@ -1,8 +1,8 @@ use futures::channel::mpsc; use futures::executor::block_on; use futures::future::poll_fn; -use futures::stream::StreamExt; use futures::sink::SinkExt; +use futures::stream::StreamExt; use std::sync::atomic::{AtomicUsize, Ordering}; use std::thread; @@ -11,9 +11,7 @@ fn sequence() { let (tx, rx) = mpsc::channel(1); let amt = 20; - let t = thread::spawn(move || { - block_on(send_sequence(amt, tx)) - }); + let t = thread::spawn(move || block_on(send_sequence(amt, tx))); let list: Vec<_> = block_on(rx.collect()); let mut list = list.into_iter(); for i in (1..=amt).rev() { @@ -34,9 +32,7 @@ async fn send_sequence(n: u32, mut sender: mpsc::Sender) { fn drop_sender() { let (tx, mut rx) = mpsc::channel::(1); drop(tx); - let f = poll_fn(|cx| { - rx.poll_next_unpin(cx) - }); + let f = poll_fn(|cx| rx.poll_next_unpin(cx)); assert_eq!(block_on(f), None) } diff --git a/futures-channel/tests/mpsc-close.rs b/futures-channel/tests/mpsc-close.rs index 50852eb71d..bf938936c4 100644 --- a/futures-channel/tests/mpsc-close.rs +++ b/futures-channel/tests/mpsc-close.rs @@ -9,9 +9,7 @@ use std::thread; fn smoke() { let (mut sender, receiver) = mpsc::channel(1); - let t = thread::spawn(move || { - while let Ok(()) = block_on(sender.send(42)) {} - }); + let t = thread::spawn(move || while let Ok(()) = block_on(sender.send(42)) {}); // `receiver` needs to be dropped for `sender` to stop sending and therefore before the join. block_on(receiver.take(3).for_each(|_| futures::future::ready(()))); diff --git a/futures-channel/tests/mpsc.rs b/futures-channel/tests/mpsc.rs index 61c5a50246..88cdef13d6 100644 --- a/futures-channel/tests/mpsc.rs +++ b/futures-channel/tests/mpsc.rs @@ -1,13 +1,13 @@ use futures::channel::{mpsc, oneshot}; use futures::executor::{block_on, block_on_stream}; -use futures::future::{FutureExt, poll_fn}; -use futures::stream::{Stream, StreamExt}; +use futures::future::{poll_fn, FutureExt}; +use futures::pin_mut; use futures::sink::{Sink, SinkExt}; +use futures::stream::{Stream, StreamExt}; use futures::task::{Context, Poll}; -use futures::pin_mut; use futures_test::task::{new_count_waker, noop_context}; -use std::sync::{Arc, Mutex}; use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::{Arc, Mutex}; use std::thread; trait AssertSend: Send {} @@ -77,7 +77,7 @@ fn send_shared_recv() { fn send_recv_threads() { let (mut tx, rx) = mpsc::channel::(16); - let t = thread::spawn(move|| { + let t = thread::spawn(move || { block_on(tx.send(1)).unwrap(); }); @@ -204,7 +204,7 @@ fn stress_shared_unbounded() { const NTHREADS: u32 = 8; let (tx, rx) = mpsc::unbounded::(); - let t = thread::spawn(move|| { + let t = thread::spawn(move || { let result: Vec<_> = block_on(rx.collect()); assert_eq!(result.len(), (AMT * NTHREADS) as usize); for item in result { @@ -215,7 +215,7 @@ fn stress_shared_unbounded() { for _ in 0..NTHREADS { let tx = tx.clone(); - thread::spawn(move|| { + thread::spawn(move || { for _ in 0..AMT { tx.unbounded_send(1).unwrap(); } @@ -233,7 +233,7 @@ fn stress_shared_bounded_hard() { const NTHREADS: u32 = 8; let (tx, rx) = mpsc::channel::(0); - let t = thread::spawn(move|| { + let t = thread::spawn(move || { let result: Vec<_> = block_on(rx.collect()); assert_eq!(result.len(), (AMT * NTHREADS) as usize); for item in result { @@ -297,9 +297,9 @@ fn stress_receiver_multi_task_bounded_hard() { } Poll::Ready(None) => { *rx_opt = None; - break - }, - Poll::Pending => {}, + break; + } + Poll::Pending => {} } } } else { @@ -311,7 +311,6 @@ fn stress_receiver_multi_task_bounded_hard() { th.push(t); } - for i in 0..AMT { block_on(tx.send(i)).unwrap(); } @@ -328,7 +327,7 @@ fn stress_receiver_multi_task_bounded_hard() { /// after sender dropped. #[test] fn stress_drop_sender() { - fn list() -> impl Stream { + fn list() -> impl Stream { let (tx, rx) = mpsc::channel(1); thread::spawn(move || { block_on(send_one_two_three(tx)); @@ -407,9 +406,7 @@ fn stress_poll_ready() { let mut threads = Vec::new(); for _ in 0..NTHREADS { let sender = tx.clone(); - threads.push(thread::spawn(move || { - block_on(stress_poll_ready_sender(sender, AMT)) - })); + threads.push(thread::spawn(move || block_on(stress_poll_ready_sender(sender, AMT)))); } drop(tx); @@ -436,7 +433,7 @@ fn try_send_1() { for i in 0..N { loop { if tx.try_send(i).is_ok() { - break + break; } } } @@ -542,8 +539,8 @@ fn is_connected_to() { #[test] fn hash_receiver() { - use std::hash::Hasher; use std::collections::hash_map::DefaultHasher; + use std::hash::Hasher; let mut hasher_a1 = DefaultHasher::new(); let mut hasher_a2 = DefaultHasher::new(); diff --git a/futures-channel/tests/oneshot.rs b/futures-channel/tests/oneshot.rs index a22d039e40..979cd8a15a 100644 --- a/futures-channel/tests/oneshot.rs +++ b/futures-channel/tests/oneshot.rs @@ -1,6 +1,6 @@ use futures::channel::oneshot::{self, Sender}; use futures::executor::block_on; -use futures::future::{FutureExt, poll_fn}; +use futures::future::{poll_fn, FutureExt}; use futures::task::{Context, Poll}; use futures_test::task::panic_waker_ref; use std::sync::mpsc; @@ -70,7 +70,7 @@ fn close() { rx.close(); block_on(poll_fn(|cx| { match rx.poll_unpin(cx) { - Poll::Ready(Err(_)) => {}, + Poll::Ready(Err(_)) => {} _ => panic!(), }; assert!(tx.poll_canceled(cx).is_ready()); diff --git a/futures-core/src/future.rs b/futures-core/src/future.rs index 02460cc5a3..8271c46114 100644 --- a/futures-core/src/future.rs +++ b/futures-core/src/future.rs @@ -66,14 +66,12 @@ pub trait TryFuture: Future + private_try_future::Sealed { /// 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: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll>; + fn try_poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; } impl TryFuture for F - where F: ?Sized + Future> +where + F: ?Sized + Future>, { type Ok = T; type Error = E; @@ -86,8 +84,8 @@ impl TryFuture for F #[cfg(feature = "alloc")] mod if_alloc { - use alloc::boxed::Box; use super::*; + use alloc::boxed::Box; impl FusedFuture for Box { fn is_terminated(&self) -> bool { diff --git a/futures-core/src/lib.rs b/futures-core/src/lib.rs index b7e21b4587..e621869267 100644 --- a/futures-core/src/lib.rs +++ b/futures-core/src/lib.rs @@ -1,21 +1,16 @@ //! Core traits and types for asynchronous operations in Rust. #![cfg_attr(feature = "cfg-target-has-atomic", feature(cfg_target_has_atomic))] - #![cfg_attr(not(feature = "std"), no_std)] - #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms, unreachable_pub)] // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-core/0.3.8")] #[cfg(all(feature = "cfg-target-has-atomic", not(feature = "unstable")))] @@ -25,10 +20,12 @@ compile_error!("The `cfg-target-has-atomic` feature requires the `unstable` feat extern crate alloc; pub mod future; -#[doc(hidden)] pub use self::future::{Future, FusedFuture, TryFuture}; +#[doc(hidden)] +pub use self::future::{FusedFuture, Future, TryFuture}; pub mod stream; -#[doc(hidden)] pub use self::stream::{Stream, FusedStream, TryStream}; +#[doc(hidden)] +pub use self::stream::{FusedStream, Stream, TryStream}; #[macro_use] pub mod task; diff --git a/futures-core/src/stream.rs b/futures-core/src/stream.rs index 4a13e3bd7d..ad5350b795 100644 --- a/futures-core/src/stream.rs +++ b/futures-core/src/stream.rs @@ -63,10 +63,7 @@ pub trait Stream { /// calls. /// /// [`fuse`]: https://docs.rs/futures/0.3/futures/stream/trait.StreamExt.html#method.fuse - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll>; + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; /// Returns the bounds on the remaining length of the stream. /// @@ -103,10 +100,7 @@ pub trait Stream { impl Stream for &mut S { type Item = S::Item; - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { S::poll_next(Pin::new(&mut **self), cx) } @@ -122,10 +116,7 @@ where { type Item = ::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.get_mut().as_mut().poll_next(cx) } @@ -185,35 +176,36 @@ pub trait TryStream: Stream + private_try_stream::Sealed { /// This method is a stopgap for a compiler limitation that prevents us from /// directly inheriting from the `Stream` trait; in the future it won't be /// needed. - fn try_poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll>>; + fn try_poll_next( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>>; } impl TryStream for S - where S: ?Sized + Stream> +where + S: ?Sized + Stream>, { type Ok = T; type Error = E; - fn try_poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll>> - { + fn try_poll_next( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>> { self.poll_next(cx) } } #[cfg(feature = "alloc")] mod if_alloc { - use alloc::boxed::Box; use super::*; + use alloc::boxed::Box; impl Stream for Box { type Item = S::Item; - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { Pin::new(&mut **self).poll_next(cx) } @@ -226,10 +218,7 @@ mod if_alloc { impl Stream for std::panic::AssertUnwindSafe { type Item = S::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { unsafe { self.map_unchecked_mut(|x| &mut x.0) }.poll_next(cx) } diff --git a/futures-core/src/task/__internal/atomic_waker.rs b/futures-core/src/task/__internal/atomic_waker.rs index a2f6191368..bbf8939bc0 100644 --- a/futures-core/src/task/__internal/atomic_waker.rs +++ b/futures-core/src/task/__internal/atomic_waker.rs @@ -1,7 +1,7 @@ use core::cell::UnsafeCell; use core::fmt; use core::sync::atomic::AtomicUsize; -use core::sync::atomic::Ordering::{Acquire, Release, AcqRel}; +use core::sync::atomic::Ordering::{AcqRel, Acquire, Release}; use core::task::Waker; /// A synchronization primitive for task wakeup. @@ -202,10 +202,7 @@ impl AtomicWaker { trait AssertSync: Sync {} impl AssertSync for Waker {} - Self { - state: AtomicUsize::new(WAITING), - waker: UnsafeCell::new(None), - } + Self { state: AtomicUsize::new(WAITING), waker: UnsafeCell::new(None) } } /// Registers the waker to be notified on calls to `wake`. @@ -275,8 +272,7 @@ impl AtomicWaker { // nothing to acquire, only release. In case of concurrent // wakers, we need to acquire their releases, so success needs // to do both. - let res = self.state.compare_exchange( - REGISTERING, WAITING, AcqRel, Acquire); + let res = self.state.compare_exchange(REGISTERING, WAITING, AcqRel, Acquire); match res { Ok(_) => { @@ -340,9 +336,7 @@ impl AtomicWaker { // // We just want to maintain memory safety. It is ok to drop the // call to `register`. - debug_assert!( - state == REGISTERING || - state == REGISTERING | WAKING); + debug_assert!(state == REGISTERING || state == REGISTERING | WAKING); } } } @@ -387,9 +381,8 @@ impl AtomicWaker { // not. // debug_assert!( - state == REGISTERING || - state == REGISTERING | WAKING || - state == WAKING); + state == REGISTERING || state == REGISTERING | WAKING || state == WAKING + ); None } } diff --git a/futures-core/src/task/mod.rs b/futures-core/src/task/mod.rs index 362b376e04..5dde0804fe 100644 --- a/futures-core/src/task/mod.rs +++ b/futures-core/src/task/mod.rs @@ -6,4 +6,4 @@ mod poll; #[doc(hidden)] pub mod __internal; -pub use core::task::{Context, Poll, Waker, RawWaker, RawWakerVTable}; +pub use core::task::{Context, Poll, RawWaker, RawWakerVTable, Waker}; diff --git a/futures-core/src/task/poll.rs b/futures-core/src/task/poll.rs index 8fe294c580..930d888d2d 100644 --- a/futures-core/src/task/poll.rs +++ b/futures-core/src/task/poll.rs @@ -3,9 +3,10 @@ /// This macro bakes in propagation of `Pending` signals by returning early. #[macro_export] macro_rules! ready { - ($e:expr $(,)?) => (match $e { - $crate::__private::Poll::Ready(t) => t, - $crate::__private::Poll::Pending => - return $crate::__private::Poll::Pending, - }) + ($e:expr $(,)?) => { + match $e { + $crate::__private::Poll::Ready(t) => t, + $crate::__private::Poll::Pending => return $crate::__private::Poll::Pending, + } + }; } diff --git a/futures-executor/benches/thread_notify.rs b/futures-executor/benches/thread_notify.rs index d8fbec4555..88d0447cf6 100644 --- a/futures-executor/benches/thread_notify.rs +++ b/futures-executor/benches/thread_notify.rs @@ -102,10 +102,7 @@ fn thread_yield_multi_thread(b: &mut Bencher) { }); b.iter(move || { - let y = Yield { - rem: NUM, - tx: tx.clone(), - }; + let y = Yield { rem: NUM, tx: tx.clone() }; block_on(y); }); diff --git a/futures-executor/src/lib.rs b/futures-executor/src/lib.rs index 7e3bf5bebe..c351061c3a 100644 --- a/futures-executor/src/lib.rs +++ b/futures-executor/src/lib.rs @@ -4,21 +4,16 @@ //! library is activated, and it is activated by default. #![cfg_attr(not(feature = "std"), no_std)] - #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms, unreachable_pub)] // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-executor/0.3.8")] - #![cfg_attr(docsrs, feature(doc_cfg))] #[cfg(feature = "std")] @@ -26,14 +21,14 @@ mod local_pool; #[cfg(feature = "std")] pub use crate::local_pool::{block_on, block_on_stream, BlockingStream, LocalPool, LocalSpawner}; -#[cfg(feature = "thread-pool")] -#[cfg(feature = "std")] -mod unpark_mutex; #[cfg(feature = "thread-pool")] #[cfg_attr(docsrs, doc(cfg(feature = "thread-pool")))] #[cfg(feature = "std")] mod thread_pool; #[cfg(feature = "thread-pool")] +#[cfg(feature = "std")] +mod unpark_mutex; +#[cfg(feature = "thread-pool")] #[cfg_attr(docsrs, doc(cfg(feature = "thread-pool")))] #[cfg(feature = "std")] pub use crate::thread_pool::{ThreadPool, ThreadPoolBuilder}; diff --git a/futures-executor/src/local_pool.rs b/futures-executor/src/local_pool.rs index 156d5cc642..bee96d8db9 100644 --- a/futures-executor/src/local_pool.rs +++ b/futures-executor/src/local_pool.rs @@ -10,7 +10,10 @@ use futures_util::stream::StreamExt; use std::cell::RefCell; use std::ops::{Deref, DerefMut}; use std::rc::{Rc, Weak}; -use std::sync::{Arc, atomic::{AtomicBool, Ordering}}; +use std::sync::{ + atomic::{AtomicBool, Ordering}, + Arc, +}; use std::thread::{self, Thread}; /// A single-threaded task pool for polling futures to completion. @@ -119,17 +122,12 @@ fn poll_executor) -> T>(mut f: F) -> T { impl LocalPool { /// Create a new, empty pool of tasks. pub fn new() -> Self { - Self { - pool: FuturesUnordered::new(), - incoming: Default::default(), - } + Self { pool: FuturesUnordered::new(), incoming: Default::default() } } /// Get a clonable handle to the pool as a [`Spawn`]. pub fn spawner(&self) -> LocalSpawner { - LocalSpawner { - incoming: Rc::downgrade(&self.incoming), - } + LocalSpawner { incoming: Rc::downgrade(&self.incoming) } } /// Run all tasks in the pool to completion. diff --git a/futures-executor/src/thread_pool.rs b/futures-executor/src/thread_pool.rs index 741e6d9c6c..f2347dbbdf 100644 --- a/futures-executor/src/thread_pool.rs +++ b/futures-executor/src/thread_pool.rs @@ -2,8 +2,8 @@ use crate::enter; use crate::unpark_mutex::UnparkMutex; use futures_core::future::Future; use futures_core::task::{Context, Poll}; +use futures_task::{waker_ref, ArcWake}; use futures_task::{FutureObj, Spawn, SpawnError}; -use futures_task::{ArcWake, waker_ref}; use futures_util::future::FutureExt; use std::cmp; use std::fmt; @@ -54,9 +54,7 @@ struct PoolState { impl fmt::Debug for ThreadPool { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("ThreadPool") - .field("size", &self.state.size) - .finish() + f.debug_struct("ThreadPool").field("size", &self.state.size).finish() } } @@ -100,10 +98,7 @@ impl ThreadPool { pub fn spawn_obj_ok(&self, future: FutureObj<'static, ()>) { let task = Task { future, - wake_handle: Arc::new(WakeHandle { - exec: self.clone(), - mutex: UnparkMutex::new(), - }), + wake_handle: Arc::new(WakeHandle { exec: self.clone(), mutex: UnparkMutex::new() }), exec: self.clone(), }; self.state.send(Message::Run(task)); @@ -132,10 +127,7 @@ impl ThreadPool { } impl Spawn for ThreadPool { - fn spawn_obj( - &self, - future: FutureObj<'static, ()>, - ) -> Result<(), SpawnError> { + fn spawn_obj(&self, future: FutureObj<'static, ()>) -> Result<(), SpawnError> { self.spawn_obj_ok(future); Ok(()) } @@ -146,10 +138,12 @@ impl PoolState { self.tx.lock().unwrap().send(msg).unwrap(); } - fn work(&self, - idx: usize, - after_start: Option>, - before_stop: Option>) { + fn work( + &self, + idx: usize, + after_start: Option>, + before_stop: Option>, + ) { let _scope = enter().unwrap(); if let Some(after_start) = after_start { after_start(idx); @@ -241,7 +235,8 @@ impl ThreadPoolBuilder { /// The closure provided will receive an index corresponding to the worker /// thread it's running on. pub fn after_start(&mut self, f: F) -> &mut Self - where F: Fn(usize) + Send + Sync + 'static + where + F: Fn(usize) + Send + Sync + 'static, { self.after_start = Some(Arc::new(f)); self @@ -256,7 +251,8 @@ impl ThreadPoolBuilder { /// The closure provided will receive an index corresponding to the worker /// thread it's running on. pub fn before_stop(&mut self, f: F) -> &mut Self - where F: Fn(usize) + Send + Sync + 'static + where + F: Fn(usize) + Send + Sync + 'static, { self.before_stop = Some(Arc::new(f)); self @@ -328,14 +324,11 @@ impl Task { Poll::Pending => {} Poll::Ready(()) => return wake_handle.mutex.complete(), } - let task = Self { - future, - wake_handle: wake_handle.clone(), - exec, - }; + let task = Self { future, wake_handle: wake_handle.clone(), exec }; match wake_handle.mutex.wait(task) { Ok(()) => return, // we've waited - Err(task) => { // someone's notified us + Err(task) => { + // someone's notified us future = task.future; exec = task.exec; } @@ -347,9 +340,7 @@ impl Task { impl fmt::Debug for Task { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Task") - .field("contents", &"...") - .finish() + f.debug_struct("Task").field("contents", &"...").finish() } } @@ -372,7 +363,9 @@ mod tests { let (tx, rx) = mpsc::sync_channel(2); let _cpu_pool = ThreadPoolBuilder::new() .pool_size(2) - .after_start(move |_| tx.send(1).unwrap()).create().unwrap(); + .after_start(move |_| tx.send(1).unwrap()) + .create() + .unwrap(); // After ThreadPoolBuilder is deconstructed, the tx should be droped // so that we can use rx as an iterator. diff --git a/futures-executor/src/unpark_mutex.rs b/futures-executor/src/unpark_mutex.rs index c49c64cc39..ac5112cfa2 100644 --- a/futures-executor/src/unpark_mutex.rs +++ b/futures-executor/src/unpark_mutex.rs @@ -29,25 +29,22 @@ unsafe impl Sync for UnparkMutex {} // transitions: // The task is blocked, waiting on an event -const WAITING: usize = 0; // --> POLLING +const WAITING: usize = 0; // --> POLLING // The task is actively being polled by a thread; arrival of additional events // of interest should move it to the REPOLL state -const POLLING: usize = 1; // --> WAITING, REPOLL, or COMPLETE +const POLLING: usize = 1; // --> WAITING, REPOLL, or COMPLETE // The task is actively being polled, but will need to be re-polled upon // completion to ensure that all events were observed. -const REPOLL: usize = 2; // --> POLLING +const REPOLL: usize = 2; // --> POLLING // The task has finished executing (either successfully or with an error/panic) -const COMPLETE: usize = 3; // No transitions out +const COMPLETE: usize = 3; // No transitions out impl UnparkMutex { pub(crate) fn new() -> Self { - Self { - status: AtomicUsize::new(WAITING), - inner: UnsafeCell::new(None), - } + Self { status: AtomicUsize::new(WAITING), inner: UnsafeCell::new(None) } } /// Attempt to "notify" the mutex that a poll should occur. @@ -62,8 +59,7 @@ impl UnparkMutex { match status { // The task is idle, so try to run it immediately. WAITING => { - match self.status.compare_exchange(WAITING, POLLING, - SeqCst, SeqCst) { + match self.status.compare_exchange(WAITING, POLLING, SeqCst, SeqCst) { Ok(_) => { let data = unsafe { // SAFETY: we've ensured mutual exclusion via @@ -82,13 +78,10 @@ impl UnparkMutex { // The task is being polled, so we need to record that it should // be *repolled* when complete. - POLLING => { - match self.status.compare_exchange(POLLING, REPOLL, - SeqCst, SeqCst) { - Ok(_) => return Err(()), - Err(cur) => status = cur, - } - } + POLLING => match self.status.compare_exchange(POLLING, REPOLL, SeqCst, SeqCst) { + Ok(_) => return Err(()), + Err(cur) => status = cur, + }, // The task is already scheduled for polling, or is complete, so // we've got nothing to do. diff --git a/futures-executor/tests/local_pool.rs b/futures-executor/tests/local_pool.rs index b31f1034cb..56e6daa077 100644 --- a/futures-executor/tests/local_pool.rs +++ b/futures-executor/tests/local_pool.rs @@ -1,14 +1,14 @@ use futures::channel::oneshot; use futures::executor::LocalPool; -use futures::future::{self, Future, lazy, poll_fn}; -use futures::task::{Context, Poll, Spawn, LocalSpawn, Waker}; +use futures::future::{self, lazy, poll_fn, Future}; +use futures::task::{Context, LocalSpawn, Poll, Spawn, Waker}; use std::cell::{Cell, RefCell}; use std::pin::Pin; use std::rc::Rc; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::Arc; use std::thread; use std::time::Duration; -use std::sync::Arc; -use std::sync::atomic::{Ordering, AtomicBool}; struct Pending(Rc<()>); @@ -52,9 +52,14 @@ fn run_until_executes_spawned() { let (tx, rx) = oneshot::channel(); let mut pool = LocalPool::new(); let spawn = pool.spawner(); - spawn.spawn_local_obj(Box::pin(lazy(move |_| { - tx.send(()).unwrap(); - })).into()).unwrap(); + spawn + .spawn_local_obj( + Box::pin(lazy(move |_| { + tx.send(()).unwrap(); + })) + .into(), + ) + .unwrap(); pool.run_until(rx).unwrap(); } @@ -74,18 +79,27 @@ fn run_executes_spawned() { let spawn = pool.spawner(); let spawn2 = pool.spawner(); - spawn.spawn_local_obj(Box::pin(lazy(move |_| { - spawn2.spawn_local_obj(Box::pin(lazy(move |_| { - cnt2.set(cnt2.get() + 1); - })).into()).unwrap(); - })).into()).unwrap(); + spawn + .spawn_local_obj( + Box::pin(lazy(move |_| { + spawn2 + .spawn_local_obj( + Box::pin(lazy(move |_| { + cnt2.set(cnt2.get() + 1); + })) + .into(), + ) + .unwrap(); + })) + .into(), + ) + .unwrap(); pool.run(); assert_eq!(cnt.get(), 1); } - #[test] fn run_spawn_many() { const ITER: usize = 200; @@ -97,9 +111,14 @@ fn run_spawn_many() { for _ in 0..ITER { let cnt = cnt.clone(); - spawn.spawn_local_obj(Box::pin(lazy(move |_| { - cnt.set(cnt.get() + 1); - })).into()).unwrap(); + spawn + .spawn_local_obj( + Box::pin(lazy(move |_| { + cnt.set(cnt.get() + 1); + })) + .into(), + ) + .unwrap(); } pool.run(); @@ -126,9 +145,14 @@ fn try_run_one_executes_one_ready() { spawn.spawn_local_obj(Box::pin(pending()).into()).unwrap(); let cnt = cnt.clone(); - spawn.spawn_local_obj(Box::pin(lazy(move |_| { - cnt.set(cnt.get() + 1); - })).into()).unwrap(); + spawn + .spawn_local_obj( + Box::pin(lazy(move |_| { + cnt.set(cnt.get() + 1); + })) + .into(), + ) + .unwrap(); spawn.spawn_local_obj(Box::pin(pending()).into()).unwrap(); } @@ -154,15 +178,20 @@ fn try_run_one_returns_on_no_progress() { { let cnt = cnt.clone(); let waker = waker.clone(); - spawn.spawn_local_obj(Box::pin(poll_fn(move |ctx| { - cnt.set(cnt.get() + 1); - waker.set(Some(ctx.waker().clone())); - if cnt.get() == ITER { - Poll::Ready(()) - } else { - Poll::Pending - } - })).into()).unwrap(); + spawn + .spawn_local_obj( + Box::pin(poll_fn(move |ctx| { + cnt.set(cnt.get() + 1); + waker.set(Some(ctx.waker().clone())); + if cnt.get() == ITER { + Poll::Ready(()) + } else { + Poll::Pending + } + })) + .into(), + ) + .unwrap(); } for i in 0..ITER - 1 { @@ -185,16 +214,21 @@ fn try_run_one_runs_sub_futures() { let inner_spawner = spawn.clone(); let cnt1 = cnt.clone(); - spawn.spawn_local_obj(Box::pin(poll_fn(move |_| { - cnt1.set(cnt1.get() + 1); - - let cnt2 = cnt1.clone(); - inner_spawner.spawn_local_obj(Box::pin(lazy(move |_|{ - cnt2.set(cnt2.get() + 1) - })).into()).unwrap(); + spawn + .spawn_local_obj( + Box::pin(poll_fn(move |_| { + cnt1.set(cnt1.get() + 1); - Poll::Pending - })).into()).unwrap(); + let cnt2 = cnt1.clone(); + inner_spawner + .spawn_local_obj(Box::pin(lazy(move |_| cnt2.set(cnt2.get() + 1))).into()) + .unwrap(); + + Poll::Pending + })) + .into(), + ) + .unwrap(); pool.try_run_one(); assert_eq!(cnt.get(), 2); @@ -214,12 +248,12 @@ fn run_until_stalled_returns_multiple_times() { let cnt = Rc::new(Cell::new(0)); let cnt1 = cnt.clone(); - spawn.spawn_local_obj(Box::pin(lazy(move |_|{ cnt1.set(cnt1.get() + 1) })).into()).unwrap(); + spawn.spawn_local_obj(Box::pin(lazy(move |_| cnt1.set(cnt1.get() + 1))).into()).unwrap(); pool.run_until_stalled(); assert_eq!(cnt.get(), 1); let cnt2 = cnt.clone(); - spawn.spawn_local_obj(Box::pin(lazy(move |_|{ cnt2.set(cnt2.get() + 1) })).into()).unwrap(); + spawn.spawn_local_obj(Box::pin(lazy(move |_| cnt2.set(cnt2.get() + 1))).into()).unwrap(); pool.run_until_stalled(); assert_eq!(cnt.get(), 2); } @@ -232,16 +266,21 @@ fn run_until_stalled_runs_spawned_sub_futures() { let inner_spawner = spawn.clone(); let cnt1 = cnt.clone(); - spawn.spawn_local_obj(Box::pin(poll_fn(move |_| { - cnt1.set(cnt1.get() + 1); - - let cnt2 = cnt1.clone(); - inner_spawner.spawn_local_obj(Box::pin(lazy(move |_|{ - cnt2.set(cnt2.get() + 1) - })).into()).unwrap(); + spawn + .spawn_local_obj( + Box::pin(poll_fn(move |_| { + cnt1.set(cnt1.get() + 1); - Poll::Pending - })).into()).unwrap(); + let cnt2 = cnt1.clone(); + inner_spawner + .spawn_local_obj(Box::pin(lazy(move |_| cnt2.set(cnt2.get() + 1))).into()) + .unwrap(); + + Poll::Pending + })) + .into(), + ) + .unwrap(); pool.run_until_stalled(); assert_eq!(cnt.get(), 2); @@ -262,9 +301,14 @@ fn run_until_stalled_executes_all_ready() { spawn.spawn_local_obj(Box::pin(pending()).into()).unwrap(); let cnt = cnt.clone(); - spawn.spawn_local_obj(Box::pin(lazy(move |_| { - cnt.set(cnt.get() + 1); - })).into()).unwrap(); + spawn + .spawn_local_obj( + Box::pin(lazy(move |_| { + cnt.set(cnt.get() + 1); + })) + .into(), + ) + .unwrap(); // also add some pending tasks to test if they are ignored spawn.spawn_local_obj(Box::pin(pending()).into()).unwrap(); @@ -281,10 +325,15 @@ fn nesting_run() { let mut pool = LocalPool::new(); let spawn = pool.spawner(); - spawn.spawn_obj(Box::pin(lazy(|_| { - let mut pool = LocalPool::new(); - pool.run(); - })).into()).unwrap(); + spawn + .spawn_obj( + Box::pin(lazy(|_| { + let mut pool = LocalPool::new(); + pool.run(); + })) + .into(), + ) + .unwrap(); pool.run(); } @@ -295,10 +344,15 @@ fn nesting_run_run_until_stalled() { let mut pool = LocalPool::new(); let spawn = pool.spawner(); - spawn.spawn_obj(Box::pin(lazy(|_| { - let mut pool = LocalPool::new(); - pool.run_until_stalled(); - })).into()).unwrap(); + spawn + .spawn_obj( + Box::pin(lazy(|_| { + let mut pool = LocalPool::new(); + pool.run_until_stalled(); + })) + .into(), + ) + .unwrap(); pool.run(); } @@ -342,15 +396,9 @@ fn tasks_are_scheduled_fairly() { let mut pool = LocalPool::new(); let spawn = pool.spawner(); - spawn.spawn_local_obj(Box::pin(Spin { - state: state.clone(), - idx: 0, - }).into()).unwrap(); + spawn.spawn_local_obj(Box::pin(Spin { state: state.clone(), idx: 0 }).into()).unwrap(); - spawn.spawn_local_obj(Box::pin(Spin { - state, - idx: 1, - }).into()).unwrap(); + spawn.spawn_local_obj(Box::pin(Spin { state, idx: 1 }).into()).unwrap(); pool.run(); } @@ -363,11 +411,11 @@ fn park_unpark_independence() { let future = future::poll_fn(move |cx| { if done { - return Poll::Ready(()) + return Poll::Ready(()); } done = true; cx.waker().clone().wake(); // (*) - // some user-code that temporarily parks the thread + // some user-code that temporarily parks the thread let test = thread::current(); let latch = Arc::new(AtomicBool::new(false)); let signal = latch.clone(); @@ -384,4 +432,3 @@ fn park_unpark_independence() { futures::executor::block_on(future) } - diff --git a/futures-io/src/lib.rs b/futures-io/src/lib.rs index cfc3f22706..60289b8d0c 100644 --- a/futures-io/src/lib.rs +++ b/futures-io/src/lib.rs @@ -9,23 +9,17 @@ //! library is activated, and it is activated by default. #![cfg_attr(all(feature = "read-initializer", feature = "std"), feature(read_initializer))] - #![cfg_attr(not(feature = "std"), no_std)] - #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms, unreachable_pub)] // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-io/0.3.8")] - #![cfg_attr(docsrs, feature(doc_cfg))] #[cfg(all(feature = "read-initializer", not(feature = "unstable")))] @@ -41,19 +35,12 @@ mod if_std { // Re-export some types from `std::io` so that users don't have to deal // with conflicts when `use`ing `futures::io` and `std::io`. #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 - pub use io::{ - Error as Error, - ErrorKind as ErrorKind, - Result as Result, - IoSlice as IoSlice, - IoSliceMut as IoSliceMut, - SeekFrom as SeekFrom, - }; + pub use io::{Error, ErrorKind, IoSlice, IoSliceMut, Result, SeekFrom}; #[cfg(feature = "read-initializer")] #[cfg_attr(docsrs, doc(cfg(feature = "read-initializer")))] #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 - pub use io::Initializer as Initializer; + pub use io::Initializer; /// Read bytes asynchronously. /// @@ -99,8 +86,11 @@ mod if_std { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Poll::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_read(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) - -> Poll>; + fn poll_read( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll>; /// Attempt to read from the `AsyncRead` into `bufs` using vectored /// IO operations. @@ -124,9 +114,11 @@ mod if_std { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Poll::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_read_vectored(self: Pin<&mut Self>, cx: &mut Context<'_>, bufs: &mut [IoSliceMut<'_>]) - -> Poll> - { + fn poll_read_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &mut [IoSliceMut<'_>], + ) -> Poll> { for b in bufs { if !b.is_empty() { return self.poll_read(cx, b); @@ -163,8 +155,11 @@ mod if_std { /// /// `poll_write` must try to make progress by flushing the underlying object if /// that is the only way the underlying object can become writable again. - fn poll_write(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) - -> Poll>; + fn poll_write( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll>; /// Attempt to write bytes from `bufs` into the object using vectored /// IO operations. @@ -189,9 +184,11 @@ mod if_std { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Poll::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_write_vectored(self: Pin<&mut Self>, cx: &mut Context<'_>, bufs: &[IoSlice<'_>]) - -> Poll> - { + fn poll_write_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { for b in bufs { if !b.is_empty() { return self.poll_write(cx, b); @@ -266,8 +263,11 @@ mod if_std { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Poll::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_seek(self: Pin<&mut Self>, cx: &mut Context<'_>, pos: SeekFrom) - -> Poll>; + fn poll_seek( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + pos: SeekFrom, + ) -> Poll>; } /// Read bytes asynchronously. @@ -306,8 +306,7 @@ mod if_std { /// `Interrupted`. Implementations must convert `WouldBlock` into /// `Poll::Pending` and either internally retry or convert /// `Interrupted` into another error kind. - fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll>; + fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; /// Tells this buffer that `amt` bytes have been consumed from the buffer, /// so they should no longer be returned in calls to [`poll_read`]. @@ -334,18 +333,22 @@ mod if_std { (**self).initializer() } - fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) - -> Poll> - { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { Pin::new(&mut **self).poll_read(cx, buf) } - fn poll_read_vectored(mut self: Pin<&mut Self>, cx: &mut Context<'_>, bufs: &mut [IoSliceMut<'_>]) - -> Poll> - { + fn poll_read_vectored( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &mut [IoSliceMut<'_>], + ) -> Poll> { Pin::new(&mut **self).poll_read_vectored(cx, bufs) } - } + }; } impl AsyncRead for Box { @@ -366,15 +369,19 @@ mod if_std { (**self).initializer() } - fn poll_read(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) - -> Poll> - { + fn poll_read( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { self.get_mut().as_mut().poll_read(cx, buf) } - fn poll_read_vectored(self: Pin<&mut Self>, cx: &mut Context<'_>, bufs: &mut [IoSliceMut<'_>]) - -> Poll> - { + fn poll_read_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &mut [IoSliceMut<'_>], + ) -> Poll> { self.get_mut().as_mut().poll_read_vectored(cx, bufs) } } @@ -386,18 +393,22 @@ mod if_std { io::Read::initializer(self) } - fn poll_read(mut self: Pin<&mut Self>, _: &mut Context<'_>, buf: &mut [u8]) - -> Poll> - { + fn poll_read( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { Poll::Ready(io::Read::read(&mut *self, buf)) } - fn poll_read_vectored(mut self: Pin<&mut Self>, _: &mut Context<'_>, bufs: &mut [IoSliceMut<'_>]) - -> Poll> - { + fn poll_read_vectored( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + bufs: &mut [IoSliceMut<'_>], + ) -> Poll> { Poll::Ready(io::Read::read_vectored(&mut *self, bufs)) } - } + }; } impl AsyncRead for &[u8] { @@ -406,15 +417,19 @@ mod if_std { macro_rules! deref_async_write { () => { - fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) - -> Poll> - { + fn poll_write( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { Pin::new(&mut **self).poll_write(cx, buf) } - fn poll_write_vectored(mut self: Pin<&mut Self>, cx: &mut Context<'_>, bufs: &[IoSlice<'_>]) - -> Poll> - { + fn poll_write_vectored( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { Pin::new(&mut **self).poll_write_vectored(cx, bufs) } @@ -425,7 +440,7 @@ mod if_std { fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { Pin::new(&mut **self).poll_close(cx) } - } + }; } impl AsyncWrite for Box { @@ -441,15 +456,19 @@ mod if_std { P: DerefMut + Unpin, P::Target: AsyncWrite, { - fn poll_write(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) - -> Poll> - { + fn poll_write( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { self.get_mut().as_mut().poll_write(cx, buf) } - fn poll_write_vectored(self: Pin<&mut Self>, cx: &mut Context<'_>, bufs: &[IoSlice<'_>]) - -> Poll> - { + fn poll_write_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { self.get_mut().as_mut().poll_write_vectored(cx, bufs) } @@ -464,15 +483,19 @@ mod if_std { macro_rules! delegate_async_write_to_stdio { () => { - fn poll_write(mut self: Pin<&mut Self>, _: &mut Context<'_>, buf: &[u8]) - -> Poll> - { + fn poll_write( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { Poll::Ready(io::Write::write(&mut *self, buf)) } - fn poll_write_vectored(mut self: Pin<&mut Self>, _: &mut Context<'_>, bufs: &[IoSlice<'_>]) - -> Poll> - { + fn poll_write_vectored( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { Poll::Ready(io::Write::write_vectored(&mut *self, bufs)) } @@ -483,7 +506,7 @@ mod if_std { fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.poll_flush(cx) } - } + }; } impl AsyncWrite for Vec { @@ -492,12 +515,14 @@ mod if_std { macro_rules! deref_async_seek { () => { - fn poll_seek(mut self: Pin<&mut Self>, cx: &mut Context<'_>, pos: SeekFrom) - -> Poll> - { + fn poll_seek( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + pos: SeekFrom, + ) -> Poll> { Pin::new(&mut **self).poll_seek(cx, pos) } - } + }; } impl AsyncSeek for Box { @@ -513,25 +538,25 @@ mod if_std { P: DerefMut + Unpin, P::Target: AsyncSeek, { - fn poll_seek(self: Pin<&mut Self>, cx: &mut Context<'_>, pos: SeekFrom) - -> Poll> - { + fn poll_seek( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + pos: SeekFrom, + ) -> Poll> { self.get_mut().as_mut().poll_seek(cx, pos) } } macro_rules! deref_async_buf_read { () => { - fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll> - { + fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { Pin::new(&mut **self.get_mut()).poll_fill_buf(cx) } fn consume(mut self: Pin<&mut Self>, amt: usize) { Pin::new(&mut **self).consume(amt) } - } + }; } impl AsyncBufRead for Box { @@ -547,9 +572,7 @@ mod if_std { P: DerefMut + Unpin, P::Target: AsyncBufRead, { - fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll> - { + fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.get_mut().as_mut().poll_fill_buf(cx) } @@ -560,16 +583,14 @@ mod if_std { macro_rules! delegate_async_buf_read_to_stdio { () => { - fn poll_fill_buf(self: Pin<&mut Self>, _: &mut Context<'_>) - -> Poll> - { + fn poll_fill_buf(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(io::BufRead::fill_buf(self.get_mut())) } fn consume(self: Pin<&mut Self>, amt: usize) { io::BufRead::consume(self.get_mut(), amt) } - } + }; } impl AsyncBufRead for &[u8] { diff --git a/futures-macro/src/join.rs b/futures-macro/src/join.rs index 048b62a6cd..d427da27a0 100644 --- a/futures-macro/src/join.rs +++ b/futures-macro/src/join.rs @@ -27,10 +27,7 @@ impl Parse for Join { } } -fn bind_futures( - fut_exprs: Vec, - span: Span, -) -> (Vec, Vec) { +fn bind_futures(fut_exprs: Vec, span: Span) -> (Vec, Vec) { let mut future_let_bindings = Vec::with_capacity(fut_exprs.len()); let future_names: Vec<_> = fut_exprs .into_iter() diff --git a/futures-macro/src/lib.rs b/futures-macro/src/lib.rs index bf003395ac..1435dc4218 100644 --- a/futures-macro/src/lib.rs +++ b/futures-macro/src/lib.rs @@ -5,14 +5,11 @@ // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-join-macro/0.3.8")] // Since https://github.com/rust-lang/cargo/pull/7700 `proc_macro` is part of the prelude for diff --git a/futures-macro/src/select.rs b/futures-macro/src/select.rs index 65b31877ac..0c8e5f1ca0 100644 --- a/futures-macro/src/select.rs +++ b/futures-macro/src/select.rs @@ -3,8 +3,8 @@ use proc_macro::TokenStream; use proc_macro2::Span; use quote::{format_ident, quote}; -use syn::{parse_quote, Expr, Ident, Pat, Token}; use syn::parse::{Parse, ParseStream}; +use syn::{parse_quote, Expr, Ident, Pat, Token}; mod kw { syn::custom_keyword!(complete); @@ -63,7 +63,10 @@ impl Parse for Select { // Commas after the expression are only optional if it's a `Block` // or it is the last branch in the `match`. - let is_block = match expr { Expr::Block(_) => true, _ => false }; + let is_block = match expr { + Expr::Block(_) => true, + _ => false, + }; if is_block || input.is_empty() { input.parse::>()?; } else { @@ -76,7 +79,7 @@ impl Parse for Select { CaseKind::Normal(pat, fut_expr) => { select.normal_fut_exprs.push(fut_expr); select.normal_fut_handlers.push((pat, expr)); - }, + } } } @@ -92,22 +95,16 @@ fn declare_result_enum( result_ident: Ident, variants: usize, complete: bool, - span: Span + span: Span, ) -> (Vec, syn::ItemEnum) { // "_0", "_1", "_2" let variant_names: Vec = - (0..variants) - .map(|num| format_ident!("_{}", num, span = span)) - .collect(); + (0..variants).map(|num| format_ident!("_{}", num, span = span)).collect(); let type_parameters = &variant_names; let variants = &variant_names; - let complete_variant = if complete { - Some(quote!(Complete)) - } else { - None - }; + let complete_variant = if complete { Some(quote!(Complete)) } else { None }; let enum_item = parse_quote! { enum #result_ident<#(#type_parameters,)*> { @@ -148,7 +145,9 @@ fn select_inner(input: TokenStream, random: bool) -> TokenStream { // bind non-`Ident` future exprs w/ `let` let mut future_let_bindings = Vec::with_capacity(parsed.normal_fut_exprs.len()); - let bound_future_names: Vec<_> = parsed.normal_fut_exprs.into_iter() + let bound_future_names: Vec<_> = parsed + .normal_fut_exprs + .into_iter() .zip(variant_names.iter()) .map(|(expr, variant_name)| { match expr { @@ -164,7 +163,7 @@ fn select_inner(input: TokenStream, random: bool) -> TokenStream { __futures_crate::async_await::assert_unpin(&#path); }); path - }, + } _ => { // Bind and pin the resulting Future on the stack. This is // necessary to support direct select! calls on !Unpin @@ -188,8 +187,8 @@ fn select_inner(input: TokenStream, random: bool) -> TokenStream { // For each future, make an `&mut dyn FnMut(&mut Context<'_>) -> Option>` // to use for polling that individual future. These will then be put in an array. - let poll_functions = bound_future_names.iter().zip(variant_names.iter()) - .map(|(bound_future_name, variant_name)| { + let poll_functions = bound_future_names.iter().zip(variant_names.iter()).map( + |(bound_future_name, variant_name)| { // Below we lazily create the Pin on the Future below. // This is done in order to avoid allocating memory in the generator // for the Pin variable. @@ -216,7 +215,8 @@ fn select_inner(input: TokenStream, random: bool) -> TokenStream { &mut __futures_crate::task::Context<'_> ) -> __futures_crate::Option<__futures_crate::task::Poll<_>> = &mut #variant_name; } - }); + }, + ); let none_polled = if parsed.complete.is_some() { quote! { @@ -229,13 +229,13 @@ fn select_inner(input: TokenStream, random: bool) -> TokenStream { } }; - let branches = parsed.normal_fut_handlers.into_iter() - .zip(variant_names.iter()) - .map(|((pat, expr), variant_name)| { + let branches = parsed.normal_fut_handlers.into_iter().zip(variant_names.iter()).map( + |((pat, expr), variant_name)| { quote! { #enum_ident::#variant_name(#pat) => { #expr }, } - }); + }, + ); let branches = quote! { #( #branches )* }; let complete_branch = parsed.complete.map(|complete_expr| { diff --git a/futures-sink/src/lib.rs b/futures-sink/src/lib.rs index c477cce48f..74202cf61f 100644 --- a/futures-sink/src/lib.rs +++ b/futures-sink/src/lib.rs @@ -8,14 +8,11 @@ // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-sink/0.3.8")] #[cfg(feature = "alloc")] @@ -215,7 +212,10 @@ mod if_alloc { impl + Unpin, Item> Sink for alloc::boxed::Box { type Error = S::Error; - fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_ready( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { Pin::new(&mut **self).poll_ready(cx) } @@ -223,11 +223,17 @@ mod if_alloc { Pin::new(&mut **self).start_send(item) } - fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_flush( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { Pin::new(&mut **self).poll_flush(cx) } - fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_close( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { Pin::new(&mut **self).poll_close(cx) } } diff --git a/futures-task/src/future_obj.rs b/futures-task/src/future_obj.rs index 373be244cd..48ec12beb2 100644 --- a/futures-task/src/future_obj.rs +++ b/futures-task/src/future_obj.rs @@ -1,8 +1,8 @@ use core::{ - mem, fmt, future::Future, marker::PhantomData, + mem, pin::Pin, task::{Context, Poll}, }; @@ -26,16 +26,16 @@ impl Unpin for LocalFutureObj<'_, T> {} #[allow(single_use_lifetimes)] #[allow(clippy::transmute_ptr_to_ptr)] -unsafe fn remove_future_lifetime<'a, T>(ptr: *mut (dyn Future + 'a)) - -> *mut (dyn Future + 'static) -{ +unsafe fn remove_future_lifetime<'a, T>( + ptr: *mut (dyn Future + 'a), +) -> *mut (dyn Future + 'static) { mem::transmute(ptr) } #[allow(single_use_lifetimes)] -unsafe fn remove_drop_lifetime<'a, T>(ptr: unsafe fn (*mut (dyn Future + 'a))) - -> unsafe fn(*mut (dyn Future + 'static)) -{ +unsafe fn remove_drop_lifetime<'a, T>( + ptr: unsafe fn(*mut (dyn Future + 'a)), +) -> unsafe fn(*mut (dyn Future + 'static)) { mem::transmute(ptr) } @@ -65,8 +65,7 @@ impl<'a, T> LocalFutureObj<'a, T> { impl fmt::Debug for LocalFutureObj<'_, T> { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("LocalFutureObj") - .finish() + f.debug_struct("LocalFutureObj").finish() } } @@ -82,17 +81,13 @@ impl Future for LocalFutureObj<'_, T> { #[inline] fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - unsafe { - Pin::new_unchecked(&mut *self.future).poll(cx) - } + unsafe { Pin::new_unchecked(&mut *self.future).poll(cx) } } } impl Drop for LocalFutureObj<'_, T> { fn drop(&mut self) { - unsafe { - (self.drop_fn)(self.future) - } + unsafe { (self.drop_fn)(self.future) } } } @@ -120,8 +115,7 @@ impl<'a, T> FutureObj<'a, T> { impl fmt::Debug for FutureObj<'_, T> { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("FutureObj") - .finish() + f.debug_struct("FutureObj").finish() } } @@ -130,7 +124,7 @@ impl Future for FutureObj<'_, T> { #[inline] fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - Pin::new( &mut self.0 ).poll(cx) + Pin::new(&mut self.0).poll(cx) } } @@ -180,7 +174,7 @@ pub unsafe trait UnsafeFutureObj<'a, T>: 'a { unsafe impl<'a, T, F> UnsafeFutureObj<'a, T> for &'a mut F where - F: Future + Unpin + 'a + F: Future + Unpin + 'a, { fn into_raw(self) -> *mut (dyn Future + 'a) { self as *mut dyn Future @@ -189,8 +183,7 @@ where unsafe fn drop(_ptr: *mut (dyn Future + 'a)) {} } -unsafe impl<'a, T> UnsafeFutureObj<'a, T> for &'a mut (dyn Future + Unpin + 'a) -{ +unsafe impl<'a, T> UnsafeFutureObj<'a, T> for &'a mut (dyn Future + Unpin + 'a) { fn into_raw(self) -> *mut (dyn Future + 'a) { self as *mut dyn Future } @@ -200,7 +193,7 @@ unsafe impl<'a, T> UnsafeFutureObj<'a, T> for &'a mut (dyn Future + unsafe impl<'a, T, F> UnsafeFutureObj<'a, T> for Pin<&'a mut F> where - F: Future + 'a + F: Future + 'a, { fn into_raw(self) -> *mut (dyn Future + 'a) { unsafe { self.get_unchecked_mut() as *mut dyn Future } @@ -209,8 +202,7 @@ where unsafe fn drop(_ptr: *mut (dyn Future + 'a)) {} } -unsafe impl<'a, T> UnsafeFutureObj<'a, T> for Pin<&'a mut (dyn Future + 'a)> -{ +unsafe impl<'a, T> UnsafeFutureObj<'a, T> for Pin<&'a mut (dyn Future + 'a)> { fn into_raw(self) -> *mut (dyn Future + 'a) { unsafe { self.get_unchecked_mut() as *mut dyn Future } } @@ -224,7 +216,8 @@ mod if_alloc { use alloc::boxed::Box; unsafe impl<'a, T, F> UnsafeFutureObj<'a, T> for Box - where F: Future + 'a + where + F: Future + 'a, { fn into_raw(self) -> *mut (dyn Future + 'a) { Box::into_raw(self) @@ -257,7 +250,7 @@ mod if_alloc { unsafe impl<'a, T, F> UnsafeFutureObj<'a, T> for Pin> where - F: Future + 'a + F: Future + 'a, { fn into_raw(mut self) -> *mut (dyn Future + 'a) { let ptr = unsafe { self.as_mut().get_unchecked_mut() as *mut _ }; diff --git a/futures-task/src/lib.rs b/futures-task/src/lib.rs index 96421bb485..5fa9ed5e27 100644 --- a/futures-task/src/lib.rs +++ b/futures-task/src/lib.rs @@ -1,21 +1,16 @@ //! Tools for working with tasks. #![cfg_attr(feature = "cfg-target-has-atomic", feature(cfg_target_has_atomic))] - #![cfg_attr(not(feature = "std"), no_std)] - #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms, unreachable_pub)] // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-task/0.3.8")] #[cfg(all(feature = "cfg-target-has-atomic", not(feature = "unstable")))] @@ -32,7 +27,7 @@ macro_rules! cfg_target_has_atomic { } mod spawn; -pub use crate::spawn::{Spawn, SpawnError, LocalSpawn}; +pub use crate::spawn::{LocalSpawn, Spawn, SpawnError}; cfg_target_has_atomic! { #[cfg(feature = "alloc")] @@ -59,4 +54,4 @@ pub use crate::noop_waker::noop_waker; #[cfg(feature = "std")] pub use crate::noop_waker::noop_waker_ref; -pub use core::task::{Context, Poll, Waker, RawWaker, RawWakerVTable}; +pub use core::task::{Context, Poll, RawWaker, RawWakerVTable, Waker}; diff --git a/futures-task/src/noop_waker.rs b/futures-task/src/noop_waker.rs index 2a84bcd6f3..130fb8f42f 100644 --- a/futures-task/src/noop_waker.rs +++ b/futures-task/src/noop_waker.rs @@ -1,7 +1,7 @@ //! Utilities for creating zero-cost wakers that don't do anything. -use core::task::{RawWaker, RawWakerVTable, Waker}; use core::ptr::null; +use core::task::{RawWaker, RawWakerVTable, Waker}; #[cfg(feature = "std")] use once_cell::sync::Lazy; @@ -29,9 +29,7 @@ fn noop_raw_waker() -> RawWaker { /// ``` #[inline] pub fn noop_waker() -> Waker { - unsafe { - Waker::from_raw(noop_raw_waker()) - } + unsafe { Waker::from_raw(noop_raw_waker()) } } /// Get a static reference to a [`Waker`] which diff --git a/futures-task/src/spawn.rs b/futures-task/src/spawn.rs index a515dd4e18..50f5d0d56a 100644 --- a/futures-task/src/spawn.rs +++ b/futures-task/src/spawn.rs @@ -126,7 +126,7 @@ impl LocalSpawn for &mut Sp { #[cfg(feature = "alloc")] mod if_alloc { use super::*; - use alloc::{ boxed::Box, rc::Rc }; + use alloc::{boxed::Box, rc::Rc}; impl Spawn for Box { fn spawn_obj(&self, future: FutureObj<'static, ()>) -> Result<(), SpawnError> { diff --git a/futures-test/src/assert.rs b/futures-test/src/assert.rs index 4f1cc7d31a..75d7832482 100644 --- a/futures-test/src/assert.rs +++ b/futures-test/src/assert.rs @@ -30,9 +30,7 @@ macro_rules! assert_stream_pending { $crate::__private::assert::assert_is_unpin_stream(stream); let stream = $crate::__private::Pin::new(stream); let mut cx = $crate::task::noop_context(); - let poll = $crate::__private::stream::Stream::poll_next( - stream, &mut cx, - ); + let poll = $crate::__private::stream::Stream::poll_next(stream, &mut cx); if poll.is_ready() { panic!("assertion failed: stream is not pending"); } @@ -71,13 +69,15 @@ macro_rules! assert_stream_next { assert_eq!(x, $item); } $crate::__private::task::Poll::Ready($crate::__private::None) => { - panic!("assertion failed: expected stream to provide item but stream is at its end"); + panic!( + "assertion failed: expected stream to provide item but stream is at its end" + ); } $crate::__private::task::Poll::Pending => { panic!("assertion failed: expected stream to provide item but stream wasn't ready"); } } - }} + }}; } /// Assert that the next poll to the provided stream will return an empty @@ -117,5 +117,5 @@ macro_rules! assert_stream_done { panic!("assertion failed: expected stream to be done but was pending"); } } - }} + }}; } diff --git a/futures-test/src/assert_unmoved.rs b/futures-test/src/assert_unmoved.rs index 6e5116d228..95d9a095f2 100644 --- a/futures-test/src/assert_unmoved.rs +++ b/futures-test/src/assert_unmoved.rs @@ -34,10 +34,7 @@ unsafe impl Sync for AssertUnmoved {} impl AssertUnmoved { pub(crate) fn new(inner: T) -> Self { - Self { - inner, - this_ptr: ptr::null(), - } + Self { inner, this_ptr: ptr::null() } } fn poll_with<'a, U>(mut self: Pin<&'a mut Self>, f: impl FnOnce(Pin<&'a mut T>) -> U) -> U { @@ -46,10 +43,7 @@ impl AssertUnmoved { // First time being polled *self.as_mut().project().this_ptr = cur_this; } else { - assert_eq!( - self.this_ptr, cur_this, - "AssertUnmoved moved between poll calls" - ); + assert_eq!(self.this_ptr, cur_this, "AssertUnmoved moved between poll calls"); } f(self.project().inner) } diff --git a/futures-test/src/future/pending_once.rs b/futures-test/src/future/pending_once.rs index b36af6fe39..0fc3ef0b34 100644 --- a/futures-test/src/future/pending_once.rs +++ b/futures-test/src/future/pending_once.rs @@ -1,7 +1,7 @@ -use futures_core::future::{Future, FusedFuture}; +use futures_core::future::{FusedFuture, Future}; use futures_core::task::{Context, Poll}; -use std::pin::Pin; use pin_project::pin_project; +use std::pin::Pin; /// Combinator that guarantees one [`Poll::Pending`] before polling its inner /// future. @@ -20,20 +20,14 @@ pub struct PendingOnce { impl PendingOnce { pub(super) fn new(future: Fut) -> Self { - Self { - future, - polled_before: false, - } + Self { future, polled_before: false } } } impl Future for PendingOnce { type Output = Fut::Output; - fn poll( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.project(); if *this.polled_before { this.future.poll(cx) diff --git a/futures-test/src/interleave_pending.rs b/futures-test/src/interleave_pending.rs index 7bc8706388..91640778b2 100644 --- a/futures-test/src/interleave_pending.rs +++ b/futures-test/src/interleave_pending.rs @@ -28,10 +28,7 @@ pub struct InterleavePending { impl InterleavePending { pub(crate) fn new(inner: T) -> Self { - Self { - inner, - pended: false, - } + Self { inner, pended: false } } /// Acquires a reference to the underlying I/O object that this adaptor is diff --git a/futures-test/src/io/limited.rs b/futures-test/src/io/limited.rs index a206160099..34b72a530e 100644 --- a/futures-test/src/io/limited.rs +++ b/futures-test/src/io/limited.rs @@ -59,17 +59,11 @@ impl AsyncWrite for Limited { this.io.poll_write(cx, &buf[..cmp::min(*this.limit, buf.len())]) } - fn poll_flush( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.project().io.poll_flush(cx) } - fn poll_close( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.project().io.poll_close(cx) } } @@ -87,10 +81,7 @@ impl AsyncRead for Limited { } impl AsyncBufRead for Limited { - fn poll_fill_buf( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.project().io.poll_fill_buf(cx) } diff --git a/futures-test/src/lib.rs b/futures-test/src/lib.rs index e1b54c1880..d4b594eb0d 100644 --- a/futures-test/src/lib.rs +++ b/futures-test/src/lib.rs @@ -4,29 +4,28 @@ // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-test/0.3.8")] #[cfg(not(feature = "std"))] -compile_error!("`futures-test` must have the `std` feature activated, this is a default-active feature"); +compile_error!( + "`futures-test` must have the `std` feature activated, this is a default-active feature" +); // Not public API. #[doc(hidden)] #[cfg(feature = "std")] pub mod __private { + pub use futures_core::{future, stream, task}; pub use std::{ - option::Option::{Some, None}, + option::Option::{None, Some}, pin::Pin, result::Result::{Err, Ok}, }; - pub use futures_core::{future, stream, task}; pub mod assert { pub use crate::assert::*; diff --git a/futures-test/src/task/context.rs b/futures-test/src/task/context.rs index 602127c657..b2b0dfe31e 100644 --- a/futures-test/src/task/context.rs +++ b/futures-test/src/task/context.rs @@ -1,4 +1,4 @@ -use crate::task::{panic_waker_ref, noop_waker_ref}; +use crate::task::{noop_waker_ref, panic_waker_ref}; use futures_core::task::Context; /// Create a new [`Context`](core::task::Context) where the diff --git a/futures-test/src/task/mod.rs b/futures-test/src/task/mod.rs index 89e6aef704..ef6dd1cea4 100644 --- a/futures-test/src/task/mod.rs +++ b/futures-test/src/task/mod.rs @@ -57,4 +57,4 @@ mod record_spawner; pub use self::record_spawner::RecordSpawner; mod wake_counter; -pub use self::wake_counter::{AwokenCount, new_count_waker}; +pub use self::wake_counter::{new_count_waker, AwokenCount}; diff --git a/futures-test/src/task/panic_waker.rs b/futures-test/src/task/panic_waker.rs index d38d215958..ba6188904f 100644 --- a/futures-test/src/task/panic_waker.rs +++ b/futures-test/src/task/panic_waker.rs @@ -1,5 +1,5 @@ -use futures_core::task::{Waker, RawWaker, RawWakerVTable}; use core::ptr::null; +use futures_core::task::{RawWaker, RawWakerVTable, Waker}; use once_cell::sync::Lazy; unsafe fn clone_panic_waker(_data: *const ()) -> RawWaker { @@ -9,17 +9,13 @@ unsafe fn clone_panic_waker(_data: *const ()) -> RawWaker { unsafe fn noop(_data: *const ()) {} unsafe fn wake_panic(_data: *const ()) { - if ! std::thread::panicking() { + if !std::thread::panicking() { panic!("should not be woken"); } } -const PANIC_WAKER_VTABLE: RawWakerVTable = RawWakerVTable::new( - clone_panic_waker, - wake_panic, - wake_panic, - noop, -); +const PANIC_WAKER_VTABLE: RawWakerVTable = + RawWakerVTable::new(clone_panic_waker, wake_panic, wake_panic, noop); fn raw_panic_waker() -> RawWaker { RawWaker::new(null(), &PANIC_WAKER_VTABLE) diff --git a/futures-test/src/task/wake_counter.rs b/futures-test/src/task/wake_counter.rs index cf496c2ddb..52c63e1cc9 100644 --- a/futures-test/src/task/wake_counter.rs +++ b/futures-test/src/task/wake_counter.rs @@ -1,7 +1,7 @@ use futures_core::task::Waker; use futures_util::task::{self, ArcWake}; -use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::Arc; /// Number of times the waker was awoken. /// diff --git a/futures-test/src/track_closed.rs b/futures-test/src/track_closed.rs index 31e404c226..be883b1491 100644 --- a/futures-test/src/track_closed.rs +++ b/futures-test/src/track_closed.rs @@ -21,10 +21,7 @@ pub struct TrackClosed { impl TrackClosed { pub(crate) fn new(inner: T) -> Self { - Self { - inner, - closed: false, - } + Self { inner, closed: false } } /// Check whether this object has been closed. diff --git a/futures-util/benches/futures_unordered.rs b/futures-util/benches/futures_unordered.rs index 05e9eadb79..d5fe7a59de 100644 --- a/futures-util/benches/futures_unordered.rs +++ b/futures-util/benches/futures_unordered.rs @@ -6,7 +6,7 @@ use crate::test::Bencher; use futures::channel::oneshot; use futures::executor::block_on; use futures::future; -use futures::stream::{StreamExt, FuturesUnordered}; +use futures::stream::{FuturesUnordered, StreamExt}; use futures::task::Poll; use std::collections::VecDeque; use std::thread; @@ -34,7 +34,7 @@ fn oneshots(b: &mut Bencher) { block_on(future::poll_fn(move |cx| { loop { if let Poll::Ready(None) = rxs.poll_next_unpin(cx) { - break + break; } } Poll::Ready(()) diff --git a/futures-util/src/async_await/mod.rs b/futures-util/src/async_await/mod.rs index bdaed95b0c..5f5d4aca3f 100644 --- a/futures-util/src/async_await/mod.rs +++ b/futures-util/src/async_await/mod.rs @@ -3,8 +3,8 @@ //! This module contains a number of functions and combinators for working //! with `async`/`await` code. -use futures_core::future::{Future, FusedFuture}; -use futures_core::stream::{Stream, FusedStream}; +use futures_core::future::{FusedFuture, Future}; +use futures_core::stream::{FusedStream, Stream}; #[macro_use] mod poll; diff --git a/futures-util/src/async_await/pending.rs b/futures-util/src/async_await/pending.rs index e0cf341a8b..5d7a431811 100644 --- a/futures-util/src/async_await/pending.rs +++ b/futures-util/src/async_await/pending.rs @@ -16,7 +16,7 @@ use futures_core::task::{Context, Poll}; macro_rules! pending { () => { $crate::__private::async_await::pending_once().await - } + }; } #[doc(hidden)] diff --git a/futures-util/src/async_await/poll.rs b/futures-util/src/async_await/poll.rs index ac70a53a17..4a6e7825e2 100644 --- a/futures-util/src/async_await/poll.rs +++ b/futures-util/src/async_await/poll.rs @@ -13,7 +13,7 @@ use futures_core::task::{Context, Poll}; macro_rules! poll { ($x:expr $(,)?) => { $crate::__private::async_await::poll($x).await - } + }; } #[doc(hidden)] diff --git a/futures-util/src/compat/compat01as03.rs b/futures-util/src/compat/compat01as03.rs index bc3aee3c0a..17239a4e57 100644 --- a/futures-util/src/compat/compat01as03.rs +++ b/futures-util/src/compat/compat01as03.rs @@ -1,18 +1,15 @@ use futures_01::executor::{ - spawn as spawn01, Notify as Notify01, NotifyHandle as NotifyHandle01, - Spawn as Spawn01, UnsafeNotify as UnsafeNotify01, -}; -use futures_01::{ - Async as Async01, Future as Future01, - Stream as Stream01, + spawn as spawn01, Notify as Notify01, NotifyHandle as NotifyHandle01, Spawn as Spawn01, + UnsafeNotify as UnsafeNotify01, }; +use futures_01::{Async as Async01, Future as Future01, Stream as Stream01}; #[cfg(feature = "sink")] use futures_01::{AsyncSink as AsyncSink01, Sink as Sink01}; -use futures_core::{task as task03, future::Future as Future03, stream::Stream as Stream03}; -use std::pin::Pin; -use std::task::Context; +use futures_core::{future::Future as Future03, stream::Stream as Stream03, task as task03}; #[cfg(feature = "sink")] use futures_sink::Sink as Sink03; +use std::pin::Pin; +use std::task::Context; #[cfg(feature = "io-compat")] #[cfg_attr(docsrs, doc(cfg(feature = "io-compat")))] @@ -33,9 +30,7 @@ impl Compat01As03 { /// Wraps a futures 0.1 Future, Stream, AsyncRead, or AsyncWrite /// object in a futures 0.3-compatible wrapper. pub fn new(object: T) -> Self { - Self { - inner: spawn01(object), - } + Self { inner: spawn01(object) } } fn in_notify(&mut self, cx: &mut Context<'_>, f: impl FnOnce(&mut T) -> R) -> R { @@ -157,10 +152,7 @@ fn poll_01_to_03(x: Result, E>) -> task03::Poll> { impl Future03 for Compat01As03 { type Output = Result; - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> task03::Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> task03::Poll { poll_01_to_03(self.in_notify(cx, Future01::poll)) } } @@ -198,18 +190,10 @@ impl Unpin for Compat01As03Sink {} impl Compat01As03Sink { /// Wraps a futures 0.1 Sink object in a futures 0.3-compatible wrapper. pub fn new(inner: S) -> Self { - Self { - inner: spawn01(inner), - buffer: None, - close_started: false - } + Self { inner: spawn01(inner), buffer: None, close_started: false } } - fn in_notify( - &mut self, - cx: &mut Context<'_>, - f: impl FnOnce(&mut S) -> R, - ) -> R { + fn in_notify(&mut self, cx: &mut Context<'_>, f: impl FnOnce(&mut S) -> R) -> R { let notify = &WakerToHandle(cx.waker()); self.inner.poll_fn_notify(notify, 0, f) } @@ -256,10 +240,7 @@ where { type Error = S::SinkError; - fn start_send( - mut self: Pin<&mut Self>, - item: SinkItem, - ) -> Result<(), Self::Error> { + fn start_send(mut self: Pin<&mut Self>, item: SinkItem) -> Result<(), Self::Error> { debug_assert!(self.buffer.is_none()); self.buffer = Some(item); Ok(()) @@ -289,9 +270,7 @@ where match self.in_notify(cx, |f| match item { Some(i) => match f.start_send(i)? { AsyncSink01::Ready => f.poll_complete().map(|i| (i, None)), - AsyncSink01::NotReady(t) => { - Ok((Async01::NotReady, Some(t))) - } + AsyncSink01::NotReady(t) => Ok((Async01::NotReady, Some(t))), }, None => f.poll_complete().map(|i| (i, None)), })? { @@ -447,29 +426,35 @@ mod io { } } - fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) - -> task03::Poll> - { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> task03::Poll> { poll_01_to_03(self.in_notify(cx, |x| x.poll_read(buf))) } } impl AsyncWrite03 for Compat01As03 { - fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) - -> task03::Poll> - { + fn poll_write( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> task03::Poll> { poll_01_to_03(self.in_notify(cx, |x| x.poll_write(buf))) } - fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) - -> task03::Poll> - { + fn poll_flush( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> task03::Poll> { poll_01_to_03(self.in_notify(cx, AsyncWrite01::poll_flush)) } - fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) - -> task03::Poll> - { + fn poll_close( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> task03::Poll> { poll_01_to_03(self.in_notify(cx, AsyncWrite01::shutdown)) } } diff --git a/futures-util/src/compat/compat03as01.rs b/futures-util/src/compat/compat03as01.rs index 3f1eebbf1d..2573fe7a74 100644 --- a/futures-util/src/compat/compat03as01.rs +++ b/futures-util/src/compat/compat03as01.rs @@ -1,31 +1,19 @@ +use crate::task::{self as task03, ArcWake as ArcWake03, WakerRef}; use futures_01::{ - task as task01, Async as Async01, Future as Future01, Poll as Poll01, - Stream as Stream01, + task as task01, Async as Async01, Future as Future01, Poll as Poll01, Stream as Stream01, }; #[cfg(feature = "sink")] -use futures_01::{ - AsyncSink as AsyncSink01, Sink as Sink01, StartSend as StartSend01, -}; +use futures_01::{AsyncSink as AsyncSink01, Sink as Sink01, StartSend as StartSend01}; use futures_core::{ - task::{RawWaker, RawWakerVTable}, future::TryFuture as TryFuture03, stream::TryStream as TryStream03, + task::{RawWaker, RawWakerVTable}, }; #[cfg(feature = "sink")] use futures_sink::Sink as Sink03; -use crate::task::{ - self as task03, - ArcWake as ArcWake03, - WakerRef, -}; #[cfg(feature = "sink")] use std::marker::PhantomData; -use std::{ - mem, - pin::Pin, - sync::Arc, - task::Context, -}; +use std::{mem, pin::Pin, sync::Arc, task::Context}; /// Converts a futures 0.3 [`TryFuture`](futures_core::future::TryFuture) or /// [`TryStream`](futures_core::stream::TryStream) into a futures 0.1 @@ -80,10 +68,7 @@ impl Compat { impl CompatSink { /// Creates a new [`CompatSink`]. pub fn new(inner: T) -> Self { - Self { - inner, - _phantom: PhantomData, - } + Self { inner, _phantom: PhantomData } } /// Get a reference to 0.3 Sink contained within. @@ -102,9 +87,7 @@ impl CompatSink { } } -fn poll_03_to_01(x: task03::Poll>) - -> Result, E> -{ +fn poll_03_to_01(x: task03::Poll>) -> Result, E> { match x? { task03::Poll::Ready(t) => Ok(Async01::Ready(t)), task03::Poll::Pending => Ok(Async01::NotReady), @@ -147,17 +130,10 @@ where type SinkItem = Item; type SinkError = T::Error; - fn start_send( - &mut self, - item: Self::SinkItem, - ) -> StartSend01 { - with_sink_context(self, |mut inner, cx| { - match inner.as_mut().poll_ready(cx)? { - task03::Poll::Ready(()) => { - inner.start_send(item).map(|()| AsyncSink01::Ready) - } - task03::Poll::Pending => Ok(AsyncSink01::NotReady(item)), - } + fn start_send(&mut self, item: Self::SinkItem) -> StartSend01 { + with_sink_context(self, |mut inner, cx| match inner.as_mut().poll_ready(cx)? { + task03::Poll::Ready(()) => inner.start_send(item).map(|()| AsyncSink01::Ready), + task03::Poll::Pending => Ok(AsyncSink01::NotReady(item)), }) } @@ -190,9 +166,9 @@ impl Current { // Lazily create the `Arc` only when the waker is actually cloned. // FIXME: remove `transmute` when a `Waker` -> `RawWaker` conversion // function is landed in `core`. - mem::transmute::( - task03::waker(Arc::new(ptr_to_current(ptr).clone())) - ) + mem::transmute::(task03::waker(Arc::new( + ptr_to_current(ptr).clone(), + ))) } unsafe fn drop(_: *const ()) {} unsafe fn wake(ptr: *const ()) { @@ -243,9 +219,7 @@ mod io { use futures_io::{AsyncRead as AsyncRead03, AsyncWrite as AsyncWrite03}; use tokio_io::{AsyncRead as AsyncRead01, AsyncWrite as AsyncWrite01}; - fn poll_03_to_io(x: task03::Poll>) - -> Result - { + fn poll_03_to_io(x: task03::Poll>) -> Result { match x { task03::Poll::Ready(Ok(t)) => Ok(t), task03::Poll::Pending => Err(std::io::ErrorKind::WouldBlock.into()), diff --git a/futures-util/src/compat/executor.rs b/futures-util/src/compat/executor.rs index 82cb496a70..e25705be16 100644 --- a/futures-util/src/compat/executor.rs +++ b/futures-util/src/compat/executor.rs @@ -66,9 +66,7 @@ where fn spawn_obj(&self, future: FutureObj<'static, ()>) -> Result<(), SpawnError03> { let future = future.unit_error().compat(); - self.executor01 - .execute(future) - .map_err(|_| SpawnError03::shutdown()) + self.executor01.execute(future).map_err(|_| SpawnError03::shutdown()) } } diff --git a/futures-util/src/compat/mod.rs b/futures-util/src/compat/mod.rs index 897a50a4fd..aed704eabe 100644 --- a/futures-util/src/compat/mod.rs +++ b/futures-util/src/compat/mod.rs @@ -4,16 +4,16 @@ //! library is activated. mod executor; -pub use self::executor::{Executor01CompatExt, Executor01Future, Executor01As03}; +pub use self::executor::{Executor01As03, Executor01CompatExt, Executor01Future}; mod compat01as03; +#[cfg(feature = "io-compat")] +#[cfg_attr(docsrs, doc(cfg(feature = "io-compat")))] +pub use self::compat01as03::{AsyncRead01CompatExt, AsyncWrite01CompatExt}; pub use self::compat01as03::{Compat01As03, Future01CompatExt, Stream01CompatExt}; #[cfg(feature = "sink")] #[cfg_attr(docsrs, doc(cfg(feature = "sink")))] pub use self::compat01as03::{Compat01As03Sink, Sink01CompatExt}; -#[cfg(feature = "io-compat")] -#[cfg_attr(docsrs, doc(cfg(feature = "io-compat")))] -pub use self::compat01as03::{AsyncRead01CompatExt, AsyncWrite01CompatExt}; mod compat03as01; pub use self::compat03as01::Compat; diff --git a/futures-util/src/fns.rs b/futures-util/src/fns.rs index cb623918dd..37ee03e6df 100644 --- a/futures-util/src/fns.rs +++ b/futures-util/src/fns.rs @@ -1,5 +1,5 @@ -use core::marker::PhantomData; use core::fmt::{self, Debug}; +use core::marker::PhantomData; pub trait FnOnce1 { type Output; @@ -8,7 +8,7 @@ pub trait FnOnce1 { impl FnOnce1 for T where - T: FnOnce(A) -> R + T: FnOnce(A) -> R, { type Output = R; fn call_once(self, arg: A) -> R { @@ -22,7 +22,7 @@ pub trait FnMut1: FnOnce1 { impl FnMut1 for T where - T: FnMut(A) -> R + T: FnMut(A) -> R, { fn call_mut(&mut self, arg: A) -> R { self(arg) @@ -37,7 +37,7 @@ pub trait Fn1: FnMut1 { impl Fn1 for T where - T: Fn(A) -> R + T: Fn(A) -> R, { fn call(&self, arg: A) -> R { self(arg) @@ -143,7 +143,7 @@ pub struct InspectFn(F); #[allow(single_use_lifetimes)] // https://github.com/rust-lang/rust/issues/55058 impl FnOnce1 for InspectFn where - F: for<'a> FnOnce1<&'a A, Output=()>, + F: for<'a> FnOnce1<&'a A, Output = ()>, { type Output = A; fn call_once(self, arg: A) -> Self::Output { @@ -154,7 +154,7 @@ where #[allow(single_use_lifetimes)] // https://github.com/rust-lang/rust/issues/55058 impl FnMut1 for InspectFn where - F: for<'a> FnMut1<&'a A, Output=()>, + F: for<'a> FnMut1<&'a A, Output = ()>, { fn call_mut(&mut self, arg: A) -> Self::Output { self.0.call_mut(&arg); @@ -164,7 +164,7 @@ where #[allow(single_use_lifetimes)] // https://github.com/rust-lang/rust/issues/55058 impl Fn1 for InspectFn where - F: for<'a> Fn1<&'a A, Output=()>, + F: for<'a> Fn1<&'a A, Output = ()>, { fn call(&self, arg: A) -> Self::Output { self.0.call(&arg); @@ -244,27 +244,33 @@ pub struct InspectOkFn(F); impl<'a, F, T, E> FnOnce1<&'a Result> for InspectOkFn where - F: FnOnce1<&'a T, Output=()> + F: FnOnce1<&'a T, Output = ()>, { type Output = (); fn call_once(self, arg: &'a Result) -> Self::Output { - if let Ok(x) = arg { self.0.call_once(x) } + if let Ok(x) = arg { + self.0.call_once(x) + } } } impl<'a, F, T, E> FnMut1<&'a Result> for InspectOkFn where - F: FnMut1<&'a T, Output=()>, + F: FnMut1<&'a T, Output = ()>, { fn call_mut(&mut self, arg: &'a Result) -> Self::Output { - if let Ok(x) = arg { self.0.call_mut(x) } + if let Ok(x) = arg { + self.0.call_mut(x) + } } } impl<'a, F, T, E> Fn1<&'a Result> for InspectOkFn where - F: Fn1<&'a T, Output=()>, + F: Fn1<&'a T, Output = ()>, { fn call(&self, arg: &'a Result) -> Self::Output { - if let Ok(x) = arg { self.0.call(x) } + if let Ok(x) = arg { + self.0.call(x) + } } } pub(crate) fn inspect_ok_fn(f: F) -> InspectOkFn { @@ -276,27 +282,33 @@ pub struct InspectErrFn(F); impl<'a, F, T, E> FnOnce1<&'a Result> for InspectErrFn where - F: FnOnce1<&'a E, Output=()> + F: FnOnce1<&'a E, Output = ()>, { type Output = (); fn call_once(self, arg: &'a Result) -> Self::Output { - if let Err(x) = arg { self.0.call_once(x) } + if let Err(x) = arg { + self.0.call_once(x) + } } } impl<'a, F, T, E> FnMut1<&'a Result> for InspectErrFn where - F: FnMut1<&'a E, Output=()>, + F: FnMut1<&'a E, Output = ()>, { fn call_mut(&mut self, arg: &'a Result) -> Self::Output { - if let Err(x) = arg { self.0.call_mut(x) } + if let Err(x) = arg { + self.0.call_mut(x) + } } } impl<'a, F, T, E> Fn1<&'a Result> for InspectErrFn where - F: Fn1<&'a E, Output=()>, + F: Fn1<&'a E, Output = ()>, { fn call(&self, arg: &'a Result) -> Self::Output { - if let Err(x) = arg { self.0.call(x) } + if let Err(x) = arg { + self.0.call(x) + } } } pub(crate) fn inspect_err_fn(f: F) -> InspectErrFn { @@ -313,7 +325,7 @@ pub struct UnwrapOrElseFn(F); impl FnOnce1> for UnwrapOrElseFn where - F: FnOnce1, + F: FnOnce1, { type Output = T; fn call_once(self, arg: Result) -> Self::Output { @@ -322,7 +334,7 @@ where } impl FnMut1> for UnwrapOrElseFn where - F: FnMut1, + F: FnMut1, { fn call_mut(&mut self, arg: Result) -> Self::Output { arg.unwrap_or_else(|x| self.0.call_mut(x)) @@ -330,7 +342,7 @@ where } impl Fn1> for UnwrapOrElseFn where - F: Fn1, + F: Fn1, { fn call(&self, arg: Result) -> Self::Output { arg.unwrap_or_else(|x| self.0.call(x)) @@ -347,7 +359,10 @@ impl Default for IntoFn { Self(PhantomData) } } -impl FnOnce1 for IntoFn where A: Into { +impl FnOnce1 for IntoFn +where + A: Into, +{ type Output = T; fn call_once(self, arg: A) -> Self::Output { arg.into() diff --git a/futures-util/src/future/future/catch_unwind.rs b/futures-util/src/future/future/catch_unwind.rs index 123f5e3517..b6ea112cac 100644 --- a/futures-util/src/future/future/catch_unwind.rs +++ b/futures-util/src/future/future/catch_unwind.rs @@ -1,6 +1,6 @@ use core::any::Any; use core::pin::Pin; -use std::panic::{catch_unwind, UnwindSafe, AssertUnwindSafe}; +use std::panic::{catch_unwind, AssertUnwindSafe, UnwindSafe}; use futures_core::future::Future; use futures_core::task::{Context, Poll}; @@ -12,14 +12,18 @@ use pin_project::pin_project; #[must_use = "futures do nothing unless you `.await` or poll them"] pub struct CatchUnwind(#[pin] Fut); -impl CatchUnwind where Fut: Future + UnwindSafe { +impl CatchUnwind +where + Fut: Future + UnwindSafe, +{ pub(super) fn new(future: Fut) -> Self { Self(future) } } impl Future for CatchUnwind - where Fut: Future + UnwindSafe, +where + Fut: Future + UnwindSafe, { type Output = Result>; diff --git a/futures-util/src/future/future/flatten.rs b/futures-util/src/future/future/flatten.rs index d621c1eb63..8b0d16df84 100644 --- a/futures-util/src/future/future/flatten.rs +++ b/futures-util/src/future/future/flatten.rs @@ -1,9 +1,9 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::{FusedStream, Stream}; +use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; -use futures_core::task::{Context, Poll}; use pin_project::pin_project; #[pin_project(project = FlattenProj)] @@ -21,8 +21,9 @@ impl Flatten { } impl FusedFuture for Flatten - where Fut: Future, - Fut::Output: Future, +where + Fut: Future, + Fut::Output: Future, { fn is_terminated(&self) -> bool { match self { @@ -33,8 +34,9 @@ impl FusedFuture for Flatten } impl Future for Flatten - where Fut: Future, - Fut::Output: Future, +where + Fut: Future, + Fut::Output: Future, { type Output = ::Output; @@ -44,12 +46,12 @@ impl Future for Flatten FlattenProj::First(f) => { let f = ready!(f.poll(cx)); self.set(Self::Second(f)); - }, + } FlattenProj::Second(f) => { let output = ready!(f.poll(cx)); self.set(Self::Empty); break output; - }, + } FlattenProj::Empty => panic!("Flatten polled after completion"), } }) @@ -57,8 +59,9 @@ impl Future for Flatten } impl FusedStream for Flatten - where Fut: Future, - Fut::Output: Stream, +where + Fut: Future, + Fut::Output: Stream, { fn is_terminated(&self) -> bool { match self { @@ -69,8 +72,9 @@ impl FusedStream for Flatten } impl Stream for Flatten - where Fut: Future, - Fut::Output: Stream, +where + Fut: Future, + Fut::Output: Stream, { type Item = ::Item; @@ -80,21 +84,20 @@ impl Stream for Flatten FlattenProj::First(f) => { let f = ready!(f.poll(cx)); self.set(Self::Second(f)); - }, + } FlattenProj::Second(f) => { let output = ready!(f.poll_next(cx)); if output.is_none() { self.set(Self::Empty); } break output; - }, + } FlattenProj::Empty => break None, } }) } } - #[cfg(feature = "sink")] impl Sink for Flatten where @@ -103,19 +106,16 @@ where { type Error = >::Error; - fn poll_ready( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { Poll::Ready(loop { match self.as_mut().project() { FlattenProj::First(f) => { let f = ready!(f.poll(cx)); self.set(Self::Second(f)); - }, + } FlattenProj::Second(f) => { break ready!(f.poll_ready(cx)); - }, + } FlattenProj::Empty => panic!("poll_ready called after eof"), } }) @@ -137,10 +137,7 @@ where } } - fn poll_close( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let res = match self.as_mut().project() { FlattenProj::Second(f) => f.poll_close(cx), _ => Poll::Ready(Ok(())), diff --git a/futures-util/src/future/future/fuse.rs b/futures-util/src/future/future/fuse.rs index f583586029..e2d667de9a 100644 --- a/futures-util/src/future/future/fuse.rs +++ b/futures-util/src/future/future/fuse.rs @@ -1,5 +1,5 @@ use core::pin::Pin; -use futures_core::future::{Future, FusedFuture}; +use futures_core::future::{FusedFuture, Future}; use futures_core::task::{Context, Poll}; use pin_project::pin_project; @@ -81,7 +81,7 @@ impl Future for Fuse { let output = ready!(fut.poll(cx)); self.project().0.set(None); output - }, + } None => return Poll::Pending, }) } diff --git a/futures-util/src/future/future/map.rs b/futures-util/src/future/future/map.rs index e7d9a73f80..2088a3479e 100644 --- a/futures-util/src/future/future/map.rs +++ b/futures-util/src/future/future/map.rs @@ -26,8 +26,9 @@ impl Map { } impl FusedFuture for Map - where Fut: Future, - F: FnOnce1, +where + Fut: Future, + F: FnOnce1, { fn is_terminated(&self) -> bool { match self { @@ -38,8 +39,9 @@ impl FusedFuture for Map } impl Future for Map - where Fut: Future, - F: FnOnce1, +where + Fut: Future, + F: FnOnce1, { type Output = T; @@ -51,7 +53,7 @@ impl Future for Map MapProjOwn::Incomplete { f, .. } => Poll::Ready(f.call_once(output)), MapProjOwn::Complete => unreachable!(), } - }, + } MapProj::Complete => panic!("Map must not be polled after it returned `Poll::Ready`"), } } diff --git a/futures-util/src/future/future/mod.rs b/futures-util/src/future/future/mod.rs index f01d3462f7..35bf6860d2 100644 --- a/futures-util/src/future/future/mod.rs +++ b/futures-util/src/future/future/mod.rs @@ -7,10 +7,10 @@ use alloc::boxed::Box; use core::pin::Pin; -use crate::future::{assert_future, Either}; -use crate::stream::assert_stream; use crate::fns::{inspect_fn, into_fn, ok_fn, InspectFn, IntoFn, OkFn}; +use crate::future::{assert_future, Either}; use crate::never::Never; +use crate::stream::assert_stream; #[cfg(feature = "alloc")] use futures_core::future::{BoxFuture, LocalBoxFuture}; use futures_core::{ diff --git a/futures-util/src/future/future/remote_handle.rs b/futures-util/src/future/future/remote_handle.rs index 598f63c5e7..ab97916c7f 100644 --- a/futures-util/src/future/future/remote_handle.rs +++ b/futures-util/src/future/future/remote_handle.rs @@ -1,22 +1,22 @@ use { crate::future::{CatchUnwind, FutureExt}, - futures_channel::oneshot::{self, Sender, Receiver}, + futures_channel::oneshot::{self, Receiver, Sender}, futures_core::{ future::Future, task::{Context, Poll}, }, + pin_project::pin_project, std::{ any::Any, fmt, panic::{self, AssertUnwindSafe}, pin::Pin, sync::{ - Arc, atomic::{AtomicBool, Ordering}, + Arc, }, thread, }, - pin_project::pin_project, }; /// The handle to a remote future returned by @@ -83,9 +83,7 @@ pub struct Remote { impl fmt::Debug for Remote { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_tuple("Remote") - .field(&self.future) - .finish() + f.debug_tuple("Remote").field(&self.future).finish() } } @@ -98,7 +96,7 @@ impl Future for Remote { if let Poll::Ready(_) = this.tx.as_mut().unwrap().poll_canceled(cx) { if !this.keep_running.load(Ordering::SeqCst) { // Cancelled, bail out - return Poll::Ready(()) + return Poll::Ready(()); } } diff --git a/futures-util/src/future/future/shared.rs b/futures-util/src/future/future/shared.rs index 66fc5d3406..3ccb67d74b 100644 --- a/futures-util/src/future/future/shared.rs +++ b/futures-util/src/future/future/shared.rs @@ -90,10 +90,7 @@ impl Shared { }), }; - Self { - inner: Some(Arc::new(inner)), - waker_key: NULL_WAKER_KEY, - } + Self { inner: Some(Arc::new(inner)), waker_key: NULL_WAKER_KEY } } } @@ -197,10 +194,7 @@ where fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = &mut *self; - let inner = this - .inner - .take() - .expect("Shared future polled again after completion"); + let inner = this.inner.take().expect("Shared future polled again after completion"); // Fast path for when the wrapped future has already completed if inner.notifier.state.load(Acquire) == COMPLETE { @@ -297,10 +291,7 @@ where Fut: Future, { fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - waker_key: NULL_WAKER_KEY, - } + Self { inner: self.inner.clone(), waker_key: NULL_WAKER_KEY } } } @@ -334,16 +325,12 @@ impl ArcWake for Notifier { } } -impl WeakShared -{ +impl WeakShared { /// Attempts to upgrade this [`WeakShared`] into a [`Shared`]. /// /// Returns [`None`] if all clones of the [`Shared`] have been dropped or polled /// to completion. pub fn upgrade(&self) -> Option> { - Some(Shared { - inner: Some(self.0.upgrade()?), - waker_key: NULL_WAKER_KEY, - }) + Some(Shared { inner: Some(self.0.upgrade()?), waker_key: NULL_WAKER_KEY }) } } diff --git a/futures-util/src/future/join.rs b/futures-util/src/future/join.rs index 0005b08173..1f01a169b0 100644 --- a/futures-util/src/future/join.rs +++ b/futures-util/src/future/join.rs @@ -1,9 +1,9 @@ #![allow(non_snake_case)] -use crate::future::{MaybeDone, maybe_done}; +use crate::future::{maybe_done, MaybeDone}; use core::fmt; use core::pin::Pin; -use futures_core::future::{Future, FusedFuture}; +use futures_core::future::{FusedFuture, Future}; use futures_core::task::{Context, Poll}; use pin_project::pin_project; diff --git a/futures-util/src/future/join_all.rs b/futures-util/src/future/join_all.rs index 0c8357ca68..7e505c59ab 100644 --- a/futures-util/src/future/join_all.rs +++ b/futures-util/src/future/join_all.rs @@ -1,14 +1,14 @@ //! Definition of the `JoinAll` combinator, waiting for all of a list of futures //! to finish. +use alloc::boxed::Box; +use alloc::vec::Vec; use core::fmt; use core::future::Future; use core::iter::FromIterator; use core::mem; use core::pin::Pin; use core::task::{Context, Poll}; -use alloc::boxed::Box; -use alloc::vec::Vec; use super::MaybeDone; @@ -16,9 +16,7 @@ fn iter_pin_mut(slice: Pin<&mut [T]>) -> impl Iterator> { // Safety: `std` _could_ make this unsound if it were to decide Pin's // invariants aren't required to transmit through slices. Otherwise this has // the same safety as a normal field pin projection. - unsafe { slice.get_unchecked_mut() } - .iter_mut() - .map(|t| unsafe { Pin::new_unchecked(t) }) + unsafe { slice.get_unchecked_mut() }.iter_mut().map(|t| unsafe { Pin::new_unchecked(t) }) } /// Future for the [`join_all`] function. @@ -36,9 +34,7 @@ where F::Output: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("JoinAll") - .field("elems", &self.elems) - .finish() + f.debug_struct("JoinAll").field("elems", &self.elems).finish() } } @@ -105,9 +101,7 @@ where if all_done { let mut elems = mem::replace(&mut self.elems, Box::pin([])); - let result = iter_pin_mut(elems.as_mut()) - .map(|e| e.take_output().unwrap()) - .collect(); + let result = iter_pin_mut(elems.as_mut()).map(|e| e.take_output().unwrap()).collect(); Poll::Ready(result) } else { Poll::Pending diff --git a/futures-util/src/future/lazy.rs b/futures-util/src/future/lazy.rs index 409717a2ab..a46c94f1fa 100644 --- a/futures-util/src/future/lazy.rs +++ b/futures-util/src/future/lazy.rs @@ -6,7 +6,7 @@ use futures_core::task::{Context, Poll}; #[derive(Debug)] #[must_use = "futures do nothing unless you `.await` or poll them"] pub struct Lazy { - f: Option + f: Option, } // safe because we never generate `Pin<&mut F>` @@ -32,19 +32,24 @@ impl Unpin for Lazy {} /// # }); /// ``` pub fn lazy(f: F) -> Lazy - where F: FnOnce(&mut Context<'_>) -> R, +where + F: FnOnce(&mut Context<'_>) -> R, { Lazy { f: Some(f) } } impl FusedFuture for Lazy - where F: FnOnce(&mut Context<'_>) -> R, +where + F: FnOnce(&mut Context<'_>) -> R, { - fn is_terminated(&self) -> bool { self.f.is_none() } + fn is_terminated(&self) -> bool { + self.f.is_none() + } } impl Future for Lazy - where F: FnOnce(&mut Context<'_>) -> R, +where + F: FnOnce(&mut Context<'_>) -> R, { type Output = R; diff --git a/futures-util/src/future/mod.rs b/futures-util/src/future/mod.rs index 6a497477e5..0f4e149e24 100644 --- a/futures-util/src/future/mod.rs +++ b/futures-util/src/future/mod.rs @@ -13,7 +13,7 @@ pub use futures_task::{FutureObj, LocalFutureObj, UnsafeFutureObj}; #[allow(clippy::module_inception)] mod future; pub use self::future::{ - Flatten, Fuse, FutureExt, Inspect, IntoStream, Map, NeverError, Then, UnitError, MapInto, + Flatten, Fuse, FutureExt, Inspect, IntoStream, Map, MapInto, NeverError, Then, UnitError, }; #[deprecated(note = "This is now an alias for [Flatten](Flatten)")] @@ -32,8 +32,8 @@ pub use self::future::{Shared, WeakShared}; mod try_future; pub use self::try_future::{ - AndThen, ErrInto, OkInto, InspectErr, InspectOk, IntoFuture, MapErr, MapOk, OrElse, TryFlattenStream, - TryFutureExt, UnwrapOrElse, MapOkOrElse, TryFlatten, + AndThen, ErrInto, InspectErr, InspectOk, IntoFuture, MapErr, MapOk, MapOkOrElse, OkInto, + OrElse, TryFlatten, TryFlattenStream, TryFutureExt, UnwrapOrElse, }; #[cfg(feature = "sink")] diff --git a/futures-util/src/future/option.rs b/futures-util/src/future/option.rs index b58911264c..4b959aca91 100644 --- a/futures-util/src/future/option.rs +++ b/futures-util/src/future/option.rs @@ -1,7 +1,7 @@ //! Definition of the `Option` (optional step) combinator use core::pin::Pin; -use futures_core::future::{Future, FusedFuture}; +use futures_core::future::{FusedFuture, Future}; use futures_core::task::{Context, Poll}; use pin_project::pin_project; @@ -30,10 +30,7 @@ pub struct OptionFuture(#[pin] Option); impl Future for OptionFuture { type Output = Option; - fn poll( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { match self.project().0.as_pin_mut() { Some(x) => x.poll(cx).map(Some), None => Poll::Ready(None), diff --git a/futures-util/src/future/pending.rs b/futures-util/src/future/pending.rs index 5a7bbb8d59..813db073f2 100644 --- a/futures-util/src/future/pending.rs +++ b/futures-util/src/future/pending.rs @@ -33,9 +33,7 @@ impl FusedFuture for Pending { /// # }); /// ``` pub fn pending() -> Pending { - Pending { - _data: marker::PhantomData, - } + Pending { _data: marker::PhantomData } } impl Future for Pending { @@ -46,8 +44,7 @@ impl Future for Pending { } } -impl Unpin for Pending { -} +impl Unpin for Pending {} impl Clone for Pending { fn clone(&self) -> Self { diff --git a/futures-util/src/future/poll_fn.rs b/futures-util/src/future/poll_fn.rs index b7b10be85d..c223f9ac2c 100644 --- a/futures-util/src/future/poll_fn.rs +++ b/futures-util/src/future/poll_fn.rs @@ -34,7 +34,7 @@ impl Unpin for PollFn {} /// ``` pub fn poll_fn(f: F) -> PollFn where - F: FnMut(&mut Context<'_>) -> Poll + F: FnMut(&mut Context<'_>) -> Poll, { PollFn { f } } @@ -46,7 +46,8 @@ impl fmt::Debug for PollFn { } impl Future for PollFn - where F: FnMut(&mut Context<'_>) -> Poll, +where + F: FnMut(&mut Context<'_>) -> Poll, { type Output = T; diff --git a/futures-util/src/future/select.rs b/futures-util/src/future/select.rs index bc247796dd..0d4947971f 100644 --- a/futures-util/src/future/select.rs +++ b/futures-util/src/future/select.rs @@ -1,7 +1,7 @@ +use crate::future::{Either, FutureExt}; use core::pin::Pin; -use futures_core::future::{Future, FusedFuture}; +use futures_core::future::{FusedFuture, Future}; use futures_core::task::{Context, Poll}; -use crate::future::{Either, FutureExt}; /// Future for the [`select()`] function. #[must_use = "futures do nothing unless you `.await` or poll them"] @@ -73,7 +73,9 @@ impl Unpin for Select {} /// } /// ``` pub fn select(future1: A, future2: B) -> Select - where A: Future + Unpin, B: Future + Unpin +where + A: Future + Unpin, + B: Future + Unpin, { Select { inner: Some((future1, future2)) } } @@ -95,7 +97,7 @@ where self.inner = Some((a, b)); Poll::Pending } - } + }, } } } diff --git a/futures-util/src/future/select_all.rs b/futures-util/src/future/select_all.rs index 9f7fb245bf..7fdf37ad73 100644 --- a/futures-util/src/future/select_all.rs +++ b/futures-util/src/future/select_all.rs @@ -1,8 +1,8 @@ use crate::future::FutureExt; +use alloc::vec::Vec; use core::iter::FromIterator; use core::mem; use core::pin::Pin; -use alloc::vec::Vec; use futures_core::future::Future; use futures_core::task::{Context, Poll}; @@ -31,12 +31,11 @@ impl Unpin for SelectAll {} /// /// This function will panic if the iterator specified contains no items. pub fn select_all(iter: I) -> SelectAll - where I: IntoIterator, - I::Item: Future + Unpin, +where + I: IntoIterator, + I::Item: Future + Unpin, { - let ret = SelectAll { - inner: iter.into_iter().collect() - }; + let ret = SelectAll { inner: iter.into_iter().collect() }; assert!(!ret.inner.is_empty()); ret } @@ -45,11 +44,9 @@ impl Future for SelectAll { type Output = (Fut::Output, usize, Vec); fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let item = self.inner.iter_mut().enumerate().find_map(|(i, f)| { - match f.poll_unpin(cx) { - Poll::Pending => None, - Poll::Ready(e) => Some((i, e)), - } + let item = self.inner.iter_mut().enumerate().find_map(|(i, f)| match f.poll_unpin(cx) { + Poll::Pending => None, + Poll::Ready(e) => Some((i, e)), }); match item { Some((idx, res)) => { diff --git a/futures-util/src/future/select_ok.rs b/futures-util/src/future/select_ok.rs index 7f4f4d65f4..9552dbdfdb 100644 --- a/futures-util/src/future/select_ok.rs +++ b/futures-util/src/future/select_ok.rs @@ -1,8 +1,8 @@ use crate::future::TryFutureExt; +use alloc::vec::Vec; use core::iter::FromIterator; use core::mem; use core::pin::Pin; -use alloc::vec::Vec; use futures_core::future::{Future, TryFuture}; use futures_core::task::{Context, Poll}; @@ -29,12 +29,11 @@ impl Unpin for SelectOk {} /// /// This function will panic if the iterator specified contains no items. pub fn select_ok(iter: I) -> SelectOk - where I: IntoIterator, - I::Item: TryFuture + Unpin, +where + I: IntoIterator, + I::Item: TryFuture + Unpin, { - let ret = SelectOk { - inner: iter.into_iter().collect() - }; + let ret = SelectOk { inner: iter.into_iter().collect() }; assert!(!ret.inner.is_empty(), "iterator provided to select_ok was empty"); ret } @@ -45,12 +44,11 @@ impl Future for SelectOk { fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { // loop until we've either exhausted all errors, a success was hit, or nothing is ready loop { - let item = self.inner.iter_mut().enumerate().find_map(|(i, f)| { - match f.try_poll_unpin(cx) { + let item = + self.inner.iter_mut().enumerate().find_map(|(i, f)| match f.try_poll_unpin(cx) { Poll::Pending => None, Poll::Ready(e) => Some((i, e)), - } - }); + }); match item { Some((idx, res)) => { // always remove Ok or Err, if it's not the last Err continue looping @@ -58,18 +56,18 @@ impl Future for SelectOk { match res { Ok(e) => { let rest = mem::replace(&mut self.inner, Vec::new()); - return Poll::Ready(Ok((e, rest))) + return Poll::Ready(Ok((e, rest))); } Err(e) => { if self.inner.is_empty() { - return Poll::Ready(Err(e)) + return Poll::Ready(Err(e)); } } } } None => { // based on the filter above, nothing is ready, return - return Poll::Pending + return Poll::Pending; } } } diff --git a/futures-util/src/future/try_future/into_future.rs b/futures-util/src/future/try_future/into_future.rs index 5c43bdd4f2..59ff266b0c 100644 --- a/futures-util/src/future/try_future/into_future.rs +++ b/futures-util/src/future/try_future/into_future.rs @@ -17,17 +17,16 @@ impl IntoFuture { } impl FusedFuture for IntoFuture { - fn is_terminated(&self) -> bool { self.0.is_terminated() } + fn is_terminated(&self) -> bool { + self.0.is_terminated() + } } impl Future for IntoFuture { type Output = Result; #[inline] - fn poll( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { self.project().0.try_poll(cx) } } diff --git a/futures-util/src/future/try_future/try_flatten.rs b/futures-util/src/future/try_future/try_flatten.rs index a0c69ca634..a82cc9f79a 100644 --- a/futures-util/src/future/try_future/try_flatten.rs +++ b/futures-util/src/future/try_future/try_flatten.rs @@ -1,9 +1,9 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; use futures_core::stream::{FusedStream, Stream, TryStream}; +use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; -use futures_core::task::{Context, Poll}; use pin_project::pin_project; #[pin_project(project = TryFlattenProj)] @@ -21,8 +21,9 @@ impl TryFlatten { } impl FusedFuture for TryFlatten - where Fut: TryFuture, - Fut::Ok: TryFuture, +where + Fut: TryFuture, + Fut::Ok: TryFuture, { fn is_terminated(&self) -> bool { match self { @@ -33,28 +34,27 @@ impl FusedFuture for TryFlatten } impl Future for TryFlatten - where Fut: TryFuture, - Fut::Ok: TryFuture, +where + Fut: TryFuture, + Fut::Ok: TryFuture, { type Output = Result<::Ok, Fut::Error>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { Poll::Ready(loop { match self.as_mut().project() { - TryFlattenProj::First(f) => { - match ready!(f.try_poll(cx)) { - Ok(f) => self.set(Self::Second(f)), - Err(e) => { - self.set(Self::Empty); - break Err(e); - } + TryFlattenProj::First(f) => match ready!(f.try_poll(cx)) { + Ok(f) => self.set(Self::Second(f)), + Err(e) => { + self.set(Self::Empty); + break Err(e); } }, TryFlattenProj::Second(f) => { let output = ready!(f.try_poll(cx)); self.set(Self::Empty); break output; - }, + } TryFlattenProj::Empty => panic!("TryFlatten polled after completion"), } }) @@ -62,8 +62,9 @@ impl Future for TryFlatten } impl FusedStream for TryFlatten - where Fut: TryFuture, - Fut::Ok: TryStream, +where + Fut: TryFuture, + Fut::Ok: TryStream, { fn is_terminated(&self) -> bool { match self { @@ -74,21 +75,20 @@ impl FusedStream for TryFlatten } impl Stream for TryFlatten - where Fut: TryFuture, - Fut::Ok: TryStream, +where + Fut: TryFuture, + Fut::Ok: TryStream, { type Item = Result<::Ok, Fut::Error>; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { Poll::Ready(loop { match self.as_mut().project() { - TryFlattenProj::First(f) => { - match ready!(f.try_poll(cx)) { - Ok(f) => self.set(Self::Second(f)), - Err(e) => { - self.set(Self::Empty); - break Some(Err(e)); - } + TryFlattenProj::First(f) => match ready!(f.try_poll(cx)) { + Ok(f) => self.set(Self::Second(f)), + Err(e) => { + self.set(Self::Empty); + break Some(Err(e)); } }, TryFlattenProj::Second(f) => { @@ -97,40 +97,34 @@ impl Stream for TryFlatten self.set(Self::Empty); } break output; - }, + } TryFlattenProj::Empty => break None, } }) } } - #[cfg(feature = "sink")] impl Sink for TryFlatten where Fut: TryFuture, - Fut::Ok: Sink, + Fut::Ok: Sink, { type Error = Fut::Error; - fn poll_ready( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { Poll::Ready(loop { match self.as_mut().project() { - TryFlattenProj::First(f) => { - match ready!(f.try_poll(cx)) { - Ok(f) => self.set(Self::Second(f)), - Err(e) => { - self.set(Self::Empty); - break Err(e); - } + TryFlattenProj::First(f) => match ready!(f.try_poll(cx)) { + Ok(f) => self.set(Self::Second(f)), + Err(e) => { + self.set(Self::Empty); + break Err(e); } }, TryFlattenProj::Second(f) => { break ready!(f.poll_ready(cx)); - }, + } TryFlattenProj::Empty => panic!("poll_ready called after eof"), } }) @@ -152,10 +146,7 @@ where } } - fn poll_close( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let res = match self.as_mut().project() { TryFlattenProj::Second(f) => f.poll_close(cx), _ => Poll::Ready(Ok(())), diff --git a/futures-util/src/future/try_future/try_flatten_err.rs b/futures-util/src/future/try_future/try_flatten_err.rs index a1383162d6..de47a4e657 100644 --- a/futures-util/src/future/try_future/try_flatten_err.rs +++ b/futures-util/src/future/try_future/try_flatten_err.rs @@ -18,8 +18,9 @@ impl TryFlattenErr { } impl FusedFuture for TryFlattenErr - where Fut: TryFuture, - Fut::Error: TryFuture, +where + Fut: TryFuture, + Fut::Error: TryFuture, { fn is_terminated(&self) -> bool { match self { @@ -30,28 +31,27 @@ impl FusedFuture for TryFlattenErr } impl Future for TryFlattenErr - where Fut: TryFuture, - Fut::Error: TryFuture, +where + Fut: TryFuture, + Fut::Error: TryFuture, { type Output = Result::Error>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { Poll::Ready(loop { match self.as_mut().project() { - TryFlattenErrProj::First(f) => { - match ready!(f.try_poll(cx)) { - Err(f) => self.set(Self::Second(f)), - Ok(e) => { - self.set(Self::Empty); - break Ok(e); - } + TryFlattenErrProj::First(f) => match ready!(f.try_poll(cx)) { + Err(f) => self.set(Self::Second(f)), + Ok(e) => { + self.set(Self::Empty); + break Ok(e); } }, TryFlattenErrProj::Second(f) => { let output = ready!(f.try_poll(cx)); self.set(Self::Empty); break output; - }, + } TryFlattenErrProj::Empty => panic!("TryFlattenErr polled after completion"), } }) diff --git a/futures-util/src/future/try_join.rs b/futures-util/src/future/try_join.rs index c12dbb139f..a6af51b4bd 100644 --- a/futures-util/src/future/try_join.rs +++ b/futures-util/src/future/try_join.rs @@ -1,6 +1,6 @@ #![allow(non_snake_case)] -use crate::future::{TryMaybeDone, try_maybe_done}; +use crate::future::{try_maybe_done, TryMaybeDone}; use core::fmt; use core::pin::Pin; use futures_core::future::{Future, TryFuture}; diff --git a/futures-util/src/future/try_join_all.rs b/futures-util/src/future/try_join_all.rs index 4de0a79b94..25fa980db7 100644 --- a/futures-util/src/future/try_join_all.rs +++ b/futures-util/src/future/try_join_all.rs @@ -1,14 +1,14 @@ //! Definition of the `TryJoinAll` combinator, waiting for all of a list of //! futures to finish with either success or error. +use alloc::boxed::Box; +use alloc::vec::Vec; use core::fmt; use core::future::Future; use core::iter::FromIterator; use core::mem; use core::pin::Pin; use core::task::{Context, Poll}; -use alloc::boxed::Box; -use alloc::vec::Vec; use super::{TryFuture, TryMaybeDone}; @@ -16,15 +16,13 @@ fn iter_pin_mut(slice: Pin<&mut [T]>) -> impl Iterator> { // Safety: `std` _could_ make this unsound if it were to decide Pin's // invariants aren't required to transmit through slices. Otherwise this has // the same safety as a normal field pin projection. - unsafe { slice.get_unchecked_mut() } - .iter_mut() - .map(|t| unsafe { Pin::new_unchecked(t) }) + unsafe { slice.get_unchecked_mut() }.iter_mut().map(|t| unsafe { Pin::new_unchecked(t) }) } enum FinalState { Pending, AllDone, - Error(E) + Error(E), } /// Future for the [`try_join_all`] function. @@ -43,9 +41,7 @@ where F::Error: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("TryJoinAll") - .field("elems", &self.elems) - .finish() + f.debug_struct("TryJoinAll").field("elems", &self.elems).finish() } } @@ -93,9 +89,7 @@ where I::Item: TryFuture, { let elems: Box<[_]> = i.into_iter().map(TryMaybeDone::Future).collect(); - TryJoinAll { - elems: elems.into(), - } + TryJoinAll { elems: elems.into() } } impl Future for TryJoinAll @@ -110,7 +104,7 @@ where for elem in iter_pin_mut(self.elems.as_mut()) { match elem.try_poll(cx) { Poll::Pending => state = FinalState::Pending, - Poll::Ready(Ok(())) => {}, + Poll::Ready(Ok(())) => {} Poll::Ready(Err(e)) => { state = FinalState::Error(e); break; @@ -122,15 +116,14 @@ where FinalState::Pending => Poll::Pending, FinalState::AllDone => { let mut elems = mem::replace(&mut self.elems, Box::pin([])); - let results = iter_pin_mut(elems.as_mut()) - .map(|e| e.take_output().unwrap()) - .collect(); + let results = + iter_pin_mut(elems.as_mut()).map(|e| e.take_output().unwrap()).collect(); Poll::Ready(Ok(results)) - }, + } FinalState::Error(e) => { let _ = mem::replace(&mut self.elems, Box::pin([])); Poll::Ready(Err(e)) - }, + } } } } diff --git a/futures-util/src/future/try_maybe_done.rs b/futures-util/src/future/try_maybe_done.rs index d033cee8d9..1ecaed9d96 100644 --- a/futures-util/src/future/try_maybe_done.rs +++ b/futures-util/src/future/try_maybe_done.rs @@ -44,12 +44,12 @@ impl TryMaybeDone { #[inline] pub fn take_output(self: Pin<&mut Self>) -> Option { match &*self { - Self::Done(_) => {}, + Self::Done(_) => {} Self::Future(_) | Self::Gone => return None, } match self.project_replace(Self::Gone) { TryMaybeDoneProjOwn::Done(output) => Some(output), - _ => unreachable!() + _ => unreachable!(), } } } @@ -68,16 +68,14 @@ impl Future for TryMaybeDone { fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { match self.as_mut().project() { - TryMaybeDoneProj::Future(f) => { - match ready!(f.try_poll(cx)) { - Ok(res) => self.set(Self::Done(res)), - Err(e) => { - self.set(Self::Gone); - return Poll::Ready(Err(e)); - } + TryMaybeDoneProj::Future(f) => match ready!(f.try_poll(cx)) { + Ok(res) => self.set(Self::Done(res)), + Err(e) => { + self.set(Self::Gone); + return Poll::Ready(Err(e)); } }, - TryMaybeDoneProj::Done(_) => {}, + TryMaybeDoneProj::Done(_) => {} TryMaybeDoneProj::Gone => panic!("TryMaybeDone polled after value taken"), } Poll::Ready(Ok(())) diff --git a/futures-util/src/future/try_select.rs b/futures-util/src/future/try_select.rs index 56564f5b5c..a3e8bb902f 100644 --- a/futures-util/src/future/try_select.rs +++ b/futures-util/src/future/try_select.rs @@ -1,7 +1,7 @@ +use crate::future::{Either, TryFutureExt}; use core::pin::Pin; use futures_core::future::{Future, TryFuture}; use futures_core::task::{Context, Poll}; -use crate::future::{Either, TryFutureExt}; /// Future for the [`try_select()`] function. #[must_use = "futures do nothing unless you `.await` or poll them"] @@ -48,19 +48,20 @@ impl Unpin for TrySelect {} /// } /// ``` pub fn try_select(future1: A, future2: B) -> TrySelect - where A: TryFuture + Unpin, B: TryFuture + Unpin +where + A: TryFuture + Unpin, + B: TryFuture + Unpin, { TrySelect { inner: Some((future1, future2)) } } impl Future for TrySelect - where A: TryFuture, B: TryFuture +where + A: TryFuture, + B: TryFuture, { #[allow(clippy::type_complexity)] - type Output = Result< - Either<(A::Ok, B), (B::Ok, A)>, - Either<(A::Error, B), (B::Error, A)>, - >; + type Output = Result, Either<(A::Error, B), (B::Error, A)>>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let (mut a, mut b) = self.inner.take().expect("cannot poll Select twice"); @@ -74,7 +75,7 @@ impl Future for TrySelect self.inner = Some((a, b)); Poll::Pending } - } + }, } } } diff --git a/futures-util/src/io/allow_std.rs b/futures-util/src/io/allow_std.rs index 9aa8eb4e71..1d13e0c177 100644 --- a/futures-util/src/io/allow_std.rs +++ b/futures-util/src/io/allow_std.rs @@ -1,9 +1,9 @@ use futures_core::task::{Context, Poll}; #[cfg(feature = "read-initializer")] use futures_io::Initializer; -use futures_io::{AsyncRead, AsyncWrite, AsyncSeek, AsyncBufRead, IoSlice, IoSliceMut, SeekFrom}; -use std::{fmt, io}; +use futures_io::{AsyncBufRead, AsyncRead, AsyncSeek, AsyncWrite, IoSlice, IoSliceMut, SeekFrom}; use std::pin::Pin; +use std::{fmt, io}; /// A simple wrapper type which allows types which implement only /// implement `std::io::Read` or `std::io::Write` @@ -35,7 +35,7 @@ macro_rules! try_with_interrupt { } } } - } + }; } impl AllowStdIo { @@ -60,7 +60,10 @@ impl AllowStdIo { } } -impl io::Write for AllowStdIo where T: io::Write { +impl io::Write for AllowStdIo +where + T: io::Write, +{ fn write(&mut self, buf: &[u8]) -> io::Result { self.0.write(buf) } @@ -78,16 +81,23 @@ impl io::Write for AllowStdIo where T: io::Write { } } -impl AsyncWrite for AllowStdIo where T: io::Write { - fn poll_write(mut self: Pin<&mut Self>, _: &mut Context<'_>, buf: &[u8]) - -> Poll> - { +impl AsyncWrite for AllowStdIo +where + T: io::Write, +{ + fn poll_write( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { Poll::Ready(Ok(try_with_interrupt!(self.0.write(buf)))) } - fn poll_write_vectored(mut self: Pin<&mut Self>, _: &mut Context<'_>, bufs: &[IoSlice<'_>]) - -> Poll> - { + fn poll_write_vectored( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { Poll::Ready(Ok(try_with_interrupt!(self.0.write_vectored(bufs)))) } @@ -101,7 +111,10 @@ impl AsyncWrite for AllowStdIo where T: io::Write { } } -impl io::Read for AllowStdIo where T: io::Read { +impl io::Read for AllowStdIo +where + T: io::Read, +{ fn read(&mut self, buf: &mut [u8]) -> io::Result { self.0.read(buf) } @@ -123,16 +136,23 @@ impl io::Read for AllowStdIo where T: io::Read { } } -impl AsyncRead for AllowStdIo where T: io::Read { - fn poll_read(mut self: Pin<&mut Self>, _: &mut Context<'_>, buf: &mut [u8]) - -> Poll> - { +impl AsyncRead for AllowStdIo +where + T: io::Read, +{ + fn poll_read( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { Poll::Ready(Ok(try_with_interrupt!(self.0.read(buf)))) } - fn poll_read_vectored(mut self: Pin<&mut Self>, _: &mut Context<'_>, bufs: &mut [IoSliceMut<'_>]) - -> Poll> - { + fn poll_read_vectored( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + bufs: &mut [IoSliceMut<'_>], + ) -> Poll> { Poll::Ready(Ok(try_with_interrupt!(self.0.read_vectored(bufs)))) } @@ -142,21 +162,32 @@ impl AsyncRead for AllowStdIo where T: io::Read { } } -impl io::Seek for AllowStdIo where T: io::Seek { +impl io::Seek for AllowStdIo +where + T: io::Seek, +{ fn seek(&mut self, pos: SeekFrom) -> io::Result { self.0.seek(pos) } } -impl AsyncSeek for AllowStdIo where T: io::Seek { - fn poll_seek(mut self: Pin<&mut Self>, _: &mut Context<'_>, pos: SeekFrom) - -> Poll> - { +impl AsyncSeek for AllowStdIo +where + T: io::Seek, +{ + fn poll_seek( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + pos: SeekFrom, + ) -> Poll> { Poll::Ready(Ok(try_with_interrupt!(self.0.seek(pos)))) } } -impl io::BufRead for AllowStdIo where T: io::BufRead { +impl io::BufRead for AllowStdIo +where + T: io::BufRead, +{ fn fill_buf(&mut self) -> io::Result<&[u8]> { self.0.fill_buf() } @@ -165,10 +196,11 @@ impl io::BufRead for AllowStdIo where T: io::BufRead { } } -impl AsyncBufRead for AllowStdIo where T: io::BufRead { - fn poll_fill_buf(mut self: Pin<&mut Self>, _: &mut Context<'_>) - -> Poll> - { +impl AsyncBufRead for AllowStdIo +where + T: io::BufRead, +{ + fn poll_fill_buf(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { let this: *mut Self = &mut *self as *mut _; Poll::Ready(Ok(try_with_interrupt!(unsafe { &mut *this }.0.fill_buf()))) } diff --git a/futures-util/src/io/buf_reader.rs b/futures-util/src/io/buf_reader.rs index 2755667219..381c19ed7c 100644 --- a/futures-util/src/io/buf_reader.rs +++ b/futures-util/src/io/buf_reader.rs @@ -1,3 +1,4 @@ +use super::DEFAULT_BUF_SIZE; use futures_core::task::{Context, Poll}; #[cfg(feature = "read-initializer")] use futures_io::Initializer; @@ -6,7 +7,6 @@ use pin_project::pin_project; use std::io::{self, Read}; use std::pin::Pin; use std::{cmp, fmt}; -use super::DEFAULT_BUF_SIZE; /// The `BufReader` struct adds buffering to any reader. /// @@ -49,12 +49,7 @@ impl BufReader { let mut buffer = Vec::with_capacity(capacity); buffer.set_len(capacity); super::initialize(&inner, &mut buffer); - Self { - inner, - buffer: buffer.into_boxed_slice(), - pos: 0, - cap: 0, - } + Self { inner, buffer: buffer.into_boxed_slice(), pos: 0, cap: 0 } } } @@ -121,10 +116,7 @@ impl AsyncRead for BufReader { } impl AsyncBufRead for BufReader { - fn poll_fill_buf( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); // If we've reached the end of our internal buffer then we need to fetch @@ -190,7 +182,8 @@ impl AsyncSeek for BufReader { // support seeking by i64::min_value() so we need to handle underflow when subtracting // remainder. if let Some(offset) = n.checked_sub(remainder) { - result = ready!(self.as_mut().project().inner.poll_seek(cx, SeekFrom::Current(offset)))?; + result = + ready!(self.as_mut().project().inner.poll_seek(cx, SeekFrom::Current(offset)))?; } else { // seek backwards by our remainder, and then by the offset ready!(self.as_mut().project().inner.poll_seek(cx, SeekFrom::Current(-remainder)))?; diff --git a/futures-util/src/io/buf_writer.rs b/futures-util/src/io/buf_writer.rs index ed9196dfb7..6c399782d0 100644 --- a/futures-util/src/io/buf_writer.rs +++ b/futures-util/src/io/buf_writer.rs @@ -1,10 +1,10 @@ +use super::DEFAULT_BUF_SIZE; use futures_core::task::{Context, Poll}; use futures_io::{AsyncBufRead, AsyncRead, AsyncSeek, AsyncWrite, IoSlice, SeekFrom}; use pin_project::pin_project; use std::fmt; use std::io::{self, Write}; use std::pin::Pin; -use super::DEFAULT_BUF_SIZE; /// Wraps a writer and buffers its output. /// @@ -44,11 +44,7 @@ impl BufWriter { /// Creates a new `BufWriter` with the specified buffer capacity. pub fn with_capacity(cap: usize, inner: W) -> Self { - Self { - inner, - buf: Vec::with_capacity(cap), - written: 0, - } + Self { inner, buf: Vec::with_capacity(cap), written: 0 } } fn flush_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { diff --git a/futures-util/src/io/chain.rs b/futures-util/src/io/chain.rs index 336307f250..9e8b6bc4d2 100644 --- a/futures-util/src/io/chain.rs +++ b/futures-util/src/io/chain.rs @@ -24,11 +24,7 @@ where U: AsyncRead, { pub(super) fn new(first: T, second: U) -> Self { - Self { - first, - second, - done_first: false, - } + Self { first, second, done_first: false } } /// Gets references to the underlying readers in this `Chain`. diff --git a/futures-util/src/io/copy.rs b/futures-util/src/io/copy.rs index 491a680a4a..7939d31c1f 100644 --- a/futures-util/src/io/copy.rs +++ b/futures-util/src/io/copy.rs @@ -1,10 +1,10 @@ +use super::{copy_buf, BufReader, CopyBuf}; use futures_core::future::Future; use futures_core::task::{Context, Poll}; use futures_io::{AsyncRead, AsyncWrite}; +use pin_project::pin_project; use std::io; use std::pin::Pin; -use super::{BufReader, copy_buf, CopyBuf}; -use pin_project::pin_project; /// Creates a future which copies all the bytes from one object to another. /// @@ -36,9 +36,7 @@ where R: AsyncRead, W: AsyncWrite + Unpin + ?Sized, { - Copy { - inner: copy_buf(BufReader::new(reader), writer), - } + Copy { inner: copy_buf(BufReader::new(reader), writer) } } /// Future for the [`copy()`] function. diff --git a/futures-util/src/io/copy_buf.rs b/futures-util/src/io/copy_buf.rs index f47144a508..5151436eeb 100644 --- a/futures-util/src/io/copy_buf.rs +++ b/futures-util/src/io/copy_buf.rs @@ -1,9 +1,9 @@ use futures_core::future::Future; use futures_core::task::{Context, Poll}; use futures_io::{AsyncBufRead, AsyncWrite}; +use pin_project::pin_project; use std::io; use std::pin::Pin; -use pin_project::pin_project; /// Creates a future which copies all the bytes from one object to another. /// @@ -35,11 +35,7 @@ where R: AsyncBufRead, W: AsyncWrite + Unpin + ?Sized, { - CopyBuf { - reader, - writer, - amt: 0, - } + CopyBuf { reader, writer, amt: 0 } } /// Future for the [`copy_buf()`] function. @@ -54,8 +50,9 @@ pub struct CopyBuf<'a, R, W: ?Sized> { } impl Future for CopyBuf<'_, R, W> - where R: AsyncBufRead, - W: AsyncWrite + Unpin + ?Sized, +where + R: AsyncBufRead, + W: AsyncWrite + Unpin + ?Sized, { type Output = io::Result; @@ -70,7 +67,7 @@ impl Future for CopyBuf<'_, R, W> let i = ready!(Pin::new(&mut this.writer).poll_write(cx, buffer))?; if i == 0 { - return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) + return Poll::Ready(Err(io::ErrorKind::WriteZero.into())); } *this.amt += i as u64; this.reader.as_mut().consume(i); diff --git a/futures-util/src/io/cursor.rs b/futures-util/src/io/cursor.rs index b11dbf54f4..a458f50f30 100644 --- a/futures-util/src/io/cursor.rs +++ b/futures-util/src/io/cursor.rs @@ -43,9 +43,7 @@ impl Cursor { /// # force_inference(&buff); /// ``` pub fn new(inner: T) -> Self { - Self { - inner: io::Cursor::new(inner), - } + Self { inner: io::Cursor::new(inner) } } /// Consumes this cursor, returning the underlying value. @@ -199,15 +197,19 @@ where macro_rules! delegate_async_write_to_stdio { () => { - fn poll_write(mut self: Pin<&mut Self>, _: &mut Context<'_>, buf: &[u8]) - -> Poll> - { + fn poll_write( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { Poll::Ready(io::Write::write(&mut self.inner, buf)) } - fn poll_write_vectored(mut self: Pin<&mut Self>, _: &mut Context<'_>, bufs: &[IoSlice<'_>]) - -> Poll> - { + fn poll_write_vectored( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { Poll::Ready(io::Write::write_vectored(&mut self.inner, bufs)) } @@ -218,7 +220,7 @@ macro_rules! delegate_async_write_to_stdio { fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.poll_flush(cx) } - } + }; } impl AsyncWrite for Cursor<&mut [u8]> { diff --git a/futures-util/src/io/fill_buf.rs b/futures-util/src/io/fill_buf.rs index 6fb3ec73aa..19b0d2044c 100644 --- a/futures-util/src/io/fill_buf.rs +++ b/futures-util/src/io/fill_buf.rs @@ -20,7 +20,8 @@ impl<'a, R: AsyncBufRead + ?Sized + Unpin> FillBuf<'a, R> { } impl<'a, R> Future for FillBuf<'a, R> - where R: AsyncBufRead + ?Sized + Unpin, +where + R: AsyncBufRead + ?Sized + Unpin, { type Output = io::Result<&'a [u8]>; diff --git a/futures-util/src/io/flush.rs b/futures-util/src/io/flush.rs index ece0a7cdc0..b75d14c5d3 100644 --- a/futures-util/src/io/flush.rs +++ b/futures-util/src/io/flush.rs @@ -20,7 +20,8 @@ impl<'a, W: AsyncWrite + ?Sized + Unpin> Flush<'a, W> { } impl Future for Flush<'_, W> - where W: AsyncWrite + ?Sized + Unpin, +where + W: AsyncWrite + ?Sized + Unpin, { type Output = io::Result<()>; diff --git a/futures-util/src/io/into_sink.rs b/futures-util/src/io/into_sink.rs index 3fa5b8923f..a66d87bbda 100644 --- a/futures-util/src/io/into_sink.rs +++ b/futures-util/src/io/into_sink.rs @@ -1,9 +1,9 @@ use futures_core::task::{Context, Poll}; use futures_io::AsyncWrite; use futures_sink::Sink; +use pin_project::pin_project; use std::io; use std::pin::Pin; -use pin_project::pin_project; #[derive(Debug)] struct Block { @@ -34,8 +34,7 @@ impl> IntoSink { fn poll_flush_buffer( self: Pin<&mut Self>, cx: &mut Context<'_>, - ) -> Poll> - { + ) -> Poll> { let mut this = self.project(); if let Some(buffer) = this.buffer { @@ -51,47 +50,30 @@ impl> IntoSink { *this.buffer = None; Poll::Ready(Ok(())) } - } impl> Sink for IntoSink { type Error = io::Error; - fn poll_ready( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> - { + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.poll_flush_buffer(cx))?; Poll::Ready(Ok(())) } #[allow(clippy::debug_assert_with_mut_call)] - fn start_send( - self: Pin<&mut Self>, - item: Item, - ) -> Result<(), Self::Error> - { + fn start_send(self: Pin<&mut Self>, item: Item) -> Result<(), Self::Error> { debug_assert!(self.buffer.is_none()); *self.project().buffer = Some(Block { offset: 0, bytes: item }); Ok(()) } - fn poll_flush( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> - { + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().poll_flush_buffer(cx))?; ready!(self.project().writer.poll_flush(cx))?; Poll::Ready(Ok(())) } - fn poll_close( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> - { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().poll_flush_buffer(cx))?; ready!(self.project().writer.poll_close(cx))?; Poll::Ready(Ok(())) diff --git a/futures-util/src/io/lines.rs b/futures-util/src/io/lines.rs index 90b993da30..01fc78f2a8 100644 --- a/futures-util/src/io/lines.rs +++ b/futures-util/src/io/lines.rs @@ -1,11 +1,11 @@ +use super::read_line::read_line_internal; use futures_core::stream::Stream; use futures_core::task::{Context, Poll}; use futures_io::AsyncBufRead; +use pin_project::pin_project; use std::io; use std::mem; use std::pin::Pin; -use super::read_line::read_line_internal; -use pin_project::pin_project; /// Stream for the [`lines`](super::AsyncBufReadExt::lines) method. @@ -22,12 +22,7 @@ pub struct Lines { impl Lines { pub(super) fn new(reader: R) -> Self { - Self { - reader, - buf: String::new(), - bytes: Vec::new(), - read: 0, - } + Self { reader, buf: String::new(), bytes: Vec::new(), read: 0 } } } @@ -38,7 +33,7 @@ impl Stream for Lines { let this = self.project(); let n = ready!(read_line_internal(this.reader, cx, this.buf, this.bytes, this.read))?; if n == 0 && this.buf.is_empty() { - return Poll::Ready(None) + return Poll::Ready(None); } if this.buf.ends_with('\n') { this.buf.pop(); diff --git a/futures-util/src/io/mod.rs b/futures-util/src/io/mod.rs index 51ee995d11..3a1832ac28 100644 --- a/futures-util/src/io/mod.rs +++ b/futures-util/src/io/mod.rs @@ -12,15 +12,15 @@ #[cfg(feature = "io-compat")] #[cfg_attr(docsrs, doc(cfg(feature = "io-compat")))] use crate::compat::Compat; -use std::{ptr, pin::Pin}; +use std::{pin::Pin, ptr}; -pub use futures_io::{ - AsyncRead, AsyncWrite, AsyncSeek, AsyncBufRead, Error, ErrorKind, - IoSlice, IoSliceMut, Result, SeekFrom, -}; #[cfg(feature = "read-initializer")] #[cfg_attr(docsrs, doc(cfg(feature = "read-initializer")))] pub use futures_io::Initializer; +pub use futures_io::{ + AsyncBufRead, AsyncRead, AsyncSeek, AsyncWrite, Error, ErrorKind, IoSlice, IoSliceMut, Result, + SeekFrom, +}; // used by `BufReader` and `BufWriter` // https://github.com/rust-lang/rust/blob/master/src/libstd/sys_common/io.rs#L1 @@ -114,7 +114,7 @@ mod sink; pub use self::sink::{sink, Sink}; mod split; -pub use self::split::{ReadHalf, WriteHalf, ReuniteError}; +pub use self::split::{ReadHalf, ReuniteError, WriteHalf}; mod take; pub use self::take::Take; @@ -194,7 +194,8 @@ pub trait AsyncReadExt: AsyncRead { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn read<'a>(&'a mut self, buf: &'a mut [u8]) -> Read<'a, Self> - where Self: Unpin, + where + Self: Unpin, { Read::new(self, buf) } @@ -205,7 +206,8 @@ pub trait AsyncReadExt: AsyncRead { /// The returned future will resolve to the number of bytes read once the read /// operation is completed. fn read_vectored<'a>(&'a mut self, bufs: &'a mut [IoSliceMut<'a>]) -> ReadVectored<'a, Self> - where Self: Unpin, + where + Self: Unpin, { ReadVectored::new(self, bufs) } @@ -247,11 +249,9 @@ pub trait AsyncReadExt: AsyncRead { /// assert_eq!(result.unwrap_err().kind(), io::ErrorKind::UnexpectedEof); /// # }); /// ``` - fn read_exact<'a>( - &'a mut self, - buf: &'a mut [u8], - ) -> ReadExact<'a, Self> - where Self: Unpin, + fn read_exact<'a>(&'a mut self, buf: &'a mut [u8]) -> ReadExact<'a, Self> + where + Self: Unpin, { ReadExact::new(self, buf) } @@ -275,11 +275,9 @@ pub trait AsyncReadExt: AsyncRead { /// assert_eq!(output, vec![1, 2, 3, 4]); /// # Ok::<(), Box>(()) }).unwrap(); /// ``` - fn read_to_end<'a>( - &'a mut self, - buf: &'a mut Vec, - ) -> ReadToEnd<'a, Self> - where Self: Unpin, + fn read_to_end<'a>(&'a mut self, buf: &'a mut Vec) -> ReadToEnd<'a, Self> + where + Self: Unpin, { ReadToEnd::new(self, buf) } @@ -303,11 +301,9 @@ pub trait AsyncReadExt: AsyncRead { /// assert_eq!(buffer, String::from("1234")); /// # Ok::<(), Box>(()) }).unwrap(); /// ``` - fn read_to_string<'a>( - &'a mut self, - buf: &'a mut String, - ) -> ReadToString<'a, Self> - where Self: Unpin, + fn read_to_string<'a>(&'a mut self, buf: &'a mut String) -> ReadToString<'a, Self> + where + Self: Unpin, { ReadToString::new(self, buf) } @@ -342,7 +338,8 @@ pub trait AsyncReadExt: AsyncRead { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn split(self) -> (ReadHalf, WriteHalf) - where Self: AsyncWrite + Sized, + where + Self: AsyncWrite + Sized, { split::split(self) } @@ -367,7 +364,8 @@ pub trait AsyncReadExt: AsyncRead { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn take(self, limit: u64) -> Take - where Self: Sized + where + Self: Sized, { Take::new(self, limit) } @@ -381,7 +379,8 @@ pub trait AsyncReadExt: AsyncRead { #[cfg(feature = "io-compat")] #[cfg_attr(docsrs, doc(cfg(feature = "io-compat")))] fn compat(self) -> Compat - where Self: Sized + Unpin, + where + Self: Sized + Unpin, { Compat::new(self) } @@ -414,14 +413,16 @@ pub trait AsyncWriteExt: AsyncWrite { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn flush(&mut self) -> Flush<'_, Self> - where Self: Unpin, + where + Self: Unpin, { Flush::new(self) } /// Creates a future which will entirely close this `AsyncWrite`. fn close(&mut self) -> Close<'_, Self> - where Self: Unpin, + where + Self: Unpin, { Close::new(self) } @@ -431,7 +432,8 @@ pub trait AsyncWriteExt: AsyncWrite { /// The returned future will resolve to the number of bytes written once the write /// operation is completed. fn write<'a>(&'a mut self, buf: &'a [u8]) -> Write<'a, Self> - where Self: Unpin, + where + Self: Unpin, { Write::new(self, buf) } @@ -442,7 +444,8 @@ pub trait AsyncWriteExt: AsyncWrite { /// The returned future will resolve to the number of bytes written once the write /// operation is completed. fn write_vectored<'a>(&'a mut self, bufs: &'a [IoSlice<'a>]) -> WriteVectored<'a, Self> - where Self: Unpin, + where + Self: Unpin, { WriteVectored::new(self, bufs) } @@ -468,7 +471,8 @@ pub trait AsyncWriteExt: AsyncWrite { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn write_all<'a>(&'a mut self, buf: &'a [u8]) -> WriteAll<'a, Self> - where Self: Unpin, + where + Self: Unpin, { WriteAll::new(self, buf) } @@ -534,7 +538,8 @@ pub trait AsyncWriteExt: AsyncWrite { #[cfg(feature = "io-compat")] #[cfg_attr(docsrs, doc(cfg(feature = "io-compat")))] fn compat_write(self) -> Compat - where Self: Sized + Unpin, + where + Self: Sized + Unpin, { Compat::new(self) } @@ -568,7 +573,8 @@ pub trait AsyncWriteExt: AsyncWrite { #[cfg(feature = "sink")] #[cfg_attr(docsrs, doc(cfg(feature = "sink")))] fn into_sink>(self) -> IntoSink - where Self: Sized, + where + Self: Sized, { IntoSink::new(self) } @@ -584,7 +590,8 @@ pub trait AsyncSeekExt: AsyncSeek { /// In the case of an error the buffer and the object will be discarded, with /// the error yielded. fn seek(&mut self, pos: SeekFrom) -> Seek<'_, Self> - where Self: Unpin, + where + Self: Unpin, { Seek::new(self, pos) } @@ -618,7 +625,8 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn fill_buf(&mut self) -> FillBuf<'_, Self> - where Self: Unpin, + where + Self: Unpin, { FillBuf::new(self) } @@ -641,7 +649,8 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn consume_unpin(&mut self, amt: usize) - where Self: Unpin, + where + Self: Unpin, { Pin::new(self).consume(amt) } @@ -687,12 +696,9 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// assert_eq!(buf, b""); /// # Ok::<(), Box>(()) }).unwrap(); /// ``` - fn read_until<'a>( - &'a mut self, - byte: u8, - buf: &'a mut Vec, - ) -> ReadUntil<'a, Self> - where Self: Unpin, + fn read_until<'a>(&'a mut self, byte: u8, buf: &'a mut Vec) -> ReadUntil<'a, Self> + where + Self: Unpin, { ReadUntil::new(self, byte, buf) } @@ -749,7 +755,8 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn read_line<'a>(&'a mut self, buf: &'a mut String) -> ReadLine<'a, Self> - where Self: Unpin, + where + Self: Unpin, { ReadLine::new(self, buf) } @@ -787,7 +794,8 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// # Ok::<(), Box>(()) }).unwrap(); /// ``` fn lines(self) -> Lines - where Self: Sized, + where + Self: Sized, { Lines::new(self) } diff --git a/futures-util/src/io/read_exact.rs b/futures-util/src/io/read_exact.rs index 879b81bcbe..265432080b 100644 --- a/futures-util/src/io/read_exact.rs +++ b/futures-util/src/io/read_exact.rs @@ -33,7 +33,7 @@ impl Future for ReadExact<'_, R> { this.buf = rest; } if n == 0 { - return Poll::Ready(Err(io::ErrorKind::UnexpectedEof.into())) + return Poll::Ready(Err(io::ErrorKind::UnexpectedEof.into())); } } Poll::Ready(Ok(())) diff --git a/futures-util/src/io/read_line.rs b/futures-util/src/io/read_line.rs index 59fd8728ff..a2759c02e2 100644 --- a/futures-util/src/io/read_line.rs +++ b/futures-util/src/io/read_line.rs @@ -1,3 +1,4 @@ +use super::read_until::read_until_internal; use futures_core::future::Future; use futures_core::task::{Context, Poll}; use futures_io::AsyncBufRead; @@ -5,7 +6,6 @@ use std::io; use std::mem; use std::pin::Pin; use std::str; -use super::read_until::read_until_internal; /// Future for the [`read_line`](super::AsyncBufReadExt::read_line) method. #[derive(Debug)] @@ -21,12 +21,7 @@ impl Unpin for ReadLine<'_, R> {} impl<'a, R: AsyncBufRead + ?Sized + Unpin> ReadLine<'a, R> { pub(super) fn new(reader: &'a mut R, buf: &'a mut String) -> Self { - Self { - reader, - bytes: mem::replace(buf, String::new()).into_bytes(), - buf, - read: 0, - } + Self { reader, bytes: mem::replace(buf, String::new()).into_bytes(), buf, read: 0 } } } diff --git a/futures-util/src/io/read_to_end.rs b/futures-util/src/io/read_to_end.rs index 70b057829c..5f00b66d30 100644 --- a/futures-util/src/io/read_to_end.rs +++ b/futures-util/src/io/read_to_end.rs @@ -19,19 +19,20 @@ impl Unpin for ReadToEnd<'_, R> {} impl<'a, R: AsyncRead + ?Sized + Unpin> ReadToEnd<'a, R> { pub(super) fn new(reader: &'a mut R, buf: &'a mut Vec) -> Self { let start_len = buf.len(); - Self { - reader, - buf, - start_len, - } + Self { reader, buf, start_len } } } -struct Guard<'a> { buf: &'a mut Vec, len: usize } +struct Guard<'a> { + buf: &'a mut Vec, + len: usize, +} impl Drop for Guard<'_> { fn drop(&mut self) { - unsafe { self.buf.set_len(self.len); } + unsafe { + self.buf.set_len(self.len); + } } } @@ -79,7 +80,8 @@ pub(super) fn read_to_end_internal( } impl Future for ReadToEnd<'_, A> - where A: AsyncRead + ?Sized + Unpin, +where + A: AsyncRead + ?Sized + Unpin, { type Output = io::Result; diff --git a/futures-util/src/io/read_to_string.rs b/futures-util/src/io/read_to_string.rs index 087ce38c34..e1545a1c6b 100644 --- a/futures-util/src/io/read_to_string.rs +++ b/futures-util/src/io/read_to_string.rs @@ -21,12 +21,7 @@ impl Unpin for ReadToString<'_, R> {} impl<'a, R: AsyncRead + ?Sized + Unpin> ReadToString<'a, R> { pub(super) fn new(reader: &'a mut R, buf: &'a mut String) -> Self { let start_len = buf.len(); - Self { - reader, - bytes: mem::replace(buf, String::new()).into_bytes(), - buf, - start_len, - } + Self { reader, bytes: mem::replace(buf, String::new()).into_bytes(), buf, start_len } } } @@ -40,10 +35,7 @@ fn read_to_string_internal( let ret = ready!(read_to_end_internal(reader, cx, bytes, start_len)); if str::from_utf8(bytes).is_err() { Poll::Ready(ret.and_then(|_| { - Err(io::Error::new( - io::ErrorKind::InvalidData, - "stream did not contain valid UTF-8", - )) + Err(io::Error::new(io::ErrorKind::InvalidData, "stream did not contain valid UTF-8")) })) } else { debug_assert!(buf.is_empty()); diff --git a/futures-util/src/io/split.rs b/futures-util/src/io/split.rs index ccddd04979..082cc81ebf 100644 --- a/futures-util/src/io/split.rs +++ b/futures-util/src/io/split.rs @@ -1,7 +1,7 @@ use crate::lock::BiLock; +use core::fmt; use futures_core::task::{Context, Poll}; use futures_io::{AsyncRead, AsyncWrite, IoSlice, IoSliceMut}; -use core::fmt; use std::io; use std::pin::Pin; @@ -17,12 +17,9 @@ pub struct WriteHalf { handle: BiLock, } -fn lock_and_then( - lock: &BiLock, - cx: &mut Context<'_>, - f: F -) -> Poll> - where F: FnOnce(Pin<&mut T>, &mut Context<'_>) -> Poll> +fn lock_and_then(lock: &BiLock, cx: &mut Context<'_>, f: F) -> Poll> +where + F: FnOnce(Pin<&mut T>, &mut Context<'_>) -> Poll>, { let mut l = ready!(lock.poll_lock(cx)); f(l.as_pin_mut(), cx) @@ -38,9 +35,9 @@ impl ReadHalf { /// together. Succeeds only if the `ReadHalf` and `WriteHalf` are /// a matching pair originating from the same call to `AsyncReadExt::split`. pub fn reunite(self, other: WriteHalf) -> Result> { - self.handle.reunite(other.handle).map_err(|err| { - ReuniteError(ReadHalf { handle: err.0 }, WriteHalf { handle: err.1 }) - }) + self.handle + .reunite(other.handle) + .map_err(|err| ReuniteError(ReadHalf { handle: err.0 }, WriteHalf { handle: err.1 })) } } @@ -54,29 +51,37 @@ impl WriteHalf { } impl AsyncRead for ReadHalf { - fn poll_read(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) - -> Poll> - { + fn poll_read( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { lock_and_then(&self.handle, cx, |l, cx| l.poll_read(cx, buf)) } - fn poll_read_vectored(self: Pin<&mut Self>, cx: &mut Context<'_>, bufs: &mut [IoSliceMut<'_>]) - -> Poll> - { + fn poll_read_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &mut [IoSliceMut<'_>], + ) -> Poll> { lock_and_then(&self.handle, cx, |l, cx| l.poll_read_vectored(cx, bufs)) } } impl AsyncWrite for WriteHalf { - fn poll_write(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) - -> Poll> - { + fn poll_write( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { lock_and_then(&self.handle, cx, |l, cx| l.poll_write(cx, buf)) } - fn poll_write_vectored(self: Pin<&mut Self>, cx: &mut Context<'_>, bufs: &[IoSlice<'_>]) - -> Poll> - { + fn poll_write_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { lock_and_then(&self.handle, cx, |l, cx| l.poll_write_vectored(cx, bufs)) } @@ -95,9 +100,7 @@ pub struct ReuniteError(pub ReadHalf, pub WriteHalf); impl fmt::Debug for ReuniteError { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_tuple("ReuniteError") - .field(&"...") - .finish() + f.debug_tuple("ReuniteError").field(&"...").finish() } } diff --git a/futures-util/src/io/take.rs b/futures-util/src/io/take.rs index 6179486bb9..fde70c207d 100644 --- a/futures-util/src/io/take.rs +++ b/futures-util/src/io/take.rs @@ -1,10 +1,10 @@ use futures_core::task::{Context, Poll}; #[cfg(feature = "read-initializer")] use futures_io::Initializer; -use futures_io::{AsyncRead, AsyncBufRead}; +use futures_io::{AsyncBufRead, AsyncRead}; use pin_project::pin_project; -use std::{cmp, io}; use std::pin::Pin; +use std::{cmp, io}; /// Reader for the [`take`](super::AsyncReadExt::take) method. #[pin_project] diff --git a/futures-util/src/io/window.rs b/futures-util/src/io/window.rs index 3424197d75..77b7267c69 100644 --- a/futures-util/src/io/window.rs +++ b/futures-util/src/io/window.rs @@ -30,10 +30,7 @@ impl> Window { /// Further methods can be called on the returned `Window` to alter the /// window into the data provided. pub fn new(t: T) -> Self { - Self { - range: 0..t.as_ref().len(), - inner: t, - } + Self { range: 0..t.as_ref().len(), inner: t } } /// Gets a shared reference to the underlying buffer inside of this diff --git a/futures-util/src/io/write_all_vectored.rs b/futures-util/src/io/write_all_vectored.rs index 82772d901e..298034be6f 100644 --- a/futures-util/src/io/write_all_vectored.rs +++ b/futures-util/src/io/write_all_vectored.rs @@ -55,11 +55,7 @@ mod tests { /// Create a new writer that reads from at most `n_bufs` and reads /// `per_call` bytes (in total) per call to write. fn test_writer(n_bufs: usize, per_call: usize) -> TestWriter { - TestWriter { - n_bufs, - per_call, - written: Vec::new(), - } + TestWriter { n_bufs, per_call, written: Vec::new() } } // TODO: maybe move this the future-test crate? @@ -109,10 +105,9 @@ mod tests { let expected = $expected; match $e { Poll::Ready(Ok(ok)) if ok == expected => {} - got => panic!( - "unexpected result, got: {:?}, wanted: Ready(Ok({:?}))", - got, expected - ), + got => { + panic!("unexpected result, got: {:?}, wanted: Ready(Ok({:?}))", got, expected) + } } }; } @@ -153,11 +148,7 @@ mod tests { assert_poll_ok!(dst.as_mut().poll_write_vectored(&mut cx, bufs), 3); // Read at most 3 bytes from three buffers. - let bufs = &[ - IoSlice::new(&[3]), - IoSlice::new(&[4]), - IoSlice::new(&[5, 5]), - ]; + let bufs = &[IoSlice::new(&[3]), IoSlice::new(&[4]), IoSlice::new(&[5, 5])]; assert_poll_ok!(dst.as_mut().poll_write_vectored(&mut cx, bufs), 3); assert_eq!(dst.written, &[1, 2, 2, 3, 4, 5]); diff --git a/futures-util/src/lib.rs b/futures-util/src/lib.rs index 34ae11f369..4af544a503 100644 --- a/futures-util/src/lib.rs +++ b/futures-util/src/lib.rs @@ -4,22 +4,17 @@ #![cfg_attr(feature = "cfg-target-has-atomic", feature(cfg_target_has_atomic))] #![cfg_attr(feature = "read-initializer", feature(read_initializer))] #![cfg_attr(feature = "write-all-vectored", feature(io_slice_advance))] - #![cfg_attr(not(feature = "std"), no_std)] #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms, unreachable_pub)] // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures-util/0.3.8")] - #![cfg_attr(docsrs, feature(doc_cfg))] #[cfg(all(feature = "cfg-target-has-atomic", not(feature = "unstable")))] @@ -54,7 +49,7 @@ pub use self::async_await::*; pub mod __private { pub use crate::*; pub use core::{ - option::Option::{self, Some, None}, + option::Option::{self, None, Some}, pin::Pin, result::Result::{Err, Ok}, }; @@ -81,10 +76,7 @@ macro_rules! delegate_sink { self.project().$field.poll_ready(cx) } - fn start_send( - self: core::pin::Pin<&mut Self>, - item: $item, - ) -> Result<(), Self::Error> { + fn start_send(self: core::pin::Pin<&mut Self>, item: $item) -> Result<(), Self::Error> { self.project().$field.start_send(item) } @@ -101,7 +93,7 @@ macro_rules! delegate_sink { ) -> core::task::Poll> { self.project().$field.poll_close(cx) } - } + }; } macro_rules! delegate_future { @@ -112,7 +104,7 @@ macro_rules! delegate_future { ) -> core::task::Poll { self.project().$field.poll(cx) } - } + }; } macro_rules! delegate_stream { @@ -126,34 +118,40 @@ macro_rules! delegate_stream { fn size_hint(&self) -> (usize, Option) { self.$field.size_hint() } - } + }; } #[cfg(feature = "io")] #[cfg(feature = "std")] macro_rules! delegate_async_write { ($field:ident) => { - fn poll_write(self: core::pin::Pin<&mut Self>, cx: &mut core::task::Context<'_>, buf: &[u8]) - -> core::task::Poll> - { + fn poll_write( + self: core::pin::Pin<&mut Self>, + cx: &mut core::task::Context<'_>, + buf: &[u8], + ) -> core::task::Poll> { self.project().$field.poll_write(cx, buf) } - fn poll_write_vectored(self: core::pin::Pin<&mut Self>, cx: &mut core::task::Context<'_>, bufs: &[std::io::IoSlice<'_>]) - -> core::task::Poll> - { + fn poll_write_vectored( + self: core::pin::Pin<&mut Self>, + cx: &mut core::task::Context<'_>, + bufs: &[std::io::IoSlice<'_>], + ) -> core::task::Poll> { self.project().$field.poll_write_vectored(cx, bufs) } - fn poll_flush(self: core::pin::Pin<&mut Self>, cx: &mut core::task::Context<'_>) - -> core::task::Poll> - { + fn poll_flush( + self: core::pin::Pin<&mut Self>, + cx: &mut core::task::Context<'_>, + ) -> core::task::Poll> { self.project().$field.poll_flush(cx) } - fn poll_close(self: core::pin::Pin<&mut Self>, cx: &mut core::task::Context<'_>) - -> core::task::Poll> - { + fn poll_close( + self: core::pin::Pin<&mut Self>, + cx: &mut core::task::Context<'_>, + ) -> core::task::Poll> { self.project().$field.poll_close(cx) } - } + }; } #[cfg(feature = "io")] @@ -165,18 +163,22 @@ macro_rules! delegate_async_read { self.$field.initializer() } - fn poll_read(self: core::pin::Pin<&mut Self>, cx: &mut core::task::Context<'_>, buf: &mut [u8]) - -> core::task::Poll> - { + fn poll_read( + self: core::pin::Pin<&mut Self>, + cx: &mut core::task::Context<'_>, + buf: &mut [u8], + ) -> core::task::Poll> { self.project().$field.poll_read(cx, buf) } - fn poll_read_vectored(self: core::pin::Pin<&mut Self>, cx: &mut core::task::Context<'_>, bufs: &mut [std::io::IoSliceMut<'_>]) - -> core::task::Poll> - { + fn poll_read_vectored( + self: core::pin::Pin<&mut Self>, + cx: &mut core::task::Context<'_>, + bufs: &mut [std::io::IoSliceMut<'_>], + ) -> core::task::Poll> { self.project().$field.poll_read_vectored(cx, bufs) } - } + }; } #[cfg(feature = "io")] @@ -193,7 +195,7 @@ macro_rules! delegate_async_buf_read { fn consume(self: core::pin::Pin<&mut Self>, amt: usize) { self.project().$field.consume(amt) } - } + }; } macro_rules! delegate_access_inner { @@ -308,16 +310,19 @@ macro_rules! delegate_all { } pub mod future; -#[doc(hidden)] pub use crate::future::{FutureExt, TryFutureExt}; +#[doc(hidden)] +pub use crate::future::{FutureExt, TryFutureExt}; pub mod stream; -#[doc(hidden)] pub use crate::stream::{StreamExt, TryStreamExt}; +#[doc(hidden)] +pub use crate::stream::{StreamExt, TryStreamExt}; #[cfg(feature = "sink")] #[cfg_attr(docsrs, doc(cfg(feature = "sink")))] pub mod sink; #[cfg(feature = "sink")] -#[doc(hidden)] pub use crate::sink::SinkExt; +#[doc(hidden)] +pub use crate::sink::SinkExt; pub mod task; @@ -333,11 +338,11 @@ pub mod compat; pub mod io; #[cfg(feature = "io")] #[cfg(feature = "std")] -#[doc(hidden)] pub use crate::io::{AsyncReadExt, AsyncWriteExt, AsyncSeekExt, AsyncBufReadExt}; +#[doc(hidden)] +pub use crate::io::{AsyncBufReadExt, AsyncReadExt, AsyncSeekExt, AsyncWriteExt}; mod fns; - cfg_target_has_atomic! { #[cfg(feature = "alloc")] pub mod lock; diff --git a/futures-util/src/sink/buffer.rs b/futures-util/src/sink/buffer.rs index 0976fb486c..3f35f5e7c4 100644 --- a/futures-util/src/sink/buffer.rs +++ b/futures-util/src/sink/buffer.rs @@ -1,9 +1,9 @@ -use futures_core::stream::{Stream, FusedStream}; +use alloc::collections::VecDeque; +use core::pin::Pin; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_project::pin_project; -use core::pin::Pin; -use alloc::collections::VecDeque; /// Sink for the [`buffer`](super::SinkExt::buffer) method. #[pin_project] @@ -20,19 +20,12 @@ pub struct Buffer { impl, Item> Buffer { pub(super) fn new(sink: Si, capacity: usize) -> Self { - Self { - sink, - buf: VecDeque::with_capacity(capacity), - capacity, - } + Self { sink, buf: VecDeque::with_capacity(capacity), capacity } } delegate_access_inner!(sink, Si, ()); - fn try_empty_buffer( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn try_empty_buffer(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); ready!(this.sink.as_mut().poll_ready(cx))?; while let Some(item) = this.buf.pop_front() { @@ -46,7 +39,10 @@ impl, Item> Buffer { } // Forwarding impl of Stream from the underlying sink -impl Stream for Buffer where S: Sink + Stream { +impl Stream for Buffer +where + S: Sink + Stream, +{ type Item = S::Item; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { @@ -58,7 +54,10 @@ impl Stream for Buffer where S: Sink + Stream { } } -impl FusedStream for Buffer where S: Sink + FusedStream { +impl FusedStream for Buffer +where + S: Sink + FusedStream, +{ fn is_terminated(&self) -> bool { self.sink.is_terminated() } @@ -67,10 +66,7 @@ impl FusedStream for Buffer where S: Sink + FusedStream impl, Item> Sink for Buffer { type Error = Si::Error; - fn poll_ready( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { if self.capacity == 0 { return self.project().sink.poll_ready(cx); } @@ -84,10 +80,7 @@ impl, Item> Sink for Buffer { } } - fn start_send( - self: Pin<&mut Self>, - item: Item, - ) -> Result<(), Self::Error> { + fn start_send(self: Pin<&mut Self>, item: Item) -> Result<(), Self::Error> { if self.capacity == 0 { self.project().sink.start_send(item) } else { @@ -97,20 +90,14 @@ impl, Item> Sink for Buffer { } #[allow(clippy::debug_assert_with_mut_call)] - fn poll_flush( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().try_empty_buffer(cx))?; debug_assert!(self.buf.is_empty()); self.project().sink.poll_flush(cx) } #[allow(clippy::debug_assert_with_mut_call)] - fn poll_close( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().try_empty_buffer(cx))?; debug_assert!(self.buf.is_empty()); self.project().sink.poll_close(cx) diff --git a/futures-util/src/sink/close.rs b/futures-util/src/sink/close.rs index 4421d10deb..f8cbf1f4d5 100644 --- a/futures-util/src/sink/close.rs +++ b/futures-util/src/sink/close.rs @@ -19,20 +19,14 @@ impl Unpin for Close<'_, Si, Item> {} /// The sink itself is returned after closeing is complete. impl<'a, Si: Sink + Unpin + ?Sized, Item> Close<'a, Si, Item> { pub(super) fn new(sink: &'a mut Si) -> Self { - Self { - sink, - _phantom: PhantomData, - } + Self { sink, _phantom: PhantomData } } } impl + Unpin + ?Sized, Item> Future for Close<'_, Si, Item> { type Output = Result<(), Si::Error>; - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { Pin::new(&mut self.sink).poll_close(cx) } } diff --git a/futures-util/src/sink/drain.rs b/futures-util/src/sink/drain.rs index 46de83b7c2..f46f7f25b5 100644 --- a/futures-util/src/sink/drain.rs +++ b/futures-util/src/sink/drain.rs @@ -34,31 +34,19 @@ impl Unpin for Drain {} impl Sink for Drain { type Error = Never; - fn poll_ready( - self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } - fn start_send( - self: Pin<&mut Self>, - _item: T, - ) -> Result<(), Self::Error> { + fn start_send(self: Pin<&mut Self>, _item: T) -> Result<(), Self::Error> { Ok(()) } - fn poll_flush( - self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close( - self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } } diff --git a/futures-util/src/sink/err_into.rs b/futures-util/src/sink/err_into.rs index edad790130..0c98a83f52 100644 --- a/futures-util/src/sink/err_into.rs +++ b/futures-util/src/sink/err_into.rs @@ -1,6 +1,6 @@ use crate::sink::{SinkExt, SinkMapErr}; -use futures_core::stream::{Stream, FusedStream}; -use futures_sink::{Sink}; +use futures_core::stream::{FusedStream, Stream}; +use futures_sink::Sink; use pin_project::pin_project; /// Sink for the [`sink_err_into`](super::SinkExt::sink_err_into) method. @@ -13,21 +13,21 @@ pub struct SinkErrInto, Item, E> { } impl SinkErrInto - where Si: Sink, - Si::Error: Into, +where + Si: Sink, + Si::Error: Into, { pub(super) fn new(sink: Si) -> Self { - Self { - sink: SinkExt::sink_map_err(sink, Into::into), - } + Self { sink: SinkExt::sink_map_err(sink, Into::into) } } delegate_access_inner!(sink, Si, (.)); } impl Sink for SinkErrInto - where Si: Sink, - Si::Error: Into, +where + Si: Sink, + Si::Error: Into, { type Error = E; @@ -36,8 +36,9 @@ impl Sink for SinkErrInto // Forwarding impl of Stream from the underlying sink impl Stream for SinkErrInto - where S: Sink + Stream, - S::Error: Into +where + S: Sink + Stream, + S::Error: Into, { type Item = S::Item; @@ -45,8 +46,9 @@ impl Stream for SinkErrInto } impl FusedStream for SinkErrInto - where S: Sink + FusedStream, - S::Error: Into +where + S: Sink + FusedStream, + S::Error: Into, { fn is_terminated(&self) -> bool { self.sink.is_terminated() diff --git a/futures-util/src/sink/fanout.rs b/futures-util/src/sink/fanout.rs index 19d6bc9f2a..e735d675f5 100644 --- a/futures-util/src/sink/fanout.rs +++ b/futures-util/src/sink/fanout.rs @@ -14,7 +14,7 @@ pub struct Fanout { #[pin] sink1: Si1, #[pin] - sink2: Si2 + sink2: Si2, } impl Fanout { @@ -49,36 +49,32 @@ impl Fanout { impl Debug for Fanout { fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { - f.debug_struct("Fanout") - .field("sink1", &self.sink1) - .field("sink2", &self.sink2) - .finish() + f.debug_struct("Fanout").field("sink1", &self.sink1).field("sink2", &self.sink2).finish() } } impl Sink for Fanout - where Si1: Sink, - Item: Clone, - Si2: Sink +where + Si1: Sink, + Item: Clone, + Si2: Sink, { type Error = Si1::Error; - fn poll_ready( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); let sink1_ready = this.sink1.poll_ready(cx)?.is_ready(); let sink2_ready = this.sink2.poll_ready(cx)?.is_ready(); let ready = sink1_ready && sink2_ready; - if ready { Poll::Ready(Ok(())) } else { Poll::Pending } + if ready { + Poll::Ready(Ok(())) + } else { + Poll::Pending + } } - fn start_send( - self: Pin<&mut Self>, - item: Item, - ) -> Result<(), Self::Error> { + fn start_send(self: Pin<&mut Self>, item: Item) -> Result<(), Self::Error> { let this = self.project(); this.sink1.start_send(item.clone())?; @@ -86,27 +82,29 @@ impl Sink for Fanout Ok(()) } - fn poll_flush( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); let sink1_ready = this.sink1.poll_flush(cx)?.is_ready(); let sink2_ready = this.sink2.poll_flush(cx)?.is_ready(); let ready = sink1_ready && sink2_ready; - if ready { Poll::Ready(Ok(())) } else { Poll::Pending } + if ready { + Poll::Ready(Ok(())) + } else { + Poll::Pending + } } - fn poll_close( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); let sink1_ready = this.sink1.poll_close(cx)?.is_ready(); let sink2_ready = this.sink2.poll_close(cx)?.is_ready(); let ready = sink1_ready && sink2_ready; - if ready { Poll::Ready(Ok(())) } else { Poll::Pending } + if ready { + Poll::Ready(Ok(())) + } else { + Poll::Pending + } } } diff --git a/futures-util/src/sink/flush.rs b/futures-util/src/sink/flush.rs index c06a22185e..35a8372de7 100644 --- a/futures-util/src/sink/flush.rs +++ b/futures-util/src/sink/flush.rs @@ -23,20 +23,14 @@ impl Unpin for Flush<'_, Si, Item> {} /// all current requests are processed. impl<'a, Si: Sink + Unpin + ?Sized, Item> Flush<'a, Si, Item> { pub(super) fn new(sink: &'a mut Si) -> Self { - Self { - sink, - _phantom: PhantomData, - } + Self { sink, _phantom: PhantomData } } } impl + Unpin + ?Sized, Item> Future for Flush<'_, Si, Item> { type Output = Result<(), Si::Error>; - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { Pin::new(&mut self.sink).poll_flush(cx) } } diff --git a/futures-util/src/sink/map_err.rs b/futures-util/src/sink/map_err.rs index e77710cc3e..266aeff47d 100644 --- a/futures-util/src/sink/map_err.rs +++ b/futures-util/src/sink/map_err.rs @@ -1,7 +1,7 @@ use core::pin::Pin; -use futures_core::stream::{Stream, FusedStream}; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; -use futures_sink::{Sink}; +use futures_sink::Sink; use pin_project::pin_project; /// Sink for the [`sink_map_err`](super::SinkExt::sink_map_err) method. @@ -27,36 +27,25 @@ impl SinkMapErr { } impl Sink for SinkMapErr - where Si: Sink, - F: FnOnce(Si::Error) -> E, +where + Si: Sink, + F: FnOnce(Si::Error) -> E, { type Error = E; - fn poll_ready( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.as_mut().project().sink.poll_ready(cx).map_err(|e| self.as_mut().take_f()(e)) } - fn start_send( - mut self: Pin<&mut Self>, - item: Item, - ) -> Result<(), Self::Error> { + fn start_send(mut self: Pin<&mut Self>, item: Item) -> Result<(), Self::Error> { self.as_mut().project().sink.start_send(item).map_err(|e| self.as_mut().take_f()(e)) } - fn poll_flush( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.as_mut().project().sink.poll_flush(cx).map_err(|e| self.as_mut().take_f()(e)) } - fn poll_close( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.as_mut().project().sink.poll_close(cx).map_err(|e| self.as_mut().take_f()(e)) } } diff --git a/futures-util/src/sink/mod.rs b/futures-util/src/sink/mod.rs index b0e2c83f00..eeb1d25d9e 100644 --- a/futures-util/src/sink/mod.rs +++ b/futures-util/src/sink/mod.rs @@ -6,11 +6,11 @@ //! This module is only available when the `sink` feature of this //! library is activated, and it is activated by default. +use crate::future::Either; use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::{Stream, TryStream}; use futures_core::task::{Context, Poll}; -use crate::future::Either; #[cfg(feature = "compat")] use crate::compat::CompatSink; @@ -69,10 +69,11 @@ pub trait SinkExt: Sink { /// Note that this function consumes the given sink, returning a wrapped /// version, much like `Iterator::map`. fn with(self, f: F) -> With - where F: FnMut(U) -> Fut, - Fut: Future>, - E: From, - Self: Sized + where + F: FnMut(U) -> Fut, + Fut: Future>, + E: From, + Self: Sized, { With::new(self, f) } @@ -110,9 +111,10 @@ pub trait SinkExt: Sink { /// # }); /// ``` fn with_flat_map(self, f: F) -> WithFlatMap - where F: FnMut(U) -> St, - St: Stream>, - Self: Sized + where + F: FnMut(U) -> St, + St: Stream>, + Self: Sized, { WithFlatMap::new(self, f) } @@ -133,8 +135,9 @@ pub trait SinkExt: Sink { /// Transforms the error returned by the sink. fn sink_map_err(self, f: F) -> SinkMapErr - where F: FnOnce(Self::Error) -> E, - Self: Sized, + where + F: FnOnce(Self::Error) -> E, + Self: Sized, { SinkMapErr::new(self, f) } @@ -143,13 +146,13 @@ pub trait SinkExt: Sink { /// /// If wanting to map errors of a `Sink + Stream`, use `.sink_err_into().err_into()`. fn sink_err_into(self) -> err_into::SinkErrInto - where Self: Sized, - Self::Error: Into, + where + Self: Sized, + Self::Error: Into, { SinkErrInto::new(self) } - /// Adds a fixed-size buffer to the current sink. /// /// The resulting sink will buffer up to `capacity` items when the @@ -164,14 +167,16 @@ pub trait SinkExt: Sink { /// library is activated, and it is activated by default. #[cfg(feature = "alloc")] fn buffer(self, capacity: usize) -> Buffer - where Self: Sized, + where + Self: Sized, { Buffer::new(self, capacity) } /// Close the sink. fn close(&mut self) -> Close<'_, Self, Item> - where Self: Unpin, + where + Self: Unpin, { Close::new(self) } @@ -181,9 +186,10 @@ pub trait SinkExt: Sink { /// This adapter clones each incoming item and forwards it to both this as well as /// the other sink at the same time. fn fanout(self, other: Si) -> Fanout - where Self: Sized, - Item: Clone, - Si: Sink + where + Self: Sized, + Item: Clone, + Si: Sink, { Fanout::new(self, other) } @@ -193,7 +199,8 @@ pub trait SinkExt: Sink { /// This adapter is intended to be used when you want to stop sending to the sink /// until all current requests are processed. fn flush(&mut self) -> Flush<'_, Self, Item> - where Self: Unpin, + where + Self: Unpin, { Flush::new(self) } @@ -205,7 +212,8 @@ pub trait SinkExt: Sink { /// to batch together items to send via `send_all`, rather than flushing /// between each item.** fn send(&mut self, item: Item) -> Send<'_, Self, Item> - where Self: Unpin, + where + Self: Unpin, { Send::new(self, item) } @@ -221,12 +229,10 @@ 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, St>( - &'a mut self, - stream: &'a mut St - ) -> SendAll<'a, Self, St> - where St: TryStream + Stream + Unpin + ?Sized, - Self: Unpin, + fn send_all<'a, St>(&'a mut self, stream: &'a mut St) -> SendAll<'a, Self, St> + where + St: TryStream + Stream + Unpin + ?Sized, + Self: Unpin, { SendAll::new(self, stream) } @@ -237,8 +243,9 @@ pub trait SinkExt: Sink { /// This can be used in combination with the `right_sink` method to write `if` /// statements that evaluate to different streams in different branches. fn left_sink(self) -> Either - where Si2: Sink, - Self: Sized + where + Si2: Sink, + Self: Sized, { Either::Left(self) } @@ -249,8 +256,9 @@ pub trait SinkExt: Sink { /// This can be used in combination with the `left_sink` method to write `if` /// statements that evaluate to different streams in different branches. fn right_sink(self) -> Either - where Si1: Sink, - Self: Sized + where + Si1: Sink, + Self: Sized, { Either::Right(self) } @@ -260,15 +268,17 @@ pub trait SinkExt: Sink { #[cfg(feature = "compat")] #[cfg_attr(docsrs, doc(cfg(feature = "compat")))] fn compat(self) -> CompatSink - where Self: Sized + Unpin, + where + Self: Sized + Unpin, { CompatSink::new(self) } - + /// A convenience method for calling [`Sink::poll_ready`] on [`Unpin`] /// sink types. fn poll_ready_unpin(&mut self, cx: &mut Context<'_>) -> Poll> - where Self: Unpin + where + Self: Unpin, { Pin::new(self).poll_ready(cx) } @@ -276,7 +286,8 @@ pub trait SinkExt: Sink { /// A convenience method for calling [`Sink::start_send`] on [`Unpin`] /// sink types. fn start_send_unpin(&mut self, item: Item) -> Result<(), Self::Error> - where Self: Unpin + where + Self: Unpin, { Pin::new(self).start_send(item) } @@ -284,7 +295,8 @@ pub trait SinkExt: Sink { /// A convenience method for calling [`Sink::poll_flush`] on [`Unpin`] /// sink types. fn poll_flush_unpin(&mut self, cx: &mut Context<'_>) -> Poll> - where Self: Unpin + where + Self: Unpin, { Pin::new(self).poll_flush(cx) } @@ -292,7 +304,8 @@ pub trait SinkExt: Sink { /// A convenience method for calling [`Sink::poll_close`] on [`Unpin`] /// sink types. fn poll_close_unpin(&mut self, cx: &mut Context<'_>) -> Poll> - where Self: Unpin + where + Self: Unpin, { Pin::new(self).poll_close(cx) } diff --git a/futures-util/src/sink/send.rs b/futures-util/src/sink/send.rs index 86dc96e6b6..e94669d31f 100644 --- a/futures-util/src/sink/send.rs +++ b/futures-util/src/sink/send.rs @@ -16,20 +16,14 @@ impl Unpin for Send<'_, Si, Item> {} impl<'a, Si: Sink + Unpin + ?Sized, Item> Send<'a, Si, Item> { pub(super) fn new(sink: &'a mut Si, item: Item) -> Self { - Self { - sink, - item: Some(item), - } + Self { sink, item: Some(item) } } } impl + Unpin + ?Sized, Item> Future for Send<'_, Si, Item> { type Output = Result<(), Si::Error>; - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = &mut *self; if let Some(item) = this.item.take() { let mut sink = Pin::new(&mut this.sink); diff --git a/futures-util/src/sink/send_all.rs b/futures-util/src/sink/send_all.rs index b52191e095..55c28ed628 100644 --- a/futures-util/src/sink/send_all.rs +++ b/futures-util/src/sink/send_all.rs @@ -1,8 +1,8 @@ -use crate::stream::{StreamExt, TryStreamExt, Fuse}; +use crate::stream::{Fuse, StreamExt, TryStreamExt}; use core::fmt; use core::pin::Pin; use futures_core::future::Future; -use futures_core::stream::{TryStream, Stream}; +use futures_core::stream::{Stream, TryStream}; use futures_core::task::{Context, Poll}; use futures_sink::Sink; @@ -39,22 +39,16 @@ impl Unpin for SendAll<'_, Si, St> where Si: Unpin + ?Sized, St: TryStream + Unpin + ?Sized, -{} +{ +} impl<'a, Si, St, Ok, Error> SendAll<'a, Si, St> where Si: Sink + Unpin + ?Sized, St: TryStream + Stream + Unpin + ?Sized, { - pub(super) fn new( - sink: &'a mut Si, - stream: &'a mut St, - ) -> Self { - Self { - sink, - stream: stream.fuse(), - buffered: None, - } + pub(super) fn new(sink: &'a mut Si, stream: &'a mut St) -> Self { + Self { sink, stream: stream.fuse(), buffered: None } } fn try_start_send( @@ -64,9 +58,7 @@ where ) -> Poll> { debug_assert!(self.buffered.is_none()); match Pin::new(&mut self.sink).poll_ready(cx)? { - Poll::Ready(()) => { - Poll::Ready(Pin::new(&mut self.sink).start_send(item)) - } + Poll::Ready(()) => Poll::Ready(Pin::new(&mut self.sink).start_send(item)), Poll::Pending => { self.buffered = Some(item); Poll::Pending @@ -82,10 +74,7 @@ where { type Output = Result<(), Error>; - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = &mut *self; // If we've got an item buffered already, we need to write it to the // sink before we can do anything else @@ -95,16 +84,14 @@ where loop { match this.stream.try_poll_next_unpin(cx)? { - Poll::Ready(Some(item)) => { - ready!(this.try_start_send(cx, item))? - } + Poll::Ready(Some(item)) => ready!(this.try_start_send(cx, item))?, Poll::Ready(None) => { ready!(Pin::new(&mut this.sink).poll_flush(cx))?; - return Poll::Ready(Ok(())) + return Poll::Ready(Ok(())); } Poll::Pending => { ready!(Pin::new(&mut this.sink).poll_flush(cx))?; - return Poll::Pending + return Poll::Pending; } } } diff --git a/futures-util/src/sink/with.rs b/futures-util/src/sink/with.rs index a17f88af9b..f87de8c807 100644 --- a/futures-util/src/sink/with.rs +++ b/futures-util/src/sink/with.rs @@ -25,37 +25,31 @@ where Fut: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("With") - .field("sink", &self.sink) - .field("state", &self.state) - .finish() + f.debug_struct("With").field("sink", &self.sink).field("state", &self.state).finish() } } impl With -where Si: Sink, - F: FnMut(U) -> Fut, - Fut: Future, +where + Si: Sink, + F: FnMut(U) -> Fut, + Fut: Future, { pub(super) fn new(sink: Si, f: F) -> Self - where - Fut: Future>, - E: From, + where + Fut: Future>, + E: From, { - Self { - state: None, - sink, - f, - _phantom: PhantomData, - } + Self { state: None, sink, f, _phantom: PhantomData } } } // Forwarding impl of Stream from the underlying sink impl Stream for With - where S: Stream + Sink, - F: FnMut(U) -> Fut, - Fut: Future +where + S: Stream + Sink, + F: FnMut(U) -> Fut, + Fut: Future, { type Item = S::Item; @@ -63,18 +57,16 @@ impl Stream for With } impl With - where Si: Sink, - F: FnMut(U) -> Fut, - Fut: Future>, - E: From, +where + Si: Sink, + F: FnMut(U) -> Fut, + Fut: Future>, + E: From, { delegate_access_inner!(sink, Si, ()); /// Completes the processing of previous item if any. - fn poll( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); let item = match this.state.as_mut().as_pin_mut() { @@ -88,26 +80,21 @@ impl With } impl Sink for With - where Si: Sink, - F: FnMut(U) -> Fut, - Fut: Future>, - E: From, +where + Si: Sink, + F: FnMut(U) -> Fut, + Fut: Future>, + E: From, { type Error = E; - fn poll_ready( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().poll(cx))?; ready!(self.project().sink.poll_ready(cx)?); Poll::Ready(Ok(())) } - fn start_send( - self: Pin<&mut Self>, - item: U, - ) -> Result<(), Self::Error> { + fn start_send(self: Pin<&mut Self>, item: U) -> Result<(), Self::Error> { let mut this = self.project(); assert!(this.state.is_none()); @@ -115,19 +102,13 @@ impl Sink for With Ok(()) } - fn poll_flush( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().poll(cx))?; ready!(self.project().sink.poll_flush(cx)?); Poll::Ready(Ok(())) } - fn poll_close( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().poll(cx))?; ready!(self.project().sink.poll_close(cx)?); Poll::Ready(Ok(())) diff --git a/futures-util/src/sink/with_flat_map.rs b/futures-util/src/sink/with_flat_map.rs index e5376a8ff6..f1d3eec9cc 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::fmt; use core::marker::PhantomData; use core::pin::Pin; -use futures_core::stream::{Stream, FusedStream}; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_project::pin_project; @@ -41,21 +41,12 @@ where St: Stream>, { pub(super) fn new(sink: Si, f: F) -> Self { - Self { - sink, - f, - stream: None, - buffer: None, - _marker: PhantomData, - } + Self { sink, f, stream: None, buffer: None, _marker: PhantomData } } delegate_access_inner!(sink, Si, ()); - fn try_empty_stream( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn try_empty_stream(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); if this.buffer.is_some() { @@ -110,17 +101,11 @@ where { type Error = Si::Error; - fn poll_ready( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.try_empty_stream(cx) } - fn start_send( - self: Pin<&mut Self>, - item: U, - ) -> Result<(), Self::Error> { + fn start_send(self: Pin<&mut Self>, item: U) -> Result<(), Self::Error> { let mut this = self.project(); assert!(this.stream.is_none()); @@ -128,18 +113,12 @@ where Ok(()) } - fn poll_flush( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().try_empty_stream(cx)?); self.project().sink.poll_flush(cx) } - fn poll_close( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { ready!(self.as_mut().try_empty_stream(cx)?); self.project().sink.poll_close(cx) } diff --git a/futures-util/src/stream/empty.rs b/futures-util/src/stream/empty.rs index d228b31581..e73d8c7fc5 100644 --- a/futures-util/src/stream/empty.rs +++ b/futures-util/src/stream/empty.rs @@ -7,16 +7,14 @@ use futures_core::task::{Context, Poll}; #[derive(Debug)] #[must_use = "streams do nothing unless polled"] pub struct Empty { - _phantom: PhantomData + _phantom: PhantomData, } /// Creates a stream which contains no elements. /// /// The returned stream will always return `Ready(None)` when polled. pub fn empty() -> Empty { - Empty { - _phantom: PhantomData - } + Empty { _phantom: PhantomData } } impl Unpin for Empty {} diff --git a/futures-util/src/stream/iter.rs b/futures-util/src/stream/iter.rs index cab8cd81b1..77cfd47da5 100644 --- a/futures-util/src/stream/iter.rs +++ b/futures-util/src/stream/iter.rs @@ -26,15 +26,15 @@ impl Unpin for Iter {} /// # }); /// ``` pub fn iter(i: I) -> Iter - where I: IntoIterator, +where + I: IntoIterator, { - Iter { - iter: i.into_iter(), - } + Iter { iter: i.into_iter() } } impl Stream for Iter - where I: Iterator, +where + I: Iterator, { type Item = I::Item; diff --git a/futures-util/src/stream/mod.rs b/futures-util/src/stream/mod.rs index 1b95837fc0..11f2fefa1c 100644 --- a/futures-util/src/stream/mod.rs +++ b/futures-util/src/stream/mod.rs @@ -15,7 +15,7 @@ mod stream; pub use self::stream::{ Chain, Collect, Concat, Cycle, Enumerate, Filter, FilterMap, FlatMap, Flatten, Fold, ForEach, Fuse, Inspect, Map, Next, Peek, Peekable, Scan, SelectNextSome, Skip, SkipWhile, StreamExt, - StreamFuture, Take, TakeWhile, TakeUntil, Then, Zip, + StreamFuture, Take, TakeUntil, TakeWhile, Then, Unzip, Zip, }; #[cfg(feature = "std")] @@ -104,8 +104,8 @@ cfg_target_has_atomic! { // Just a helper function to ensure the futures we're returning all have the // right implementations. pub(crate) fn assert_stream(stream: S) -> S - where - S: Stream, +where + S: Stream, { stream } diff --git a/futures-util/src/stream/once.rs b/futures-util/src/stream/once.rs index 3a8fef65d9..e0181b5ade 100644 --- a/futures-util/src/stream/once.rs +++ b/futures-util/src/stream/once.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::Future; -use futures_core::stream::{Stream, FusedStream}; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; use pin_project::pin_project; @@ -25,7 +25,7 @@ pub fn once(future: Fut) -> Once { #[must_use = "streams do nothing unless polled"] pub struct Once { #[pin] - future: Option + future: Option, } impl Once { diff --git a/futures-util/src/stream/repeat.rs b/futures-util/src/stream/repeat.rs index 6a2637d3f5..e402b8a131 100644 --- a/futures-util/src/stream/repeat.rs +++ b/futures-util/src/stream/repeat.rs @@ -1,5 +1,5 @@ use core::pin::Pin; -use futures_core::stream::{Stream, FusedStream}; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; /// Stream for the [`repeat`] function. @@ -24,7 +24,8 @@ pub struct Repeat { /// # }); /// ``` pub fn repeat(item: T) -> Repeat - where T: Clone +where + T: Clone, { Repeat { item } } @@ -32,7 +33,8 @@ pub fn repeat(item: T) -> Repeat impl Unpin for Repeat {} impl Stream for Repeat - where T: Clone +where + T: Clone, { type Item = T; @@ -46,7 +48,8 @@ impl Stream for Repeat } impl FusedStream for Repeat - where T: Clone, +where + T: Clone, { fn is_terminated(&self) -> bool { false diff --git a/futures-util/src/stream/select.rs b/futures-util/src/stream/select.rs index 7666386431..09d8d44902 100644 --- a/futures-util/src/stream/select.rs +++ b/futures-util/src/stream/select.rs @@ -1,4 +1,4 @@ -use crate::stream::{StreamExt, Fuse}; +use crate::stream::{Fuse, StreamExt}; use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; @@ -27,14 +27,11 @@ pub struct Select { /// Note that this function consumes both streams and returns a wrapped /// version of them. pub fn select(stream1: St1, stream2: St2) -> Select - where St1: Stream, - St2: Stream +where + St1: Stream, + St2: Stream, { - Select { - stream1: stream1.fuse(), - stream2: stream2.fuse(), - flag: false, - } + Select { stream1: stream1.fuse(), stream2: stream2.fuse(), flag: false } } impl Select { @@ -73,8 +70,9 @@ impl Select { } impl FusedStream for Select - where St1: Stream, - St2: Stream +where + St1: Stream, + St2: Stream, { fn is_terminated(&self) -> bool { self.stream1.is_terminated() && self.stream2.is_terminated() @@ -82,15 +80,13 @@ impl FusedStream for Select } impl Stream for Select - where St1: Stream, - St2: Stream +where + St1: Stream, + St2: Stream, { type Item = St1::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); if !*this.flag { poll_inner(this.flag, this.stream1, this.stream2, cx) @@ -104,24 +100,24 @@ fn poll_inner( flag: &mut bool, a: Pin<&mut St1>, b: Pin<&mut St2>, - cx: &mut Context<'_> + cx: &mut Context<'_>, ) -> Poll> - where St1: Stream, St2: Stream +where + St1: Stream, + St2: Stream, { let a_done = match a.poll_next(cx) { Poll::Ready(Some(item)) => { // give the other stream a chance to go first next time *flag = !*flag; - return Poll::Ready(Some(item)) - }, + return Poll::Ready(Some(item)); + } Poll::Ready(None) => true, Poll::Pending => false, }; match b.poll_next(cx) { - Poll::Ready(Some(item)) => { - Poll::Ready(Some(item)) - } + Poll::Ready(Some(item)) => Poll::Ready(Some(item)), Poll::Ready(None) if a_done => Poll::Ready(None), Poll::Ready(None) | Poll::Pending => Poll::Pending, } diff --git a/futures-util/src/stream/stream/catch_unwind.rs b/futures-util/src/stream/stream/catch_unwind.rs index 20769ae23f..bf224d9deb 100644 --- a/futures-util/src/stream/stream/catch_unwind.rs +++ b/futures-util/src/stream/stream/catch_unwind.rs @@ -1,9 +1,9 @@ -use futures_core::stream::{Stream, FusedStream}; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; use pin_project::pin_project; use std::any::Any; +use std::panic::{catch_unwind, AssertUnwindSafe, UnwindSafe}; use std::pin::Pin; -use std::panic::{catch_unwind, UnwindSafe, AssertUnwindSafe}; /// Stream for the [`catch_unwind`](super::StreamExt::catch_unwind) method. #[pin_project] @@ -26,25 +26,20 @@ impl CatchUnwind { impl Stream for CatchUnwind { type Item = Result>; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); if *this.caught_unwind { Poll::Ready(None) } else { - let res = catch_unwind(AssertUnwindSafe(|| { - this.stream.as_mut().poll_next(cx) - })); + let res = catch_unwind(AssertUnwindSafe(|| this.stream.as_mut().poll_next(cx))); match res { Ok(poll) => poll.map(|opt| opt.map(Ok)), Err(e) => { *this.caught_unwind = true; Poll::Ready(Some(Err(e))) - }, + } } } } diff --git a/futures-util/src/stream/stream/chain.rs b/futures-util/src/stream/stream/chain.rs index 757ee69a3b..5b6d05474e 100644 --- a/futures-util/src/stream/stream/chain.rs +++ b/futures-util/src/stream/stream/chain.rs @@ -16,20 +16,19 @@ pub struct Chain { // All interactions with `Pin<&mut Chain<..>>` happen through these methods impl Chain -where St1: Stream, - St2: Stream, +where + St1: Stream, + St2: Stream, { pub(super) fn new(stream1: St1, stream2: St2) -> Self { - Self { - first: Some(stream1), - second: stream2, - } + Self { first: Some(stream1), second: stream2 } } } impl FusedStream for Chain -where St1: Stream, - St2: FusedStream, +where + St1: Stream, + St2: FusedStream, { fn is_terminated(&self) -> bool { self.first.is_none() && self.second.is_terminated() @@ -37,19 +36,17 @@ where St1: Stream, } impl Stream for Chain -where St1: Stream, - St2: Stream, +where + St1: Stream, + St2: Stream, { type Item = St1::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); if let Some(first) = this.first.as_mut().as_pin_mut() { if let Some(item) = ready!(first.poll_next(cx)) { - return Poll::Ready(Some(item)) + return Poll::Ready(Some(item)); } } this.first.set(None); @@ -65,7 +62,7 @@ where St1: Stream, let upper = match (first_upper, second_upper) { (Some(x), Some(y)) => x.checked_add(y), - _ => None + _ => None, }; (lower, upper) diff --git a/futures-util/src/stream/stream/chunks.rs b/futures-util/src/stream/stream/chunks.rs index 3889f888f2..9d1a3ef76d 100644 --- a/futures-util/src/stream/stream/chunks.rs +++ b/futures-util/src/stream/stream/chunks.rs @@ -1,12 +1,12 @@ use crate::stream::Fuse; -use futures_core::stream::{Stream, FusedStream}; +use alloc::vec::Vec; +use core::mem; +use core::pin::Pin; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; use pin_project::pin_project; -use core::mem; -use core::pin::Pin; -use alloc::vec::Vec; /// Stream for the [`chunks`](super::StreamExt::chunks) method. #[pin_project] @@ -19,7 +19,10 @@ pub struct Chunks { cap: usize, // https://github.com/rust-lang/futures-rs/issues/1475 } -impl Chunks where St: Stream { +impl Chunks +where + St: Stream, +{ pub(super) fn new(stream: St, capacity: usize) -> Self { assert!(capacity > 0); @@ -41,10 +44,7 @@ impl Chunks where St: Stream { impl Stream for Chunks { type Item = Vec; - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.as_mut().project(); loop { match ready!(this.stream.as_mut().poll_next(cx)) { @@ -54,7 +54,7 @@ impl Stream for Chunks { Some(item) => { this.items.push(item); if this.items.len() >= *this.cap { - return Poll::Ready(Some(self.take())) + return Poll::Ready(Some(self.take())); } } diff --git a/futures-util/src/stream/stream/collect.rs b/futures-util/src/stream/stream/collect.rs index eeb2c7461d..68d133b7eb 100644 --- a/futures-util/src/stream/stream/collect.rs +++ b/futures-util/src/stream/stream/collect.rs @@ -21,16 +21,14 @@ impl Collect { } pub(super) fn new(stream: St) -> Self { - Self { - stream, - collection: Default::default(), - } + Self { stream, collection: Default::default() } } } impl FusedFuture for Collect -where St: FusedStream, - C: Default + Extend +where + St: FusedStream, + C: Default + Extend, { fn is_terminated(&self) -> bool { self.stream.is_terminated() @@ -38,8 +36,9 @@ where St: FusedStream, } impl Future for Collect -where St: Stream, - C: Default + Extend +where + St: Stream, + C: Default + Extend, { type Output = C; diff --git a/futures-util/src/stream/stream/concat.rs b/futures-util/src/stream/stream/concat.rs index 25d46cb9dd..582b80219f 100644 --- a/futures-util/src/stream/stream/concat.rs +++ b/futures-util/src/stream/stream/concat.rs @@ -1,6 +1,6 @@ use core::pin::Pin; -use futures_core::future::{Future, FusedFuture}; -use futures_core::stream::{Stream, FusedStream}; +use futures_core::future::{FusedFuture, Future}; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; use pin_project::pin_project; @@ -15,35 +15,28 @@ pub struct Concat { } impl Concat -where St: Stream, - St::Item: Extend<::Item> + - IntoIterator + Default, +where + St: Stream, + St::Item: Extend<::Item> + IntoIterator + Default, { pub(super) fn new(stream: St) -> Self { - Self { - stream, - accum: None, - } + Self { stream, accum: None } } } impl Future for Concat -where St: Stream, - St::Item: Extend<::Item> + - IntoIterator + Default, +where + St: Stream, + St::Item: Extend<::Item> + IntoIterator + Default, { type Output = St::Item; - fn poll( - self: Pin<&mut Self>, cx: &mut Context<'_> - ) -> Poll { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let mut this = self.project(); loop { match ready!(this.stream.as_mut().poll_next(cx)) { - None => { - return Poll::Ready(this.accum.take().unwrap_or_default()) - } + None => return Poll::Ready(this.accum.take().unwrap_or_default()), Some(e) => { if let Some(a) = this.accum { a.extend(e) @@ -57,9 +50,9 @@ where St: Stream, } impl FusedFuture for Concat -where St: FusedStream, - St::Item: Extend<::Item> + - IntoIterator + Default, +where + St: FusedStream, + St::Item: Extend<::Item> + IntoIterator + Default, { fn is_terminated(&self) -> bool { self.accum.is_none() && self.stream.is_terminated() diff --git a/futures-util/src/stream/stream/cycle.rs b/futures-util/src/stream/stream/cycle.rs index b6bff8c2ca..ce97148c55 100644 --- a/futures-util/src/stream/stream/cycle.rs +++ b/futures-util/src/stream/stream/cycle.rs @@ -19,10 +19,7 @@ where St: Clone + Stream, { pub(super) fn new(stream: St) -> Self { - Self { - orig: stream.clone(), - stream, - } + Self { orig: stream.clone(), stream } } } diff --git a/futures-util/src/stream/stream/enumerate.rs b/futures-util/src/stream/stream/enumerate.rs index 2b8f8cdd17..4692b2df6e 100644 --- a/futures-util/src/stream/stream/enumerate.rs +++ b/futures-util/src/stream/stream/enumerate.rs @@ -17,10 +17,7 @@ pub struct Enumerate { impl Enumerate { pub(super) fn new(stream: St) -> Self { - Self { - stream, - count: 0, - } + Self { stream, count: 0 } } delegate_access_inner!(stream, St, ()); @@ -35,10 +32,7 @@ impl FusedStream for Enumerate { impl Stream for Enumerate { type Item = (usize, St::Item); - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); match ready!(this.stream.poll_next(cx)) { diff --git a/futures-util/src/stream/stream/filter.rs b/futures-util/src/stream/stream/filter.rs index 301d5c59b8..cfd1c54908 100644 --- a/futures-util/src/stream/stream/filter.rs +++ b/futures-util/src/stream/stream/filter.rs @@ -1,3 +1,4 @@ +use crate::fns::FnMut1; use core::fmt; use core::pin::Pin; use futures_core::future::Future; @@ -6,13 +7,13 @@ use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; use pin_project::pin_project; -use crate::fns::FnMut1; /// Stream for the [`filter`](super::StreamExt::filter) method. #[pin_project] #[must_use = "streams do nothing unless polled"] pub struct Filter - where St: Stream, +where + St: Stream, { #[pin] stream: St, @@ -39,26 +40,23 @@ where #[allow(single_use_lifetimes)] // https://github.com/rust-lang/rust/issues/55058 impl Filter -where St: Stream, - F: for<'a> FnMut1<&'a St::Item, Output=Fut>, - Fut: Future, +where + St: Stream, + F: for<'a> FnMut1<&'a St::Item, Output = Fut>, + Fut: Future, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - f, - pending_fut: None, - pending_item: None, - } + Self { stream, f, pending_fut: None, pending_item: None } } delegate_access_inner!(stream, St, ()); } impl FusedStream for Filter - where St: Stream + FusedStream, - F: FnMut(&St::Item) -> Fut, - Fut: Future, +where + St: Stream + FusedStream, + F: FnMut(&St::Item) -> Fut, + Fut: Future, { fn is_terminated(&self) -> bool { self.pending_fut.is_none() && self.stream.is_terminated() @@ -67,16 +65,14 @@ impl FusedStream for Filter #[allow(single_use_lifetimes)] // https://github.com/rust-lang/rust/issues/55058 impl Stream for Filter - where St: Stream, - F: for<'a> FnMut1<&'a St::Item, Output=Fut>, - Fut: Future, +where + St: Stream, + F: for<'a> FnMut1<&'a St::Item, Output = Fut>, + Fut: Future, { type Item = St::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); Poll::Ready(loop { if let Some(fut) = this.pending_fut.as_mut().as_pin_mut() { @@ -109,9 +105,10 @@ impl Stream for Filter // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for Filter - where S: Stream + Sink, - F: FnMut(&S::Item) -> Fut, - Fut: Future, +where + S: Stream + Sink, + F: FnMut(&S::Item) -> Fut, + Fut: Future, { type Error = S::Error; diff --git a/futures-util/src/stream/stream/filter_map.rs b/futures-util/src/stream/stream/filter_map.rs index ec9cc2bd66..aaecf14c5d 100644 --- a/futures-util/src/stream/stream/filter_map.rs +++ b/futures-util/src/stream/stream/filter_map.rs @@ -1,3 +1,4 @@ +use crate::fns::FnMut1; use core::fmt; use core::pin::Pin; use futures_core::future::Future; @@ -6,7 +7,6 @@ use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; use pin_project::pin_project; -use crate::fns::FnMut1; /// Stream for the [`filter_map`](super::StreamExt::filter_map) method. #[pin_project] @@ -33,9 +33,10 @@ where } impl FilterMap - where St: Stream, - F: FnMut(St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(St::Item) -> Fut, + Fut: Future, { pub(super) fn new(stream: St, f: F) -> Self { Self { stream, f, pending: None } @@ -45,9 +46,10 @@ impl FilterMap } impl FusedStream for FilterMap - where St: Stream + FusedStream, - F: FnMut1, - Fut: Future>, +where + St: Stream + FusedStream, + F: FnMut1, + Fut: Future>, { fn is_terminated(&self) -> bool { self.pending.is_none() && self.stream.is_terminated() @@ -55,16 +57,14 @@ impl FusedStream for FilterMap } impl Stream for FilterMap - where St: Stream, - F: FnMut1, - Fut: Future>, +where + St: Stream, + F: FnMut1, + Fut: Future>, { type Item = T; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); Poll::Ready(loop { if let Some(p) = this.pending.as_mut().as_pin_mut() { @@ -98,9 +98,10 @@ impl Stream for FilterMap // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for FilterMap - where S: Stream + Sink, - F: FnMut1, - Fut: Future, +where + S: Stream + Sink, + F: FnMut1, + Fut: Future, { type Error = S::Error; diff --git a/futures-util/src/stream/stream/fold.rs b/futures-util/src/stream/stream/fold.rs index 1ab76e6936..59f75e7922 100644 --- a/futures-util/src/stream/stream/fold.rs +++ b/futures-util/src/stream/stream/fold.rs @@ -33,24 +33,21 @@ where } impl Fold -where St: Stream, - F: FnMut(T, St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(T, St::Item) -> Fut, + Fut: Future, { pub(super) fn new(stream: St, f: F, t: T) -> Self { - Self { - stream, - f, - accum: Some(t), - future: None, - } + Self { stream, f, accum: Some(t), future: None } } } impl FusedFuture for Fold - where St: Stream, - F: FnMut(T, St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(T, St::Item) -> Fut, + Fut: Future, { fn is_terminated(&self) -> bool { self.accum.is_none() && self.future.is_none() @@ -58,9 +55,10 @@ impl FusedFuture for Fold } impl Future for Fold - where St: Stream, - F: FnMut(T, St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(T, St::Item) -> Fut, + Fut: Future, { type Output = T; diff --git a/futures-util/src/stream/stream/for_each.rs b/futures-util/src/stream/stream/for_each.rs index 393d86ea0f..513899f788 100644 --- a/futures-util/src/stream/stream/for_each.rs +++ b/futures-util/src/stream/stream/for_each.rs @@ -30,23 +30,21 @@ where } impl ForEach -where St: Stream, - F: FnMut(St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(St::Item) -> Fut, + Fut: Future, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - f, - future: None, - } + Self { stream, f, future: None } } } impl FusedFuture for ForEach - where St: FusedStream, - F: FnMut(St::Item) -> Fut, - Fut: Future, +where + St: FusedStream, + F: FnMut(St::Item) -> Fut, + Fut: Future, { fn is_terminated(&self) -> bool { self.future.is_none() && self.stream.is_terminated() @@ -54,9 +52,10 @@ impl FusedFuture for ForEach } impl Future for ForEach - where St: Stream, - F: FnMut(St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(St::Item) -> Fut, + Fut: Future, { type Output = (); diff --git a/futures-util/src/stream/stream/forward.rs b/futures-util/src/stream/stream/forward.rs index 195981515a..89c11067d5 100644 --- a/futures-util/src/stream/stream/forward.rs +++ b/futures-util/src/stream/stream/forward.rs @@ -20,11 +20,7 @@ pub struct Forward { impl Forward { pub(crate) fn new(stream: St, sink: Si) -> Self { - Self { - sink: Some(sink), - stream: Fuse::new(stream), - buffered_item: None, - } + Self { sink: Some(sink), stream: Fuse::new(stream), buffered_item: None } } } @@ -45,10 +41,7 @@ where { type Output = Result<(), E>; - fn poll( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let ForwardProj { mut sink, mut stream, buffered_item } = self.project(); let mut si = sink.as_mut().as_pin_mut().expect("polled `Forward` after completion"); @@ -67,11 +60,11 @@ where Poll::Ready(None) => { ready!(si.poll_close(cx))?; sink.set(None); - return Poll::Ready(Ok(())) + return Poll::Ready(Ok(())); } Poll::Pending => { ready!(si.poll_flush(cx))?; - return Poll::Pending + return Poll::Pending; } } } diff --git a/futures-util/src/stream/stream/fuse.rs b/futures-util/src/stream/stream/fuse.rs index f05b48c137..819e00bfc9 100644 --- a/futures-util/src/stream/stream/fuse.rs +++ b/futures-util/src/stream/stream/fuse.rs @@ -41,10 +41,7 @@ impl FusedStream for Fuse { impl Stream for Fuse { type Item = S::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.project(); if *this.done { diff --git a/futures-util/src/stream/stream/into_future.rs b/futures-util/src/stream/stream/into_future.rs index 2ec17a5dc4..dc1a7505c3 100644 --- a/futures-util/src/stream/stream/into_future.rs +++ b/futures-util/src/stream/stream/into_future.rs @@ -78,10 +78,7 @@ impl FusedFuture for StreamFuture { impl Future for StreamFuture { type Output = (Option, St); - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let item = { let s = self.stream.as_mut().expect("polling StreamFuture twice"); ready!(s.poll_next_unpin(cx)) diff --git a/futures-util/src/stream/stream/map.rs b/futures-util/src/stream/stream/map.rs index a78a47979e..44b043477d 100644 --- a/futures-util/src/stream/stream/map.rs +++ b/futures-util/src/stream/stream/map.rs @@ -22,9 +22,7 @@ where St: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Map") - .field("stream", &self.stream) - .finish() + f.debug_struct("Map").field("stream", &self.stream).finish() } } @@ -37,8 +35,9 @@ impl Map { } impl FusedStream for Map - where St: FusedStream, - F: FnMut1, +where + St: FusedStream, + F: FnMut1, { fn is_terminated(&self) -> bool { self.stream.is_terminated() @@ -46,15 +45,13 @@ impl FusedStream for Map } impl Stream for Map - where St: Stream, - F: FnMut1, +where + St: Stream, + F: FnMut1, { type Item = F::Output; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); let res = ready!(this.stream.as_mut().poll_next(cx)); Poll::Ready(res.map(|x| this.f.call_mut(x))) @@ -68,8 +65,9 @@ impl Stream for Map // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for Map - where St: Stream + Sink, - F: FnMut1, +where + St: Stream + Sink, + F: FnMut1, { type Error = St::Error; diff --git a/futures-util/src/stream/stream/mod.rs b/futures-util/src/stream/stream/mod.rs index 84b0acd862..29f82baef8 100644 --- a/futures-util/src/stream/stream/mod.rs +++ b/futures-util/src/stream/stream/mod.rs @@ -19,7 +19,7 @@ use futures_core::{ #[cfg(feature = "sink")] use futures_sink::Sink; -use crate::fns::{InspectFn, inspect_fn}; +use crate::fns::{inspect_fn, InspectFn}; mod chain; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 @@ -29,6 +29,10 @@ mod collect; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::collect::Collect; +mod unzip; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::unzip::Unzip; + mod concat; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::concat::Concat; @@ -477,6 +481,45 @@ pub trait StreamExt: Stream { assert_future::(Collect::new(self)) } + /// Converts a stream of pairs into a future, which + /// resolves to pair of containers. + /// + /// `unzip()` produces a future, which resolves to two + /// collections: one from the left elements of the pairs, + /// and one from the right elements. + /// + /// The returned future will be resolved when the stream terminates. + /// + /// # Examples + /// + /// ``` + /// # futures::executor::block_on(async { + /// use futures::channel::mpsc; + /// use futures::stream::StreamExt; + /// use std::thread; + /// + /// let (tx, rx) = mpsc::unbounded(); + /// + /// thread::spawn(move || { + /// tx.unbounded_send((1, 2)).unwrap(); + /// tx.unbounded_send((3, 4)).unwrap(); + /// tx.unbounded_send((5, 6)).unwrap(); + /// }); + /// + /// let (o1, o2): (Vec<_>, Vec<_>) = rx.unzip().await; + /// assert_eq!(o1, vec![1, 3, 5]); + /// assert_eq!(o2, vec![2, 4, 6]); + /// # }); + /// ``` + fn unzip(self) -> Unzip + where + FromA: Default + Extend, + FromB: Default + Extend, + Self: Sized + Stream, + { + assert_future::<(FromA, FromB), _>(Unzip::new(self)) + } + /// Concatenate all items of a stream into a single extendable /// destination, returning a future representing the end result. /// @@ -1269,8 +1312,8 @@ pub trait StreamExt: Stream { /// This method will panic if `capacity` is zero. #[cfg(feature = "alloc")] fn ready_chunks(self, capacity: usize) -> ReadyChunks - where - Self: Sized, + where + Self: Sized, { ReadyChunks::new(self, capacity) } diff --git a/futures-util/src/stream/stream/next.rs b/futures-util/src/stream/stream/next.rs index 6949878bef..8d8347aa03 100644 --- a/futures-util/src/stream/stream/next.rs +++ b/futures-util/src/stream/stream/next.rs @@ -28,10 +28,7 @@ impl FusedFuture for Next<'_, St> { impl Future for Next<'_, St> { type Output = Option; - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { self.stream.poll_next_unpin(cx) } } diff --git a/futures-util/src/stream/stream/peek.rs b/futures-util/src/stream/stream/peek.rs index 0386c43a63..8b6b03e2d2 100644 --- a/futures-util/src/stream/stream/peek.rs +++ b/futures-util/src/stream/stream/peek.rs @@ -24,10 +24,7 @@ pub struct Peekable { impl Peekable { pub(super) fn new(stream: St) -> Self { - Self { - stream: stream.fuse(), - peeked: None, - } + Self { stream: stream.fuse(), peeked: None } } delegate_access_inner!(stream, St, (.)); @@ -42,10 +39,7 @@ impl Peekable { /// /// This method polls the underlying stream and return either a reference /// to the next item if the stream is ready or passes through any errors. - pub fn poll_peek( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + pub fn poll_peek(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); Poll::Ready(loop { @@ -113,9 +107,7 @@ where St::Item: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Peek") - .field("inner", &self.inner) - .finish() + f.debug_struct("Peek").field("inner", &self.inner).finish() } } diff --git a/futures-util/src/stream/stream/ready_chunks.rs b/futures-util/src/stream/stream/ready_chunks.rs index 6cc51a5d78..5c0f209a9c 100644 --- a/futures-util/src/stream/stream/ready_chunks.rs +++ b/futures-util/src/stream/stream/ready_chunks.rs @@ -1,12 +1,12 @@ use crate::stream::Fuse; -use futures_core::stream::{Stream, FusedStream}; +use alloc::vec::Vec; +use core::mem; +use core::pin::Pin; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; use pin_project::pin_project; -use core::mem; -use core::pin::Pin; -use alloc::vec::Vec; /// Stream for the [`ready_chunks`](super::StreamExt::ready_chunks) method. #[pin_project] @@ -19,7 +19,10 @@ pub struct ReadyChunks { cap: usize, // https://github.com/rust-lang/futures-rs/issues/1475 } -impl ReadyChunks where St: Stream { +impl ReadyChunks +where + St: Stream, +{ pub(super) fn new(stream: St, capacity: usize) -> Self { assert!(capacity > 0); @@ -36,10 +39,7 @@ impl ReadyChunks where St: Stream { impl Stream for ReadyChunks { type Item = Vec; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); loop { @@ -60,7 +60,10 @@ impl Stream for ReadyChunks { Poll::Ready(Some(item)) => { this.items.push(item); if this.items.len() >= *this.cap { - return Poll::Ready(Some(mem::replace(this.items, Vec::with_capacity(*this.cap)))) + return Poll::Ready(Some(mem::replace( + this.items, + Vec::with_capacity(*this.cap), + ))); } } diff --git a/futures-util/src/stream/stream/scan.rs b/futures-util/src/stream/stream/scan.rs index 400ed984e0..5aa6dc867d 100644 --- a/futures-util/src/stream/stream/scan.rs +++ b/futures-util/src/stream/stream/scan.rs @@ -54,14 +54,7 @@ where Fut: Future>, { pub(super) fn new(stream: St, initial_state: S, f: F) -> Self { - Self { - stream, - state_f: Some(StateFn { - state: initial_state, - f, - }), - future: None, - } + Self { stream, state_f: Some(StateFn { state: initial_state, f }), future: None } } delegate_access_inner!(stream, St, ()); diff --git a/futures-util/src/stream/stream/select_next_some.rs b/futures-util/src/stream/stream/select_next_some.rs index 00d38eb63c..4209203c4b 100644 --- a/futures-util/src/stream/stream/select_next_some.rs +++ b/futures-util/src/stream/stream/select_next_some.rs @@ -1,8 +1,8 @@ +use crate::stream::StreamExt; use core::pin::Pin; +use futures_core::future::{FusedFuture, Future}; use futures_core::stream::FusedStream; -use futures_core::future::{Future, FusedFuture}; use futures_core::task::{Context, Poll}; -use crate::stream::StreamExt; /// Future for the [`select_next_some`](super::StreamExt::select_next_some) /// method. diff --git a/futures-util/src/stream/stream/skip.rs b/futures-util/src/stream/stream/skip.rs index 4e62251219..2dae231018 100644 --- a/futures-util/src/stream/stream/skip.rs +++ b/futures-util/src/stream/stream/skip.rs @@ -17,10 +17,7 @@ pub struct Skip { impl Skip { pub(super) fn new(stream: St, n: usize) -> Self { - Self { - stream, - remaining: n, - } + Self { stream, remaining: n } } delegate_access_inner!(stream, St, ()); @@ -35,10 +32,7 @@ impl FusedStream for Skip { impl Stream for Skip { type Item = St::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); while *this.remaining > 0 { diff --git a/futures-util/src/stream/stream/skip_while.rs b/futures-util/src/stream/stream/skip_while.rs index dba11d9253..f0d2066d0a 100644 --- a/futures-util/src/stream/stream/skip_while.rs +++ b/futures-util/src/stream/stream/skip_while.rs @@ -10,7 +10,10 @@ use pin_project::pin_project; /// Stream for the [`skip_while`](super::StreamExt::skip_while) method. #[pin_project] #[must_use = "streams do nothing unless polled"] -pub struct SkipWhile where St: Stream { +pub struct SkipWhile +where + St: Stream, +{ #[pin] stream: St, f: F, @@ -37,27 +40,23 @@ where } impl SkipWhile - where St: Stream, - F: FnMut(&St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(&St::Item) -> Fut, + Fut: Future, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - f, - pending_fut: None, - pending_item: None, - done_skipping: false, - } + Self { stream, f, pending_fut: None, pending_item: None, done_skipping: false } } delegate_access_inner!(stream, St, ()); } impl FusedStream for SkipWhile - where St: FusedStream, - F: FnMut(&St::Item) -> Fut, - Fut: Future, +where + St: FusedStream, + F: FnMut(&St::Item) -> Fut, + Fut: Future, { fn is_terminated(&self) -> bool { self.pending_item.is_none() && self.stream.is_terminated() @@ -65,16 +64,14 @@ impl FusedStream for SkipWhile } impl Stream for SkipWhile - where St: Stream, - F: FnMut(&St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(&St::Item) -> Fut, + Fut: Future, { type Item = St::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); if *this.done_skipping { @@ -117,9 +114,10 @@ impl Stream for SkipWhile // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for SkipWhile - where S: Stream + Sink, - F: FnMut(&S::Item) -> Fut, - Fut: Future, +where + S: Stream + Sink, + F: FnMut(&S::Item) -> Fut, + Fut: Future, { type Error = S::Error; diff --git a/futures-util/src/stream/stream/take.rs b/futures-util/src/stream/stream/take.rs index 4a7cfc59fe..b3743055b2 100644 --- a/futures-util/src/stream/stream/take.rs +++ b/futures-util/src/stream/stream/take.rs @@ -1,6 +1,6 @@ use core::cmp; use core::pin::Pin; -use futures_core::stream::{Stream, FusedStream}; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; @@ -18,24 +18,19 @@ pub struct Take { impl Take { pub(super) fn new(stream: St, n: usize) -> Self { - Self { - stream, - remaining: n, - } + Self { stream, remaining: n } } delegate_access_inner!(stream, St, ()); } impl Stream for Take - where St: Stream, +where + St: Stream, { type Item = St::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { if self.remaining == 0 { Poll::Ready(None) } else { @@ -61,7 +56,7 @@ impl Stream for Take let upper = match upper { Some(x) if x < self.remaining as usize => Some(x), - _ => Some(self.remaining as usize) + _ => Some(self.remaining as usize), }; (lower, upper) @@ -69,7 +64,8 @@ impl Stream for Take } impl FusedStream for Take - where St: FusedStream, +where + St: FusedStream, { fn is_terminated(&self) -> bool { self.remaining == 0 || self.stream.is_terminated() @@ -79,7 +75,8 @@ impl FusedStream for Take // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for Take - where S: Stream + Sink, +where + S: Stream + Sink, { type Error = S::Error; diff --git a/futures-util/src/stream/stream/take_until.rs b/futures-util/src/stream/stream/take_until.rs index 11be3fab2d..7952c65930 100644 --- a/futures-util/src/stream/stream/take_until.rs +++ b/futures-util/src/stream/stream/take_until.rs @@ -32,10 +32,7 @@ where Fut: Future + fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("TakeUntil") - .field("stream", &self.stream) - .field("fut", &self.fut) - .finish() + f.debug_struct("TakeUntil").field("stream", &self.stream).field("fut", &self.fut).finish() } } @@ -45,12 +42,7 @@ where Fut: Future, { pub(super) fn new(stream: St, fut: Fut) -> Self { - Self { - stream, - fut: Some(fut), - fut_result: None, - free: false, - } + Self { stream, fut: Some(fut), fut_result: None, free: false } } delegate_access_inner!(stream, St, ()); diff --git a/futures-util/src/stream/stream/take_while.rs b/futures-util/src/stream/stream/take_while.rs index 866d5b886e..d1796195a6 100644 --- a/futures-util/src/stream/stream/take_while.rs +++ b/futures-util/src/stream/stream/take_while.rs @@ -1,7 +1,7 @@ use core::fmt; use core::pin::Pin; use futures_core::future::Future; -use futures_core::stream::{Stream, FusedStream}; +use futures_core::stream::{FusedStream, Stream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; @@ -37,34 +37,27 @@ where } impl TakeWhile - where St: Stream, - F: FnMut(&St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(&St::Item) -> Fut, + Fut: Future, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - f, - pending_fut: None, - pending_item: None, - done_taking: false, - } + Self { stream, f, pending_fut: None, pending_item: None, done_taking: false } } delegate_access_inner!(stream, St, ()); } impl Stream for TakeWhile - where St: Stream, - F: FnMut(&St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(&St::Item) -> Fut, + Fut: Future, { type Item = St::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { if self.done_taking { return Poll::Ready(None); } @@ -107,9 +100,10 @@ impl Stream for TakeWhile } impl FusedStream for TakeWhile - where St: FusedStream, - F: FnMut(&St::Item) -> Fut, - Fut: Future, +where + St: FusedStream, + F: FnMut(&St::Item) -> Fut, + Fut: Future, { fn is_terminated(&self) -> bool { self.done_taking || self.pending_item.is_none() && self.stream.is_terminated() @@ -119,7 +113,8 @@ impl FusedStream for TakeWhile // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for TakeWhile - where S: Stream + Sink, +where + S: Stream + Sink, { type Error = S::Error; diff --git a/futures-util/src/stream/stream/then.rs b/futures-util/src/stream/stream/then.rs index 0b7dda4057..f6b18c3244 100644 --- a/futures-util/src/stream/stream/then.rs +++ b/futures-util/src/stream/stream/then.rs @@ -24,32 +24,27 @@ where Fut: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Then") - .field("stream", &self.stream) - .field("future", &self.future) - .finish() + f.debug_struct("Then").field("stream", &self.stream).field("future", &self.future).finish() } } impl Then - where St: Stream, - F: FnMut(St::Item) -> Fut, +where + St: Stream, + F: FnMut(St::Item) -> Fut, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - future: None, - f, - } + Self { stream, future: None, f } } delegate_access_inner!(stream, St, ()); } impl FusedStream for Then - where St: FusedStream, - F: FnMut(St::Item) -> Fut, - Fut: Future, +where + St: FusedStream, + F: FnMut(St::Item) -> Fut, + Fut: Future, { fn is_terminated(&self) -> bool { self.future.is_none() && self.stream.is_terminated() @@ -57,16 +52,14 @@ impl FusedStream for Then } impl Stream for Then - where St: Stream, - F: FnMut(St::Item) -> Fut, - Fut: Future, +where + St: Stream, + F: FnMut(St::Item) -> Fut, + Fut: Future, { type Item = Fut::Output; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); Poll::Ready(loop { @@ -97,7 +90,8 @@ impl Stream for Then // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for Then - where S: Sink, +where + S: Sink, { type Error = S::Error; diff --git a/futures-util/src/stream/stream/unzip.rs b/futures-util/src/stream/stream/unzip.rs new file mode 100644 index 0000000000..3c4435f566 --- /dev/null +++ b/futures-util/src/stream/stream/unzip.rs @@ -0,0 +1,61 @@ +use core::mem; +use core::pin::Pin; +use futures_core::future::{FusedFuture, Future}; +use futures_core::stream::{FusedStream, Stream}; +use futures_core::task::{Context, Poll}; +use pin_project::pin_project; + +/// Future for the [`unzip`](super::StreamExt::unzip) method. +#[pin_project] +#[derive(Debug)] +#[must_use = "futures do nothing unless you `.await` or poll them"] +pub struct Unzip { + #[pin] + stream: St, + left: FromA, + right: FromB, +} + +impl Unzip { + fn finish(self: Pin<&mut Self>) -> (FromA, FromB) { + let this = self.project(); + (mem::replace(this.left, Default::default()), mem::replace(this.right, Default::default())) + } + + pub(super) fn new(stream: St) -> Self { + Self { stream, left: Default::default(), right: Default::default() } + } +} + +impl FusedFuture for Unzip +where + St: FusedStream, + FromA: Default + Extend, + FromB: Default + Extend, +{ + fn is_terminated(&self) -> bool { + self.stream.is_terminated() + } +} + +impl Future for Unzip +where + St: Stream, + FromA: Default + Extend, + FromB: Default + Extend, +{ + type Output = (FromA, FromB); + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<(FromA, FromB)> { + let mut this = self.as_mut().project(); + loop { + match ready!(this.stream.as_mut().poll_next(cx)) { + Some(e) => { + this.left.extend(Some(e.0)); + this.right.extend(Some(e.1)); + } + None => return Poll::Ready(self.finish()), + } + } + } +} diff --git a/futures-util/src/stream/stream/zip.rs b/futures-util/src/stream/stream/zip.rs index 244748e9e5..1aa3e941c9 100644 --- a/futures-util/src/stream/stream/zip.rs +++ b/futures-util/src/stream/stream/zip.rs @@ -1,4 +1,4 @@ -use crate::stream::{StreamExt, Fuse}; +use crate::stream::{Fuse, StreamExt}; use core::cmp; use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; @@ -20,12 +20,7 @@ pub struct Zip { impl Zip { pub(super) fn new(stream1: St1, stream2: St2) -> Self { - Self { - stream1: stream1.fuse(), - stream2: stream2.fuse(), - queued1: None, - queued2: None, - } + Self { stream1: stream1.fuse(), stream2: stream2.fuse(), queued1: None, queued2: None } } /// Acquires a reference to the underlying streams that this combinator is @@ -63,7 +58,9 @@ impl Zip { } impl FusedStream for Zip - where St1: Stream, St2: Stream, +where + St1: Stream, + St2: Stream, { fn is_terminated(&self) -> bool { self.stream1.is_terminated() && self.stream2.is_terminated() @@ -71,14 +68,13 @@ impl FusedStream for Zip } impl Stream for Zip - where St1: Stream, St2: Stream +where + St1: Stream, + St2: Stream, { type Item = (St1::Item, St2::Item); - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); if this.queued1.is_none() { @@ -123,7 +119,7 @@ impl Stream for Zip } (Some(x), None) => x.checked_add(queued1_len), (None, Some(y)) => y.checked_add(queued2_len), - (None, None) => None + (None, None) => None, }; (lower, upper) diff --git a/futures-util/src/stream/try_stream/and_then.rs b/futures-util/src/stream/try_stream/and_then.rs index 4d24c4fbb3..8cfb8d6183 100644 --- a/futures-util/src/stream/try_stream/and_then.rs +++ b/futures-util/src/stream/try_stream/and_then.rs @@ -1,7 +1,7 @@ use core::fmt; use core::pin::Pin; use futures_core::future::TryFuture; -use futures_core::stream::{Stream, TryStream, FusedStream}; +use futures_core::stream::{FusedStream, Stream, TryStream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; @@ -32,9 +32,10 @@ where } impl AndThen - where St: TryStream, - F: FnMut(St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(St::Ok) -> Fut, + Fut: TryFuture, { pub(super) fn new(stream: St, f: F) -> Self { Self { stream, future: None, f } @@ -44,16 +45,14 @@ impl AndThen } impl Stream for AndThen - where St: TryStream, - F: FnMut(St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(St::Ok) -> Fut, + Fut: TryFuture, { type Item = Result; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); Poll::Ready(loop { @@ -82,9 +81,10 @@ impl Stream for AndThen } impl FusedStream for AndThen - where St: TryStream + FusedStream, - F: FnMut(St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream + FusedStream, + F: FnMut(St::Ok) -> Fut, + Fut: TryFuture, { fn is_terminated(&self) -> bool { self.future.is_none() && self.stream.is_terminated() @@ -94,7 +94,8 @@ impl FusedStream for AndThen // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for AndThen - where S: Sink, +where + S: Sink, { type Error = S::Error; diff --git a/futures-util/src/stream/try_stream/into_async_read.rs b/futures-util/src/stream/try_stream/into_async_read.rs index 99e3e3b7c2..482d07beee 100644 --- a/futures-util/src/stream/try_stream/into_async_read.rs +++ b/futures-util/src/stream/try_stream/into_async_read.rs @@ -2,7 +2,7 @@ use crate::stream::TryStreamExt; use core::pin::Pin; use futures_core::stream::TryStream; use futures_core::task::{Context, Poll}; -use futures_io::{AsyncRead, AsyncWrite, AsyncBufRead}; +use futures_io::{AsyncBufRead, AsyncRead, AsyncWrite}; use std::cmp; use std::io::{Error, Result}; @@ -39,10 +39,7 @@ where St::Ok: AsRef<[u8]>, { pub(super) fn new(stream: St) -> Self { - Self { - stream, - state: ReadState::PendingChunk, - } + Self { stream, state: ReadState::PendingChunk } } } @@ -62,9 +59,7 @@ where let chunk = chunk.as_ref(); let len = cmp::min(buf.len(), chunk.len() - *chunk_start); - buf[..len].copy_from_slice( - &chunk[*chunk_start..*chunk_start + len], - ); + buf[..len].copy_from_slice(&chunk[*chunk_start..*chunk_start + len]); *chunk_start += len; if chunk.len() == *chunk_start { @@ -73,26 +68,21 @@ where return Poll::Ready(Ok(len)); } - ReadState::PendingChunk => { - match ready!(self.stream.try_poll_next_unpin(cx)) { - Some(Ok(chunk)) => { - if !chunk.as_ref().is_empty() { - self.state = ReadState::Ready { - chunk, - chunk_start: 0, - }; - } - } - Some(Err(err)) => { - self.state = ReadState::Eof; - return Poll::Ready(Err(err)); - } - None => { - self.state = ReadState::Eof; - return Poll::Ready(Ok(0)); + ReadState::PendingChunk => match ready!(self.stream.try_poll_next_unpin(cx)) { + Some(Ok(chunk)) => { + if !chunk.as_ref().is_empty() { + self.state = ReadState::Ready { chunk, chunk_start: 0 }; } } - } + Some(Err(err)) => { + self.state = ReadState::Eof; + return Poll::Ready(Err(err)); + } + None => { + self.state = ReadState::Eof; + return Poll::Ready(Ok(0)); + } + }, ReadState::Eof => { return Poll::Ready(Ok(0)); } @@ -109,23 +99,17 @@ where fn poll_write( mut self: Pin<&mut Self>, cx: &mut Context<'_>, - buf: &[u8] + buf: &[u8], ) -> Poll> { - Pin::new( &mut self.stream ).poll_write( cx, buf ) + Pin::new(&mut self.stream).poll_write(cx, buf) } - fn poll_flush( - mut self: Pin<&mut Self>, - cx: &mut Context<'_> - ) -> Poll> { - Pin::new( &mut self.stream ).poll_flush( cx ) + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut self.stream).poll_flush(cx) } - fn poll_close( - mut self: Pin<&mut Self>, - cx: &mut Context<'_> - ) -> Poll> { - Pin::new( &mut self.stream ).poll_close( cx ) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut self.stream).poll_close(cx) } } @@ -134,18 +118,12 @@ where St: TryStream + Unpin, St::Ok: AsRef<[u8]>, { - fn poll_fill_buf( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_fill_buf(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { while let ReadState::PendingChunk = self.state { match ready!(self.stream.try_poll_next_unpin(cx)) { Some(Ok(chunk)) => { if !chunk.as_ref().is_empty() { - self.state = ReadState::Ready { - chunk, - chunk_start: 0, - }; + self.state = ReadState::Ready { chunk, chunk_start: 0 }; } } Some(Err(err)) => { @@ -168,12 +146,11 @@ where Poll::Ready(Ok(&[])) } - fn consume( - mut self: Pin<&mut Self>, - amount: usize, - ) { - // https://github.com/rust-lang/futures-rs/pull/1556#discussion_r281644295 - if amount == 0 { return } + fn consume(mut self: Pin<&mut Self>, amount: usize) { + // https://github.com/rust-lang/futures-rs/pull/1556#discussion_r281644295 + if amount == 0 { + return; + } if let ReadState::Ready { chunk, chunk_start } = &mut self.state { *chunk_start += amount; debug_assert!(*chunk_start <= chunk.as_ref().len()); diff --git a/futures-util/src/stream/try_stream/into_stream.rs b/futures-util/src/stream/try_stream/into_stream.rs index b466673c82..2f32d3946a 100644 --- a/futures-util/src/stream/try_stream/into_stream.rs +++ b/futures-util/src/stream/try_stream/into_stream.rs @@ -33,10 +33,7 @@ impl Stream for IntoStream { type Item = Result; #[inline] - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.project().stream.try_poll_next(cx) } diff --git a/futures-util/src/stream/try_stream/mod.rs b/futures-util/src/stream/try_stream/mod.rs index 6a48a4c4b4..0f339a860f 100644 --- a/futures-util/src/stream/try_stream/mod.rs +++ b/futures-util/src/stream/try_stream/mod.rs @@ -5,16 +5,17 @@ #[cfg(feature = "compat")] use crate::compat::Compat; +use crate::fns::{ + inspect_err_fn, inspect_ok_fn, into_fn, map_err_fn, map_ok_fn, InspectErrFn, InspectOkFn, + IntoFn, MapErrFn, MapOkFn, +}; +use crate::stream::{Inspect, Map}; use core::pin::Pin; use futures_core::{ future::{Future, TryFuture}, stream::TryStream, task::{Context, Poll}, }; -use crate::fns::{ - InspectOkFn, inspect_ok_fn, InspectErrFn, inspect_err_fn, MapErrFn, map_err_fn, IntoFn, into_fn, MapOkFn, map_ok_fn, -}; -use crate::stream::{Map, Inspect}; mod and_then; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 diff --git a/futures-util/src/stream/try_stream/or_else.rs b/futures-util/src/stream/try_stream/or_else.rs index d972c0f064..618bd1d021 100644 --- a/futures-util/src/stream/try_stream/or_else.rs +++ b/futures-util/src/stream/try_stream/or_else.rs @@ -1,7 +1,7 @@ use core::fmt; use core::pin::Pin; use futures_core::future::TryFuture; -use futures_core::stream::{Stream, TryStream, FusedStream}; +use futures_core::stream::{FusedStream, Stream, TryStream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; @@ -32,9 +32,10 @@ where } impl OrElse - where St: TryStream, - F: FnMut(St::Error) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(St::Error) -> Fut, + Fut: TryFuture, { pub(super) fn new(stream: St, f: F) -> Self { Self { stream, future: None, f } @@ -44,16 +45,14 @@ impl OrElse } impl Stream for OrElse - where St: TryStream, - F: FnMut(St::Error) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(St::Error) -> Fut, + Fut: TryFuture, { type Item = Result; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); Poll::Ready(loop { @@ -66,7 +65,7 @@ impl Stream for OrElse Some(Ok(item)) => break Some(Ok(item)), Some(Err(e)) => { this.future.set(Some((this.f)(e))); - }, + } None => break None, } } @@ -86,9 +85,10 @@ impl Stream for OrElse } impl FusedStream for OrElse - where St: TryStream + FusedStream, - F: FnMut(St::Error) -> Fut, - Fut: TryFuture, +where + St: TryStream + FusedStream, + F: FnMut(St::Error) -> Fut, + Fut: TryFuture, { fn is_terminated(&self) -> bool { self.future.is_none() && self.stream.is_terminated() @@ -98,7 +98,8 @@ impl FusedStream for OrElse // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for OrElse - where S: Sink, +where + S: Sink, { type Error = S::Error; diff --git a/futures-util/src/stream/try_stream/try_collect.rs b/futures-util/src/stream/try_stream/try_collect.rs index bd85e816ba..dc61a4749e 100644 --- a/futures-util/src/stream/try_stream/try_collect.rs +++ b/futures-util/src/stream/try_stream/try_collect.rs @@ -17,10 +17,7 @@ pub struct TryCollect { impl TryCollect { pub(super) fn new(s: St) -> Self { - Self { - stream: s, - items: Default::default(), - } + Self { stream: s, items: Default::default() } } } @@ -41,10 +38,7 @@ where { type Output = Result; - fn poll( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let mut this = self.project(); Poll::Ready(Ok(loop { match ready!(this.stream.as_mut().try_poll_next(cx)?) { diff --git a/futures-util/src/stream/try_stream/try_concat.rs b/futures-util/src/stream/try_stream/try_concat.rs index ea992280cb..e957999b73 100644 --- a/futures-util/src/stream/try_stream/try_concat.rs +++ b/futures-util/src/stream/try_stream/try_concat.rs @@ -20,10 +20,7 @@ where St::Ok: Extend<::Item> + IntoIterator + Default, { pub(super) fn new(stream: St) -> Self { - Self { - stream, - accum: None, - } + Self { stream, accum: None } } } diff --git a/futures-util/src/stream/try_stream/try_filter.rs b/futures-util/src/stream/try_stream/try_filter.rs index f38f36f908..b9a420c3da 100644 --- a/futures-util/src/stream/try_stream/try_filter.rs +++ b/futures-util/src/stream/try_stream/try_filter.rs @@ -1,7 +1,7 @@ use core::fmt; use core::pin::Pin; use futures_core::future::Future; -use futures_core::stream::{Stream, TryStream, FusedStream}; +use futures_core::stream::{FusedStream, Stream, TryStream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; @@ -12,7 +12,8 @@ use pin_project::pin_project; #[pin_project] #[must_use = "streams do nothing unless polled"] pub struct TryFilter - where St: TryStream +where + St: TryStream, { #[pin] stream: St, @@ -38,24 +39,21 @@ where } impl TryFilter - where St: TryStream +where + St: TryStream, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - f, - pending_fut: None, - pending_item: None, - } + Self { stream, f, pending_fut: None, pending_item: None } } delegate_access_inner!(stream, St, ()); } impl FusedStream for TryFilter - where St: TryStream + FusedStream, - F: FnMut(&St::Ok) -> Fut, - Fut: Future, +where + St: TryStream + FusedStream, + F: FnMut(&St::Ok) -> Fut, + Fut: Future, { fn is_terminated(&self) -> bool { self.pending_fut.is_none() && self.stream.is_terminated() @@ -63,16 +61,14 @@ impl FusedStream for TryFilter } impl Stream for TryFilter - where St: TryStream, - Fut: Future, - F: FnMut(&St::Ok) -> Fut, +where + St: TryStream, + Fut: Future, + F: FnMut(&St::Ok) -> Fut, { type Item = Result; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); Poll::Ready(loop { @@ -106,7 +102,8 @@ impl Stream for TryFilter // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for TryFilter - where S: TryStream + Sink, +where + S: TryStream + Sink, { type Error = E; diff --git a/futures-util/src/stream/try_stream/try_filter_map.rs b/futures-util/src/stream/try_stream/try_filter_map.rs index 519638d525..1c7a52e5c7 100644 --- a/futures-util/src/stream/try_stream/try_filter_map.rs +++ b/futures-util/src/stream/try_stream/try_filter_map.rs @@ -1,7 +1,7 @@ use core::fmt; use core::pin::Pin; -use futures_core::future::{TryFuture}; -use futures_core::stream::{Stream, TryStream, FusedStream}; +use futures_core::future::TryFuture; +use futures_core::stream::{FusedStream, Stream, TryStream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; @@ -41,9 +41,10 @@ impl TryFilterMap { } impl FusedStream for TryFilterMap - where St: TryStream + FusedStream, - Fut: TryFuture, Error = St::Error>, - F: FnMut(St::Ok) -> Fut, +where + St: TryStream + FusedStream, + Fut: TryFuture, Error = St::Error>, + F: FnMut(St::Ok) -> Fut, { fn is_terminated(&self) -> bool { self.pending.is_none() && self.stream.is_terminated() @@ -51,16 +52,14 @@ impl FusedStream for TryFilterMap } impl Stream for TryFilterMap - where St: TryStream, - Fut: TryFuture, Error = St::Error>, - F: FnMut(St::Ok) -> Fut, +where + St: TryStream, + Fut: TryFuture, Error = St::Error>, + F: FnMut(St::Ok) -> Fut, { type Item = Result; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); Poll::Ready(loop { @@ -96,7 +95,8 @@ impl Stream for TryFilterMap // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for TryFilterMap - where S: Sink, +where + S: Sink, { type Error = S::Error; diff --git a/futures-util/src/stream/try_stream/try_fold.rs b/futures-util/src/stream/try_stream/try_fold.rs index 42b16f9412..46afa060b1 100644 --- a/futures-util/src/stream/try_stream/try_fold.rs +++ b/futures-util/src/stream/try_stream/try_fold.rs @@ -33,24 +33,21 @@ where } impl TryFold -where St: TryStream, - F: FnMut(T, St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(T, St::Ok) -> Fut, + Fut: TryFuture, { pub(super) fn new(stream: St, f: F, t: T) -> Self { - Self { - stream, - f, - accum: Some(t), - future: None, - } + Self { stream, f, accum: Some(t), future: None } } } impl FusedFuture for TryFold - where St: TryStream, - F: FnMut(T, St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(T, St::Ok) -> Fut, + Fut: TryFuture, { fn is_terminated(&self) -> bool { self.accum.is_none() && self.future.is_none() @@ -58,9 +55,10 @@ impl FusedFuture for TryFold } impl Future for TryFold - where St: TryStream, - F: FnMut(T, St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(T, St::Ok) -> Fut, + Fut: TryFuture, { type Output = Result; diff --git a/futures-util/src/stream/try_stream/try_for_each.rs b/futures-util/src/stream/try_stream/try_for_each.rs index 7fecc5b618..4473400980 100644 --- a/futures-util/src/stream/try_stream/try_for_each.rs +++ b/futures-util/src/stream/try_stream/try_for_each.rs @@ -30,23 +30,21 @@ where } impl TryForEach -where St: TryStream, - F: FnMut(St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(St::Ok) -> Fut, + Fut: TryFuture, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - f, - future: None, - } + Self { stream, f, future: None } } } impl Future for TryForEach - where St: TryStream, - F: FnMut(St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(St::Ok) -> Fut, + Fut: TryFuture, { type Output = Result<(), St::Error>; diff --git a/futures-util/src/stream/try_stream/try_next.rs b/futures-util/src/stream/try_stream/try_next.rs index 1bc00fbc2d..13fcf80cae 100644 --- a/futures-util/src/stream/try_stream/try_next.rs +++ b/futures-util/src/stream/try_stream/try_next.rs @@ -28,10 +28,7 @@ impl FusedFuture for TryNext<'_, S impl Future for TryNext<'_, St> { type Output = Result, St::Error>; - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { self.stream.try_poll_next_unpin(cx)?.map(Ok) } } diff --git a/futures-util/src/stream/try_stream/try_skip_while.rs b/futures-util/src/stream/try_stream/try_skip_while.rs index eeda8e1126..df2dc011b9 100644 --- a/futures-util/src/stream/try_stream/try_skip_while.rs +++ b/futures-util/src/stream/try_stream/try_skip_while.rs @@ -1,7 +1,7 @@ use core::fmt; use core::pin::Pin; use futures_core::future::TryFuture; -use futures_core::stream::{Stream, TryStream, FusedStream}; +use futures_core::stream::{FusedStream, Stream, TryStream}; use futures_core::task::{Context, Poll}; #[cfg(feature = "sink")] use futures_sink::Sink; @@ -11,7 +11,10 @@ use pin_project::pin_project; /// method. #[pin_project] #[must_use = "streams do nothing unless polled"] -pub struct TrySkipWhile where St: TryStream { +pub struct TrySkipWhile +where + St: TryStream, +{ #[pin] stream: St, f: F, @@ -38,34 +41,27 @@ where } impl TrySkipWhile - where St: TryStream, - F: FnMut(&St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(&St::Ok) -> Fut, + Fut: TryFuture, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - f, - pending_fut: None, - pending_item: None, - done_skipping: false, - } + Self { stream, f, pending_fut: None, pending_item: None, done_skipping: false } } delegate_access_inner!(stream, St, ()); } impl Stream for TrySkipWhile - where St: TryStream, - F: FnMut(&St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream, + F: FnMut(&St::Ok) -> Fut, + Fut: TryFuture, { type Item = Result; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); if *this.done_skipping { @@ -103,9 +99,10 @@ impl Stream for TrySkipWhile } impl FusedStream for TrySkipWhile - where St: TryStream + FusedStream, - F: FnMut(&St::Ok) -> Fut, - Fut: TryFuture, +where + St: TryStream + FusedStream, + F: FnMut(&St::Ok) -> Fut, + Fut: TryFuture, { fn is_terminated(&self) -> bool { self.pending_item.is_none() && self.stream.is_terminated() @@ -115,7 +112,8 @@ impl FusedStream for TrySkipWhile // Forwarding impl of Sink from the underlying stream #[cfg(feature = "sink")] impl Sink for TrySkipWhile - where S: TryStream + Sink, +where + S: TryStream + Sink, { type Error = E; diff --git a/futures-util/src/stream/try_stream/try_take_while.rs b/futures-util/src/stream/try_stream/try_take_while.rs index f9feff324f..9fde763049 100644 --- a/futures-util/src/stream/try_stream/try_take_while.rs +++ b/futures-util/src/stream/try_stream/try_take_while.rs @@ -47,13 +47,7 @@ where Fut: TryFuture, { pub(super) fn new(stream: St, f: F) -> Self { - Self { - stream, - f, - pending_fut: None, - pending_item: None, - done_taking: false, - } + Self { stream, f, pending_fut: None, pending_item: None, done_taking: false } } delegate_access_inner!(stream, St, ()); diff --git a/futures-util/src/stream/try_stream/try_unfold.rs b/futures-util/src/stream/try_stream/try_unfold.rs index a6b31fed3d..8e4833eb5c 100644 --- a/futures-util/src/stream/try_stream/try_unfold.rs +++ b/futures-util/src/stream/try_stream/try_unfold.rs @@ -59,11 +59,7 @@ where F: FnMut(T) -> Fut, Fut: TryFuture>, { - TryUnfold { - f, - state: Some(init), - fut: None, - } + TryUnfold { f, state: Some(init), fut: None } } /// Stream for the [`try_unfold`] function. @@ -82,10 +78,7 @@ where Fut: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("TryUnfold") - .field("state", &self.state) - .field("fut", &self.fut) - .finish() + f.debug_struct("TryUnfold").field("state", &self.state).field("fut", &self.fut).finish() } } @@ -96,10 +89,7 @@ where { type Item = Result; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); if let Some(state) = this.state.take() { diff --git a/futures-util/src/stream/unfold.rs b/futures-util/src/stream/unfold.rs index b6f8eae0b2..0360304f0b 100644 --- a/futures-util/src/stream/unfold.rs +++ b/futures-util/src/stream/unfold.rs @@ -45,14 +45,11 @@ use pin_project::pin_project; /// # }); /// ``` pub fn unfold(init: T, f: F) -> Unfold - where F: FnMut(T) -> Fut, - Fut: Future>, +where + F: FnMut(T) -> Fut, + Fut: Future>, { - Unfold { - f, - state: Some(init), - fut: None, - } + Unfold { f, state: Some(init), fut: None } } /// Stream for the [`unfold`] function. @@ -71,16 +68,14 @@ where Fut: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Unfold") - .field("state", &self.state) - .field("fut", &self.fut) - .finish() + f.debug_struct("Unfold").field("state", &self.state).field("fut", &self.fut).finish() } } impl FusedStream for Unfold - where F: FnMut(T) -> Fut, - Fut: Future>, +where + F: FnMut(T) -> Fut, + Fut: Future>, { fn is_terminated(&self) -> bool { self.state.is_none() && self.fut.is_none() @@ -88,23 +83,25 @@ impl FusedStream for Unfold } impl Stream for Unfold - where F: FnMut(T) -> Fut, - Fut: Future>, +where + F: FnMut(T) -> Fut, + Fut: Future>, { type Item = Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut this = self.project(); if let Some(state) = this.state.take() { this.fut.set(Some((this.f)(state))); } - let step = ready!(this.fut.as_mut().as_pin_mut() - .expect("Unfold must not be polled after it returned `Poll::Ready(None)`").poll(cx)); + let step = ready!(this + .fut + .as_mut() + .as_pin_mut() + .expect("Unfold must not be polled after it returned `Poll::Ready(None)`") + .poll(cx)); this.fut.set(None); if let Some((item, next_state)) = step { diff --git a/futures-util/src/task/mod.rs b/futures-util/src/task/mod.rs index fb3b7adacd..ac5aaa886f 100644 --- a/futures-util/src/task/mod.rs +++ b/futures-util/src/task/mod.rs @@ -14,14 +14,11 @@ cfg_target_has_atomic! { } mod spawn; -pub use self::spawn::{SpawnExt, LocalSpawnExt}; +pub use self::spawn::{LocalSpawnExt, SpawnExt}; -pub use futures_core::task::{Context, Poll, Waker, RawWaker, RawWakerVTable}; +pub use futures_core::task::{Context, Poll, RawWaker, RawWakerVTable, Waker}; -pub use futures_task::{ - Spawn, LocalSpawn, SpawnError, - FutureObj, LocalFutureObj, UnsafeFutureObj, -}; +pub use futures_task::{FutureObj, LocalFutureObj, LocalSpawn, Spawn, SpawnError, UnsafeFutureObj}; pub use futures_task::noop_waker; #[cfg(feature = "std")] diff --git a/futures/src/lib.rs b/futures/src/lib.rs index 01f091a16a..cb62944cb4 100644 --- a/futures/src/lib.rs +++ b/futures/src/lib.rs @@ -80,23 +80,17 @@ #![cfg_attr(feature = "cfg-target-has-atomic", feature(cfg_target_has_atomic))] #![cfg_attr(feature = "read-initializer", feature(read_initializer))] - #![cfg_attr(not(feature = "std"), no_std)] - #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms, unreachable_pub)] // It cannot be included in the published code because this lints have false positives in the minimum required version. #![cfg_attr(test, warn(single_use_lifetimes))] #![warn(clippy::all)] - // mem::take requires Rust 1.40, matches! requires Rust 1.42 // Can be removed if the minimum supported version increased or if https://github.com/rust-lang/rust-clippy/issues/3941 // get's implemented. #![allow(clippy::mem_replace_with_default, clippy::match_like_matches_macro)] - #![doc(test(attr(deny(warnings), allow(dead_code, unused_assignments, unused_variables))))] - #![doc(html_root_url = "https://docs.rs/futures/0.3.8")] - #![cfg_attr(docsrs, feature(doc_cfg))] #[cfg(all(feature = "cfg-target-has-atomic", not(feature = "unstable")))] @@ -108,28 +102,36 @@ compile_error!("The `bilock` feature requires the `unstable` feature as an expli #[cfg(all(feature = "read-initializer", not(feature = "unstable")))] compile_error!("The `read-initializer` feature requires the `unstable` feature as an explicit opt-in to unstable features"); -#[doc(hidden)] pub use futures_core::future::{Future, TryFuture}; -#[doc(hidden)] pub use futures_util::future::{FutureExt, TryFutureExt}; +#[doc(hidden)] +pub use futures_core::future::{Future, TryFuture}; +#[doc(hidden)] +pub use futures_util::future::{FutureExt, TryFutureExt}; -#[doc(hidden)] pub use futures_core::stream::{Stream, TryStream}; -#[doc(hidden)] pub use futures_util::stream::{StreamExt, TryStreamExt}; +#[doc(hidden)] +pub use futures_core::stream::{Stream, TryStream}; +#[doc(hidden)] +pub use futures_util::stream::{StreamExt, TryStreamExt}; -#[doc(hidden)] pub use futures_sink::Sink; -#[doc(hidden)] pub use futures_util::sink::SinkExt; +#[doc(hidden)] +pub use futures_sink::Sink; +#[doc(hidden)] +pub use futures_util::sink::SinkExt; #[cfg(feature = "std")] -#[doc(hidden)] pub use futures_io::{AsyncRead, AsyncWrite, AsyncSeek, AsyncBufRead}; +#[doc(hidden)] +pub use futures_io::{AsyncBufRead, AsyncRead, AsyncSeek, AsyncWrite}; #[cfg(feature = "std")] -#[doc(hidden)] pub use futures_util::{AsyncReadExt, AsyncWriteExt, AsyncSeekExt, AsyncBufReadExt}; +#[doc(hidden)] +pub use futures_util::{AsyncBufReadExt, AsyncReadExt, AsyncSeekExt, AsyncWriteExt}; // Macro reexports pub use futures_core::ready; // Readiness propagation pub use futures_util::pin_mut; -#[cfg(feature = "async-await")] -pub use futures_util::{pending, poll, join, try_join, select_biased}; // Async-await #[cfg(feature = "std")] #[cfg(feature = "async-await")] pub use futures_util::select; +#[cfg(feature = "async-await")] +pub use futures_util::{join, pending, poll, select_biased, try_join}; // Async-await #[cfg_attr(feature = "cfg-target-has-atomic", cfg(target_has_atomic = "ptr"))] #[cfg(feature = "alloc")] @@ -163,24 +165,13 @@ pub mod compat { //! library is activated. pub use futures_util::compat::{ - Compat, - CompatSink, - Compat01As03, - Compat01As03Sink, - Executor01Future, - Executor01As03, - Executor01CompatExt, - Future01CompatExt, - Stream01CompatExt, - Sink01CompatExt, + Compat, Compat01As03, Compat01As03Sink, CompatSink, Executor01As03, Executor01CompatExt, + Executor01Future, Future01CompatExt, Sink01CompatExt, Stream01CompatExt, }; #[cfg(feature = "io-compat")] #[cfg_attr(docsrs, doc(cfg(feature = "io-compat")))] - pub use futures_util::compat::{ - AsyncRead01CompatExt, - AsyncWrite01CompatExt, - }; + pub use futures_util::compat::{AsyncRead01CompatExt, AsyncWrite01CompatExt}; } #[cfg(feature = "executor")] @@ -227,10 +218,8 @@ pub mod executor { //! completion on the current thread. pub use futures_executor::{ - BlockingStream, - Enter, EnterError, - LocalSpawner, LocalPool, - block_on, block_on_stream, enter, + block_on, block_on_stream, enter, BlockingStream, Enter, EnterError, LocalPool, + LocalSpawner, }; #[cfg(feature = "thread-pool")] @@ -251,9 +240,7 @@ pub mod future { //! [`ready`](crate::future::ready), which constructs a future with an //! immediate defined value. - pub use futures_core::future::{ - Future, TryFuture, FusedFuture, - }; + pub use futures_core::future::{FusedFuture, Future, TryFuture}; #[cfg(feature = "alloc")] pub use futures_core::future::{BoxFuture, LocalBoxFuture}; @@ -261,48 +248,25 @@ pub mod future { pub use futures_task::{FutureObj, LocalFutureObj, UnsafeFutureObj}; pub use futures_util::future::{ - lazy, Lazy, - maybe_done, MaybeDone, - pending, Pending, - poll_fn, PollFn, - ready, ok, err, Ready, - join, join3, join4, join5, - Join, Join3, Join4, Join5, - select, Select, - try_join, try_join3, try_join4, try_join5, - TryJoin, TryJoin3, TryJoin4, TryJoin5, - try_select, TrySelect, - Either, - OptionFuture, - - FutureExt, - FlattenStream, Flatten, Fuse, Inspect, IntoStream, Map, Then, UnitError, - NeverError, - - TryFutureExt, - AndThen, ErrInto, FlattenSink, IntoFuture, MapErr, MapOk, OrElse, - InspectOk, InspectErr, TryFlattenStream, UnwrapOrElse, + err, join, join3, join4, join5, lazy, maybe_done, ok, pending, poll_fn, ready, select, + try_join, try_join3, try_join4, try_join5, try_select, AndThen, Either, ErrInto, Flatten, + FlattenSink, FlattenStream, Fuse, FutureExt, Inspect, InspectErr, InspectOk, IntoFuture, + IntoStream, Join, Join3, Join4, Join5, Lazy, Map, MapErr, MapOk, MaybeDone, NeverError, + OptionFuture, OrElse, Pending, PollFn, Ready, Select, Then, TryFlattenStream, TryFutureExt, + TryJoin, TryJoin3, TryJoin4, TryJoin5, TrySelect, UnitError, UnwrapOrElse, }; #[cfg(feature = "alloc")] pub use futures_util::future::{ - join_all, JoinAll, - select_all, SelectAll, - try_join_all, TryJoinAll, - select_ok, SelectOk, + join_all, select_all, select_ok, try_join_all, JoinAll, SelectAll, SelectOk, TryJoinAll, }; #[cfg_attr(feature = "cfg-target-has-atomic", cfg(target_has_atomic = "ptr"))] #[cfg(feature = "alloc")] - pub use futures_util::future::{ - abortable, Abortable, AbortHandle, AbortRegistration, Aborted, - }; + pub use futures_util::future::{abortable, AbortHandle, AbortRegistration, Abortable, Aborted}; #[cfg(feature = "std")] - pub use futures_util::future::{ - Remote, RemoteHandle, - CatchUnwind, Shared, WeakShared, - }; + pub use futures_util::future::{CatchUnwind, Remote, RemoteHandle, Shared, WeakShared}; } #[cfg(feature = "std")] @@ -332,8 +296,8 @@ pub mod io { //! library is activated, and it is activated by default. pub use futures_io::{ - AsyncRead, AsyncWrite, AsyncSeek, AsyncBufRead, Error, ErrorKind, - IoSlice, IoSliceMut, Result, SeekFrom, + AsyncBufRead, AsyncRead, AsyncSeek, AsyncWrite, Error, ErrorKind, IoSlice, IoSliceMut, + Result, SeekFrom, }; #[cfg(feature = "read-initializer")] @@ -341,12 +305,11 @@ pub mod io { pub use futures_io::Initializer; pub use futures_util::io::{ - AsyncReadExt, AsyncWriteExt, AsyncSeekExt, AsyncBufReadExt, AllowStdIo, - BufReader, BufWriter, Cursor, Chain, Close, copy, Copy, copy_buf, CopyBuf, - empty, Empty, FillBuf, Flush, IntoSink, Lines, Read, ReadExact, ReadHalf, - ReadLine, ReadToEnd, ReadToString, ReadUntil, ReadVectored, repeat, - Repeat, ReuniteError, Seek, sink, Sink, Take, Window, Write, WriteAll, WriteHalf, - WriteVectored, + copy, copy_buf, empty, repeat, sink, AllowStdIo, AsyncBufReadExt, AsyncReadExt, + AsyncSeekExt, AsyncWriteExt, BufReader, BufWriter, Chain, Close, Copy, CopyBuf, Cursor, + Empty, FillBuf, Flush, IntoSink, Lines, Read, ReadExact, ReadHalf, ReadLine, ReadToEnd, + ReadToString, ReadUntil, ReadVectored, Repeat, ReuniteError, Seek, Sink, Take, Window, + Write, WriteAll, WriteHalf, WriteVectored, }; } @@ -363,7 +326,7 @@ pub mod lock { pub use futures_util::lock::{BiLock, BiLockAcquire, BiLockGuard, ReuniteError}; #[cfg(feature = "std")] - pub use futures_util::lock::{MappedMutexGuard, Mutex, MutexLockFuture, MutexGuard}; + pub use futures_util::lock::{MappedMutexGuard, Mutex, MutexGuard, MutexLockFuture}; } pub mod prelude { @@ -381,25 +344,23 @@ pub mod prelude { //! The prelude may grow over time as additional items see ubiquitous use. pub use crate::future::{self, Future, TryFuture}; - pub use crate::stream::{self, Stream, TryStream}; pub use crate::sink::{self, Sink}; + pub use crate::stream::{self, Stream, TryStream}; #[doc(no_inline)] pub use crate::future::{FutureExt as _, TryFutureExt as _}; #[doc(no_inline)] - pub use crate::stream::{StreamExt as _, TryStreamExt as _}; - #[doc(no_inline)] pub use crate::sink::SinkExt as _; + #[doc(no_inline)] + pub use crate::stream::{StreamExt as _, TryStreamExt as _}; #[cfg(feature = "std")] - pub use crate::io::{ - AsyncRead, AsyncWrite, AsyncSeek, AsyncBufRead, - }; + pub use crate::io::{AsyncBufRead, AsyncRead, AsyncSeek, AsyncWrite}; #[cfg(feature = "std")] #[doc(no_inline)] pub use crate::io::{ - AsyncReadExt as _, AsyncWriteExt as _, AsyncSeekExt as _, AsyncBufReadExt as _, + AsyncBufReadExt as _, AsyncReadExt as _, AsyncSeekExt as _, AsyncWriteExt as _, }; } @@ -416,8 +377,7 @@ pub mod sink { pub use futures_sink::Sink; pub use futures_util::sink::{ - Close, Flush, Send, SendAll, SinkErrInto, SinkMapErr, With, - SinkExt, Fanout, Drain, drain, + drain, Close, Drain, Fanout, Flush, Send, SendAll, SinkErrInto, SinkExt, SinkMapErr, With, WithFlatMap, }; @@ -437,68 +397,57 @@ pub mod stream { //! - Top-level stream constructors like [`iter`](crate::stream::iter) //! which creates a stream from an iterator. - pub use futures_core::stream::{ - Stream, TryStream, FusedStream, - }; + pub use futures_core::stream::{FusedStream, Stream, TryStream}; #[cfg(feature = "alloc")] pub use futures_core::stream::{BoxStream, LocalBoxStream}; pub use futures_util::stream::{ - iter, Iter, - repeat, Repeat, - empty, Empty, - pending, Pending, - once, Once, - poll_fn, PollFn, - select, Select, - unfold, Unfold, - try_unfold, TryUnfold, - - StreamExt, - Chain, Collect, Concat, Cycle, Enumerate, Filter, FilterMap, FlatMap, Flatten, - Fold, Forward, ForEach, Fuse, StreamFuture, Inspect, Map, Next, - SelectNextSome, Peek, Peekable, Scan, Skip, SkipWhile, Take, TakeUntil, - TakeWhile, Then, Zip, - - TryStreamExt, - AndThen, ErrInto, MapOk, MapErr, OrElse, - InspectOk, InspectErr, - TryNext, TryForEach, TryFilter, TryFilterMap, TryFlatten, - TryCollect, TryConcat, TryFold, TrySkipWhile, TryTakeWhile, - IntoStream, + empty, iter, once, pending, poll_fn, repeat, select, try_unfold, unfold, AndThen, Chain, + Collect, Concat, Cycle, Empty, Enumerate, ErrInto, Filter, FilterMap, FlatMap, Flatten, + Fold, ForEach, Forward, Fuse, Inspect, InspectErr, InspectOk, IntoStream, Iter, Map, + MapErr, MapOk, Next, Once, OrElse, Peek, Peekable, Pending, PollFn, Repeat, Scan, Select, + SelectNextSome, Skip, SkipWhile, StreamExt, StreamFuture, Take, TakeUntil, TakeWhile, Then, + TryCollect, TryConcat, TryFilter, TryFilterMap, TryFlatten, TryFold, TryForEach, TryNext, + TrySkipWhile, TryStreamExt, TryTakeWhile, TryUnfold, Unfold, Unzip, Zip, }; #[cfg(feature = "alloc")] pub use futures_util::stream::{ // For StreamExt: - Chunks, ReadyChunks, + Chunks, + ReadyChunks, }; #[cfg_attr(feature = "cfg-target-has-atomic", cfg(target_has_atomic = "ptr"))] #[cfg(feature = "alloc")] pub use futures_util::stream::{ + futures_unordered, + select_all, + // For StreamExt: + BufferUnordered, + Buffered, + ForEachConcurrent, FuturesOrdered, - futures_unordered, FuturesUnordered, + FuturesUnordered, - // For StreamExt: - BufferUnordered, Buffered, ForEachConcurrent, SplitStream, SplitSink, ReuniteError, - select_all, SelectAll, + SelectAll, + SplitSink, + SplitStream, }; #[cfg(feature = "std")] - pub use futures_util::stream::{ - // For StreamExt: - CatchUnwind, - }; + pub use futures_util::stream::CatchUnwind; #[cfg_attr(feature = "cfg-target-has-atomic", cfg(target_has_atomic = "ptr"))] #[cfg(feature = "alloc")] pub use futures_util::stream::{ // For TryStreamExt: - TryBufferUnordered, TryBuffered, TryForEachConcurrent, + TryBufferUnordered, + TryBuffered, + TryForEachConcurrent, }; #[cfg(feature = "std")] @@ -518,11 +467,10 @@ pub mod task { //! The remaining types and traits in the module are used for implementing //! executors or dealing with synchronization issues around task wakeup. - pub use futures_core::task::{Context, Poll, Waker, RawWaker, RawWakerVTable}; + pub use futures_core::task::{Context, Poll, RawWaker, RawWakerVTable, Waker}; pub use futures_task::{ - Spawn, LocalSpawn, SpawnError, - FutureObj, LocalFutureObj, UnsafeFutureObj, + FutureObj, LocalFutureObj, LocalSpawn, Spawn, SpawnError, UnsafeFutureObj, }; pub use futures_util::task::noop_waker; @@ -531,11 +479,11 @@ pub mod task { pub use futures_util::task::noop_waker_ref; #[cfg(feature = "alloc")] - pub use futures_util::task::{SpawnExt, LocalSpawnExt}; + pub use futures_util::task::{LocalSpawnExt, SpawnExt}; #[cfg_attr(feature = "cfg-target-has-atomic", cfg(target_has_atomic = "ptr"))] #[cfg(feature = "alloc")] - pub use futures_util::task::{waker, waker_ref, WakerRef, ArcWake}; + pub use futures_util::task::{waker, waker_ref, ArcWake, WakerRef}; #[cfg_attr(feature = "cfg-target-has-atomic", cfg(target_has_atomic = "ptr"))] pub use futures_util::task::AtomicWaker; diff --git a/futures/tests/_require_features.rs b/futures/tests/_require_features.rs index da76dcd1e9..8046cc99a4 100644 --- a/futures/tests/_require_features.rs +++ b/futures/tests/_require_features.rs @@ -1,8 +1,13 @@ #[cfg(not(all( - feature = "std", feature = "alloc", feature = "async-await", - feature = "compat", feature = "io-compat", - feature = "executor", feature = "thread-pool", + feature = "std", + feature = "alloc", + feature = "async-await", + feature = "compat", + feature = "io-compat", + feature = "executor", + feature = "thread-pool", )))] -compile_error!("`futures` tests must have all stable features activated: \ +compile_error!( + "`futures` tests must have all stable features activated: \ use `--all-features` or `--features default,thread-pool,io-compat`" ); diff --git a/futures/tests/abortable.rs b/futures/tests/abortable.rs index 6b5a25c365..db9092ce32 100644 --- a/futures/tests/abortable.rs +++ b/futures/tests/abortable.rs @@ -1,8 +1,8 @@ #[test] fn abortable_works() { use futures::channel::oneshot; - use futures::future::{abortable, Aborted}; use futures::executor::block_on; + use futures::future::{abortable, Aborted}; let (_tx, a_rx) = oneshot::channel::<()>(); let (abortable_rx, abort_handle) = abortable(a_rx); @@ -34,8 +34,8 @@ fn abortable_awakens() { #[test] fn abortable_resolves() { use futures::channel::oneshot; - use futures::future::abortable; use futures::executor::block_on; + use futures::future::abortable; let (tx, a_rx) = oneshot::channel::<()>(); let (abortable_rx, _abort_handle) = abortable(a_rx); diff --git a/futures/tests/arc_wake.rs b/futures/tests/arc_wake.rs index d19a83dfac..849f58f63e 100644 --- a/futures/tests/arc_wake.rs +++ b/futures/tests/arc_wake.rs @@ -8,9 +8,7 @@ mod countingwaker { impl CountingWaker { fn new() -> Self { - Self { - nr_wake: Mutex::new(0), - } + Self { nr_wake: Mutex::new(0) } } fn wakes(&self) -> i32 { @@ -75,7 +73,10 @@ fn proper_refcount_on_wake_panic() { let some_w = Arc::new(PanicWaker); let w1: Waker = task::waker(some_w.clone()); - assert_eq!("WAKE UP", *std::panic::catch_unwind(|| w1.wake_by_ref()).unwrap_err().downcast::<&str>().unwrap()); + assert_eq!( + "WAKE UP", + *std::panic::catch_unwind(|| w1.wake_by_ref()).unwrap_err().downcast::<&str>().unwrap() + ); assert_eq!(2, Arc::strong_count(&some_w)); // some_w + w1 drop(w1); assert_eq!(1, Arc::strong_count(&some_w)); // some_w diff --git a/futures/tests/async_await_macros.rs b/futures/tests/async_await_macros.rs index bd586d6e52..fc1902298b 100644 --- a/futures/tests/async_await_macros.rs +++ b/futures/tests/async_await_macros.rs @@ -1,8 +1,8 @@ #[test] fn poll_and_pending() { - use futures::{pending, pin_mut, poll}; use futures::executor::block_on; use futures::task::Poll; + use futures::{pending, pin_mut, poll}; let pending_once = async { pending!() }; block_on(async { @@ -14,10 +14,10 @@ fn poll_and_pending() { #[test] fn join() { - use futures::{pin_mut, poll, join}; use futures::channel::oneshot; use futures::executor::block_on; use futures::task::Poll; + use futures::{join, pin_mut, poll}; let (tx1, rx1) = oneshot::channel::(); let (tx2, rx2) = oneshot::channel::(); @@ -39,10 +39,10 @@ fn join() { #[test] fn select() { - use futures::select; use futures::channel::oneshot; use futures::executor::block_on; use futures::future::FutureExt; + use futures::select; let (tx1, rx1) = oneshot::channel::(); let (_tx2, rx2) = oneshot::channel::(); @@ -85,9 +85,9 @@ fn select_biased() { #[test] fn select_streams() { - use futures::select; use futures::channel::mpsc; use futures::executor::block_on; + use futures::select; use futures::sink::SinkExt; use futures::stream::StreamExt; @@ -134,10 +134,10 @@ fn select_streams() { #[test] fn select_can_move_uncompleted_futures() { - use futures::select; use futures::channel::oneshot; use futures::executor::block_on; use futures::future::FutureExt; + use futures::select; let (tx1, rx1) = oneshot::channel::(); let (tx2, rx2) = oneshot::channel::(); @@ -165,9 +165,9 @@ fn select_can_move_uncompleted_futures() { #[test] fn select_nested() { - use futures::select; use futures::executor::block_on; use futures::future; + use futures::select; let mut outer_fut = future::ready(1); let mut inner_fut = future::ready(2); @@ -185,8 +185,8 @@ fn select_nested() { #[test] fn select_size() { - use futures::select; use futures::future; + use futures::select; let fut = async { let mut ready = future::ready(0i32); @@ -209,14 +209,12 @@ fn select_size() { #[test] fn select_on_non_unpin_expressions() { - use futures::select; use futures::executor::block_on; use futures::future::FutureExt; + use futures::select; // The returned Future is !Unpin - let make_non_unpin_fut = || { async { - 5 - }}; + let make_non_unpin_fut = || async { 5 }; let res = block_on(async { let select_res; @@ -231,14 +229,12 @@ fn select_on_non_unpin_expressions() { #[test] fn select_on_non_unpin_expressions_with_default() { - use futures::select; use futures::executor::block_on; use futures::future::FutureExt; + use futures::select; // The returned Future is !Unpin - let make_non_unpin_fut = || { async { - 5 - }}; + let make_non_unpin_fut = || async { 5 }; let res = block_on(async { let select_res; @@ -254,13 +250,11 @@ fn select_on_non_unpin_expressions_with_default() { #[test] fn select_on_non_unpin_size() { - use futures::select; use futures::future::FutureExt; + use futures::select; // The returned Future is !Unpin - let make_non_unpin_fut = || { async { - 5 - }}; + let make_non_unpin_fut = || async { 5 }; let fut = async { let select_res; @@ -276,9 +270,9 @@ fn select_on_non_unpin_size() { #[test] fn select_can_be_used_as_expression() { - use futures::select; use futures::executor::block_on; use futures::future; + use futures::select; block_on(async { let res = select! { @@ -291,9 +285,9 @@ fn select_can_be_used_as_expression() { #[test] fn select_with_default_can_be_used_as_expression() { - use futures::select; use futures::executor::block_on; - use futures::future::{FutureExt, poll_fn}; + use futures::future::{poll_fn, FutureExt}; + use futures::select; use futures::task::{Context, Poll}; fn poll_always_pending(_cx: &mut Context<'_>) -> Poll { @@ -312,9 +306,9 @@ fn select_with_default_can_be_used_as_expression() { #[test] fn select_with_complete_can_be_used_as_expression() { - use futures::select; use futures::executor::block_on; use futures::future; + use futures::select; block_on(async { let res = select! { @@ -330,9 +324,9 @@ fn select_with_complete_can_be_used_as_expression() { #[test] #[allow(unused_assignments)] fn select_on_mutable_borrowing_future_with_same_borrow_in_block() { - use futures::select; use futures::executor::block_on; use futures::future::FutureExt; + use futures::select; async fn require_mutable(_: &mut i32) {} async fn async_noop() {} @@ -351,9 +345,9 @@ fn select_on_mutable_borrowing_future_with_same_borrow_in_block() { #[test] #[allow(unused_assignments)] fn select_on_mutable_borrowing_future_with_same_borrow_in_block_and_default() { - use futures::select; use futures::executor::block_on; use futures::future::FutureExt; + use futures::select; async fn require_mutable(_: &mut i32) {} async fn async_noop() {} @@ -374,8 +368,8 @@ fn select_on_mutable_borrowing_future_with_same_borrow_in_block_and_default() { #[test] fn join_size() { - use futures::join; use futures::future; + use futures::join; let fut = async { let ready = future::ready(0i32); @@ -393,8 +387,8 @@ fn join_size() { #[test] fn try_join_size() { - use futures::try_join; use futures::future; + use futures::try_join; let fut = async { let ready = future::ready(Ok::(0)); @@ -414,19 +408,12 @@ fn try_join_size() { fn join_doesnt_require_unpin() { use futures::join; - let _ = async { - join!(async {}, async {}) - }; + let _ = async { join!(async {}, async {}) }; } #[test] fn try_join_doesnt_require_unpin() { use futures::try_join; - let _ = async { - try_join!( - async { Ok::<(), ()>(()) }, - async { Ok::<(), ()>(()) }, - ) - }; + let _ = async { try_join!(async { Ok::<(), ()>(()) }, async { Ok::<(), ()>(()) },) }; } diff --git a/futures/tests/basic_combinators.rs b/futures/tests/basic_combinators.rs index fa65b6f5f1..372ab48b79 100644 --- a/futures/tests/basic_combinators.rs +++ b/futures/tests/basic_combinators.rs @@ -13,17 +13,21 @@ fn basic_future_combinators() { tx1.send(x).unwrap(); // Send 1 tx1.send(2).unwrap(); // Send 2 future::ready(3) - }).map(move |x| { + }) + .map(move |x| { tx2.send(x).unwrap(); // Send 3 tx2.send(4).unwrap(); // Send 4 5 - }).map(move |x| { + }) + .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 + for i in 1..=5 { + assert_eq!(rx.recv(), Ok(i)); + } // Check it assert!(rx.recv().is_err()); // Should be done } @@ -93,6 +97,8 @@ fn basic_try_future_combinators() { 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 + 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/buffer_unordered.rs b/futures/tests/buffer_unordered.rs index 5c8b8bf7e5..f138670a8f 100644 --- a/futures/tests/buffer_unordered.rs +++ b/futures/tests/buffer_unordered.rs @@ -1,7 +1,7 @@ #[test] #[ignore] // FIXME: https://github.com/rust-lang/futures-rs/issues/1790 fn works() { - use futures::channel::{oneshot, mpsc}; + use futures::channel::{mpsc, oneshot}; use futures::executor::{block_on, block_on_stream}; use futures::sink::SinkExt; use futures::stream::StreamExt; diff --git a/futures/tests/compat.rs b/futures/tests/compat.rs index ba4c897bd6..3760869e01 100644 --- a/futures/tests/compat.rs +++ b/futures/tests/compat.rs @@ -1,14 +1,12 @@ -use tokio::timer::Delay; -use tokio::runtime::Runtime; -use std::time::Instant; -use futures::prelude::*; use futures::compat::Future01CompatExt; +use futures::prelude::*; +use std::time::Instant; +use tokio::runtime::Runtime; +use tokio::timer::Delay; #[test] fn can_use_01_futures_in_a_03_future_running_on_a_01_executor() { - let f = async { - Delay::new(Instant::now()).compat().await - }; + let f = async { Delay::new(Instant::now()).compat().await }; let mut runtime = Runtime::new().unwrap(); runtime.block_on(f.boxed().compat()).unwrap(); diff --git a/futures/tests/eager_drop.rs b/futures/tests/eager_drop.rs index 11edb1b6de..c1f01ae562 100644 --- a/futures/tests/eager_drop.rs +++ b/futures/tests/eager_drop.rs @@ -10,7 +10,10 @@ fn map_ok() { let (tx2, rx2) = mpsc::channel::<()>(); future::ready::>(Err(1)) - .map_ok(move |_| { let _tx1 = tx1; panic!("should not run"); }) + .map_ok(move |_| { + let _tx1 = tx1; + panic!("should not run"); + }) .map(move |_| { assert!(rx1.recv().is_err()); tx2.send(()).unwrap() @@ -32,7 +35,10 @@ fn map_err() { let (tx2, rx2) = mpsc::channel::<()>(); future::ready::>(Ok(1)) - .map_err(move |_| { let _tx1 = tx1; panic!("should not run"); }) + .map_err(move |_| { + let _tx1 = tx1; + panic!("should not run"); + }) .map(move |_| { assert!(rx1.recv().is_err()); tx2.send(()).unwrap() @@ -44,7 +50,7 @@ fn map_err() { mod channelled { use futures::future::Future; - use futures::task::{Context,Poll}; + use futures::task::{Context, Poll}; use pin_project::pin_project; use std::pin::Pin; @@ -74,7 +80,7 @@ mod channelled { let (tx1, rx1) = mpsc::channel::<()>(); let (tx2, rx2) = mpsc::channel::<()>(); - FutureData { _data: tx1, future: rx0.unwrap_or_else(|_| { panic!() }) } + FutureData { _data: tx1, future: rx0.unwrap_or_else(|_| panic!()) } .then(move |_| { assert!(rx1.recv().is_err()); // tx1 should have been dropped tx2.send(()).unwrap(); @@ -98,7 +104,7 @@ mod channelled { let (tx1, rx1) = mpsc::channel::<()>(); let (tx2, rx2) = mpsc::channel::<()>(); - FutureData { _data: tx1, future: rx0.unwrap_or_else(|_| { panic!() }) } + 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(); @@ -122,7 +128,7 @@ mod channelled { let (tx1, rx1) = mpsc::channel::<()>(); let (tx2, rx2) = mpsc::channel::<()>(); - FutureData { _data: tx1, future: rx0.unwrap_or_else(|_| { panic!() }) } + 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(); diff --git a/futures/tests/future_obj.rs b/futures/tests/future_obj.rs index c6b18fc85c..0e5253464e 100644 --- a/futures/tests/future_obj.rs +++ b/futures/tests/future_obj.rs @@ -1,6 +1,6 @@ -use futures::future::{Future, FutureObj, FutureExt}; -use std::pin::Pin; +use futures::future::{Future, FutureExt, FutureObj}; use futures::task::{Context, Poll}; +use std::pin::Pin; #[test] fn dropping_does_not_segfault() { diff --git a/futures/tests/future_try_flatten_stream.rs b/futures/tests/future_try_flatten_stream.rs index 4a614f9c14..d302a78bd4 100644 --- a/futures/tests/future_try_flatten_stream.rs +++ b/futures/tests/future_try_flatten_stream.rs @@ -25,7 +25,7 @@ fn failed_future() { use futures::task::{Context, Poll}; struct PanickingStream { - _marker: PhantomData<(T, E)> + _marker: PhantomData<(T, E)>, } impl Stream for PanickingStream { @@ -47,10 +47,10 @@ fn failed_future() { fn assert_impls() { use core::marker::PhantomData; use core::pin::Pin; + use futures::future::{ok, TryFutureExt}; use futures::sink::Sink; use futures::stream::Stream; use futures::task::{Context, Poll}; - use futures::future::{ok, TryFutureExt}; struct StreamSink(PhantomData<(T, E, Item)>); diff --git a/futures/tests/futures_ordered.rs b/futures/tests/futures_ordered.rs index 7f21c829e3..5a65099ee0 100644 --- a/futures/tests/futures_ordered.rs +++ b/futures/tests/futures_ordered.rs @@ -2,7 +2,7 @@ fn works_1() { use futures::channel::oneshot; use futures::executor::block_on_stream; - use futures::stream::{StreamExt, FuturesOrdered}; + use futures::stream::{FuturesOrdered, StreamExt}; use futures_test::task::noop_context; let (a_tx, a_rx) = oneshot::channel::(); @@ -28,17 +28,16 @@ fn works_1() { fn works_2() { use futures::channel::oneshot; use futures::future::{join, FutureExt}; - use futures::stream::{StreamExt, FuturesOrdered}; + use futures::stream::{FuturesOrdered, StreamExt}; use futures_test::task::noop_context; let (a_tx, a_rx) = oneshot::channel::(); let (b_tx, b_rx) = oneshot::channel::(); let (c_tx, c_rx) = oneshot::channel::(); - let mut stream = vec![ - a_rx.boxed(), - join(b_rx, c_rx).map(|(a, b)| Ok(a? + b?)).boxed(), - ].into_iter().collect::>(); + let mut stream = vec![a_rx.boxed(), join(b_rx, c_rx).map(|(a, b)| Ok(a? + b?)).boxed()] + .into_iter() + .collect::>(); let mut cx = noop_context(); a_tx.send(33).unwrap(); @@ -53,22 +52,20 @@ fn works_2() { fn from_iterator() { use futures::executor::block_on; use futures::future; - use futures::stream::{StreamExt, FuturesOrdered}; + use futures::stream::{FuturesOrdered, StreamExt}; - let stream = vec![ - future::ready::(1), - future::ready::(2), - future::ready::(3) - ].into_iter().collect::>(); + let stream = vec![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]); + assert_eq!(block_on(stream.collect::>()), vec![1, 2, 3]); } #[test] fn queue_never_unblocked() { use futures::channel::oneshot; use futures::future::{self, Future, TryFutureExt}; - use futures::stream::{StreamExt, FuturesOrdered}; + use futures::stream::{FuturesOrdered, StreamExt}; use futures_test::task::noop_context; use std::any::Any; @@ -78,10 +75,14 @@ fn queue_never_unblocked() { let mut stream = vec![ Box::new(a_rx) as Box + Unpin>, - Box::new(future::try_select(b_rx, c_rx) - .map_err(|e| e.factor_first().0) - .and_then(|e| future::ok(Box::new(e) as Box))) as _, - ].into_iter().collect::>(); + Box::new( + future::try_select(b_rx, c_rx) + .map_err(|e| e.factor_first().0) + .and_then(|e| future::ok(Box::new(e) as Box)), + ) as _, + ] + .into_iter() + .collect::>(); let cx = &mut noop_context(); for _ in 0..10 { diff --git a/futures/tests/futures_unordered.rs b/futures/tests/futures_unordered.rs index 4dd0deb167..076afb601b 100644 --- a/futures/tests/futures_unordered.rs +++ b/futures/tests/futures_unordered.rs @@ -40,11 +40,8 @@ fn works_1() { let (b_tx, b_rx) = oneshot::channel::(); let (c_tx, c_rx) = oneshot::channel::(); - let mut iter = block_on_stream( - vec![a_rx, b_rx, c_rx] - .into_iter() - .collect::>(), - ); + let mut iter = + block_on_stream(vec![a_rx, b_rx, c_rx].into_iter().collect::>()); b_tx.send(99).unwrap(); assert_eq!(Some(Ok(99)), iter.next()); @@ -68,12 +65,9 @@ fn works_2() { let (b_tx, b_rx) = oneshot::channel::(); let (c_tx, c_rx) = oneshot::channel::(); - let mut stream = vec![ - a_rx.boxed(), - join(b_rx, c_rx).map(|(a, b)| Ok(a? + b?)).boxed(), - ] - .into_iter() - .collect::>(); + let mut stream = vec![a_rx.boxed(), join(b_rx, c_rx).map(|(a, b)| Ok(a? + b?)).boxed()] + .into_iter() + .collect::>(); a_tx.send(9).unwrap(); b_tx.send(10).unwrap(); @@ -91,24 +85,20 @@ fn from_iterator() { use futures::future; use futures::stream::{FuturesUnordered, StreamExt}; - let stream = vec![ - future::ready::(1), - future::ready::(2), - future::ready::(3), - ] - .into_iter() - .collect::>(); + let stream = vec![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]); } #[test] fn finished_future() { - use std::marker::Unpin; use futures::channel::oneshot; use futures::future::{self, Future, FutureExt}; use futures::stream::{FuturesUnordered, StreamExt}; use futures_test::task::noop_context; + use std::marker::Unpin; let (_a_tx, a_rx) = oneshot::channel::(); let (b_tx, b_rx) = oneshot::channel::(); @@ -143,9 +133,7 @@ fn iter_mut_cancel() { let (b_tx, b_rx) = oneshot::channel::(); let (c_tx, c_rx) = oneshot::channel::(); - let mut stream = vec![a_rx, b_rx, c_rx] - .into_iter() - .collect::>(); + let mut stream = vec![a_rx, b_rx, c_rx].into_iter().collect::>(); for rx in stream.iter_mut() { rx.close(); @@ -168,13 +156,10 @@ fn iter_mut_len() { use futures::future; use futures::stream::FuturesUnordered; - let mut stream = vec![ - future::pending::<()>(), - future::pending::<()>(), - future::pending::<()>(), - ] - .into_iter() - .collect::>(); + let mut stream = + vec![future::pending::<()>(), future::pending::<()>(), future::pending::<()>()] + .into_iter() + .collect::>(); let mut iter_mut = stream.iter_mut(); assert_eq!(iter_mut.len(), 3); @@ -246,13 +231,9 @@ fn iter_len() { use futures::future; use futures::stream::FuturesUnordered; - let stream = vec![ - future::pending::<()>(), - future::pending::<()>(), - future::pending::<()>(), - ] - .into_iter() - .collect::>(); + let stream = vec![future::pending::<()>(), future::pending::<()>(), future::pending::<()>()] + .into_iter() + .collect::>(); let mut iter = stream.iter(); assert_eq!(iter.len(), 3); diff --git a/futures/tests/inspect.rs b/futures/tests/inspect.rs index 375778b63d..6107b90415 100644 --- a/futures/tests/inspect.rs +++ b/futures/tests/inspect.rs @@ -6,7 +6,9 @@ fn smoke() { let mut counter = 0; { - let work = future::ready::(40).inspect(|val| { counter += *val; }); + let work = future::ready::(40).inspect(|val| { + counter += *val; + }); assert_eq!(block_on(work), 40); } diff --git a/futures/tests/io_buf_reader.rs b/futures/tests/io_buf_reader.rs index f8f9d140e3..97915821ed 100644 --- a/futures/tests/io_buf_reader.rs +++ b/futures/tests/io_buf_reader.rs @@ -1,7 +1,7 @@ macro_rules! run_fill_buf { ($reader:expr) => {{ - use futures_test::task::noop_context; use futures::task::Poll; + use futures_test::task::noop_context; use std::pin::Pin; let mut cx = noop_context(); @@ -16,9 +16,9 @@ macro_rules! run_fill_buf { mod util { use futures::future::Future; pub fn run(mut f: F) -> F::Output { - use futures_test::task::noop_context; - use futures::task::Poll; use futures::future::FutureExt; + use futures::task::Poll; + use futures_test::task::noop_context; let mut cx = noop_context(); loop { @@ -30,10 +30,10 @@ mod util { } mod maybe_pending { - use futures::task::{Context,Poll}; - use std::{cmp,io}; + use futures::io::{AsyncBufRead, AsyncRead}; + use futures::task::{Context, Poll}; use std::pin::Pin; - use futures::io::{AsyncRead,AsyncBufRead}; + use std::{cmp, io}; pub struct MaybePending<'a> { inner: &'a [u8], @@ -48,9 +48,11 @@ mod maybe_pending { } impl AsyncRead for MaybePending<'_> { - fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) - -> Poll> - { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { if self.ready_read { self.ready_read = false; Pin::new(&mut self.inner).poll_read(cx, buf) @@ -62,12 +64,12 @@ mod maybe_pending { } impl AsyncBufRead for MaybePending<'_> { - fn poll_fill_buf(mut self: Pin<&mut Self>, _: &mut Context<'_>) - -> Poll> - { + fn poll_fill_buf(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { if self.ready_fill_buf { self.ready_fill_buf = false; - if self.inner.is_empty() { return Poll::Ready(Ok(&[])) } + if self.inner.is_empty() { + return Poll::Ready(Ok(&[])); + } let len = cmp::min(2, self.inner.len()); Poll::Ready(Ok(&self.inner[0..len])) } else { @@ -125,7 +127,7 @@ fn test_buffered_reader() { #[test] fn test_buffered_reader_seek() { use futures::executor::block_on; - use futures::io::{AsyncSeekExt, AsyncBufRead, BufReader, Cursor, SeekFrom}; + use futures::io::{AsyncBufRead, AsyncSeekExt, BufReader, Cursor, SeekFrom}; use std::pin::Pin; use util::run; @@ -145,12 +147,12 @@ fn test_buffered_reader_seek() { #[test] fn test_buffered_reader_seek_underflow() { use futures::executor::block_on; - use futures::io::{AsyncSeekExt, AsyncBufRead, AllowStdIo, BufReader, SeekFrom}; + use futures::io::{AllowStdIo, AsyncBufRead, AsyncSeekExt, BufReader, SeekFrom}; use std::io; // gimmick reader that yields its position modulo 256 for each byte struct PositionReader { - pos: u64 + pos: u64, } impl io::Read for PositionReader { fn read(&mut self, buf: &mut [u8]) -> io::Result { @@ -181,7 +183,7 @@ fn test_buffered_reader_seek_underflow() { let mut reader = BufReader::with_capacity(5, AllowStdIo::new(PositionReader { pos: 0 })); assert_eq!(run_fill_buf!(reader).ok(), Some(&[0, 1, 2, 3, 4][..])); - assert_eq!(block_on(reader.seek(SeekFrom::End(-5))).ok(), Some(u64::max_value()-5)); + assert_eq!(block_on(reader.seek(SeekFrom::End(-5))).ok(), Some(u64::max_value() - 5)); assert_eq!(run_fill_buf!(reader).ok().map(|s| s.len()), Some(5)); // the following seek will require two underlying seeks let expected = 9_223_372_036_854_775_802; @@ -195,7 +197,7 @@ fn test_buffered_reader_seek_underflow() { #[test] fn test_short_reads() { use futures::executor::block_on; - use futures::io::{AsyncReadExt, AllowStdIo, BufReader}; + use futures::io::{AllowStdIo, AsyncReadExt, BufReader}; use std::io; /// A dummy reader intended at testing short-reads propagation. @@ -228,8 +230,8 @@ fn test_short_reads() { #[test] fn maybe_pending() { use futures::io::{AsyncReadExt, BufReader}; - use util::run; use maybe_pending::MaybePending; + use util::run; let inner: &[u8] = &[5, 6, 7, 0, 1, 2, 3, 4]; let mut reader = BufReader::with_capacity(2, MaybePending::new(inner)); @@ -269,8 +271,8 @@ fn maybe_pending() { #[test] fn maybe_pending_buf_read() { use futures::io::{AsyncBufReadExt, BufReader}; - use util::run; use maybe_pending::MaybePending; + use util::run; let inner = MaybePending::new(&[0, 1, 2, 3, 1, 0]); let mut reader = BufReader::with_capacity(2, inner); @@ -291,10 +293,10 @@ fn maybe_pending_buf_read() { // https://github.com/rust-lang/futures-rs/pull/1573#discussion_r281162309 #[test] fn maybe_pending_seek() { - use futures::io::{AsyncBufRead, AsyncSeek, AsyncSeekExt, AsyncRead, BufReader, - Cursor, SeekFrom + use futures::io::{ + AsyncBufRead, AsyncRead, AsyncSeek, AsyncSeekExt, BufReader, Cursor, SeekFrom, }; - use futures::task::{Context,Poll}; + use futures::task::{Context, Poll}; use std::io; use std::pin::Pin; use util::run; @@ -310,17 +312,20 @@ fn maybe_pending_seek() { } impl AsyncRead for MaybePendingSeek<'_> { - fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) - -> Poll> - { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { Pin::new(&mut self.inner).poll_read(cx, buf) } } impl AsyncBufRead for MaybePendingSeek<'_> { - fn poll_fill_buf(mut self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll> - { + fn poll_fill_buf( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { let this: *mut Self = &mut *self as *mut _; Pin::new(&mut unsafe { &mut *this }.inner).poll_fill_buf(cx) } @@ -331,9 +336,11 @@ fn maybe_pending_seek() { } impl AsyncSeek for MaybePendingSeek<'_> { - fn poll_seek(mut self: Pin<&mut Self>, cx: &mut Context<'_>, pos: SeekFrom) - -> Poll> - { + fn poll_seek( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + pos: SeekFrom, + ) -> Poll> { if self.ready { self.ready = false; Pin::new(&mut self.inner).poll_seek(cx, pos) diff --git a/futures/tests/io_buf_writer.rs b/futures/tests/io_buf_writer.rs index d58a6d801f..cd9819439a 100644 --- a/futures/tests/io_buf_writer.rs +++ b/futures/tests/io_buf_writer.rs @@ -197,7 +197,9 @@ fn maybe_pending_buf_writer_inner_flushes() { #[test] fn maybe_pending_buf_writer_seek() { - use futures::io::{AsyncSeek, AsyncSeekExt, AsyncWrite, AsyncWriteExt, BufWriter, Cursor, SeekFrom}; + use futures::io::{ + AsyncSeek, AsyncSeekExt, AsyncWrite, AsyncWriteExt, BufWriter, Cursor, SeekFrom, + }; use futures::task::{Context, Poll}; use std::io; use std::pin::Pin; @@ -241,9 +243,11 @@ fn maybe_pending_buf_writer_seek() { } impl AsyncSeek for MaybePendingSeek { - fn poll_seek(mut self: Pin<&mut Self>, cx: &mut Context<'_>, pos: SeekFrom) - -> Poll> - { + fn poll_seek( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + pos: SeekFrom, + ) -> Poll> { if self.ready_seek { self.ready_seek = false; Pin::new(&mut self.inner).poll_seek(cx, pos) diff --git a/futures/tests/io_lines.rs b/futures/tests/io_lines.rs index 2552c7c40a..57dd567577 100644 --- a/futures/tests/io_lines.rs +++ b/futures/tests/io_lines.rs @@ -2,9 +2,9 @@ mod util { use futures::future::Future; pub fn run(mut f: F) -> F::Output { - use futures_test::task::noop_context; - use futures::task::Poll; use futures::future::FutureExt; + use futures::task::Poll; + use futures_test::task::noop_context; let mut cx = noop_context(); loop { @@ -18,8 +18,8 @@ mod util { #[test] fn lines() { use futures::executor::block_on; - use futures::stream::StreamExt; use futures::io::{AsyncBufReadExt, Cursor}; + use futures::stream::StreamExt; macro_rules! block_on_next { ($expr:expr) => { @@ -41,8 +41,8 @@ fn lines() { #[test] fn maybe_pending() { - use futures::stream::{self, StreamExt, TryStreamExt}; use futures::io::AsyncBufReadExt; + use futures::stream::{self, StreamExt, TryStreamExt}; use futures_test::io::AsyncReadTestExt; use util::run; @@ -53,10 +53,8 @@ fn maybe_pending() { }; } - let buf = stream::iter(vec![&b"12"[..], &b"\r"[..]]) - .map(Ok) - .into_async_read() - .interleave_pending(); + let buf = + stream::iter(vec![&b"12"[..], &b"\r"[..]]).map(Ok).into_async_read().interleave_pending(); let mut s = buf.lines(); assert_eq!(run_next!(s), "12\r".to_string()); assert!(run(s.next()).is_none()); diff --git a/futures/tests/io_read.rs b/futures/tests/io_read.rs index 5902ad0ed9..9c70683b41 100644 --- a/futures/tests/io_read.rs +++ b/futures/tests/io_read.rs @@ -18,7 +18,7 @@ mod mock_reader { fn poll_read( self: Pin<&mut Self>, _cx: &mut Context<'_>, - buf: &mut [u8] + buf: &mut [u8], ) -> Poll> { (self.get_mut().fun)(buf) } diff --git a/futures/tests/io_read_exact.rs b/futures/tests/io_read_exact.rs index bd4b36deaf..b76b3b6c59 100644 --- a/futures/tests/io_read_exact.rs +++ b/futures/tests/io_read_exact.rs @@ -8,7 +8,7 @@ fn read_exact() { let res = block_on(reader.read_exact(&mut out)); // read 3 bytes out assert!(res.is_ok()); - assert_eq!(out, [1,2,3]); + assert_eq!(out, [1, 2, 3]); assert_eq!(reader.len(), 2); let res = block_on(reader.read_exact(&mut out)); // read another 3 bytes, but only 2 bytes left diff --git a/futures/tests/io_read_line.rs b/futures/tests/io_read_line.rs index 51e8126ada..7f07b0a88b 100644 --- a/futures/tests/io_read_line.rs +++ b/futures/tests/io_read_line.rs @@ -37,8 +37,8 @@ fn maybe_pending() { } } - use futures::stream::{self, StreamExt, TryStreamExt}; use futures::io::AsyncBufReadExt; + use futures::stream::{self, StreamExt, TryStreamExt}; use futures_test::io::AsyncReadTestExt; let mut buf = b"12".interleave_pending(); @@ -46,10 +46,8 @@ fn maybe_pending() { assert_eq!(run(buf.read_line(&mut v)).unwrap(), 2); assert_eq!(v, "12"); - let mut buf = stream::iter(vec![&b"12"[..], &b"\n\n"[..]]) - .map(Ok) - .into_async_read() - .interleave_pending(); + let mut buf = + stream::iter(vec![&b"12"[..], &b"\n\n"[..]]).map(Ok).into_async_read().interleave_pending(); let mut v = String::new(); assert_eq!(run(buf.read_line(&mut v)).unwrap(), 3); assert_eq!(v, "12\n"); diff --git a/futures/tests/io_read_to_string.rs b/futures/tests/io_read_to_string.rs index 2e9c00a138..b89800e621 100644 --- a/futures/tests/io_read_to_string.rs +++ b/futures/tests/io_read_to_string.rs @@ -21,14 +21,14 @@ fn read_to_string() { #[test] fn interleave_pending() { use futures::future::Future; - use futures::stream::{self, StreamExt, TryStreamExt}; use futures::io::AsyncReadExt; + use futures::stream::{self, StreamExt, TryStreamExt}; use futures_test::io::AsyncReadTestExt; fn run(mut f: F) -> F::Output { use futures::future::FutureExt; - use futures_test::task::noop_context; use futures::task::Poll; + use futures_test::task::noop_context; let mut cx = noop_context(); loop { diff --git a/futures/tests/io_read_until.rs b/futures/tests/io_read_until.rs index 6fa22eee65..b952427de1 100644 --- a/futures/tests/io_read_until.rs +++ b/futures/tests/io_read_until.rs @@ -26,8 +26,8 @@ fn maybe_pending() { fn run(mut f: F) -> F::Output { use futures::future::FutureExt; - use futures_test::task::noop_context; use futures::task::Poll; + use futures_test::task::noop_context; let mut cx = noop_context(); loop { @@ -37,8 +37,8 @@ fn maybe_pending() { } } - use futures::stream::{self, StreamExt, TryStreamExt}; use futures::io::AsyncBufReadExt; + use futures::stream::{self, StreamExt, TryStreamExt}; use futures_test::io::AsyncReadTestExt; let mut buf = b"12".interleave_pending(); diff --git a/futures/tests/io_write.rs b/futures/tests/io_write.rs index 363f32b1a6..9ca673af4e 100644 --- a/futures/tests/io_write.rs +++ b/futures/tests/io_write.rs @@ -74,11 +74,7 @@ fn write_vectored_first_non_empty() { Poll::Ready(Ok(4)) }); let cx = &mut panic_context(); - let bufs = &mut [ - io::IoSlice::new(&[]), - io::IoSlice::new(&[]), - io::IoSlice::new(b"four") - ]; + let bufs = &mut [io::IoSlice::new(&[]), io::IoSlice::new(&[]), io::IoSlice::new(b"four")]; let res = Pin::new(&mut writer).poll_write_vectored(cx, bufs); let res = res.map_err(|e| e.kind()); diff --git a/futures/tests/join_all.rs b/futures/tests/join_all.rs index c322e58a13..f4cb39e690 100644 --- a/futures/tests/join_all.rs +++ b/futures/tests/join_all.rs @@ -1,6 +1,6 @@ mod util { - use std::future::Future; use std::fmt::Debug; + use std::future::Future; pub fn assert_done(actual_fut: F, expected: T) where @@ -16,7 +16,7 @@ mod util { #[test] fn collect_collects() { - use futures_util::future::{join_all,ready}; + use futures_util::future::{join_all, ready}; util::assert_done(|| Box::new(join_all(vec![ready(1), ready(2)])), vec![1, 2]); util::assert_done(|| Box::new(join_all(vec![ready(1)])), vec![1]); @@ -28,7 +28,7 @@ fn collect_collects() { #[test] fn join_all_iter_lifetime() { - use futures_util::future::{join_all,ready}; + use futures_util::future::{join_all, ready}; use std::future::Future; // In futures-rs version 0.1, this function would fail to typecheck due to an overly // conservative type parameterization of `JoinAll`. @@ -37,12 +37,12 @@ fn join_all_iter_lifetime() { Box::new(join_all(iter)) } - util::assert_done(|| sizes(vec![&[1,2,3], &[], &[0]]), vec![3_usize, 0, 1]); + util::assert_done(|| sizes(vec![&[1, 2, 3], &[], &[0]]), vec![3_usize, 0, 1]); } #[test] fn join_all_from_iter() { - use futures_util::future::{JoinAll,ready}; + use futures_util::future::{ready, JoinAll}; util::assert_done( || Box::new(vec![ready(1), ready(2)].into_iter().collect::>()), diff --git a/futures/tests/macro-reexport/src/lib.rs b/futures/tests/macro-reexport/src/lib.rs index 49691c9e07..82f939b343 100644 --- a/futures/tests/macro-reexport/src/lib.rs +++ b/futures/tests/macro-reexport/src/lib.rs @@ -1,8 +1,7 @@ // normal reexport -pub use futures03::{join, try_join, select, select_biased}; +pub use futures03::{join, select, select_biased, try_join}; // reexport + rename pub use futures03::{ - join as join2, try_join as try_join2, - select as select2, select_biased as select_biased2, + join as join2, select as select2, select_biased as select_biased2, try_join as try_join2, }; diff --git a/futures/tests/macro-tests/src/main.rs b/futures/tests/macro-tests/src/main.rs index 5d11f60834..c654e0c92c 100644 --- a/futures/tests/macro-tests/src/main.rs +++ b/futures/tests/macro-tests/src/main.rs @@ -65,5 +65,4 @@ fn main() { _ = b => unreachable!(), }; }); - } diff --git a/futures/tests/macro_comma_support.rs b/futures/tests/macro_comma_support.rs index ca131639dd..50c39b6df4 100644 --- a/futures/tests/macro_comma_support.rs +++ b/futures/tests/macro_comma_support.rs @@ -1,11 +1,6 @@ #[test] fn ready() { - use futures::{ - executor::block_on, - future, - task::Poll, - ready, - }; + use futures::{executor::block_on, future, ready, task::Poll}; block_on(future::poll_fn(|_| { ready!(Poll::Ready(()),); @@ -15,11 +10,7 @@ fn ready() { #[test] fn poll() { - use futures::{ - executor::block_on, - future::FutureExt, - poll, - }; + use futures::{executor::block_on, future::FutureExt, poll}; block_on(async { let _ = poll!(async {}.boxed(),); @@ -28,10 +19,7 @@ fn poll() { #[test] fn join() { - use futures::{ - executor::block_on, - join - }; + use futures::{executor::block_on, join}; block_on(async { let future1 = async { 1 }; @@ -42,11 +30,7 @@ fn join() { #[test] fn try_join() { - use futures::{ - executor::block_on, - future::FutureExt, - try_join, - }; + use futures::{executor::block_on, future::FutureExt, try_join}; block_on(async { let future1 = async { 1 }.never_error(); diff --git a/futures/tests/mutex.rs b/futures/tests/mutex.rs index 68e0301426..ba2799a5ed 100644 --- a/futures/tests/mutex.rs +++ b/futures/tests/mutex.rs @@ -45,10 +45,7 @@ fn mutex_contested() { use std::sync::Arc; let (tx, mut rx) = mpsc::unbounded(); - let pool = futures::executor::ThreadPool::builder() - .pool_size(16) - .create() - .unwrap(); + let pool = futures::executor::ThreadPool::builder().pool_size(16).create().unwrap(); let tx = Arc::new(tx); let mutex = Arc::new(Mutex::new(0)); diff --git a/futures/tests/recurse.rs b/futures/tests/recurse.rs index 87a4ff27c0..c01ce67b31 100644 --- a/futures/tests/recurse.rs +++ b/futures/tests/recurse.rs @@ -1,7 +1,7 @@ #[test] fn lots() { use futures::executor::block_on; - use futures::future::{self, FutureExt, BoxFuture}; + use futures::future::{self, BoxFuture, FutureExt}; use std::sync::mpsc; use std::thread; @@ -15,8 +15,6 @@ fn lots() { } let (tx, rx) = mpsc::channel(); - thread::spawn(|| { - block_on(do_it((1_000, 0)).map(move |x| tx.send(x).unwrap())) - }); + 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/select_all.rs b/futures/tests/select_all.rs index 540db2c410..905b8a2fec 100644 --- a/futures/tests/select_all.rs +++ b/futures/tests/select_all.rs @@ -4,11 +4,7 @@ fn smoke() { use futures::future::{ready, select_all}; use std::collections::HashSet; - let v = vec![ - ready(1), - ready(2), - ready(3), - ]; + let v = vec![ready(1), ready(2), ready(3)]; let mut c = vec![1, 2, 3].into_iter().collect::>(); diff --git a/futures/tests/select_ok.rs b/futures/tests/select_ok.rs index 81cadb7bae..7a9409e5cc 100644 --- a/futures/tests/select_ok.rs +++ b/futures/tests/select_ok.rs @@ -3,12 +3,7 @@ fn ignore_err() { use futures::executor::block_on; use futures::future::{err, ok, select_ok}; - let v = vec![ - err(1), - err(2), - ok(3), - ok(4), - ]; + let v = vec![err(1), err(2), ok(3), ok(4)]; let (i, v) = block_on(select_ok(v)).ok().unwrap(); assert_eq!(i, 3); @@ -26,11 +21,7 @@ fn last_err() { use futures::executor::block_on; use futures::future::{err, ok, select_ok}; - let v = vec![ - ok(1), - err(2), - err(3), - ]; + let v = vec![ok(1), err(2), err(3)]; let (i, v) = block_on(select_ok(v)).ok().unwrap(); assert_eq!(i, 1); diff --git a/futures/tests/shared.rs b/futures/tests/shared.rs index cc0c6a20cf..b51b7afeb2 100644 --- a/futures/tests/shared.rs +++ b/futures/tests/shared.rs @@ -134,9 +134,7 @@ fn peek() { } // Once the Shared has been polled, the value is peekable on the clone. - spawn - .spawn_local_obj(LocalFutureObj::new(Box::new(f1.map(|_| ())))) - .unwrap(); + spawn.spawn_local_obj(LocalFutureObj::new(Box::new(f1.map(|_| ())))).unwrap(); local_pool.run(); for _ in 0..2 { assert_eq!(*f2.peek().unwrap(), Ok(42)); @@ -233,8 +231,5 @@ fn shared_future_that_wakes_itself_until_pending_is_returned() { // The join future can only complete if the second future gets a chance to run after the first // has returned pending - assert_eq!( - block_on(futures::future::join(fut, async { proceed.set(true) })), - ((), ()) - ); + assert_eq!(block_on(futures::future::join(fut, async { proceed.set(true) })), ((), ())); } diff --git a/futures/tests/sink.rs b/futures/tests/sink.rs index 8ed201e394..12abd06e7e 100644 --- a/futures/tests/sink.rs +++ b/futures/tests/sink.rs @@ -32,8 +32,8 @@ mod unwrap { mod flag_cx { use futures::task::{self, ArcWake, Context}; - use std::sync::Arc; use std::sync::atomic::{AtomicBool, Ordering}; + use std::sync::Arc; // An Unpark struct that records unpark events for inspection pub struct Flag(AtomicBool); @@ -129,7 +129,10 @@ mod manual_flush { Ok(()) } - fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_flush( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { if self.data.is_empty() { Poll::Ready(Ok(())) } else { @@ -145,10 +148,7 @@ mod manual_flush { impl ManualFlush { pub fn new() -> Self { - Self { - data: Vec::new(), - waiting_tasks: Vec::new(), - } + Self { data: Vec::new(), waiting_tasks: Vec::new() } } pub fn force_flush(&mut self) -> Vec { @@ -179,10 +179,7 @@ mod allowance { impl Allow { pub fn new() -> Self { - Self { - flag: Cell::new(false), - tasks: RefCell::new(Vec::new()), - } + Self { flag: Cell::new(false), tasks: RefCell::new(Vec::new()) } } pub fn check(&self, cx: &mut Context<'_>) -> bool { @@ -230,10 +227,7 @@ mod allowance { pub fn manual_allow() -> (ManualAllow, Rc) { let allow = Rc::new(Allow::new()); - let manual_allow = ManualAllow { - data: Vec::new(), - allow: allow.clone(), - }; + let manual_allow = ManualAllow { data: Vec::new(), allow: allow.clone() }; (manual_allow, allow) } } @@ -244,11 +238,8 @@ fn either_sink() { use std::collections::VecDeque; use std::pin::Pin; - let mut s = if true { - Vec::::new().left_sink() - } else { - VecDeque::::new().right_sink() - }; + let mut s = + if true { Vec::::new().left_sink() } else { VecDeque::::new().right_sink() }; Pin::new(&mut s).start_send(0).unwrap(); } @@ -325,9 +316,9 @@ fn mpsc_blocking_start_send() { use futures::executor::block_on; use futures::future::{self, FutureExt}; - use start_send_fut::StartSendFut; use flag_cx::flag_cx; use sassert_next::sassert_next; + use start_send_fut::StartSendFut; use unwrap::unwrap; let (mut tx, mut rx) = mpsc::channel::(0); @@ -461,8 +452,8 @@ fn with_flush_propagate() { use futures::sink::{Sink, SinkExt}; use std::pin::Pin; - use manual_flush::ManualFlush; use flag_cx::flag_cx; + use manual_flush::ManualFlush; use unwrap::unwrap; let mut sink = ManualFlush::new().with(future::ok::, ()>); @@ -508,10 +499,10 @@ fn buffer() { use futures::future::FutureExt; use futures::sink::SinkExt; - use start_send_fut::StartSendFut; + use allowance::manual_allow; use flag_cx::flag_cx; + use start_send_fut::StartSendFut; use unwrap::unwrap; - use allowance::manual_allow; let (sink, allow) = manual_allow::(); let sink = sink.buffer(2); @@ -553,8 +544,8 @@ fn fanout_backpressure() { use futures::sink::SinkExt; use futures::stream::StreamExt; - use start_send_fut::StartSendFut; use flag_cx::flag_cx; + use start_send_fut::StartSendFut; use unwrap::unwrap; let (left_send, mut left_recv) = mpsc::channel(0); @@ -604,10 +595,7 @@ fn sink_map_err() { } let tx = mpsc::channel(0).0; - assert_eq!( - Pin::new(&mut tx.sink_map_err(|_| ())).start_send(()), - Err(()) - ); + assert_eq!(Pin::new(&mut tx.sink_map_err(|_| ())).start_send(()), Err(())); } #[test] @@ -636,8 +624,5 @@ fn err_into() { } let tx = mpsc::channel(0).0; - assert_eq!( - Pin::new(&mut tx.sink_err_into()).start_send(()), - Err(ErrIntoTest) - ); + assert_eq!(Pin::new(&mut tx.sink_err_into()).start_send(()), Err(ErrIntoTest)); } diff --git a/futures/tests/split.rs b/futures/tests/split.rs index 86c2fc6b82..34c81a0cbc 100644 --- a/futures/tests/split.rs +++ b/futures/tests/split.rs @@ -18,10 +18,7 @@ fn test_split() { impl Stream for Join { type Item = T::Item; - fn poll_next( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.project().stream.poll_next(cx) } } @@ -29,41 +26,26 @@ fn test_split() { impl, Item> Sink for Join { type Error = U::Error; - fn poll_ready( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.project().sink.poll_ready(cx) } - fn start_send( - self: Pin<&mut Self>, - item: Item, - ) -> Result<(), Self::Error> { + fn start_send(self: Pin<&mut Self>, item: Item) -> Result<(), Self::Error> { self.project().sink.start_send(item) } - fn poll_flush( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.project().sink.poll_flush(cx) } - fn poll_close( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { self.project().sink.poll_close(cx) } } let mut dest: Vec = Vec::new(); { - let join = Join { - stream: stream::iter(vec![10, 20, 30]), - sink: &mut dest - }; + 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"); diff --git a/futures/tests/stream.rs b/futures/tests/stream.rs index 14b283db7a..dc54c21388 100644 --- a/futures/tests/stream.rs +++ b/futures/tests/stream.rs @@ -20,16 +20,11 @@ fn flat_map() { use futures::stream::{self, StreamExt}; futures::executor::block_on(async { - let st = stream::iter(vec![ - stream::iter(0..=4u8), - stream::iter(6..=10), - stream::iter(0..=2), - ]); + let st = + stream::iter(vec![stream::iter(0..=4u8), stream::iter(6..=10), stream::iter(0..=2)]); - let values: Vec<_> = st - .flat_map(|s| s.filter(|v| futures::future::ready(v % 2 == 0))) - .collect() - .await; + let values: Vec<_> = + st.flat_map(|s| s.filter(|v| futures::future::ready(v % 2 == 0))).collect().await; assert_eq!(values, vec![0, 2, 4, 6, 8, 10, 0, 2]); }); @@ -135,8 +130,8 @@ fn ready_chunks_panic_on_cap_zero() { #[test] fn ready_chunks() { use futures::channel::mpsc; - use futures::stream::StreamExt; use futures::sink::SinkExt; + use futures::stream::StreamExt; use futures::FutureExt; use futures_test::task::noop_context; @@ -154,7 +149,7 @@ fn ready_chunks() { tx.send(2).await.unwrap(); tx.send(3).await.unwrap(); tx.send(4).await.unwrap(); - assert_eq!(s.next().await.unwrap(), vec![2,3]); + assert_eq!(s.next().await.unwrap(), vec![2, 3]); assert_eq!(s.next().await.unwrap(), vec![4]); }); } diff --git a/futures/tests/stream_into_async_read.rs b/futures/tests/stream_into_async_read.rs index eb78b59e50..83ff2403b6 100644 --- a/futures/tests/stream_into_async_read.rs +++ b/futures/tests/stream_into_async_read.rs @@ -4,7 +4,7 @@ fn test_into_async_read() { use futures::io::AsyncRead; use futures::stream::{self, TryStreamExt}; use futures::task::Poll; - use futures_test::{task::noop_context, stream::StreamTestExt}; + use futures_test::{stream::StreamTestExt, task::noop_context}; macro_rules! assert_read { ($reader:expr, $buf:expr, $item:expr) => { @@ -57,7 +57,7 @@ fn test_into_async_bufread() -> std::io::Result<()> { use futures::io::AsyncBufRead; use futures::stream::{self, TryStreamExt}; use futures::task::Poll; - use futures_test::{task::noop_context, stream::StreamTestExt}; + use futures_test::{stream::StreamTestExt, task::noop_context}; macro_rules! assert_fill_buf { ($reader:expr, $buf:expr) => { diff --git a/futures/tests/stream_select_next_some.rs b/futures/tests/stream_select_next_some.rs index bec5262c1d..fabc960943 100644 --- a/futures/tests/stream_select_next_some.rs +++ b/futures/tests/stream_select_next_some.rs @@ -30,8 +30,8 @@ fn is_terminated() { #[test] fn select() { - use futures::{future, select}; use futures::stream::{FuturesUnordered, StreamExt}; + use futures::{future, select}; use futures_test::future::FutureTestExt; // Checks that even though `async_tasks` will yield a `None` and return diff --git a/futures/tests/try_join.rs b/futures/tests/try_join.rs index 6c6d0843d5..8b0b38c1a3 100644 --- a/futures/tests/try_join.rs +++ b/futures/tests/try_join.rs @@ -1,6 +1,6 @@ #![deny(unreachable_code)] -use futures::{try_join, executor::block_on}; +use futures::{executor::block_on, try_join}; // TODO: This abuses https://github.com/rust-lang/rust/issues/58733 in order to // test behaviour of the `try_join!` macro with the never type before it is @@ -14,7 +14,6 @@ impl MyTrait for fn() -> T { } type Never = ! as MyTrait>::Output; - #[test] fn try_join_never_error() { block_on(async { diff --git a/futures/tests/try_join_all.rs b/futures/tests/try_join_all.rs index 8e579a2800..18eaf57be8 100644 --- a/futures/tests/try_join_all.rs +++ b/futures/tests/try_join_all.rs @@ -1,7 +1,7 @@ mod util { - use std::future::Future; use futures::executor::block_on; use std::fmt::Debug; + use std::future::Future; pub fn assert_done(actual_fut: F, expected: T) where @@ -42,7 +42,7 @@ fn try_join_all_iter_lifetime() { Box::new(try_join_all(iter)) } - assert_done(|| sizes(vec![&[1,2,3], &[], &[0]]), Ok(vec![3_usize, 0, 1])); + assert_done(|| sizes(vec![&[1, 2, 3], &[], &[0]]), Ok(vec![3_usize, 0, 1])); } #[test] diff --git a/futures/tests/unfold.rs b/futures/tests/unfold.rs index 95722cf8a6..944446401e 100644 --- a/futures/tests/unfold.rs +++ b/futures/tests/unfold.rs @@ -2,9 +2,7 @@ use futures::future; use futures::stream; use futures_test::future::FutureTestExt; -use futures_test::{ - assert_stream_done, assert_stream_next, assert_stream_pending, -}; +use futures_test::{assert_stream_done, assert_stream_next, assert_stream_pending}; #[test] fn unfold1() {