From 53af42cc382cb891dee859625190cbe16b85ee27 Mon Sep 17 00:00:00 2001 From: Taiki Endo Date: Mon, 8 Apr 2019 14:22:36 +0900 Subject: [PATCH 1/2] Update to new futures_api (cx) --- .travis.yml | 2 +- Cargo.toml | 1 + futures-channel/benches/sync_mpsc.rs | 40 ++++---- futures-channel/src/mpsc/mod.rs | 24 ++--- futures-channel/src/oneshot.rs | 18 ++-- futures-channel/tests/channel.rs | 4 +- futures-channel/tests/mpsc.rs | 40 ++++---- futures-channel/tests/oneshot.rs | 22 ++--- futures-core/src/future/future_obj.rs | 30 +++--- futures-core/src/future/mod.rs | 8 +- futures-core/src/stream/mod.rs | 34 +++---- futures-core/src/stream/stream_obj.rs | 30 +++--- .../src/task/__internal/atomic_waker.rs | 6 +- futures-core/src/task/mod.rs | 2 +- futures-executor/benches/thread_notify.rs | 14 +-- futures-executor/src/local_pool.rs | 28 +++--- futures-executor/src/thread_pool.rs | 5 +- futures-executor/tests/local_pool.rs | 8 +- futures-io/src/lib.rs | 96 +++++++++---------- futures-io/tests/cursor.rs | 10 +- futures-select-macro/src/lib.rs | 18 ++-- futures-sink/src/channel_impls.rs | 26 ++--- futures-sink/src/lib.rs | 80 ++++++++-------- futures-test/src/assert.rs | 12 +-- futures-test/src/future/assert_unmoved.rs | 13 +-- futures-test/src/future/mod.rs | 8 +- futures-test/src/future/pending_once.rs | 8 +- futures-test/src/task/panic_waker.rs | 18 ++-- futures-test/src/task/wake_counter.rs | 6 +- futures-util/benches/futures_unordered.rs | 4 +- futures-util/benches_disabled/bilock.rs | 6 +- futures-util/src/async_await/join.rs | 8 +- futures-util/src/async_await/pending.rs | 4 +- futures-util/src/async_await/poll.rs | 6 +- futures-util/src/compat/compat01as03.rs | 50 +++++----- futures-util/src/compat/compat03as01.rs | 37 ++++--- futures-util/src/future/abortable.rs | 8 +- futures-util/src/future/catch_unwind.rs | 6 +- futures-util/src/future/chain.rs | 8 +- futures-util/src/future/disabled/select.rs | 6 +- .../src/future/disabled/select_all.rs | 4 +- futures-util/src/future/disabled/select_ok.rs | 4 +- futures-util/src/future/empty.rs | 4 +- futures-util/src/future/flatten.rs | 6 +- futures-util/src/future/flatten_stream.rs | 8 +- futures-util/src/future/fuse.rs | 6 +- futures-util/src/future/inspect.rs | 6 +- futures-util/src/future/into_stream.rs | 6 +- futures-util/src/future/join.rs | 6 +- futures-util/src/future/join_all.rs | 6 +- futures-util/src/future/lazy.rs | 10 +- futures-util/src/future/map.rs | 6 +- futures-util/src/future/maybe_done.rs | 6 +- futures-util/src/future/mod.rs | 6 +- futures-util/src/future/option.rs | 6 +- futures-util/src/future/poll_fn.rs | 14 +-- futures-util/src/future/ready.rs | 4 +- futures-util/src/future/remote_handle.rs | 12 +-- futures-util/src/future/shared.rs | 16 ++-- futures-util/src/future/then.rs | 6 +- futures-util/src/future/unit_error.rs | 6 +- futures-util/src/io/allow_std.rs | 12 +-- futures-util/src/io/close.rs | 6 +- futures-util/src/io/copy_into.rs | 10 +- futures-util/src/io/disabled/lines.rs | 2 +- futures-util/src/io/flush.rs | 6 +- futures-util/src/io/read.rs | 6 +- futures-util/src/io/read_exact.rs | 6 +- futures-util/src/io/read_to_end.rs | 10 +- futures-util/src/io/split.rs | 34 +++---- futures-util/src/io/write_all.rs | 6 +- futures-util/src/lib.rs | 12 +-- futures-util/src/lock/bilock.rs | 10 +- futures-util/src/lock/mutex.rs | 8 +- futures-util/src/sink/buffer.rs | 31 +++--- futures-util/src/sink/close.rs | 6 +- futures-util/src/sink/drain.rs | 8 +- futures-util/src/sink/err_into.rs | 6 +- futures-util/src/sink/fanout.rs | 20 ++-- futures-util/src/sink/flush.rs | 6 +- futures-util/src/sink/map_err.rs | 18 ++-- futures-util/src/sink/send.rs | 8 +- futures-util/src/sink/send_all.rs | 18 ++-- futures-util/src/sink/with.rs | 26 ++--- futures-util/src/sink/with_flat_map.rs | 30 +++--- futures-util/src/stream/buffer_unordered.rs | 8 +- futures-util/src/stream/buffered.rs | 8 +- futures-util/src/stream/catch_unwind.rs | 6 +- futures-util/src/stream/chain.rs | 8 +- futures-util/src/stream/chunks.rs | 6 +- futures-util/src/stream/collect.rs | 6 +- futures-util/src/stream/concat.rs | 6 +- futures-util/src/stream/empty.rs | 4 +- futures-util/src/stream/filter.rs | 8 +- futures-util/src/stream/filter_map.rs | 8 +- futures-util/src/stream/flatten.rs | 8 +- futures-util/src/stream/fold.rs | 8 +- futures-util/src/stream/for_each.rs | 8 +- .../src/stream/for_each_concurrent.rs | 8 +- futures-util/src/stream/forward.rs | 18 ++-- futures-util/src/stream/fuse.rs | 6 +- futures-util/src/stream/futures_ordered.rs | 10 +- .../src/stream/futures_unordered/mod.rs | 11 ++- futures-util/src/stream/inspect.rs | 6 +- futures-util/src/stream/into_future.rs | 6 +- futures-util/src/stream/iter.rs | 4 +- futures-util/src/stream/map.rs | 6 +- futures-util/src/stream/mod.rs | 6 +- futures-util/src/stream/next.rs | 6 +- futures-util/src/stream/once.rs | 6 +- futures-util/src/stream/peek.rs | 10 +- futures-util/src/stream/poll_fn.rs | 10 +- futures-util/src/stream/repeat.rs | 4 +- futures-util/src/stream/select.rs | 14 +-- futures-util/src/stream/select_all.rs | 8 +- futures-util/src/stream/select_next_some.rs | 8 +- futures-util/src/stream/skip.rs | 8 +- futures-util/src/stream/skip_while.rs | 10 +- futures-util/src/stream/split.rs | 28 +++--- futures-util/src/stream/take.rs | 6 +- futures-util/src/stream/take_while.rs | 8 +- futures-util/src/stream/then.rs | 8 +- futures-util/src/stream/unfold.rs | 6 +- futures-util/src/stream/zip.rs | 8 +- futures-util/src/task/mod.rs | 12 +-- futures-util/src/task/noop_waker.rs | 15 +-- futures-util/src/task/waker_ref.rs | 10 +- futures-util/src/try_future/and_then.rs | 6 +- futures-util/src/try_future/err_into.rs | 6 +- futures-util/src/try_future/flatten_sink.rs | 18 ++-- futures-util/src/try_future/into_future.rs | 6 +- futures-util/src/try_future/map_err.rs | 6 +- futures-util/src/try_future/map_ok.rs | 6 +- futures-util/src/try_future/or_else.rs | 6 +- futures-util/src/try_future/try_chain.rs | 8 +- futures-util/src/try_future/try_join.rs | 8 +- futures-util/src/try_future/try_join_all.rs | 6 +- futures-util/src/try_future/unwrap_or_else.rs | 6 +- futures-util/src/try_stream/err_into.rs | 6 +- .../src/try_stream/into_async_read.rs | 10 +- futures-util/src/try_stream/into_stream.rs | 6 +- futures-util/src/try_stream/map_err.rs | 6 +- futures-util/src/try_stream/map_ok.rs | 6 +- futures-util/src/try_stream/mod.rs | 6 +- .../src/try_stream/try_buffer_unordered.rs | 8 +- futures-util/src/try_stream/try_collect.rs | 6 +- futures-util/src/try_stream/try_concat.rs | 6 +- futures-util/src/try_stream/try_filter_map.rs | 8 +- futures-util/src/try_stream/try_fold.rs | 8 +- futures-util/src/try_stream/try_for_each.rs | 8 +- .../src/try_stream/try_for_each_concurrent.rs | 8 +- futures-util/src/try_stream/try_next.rs | 6 +- futures-util/src/try_stream/try_skip_while.rs | 10 +- futures-util/tests/atomic_waker.rs | 4 +- futures-util/tests/futures_unordered.rs | 10 +- futures-util/tests/mutex.rs | 13 +-- futures-util/tests/select_all.rs | 10 +- futures-util/tests/select_next_some.rs | 11 ++- futures/src/lib.rs | 4 +- futures/tests/abortable.rs | 9 +- futures/tests/eager_drop.rs | 6 +- futures/tests/fuse.rs | 8 +- futures/tests/future_obj.rs | 4 +- futures/tests/futures_ordered.rs | 21 ++-- futures/tests/futures_unordered.rs | 20 ++-- futures/tests/split.rs | 18 ++-- .../tests_disabled/future_flatten_stream.rs | 2 +- futures/tests_disabled/ready_queue.rs | 48 +++++----- futures/tests_disabled/sink.rs | 78 +++++++-------- futures/tests_disabled/stream.rs | 10 +- 170 files changed, 1022 insertions(+), 1009 deletions(-) diff --git a/.travis.yml b/.travis.yml index af161349f2..6be96649d1 100644 --- a/.travis.yml +++ b/.travis.yml @@ -25,7 +25,7 @@ matrix: # When updating this, the reminder to update the minimum required version in README.md. - name: cargo test (minimum required version) - rust: nightly-2019-02-15 + rust: nightly-2019-04-08 - name: cargo clippy rust: nightly diff --git a/Cargo.toml b/Cargo.toml index f02526a563..430205d8ca 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -5,6 +5,7 @@ members = [ "futures-channel", "futures-executor", "futures-io", + "futures-select-macro", "futures-sink", "futures-util", "futures-test", diff --git a/futures-channel/benches/sync_mpsc.rs b/futures-channel/benches/sync_mpsc.rs index 9df44b32a9..f28d2fdde8 100644 --- a/futures-channel/benches/sync_mpsc.rs +++ b/futures-channel/benches/sync_mpsc.rs @@ -9,7 +9,7 @@ use { ready, stream::{Stream, StreamExt}, sink::Sink, - task::{Waker, Poll}, + task::{Context, Poll}, }, futures_test::task::noop_waker_ref, std::pin::Pin, @@ -18,7 +18,7 @@ use { /// Single producer, single consumer #[bench] fn unbounded_1_tx(b: &mut Bencher) { - let waker = noop_waker_ref(); + let mut cx = Context::from_waker(noop_waker_ref()); b.iter(|| { let (tx, mut rx) = mpsc::unbounded(); @@ -27,12 +27,12 @@ fn unbounded_1_tx(b: &mut Bencher) { for i in 0..1000 { // Poll, not ready, park - assert_eq!(Poll::Pending, rx.poll_next_unpin(waker)); + assert_eq!(Poll::Pending, rx.poll_next_unpin(&mut cx)); UnboundedSender::unbounded_send(&tx, i).unwrap(); // Now poll ready - assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(waker)); + assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(&mut cx)); } }) } @@ -40,7 +40,7 @@ fn unbounded_1_tx(b: &mut Bencher) { /// 100 producers, single consumer #[bench] fn unbounded_100_tx(b: &mut Bencher) { - let waker = noop_waker_ref(); + let mut cx = Context::from_waker(noop_waker_ref()); b.iter(|| { let (tx, mut rx) = mpsc::unbounded(); @@ -49,11 +49,11 @@ fn unbounded_100_tx(b: &mut Bencher) { // 1000 send/recv operations total, result should be divided by 1000 for _ in 0..10 { for i in 0..tx.len() { - assert_eq!(Poll::Pending, rx.poll_next_unpin(waker)); + assert_eq!(Poll::Pending, rx.poll_next_unpin(&mut cx)); UnboundedSender::unbounded_send(&tx[i], i).unwrap(); - assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(waker)); + assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(&mut cx)); } } }) @@ -61,14 +61,14 @@ fn unbounded_100_tx(b: &mut Bencher) { #[bench] fn unbounded_uncontended(b: &mut Bencher) { - let waker = noop_waker_ref(); + let mut cx = Context::from_waker(noop_waker_ref()); b.iter(|| { let (tx, mut rx) = mpsc::unbounded(); for i in 0..1000 { UnboundedSender::unbounded_send(&tx, i).expect("send"); // No need to create a task, because poll is not going to park. - assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(waker)); + assert_eq!(Poll::Ready(Some(i)), rx.poll_next_unpin(&mut cx)); } }) } @@ -84,16 +84,16 @@ struct TestSender { impl Stream for TestSender { type Item = u32; - fn poll_next(mut self: Pin<&mut Self>, waker: &Waker) + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = &mut *self; let mut tx = Pin::new(&mut this.tx); - ready!(tx.as_mut().poll_ready(waker)).unwrap(); + ready!(tx.as_mut().poll_ready(cx)).unwrap(); tx.as_mut().start_send(this.last + 1).unwrap(); this.last += 1; - assert_eq!(Poll::Ready(Ok(())), tx.as_mut().poll_flush(waker)); + assert_eq!(Poll::Ready(Ok(())), tx.as_mut().poll_flush(cx)); Poll::Ready(Some(this.last)) } } @@ -101,16 +101,16 @@ impl Stream for TestSender { /// Single producers, single consumer #[bench] fn bounded_1_tx(b: &mut Bencher) { - let waker = noop_waker_ref(); + let mut cx = Context::from_waker(noop_waker_ref()); b.iter(|| { let (tx, mut rx) = mpsc::channel(0); let mut tx = TestSender { tx, last: 0 }; for i in 0..1000 { - assert_eq!(Poll::Ready(Some(i + 1)), tx.poll_next_unpin(waker)); - assert_eq!(Poll::Pending, tx.poll_next_unpin(waker)); - assert_eq!(Poll::Ready(Some(i + 1)), rx.poll_next_unpin(waker)); + assert_eq!(Poll::Ready(Some(i + 1)), tx.poll_next_unpin(&mut cx)); + assert_eq!(Poll::Pending, tx.poll_next_unpin(&mut cx)); + assert_eq!(Poll::Ready(Some(i + 1)), rx.poll_next_unpin(&mut cx)); } }) } @@ -118,7 +118,7 @@ fn bounded_1_tx(b: &mut Bencher) { /// 100 producers, single consumer #[bench] fn bounded_100_tx(b: &mut Bencher) { - let waker = noop_waker_ref(); + let mut cx = Context::from_waker(noop_waker_ref()); b.iter(|| { // Each sender can send one item after specified capacity let (tx, mut rx) = mpsc::channel(0); @@ -133,11 +133,11 @@ fn bounded_100_tx(b: &mut Bencher) { for i in 0..10 { for j in 0..tx.len() { // Send an item - assert_eq!(Poll::Ready(Some(i + 1)), tx[j].poll_next_unpin(waker)); + assert_eq!(Poll::Ready(Some(i + 1)), tx[j].poll_next_unpin(&mut cx)); // Then block - assert_eq!(Poll::Pending, tx[j].poll_next_unpin(waker)); + assert_eq!(Poll::Pending, tx[j].poll_next_unpin(&mut cx)); // Recv the item - assert_eq!(Poll::Ready(Some(i + 1)), rx.poll_next_unpin(waker)); + assert_eq!(Poll::Ready(Some(i + 1)), rx.poll_next_unpin(&mut cx)); } } }) diff --git a/futures-channel/src/mpsc/mod.rs b/futures-channel/src/mpsc/mod.rs index 218c97130c..8f7cb17d5b 100644 --- a/futures-channel/src/mpsc/mod.rs +++ b/futures-channel/src/mpsc/mod.rs @@ -79,7 +79,7 @@ // by the queue structure. use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll, Waker}; use futures_core::task::__internal::AtomicWaker; use std::any::Any; use std::error::Error; @@ -555,7 +555,7 @@ impl SenderInner { /// - `Err(SendError)` if the receiver has been dropped. fn poll_ready( &mut self, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { let state = decode_state(self.inner.state.load(SeqCst)); if !state.is_open { @@ -564,7 +564,7 @@ impl SenderInner { })); } - self.poll_unparked(Some(waker)).map(Ok) + self.poll_unparked(Some(cx)).map(Ok) } /// Returns whether this channel is closed without needing a context. @@ -582,7 +582,7 @@ impl SenderInner { self.inner.recv_task.wake(); } - fn poll_unparked(&mut self, waker: Option<&Waker>) -> Poll<()> { + fn poll_unparked(&mut self, cx: Option<&mut Context<'_>>) -> Poll<()> { // First check the `maybe_parked` variable. This avoids acquiring the // lock in most cases if self.maybe_parked { @@ -600,7 +600,7 @@ impl SenderInner { // // Update the task in case the `Sender` has been moved to another // task - task.task = waker.cloned(); + task.task = cx.map(|cx| cx.waker().clone()); Poll::Pending } else { @@ -649,12 +649,12 @@ impl Sender { /// - `Err(SendError)` if the receiver has been dropped. pub fn poll_ready( &mut self, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { let inner = self.0.as_mut().ok_or(SendError { kind: SendErrorKind::Disconnected, })?; - inner.poll_ready(waker) + inner.poll_ready(cx) } /// Returns whether this channel is closed without needing a context. @@ -679,7 +679,7 @@ impl UnboundedSender { /// Check if the channel is ready to receive a message. pub fn poll_ready( &self, - _: &Waker, + _: &mut Context<'_>, ) -> Poll> { let inner = self.0.as_ref().ok_or(SendError { kind: SendErrorKind::Disconnected, @@ -904,7 +904,7 @@ impl Stream for Receiver { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { // Try to read a message off of the message queue. match self.next_message() { @@ -916,7 +916,7 @@ impl Stream for Receiver { }, Poll::Pending => { // There are no messages to read, in this case, park. - self.inner.as_ref().unwrap().recv_task.register(waker); + self.inner.as_ref().unwrap().recv_task.register(cx.waker()); // Check queue again after parking to prevent race condition: // a message could be added to the queue after previous `next_message` // before `register` call. @@ -971,9 +971,9 @@ impl Stream for UnboundedReceiver { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - Pin::new(&mut self.0).poll_next(waker) + Pin::new(&mut self.0).poll_next(cx) } } diff --git a/futures-channel/src/oneshot.rs b/futures-channel/src/oneshot.rs index ff7b9dacfc..c54e22d9ca 100644 --- a/futures-channel/src/oneshot.rs +++ b/futures-channel/src/oneshot.rs @@ -1,7 +1,7 @@ //! A channel for sending a single message between asynchronous tasks. use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll, Waker}; use std::pin::Pin; use std::sync::Arc; use std::sync::atomic::AtomicBool; @@ -154,7 +154,7 @@ impl Inner { } } - fn poll_cancel(&self, waker: &Waker) -> Poll<()> { + fn poll_cancel(&self, cx: &mut Context<'_>) -> Poll<()> { // Fast path up first, just read the flag and see if our other half is // gone. This flag is set both in our destructor and the oneshot // destructor, but our destructor hasn't run yet so if it's set then the @@ -176,7 +176,7 @@ impl Inner { // `Receiver` may have been dropped. The first thing it does is set the // flag, and if it fails to acquire the lock it assumes that we'll see // the flag later on. So... we then try to see the flag later on! - let handle = waker.clone(); + let handle = cx.waker().clone(); match self.tx_task.try_lock() { Some(mut p) => *p = Some(handle), None => return Poll::Ready(()), @@ -249,7 +249,7 @@ impl Inner { } } - fn recv(&self, waker: &Waker) -> Poll> { + fn recv(&self, cx: &mut Context<'_>) -> Poll> { // Check to see if some data has arrived. If it hasn't then we need to // block our task. // @@ -260,7 +260,7 @@ impl Inner { let done = if self.complete.load(SeqCst) { true } else { - let task = waker.clone(); + let task = cx.waker().clone(); match self.rx_task.try_lock() { Some(mut slot) => { *slot = Some(task); false }, None => true, @@ -348,8 +348,8 @@ impl Sender { /// alive and may be able to receive a message if sent. The current task, /// however, is scheduled to receive a notification if the corresponding /// `Receiver` goes away. - pub fn poll_cancel(&mut self, waker: &Waker) -> Poll<()> { - self.inner.poll_cancel(waker) + pub fn poll_cancel(&mut self, cx: &mut Context<'_>) -> Poll<()> { + self.inner.poll_cancel(cx) } /// Tests to see whether this `Sender`'s corresponding `Receiver` @@ -416,9 +416,9 @@ impl Future for Receiver { fn poll( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.inner.recv(waker) + self.inner.recv(cx) } } diff --git a/futures-channel/tests/channel.rs b/futures-channel/tests/channel.rs index 21ab87105a..7866c4d886 100644 --- a/futures-channel/tests/channel.rs +++ b/futures-channel/tests/channel.rs @@ -36,8 +36,8 @@ async fn send_sequence(n: u32, mut sender: mpsc::Sender) { fn drop_sender() { let (tx, mut rx) = mpsc::channel::(1); drop(tx); - let f = poll_fn(|lw| { - rx.poll_next_unpin(lw) + let f = poll_fn(|cx| { + rx.poll_next_unpin(cx) }); assert_eq!(block_on(f), None) } diff --git a/futures-channel/tests/mpsc.rs b/futures-channel/tests/mpsc.rs index ad99f0b308..924a557c5d 100644 --- a/futures-channel/tests/mpsc.rs +++ b/futures-channel/tests/mpsc.rs @@ -5,7 +5,7 @@ use futures::executor::{block_on, block_on_stream}; use futures::future::{FutureExt, poll_fn}; use futures::stream::{Stream, StreamExt}; use futures::sink::{Sink, SinkExt}; -use futures::task::Poll; +use futures::task::{Context, Poll}; use futures_test::task::noop_waker_ref; use pin_utils::pin_mut; use std::sync::{Arc, Mutex}; @@ -29,34 +29,34 @@ fn send_recv() { #[test] fn send_recv_no_buffer() { // Run on a task context - block_on(poll_fn(move |lw| { + block_on(poll_fn(move |cx| { let (tx, rx) = mpsc::channel::(0); pin_mut!(tx, rx); - assert!(tx.as_mut().poll_flush(lw).is_ready()); - assert!(tx.as_mut().poll_ready(lw).is_ready()); + assert!(tx.as_mut().poll_flush(cx).is_ready()); + assert!(tx.as_mut().poll_ready(cx).is_ready()); // Send first message assert!(tx.as_mut().start_send(1).is_ok()); - assert!(tx.as_mut().poll_ready(lw).is_pending()); + assert!(tx.as_mut().poll_ready(cx).is_pending()); // poll_ready said Pending, so no room in buffer, therefore new sends // should get rejected with is_full. assert!(tx.as_mut().start_send(0).unwrap_err().is_full()); - assert!(tx.as_mut().poll_ready(lw).is_pending()); + assert!(tx.as_mut().poll_ready(cx).is_pending()); // Take the value - assert_eq!(rx.as_mut().poll_next(lw), Poll::Ready(Some(1))); - assert!(tx.as_mut().poll_ready(lw).is_ready()); + assert_eq!(rx.as_mut().poll_next(cx), Poll::Ready(Some(1))); + assert!(tx.as_mut().poll_ready(cx).is_ready()); // Send second message - assert!(tx.as_mut().poll_ready(lw).is_ready()); + assert!(tx.as_mut().poll_ready(cx).is_ready()); assert!(tx.as_mut().start_send(2).is_ok()); - assert!(tx.as_mut().poll_ready(lw).is_pending()); + assert!(tx.as_mut().poll_ready(cx).is_pending()); // Take the value - assert_eq!(rx.as_mut().poll_next(lw), Poll::Ready(Some(2))); - assert!(tx.as_mut().poll_ready(lw).is_ready()); + assert_eq!(rx.as_mut().poll_next(cx), Poll::Ready(Some(2))); + assert!(tx.as_mut().poll_ready(cx).is_ready()); Poll::Ready(()) })); @@ -119,11 +119,11 @@ fn recv_close_gets_none() { let (mut tx, mut rx) = mpsc::channel::(10); // Run on a task context - block_on(poll_fn(move |lw| { + block_on(poll_fn(move |cx| { rx.close(); - assert_eq!(rx.poll_next_unpin(lw), Poll::Ready(None)); - match tx.poll_ready(lw) { + assert_eq!(rx.poll_next_unpin(cx), Poll::Ready(None)); + match tx.poll_ready(cx) { Poll::Pending | Poll::Ready(Ok(_)) => panic!(), Poll::Ready(Err(e)) => assert!(e.is_disconnected()), }; @@ -139,8 +139,8 @@ fn tx_close_gets_none() { let (_, mut rx) = mpsc::channel::(10); // Run on a task context - block_on(poll_fn(move |lw| { - assert_eq!(rx.poll_next_unpin(lw), Poll::Ready(None)); + block_on(poll_fn(move |cx| { + assert_eq!(rx.poll_next_unpin(cx), Poll::Ready(None)); Poll::Ready(()) })); } @@ -304,7 +304,7 @@ fn stress_receiver_multi_task_bounded_hard() { } else { // Just poll let n = n.clone(); - match rx.poll_next_unpin(noop_waker_ref()) { + match rx.poll_next_unpin(&mut Context::from_waker(noop_waker_ref())) { Poll::Ready(Some(_)) => { n.fetch_add(1, Ordering::Relaxed); } @@ -472,8 +472,8 @@ fn try_send_2() { let (readytx, readyrx) = oneshot::channel::<()>(); let th = thread::spawn(move || { - block_on(poll_fn(|lw| { - assert!(tx.poll_ready(lw).is_pending()); + block_on(poll_fn(|cx| { + assert!(tx.poll_ready(cx).is_pending()); Poll::Ready(()) })); diff --git a/futures-channel/tests/oneshot.rs b/futures-channel/tests/oneshot.rs index c66acbc038..06b0e3599f 100644 --- a/futures-channel/tests/oneshot.rs +++ b/futures-channel/tests/oneshot.rs @@ -3,7 +3,7 @@ use futures::channel::oneshot::{self, Sender}; use futures::executor::block_on; use futures::future::{Future, FutureExt, poll_fn}; -use futures::task::{Waker, Poll}; +use futures::task::{Context, Poll}; use std::pin::Pin; use std::sync::mpsc; use std::thread; @@ -12,12 +12,12 @@ use std::thread; fn smoke_poll() { let (mut tx, rx) = oneshot::channel::(); let mut rx = Some(rx); - let f = poll_fn(|waker| { - assert!(tx.poll_cancel(waker).is_pending()); - assert!(tx.poll_cancel(waker).is_pending()); + let f = poll_fn(|cx| { + assert!(tx.poll_cancel(cx).is_pending()); + assert!(tx.poll_cancel(cx).is_pending()); drop(rx.take()); - assert!(tx.poll_cancel(waker).is_ready()); - assert!(tx.poll_cancel(waker).is_ready()); + assert!(tx.poll_cancel(cx).is_ready()); + assert!(tx.poll_cancel(cx).is_ready()); Poll::Ready(()) }); @@ -42,8 +42,8 @@ struct WaitForCancel { impl Future for WaitForCancel { type Output = (); - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - self.tx.poll_cancel(waker) + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.tx.poll_cancel(cx) } } @@ -73,12 +73,12 @@ fn cancel_lots() { fn close() { let (mut tx, mut rx) = oneshot::channel::(); rx.close(); - block_on(poll_fn(|waker| { - match rx.poll_unpin(waker) { + block_on(poll_fn(|cx| { + match rx.poll_unpin(cx) { Poll::Ready(Err(_)) => {}, _ => panic!(), }; - assert!(tx.poll_cancel(waker).is_ready()); + assert!(tx.poll_cancel(cx).is_ready()); Poll::Ready(()) })); } diff --git a/futures-core/src/future/future_obj.rs b/futures-core/src/future/future_obj.rs index d765cabcf5..66b5d5c754 100644 --- a/futures-core/src/future/future_obj.rs +++ b/futures-core/src/future/future_obj.rs @@ -3,7 +3,7 @@ use core::{ future::Future, marker::PhantomData, pin::Pin, - task::{Waker, Poll}, + task::{Context, Poll}, }; /// A custom trait object for polling futures, roughly akin to @@ -14,7 +14,7 @@ use core::{ /// `Box` is not available in no_std contexts. pub struct LocalFutureObj<'a, T> { ptr: *mut (), - poll_fn: unsafe fn(*mut (), &Waker) -> Poll, + poll_fn: unsafe fn(*mut (), &mut Context<'_>) -> Poll, drop_fn: unsafe fn(*mut ()), _marker: PhantomData<&'a ()>, } @@ -61,9 +61,9 @@ impl<'a, T> Future for LocalFutureObj<'a, T> { type Output = T; #[inline] - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { unsafe { - ((*self).poll_fn)((*self).ptr, waker) + ((*self).poll_fn)((*self).ptr, cx) } } } @@ -111,11 +111,11 @@ impl<'a, T> Future for FutureObj<'a, T> { type Output = T; #[inline] - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let pinned_field: Pin<&mut LocalFutureObj<'a, T>> = unsafe { Pin::map_unchecked_mut(self, |x| &mut x.0) }; - LocalFutureObj::poll(pinned_field, waker) + LocalFutureObj::poll(pinned_field, cx) } } @@ -140,7 +140,7 @@ pub unsafe trait UnsafeFutureObj<'a, T>: 'a { /// `poll` with the result of `into_raw` until `drop` is called; such calls /// are not, however, allowed to race with each other or with calls to /// `drop`. - unsafe fn poll(ptr: *mut (), waker: &Waker) -> Poll; + unsafe fn poll(ptr: *mut (), cx: &mut Context<'_>) -> Poll; /// Drops the future represented by the given void pointer. /// @@ -160,9 +160,9 @@ where self as *mut F as *mut () } - unsafe fn poll(ptr: *mut (), waker: &Waker) -> Poll { + unsafe fn poll(ptr: *mut (), cx: &mut Context<'_>) -> Poll { let p: Pin<&mut F> = Pin::new_unchecked(&mut *(ptr as *mut F)); - F::poll(p, waker) + F::poll(p, cx) } unsafe fn drop(_ptr: *mut ()) {} @@ -177,9 +177,9 @@ where mut_ref as *mut F as *mut () } - unsafe fn poll(ptr: *mut (), waker: &Waker) -> Poll { + unsafe fn poll(ptr: *mut (), cx: &mut Context<'_>) -> Poll { let future: Pin<&mut F> = Pin::new_unchecked(&mut *(ptr as *mut F)); - F::poll(future, waker) + F::poll(future, cx) } unsafe fn drop(_ptr: *mut ()) {} @@ -198,10 +198,10 @@ mod if_alloc { Box::into_raw(self) as *mut () } - unsafe fn poll(ptr: *mut (), waker: &Waker) -> Poll { + unsafe fn poll(ptr: *mut (), cx: &mut Context<'_>) -> Poll { let ptr = ptr as *mut F; let pin: Pin<&mut F> = Pin::new_unchecked(&mut *ptr); - F::poll(pin, waker) + F::poll(pin, cx) } unsafe fn drop(ptr: *mut ()) { @@ -220,10 +220,10 @@ mod if_alloc { ptr } - unsafe fn poll(ptr: *mut (), waker: &Waker) -> Poll { + unsafe fn poll(ptr: *mut (), cx: &mut Context<'_>) -> Poll { let ptr = ptr as *mut F; let pin: Pin<&mut F> = Pin::new_unchecked(&mut *ptr); - F::poll(pin, waker) + F::poll(pin, cx) } unsafe fn drop(ptr: *mut ()) { diff --git a/futures-core/src/future/mod.rs b/futures-core/src/future/mod.rs index 0d539b3876..4b821efca8 100644 --- a/futures-core/src/future/mod.rs +++ b/futures-core/src/future/mod.rs @@ -1,7 +1,7 @@ //! Futures. -use crate::task::{Waker, Poll}; use core::pin::Pin; +use core::task::{Context, Poll}; pub use core::future::Future; @@ -74,7 +74,7 @@ pub trait TryFuture { /// needed. fn try_poll( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll>; } @@ -85,7 +85,7 @@ impl TryFuture for F type Error = E; #[inline] - fn try_poll(self: Pin<&mut Self>, waker: &Waker) -> Poll { - self.poll(waker) + fn try_poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.poll(cx) } } diff --git a/futures-core/src/stream/mod.rs b/futures-core/src/stream/mod.rs index cddc11e150..f0b50d174f 100644 --- a/futures-core/src/stream/mod.rs +++ b/futures-core/src/stream/mod.rs @@ -1,8 +1,8 @@ //! Asynchronous streams. -use crate::task::{Waker, Poll}; use core::ops; use core::pin::Pin; +use core::task::{Context, Poll}; #[cfg(feature = "either")] use either::Either; @@ -54,7 +54,7 @@ pub trait Stream { /// calls. fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll>; } @@ -63,9 +63,9 @@ impl<'a, S: ?Sized + Stream + Unpin> Stream for &'a mut S { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - S::poll_next(Pin::new(&mut **self), waker) + S::poll_next(Pin::new(&mut **self), cx) } } @@ -78,9 +78,9 @@ where fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - Pin::get_mut(self).as_mut().poll_next(waker) + Pin::get_mut(self).as_mut().poll_next(cx) } } @@ -91,11 +91,11 @@ impl Stream for Either { type Item = A::Item; - fn poll_next(self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { unsafe { match Pin::get_unchecked_mut(self) { - Either::Left(a) => Pin::new_unchecked(a).poll_next(waker), - Either::Right(b) => Pin::new_unchecked(b).poll_next(waker), + Either::Left(a) => Pin::new_unchecked(a).poll_next(cx), + Either::Right(b) => Pin::new_unchecked(b).poll_next(cx), } } } @@ -128,7 +128,7 @@ pub trait TryStream { /// This method is a stopgap for a compiler limitation that prevents us from /// directly inheriting from the `Stream` trait; in the future it won't be /// needed. - fn try_poll_next(self: Pin<&mut Self>, waker: &Waker) + fn try_poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>>; } @@ -138,10 +138,10 @@ impl TryStream for S type Ok = T; type Error = E; - fn try_poll_next(self: Pin<&mut Self>, waker: &Waker) + fn try_poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>> { - self.poll_next(waker) + self.poll_next(cx) } } @@ -155,9 +155,9 @@ mod if_alloc { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - Pin::new(&mut **self).poll_next(waker) + Pin::new(&mut **self).poll_next(cx) } } @@ -167,9 +167,9 @@ mod if_alloc { fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - unsafe { Pin::map_unchecked_mut(self, |x| &mut x.0) }.poll_next(waker) + unsafe { Pin::map_unchecked_mut(self, |x| &mut x.0) }.poll_next(cx) } } @@ -178,7 +178,7 @@ mod if_alloc { fn poll_next( mut self: Pin<&mut Self>, - _lw: &Waker, + _cx: &mut Context<'_>, ) -> Poll> { Poll::Ready(self.pop_front()) } diff --git a/futures-core/src/stream/stream_obj.rs b/futures-core/src/stream/stream_obj.rs index 70559ecffd..b1a20b73a8 100644 --- a/futures-core/src/stream/stream_obj.rs +++ b/futures-core/src/stream/stream_obj.rs @@ -1,8 +1,8 @@ use super::Stream; -use crate::task::{Waker, Poll}; use core::fmt; use core::marker::PhantomData; use core::pin::Pin; +use core::task::{Context, Poll}; /// A custom trait object for polling streams, roughly akin to /// `Box + 'a>`. @@ -12,7 +12,7 @@ use core::pin::Pin; /// `Box` is not available in no_std contexts. pub struct LocalStreamObj<'a, T> { ptr: *mut (), - poll_next_fn: unsafe fn(*mut (), &Waker) -> Poll>, + poll_next_fn: unsafe fn(*mut (), &mut Context<'_>) -> Poll>, drop_fn: unsafe fn(*mut ()), _marker: PhantomData<&'a ()>, } @@ -60,9 +60,9 @@ impl<'a, T> Stream for LocalStreamObj<'a, T> { #[inline] fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - unsafe { (self.poll_next_fn)(self.ptr, waker) } + unsafe { (self.poll_next_fn)(self.ptr, cx) } } } @@ -108,10 +108,10 @@ impl<'a, T> Stream for StreamObj<'a, T> { #[inline] fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { let pinned_field = unsafe { Pin::map_unchecked_mut(self, |x| &mut x.0) }; - pinned_field.poll_next(waker) + pinned_field.poll_next(cx) } } @@ -138,7 +138,7 @@ pub unsafe trait UnsafeStreamObj<'a, T>: 'a { /// `drop`. unsafe fn poll_next( ptr: *mut (), - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll>; /// Drops the stream represented by the given void pointer. @@ -161,9 +161,9 @@ where unsafe fn poll_next( ptr: *mut (), - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - Pin::new_unchecked(&mut *(ptr as *mut F)).poll_next(waker) + Pin::new_unchecked(&mut *(ptr as *mut F)).poll_next(cx) } unsafe fn drop(_ptr: *mut ()) {} @@ -179,9 +179,9 @@ where unsafe fn poll_next( ptr: *mut (), - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - Pin::new_unchecked(&mut *(ptr as *mut F)).poll_next(waker) + Pin::new_unchecked(&mut *(ptr as *mut F)).poll_next(cx) } unsafe fn drop(_ptr: *mut ()) {} @@ -200,10 +200,10 @@ mod if_alloc { Box::into_raw(self) as *mut () } - unsafe fn poll_next(ptr: *mut (), waker: &Waker) -> Poll> { + unsafe fn poll_next(ptr: *mut (), cx: &mut Context<'_>) -> Poll> { let ptr = ptr as *mut F; let pin: Pin<&mut F> = Pin::new_unchecked(&mut *ptr); - pin.poll_next(waker) + pin.poll_next(cx) } unsafe fn drop(ptr: *mut ()) { @@ -221,10 +221,10 @@ mod if_alloc { ptr } - unsafe fn poll_next(ptr: *mut (), waker: &Waker) -> Poll> { + unsafe fn poll_next(ptr: *mut (), cx: &mut Context<'_>) -> Poll> { let ptr = ptr as *mut F; let pin: Pin<&mut F> = Pin::new_unchecked(&mut *ptr); - pin.poll_next(waker) + pin.poll_next(cx) } unsafe fn drop(ptr: *mut ()) { diff --git a/futures-core/src/task/__internal/atomic_waker.rs b/futures-core/src/task/__internal/atomic_waker.rs index 039f3e198c..e3c9feb74d 100644 --- a/futures-core/src/task/__internal/atomic_waker.rs +++ b/futures-core/src/task/__internal/atomic_waker.rs @@ -171,7 +171,7 @@ impl AtomicWaker { /// ``` /// #![feature(futures_api)] /// use futures::future::Future; - /// use futures::task::{Waker, Poll, AtomicWaker}; + /// use futures::task::{Context, Poll, AtomicWaker}; /// use std::sync::atomic::AtomicBool; /// use std::sync::atomic::Ordering::SeqCst; /// use std::pin::Pin; @@ -184,10 +184,10 @@ impl AtomicWaker { /// impl Future for Flag { /// type Output = (); /// - /// fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll<()> { + /// fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> { /// // Register **before** checking `set` to avoid a race condition /// // that would result in lost notifications. - /// self.waker.register(waker); + /// self.waker.register(cx.waker()); /// /// if self.set.load(SeqCst) { /// Poll::Ready(()) diff --git a/futures-core/src/task/mod.rs b/futures-core/src/task/mod.rs index 239921a9f4..1747b55659 100644 --- a/futures-core/src/task/mod.rs +++ b/futures-core/src/task/mod.rs @@ -5,4 +5,4 @@ mod spawn; pub mod __internal; pub use self::spawn::{Spawn, LocalSpawn, SpawnError}; -pub use core::task::{Poll, Waker, RawWaker, RawWakerVTable}; +pub use core::task::{Context, Poll, Waker, RawWaker, RawWakerVTable}; diff --git a/futures-executor/benches/thread_notify.rs b/futures-executor/benches/thread_notify.rs index e36e463d09..a26ddde0cb 100644 --- a/futures-executor/benches/thread_notify.rs +++ b/futures-executor/benches/thread_notify.rs @@ -5,7 +5,7 @@ use crate::test::Bencher; use futures::executor::block_on; use futures::future::Future; -use futures::task::{Poll, Waker}; +use futures::task::{Context, Poll, Waker}; use std::pin::Pin; #[bench] @@ -19,12 +19,12 @@ fn thread_yield_single_thread_one_wait(b: &mut Bencher) { impl Future for Yield { type Output = (); - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { if self.rem == 0 { Poll::Ready(()) } else { self.rem -= 1; - waker.wake(); + cx.waker().wake(); Poll::Pending } } @@ -47,12 +47,12 @@ fn thread_yield_single_thread_many_wait(b: &mut Bencher) { impl Future for Yield { type Output = (); - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { if self.rem == 0 { Poll::Ready(()) } else { self.rem -= 1; - waker.wake(); + cx.waker().wake(); Poll::Pending } } @@ -84,12 +84,12 @@ fn thread_yield_multi_thread(b: &mut Bencher) { impl Future for Yield { type Output = (); - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { if self.rem == 0 { Poll::Ready(()) } else { self.rem -= 1; - self.tx.send(waker.clone()).unwrap(); + self.tx.send(cx.waker().clone()).unwrap(); Poll::Pending } } diff --git a/futures-executor/src/local_pool.rs b/futures-executor/src/local_pool.rs index 06fa50933c..f729db6dea 100644 --- a/futures-executor/src/local_pool.rs +++ b/futures-executor/src/local_pool.rs @@ -1,11 +1,8 @@ use crate::enter; use futures_core::future::{Future, FutureObj, LocalFutureObj}; use futures_core::stream::{Stream}; -use futures_core::task::{ - Poll, Waker, - Spawn, LocalSpawn, SpawnError, -}; -use futures_util::task::{WakerRef, waker_ref, ArcWake}; +use futures_core::task::{Context, Poll, Spawn, LocalSpawn, SpawnError}; +use futures_util::task::{waker_ref, ArcWake}; use futures_util::stream::FuturesUnordered; use futures_util::stream::StreamExt; use pin_utils::pin_mut; @@ -60,15 +57,16 @@ impl ArcWake for ThreadNotify { // Set up and run a basic single-threaded spawner loop, invoking `f` on each // turn. -fn run_executor Poll>(mut f: F) -> T { +fn run_executor) -> Poll>(mut f: F) -> T { let _enter = enter() .expect("cannot execute `LocalPool` executor from within \ another executor"); CURRENT_THREAD_NOTIFY.with(|thread_notify| { - let waker: WakerRef<'_> = waker_ref(thread_notify); + let waker = waker_ref(thread_notify); + let mut cx = Context::from_waker(&waker); loop { - if let Poll::Ready(t) = f(&waker) { + if let Poll::Ready(t) = f(&mut cx) { return t; } thread::park(); @@ -112,7 +110,7 @@ impl LocalPool { /// The function will block the calling thread until *all* tasks in the pool /// are complete, including any spawned while running existing tasks. pub fn run(&mut self) { - run_executor(|waker| self.poll_pool(waker)) + run_executor(|cx| self.poll_pool(cx)) } /// Runs all the tasks in the pool until the given future completes. @@ -142,23 +140,23 @@ impl LocalPool { pub fn run_until(&mut self, future: F) -> F::Output { pin_mut!(future); - run_executor(|waker| { + run_executor(|cx| { { // if our main task is done, so are we - let result = future.as_mut().poll(waker); + let result = future.as_mut().poll(cx); if let Poll::Ready(output) = result { return Poll::Ready(output); } } - let _ = self.poll_pool(waker); + let _ = self.poll_pool(cx); Poll::Pending }) } // Make maximal progress on the entire pool of spawned task, returning `Ready` // if the pool is empty and `Pending` if no further progress can be made. - fn poll_pool(&mut self, waker: &Waker) -> Poll<()> { + fn poll_pool(&mut self, cx: &mut Context<'_>) -> Poll<()> { // state for the FuturesUnordered, which will never be used loop { // empty the incoming queue of newly-spawned tasks @@ -169,7 +167,7 @@ impl LocalPool { } } - let ret = self.pool.poll_next_unpin(waker); + let ret = self.pool.poll_next_unpin(cx); // we queued up some new tasks; add them and poll again if !self.incoming.borrow().is_empty() { continue; @@ -199,7 +197,7 @@ impl Default for LocalPool { /// spawned tasks. pub fn block_on(f: F) -> F::Output { pin_mut!(f); - run_executor(|waker| f.as_mut().poll(waker)) + run_executor(|cx| f.as_mut().poll(cx)) } /// Turn a stream into a blocking iterator. diff --git a/futures-executor/src/thread_pool.rs b/futures-executor/src/thread_pool.rs index 0a9879d2aa..227db93df1 100644 --- a/futures-executor/src/thread_pool.rs +++ b/futures-executor/src/thread_pool.rs @@ -1,7 +1,7 @@ use crate::enter; use crate::unpark_mutex::UnparkMutex; use futures_core::future::{Future, FutureObj}; -use futures_core::task::{Poll, Spawn, SpawnError}; +use futures_core::task::{Context, Poll, Spawn, SpawnError}; use futures_util::future::FutureExt; use futures_util::task::{ArcWake, waker_ref}; use num_cpus; @@ -300,6 +300,7 @@ impl Task { pub fn run(self) { let Task { mut future, wake_handle, mut exec } = self; let waker = waker_ref(&wake_handle); + let mut cx = Context::from_waker(&waker); // Safety: The ownership of this `Task` object is evidence that // we are in the `POLLING`/`REPOLL` state for the mutex. @@ -307,7 +308,7 @@ impl Task { wake_handle.mutex.start_poll(); loop { - let res = future.poll_unpin(&waker); + let res = future.poll_unpin(&mut cx); match res { Poll::Pending => {} Poll::Ready(()) => return wake_handle.mutex.complete(), diff --git a/futures-executor/tests/local_pool.rs b/futures-executor/tests/local_pool.rs index 93bcc13d92..2bebc832ad 100644 --- a/futures-executor/tests/local_pool.rs +++ b/futures-executor/tests/local_pool.rs @@ -3,7 +3,7 @@ use futures::channel::oneshot; use futures::executor::LocalPool; use futures::future::{Future, lazy}; -use futures::task::{Waker, Poll, Spawn, LocalSpawn}; +use futures::task::{Context, Poll, Spawn, LocalSpawn}; use std::cell::{Cell, RefCell}; use std::pin::Pin; use std::rc::Rc; @@ -13,7 +13,7 @@ struct Pending(Rc<()>); impl Future for Pending { type Output = (); - fn poll(self: Pin<&mut Self>, _waker: &Waker) -> Poll<()> { + fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<()> { Poll::Pending } } @@ -128,7 +128,7 @@ fn tasks_are_scheduled_fairly() { impl Future for Spin { type Output = (); - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll<()> { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> { let mut state = self.state.borrow_mut(); if self.idx == 0 { @@ -147,7 +147,7 @@ fn tasks_are_scheduled_fairly() { return Poll::Ready(()); } - waker.wake(); + cx.waker().wake(); Poll::Pending } } diff --git a/futures-io/src/lib.rs b/futures-io/src/lib.rs index c615476ba8..5e8662b5ff 100644 --- a/futures-io/src/lib.rs +++ b/futures-io/src/lib.rs @@ -14,7 +14,7 @@ #[cfg(feature = "std")] mod if_std { - use futures_core::task::{Waker, Poll}; + use futures_core::task::{Context, Poll}; use std::boxed::Box; use std::cmp; use std::io as StdIo; @@ -100,7 +100,7 @@ mod if_std { /// /// If no data is available for reading, the method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `waker.wake()`) to receive a notification when the object becomes + /// `cx.waker().wake()`) to receive a notification when the object becomes /// readable or is closed. /// /// # Implementation @@ -109,7 +109,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_read(self: Pin<&mut Self>, waker: &Waker, buf: &mut [u8]) + fn poll_read(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) -> Poll>; /// Attempt to read from the `AsyncRead` into `vec` using vectored @@ -122,7 +122,7 @@ mod if_std { /// /// If no data is available for reading, the method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `waker.wake()`) to receive a notification when the object becomes + /// `cx.waker().wake()`) to receive a notification when the object becomes /// readable or is closed. /// By default, this method delegates to using `poll_read` on the first /// buffer in `vec`. Objects which support vectored IO should override @@ -134,11 +134,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_vectored_read(self: Pin<&mut Self>, waker: &Waker, vec: &mut [&mut IoVec]) + fn poll_vectored_read(self: Pin<&mut Self>, cx: &mut Context<'_>, vec: &mut [&mut IoVec]) -> Poll> { if let Some(ref mut first_iovec) = vec.get_mut(0) { - self.poll_read(waker, first_iovec) + self.poll_read(cx, first_iovec) } else { // `vec` is empty. Poll::Ready(Ok(0)) @@ -160,7 +160,7 @@ mod if_std { /// /// If the object is not ready for writing, the method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `waker.wake()`) to receive a notification when the object becomes + /// `cx.waker().wake()`) to receive a notification when the object becomes /// readable or is closed. /// /// # Implementation @@ -169,7 +169,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_write(self: Pin<&mut Self>, waker: &Waker, buf: &[u8]) + fn poll_write(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) -> Poll>; /// Attempt to write bytes from `vec` into the object using vectored @@ -182,7 +182,7 @@ mod if_std { /// /// If the object is not ready for writing, the method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `waker.wake()`) to receive a notification when the object becomes + /// `cx.waker().wake()`) to receive a notification when the object becomes /// readable or is closed. /// /// By default, this method delegates to using `poll_write` on the first @@ -195,11 +195,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_vectored_write(self: Pin<&mut Self>, waker: &Waker, vec: &[&IoVec]) + fn poll_vectored_write(self: Pin<&mut Self>, cx: &mut Context<'_>, vec: &[&IoVec]) -> Poll> { if let Some(ref first_iovec) = vec.get(0) { - self.poll_write(waker, &*first_iovec) + self.poll_write(cx, &*first_iovec) } else { // `vec` is empty. Poll::Ready(Ok(0)) @@ -213,7 +213,7 @@ mod if_std { /// /// If flushing cannot immediately complete, this method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `waker.wake()`) to receive a notification when the object can make + /// `cx.waker().wake()`) to receive a notification when the object can make /// progress towards flushing. /// /// # Implementation @@ -222,7 +222,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_flush(self: Pin<&mut Self>, waker: &Waker) -> Poll>; + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; /// Attempt to close the object. /// @@ -230,7 +230,7 @@ mod if_std { /// /// If closing cannot immediately complete, this function returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `waker.wake()`) to receive a notification when the object can make + /// `cx.waker().wake()`) to receive a notification when the object can make /// progress towards closing. /// /// # Implementation @@ -239,7 +239,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_close(self: Pin<&mut Self>, waker: &Waker) -> Poll>; + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; } macro_rules! deref_async_read { @@ -248,16 +248,16 @@ mod if_std { (**self).initializer() } - fn poll_read(mut self: Pin<&mut Self>, waker: &Waker, buf: &mut [u8]) + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) -> Poll> { - Pin::new(&mut **self).poll_read(waker, buf) + Pin::new(&mut **self).poll_read(cx, buf) } - fn poll_vectored_read(mut self: Pin<&mut Self>, waker: &Waker, vec: &mut [&mut IoVec]) + fn poll_vectored_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, vec: &mut [&mut IoVec]) -> Poll> { - Pin::new(&mut **self).poll_vectored_read(waker, vec) + Pin::new(&mut **self).poll_vectored_read(cx, vec) } } } @@ -275,16 +275,16 @@ mod if_std { (**self).initializer() } - fn poll_read(mut self: Pin<&mut Self>, waker: &Waker, buf: &mut [u8]) + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) -> Poll> { - T::poll_read((*self).as_mut(), waker, buf) + T::poll_read((*self).as_mut(), cx, buf) } - fn poll_vectored_read(mut self: Pin<&mut Self>, waker: &Waker, vec: &mut [&mut IoVec]) + fn poll_vectored_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, vec: &mut [&mut IoVec]) -> Poll> { - T::poll_vectored_read((*self).as_mut(), waker, vec) + T::poll_vectored_read((*self).as_mut(), cx, vec) } } @@ -296,7 +296,7 @@ mod if_std { Initializer::nop() } - fn poll_read(mut self: Pin<&mut Self>, _: &Waker, buf: &mut [u8]) + fn poll_read(mut self: Pin<&mut Self>, _: &mut Context<'_>, buf: &mut [u8]) -> Poll> { Poll::Ready(StdIo::Read::read(&mut *self, buf)) @@ -318,24 +318,24 @@ mod if_std { macro_rules! deref_async_write { () => { - fn poll_write(mut self: Pin<&mut Self>, waker: &Waker, buf: &[u8]) + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) -> Poll> { - Pin::new(&mut **self).poll_write(waker, buf) + Pin::new(&mut **self).poll_write(cx, buf) } - fn poll_vectored_write(mut self: Pin<&mut Self>, waker: &Waker, vec: &[&IoVec]) + fn poll_vectored_write(mut self: Pin<&mut Self>, cx: &mut Context<'_>, vec: &[&IoVec]) -> Poll> { - Pin::new(&mut **self).poll_vectored_write(waker, vec) + Pin::new(&mut **self).poll_vectored_write(cx, vec) } - fn poll_flush(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - Pin::new(&mut **self).poll_flush(waker) + 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>, waker: &Waker) -> Poll> { - Pin::new(&mut **self).poll_close(waker) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut **self).poll_close(cx) } } } @@ -349,41 +349,41 @@ mod if_std { } impl<'a, T: ?Sized + AsyncWrite> AsyncWrite for Pin<&'a mut T> { - fn poll_write(mut self: Pin<&mut Self>, waker: &Waker, buf: &[u8]) + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) -> Poll> { - T::poll_write((*self).as_mut(), waker, buf) + T::poll_write((*self).as_mut(), cx, buf) } - fn poll_vectored_write(mut self: Pin<&mut Self>, waker: &Waker, vec: &[&IoVec]) + fn poll_vectored_write(mut self: Pin<&mut Self>, cx: &mut Context<'_>, vec: &[&IoVec]) -> Poll> { - T::poll_vectored_write((*self).as_mut(), waker, vec) + T::poll_vectored_write((*self).as_mut(), cx, vec) } - fn poll_flush(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - T::poll_flush((*self).as_mut(), waker) + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + T::poll_flush((*self).as_mut(), cx) } - fn poll_close(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - T::poll_close((*self).as_mut(), waker) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + T::poll_close((*self).as_mut(), cx) } } macro_rules! delegate_async_write_to_stdio { () => { - fn poll_write(mut self: Pin<&mut Self>, _: &Waker, buf: &[u8]) + fn poll_write(mut self: Pin<&mut Self>, _: &mut Context<'_>, buf: &[u8]) -> Poll> { Poll::Ready(StdIo::Write::write(&mut *self, buf)) } - fn poll_flush(mut self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_flush(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(StdIo::Write::flush(&mut *self)) } - fn poll_close(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - self.poll_flush(waker) + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) } } } @@ -391,7 +391,7 @@ mod if_std { impl + Unpin> AsyncWrite for StdIo::Cursor { fn poll_write( mut self: Pin<&mut Self>, - _: &Waker, + _: &mut Context<'_>, buf: &[u8], ) -> Poll> { let position = self.position(); @@ -406,12 +406,12 @@ mod if_std { Poll::Ready(result) } - fn poll_flush(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(StdIo::Write::flush(&mut self.get_mut().get_mut().as_mut())) } - fn poll_close(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - self.poll_flush(waker) + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) } } diff --git a/futures-io/tests/cursor.rs b/futures-io/tests/cursor.rs index 69560e0895..27eafe3683 100644 --- a/futures-io/tests/cursor.rs +++ b/futures-io/tests/cursor.rs @@ -10,11 +10,11 @@ use std::pin::Pin; #[test] fn cursor_asyncwrite_asmut() { let mut cursor = Cursor::new([0; 5]); - futures::executor::block_on(lazy(|ctx| { - assert_matches!(Pin::new(&mut cursor).poll_write(ctx, &[1, 2]), Poll::Ready(Ok(2))); - assert_matches!(Pin::new(&mut cursor).poll_write(ctx, &[3, 4]), Poll::Ready(Ok(2))); - assert_matches!(Pin::new(&mut cursor).poll_write(ctx, &[5, 6]), Poll::Ready(Ok(1))); - assert_matches!(Pin::new(&mut cursor).poll_write(ctx, &[6, 7]), Poll::Ready(Ok(0))); + futures::executor::block_on(lazy(|cx| { + assert_matches!(Pin::new(&mut cursor).poll_write(cx, &[1, 2]), Poll::Ready(Ok(2))); + assert_matches!(Pin::new(&mut cursor).poll_write(cx, &[3, 4]), Poll::Ready(Ok(2))); + assert_matches!(Pin::new(&mut cursor).poll_write(cx, &[5, 6]), Poll::Ready(Ok(1))); + assert_matches!(Pin::new(&mut cursor).poll_write(cx, &[6, 7]), Poll::Ready(Ok(0))); })); assert_eq!(cursor.into_inner(), [1, 2, 3, 4, 5]); } diff --git a/futures-select-macro/src/lib.rs b/futures-select-macro/src/lib.rs index a21f8be25d..26df020211 100644 --- a/futures-select-macro/src/lib.rs +++ b/futures-select-macro/src/lib.rs @@ -180,23 +180,23 @@ pub fn select(input: TokenStream) -> TokenStream { }) .collect(); - // For each future, make an `&mut dyn FnMut(&Waker) -> Option>` + // 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)| { quote! { - let mut #variant_name = |__waker: &_| { + let mut #variant_name = |__cx: &mut #futures_crate::task::Context<'_>| { if #futures_crate::future::FusedFuture::is_terminated(&#bound_future_name) { None } else { Some(#futures_crate::future::FutureExt::poll_unpin( &mut #bound_future_name, - __waker, + __cx, ).map(#enum_ident::#variant_name)) } }; let #variant_name: &mut dyn FnMut( - &#futures_crate::task::Waker + &mut #futures_crate::task::Context<'_> ) -> Option<#futures_crate::task::Poll<_>> = &mut #variant_name; } }); @@ -235,7 +235,9 @@ pub fn select(input: TokenStream) -> TokenStream { let await_and_select = if let Some(default_expr) = parsed.default { quote! { if let #futures_crate::task::Poll::Ready(x) = - __poll_fn(#futures_crate::task::noop_waker_ref()) + __poll_fn(&mut #futures_crate::task::Context::from_waker( + #futures_crate::task::noop_waker_ref() + )) { match x { #branches } } else { @@ -254,7 +256,7 @@ pub fn select(input: TokenStream) -> TokenStream { #enum_item #( #future_let_bindings )* - let mut __poll_fn = |__waker: &#futures_crate::task::Waker| { + let mut __poll_fn = |__cx: &mut #futures_crate::task::Context<'_>| { let mut __any_polled = false; #( #poll_functions )* @@ -266,9 +268,9 @@ pub fn select(input: TokenStream) -> TokenStream { ); for poller in &mut __select_arr { let poller: &mut &mut dyn FnMut( - &#futures_crate::task::Waker + &mut #futures_crate::task::Context<'_> ) -> Option<#futures_crate::task::Poll<_>> = poller; - match poller(__waker) { + match poller(__cx) { Some(x @ #futures_crate::task::Poll::Ready(_)) => return x, Some(#futures_crate::task::Poll::Pending) => { diff --git a/futures-sink/src/channel_impls.rs b/futures-sink/src/channel_impls.rs index ad0e165db9..ccba3e2c8a 100644 --- a/futures-sink/src/channel_impls.rs +++ b/futures-sink/src/channel_impls.rs @@ -1,24 +1,24 @@ use crate::{Sink, Poll}; -use futures_core::task::Waker; +use futures_core::task::Context; use futures_channel::mpsc::{Sender, SendError, TrySendError, UnboundedSender}; use std::pin::Pin; impl Sink for Sender { type SinkError = SendError; - fn poll_ready(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - (*self).poll_ready(waker) + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + (*self).poll_ready(cx) } fn start_send(mut self: Pin<&mut Self>, msg: T) -> Result<(), Self::SinkError> { (*self).start_send(msg) } - fn poll_flush(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(mut self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { self.disconnect(); Poll::Ready(Ok(())) } @@ -27,19 +27,19 @@ impl Sink for Sender { impl Sink for UnboundedSender { type SinkError = SendError; - fn poll_ready(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - UnboundedSender::poll_ready(&*self, waker) + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + UnboundedSender::poll_ready(&*self, cx) } fn start_send(mut self: Pin<&mut Self>, msg: T) -> Result<(), Self::SinkError> { UnboundedSender::start_send(&mut *self, msg) } - fn poll_flush(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(mut self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { self.disconnect(); Poll::Ready(Ok(())) } @@ -48,8 +48,8 @@ impl Sink for UnboundedSender { impl<'a, T> Sink for &'a UnboundedSender { type SinkError = SendError; - fn poll_ready(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - UnboundedSender::poll_ready(*self, waker) + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + UnboundedSender::poll_ready(*self, cx) } fn start_send(self: Pin<&mut Self>, msg: T) -> Result<(), Self::SinkError> { @@ -57,11 +57,11 @@ impl<'a, T> Sink for &'a UnboundedSender { .map_err(TrySendError::into_send_error) } - fn poll_flush(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_close(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { self.close_channel(); Poll::Ready(Ok(())) } diff --git a/futures-sink/src/lib.rs b/futures-sink/src/lib.rs index f34ce80f7a..3bb1316b52 100644 --- a/futures-sink/src/lib.rs +++ b/futures-sink/src/lib.rs @@ -18,7 +18,7 @@ extern crate alloc; #[cfg(feature = "std")] extern crate std as alloc; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use core::pin::Pin; /// A `Sink` is a value into which other values can be sent, asynchronously. @@ -59,12 +59,12 @@ pub trait Sink { /// /// This method returns `Poll::Ready` once the underlying sink is ready to /// receive data. If this method returns `Poll::Pending`, the current task - /// is registered to be notified (via `waker.wake()`) when `poll_ready` + /// is registered to be notified (via `cx.waker().wake()`) when `poll_ready` /// should be called again. /// /// In most cases, if the sink encounters an error, the sink will /// permanently be unable to receive items. - fn poll_ready(self: Pin<&mut Self>, waker: &Waker) -> Poll>; + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; /// Begin the process of sending a value to the sink. /// Each call to this function must be proceeded by a successful call to @@ -95,12 +95,12 @@ pub trait Sink { /// via `start_send` have been flushed. /// /// Returns `Ok(Poll::Pending)` if there is more work left to do, in which - /// case the current task is scheduled (via `waker.wake()`) to wake up when + /// case the current task is scheduled (via `cx.waker().wake()`) to wake up when /// `poll_flush` should be called again. /// /// In most cases, if the sink encounters an error, the sink will /// permanently be unable to receive items. - fn poll_flush(self: Pin<&mut Self>, waker: &Waker) -> Poll>; + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; /// Flush any remaining output and close this sink, if necessary. /// @@ -108,51 +108,51 @@ pub trait Sink { /// has been successfully closed. /// /// Returns `Ok(Poll::Pending)` if there is more work left to do, in which - /// case the current task is scheduled (via `waker.wake()`) to wake up when + /// case the current task is scheduled (via `cx.waker().wake()`) to wake up when /// `poll_close` should be called again. /// /// If this function encounters an error, the sink should be considered to /// have failed permanently, and no more `Sink` methods should be called. - fn poll_close(self: Pin<&mut Self>, waker: &Waker) -> Poll>; + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; } impl<'a, S: ?Sized + Sink + Unpin, Item> Sink for &'a mut S { type SinkError = S::SinkError; - fn poll_ready(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - Pin::new(&mut **self).poll_ready(waker) + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut **self).poll_ready(cx) } fn start_send(mut self: Pin<&mut Self>, item: Item) -> Result<(), Self::SinkError> { Pin::new(&mut **self).start_send(item) } - fn poll_flush(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - Pin::new(&mut **self).poll_flush(waker) + 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>, waker: &Waker) -> Poll> { - Pin::new(&mut **self).poll_close(waker) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut **self).poll_close(cx) } } impl<'a, S: ?Sized + Sink, Item> Sink for Pin<&'a mut S> { type SinkError = S::SinkError; - fn poll_ready(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - S::poll_ready((*self).as_mut(), waker) + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + S::poll_ready((*self).as_mut(), cx) } fn start_send(mut self: Pin<&mut Self>, item: Item) -> Result<(), Self::SinkError> { S::start_send((*self).as_mut(), item) } - fn poll_flush(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - S::poll_flush((*self).as_mut(), waker) + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + S::poll_flush((*self).as_mut(), cx) } - fn poll_close(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - S::poll_close((*self).as_mut(), waker) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + S::poll_close((*self).as_mut(), cx) } } @@ -171,7 +171,7 @@ mod if_alloc { impl Sink for ::alloc::vec::Vec { type SinkError = VecSinkError; - fn poll_ready(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } @@ -181,11 +181,11 @@ mod if_alloc { Ok(()) } - fn poll_flush(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_close(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } } @@ -193,7 +193,7 @@ mod if_alloc { impl Sink for ::alloc::collections::VecDeque { type SinkError = VecSinkError; - fn poll_ready(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } @@ -203,11 +203,11 @@ mod if_alloc { Ok(()) } - fn poll_flush(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } - fn poll_close(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_close(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } } @@ -215,20 +215,20 @@ mod if_alloc { impl + Unpin, Item> Sink for ::alloc::boxed::Box { type SinkError = S::SinkError; - fn poll_ready(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - Pin::new(&mut **self).poll_ready(waker) + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut **self).poll_ready(cx) } fn start_send(mut self: Pin<&mut Self>, item: Item) -> Result<(), Self::SinkError> { Pin::new(&mut **self).start_send(item) } - fn poll_flush(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - Pin::new(&mut **self).poll_flush(waker) + 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>, waker: &Waker) -> Poll> { - Pin::new(&mut **self).poll_close(waker) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut **self).poll_close(cx) } } } @@ -245,11 +245,11 @@ impl Sink for Either { type SinkError = A::SinkError; - fn poll_ready(self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { unsafe { match Pin::get_unchecked_mut(self) { - Either::Left(x) => Pin::new_unchecked(x).poll_ready(waker), - Either::Right(x) => Pin::new_unchecked(x).poll_ready(waker), + Either::Left(x) => Pin::new_unchecked(x).poll_ready(cx), + Either::Right(x) => Pin::new_unchecked(x).poll_ready(cx), } } } @@ -263,20 +263,20 @@ impl Sink for Either } } - fn poll_flush(self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { unsafe { match Pin::get_unchecked_mut(self) { - Either::Left(x) => Pin::new_unchecked(x).poll_flush(waker), - Either::Right(x) => Pin::new_unchecked(x).poll_flush(waker), + Either::Left(x) => Pin::new_unchecked(x).poll_flush(cx), + Either::Right(x) => Pin::new_unchecked(x).poll_flush(cx), } } } - fn poll_close(self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { unsafe { match Pin::get_unchecked_mut(self) { - Either::Left(x) => Pin::new_unchecked(x).poll_close(waker), - Either::Right(x) => Pin::new_unchecked(x).poll_close(waker), + Either::Left(x) => Pin::new_unchecked(x).poll_close(cx), + Either::Right(x) => Pin::new_unchecked(x).poll_close(cx), } } } diff --git a/futures-test/src/assert.rs b/futures-test/src/assert.rs index b57610707a..25b3dcfd18 100644 --- a/futures-test/src/assert.rs +++ b/futures-test/src/assert.rs @@ -30,9 +30,9 @@ macro_rules! assert_stream_pending { let mut stream = &mut $stream; $crate::assert::assert_is_unpin_stream(stream); let stream = $crate::std_reexport::pin::Pin::new(stream); - let lw = &$crate::task::noop_waker_ref(); + let mut cx = $crate::std_reexport::task::Context::from_waker($crate::task::noop_waker_ref()); let poll = $crate::futures_core_reexport::stream::Stream::poll_next( - stream, lw, + stream, &mut cx, ); if poll.is_ready() { panic!("assertion failed: stream is not pending"); @@ -67,8 +67,8 @@ macro_rules! assert_stream_next { let mut stream = &mut $stream; $crate::assert::assert_is_unpin_stream(stream); let stream = $crate::std_reexport::pin::Pin::new(stream); - let lw = &$crate::task::noop_waker_ref(); - match $crate::futures_core_reexport::stream::Stream::poll_next(stream, lw) { + let mut cx = $crate::std_reexport::task::Context::from_waker($crate::task::noop_waker_ref()); + match $crate::futures_core_reexport::stream::Stream::poll_next(stream, &mut cx) { $crate::futures_core_reexport::task::Poll::Ready(Some(x)) => { assert_eq!(x, $item); } @@ -110,8 +110,8 @@ macro_rules! assert_stream_done { let mut stream = &mut $stream; $crate::assert::assert_is_unpin_stream(stream); let stream = $crate::std_reexport::pin::Pin::new(stream); - let lw = &$crate::task::noop_waker_ref(); - match $crate::futures_core_reexport::stream::Stream::poll_next(stream, lw) { + let mut cx = $crate::std_reexport::task::Context::from_waker($crate::task::noop_waker_ref()); + match $crate::futures_core_reexport::stream::Stream::poll_next(stream, &mut cx) { $crate::futures_core_reexport::task::Poll::Ready(Some(_)) => { panic!("assertion failed: expected stream to be done but had more elements"); } diff --git a/futures-test/src/future/assert_unmoved.rs b/futures-test/src/future/assert_unmoved.rs index a4339efae5..50778ebd70 100644 --- a/futures-test/src/future/assert_unmoved.rs +++ b/futures-test/src/future/assert_unmoved.rs @@ -1,5 +1,5 @@ use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::marker::PhantomPinned; use std::pin::Pin; @@ -35,7 +35,7 @@ impl Future for AssertUnmoved { fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { let cur_this = &*self as *const Self; if self.this_ptr.is_null() { @@ -44,7 +44,7 @@ impl Future for AssertUnmoved { } else { assert_eq!(self.this_ptr, cur_this, "Future moved between poll calls"); } - self.as_mut().future().poll(waker) + self.as_mut().future().poll(cx) } } @@ -62,7 +62,7 @@ impl Drop for AssertUnmoved { #[cfg(test)] mod tests { use futures_core::future::Future; - use futures_core::task::Poll; + use futures_core::task::{Context, Poll}; use futures_util::future::empty; use futures_util::task::noop_waker; use std::pin::Pin; @@ -81,17 +81,18 @@ mod tests { fn dont_double_panic() { // This test should only panic, not abort the process. let waker = noop_waker(); + let mut cx = Context::from_waker(&waker); // First we allocate the future on the stack and poll it. let mut future = AssertUnmoved::new(empty::<()>()); let pinned_future = unsafe { Pin::new_unchecked(&mut future) }; - assert_eq!(pinned_future.poll(&waker), Poll::Pending); + assert_eq!(pinned_future.poll(&mut cx), Poll::Pending); // Next we move it back to the heap and poll it again. This second call // should panic (as the future is moved), but we shouldn't panic again // whilst dropping `AssertUnmoved`. let mut future = Box::new(future); let pinned_boxed_future = unsafe { Pin::new_unchecked(&mut *future) }; - assert_eq!(pinned_boxed_future.poll(&waker), Poll::Pending); + assert_eq!(pinned_boxed_future.poll(&mut cx), Poll::Pending); } } diff --git a/futures-test/src/future/mod.rs b/futures-test/src/future/mod.rs index 713d745d39..ea9796fdcf 100644 --- a/futures-test/src/future/mod.rs +++ b/futures-test/src/future/mod.rs @@ -34,7 +34,7 @@ pub trait FutureTestExt: Future { /// /// ``` /// #![feature(async_await, futures_api)] - /// use futures::task::Poll; + /// use futures::task::{Context, Poll}; /// use futures::future::FutureExt; /// use futures_test::task; /// use futures_test::future::FutureTestExt; @@ -43,10 +43,10 @@ pub trait FutureTestExt: Future { /// let future = (async { 5 }).pending_once(); /// pin_mut!(future); /// - /// let lw = &task::noop_waker_ref(); + /// let mut cx = Context::from_waker(task::noop_waker_ref()); /// - /// assert_eq!(future.poll_unpin(lw), Poll::Pending); - /// assert_eq!(future.poll_unpin(lw), Poll::Ready(5)); + /// assert_eq!(future.poll_unpin(&mut cx), Poll::Pending); + /// assert_eq!(future.poll_unpin(&mut cx), Poll::Ready(5)); /// ``` fn pending_once(self) -> PendingOnce where diff --git a/futures-test/src/future/pending_once.rs b/futures-test/src/future/pending_once.rs index 238513fa43..fd3f02743e 100644 --- a/futures-test/src/future/pending_once.rs +++ b/futures-test/src/future/pending_once.rs @@ -1,5 +1,5 @@ use futures_core::future::{Future, FusedFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use std::pin::Pin; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -33,13 +33,13 @@ impl Future for PendingOnce { fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { if self.polled_before { - self.as_mut().future().poll(waker) + self.as_mut().future().poll(cx) } else { *self.as_mut().polled_before() = true; - waker.wake(); + cx.waker().wake(); Poll::Pending } } diff --git a/futures-test/src/task/panic_waker.rs b/futures-test/src/task/panic_waker.rs index 13a113fc63..c9692d9415 100644 --- a/futures-test/src/task/panic_waker.rs +++ b/futures-test/src/task/panic_waker.rs @@ -13,11 +13,11 @@ unsafe fn wake_panic(_data: *const()) { panic!("should not be woken"); } -const PANIC_WAKER_VTABLE: RawWakerVTable = RawWakerVTable { - clone: clone_panic_waker, - drop: noop, - wake: wake_panic, -}; +const PANIC_WAKER_VTABLE: RawWakerVTable = RawWakerVTable::new( + clone_panic_waker, + wake_panic, + noop, +); fn raw_panic_waker() -> RawWaker { RawWaker::new(null(), &PANIC_WAKER_VTABLE) @@ -33,8 +33,8 @@ fn raw_panic_waker() -> RawWaker { /// #![feature(futures_api)] /// use futures_test::task::panic_waker; /// -/// let lw = panic_waker(); -/// lw.wake(); // Will panic +/// let waker = panic_waker(); +/// waker.wake(); // Will panic /// ``` pub fn panic_waker() -> Waker { unsafe { Waker::new_unchecked(raw_panic_waker()) } @@ -51,8 +51,8 @@ pub fn panic_waker() -> Waker { /// use futures::task; /// use futures_test::task::panic_waker_ref; /// -/// let lw = panic_waker_ref(); -/// lw.wake(); // Will panic +/// let waker = panic_waker_ref(); +/// waker.wake(); // Will panic /// ``` pub fn panic_waker_ref() -> &'static Waker { thread_local! { diff --git a/futures-test/src/task/wake_counter.rs b/futures-test/src/task/wake_counter.rs index 3c43a0f293..e1c85812db 100644 --- a/futures-test/src/task/wake_counter.rs +++ b/futures-test/src/task/wake_counter.rs @@ -45,12 +45,12 @@ impl ArcWake for WakerInner { /// #![feature(futures_api)] /// use futures_test::task::new_count_waker; /// -/// let (lw, count) = new_count_waker(); +/// let (waker, count) = new_count_waker(); /// /// assert_eq!(count, 0); /// -/// lw.wake(); -/// lw.wake(); +/// waker.wake(); +/// waker.wake(); /// /// assert_eq!(count, 2); /// ``` diff --git a/futures-util/benches/futures_unordered.rs b/futures-util/benches/futures_unordered.rs index 80b87d0aba..9608eafa0d 100644 --- a/futures-util/benches/futures_unordered.rs +++ b/futures-util/benches/futures_unordered.rs @@ -31,9 +31,9 @@ fn oneshots(b: &mut Bencher) { } }); - block_on(future::poll_fn(move |lw| { + block_on(future::poll_fn(move |cx| { loop { - if let Poll::Ready(None) = rxs.poll_next_unpin(lw) { + if let Poll::Ready(None) = rxs.poll_next_unpin(cx) { break } } diff --git a/futures-util/benches_disabled/bilock.rs b/futures-util/benches_disabled/bilock.rs index b6e5494db9..2c97383346 100644 --- a/futures-util/benches_disabled/bilock.rs +++ b/futures-util/benches_disabled/bilock.rs @@ -2,7 +2,7 @@ #[cfg(feature = "bench")] mod bench { -use futures::task::Waker; +use futures::task::{Context, Waker}; use futures::executor::LocalPool; use futures_util::lock::BiLock; use futures_util::lock::BiLockAcquire; @@ -46,8 +46,8 @@ impl Stream for LockStream { type Item = BiLockAcquired; type Error = (); - fn poll_next(&mut self, waker: &Waker) -> Poll, Self::Error> { - self.lock.poll(waker).map(|a| match a { + fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll, Self::Error> { + self.lock.poll(cx).map(|a| match a { Poll::Ready(a) => Poll::Ready(Some(a)), Poll::Pending => Poll::Pending, }) diff --git a/futures-util/src/async_await/join.rs b/futures-util/src/async_await/join.rs index b367dff707..a8f9d33fca 100644 --- a/futures-util/src/async_await/join.rs +++ b/futures-util/src/async_await/join.rs @@ -29,11 +29,11 @@ macro_rules! join { // is no longer accessible by the end user. let mut $fut = $crate::future::maybe_done($fut); )* - await!($crate::future::poll_fn(move |lw| { + await!($crate::future::poll_fn(move |cx| { let mut all_done = true; $( if $crate::core_reexport::future::Future::poll( - unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }, lw).is_pending() + unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }, cx).is_pending() { all_done = false; } @@ -97,11 +97,11 @@ macro_rules! try_join { let mut $fut = $crate::future::maybe_done($fut); )* - let res: $crate::core_reexport::result::Result<_, _> = await!($crate::future::poll_fn(move |lw| { + let res: $crate::core_reexport::result::Result<_, _> = await!($crate::future::poll_fn(move |cx| { let mut all_done = true; $( if $crate::core_reexport::future::Future::poll( - unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }, lw).is_pending() + unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }, cx).is_pending() { all_done = false; } else if unsafe { $crate::core_reexport::pin::Pin::new_unchecked(&mut $fut) }.output_mut().unwrap().is_err() { diff --git a/futures-util/src/async_await/pending.rs b/futures-util/src/async_await/pending.rs index 10cf2572ee..0fe5056d6e 100644 --- a/futures-util/src/async_await/pending.rs +++ b/futures-util/src/async_await/pending.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// A macro which yields to the event loop once. /// @@ -30,7 +30,7 @@ pub struct PendingOnce { impl Future for PendingOnce { type Output = (); - fn poll(mut self: Pin<&mut Self>, _: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll { if self.is_ready { Poll::Ready(()) } else { diff --git a/futures-util/src/async_await/poll.rs b/futures-util/src/async_await/poll.rs index 3479f988a6..38463416d6 100644 --- a/futures-util/src/async_await/poll.rs +++ b/futures-util/src/async_await/poll.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// A macro which returns the result of polling a future once within the /// current `async` context. @@ -26,7 +26,7 @@ pub struct PollOnce { impl Future for PollOnce { type Output = Poll; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - Poll::Ready(Pin::new(&mut self.future).poll(waker)) + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + Poll::Ready(Pin::new(&mut self.future).poll(cx)) } } diff --git a/futures-util/src/compat/compat01as03.rs b/futures-util/src/compat/compat01as03.rs index a3d327ef06..5eeb231883 100644 --- a/futures-util/src/compat/compat01as03.rs +++ b/futures-util/src/compat/compat01as03.rs @@ -8,7 +8,7 @@ use futures_01::{ }; use futures_core::{task as task03, Future as Future03, Stream as Stream03}; use std::pin::Pin; -use std::task::Waker; +use std::task::Context; use futures_sink::Sink as Sink03; #[cfg(feature = "io-compat")] @@ -33,8 +33,8 @@ impl Compat01As03 { } } - fn in_notify(&mut self, waker: &Waker, f: impl FnOnce(&mut T) -> R) -> R { - let notify = &WakerToHandle(waker); + fn in_notify(&mut self, cx: &mut Context<'_>, f: impl FnOnce(&mut T) -> R) -> R { + let notify = &WakerToHandle(cx.waker()); self.inner.poll_fn_notify(notify, 0, f) } } @@ -138,9 +138,9 @@ impl Future03 for Compat01As03 { fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> task03::Poll { - poll_01_to_03(self.in_notify(waker, Future01::poll)) + poll_01_to_03(self.in_notify(cx, Future01::poll)) } } @@ -149,9 +149,9 @@ impl Stream03 for Compat01As03 { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> task03::Poll> { - match self.in_notify(waker, Stream01::poll) { + match self.in_notify(cx, Stream01::poll) { Ok(Async01::Ready(Some(t))) => task03::Poll::Ready(Some(Ok(t))), Ok(Async01::Ready(None)) => task03::Poll::Ready(None), Ok(Async01::NotReady) => task03::Poll::Pending, @@ -183,10 +183,10 @@ impl Compat01As03Sink { fn in_notify( &mut self, - waker: &Waker, + cx: &mut Context<'_>, f: impl FnOnce(&mut S) -> R, ) -> R { - let notify = &WakerToHandle(waker); + let notify = &WakerToHandle(cx.waker()); self.inner.poll_fn_notify(notify, 0, f) } } @@ -199,9 +199,9 @@ where fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> task03::Poll> { - match self.in_notify(waker, Stream01::poll) { + match self.in_notify(cx, Stream01::poll) { Ok(Async01::Ready(Some(t))) => task03::Poll::Ready(Some(Ok(t))), Ok(Async01::Ready(None)) => task03::Poll::Ready(None), Ok(Async01::NotReady) => task03::Poll::Pending, @@ -227,10 +227,10 @@ where fn poll_ready( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> task03::Poll> { match self.buffer.take() { - Some(item) => match self.in_notify(waker, |f| f.start_send(item)) { + Some(item) => match self.in_notify(cx, |f| f.start_send(item)) { Ok(AsyncSink01::Ready) => task03::Poll::Ready(Ok(())), Ok(AsyncSink01::NotReady(i)) => { self.buffer = Some(i); @@ -244,10 +244,10 @@ where fn poll_flush( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> task03::Poll> { let item = self.buffer.take(); - match self.in_notify(waker, |f| match item { + match self.in_notify(cx, |f| match item { Some(i) => match f.start_send(i) { Ok(AsyncSink01::Ready) => f.poll_complete().map(|i| (i, None)), Ok(AsyncSink01::NotReady(t)) => { @@ -268,12 +268,12 @@ where fn poll_close( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> task03::Poll> { let item = self.buffer.take(); let close_started = self.close_started; - let result = self.in_notify(waker, |f| { + let result = self.in_notify(cx, |f| { if !close_started { if let Some(item) = item { if let AsyncSink01::NotReady(item) = f.start_send(item)? { @@ -408,30 +408,30 @@ mod io { } } - fn poll_read(mut self: Pin<&mut Self>, waker: &task03::Waker, buf: &mut [u8]) + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) -> task03::Poll> { - poll_01_to_03(self.in_notify(waker, |x| x.poll_read(buf))) + 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>, waker: &task03::Waker, buf: &[u8]) + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) -> task03::Poll> { - poll_01_to_03(self.in_notify(waker, |x| x.poll_write(buf))) + poll_01_to_03(self.in_notify(cx, |x| x.poll_write(buf))) } - fn poll_flush(mut self: Pin<&mut Self>, waker: &task03::Waker) + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> task03::Poll> { - poll_01_to_03(self.in_notify(waker, AsyncWrite01::poll_flush)) + poll_01_to_03(self.in_notify(cx, AsyncWrite01::poll_flush)) } - fn poll_close(mut self: Pin<&mut Self>, waker: &task03::Waker) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> task03::Poll> { - poll_01_to_03(self.in_notify(waker, AsyncWrite01::shutdown)) + 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 60ac479af9..f1978ccdbe 100644 --- a/futures-util/src/compat/compat03as01.rs +++ b/futures-util/src/compat/compat03as01.rs @@ -19,6 +19,7 @@ use std::{ mem, pin::Pin, sync::Arc, + task::Context, }; /// Converts a futures 0.3 [`TryFuture`](futures_core::future::TryFuture), @@ -90,7 +91,7 @@ where type Error = Fut::Error; fn poll(&mut self) -> Poll01 { - with_context(self, |inner, waker| poll_03_to_01(inner.try_poll(waker))) + with_context(self, |inner, cx| poll_03_to_01(inner.try_poll(cx))) } } @@ -102,7 +103,7 @@ where type Error = St::Error; fn poll(&mut self) -> Poll01, Self::Error> { - with_context(self, |inner, waker| match inner.try_poll_next(waker) { + with_context(self, |inner, cx| match inner.try_poll_next(cx) { task03::Poll::Ready(None) => Ok(Async01::Ready(None)), task03::Poll::Ready(Some(Ok(t))) => Ok(Async01::Ready(Some(t))), task03::Poll::Pending => Ok(Async01::NotReady), @@ -122,8 +123,8 @@ where &mut self, item: Self::SinkItem, ) -> StartSend01 { - with_sink_context(self, |mut inner, waker| { - match inner.as_mut().poll_ready(waker) { + with_sink_context(self, |mut inner, cx| { + match inner.as_mut().poll_ready(cx) { task03::Poll::Ready(Ok(())) => { inner.start_send(item).map(|()| AsyncSink01::Ready) } @@ -134,11 +135,11 @@ where } fn poll_complete(&mut self) -> Poll01<(), Self::SinkError> { - with_sink_context(self, |inner, waker| poll_03_to_01(inner.poll_flush(waker))) + with_sink_context(self, |inner, cx| poll_03_to_01(inner.poll_flush(cx))) } fn close(&mut self) -> Poll01<(), Self::SinkError> { - with_sink_context(self, |inner, waker| poll_03_to_01(inner.poll_close(waker))) + with_sink_context(self, |inner, cx| poll_03_to_01(inner.poll_close(cx))) } } @@ -173,7 +174,7 @@ impl Current { } let ptr = current_to_ptr(self); - let vtable = &RawWakerVTable { clone, drop, wake }; + let vtable = &RawWakerVTable::new(clone, wake, drop); unsafe { WakerRef::new(task03::Waker::new_unchecked(RawWaker::new(ptr, vtable))) } @@ -189,21 +190,23 @@ impl ArcWake03 for Current { fn with_context(compat: &mut Compat, f: F) -> R where T: Unpin, - F: FnOnce(Pin<&mut T>, &task03::Waker) -> R, + F: FnOnce(Pin<&mut T>, &mut Context<'_>) -> R, { let current = Current::new(); let waker = current.as_waker(); - f(Pin::new(&mut compat.inner), &waker) + let mut cx = Context::from_waker(&waker); + f(Pin::new(&mut compat.inner), &mut cx) } fn with_sink_context(compat: &mut CompatSink, f: F) -> R where T: Unpin, - F: FnOnce(Pin<&mut T>, &task03::Waker) -> R, + F: FnOnce(Pin<&mut T>, &mut Context<'_>) -> R, { let current = Current::new(); let waker = current.as_waker(); - f(Pin::new(&mut compat.inner), &waker) + let mut cx = Context::from_waker(&waker); + f(Pin::new(&mut compat.inner), &mut cx) } #[cfg(feature = "io-compat")] @@ -226,7 +229,8 @@ mod io { fn read(&mut self, buf: &mut [u8]) -> std::io::Result { let current = Current::new(); let waker = current.as_waker(); - poll_03_to_io(Pin::new(&mut self.inner).poll_read(&waker, buf)) + let mut cx = Context::from_waker(&waker); + poll_03_to_io(Pin::new(&mut self.inner).poll_read(&mut cx, buf)) } } @@ -245,13 +249,15 @@ mod io { fn write(&mut self, buf: &[u8]) -> std::io::Result { let current = Current::new(); let waker = current.as_waker(); - poll_03_to_io(Pin::new(&mut self.inner).poll_write(&waker, buf)) + let mut cx = Context::from_waker(&waker); + poll_03_to_io(Pin::new(&mut self.inner).poll_write(&mut cx, buf)) } fn flush(&mut self) -> std::io::Result<()> { let current = Current::new(); let waker = current.as_waker(); - poll_03_to_io(Pin::new(&mut self.inner).poll_flush(&waker)) + let mut cx = Context::from_waker(&waker); + poll_03_to_io(Pin::new(&mut self.inner).poll_flush(&mut cx)) } } @@ -259,7 +265,8 @@ mod io { fn shutdown(&mut self) -> std::io::Result> { let current = Current::new(); let waker = current.as_waker(); - poll_03_to_01(Pin::new(&mut self.inner).poll_close(&waker)) + let mut cx = Context::from_waker(&waker); + poll_03_to_01(Pin::new(&mut self.inner).poll_close(&mut cx)) } } } diff --git a/futures-util/src/future/abortable.rs b/futures-util/src/future/abortable.rs index 98e8d6fa2e..ab176f4b6b 100644 --- a/futures-util/src/future/abortable.rs +++ b/futures-util/src/future/abortable.rs @@ -1,6 +1,6 @@ use crate::task::AtomicWaker; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; use core::pin::Pin; use core::sync::atomic::{AtomicBool, Ordering}; @@ -121,19 +121,19 @@ pub struct Aborted; impl Future for Abortable where Fut: Future { type Output = Result; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { // Check if the future has been aborted if self.inner.cancel.load(Ordering::Relaxed) { return Poll::Ready(Err(Aborted)) } // attempt to complete the future - if let Poll::Ready(x) = self.as_mut().future().poll(waker) { + if let Poll::Ready(x) = self.as_mut().future().poll(cx) { return Poll::Ready(Ok(x)) } // Register to receive a wakeup if the future is aborted in the... future - self.inner.waker.register(waker); + self.inner.waker.register(cx.waker()); // Check to see if the future was aborted between the first check and // registration. diff --git a/futures-util/src/future/catch_unwind.rs b/futures-util/src/future/catch_unwind.rs index dfdaae9683..b344ab2116 100644 --- a/futures-util/src/future/catch_unwind.rs +++ b/futures-util/src/future/catch_unwind.rs @@ -1,5 +1,5 @@ use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; use std::any::Any; use std::pin::Pin; @@ -26,8 +26,8 @@ impl Future for CatchUnwind { type Output = Result>; - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll { - match catch_unwind(AssertUnwindSafe(|| self.future().poll(waker))) { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match catch_unwind(AssertUnwindSafe(|| self.future().poll(cx))) { Ok(res) => res.map(Ok), Err(e) => Poll::Ready(Err(e)) } diff --git a/futures-util/src/future/chain.rs b/futures-util/src/future/chain.rs index efc79242b1..b9a438d453 100644 --- a/futures-util/src/future/chain.rs +++ b/futures-util/src/future/chain.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; #[must_use = "futures do nothing unless polled"] #[derive(Debug)] @@ -26,7 +26,7 @@ impl Chain pub(crate) fn poll( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, f: F, ) -> Poll where F: FnOnce(Fut1::Output, Data) -> Fut2, @@ -39,13 +39,13 @@ impl Chain loop { let (output, data) = match this { Chain::First(fut1, data) => { - match unsafe { Pin::new_unchecked(fut1) }.poll(waker) { + match unsafe { Pin::new_unchecked(fut1) }.poll(cx) { Poll::Pending => return Poll::Pending, Poll::Ready(output) => (output, data.take().unwrap()), } } Chain::Second(fut2) => { - return unsafe { Pin::new_unchecked(fut2) }.poll(waker); + return unsafe { Pin::new_unchecked(fut2) }.poll(cx); } Chain::Empty => unreachable!() }; diff --git a/futures-util/src/future/disabled/select.rs b/futures-util/src/future/disabled/select.rs index 7cda497c2d..120e17f2d3 100644 --- a/futures-util/src/future/disabled/select.rs +++ b/futures-util/src/future/disabled/select.rs @@ -18,12 +18,12 @@ impl Future for Select where A: Future, B: Future { type Item = Either<(A::Item, B), (B::Item, A)>; type Error = Either<(A::Error, B), (B::Error, A)>; - fn poll(&mut self, waker: &Waker) -> Poll { + fn poll(&mut self, cx: &mut Context<'_>) -> Poll { let (mut a, mut b) = self.inner.take().expect("cannot poll Select twice"); - match a.poll(waker) { + match a.poll(cx) { Err(e) => Err(Either::Left((e, b))), Ok(Poll::Ready(x)) => Ok(Poll::Ready(Either::Left((x, b)))), - Ok(Poll::Pending) => match b.poll(waker) { + Ok(Poll::Pending) => match b.poll(cx) { Err(e) => Err(Either::Right((e, a))), Ok(Poll::Ready(x)) => Ok(Poll::Ready(Either::Right((x, a)))), Ok(Poll::Pending) => { diff --git a/futures-util/src/future/disabled/select_all.rs b/futures-util/src/future/disabled/select_all.rs index 46e5c916b5..f808c29709 100644 --- a/futures-util/src/future/disabled/select_all.rs +++ b/futures-util/src/future/disabled/select_all.rs @@ -46,9 +46,9 @@ impl Future for SelectAll type Item = (A::Item, usize, Vec); type Error = (A::Error, usize, Vec); - fn poll(&mut self, waker: &Waker) -> Poll { + fn poll(&mut self, cx: &mut Context<'_>) -> Poll { let item = self.inner.iter_mut().enumerate().filter_map(|(i, f)| { - match f.poll(waker) { + match f.poll(cx) { Ok(Poll::Pending) => None, Ok(Poll::Ready(e)) => Some((i, Ok(e))), Err(e) => Some((i, Err(e))), diff --git a/futures-util/src/future/disabled/select_ok.rs b/futures-util/src/future/disabled/select_ok.rs index 16c01e1f78..b0df58b6ac 100644 --- a/futures-util/src/future/disabled/select_ok.rs +++ b/futures-util/src/future/disabled/select_ok.rs @@ -41,11 +41,11 @@ impl Future for SelectOk where A: Future { type Item = (A::Item, Vec); type Error = A::Error; - fn poll(&mut self, waker: &Waker) -> Poll { + fn poll(&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().filter_map(|(i, f)| { - match f.poll(waker) { + match f.poll(cx) { Ok(Poll::Pending) => None, Ok(Poll::Ready(e)) => Some((i, Ok(e))), Err(e) => Some((i, Err(e))), diff --git a/futures-util/src/future/empty.rs b/futures-util/src/future/empty.rs index df2dac51be..5cf183225c 100644 --- a/futures-util/src/future/empty.rs +++ b/futures-util/src/future/empty.rs @@ -1,7 +1,7 @@ use core::marker; use core::pin::Pin; use futures_core::future::{Future, FusedFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Future for the [`empty`] function. #[derive(Debug)] @@ -38,7 +38,7 @@ pub fn empty() -> Empty { impl Future for Empty { type Output = T; - fn poll(self: Pin<&mut Self>, _: &Waker) -> Poll { + fn poll(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll { Poll::Pending } } diff --git a/futures-util/src/future/flatten.rs b/futures-util/src/future/flatten.rs index d3c10f0ec9..5876673dc2 100644 --- a/futures-util/src/future/flatten.rs +++ b/futures-util/src/future/flatten.rs @@ -2,7 +2,7 @@ use super::chain::Chain; use core::fmt; use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`flatten`](super::FutureExt::flatten) method. @@ -51,7 +51,7 @@ impl Future for Flatten { type Output = ::Output; - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll { - self.state().poll(waker, |a, ()| a) + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.state().poll(cx, |a, ()| a) } } diff --git a/futures-util/src/future/flatten_stream.rs b/futures-util/src/future/flatten_stream.rs index 44837662b2..fe3da461bf 100644 --- a/futures-util/src/future/flatten_stream.rs +++ b/futures-util/src/future/flatten_stream.rs @@ -2,7 +2,7 @@ use core::fmt; use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Stream for the [`flatten_stream`](super::FutureExt::flatten_stream) method. #[must_use = "streams do nothing unless polled"] @@ -55,14 +55,14 @@ impl Stream for FlattenStream { type Item = ::Item; - fn poll_next(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { // safety: data is never moved via the resulting &mut reference let stream = match &mut unsafe { Pin::get_unchecked_mut(self.as_mut()) }.state { State::Future(f) => { // safety: the future we're re-pinning here will never be moved; // it will just be polled, then dropped in place - match unsafe { Pin::new_unchecked(f) }.poll(waker) { + match unsafe { Pin::new_unchecked(f) }.poll(cx) { Poll::Pending => { // State is not changed, early return. return Poll::Pending @@ -78,7 +78,7 @@ impl Stream for FlattenStream State::Stream(s) => { // safety: the stream we're repinning here will never be moved; // it will just be polled, then dropped in place - return unsafe { Pin::new_unchecked(s) }.poll_next(waker); + return unsafe { Pin::new_unchecked(s) }.poll_next(cx); } }; diff --git a/futures-util/src/future/fuse.rs b/futures-util/src/future/fuse.rs index 46988f9897..dc7319d2e7 100644 --- a/futures-util/src/future/fuse.rs +++ b/futures-util/src/future/fuse.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{Future, FusedFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`fuse`](super::FutureExt::fuse) method. @@ -29,12 +29,12 @@ impl FusedFuture for Fuse { impl Future for Fuse { type Output = Fut::Output; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { // safety: we use this &mut only for matching, not for movement let v = match self.as_mut().future().as_pin_mut() { Some(fut) => { // safety: this re-pinned future will never move before being dropped - match fut.poll(waker) { + match fut.poll(cx) { Poll::Pending => return Poll::Pending, Poll::Ready(v) => v } diff --git a/futures-util/src/future/inspect.rs b/futures-util/src/future/inspect.rs index c5cbe00dae..05e59e8c37 100644 --- a/futures-util/src/future/inspect.rs +++ b/futures-util/src/future/inspect.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`inspect`](super::FutureExt::inspect) method. @@ -35,8 +35,8 @@ impl Future for Inspect { type Output = Fut::Output; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - let e = match self.as_mut().future().poll(waker) { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let e = match self.as_mut().future().poll(cx) { Poll::Pending => return Poll::Pending, Poll::Ready(e) => e, }; diff --git a/futures-util/src/future/into_stream.rs b/futures-util/src/future/into_stream.rs index a3920d47d1..cb85a163eb 100644 --- a/futures-util/src/future/into_stream.rs +++ b/futures-util/src/future/into_stream.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Stream for the [`into_stream`](super::FutureExt::into_stream) method. @@ -24,10 +24,10 @@ impl IntoStream { impl Stream for IntoStream { type Item = Fut::Output; - fn poll_next(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let v = match self.as_mut().future().as_pin_mut() { Some(fut) => { - match fut.poll(waker) { + match fut.poll(cx) { Poll::Pending => return Poll::Pending, Poll::Ready(v) => v } diff --git a/futures-util/src/future/join.rs b/futures-util/src/future/join.rs index 008b8566ea..3327616f4d 100644 --- a/futures-util/src/future/join.rs +++ b/futures-util/src/future/join.rs @@ -4,7 +4,7 @@ use crate::future::{MaybeDone, maybe_done}; use core::fmt; use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; macro_rules! generate { @@ -48,11 +48,11 @@ macro_rules! generate { #[allow(clippy::useless_let_if_seq)] fn poll( - mut self: Pin<&mut Self>, waker: &Waker + mut self: Pin<&mut Self>, cx: &mut Context<'_> ) -> Poll { let mut all_done = true; $( - if self.as_mut().$Fut().poll(waker).is_pending() { + if self.as_mut().$Fut().poll(cx).is_pending() { all_done = false; } )* diff --git a/futures-util/src/future/join_all.rs b/futures-util/src/future/join_all.rs index 72af6bd4c2..14b3b14a6d 100644 --- a/futures-util/src/future/join_all.rs +++ b/futures-util/src/future/join_all.rs @@ -6,7 +6,7 @@ use core::future::Future; use core::iter::FromIterator; use core::mem; use core::pin::Pin; -use core::task::{Poll, Waker}; +use core::task::{Context, Poll}; use alloc::prelude::v1::*; #[derive(Debug)] @@ -124,12 +124,12 @@ where { type Output = Vec; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let mut all_done = true; for mut elem in iter_pin_mut(self.elems.as_mut()) { if let Some(pending) = elem.as_mut().pending_pin_mut() { - if let Poll::Ready(output) = pending.poll(waker) { + if let Poll::Ready(output) = pending.poll(cx) { elem.set(ElemState::Done(Some(output))); } else { all_done = false; diff --git a/futures-util/src/future/lazy.rs b/futures-util/src/future/lazy.rs index 1e76ca61e4..1e0e9660bd 100644 --- a/futures-util/src/future/lazy.rs +++ b/futures-util/src/future/lazy.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Future for the [`lazy`] function. #[derive(Debug)] @@ -33,7 +33,7 @@ impl Unpin for Lazy {} /// # }); /// ``` pub fn lazy(f: F) -> Lazy - where F: FnOnce(&Waker) -> R, + where F: FnOnce(&mut Context<'_>) -> R, { Lazy { f: Some(f) } } @@ -43,11 +43,11 @@ impl FusedFuture for Lazy { } impl Future for Lazy - where F: FnOnce(&Waker) -> R, + where F: FnOnce(&mut Context<'_>) -> R, { type Output = R; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - Poll::Ready((self.f.take().unwrap())(waker)) + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + Poll::Ready((self.f.take().unwrap())(cx)) } } diff --git a/futures-util/src/future/map.rs b/futures-util/src/future/map.rs index 66498a75d9..780d608821 100644 --- a/futures-util/src/future/map.rs +++ b/futures-util/src/future/map.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`map`](super::FutureExt::map) method. @@ -33,8 +33,8 @@ impl Future for Map { type Output = T; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - match self.as_mut().future().poll(waker) { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match self.as_mut().future().poll(cx) { Poll::Pending => Poll::Pending, Poll::Ready(output) => { let f = self.f().take() diff --git a/futures-util/src/future/maybe_done.rs b/futures-util/src/future/maybe_done.rs index dfcd38581e..71e59eac10 100644 --- a/futures-util/src/future/maybe_done.rs +++ b/futures-util/src/future/maybe_done.rs @@ -3,7 +3,7 @@ use core::mem; use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// A future that may have completed. /// @@ -92,11 +92,11 @@ impl FusedFuture for MaybeDone { impl Future for MaybeDone { type Output = (); - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let res = unsafe { match Pin::get_unchecked_mut(self.as_mut()) { MaybeDone::Future(a) => { - if let Poll::Ready(res) = Pin::new_unchecked(a).poll(waker) { + if let Poll::Ready(res) = Pin::new_unchecked(a).poll(cx) { res } else { return Poll::Pending diff --git a/futures-util/src/future/mod.rs b/futures-util/src/future/mod.rs index 1605a41d23..63ca164e06 100644 --- a/futures-util/src/future/mod.rs +++ b/futures-util/src/future/mod.rs @@ -6,7 +6,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; #[cfg(feature = "alloc")] use alloc::boxed::Box; @@ -671,10 +671,10 @@ pub trait FutureExt: Future { } /// A convenience for calling `Future::poll` on `Unpin` future types. - fn poll_unpin(&mut self, waker: &Waker) -> Poll + fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll where Self: Unpin + Sized { - Pin::new(self).poll(waker) + Pin::new(self).poll(cx) } } diff --git a/futures-util/src/future/option.rs b/futures-util/src/future/option.rs index 4c84ce35a3..3d7bc35be4 100644 --- a/futures-util/src/future/option.rs +++ b/futures-util/src/future/option.rs @@ -2,7 +2,7 @@ use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// A future representing a value which may or may not be present. @@ -38,10 +38,10 @@ impl Future for OptionFuture { fn poll( self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll { match self.option().as_pin_mut() { - Some(x) => x.poll(waker).map(Some), + Some(x) => x.poll(cx).map(Some), None => Poll::Ready(None), } } diff --git a/futures-util/src/future/poll_fn.rs b/futures-util/src/future/poll_fn.rs index 3e6adea928..501b77b995 100644 --- a/futures-util/src/future/poll_fn.rs +++ b/futures-util/src/future/poll_fn.rs @@ -2,7 +2,7 @@ use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Future for the [`poll_fn`] function. #[derive(Debug)] @@ -23,9 +23,9 @@ impl Unpin for PollFn {} /// #![feature(async_await, await_macro, futures_api)] /// # futures::executor::block_on(async { /// use futures::future::poll_fn; -/// use futures::task::{Waker, Poll}; +/// use futures::task::{Context, Poll}; /// -/// fn read_line(waker: &Waker) -> Poll { +/// fn read_line(_cx: &mut Context<'_>) -> Poll { /// Poll::Ready("Hello, World!".into()) /// } /// @@ -35,17 +35,17 @@ impl Unpin for PollFn {} /// ``` pub fn poll_fn(f: F) -> PollFn where - F: FnMut(&Waker) -> Poll + F: FnMut(&mut Context<'_>) -> Poll { PollFn { f } } impl Future for PollFn - where F: FnMut(&Waker) -> Poll, + where F: FnMut(&mut Context<'_>) -> Poll, { type Output = T; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - (&mut self.f)(waker) + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + (&mut self.f)(cx) } } diff --git a/futures-util/src/future/ready.rs b/futures-util/src/future/ready.rs index 817d584a40..761367ae93 100644 --- a/futures-util/src/future/ready.rs +++ b/futures-util/src/future/ready.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Future for the [`ready`](ready()) function. #[derive(Debug, Clone)] @@ -19,7 +19,7 @@ impl Future for Ready { type Output = T; #[inline] - fn poll(mut self: Pin<&mut Self>, _waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll { Poll::Ready(self.0.take().unwrap()) } } diff --git a/futures-util/src/future/remote_handle.rs b/futures-util/src/future/remote_handle.rs index a16f0dee2c..ad28460653 100644 --- a/futures-util/src/future/remote_handle.rs +++ b/futures-util/src/future/remote_handle.rs @@ -3,7 +3,7 @@ use { futures_channel::oneshot::{self, Sender, Receiver}, futures_core::{ future::Future, - task::{Waker, Poll}, + task::{Context, Poll}, }, pin_utils::{unsafe_pinned, unsafe_unpinned}, std::{ @@ -41,8 +41,8 @@ impl RemoteHandle { impl Future for RemoteHandle { type Output = T; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - match self.rx.poll_unpin(waker) { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match self.rx.poll_unpin(cx) { Poll::Ready(Ok(Ok(output))) => Poll::Ready(output), Poll::Ready(Ok(Err(e))) => panic::resume_unwind(e), Poll::Ready(Err(e)) => panic::resume_unwind(Box::new(e)), @@ -80,15 +80,15 @@ impl Remote { impl Future for Remote { type Output = (); - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll<()> { - if let Poll::Ready(_) = self.as_mut().tx().as_mut().unwrap().poll_cancel(waker) { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> { + if let Poll::Ready(_) = self.as_mut().tx().as_mut().unwrap().poll_cancel(cx) { if !self.keep_running.load(Ordering::SeqCst) { // Cancelled, bail out return Poll::Ready(()) } } - let output = match self.as_mut().future().poll(waker) { + let output = match self.as_mut().future().poll(cx) { Poll::Ready(output) => output, Poll::Pending => return Poll::Pending, }; diff --git a/futures-util/src/future/shared.rs b/futures-util/src/future/shared.rs index b9f1f17598..a92341e794 100644 --- a/futures-util/src/future/shared.rs +++ b/futures-util/src/future/shared.rs @@ -1,6 +1,6 @@ use crate::task::{ArcWake, waker_ref}; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll, Waker}; use slab::Slab; use std::cell::UnsafeCell; use std::fmt; @@ -108,7 +108,7 @@ where } /// Registers the current task to receive a wakeup when `Inner` is awoken. - fn set_waker(&mut self, waker: &Waker) { + fn set_waker(&mut self, cx: &mut Context<'_>) { // Acquire the lock first before checking COMPLETE to ensure there // isn't a race. let mut wakers_guard = if let Some(inner) = self.inner.as_ref() { @@ -124,7 +124,7 @@ where }; if self.waker_key == NULL_WAKER_KEY { - self.waker_key = wakers.insert(Some(waker.clone())); + self.waker_key = wakers.insert(Some(cx.waker().clone())); } else { let waker_slot = &mut wakers[self.waker_key]; let needs_replacement = if let Some(_old_waker) = waker_slot { @@ -137,7 +137,7 @@ where true }; if needs_replacement { - *waker_slot = Some(waker.clone()); + *waker_slot = Some(cx.waker().clone()); } } debug_assert!(self.waker_key != NULL_WAKER_KEY); @@ -185,10 +185,10 @@ where { type Output = Fut::Output; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = &mut *self; - this.set_waker(waker); + this.set_waker(cx); let inner = if let Some(inner) = this.inner.as_ref() { inner @@ -215,7 +215,7 @@ where } let waker = waker_ref(&inner.notifier); - let waker = &waker; + let mut cx = Context::from_waker(&waker); struct Reset<'a>(&'a AtomicUsize); @@ -239,7 +239,7 @@ where } }; - let poll = future.poll(&waker); + let poll = future.poll(&mut cx); match poll { Poll::Pending => { diff --git a/futures-util/src/future/then.rs b/futures-util/src/future/then.rs index 3d8c2cecc9..371e721684 100644 --- a/futures-util/src/future/then.rs +++ b/futures-util/src/future/then.rs @@ -1,7 +1,7 @@ use super::Chain; use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`then`](super::FutureExt::then) method. @@ -36,7 +36,7 @@ impl Future for Then { type Output = Fut2::Output; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - self.as_mut().chain().poll(waker, |output, f| f(output)) + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.as_mut().chain().poll(cx, |output, f| f(output)) } } diff --git a/futures-util/src/future/unit_error.rs b/futures-util/src/future/unit_error.rs index 12221d9251..e72d742859 100644 --- a/futures-util/src/future/unit_error.rs +++ b/futures-util/src/future/unit_error.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Future for the `unit_error` combinator, turning a `Future` into a `TryFuture`. @@ -32,7 +32,7 @@ impl Future for UnitError { type Output = Result; - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - self.future().poll(waker).map(Ok) + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.future().poll(cx).map(Ok) } } diff --git a/futures-util/src/io/allow_std.rs b/futures-util/src/io/allow_std.rs index 7967db6452..0d99bf3ce7 100644 --- a/futures-util/src/io/allow_std.rs +++ b/futures-util/src/io/allow_std.rs @@ -1,4 +1,4 @@ -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_io::{AsyncRead, AsyncWrite}; use std::{fmt, io}; use std::pin::Pin; @@ -76,19 +76,19 @@ 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>, _: &Waker, buf: &[u8]) + 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_flush(mut self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_flush(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { try_with_interrupt!(self.0.flush()); Poll::Ready(Ok(())) } - fn poll_close(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - self.poll_flush(waker) + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) } } @@ -110,7 +110,7 @@ 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>, _: &Waker, buf: &mut [u8]) + fn poll_read(mut self: Pin<&mut Self>, _: &mut Context<'_>, buf: &mut [u8]) -> Poll> { Poll::Ready(Ok(try_with_interrupt!(self.0.read(buf)))) diff --git a/futures-util/src/io/close.rs b/futures-util/src/io/close.rs index 0b039b1c9c..de0c3f76cd 100644 --- a/futures-util/src/io/close.rs +++ b/futures-util/src/io/close.rs @@ -1,5 +1,5 @@ use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_io::AsyncWrite; use std::io; use std::pin::Pin; @@ -21,7 +21,7 @@ impl<'a, W: AsyncWrite + ?Sized + Unpin> Close<'a, W> { impl Future for Close<'_, W> { type Output = io::Result<()>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - Pin::new(&mut *self.writer).poll_close(waker) + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + Pin::new(&mut *self.writer).poll_close(cx) } } diff --git a/futures-util/src/io/copy_into.rs b/futures-util/src/io/copy_into.rs index fc63362a50..c7134932e6 100644 --- a/futures-util/src/io/copy_into.rs +++ b/futures-util/src/io/copy_into.rs @@ -1,5 +1,5 @@ use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_io::{AsyncRead, AsyncWrite}; use std::boxed::Box; use std::io; @@ -39,13 +39,13 @@ impl Future for CopyInto<'_, R, W> { type Output = io::Result; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = &mut *self; loop { // If our buffer is empty, then we need to read some data to // continue. if this.pos == this.cap && !this.read_done { - let n = try_ready!(Pin::new(&mut this.reader).poll_read(waker, &mut this.buf)); + let n = try_ready!(Pin::new(&mut this.reader).poll_read(cx, &mut this.buf)); if n == 0 { this.read_done = true; } else { @@ -56,7 +56,7 @@ impl Future for CopyInto<'_, R, W> // If our buffer has some data, let's write it out! while this.pos < this.cap { - let i = try_ready!(Pin::new(&mut this.writer).poll_write(waker, &this.buf[this.pos..this.cap])); + let i = try_ready!(Pin::new(&mut this.writer).poll_write(cx, &this.buf[this.pos..this.cap])); if i == 0 { return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) } else { @@ -69,7 +69,7 @@ impl Future for CopyInto<'_, R, W> // data and finish the transfer. // done with the entire transfer. if this.pos == this.cap && this.read_done { - try_ready!(Pin::new(&mut this.writer).poll_flush(waker)); + try_ready!(Pin::new(&mut this.writer).poll_flush(cx)); return Poll::Ready(Ok(this.amt)); } } diff --git a/futures-util/src/io/disabled/lines.rs b/futures-util/src/io/disabled/lines.rs index b8e1061981..cd69f38566 100644 --- a/futures-util/src/io/disabled/lines.rs +++ b/futures-util/src/io/disabled/lines.rs @@ -44,7 +44,7 @@ impl Stream for Lines type Item = String; type Error = io::Error; - fn poll(&mut self, waker: &Waker) -> Poll, io::Error> { + fn poll(&mut self, cx: &mut Context<'_>) -> Poll, io::Error> { let n = ready!(self.io.read_line(&mut self.line)); if n == 0 && self.line.len() == 0 { return Ok(None.into()) diff --git a/futures-util/src/io/flush.rs b/futures-util/src/io/flush.rs index 4b620a9937..eee50afb52 100644 --- a/futures-util/src/io/flush.rs +++ b/futures-util/src/io/flush.rs @@ -1,5 +1,5 @@ use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_io::AsyncWrite; use std::io; use std::pin::Pin; @@ -23,7 +23,7 @@ impl Future for Flush<'_, W> { type Output = io::Result<()>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { - Pin::new(&mut *self.writer).poll_flush(waker) + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + Pin::new(&mut *self.writer).poll_flush(cx) } } diff --git a/futures-util/src/io/read.rs b/futures-util/src/io/read.rs index d833c8fbae..5287a02939 100644 --- a/futures-util/src/io/read.rs +++ b/futures-util/src/io/read.rs @@ -1,6 +1,6 @@ use crate::io::AsyncRead; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use std::io; use std::pin::Pin; @@ -22,8 +22,8 @@ impl<'a, R: AsyncRead + ?Sized + Unpin> Read<'a, R> { impl Future for Read<'_, R> { type Output = io::Result; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = &mut *self; - Pin::new(&mut this.reader).poll_read(waker, this.buf) + Pin::new(&mut this.reader).poll_read(cx, this.buf) } } diff --git a/futures-util/src/io/read_exact.rs b/futures-util/src/io/read_exact.rs index 655ca5678f..4e76e10e24 100644 --- a/futures-util/src/io/read_exact.rs +++ b/futures-util/src/io/read_exact.rs @@ -1,6 +1,6 @@ use crate::io::AsyncRead; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use std::io; use std::mem; use std::pin::Pin; @@ -23,10 +23,10 @@ impl<'a, R: AsyncRead + ?Sized + Unpin> ReadExact<'a, R> { impl Future for ReadExact<'_, R> { type Output = io::Result<()>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = &mut *self; while !this.buf.is_empty() { - let n = try_ready!(Pin::new(&mut this.reader).poll_read(waker, this.buf)); + let n = try_ready!(Pin::new(&mut this.reader).poll_read(cx, this.buf)); { let (_, rest) = mem::replace(&mut this.buf, &mut []).split_at_mut(n); this.buf = rest; diff --git a/futures-util/src/io/read_to_end.rs b/futures-util/src/io/read_to_end.rs index 68e0065d1b..df3f08bcf6 100644 --- a/futures-util/src/io/read_to_end.rs +++ b/futures-util/src/io/read_to_end.rs @@ -1,5 +1,5 @@ use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_io::AsyncRead; use std::io; use std::pin::Pin; @@ -39,7 +39,7 @@ impl Drop for Guard<'_> { // readers, we need to make sure to truncate that if any of this panics. fn read_to_end_internal( mut rd: Pin<&mut R>, - waker: &Waker, + cx: &mut Context<'_>, buf: &mut Vec, ) -> Poll> { let mut g = Guard { len: buf.len(), buf }; @@ -54,7 +54,7 @@ fn read_to_end_internal( } } - match rd.as_mut().poll_read(waker, &mut g.buf[g.len..]) { + match rd.as_mut().poll_read(cx, &mut g.buf[g.len..]) { Poll::Ready(Ok(0)) => { ret = Poll::Ready(Ok(())); break; @@ -76,8 +76,8 @@ impl Future for ReadToEnd<'_, A> { type Output = io::Result<()>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = &mut *self; - read_to_end_internal(Pin::new(&mut this.reader), waker, this.buf) + read_to_end_internal(Pin::new(&mut this.reader), cx, this.buf) } } diff --git a/futures-util/src/io/split.rs b/futures-util/src/io/split.rs index c9a125de95..333fc17983 100644 --- a/futures-util/src/io/split.rs +++ b/futures-util/src/io/split.rs @@ -1,5 +1,5 @@ use crate::lock::BiLock; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_io::{AsyncRead, AsyncWrite, IoVec}; use std::io; use std::pin::Pin; @@ -18,15 +18,15 @@ pub struct WriteHalf { fn lock_and_then( lock: &BiLock, - waker: &Waker, + cx: &mut Context<'_>, f: F ) -> Poll> - where F: FnOnce(Pin<&mut T>, &Waker) -> Poll> + where F: FnOnce(Pin<&mut T>, &mut Context<'_>) -> Poll> { - match lock.poll_lock(waker) { + match lock.poll_lock(cx) { // Safety: the value behind the bilock used by `ReadHalf` and `WriteHalf` is never exposed // as a `Pin<&mut T>` anywhere other than here as a way to get to `&mut T`. - Poll::Ready(mut l) => f(l.as_pin_mut(), waker), + Poll::Ready(mut l) => f(l.as_pin_mut(), cx), Poll::Pending => Poll::Pending, } } @@ -37,37 +37,37 @@ pub fn split(t: T) -> (ReadHalf, WriteHalf) { } impl AsyncRead for ReadHalf { - fn poll_read(self: Pin<&mut Self>, waker: &Waker, buf: &mut [u8]) + fn poll_read(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &mut [u8]) -> Poll> { - lock_and_then(&self.handle, waker, |l, waker| l.poll_read(waker, buf)) + lock_and_then(&self.handle, cx, |l, cx| l.poll_read(cx, buf)) } - fn poll_vectored_read(self: Pin<&mut Self>, waker: &Waker, vec: &mut [&mut IoVec]) + fn poll_vectored_read(self: Pin<&mut Self>, cx: &mut Context<'_>, vec: &mut [&mut IoVec]) -> Poll> { - lock_and_then(&self.handle, waker, |l, waker| l.poll_vectored_read(waker, vec)) + lock_and_then(&self.handle, cx, |l, cx| l.poll_vectored_read(cx, vec)) } } impl AsyncWrite for WriteHalf { - fn poll_write(self: Pin<&mut Self>, waker: &Waker, buf: &[u8]) + fn poll_write(self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8]) -> Poll> { - lock_and_then(&self.handle, waker, |l, waker| l.poll_write(waker, buf)) + lock_and_then(&self.handle, cx, |l, cx| l.poll_write(cx, buf)) } - fn poll_vectored_write(self: Pin<&mut Self>, waker: &Waker, vec: &[&IoVec]) + fn poll_vectored_write(self: Pin<&mut Self>, cx: &mut Context<'_>, vec: &[&IoVec]) -> Poll> { - lock_and_then(&self.handle, waker, |l, waker| l.poll_vectored_write(waker, vec)) + lock_and_then(&self.handle, cx, |l, cx| l.poll_vectored_write(cx, vec)) } - fn poll_flush(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - lock_and_then(&self.handle, waker, |l, waker| l.poll_flush(waker)) + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + lock_and_then(&self.handle, cx, |l, cx| l.poll_flush(cx)) } - fn poll_close(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - lock_and_then(&self.handle, waker, |l, waker| l.poll_close(waker)) + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + lock_and_then(&self.handle, cx, |l, cx| l.poll_close(cx)) } } diff --git a/futures-util/src/io/write_all.rs b/futures-util/src/io/write_all.rs index d64615c950..30d3ac426f 100644 --- a/futures-util/src/io/write_all.rs +++ b/futures-util/src/io/write_all.rs @@ -1,5 +1,5 @@ use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_io::AsyncWrite; use std::io; use std::mem; @@ -23,10 +23,10 @@ impl<'a, W: AsyncWrite + ?Sized + Unpin> WriteAll<'a, W> { impl Future for WriteAll<'_, W> { type Output = io::Result<()>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = &mut *self; while !this.buf.is_empty() { - let n = try_ready!(Pin::new(&mut this.writer).poll_write(waker, this.buf)); + let n = try_ready!(Pin::new(&mut this.writer).poll_write(cx, this.buf)); { let (_, rest) = mem::replace(&mut this.buf, &[]).split_at(n); this.buf = rest; diff --git a/futures-util/src/lib.rs b/futures-util/src/lib.rs index afd77f668d..129b1dd75d 100644 --- a/futures-util/src/lib.rs +++ b/futures-util/src/lib.rs @@ -49,9 +49,9 @@ macro_rules! delegate_sink { ($field:ident, $item:ty) => { fn poll_ready( self: Pin<&mut Self>, - waker: &$crate::core_reexport::task::Waker, + cx: &mut $crate::core_reexport::task::Context<'_>, ) -> $crate::core_reexport::task::Poll> { - self.$field().poll_ready(waker) + self.$field().poll_ready(cx) } fn start_send( @@ -63,16 +63,16 @@ macro_rules! delegate_sink { fn poll_flush( self: Pin<&mut Self>, - waker: &$crate::core_reexport::task::Waker + cx: &mut $crate::core_reexport::task::Context<'_>, ) -> $crate::core_reexport::task::Poll> { - self.$field().poll_flush(waker) + self.$field().poll_flush(cx) } fn poll_close( self: Pin<&mut Self>, - waker: &$crate::core_reexport::task::Waker + cx: &mut $crate::core_reexport::task::Context<'_>, ) -> $crate::core_reexport::task::Poll> { - self.$field().poll_close(waker) + self.$field().poll_close(cx) } } } diff --git a/futures-util/src/lock/bilock.rs b/futures-util/src/lock/bilock.rs index 0b4cf473ed..71550498f3 100644 --- a/futures-util/src/lock/bilock.rs +++ b/futures-util/src/lock/bilock.rs @@ -2,7 +2,7 @@ #![allow(unused)] use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll, Waker}; use core::cell::UnsafeCell; use core::fmt; use core::mem; @@ -88,7 +88,7 @@ impl BiLock { /// /// This function will panic if called outside the context of a future's /// task. - pub fn poll_lock(&self, waker: &Waker) -> Poll> { + pub fn poll_lock(&self, cx: &mut Context<'_>) -> Poll> { loop { match self.arc.state.swap(1, SeqCst) { // Woohoo, we grabbed the lock! @@ -105,7 +105,7 @@ impl BiLock { } // type ascription for safety's sake! - let me: Box = Box::new(waker.clone()); + let me: Box = Box::new(cx.waker().clone()); let me = Box::into_raw(me) as usize; match self.arc.state.compare_exchange(1, me, SeqCst, SeqCst) { @@ -271,7 +271,7 @@ impl<'a, T> Unpin for BiLockAcquire<'a, T> {} impl<'a, T> Future for BiLockAcquire<'a, T> { type Output = BiLockGuard<'a, T>; - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll { - self.bilock.poll_lock(waker) + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.bilock.poll_lock(cx) } } diff --git a/futures-util/src/lock/mutex.rs b/futures-util/src/lock/mutex.rs index 0767f0fbf3..5f03b8308b 100644 --- a/futures-util/src/lock/mutex.rs +++ b/futures-util/src/lock/mutex.rs @@ -1,5 +1,5 @@ use futures_core::future::{FusedFuture, Future}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll, Waker}; use slab::Slab; use std::{fmt, mem, usize}; use std::cell::UnsafeCell; @@ -147,7 +147,7 @@ impl FusedFuture for MutexLockFuture<'_, T> { impl<'a, T> Future for MutexLockFuture<'a, T> { type Output = MutexGuard<'a, T>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let mutex = self.mutex.expect("polled MutexLockFuture after completion"); if let Some(lock) = mutex.try_lock() { @@ -159,12 +159,12 @@ impl<'a, T> Future for MutexLockFuture<'a, T> { { let mut waiters = mutex.waiters.lock().unwrap(); if self.wait_key == WAIT_KEY_NONE { - self.wait_key = waiters.insert(Waiter::Waiting(waker.clone())); + self.wait_key = waiters.insert(Waiter::Waiting(cx.waker().clone())); if waiters.len() == 1 { mutex.state.fetch_or(HAS_WAITERS, Ordering::Relaxed); // released by mutex unlock } } else { - waiters[self.wait_key].register(waker); + waiters[self.wait_key].register(cx.waker()); } } diff --git a/futures-util/src/sink/buffer.rs b/futures-util/src/sink/buffer.rs index c920030905..6f926d95b9 100644 --- a/futures-util/src/sink/buffer.rs +++ b/futures-util/src/sink/buffer.rs @@ -1,5 +1,5 @@ use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use core::pin::Pin; @@ -23,7 +23,6 @@ impl, Item> Buffer { unsafe_unpinned!(buf: VecDeque); unsafe_unpinned!(capacity: usize); - pub(super) fn new(sink: Si, capacity: usize) -> Self { Buffer { sink, @@ -39,15 +38,15 @@ impl, Item> Buffer { fn try_empty_buffer( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { - try_ready!(self.as_mut().sink().poll_ready(waker)); + try_ready!(self.as_mut().sink().poll_ready(cx)); while let Some(item) = self.as_mut().buf().pop_front() { if let Err(e) = self.as_mut().sink().start_send(item) { return Poll::Ready(Err(e)); } if !self.buf.is_empty() { - try_ready!(self.as_mut().sink().poll_ready(waker)); + try_ready!(self.as_mut().sink().poll_ready(cx)); } } Poll::Ready(Ok(())) @@ -58,8 +57,8 @@ impl, Item> Buffer { impl Stream for Buffer where S: Sink + Stream { type Item = S::Item; - fn poll_next(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - self.sink().poll_next(waker) + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.sink().poll_next(cx) } } @@ -68,13 +67,13 @@ impl, Item> Sink for Buffer { fn poll_ready( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { if self.capacity == 0 { - return self.as_mut().sink().poll_ready(waker); + return self.as_mut().sink().poll_ready(cx); } - if let Poll::Ready(Err(e)) = self.as_mut().try_empty_buffer(waker) { + if let Poll::Ready(Err(e)) = self.as_mut().try_empty_buffer(cx) { return Poll::Ready(Err(e)); } @@ -99,19 +98,19 @@ impl, Item> Sink for Buffer { fn poll_flush( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - try_ready!(self.as_mut().try_empty_buffer(waker)); + try_ready!(self.as_mut().try_empty_buffer(cx)); debug_assert!(self.as_mut().buf().is_empty()); - self.as_mut().sink().poll_flush(waker) + self.as_mut().sink().poll_flush(cx) } fn poll_close( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - try_ready!(self.as_mut().try_empty_buffer(waker)); + try_ready!(self.as_mut().try_empty_buffer(cx)); debug_assert!(self.as_mut().buf().is_empty()); - self.as_mut().sink().poll_close(waker) + self.as_mut().sink().poll_close(cx) } } diff --git a/futures-util/src/sink/close.rs b/futures-util/src/sink/close.rs index d655d6d7c4..98ba320e57 100644 --- a/futures-util/src/sink/close.rs +++ b/futures-util/src/sink/close.rs @@ -1,7 +1,7 @@ use core::marker::PhantomData; use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; /// Future for the [`close`](super::SinkExt::close) method. @@ -29,8 +29,8 @@ impl + Unpin + ?Sized, Item> Future for Close<'_, Si, Item> { fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - Pin::new(&mut self.sink).poll_close(waker) + 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 1cde4001f7..2e5d505dd7 100644 --- a/futures-util/src/sink/drain.rs +++ b/futures-util/src/sink/drain.rs @@ -1,7 +1,7 @@ use core::fmt; use core::marker::PhantomData; use core::pin::Pin; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; /// Sink for the [`drain`] function. @@ -40,7 +40,7 @@ impl Sink for Drain { fn poll_ready( self: Pin<&mut Self>, - _waker: &Waker, + _cx: &mut Context<'_>, ) -> Poll> { Poll::Ready(Ok(())) } @@ -54,14 +54,14 @@ impl Sink for Drain { fn poll_flush( self: Pin<&mut Self>, - _waker: &Waker, + _cx: &mut Context<'_>, ) -> Poll> { Poll::Ready(Ok(())) } fn poll_close( self: Pin<&mut Self>, - _waker: &Waker, + _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 318bed209c..12057a6f63 100644 --- a/futures-util/src/sink/err_into.rs +++ b/futures-util/src/sink/err_into.rs @@ -1,7 +1,7 @@ use crate::sink::{SinkExt, SinkMapErr}; use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::{Sink}; use pin_utils::unsafe_pinned; @@ -60,8 +60,8 @@ impl Stream for SinkErrInto fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.sink().poll_next(waker) + self.sink().poll_next(cx) } } diff --git a/futures-util/src/sink/fanout.rs b/futures-util/src/sink/fanout.rs index 8ecc3c4a50..864dcfd36d 100644 --- a/futures-util/src/sink/fanout.rs +++ b/futures-util/src/sink/fanout.rs @@ -1,6 +1,6 @@ use core::fmt::{Debug, Formatter, Result as FmtResult}; use core::pin::Pin; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::unsafe_pinned; @@ -48,10 +48,10 @@ impl Sink for Fanout fn poll_ready( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - let sink1_ready = self.as_mut().sink1().poll_ready(waker)?.is_ready(); - let sink2_ready = self.as_mut().sink2().poll_ready(waker)?.is_ready(); + let sink1_ready = self.as_mut().sink1().poll_ready(cx)?.is_ready(); + let sink2_ready = self.as_mut().sink2().poll_ready(cx)?.is_ready(); let ready = sink1_ready && sink2_ready; if ready { Poll::Ready(Ok(())) } else { Poll::Pending } } @@ -67,20 +67,20 @@ impl Sink for Fanout fn poll_flush( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - let sink1_ready = self.as_mut().sink1().poll_flush(waker)?.is_ready(); - let sink2_ready = self.as_mut().sink2().poll_flush(waker)?.is_ready(); + let sink1_ready = self.as_mut().sink1().poll_flush(cx)?.is_ready(); + let sink2_ready = self.as_mut().sink2().poll_flush(cx)?.is_ready(); let ready = sink1_ready && sink2_ready; if ready { Poll::Ready(Ok(())) } else { Poll::Pending } } fn poll_close( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - let sink1_ready = self.as_mut().sink1().poll_close(waker)?.is_ready(); - let sink2_ready = self.as_mut().sink2().poll_close(waker)?.is_ready(); + let sink1_ready = self.as_mut().sink1().poll_close(cx)?.is_ready(); + let sink2_ready = self.as_mut().sink2().poll_close(cx)?.is_ready(); let ready = sink1_ready && sink2_ready; if ready { Poll::Ready(Ok(())) } else { Poll::Pending } } diff --git a/futures-util/src/sink/flush.rs b/futures-util/src/sink/flush.rs index 9aef125f93..90b96a90c0 100644 --- a/futures-util/src/sink/flush.rs +++ b/futures-util/src/sink/flush.rs @@ -1,7 +1,7 @@ use core::marker::PhantomData; use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; /// Future for the [`flush`](super::SinkExt::flush) method. @@ -35,8 +35,8 @@ impl + Unpin + ?Sized, Item> Future for Flush<'_, Si, Item> { fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - Pin::new(&mut self.sink).poll_flush(waker) + 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 bfb88a7af5..c4f911dbb0 100644 --- a/futures-util/src/sink/map_err.rs +++ b/futures-util/src/sink/map_err.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::{Sink}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -59,10 +59,10 @@ impl Sink for SinkMapErr fn poll_ready( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 - self.as_mut().sink().poll_ready(waker).map_err(|e| self.as_mut().take_f()(e)) + self.as_mut().sink().poll_ready(cx).map_err(|e| self.as_mut().take_f()(e)) } fn start_send( @@ -75,18 +75,18 @@ impl Sink for SinkMapErr fn poll_flush( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 - self.as_mut().sink().poll_flush(waker).map_err(|e| self.as_mut().take_f()(e)) + self.as_mut().sink().poll_flush(cx).map_err(|e| self.as_mut().take_f()(e)) } fn poll_close( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 - self.as_mut().sink().poll_close(waker).map_err(|e| self.as_mut().take_f()(e)) + self.as_mut().sink().poll_close(cx).map_err(|e| self.as_mut().take_f()(e)) } } @@ -95,8 +95,8 @@ impl Stream for SinkMapErr { fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.sink().poll_next(waker) + self.sink().poll_next(cx) } } diff --git a/futures-util/src/sink/send.rs b/futures-util/src/sink/send.rs index 7e55f6379e..3d6135f82a 100644 --- a/futures-util/src/sink/send.rs +++ b/futures-util/src/sink/send.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::Future; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; /// Future for the [`send`](super::SinkExt::send) method. @@ -28,12 +28,12 @@ impl + Unpin + ?Sized, Item> Future for Send<'_, Si, Item> { fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { let this = &mut *self; if let Some(item) = this.item.take() { let mut sink = Pin::new(&mut this.sink); - match sink.as_mut().poll_ready(waker) { + match sink.as_mut().poll_ready(cx) { Poll::Ready(Ok(())) => { if let Err(e) = sink.as_mut().start_send(item) { return Poll::Ready(Err(e)); @@ -49,7 +49,7 @@ impl + Unpin + ?Sized, Item> Future for Send<'_, Si, Item> { // we're done sending the item, but want to block on flushing the // sink - try_ready!(Pin::new(&mut this.sink).poll_flush(waker)); + try_ready!(Pin::new(&mut this.sink).poll_flush(cx)); Poll::Ready(Ok(())) } diff --git a/futures-util/src/sink/send_all.rs b/futures-util/src/sink/send_all.rs index 43c5a525c0..582a653fed 100644 --- a/futures-util/src/sink/send_all.rs +++ b/futures-util/src/sink/send_all.rs @@ -2,7 +2,7 @@ use crate::stream::{StreamExt, Fuse}; use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; /// Future for the [`send_all`](super::SinkExt::send_all) method. @@ -43,11 +43,11 @@ where fn try_start_send( &mut self, - waker: &Waker, + cx: &mut Context<'_>, item: St::Item, ) -> Poll> { debug_assert!(self.buffered.is_none()); - match Pin::new(&mut self.sink).poll_ready(waker) { + match Pin::new(&mut self.sink).poll_ready(cx) { Poll::Ready(Ok(())) => { Poll::Ready(Pin::new(&mut self.sink).start_send(item)) } @@ -69,26 +69,26 @@ where fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + 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 if let Some(item) = this.buffered.take() { - try_ready!(this.try_start_send(waker, item)) + try_ready!(this.try_start_send(cx, item)) } loop { - match this.stream.poll_next_unpin(waker) { + match this.stream.poll_next_unpin(cx) { Poll::Ready(Some(item)) => { - try_ready!(this.try_start_send(waker, item)) + try_ready!(this.try_start_send(cx, item)) } Poll::Ready(None) => { - try_ready!(Pin::new(&mut this.sink).poll_flush(waker)); + try_ready!(Pin::new(&mut this.sink).poll_flush(cx)); return Poll::Ready(Ok(())) } Poll::Pending => { - try_ready!(Pin::new(&mut this.sink).poll_flush(waker)); + try_ready!(Pin::new(&mut this.sink).poll_flush(cx)); return Poll::Pending } } diff --git a/futures-util/src/sink/with.rs b/futures-util/src/sink/with.rs index 50286338d7..120d920159 100644 --- a/futures-util/src/sink/with.rs +++ b/futures-util/src/sink/with.rs @@ -3,7 +3,7 @@ use core::mem; use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -86,9 +86,9 @@ impl Stream for With fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.sink().poll_next(waker) + self.sink().poll_next(cx) } } @@ -118,11 +118,11 @@ impl With fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { let buffered = match self.as_mut().state().as_pin_mut() { State::Empty => return Poll::Ready(Ok(())), - State::Process(fut) => Some(try_ready!(fut.poll(waker))), + State::Process(fut) => Some(try_ready!(fut.poll(cx))), State::Buffered(_) => None, }; if let Some(buffered) = buffered { @@ -146,9 +146,9 @@ impl Sink for With fn poll_ready( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.poll(waker) + self.poll(cx) } fn start_send( @@ -162,19 +162,19 @@ impl Sink for With fn poll_flush( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - try_ready!(self.as_mut().poll(waker)); - try_ready!(self.as_mut().sink().poll_flush(waker)); + try_ready!(self.as_mut().poll(cx)); + try_ready!(self.as_mut().sink().poll_flush(cx)); Poll::Ready(Ok(())) } fn poll_close( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - try_ready!(self.as_mut().poll(waker)); - try_ready!(self.as_mut().sink().poll_close(waker)); + try_ready!(self.as_mut().poll(cx)); + try_ready!(self.as_mut().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 31128cfdbf..84fe0461f4 100644 --- a/futures-util/src/sink/with_flat_map.rs +++ b/futures-util/src/sink/with_flat_map.rs @@ -1,7 +1,7 @@ use core::marker::PhantomData; use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -74,7 +74,7 @@ where fn try_empty_stream( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { let WithFlatMap { sink, stream, buffer, .. } = unsafe { Pin::get_unchecked_mut(self) }; @@ -82,14 +82,14 @@ where let mut stream = unsafe { Pin::new_unchecked(stream) }; if buffer.is_some() { - try_ready!(sink.as_mut().poll_ready(waker)); + try_ready!(sink.as_mut().poll_ready(cx)); let item = buffer.take().unwrap(); try_ready!(Poll::Ready(sink.as_mut().start_send(item))); } if let Some(mut some_stream) = stream.as_mut().as_pin_mut() { - while let Some(x) = ready!(some_stream.as_mut().poll_next(waker)) { + while let Some(x) = ready!(some_stream.as_mut().poll_next(cx)) { let item = try_ready!(Poll::Ready(x)); - match sink.as_mut().poll_ready(waker)? { + match sink.as_mut().poll_ready(cx)? { Poll::Ready(()) => sink.as_mut().start_send(item)?, Poll::Pending => { *buffer = Some(item); @@ -112,9 +112,9 @@ where type Item = S::Item; fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.sink().poll_next(waker) + self.sink().poll_next(cx) } } @@ -128,9 +128,9 @@ where fn poll_ready( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.try_empty_stream(waker) + self.try_empty_stream(cx) } fn start_send( @@ -145,22 +145,22 @@ where fn poll_flush( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - match self.as_mut().try_empty_stream(waker) { + match self.as_mut().try_empty_stream(cx) { Poll::Pending => Poll::Pending, - Poll::Ready(Ok(())) => self.as_mut().sink().poll_flush(waker), + Poll::Ready(Ok(())) => self.as_mut().sink().poll_flush(cx), Poll::Ready(Err(e)) => Poll::Ready(Err(e)), } } fn poll_close( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - match self.as_mut().try_empty_stream(waker) { + match self.as_mut().try_empty_stream(cx) { Poll::Pending => Poll::Pending, - Poll::Ready(Ok(())) => self.as_mut().sink().poll_close(waker), + Poll::Ready(Ok(())) => self.as_mut().sink().poll_close(cx), Poll::Ready(Err(e)) => Poll::Ready(Err(e)), } } diff --git a/futures-util/src/stream/buffer_unordered.rs b/futures-util/src/stream/buffer_unordered.rs index 1facf7792c..e752925ce5 100644 --- a/futures-util/src/stream/buffer_unordered.rs +++ b/futures-util/src/stream/buffer_unordered.rs @@ -1,7 +1,7 @@ use crate::stream::{Fuse, FuturesUnordered}; use futures_core::future::Future; use futures_core::stream::{Stream, FusedStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use core::fmt; @@ -103,19 +103,19 @@ where fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { // First up, try to spawn off as many futures as possible by filling up // our slab of futures. while self.in_progress_queue.len() < self.max { - match self.as_mut().stream().poll_next(waker) { + match self.as_mut().stream().poll_next(cx) { Poll::Ready(Some(fut)) => self.as_mut().in_progress_queue().push(fut), Poll::Ready(None) | Poll::Pending => break, } } // Attempt to pull the next value from the in_progress_queue - match Pin::new(self.as_mut().in_progress_queue()).poll_next(waker) { + match Pin::new(self.as_mut().in_progress_queue()).poll_next(cx) { x @ Poll::Pending | x @ Poll::Ready(Some(_)) => return x, Poll::Ready(None) => {} } diff --git a/futures-util/src/stream/buffered.rs b/futures-util/src/stream/buffered.rs index b409be2af2..dbc54a482b 100644 --- a/futures-util/src/stream/buffered.rs +++ b/futures-util/src/stream/buffered.rs @@ -1,7 +1,7 @@ use crate::stream::{Fuse, FuturesOrdered}; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use core::fmt; @@ -98,19 +98,19 @@ where fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { // Try to spawn off as many futures as possible by filling up // our in_progress_queue of futures. while self.in_progress_queue.len() < self.max { - match self.as_mut().stream().poll_next(waker) { + match self.as_mut().stream().poll_next(cx) { Poll::Ready(Some(fut)) => self.as_mut().in_progress_queue().push(fut), Poll::Ready(None) | Poll::Pending => break, } } // Attempt to pull the next value from the in_progress_queue - let res = Pin::new(self.as_mut().in_progress_queue()).poll_next(waker); + let res = Pin::new(self.as_mut().in_progress_queue()).poll_next(cx); if let Some(val) = ready!(res) { return Poll::Ready(Some(val)) } diff --git a/futures-util/src/stream/catch_unwind.rs b/futures-util/src/stream/catch_unwind.rs index bcd5d8551f..6fd864b1d5 100644 --- a/futures-util/src/stream/catch_unwind.rs +++ b/futures-util/src/stream/catch_unwind.rs @@ -1,5 +1,5 @@ use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use std::any::Any; use std::pin::Pin; @@ -29,13 +29,13 @@ impl Stream for CatchUnwind fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { if *self.as_mut().caught_unwind() { Poll::Ready(None) } else { let res = catch_unwind(AssertUnwindSafe(|| { - self.as_mut().stream().poll_next(waker) + self.as_mut().stream().poll_next(cx) })); match res { diff --git a/futures-util/src/stream/chain.rs b/futures-util/src/stream/chain.rs index 73f1db280d..084cd5a62b 100644 --- a/futures-util/src/stream/chain.rs +++ b/futures-util/src/stream/chain.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Stream for the [`chain`](super::StreamExt::chain) method. @@ -41,14 +41,14 @@ where St1: Stream, fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { if let Some(first) = self.as_mut().first().as_pin_mut() { - if let Some(item) = ready!(first.poll_next(waker)) { + if let Some(item) = ready!(first.poll_next(cx)) { return Poll::Ready(Some(item)) } } self.as_mut().first().set(None); - self.as_mut().second().poll_next(waker) + self.as_mut().second().poll_next(cx) } } diff --git a/futures-util/src/stream/chunks.rs b/futures-util/src/stream/chunks.rs index 199e4a95d4..234c24bd3e 100644 --- a/futures-util/src/stream/chunks.rs +++ b/futures-util/src/stream/chunks.rs @@ -1,6 +1,6 @@ use crate::stream::Fuse; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use core::mem; @@ -66,10 +66,10 @@ impl Stream for Chunks { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { loop { - match ready!(self.as_mut().stream().poll_next(waker)) { + match ready!(self.as_mut().stream().poll_next(cx)) { // Push the item into the buffer and check whether it is full. // If so, replace our buffer with a new and empty one and return // the full one. diff --git a/futures-util/src/stream/collect.rs b/futures-util/src/stream/collect.rs index 7ca29b190c..2bbfdc184d 100644 --- a/futures-util/src/stream/collect.rs +++ b/futures-util/src/stream/collect.rs @@ -2,7 +2,7 @@ use core::mem; use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`collect`](super::StreamExt::collect) method. @@ -43,9 +43,9 @@ where St: Stream, { type Output = C; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { - match ready!(self.as_mut().stream().poll_next(waker)) { + match ready!(self.as_mut().stream().poll_next(cx)) { Some(e) => self.as_mut().collection().extend(Some(e)), None => return Poll::Ready(self.as_mut().finish()), } diff --git a/futures-util/src/stream/concat.rs b/futures-util/src/stream/concat.rs index 72cdfa2eb8..16b3f51d5c 100644 --- a/futures-util/src/stream/concat.rs +++ b/futures-util/src/stream/concat.rs @@ -3,7 +3,7 @@ use core::pin::Pin; use core::default::Default; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`concat`](super::StreamExt::concat) method. @@ -50,10 +50,10 @@ where St: Stream, type Output = St::Item; fn poll( - mut self: Pin<&mut Self>, waker: &Waker + mut self: Pin<&mut Self>, cx: &mut Context<'_> ) -> Poll { loop { - match self.as_mut().stream().poll_next(waker) { + match self.as_mut().stream().poll_next(cx) { Poll::Pending => return Poll::Pending, Poll::Ready(None) => { return Poll::Ready(self.as_mut().accum().take().unwrap_or_default()) diff --git a/futures-util/src/stream/empty.rs b/futures-util/src/stream/empty.rs index 5401d77ad2..6cc28baa13 100644 --- a/futures-util/src/stream/empty.rs +++ b/futures-util/src/stream/empty.rs @@ -1,7 +1,7 @@ use core::marker::PhantomData; use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Stream for the [`empty`] function. #[derive(Debug)] @@ -24,7 +24,7 @@ impl Unpin for Empty {} impl Stream for Empty { type Item = T; - fn poll_next(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(None) } } diff --git a/futures-util/src/stream/filter.rs b/futures-util/src/stream/filter.rs index b6cb9a7286..b37edaac63 100644 --- a/futures-util/src/stream/filter.rs +++ b/futures-util/src/stream/filter.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -87,11 +87,11 @@ impl Stream for Filter fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { loop { if self.as_mut().pending_fut().as_pin_mut().is_none() { - let item = match ready!(self.as_mut().stream().poll_next(waker)) { + let item = match ready!(self.as_mut().stream().poll_next(cx)) { Some(e) => e, None => return Poll::Ready(None), }; @@ -100,7 +100,7 @@ impl Stream for Filter *self.as_mut().pending_item() = Some(item); } - let yield_item = ready!(self.as_mut().pending_fut().as_pin_mut().unwrap().poll(waker)); + let yield_item = ready!(self.as_mut().pending_fut().as_pin_mut().unwrap().poll(cx)); self.as_mut().pending_fut().set(None); let item = self.as_mut().pending_item().take().unwrap(); diff --git a/futures-util/src/stream/filter_map.rs b/futures-util/src/stream/filter_map.rs index 84a0a5c925..6275df0e8f 100644 --- a/futures-util/src/stream/filter_map.rs +++ b/futures-util/src/stream/filter_map.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -80,11 +80,11 @@ impl Stream for FilterMap fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { loop { if self.as_mut().pending().as_pin_mut().is_none() { - let item = match ready!(self.as_mut().stream().poll_next(waker)) { + let item = match ready!(self.as_mut().stream().poll_next(cx)) { Some(e) => e, None => return Poll::Ready(None), }; @@ -92,7 +92,7 @@ impl Stream for FilterMap self.as_mut().pending().set(Some(fut)); } - let item = ready!(self.as_mut().pending().as_pin_mut().unwrap().poll(waker)); + let item = ready!(self.as_mut().pending().as_pin_mut().unwrap().poll(cx)); self.as_mut().pending().set(None); if item.is_some() { return Poll::Ready(item); diff --git a/futures-util/src/stream/flatten.rs b/futures-util/src/stream/flatten.rs index 3bfd3f56d8..608641582d 100644 --- a/futures-util/src/stream/flatten.rs +++ b/futures-util/src/stream/flatten.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::unsafe_pinned; @@ -68,16 +68,16 @@ impl Stream for Flatten fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { loop { if self.as_mut().next().as_pin_mut().is_none() { - match ready!(self.as_mut().stream().poll_next(waker)) { + match ready!(self.as_mut().stream().poll_next(cx)) { Some(e) => self.as_mut().next().set(Some(e)), None => return Poll::Ready(None), } } - let item = ready!(self.as_mut().next().as_pin_mut().unwrap().poll_next(waker)); + let item = ready!(self.as_mut().next().as_pin_mut().unwrap().poll_next(cx)); if item.is_some() { return Poll::Ready(item); } else { diff --git a/futures-util/src/stream/fold.rs b/futures-util/src/stream/fold.rs index 16e3b5db0c..aeb2eb1a33 100644 --- a/futures-util/src/stream/fold.rs +++ b/futures-util/src/stream/fold.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`fold`](super::StreamExt::fold) method. @@ -49,16 +49,16 @@ impl Future for Fold { type Output = T; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { // we're currently processing a future to produce a new accum value if self.as_mut().accum().is_none() { - let accum = ready!(self.as_mut().future().as_pin_mut().unwrap().poll(waker)); + let accum = ready!(self.as_mut().future().as_pin_mut().unwrap().poll(cx)); *self.as_mut().accum() = Some(accum); self.as_mut().future().set(None); } - let item = ready!(self.as_mut().stream().poll_next(waker)); + let item = ready!(self.as_mut().stream().poll_next(cx)); let accum = self.as_mut().accum().take() .expect("Fold polled after completion"); diff --git a/futures-util/src/stream/for_each.rs b/futures-util/src/stream/for_each.rs index ce60650a94..a2ce71bfc4 100644 --- a/futures-util/src/stream/for_each.rs +++ b/futures-util/src/stream/for_each.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`for_each`](super::StreamExt::for_each) method. @@ -50,14 +50,14 @@ impl Future for ForEach { type Output = (); - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll<()> { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> { loop { if let Some(future) = self.as_mut().future().as_pin_mut() { - ready!(future.poll(waker)); + ready!(future.poll(cx)); } self.as_mut().future().as_mut().set(None); - match ready!(self.as_mut().stream().poll_next(waker)) { + match ready!(self.as_mut().stream().poll_next(cx)) { Some(e) => { let future = (self.as_mut().f())(e); self.as_mut().future().set(Some(future)); diff --git a/futures-util/src/stream/for_each_concurrent.rs b/futures-util/src/stream/for_each_concurrent.rs index edac28b915..8ce10e91b8 100644 --- a/futures-util/src/stream/for_each_concurrent.rs +++ b/futures-util/src/stream/for_each_concurrent.rs @@ -3,7 +3,7 @@ use core::pin::Pin; use core::num::NonZeroUsize; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`for_each_concurrent`](super::StreamExt::for_each_concurrent) @@ -56,7 +56,7 @@ impl Future for ForEachConcurrent { type Output = (); - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll<()> { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> { loop { let mut made_progress_this_iter = false; @@ -66,7 +66,7 @@ impl Future for ForEachConcurrent if self.limit.map(|limit| limit.get() > current_len).unwrap_or(true) { let mut stream_completed = false; let elem = if let Some(stream) = self.as_mut().stream().as_pin_mut() { - match stream.poll_next(waker) { + match stream.poll_next(cx) { Poll::Ready(Some(elem)) => { made_progress_this_iter = true; Some(elem) @@ -89,7 +89,7 @@ impl Future for ForEachConcurrent } } - match self.as_mut().futures().poll_next_unpin(waker) { + match self.as_mut().futures().poll_next_unpin(cx) { Poll::Ready(Some(())) => made_progress_this_iter = true, Poll::Ready(None) => { if self.as_mut().stream().is_none() { diff --git a/futures-util/src/stream/forward.rs b/futures-util/src/stream/forward.rs index e6f17bf45a..d55d6ce71d 100644 --- a/futures-util/src/stream/forward.rs +++ b/futures-util/src/stream/forward.rs @@ -2,7 +2,7 @@ use crate::stream::{StreamExt, Fuse}; use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -38,13 +38,13 @@ where fn try_start_send( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, item: St::Ok, ) -> Poll> { debug_assert!(self.buffered_item.is_none()); { let mut sink = self.as_mut().sink().as_pin_mut().unwrap(); - if sink.as_mut().poll_ready(waker)?.is_ready() { + if sink.as_mut().poll_ready(cx)?.is_ready() { return Poll::Ready(sink.start_send(item)); } } @@ -68,28 +68,28 @@ where fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { // If we've got an item buffered already, we need to write it to the // sink before we can do anything else if let Some(item) = self.as_mut().buffered_item().take() { - try_ready!(self.as_mut().try_start_send(waker, item)); + try_ready!(self.as_mut().try_start_send(cx, item)); } loop { - match self.as_mut().stream().poll_next(waker) { + match self.as_mut().stream().poll_next(cx) { Poll::Ready(Some(Ok(item))) => - try_ready!(self.as_mut().try_start_send(waker, item)), + try_ready!(self.as_mut().try_start_send(cx, item)), Poll::Ready(Some(Err(e))) => return Poll::Ready(Err(e)), Poll::Ready(None) => { try_ready!(self.as_mut().sink().as_pin_mut().expect(INVALID_POLL) - .poll_close(waker)); + .poll_close(cx)); self.as_mut().sink().set(None); return Poll::Ready(Ok(())) } Poll::Pending => { try_ready!(self.as_mut().sink().as_pin_mut().expect(INVALID_POLL) - .poll_flush(waker)); + .poll_flush(cx)); return Poll::Pending } } diff --git a/futures-util/src/stream/fuse.rs b/futures-util/src/stream/fuse.rs index 20f5b0cbcb..7eab4596ff 100644 --- a/futures-util/src/stream/fuse.rs +++ b/futures-util/src/stream/fuse.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -76,13 +76,13 @@ impl Stream for Fuse { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { if *self.as_mut().done() { return Poll::Ready(None); } - let item = ready!(self.as_mut().stream().poll_next(waker)); + let item = ready!(self.as_mut().stream().poll_next(cx)); if item.is_none() { *self.as_mut().done() = true; } diff --git a/futures-util/src/stream/futures_ordered.rs b/futures-util/src/stream/futures_ordered.rs index c9798762da..2d40a97798 100644 --- a/futures-util/src/stream/futures_ordered.rs +++ b/futures-util/src/stream/futures_ordered.rs @@ -1,7 +1,7 @@ use crate::stream::FuturesUnordered; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; use core::cmp::{Eq, PartialEq, PartialOrd, Ord, Ordering}; use core::fmt::{self, Debug}; @@ -48,9 +48,9 @@ impl Future for OrderWrapper fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - self.as_mut().data().as_mut().poll(waker) + self.as_mut().data().as_mut().poll(cx) .map(|output| OrderWrapper { data: output, index: self.index }) } } @@ -152,7 +152,7 @@ impl Stream for FuturesOrdered { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_> ) -> Poll> { let this = &mut *self; @@ -165,7 +165,7 @@ impl Stream for FuturesOrdered { } loop { - match Pin::new(&mut this.in_progress_queue).poll_next(waker) { + match Pin::new(&mut this.in_progress_queue).poll_next(cx) { Poll::Ready(Some(output)) => { if output.index == this.next_outgoing_index { this.next_outgoing_index += 1; diff --git a/futures-util/src/stream/futures_unordered/mod.rs b/futures-util/src/stream/futures_unordered/mod.rs index 7e4c9f217a..a648e0e1ed 100644 --- a/futures-util/src/stream/futures_unordered/mod.rs +++ b/futures-util/src/stream/futures_unordered/mod.rs @@ -3,7 +3,7 @@ use crate::task::{AtomicWaker}; use futures_core::future::{Future, FutureObj, LocalFutureObj}; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll, Spawn, LocalSpawn, SpawnError}; +use futures_core::task::{Context, Poll, Spawn, LocalSpawn, SpawnError}; use core::cell::UnsafeCell; use core::fmt::{self, Debug}; use core::iter::FromIterator; @@ -287,11 +287,11 @@ impl FuturesUnordered { impl Stream for FuturesUnordered { type Item = Fut::Output; - fn poll_next(mut self: Pin<&mut Self>, waker: &Waker) + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { // Ensure `parent` is correctly set. - self.ready_to_run_queue.waker.register(waker); + self.ready_to_run_queue.waker.register(cx.waker()); loop { // Safety: &mut self guarantees the mutual exclusion `dequeue` @@ -311,7 +311,7 @@ impl Stream for FuturesUnordered { // At this point, it may be worth yielding the thread & // spinning a few times... but for now, just yield using the // task system. - waker.wake(); + cx.waker().wake(); return Poll::Pending; } Dequeue::Data(task) => task, @@ -403,11 +403,12 @@ impl Stream for FuturesUnordered { // deallocating the task if need be. let res = { let waker = Task::waker_ref(bomb.task.as_ref().unwrap()); + let mut cx = Context::from_waker(&waker); // Safety: We won't move the future ever again let future = unsafe { Pin::new_unchecked(future) }; - future.poll(&waker) + future.poll(&mut cx) }; match res { diff --git a/futures-util/src/stream/inspect.rs b/futures-util/src/stream/inspect.rs index 77c20c12b3..8873773374 100644 --- a/futures-util/src/stream/inspect.rs +++ b/futures-util/src/stream/inspect.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -63,9 +63,9 @@ impl Stream for Inspect fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { - let item = ready!(self.as_mut().stream().poll_next(waker)); + let item = ready!(self.as_mut().stream().poll_next(cx)); Poll::Ready(item.map(|e| { (self.as_mut().f())(&e); e diff --git a/futures-util/src/stream/into_future.rs b/futures-util/src/stream/into_future.rs index 1e17255e57..b1a7fe86ec 100644 --- a/futures-util/src/stream/into_future.rs +++ b/futures-util/src/stream/into_future.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Future for the [`into_future`](super::StreamExt::into_future) method. #[derive(Debug)] @@ -66,11 +66,11 @@ impl Future for StreamFuture { fn poll( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll { let item = { let s = self.stream.as_mut().expect("polling StreamFuture twice"); - ready!(Pin::new(s).poll_next(waker)) + ready!(Pin::new(s).poll_next(cx)) }; let stream = self.stream.take().unwrap(); Poll::Ready((item, stream)) diff --git a/futures-util/src/stream/iter.rs b/futures-util/src/stream/iter.rs index 5e3ea6a713..6b32fbf68c 100644 --- a/futures-util/src/stream/iter.rs +++ b/futures-util/src/stream/iter.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Stream for the [`iter`] function. #[derive(Debug)] @@ -37,7 +37,7 @@ impl Stream for Iter { type Item = I::Item; - fn poll_next(mut self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(self.iter.next()) } } diff --git a/futures-util/src/stream/map.rs b/futures-util/src/stream/map.rs index 32a6aa8705..17ccc419a8 100644 --- a/futures-util/src/stream/map.rs +++ b/futures-util/src/stream/map.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -63,9 +63,9 @@ impl Stream for Map fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { - let option = ready!(self.as_mut().stream().poll_next(waker)); + let option = ready!(self.as_mut().stream().poll_next(cx)); Poll::Ready(option.map(self.as_mut().f())) } } diff --git a/futures-util/src/stream/mod.rs b/futures-util/src/stream/mod.rs index 415831f550..6aa2c580ab 100644 --- a/futures-util/src/stream/mod.rs +++ b/futures-util/src/stream/mod.rs @@ -7,7 +7,7 @@ use core::pin::Pin; use either::Either; use futures_core::future::Future; use futures_core::stream::{FusedStream, Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; #[cfg(feature = "alloc")] use alloc::boxed::Box; @@ -1058,11 +1058,11 @@ pub trait StreamExt: Stream { /// stream types. fn poll_next_unpin( &mut self, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> where Self: Unpin + Sized { - Pin::new(self).poll_next(waker) + Pin::new(self).poll_next(cx) } /// Returns a [`Future`] that resolves when the next item in this stream is diff --git a/futures-util/src/stream/next.rs b/futures-util/src/stream/next.rs index a3d9c49561..ae085b30fa 100644 --- a/futures-util/src/stream/next.rs +++ b/futures-util/src/stream/next.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Future for the [`next`](super::StreamExt::next) method. #[derive(Debug)] @@ -29,8 +29,8 @@ impl Future for Next<'_, St> { fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - Pin::new(&mut *self.stream).poll_next(waker) + Pin::new(&mut *self.stream).poll_next(cx) } } diff --git a/futures-util/src/stream/once.rs b/futures-util/src/stream/once.rs index 27b5e207c8..966d35233f 100644 --- a/futures-util/src/stream/once.rs +++ b/futures-util/src/stream/once.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Creates a stream of single element @@ -39,10 +39,10 @@ impl Stream for Once { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { let val = if let Some(f) = self.as_mut().future().as_pin_mut() { - ready!(f.poll(waker)) + ready!(f.poll(cx)) } else { return Poll::Ready(None) }; diff --git a/futures-util/src/stream/peek.rs b/futures-util/src/stream/peek.rs index 0052549f17..3ba16d6e73 100644 --- a/futures-util/src/stream/peek.rs +++ b/futures-util/src/stream/peek.rs @@ -1,7 +1,7 @@ use crate::stream::{StreamExt, Fuse}; use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -36,13 +36,13 @@ impl Peekable { /// to the next item if the stream is ready or passes through any errors. pub fn peek<'a>( mut self: Pin<&'a mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { if self.peeked.is_some() { let this: &Self = self.into_ref().get_ref(); return Poll::Ready(this.peeked.as_ref()) } - match ready!(self.as_mut().stream().poll_next(waker)) { + match ready!(self.as_mut().stream().poll_next(cx)) { None => Poll::Ready(None), Some(item) => { *self.as_mut().peeked() = Some(item); @@ -64,12 +64,12 @@ impl Stream for Peekable { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { if let Some(item) = self.as_mut().peeked().take() { return Poll::Ready(Some(item)) } - self.as_mut().stream().poll_next(waker) + self.as_mut().stream().poll_next(cx) } } diff --git a/futures-util/src/stream/poll_fn.rs b/futures-util/src/stream/poll_fn.rs index a03ad0d212..ecbb5aa1d3 100644 --- a/futures-util/src/stream/poll_fn.rs +++ b/futures-util/src/stream/poll_fn.rs @@ -2,7 +2,7 @@ use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Stream for the [`poll_fn`] function. #[derive(Debug)] @@ -34,18 +34,18 @@ impl Unpin for PollFn {} /// ``` pub fn poll_fn(f: F) -> PollFn where - F: FnMut(&Waker) -> Poll>, + F: FnMut(&mut Context<'_>) -> Poll>, { PollFn { f } } impl Stream for PollFn where - F: FnMut(&Waker) -> Poll>, + F: FnMut(&mut Context<'_>) -> Poll>, { type Item = T; - fn poll_next(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { - (&mut self.f)(waker) + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + (&mut self.f)(cx) } } diff --git a/futures-util/src/stream/repeat.rs b/futures-util/src/stream/repeat.rs index 65bb2128a3..08c6ef054b 100644 --- a/futures-util/src/stream/repeat.rs +++ b/futures-util/src/stream/repeat.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Stream for the [`repeat`] function. #[derive(Debug)] @@ -35,7 +35,7 @@ impl Stream for Repeat { type Item = T; - fn poll_next(self: Pin<&mut Self>, _: &Waker) -> Poll> { + fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Ready(Some(self.item.clone())) } } diff --git a/futures-util/src/stream/select.rs b/futures-util/src/stream/select.rs index 0e3ee361a0..a06036c116 100644 --- a/futures-util/src/stream/select.rs +++ b/futures-util/src/stream/select.rs @@ -1,7 +1,7 @@ use crate::stream::{StreamExt, Fuse}; use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; /// Stream for the [`select`](super::StreamExt::select) method. #[derive(Debug)] @@ -41,7 +41,7 @@ impl Stream for Select fn poll_next( self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { let Select { flag, stream1, stream2 } = unsafe { Pin::get_unchecked_mut(self) }; @@ -49,9 +49,9 @@ impl Stream for Select let stream2 = unsafe { Pin::new_unchecked(stream2) }; if !*flag { - poll_inner(flag, stream1, stream2, waker) + poll_inner(flag, stream1, stream2, cx) } else { - poll_inner(flag, stream2, stream1, waker) + poll_inner(flag, stream2, stream1, cx) } } } @@ -60,11 +60,11 @@ fn poll_inner( flag: &mut bool, a: Pin<&mut St1>, b: Pin<&mut St2>, - waker: &Waker + cx: &mut Context<'_> ) -> Poll> where St1: Stream, St2: Stream { - let a_done = match a.poll_next(waker) { + 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; @@ -74,7 +74,7 @@ fn poll_inner( Poll::Pending => false, }; - match b.poll_next(waker) { + match b.poll_next(cx) { Poll::Ready(Some(item)) => { Poll::Ready(Some(item)) } diff --git a/futures-util/src/stream/select_all.rs b/futures-util/src/stream/select_all.rs index d48542df5c..1faa61038f 100644 --- a/futures-util/src/stream/select_all.rs +++ b/futures-util/src/stream/select_all.rs @@ -4,7 +4,7 @@ use core::fmt::{self, Debug}; use core::pin::Pin; use futures_core::{Poll, Stream, FusedStream}; -use futures_core::task::Waker; +use futures_core::task::Context; use crate::stream::{StreamExt, StreamFuture, FuturesUnordered}; @@ -74,9 +74,9 @@ impl Stream for SelectAll { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - match self.inner.poll_next_unpin(waker) { + match self.inner.poll_next_unpin(cx) { Poll::Pending => Poll::Pending, Poll::Ready(Some((Some(item), remaining))) => { self.push(remaining); @@ -86,7 +86,7 @@ impl Stream for SelectAll { // FuturesUnordered thinks it isn't terminated // because it yielded a Some. Here we poll it // so it can realize it is terminated. - let _ = self.inner.poll_next_unpin(waker); + let _ = self.inner.poll_next_unpin(cx); Poll::Ready(None) } Poll::Ready(_) => Poll::Ready(None), diff --git a/futures-util/src/stream/select_next_some.rs b/futures-util/src/stream/select_next_some.rs index d990d5a6b9..565d0201eb 100644 --- a/futures-util/src/stream/select_next_some.rs +++ b/futures-util/src/stream/select_next_some.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::stream::{Stream, FusedStream}; use futures_core::future::{Future, FusedFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use crate::stream::StreamExt; /// Future for the [`select_next_some`](super::StreamExt::select_next_some) @@ -27,14 +27,14 @@ impl<'a, St: FusedStream> FusedFuture for SelectNextSome<'a, St> { impl<'a, St: Stream + FusedStream + Unpin> Future for SelectNextSome<'a, St> { type Output = St::Item; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { assert!(!self.stream.is_terminated(), "SelectNextSome polled after terminated"); - if let Some(item) = ready!(self.stream.poll_next_unpin(waker)) { + if let Some(item) = ready!(self.stream.poll_next_unpin(cx)) { Poll::Ready(item) } else { debug_assert!(self.stream.is_terminated()); - waker.wake(); + cx.waker().wake(); Poll::Pending } } diff --git a/futures-util/src/stream/skip.rs b/futures-util/src/stream/skip.rs index 288f0f14bb..d35484ebfd 100644 --- a/futures-util/src/stream/skip.rs +++ b/futures-util/src/stream/skip.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -60,16 +60,16 @@ impl Stream for Skip { fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { while self.remaining > 0 { - match ready!(self.as_mut().stream().poll_next(waker)) { + match ready!(self.as_mut().stream().poll_next(cx)) { Some(_) => *self.as_mut().remaining() -= 1, None => return Poll::Ready(None), } } - self.as_mut().stream().poll_next(waker) + self.as_mut().stream().poll_next(cx) } } diff --git a/futures-util/src/stream/skip_while.rs b/futures-util/src/stream/skip_while.rs index 9c62316971..0cbb65d17c 100644 --- a/futures-util/src/stream/skip_while.rs +++ b/futures-util/src/stream/skip_while.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -78,15 +78,15 @@ impl Stream for SkipWhile fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { if self.done_skipping { - return self.as_mut().stream().poll_next(waker); + return self.as_mut().stream().poll_next(cx); } loop { if self.pending_item.is_none() { - let item = match ready!(self.as_mut().stream().poll_next(waker)) { + let item = match ready!(self.as_mut().stream().poll_next(cx)) { Some(e) => e, None => return Poll::Ready(None), }; @@ -95,7 +95,7 @@ impl Stream for SkipWhile *self.as_mut().pending_item() = Some(item); } - let skipped = ready!(self.as_mut().pending_fut().as_pin_mut().unwrap().poll(waker)); + let skipped = ready!(self.as_mut().pending_fut().as_pin_mut().unwrap().poll(cx)); let item = self.as_mut().pending_item().take().unwrap(); self.as_mut().pending_fut().set(None); diff --git a/futures-util/src/stream/split.rs b/futures-util/src/stream/split.rs index 875d363425..3e9956f355 100644 --- a/futures-util/src/stream/split.rs +++ b/futures-util/src/stream/split.rs @@ -1,5 +1,5 @@ use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use core::fmt; use core::pin::Pin; @@ -31,9 +31,9 @@ impl SplitStream { impl Stream for SplitStream { type Item = S::Item; - fn poll_next(self: Pin<&mut Self>, waker: &Waker) -> Poll> { - match self.0.poll_lock(waker) { - Poll::Ready(mut inner) => inner.as_pin_mut().poll_next(waker), + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + match self.0.poll_lock(cx) { + Poll::Ready(mut inner) => inner.as_pin_mut().poll_next(cx), Poll::Pending => Poll::Pending, } } @@ -70,12 +70,12 @@ impl + Unpin, Item> SplitSink { impl, Item> Sink for SplitSink { type SinkError = S::SinkError; - fn poll_ready(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { if self.slot.is_none() { return Poll::Ready(Ok(())); } - try_ready!(self.as_mut().poll_flush(waker)); + try_ready!(self.as_mut().poll_flush(cx)); } } @@ -84,33 +84,33 @@ impl, Item> Sink for SplitSink { Ok(()) } - fn poll_flush(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = &mut *self; - match this.lock.poll_lock(waker) { + match this.lock.poll_lock(cx) { Poll::Ready(mut inner) => { if this.slot.is_some() { - try_ready!(inner.as_pin_mut().poll_ready(waker)); + try_ready!(inner.as_pin_mut().poll_ready(cx)); if let Err(e) = inner.as_pin_mut().start_send(this.slot.take().unwrap()) { return Poll::Ready(Err(e)); } } - inner.as_pin_mut().poll_flush(waker) + inner.as_pin_mut().poll_flush(cx) } Poll::Pending => Poll::Pending, } } - fn poll_close(mut self: Pin<&mut Self>, waker: &Waker) -> Poll> { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = &mut *self; - match this.lock.poll_lock(waker) { + match this.lock.poll_lock(cx) { Poll::Ready(mut inner) => { if this.slot.is_some() { - try_ready!(inner.as_pin_mut().poll_ready(waker)); + try_ready!(inner.as_pin_mut().poll_ready(cx)); if let Err(e) = inner.as_pin_mut().start_send(this.slot.take().unwrap()) { return Poll::Ready(Err(e)); } } - inner.as_pin_mut().poll_close(waker) + inner.as_pin_mut().poll_close(cx) } Poll::Pending => Poll::Pending, } diff --git a/futures-util/src/stream/take.rs b/futures-util/src/stream/take.rs index 4a2e776447..9718630c84 100644 --- a/futures-util/src/stream/take.rs +++ b/futures-util/src/stream/take.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -56,12 +56,12 @@ impl Stream for Take fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { if self.remaining == 0 { Poll::Ready(None) } else { - let next = ready!(self.as_mut().stream().poll_next(waker)); + let next = ready!(self.as_mut().stream().poll_next(cx)); match next { Some(_) => *self.as_mut().remaining() -= 1, None => *self.as_mut().remaining() = 0, diff --git a/futures-util/src/stream/take_while.rs b/futures-util/src/stream/take_while.rs index 3d97f64d49..e2e4df319a 100644 --- a/futures-util/src/stream/take_while.rs +++ b/futures-util/src/stream/take_while.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::Stream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -72,14 +72,14 @@ impl Stream for TakeWhile fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { if self.done_taking { return Poll::Ready(None); } if self.pending_item.is_none() { - let item = match ready!(self.as_mut().stream().poll_next(waker)) { + let item = match ready!(self.as_mut().stream().poll_next(cx)) { Some(e) => e, None => return Poll::Ready(None), }; @@ -88,7 +88,7 @@ impl Stream for TakeWhile *self.as_mut().pending_item() = Some(item); } - let take = ready!(self.as_mut().pending_fut().as_pin_mut().unwrap().poll(waker)); + let take = ready!(self.as_mut().pending_fut().as_pin_mut().unwrap().poll(cx)); self.as_mut().pending_fut().set(None); let item = self.as_mut().pending_item().take().unwrap(); diff --git a/futures-util/src/stream/then.rs b/futures-util/src/stream/then.rs index 513b1530e6..11bbb910e1 100644 --- a/futures-util/src/stream/then.rs +++ b/futures-util/src/stream/then.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -48,10 +48,10 @@ impl Stream for Then fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { if self.as_mut().future().as_pin_mut().is_none() { - let item = match ready!(self.as_mut().stream().poll_next(waker)) { + let item = match ready!(self.as_mut().stream().poll_next(cx)) { None => return Poll::Ready(None), Some(e) => e, }; @@ -59,7 +59,7 @@ impl Stream for Then self.as_mut().future().set(Some(fut)); } - let e = ready!(self.as_mut().future().as_pin_mut().unwrap().poll(waker)); + let e = ready!(self.as_mut().future().as_pin_mut().unwrap().poll(cx)); self.as_mut().future().set(None); Poll::Ready(Some(e)) } diff --git a/futures-util/src/stream/unfold.rs b/futures-util/src/stream/unfold.rs index eb00f4a992..cef3271ca4 100644 --- a/futures-util/src/stream/unfold.rs +++ b/futures-util/src/stream/unfold.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Creates a `Stream` from a seed and a closure returning a `Future`. @@ -89,14 +89,14 @@ impl Stream for Unfold fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { if let Some(state) = self.as_mut().state().take() { let fut = (self.as_mut().f())(state); Pin::set(&mut self.as_mut().fut(), Some(fut)); } - let step = ready!(self.as_mut().fut().as_pin_mut().unwrap().poll(waker)); + let step = ready!(self.as_mut().fut().as_pin_mut().unwrap().poll(cx)); self.as_mut().fut().set(None); if let Some((item, next_state)) = step { diff --git a/futures-util/src/stream/zip.rs b/futures-util/src/stream/zip.rs index 9f5efb8800..abd43df6ad 100644 --- a/futures-util/src/stream/zip.rs +++ b/futures-util/src/stream/zip.rs @@ -1,7 +1,7 @@ use crate::stream::{StreamExt, Fuse}; use core::pin::Pin; use futures_core::stream::{FusedStream, Stream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Stream for the [`zip`](super::StreamExt::zip) method. @@ -47,16 +47,16 @@ impl Stream for Zip fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll> { if self.queued1.is_none() { - match self.as_mut().stream1().poll_next(waker) { + match self.as_mut().stream1().poll_next(cx) { Poll::Ready(Some(item1)) => *self.as_mut().queued1() = Some(item1), Poll::Ready(None) | Poll::Pending => {} } } if self.as_mut().queued2().is_none() { - match self.as_mut().stream2().poll_next(waker) { + match self.as_mut().stream2().poll_next(cx) { Poll::Ready(Some(item2)) => *self.as_mut().queued2() = Some(item2), Poll::Ready(None) | Poll::Pending => {} } diff --git a/futures-util/src/task/mod.rs b/futures-util/src/task/mod.rs index d686272124..28139ac6b1 100644 --- a/futures-util/src/task/mod.rs +++ b/futures-util/src/task/mod.rs @@ -6,11 +6,11 @@ cfg_target_has_atomic! { #[cfg(feature = "alloc")] macro_rules! waker_vtable { ($ty:ident) => { - &RawWakerVTable { - clone: clone_arc_raw::<$ty>, - drop: drop_arc_raw::<$ty>, - wake: wake_arc_raw::<$ty>, - } + &RawWakerVTable::new( + clone_arc_raw::<$ty>, + wake_arc_raw::<$ty>, + drop_arc_raw::<$ty>, + ) }; } @@ -37,4 +37,4 @@ pub use self::spawn::{SpawnExt, LocalSpawnExt}; // re-export for `select!` #[doc(hidden)] -pub use futures_core::task::{Waker, Poll}; +pub use futures_core::task::{Context, Poll, Waker}; diff --git a/futures-util/src/task/noop_waker.rs b/futures-util/src/task/noop_waker.rs index 667d413a88..7eb9c55aed 100644 --- a/futures-util/src/task/noop_waker.rs +++ b/futures-util/src/task/noop_waker.rs @@ -11,11 +11,7 @@ unsafe fn noop_clone(_data: *const()) -> RawWaker { unsafe fn noop(_data: *const()) { } -const NOOP_WAKER_VTABLE: RawWakerVTable = RawWakerVTable { - clone: noop_clone, - drop: noop, - wake: noop, -}; +const NOOP_WAKER_VTABLE: RawWakerVTable = RawWakerVTable::new(noop_clone, noop, noop); fn noop_raw_waker() -> RawWaker { RawWaker::new(null(), &NOOP_WAKER_VTABLE) @@ -30,8 +26,8 @@ fn noop_raw_waker() -> RawWaker { /// ``` /// #![feature(futures_api)] /// use futures::task::noop_waker; -/// let lw = noop_waker(); -/// lw.wake(); +/// let waker = noop_waker(); +/// waker.wake(); /// ``` #[inline] pub fn noop_waker() -> Waker { @@ -49,8 +45,8 @@ pub fn noop_waker() -> Waker { /// ``` /// #![feature(futures_api)] /// use futures::task::noop_waker_ref; -/// let lw = noop_waker_ref(); -/// lw.wake(); +/// let waker = noop_waker_ref(); +/// waker.wake(); /// ``` #[inline] #[cfg(feature = "std")] @@ -61,4 +57,3 @@ pub fn noop_waker_ref() -> &'static Waker { } NOOP_WAKER_INSTANCE.with(|l| unsafe { &*l.get() }) } - diff --git a/futures-util/src/task/waker_ref.rs b/futures-util/src/task/waker_ref.rs index cd5c39dc06..e8dc82c535 100644 --- a/futures-util/src/task/waker_ref.rs +++ b/futures-util/src/task/waker_ref.rs @@ -57,11 +57,11 @@ where // Similar to `waker_vtable`, but with a no-op `drop` function. // Clones of the resulting `RawWaker` will still be dropped normally. - let vtable = &RawWakerVTable { - clone: clone_arc_raw::, - drop: noop, - wake: wake_arc_raw::, - }; + let vtable = &RawWakerVTable::new( + clone_arc_raw::, + wake_arc_raw::, + noop, + ); let waker = unsafe { Waker::new_unchecked(RawWaker::new(ptr, vtable)) diff --git a/futures-util/src/try_future/and_then.rs b/futures-util/src/try_future/and_then.rs index 718abb23e3..dc9d180c76 100644 --- a/futures-util/src/try_future/and_then.rs +++ b/futures-util/src/try_future/and_then.rs @@ -1,7 +1,7 @@ use super::{TryChain, TryChainAction}; use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`and_then`](super::TryFutureExt::and_then) method. @@ -42,8 +42,8 @@ impl Future for AndThen { type Output = Result; - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll { - self.try_chain().poll(waker, |result, async_op| { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.try_chain().poll(cx, |result, async_op| { match result { Ok(ok) => TryChainAction::Future(async_op(ok)), Err(err) => TryChainAction::Output(Err(err)), diff --git a/futures-util/src/try_future/err_into.rs b/futures-util/src/try_future/err_into.rs index 3986f1a9fc..2c57e98829 100644 --- a/futures-util/src/try_future/err_into.rs +++ b/futures-util/src/try_future/err_into.rs @@ -1,7 +1,7 @@ use core::marker::PhantomData; use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`err_into`](super::TryFutureExt::err_into) method. @@ -37,9 +37,9 @@ impl Future for ErrInto fn poll( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - self.future().try_poll(waker) + self.future().try_poll(cx) .map(|res| res.map_err(Into::into)) } } diff --git a/futures-util/src/try_future/flatten_sink.rs b/futures-util/src/try_future/flatten_sink.rs index 3a9c89c3ea..b22bf2dcf1 100644 --- a/futures-util/src/try_future/flatten_sink.rs +++ b/futures-util/src/try_future/flatten_sink.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::TryFuture; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; #[derive(Debug)] @@ -48,16 +48,16 @@ where fn poll_ready( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { let resolved_stream = match self.as_mut().project_pin() { - Ready(s) => return s.poll_ready(waker), - Waiting(f) => try_ready!(f.try_poll(waker)), + Ready(s) => return s.poll_ready(cx), + Waiting(f) => try_ready!(f.try_poll(cx)), Closed => panic!("poll_ready called after eof"), }; Pin::set(&mut self.as_mut(), FlattenSink(Ready(resolved_stream))); if let Ready(resolved_stream) = self.project_pin() { - resolved_stream.poll_ready(waker) + resolved_stream.poll_ready(cx) } else { unreachable!() } @@ -76,10 +76,10 @@ where fn poll_flush( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { match self.project_pin() { - Ready(s) => s.poll_flush(waker), + Ready(s) => s.poll_flush(cx), // if sink not yet resolved, nothing written ==> everything flushed Waiting(_) => Poll::Ready(Ok(())), Closed => panic!("poll_flush called after eof"), @@ -88,10 +88,10 @@ where fn poll_close( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { let res = match self.as_mut().project_pin() { - Ready(s) => s.poll_close(waker), + Ready(s) => s.poll_close(cx), Waiting(_) | Closed => Poll::Ready(Ok(())), }; if res.is_ready() { diff --git a/futures-util/src/try_future/into_future.rs b/futures-util/src/try_future/into_future.rs index 86e25921a2..8092702871 100644 --- a/futures-util/src/try_future/into_future.rs +++ b/futures-util/src/try_future/into_future.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`into_future`](super::TryFutureExt::into_future) method. @@ -29,8 +29,8 @@ impl Future for IntoFuture { #[inline] fn poll( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - self.future().try_poll(waker) + self.future().try_poll(cx) } } diff --git a/futures-util/src/try_future/map_err.rs b/futures-util/src/try_future/map_err.rs index 404d9574d0..47306ad160 100644 --- a/futures-util/src/try_future/map_err.rs +++ b/futures-util/src/try_future/map_err.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`map_err`](super::TryFutureExt::map_err) method. @@ -35,9 +35,9 @@ impl Future for MapErr fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - match self.as_mut().future().try_poll(waker) { + match self.as_mut().future().try_poll(cx) { Poll::Pending => Poll::Pending, Poll::Ready(result) => { let f = self.as_mut().f().take() diff --git a/futures-util/src/try_future/map_ok.rs b/futures-util/src/try_future/map_ok.rs index fa4daba27d..f98ab76cdf 100644 --- a/futures-util/src/try_future/map_ok.rs +++ b/futures-util/src/try_future/map_ok.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`map_ok`](super::TryFutureExt::map_ok) method. @@ -37,9 +37,9 @@ impl Future for MapOk fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - match self.as_mut().future().try_poll(waker) { + match self.as_mut().future().try_poll(cx) { Poll::Pending => Poll::Pending, Poll::Ready(result) => { let op = self.as_mut().f().take() diff --git a/futures-util/src/try_future/or_else.rs b/futures-util/src/try_future/or_else.rs index 68d54423f4..595f1dfb0e 100644 --- a/futures-util/src/try_future/or_else.rs +++ b/futures-util/src/try_future/or_else.rs @@ -1,7 +1,7 @@ use super::{TryChain, TryChainAction}; use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; /// Future for the [`or_else`](super::TryFutureExt::or_else) method. @@ -44,9 +44,9 @@ impl Future for OrElse fn poll( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - self.try_chain().poll(waker, |result, async_op| { + self.try_chain().poll(cx, |result, async_op| { match result { Ok(ok) => TryChainAction::Output(Ok(ok)), Err(err) => TryChainAction::Future(async_op(err)), diff --git a/futures-util/src/try_future/try_chain.rs b/futures-util/src/try_future/try_chain.rs index 2a2356cef7..2816cc88fd 100644 --- a/futures-util/src/try_future/try_chain.rs +++ b/futures-util/src/try_future/try_chain.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::TryFuture; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; #[must_use = "futures do nothing unless polled"] #[derive(Debug)] @@ -34,7 +34,7 @@ impl TryChain pub(crate) fn poll( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, f: F, ) -> Poll> where F: FnOnce(Result, Data) -> TryChainAction, @@ -48,14 +48,14 @@ impl TryChain let (output, data) = match this { TryChain::First(fut1, data) => { // Poll the first future - match unsafe { Pin::new_unchecked(fut1) }.try_poll(waker) { + match unsafe { Pin::new_unchecked(fut1) }.try_poll(cx) { Poll::Pending => return Poll::Pending, Poll::Ready(output) => (output, data.take().unwrap()), } } TryChain::Second(fut2) => { // Poll the second future - return unsafe { Pin::new_unchecked(fut2) }.try_poll(waker) + return unsafe { Pin::new_unchecked(fut2) }.try_poll(cx) } TryChain::Empty => { panic!("future must not be polled after it returned `Poll::Ready`"); diff --git a/futures-util/src/try_future/try_join.rs b/futures-util/src/try_future/try_join.rs index 8d63de430a..5de31b3a68 100644 --- a/futures-util/src/try_future/try_join.rs +++ b/futures-util/src/try_future/try_join.rs @@ -5,7 +5,7 @@ use crate::try_future::{TryFutureExt, IntoFuture}; use core::fmt; use core::pin::Pin; use futures_core::future::{Future, TryFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::unsafe_pinned; macro_rules! generate { @@ -70,17 +70,17 @@ macro_rules! generate { #[allow(clippy::useless_let_if_seq)] fn poll( - mut self: Pin<&mut Self>, waker: &Waker + mut self: Pin<&mut Self>, cx: &mut Context<'_> ) -> Poll { let mut all_done = true; - if self.as_mut().Fut1().poll(waker).is_pending() { + if self.as_mut().Fut1().poll(cx).is_pending() { all_done = false; } else if self.as_mut().Fut1().output_mut().unwrap().is_err() { return Poll::Ready(Err( self.as_mut().Fut1().take_output().unwrap().err().unwrap())); } $( - if self.as_mut().$Fut().poll(waker).is_pending() { + if self.as_mut().$Fut().poll(cx).is_pending() { all_done = false; } else if self.as_mut().$Fut().output_mut().unwrap().is_err() { return Poll::Ready(Err( diff --git a/futures-util/src/try_future/try_join_all.rs b/futures-util/src/try_future/try_join_all.rs index 362c712a01..4fb37f5d12 100644 --- a/futures-util/src/try_future/try_join_all.rs +++ b/futures-util/src/try_future/try_join_all.rs @@ -6,7 +6,7 @@ use core::future::Future; use core::iter::FromIterator; use core::mem; use core::pin::Pin; -use core::task::{Poll, Waker}; +use core::task::{Context, Poll}; use alloc::prelude::v1::*; use super::TryFuture; @@ -134,12 +134,12 @@ where { type Output = Result, F::Error>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let mut state = FinalState::AllDone; for mut elem in iter_pin_mut(self.elems.as_mut()) { if let Some(pending) = elem.as_mut().pending_pin_mut() { - match pending.try_poll(waker) { + match pending.try_poll(cx) { Poll::Pending => state = FinalState::Pending, Poll::Ready(output) => match output { Ok(item) => elem.set(ElemState::Done(Some(item))), diff --git a/futures-util/src/try_future/unwrap_or_else.rs b/futures-util/src/try_future/unwrap_or_else.rs index 2661e9c6ae..3e142c7732 100644 --- a/futures-util/src/try_future/unwrap_or_else.rs +++ b/futures-util/src/try_future/unwrap_or_else.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`unwrap_or_else`](super::TryFutureExt::unwrap_or_else) @@ -38,9 +38,9 @@ impl Future for UnwrapOrElse fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - match self.as_mut().future().try_poll(waker) { + match self.as_mut().future().try_poll(cx) { Poll::Pending => Poll::Pending, Poll::Ready(result) => { let op = self.as_mut().f().take() diff --git a/futures-util/src/try_stream/err_into.rs b/futures-util/src/try_stream/err_into.rs index 23136fd941..9d29e3d810 100644 --- a/futures-util/src/try_stream/err_into.rs +++ b/futures-util/src/try_stream/err_into.rs @@ -1,7 +1,7 @@ use core::marker::PhantomData; use core::pin::Pin; use futures_core::stream::{FusedStream, Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::unsafe_pinned; @@ -38,9 +38,9 @@ where fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.stream().try_poll_next(waker) + self.stream().try_poll_next(cx) .map(|res| res.map(|some| some.map_err(Into::into))) } } diff --git a/futures-util/src/try_stream/into_async_read.rs b/futures-util/src/try_stream/into_async_read.rs index d1d0ad1c5b..eaa34ce79a 100644 --- a/futures-util/src/try_stream/into_async_read.rs +++ b/futures-util/src/try_stream/into_async_read.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::TryStream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_io::AsyncRead; use std::cmp; use std::io::{Error, Result}; @@ -51,7 +51,7 @@ where { fn poll_read( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, buf: &mut [u8], ) -> Poll> { loop { @@ -72,7 +72,7 @@ where return Poll::Ready(Ok(len)); } ReadState::PendingChunk => { - match ready!(Pin::new(&mut self.stream).try_poll_next(waker)) { + match ready!(Pin::new(&mut self.stream).try_poll_next(cx)) { Some(Ok(chunk)) => { self.state = ReadState::Ready { chunk, @@ -107,8 +107,8 @@ mod tests { macro_rules! assert_read { ($reader:expr, $buf:expr, $item:expr) => { - let waker = noop_waker_ref(); - match Pin::new(&mut $reader).poll_read(waker, $buf) { + let mut cx = Context::from_waker(noop_waker_ref()); + match Pin::new(&mut $reader).poll_read(&mut cx, $buf) { Poll::Ready(Ok(x)) => { assert_eq!(x, $item); } diff --git a/futures-util/src/try_stream/into_stream.rs b/futures-util/src/try_stream/into_stream.rs index 53a7eb7806..5e83e5e880 100644 --- a/futures-util/src/try_stream/into_stream.rs +++ b/futures-util/src/try_stream/into_stream.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::unsafe_pinned; @@ -49,9 +49,9 @@ impl Stream for IntoStream { #[inline] fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.stream().try_poll_next(waker) + self.stream().try_poll_next(cx) } } diff --git a/futures-util/src/try_stream/map_err.rs b/futures-util/src/try_stream/map_err.rs index 462b506abb..911018b9ec 100644 --- a/futures-util/src/try_stream/map_err.rs +++ b/futures-util/src/try_stream/map_err.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -40,9 +40,9 @@ where #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - match self.as_mut().stream().try_poll_next(waker) { + match self.as_mut().stream().try_poll_next(cx) { Poll::Pending => Poll::Pending, Poll::Ready(opt) => Poll::Ready(opt.map(|res| res.map_err(|e| self.as_mut().f()(e)))), diff --git a/futures-util/src/try_stream/map_ok.rs b/futures-util/src/try_stream/map_ok.rs index fabbba69fb..9b2d9dafde 100644 --- a/futures-util/src/try_stream/map_ok.rs +++ b/futures-util/src/try_stream/map_ok.rs @@ -1,6 +1,6 @@ use core::pin::Pin; use futures_core::stream::{FusedStream, Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -40,9 +40,9 @@ where #[allow(clippy::redundant_closure)] // https://github.com/rust-lang-nursery/rust-clippy/issues/1439 fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - match self.as_mut().stream().try_poll_next(waker) { + match self.as_mut().stream().try_poll_next(cx) { Poll::Pending => Poll::Pending, Poll::Ready(opt) => Poll::Ready(opt.map(|res| res.map(|x| self.as_mut().f()(x)))), diff --git a/futures-util/src/try_stream/mod.rs b/futures-util/src/try_stream/mod.rs index 6cbea94179..7557f28fff 100644 --- a/futures-util/src/try_stream/mod.rs +++ b/futures-util/src/try_stream/mod.rs @@ -6,7 +6,7 @@ use core::pin::Pin; use futures_core::future::TryFuture; use futures_core::stream::TryStream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; #[cfg(feature = "compat")] use crate::compat::Compat; @@ -564,11 +564,11 @@ pub trait TryStreamExt: TryStream { /// stream types. fn try_poll_next_unpin( &mut self, - waker: &Waker + cx: &mut Context<'_>, ) -> Poll>> where Self: Unpin, { - Pin::new(self).try_poll_next(waker) + Pin::new(self).try_poll_next(cx) } /// Wraps a [`TryStream`] into a stream compatible with libraries using diff --git a/futures-util/src/try_stream/try_buffer_unordered.rs b/futures-util/src/try_stream/try_buffer_unordered.rs index bd7ffd2f8f..6bdd126ca3 100644 --- a/futures-util/src/try_stream/try_buffer_unordered.rs +++ b/futures-util/src/try_stream/try_buffer_unordered.rs @@ -3,7 +3,7 @@ use crate::try_future::{IntoFuture, TryFutureExt}; use crate::try_stream::IntoStream; use futures_core::future::TryFuture; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; use core::pin::Pin; @@ -71,12 +71,12 @@ impl Stream for TryBufferUnordered fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { // First up, try to spawn off as many futures as possible by filling up // our slab of futures. Propagate errors from the stream immediately. while self.in_progress_queue.len() < self.max { - match self.as_mut().stream().poll_next(waker) { + match self.as_mut().stream().poll_next(cx) { Poll::Ready(Some(Ok(fut))) => self.as_mut().in_progress_queue().push(fut.into_future()), Poll::Ready(Some(Err(e))) => return Poll::Ready(Some(Err(e))), Poll::Ready(None) | Poll::Pending => break, @@ -84,7 +84,7 @@ impl Stream for TryBufferUnordered } // Attempt to pull the next value from the in_progress_queue - match Pin::new(self.as_mut().in_progress_queue()).poll_next(waker) { + match Pin::new(self.as_mut().in_progress_queue()).poll_next(cx) { x @ Poll::Pending | x @ Poll::Ready(Some(_)) => return x, Poll::Ready(None) => {} } diff --git a/futures-util/src/try_stream/try_collect.rs b/futures-util/src/try_stream/try_collect.rs index ee6d7f8b19..71776a24e0 100644 --- a/futures-util/src/try_stream/try_collect.rs +++ b/futures-util/src/try_stream/try_collect.rs @@ -2,7 +2,7 @@ use core::mem; use core::pin::Pin; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::{FusedStream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`try_collect`](super::TryStreamExt::try_collect) method. @@ -44,10 +44,10 @@ impl Future for TryCollect fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { loop { - match ready!(self.as_mut().stream().try_poll_next(waker)) { + match ready!(self.as_mut().stream().try_poll_next(cx)) { Some(Ok(x)) => self.as_mut().items().extend(Some(x)), Some(Err(e)) => return Poll::Ready(Err(e)), None => return Poll::Ready(Ok(self.as_mut().finish())), diff --git a/futures-util/src/try_stream/try_concat.rs b/futures-util/src/try_stream/try_concat.rs index 3febb72067..f625c6dc2f 100644 --- a/futures-util/src/try_stream/try_concat.rs +++ b/futures-util/src/try_stream/try_concat.rs @@ -2,7 +2,7 @@ use core::default::Default; use core::pin::Pin; use futures_core::future::Future; use futures_core::stream::TryStream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`try_concat`](super::TryStreamExt::try_concat) method. @@ -38,9 +38,9 @@ where { type Output = Result; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { - match ready!(self.as_mut().stream().try_poll_next(waker)) { + match ready!(self.as_mut().stream().try_poll_next(cx)) { Some(Ok(x)) => { let accum = self.as_mut().accum(); if let Some(a) = accum { diff --git a/futures-util/src/try_stream/try_filter_map.rs b/futures-util/src/try_stream/try_filter_map.rs index 6d4fc817ac..7f08b058db 100644 --- a/futures-util/src/try_stream/try_filter_map.rs +++ b/futures-util/src/try_stream/try_filter_map.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::{TryFuture}; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -61,11 +61,11 @@ impl Stream for TryFilterMap fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll>> { loop { if self.pending.is_none() { - let item = match ready!(self.as_mut().stream().try_poll_next(waker)) { + let item = match ready!(self.as_mut().stream().try_poll_next(cx)) { Some(Ok(x)) => x, Some(Err(e)) => return Poll::Ready(Some(Err(e))), None => return Poll::Ready(None), @@ -74,7 +74,7 @@ impl Stream for TryFilterMap self.as_mut().pending().set(Some(fut)); } - let result = ready!(self.as_mut().pending().as_pin_mut().unwrap().try_poll(waker)); + let result = ready!(self.as_mut().pending().as_pin_mut().unwrap().try_poll(cx)); self.as_mut().pending().set(None); match result { Ok(Some(x)) => return Poll::Ready(Some(Ok(x))), diff --git a/futures-util/src/try_stream/try_fold.rs b/futures-util/src/try_stream/try_fold.rs index 9b90fef2ea..8e7e0a13ca 100644 --- a/futures-util/src/try_stream/try_fold.rs +++ b/futures-util/src/try_stream/try_fold.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::{FusedFuture, Future, TryFuture}; use futures_core::stream::TryStream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`try_fold`](super::TryStreamExt::try_fold) method. @@ -49,14 +49,14 @@ impl Future for TryFold { type Output = Result; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { // we're currently processing a future to produce a new accum value if self.as_mut().accum().is_none() { let accum = match ready!( self.as_mut().future().as_pin_mut() .expect("TryFold polled after completion") - .try_poll(waker) + .try_poll(cx) ) { Ok(accum) => accum, Err(e) => { @@ -69,7 +69,7 @@ impl Future for TryFold self.as_mut().future().set(None); } - let item = match ready!(self.as_mut().stream().try_poll_next(waker)) { + let item = match ready!(self.as_mut().stream().try_poll_next(cx)) { Some(Ok(item)) => Some(item), Some(Err(e)) => { // Indicate that the future can no longer be polled. diff --git a/futures-util/src/try_stream/try_for_each.rs b/futures-util/src/try_stream/try_for_each.rs index 7de42a530b..c285ff27bf 100644 --- a/futures-util/src/try_stream/try_for_each.rs +++ b/futures-util/src/try_stream/try_for_each.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::{Future, TryFuture}; use futures_core::stream::TryStream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the [`try_for_each`](super::TryStreamExt::try_for_each) method. @@ -40,14 +40,14 @@ impl Future for TryForEach { type Output = Result<(), St::Error>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { if let Some(future) = self.as_mut().future().as_pin_mut() { - try_ready!(future.try_poll(waker)); + try_ready!(future.try_poll(cx)); } Pin::set(&mut self.as_mut().future(), None); - match ready!(self.as_mut().stream().try_poll_next(waker)) { + match ready!(self.as_mut().stream().try_poll_next(cx)) { Some(Ok(e)) => { let future = (self.as_mut().f())(e); Pin::set(&mut self.as_mut().future(), Some(future)); diff --git a/futures-util/src/try_stream/try_for_each_concurrent.rs b/futures-util/src/try_stream/try_for_each_concurrent.rs index ffbb82f6c2..190ef35a96 100644 --- a/futures-util/src/try_stream/try_for_each_concurrent.rs +++ b/futures-util/src/try_stream/try_for_each_concurrent.rs @@ -4,7 +4,7 @@ use core::pin::Pin; use core::num::NonZeroUsize; use futures_core::future::{FusedFuture, Future}; use futures_core::stream::TryStream; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use pin_utils::{unsafe_pinned, unsafe_unpinned}; /// Future for the @@ -58,7 +58,7 @@ impl Future for TryForEachConcurrent { type Output = Result<(), St::Error>; - fn poll(mut self: Pin<&mut Self>, waker: &Waker) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { let mut made_progress_this_iter = false; @@ -67,7 +67,7 @@ impl Future for TryForEachConcurrent // Check if we've already created a number of futures greater than `limit` if self.limit.map(|limit| limit.get() > current_len).unwrap_or(true) { let poll_res = match self.as_mut().stream().as_pin_mut() { - Some(stream) => stream.try_poll_next(waker), + Some(stream) => stream.try_poll_next(cx), None => Poll::Ready(None), }; @@ -96,7 +96,7 @@ impl Future for TryForEachConcurrent } } - match self.as_mut().futures().poll_next_unpin(waker) { + match self.as_mut().futures().poll_next_unpin(cx) { Poll::Ready(Some(Ok(()))) => made_progress_this_iter = true, Poll::Ready(None) => { if self.as_mut().stream().is_none() { diff --git a/futures-util/src/try_stream/try_next.rs b/futures-util/src/try_stream/try_next.rs index 79b7f9aa1a..a1d2a361ca 100644 --- a/futures-util/src/try_stream/try_next.rs +++ b/futures-util/src/try_stream/try_next.rs @@ -1,6 +1,6 @@ use futures_core::future::{FusedFuture, Future}; use futures_core::stream::{FusedStream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use core::pin::Pin; /// Future for the [`try_next`](super::TryStreamExt::try_next) method. @@ -29,9 +29,9 @@ impl Future for TryNext<'_, St> { fn poll( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll { - match Pin::new(&mut *self.stream).try_poll_next(waker) { + match Pin::new(&mut *self.stream).try_poll_next(cx) { Poll::Ready(Some(Ok(x))) => Poll::Ready(Ok(Some(x))), Poll::Ready(Some(Err(e))) => Poll::Ready(Err(e)), Poll::Ready(None) => Poll::Ready(Ok(None)), diff --git a/futures-util/src/try_stream/try_skip_while.rs b/futures-util/src/try_stream/try_skip_while.rs index dcc346b911..42b4b533b9 100644 --- a/futures-util/src/try_stream/try_skip_while.rs +++ b/futures-util/src/try_stream/try_skip_while.rs @@ -1,7 +1,7 @@ use core::pin::Pin; use futures_core::future::TryFuture; use futures_core::stream::{Stream, TryStream}; -use futures_core::task::{Waker, Poll}; +use futures_core::task::{Context, Poll}; use futures_sink::Sink; use pin_utils::{unsafe_pinned, unsafe_unpinned}; @@ -73,15 +73,15 @@ impl Stream for TrySkipWhile fn poll_next( mut self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { if self.done_skipping { - return self.as_mut().stream().try_poll_next(waker); + return self.as_mut().stream().try_poll_next(cx); } loop { if self.pending_item.is_none() { - let item = match ready!(self.as_mut().stream().try_poll_next(waker)?) { + let item = match ready!(self.as_mut().stream().try_poll_next(cx)?) { Some(e) => e, None => return Poll::Ready(None), }; @@ -90,7 +90,7 @@ impl Stream for TrySkipWhile *self.as_mut().pending_item() = Some(item); } - let skipped = ready!(self.as_mut().pending_fut().as_pin_mut().unwrap().try_poll(waker)?); + let skipped = ready!(self.as_mut().pending_fut().as_pin_mut().unwrap().try_poll(cx)?); let item = self.as_mut().pending_item().take().unwrap(); self.as_mut().pending_fut().set(None); diff --git a/futures-util/tests/atomic_waker.rs b/futures-util/tests/atomic_waker.rs index 25dc4e1948..7a1e6da444 100644 --- a/futures-util/tests/atomic_waker.rs +++ b/futures-util/tests/atomic_waker.rs @@ -24,14 +24,14 @@ fn basic() { let t = thread::spawn(move || { let mut pending_count = 0; - block_on(poll_fn(move |lw| { + block_on(poll_fn(move |cx| { if woken_copy.load(Ordering::Relaxed) == 1 { Poll::Ready(()) } else { // Assert we return pending exactly once assert_eq!(0, pending_count); pending_count += 1; - atomic_waker_copy.register(lw); + atomic_waker_copy.register(cx.waker()); returned_pending_copy.store(1, Ordering::Relaxed); diff --git a/futures-util/tests/futures_unordered.rs b/futures-util/tests/futures_unordered.rs index 867b13b69e..2ad5d1302a 100644 --- a/futures-util/tests/futures_unordered.rs +++ b/futures-util/tests/futures_unordered.rs @@ -1,17 +1,17 @@ #![feature(async_await, await_macro, futures_api)] use futures::future; -use futures::task::Poll; +use futures::task::{Context, Poll}; use futures::stream::{FusedStream, FuturesUnordered, StreamExt}; use futures_test::task::noop_waker_ref; #[test] fn is_terminated() { - let lw = noop_waker_ref(); + let mut cx = Context::from_waker(noop_waker_ref()); let mut tasks = FuturesUnordered::new(); assert_eq!(tasks.is_terminated(), false); - assert_eq!(tasks.poll_next_unpin(lw), Poll::Ready(None)); + assert_eq!(tasks.poll_next_unpin(&mut cx), Poll::Ready(None)); assert_eq!(tasks.is_terminated(), true); // Test that the sentinel value doesn't leak @@ -26,8 +26,8 @@ fn is_terminated() { assert_eq!(tasks.iter_mut().len(), 1); assert_eq!(tasks.is_terminated(), false); - assert_eq!(tasks.poll_next_unpin(lw), Poll::Ready(Some(1))); + assert_eq!(tasks.poll_next_unpin(&mut cx), Poll::Ready(Some(1))); assert_eq!(tasks.is_terminated(), false); - assert_eq!(tasks.poll_next_unpin(lw), Poll::Ready(None)); + assert_eq!(tasks.poll_next_unpin(&mut cx), Poll::Ready(None)); assert_eq!(tasks.is_terminated(), true); } diff --git a/futures-util/tests/mutex.rs b/futures-util/tests/mutex.rs index 8a332d6d75..f6a828fdb8 100644 --- a/futures-util/tests/mutex.rs +++ b/futures-util/tests/mutex.rs @@ -4,7 +4,7 @@ use futures::channel::mpsc; use futures::future::{ready, FutureExt}; use futures::lock::Mutex; use futures::stream::StreamExt; -use futures::task::SpawnExt; +use futures::task::{Context, SpawnExt}; use futures_test::future::FutureTestExt; use futures_test::task::{panic_waker_ref, new_count_waker}; use std::sync::Arc; @@ -13,25 +13,26 @@ use std::sync::Arc; fn mutex_acquire_uncontested() { let mutex = Mutex::new(()); for _ in 0..10 { - assert!(mutex.lock().poll_unpin(panic_waker_ref()).is_ready()); + assert!(mutex.lock().poll_unpin(&mut Context::from_waker(panic_waker_ref())).is_ready()); } } #[test] fn mutex_wakes_waiters() { let mutex = Mutex::new(()); - let (lw, counter) = new_count_waker(); - let lock = mutex.lock().poll_unpin(panic_waker_ref()); + let (waker, counter) = new_count_waker(); + let lock = mutex.lock().poll_unpin(&mut Context::from_waker(panic_waker_ref())); assert!(lock.is_ready()); + let mut cx = Context::from_waker(&waker); let mut waiter = mutex.lock(); - assert!(waiter.poll_unpin(&lw).is_pending()); + assert!(waiter.poll_unpin(&mut cx).is_pending()); assert_eq!(counter, 0); drop(lock); assert_eq!(counter, 1); - assert!(waiter.poll_unpin(panic_waker_ref()).is_ready()); + assert!(waiter.poll_unpin(&mut Context::from_waker(panic_waker_ref())).is_ready()); } #[test] diff --git a/futures-util/tests/select_all.rs b/futures-util/tests/select_all.rs index 5150f6f54c..4193e9d2bd 100644 --- a/futures-util/tests/select_all.rs +++ b/futures-util/tests/select_all.rs @@ -2,18 +2,18 @@ use futures::future; use futures::FutureExt; -use futures::task::Poll; +use futures::task::{Context, Poll}; use futures::stream::FusedStream; use futures::stream::{SelectAll, StreamExt}; use futures_test::task::noop_waker_ref; #[test] fn is_terminated() { - let lw = noop_waker_ref(); + let mut cx = Context::from_waker(noop_waker_ref()); let mut tasks = SelectAll::new(); assert_eq!(tasks.is_terminated(), false); - assert_eq!(tasks.poll_next_unpin(lw), Poll::Ready(None)); + assert_eq!(tasks.poll_next_unpin(&mut cx), Poll::Ready(None)); assert_eq!(tasks.is_terminated(), true); // Test that the sentinel value doesn't leak @@ -26,8 +26,8 @@ fn is_terminated() { assert_eq!(tasks.len(), 1); assert_eq!(tasks.is_terminated(), false); - assert_eq!(tasks.poll_next_unpin(lw), Poll::Ready(Some(1))); + assert_eq!(tasks.poll_next_unpin(&mut cx), Poll::Ready(Some(1))); assert_eq!(tasks.is_terminated(), false); - assert_eq!(tasks.poll_next_unpin(lw), Poll::Ready(None)); + assert_eq!(tasks.poll_next_unpin(&mut cx), Poll::Ready(None)); assert_eq!(tasks.is_terminated(), true); } diff --git a/futures-util/tests/select_next_some.rs b/futures-util/tests/select_next_some.rs index 4179414676..0485146949 100644 --- a/futures-util/tests/select_next_some.rs +++ b/futures-util/tests/select_next_some.rs @@ -3,19 +3,20 @@ use futures::{future, select}; use futures::future::{FusedFuture, FutureExt}; use futures::stream::{FuturesUnordered, StreamExt}; -use futures::task::Poll; +use futures::task::{Context, Poll}; use futures_test::future::FutureTestExt; use futures_test::task::new_count_waker; #[test] fn is_terminated() { - let (lw, counter) = new_count_waker(); + let (waker, counter) = new_count_waker(); + let mut cx = Context::from_waker(&waker); let mut tasks = FuturesUnordered::new(); let mut select_next_some = tasks.select_next_some(); assert_eq!(select_next_some.is_terminated(), false); - assert_eq!(select_next_some.poll_unpin(&lw), Poll::Pending); + assert_eq!(select_next_some.poll_unpin(&mut cx), Poll::Pending); assert_eq!(counter, 1); assert_eq!(select_next_some.is_terminated(), true); drop(select_next_some); @@ -24,9 +25,9 @@ fn is_terminated() { let mut select_next_some = tasks.select_next_some(); assert_eq!(select_next_some.is_terminated(), false); - assert_eq!(select_next_some.poll_unpin(&lw), Poll::Ready(1)); + assert_eq!(select_next_some.poll_unpin(&mut cx), Poll::Ready(1)); assert_eq!(select_next_some.is_terminated(), false); - assert_eq!(select_next_some.poll_unpin(&lw), Poll::Pending); + assert_eq!(select_next_some.poll_unpin(&mut cx), Poll::Pending); assert_eq!(select_next_some.is_terminated(), true); } diff --git a/futures/src/lib.rs b/futures/src/lib.rs index aa9cbe8697..e8ac2df21f 100644 --- a/futures/src/lib.rs +++ b/futures/src/lib.rs @@ -402,13 +402,15 @@ pub mod task { //! This module contains: //! //! - [`Spawn`](crate::task::Spawn), a trait for spawning new tasks. + //! - [`Context`](crate::task::Context), a context of an asynchronous task, + //! including a handle for waking up the task. //! - [`Waker`](crate::task::Waker), a handle for waking up a 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::{ - Poll, Spawn, LocalSpawn, SpawnError, + Context, Poll, Spawn, LocalSpawn, SpawnError, Waker, RawWaker, RawWakerVTable }; diff --git a/futures/tests/abortable.rs b/futures/tests/abortable.rs index 15e49cf6d3..dbedc230a5 100644 --- a/futures/tests/abortable.rs +++ b/futures/tests/abortable.rs @@ -3,7 +3,7 @@ use futures::channel::oneshot; use futures::executor::block_on; use futures::future::{abortable, Aborted, FutureExt}; -use futures::task::Poll; +use futures::task::{Context, Poll}; use futures_test::task::new_count_waker; #[test] @@ -20,13 +20,14 @@ fn abortable_awakens() { let (_tx, a_rx) = oneshot::channel::<()>(); let (mut abortable_rx, abort_handle) = abortable(a_rx); - let (lw, counter) = new_count_waker(); + let (waker, counter) = new_count_waker(); + let mut cx = Context::from_waker(&waker); assert_eq!(counter, 0); - assert_eq!(Poll::Pending, abortable_rx.poll_unpin(&lw)); + assert_eq!(Poll::Pending, abortable_rx.poll_unpin(&mut cx)); assert_eq!(counter, 0); abort_handle.abort(); assert_eq!(counter, 1); - assert_eq!(Poll::Ready(Err(Aborted)), abortable_rx.poll_unpin(&lw)); + assert_eq!(Poll::Ready(Err(Aborted)), abortable_rx.poll_unpin(&mut cx)); } #[test] diff --git a/futures/tests/eager_drop.rs b/futures/tests/eager_drop.rs index c6bcb460ae..c1237911dd 100644 --- a/futures/tests/eager_drop.rs +++ b/futures/tests/eager_drop.rs @@ -2,7 +2,7 @@ use futures::channel::oneshot; use futures::future::{self, Future, FutureExt, TryFutureExt}; -use futures::task::{Waker, Poll}; +use futures::task::{Context, Poll}; use futures_test::future::FutureTestExt; use pin_utils::unsafe_pinned; use std::pin::Pin; @@ -56,8 +56,8 @@ impl FutureData { impl Future for FutureData { type Output = F::Output; - fn poll(self: Pin<&mut Self>, waker: &Waker) -> Poll { - self.future().poll(waker) + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.future().poll(cx) } } diff --git a/futures/tests/fuse.rs b/futures/tests/fuse.rs index 8ac18535ba..1386845eab 100644 --- a/futures/tests/fuse.rs +++ b/futures/tests/fuse.rs @@ -1,12 +1,14 @@ #![feature(futures_api)] use futures::future::{self, FutureExt}; +use futures::task::Context; use futures_test::task::panic_waker; #[test] fn fuse() { let mut future = future::ready::(2).fuse(); - let lw = &mut panic_waker(); - assert!(future.poll_unpin(lw).is_ready()); - assert!(future.poll_unpin(lw).is_pending()); + let waker = panic_waker(); + let mut cx = Context::from_waker(&waker); + assert!(future.poll_unpin(&mut cx).is_ready()); + assert!(future.poll_unpin(&mut cx).is_pending()); } diff --git a/futures/tests/future_obj.rs b/futures/tests/future_obj.rs index 19da856af2..14ab618eb0 100644 --- a/futures/tests/future_obj.rs +++ b/futures/tests/future_obj.rs @@ -2,7 +2,7 @@ use futures::future::{Future, FutureExt, FutureObj}; use std::pin::Pin; -use futures::task::{Waker, Poll}; +use futures::task::{Context, Poll}; #[test] fn dropping_does_not_segfault() { @@ -18,7 +18,7 @@ fn dropping_drops_the_future() { impl<'a> Future for Inc<'a> { type Output = (); - fn poll(self: Pin<&mut Self>, _: &Waker) -> Poll<()> { + fn poll(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll<()> { unimplemented!() } } diff --git a/futures/tests/futures_ordered.rs b/futures/tests/futures_ordered.rs index 57db1db86e..4bbbd2df56 100644 --- a/futures/tests/futures_ordered.rs +++ b/futures/tests/futures_ordered.rs @@ -4,6 +4,7 @@ use futures::channel::oneshot; use futures::executor::{block_on, block_on_stream}; use futures::future::{self, FutureExt, FutureObj}; use futures::stream::{StreamExt, FuturesOrdered}; +use futures::task::Context; use futures_test::task::noop_waker_ref; #[test] @@ -15,7 +16,7 @@ fn works_1() { let mut stream = vec![a_rx, b_rx, c_rx].into_iter().collect::>(); b_tx.send(99).unwrap(); - assert!(stream.poll_next_unpin(&noop_waker_ref()).is_pending()); + assert!(stream.poll_next_unpin(&mut Context::from_waker(noop_waker_ref())).is_pending()); a_tx.send(33).unwrap(); c_tx.send(33).unwrap(); @@ -38,13 +39,13 @@ fn works_2() { FutureObj::new(Box::new(b_rx.join(c_rx).map(|(a, b)| Ok(a? + b?)))), ].into_iter().collect::>(); - let lw = &noop_waker_ref(); + let mut cx = Context::from_waker(noop_waker_ref()); a_tx.send(33).unwrap(); b_tx.send(33).unwrap(); - assert!(stream.poll_next_unpin(lw).is_ready()); - assert!(stream.poll_next_unpin(lw).is_pending()); + assert!(stream.poll_next_unpin(&mut cx).is_ready()); + assert!(stream.poll_next_unpin(&mut cx).is_pending()); c_tx.send(33).unwrap(); - assert!(stream.poll_next_unpin(lw).is_ready()); + assert!(stream.poll_next_unpin(&mut cx).is_ready()); } #[test] @@ -70,15 +71,15 @@ fn queue_never_unblocked() { Box::new(b_rx.select(c_rx).then(|res| Ok(Box::new(res) as Box))) as _, ]); - with_no_spawn_context(|lw| { + with_no_spawn_context(|cx| { for _ in 0..10 { - assert!(stream.poll_next(lw).unwrap().is_pending()); + assert!(stream.poll_next(cx).unwrap().is_pending()); } b_tx.send(Box::new(())).unwrap(); - assert!(stream.poll_next(lw).unwrap().is_pending()); + assert!(stream.poll_next(cx).unwrap().is_pending()); c_tx.send(Box::new(())).unwrap(); - assert!(stream.poll_next(lw).unwrap().is_pending()); - assert!(stream.poll_next(lw).unwrap().is_pending()); + assert!(stream.poll_next(cx).unwrap().is_pending()); + assert!(stream.poll_next(cx).unwrap().is_pending()); }) }*/ diff --git a/futures/tests/futures_unordered.rs b/futures/tests/futures_unordered.rs index dcad547b39..02a406a5a7 100644 --- a/futures/tests/futures_unordered.rs +++ b/futures/tests/futures_unordered.rs @@ -4,7 +4,7 @@ use futures::channel::oneshot; use futures::executor::{block_on, block_on_stream}; use futures::future::{self, FutureExt, FutureObj}; use futures::stream::{StreamExt, FuturesUnordered}; -use futures::task::Poll; +use futures::task::{Context, Poll}; use futures_test::{assert_stream_done, assert_stream_next}; use futures_test::future::FutureTestExt; use futures_test::task::noop_waker_ref; @@ -42,11 +42,11 @@ fn works_2() { a_tx.send(9).unwrap(); b_tx.send(10).unwrap(); - let lw = &noop_waker_ref(); - assert_eq!(stream.poll_next_unpin(lw), Poll::Ready(Some(Ok(9)))); + let mut cx = Context::from_waker(noop_waker_ref()); + assert_eq!(stream.poll_next_unpin(&mut cx), Poll::Ready(Some(Ok(9)))); c_tx.send(20).unwrap(); - assert_eq!(stream.poll_next_unpin(lw), Poll::Ready(Some(Ok(30)))); - assert_eq!(stream.poll_next_unpin(lw), Poll::Ready(None)); + assert_eq!(stream.poll_next_unpin(&mut cx), Poll::Ready(Some(Ok(30)))); + assert_eq!(stream.poll_next_unpin(&mut cx), Poll::Ready(None)); } #[test] @@ -72,16 +72,16 @@ fn finished_future() { //FutureObj::new(Box::new(b_rx.select(c_rx))), ]); - support::with_noop_waker_context(f)(|lw| { + support::with_noop_waker_context(f)(|cx| { for _ in 0..10 { - assert!(stream.poll_next_unpin(lw).is_pending()); + assert!(stream.poll_next_unpin(cx).is_pending()); } b_tx.send(12).unwrap(); - assert!(stream.poll_next_unpin(lw).is_ready()); + assert!(stream.poll_next_unpin(cx).is_ready()); c_tx.send(3).unwrap(); - assert!(stream.poll_next_unpin(lw).is_pending()); - assert!(stream.poll_next_unpin(lw).is_pending()); + assert!(stream.poll_next_unpin(cx).is_pending()); + assert!(stream.poll_next_unpin(cx).is_pending()); }) }*/ diff --git a/futures/tests/split.rs b/futures/tests/split.rs index 82390c4191..84ae5049d5 100644 --- a/futures/tests/split.rs +++ b/futures/tests/split.rs @@ -3,7 +3,7 @@ use futures::executor::block_on; use futures::sink::{Sink, SinkExt}; use futures::stream::{self, Stream, StreamExt}; -use futures::task::{Waker, Poll}; +use futures::task::{Context, Poll}; use pin_utils::unsafe_pinned; use std::pin::Pin; @@ -22,9 +22,9 @@ impl Stream for Join { fn poll_next( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.stream().poll_next(waker) + self.stream().poll_next(cx) } } @@ -33,9 +33,9 @@ impl, Item> Sink for Join { fn poll_ready( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.sink().poll_ready(waker) + self.sink().poll_ready(cx) } fn start_send( @@ -47,16 +47,16 @@ impl, Item> Sink for Join { fn poll_flush( self: Pin<&mut Self>, - waker: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.sink().poll_flush(waker) + self.sink().poll_flush(cx) } fn poll_close( self: Pin<&mut Self>, - lw: &Waker, + cx: &mut Context<'_>, ) -> Poll> { - self.sink().poll_close(lw) + self.sink().poll_close(cx) } } diff --git a/futures/tests_disabled/future_flatten_stream.rs b/futures/tests_disabled/future_flatten_stream.rs index 3592d4d9cb..fafaf38f81 100644 --- a/futures/tests_disabled/future_flatten_stream.rs +++ b/futures/tests_disabled/future_flatten_stream.rs @@ -25,7 +25,7 @@ impl Stream for PanickingStream { type Item = T; type Error = E; - fn poll_next(&mut self, _: &Waker) -> Poll, Self::Error> { + fn poll_next(&mut self, _: &mut Context<'_>) -> Poll, Self::Error> { panic!() } } diff --git a/futures/tests_disabled/ready_queue.rs b/futures/tests_disabled/ready_queue.rs index db0b69a5e4..c3e07baabf 100644 --- a/futures/tests_disabled/ready_queue.rs +++ b/futures/tests_disabled/ready_queue.rs @@ -12,7 +12,7 @@ impl AssertSendSync for FuturesUnordered<()> {} #[test] fn basic_usage() { - block_on(future::lazy(move |lw| { + block_on(future::lazy(move |cx| { let mut queue = FuturesUnordered::new(); let (tx1, rx1) = oneshot::channel(); let (tx2, rx2) = oneshot::channel(); @@ -22,19 +22,19 @@ fn basic_usage() { queue.push(rx2); queue.push(rx3); - assert!(!queue.poll_next(lw).unwrap().is_ready()); + assert!(!queue.poll_next(cx).unwrap().is_ready()); tx2.send("hello").unwrap(); - assert_eq!(Ready(Some("hello")), queue.poll_next(lw).unwrap()); - assert!(!queue.poll_next(lw).unwrap().is_ready()); + assert_eq!(Ready(Some("hello")), queue.poll_next(cx).unwrap()); + assert!(!queue.poll_next(cx).unwrap().is_ready()); tx1.send("world").unwrap(); tx3.send("world2").unwrap(); - assert_eq!(Ready(Some("world")), queue.poll_next(lw).unwrap()); - assert_eq!(Ready(Some("world2")), queue.poll_next(lw).unwrap()); - assert_eq!(Ready(None), queue.poll_next(lw).unwrap()); + assert_eq!(Ready(Some("world")), queue.poll_next(cx).unwrap()); + assert_eq!(Ready(Some("world2")), queue.poll_next(cx).unwrap()); + assert_eq!(Ready(None), queue.poll_next(cx).unwrap()); Ok::<_, ()>(()) })).unwrap(); @@ -42,7 +42,7 @@ fn basic_usage() { #[test] fn resolving_errors() { - block_on(future::lazy(move |lw| { + block_on(future::lazy(move |cx| { let mut queue = FuturesUnordered::new(); let (tx1, rx1) = oneshot::channel(); let (tx2, rx2) = oneshot::channel(); @@ -52,19 +52,19 @@ fn resolving_errors() { queue.push(rx2); queue.push(rx3); - assert!(!queue.poll_next(lw).unwrap().is_ready()); + assert!(!queue.poll_next(cx).unwrap().is_ready()); drop(tx2); - assert!(queue.poll_next(lw).is_err()); - assert!(!queue.poll_next(lw).unwrap().is_ready()); + assert!(queue.poll_next(cx).is_err()); + assert!(!queue.poll_next(cx).unwrap().is_ready()); drop(tx1); tx3.send("world2").unwrap(); - assert!(queue.poll_next(lw).is_err()); - assert_eq!(Ready(Some("world2")), queue.poll_next(lw).unwrap()); - assert_eq!(Ready(None), queue.poll_next(lw).unwrap()); + assert!(queue.poll_next(cx).is_err()); + assert_eq!(Ready(Some("world2")), queue.poll_next(cx).unwrap()); + assert_eq!(Ready(None), queue.poll_next(cx).unwrap()); Ok::<_, ()>(()) })).unwrap(); @@ -82,16 +82,16 @@ fn dropping_ready_queue() { queue.push(rx2); queue.push(rx3); - support::noop_waker_lw(|lw| { - assert!(!tx1.poll_cancel(lw).unwrap().is_ready()); - assert!(!tx2.poll_cancel(lw).unwrap().is_ready()); - assert!(!tx3.poll_cancel(lw).unwrap().is_ready()); + support::noop_waker_lw(|cx| { + assert!(!tx1.poll_cancel(cx).unwrap().is_ready()); + assert!(!tx2.poll_cancel(cx).unwrap().is_ready()); + assert!(!tx3.poll_cancel(cx).unwrap().is_ready()); drop(queue); - assert!(tx1.poll_cancel(lw).unwrap().is_ready()); - assert!(tx2.poll_cancel(lw).unwrap().is_ready()); - assert!(tx3.poll_cancel(lw).unwrap().is_ready()); + assert!(tx1.poll_cancel(cx).unwrap().is_ready()); + assert!(tx2.poll_cancel(cx).unwrap().is_ready()); + assert!(tx3.poll_cancel(cx).unwrap().is_ready()); }); Ok::<_, ()>(()).into_future() @@ -149,16 +149,16 @@ fn stress() { #[test] fn panicking_future_dropped() { - block_on(future::lazy(move |lw| { + block_on(future::lazy(move |cx| { let mut queue = FuturesUnordered::new(); queue.push(future::poll_fn(|_| -> Poll { panic!() })); - let r = panic::catch_unwind(AssertUnwindSafe(|| queue.poll_next(lw))); + let r = panic::catch_unwind(AssertUnwindSafe(|| queue.poll_next(cx))); assert!(r.is_err()); assert!(queue.is_empty()); - assert_eq!(Ready(None), queue.poll_next(lw).unwrap()); + assert_eq!(Ready(None), queue.poll_next(cx).unwrap()); Ok::<_, ()>(()) })).unwrap(); diff --git a/futures/tests_disabled/sink.rs b/futures/tests_disabled/sink.rs index 1ea9faa084..b0c24d0f25 100644 --- a/futures/tests_disabled/sink.rs +++ b/futures/tests_disabled/sink.rs @@ -97,16 +97,16 @@ impl Wake for Flag { } } -fn flag_lw(f: F) -> R - where F: FnOnce(Arc, &Waker) -> R +fn flag_cx(f: F) -> R + where F: FnOnce(Arc, &mut Context<'_>) -> R { let flag = Flag::new(); let map = &mut task::LocalMap::new(); let waker = Waker::from(flag.clone()); let exec = &mut support::PanicExec; - let lw = &Waker::new(map, &waker, exec); - f(flag, lw) + let cx = &mut Context::new(map, &waker, exec); + f(flag, cx) } // Sends a value on an i32 channel sink @@ -122,10 +122,10 @@ impl Future for StartSendFut { type Item = S; type Error = S::SinkError; - fn poll(&mut self, waker: &Waker) -> Poll { + fn poll(&mut self, cx: &mut Context<'_>) -> Poll { { let inner = self.0.as_mut().unwrap(); - try_ready!(inner.poll_ready(waker)); + try_ready!(inner.poll_ready(cx)); inner.start_send(self.1.take().unwrap())?; } Ok(Poll::Ready(self.0.take().unwrap())) @@ -141,15 +141,15 @@ fn mpsc_blocking_start_send() { block_on(futures::future::lazy(|_| { tx.start_send(0).unwrap(); - flag_lw(|flag, waker| { + flag_cx(|flag, cx| { let mut task = StartSendFut::new(tx, 1); - assert!(task.poll(waker).unwrap().is_pending()); + assert!(task.poll(cx).unwrap().is_pending()); assert!(!flag.get()); sassert_next(&mut rx, 0); assert!(flag.get()); flag.set(false); - assert!(task.poll(waker).unwrap().is_ready()); + assert!(task.poll(cx).unwrap().is_ready()); assert!(!flag.get()); sassert_next(&mut rx, 1); @@ -171,13 +171,13 @@ fn with_flush() { assert_eq!(sink.start_send(0), Ok(())); - flag_lw(|flag, waker| { + flag_cx(|flag, cx| { let mut task = sink.flush(); - assert!(task.poll(waker).unwrap().is_pending()); + assert!(task.poll(cx).unwrap().is_pending()); tx.send(()).unwrap(); assert!(flag.get()); - let sink = match task.poll(waker).unwrap() { + let sink = match task.poll(cx).unwrap() { Poll::Ready(sink) => sink, _ => panic!() }; @@ -222,7 +222,7 @@ impl Sink for ManualFlush { type SinkItem = Option; // Pass None to flush type SinkError = (); - fn poll_ready(&mut self, _: &Waker) -> Poll<(), Self::SinkError> { + fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll<(), Self::SinkError> { Ok(Poll::Ready(())) } @@ -235,17 +235,17 @@ impl Sink for ManualFlush { Ok(()) } - fn poll_flush(&mut self, waker: &Waker) -> Poll<(), Self::SinkError> { + fn poll_flush(&mut self, cx: &mut Context<'_>) -> Poll<(), Self::SinkError> { if self.data.is_empty() { Ok(Poll::Ready(())) } else { - self.waiting_tasks.push(waker.waker().clone()); + self.waiting_tasks.push(cx.waker().clone()); Ok(Poll::Pending) } } - fn poll_close(&mut self, waker: &Waker) -> Poll<(), Self::SinkError> { - self.poll_flush(waker) + fn poll_close(&mut self, cx: &mut Context<'_>) -> Poll<(), Self::SinkError> { + self.poll_flush(cx) } } @@ -270,18 +270,18 @@ impl ManualFlush { // but doesn't claim to be flushed until the underlying sink is fn with_flush_propagate() { let mut sink = ManualFlush::new().with(|x| -> Result, ()> { Ok(x) }); - flag_lw(|flag, waker| { - assert!(sink.poll_ready(waker).unwrap().is_ready()); + flag_cx(|flag, cx| { + assert!(sink.poll_ready(cx).unwrap().is_ready()); sink.start_send(Some(0)).unwrap(); - assert!(sink.poll_ready(waker).unwrap().is_ready()); + assert!(sink.poll_ready(cx).unwrap().is_ready()); sink.start_send(Some(1)).unwrap(); let mut task = sink.flush(); - assert!(task.poll(waker).unwrap().is_pending()); + assert!(task.poll(cx).unwrap().is_pending()); assert!(!flag.get()); assert_eq!(task.get_mut().unwrap().get_mut().force_flush(), vec![0, 1]); assert!(flag.get()); - assert!(task.poll(waker).unwrap().is_ready()); + assert!(task.poll(cx).unwrap().is_ready()); }) } @@ -317,11 +317,11 @@ impl Allow { } } - fn check(&self, waker: &Waker) -> bool { + fn check(&self, cx: &mut Context<'_>) -> bool { if self.flag.get() { true } else { - self.tasks.borrow_mut().push(waker.waker().clone()); + self.tasks.borrow_mut().push(cx.waker().clone()); false } } @@ -339,8 +339,8 @@ impl Sink for ManualAllow { type SinkItem = T; type SinkError = Never; - fn poll_ready(&mut self, waker: &Waker) -> Poll<(), Self::SinkError> { - if self.allow.check(waker) { + fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<(), Self::SinkError> { + if self.allow.check(cx) { Ok(Poll::Ready(())) } else { Ok(Poll::Pending) @@ -352,11 +352,11 @@ impl Sink for ManualAllow { Ok(()) } - fn poll_flush(&mut self, _: &Waker) -> Poll<(), Self::SinkError> { + fn poll_flush(&mut self, _: &mut Context<'_>) -> Poll<(), Self::SinkError> { Ok(Poll::Ready(())) } - fn poll_close(&mut self, _: &Waker) -> Poll<(), Self::SinkError> { + fn poll_close(&mut self, _: &mut Context<'_>) -> Poll<(), Self::SinkError> { Ok(Poll::Ready(())) } } @@ -380,13 +380,13 @@ fn buffer() { let sink = block_on(StartSendFut::new(sink, 0)).unwrap(); let sink = block_on(StartSendFut::new(sink, 1)).unwrap(); - flag_lw(|flag, waker| { + flag_cx(|flag, cx| { let mut task = sink.send(2); - assert!(task.poll(waker).unwrap().is_pending()); + assert!(task.poll(cx).unwrap().is_pending()); assert!(!flag.get()); allow.start(); assert!(flag.get()); - match task.poll(waker).unwrap() { + match task.poll(cx).unwrap() { Poll::Ready(sink) => { assert_eq!(sink.get_ref().data, vec![0, 1, 2]); } @@ -415,18 +415,18 @@ fn fanout_backpressure() { let sink = block_on(StartSendFut::new(sink, 0)).unwrap(); - flag_lw(|flag, waker| { + flag_cx(|flag, cx| { let mut task = sink.send(2); assert!(!flag.get()); - assert!(task.poll(waker).unwrap().is_pending()); + assert!(task.poll(cx).unwrap().is_pending()); let (item, left_recv) = block_on(left_recv.next()).unwrap(); assert_eq!(item, Some(0)); assert!(flag.get()); - assert!(task.poll(waker).unwrap().is_pending()); + assert!(task.poll(cx).unwrap().is_pending()); let (item, right_recv) = block_on(right_recv.next()).unwrap(); assert_eq!(item, Some(0)); assert!(flag.get()); - assert!(task.poll(waker).unwrap().is_ready()); + assert!(task.poll(cx).unwrap().is_ready()); // make sure receivers live until end of test to prevent send errors drop(left_recv); drop(right_recv); @@ -435,11 +435,11 @@ fn fanout_backpressure() { #[test] fn map_err() { - panic_waker_lw(|waker| { + panic_waker_cx(|cx| { let (tx, _rx) = mpsc::channel(1); let mut tx = tx.sink_map_err(|_| ()); assert_eq!(tx.start_send(()), Ok(())); - assert_eq!(tx.poll_flush(waker), Ok(Poll::Ready(()))); + assert_eq!(tx.poll_flush(cx), Ok(Poll::Ready(()))); }); let tx = mpsc::channel(0).0; @@ -457,11 +457,11 @@ impl From for FromErrTest { #[test] fn from_err() { - panic_waker_lw(|lw| { + panic_waker_cx(|cx| { let (tx, _rx) = mpsc::channel(1); let mut tx: SinkErrInto, FromErrTest> = tx.sink_err_into(); assert_eq!(tx.start_send(()), Ok(())); - assert_eq!(tx.poll_flush(lw), Ok(Poll::Ready(()))); + assert_eq!(tx.poll_flush(cx), Ok(Poll::Ready(()))); }); let tx = mpsc::channel(0).0; diff --git a/futures/tests_disabled/stream.rs b/futures/tests_disabled/stream.rs index 612450e1a3..9a27c08f75 100644 --- a/futures/tests_disabled/stream.rs +++ b/futures/tests_disabled/stream.rs @@ -25,7 +25,7 @@ impl Stream for Iter type Item = T; type Error = E; - fn poll_next(&mut self, _: &Waker) -> Poll, E> { + fn poll_next(&mut self, _: &mut Context<'_>) -> Poll, E> { match self.iter.next() { Some(Ok(e)) => Ok(Poll::Ready(Some(e))), Some(Err(e)) => Err(e), @@ -288,13 +288,13 @@ fn peek() { type Item = (); type Error = u32; - fn poll(&mut self, waker: &Waker) -> Poll<(), u32> { + fn poll(&mut self, cx: &mut Context<'_>) -> Poll<(), u32> { { - let res = try_ready!(self.inner.peek(waker)); + let res = try_ready!(self.inner.peek(cx)); assert_eq!(res, Some(&1)); } - assert_eq!(self.inner.peek(waker).unwrap(), Some(&1).into()); - assert_eq!(self.inner.poll_next(waker).unwrap(), Some(1).into()); + assert_eq!(self.inner.peek(cx).unwrap(), Some(&1).into()); + assert_eq!(self.inner.poll_next(cx).unwrap(), Some(1).into()); Ok(Poll::Ready(())) } } From ad8f3c48e943f7cbd0ce7bce18544fc9428a2a35 Mon Sep 17 00:00:00 2001 From: Taiki Endo Date: Mon, 15 Apr 2019 23:34:48 +0900 Subject: [PATCH 2/2] Update to new futures_api (wake-by-ref) --- .travis.yml | 2 +- .../src/task/__internal/atomic_waker.rs | 2 +- futures-executor/benches/thread_notify.rs | 4 +- futures-executor/src/local_pool.rs | 2 +- futures-executor/src/thread_pool.rs | 2 +- futures-executor/tests/local_pool.rs | 3 +- futures-io/src/lib.rs | 12 +++--- futures-sink/src/lib.rs | 6 +-- futures-test/src/future/pending_once.rs | 2 +- futures-test/src/task/panic_waker.rs | 12 +++--- futures-test/src/task/wake_counter.rs | 4 +- futures-util/src/compat/compat01as03.rs | 2 +- futures-util/src/compat/compat03as01.rs | 6 +-- futures-util/src/future/shared.rs | 2 +- .../src/stream/futures_unordered/mod.rs | 2 +- .../src/stream/futures_unordered/task.rs | 2 +- futures-util/src/stream/select_next_some.rs | 2 +- futures-util/src/task/arc_wake.rs | 43 ++++++++++++++----- futures-util/src/task/mod.rs | 1 + futures-util/src/task/noop_waker.rs | 11 +++-- futures-util/src/task/waker_ref.rs | 7 +-- 21 files changed, 75 insertions(+), 54 deletions(-) diff --git a/.travis.yml b/.travis.yml index 6be96649d1..39a2fc1f24 100644 --- a/.travis.yml +++ b/.travis.yml @@ -25,7 +25,7 @@ matrix: # When updating this, the reminder to update the minimum required version in README.md. - name: cargo test (minimum required version) - rust: nightly-2019-04-08 + rust: nightly-2019-04-13 - name: cargo clippy rust: nightly diff --git a/futures-core/src/task/__internal/atomic_waker.rs b/futures-core/src/task/__internal/atomic_waker.rs index e3c9feb74d..11772ec4f4 100644 --- a/futures-core/src/task/__internal/atomic_waker.rs +++ b/futures-core/src/task/__internal/atomic_waker.rs @@ -241,7 +241,7 @@ impl AtomicWaker { // Currently in the process of waking the task, i.e., // `wake` is currently being called on the old task handle. // So, we call wake on the new waker - waker.wake(); + waker.wake_by_ref(); } state => { // In this case, a concurrent thread is holding the diff --git a/futures-executor/benches/thread_notify.rs b/futures-executor/benches/thread_notify.rs index a26ddde0cb..b317a864db 100644 --- a/futures-executor/benches/thread_notify.rs +++ b/futures-executor/benches/thread_notify.rs @@ -24,7 +24,7 @@ fn thread_yield_single_thread_one_wait(b: &mut Bencher) { Poll::Ready(()) } else { self.rem -= 1; - cx.waker().wake(); + cx.waker().wake_by_ref(); Poll::Pending } } @@ -52,7 +52,7 @@ fn thread_yield_single_thread_many_wait(b: &mut Bencher) { Poll::Ready(()) } else { self.rem -= 1; - cx.waker().wake(); + cx.waker().wake_by_ref(); Poll::Pending } } diff --git a/futures-executor/src/local_pool.rs b/futures-executor/src/local_pool.rs index f729db6dea..9bdd7941cb 100644 --- a/futures-executor/src/local_pool.rs +++ b/futures-executor/src/local_pool.rs @@ -50,7 +50,7 @@ thread_local! { } impl ArcWake for ThreadNotify { - fn wake(arc_self: &Arc) { + fn wake_by_ref(arc_self: &Arc) { arc_self.thread.unpark(); } } diff --git a/futures-executor/src/thread_pool.rs b/futures-executor/src/thread_pool.rs index 227db93df1..c1c89a7007 100644 --- a/futures-executor/src/thread_pool.rs +++ b/futures-executor/src/thread_pool.rs @@ -339,7 +339,7 @@ impl fmt::Debug for Task { } impl ArcWake for WakeHandle { - fn wake(arc_self: &Arc) { + fn wake_by_ref(arc_self: &Arc) { match arc_self.mutex.notify() { Ok(task) => arc_self.exec.state.send(Message::Run(task)), Err(()) => {} diff --git a/futures-executor/tests/local_pool.rs b/futures-executor/tests/local_pool.rs index 2bebc832ad..c5eceea8a8 100644 --- a/futures-executor/tests/local_pool.rs +++ b/futures-executor/tests/local_pool.rs @@ -147,7 +147,7 @@ fn tasks_are_scheduled_fairly() { return Poll::Ready(()); } - cx.waker().wake(); + cx.waker().wake_by_ref(); Poll::Pending } } @@ -167,4 +167,3 @@ fn tasks_are_scheduled_fairly() { pool.run(); } - diff --git a/futures-io/src/lib.rs b/futures-io/src/lib.rs index 5e8662b5ff..39cd686940 100644 --- a/futures-io/src/lib.rs +++ b/futures-io/src/lib.rs @@ -100,7 +100,7 @@ mod if_std { /// /// If no data is available for reading, the method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `cx.waker().wake()`) to receive a notification when the object becomes + /// `cx.waker().wake_by_ref()`) to receive a notification when the object becomes /// readable or is closed. /// /// # Implementation @@ -122,7 +122,7 @@ mod if_std { /// /// If no data is available for reading, the method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `cx.waker().wake()`) to receive a notification when the object becomes + /// `cx.waker().wake_by_ref()`) to receive a notification when the object becomes /// readable or is closed. /// By default, this method delegates to using `poll_read` on the first /// buffer in `vec`. Objects which support vectored IO should override @@ -160,7 +160,7 @@ mod if_std { /// /// If the object is not ready for writing, the method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `cx.waker().wake()`) to receive a notification when the object becomes + /// `cx.waker().wake_by_ref()`) to receive a notification when the object becomes /// readable or is closed. /// /// # Implementation @@ -182,7 +182,7 @@ mod if_std { /// /// If the object is not ready for writing, the method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `cx.waker().wake()`) to receive a notification when the object becomes + /// `cx.waker().wake_by_ref()`) to receive a notification when the object becomes /// readable or is closed. /// /// By default, this method delegates to using `poll_write` on the first @@ -213,7 +213,7 @@ mod if_std { /// /// If flushing cannot immediately complete, this method returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `cx.waker().wake()`) to receive a notification when the object can make + /// `cx.waker().wake_by_ref()`) to receive a notification when the object can make /// progress towards flushing. /// /// # Implementation @@ -230,7 +230,7 @@ mod if_std { /// /// If closing cannot immediately complete, this function returns /// `Ok(Poll::Pending)` and arranges for the current task (via - /// `cx.waker().wake()`) to receive a notification when the object can make + /// `cx.waker().wake_by_ref()`) to receive a notification when the object can make /// progress towards closing. /// /// # Implementation diff --git a/futures-sink/src/lib.rs b/futures-sink/src/lib.rs index 3bb1316b52..b0a89bd808 100644 --- a/futures-sink/src/lib.rs +++ b/futures-sink/src/lib.rs @@ -59,7 +59,7 @@ pub trait Sink { /// /// This method returns `Poll::Ready` once the underlying sink is ready to /// receive data. If this method returns `Poll::Pending`, the current task - /// is registered to be notified (via `cx.waker().wake()`) when `poll_ready` + /// is registered to be notified (via `cx.waker().wake_by_ref()`) when `poll_ready` /// should be called again. /// /// In most cases, if the sink encounters an error, the sink will @@ -95,7 +95,7 @@ pub trait Sink { /// via `start_send` have been flushed. /// /// Returns `Ok(Poll::Pending)` if there is more work left to do, in which - /// case the current task is scheduled (via `cx.waker().wake()`) to wake up when + /// case the current task is scheduled (via `cx.waker().wake_by_ref()`) to wake up when /// `poll_flush` should be called again. /// /// In most cases, if the sink encounters an error, the sink will @@ -108,7 +108,7 @@ pub trait Sink { /// has been successfully closed. /// /// Returns `Ok(Poll::Pending)` if there is more work left to do, in which - /// case the current task is scheduled (via `cx.waker().wake()`) to wake up when + /// case the current task is scheduled (via `cx.waker().wake_by_ref()`) to wake up when /// `poll_close` should be called again. /// /// If this function encounters an error, the sink should be considered to diff --git a/futures-test/src/future/pending_once.rs b/futures-test/src/future/pending_once.rs index fd3f02743e..47bbdf5a0b 100644 --- a/futures-test/src/future/pending_once.rs +++ b/futures-test/src/future/pending_once.rs @@ -39,7 +39,7 @@ impl Future for PendingOnce { self.as_mut().future().poll(cx) } else { *self.as_mut().polled_before() = true; - cx.waker().wake(); + cx.waker().wake_by_ref(); Poll::Pending } } diff --git a/futures-test/src/task/panic_waker.rs b/futures-test/src/task/panic_waker.rs index c9692d9415..6e5295cc37 100644 --- a/futures-test/src/task/panic_waker.rs +++ b/futures-test/src/task/panic_waker.rs @@ -2,20 +2,20 @@ use futures_core::task::{Waker, RawWaker, RawWakerVTable}; use core::cell::UnsafeCell; use core::ptr::null; -unsafe fn clone_panic_waker(_data: *const()) -> RawWaker { +unsafe fn clone_panic_waker(_data: *const ()) -> RawWaker { raw_panic_waker() } -unsafe fn noop(_data: *const()) { -} +unsafe fn noop(_data: *const ()) {} -unsafe fn wake_panic(_data: *const()) { +unsafe fn wake_panic(_data: *const ()) { panic!("should not be woken"); } const PANIC_WAKER_VTABLE: RawWakerVTable = RawWakerVTable::new( clone_panic_waker, wake_panic, + wake_panic, noop, ); @@ -37,7 +37,7 @@ fn raw_panic_waker() -> RawWaker { /// waker.wake(); // Will panic /// ``` pub fn panic_waker() -> Waker { - unsafe { Waker::new_unchecked(raw_panic_waker()) } + unsafe { Waker::from_raw(raw_panic_waker()) } } /// Get a global reference to a @@ -52,7 +52,7 @@ pub fn panic_waker() -> Waker { /// use futures_test::task::panic_waker_ref; /// /// let waker = panic_waker_ref(); -/// waker.wake(); // Will panic +/// waker.wake_by_ref(); // Will panic /// ``` pub fn panic_waker_ref() -> &'static Waker { thread_local! { diff --git a/futures-test/src/task/wake_counter.rs b/futures-test/src/task/wake_counter.rs index e1c85812db..e4d5df188f 100644 --- a/futures-test/src/task/wake_counter.rs +++ b/futures-test/src/task/wake_counter.rs @@ -30,7 +30,7 @@ struct WakerInner { } impl ArcWake for WakerInner { - fn wake(arc_self: &Arc) { + fn wake_by_ref(arc_self: &Arc) { let _ = arc_self.count.fetch_add(1, Ordering::SeqCst); } } @@ -49,7 +49,7 @@ impl ArcWake for WakerInner { /// /// assert_eq!(count, 0); /// -/// waker.wake(); +/// waker.wake_by_ref(); /// waker.wake(); /// /// assert_eq!(count, 2); diff --git a/futures-util/src/compat/compat01as03.rs b/futures-util/src/compat/compat01as03.rs index 5eeb231883..36e6b5d27c 100644 --- a/futures-util/src/compat/compat01as03.rs +++ b/futures-util/src/compat/compat01as03.rs @@ -316,7 +316,7 @@ impl<'a> From> for NotifyHandle01 { impl Notify01 for NotifyWaker { fn notify(&self, _: usize) { - self.0.wake(); + self.0.wake_by_ref(); } } diff --git a/futures-util/src/compat/compat03as01.rs b/futures-util/src/compat/compat03as01.rs index f1978ccdbe..400b3996a9 100644 --- a/futures-util/src/compat/compat03as01.rs +++ b/futures-util/src/compat/compat03as01.rs @@ -174,15 +174,15 @@ impl Current { } let ptr = current_to_ptr(self); - let vtable = &RawWakerVTable::new(clone, wake, drop); + let vtable = &RawWakerVTable::new(clone, wake, wake, drop); unsafe { - WakerRef::new(task03::Waker::new_unchecked(RawWaker::new(ptr, vtable))) + WakerRef::new(task03::Waker::from_raw(RawWaker::new(ptr, vtable))) } } } impl ArcWake03 for Current { - fn wake(arc_self: &Arc) { + fn wake_by_ref(arc_self: &Arc) { arc_self.0.notify(); } } diff --git a/futures-util/src/future/shared.rs b/futures-util/src/future/shared.rs index a92341e794..2651e87ab5 100644 --- a/futures-util/src/future/shared.rs +++ b/futures-util/src/future/shared.rs @@ -315,7 +315,7 @@ where } impl ArcWake for Notifier { - fn wake(arc_self: &Arc) { + fn wake_by_ref(arc_self: &Arc) { arc_self.state.compare_and_swap(POLLING, REPOLL, SeqCst); let wakers = &mut *arc_self.wakers.lock().unwrap(); diff --git a/futures-util/src/stream/futures_unordered/mod.rs b/futures-util/src/stream/futures_unordered/mod.rs index a648e0e1ed..84ebb029d9 100644 --- a/futures-util/src/stream/futures_unordered/mod.rs +++ b/futures-util/src/stream/futures_unordered/mod.rs @@ -311,7 +311,7 @@ impl Stream for FuturesUnordered { // At this point, it may be worth yielding the thread & // spinning a few times... but for now, just yield using the // task system. - cx.waker().wake(); + cx.waker().wake_by_ref(); return Poll::Pending; } Dequeue::Data(task) => task, diff --git a/futures-util/src/stream/futures_unordered/task.rs b/futures-util/src/stream/futures_unordered/task.rs index cbfec0c7ea..1ea4d3c040 100644 --- a/futures-util/src/stream/futures_unordered/task.rs +++ b/futures-util/src/stream/futures_unordered/task.rs @@ -36,7 +36,7 @@ unsafe impl Send for Task {} unsafe impl Sync for Task {} impl ArcWake for Task { - fn wake(arc_self: &Arc) { + fn wake_by_ref(arc_self: &Arc) { let inner = match arc_self.ready_to_run_queue.upgrade() { Some(inner) => inner, None => return, diff --git a/futures-util/src/stream/select_next_some.rs b/futures-util/src/stream/select_next_some.rs index 565d0201eb..6063e8e0f4 100644 --- a/futures-util/src/stream/select_next_some.rs +++ b/futures-util/src/stream/select_next_some.rs @@ -34,7 +34,7 @@ impl<'a, St: Stream + FusedStream + Unpin> Future for SelectNextSome<'a, St> { Poll::Ready(item) } else { debug_assert!(self.stream.is_terminated()); - cx.waker().wake(); + cx.waker().wake_by_ref(); Poll::Pending } } diff --git a/futures-util/src/task/arc_wake.rs b/futures-util/src/task/arc_wake.rs index f7333e06b6..cf594b6a7a 100644 --- a/futures-util/src/task/arc_wake.rs +++ b/futures-util/src/task/arc_wake.rs @@ -17,7 +17,22 @@ pub trait ArcWake { /// /// Executors generally maintain a queue of "ready" tasks; `wake` should place /// the associated task onto this queue. - fn wake(arc_self: &Arc); + fn wake(self: Arc) { + Self::wake_by_ref(&self) + } + + /// Indicates that the associated task is ready to make progress and should + /// be `poll`ed. + /// + /// This function can be called from an arbitrary thread, including threads which + /// did not create the `ArcWake` based `Waker`. + /// + /// Executors generally maintain a queue of "ready" tasks; `wake_by_ref` should place + /// the associated task onto this queue. + /// + /// This function is similar to `wake`, but must not consume the provided data + /// pointer. + fn wake_by_ref(arc_self: &Arc); /// Creates a `Waker` from an Arc, if T implements `ArcWake`. /// @@ -25,10 +40,10 @@ pub trait ArcWake { /// the `wake()` function that is defined inside this trait will get called. fn into_waker(self: Arc) -> Waker where Self: Sized { - let ptr = Arc::into_raw(self) as *const(); + let ptr = Arc::into_raw(self) as *const (); unsafe { - Waker::new_unchecked(RawWaker::new(ptr, waker_vtable!(Self))) + Waker::from_raw(RawWaker::new(ptr, waker_vtable!(Self))) } } } @@ -36,7 +51,7 @@ pub trait ArcWake { // FIXME: panics on Arc::clone / refcount changes could wreak havoc on the // code here. We should guard against this by aborting. -unsafe fn increase_refcount(data: *const()) { +unsafe fn increase_refcount(data: *const ()) { // Retain Arc by creating a copy let arc: Arc = Arc::from_raw(data as *const T); let arc_clone = arc.clone(); @@ -46,19 +61,25 @@ unsafe fn increase_refcount(data: *const()) { } // used by `waker_ref` -pub(super) unsafe fn clone_arc_raw(data: *const()) -> RawWaker { +pub(super) unsafe fn clone_arc_raw(data: *const ()) -> RawWaker { increase_refcount::(data); RawWaker::new(data, waker_vtable!(T)) } -unsafe fn drop_arc_raw(data: *const()) { +unsafe fn drop_arc_raw(data: *const ()) { drop(Arc::::from_raw(data as *const T)) } // used by `waker_ref` -pub(super) unsafe fn wake_arc_raw(data: *const()) { +pub(super) unsafe fn wake_arc_raw(data: *const ()) { + let arc: Arc = Arc::from_raw(data as *const T); + ArcWake::wake(arc); +} + +// used by `waker_ref` +pub(super) unsafe fn wake_by_ref_arc_raw(data: *const ()) { let arc: Arc = Arc::from_raw(data as *const T); - ArcWake::wake(&arc); + ArcWake::wake_by_ref(&arc); mem::forget(arc); } @@ -84,7 +105,7 @@ mod tests { } impl ArcWake for CountingWaker { - fn wake(arc_self: &Arc) { + fn wake_by_ref(arc_self: &Arc) { let mut lock = arc_self.nr_wake.lock().unwrap(); *lock += 1; } @@ -96,13 +117,13 @@ mod tests { let w1: Waker = ArcWake::into_waker(some_w.clone()); assert_eq!(2, Arc::strong_count(&some_w)); - w1.wake(); + w1.wake_by_ref(); assert_eq!(1, some_w.wakes()); let w2 = w1.clone(); assert_eq!(3, Arc::strong_count(&some_w)); - w2.wake(); + w2.wake_by_ref(); assert_eq!(2, some_w.wakes()); drop(w2); diff --git a/futures-util/src/task/mod.rs b/futures-util/src/task/mod.rs index 28139ac6b1..f8069386d0 100644 --- a/futures-util/src/task/mod.rs +++ b/futures-util/src/task/mod.rs @@ -9,6 +9,7 @@ cfg_target_has_atomic! { &RawWakerVTable::new( clone_arc_raw::<$ty>, wake_arc_raw::<$ty>, + wake_by_ref_arc_raw::<$ty>, drop_arc_raw::<$ty>, ) }; diff --git a/futures-util/src/task/noop_waker.rs b/futures-util/src/task/noop_waker.rs index 7eb9c55aed..1378c13cec 100644 --- a/futures-util/src/task/noop_waker.rs +++ b/futures-util/src/task/noop_waker.rs @@ -4,14 +4,13 @@ use core::ptr::null; #[cfg(feature = "std")] use core::cell::UnsafeCell; -unsafe fn noop_clone(_data: *const()) -> RawWaker { +unsafe fn noop_clone(_data: *const ()) -> RawWaker { noop_raw_waker() } -unsafe fn noop(_data: *const()) { -} +unsafe fn noop(_data: *const ()) {} -const NOOP_WAKER_VTABLE: RawWakerVTable = RawWakerVTable::new(noop_clone, noop, noop); +const NOOP_WAKER_VTABLE: RawWakerVTable = RawWakerVTable::new(noop_clone, noop, noop, noop); fn noop_raw_waker() -> RawWaker { RawWaker::new(null(), &NOOP_WAKER_VTABLE) @@ -32,7 +31,7 @@ fn noop_raw_waker() -> RawWaker { #[inline] pub fn noop_waker() -> Waker { unsafe { - Waker::new_unchecked(noop_raw_waker()) + Waker::from_raw(noop_raw_waker()) } } @@ -46,7 +45,7 @@ pub fn noop_waker() -> Waker { /// #![feature(futures_api)] /// use futures::task::noop_waker_ref; /// let waker = noop_waker_ref(); -/// waker.wake(); +/// waker.wake_by_ref(); /// ``` #[inline] #[cfg(feature = "std")] diff --git a/futures-util/src/task/waker_ref.rs b/futures-util/src/task/waker_ref.rs index e8dc82c535..c4e779826d 100644 --- a/futures-util/src/task/waker_ref.rs +++ b/futures-util/src/task/waker_ref.rs @@ -1,6 +1,6 @@ #![allow(clippy::cast_ptr_alignment)] // clippy is too strict here -use super::arc_wake::{ArcWake, clone_arc_raw, wake_arc_raw}; +use super::arc_wake::{ArcWake, clone_arc_raw, wake_arc_raw, wake_by_ref_arc_raw}; use alloc::sync::Arc; use core::marker::PhantomData; use core::ops::Deref; @@ -53,18 +53,19 @@ where { // This uses the same mechanism as Arc::into_raw, without needing a reference. // This is potentially not stable - let ptr = &*wake as &W as *const W as *const(); + let ptr = &*wake as &W as *const W as *const (); // Similar to `waker_vtable`, but with a no-op `drop` function. // Clones of the resulting `RawWaker` will still be dropped normally. let vtable = &RawWakerVTable::new( clone_arc_raw::, wake_arc_raw::, + wake_by_ref_arc_raw::, noop, ); let waker = unsafe { - Waker::new_unchecked(RawWaker::new(ptr, vtable)) + Waker::from_raw(RawWaker::new(ptr, vtable)) }; WakerRef::new(waker) }