From f13ea3ebcacdd822c3ec35aa13068e4d1fa4bd96 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sat, 1 Jul 2023 01:54:58 -0400 Subject: [PATCH 01/36] Method recv_many returns a Vec for mpsc channels Circumstances may exist where multiple messages are queued for the receiver by the time the receiving task runs -- and the receiving task itself may benefit from batch as opposed to item-by-item processing, e.g., update a row based on a primary key but only the most recent update is needed; recv_many enables the receiver to process messages last-first and skip earlier updates for rows already updated. The change involved introducing a peek function to sync/mpsc/list.rs; it is based on fn pop. fn recv_many returns Poll> contrasting recv's Poll>. The new function returns Ready(vec![]) where recv returned Ready(None). The mpsc.rs trait Semaphore was extended with two functions, num_acquired() and add_permits(n) -- which are used to size the vector and update the underlying semaphores in a single call. Simple tests were added to tests/sync_mpsc.rs. --- tokio/src/sync/mpsc/bounded.rs | 14 +++-- tokio/src/sync/mpsc/chan.rs | 92 ++++++++++++++++++++++++++++++++ tokio/src/sync/mpsc/list.rs | 16 ++++++ tokio/src/sync/mpsc/unbounded.rs | 13 ++++- tokio/tests/sync_mpsc.rs | 30 +++++++++++ 5 files changed, 160 insertions(+), 5 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 8924dc22292..f518e4c29ec 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -44,10 +44,10 @@ pub struct Sender { /// async fn main() { /// let (tx, _rx) = channel::(15); /// let tx_weak = tx.downgrade(); -/// +/// /// // Upgrading will succeed because `tx` still exists. /// assert!(tx_weak.upgrade().is_some()); -/// +/// /// // If we drop `tx`, then it will fail. /// drop(tx); /// assert!(tx_weak.clone().upgrade().is_none()); @@ -229,6 +229,14 @@ impl Receiver { use crate::future::poll_fn; poll_fn(|cx| self.chan.recv(cx)).await } + /// Receives the all available values for this receiver. + /// + /// Returns an empty vector if has been closed and there are + /// no remaining messages in the channel's buffer. + pub async fn recv_many(&mut self) -> Vec { + use crate::future::poll_fn; + poll_fn(|cx| self.chan.recv_many(cx)).await + } /// Tries to receive the next value for this receiver. /// @@ -1072,7 +1080,7 @@ impl Sender { /// #[tokio::main] /// async fn main() { /// let (tx, _rx) = mpsc::channel::<()>(5); - /// + /// /// // both max capacity and capacity are the same at first /// assert_eq!(tx.max_capacity(), 5); /// assert_eq!(tx.capacity(), 5); diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index c7c0caf6c0e..97759bcbeb9 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -41,9 +41,13 @@ pub(crate) trait Semaphore { fn add_permit(&self); + fn add_permits(&self, n:usize); + fn close(&self); fn is_closed(&self) -> bool; + + fn num_acquired(&self) -> usize; } pub(super) struct Chan { @@ -293,6 +297,72 @@ impl Rx { }) } + pub(crate) fn recv_many(&mut self, cx: &mut Context<'_>) -> Poll> { + use super::block::Read::*; + + ready!(crate::trace::trace_leaf(cx)); + + // Keep track of task budget + let coop = ready!(crate::runtime::coop::poll_proceed(cx)); + + self.inner.rx_fields.with_mut(|rx_fields_ptr| { + let rx_fields = unsafe { &mut *rx_fields_ptr }; + + macro_rules! try_recv { + () => { + match rx_fields.list.pop(&self.inner.tx) { + Some(Value(value)) => { + let capacity = self.inner.semaphore.num_acquired(); + let mut result : Vec = Vec::with_capacity( capacity ); + result.push(value); + let mut next = rx_fields.list.peek(); + while (result.len() < capacity && match next { + Some(Value(value)) => { + rx_fields.list.pop(&self.inner.tx); + result.push(value); + next = rx_fields.list.peek(); + true } + _ => false + }) {} + self.inner.semaphore.add_permits(result.len()); + coop.made_progress(); + return Ready(result); + } + Some(Closed) => { + // TODO: This check may not be required as it most + // likely can only return `true` at this point. A + // channel is closed when all tx handles are + // dropped. Dropping a tx handle releases memory, + // which ensures that if dropping the tx handle is + // visible, then all messages sent are also visible. + assert!(self.inner.semaphore.is_idle()); + coop.made_progress(); + return Ready(vec![]); + } + None => {} // fall through + } + }; + } + + try_recv!(); + + self.inner.rx_waker.register_by_ref(cx.waker()); + + // It is possible that a value was pushed between attempting to read + // and registering the task, so we have to check the channel a + // second time here. + try_recv!(); + + if rx_fields.rx_closed && self.inner.semaphore.is_idle() { + coop.made_progress(); + Ready(vec![]) + } else { + Pending + } + }) + } + + /// Try to receive the next value. pub(crate) fn try_recv(&mut self) -> Result { use super::list::TryPopResult; @@ -389,6 +459,10 @@ impl Semaphore for bounded::Semaphore { self.semaphore.release(1) } + fn add_permits(&self, n:usize) { + self.semaphore.release(n) + } + fn is_idle(&self) -> bool { self.semaphore.available_permits() == self.bound } @@ -400,6 +474,10 @@ impl Semaphore for bounded::Semaphore { fn is_closed(&self) -> bool { self.semaphore.is_closed() } + + fn num_acquired(&self) -> usize { + self.bound - self.semaphore.available_permits() + } } // ===== impl Semaphore for AtomicUsize ===== @@ -414,6 +492,15 @@ impl Semaphore for unbounded::Semaphore { } } + fn add_permits(&self, n:usize) { + let prev = self.0.fetch_sub(n << 1, Release); + + if prev >> 1 == 0 { + // Something went wrong + process::abort(); + } + } + fn is_idle(&self) -> bool { self.0.load(Acquire) >> 1 == 0 } @@ -425,4 +512,9 @@ impl Semaphore for unbounded::Semaphore { fn is_closed(&self) -> bool { self.0.load(Acquire) & 1 == 1 } + + fn num_acquired(&self) -> usize { + self.0.load(Acquire) >> 1 + } + } diff --git a/tokio/src/sync/mpsc/list.rs b/tokio/src/sync/mpsc/list.rs index 10b29575bdb..7869db82fc1 100644 --- a/tokio/src/sync/mpsc/list.rs +++ b/tokio/src/sync/mpsc/list.rs @@ -252,6 +252,22 @@ impl Rx { } } + /// Peeks at the next value on the queue, if any. + pub(crate) fn peek(&mut self) -> Option> { + // Advance `head`, if needed + if !self.try_advancing_head() { + return None; + } + + unsafe { + let block = self.head.as_ref(); + + let ret = block.read(self.index); + + ret + } + } + /// Pops the next value off the queue, detecting whether the block /// is busy or empty on failure. /// diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index cd83fc12513..c03e44964f7 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -34,10 +34,10 @@ pub struct UnboundedSender { /// async fn main() { /// let (tx, _rx) = unbounded_channel::(); /// let tx_weak = tx.downgrade(); -/// +/// /// // Upgrading will succeed because `tx` still exists. /// assert!(tx_weak.upgrade().is_some()); -/// +/// /// // If we drop `tx`, then it will fail. /// drop(tx); /// assert!(tx_weak.clone().upgrade().is_none()); @@ -172,6 +172,15 @@ impl UnboundedReceiver { poll_fn(|cx| self.poll_recv(cx)).await } + /// Receives the all available values for this receiver. + /// + /// Returns an empty vector if has been closed and there are + /// no remaining messages in the channel's buffer. + pub async fn recv_many(&mut self) -> Vec { + use crate::future::poll_fn; + poll_fn(|cx| self.chan.recv_many(cx)).await + } + /// Tries to receive the next value for this receiver. /// /// This method returns the [`Empty`] error if the channel is currently diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index d2b7078b4ea..df652abd473 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -120,6 +120,21 @@ async fn async_send_recv_with_buffer() { assert_eq!(None, rx.recv().await); } +#[tokio::test] +#[cfg(feature = "full")] +async fn async_send_recv_many_with_buffer() { + let (tx, mut rx) = mpsc::channel(16); + + tokio::spawn(async move { + assert_ok!(tx.send(1).await); + assert_ok!(tx.send(2).await); + }); + + assert_eq!(vec![1, 2], rx.recv_many().await); + assert_eq!(None, rx.recv().await); +} + + #[tokio::test] #[cfg(feature = "full")] async fn start_send_past_cap() { @@ -176,6 +191,21 @@ async fn send_recv_unbounded() { assert!(rx.recv().await.is_none()); } +#[maybe_tokio_test] +async fn send_recv_many_unbounded() { + let (tx, mut rx) = mpsc::unbounded_channel::(); + + // Using `try_send` + assert_ok!(tx.send(1)); + assert_ok!(tx.send(2)); + + assert_eq!(rx.recv_many().await, vec![1,2]); + + drop(tx); + + assert!(rx.recv().await.is_none()); +} + #[tokio::test] #[cfg(feature = "full")] async fn async_send_recv_unbounded() { From 01334251828613c646160411804748519c302c8f Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Fri, 8 Sep 2023 23:36:46 -0400 Subject: [PATCH 02/36] Update recv_many to populate a passed-in buffer. The implementation uses a passed-in buffer to return results. The buffer is always cleared, and its capacity may be increased to match the number of queued messages. This avoids excessive memory allocation. This commit also adds new benchmarks in benches/sync_mpsc_recv_many.rs It borrows substantially from benches/sync_mpsc.rs The benchmarks are framed around the idea that only the last value is needed, e.g., what might be typical in a dashboard. The following benchmarks compare extracting the last value from 5_000 recv's vs. a variable number of recv_many: contention_bounded_updater_recv vs. contention_bounded_updater_recv_many contention_bounded_full_updater_recv vs. contention_bounded_full_updater_recv_many contention_unbounded_updater_recv vs. contention_unbounded_updater_recv_many uncontented_bounded_updater_recv vs. uncontented_bounded_updater_recv_many uncontented_unbounded_updater_recv vs. uncontented_unbounded_updater_recv_many Initial tests suggests a ~30% performance improvement for the uncontented tests. The benchmarks contention_bounded_updater_publish_recv and contention_bounded_updater_publish_recv_many are different; first the number of messages recevied are only 1_000. In these test, we contemplate an actor who needs to receive a message and forward it over an API that takes 1ns to complete. An actor accepting a single message-at-a-time will hit the 1ns lag repeatedly; the actor that can pick only the last message out of the buffer from recv_many has much less work. --- benches/Cargo.toml | 5 + benches/sync_mpsc_recv_many.rs | 382 +++++++++++++++++++++++++++++++ tokio/src/sync/mpsc/bounded.rs | 10 +- tokio/src/sync/mpsc/chan.rs | 23 +- tokio/src/sync/mpsc/unbounded.rs | 10 +- tokio/tests/sync_mpsc.rs | 34 ++- 6 files changed, 439 insertions(+), 25 deletions(-) create mode 100644 benches/sync_mpsc_recv_many.rs diff --git a/benches/Cargo.toml b/benches/Cargo.toml index e0b162b422e..488333e146c 100644 --- a/benches/Cargo.toml +++ b/benches/Cargo.toml @@ -30,6 +30,11 @@ name = "sync_mpsc" path = "sync_mpsc.rs" harness = false +[[bench]] +name = "sync_mpsc_recv_many" +path = "sync_mpsc_recv_many.rs" +harness = false + [[bench]] name = "sync_mpsc_oneshot" path = "sync_mpsc_oneshot.rs" diff --git a/benches/sync_mpsc_recv_many.rs b/benches/sync_mpsc_recv_many.rs new file mode 100644 index 00000000000..4f94d2f9514 --- /dev/null +++ b/benches/sync_mpsc_recv_many.rs @@ -0,0 +1,382 @@ +use bencher::Bencher; +use tokio::sync::mpsc; + +fn rt() -> tokio::runtime::Runtime { + tokio::runtime::Builder::new_multi_thread() + .worker_threads(6) + .build() + .unwrap() +} + +// Simulate a use case of an actor that must update +// a resource, but resource only needs last value +fn publish_last_value(last_value:usize) -> usize { + std::thread::sleep(std::time::Duration::from_nanos(1)); + last_value +} + +fn contention_bounded_updater_recv(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(1_000_000); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).await.unwrap(); + } + }); + } + + let mut last_value = 0usize; + for _ in 0..1_000 * 5 { + let Some(v) = rx.recv().await else {continue}; + last_value = v + } + last_value + }) + }); +} + +fn contention_bounded_updater_recv_many(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(1_000_000); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).await.unwrap(); + } + }); + } + + let mut last_value = 0usize; + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + let count = rx.recv_many(&mut buffer).await; + total += count; + if count > 0 { + last_value = buffer[buffer.len()-1] + } + } + last_value + }) + }); +} + + +fn contention_bounded_updater_publish_recv(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(1_000_000); + + for _ in 0..1 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).await.unwrap(); + } + }); + } + + for _ in 0..1_000 { + let Some(v) = rx.recv().await else {continue}; + let _ = publish_last_value(v); + } + }) + }); +} + +fn contention_bounded_updater_publish_recv_many(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(1_000_000); + + for _ in 0..1 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).await.unwrap(); + } + }); + } + + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 1 { + let count = rx.recv_many(&mut buffer).await; + total += count; + if count > 0 { + publish_last_value(buffer[buffer.len()-1]); + } + } + }) + }); +} + +fn contention_bounded_full_updater_recv(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(100); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).await.unwrap(); + } + }); + } + + let mut last_value = 0usize; + for _ in 0..1_000 * 5 { + let Some(v) = rx.recv().await else {continue}; + last_value = v + } + last_value + }) + }); +} + +fn contention_bounded_full_updater_recv_many(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(100); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).await.unwrap(); + } + }); + } + + let mut last_value = 0usize; + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + let count = rx.recv_many(&mut buffer).await; + total += count; + if count > 0 { + last_value = buffer[buffer.len()-1] + } + } + last_value + }) + }); +} + +fn contention_unbounded_updater_recv(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::unbounded_channel::(); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).unwrap(); + } + }); + } + + let mut last_value = 0usize; + for _ in 0..1_000 * 5 { + let Some(v) = rx.recv().await else {continue}; + last_value = v + } + last_value + }) + }); +} + +fn contention_unbounded_updater_recv_many(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::unbounded_channel::(); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).unwrap(); + } + }); + } + + let mut last_value = 0usize; + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + let count = rx.recv_many(&mut buffer).await; + total += count; + if count > 0 { + last_value = buffer[buffer.len()-1] + } + } + last_value + }) + }); +} + +fn uncontented_bounded_updater_recv(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(1_000_000); + + for i in 0..5000 { + tx.send(i).await.unwrap(); + } + + let mut last_value = 0usize; + for _ in 0..5_000 { + let Some(v) = rx.recv().await else {continue}; + last_value = v + } + last_value + }) + }); +} + +fn uncontented_bounded_updater_recv_many(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(1_000_000); + + for i in 0..5000 { + tx.send(i).await.unwrap(); + } + + let mut last_value = 0usize; + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + let count = rx.recv_many(&mut buffer).await; + total += count; + if count > 0 { + last_value = buffer[buffer.len()-1] + } + } + last_value + }) + }); +} + + +fn uncontented_unbounded_updater_recv(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::unbounded_channel::(); + + for i in 0..5000 { + tx.send(i).unwrap(); + } + + let mut last_value = 0usize; + for _ in 0..5_000 { + let Some(v) = rx.recv().await else {continue}; + last_value = v + } + last_value + }) + }); +} + +fn uncontented_unbounded_updater_recv_many(b: &mut Bencher) { + let rt = rt(); + + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::unbounded_channel::(); + + for i in 0..5000 { + tx.send(i).unwrap(); + } + + let mut last_value = 0usize; + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + let count = rx.recv_many(&mut buffer).await; + total += count; + if count > 0 { + last_value = buffer[buffer.len()-1] + } + } + last_value + }) + }); +} + +bencher::benchmark_group!( + contention_bounded_updater, + contention_bounded_updater_recv, + contention_bounded_updater_recv_many +); + +bencher::benchmark_group!( + contention_bounded_updater_publish, + contention_bounded_updater_publish_recv, + contention_bounded_updater_publish_recv_many +); + + +bencher::benchmark_group!( + contention_bounded_full_updater, + contention_bounded_full_updater_recv, + contention_bounded_full_updater_recv_many +); + +bencher::benchmark_group!( + contention_unbounded_updater, + contention_unbounded_updater_recv, + contention_unbounded_updater_recv_many +); + +bencher::benchmark_group!( + uncontented_bounded_updater, + uncontented_bounded_updater_recv, + uncontented_bounded_updater_recv_many +); + +bencher::benchmark_group!( + uncontented_unbounded_updater, + uncontented_unbounded_updater_recv, + uncontented_unbounded_updater_recv_many +); + +bencher::benchmark_main!( + contention_bounded_updater, + contention_bounded_updater_publish, + contention_bounded_full_updater, + contention_unbounded_updater, + uncontented_bounded_updater, + uncontented_unbounded_updater +); diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index f518e4c29ec..8f9ea1677d0 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -229,13 +229,13 @@ impl Receiver { use crate::future::poll_fn; poll_fn(|cx| self.chan.recv(cx)).await } - /// Receives the all available values for this receiver. + /// Receives the all available values for this receiver /// - /// Returns an empty vector if has been closed and there are - /// no remaining messages in the channel's buffer. - pub async fn recv_many(&mut self) -> Vec { + /// Returns the number of elements populated in the passed-in + /// result buffer. The capacity of the buffer is not reduced. + pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { use crate::future::poll_fn; - poll_fn(|cx| self.chan.recv_many(cx)).await + poll_fn(|cx| self.chan.recv_many(cx, buffer)).await } /// Tries to receive the next value for this receiver. diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index 97759bcbeb9..5d89806747f 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -297,7 +297,8 @@ impl Rx { }) } - pub(crate) fn recv_many(&mut self, cx: &mut Context<'_>) -> Poll> { + /// Receives available values into result buffer whose capacity increases as needed + pub(crate) fn recv_many(&mut self, cx: &mut Context<'_>, buffer: &mut Vec) -> Poll { use super::block::Read::*; ready!(crate::trace::trace_leaf(cx)); @@ -305,6 +306,8 @@ impl Rx { // Keep track of task budget let coop = ready!(crate::runtime::coop::poll_proceed(cx)); + buffer.clear(); + self.inner.rx_fields.with_mut(|rx_fields_ptr| { let rx_fields = unsafe { &mut *rx_fields_ptr }; @@ -313,20 +316,22 @@ impl Rx { match rx_fields.list.pop(&self.inner.tx) { Some(Value(value)) => { let capacity = self.inner.semaphore.num_acquired(); - let mut result : Vec = Vec::with_capacity( capacity ); - result.push(value); + if buffer.capacity() < capacity { + buffer.reserve(capacity); + } + buffer.push(value); let mut next = rx_fields.list.peek(); - while (result.len() < capacity && match next { + while (buffer.len() < capacity && match next { Some(Value(value)) => { rx_fields.list.pop(&self.inner.tx); - result.push(value); + buffer.push(value); next = rx_fields.list.peek(); true } _ => false }) {} - self.inner.semaphore.add_permits(result.len()); + self.inner.semaphore.add_permits(buffer.len()); coop.made_progress(); - return Ready(result); + return Ready(buffer.len()); } Some(Closed) => { // TODO: This check may not be required as it most @@ -337,7 +342,7 @@ impl Rx { // visible, then all messages sent are also visible. assert!(self.inner.semaphore.is_idle()); coop.made_progress(); - return Ready(vec![]); + return Ready(0usize); } None => {} // fall through } @@ -355,7 +360,7 @@ impl Rx { if rx_fields.rx_closed && self.inner.semaphore.is_idle() { coop.made_progress(); - Ready(vec![]) + Ready(0usize) } else { Pending } diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index c03e44964f7..03d896bd853 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -172,13 +172,13 @@ impl UnboundedReceiver { poll_fn(|cx| self.poll_recv(cx)).await } - /// Receives the all available values for this receiver. + /// Receives the all available values for this receiver /// - /// Returns an empty vector if has been closed and there are - /// no remaining messages in the channel's buffer. - pub async fn recv_many(&mut self) -> Vec { + /// Returns the number of elements populated in the passed-in + /// result buffer. The capacity of the buffer is not reduced. + pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { use crate::future::poll_fn; - poll_fn(|cx| self.chan.recv_many(cx)).await + poll_fn(|cx| self.chan.recv_many(cx, buffer)).await } /// Tries to receive the next value for this receiver. diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index df652abd473..586a5ebcdf8 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -123,14 +123,31 @@ async fn async_send_recv_with_buffer() { #[tokio::test] #[cfg(feature = "full")] async fn async_send_recv_many_with_buffer() { - let (tx, mut rx) = mpsc::channel(16); + let (tx, mut rx) = mpsc::channel(2); tokio::spawn(async move { assert_ok!(tx.send(1).await); assert_ok!(tx.send(2).await); + assert_ok!(tx.send(7).await); }); - assert_eq!(vec![1, 2], rx.recv_many().await); + let mut buffer = vec![0; 16]; + let mut recv_count = 0usize; + let mut sum = 0; + let mut iter = 0; + while recv_count < 3 { + let n = rx.recv_many(&mut buffer).await; + recv_count += n; + assert_eq!(buffer.len(), n); + sum += buffer.iter().sum::(); + iter += 1; + assert!(buffer.capacity() >= 16); + } + + assert_eq!(3, recv_count); + assert_eq!(10, sum); + assert!(iter > 1); + assert_eq!(None, rx.recv().await); } @@ -196,10 +213,15 @@ async fn send_recv_many_unbounded() { let (tx, mut rx) = mpsc::unbounded_channel::(); // Using `try_send` - assert_ok!(tx.send(1)); - assert_ok!(tx.send(2)); - - assert_eq!(rx.recv_many().await, vec![1,2]); + assert_ok!(tx.send(7)); + assert_ok!(tx.send(13)); + assert_ok!(tx.send(100)); + assert_ok!(tx.send(1002)); + + let mut buffer = vec![0; 0]; + assert_eq!(rx.recv_many(&mut buffer).await, 4); + assert_eq!(vec![7,13,100,1002], buffer); + assert!(buffer.capacity() >= 4); drop(tx); From 7d0d8db2ac03b3d0a5e762f9e9d9d958c23421c0 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Fri, 15 Sep 2023 18:55:22 -0400 Subject: [PATCH 03/36] fix rustfmt issues --- benches/sync_mpsc_recv_many.rs | 19 ++++++++----------- tokio/src/sync/mpsc/chan.rs | 27 ++++++++++++++------------- tokio/tests/sync_mpsc.rs | 3 +-- 3 files changed, 23 insertions(+), 26 deletions(-) diff --git a/benches/sync_mpsc_recv_many.rs b/benches/sync_mpsc_recv_many.rs index 4f94d2f9514..cfe5dc1fd81 100644 --- a/benches/sync_mpsc_recv_many.rs +++ b/benches/sync_mpsc_recv_many.rs @@ -10,7 +10,7 @@ fn rt() -> tokio::runtime::Runtime { // Simulate a use case of an actor that must update // a resource, but resource only needs last value -fn publish_last_value(last_value:usize) -> usize { +fn publish_last_value(last_value: usize) -> usize { std::thread::sleep(std::time::Duration::from_nanos(1)); last_value } @@ -64,7 +64,7 @@ fn contention_bounded_updater_recv_many(b: &mut Bencher) { let count = rx.recv_many(&mut buffer).await; total += count; if count > 0 { - last_value = buffer[buffer.len()-1] + last_value = buffer[buffer.len() - 1] } } last_value @@ -72,7 +72,6 @@ fn contention_bounded_updater_recv_many(b: &mut Bencher) { }); } - fn contention_bounded_updater_publish_recv(b: &mut Bencher) { let rt = rt(); @@ -89,7 +88,7 @@ fn contention_bounded_updater_publish_recv(b: &mut Bencher) { }); } - for _ in 0..1_000 { + for _ in 0..1_000 { let Some(v) = rx.recv().await else {continue}; let _ = publish_last_value(v); } @@ -119,7 +118,7 @@ fn contention_bounded_updater_publish_recv_many(b: &mut Bencher) { let count = rx.recv_many(&mut buffer).await; total += count; if count > 0 { - publish_last_value(buffer[buffer.len()-1]); + publish_last_value(buffer[buffer.len() - 1]); } } }) @@ -175,7 +174,7 @@ fn contention_bounded_full_updater_recv_many(b: &mut Bencher) { let count = rx.recv_many(&mut buffer).await; total += count; if count > 0 { - last_value = buffer[buffer.len()-1] + last_value = buffer[buffer.len() - 1] } } last_value @@ -232,7 +231,7 @@ fn contention_unbounded_updater_recv_many(b: &mut Bencher) { let count = rx.recv_many(&mut buffer).await; total += count; if count > 0 { - last_value = buffer[buffer.len()-1] + last_value = buffer[buffer.len() - 1] } } last_value @@ -279,7 +278,7 @@ fn uncontented_bounded_updater_recv_many(b: &mut Bencher) { let count = rx.recv_many(&mut buffer).await; total += count; if count > 0 { - last_value = buffer[buffer.len()-1] + last_value = buffer[buffer.len() - 1] } } last_value @@ -287,7 +286,6 @@ fn uncontented_bounded_updater_recv_many(b: &mut Bencher) { }); } - fn uncontented_unbounded_updater_recv(b: &mut Bencher) { let rt = rt(); @@ -327,7 +325,7 @@ fn uncontented_unbounded_updater_recv_many(b: &mut Bencher) { let count = rx.recv_many(&mut buffer).await; total += count; if count > 0 { - last_value = buffer[buffer.len()-1] + last_value = buffer[buffer.len() - 1] } } last_value @@ -347,7 +345,6 @@ bencher::benchmark_group!( contention_bounded_updater_publish_recv_many ); - bencher::benchmark_group!( contention_bounded_full_updater, contention_bounded_full_updater_recv, diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index 5d89806747f..bba8e0ccc77 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -41,7 +41,7 @@ pub(crate) trait Semaphore { fn add_permit(&self); - fn add_permits(&self, n:usize); + fn add_permits(&self, n: usize); fn close(&self); @@ -321,14 +321,17 @@ impl Rx { } buffer.push(value); let mut next = rx_fields.list.peek(); - while (buffer.len() < capacity && match next { - Some(Value(value)) => { - rx_fields.list.pop(&self.inner.tx); - buffer.push(value); - next = rx_fields.list.peek(); - true } - _ => false - }) {} + while (buffer.len() < capacity + && match next { + Some(Value(value)) => { + rx_fields.list.pop(&self.inner.tx); + buffer.push(value); + next = rx_fields.list.peek(); + true + } + _ => false, + }) + {} self.inner.semaphore.add_permits(buffer.len()); coop.made_progress(); return Ready(buffer.len()); @@ -367,7 +370,6 @@ impl Rx { }) } - /// Try to receive the next value. pub(crate) fn try_recv(&mut self) -> Result { use super::list::TryPopResult; @@ -464,7 +466,7 @@ impl Semaphore for bounded::Semaphore { self.semaphore.release(1) } - fn add_permits(&self, n:usize) { + fn add_permits(&self, n: usize) { self.semaphore.release(n) } @@ -497,7 +499,7 @@ impl Semaphore for unbounded::Semaphore { } } - fn add_permits(&self, n:usize) { + fn add_permits(&self, n: usize) { let prev = self.0.fetch_sub(n << 1, Release); if prev >> 1 == 0 { @@ -521,5 +523,4 @@ impl Semaphore for unbounded::Semaphore { fn num_acquired(&self) -> usize { self.0.load(Acquire) >> 1 } - } diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index 586a5ebcdf8..fe675d81c77 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -151,7 +151,6 @@ async fn async_send_recv_many_with_buffer() { assert_eq!(None, rx.recv().await); } - #[tokio::test] #[cfg(feature = "full")] async fn start_send_past_cap() { @@ -220,7 +219,7 @@ async fn send_recv_many_unbounded() { let mut buffer = vec![0; 0]; assert_eq!(rx.recv_many(&mut buffer).await, 4); - assert_eq!(vec![7,13,100,1002], buffer); + assert_eq!(vec![7, 13, 100, 1002], buffer); assert!(buffer.capacity() >= 4); drop(tx); From 487f887d5a8a0c6728be1d016377a224acdc4c71 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Fri, 15 Sep 2023 18:58:16 -0400 Subject: [PATCH 04/36] fix returning the result of a binding from a block --- tokio/src/sync/mpsc/list.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tokio/src/sync/mpsc/list.rs b/tokio/src/sync/mpsc/list.rs index 7869db82fc1..9cb15517304 100644 --- a/tokio/src/sync/mpsc/list.rs +++ b/tokio/src/sync/mpsc/list.rs @@ -262,9 +262,7 @@ impl Rx { unsafe { let block = self.head.as_ref(); - let ret = block.read(self.index); - - ret + block.read(self.index) } } From 7086b34868e2fa66167f690d33721fa77900102b Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Fri, 15 Sep 2023 19:07:38 -0400 Subject: [PATCH 05/36] fix rustfmt braces around continue --- benches/sync_mpsc_recv_many.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/benches/sync_mpsc_recv_many.rs b/benches/sync_mpsc_recv_many.rs index cfe5dc1fd81..44b9e8b65fe 100644 --- a/benches/sync_mpsc_recv_many.rs +++ b/benches/sync_mpsc_recv_many.rs @@ -33,7 +33,7 @@ fn contention_bounded_updater_recv(b: &mut Bencher) { let mut last_value = 0usize; for _ in 0..1_000 * 5 { - let Some(v) = rx.recv().await else {continue}; + let Some(v) = rx.recv().await else { continue }; last_value = v } last_value @@ -89,7 +89,7 @@ fn contention_bounded_updater_publish_recv(b: &mut Bencher) { } for _ in 0..1_000 { - let Some(v) = rx.recv().await else {continue}; + let Some(v) = rx.recv().await else { continue }; let _ = publish_last_value(v); } }) @@ -143,7 +143,7 @@ fn contention_bounded_full_updater_recv(b: &mut Bencher) { let mut last_value = 0usize; for _ in 0..1_000 * 5 { - let Some(v) = rx.recv().await else {continue}; + let Some(v) = rx.recv().await else { continue }; last_value = v } last_value @@ -200,7 +200,7 @@ fn contention_unbounded_updater_recv(b: &mut Bencher) { let mut last_value = 0usize; for _ in 0..1_000 * 5 { - let Some(v) = rx.recv().await else {continue}; + let Some(v) = rx.recv().await else { continue }; last_value = v } last_value @@ -252,7 +252,7 @@ fn uncontented_bounded_updater_recv(b: &mut Bencher) { let mut last_value = 0usize; for _ in 0..5_000 { - let Some(v) = rx.recv().await else {continue}; + let Some(v) = rx.recv().await else { continue }; last_value = v } last_value @@ -299,7 +299,7 @@ fn uncontented_unbounded_updater_recv(b: &mut Bencher) { let mut last_value = 0usize; for _ in 0..5_000 { - let Some(v) = rx.recv().await else {continue}; + let Some(v) = rx.recv().await else { continue }; last_value = v } last_value From 9a170b0856ec9fcec921e4e5a827d4d059015b35 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Fri, 15 Sep 2023 20:12:23 -0400 Subject: [PATCH 06/36] Update from bencher and merge benchmarks into sync_mpsc.rs Previous benchmarks in sync_mpsc_recv_many.rs included a contrived example that is removed. --- benches/sync_mpsc.rs | 130 +++++++++++ benches/sync_mpsc_recv_many.rs | 379 --------------------------------- 2 files changed, 130 insertions(+), 379 deletions(-) delete mode 100644 benches/sync_mpsc_recv_many.rs diff --git a/benches/sync_mpsc.rs b/benches/sync_mpsc.rs index d6545e8047f..7b07d9373c8 100644 --- a/benches/sync_mpsc.rs +++ b/benches/sync_mpsc.rs @@ -73,6 +73,33 @@ fn contention_bounded(g: &mut BenchmarkGroup) { }); } +fn contention_bounded_recv_many(g: &mut BenchmarkGroup) { + let rt = rt(); + + g.bench_function("bounded_recv_many", |b| { + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(1_000_000); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).await.unwrap(); + } + }); + } + + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + total += rx.recv_many(&mut buffer).await; + } + }) + }) + }); +} + fn contention_bounded_full(g: &mut BenchmarkGroup) { let rt = rt(); @@ -98,6 +125,33 @@ fn contention_bounded_full(g: &mut BenchmarkGroup) { }); } +fn contention_bounded_full_recv_many(g: &mut BenchmarkGroup) { + let rt = rt(); + + g.bench_function("bounded_full_recv_many", |b| { + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(100); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).await.unwrap(); + } + }); + } + + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + total += rx.recv_many(&mut buffer).await; + } + }) + }) + }); +} + fn contention_unbounded(g: &mut BenchmarkGroup) { let rt = rt(); @@ -123,6 +177,33 @@ fn contention_unbounded(g: &mut BenchmarkGroup) { }); } +fn contention_unbounded_recv_many(g: &mut BenchmarkGroup) { + let rt = rt(); + + g.bench_function("unbounded_recv_many", |b| { + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::unbounded_channel::(); + + for _ in 0..5 { + let tx = tx.clone(); + tokio::spawn(async move { + for i in 0..1000 { + tx.send(i).unwrap(); + } + }); + } + + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + total += rx.recv_many(&mut buffer).await; + } + }) + }) + }); +} + fn uncontented_bounded(g: &mut BenchmarkGroup) { let rt = rt(); @@ -143,6 +224,28 @@ fn uncontented_bounded(g: &mut BenchmarkGroup) { }); } +fn uncontented_bounded_recv_many(g: &mut BenchmarkGroup) { + let rt = rt(); + + g.bench_function("bounded_recv_many", |b| { + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::channel::(1_000_000); + + for i in 0..5000 { + tx.send(i).await.unwrap(); + } + + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + total += rx.recv_many(&mut buffer).await; + } + }) + }) + }); +} + fn uncontented_unbounded(g: &mut BenchmarkGroup) { let rt = rt(); @@ -163,6 +266,28 @@ fn uncontented_unbounded(g: &mut BenchmarkGroup) { }); } +fn uncontented_unbounded_recv_many(g: &mut BenchmarkGroup) { + let rt = rt(); + + g.bench_function("unbounded_recv_many", |b| { + b.iter(|| { + rt.block_on(async move { + let (tx, mut rx) = mpsc::unbounded_channel::(); + + for i in 0..5000 { + tx.send(i).unwrap(); + } + + let mut buffer = Vec::::with_capacity(5_000); + let mut total = 0; + while total < 1_000 * 5 { + total += rx.recv_many(&mut buffer).await; + } + }) + }) + }); +} + fn bench_create_medium(c: &mut Criterion) { let mut group = c.benchmark_group("create_medium"); create_medium::<1>(&mut group); @@ -181,15 +306,20 @@ fn bench_send(c: &mut Criterion) { fn bench_contention(c: &mut Criterion) { let mut group = c.benchmark_group("contention"); contention_bounded(&mut group); + contention_bounded_recv_many(&mut group); contention_bounded_full(&mut group); + contention_bounded_full_recv_many(&mut group); contention_unbounded(&mut group); + contention_unbounded_recv_many(&mut group); group.finish(); } fn bench_uncontented(c: &mut Criterion) { let mut group = c.benchmark_group("uncontented"); uncontented_bounded(&mut group); + uncontented_bounded_recv_many(&mut group); uncontented_unbounded(&mut group); + uncontented_unbounded_recv_many(&mut group); group.finish(); } diff --git a/benches/sync_mpsc_recv_many.rs b/benches/sync_mpsc_recv_many.rs deleted file mode 100644 index 44b9e8b65fe..00000000000 --- a/benches/sync_mpsc_recv_many.rs +++ /dev/null @@ -1,379 +0,0 @@ -use bencher::Bencher; -use tokio::sync::mpsc; - -fn rt() -> tokio::runtime::Runtime { - tokio::runtime::Builder::new_multi_thread() - .worker_threads(6) - .build() - .unwrap() -} - -// Simulate a use case of an actor that must update -// a resource, but resource only needs last value -fn publish_last_value(last_value: usize) -> usize { - std::thread::sleep(std::time::Duration::from_nanos(1)); - last_value -} - -fn contention_bounded_updater_recv(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::channel::(1_000_000); - - for _ in 0..5 { - let tx = tx.clone(); - tokio::spawn(async move { - for i in 0..1000 { - tx.send(i).await.unwrap(); - } - }); - } - - let mut last_value = 0usize; - for _ in 0..1_000 * 5 { - let Some(v) = rx.recv().await else { continue }; - last_value = v - } - last_value - }) - }); -} - -fn contention_bounded_updater_recv_many(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::channel::(1_000_000); - - for _ in 0..5 { - let tx = tx.clone(); - tokio::spawn(async move { - for i in 0..1000 { - tx.send(i).await.unwrap(); - } - }); - } - - let mut last_value = 0usize; - let mut buffer = Vec::::with_capacity(5_000); - let mut total = 0; - while total < 1_000 * 5 { - let count = rx.recv_many(&mut buffer).await; - total += count; - if count > 0 { - last_value = buffer[buffer.len() - 1] - } - } - last_value - }) - }); -} - -fn contention_bounded_updater_publish_recv(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::channel::(1_000_000); - - for _ in 0..1 { - let tx = tx.clone(); - tokio::spawn(async move { - for i in 0..1000 { - tx.send(i).await.unwrap(); - } - }); - } - - for _ in 0..1_000 { - let Some(v) = rx.recv().await else { continue }; - let _ = publish_last_value(v); - } - }) - }); -} - -fn contention_bounded_updater_publish_recv_many(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::channel::(1_000_000); - - for _ in 0..1 { - let tx = tx.clone(); - tokio::spawn(async move { - for i in 0..1000 { - tx.send(i).await.unwrap(); - } - }); - } - - let mut buffer = Vec::::with_capacity(5_000); - let mut total = 0; - while total < 1_000 * 1 { - let count = rx.recv_many(&mut buffer).await; - total += count; - if count > 0 { - publish_last_value(buffer[buffer.len() - 1]); - } - } - }) - }); -} - -fn contention_bounded_full_updater_recv(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::channel::(100); - - for _ in 0..5 { - let tx = tx.clone(); - tokio::spawn(async move { - for i in 0..1000 { - tx.send(i).await.unwrap(); - } - }); - } - - let mut last_value = 0usize; - for _ in 0..1_000 * 5 { - let Some(v) = rx.recv().await else { continue }; - last_value = v - } - last_value - }) - }); -} - -fn contention_bounded_full_updater_recv_many(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::channel::(100); - - for _ in 0..5 { - let tx = tx.clone(); - tokio::spawn(async move { - for i in 0..1000 { - tx.send(i).await.unwrap(); - } - }); - } - - let mut last_value = 0usize; - let mut buffer = Vec::::with_capacity(5_000); - let mut total = 0; - while total < 1_000 * 5 { - let count = rx.recv_many(&mut buffer).await; - total += count; - if count > 0 { - last_value = buffer[buffer.len() - 1] - } - } - last_value - }) - }); -} - -fn contention_unbounded_updater_recv(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::unbounded_channel::(); - - for _ in 0..5 { - let tx = tx.clone(); - tokio::spawn(async move { - for i in 0..1000 { - tx.send(i).unwrap(); - } - }); - } - - let mut last_value = 0usize; - for _ in 0..1_000 * 5 { - let Some(v) = rx.recv().await else { continue }; - last_value = v - } - last_value - }) - }); -} - -fn contention_unbounded_updater_recv_many(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::unbounded_channel::(); - - for _ in 0..5 { - let tx = tx.clone(); - tokio::spawn(async move { - for i in 0..1000 { - tx.send(i).unwrap(); - } - }); - } - - let mut last_value = 0usize; - let mut buffer = Vec::::with_capacity(5_000); - let mut total = 0; - while total < 1_000 * 5 { - let count = rx.recv_many(&mut buffer).await; - total += count; - if count > 0 { - last_value = buffer[buffer.len() - 1] - } - } - last_value - }) - }); -} - -fn uncontented_bounded_updater_recv(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::channel::(1_000_000); - - for i in 0..5000 { - tx.send(i).await.unwrap(); - } - - let mut last_value = 0usize; - for _ in 0..5_000 { - let Some(v) = rx.recv().await else { continue }; - last_value = v - } - last_value - }) - }); -} - -fn uncontented_bounded_updater_recv_many(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::channel::(1_000_000); - - for i in 0..5000 { - tx.send(i).await.unwrap(); - } - - let mut last_value = 0usize; - let mut buffer = Vec::::with_capacity(5_000); - let mut total = 0; - while total < 1_000 * 5 { - let count = rx.recv_many(&mut buffer).await; - total += count; - if count > 0 { - last_value = buffer[buffer.len() - 1] - } - } - last_value - }) - }); -} - -fn uncontented_unbounded_updater_recv(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::unbounded_channel::(); - - for i in 0..5000 { - tx.send(i).unwrap(); - } - - let mut last_value = 0usize; - for _ in 0..5_000 { - let Some(v) = rx.recv().await else { continue }; - last_value = v - } - last_value - }) - }); -} - -fn uncontented_unbounded_updater_recv_many(b: &mut Bencher) { - let rt = rt(); - - b.iter(|| { - rt.block_on(async move { - let (tx, mut rx) = mpsc::unbounded_channel::(); - - for i in 0..5000 { - tx.send(i).unwrap(); - } - - let mut last_value = 0usize; - let mut buffer = Vec::::with_capacity(5_000); - let mut total = 0; - while total < 1_000 * 5 { - let count = rx.recv_many(&mut buffer).await; - total += count; - if count > 0 { - last_value = buffer[buffer.len() - 1] - } - } - last_value - }) - }); -} - -bencher::benchmark_group!( - contention_bounded_updater, - contention_bounded_updater_recv, - contention_bounded_updater_recv_many -); - -bencher::benchmark_group!( - contention_bounded_updater_publish, - contention_bounded_updater_publish_recv, - contention_bounded_updater_publish_recv_many -); - -bencher::benchmark_group!( - contention_bounded_full_updater, - contention_bounded_full_updater_recv, - contention_bounded_full_updater_recv_many -); - -bencher::benchmark_group!( - contention_unbounded_updater, - contention_unbounded_updater_recv, - contention_unbounded_updater_recv_many -); - -bencher::benchmark_group!( - uncontented_bounded_updater, - uncontented_bounded_updater_recv, - uncontented_bounded_updater_recv_many -); - -bencher::benchmark_group!( - uncontented_unbounded_updater, - uncontented_unbounded_updater_recv, - uncontented_unbounded_updater_recv_many -); - -bencher::benchmark_main!( - contention_bounded_updater, - contention_bounded_updater_publish, - contention_bounded_full_updater, - contention_unbounded_updater, - uncontented_bounded_updater, - uncontented_unbounded_updater -); From 01b2270e625896ed68a4d3a9d987f291c38f7cec Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Fri, 15 Sep 2023 20:44:57 -0400 Subject: [PATCH 07/36] fix Cargo.toml to reflect deletion of old benchmark --- benches/Cargo.toml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/benches/Cargo.toml b/benches/Cargo.toml index 488333e146c..e0b162b422e 100644 --- a/benches/Cargo.toml +++ b/benches/Cargo.toml @@ -30,11 +30,6 @@ name = "sync_mpsc" path = "sync_mpsc.rs" harness = false -[[bench]] -name = "sync_mpsc_recv_many" -path = "sync_mpsc_recv_many.rs" -harness = false - [[bench]] name = "sync_mpsc_oneshot" path = "sync_mpsc_oneshot.rs" From 8c8b06f009a5385866e8a094237dbda77f63fea4 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 24 Sep 2023 20:15:51 -0400 Subject: [PATCH 08/36] Simplified recv_many implementation Removed methods num_acquired and peek; allow caller to control the number of retrieved elements by setting the capacity of the result buffer. In the event the result buffer has no capacity, the capacity is set to BLOCK_CAP. Added examples to the documentation. --- tokio/src/sync/mpsc/bounded.rs | 30 +++++++++-- tokio/src/sync/mpsc/chan.rs | 86 ++++++++++++++------------------ tokio/src/sync/mpsc/list.rs | 14 ------ tokio/src/sync/mpsc/unbounded.rs | 31 ++++++++++-- 4 files changed, 93 insertions(+), 68 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 8f9ea1677d0..639e339a691 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -229,10 +229,34 @@ impl Receiver { use crate::future::poll_fn; poll_fn(|cx| self.chan.recv(cx)).await } - /// Receives the all available values for this receiver + + /// Receives the next values for this receiver and populates `buffer`. + /// + /// This method returns the number of values populated in `buffer`, + /// which is cleared on each call. + /// + /// If the buffer initially has zero capacity, the method reserves + /// `BLOCK_CAP` elements, otherwise the capacity is not changed. + /// The number of returned values can be at most the capacity of + /// the buffer. + /// + /// # Example + /// ``` + /// use tokio::sync::mpsc; /// - /// Returns the number of elements populated in the passed-in - /// result buffer. The capacity of the buffer is not reduced. + /// #[tokio::main] + /// async fn main() { + /// let (tx, mut rx) = mpsc::channel(100); + /// + /// tx.send("hello").await.unwrap(); + /// tx.send("world").await.unwrap(); + /// + /// // At most 5 at a time + /// let mut buffer : Vec<&str> = Vec::with_capacity(5); + /// assert_eq!(2, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["hello","world"], buffer); + /// } + /// ``` pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { use crate::future::poll_fn; poll_fn(|cx| self.chan.recv_many(cx, buffer)).await diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index bba8e0ccc77..a30ffd84833 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -46,8 +46,6 @@ pub(crate) trait Semaphore { fn close(&self); fn is_closed(&self) -> bool; - - fn num_acquired(&self) -> usize; } pub(super) struct Chan { @@ -297,57 +295,56 @@ impl Rx { }) } - /// Receives available values into result buffer whose capacity increases as needed + /// Receives values into `buffer` up to its capacity + /// + /// If the buffer initially has 0 capacity, reserves `super::BLOCK_CAP` elements pub(crate) fn recv_many(&mut self, cx: &mut Context<'_>, buffer: &mut Vec) -> Poll { - use super::block::Read::*; + use super::block::Read; + if buffer.capacity() == 0 { + buffer.reserve(super::BLOCK_CAP); + } + buffer.clear(); ready!(crate::trace::trace_leaf(cx)); // Keep track of task budget let coop = ready!(crate::runtime::coop::poll_proceed(cx)); - buffer.clear(); - self.inner.rx_fields.with_mut(|rx_fields_ptr| { let rx_fields = unsafe { &mut *rx_fields_ptr }; - macro_rules! try_recv { () => { - match rx_fields.list.pop(&self.inner.tx) { - Some(Value(value)) => { - let capacity = self.inner.semaphore.num_acquired(); - if buffer.capacity() < capacity { - buffer.reserve(capacity); + let mut working = true; + while (working && buffer.len() < buffer.capacity()) { + match rx_fields.list.pop(&self.inner.tx) { + Some(Read::Value(value)) => { + buffer.push(value); + } + + Some(Read::Closed) => { + if !buffer.is_empty() { + self.inner.semaphore.add_permits(buffer.len()); + } + // TODO: This check may not be required as it most + // likely can only return `true` at this point. A + // channel is closed when all tx handles are + // dropped. Dropping a tx handle releases memory, + // which ensures that if dropping the tx handle is + // visible, then all messages sent are also visible. + assert!(self.inner.semaphore.is_idle()); + coop.made_progress(); + return Ready(buffer.len()); + } + + None => { + working = false; // fall through } - buffer.push(value); - let mut next = rx_fields.list.peek(); - while (buffer.len() < capacity - && match next { - Some(Value(value)) => { - rx_fields.list.pop(&self.inner.tx); - buffer.push(value); - next = rx_fields.list.peek(); - true - } - _ => false, - }) - {} - self.inner.semaphore.add_permits(buffer.len()); - coop.made_progress(); - return Ready(buffer.len()); - } - Some(Closed) => { - // TODO: This check may not be required as it most - // likely can only return `true` at this point. A - // channel is closed when all tx handles are - // dropped. Dropping a tx handle releases memory, - // which ensures that if dropping the tx handle is - // visible, then all messages sent are also visible. - assert!(self.inner.semaphore.is_idle()); - coop.made_progress(); - return Ready(0usize); } - None => {} // fall through + } + if !buffer.is_empty() { + self.inner.semaphore.add_permits(buffer.len()); + coop.made_progress(); + return Ready(buffer.len()); } }; } @@ -362,6 +359,7 @@ impl Rx { try_recv!(); if rx_fields.rx_closed && self.inner.semaphore.is_idle() { + assert!(buffer.is_empty()); coop.made_progress(); Ready(0usize) } else { @@ -481,10 +479,6 @@ impl Semaphore for bounded::Semaphore { fn is_closed(&self) -> bool { self.semaphore.is_closed() } - - fn num_acquired(&self) -> usize { - self.bound - self.semaphore.available_permits() - } } // ===== impl Semaphore for AtomicUsize ===== @@ -519,8 +513,4 @@ impl Semaphore for unbounded::Semaphore { fn is_closed(&self) -> bool { self.0.load(Acquire) & 1 == 1 } - - fn num_acquired(&self) -> usize { - self.0.load(Acquire) >> 1 - } } diff --git a/tokio/src/sync/mpsc/list.rs b/tokio/src/sync/mpsc/list.rs index 9cb15517304..10b29575bdb 100644 --- a/tokio/src/sync/mpsc/list.rs +++ b/tokio/src/sync/mpsc/list.rs @@ -252,20 +252,6 @@ impl Rx { } } - /// Peeks at the next value on the queue, if any. - pub(crate) fn peek(&mut self) -> Option> { - // Advance `head`, if needed - if !self.try_advancing_head() { - return None; - } - - unsafe { - let block = self.head.as_ref(); - - block.read(self.index) - } - } - /// Pops the next value off the queue, detecting whether the block /// is busy or empty on failure. /// diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 03d896bd853..cf953fa012d 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -172,10 +172,35 @@ impl UnboundedReceiver { poll_fn(|cx| self.poll_recv(cx)).await } - /// Receives the all available values for this receiver + /// Receives the next values for this receiver and populates `buffer`. /// - /// Returns the number of elements populated in the passed-in - /// result buffer. The capacity of the buffer is not reduced. + /// This method returns the number of values populated in `buffer`, + /// which is cleared on each call. + /// + /// If the buffer initially has zero capacity, the method reserves + /// `BLOCK_CAP` elements, otherwise the capacity is not changed. + /// The number of returned values can be at most the capacity of + /// the buffer. + /// + /// # Example: + /// + /// ``` + /// use tokio::sync::mpsc; + /// + /// #[tokio::main] + /// async fn main() { + /// let (tx, mut rx) = mpsc::unbounded_channel(); + /// + /// tokio::spawn(async move { + /// tx.send("hello").unwrap(); + /// }); + /// + /// let mut buffer : Vec<&str> = Vec::with_capacity(5); + /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["hello"], buffer); + /// assert_eq!(0, rx.recv_many(&mut buffer).await); + /// assert!(buffer.is_empty()); + /// } pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { use crate::future::poll_fn; poll_fn(|cx| self.chan.recv_many(cx, buffer)).await From dc81be3af1750c48f4c739c54e62834e7dcefe8e Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 24 Sep 2023 22:08:33 -0400 Subject: [PATCH 09/36] Fix test send_recv_unbounded for arbitrarty BLOCK_CAP values Also, update test async_send_recv_many_with_buffer to test that buffer capacity is not increased by recv_many. --- tokio/tests/sync_mpsc.rs | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index fe675d81c77..95228351956 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -131,7 +131,8 @@ async fn async_send_recv_many_with_buffer() { assert_ok!(tx.send(7).await); }); - let mut buffer = vec![0; 16]; + let mut buffer = vec![0; 3]; + let initial_capacity = buffer.capacity(); let mut recv_count = 0usize; let mut sum = 0; let mut iter = 0; @@ -141,7 +142,7 @@ async fn async_send_recv_many_with_buffer() { assert_eq!(buffer.len(), n); sum += buffer.iter().sum::(); iter += 1; - assert!(buffer.capacity() >= 16); + assert!(buffer.capacity() == initial_capacity); } assert_eq!(3, recv_count); @@ -217,10 +218,17 @@ async fn send_recv_many_unbounded() { assert_ok!(tx.send(100)); assert_ok!(tx.send(1002)); - let mut buffer = vec![0; 0]; - assert_eq!(rx.recv_many(&mut buffer).await, 4); - assert_eq!(vec![7, 13, 100, 1002], buffer); - assert!(buffer.capacity() >= 4); + let mut buffer : Vec = Vec::new(); + assert!(buffer.capacity() == 0); + let mut count = 0; + let mut sum = 0; + while count < 4 { + count += rx.recv_many(&mut buffer).await; + sum += buffer.iter().sum::() + } + assert_eq!(count,4); + assert_eq!(sum,1122); + assert!(buffer.capacity() > 0); drop(tx); From 8f5b00c8aa8aee991af4181ae71aa2b591e57609 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 24 Sep 2023 22:17:17 -0400 Subject: [PATCH 10/36] Fix for rustfmt --- tokio/tests/sync_mpsc.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index 95228351956..6912a543e9d 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -218,7 +218,7 @@ async fn send_recv_many_unbounded() { assert_ok!(tx.send(100)); assert_ok!(tx.send(1002)); - let mut buffer : Vec = Vec::new(); + let mut buffer: Vec = Vec::new(); assert!(buffer.capacity() == 0); let mut count = 0; let mut sum = 0; @@ -226,8 +226,8 @@ async fn send_recv_many_unbounded() { count += rx.recv_many(&mut buffer).await; sum += buffer.iter().sum::() } - assert_eq!(count,4); - assert_eq!(sum,1122); + assert_eq!(count, 4); + assert_eq!(sum, 1122); assert!(buffer.capacity() > 0); drop(tx); From 3c1dc332fad70e0ddf5dd3eee21065c393ac3324 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Mon, 25 Sep 2023 22:57:17 -0400 Subject: [PATCH 11/36] switch to break to exit loop --- tokio/src/sync/mpsc/chan.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index a30ffd84833..2a9d1546ede 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -314,8 +314,7 @@ impl Rx { let rx_fields = unsafe { &mut *rx_fields_ptr }; macro_rules! try_recv { () => { - let mut working = true; - while (working && buffer.len() < buffer.capacity()) { + while (buffer.len() < buffer.capacity()) { match rx_fields.list.pop(&self.inner.tx) { Some(Read::Value(value)) => { buffer.push(value); @@ -337,7 +336,7 @@ impl Rx { } None => { - working = false; // fall through + break; // fall through } } } From 757bc3315a44505fb26ab1e7ad0b0fcd37aba84e Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Tue, 26 Sep 2023 20:52:54 -0400 Subject: [PATCH 12/36] resolve return value 0 for recv_many --- tokio/src/sync/mpsc/bounded.rs | 6 +++++- tokio/src/sync/mpsc/unbounded.rs | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 639e339a691..fd9d0433943 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -233,7 +233,11 @@ impl Receiver { /// Receives the next values for this receiver and populates `buffer`. /// /// This method returns the number of values populated in `buffer`, - /// which is cleared on each call. + /// which is cleared on each call; it returns 0 if the channel has been + /// closed and there are no remaining messages in the channel. + /// If there are no messages in the channel's buffer, but the channel has + /// not yet been closed, this method will sleep until a message is sent or + /// the channel is closed. /// /// If the buffer initially has zero capacity, the method reserves /// `BLOCK_CAP` elements, otherwise the capacity is not changed. diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index cf953fa012d..db2697ca90e 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -175,7 +175,11 @@ impl UnboundedReceiver { /// Receives the next values for this receiver and populates `buffer`. /// /// This method returns the number of values populated in `buffer`, - /// which is cleared on each call. + /// which is cleared on each call; it returns 0 if the channel has been + /// closed and there are no remaining messages in the channel. + /// If there are no messages in the channel's buffer, but the channel has + /// not yet been closed, this method will sleep until a message is sent or + /// the channel is closed. /// /// If the buffer initially has zero capacity, the method reserves /// `BLOCK_CAP` elements, otherwise the capacity is not changed. From 09488578183824cec99512df1835e943b82757c8 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Wed, 27 Sep 2023 16:52:44 -0400 Subject: [PATCH 13/36] Change to recv_many to no longer clear the output buffer This commit changes the behavior of recv_many to no longer clear the supplied buffer on each call. Now, recv_many will append new messages to the end of the supplied buffer until it reaches capacity. The return value is the number of new messages added to the buffer. This may be useful for a caller who expects a fixed number of messages and wishes to avoid copying. At the time of the call, recv_many now asserts that the capacity of the buffer exceeds its length (fail-fast). The caller must appropriately clear/resize the buffer between calls if it is to be reused. The method recv_many no longer changes the capacity of the supplied buffer. Unit and doc and doc tests were updated to test and document the new behavior. --- tokio/src/sync/mpsc/bounded.rs | 36 +++++++++++++++++++------------- tokio/src/sync/mpsc/chan.rs | 28 +++++++++++++++---------- tokio/src/sync/mpsc/unbounded.rs | 16 +++++++------- tokio/tests/sync_mpsc.rs | 36 +++++++++++++++++--------------- 4 files changed, 64 insertions(+), 52 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index fd9d0433943..28b26c26494 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -230,19 +230,17 @@ impl Receiver { poll_fn(|cx| self.chan.recv(cx)).await } - /// Receives the next values for this receiver and populates `buffer`. + /// Receives the next values for this receiver and extends `buffer`. /// - /// This method returns the number of values populated in `buffer`, - /// which is cleared on each call; it returns 0 if the channel has been - /// closed and there are no remaining messages in the channel. + /// This method returns the number of values populated in `buffer`; + /// it returns 0 if the channel has been closed and there + /// are no remaining messages in the channel. /// If there are no messages in the channel's buffer, but the channel has /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. /// - /// If the buffer initially has zero capacity, the method reserves - /// `BLOCK_CAP` elements, otherwise the capacity is not changed. - /// The number of returned values can be at most the capacity of - /// the buffer. + /// Asserts that the passed-in buffer has capacity greater than its + /// length. /// /// # Example /// ``` @@ -252,13 +250,21 @@ impl Receiver { /// async fn main() { /// let (tx, mut rx) = mpsc::channel(100); /// - /// tx.send("hello").await.unwrap(); - /// tx.send("world").await.unwrap(); - /// - /// // At most 5 at a time - /// let mut buffer : Vec<&str> = Vec::with_capacity(5); - /// assert_eq!(2, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["hello","world"], buffer); + /// tx.send("first").await.unwrap(); + /// tx.send("second").await.unwrap(); + /// tx.send("third").await.unwrap(); + /// + /// // At most 2 at a time + /// let mut buffer : Vec<&str> = Vec::with_capacity(2); + /// assert_eq!(2, rx.recv_many(&mut buffer).await, "did not fill to capacity"); + /// assert_eq!(vec!["first","second"], buffer); + /// buffer.clear(); // Reuse the buffer + /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["third"], buffer); + /// tx.send("fourth").await.unwrap(); + /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["third","fourth"], buffer); + /// // assert_eq!(2, rx.recv_many(&mut buffer).await); // 'buffer must have non-zero unused capacity' /// } /// ``` pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index 2a9d1546ede..d8b5e1b5104 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -300,13 +300,15 @@ impl Rx { /// If the buffer initially has 0 capacity, reserves `super::BLOCK_CAP` elements pub(crate) fn recv_many(&mut self, cx: &mut Context<'_>, buffer: &mut Vec) -> Poll { use super::block::Read; - if buffer.capacity() == 0 { - buffer.reserve(super::BLOCK_CAP); - } - buffer.clear(); - ready!(crate::trace::trace_leaf(cx)); + assert!( + buffer.capacity() > buffer.len(), + "buffer must have non-zero unused capacity" + ); + + let initial_length = buffer.len(); + ready!(crate::trace::trace_leaf(cx)); // Keep track of task budget let coop = ready!(crate::runtime::coop::poll_proceed(cx)); @@ -321,8 +323,9 @@ impl Rx { } Some(Read::Closed) => { - if !buffer.is_empty() { - self.inner.semaphore.add_permits(buffer.len()); + let number_added = buffer.len() - initial_length; + if number_added > 0 { + self.inner.semaphore.add_permits(number_added); } // TODO: This check may not be required as it most // likely can only return `true` at this point. A @@ -332,7 +335,7 @@ impl Rx { // visible, then all messages sent are also visible. assert!(self.inner.semaphore.is_idle()); coop.made_progress(); - return Ready(buffer.len()); + return Ready(number_added); } None => { @@ -340,10 +343,13 @@ impl Rx { } } } - if !buffer.is_empty() { - self.inner.semaphore.add_permits(buffer.len()); + let number_added = buffer.len() - initial_length; + if number_added > 0 { + if number_added > 0 { + self.inner.semaphore.add_permits(number_added); + } coop.made_progress(); - return Ready(buffer.len()); + return Ready(number_added); } }; } diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index db2697ca90e..87ecde458c2 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -172,19 +172,17 @@ impl UnboundedReceiver { poll_fn(|cx| self.poll_recv(cx)).await } - /// Receives the next values for this receiver and populates `buffer`. + /// Receives the next values for this receiver and extends `buffer`. /// - /// This method returns the number of values populated in `buffer`, - /// which is cleared on each call; it returns 0 if the channel has been - /// closed and there are no remaining messages in the channel. + /// This method returns the number of values populated in `buffer`; + /// it returns 0 if the channel has been closed and there + /// are no remaining messages in the channel. /// If there are no messages in the channel's buffer, but the channel has /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. /// - /// If the buffer initially has zero capacity, the method reserves - /// `BLOCK_CAP` elements, otherwise the capacity is not changed. - /// The number of returned values can be at most the capacity of - /// the buffer. + /// Asserts that the passed-in buffer has capacity greater than its + /// length. /// /// # Example: /// @@ -203,7 +201,7 @@ impl UnboundedReceiver { /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["hello"], buffer); /// assert_eq!(0, rx.recv_many(&mut buffer).await); - /// assert!(buffer.is_empty()); + /// // assert_eq!(vec!["hello"], buffer); /// } pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { use crate::future::poll_fn; diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index 6912a543e9d..6f62a85544f 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -120,6 +120,17 @@ async fn async_send_recv_with_buffer() { assert_eq!(None, rx.recv().await); } +#[tokio::test] +#[cfg(feature = "full")] +#[should_panic(expected = "buffer must have non-zero unused capacity")] +async fn async_send_recv_many_zero() { + let (tx, mut rx) = mpsc::channel(2); + assert_ok!(tx.send(7).await); + let mut buffer = vec![0; 0]; + assert_eq!(0, buffer.capacity()); + assert_eq!(1, rx.recv_many(&mut buffer).await); +} + #[tokio::test] #[cfg(feature = "full")] async fn async_send_recv_many_with_buffer() { @@ -131,24 +142,17 @@ async fn async_send_recv_many_with_buffer() { assert_ok!(tx.send(7).await); }); - let mut buffer = vec![0; 3]; - let initial_capacity = buffer.capacity(); + let mut buffer = Vec::::with_capacity(3); let mut recv_count = 0usize; - let mut sum = 0; - let mut iter = 0; while recv_count < 3 { let n = rx.recv_many(&mut buffer).await; recv_count += n; - assert_eq!(buffer.len(), n); - sum += buffer.iter().sum::(); - iter += 1; - assert!(buffer.capacity() == initial_capacity); + assert_eq!(buffer.len(), recv_count); + assert!(buffer.iter().sum::() <= 10); } - + assert_eq!(buffer.iter().sum::(), 10); assert_eq!(3, recv_count); - assert_eq!(10, sum); - assert!(iter > 1); - + assert_eq!(recv_count, buffer.len()); assert_eq!(None, rx.recv().await); } @@ -218,16 +222,14 @@ async fn send_recv_many_unbounded() { assert_ok!(tx.send(100)); assert_ok!(tx.send(1002)); - let mut buffer: Vec = Vec::new(); - assert!(buffer.capacity() == 0); + let mut buffer: Vec = Vec::with_capacity(4); let mut count = 0; - let mut sum = 0; while count < 4 { count += rx.recv_many(&mut buffer).await; - sum += buffer.iter().sum::() } assert_eq!(count, 4); - assert_eq!(sum, 1122); + assert_eq!(count, buffer.len()); + assert_eq!(buffer.iter().sum::(), 1122); assert!(buffer.capacity() > 0); drop(tx); From 1dc2dbf5ec260ee3ac03d37d41978201abfd01ba Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Wed, 27 Sep 2023 20:24:38 -0400 Subject: [PATCH 14/36] Filled buffer handling now fail-safe, not fail-fast Implements approach of Darksonn, where the buffer is extended when the capacity is equal to the length at the time of the call to recv_many. --- tokio/src/sync/mpsc/bounded.rs | 11 ++++++----- tokio/src/sync/mpsc/chan.rs | 9 ++++----- tokio/src/sync/mpsc/unbounded.rs | 5 ++--- tokio/tests/sync_mpsc.rs | 13 +------------ 4 files changed, 13 insertions(+), 25 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 28b26c26494..dd1522ed4c3 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -239,8 +239,8 @@ impl Receiver { /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. /// - /// Asserts that the passed-in buffer has capacity greater than its - /// length. + /// If at the time of the call the buffer has no unused capacity, + /// `BLOCK_CAP` additional elements are reserved. /// /// # Example /// ``` @@ -256,15 +256,16 @@ impl Receiver { /// /// // At most 2 at a time /// let mut buffer : Vec<&str> = Vec::with_capacity(2); - /// assert_eq!(2, rx.recv_many(&mut buffer).await, "did not fill to capacity"); + /// assert_eq!(2, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first","second"], buffer); /// buffer.clear(); // Reuse the buffer /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["third"], buffer); /// tx.send("fourth").await.unwrap(); + /// tx.send("fifth").await.unwrap(); /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["third","fourth"], buffer); - /// // assert_eq!(2, rx.recv_many(&mut buffer).await); // 'buffer must have non-zero unused capacity' + /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["third","fourth","fifth"], buffer); /// } /// ``` pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index d8b5e1b5104..db464eda81f 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -297,14 +297,13 @@ impl Rx { /// Receives values into `buffer` up to its capacity /// - /// If the buffer initially has 0 capacity, reserves `super::BLOCK_CAP` elements + /// If the buffer initially has 0 remaining capacity, reserves `super::BLOCK_CAP` elements pub(crate) fn recv_many(&mut self, cx: &mut Context<'_>, buffer: &mut Vec) -> Poll { use super::block::Read; - assert!( - buffer.capacity() > buffer.len(), - "buffer must have non-zero unused capacity" - ); + if buffer.len() == buffer.capacity() { + buffer.reserve(super::BLOCK_CAP); + } let initial_length = buffer.len(); diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 87ecde458c2..49855eeb5c7 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -181,8 +181,8 @@ impl UnboundedReceiver { /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. /// - /// Asserts that the passed-in buffer has capacity greater than its - /// length. + /// If at the time of the call the buffer has no unused capacity, + /// `BLOCK_CAP` additional elements are reserved. /// /// # Example: /// @@ -201,7 +201,6 @@ impl UnboundedReceiver { /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["hello"], buffer); /// assert_eq!(0, rx.recv_many(&mut buffer).await); - /// // assert_eq!(vec!["hello"], buffer); /// } pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { use crate::future::poll_fn; diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index 6f62a85544f..e57f559abc4 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -120,17 +120,6 @@ async fn async_send_recv_with_buffer() { assert_eq!(None, rx.recv().await); } -#[tokio::test] -#[cfg(feature = "full")] -#[should_panic(expected = "buffer must have non-zero unused capacity")] -async fn async_send_recv_many_zero() { - let (tx, mut rx) = mpsc::channel(2); - assert_ok!(tx.send(7).await); - let mut buffer = vec![0; 0]; - assert_eq!(0, buffer.capacity()); - assert_eq!(1, rx.recv_many(&mut buffer).await); -} - #[tokio::test] #[cfg(feature = "full")] async fn async_send_recv_many_with_buffer() { @@ -222,7 +211,7 @@ async fn send_recv_many_unbounded() { assert_ok!(tx.send(100)); assert_ok!(tx.send(1002)); - let mut buffer: Vec = Vec::with_capacity(4); + let mut buffer: Vec = Vec::with_capacity(0); let mut count = 0; while count < 4 { count += rx.recv_many(&mut buffer).await; From 6bf430cde27e16b04c649381f87f382462b6620f Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sat, 30 Sep 2023 00:39:53 -0400 Subject: [PATCH 15/36] improved doctest example --- tokio/src/sync/mpsc/bounded.rs | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index dd1522ed4c3..75c46bb2559 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -254,16 +254,27 @@ impl Receiver { /// tx.send("second").await.unwrap(); /// tx.send("third").await.unwrap(); /// - /// // At most 2 at a time + /// // The initial capacity allows up to 2 messages + /// // to be placed in the buffer. /// let mut buffer : Vec<&str> = Vec::with_capacity(2); /// assert_eq!(2, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first","second"], buffer); - /// buffer.clear(); // Reuse the buffer + /// + /// buffer.clear(); /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["third"], buffer); + /// /// tx.send("fourth").await.unwrap(); /// tx.send("fifth").await.unwrap(); + /// + /// // As the passed-in buffer has unused capacity + /// // recv_many will use it up, but it will not + /// // increase the buffer's capacity. /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(2, buffer.capacity()); + /// + /// // With the buffer full, the next call to recv_many + /// // reserves additional capacity. /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["third","fourth","fifth"], buffer); /// } From cdc36659677f717c39bfd3e63f36d801699c166c Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sat, 30 Sep 2023 13:16:22 -0400 Subject: [PATCH 16/36] fixed nested if; resolve documentation issues for recv_many --- tokio/src/sync/mpsc/bounded.rs | 38 +++++++++++++++++++++++--------- tokio/src/sync/mpsc/chan.rs | 4 +--- tokio/src/sync/mpsc/unbounded.rs | 29 ++++++++++++++++++------ 3 files changed, 51 insertions(+), 20 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 75c46bb2559..1cc8757a8c3 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -232,17 +232,35 @@ impl Receiver { /// Receives the next values for this receiver and extends `buffer`. /// - /// This method returns the number of values populated in `buffer`; - /// it returns 0 if the channel has been closed and there - /// are no remaining messages in the channel. - /// If there are no messages in the channel's buffer, but the channel has + /// This method returns the number of values populated in `buffer`. + /// + /// This method returns 0 if the channel has been closed and there are + /// no remaining messages in the channel's queue. This indicates that no + /// further values can ever be received from this `Receiver`. The channel is + /// closed when all senders have been dropped, or when [`close`] is called. + /// + /// If there are no messages in the channel's queue, but the channel has /// not yet been closed, this method will sleep until a message is sent or - /// the channel is closed. + /// the channel is closed. Note that if [`close`] is called, but there are + /// still outstanding [`Permits`] from before it was closed, the channel is + /// not considered closed by `recv` until the permits are released. + /// + /// If at the time of the call `buffer` has no unused capacity, + /// additional elements are reserved. Otherwise `recv_many` populates + /// `buffer` with no more elements than its unused capacity. /// - /// If at the time of the call the buffer has no unused capacity, - /// `BLOCK_CAP` additional elements are reserved. + /// # Cancel safety + /// + /// This method is cancel safe. If `recv` is used as the event in a + /// [`tokio::select!`](crate::select) statement and some other branch + /// completes first, it is guaranteed that no messages were received on this + /// channel. + /// + /// [`close`]: Self::close + /// [`Permits`]: struct@crate::sync::mpsc::Permit/// + /// + /// # Examples /// - /// # Example /// ``` /// use tokio::sync::mpsc; /// @@ -268,12 +286,12 @@ impl Receiver { /// tx.send("fifth").await.unwrap(); /// /// // As the passed-in buffer has unused capacity - /// // recv_many will use it up, but it will not + /// // recv_many will use it up; it will not /// // increase the buffer's capacity. /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(2, buffer.capacity()); /// - /// // With the buffer full, the next call to recv_many + /// // With the buffer full, the next call to `recv_many` /// // reserves additional capacity. /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["third","fourth","fifth"], buffer); diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index db464eda81f..2500990f757 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -344,9 +344,7 @@ impl Rx { } let number_added = buffer.len() - initial_length; if number_added > 0 { - if number_added > 0 { - self.inner.semaphore.add_permits(number_added); - } + self.inner.semaphore.add_permits(number_added); coop.made_progress(); return Ready(number_added); } diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 49855eeb5c7..f63ea339e63 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -174,17 +174,31 @@ impl UnboundedReceiver { /// Receives the next values for this receiver and extends `buffer`. /// - /// This method returns the number of values populated in `buffer`; - /// it returns 0 if the channel has been closed and there - /// are no remaining messages in the channel. - /// If there are no messages in the channel's buffer, but the channel has + /// This method returns the number of values populated in `buffer`. + /// + /// This method returns 0 if the channel has been closed and there are + /// no remaining messages in the channel's queue. This indicates that no + /// further values can ever be received from this `Receiver`. The channel is + /// closed when all senders have been dropped, or when [`close`] is called. + /// + /// If there are no messages in the channel's queue, but the channel has /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. /// - /// If at the time of the call the buffer has no unused capacity, - /// `BLOCK_CAP` additional elements are reserved. + /// If at the time of the call `buffer` has no unused capacity, + /// additional elements are reserved. Otherwise `recv_many` populates + /// `buffer` with no more elements than its unused capacity. + /// + /// # Cancel safety + /// + /// This method is cancel safe. If `recv` is used as the event in a + /// [`tokio::select!`](crate::select) statement and some other branch + /// completes first, it is guaranteed that no messages were received on this + /// channel. /// - /// # Example: + /// [`close`]: Self::close + /// + /// # Examples /// /// ``` /// use tokio::sync::mpsc; @@ -202,6 +216,7 @@ impl UnboundedReceiver { /// assert_eq!(vec!["hello"], buffer); /// assert_eq!(0, rx.recv_many(&mut buffer).await); /// } + /// ``` pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { use crate::future::poll_fn; poll_fn(|cx| self.chan.recv_many(cx, buffer)).await From 4c1321f11d1974245f32714e6526470c3bc7390d Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sat, 30 Sep 2023 22:05:15 -0400 Subject: [PATCH 17/36] Update tokio/src/sync/mpsc/unbounded.rs Co-authored-by: Alice Ryhl --- tokio/src/sync/mpsc/unbounded.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index f63ea339e63..127d5bd83e8 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -174,7 +174,7 @@ impl UnboundedReceiver { /// Receives the next values for this receiver and extends `buffer`. /// - /// This method returns the number of values populated in `buffer`. + /// This method returns the number of values added to `buffer`. /// /// This method returns 0 if the channel has been closed and there are /// no remaining messages in the channel's queue. This indicates that no From 27ae226e4cc7a5ab08720ef3477b0c921d079ac6 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sat, 30 Sep 2023 22:05:38 -0400 Subject: [PATCH 18/36] Update tokio/src/sync/mpsc/unbounded.rs Co-authored-by: Alice Ryhl --- tokio/src/sync/mpsc/unbounded.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 127d5bd83e8..2efe2222a80 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -191,7 +191,7 @@ impl UnboundedReceiver { /// /// # Cancel safety /// - /// This method is cancel safe. If `recv` is used as the event in a + /// This method is cancel safe. If `recv_many` is used as the event in a /// [`tokio::select!`](crate::select) statement and some other branch /// completes first, it is guaranteed that no messages were received on this /// channel. From 9a1d9bcf7eff9979788cb321e75bc16d81adcf69 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 1 Oct 2023 00:36:40 -0400 Subject: [PATCH 19/36] Updated doctests to be similar; clarified documentation --- tokio/src/sync/mpsc/bounded.rs | 72 +++++++++++++++----------------- tokio/src/sync/mpsc/unbounded.rs | 47 ++++++++++++++------- 2 files changed, 65 insertions(+), 54 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 1cc8757a8c3..26dc0a2c9d9 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -232,12 +232,7 @@ impl Receiver { /// Receives the next values for this receiver and extends `buffer`. /// - /// This method returns the number of values populated in `buffer`. - /// - /// This method returns 0 if the channel has been closed and there are - /// no remaining messages in the channel's queue. This indicates that no - /// further values can ever be received from this `Receiver`. The channel is - /// closed when all senders have been dropped, or when [`close`] is called. + /// This method returns the number of values added to `buffer`. /// /// If there are no messages in the channel's queue, but the channel has /// not yet been closed, this method will sleep until a message is sent or @@ -245,13 +240,19 @@ impl Receiver { /// still outstanding [`Permits`] from before it was closed, the channel is /// not considered closed by `recv` until the permits are released. /// - /// If at the time of the call `buffer` has no unused capacity, - /// additional elements are reserved. Otherwise `recv_many` populates - /// `buffer` with no more elements than its unused capacity. + /// This method returns 0 only if the channel has been closed and there are + /// no remaining messages in the channel's queue. This indicates that no + /// further values can ever be received from this `Receiver`. The channel is + /// closed when all senders have been dropped, or when [`close`] is called. + /// + /// If at the time of the call `buffer` has unused capacity, + /// `recv_many` extends the buffer with no more elements than + /// its unused capacity. Should `buffer` have no initial unused + /// capacity, additional elements are first reserved. /// /// # Cancel safety /// - /// This method is cancel safe. If `recv` is used as the event in a + /// This method is cancel safe. If `recv_many` is used as the event in a /// [`tokio::select!`](crate::select) statement and some other branch /// completes first, it is guaranteed that no messages were received on this /// channel. @@ -266,35 +267,30 @@ impl Receiver { /// /// #[tokio::main] /// async fn main() { + /// let mut buffer: Vec<&str> = Vec::with_capacity(2); /// let (tx, mut rx) = mpsc::channel(100); - /// - /// tx.send("first").await.unwrap(); - /// tx.send("second").await.unwrap(); - /// tx.send("third").await.unwrap(); - /// - /// // The initial capacity allows up to 2 messages - /// // to be placed in the buffer. - /// let mut buffer : Vec<&str> = Vec::with_capacity(2); - /// assert_eq!(2, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first","second"], buffer); - /// - /// buffer.clear(); - /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["third"], buffer); - /// - /// tx.send("fourth").await.unwrap(); - /// tx.send("fifth").await.unwrap(); - /// - /// // As the passed-in buffer has unused capacity - /// // recv_many will use it up; it will not - /// // increase the buffer's capacity. - /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(2, buffer.capacity()); - /// - /// // With the buffer full, the next call to `recv_many` - /// // reserves additional capacity. - /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["third","fourth","fifth"], buffer); + /// { + /// let tx2 = tx.clone(); + /// tx2.send("first").await.unwrap(); + /// tx2.send("second").await.unwrap(); + /// // Initial capacity allows up to 2 values + /// // to be added to the buffer. + /// assert_eq!(2, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second"], buffer); + /// tokio::spawn(async move { + /// tx.send("third").await.unwrap(); + /// }); + /// // The 'tx' is dropped, but `recv_many` + /// // is guaranteed not to return 0 as the channel + /// // is not yet closed. With the buffer full, the next + /// // call to `recv_many` reserves additional capacity. + /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second", "third"], buffer); + /// } + /// // The channel is now closed and `recv_many` returns 0; + /// // the buffer is unchanged. + /// assert_eq!(0, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second", "third"], buffer); /// } /// ``` pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 2efe2222a80..6b0dd206699 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -176,18 +176,19 @@ impl UnboundedReceiver { /// /// This method returns the number of values added to `buffer`. /// - /// This method returns 0 if the channel has been closed and there are - /// no remaining messages in the channel's queue. This indicates that no - /// further values can ever be received from this `Receiver`. The channel is - /// closed when all senders have been dropped, or when [`close`] is called. - /// /// If there are no messages in the channel's queue, but the channel has /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. /// - /// If at the time of the call `buffer` has no unused capacity, - /// additional elements are reserved. Otherwise `recv_many` populates - /// `buffer` with no more elements than its unused capacity. + /// This method returns 0 only if the channel has been closed and there are + /// no remaining messages in the channel's queue. This indicates that no + /// further values can ever be received from this `Receiver`. The channel is + /// closed when all senders have been dropped, or when [`close`] is called. + /// + /// If at the time of the call `buffer` has unused capacity, + /// `recv_many` extends the buffer with no more elements than + /// its unused capacity. Should `buffer` have no initial unused + /// capacity, additional elements are first reserved. /// /// # Cancel safety /// @@ -205,16 +206,30 @@ impl UnboundedReceiver { /// /// #[tokio::main] /// async fn main() { + /// let mut buffer: Vec<&str> = Vec::with_capacity(2); /// let (tx, mut rx) = mpsc::unbounded_channel(); - /// - /// tokio::spawn(async move { - /// tx.send("hello").unwrap(); - /// }); - /// - /// let mut buffer : Vec<&str> = Vec::with_capacity(5); - /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["hello"], buffer); + /// { + /// let tx2 = tx.clone(); + /// tx2.send("first").unwrap(); + /// tx2.send("second").unwrap(); + /// // Initial capacity allows up to 2 values + /// // to be added to the buffer. + /// assert_eq!(2, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second"], buffer); + /// tokio::spawn(async move { + /// tx.send("third").unwrap(); + /// }); + /// // The 'tx' is dropped, but `recv_many` + /// // is guaranteed not to return 0 as the channel + /// // is not yet closed. With the buffer full, the next + /// // call to `recv_many` reserves additional capacity. + /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second", "third"], buffer); + /// } + /// // The channel is now closed and `recv_many` returns 0; + /// // the buffer is unchanged. /// assert_eq!(0, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second", "third"], buffer); /// } /// ``` pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { From 7fca954d1aa1bae50744576035d3b4de7a167f0d Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 1 Oct 2023 01:27:58 -0400 Subject: [PATCH 20/36] accomodate that with_capacity(n) gives capacity of at least n --- tokio/src/sync/mpsc/bounded.rs | 9 ++++----- tokio/src/sync/mpsc/unbounded.rs | 9 ++++----- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 26dc0a2c9d9..6d4e9e0337c 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -273,8 +273,8 @@ impl Receiver { /// let tx2 = tx.clone(); /// tx2.send("first").await.unwrap(); /// tx2.send("second").await.unwrap(); - /// // Initial capacity allows up to 2 values - /// // to be added to the buffer. + /// // Initial capacity ensures both values + /// // can be added to the buffer. /// assert_eq!(2, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second"], buffer); /// tokio::spawn(async move { @@ -282,13 +282,12 @@ impl Receiver { /// }); /// // The 'tx' is dropped, but `recv_many` /// // is guaranteed not to return 0 as the channel - /// // is not yet closed. With the buffer full, the next + /// // is not yet closed. If the buffer is full, the next /// // call to `recv_many` reserves additional capacity. /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second", "third"], buffer); /// } - /// // The channel is now closed and `recv_many` returns 0; - /// // the buffer is unchanged. + /// // The channel is now closed and `recv_many` returns 0. /// assert_eq!(0, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second", "third"], buffer); /// } diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 6b0dd206699..d9c42a9cfac 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -212,8 +212,8 @@ impl UnboundedReceiver { /// let tx2 = tx.clone(); /// tx2.send("first").unwrap(); /// tx2.send("second").unwrap(); - /// // Initial capacity allows up to 2 values - /// // to be added to the buffer. + /// // Initial capacity ensures both values + /// // can be added to the buffer. /// assert_eq!(2, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second"], buffer); /// tokio::spawn(async move { @@ -221,13 +221,12 @@ impl UnboundedReceiver { /// }); /// // The 'tx' is dropped, but `recv_many` /// // is guaranteed not to return 0 as the channel - /// // is not yet closed. With the buffer full, the next + /// // is not yet closed. If the buffer is full, the next /// // call to `recv_many` reserves additional capacity. /// assert_eq!(1, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second", "third"], buffer); /// } - /// // The channel is now closed and `recv_many` returns 0; - /// // the buffer is unchanged. + /// // The channel is now closed and `recv_many` returns 0. /// assert_eq!(0, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second", "third"], buffer); /// } From fe44b323cee7028cc1f4fd1a735f09f78908da4a Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 1 Oct 2023 09:50:40 -0400 Subject: [PATCH 21/36] Improved documentation for `recv_many` and added test The documentation is changed to better highlight the limited circumstance in which `recv_many` may return 0, '..will never return 0 unless...'. Two new tests are somewhat analogous to the doctests were added: - send_recv_many_bounded_capacity() - send_recv_many_unbounded_capacity() These ensure that for buffers that have not reached their capacity a call to `recv_many` only fills up to the capacity leaving other messages still queued. A subsequent call on a filled buffer would reserve additional elements. The test also looks at pending messages holding the channel open -- compared to the doctest example which showcases non-dropped senders. --- tokio/src/sync/mpsc/bounded.rs | 9 ++-- tokio/src/sync/mpsc/unbounded.rs | 9 ++-- tokio/tests/sync_mpsc.rs | 84 ++++++++++++++++++++++++++++++++ 3 files changed, 94 insertions(+), 8 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 6d4e9e0337c..fc54f89e162 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -240,10 +240,11 @@ impl Receiver { /// still outstanding [`Permits`] from before it was closed, the channel is /// not considered closed by `recv` until the permits are released. /// - /// This method returns 0 only if the channel has been closed and there are - /// no remaining messages in the channel's queue. This indicates that no - /// further values can ever be received from this `Receiver`. The channel is - /// closed when all senders have been dropped, or when [`close`] is called. + /// This method will never return 0 unless the channel has been closed and + /// there are no remaining messages in the channel's queue. This indicates + /// that no further values can ever be received from this `Receiver`. The + /// channel is closed when all senders have been dropped, or when [`close`] + /// is called. /// /// If at the time of the call `buffer` has unused capacity, /// `recv_many` extends the buffer with no more elements than diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index d9c42a9cfac..2ef68f7bbbf 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -180,10 +180,11 @@ impl UnboundedReceiver { /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. /// - /// This method returns 0 only if the channel has been closed and there are - /// no remaining messages in the channel's queue. This indicates that no - /// further values can ever be received from this `Receiver`. The channel is - /// closed when all senders have been dropped, or when [`close`] is called. + /// This method will never return 0 unless the channel has been closed and + /// there are no remaining messages in the channel's queue. This indicates + /// that no further values can ever be received from this `Receiver`. The + /// channel is closed when all senders have been dropped, or when [`close`] + /// is called. /// /// If at the time of the call `buffer` has unused capacity, /// `recv_many` extends the buffer with no more elements than diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index e57f559abc4..a7eae798597 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -226,6 +226,90 @@ async fn send_recv_many_unbounded() { assert!(rx.recv().await.is_none()); } +#[tokio::test] +#[cfg(feature = "full")] +async fn send_recv_many_bounded_capacity() { + let mut expected: Vec; + let mut buffer: Vec = Vec::with_capacity(9); // capacity >= 9 + let (tx, mut rx) = mpsc::channel(100); + + expected = (0..buffer.capacity()) + .map(|x: usize| format!("{x}")) + .collect::>(); + for x in expected.clone() { + tx.send(x).await.unwrap() + } + tx.send("one more".to_string()).await.unwrap(); + + // With unused buffer capacity, `recv_may` uses as most + // the unused capacity -- so "one more" isn't received in + // the next call + assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer).await); + assert_eq!(expected, buffer); + + // The buffer is now full and `recv_many` will reserve capacity + // on the next call. + assert_eq!(1, rx.recv_many(&mut buffer).await); + expected.push("one more".to_string()); + assert_eq!(expected, buffer); + + tokio::spawn(async move { + tx.send("final".to_string()).await.unwrap(); + }); + + // 'tx' is dropped, but `recv_many` is guaranteed not + // to return 0 as the channel has outstanding permits + assert_eq!(1, rx.recv_many(&mut buffer).await); + expected.push("final".to_string()); + assert_eq!(expected, buffer); + // The channel is now closed and `recv_many` returns 0. + assert_eq!(0, rx.recv_many(&mut buffer).await); + assert_eq!(expected, buffer); +} + +#[tokio::test] +#[cfg(feature = "full")] +async fn send_recv_many_unbounded_capacity() { + let mut expected: Vec; + let mut buffer: Vec = Vec::with_capacity(9); // capacity >= 9 + let (tx, mut rx) = mpsc::unbounded_channel(); + + let initial_capacity = buffer.capacity(); + expected = (0..initial_capacity) + .map(|x: usize| format!("{x}")) + .collect::>(); + for x in expected.clone() { + tx.send(x).unwrap() + } + tx.send("one more".to_string()).unwrap(); + + // With unused buffer capacity, `recv_may` uses as most + // the unused capacity -- so "one more" isn't received in + // the next call + assert_eq!(initial_capacity, buffer.capacity()); + assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer).await); + assert_eq!(expected, buffer); + + // The buffer is now full and `recv_many` will reserve capacity + // on the next call. + assert_eq!(1, rx.recv_many(&mut buffer).await); + expected.push("one more".to_string()); + assert_eq!(expected, buffer); + + tokio::spawn(async move { + tx.send("final".to_string()).unwrap(); + }); + + // 'tx' is dropped, but `recv_many` is guaranteed not + // to return 0 as the channel has outstanding permits + assert_eq!(1, rx.recv_many(&mut buffer).await); + expected.push("final".to_string()); + assert_eq!(expected, buffer); + // The channel is now closed and `recv_many` returns 0. + assert_eq!(0, rx.recv_many(&mut buffer).await); + assert_eq!(expected, buffer); +} + #[tokio::test] #[cfg(feature = "full")] async fn async_send_recv_unbounded() { From aac0eda6d25b8f3c6ea6ea9f4baa1d7dde4fae42 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 1 Oct 2023 11:40:14 -0400 Subject: [PATCH 22/36] Update tokio/src/sync/mpsc/bounded.rs Co-authored-by: Alice Ryhl --- tokio/src/sync/mpsc/bounded.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index fc54f89e162..ee5f1a387b9 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -238,7 +238,7 @@ impl Receiver { /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. Note that if [`close`] is called, but there are /// still outstanding [`Permits`] from before it was closed, the channel is - /// not considered closed by `recv` until the permits are released. + /// not considered closed by `recv_many` until the permits are released. /// /// This method will never return 0 unless the channel has been closed and /// there are no remaining messages in the channel's queue. This indicates From 088537cf536babdd1413686bf3ec410b4ff20a49 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 1 Oct 2023 11:40:34 -0400 Subject: [PATCH 23/36] Update tokio/src/sync/mpsc/bounded.rs Co-authored-by: Alice Ryhl --- tokio/src/sync/mpsc/bounded.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index ee5f1a387b9..5758c95a4fa 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -259,7 +259,7 @@ impl Receiver { /// channel. /// /// [`close`]: Self::close - /// [`Permits`]: struct@crate::sync::mpsc::Permit/// + /// [`Permits`]: struct@crate::sync::mpsc::Permit /// /// # Examples /// From effb0e58dc24ad224427fca8dff874b65464da15 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 1 Oct 2023 12:27:38 -0400 Subject: [PATCH 24/36] change recv_many doctests to use drop not scope --- tokio/src/sync/mpsc/bounded.rs | 41 +++++++++++++++++--------------- tokio/src/sync/mpsc/unbounded.rs | 41 +++++++++++++++++--------------- tokio/tests/sync_mpsc.rs | 4 +++- 3 files changed, 47 insertions(+), 39 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 5758c95a4fa..c63383d2825 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -270,25 +270,28 @@ impl Receiver { /// async fn main() { /// let mut buffer: Vec<&str> = Vec::with_capacity(2); /// let (tx, mut rx) = mpsc::channel(100); - /// { - /// let tx2 = tx.clone(); - /// tx2.send("first").await.unwrap(); - /// tx2.send("second").await.unwrap(); - /// // Initial capacity ensures both values - /// // can be added to the buffer. - /// assert_eq!(2, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first", "second"], buffer); - /// tokio::spawn(async move { - /// tx.send("third").await.unwrap(); - /// }); - /// // The 'tx' is dropped, but `recv_many` - /// // is guaranteed not to return 0 as the channel - /// // is not yet closed. If the buffer is full, the next - /// // call to `recv_many` reserves additional capacity. - /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first", "second", "third"], buffer); - /// } - /// // The channel is now closed and `recv_many` returns 0. + /// let tx2 = tx.clone(); + /// tx2.send("first").await.unwrap(); + /// tx2.send("second").await.unwrap(); + /// // Initial capacity ensures both values + /// // can be added to the buffer. + /// assert_eq!(2, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second"], buffer); + /// + /// tokio::spawn(async move { + /// tx.send("third").await.unwrap(); + /// }); + /// + /// // The 'tx' is dropped, but `recv_many` + /// // is guaranteed not to return 0 as the channel + /// // is not yet closed. If the buffer is full, the next + /// // call to `recv_many` reserves additional capacity. + /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second", "third"], buffer); + /// + /// // Once the last sender is dropped, the channel is + /// // closed and `recv_many` returns 0. + /// drop(tx2); /// assert_eq!(0, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second", "third"], buffer); /// } diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 2ef68f7bbbf..19985c025bd 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -209,25 +209,28 @@ impl UnboundedReceiver { /// async fn main() { /// let mut buffer: Vec<&str> = Vec::with_capacity(2); /// let (tx, mut rx) = mpsc::unbounded_channel(); - /// { - /// let tx2 = tx.clone(); - /// tx2.send("first").unwrap(); - /// tx2.send("second").unwrap(); - /// // Initial capacity ensures both values - /// // can be added to the buffer. - /// assert_eq!(2, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first", "second"], buffer); - /// tokio::spawn(async move { - /// tx.send("third").unwrap(); - /// }); - /// // The 'tx' is dropped, but `recv_many` - /// // is guaranteed not to return 0 as the channel - /// // is not yet closed. If the buffer is full, the next - /// // call to `recv_many` reserves additional capacity. - /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first", "second", "third"], buffer); - /// } - /// // The channel is now closed and `recv_many` returns 0. + /// let tx2 = tx.clone(); + /// tx2.send("first").unwrap(); + /// tx2.send("second").unwrap(); + /// // Initial capacity ensures both values + /// // can be added to the buffer. + /// assert_eq!(2, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second"], buffer); + /// + /// tokio::spawn(async move { + /// tx.send("third").unwrap(); + /// }); + /// + /// // The 'tx' is dropped, but `recv_many` + /// // is guaranteed not to return 0 as the channel + /// // is not yet closed. If the buffer is full, the next + /// // call to `recv_many` reserves additional capacity. + /// assert_eq!(1, rx.recv_many(&mut buffer).await); + /// assert_eq!(vec!["first", "second", "third"], buffer); + /// + /// // Once the last sender is dropped, the channel is + /// // closed and `recv_many` returns 0. + /// drop(tx2); /// assert_eq!(0, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second", "third"], buffer); /// } diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index a7eae798597..0b612da965e 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -233,7 +233,8 @@ async fn send_recv_many_bounded_capacity() { let mut buffer: Vec = Vec::with_capacity(9); // capacity >= 9 let (tx, mut rx) = mpsc::channel(100); - expected = (0..buffer.capacity()) + let initial_capacity = buffer.capacity(); + expected = (0..initial_capacity) .map(|x: usize| format!("{x}")) .collect::>(); for x in expected.clone() { @@ -244,6 +245,7 @@ async fn send_recv_many_bounded_capacity() { // With unused buffer capacity, `recv_may` uses as most // the unused capacity -- so "one more" isn't received in // the next call + assert_eq!(initial_capacity, buffer.capacity()); assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer).await); assert_eq!(expected, buffer); From 1fb08df662bc6350d6218c316676cf24a3c70504 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 1 Oct 2023 12:30:37 -0400 Subject: [PATCH 25/36] adopt improved wording on buffer capacity behavior --- tokio/src/sync/mpsc/bounded.rs | 9 +++++---- tokio/src/sync/mpsc/unbounded.rs | 9 +++++---- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index c63383d2825..2ee62f60de3 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -246,10 +246,11 @@ impl Receiver { /// channel is closed when all senders have been dropped, or when [`close`] /// is called. /// - /// If at the time of the call `buffer` has unused capacity, - /// `recv_many` extends the buffer with no more elements than - /// its unused capacity. Should `buffer` have no initial unused - /// capacity, additional elements are first reserved. + /// If `buffer` has unused capacity, then this call will not reserve + /// additional space in `buffer`. This means that the maximum number of + /// received messages is `buffer.capacity() - buffer.len()`. However, if + /// the capacity is equal to the length, then this call will increase the + /// capacity to make space for additional elements. /// /// # Cancel safety /// diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 19985c025bd..a4873680ffc 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -186,10 +186,11 @@ impl UnboundedReceiver { /// channel is closed when all senders have been dropped, or when [`close`] /// is called. /// - /// If at the time of the call `buffer` has unused capacity, - /// `recv_many` extends the buffer with no more elements than - /// its unused capacity. Should `buffer` have no initial unused - /// capacity, additional elements are first reserved. + /// If `buffer` has unused capacity, then this call will not reserve + /// additional space in `buffer`. This means that the maximum number of + /// received messages is `buffer.capacity() - buffer.len()`. However, if + /// the capacity is equal to the length, then this call will increase the + /// capacity to make space for additional elements. /// /// # Cancel safety /// From bd1ef3a5d542ddc57e7100637aa0c68cc01bd575 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 1 Oct 2023 13:07:02 -0400 Subject: [PATCH 26/36] Only reserve if no unused capacity and there is 1+ value(s) to receive --- tokio/src/sync/mpsc/bounded.rs | 7 ++++--- tokio/src/sync/mpsc/chan.rs | 13 +++++++------ tokio/src/sync/mpsc/unbounded.rs | 7 ++++--- 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 2ee62f60de3..8eb008d190a 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -249,8 +249,9 @@ impl Receiver { /// If `buffer` has unused capacity, then this call will not reserve /// additional space in `buffer`. This means that the maximum number of /// received messages is `buffer.capacity() - buffer.len()`. However, if - /// the capacity is equal to the length, then this call will increase the - /// capacity to make space for additional elements. + /// the capacity is equal to the length and there is at least one message + /// in the channel's queue, then this call will increase the capacity + /// to make space for additional elements. /// /// # Cancel safety /// @@ -291,7 +292,7 @@ impl Receiver { /// assert_eq!(vec!["first", "second", "third"], buffer); /// /// // Once the last sender is dropped, the channel is - /// // closed and `recv_many` returns 0. + /// // closed and `recv_many` returns 0, capacity unchanged. /// drop(tx2); /// assert_eq!(0, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second", "third"], buffer); diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index 2500990f757..625698466a1 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -297,14 +297,11 @@ impl Rx { /// Receives values into `buffer` up to its capacity /// - /// If the buffer initially has 0 remaining capacity, reserves `super::BLOCK_CAP` elements + /// If the buffer 0 remaining capacity but a value is received, reserves `super::BLOCK_CAP` elements pub(crate) fn recv_many(&mut self, cx: &mut Context<'_>, buffer: &mut Vec) -> Poll { use super::block::Read; - if buffer.len() == buffer.capacity() { - buffer.reserve(super::BLOCK_CAP); - } - + let mut insufficient_capacity = buffer.capacity() == buffer.len(); let initial_length = buffer.len(); ready!(crate::trace::trace_leaf(cx)); @@ -315,9 +312,13 @@ impl Rx { let rx_fields = unsafe { &mut *rx_fields_ptr }; macro_rules! try_recv { () => { - while (buffer.len() < buffer.capacity()) { + while (buffer.len() < buffer.capacity() || insufficient_capacity) { match rx_fields.list.pop(&self.inner.tx) { Some(Read::Value(value)) => { + if insufficient_capacity { + buffer.reserve(super::BLOCK_CAP); + insufficient_capacity = false; + } buffer.push(value); } diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index a4873680ffc..408d276240a 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -189,8 +189,9 @@ impl UnboundedReceiver { /// If `buffer` has unused capacity, then this call will not reserve /// additional space in `buffer`. This means that the maximum number of /// received messages is `buffer.capacity() - buffer.len()`. However, if - /// the capacity is equal to the length, then this call will increase the - /// capacity to make space for additional elements. + /// the capacity is equal to the length and there is at least one message + /// in the channel's queue, then this call will increase the capacity + /// to make space for additional elements. /// /// # Cancel safety /// @@ -230,7 +231,7 @@ impl UnboundedReceiver { /// assert_eq!(vec!["first", "second", "third"], buffer); /// /// // Once the last sender is dropped, the channel is - /// // closed and `recv_many` returns 0. + /// // closed and `recv_many` returns 0, capacity unchanged. /// drop(tx2); /// assert_eq!(0, rx.recv_many(&mut buffer).await); /// assert_eq!(vec!["first", "second", "third"], buffer); From d31fd17c64808b93c14a6d2e6ff9447a7dd807f0 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Mon, 2 Oct 2023 02:07:34 -0400 Subject: [PATCH 27/36] Updated `recv_many` to accept a new argument `limit` Positive values of `limit` set the maximum number of elements that can be added to `buffer` in a single call. A zero value of `limit` sets the maximum number of elements that can be added to a default value, currently `super::BLOCK_CAP`. --- benches/sync_mpsc.rs | 10 ++-- tokio/src/sync/mpsc/bounded.rs | 43 +++++++++-------- tokio/src/sync/mpsc/chan.rs | 19 +++++--- tokio/src/sync/mpsc/unbounded.rs | 43 +++++++++-------- tokio/tests/sync_mpsc.rs | 83 +++++++++++++++++++------------- 5 files changed, 111 insertions(+), 87 deletions(-) diff --git a/benches/sync_mpsc.rs b/benches/sync_mpsc.rs index 7b07d9373c8..117b3babdde 100644 --- a/benches/sync_mpsc.rs +++ b/benches/sync_mpsc.rs @@ -93,7 +93,7 @@ fn contention_bounded_recv_many(g: &mut BenchmarkGroup) { let mut buffer = Vec::::with_capacity(5_000); let mut total = 0; while total < 1_000 * 5 { - total += rx.recv_many(&mut buffer).await; + total += rx.recv_many(&mut buffer, 5_000).await; } }) }) @@ -145,7 +145,7 @@ fn contention_bounded_full_recv_many(g: &mut BenchmarkGroup) { let mut buffer = Vec::::with_capacity(5_000); let mut total = 0; while total < 1_000 * 5 { - total += rx.recv_many(&mut buffer).await; + total += rx.recv_many(&mut buffer, 5_000).await; } }) }) @@ -197,7 +197,7 @@ fn contention_unbounded_recv_many(g: &mut BenchmarkGroup) { let mut buffer = Vec::::with_capacity(5_000); let mut total = 0; while total < 1_000 * 5 { - total += rx.recv_many(&mut buffer).await; + total += rx.recv_many(&mut buffer, 5_000).await; } }) }) @@ -239,7 +239,7 @@ fn uncontented_bounded_recv_many(g: &mut BenchmarkGroup) { let mut buffer = Vec::::with_capacity(5_000); let mut total = 0; while total < 1_000 * 5 { - total += rx.recv_many(&mut buffer).await; + total += rx.recv_many(&mut buffer, 5_000).await; } }) }) @@ -281,7 +281,7 @@ fn uncontented_unbounded_recv_many(g: &mut BenchmarkGroup) { let mut buffer = Vec::::with_capacity(5_000); let mut total = 0; while total < 1_000 * 5 { - total += rx.recv_many(&mut buffer).await; + total += rx.recv_many(&mut buffer, 5_000).await; } }) }) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 8eb008d190a..04df947fd23 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -232,7 +232,10 @@ impl Receiver { /// Receives the next values for this receiver and extends `buffer`. /// - /// This method returns the number of values added to `buffer`. + /// This method extends `buffer` by no more than a fixed number + /// of values as specified by `limit`. If `limit` is zero, + /// then a default is used. The return value is the number + /// of values added to `buffer`. /// /// If there are no messages in the channel's queue, but the channel has /// not yet been closed, this method will sleep until a message is sent or @@ -246,12 +249,7 @@ impl Receiver { /// channel is closed when all senders have been dropped, or when [`close`] /// is called. /// - /// If `buffer` has unused capacity, then this call will not reserve - /// additional space in `buffer`. This means that the maximum number of - /// received messages is `buffer.capacity() - buffer.len()`. However, if - /// the capacity is equal to the length and there is at least one message - /// in the channel's queue, then this call will increase the capacity - /// to make space for additional elements. + /// The capacity of `buffer` is increased as needed. /// /// # Cancel safety /// @@ -271,36 +269,41 @@ impl Receiver { /// #[tokio::main] /// async fn main() { /// let mut buffer: Vec<&str> = Vec::with_capacity(2); + /// let limit = 2; /// let (tx, mut rx) = mpsc::channel(100); /// let tx2 = tx.clone(); /// tx2.send("first").await.unwrap(); /// tx2.send("second").await.unwrap(); - /// // Initial capacity ensures both values - /// // can be added to the buffer. - /// assert_eq!(2, rx.recv_many(&mut buffer).await); + /// tx2.send("third").await.unwrap(); + /// + /// // Call `recv_many` to receive up to `limit` (2) values. + /// assert_eq!(2, rx.recv_many(&mut buffer, limit).await); /// assert_eq!(vec!["first", "second"], buffer); /// + /// // If the buffer is full, the next call to `recv_many` + /// // reserves additional capacity. + /// assert_eq!(1, rx.recv_many(&mut buffer, 1).await); + /// /// tokio::spawn(async move { - /// tx.send("third").await.unwrap(); + /// tx.send("fourth").await.unwrap(); /// }); /// - /// // The 'tx' is dropped, but `recv_many` + /// // 'tx' is dropped, but `recv_many` /// // is guaranteed not to return 0 as the channel - /// // is not yet closed. If the buffer is full, the next - /// // call to `recv_many` reserves additional capacity. - /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first", "second", "third"], buffer); + /// // is not yet closed. + /// assert_eq!(1, rx.recv_many(&mut buffer, 1).await); + /// assert_eq!(vec!["first", "second", "third", "fourth"], buffer); /// /// // Once the last sender is dropped, the channel is /// // closed and `recv_many` returns 0, capacity unchanged. /// drop(tx2); - /// assert_eq!(0, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first", "second", "third"], buffer); + /// assert_eq!(0, rx.recv_many(&mut buffer, limit).await); + /// assert_eq!(vec!["first", "second", "third", "fourth"], buffer); /// } /// ``` - pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { + pub async fn recv_many(&mut self, buffer: &mut Vec, limit: usize) -> usize { use crate::future::poll_fn; - poll_fn(|cx| self.chan.recv_many(cx, buffer)).await + poll_fn(|cx| self.chan.recv_many(cx, buffer, limit)).await } /// Tries to receive the next value for this receiver. diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index 625698466a1..99a687d9e14 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -297,11 +297,17 @@ impl Rx { /// Receives values into `buffer` up to its capacity /// - /// If the buffer 0 remaining capacity but a value is received, reserves `super::BLOCK_CAP` elements - pub(crate) fn recv_many(&mut self, cx: &mut Context<'_>, buffer: &mut Vec) -> Poll { + /// For `limit` > 0, receives up to limit values into `buffer`. + /// For `limit` = 0, receives at most `super::BLOCK_CAP` values. + pub(crate) fn recv_many( + &mut self, + cx: &mut Context<'_>, + buffer: &mut Vec, + limit: usize, + ) -> Poll { use super::block::Read; - let mut insufficient_capacity = buffer.capacity() == buffer.len(); + let mut to_go = if limit > 0 { limit } else { super::BLOCK_CAP }; let initial_length = buffer.len(); ready!(crate::trace::trace_leaf(cx)); @@ -312,13 +318,10 @@ impl Rx { let rx_fields = unsafe { &mut *rx_fields_ptr }; macro_rules! try_recv { () => { - while (buffer.len() < buffer.capacity() || insufficient_capacity) { + while (to_go > 0) { match rx_fields.list.pop(&self.inner.tx) { Some(Read::Value(value)) => { - if insufficient_capacity { - buffer.reserve(super::BLOCK_CAP); - insufficient_capacity = false; - } + to_go = to_go - 1; buffer.push(value); } diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 408d276240a..bef8b8bde33 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -174,7 +174,10 @@ impl UnboundedReceiver { /// Receives the next values for this receiver and extends `buffer`. /// - /// This method returns the number of values added to `buffer`. + /// This method extends `buffer` by no more than a fixed number + /// of values as specified by `limit`. If `limit` is zero, + /// then a default is used. The return value is the number + /// of values added to `buffer`. /// /// If there are no messages in the channel's queue, but the channel has /// not yet been closed, this method will sleep until a message is sent or @@ -186,12 +189,7 @@ impl UnboundedReceiver { /// channel is closed when all senders have been dropped, or when [`close`] /// is called. /// - /// If `buffer` has unused capacity, then this call will not reserve - /// additional space in `buffer`. This means that the maximum number of - /// received messages is `buffer.capacity() - buffer.len()`. However, if - /// the capacity is equal to the length and there is at least one message - /// in the channel's queue, then this call will increase the capacity - /// to make space for additional elements. + /// The capacity of `buffer` is increased as needed. /// /// # Cancel safety /// @@ -210,36 +208,41 @@ impl UnboundedReceiver { /// #[tokio::main] /// async fn main() { /// let mut buffer: Vec<&str> = Vec::with_capacity(2); + /// let limit = 2; /// let (tx, mut rx) = mpsc::unbounded_channel(); /// let tx2 = tx.clone(); /// tx2.send("first").unwrap(); /// tx2.send("second").unwrap(); - /// // Initial capacity ensures both values - /// // can be added to the buffer. - /// assert_eq!(2, rx.recv_many(&mut buffer).await); + /// tx2.send("third").unwrap(); + /// + /// // Call `recv_many` to receive up to `limit` (2) values. + /// assert_eq!(2, rx.recv_many(&mut buffer, limit).await); /// assert_eq!(vec!["first", "second"], buffer); /// + /// // If the buffer is full, the next call to `recv_many` + /// // reserves additional capacity. + /// assert_eq!(1, rx.recv_many(&mut buffer, limit).await); + /// /// tokio::spawn(async move { - /// tx.send("third").unwrap(); + /// tx.send("fourth").unwrap(); /// }); /// - /// // The 'tx' is dropped, but `recv_many` + /// // 'tx' is dropped, but `recv_many` /// // is guaranteed not to return 0 as the channel - /// // is not yet closed. If the buffer is full, the next - /// // call to `recv_many` reserves additional capacity. - /// assert_eq!(1, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first", "second", "third"], buffer); + /// // is not yet closed. + /// assert_eq!(1, rx.recv_many(&mut buffer, limit).await); + /// assert_eq!(vec!["first", "second", "third", "fourth"], buffer); /// /// // Once the last sender is dropped, the channel is /// // closed and `recv_many` returns 0, capacity unchanged. /// drop(tx2); - /// assert_eq!(0, rx.recv_many(&mut buffer).await); - /// assert_eq!(vec!["first", "second", "third"], buffer); + /// assert_eq!(0, rx.recv_many(&mut buffer, limit).await); + /// assert_eq!(vec!["first", "second", "third", "fourth"], buffer); /// } /// ``` - pub async fn recv_many(&mut self, buffer: &mut Vec) -> usize { + pub async fn recv_many(&mut self, buffer: &mut Vec, limit: usize) -> usize { use crate::future::poll_fn; - poll_fn(|cx| self.chan.recv_many(cx, buffer)).await + poll_fn(|cx| self.chan.recv_many(cx, buffer, limit)).await } /// Tries to receive the next value for this receiver. diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index 0b612da965e..543de76a426 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -131,10 +131,11 @@ async fn async_send_recv_many_with_buffer() { assert_ok!(tx.send(7).await); }); + let limit = 3; let mut buffer = Vec::::with_capacity(3); let mut recv_count = 0usize; while recv_count < 3 { - let n = rx.recv_many(&mut buffer).await; + let n = rx.recv_many(&mut buffer, limit).await; recv_count += n; assert_eq!(buffer.len(), recv_count); assert!(buffer.iter().sum::() <= 10); @@ -205,36 +206,51 @@ async fn send_recv_unbounded() { async fn send_recv_many_unbounded() { let (tx, mut rx) = mpsc::unbounded_channel::(); - // Using `try_send` assert_ok!(tx.send(7)); assert_ok!(tx.send(13)); assert_ok!(tx.send(100)); assert_ok!(tx.send(1002)); - let mut buffer: Vec = Vec::with_capacity(0); + let mut buffer: Vec = Vec::new(); let mut count = 0; while count < 4 { - count += rx.recv_many(&mut buffer).await; + count += rx.recv_many(&mut buffer, 0).await; } assert_eq!(count, 4); - assert_eq!(count, buffer.len()); - assert_eq!(buffer.iter().sum::(), 1122); - assert!(buffer.capacity() > 0); + assert_eq!(vec![7, 13, 100, 1002], buffer); + let final_capacity = buffer.capacity(); + assert!(final_capacity > 0); + + buffer.clear(); + + assert_ok!(tx.send(5)); + assert_ok!(tx.send(6)); + assert_ok!(tx.send(7)); + assert_ok!(tx.send(2)); + + count = rx.recv_many(&mut buffer, 4).await; + + assert_eq!(final_capacity, buffer.capacity()); + assert_eq!(count, 4); + assert_eq!(vec![5, 6, 7, 2], buffer); drop(tx); + // recv_many will immediately return zero if the channel + // is closed and no more messages are waiting + assert_eq!(0, rx.recv_many(&mut buffer, 4).await); + assert_eq!(0, rx.recv_many(&mut buffer, 4).await); assert!(rx.recv().await.is_none()); } #[tokio::test] #[cfg(feature = "full")] async fn send_recv_many_bounded_capacity() { - let mut expected: Vec; - let mut buffer: Vec = Vec::with_capacity(9); // capacity >= 9 + let mut buffer: Vec = Vec::with_capacity(9); + let limit = buffer.capacity(); let (tx, mut rx) = mpsc::channel(100); - let initial_capacity = buffer.capacity(); - expected = (0..initial_capacity) + let mut expected: Vec = (0..limit) .map(|x: usize| format!("{x}")) .collect::>(); for x in expected.clone() { @@ -242,16 +258,16 @@ async fn send_recv_many_bounded_capacity() { } tx.send("one more".to_string()).await.unwrap(); - // With unused buffer capacity, `recv_may` uses as most - // the unused capacity -- so "one more" isn't received in - // the next call - assert_eq!(initial_capacity, buffer.capacity()); - assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer).await); + // Here `recv_may` receives all but the last value; + // the initial capacity is adequate, so the buffer does + // not increase in side. + assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer, limit).await); assert_eq!(expected, buffer); + assert_eq!(limit, buffer.capacity()); - // The buffer is now full and `recv_many` will reserve capacity - // on the next call. - assert_eq!(1, rx.recv_many(&mut buffer).await); + // Receive up more values: + assert_eq!(1, rx.recv_many(&mut buffer, limit).await); + assert!(buffer.capacity() > limit); expected.push("one more".to_string()); assert_eq!(expected, buffer); @@ -261,23 +277,22 @@ async fn send_recv_many_bounded_capacity() { // 'tx' is dropped, but `recv_many` is guaranteed not // to return 0 as the channel has outstanding permits - assert_eq!(1, rx.recv_many(&mut buffer).await); + assert_eq!(1, rx.recv_many(&mut buffer, limit).await); expected.push("final".to_string()); assert_eq!(expected, buffer); // The channel is now closed and `recv_many` returns 0. - assert_eq!(0, rx.recv_many(&mut buffer).await); + assert_eq!(0, rx.recv_many(&mut buffer, limit).await); assert_eq!(expected, buffer); } #[tokio::test] #[cfg(feature = "full")] async fn send_recv_many_unbounded_capacity() { - let mut expected: Vec; let mut buffer: Vec = Vec::with_capacity(9); // capacity >= 9 + let limit = buffer.capacity(); let (tx, mut rx) = mpsc::unbounded_channel(); - let initial_capacity = buffer.capacity(); - expected = (0..initial_capacity) + let mut expected: Vec = (0..limit) .map(|x: usize| format!("{x}")) .collect::>(); for x in expected.clone() { @@ -285,16 +300,16 @@ async fn send_recv_many_unbounded_capacity() { } tx.send("one more".to_string()).unwrap(); - // With unused buffer capacity, `recv_may` uses as most - // the unused capacity -- so "one more" isn't received in - // the next call - assert_eq!(initial_capacity, buffer.capacity()); - assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer).await); + // Here `recv_may` receives all but the last value; + // the initial capacity is adequate, so the buffer does + // not increase in side. + assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer, limit).await); assert_eq!(expected, buffer); + assert_eq!(limit, buffer.capacity()); - // The buffer is now full and `recv_many` will reserve capacity - // on the next call. - assert_eq!(1, rx.recv_many(&mut buffer).await); + // Receive up more values: + assert_eq!(1, rx.recv_many(&mut buffer, limit).await); + assert!(buffer.capacity() > limit); expected.push("one more".to_string()); assert_eq!(expected, buffer); @@ -304,11 +319,11 @@ async fn send_recv_many_unbounded_capacity() { // 'tx' is dropped, but `recv_many` is guaranteed not // to return 0 as the channel has outstanding permits - assert_eq!(1, rx.recv_many(&mut buffer).await); + assert_eq!(1, rx.recv_many(&mut buffer, limit).await); expected.push("final".to_string()); assert_eq!(expected, buffer); // The channel is now closed and `recv_many` returns 0. - assert_eq!(0, rx.recv_many(&mut buffer).await); + assert_eq!(0, rx.recv_many(&mut buffer, limit).await); assert_eq!(expected, buffer); } From 3cd2e46c09008dfb152daedf635fef8fd8fa86d6 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Mon, 2 Oct 2023 08:57:25 -0400 Subject: [PATCH 28/36] rename to_go to remaining --- tokio/src/sync/mpsc/chan.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index 99a687d9e14..4e49ce7c252 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -307,7 +307,7 @@ impl Rx { ) -> Poll { use super::block::Read; - let mut to_go = if limit > 0 { limit } else { super::BLOCK_CAP }; + let mut remaining = if limit > 0 { limit } else { super::BLOCK_CAP }; let initial_length = buffer.len(); ready!(crate::trace::trace_leaf(cx)); @@ -318,10 +318,10 @@ impl Rx { let rx_fields = unsafe { &mut *rx_fields_ptr }; macro_rules! try_recv { () => { - while (to_go > 0) { + while (remaining > 0) { match rx_fields.list.pop(&self.inner.tx) { Some(Read::Value(value)) => { - to_go = to_go - 1; + remaining -= 1; buffer.push(value); } From 326fa53616750b65eaff7678754de1b38ad96ed1 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 8 Oct 2023 13:40:58 -0400 Subject: [PATCH 29/36] Change behavior of recv_many(buf, 0) to return 0 immediately. Returning 0 immediately when limit=0 is consistent with the current take(0) behavior. Tests were updated. --- tokio/src/sync/mpsc/bounded.rs | 20 +++++++++++--------- tokio/src/sync/mpsc/chan.rs | 8 ++++++-- tokio/src/sync/mpsc/unbounded.rs | 14 +++++++------- tokio/tests/sync_mpsc.rs | 21 ++++++++++++++++----- 4 files changed, 40 insertions(+), 23 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 04df947fd23..2cc50ce4175 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -234,19 +234,21 @@ impl Receiver { /// /// This method extends `buffer` by no more than a fixed number /// of values as specified by `limit`. If `limit` is zero, - /// then a default is used. The return value is the number - /// of values added to `buffer`. + /// the function returns immediately with `0`. + /// The return value is the number of values added to `buffer`. /// - /// If there are no messages in the channel's queue, but the channel has - /// not yet been closed, this method will sleep until a message is sent or - /// the channel is closed. Note that if [`close`] is called, but there are + /// For `limit > 0`, if there are no messages in the + /// channel's queue, but the channel has not yet been closed, this + /// method will sleep until a message is sent or the channel is closed. + /// Note that if [`close`] is called, but there are /// still outstanding [`Permits`] from before it was closed, the channel is /// not considered closed by `recv_many` until the permits are released. /// - /// This method will never return 0 unless the channel has been closed and - /// there are no remaining messages in the channel's queue. This indicates - /// that no further values can ever be received from this `Receiver`. The - /// channel is closed when all senders have been dropped, or when [`close`] + /// For non-zero values `limit`, this method will never return `0` unless + /// the channel has been closed and there are no remaining messages in the + /// channel's queue. This indicates that no further values can ever + /// be received from this `Receiver`. The channel is closed when all senders + /// have been dropped, or when [`close`] /// is called. /// /// The capacity of `buffer` is increased as needed. diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index 4e49ce7c252..d72cbab45cb 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -298,7 +298,7 @@ impl Rx { /// Receives values into `buffer` up to its capacity /// /// For `limit` > 0, receives up to limit values into `buffer`. - /// For `limit` = 0, receives at most `super::BLOCK_CAP` values. + /// For `limit` = 0, immediately returns Ready(0). pub(crate) fn recv_many( &mut self, cx: &mut Context<'_>, @@ -307,7 +307,11 @@ impl Rx { ) -> Poll { use super::block::Read; - let mut remaining = if limit > 0 { limit } else { super::BLOCK_CAP }; + if limit == 0 { + return Ready(0usize); + } + + let mut remaining = limit; let initial_length = buffer.len(); ready!(crate::trace::trace_leaf(cx)); diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index bef8b8bde33..1648a763339 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -176,18 +176,18 @@ impl UnboundedReceiver { /// /// This method extends `buffer` by no more than a fixed number /// of values as specified by `limit`. If `limit` is zero, - /// then a default is used. The return value is the number - /// of values added to `buffer`. + /// the function returns immediately with `0`. + /// The return value is the number of values added to `buffer`. /// /// If there are no messages in the channel's queue, but the channel has /// not yet been closed, this method will sleep until a message is sent or /// the channel is closed. /// - /// This method will never return 0 unless the channel has been closed and - /// there are no remaining messages in the channel's queue. This indicates - /// that no further values can ever be received from this `Receiver`. The - /// channel is closed when all senders have been dropped, or when [`close`] - /// is called. + /// For `limit > 0`, this method will never return `0` unless + /// the channel has been closed and there are no remaining messages + /// in the channel's queue. This indicates that no further values can ever + /// be received from this `Receiver`. The channel is closed when all senders + /// have been dropped, or when [`close`] is called. /// /// The capacity of `buffer` is increased as needed. /// diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index 543de76a426..f88ab1705ee 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -124,6 +124,10 @@ async fn async_send_recv_with_buffer() { #[cfg(feature = "full")] async fn async_send_recv_many_with_buffer() { let (tx, mut rx) = mpsc::channel(2); + let mut buffer = Vec::::with_capacity(3); + + // With `limit=0` does not sleep, returns immediately + assert_eq!(0, rx.recv_many(&mut buffer, 0).await); tokio::spawn(async move { assert_ok!(tx.send(1).await); @@ -132,7 +136,6 @@ async fn async_send_recv_many_with_buffer() { }); let limit = 3; - let mut buffer = Vec::::with_capacity(3); let mut recv_count = 0usize; while recv_count < 3 { let n = rx.recv_many(&mut buffer, limit).await; @@ -206,15 +209,23 @@ async fn send_recv_unbounded() { async fn send_recv_many_unbounded() { let (tx, mut rx) = mpsc::unbounded_channel::(); + let mut buffer: Vec = Vec::new(); + + // With `limit=0` does not sleep, returns immediately + rx.recv_many(&mut buffer, 0).await; + assert_eq!(0, buffer.len()); + assert_ok!(tx.send(7)); assert_ok!(tx.send(13)); assert_ok!(tx.send(100)); assert_ok!(tx.send(1002)); - let mut buffer: Vec = Vec::new(); + rx.recv_many(&mut buffer, 0).await; + assert_eq!(0, buffer.len()); + let mut count = 0; while count < 4 { - count += rx.recv_many(&mut buffer, 0).await; + count += rx.recv_many(&mut buffer, 1).await; } assert_eq!(count, 4); assert_eq!(vec![7, 13, 100, 1002], buffer); @@ -258,7 +269,7 @@ async fn send_recv_many_bounded_capacity() { } tx.send("one more".to_string()).await.unwrap(); - // Here `recv_may` receives all but the last value; + // Here `recv_many` receives all but the last value; // the initial capacity is adequate, so the buffer does // not increase in side. assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer, limit).await); @@ -300,7 +311,7 @@ async fn send_recv_many_unbounded_capacity() { } tx.send("one more".to_string()).unwrap(); - // Here `recv_may` receives all but the last value; + // Here `recv_many` receives all but the last value; // the initial capacity is adequate, so the buffer does // not increase in side. assert_eq!(buffer.capacity(), rx.recv_many(&mut buffer, limit).await); From e8950ce8eac55d9df71bb059553184ea369632b9 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Sun, 8 Oct 2023 14:07:24 -0400 Subject: [PATCH 30/36] Fix doc issues --- tokio/src/sync/mpsc/bounded.rs | 2 +- tokio/src/sync/mpsc/unbounded.rs | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 2cc50ce4175..986b0cccca7 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -244,7 +244,7 @@ impl Receiver { /// still outstanding [`Permits`] from before it was closed, the channel is /// not considered closed by `recv_many` until the permits are released. /// - /// For non-zero values `limit`, this method will never return `0` unless + /// For non-zero values of `limit`, this method will never return `0` unless /// the channel has been closed and there are no remaining messages in the /// channel's queue. This indicates that no further values can ever /// be received from this `Receiver`. The channel is closed when all senders diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 1648a763339..3a0746c9b52 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -179,11 +179,11 @@ impl UnboundedReceiver { /// the function returns immediately with `0`. /// The return value is the number of values added to `buffer`. /// - /// If there are no messages in the channel's queue, but the channel has - /// not yet been closed, this method will sleep until a message is sent or - /// the channel is closed. + /// For `limit > 0`, if there are no messages in the channel's + /// queue, but the channel has not yet been closed, this + /// method will sleep until a message is sent or the channel is closed. /// - /// For `limit > 0`, this method will never return `0` unless + /// For non-zero values of `limit`, this method will never return `0` unless /// the channel has been closed and there are no remaining messages /// in the channel's queue. This indicates that no further values can ever /// be received from this `Receiver`. The channel is closed when all senders From ead4ca662595cae84a3d5ddaaf5652d36aa70c1a Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Tue, 10 Oct 2023 21:40:53 -0400 Subject: [PATCH 31/36] Update tokio/src/sync/mpsc/chan.rs Co-authored-by: Alice Ryhl --- tokio/src/sync/mpsc/chan.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index d72cbab45cb..a423e8467a4 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -322,7 +322,7 @@ impl Rx { let rx_fields = unsafe { &mut *rx_fields_ptr }; macro_rules! try_recv { () => { - while (remaining > 0) { + while remaining > 0 { match rx_fields.list.pop(&self.inner.tx) { Some(Read::Value(value)) => { remaining -= 1; From 2fcb7152fdfb87da14f59e0a2c297580cfaacb40 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Tue, 10 Oct 2023 21:56:18 -0400 Subject: [PATCH 32/36] Update tokio/src/sync/mpsc/chan.rs Co-authored-by: Alice Ryhl --- tokio/src/sync/mpsc/chan.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index a423e8467a4..98fabc1c211 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -506,7 +506,7 @@ impl Semaphore for unbounded::Semaphore { fn add_permits(&self, n: usize) { let prev = self.0.fetch_sub(n << 1, Release); - if prev >> 1 == 0 { + if (prev >> 1) < n { // Something went wrong process::abort(); } From ba46b7cabb51d08f2c7742e0cd912e82d274c3fc Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Tue, 10 Oct 2023 22:02:46 -0400 Subject: [PATCH 33/36] check the coop budget in limit==0 case --- tokio/src/sync/mpsc/chan.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index 98fabc1c211..b30f54742e4 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -307,17 +307,19 @@ impl Rx { ) -> Poll { use super::block::Read; + ready!(crate::trace::trace_leaf(cx)); + + // Keep track of task budget + let coop = ready!(crate::runtime::coop::poll_proceed(cx)); + if limit == 0 { + coop.made_progress(); return Ready(0usize); } let mut remaining = limit; let initial_length = buffer.len(); - ready!(crate::trace::trace_leaf(cx)); - // Keep track of task budget - let coop = ready!(crate::runtime::coop::poll_proceed(cx)); - self.inner.rx_fields.with_mut(|rx_fields_ptr| { let rx_fields = unsafe { &mut *rx_fields_ptr }; macro_rules! try_recv { From b1d1cae030ac605cb1a9cff8ce640c973726ec83 Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Tue, 10 Oct 2023 22:26:04 -0400 Subject: [PATCH 34/36] clean up some recv_many tests --- tokio/tests/sync_mpsc.rs | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index f88ab1705ee..a5c15a4cfc6 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -129,24 +129,23 @@ async fn async_send_recv_many_with_buffer() { // With `limit=0` does not sleep, returns immediately assert_eq!(0, rx.recv_many(&mut buffer, 0).await); - tokio::spawn(async move { + let handle = tokio::spawn(async move { assert_ok!(tx.send(1).await); assert_ok!(tx.send(2).await); assert_ok!(tx.send(7).await); + assert_ok!(tx.send(0).await); }); let limit = 3; let mut recv_count = 0usize; - while recv_count < 3 { - let n = rx.recv_many(&mut buffer, limit).await; - recv_count += n; + while recv_count < 4 { + recv_count += rx.recv_many(&mut buffer, limit).await; assert_eq!(buffer.len(), recv_count); - assert!(buffer.iter().sum::() <= 10); } - assert_eq!(buffer.iter().sum::(), 10); - assert_eq!(3, recv_count); - assert_eq!(recv_count, buffer.len()); - assert_eq!(None, rx.recv().await); + + assert_eq!(vec![1, 2, 7, 0], buffer); + assert_eq!(0, rx.recv_many(&mut buffer, limit).await); + handle.await.unwrap(); } #[tokio::test] @@ -239,7 +238,8 @@ async fn send_recv_many_unbounded() { assert_ok!(tx.send(7)); assert_ok!(tx.send(2)); - count = rx.recv_many(&mut buffer, 4).await; + // Re-use existing capacity + count = rx.recv_many(&mut buffer, 32).await; assert_eq!(final_capacity, buffer.capacity()); assert_eq!(count, 4); @@ -250,7 +250,6 @@ async fn send_recv_many_unbounded() { // recv_many will immediately return zero if the channel // is closed and no more messages are waiting assert_eq!(0, rx.recv_many(&mut buffer, 4).await); - assert_eq!(0, rx.recv_many(&mut buffer, 4).await); assert!(rx.recv().await.is_none()); } From f162ca7140b69e0d76fd3d0c4090ca80e89ac44e Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Wed, 11 Oct 2023 09:33:52 -0400 Subject: [PATCH 35/36] Update tokio/src/sync/mpsc/bounded.rs Co-authored-by: Alice Ryhl --- tokio/src/sync/mpsc/bounded.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 986b0cccca7..a5ec89a2504 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -234,7 +234,7 @@ impl Receiver { /// /// This method extends `buffer` by no more than a fixed number /// of values as specified by `limit`. If `limit` is zero, - /// the function returns immediately with `0`. + /// the function immediately returns `0`. /// The return value is the number of values added to `buffer`. /// /// For `limit > 0`, if there are no messages in the From 9f39dc7b9a4ed30b08fea20488b53c9d96c40abb Mon Sep 17 00:00:00 2001 From: Aaron Schweiger Date: Wed, 11 Oct 2023 09:56:14 -0400 Subject: [PATCH 36/36] Reflow doc comments; adjust comment in chan.rs --- tokio/src/sync/mpsc/bounded.rs | 28 +++++++++++++--------------- tokio/src/sync/mpsc/chan.rs | 6 +++--- tokio/src/sync/mpsc/unbounded.rs | 20 ++++++++++---------- 3 files changed, 26 insertions(+), 28 deletions(-) diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index a5ec89a2504..766fa7f9a57 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -232,24 +232,22 @@ impl Receiver { /// Receives the next values for this receiver and extends `buffer`. /// - /// This method extends `buffer` by no more than a fixed number - /// of values as specified by `limit`. If `limit` is zero, - /// the function immediately returns `0`. - /// The return value is the number of values added to `buffer`. - /// - /// For `limit > 0`, if there are no messages in the - /// channel's queue, but the channel has not yet been closed, this - /// method will sleep until a message is sent or the channel is closed. - /// Note that if [`close`] is called, but there are - /// still outstanding [`Permits`] from before it was closed, the channel is - /// not considered closed by `recv_many` until the permits are released. + /// This method extends `buffer` by no more than a fixed number of values + /// as specified by `limit`. If `limit` is zero, the function immediately + /// returns `0`. The return value is the number of values added to `buffer`. + /// + /// For `limit > 0`, if there are no messages in the channel's queue, but + /// the channel has not yet been closed, this method will sleep until a + /// message is sent or the channel is closed. Note that if [`close`] is + /// called, but there are still outstanding [`Permits`] from before it was + /// closed, the channel is not considered closed by `recv_many` until the + /// permits are released. /// /// For non-zero values of `limit`, this method will never return `0` unless /// the channel has been closed and there are no remaining messages in the - /// channel's queue. This indicates that no further values can ever - /// be received from this `Receiver`. The channel is closed when all senders - /// have been dropped, or when [`close`] - /// is called. + /// channel's queue. This indicates that no further values can ever be + /// received from this `Receiver`. The channel is closed when all senders + /// have been dropped, or when [`close`] is called. /// /// The capacity of `buffer` is increased as needed. /// diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index b30f54742e4..f99059f9557 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -295,10 +295,10 @@ impl Rx { }) } - /// Receives values into `buffer` up to its capacity + /// Receives up to `limit` values into `buffer` /// - /// For `limit` > 0, receives up to limit values into `buffer`. - /// For `limit` = 0, immediately returns Ready(0). + /// For `limit > 0`, receives up to limit values into `buffer`. + /// For `limit == 0`, immediately returns Ready(0). pub(crate) fn recv_many( &mut self, cx: &mut Context<'_>, diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 3a0746c9b52..2c60e7f96cc 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -174,19 +174,19 @@ impl UnboundedReceiver { /// Receives the next values for this receiver and extends `buffer`. /// - /// This method extends `buffer` by no more than a fixed number - /// of values as specified by `limit`. If `limit` is zero, - /// the function returns immediately with `0`. - /// The return value is the number of values added to `buffer`. + /// This method extends `buffer` by no more than a fixed number of values + /// as specified by `limit`. If `limit` is zero, the function returns + /// immediately with `0`. The return value is the number of values added to + /// `buffer`. /// - /// For `limit > 0`, if there are no messages in the channel's - /// queue, but the channel has not yet been closed, this - /// method will sleep until a message is sent or the channel is closed. + /// For `limit > 0`, if there are no messages in the channel's queue, + /// but the channel has not yet been closed, this method will sleep + /// until a message is sent or the channel is closed. /// /// For non-zero values of `limit`, this method will never return `0` unless - /// the channel has been closed and there are no remaining messages - /// in the channel's queue. This indicates that no further values can ever - /// be received from this `Receiver`. The channel is closed when all senders + /// the channel has been closed and there are no remaining messages in the + /// channel's queue. This indicates that no further values can ever be + /// received from this `Receiver`. The channel is closed when all senders /// have been dropped, or when [`close`] is called. /// /// The capacity of `buffer` is increased as needed.