tokio-sync-0.1.7/benches/mpsc.rs010064400007650000024000000320551354770560000147700ustar0000000000000000#![feature(test)] extern crate futures; extern crate test; extern crate tokio_sync; type Medium = [usize; 64]; type Large = [Medium; 64]; mod tokio { use futures::{future, Async, Future, Sink, Stream}; use std::thread; use test::{self, Bencher}; use tokio_sync::mpsc::*; #[bench] fn bounded_new_medium(b: &mut Bencher) { b.iter(|| { let _ = test::black_box(&channel::(1_000)); }) } #[bench] fn unbounded_new_medium(b: &mut Bencher) { b.iter(|| { let _ = test::black_box(&unbounded_channel::()); }) } #[bench] fn bounded_new_large(b: &mut Bencher) { b.iter(|| { let _ = test::black_box(&channel::(1_000)); }) } #[bench] fn unbounded_new_large(b: &mut Bencher) { b.iter(|| { let _ = test::black_box(&unbounded_channel::()); }) } #[bench] fn send_one_message(b: &mut Bencher) { b.iter(|| { let (mut tx, mut rx) = channel(1_000); // Send tx.try_send(1).unwrap(); // Receive assert_eq!(Async::Ready(Some(1)), rx.poll().unwrap()); }) } #[bench] fn send_one_message_large(b: &mut Bencher) { b.iter(|| { let (mut tx, mut rx) = channel::(1_000); // Send let _ = tx.try_send([[0; 64]; 64]); // Receive let _ = test::black_box(&rx.poll()); }) } #[bench] fn bounded_rx_not_ready(b: &mut Bencher) { let (_tx, mut rx) = channel::(1_000); b.iter(|| { future::lazy(|| { assert!(rx.poll().unwrap().is_not_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn bounded_tx_poll_ready(b: &mut Bencher) { let (mut tx, _rx) = channel::(1); b.iter(|| { future::lazy(|| { assert!(tx.poll_ready().unwrap().is_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn bounded_tx_poll_not_ready(b: &mut Bencher) { let (mut tx, _rx) = channel::(1); tx.try_send(1).unwrap(); b.iter(|| { future::lazy(|| { assert!(tx.poll_ready().unwrap().is_not_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn unbounded_rx_not_ready(b: &mut Bencher) { let (_tx, mut rx) = unbounded_channel::(); b.iter(|| { future::lazy(|| { assert!(rx.poll().unwrap().is_not_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn unbounded_rx_not_ready_x5(b: &mut Bencher) { let (_tx, mut rx) = unbounded_channel::(); b.iter(|| { future::lazy(|| { assert!(rx.poll().unwrap().is_not_ready()); assert!(rx.poll().unwrap().is_not_ready()); assert!(rx.poll().unwrap().is_not_ready()); assert!(rx.poll().unwrap().is_not_ready()); assert!(rx.poll().unwrap().is_not_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn bounded_uncontended_1(b: &mut Bencher) { b.iter(|| { let (mut tx, mut rx) = channel(1_000); for i in 0..1000 { tx.try_send(i).unwrap(); // No need to create a task, because poll is not going to park. assert_eq!(Async::Ready(Some(i)), rx.poll().unwrap()); } }) } #[bench] fn bounded_uncontended_1_large(b: &mut Bencher) { b.iter(|| { let (mut tx, mut rx) = channel::(1_000); for i in 0..1000 { let _ = tx.try_send([[i; 64]; 64]); // No need to create a task, because poll is not going to park. let _ = test::black_box(&rx.poll()); } }) } #[bench] fn bounded_uncontended_2(b: &mut Bencher) { b.iter(|| { let (mut tx, mut rx) = channel(1000); for i in 0..1000 { tx.try_send(i).unwrap(); } for i in 0..1000 { // No need to create a task, because poll is not going to park. assert_eq!(Async::Ready(Some(i)), rx.poll().unwrap()); } }) } #[bench] fn contended_unbounded_tx(b: &mut Bencher) { let mut threads = vec![]; let mut txs = vec![]; for _ in 0..4 { let (tx, rx) = ::std::sync::mpsc::channel::>(); txs.push(tx); threads.push(thread::spawn(move || { for mut tx in rx.iter() { for i in 0..1_000 { tx.try_send(i).unwrap(); } } })); } b.iter(|| { // TODO make unbounded let (tx, rx) = channel::(1_000_000); for th in &txs { th.send(tx.clone()).unwrap(); } drop(tx); let rx = rx.wait().take(4 * 1_000); for v in rx { let _ = test::black_box(v); } }); drop(txs); for th in threads { th.join().unwrap(); } } #[bench] fn contended_bounded_tx(b: &mut Bencher) { const THREADS: usize = 4; const ITERS: usize = 100; let mut threads = vec![]; let mut txs = vec![]; for _ in 0..THREADS { let (tx, rx) = ::std::sync::mpsc::channel::>(); txs.push(tx); threads.push(thread::spawn(move || { for tx in rx.iter() { let mut tx = tx.wait(); for i in 0..ITERS { tx.send(i as i32).unwrap(); } } })); } b.iter(|| { let (tx, rx) = channel::(1); for th in &txs { th.send(tx.clone()).unwrap(); } drop(tx); let rx = rx.wait().take(THREADS * ITERS); for v in rx { let _ = test::black_box(v); } }); drop(txs); for th in threads { th.join().unwrap(); } } } mod legacy { use futures::sync::mpsc::*; use futures::{future, Async, Future, Sink, Stream}; use std::thread; use test::{self, Bencher}; #[bench] fn bounded_new_medium(b: &mut Bencher) { b.iter(|| { let _ = test::black_box(&channel::(1_000)); }) } #[bench] fn unbounded_new_medium(b: &mut Bencher) { b.iter(|| { let _ = test::black_box(&unbounded::()); }) } #[bench] fn bounded_new_large(b: &mut Bencher) { b.iter(|| { let _ = test::black_box(&channel::(1_000)); }) } #[bench] fn unbounded_new_large(b: &mut Bencher) { b.iter(|| { let _ = test::black_box(&unbounded::()); }) } #[bench] fn send_one_message(b: &mut Bencher) { b.iter(|| { let (mut tx, mut rx) = channel(1_000); // Send tx.try_send(1).unwrap(); // Receive assert_eq!(Ok(Async::Ready(Some(1))), rx.poll()); }) } #[bench] fn send_one_message_large(b: &mut Bencher) { b.iter(|| { let (mut tx, mut rx) = channel::(1_000); // Send let _ = tx.try_send([[0; 64]; 64]); // Receive let _ = test::black_box(&rx.poll()); }) } #[bench] fn bounded_rx_not_ready(b: &mut Bencher) { let (_tx, mut rx) = channel::(1_000); b.iter(|| { future::lazy(|| { assert!(rx.poll().unwrap().is_not_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn bounded_tx_poll_ready(b: &mut Bencher) { let (mut tx, _rx) = channel::(0); b.iter(|| { future::lazy(|| { assert!(tx.poll_ready().unwrap().is_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn bounded_tx_poll_not_ready(b: &mut Bencher) { let (mut tx, _rx) = channel::(0); tx.try_send(1).unwrap(); b.iter(|| { future::lazy(|| { assert!(tx.poll_ready().unwrap().is_not_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn unbounded_rx_not_ready(b: &mut Bencher) { let (_tx, mut rx) = unbounded::(); b.iter(|| { future::lazy(|| { assert!(rx.poll().unwrap().is_not_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn unbounded_rx_not_ready_x5(b: &mut Bencher) { let (_tx, mut rx) = unbounded::(); b.iter(|| { future::lazy(|| { assert!(rx.poll().unwrap().is_not_ready()); assert!(rx.poll().unwrap().is_not_ready()); assert!(rx.poll().unwrap().is_not_ready()); assert!(rx.poll().unwrap().is_not_ready()); assert!(rx.poll().unwrap().is_not_ready()); Ok::<_, ()>(()) }) .wait() .unwrap(); }) } #[bench] fn unbounded_uncontended_1(b: &mut Bencher) { b.iter(|| { let (tx, mut rx) = unbounded(); for i in 0..1000 { UnboundedSender::unbounded_send(&tx, i).expect("send"); // No need to create a task, because poll is not going to park. assert_eq!(Ok(Async::Ready(Some(i))), rx.poll()); } }) } #[bench] fn unbounded_uncontended_1_large(b: &mut Bencher) { b.iter(|| { let (tx, mut rx) = unbounded::(); for i in 0..1000 { let _ = UnboundedSender::unbounded_send(&tx, [[i; 64]; 64]); // No need to create a task, because poll is not going to park. let _ = test::black_box(&rx.poll()); } }) } #[bench] fn unbounded_uncontended_2(b: &mut Bencher) { b.iter(|| { let (tx, mut rx) = unbounded(); for i in 0..1000 { UnboundedSender::unbounded_send(&tx, i).expect("send"); } for i in 0..1000 { // No need to create a task, because poll is not going to park. assert_eq!(Ok(Async::Ready(Some(i))), rx.poll()); } }) } #[bench] fn multi_thread_unbounded_tx(b: &mut Bencher) { let mut threads = vec![]; let mut txs = vec![]; for _ in 0..4 { let (tx, rx) = ::std::sync::mpsc::channel::>(); txs.push(tx); threads.push(thread::spawn(move || { for mut tx in rx.iter() { for i in 0..1_000 { tx.try_send(i).unwrap(); } } })); } b.iter(|| { let (tx, rx) = channel::(1_000_000); for th in &txs { th.send(tx.clone()).unwrap(); } drop(tx); let rx = rx.wait().take(4 * 1_000); for v in rx { let _ = test::black_box(v); } }); drop(txs); for th in threads { th.join().unwrap(); } } #[bench] fn contended_bounded_tx(b: &mut Bencher) { const THREADS: usize = 4; const ITERS: usize = 100; let mut threads = vec![]; let mut txs = vec![]; for _ in 0..THREADS { let (tx, rx) = ::std::sync::mpsc::channel::>(); txs.push(tx); threads.push(thread::spawn(move || { for tx in rx.iter() { let mut tx = tx.wait(); for i in 0..ITERS { tx.send(i as i32).unwrap(); } } })); } b.iter(|| { let (tx, rx) = channel::(1); for th in &txs { th.send(tx.clone()).unwrap(); } drop(tx); let rx = rx.wait().take(THREADS * ITERS); for v in rx { let _ = test::black_box(v); } }); drop(txs); for th in threads { th.join().unwrap(); } } } tokio-sync-0.1.7/benches/oneshot.rs010064400007650000024000000131611354770560000155020ustar0000000000000000#![feature(test)] extern crate futures; extern crate test; extern crate tokio_sync; mod tokio { use futures::{future, Async, Future}; use test::Bencher; use tokio_sync::oneshot; #[bench] fn new(b: &mut Bencher) { b.iter(|| { let _ = ::test::black_box(&oneshot::channel::()); }) } #[bench] fn same_thread_send_recv(b: &mut Bencher) { b.iter(|| { let (tx, mut rx) = oneshot::channel(); let _ = tx.send(1); assert_eq!(Async::Ready(1), rx.poll().unwrap()); }); } #[bench] fn same_thread_recv_multi_send_recv(b: &mut Bencher) { b.iter(|| { let (tx, mut rx) = oneshot::channel(); future::lazy(|| { let _ = rx.poll(); let _ = rx.poll(); let _ = rx.poll(); let _ = rx.poll(); let _ = tx.send(1); assert_eq!(Async::Ready(1), rx.poll().unwrap()); Ok::<_, ()>(()) }) .wait() .unwrap(); }); } #[bench] fn multi_thread_send_recv(b: &mut Bencher) { const MAX: usize = 10_000_000; use std::thread; fn spin(mut f: F) -> Result { use futures::Async::Ready; loop { match f.poll() { Ok(Ready(v)) => return Ok(v), Ok(_) => {} Err(e) => return Err(e), } } } let mut ping_txs = vec![]; let mut ping_rxs = vec![]; let mut pong_txs = vec![]; let mut pong_rxs = vec![]; for _ in 0..MAX { let (tx, rx) = oneshot::channel::<()>(); ping_txs.push(Some(tx)); ping_rxs.push(Some(rx)); let (tx, rx) = oneshot::channel::<()>(); pong_txs.push(Some(tx)); pong_rxs.push(Some(rx)); } thread::spawn(move || { future::lazy(|| { for i in 0..MAX { let ping_rx = ping_rxs[i].take().unwrap(); let pong_tx = pong_txs[i].take().unwrap(); if spin(ping_rx).is_err() { return Ok(()); } pong_tx.send(()).unwrap(); } Ok::<(), ()>(()) }) .wait() .unwrap(); }); future::lazy(|| { let mut i = 0; b.iter(|| { let ping_tx = ping_txs[i].take().unwrap(); let pong_rx = pong_rxs[i].take().unwrap(); ping_tx.send(()).unwrap(); spin(pong_rx).unwrap(); i += 1; }); Ok::<(), ()>(()) }) .wait() .unwrap(); } } mod legacy { use futures::sync::oneshot; use futures::{future, Async, Future}; use test::Bencher; #[bench] fn new(b: &mut Bencher) { b.iter(|| { let _ = ::test::black_box(&oneshot::channel::()); }) } #[bench] fn same_thread_send_recv(b: &mut Bencher) { b.iter(|| { let (tx, mut rx) = oneshot::channel(); let _ = tx.send(1); assert_eq!(Async::Ready(1), rx.poll().unwrap()); }); } #[bench] fn same_thread_recv_multi_send_recv(b: &mut Bencher) { b.iter(|| { let (tx, mut rx) = oneshot::channel(); future::lazy(|| { let _ = rx.poll(); let _ = rx.poll(); let _ = rx.poll(); let _ = rx.poll(); let _ = tx.send(1); assert_eq!(Async::Ready(1), rx.poll().unwrap()); Ok::<_, ()>(()) }) .wait() .unwrap(); }); } #[bench] fn multi_thread_send_recv(b: &mut Bencher) { const MAX: usize = 10_000_000; use std::thread; fn spin(mut f: F) -> Result { use futures::Async::Ready; loop { match f.poll() { Ok(Ready(v)) => return Ok(v), Ok(_) => {} Err(e) => return Err(e), } } } let mut ping_txs = vec![]; let mut ping_rxs = vec![]; let mut pong_txs = vec![]; let mut pong_rxs = vec![]; for _ in 0..MAX { let (tx, rx) = oneshot::channel::<()>(); ping_txs.push(Some(tx)); ping_rxs.push(Some(rx)); let (tx, rx) = oneshot::channel::<()>(); pong_txs.push(Some(tx)); pong_rxs.push(Some(rx)); } thread::spawn(move || { future::lazy(|| { for i in 0..MAX { let ping_rx = ping_rxs[i].take().unwrap(); let pong_tx = pong_txs[i].take().unwrap(); if spin(ping_rx).is_err() { return Ok(()); } pong_tx.send(()).unwrap(); } Ok::<(), ()>(()) }) .wait() .unwrap(); }); future::lazy(|| { let mut i = 0; b.iter(|| { let ping_tx = ping_txs[i].take().unwrap(); let pong_rx = pong_rxs[i].take().unwrap(); ping_tx.send(()).unwrap(); spin(pong_rx).unwrap(); i += 1; }); Ok::<(), ()>(()) }) .wait() .unwrap(); } } tokio-sync-0.1.7/Cargo.toml.orig010064400007650000024000000014131354771065500147400ustar0000000000000000[package] name = "tokio-sync" # When releasing to crates.io: # - Remove path dependencies # - Update html_root_url. # - Update doc url # - Cargo.toml # - README.md # - Update CHANGELOG.md. # - Create "v0.1.x" git tag. version = "0.1.7" authors = ["Carl Lerche "] license = "MIT" repository = "https://github.com/tokio-rs/tokio" homepage = "https://tokio.rs" documentation = "https://docs.rs/tokio-sync/0.1.7/tokio_sync" description = """ Synchronization utilities. """ categories = ["asynchronous"] [dependencies] fnv = "1.0.6" futures = "0.1.19" [dev-dependencies] env_logger = { version = "0.6", default-features = false } tokio = { version = "0.1.15", path = "../tokio" } tokio-mock-task = "0.1.1" loom = { version = "0.1.1", features = ["futures"] } tokio-sync-0.1.7/Cargo.toml0000644000000022170000000000000111770ustar00# THIS FILE IS AUTOMATICALLY GENERATED BY CARGO # # When uploading crates to the registry Cargo will automatically # "normalize" Cargo.toml files for maximal compatibility # with all versions of Cargo and also rewrite `path` dependencies # to registry (e.g., crates.io) dependencies # # If you believe there's an error in this file please file an # issue against the rust-lang/cargo repository. If you're # editing this file be aware that the upstream Cargo.toml # will likely look very different (and much more reasonable) [package] name = "tokio-sync" version = "0.1.7" authors = ["Carl Lerche "] description = "Synchronization utilities.\n" homepage = "https://tokio.rs" documentation = "https://docs.rs/tokio-sync/0.1.7/tokio_sync" categories = ["asynchronous"] license = "MIT" repository = "https://github.com/tokio-rs/tokio" [dependencies.fnv] version = "1.0.6" [dependencies.futures] version = "0.1.19" [dev-dependencies.env_logger] version = "0.6" default-features = false [dev-dependencies.loom] version = "0.1.1" features = ["futures"] [dev-dependencies.tokio] version = "0.1.15" [dev-dependencies.tokio-mock-task] version = "0.1.1" tokio-sync-0.1.7/CHANGELOG.md010064400007650000024000000020371354771065500136650ustar0000000000000000# 0.1.7 (October 10, 2019) ### Fixed - memory leak when polling oneshot handle from more than one task (#1649). # 0.1.6 (June 4, 2019) ### Added - Add Sync impl for Lock (#1117). # 0.1.5 (April 22, 2019) ### Added - Add asynchronous mutual exclusion primitive (#964). # 0.1.4 (March 13, 2019) ### Fixed - Fix memory leak on channel drop (#917). ### Added - `std::error::Error` implementation for `oneshot`, `watch` error types (#967). # 0.1.3 (March 1, 2019) ### Added - `Watch`, a single value broadcast channel (#922). - `std::error::Error` implementation for more `mpsc` types (#937). # 0.1.2 (February 20, 2019) ### Fixes - `mpsc` and `Semaphore` when releasing permits (#904). - `oneshot` task handle leak (#911). ### Changes - Performance improvements in `AtomicTask` (#892). - Improved assert message when creating a channel with bound of zero (#906). ### Adds - `AtomicTask::take_task` (#895). # 0.1.1 (February 1, 2019) ### Fixes - Panic when creating a channel with bound 0 (#879). # 0.1.0 (January 24, 2019) - Initial Release tokio-sync-0.1.7/LICENSE010064400007650000024000000020461346265360200130530ustar0000000000000000Copyright (c) 2019 Tokio Contributors Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. tokio-sync-0.1.7/README.md010064400007650000024000000007111354770560000133220ustar0000000000000000# tokio-sync Synchronization utilities [Documentation](https://docs.rs/tokio-sync/0.1.6/tokio_sync/) ## Overview This crate provides synchronization utilities for usage with Tokio. ## License This project is licensed under the [MIT license](LICENSE). ### Contribution Unless you explicitly state otherwise, any contribution intentionally submitted for inclusion in Tokio by you, shall be licensed as MIT, without any additional terms or conditions. tokio-sync-0.1.7/src/lib.rs010064400007650000024000000011351354771065500137550ustar0000000000000000#![doc(html_root_url = "https://docs.rs/tokio-sync/0.1.7")] #![deny(missing_debug_implementations, missing_docs, unreachable_pub)] //! Asynchronous synchronization primitives. //! //! This crate provides primitives for synchronizing asynchronous tasks. extern crate fnv; #[macro_use] extern crate futures; macro_rules! debug { ($($t:tt)*) => { if false { println!($($t)*); } } } macro_rules! if_fuzz { ($($t:tt)*) => {{ if false { $($t)* } }} } pub mod lock; mod loom; pub mod mpsc; pub mod oneshot; pub mod semaphore; pub mod task; pub mod watch; tokio-sync-0.1.7/src/lock.rs010064400007650000024000000130051354770560000141300ustar0000000000000000//! An asynchronous `Mutex`-like type. //! //! This module provides [`Lock`], a type that acts similarly to an asynchronous `Mutex`, with one //! major difference: the [`LockGuard`] returned by `poll_lock` is not tied to the lifetime of the //! `Mutex`. This enables you to acquire a lock, and then pass that guard into a future, and then //! release it at some later point in time. //! //! This allows you to do something along the lines of: //! //! ```rust,no_run //! # #[macro_use] //! # extern crate futures; //! # extern crate tokio; //! # use futures::{future, Poll, Async, Future, Stream}; //! use tokio::sync::lock::{Lock, LockGuard}; //! struct MyType { //! lock: Lock, //! } //! //! impl Future for MyType //! where S: Stream + Send + 'static //! { //! type Item = (); //! type Error = (); //! //! fn poll(&mut self) -> Poll { //! match self.lock.poll_lock() { //! Async::Ready(mut guard) => { //! tokio::spawn(future::poll_fn(move || { //! let item = try_ready!(guard.poll().map_err(|_| ())); //! println!("item = {:?}", item); //! Ok(().into()) //! })); //! Ok(().into()) //! }, //! Async::NotReady => Ok(Async::NotReady) //! } //! } //! } //! # fn main() {} //! ``` //! //! [`Lock`]: struct.Lock.html //! [`LockGuard`]: struct.LockGuard.html use futures::Async; use semaphore; use std::cell::UnsafeCell; use std::fmt; use std::ops::{Deref, DerefMut}; use std::sync::Arc; /// An asynchronous mutual exclusion primitive useful for protecting shared data /// /// Each mutex has a type parameter (`T`) which represents the data that it is protecting. The data /// can only be accessed through the RAII guards returned from `poll_lock`, which guarantees that /// the data is only ever accessed when the mutex is locked. #[derive(Debug)] pub struct Lock { inner: Arc>, permit: semaphore::Permit, } /// A handle to a held `Lock`. /// /// As long as you have this guard, you have exclusive access to the underlying `T`. The guard /// internally keeps a reference-couned pointer to the original `Lock`, so even if the lock goes /// away, the guard remains valid. /// /// The lock is automatically released whenever the guard is dropped, at which point `poll_lock` /// will succeed yet again. #[derive(Debug)] pub struct LockGuard(Lock); // As long as T: Send, it's fine to send and share Lock between threads. // If T was not Send, sending and sharing a Lock would be bad, since you can access T through // Lock. unsafe impl Send for Lock where T: Send {} unsafe impl Sync for Lock where T: Send {} unsafe impl Sync for LockGuard where T: Send + Sync {} #[derive(Debug)] struct State { c: UnsafeCell, s: semaphore::Semaphore, } #[test] fn bounds() { fn check() {} check::>(); } impl Lock { /// Creates a new lock in an unlocked state ready for use. pub fn new(t: T) -> Self { Self { inner: Arc::new(State { c: UnsafeCell::new(t), s: semaphore::Semaphore::new(1), }), permit: semaphore::Permit::new(), } } /// Try to acquire the lock. /// /// If the lock is already held, the current task is notified when it is released. pub fn poll_lock(&mut self) -> Async> { if let Async::NotReady = self.permit.poll_acquire(&self.inner.s).unwrap_or_else(|_| { // The semaphore was closed. but, we never explicitly close it, and we have a // handle to it through the Arc, which means that this can never happen. unreachable!() }) { return Async::NotReady; } // We want to move the acquired permit into the guard, // and leave an unacquired one in self. let acquired = Self { inner: self.inner.clone(), permit: ::std::mem::replace(&mut self.permit, semaphore::Permit::new()), }; Async::Ready(LockGuard(acquired)) } } impl Drop for LockGuard { fn drop(&mut self) { if self.0.permit.is_acquired() { self.0.permit.release(&self.0.inner.s); } else if ::std::thread::panicking() { // A guard _should_ always hold its permit, but if the thread is already panicking, // we don't want to generate a panic-while-panicing, since that's just unhelpful! } else { unreachable!("Permit not held when LockGuard was dropped") } } } impl From for Lock { fn from(s: T) -> Self { Self::new(s) } } impl Clone for Lock { fn clone(&self) -> Self { Self { inner: self.inner.clone(), permit: semaphore::Permit::new(), } } } impl Default for Lock where T: Default, { fn default() -> Self { Self::new(T::default()) } } impl Deref for LockGuard { type Target = T; fn deref(&self) -> &Self::Target { assert!(self.0.permit.is_acquired()); unsafe { &*self.0.inner.c.get() } } } impl DerefMut for LockGuard { fn deref_mut(&mut self) -> &mut Self::Target { assert!(self.0.permit.is_acquired()); unsafe { &mut *self.0.inner.c.get() } } } impl fmt::Display for LockGuard { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { fmt::Display::fmt(&**self, f) } } tokio-sync-0.1.7/src/loom.rs010064400007650000024000000014461354770560000141540ustar0000000000000000pub(crate) mod futures { pub(crate) use futures::task; pub(crate) use task::AtomicTask; } pub(crate) mod sync { pub(crate) use std::sync::atomic; pub(crate) use std::sync::Arc; use std::cell::UnsafeCell; pub(crate) struct CausalCell(UnsafeCell); impl CausalCell { pub(crate) fn new(data: T) -> CausalCell { CausalCell(UnsafeCell::new(data)) } pub(crate) fn with(&self, f: F) -> R where F: FnOnce(*const T) -> R, { f(self.0.get()) } pub(crate) fn with_mut(&self, f: F) -> R where F: FnOnce(*mut T) -> R, { f(self.0.get()) } } } pub(crate) fn yield_now() { ::std::sync::atomic::spin_loop_hint(); } tokio-sync-0.1.7/src/mpsc/block.rs010064400007650000024000000307751354770560000152510ustar0000000000000000use loom::{ self, sync::atomic::{AtomicPtr, AtomicUsize}, sync::CausalCell, }; use std::mem::{self, ManuallyDrop}; use std::ops; use std::ptr::{self, NonNull}; use std::sync::atomic::Ordering::{self, AcqRel, Acquire, Release}; /// A block in a linked list. /// /// Each block in the list can hold up to `BLOCK_CAP` messages. pub(crate) struct Block { /// The start index of this block. /// /// Slots in this block have indices in `start_index .. start_index + BLOCK_CAP`. start_index: usize, /// The next block in the linked list. next: AtomicPtr>, /// Bitfield tracking slots that are ready to have their values consumed. ready_slots: AtomicUsize, /// The observed `tail_position` value *after* the block has been passed by /// `block_tail`. observed_tail_position: CausalCell, /// Array containing values pushed into the block. Values are stored in a /// continuous array in order to improve cache line behavior when reading. /// The values must be manually dropped. values: Values, } pub(crate) enum Read { Value(T), Closed, } struct Values([CausalCell>; BLOCK_CAP]); use super::BLOCK_CAP; /// Masks an index to get the block identifier const BLOCK_MASK: usize = !(BLOCK_CAP - 1); /// Masks an index to get the value offset in a block. const SLOT_MASK: usize = BLOCK_CAP - 1; /// Flag tracking that a block has gone through the sender's release routine. /// /// When this is set, the receiver may consider freeing the block. const RELEASED: usize = 1 << BLOCK_CAP; /// Flag tracking all senders dropped. /// /// When this flag is set, the send half of the channel has closed. const TX_CLOSED: usize = RELEASED << 1; /// Mask covering all bits used to track slot readiness. const READY_MASK: usize = RELEASED - 1; /// Returns the index of the first slot in the block referenced by `slot_index`. #[inline(always)] pub(crate) fn start_index(slot_index: usize) -> usize { BLOCK_MASK & slot_index } /// Returns the offset into the block referenced by `slot_index`. #[inline(always)] pub(crate) fn offset(slot_index: usize) -> usize { SLOT_MASK & slot_index } impl Block { pub(crate) fn new(start_index: usize) -> Block { Block { // The absolute index in the channel of the first slot in the block. start_index, // Pointer to the next block in the linked list. next: AtomicPtr::new(ptr::null_mut()), ready_slots: AtomicUsize::new(0), observed_tail_position: CausalCell::new(0), // Value storage values: unsafe { Values::uninitialized() }, } } /// Returns `true` if the block matches the given index pub(crate) fn is_at_index(&self, index: usize) -> bool { debug_assert!(offset(index) == 0); self.start_index == index } /// Returns the number of blocks between `self` and the block at the /// specified index. /// /// `start_index` must represent a block *after* `self`. pub(crate) fn distance(&self, other_index: usize) -> usize { debug_assert!(offset(other_index) == 0); other_index.wrapping_sub(self.start_index) / BLOCK_CAP } /// Read the value at the given offset. /// /// Returns `None` if the slot is empty. /// /// # Safety /// /// To maintain safety, the caller must ensure: /// /// * No concurrent access to the slot. pub(crate) unsafe fn read(&self, slot_index: usize) -> Option> { let offset = offset(slot_index); let ready_bits = self.ready_slots.load(Acquire); if !is_ready(ready_bits, offset) { if is_tx_closed(ready_bits) { return Some(Read::Closed); } return None; } // Get the value let value = self.values[offset].with(|ptr| ptr::read(ptr)); Some(Read::Value(ManuallyDrop::into_inner(value))) } /// Write a value to the block at the given offset. /// /// # Safety /// /// To maintain safety, the caller must ensure: /// /// * The slot is empty. /// * No concurrent access to the slot. pub(crate) unsafe fn write(&self, slot_index: usize, value: T) { // Get the offset into the block let slot_offset = offset(slot_index); self.values[slot_offset].with_mut(|ptr| { ptr::write(ptr, ManuallyDrop::new(value)); }); // Release the value. After this point, the slot ref may no longer // be used. It is possible for the receiver to free the memory at // any point. self.set_ready(slot_offset); } /// Signal to the receiver that the sender half of the list is closed. pub(crate) unsafe fn tx_close(&self) { self.ready_slots.fetch_or(TX_CLOSED, Release); } /// Reset the block to a blank state. This enables reusing blocks in the /// channel. /// /// # Safety /// /// To maintain safety, the caller must ensure: /// /// * All slots are empty. /// * The caller holds a unique pointer to the block. pub(crate) unsafe fn reclaim(&mut self) { self.start_index = 0; self.next = AtomicPtr::new(ptr::null_mut()); self.ready_slots = AtomicUsize::new(0); } /// Release the block to the rx half for freeing. /// /// This function is called by the tx half once it can be guaranteed that no /// more senders will attempt to access the block. /// /// # Safety /// /// To maintain safety, the caller must ensure: /// /// * The block will no longer be accessed by any sender. pub(crate) unsafe fn tx_release(&self, tail_position: usize) { // Track the observed tail_position. Any sender targetting a greater // tail_position is guaranteed to not access this block. self.observed_tail_position .with_mut(|ptr| *ptr = tail_position); // Set the released bit, signalling to the receiver that it is safe to // free the block's memory as soon as all slots **prior** to // `observed_tail_position` have been filled. self.ready_slots.fetch_or(RELEASED, Release); } /// Mark a slot as ready fn set_ready(&self, slot: usize) { let mask = 1 << slot; self.ready_slots.fetch_or(mask, Release); } /// Returns `true` when all slots have their `ready` bits set. /// /// This indicates that the block is in its final state and will no longer /// be mutated. /// /// # Implementation /// /// The implementation walks each slot checking the `ready` flag. It might /// be that it would make more sense to coalesce ready flags as bits in a /// single atomic cell. However, this could have negative impact on cache /// behavior as there would be many more mutations to a single slot. pub(crate) fn is_final(&self) -> bool { self.ready_slots.load(Acquire) & READY_MASK == READY_MASK } /// Returns the `observed_tail_position` value, if set pub(crate) fn observed_tail_position(&self) -> Option { if 0 == RELEASED & self.ready_slots.load(Acquire) { None } else { Some(self.observed_tail_position.with(|ptr| unsafe { *ptr })) } } /// Load the next block pub(crate) fn load_next(&self, ordering: Ordering) -> Option>> { let ret = NonNull::new(self.next.load(ordering)); debug_assert!(unsafe { ret.map(|block| block.as_ref().start_index == self.start_index.wrapping_add(BLOCK_CAP)) .unwrap_or(true) }); ret } /// Push `block` as the next block in the link. /// /// Returns Ok if successful, otherwise, a pointer to the next block in /// the list is returned. /// /// This requires that the next pointer is null. /// /// # Ordering /// /// This performs a compare-and-swap on `next` using AcqRel ordering. /// /// # Safety /// /// To maintain safety, the caller must ensure: /// /// * `block` is not freed until it has been removed from the list. pub(crate) unsafe fn try_push( &self, block: &mut NonNull>, ordering: Ordering, ) -> Result<(), NonNull>> { block.as_mut().start_index = self.start_index.wrapping_add(BLOCK_CAP); let next_ptr = self .next .compare_and_swap(ptr::null_mut(), block.as_ptr(), ordering); match NonNull::new(next_ptr) { Some(next_ptr) => Err(next_ptr), None => Ok(()), } } /// Grow the `Block` linked list by allocating and appending a new block. /// /// The next block in the linked list is returned. This may or may not be /// the one allocated by the function call. /// /// # Implementation /// /// It is assumed that `self.next` is null. A new block is allocated with /// `start_index` set to be the next block. A compare-and-swap is performed /// with AcqRel memory ordering. If the compare-and-swap is successful, the /// newly allocated block is released to other threads walking the block /// linked list. If the compare-and-swap fails, the current thread acquires /// the next block in the linked list, allowing the current thread to access /// the slots. pub(crate) fn grow(&self) -> NonNull> { // Create the new block. It is assumed that the block will become the // next one after `&self`. If this turns out to not be the case, // `start_index` is updated accordingly. let new_block = Box::new(Block::new(self.start_index + BLOCK_CAP)); let mut new_block = unsafe { NonNull::new_unchecked(Box::into_raw(new_block)) }; // Attempt to store the block. The first compare-and-swap attempt is // "unrolled" due to minor differences in logic // // `AcqRel` is used as the ordering **only** when attempting the // compare-and-swap on self.next. // // If the compare-and-swap fails, then the actual value of the cell is // returned from this function and accessed by the caller. Given this, // the memory must be acquired. // // `Release` ensures that the newly allocated block is available to // other threads acquiring the next pointer. let next = NonNull::new(self.next.compare_and_swap( ptr::null_mut(), new_block.as_ptr(), AcqRel, )); let next = match next { Some(next) => next, None => { // The compare-and-swap succeeded and the newly allocated block // is successfully pushed. return new_block; } }; // There already is a next block in the linked list. The newly allocated // block could be dropped and the discovered next block returned; // however, that would be wasteful. Instead, the linked list is walked // by repeatedly attempting to compare-and-swap the pointer into the // `next` register until the compare-and-swap succeed. // // Care is taken to update new_block's start_index field as appropriate. let mut curr = next; // TODO: Should this iteration be capped? loop { let actual = unsafe { curr.as_ref().try_push(&mut new_block, AcqRel) }; curr = match actual { Ok(_) => { return next; } Err(curr) => curr, }; // When running outside of loom, this calls `spin_loop_hint`. loom::yield_now(); } } } /// Returns `true` if the specificed slot has a value ready to be consumed. fn is_ready(bits: usize, slot: usize) -> bool { let mask = 1 << slot; mask == mask & bits } /// Returns `true` if the closed flag has been set. fn is_tx_closed(bits: usize) -> bool { TX_CLOSED == bits & TX_CLOSED } impl Values { unsafe fn uninitialized() -> Values { let mut vals = mem::uninitialized(); // When fuzzing, `CausalCell` needs to be initialized. if_fuzz! { use std::ptr; for v in &mut vals { ptr::write( v as *mut _, CausalCell::new(mem::zeroed())); } } Values(vals) } } impl ops::Index for Values { type Output = CausalCell>; fn index(&self, index: usize) -> &Self::Output { self.0.index(index) } } tokio-sync-0.1.7/src/mpsc/bounded.rs010064400007650000024000000176101354770560000155700ustar0000000000000000use super::chan; use futures::{Poll, Sink, StartSend, Stream}; use std::fmt; /// Send values to the associated `Receiver`. /// /// Instances are created by the [`channel`](fn.channel.html) function. pub struct Sender { chan: chan::Tx, } impl Clone for Sender { fn clone(&self) -> Self { Sender { chan: self.chan.clone(), } } } impl fmt::Debug for Sender { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("Sender") .field("chan", &self.chan) .finish() } } /// Receive values from the associated `Sender`. /// /// Instances are created by the [`channel`](fn.channel.html) function. pub struct Receiver { /// The channel receiver chan: chan::Rx, } impl fmt::Debug for Receiver { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("Receiver") .field("chan", &self.chan) .finish() } } /// Error returned by the `Sender`. #[derive(Debug)] pub struct SendError(()); /// Error returned by `Sender::try_send`. #[derive(Debug)] pub struct TrySendError { kind: ErrorKind, value: T, } #[derive(Debug)] enum ErrorKind { Closed, NoCapacity, } /// Error returned by `Receiver`. #[derive(Debug)] pub struct RecvError(()); /// Create a bounded mpsc channel for communicating between asynchronous tasks, /// returning the sender/receiver halves. /// /// All data sent on `Sender` will become available on `Receiver` in the same /// order as it was sent. /// /// The `Sender` can be cloned to `send` to the same channel from multiple code /// locations. Only one `Receiver` is supported. /// /// If the `Receiver` is disconnected while trying to `send`, the `send` method /// will return a `SendError`. Similarly, if `Sender` is disconnected while /// trying to `recv`, the `recv` method will return a `RecvError`. /// /// # Examples /// /// ```rust /// extern crate futures; /// extern crate tokio; /// /// use tokio::sync::mpsc::channel; /// use tokio::prelude::*; /// use futures::future::lazy; /// /// # fn some_computation() -> impl Future + Send { /// # futures::future::ok::<(), ()>(()) /// # } /// /// tokio::run(lazy(|| { /// let (tx, rx) = channel(100); /// /// tokio::spawn({ /// some_computation() /// .and_then(|value| { /// tx.send(value) /// .map_err(|_| ()) /// }) /// .map(|_| ()) /// .map_err(|_| ()) /// }); /// /// rx.for_each(|value| { /// println!("got value = {:?}", value); /// Ok(()) /// }) /// .map(|_| ()) /// .map_err(|_| ()) /// })); /// ``` pub fn channel(buffer: usize) -> (Sender, Receiver) { assert!(buffer > 0, "mpsc bounded channel requires buffer > 0"); let semaphore = (::semaphore::Semaphore::new(buffer), buffer); let (tx, rx) = chan::channel(semaphore); let tx = Sender::new(tx); let rx = Receiver::new(rx); (tx, rx) } /// Channel semaphore is a tuple of the semaphore implementation and a `usize` /// representing the channel bound. type Semaphore = (::semaphore::Semaphore, usize); impl Receiver { pub(crate) fn new(chan: chan::Rx) -> Receiver { Receiver { chan } } /// Closes the receiving half of a channel, without dropping it. /// /// This prevents any further messages from being sent on the channel while /// still enabling the receiver to drain messages that are buffered. pub fn close(&mut self) { self.chan.close(); } } impl Stream for Receiver { type Item = T; type Error = RecvError; fn poll(&mut self) -> Poll, Self::Error> { self.chan.recv().map_err(|_| RecvError(())) } } impl Sender { pub(crate) fn new(chan: chan::Tx) -> Sender { Sender { chan } } /// Check if the `Sender` is ready to handle a value. /// /// Polls the channel to determine if there is guaranteed capacity to send /// at least one item without waiting. /// /// When `poll_ready` returns `Ready`, the channel reserves capacity for one /// message for this `Sender` instance. The capacity is held until a message /// is send or the `Sender` instance is dropped. Callers should ensure a /// message is sent in a timely fashion in order to not starve other /// `Sender` instances. /// /// # Return value /// /// This method returns: /// /// - `Ok(Async::Ready(_))` if capacity is reserved for a single message. /// - `Ok(Async::NotReady)` if the channel may not have capacity, in which /// case the current task is queued to be notified once /// capacity is available; /// - `Err(SendError)` if the receiver has been dropped. pub fn poll_ready(&mut self) -> Poll<(), SendError> { self.chan.poll_ready().map_err(|_| SendError(())) } /// Attempts to send a message on this `Sender`, returning the message /// if there was an error. pub fn try_send(&mut self, message: T) -> Result<(), TrySendError> { self.chan.try_send(message)?; Ok(()) } } impl Sink for Sender { type SinkItem = T; type SinkError = SendError; fn start_send(&mut self, msg: T) -> StartSend { use futures::Async::*; use futures::AsyncSink; match self.poll_ready()? { Ready(_) => { self.try_send(msg).map_err(|_| SendError(()))?; Ok(AsyncSink::Ready) } NotReady => Ok(AsyncSink::NotReady(msg)), } } fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { use futures::Async::Ready; Ok(Ready(())) } fn close(&mut self) -> Poll<(), Self::SinkError> { use futures::Async::Ready; Ok(Ready(())) } } // ===== impl SendError ===== impl fmt::Display for SendError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for SendError { fn description(&self) -> &str { "channel closed" } } // ===== impl TrySendError ===== impl TrySendError { /// Get the inner value. pub fn into_inner(self) -> T { self.value } /// Did the send fail because the channel has been closed? pub fn is_closed(&self) -> bool { if let ErrorKind::Closed = self.kind { true } else { false } } /// Did the send fail because the channel was at capacity? pub fn is_full(&self) -> bool { if let ErrorKind::NoCapacity = self.kind { true } else { false } } } impl fmt::Display for TrySendError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for TrySendError { fn description(&self) -> &str { match self.kind { ErrorKind::Closed => "channel closed", ErrorKind::NoCapacity => "no available capacity", } } } impl From<(T, chan::TrySendError)> for TrySendError { fn from((value, err): (T, chan::TrySendError)) -> TrySendError { TrySendError { value, kind: match err { chan::TrySendError::Closed => ErrorKind::Closed, chan::TrySendError::NoPermits => ErrorKind::NoCapacity, }, } } } // ===== impl RecvError ===== impl fmt::Display for RecvError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for RecvError { fn description(&self) -> &str { "channel closed" } } tokio-sync-0.1.7/src/mpsc/chan.rs010064400007650000024000000264561354770560000150710ustar0000000000000000use super::list; use futures::Poll; use loom::{ futures::AtomicTask, sync::atomic::AtomicUsize, sync::{Arc, CausalCell}, }; use std::fmt; use std::process; use std::sync::atomic::Ordering::{AcqRel, Relaxed}; /// Channel sender pub(crate) struct Tx { inner: Arc>, permit: S::Permit, } impl fmt::Debug for Tx where S::Permit: fmt::Debug, S: fmt::Debug, { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("Tx") .field("inner", &self.inner) .field("permit", &self.permit) .finish() } } /// Channel receiver pub(crate) struct Rx { inner: Arc>, } impl fmt::Debug for Rx where S: fmt::Debug, { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("Rx").field("inner", &self.inner).finish() } } #[derive(Debug, Eq, PartialEq)] pub(crate) enum TrySendError { Closed, NoPermits, } pub(crate) trait Semaphore { type Permit; fn new_permit() -> Self::Permit; /// The permit is dropped without a value being sent. In this case, the /// permit must be returned to the semaphore. fn drop_permit(&self, permit: &mut Self::Permit); fn is_idle(&self) -> bool; fn add_permit(&self); fn poll_acquire(&self, permit: &mut Self::Permit) -> Poll<(), ()>; fn try_acquire(&self, permit: &mut Self::Permit) -> Result<(), TrySendError>; /// A value was sent into the channel and the permit held by `tx` is /// dropped. In this case, the permit should not immeditely be returned to /// the semaphore. Instead, the permit is returnred to the semaphore once /// the sent value is read by the rx handle. fn forget(&self, permit: &mut Self::Permit); fn close(&self); } struct Chan { /// Handle to the push half of the lock-free list. tx: list::Tx, /// Coordinates access to channel's capacity. semaphore: S, /// Receiver task. Notified when a value is pushed into the channel. rx_task: AtomicTask, /// Tracks the number of outstanding sender handles. /// /// When this drops to zero, the send half of the channel is closed. tx_count: AtomicUsize, /// Only accessed by `Rx` handle. rx_fields: CausalCell>, } impl fmt::Debug for Chan where S: fmt::Debug, { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("Chan") .field("tx", &self.tx) .field("semaphore", &self.semaphore) .field("rx_task", &self.rx_task) .field("tx_count", &self.tx_count) .field("rx_fields", &"...") .finish() } } /// Fields only accessed by `Rx` handle. struct RxFields { /// Channel receiver. This field is only accessed by the `Receiver` type. list: list::Rx, /// `true` if `Rx::close` is called. rx_closed: bool, } impl fmt::Debug for RxFields { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("RxFields") .field("list", &self.list) .field("rx_closed", &self.rx_closed) .finish() } } unsafe impl Send for Chan {} unsafe impl Sync for Chan {} pub(crate) fn channel(semaphore: S) -> (Tx, Rx) where S: Semaphore, { let (tx, rx) = list::channel(); let chan = Arc::new(Chan { tx, semaphore, rx_task: AtomicTask::new(), tx_count: AtomicUsize::new(1), rx_fields: CausalCell::new(RxFields { list: rx, rx_closed: false, }), }); (Tx::new(chan.clone()), Rx::new(chan)) } // ===== impl Tx ===== impl Tx where S: Semaphore, { fn new(chan: Arc>) -> Tx { Tx { inner: chan, permit: S::new_permit(), } } /// TODO: Docs pub(crate) fn poll_ready(&mut self) -> Poll<(), ()> { self.inner.semaphore.poll_acquire(&mut self.permit) } /// Send a message and notify the receiver. pub(crate) fn try_send(&mut self, value: T) -> Result<(), (T, TrySendError)> { if let Err(e) = self.inner.semaphore.try_acquire(&mut self.permit) { return Err((value, e)); } // Push the value self.inner.tx.push(value); // Notify the rx task self.inner.rx_task.notify(); // Release the permit self.inner.semaphore.forget(&mut self.permit); Ok(()) } } impl Clone for Tx where S: Semaphore, { fn clone(&self) -> Tx { // Using a Relaxed ordering here is sufficient as the caller holds a // strong ref to `self`, preventing a concurrent decrement to zero. self.inner.tx_count.fetch_add(1, Relaxed); Tx { inner: self.inner.clone(), permit: S::new_permit(), } } } impl Drop for Tx where S: Semaphore, { fn drop(&mut self) { self.inner.semaphore.drop_permit(&mut self.permit); if self.inner.tx_count.fetch_sub(1, AcqRel) != 1 { return; } // Close the list, which sends a `Close` message self.inner.tx.close(); // Notify the receiver self.inner.rx_task.notify(); } } // ===== impl Rx ===== impl Rx where S: Semaphore, { fn new(chan: Arc>) -> Rx { Rx { inner: chan } } pub(crate) fn close(&mut self) { self.inner.rx_fields.with_mut(|rx_fields_ptr| { let rx_fields = unsafe { &mut *rx_fields_ptr }; if rx_fields.rx_closed { return; } rx_fields.rx_closed = true; }); self.inner.semaphore.close(); } /// Receive the next value pub(crate) fn recv(&mut self) -> Poll, ()> { use super::block::Read::*; use futures::Async::*; 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)) => { self.inner.semaphore.add_permit(); return Ok(Ready(Some(value))); } 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()); return Ok(Ready(None)); } None => {} // fall through } }; } try_recv!(); self.inner.rx_task.register(); // 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!(); debug!( "recv; rx_closed = {:?}; is_idle = {:?}", rx_fields.rx_closed, self.inner.semaphore.is_idle() ); if rx_fields.rx_closed && self.inner.semaphore.is_idle() { Ok(Ready(None)) } else { Ok(NotReady) } }) } } impl Drop for Rx where S: Semaphore, { fn drop(&mut self) { use super::block::Read::Value; self.close(); self.inner.rx_fields.with_mut(|rx_fields_ptr| { let rx_fields = unsafe { &mut *rx_fields_ptr }; while let Some(Value(_)) = rx_fields.list.pop(&self.inner.tx) { self.inner.semaphore.add_permit(); } }) } } // ===== impl Chan ===== impl Drop for Chan { fn drop(&mut self) { use super::block::Read::Value; // Safety: the only owner of the rx fields is Chan, and eing // inside its own Drop means we're the last ones to touch it. self.rx_fields.with_mut(|rx_fields_ptr| { let rx_fields = unsafe { &mut *rx_fields_ptr }; while let Some(Value(_)) = rx_fields.list.pop(&self.tx) {} unsafe { rx_fields.list.free_blocks() }; }); } } use semaphore::TryAcquireError; impl From for TrySendError { fn from(src: TryAcquireError) -> TrySendError { if src.is_closed() { TrySendError::Closed } else if src.is_no_permits() { TrySendError::NoPermits } else { unreachable!(); } } } // ===== impl Semaphore for (::Semaphore, capacity) ===== use semaphore::Permit; impl Semaphore for (::semaphore::Semaphore, usize) { type Permit = Permit; fn new_permit() -> Permit { Permit::new() } fn drop_permit(&self, permit: &mut Permit) { permit.release(&self.0); } fn add_permit(&self) { self.0.add_permits(1) } fn is_idle(&self) -> bool { self.0.available_permits() == self.1 } fn poll_acquire(&self, permit: &mut Permit) -> Poll<(), ()> { permit.poll_acquire(&self.0).map_err(|_| ()) } fn try_acquire(&self, permit: &mut Permit) -> Result<(), TrySendError> { permit.try_acquire(&self.0)?; Ok(()) } fn forget(&self, permit: &mut Self::Permit) { permit.forget() } fn close(&self) { self.0.close(); } } // ===== impl Semaphore for AtomicUsize ===== use std::sync::atomic::Ordering::{Acquire, Release}; use std::usize; impl Semaphore for AtomicUsize { type Permit = (); fn new_permit() {} fn drop_permit(&self, _permit: &mut ()) {} fn add_permit(&self) { let prev = self.fetch_sub(2, Release); if prev >> 1 == 0 { // Something went wrong process::abort(); } } fn is_idle(&self) -> bool { self.load(Acquire) >> 1 == 0 } fn poll_acquire(&self, permit: &mut ()) -> Poll<(), ()> { use futures::Async::Ready; self.try_acquire(permit).map(Ready).map_err(|_| ()) } fn try_acquire(&self, _permit: &mut ()) -> Result<(), TrySendError> { let mut curr = self.load(Acquire); loop { if curr & 1 == 1 { return Err(TrySendError::Closed); } if curr == usize::MAX ^ 1 { // Overflowed the ref count. There is no safe way to recover, so // abort the process. In practice, this should never happen. process::abort() } match self.compare_exchange(curr, curr + 2, AcqRel, Acquire) { Ok(_) => return Ok(()), Err(actual) => { curr = actual; } } } } fn forget(&self, _permit: &mut ()) {} fn close(&self) { self.fetch_or(1, Release); } } tokio-sync-0.1.7/src/mpsc/list.rs010064400007650000024000000257271354770560000151330ustar0000000000000000//! A concurrent, lock-free, FIFO list. use super::block::{self, Block}; use loom::{ self, sync::atomic::{AtomicPtr, AtomicUsize}, }; use std::fmt; use std::ptr::NonNull; use std::sync::atomic::Ordering::{AcqRel, Acquire, Relaxed, Release}; /// List queue transmit handle pub(crate) struct Tx { /// Tail in the `Block` mpmc list. block_tail: AtomicPtr>, /// Position to push the next message. This reference a block and offset /// into the block. tail_position: AtomicUsize, } /// List queue receive handle pub(crate) struct Rx { /// Pointer to the block being processed head: NonNull>, /// Next slot index to process index: usize, /// Pointer to the next block pending release free_head: NonNull>, } pub(crate) fn channel() -> (Tx, Rx) { // Create the initial block shared between the tx and rx halves. let initial_block = Box::new(Block::new(0)); let initial_block_ptr = Box::into_raw(initial_block); let tx = Tx { block_tail: AtomicPtr::new(initial_block_ptr), tail_position: AtomicUsize::new(0), }; let head = NonNull::new(initial_block_ptr).unwrap(); let rx = Rx { head, index: 0, free_head: head, }; (tx, rx) } impl Tx { /// Push a value into the list. pub(crate) fn push(&self, value: T) { // First, claim a slot for the value. `Acquire` is used here to // synchronize with the `fetch_add` in `reclaim_blocks`. let slot_index = self.tail_position.fetch_add(1, Acquire); // Load the current block and write the value let block = self.find_block(slot_index); unsafe { // Write the value to the block block.as_ref().write(slot_index, value); } } /// Close the send half of the list /// /// Similar process as pushing a value, but instead of writing the value & /// setting the ready flag, the TX_CLOSED flag is set on the block. pub(crate) fn close(&self) { // First, claim a slot for the value. This is the last slot that will be // claimed. let slot_index = self.tail_position.fetch_add(1, Acquire); let block = self.find_block(slot_index); unsafe { block.as_ref().tx_close() } } fn find_block(&self, slot_index: usize) -> NonNull> { // The start index of the block that contains `index`. let start_index = block::start_index(slot_index); // The index offset into the block let offset = block::offset(slot_index); // Load the current head of the block let mut block_ptr = self.block_tail.load(Acquire); let block = unsafe { &*block_ptr }; // Calculate the distance between the tail ptr and the target block let distance = block.distance(start_index); // Decide if this call to `find_block` should attempt to update the // `block_tail` pointer. // // Updating `block_tail` is not always performed in order to reduce // contention. // // When set, as the routine walks the linked list, it attempts to update // `block_tail`. If the update cannot be performed, `try_updating_tail` // is unset. let mut try_updating_tail = distance > offset; // Walk the linked list of blocks until the block with `start_index` is // found. loop { let block = unsafe { &(*block_ptr) }; if block.is_at_index(start_index) { return unsafe { NonNull::new_unchecked(block_ptr) }; } let next_block = block .load_next(Acquire) // There is no allocated next block, grow the linked list. .unwrap_or_else(|| block.grow()); // If the block is **not** final, then the tail pointer cannot be // advanced any more. try_updating_tail &= block.is_final(); if try_updating_tail { // Advancing `block_tail` must happen when walking the linked // list. `block_tail` may not advance passed any blocks that are // not "final". At the point a block is finalized, it is unknown // if there are any prior blocks that are unfinalized, which // makes it impossible to advance `block_tail`. // // While walking the linked list, `block_tail` can be advanced // as long as finalized blocks are traversed. // // Release ordering is used to ensure that any subsequent reads // are able to see the memory pointed to by `block_tail`. // // Acquire is not needed as any "actual" value is not accessed. // At this point, the linked list is walked to acquire blocks. let actual = self.block_tail .compare_and_swap(block_ptr, next_block.as_ptr(), Release); if actual == block_ptr { // Synchronize with any senders let tail_position = self.tail_position.fetch_add(0, Release); unsafe { block.tx_release(tail_position); } } else { // A concurrent sender is also working on advancing // `block_tail` and this thread is falling behind. // // Stop trying to advance the tail pointer try_updating_tail = false; } } block_ptr = next_block.as_ptr(); loom::yield_now(); } } pub(crate) unsafe fn reclaim_block(&self, mut block: NonNull>) { debug!("+ reclaim_block({:p})", block); // The block has been removed from the linked list and ownership // is reclaimed. // // Before dropping the block, see if it can be reused by // inserting it back at the end of the linked list. // // First, reset the data block.as_mut().reclaim(); let mut reused = false; // Attempt to insert the block at the end // // Walk at most three times // let curr_ptr = self.block_tail.load(Acquire); // The pointer can never be null debug_assert!(!curr_ptr.is_null()); let mut curr = NonNull::new_unchecked(curr_ptr); // TODO: Unify this logic with Block::grow for _ in 0..3 { match curr.as_ref().try_push(&mut block, AcqRel) { Ok(_) => { reused = true; break; } Err(next) => { curr = next; } } } if !reused { debug!(" + block freed {:p}", block); let _ = Box::from_raw(block.as_ptr()); } } } impl fmt::Debug for Tx { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("Tx") .field("block_tail", &self.block_tail.load(Relaxed)) .field("tail_position", &self.tail_position.load(Relaxed)) .finish() } } impl Rx { /// Pop the next value off the queue pub(crate) fn pop(&mut self, tx: &Tx) -> Option> { // Advance `head`, if needed if !self.try_advancing_head() { debug!("+ !self.try_advancing_head() -> false"); return None; } self.reclaim_blocks(tx); unsafe { let block = self.head.as_ref(); let ret = block.read(self.index); if let Some(block::Read::Value(..)) = ret { self.index = self.index.wrapping_add(1); } ret } } /// Try advancing the block pointer to the block referenced by `self.index`. /// /// Returns `true` if successful, `false` if there is no next block to load. fn try_advancing_head(&mut self) -> bool { let block_index = block::start_index(self.index); loop { let next_block = { let block = unsafe { self.head.as_ref() }; if block.is_at_index(block_index) { return true; } block.load_next(Acquire) }; let next_block = match next_block { Some(next_block) => next_block, None => { return false; } }; self.head = next_block; loom::yield_now(); } } fn reclaim_blocks(&mut self, tx: &Tx) { debug!("+ reclaim_blocks()"); while self.free_head != self.head { unsafe { // Get a handle to the block that will be freed and update // `free_head` to point to the next block. let block = self.free_head; let observed_tail_position = block.as_ref().observed_tail_position(); let required_index = match observed_tail_position { Some(i) => i, None => return, }; if required_index > self.index { return; } // We may read the next pointer with `Relaxed` ordering as it is // guaranteed that the `reclaim_blocks` routine trails the `recv` // routine. Any memory accessed by `reclaim_blocks` has already // been acquired by `recv`. let next_block = block.as_ref().load_next(Relaxed); // Update the free list head self.free_head = next_block.unwrap(); // Push the emptied block onto the back of the queue, making it // available to senders. tx.reclaim_block(block); } loom::yield_now(); } } /// Effectively `Drop` all the blocks. Should only be called once, when /// the list is dropping. pub(super) unsafe fn free_blocks(&mut self) { debug!("+ free_blocks()"); debug_assert_ne!(self.free_head, NonNull::dangling()); let mut cur = Some(self.free_head); #[cfg(debug_assertions)] { // to trigger the debug assert above so as to catch that we // don't call `free_blocks` more than once. self.free_head = NonNull::dangling(); self.head = NonNull::dangling(); } while let Some(block) = cur { cur = block.as_ref().load_next(Relaxed); debug!(" + free: block = {:p}", block); drop(Box::from_raw(block.as_ptr())); } } } impl fmt::Debug for Rx { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("Rx") .field("head", &self.head) .field("index", &self.index) .field("free_head", &self.free_head) .finish() } } tokio-sync-0.1.7/src/mpsc/mod.rs010064400007650000024000000045041354770560000147250ustar0000000000000000//! A multi-producer, single-consumer queue for sending values across //! asynchronous tasks. //! //! Similar to `std`, channel creation provides [`Receiver`] and [`Sender`] //! handles. [`Receiver`] implements `Stream` and allows a task to read values //! out of the channel. If there is no message to read, the current task will be //! notified when a new value is sent. [`Sender`] implements the `Sink` trait //! and allows sending messages into the channel. If the channel is at capacity, //! the send is rejected and the task will be notified when additional capacity //! is available. In other words, the channel provides backpressure. //! //! Unbounded channels are also available using the `unbounded_channel` //! constructor. //! //! # Disconnection //! //! When all [`Sender`] handles have been dropped, it is no longer //! possible to send values into the channel. This is considered the termination //! event of the stream. As such, `Receiver::poll` returns `Ok(Ready(None))`. //! //! If the [`Receiver`] handle is dropped, then messages can no longer //! be read out of the channel. In this case, all further attempts to send will //! result in an error. //! //! # Clean Shutdown //! //! When the [`Receiver`] is dropped, it is possible for unprocessed messages to //! remain in the channel. Instead, it is usually desirable to perform a "clean" //! shutdown. To do this, the receiver first calls `close`, which will prevent //! any further messages to be sent into the channel. Then, the receiver //! consumes the channel to completion, at which point the receiver can be //! dropped. //! //! [`Sender`]: struct.Sender.html //! [`Receiver`]: struct.Receiver.html mod block; mod bounded; mod chan; mod list; mod unbounded; pub use self::bounded::{channel, Receiver, Sender}; pub use self::unbounded::{unbounded_channel, UnboundedReceiver, UnboundedSender}; pub mod error { //! Channel error types pub use super::bounded::{RecvError, SendError, TrySendError}; pub use super::unbounded::{UnboundedRecvError, UnboundedSendError, UnboundedTrySendError}; } /// The number of values a block can contain. /// /// This value must be a power of 2. It also must be smaller than the number of /// bits in `usize`. #[cfg(target_pointer_width = "64")] const BLOCK_CAP: usize = 32; #[cfg(not(target_pointer_width = "64"))] const BLOCK_CAP: usize = 16; tokio-sync-0.1.7/src/mpsc/unbounded.rs010064400007650000024000000123621354770560000161320ustar0000000000000000use super::chan; use futures::{Poll, Sink, StartSend, Stream}; use loom::sync::atomic::AtomicUsize; use std::fmt; /// Send values to the associated `UnboundedReceiver`. /// /// Instances are created by the /// [`unbounded_channel`](fn.unbounded_channel.html) function. pub struct UnboundedSender { chan: chan::Tx, } impl Clone for UnboundedSender { fn clone(&self) -> Self { UnboundedSender { chan: self.chan.clone(), } } } impl fmt::Debug for UnboundedSender { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("UnboundedSender") .field("chan", &self.chan) .finish() } } /// Receive values from the associated `UnboundedSender`. /// /// Instances are created by the /// [`unbounded_channel`](fn.unbounded_channel.html) function. pub struct UnboundedReceiver { /// The channel receiver chan: chan::Rx, } impl fmt::Debug for UnboundedReceiver { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("UnboundedReceiver") .field("chan", &self.chan) .finish() } } /// Error returned by the `UnboundedSender`. #[derive(Debug)] pub struct UnboundedSendError(()); /// Returned by `UnboundedSender::try_send` when the channel has been closed. #[derive(Debug)] pub struct UnboundedTrySendError(T); /// Error returned by `UnboundedReceiver`. #[derive(Debug)] pub struct UnboundedRecvError(()); /// Create an unbounded mpsc channel for communicating between asynchronous /// tasks. /// /// A `send` on this channel will always succeed as long as the receive half has /// not been closed. If the receiver falls behind, messages will be arbitrarily /// buffered. /// /// **Note** that the amount of available system memory is an implicit bound to /// the channel. Using an `unbounded` channel has the ability of causing the /// process to run out of memory. In this case, the process will be aborted. pub fn unbounded_channel() -> (UnboundedSender, UnboundedReceiver) { let (tx, rx) = chan::channel(AtomicUsize::new(0)); let tx = UnboundedSender::new(tx); let rx = UnboundedReceiver::new(rx); (tx, rx) } /// No capacity type Semaphore = AtomicUsize; impl UnboundedReceiver { pub(crate) fn new(chan: chan::Rx) -> UnboundedReceiver { UnboundedReceiver { chan } } /// Closes the receiving half of a channel, without dropping it. /// /// This prevents any further messages from being sent on the channel while /// still enabling the receiver to drain messages that are buffered. pub fn close(&mut self) { self.chan.close(); } } impl Stream for UnboundedReceiver { type Item = T; type Error = UnboundedRecvError; fn poll(&mut self) -> Poll, Self::Error> { self.chan.recv().map_err(|_| UnboundedRecvError(())) } } impl UnboundedSender { pub(crate) fn new(chan: chan::Tx) -> UnboundedSender { UnboundedSender { chan } } /// Attempts to send a message on this `UnboundedSender` without blocking. pub fn try_send(&mut self, message: T) -> Result<(), UnboundedTrySendError> { self.chan.try_send(message)?; Ok(()) } } impl Sink for UnboundedSender { type SinkItem = T; type SinkError = UnboundedSendError; fn start_send(&mut self, msg: T) -> StartSend { use futures::AsyncSink; self.try_send(msg).map_err(|_| UnboundedSendError(()))?; Ok(AsyncSink::Ready) } fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { use futures::Async::Ready; Ok(Ready(())) } fn close(&mut self) -> Poll<(), Self::SinkError> { use futures::Async::Ready; Ok(Ready(())) } } // ===== impl UnboundedSendError ===== impl fmt::Display for UnboundedSendError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for UnboundedSendError { fn description(&self) -> &str { "channel closed" } } // ===== impl TrySendError ===== impl UnboundedTrySendError { /// Get the inner value. pub fn into_inner(self) -> T { self.0 } } impl fmt::Display for UnboundedTrySendError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for UnboundedTrySendError { fn description(&self) -> &str { "channel closed" } } impl From<(T, chan::TrySendError)> for UnboundedTrySendError { fn from((value, err): (T, chan::TrySendError)) -> UnboundedTrySendError { assert_eq!(chan::TrySendError::Closed, err); UnboundedTrySendError(value) } } // ===== impl UnboundedRecvError ===== impl fmt::Display for UnboundedRecvError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for UnboundedRecvError { fn description(&self) -> &str { "channel closed" } } tokio-sync-0.1.7/src/oneshot.rs010064400007650000024000000364331354770560000146710ustar0000000000000000//! A channel for sending a single message between asynchronous tasks. use loom::{ futures::task::{self, Task}, sync::atomic::AtomicUsize, sync::CausalCell, }; use futures::{Async, Future, Poll}; use std::fmt; use std::mem::{self, ManuallyDrop}; use std::sync::atomic::Ordering::{self, AcqRel, Acquire}; use std::sync::Arc; /// Sends a value to the associated `Receiver`. /// /// Instances are created by the [`channel`](fn.channel.html) function. #[derive(Debug)] pub struct Sender { inner: Option>>, } /// Receive a value from the associated `Sender`. /// /// Instances are created by the [`channel`](fn.channel.html) function. #[derive(Debug)] pub struct Receiver { inner: Option>>, } pub mod error { //! Oneshot error types use std::fmt; /// Error returned by the `Future` implementation for `Receiver`. #[derive(Debug)] pub struct RecvError(pub(super) ()); /// Error returned by the `try_recv` function on `Receiver`. #[derive(Debug)] pub struct TryRecvError(pub(super) ()); // ===== impl RecvError ===== impl fmt::Display for RecvError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for RecvError { fn description(&self) -> &str { "channel closed" } } // ===== impl TryRecvError ===== impl fmt::Display for TryRecvError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for TryRecvError { fn description(&self) -> &str { "channel closed" } } } use self::error::*; struct Inner { /// Manages the state of the inner cell state: AtomicUsize, /// The value. This is set by `Sender` and read by `Receiver`. The state of /// the cell is tracked by `state`. value: CausalCell>, /// The task to notify when the receiver drops without consuming the value. tx_task: CausalCell>, /// The task to notify when the value is sent. rx_task: CausalCell>, } #[derive(Clone, Copy)] struct State(usize); /// Create a new one-shot channel for sending single values across asynchronous /// tasks. /// /// The function returns separate "send" and "receive" handles. The `Sender` /// handle is used by the producer to send the value. The `Receiver` handle is /// used by the consumer to receive the value. /// /// Each handle can be used on separate tasks. /// /// # Examples /// /// ``` /// extern crate futures; /// extern crate tokio; /// /// use tokio::sync::oneshot; /// use futures::Future; /// use std::thread; /// /// let (sender, receiver) = oneshot::channel::(); /// /// # let t = /// thread::spawn(|| { /// let future = receiver.map(|i| { /// println!("got: {:?}", i); /// }); /// // ... /// # return future; /// }); /// /// sender.send(3).unwrap(); /// # t.join().unwrap().wait().unwrap(); /// ``` pub fn channel() -> (Sender, Receiver) { let inner = Arc::new(Inner { state: AtomicUsize::new(State::new().as_usize()), value: CausalCell::new(None), tx_task: CausalCell::new(ManuallyDrop::new(unsafe { mem::uninitialized() })), rx_task: CausalCell::new(ManuallyDrop::new(unsafe { mem::uninitialized() })), }); let tx = Sender { inner: Some(inner.clone()), }; let rx = Receiver { inner: Some(inner) }; (tx, rx) } impl Sender { /// Completes this oneshot with a successful result. /// /// The function consumes `self` and notifies the `Receiver` handle that a /// value is ready to be received. /// /// If the value is successfully enqueued for the remote end to receive, /// then `Ok(())` is returned. If the receiving end was dropped before this /// function was called, however, then `Err` is returned with the value /// provided. pub fn send(mut self, t: T) -> Result<(), T> { let inner = self.inner.take().unwrap(); inner.value.with_mut(|ptr| unsafe { *ptr = Some(t); }); if !inner.complete() { return Err(inner .value .with_mut(|ptr| unsafe { (*ptr).take() }.unwrap())); } Ok(()) } /// Check if the associated [`Receiver`] handle has been dropped. /// /// # Return values /// /// If `Ok(Ready)` is returned then the associated `Receiver` has been /// dropped, which means any work required for sending should be canceled. /// /// If `Ok(NotReady)` is returned then the associated `Receiver` is still /// alive and may be able to receive a message if sent. The current task is /// registered to receive a notification if the `Receiver` handle goes away. /// /// [`Receiver`]: struct.Receiver.html pub fn poll_close(&mut self) -> Poll<(), ()> { let inner = self.inner.as_ref().unwrap(); let mut state = State::load(&inner.state, Acquire); if state.is_closed() { return Ok(Async::Ready(())); } if state.is_tx_task_set() { let will_notify = inner .tx_task .with(|ptr| unsafe { (&*ptr).will_notify_current() }); if !will_notify { state = State::unset_tx_task(&inner.state); if state.is_closed() { // Set the flag again so that the waker is released in drop State::set_tx_task(&inner.state); return Ok(Async::Ready(())); } else { unsafe { inner.drop_tx_task() }; } } } if !state.is_tx_task_set() { // Attempt to set the task unsafe { inner.set_tx_task(); } // Update the state state = State::set_tx_task(&inner.state); if state.is_closed() { return Ok(Async::Ready(())); } } Ok(Async::NotReady) } /// Check if the associated [`Receiver`] handle has been dropped. /// /// Unlike [`poll_close`], this function does not register a task for /// wakeup upon close. /// /// [`Receiver`]: struct.Receiver.html /// [`poll_close`]: struct.Sender.html#method.poll_close pub fn is_closed(&self) -> bool { let inner = self.inner.as_ref().unwrap(); let state = State::load(&inner.state, Acquire); state.is_closed() } } impl Drop for Sender { fn drop(&mut self) { if let Some(inner) = self.inner.as_ref() { inner.complete(); } } } impl Receiver { /// Prevent the associated [`Sender`] handle from sending a value. /// /// Any `send` operation which happens after calling `close` is guaranteed /// to fail. After calling `close`, `Receiver::poll`] should be called to /// receive a value if one was sent **before** the call to `close` /// completed. /// /// [`Sender`]: struct.Sender.html pub fn close(&mut self) { let inner = self.inner.as_ref().unwrap(); inner.close(); } /// Attempts to receive a value outside of the context of a task. /// /// Does not register a task if no value has been sent. /// /// A return value of `None` must be considered immediately stale (out of /// date) unless [`close`] has been called first. /// /// Returns an error if the sender was dropped. /// /// [`close`]: #method.close pub fn try_recv(&mut self) -> Result { let result = if let Some(inner) = self.inner.as_ref() { let state = State::load(&inner.state, Acquire); if state.is_complete() { match unsafe { inner.consume_value() } { Some(value) => Ok(value), None => Err(TryRecvError(())), } } else if state.is_closed() { Err(TryRecvError(())) } else { // Not ready, this does not clear `inner` return Err(TryRecvError(())); } } else { panic!("called after complete"); }; self.inner = None; result } } impl Drop for Receiver { fn drop(&mut self) { if let Some(inner) = self.inner.as_ref() { inner.close(); } } } impl Future for Receiver { type Item = T; type Error = RecvError; fn poll(&mut self) -> Poll { use futures::Async::{NotReady, Ready}; // If `inner` is `None`, then `poll()` has already completed. let ret = if let Some(inner) = self.inner.as_ref() { match inner.poll_recv() { Ok(Ready(v)) => Ok(Ready(v)), Ok(NotReady) => return Ok(NotReady), Err(e) => Err(e), } } else { panic!("called after complete"); }; self.inner = None; ret } } impl Inner { fn complete(&self) -> bool { let prev = State::set_complete(&self.state); if prev.is_closed() { return false; } if prev.is_rx_task_set() { self.rx_task.with(|ptr| unsafe { (&*ptr).notify() }); } true } fn poll_recv(&self) -> Poll { use futures::Async::{NotReady, Ready}; // Load the state let mut state = State::load(&self.state, Acquire); if state.is_complete() { match unsafe { self.consume_value() } { Some(value) => Ok(Ready(value)), None => Err(RecvError(())), } } else if state.is_closed() { Err(RecvError(())) } else { if state.is_rx_task_set() { let will_notify = self .rx_task .with(|ptr| unsafe { (&*ptr).will_notify_current() }); // Check if the task is still the same if !will_notify { // Unset the task state = State::unset_rx_task(&self.state); if state.is_complete() { // Set the flag again so that the waker is released in drop State::set_rx_task(&self.state); return match unsafe { self.consume_value() } { Some(value) => Ok(Ready(value)), None => Err(RecvError(())), }; } else { unsafe { self.drop_rx_task() }; } } } if !state.is_rx_task_set() { // Attempt to set the task unsafe { self.set_rx_task(); } // Update the state state = State::set_rx_task(&self.state); if state.is_complete() { match unsafe { self.consume_value() } { Some(value) => Ok(Ready(value)), None => Err(RecvError(())), } } else { return Ok(NotReady); } } else { return Ok(NotReady); } } } /// Called by `Receiver` to indicate that the value will never be received. fn close(&self) { let prev = State::set_closed(&self.state); if prev.is_tx_task_set() && !prev.is_complete() { self.tx_task.with(|ptr| unsafe { (&*ptr).notify() }); } } /// Consume the value. This function does not check `state`. unsafe fn consume_value(&self) -> Option { self.value.with_mut(|ptr| (*ptr).take()) } unsafe fn drop_rx_task(&self) { self.rx_task.with_mut(|ptr| ManuallyDrop::drop(&mut *ptr)) } unsafe fn drop_tx_task(&self) { self.tx_task.with_mut(|ptr| ManuallyDrop::drop(&mut *ptr)) } unsafe fn set_rx_task(&self) { self.rx_task .with_mut(|ptr| *ptr = ManuallyDrop::new(task::current())); } unsafe fn set_tx_task(&self) { self.tx_task .with_mut(|ptr| *ptr = ManuallyDrop::new(task::current())); } } unsafe impl Send for Inner {} unsafe impl Sync for Inner {} impl Drop for Inner { fn drop(&mut self) { let state = State(*self.state.get_mut()); if state.is_rx_task_set() { self.rx_task.with_mut(|ptr| unsafe { ManuallyDrop::drop(&mut *ptr); }); } if state.is_tx_task_set() { self.tx_task.with_mut(|ptr| unsafe { ManuallyDrop::drop(&mut *ptr); }); } } } impl fmt::Debug for Inner { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::sync::atomic::Ordering::Relaxed; fmt.debug_struct("Inner") .field("state", &State::load(&self.state, Relaxed)) .finish() } } const RX_TASK_SET: usize = 0b00001; const VALUE_SENT: usize = 0b00010; const CLOSED: usize = 0b00100; const TX_TASK_SET: usize = 0b01000; impl State { fn new() -> State { State(0) } fn is_complete(&self) -> bool { self.0 & VALUE_SENT == VALUE_SENT } fn set_complete(cell: &AtomicUsize) -> State { // TODO: This could be `Release`, followed by an `Acquire` fence *if* // the `RX_TASK_SET` flag is set. However, `loom` does not support // fences yet. let val = cell.fetch_or(VALUE_SENT, AcqRel); State(val) } fn is_rx_task_set(&self) -> bool { self.0 & RX_TASK_SET == RX_TASK_SET } fn set_rx_task(cell: &AtomicUsize) -> State { let val = cell.fetch_or(RX_TASK_SET, AcqRel); State(val | RX_TASK_SET) } fn unset_rx_task(cell: &AtomicUsize) -> State { let val = cell.fetch_and(!RX_TASK_SET, AcqRel); State(val & !RX_TASK_SET) } fn is_closed(&self) -> bool { self.0 & CLOSED == CLOSED } fn set_closed(cell: &AtomicUsize) -> State { // Acquire because we want all later writes (attempting to poll) to be // ordered after this. let val = cell.fetch_or(CLOSED, Acquire); State(val) } fn set_tx_task(cell: &AtomicUsize) -> State { let val = cell.fetch_or(TX_TASK_SET, AcqRel); State(val | TX_TASK_SET) } fn unset_tx_task(cell: &AtomicUsize) -> State { let val = cell.fetch_and(!TX_TASK_SET, AcqRel); State(val & !TX_TASK_SET) } fn is_tx_task_set(&self) -> bool { self.0 & TX_TASK_SET == TX_TASK_SET } fn as_usize(self) -> usize { self.0 } fn load(cell: &AtomicUsize, order: Ordering) -> State { let val = cell.load(order); State(val) } } impl fmt::Debug for State { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("State") .field("is_complete", &self.is_complete()) .field("is_closed", &self.is_closed()) .field("is_rx_task_set", &self.is_rx_task_set()) .field("is_tx_task_set", &self.is_tx_task_set()) .finish() } } tokio-sync-0.1.7/src/semaphore.rs010064400007650000024000001040471354770560000151720ustar0000000000000000//! Thread-safe, asynchronous counting semaphore. //! //! A `Semaphore` instance holds a set of permits. Permits are used to //! synchronize access to a shared resource. //! //! Before accessing the shared resource, callers acquire a permit from the //! semaphore. Once the permit is acquired, the caller then enters the critical //! section. If no permits are available, then acquiring the semaphore returns //! `NotReady`. The task is notified once a permit becomes available. use loom::{ futures::AtomicTask, sync::{ atomic::{AtomicPtr, AtomicUsize}, CausalCell, }, yield_now, }; use futures::Poll; use std::fmt; use std::ptr::{self, NonNull}; use std::sync::atomic::Ordering::{self, AcqRel, Acquire, Relaxed, Release}; use std::sync::Arc; use std::usize; /// Futures-aware semaphore. pub struct Semaphore { /// Tracks both the waiter queue tail pointer and the number of remaining /// permits. state: AtomicUsize, /// waiter queue head pointer. head: CausalCell>, /// Coordinates access to the queue head. rx_lock: AtomicUsize, /// Stub waiter node used as part of the MPSC channel algorithm. stub: Box, } /// A semaphore permit /// /// Tracks the lifecycle of a semaphore permit. /// /// An instance of `Permit` is intended to be used with a **single** instance of /// `Semaphore`. Using a single instance of `Permit` with multiple semaphore /// instances will result in unexpected behavior. /// /// `Permit` does **not** release the permit back to the semaphore on drop. It /// is the user's responsibility to ensure that `Permit::release` is called /// before dropping the permit. #[derive(Debug)] pub struct Permit { waiter: Option>, state: PermitState, } /// Error returned by `Permit::poll_acquire`. #[derive(Debug)] pub struct AcquireError(()); /// Error returned by `Permit::try_acquire`. #[derive(Debug)] pub struct TryAcquireError { kind: ErrorKind, } #[derive(Debug)] enum ErrorKind { Closed, NoPermits, } /// Node used to notify the semaphore waiter when permit is available. #[derive(Debug)] struct WaiterNode { /// Stores waiter state. /// /// See `NodeState` for more details. state: AtomicUsize, /// Task to notify when a permit is made available. task: AtomicTask, /// Next pointer in the queue of waiting senders. next: AtomicPtr, } /// Semaphore state /// /// The 2 low bits track the modes. /// /// - Closed /// - Full /// /// When not full, the rest of the `usize` tracks the total number of messages /// in the channel. When full, the rest of the `usize` is a pointer to the tail /// of the "waiting senders" queue. #[derive(Copy, Clone)] struct SemState(usize); /// Permit state #[derive(Debug, Copy, Clone, Eq, PartialEq)] enum PermitState { /// The permit has not been requested. Idle, /// Currently waiting for a permit to be made available and assigned to the /// waiter. Waiting, /// The permit has been acquired. Acquired, } /// Waiter node state #[derive(Debug, Copy, Clone, Eq, PartialEq)] #[repr(usize)] enum NodeState { /// Not waiting for a permit and the node is not in the wait queue. /// /// This is the initial state. Idle = 0, /// Not waiting for a permit but the node is in the wait queue. /// /// This happens when the waiter has previously requested a permit, but has /// since canceled the request. The node cannot be removed by the waiter, so /// this state informs the receiver to skip the node when it pops it from /// the wait queue. Queued = 1, /// Waiting for a permit and the node is in the wait queue. QueuedWaiting = 2, /// The waiter has been assigned a permit and the node has been removed from /// the queue. Assigned = 3, /// The semaphore has been closed. No more permits will be issued. Closed = 4, } // ===== impl Semaphore ===== impl Semaphore { /// Creates a new semaphore with the initial number of permits /// /// # Panics /// /// Panics if `permits` is zero. pub fn new(permits: usize) -> Semaphore { let stub = Box::new(WaiterNode::new()); let ptr = NonNull::new(&*stub as *const _ as *mut _).unwrap(); // Allocations are aligned debug_assert!(ptr.as_ptr() as usize & NUM_FLAG == 0); let state = SemState::new(permits, &stub); Semaphore { state: AtomicUsize::new(state.to_usize()), head: CausalCell::new(ptr), rx_lock: AtomicUsize::new(0), stub, } } /// Returns the current number of available permits pub fn available_permits(&self) -> usize { let curr = SemState::load(&self.state, Acquire); curr.available_permits() } /// Poll for a permit fn poll_permit(&self, mut permit: Option<&mut Permit>) -> Poll<(), AcquireError> { use futures::Async::*; // Load the current state let mut curr = SemState::load(&self.state, Acquire); debug!(" + poll_permit; sem-state = {:?}", curr); // Tracks a *mut WaiterNode representing an Arc clone. // // This avoids having to bump the ref count unless required. let mut maybe_strong: Option> = None; macro_rules! undo_strong { () => { if let Some(waiter) = maybe_strong { // The waiter was cloned, but never got queued. // Before entering `poll_permit`, the waiter was in the // `Idle` state. We must transition the node back to the // idle state. let waiter = unsafe { Arc::from_raw(waiter.as_ptr()) }; waiter.revert_to_idle(); } }; } loop { let mut next = curr; if curr.is_closed() { undo_strong!(); return Err(AcquireError::closed()); } if !next.acquire_permit(&self.stub) { debug!(" + poll_permit -- no permits"); debug_assert!(curr.waiter().is_some()); if maybe_strong.is_none() { if let Some(ref mut permit) = permit { // Get the Sender's waiter node, or initialize one let waiter = permit .waiter .get_or_insert_with(|| Arc::new(WaiterNode::new())); waiter.register(); debug!(" + poll_permit -- to_queued_waiting"); if !waiter.to_queued_waiting() { debug!(" + poll_permit; waiter already queued"); // The node is alrady queued, there is no further work // to do. return Ok(NotReady); } maybe_strong = Some(WaiterNode::into_non_null(waiter.clone())); } else { // If no `waiter`, then the task is not registered and there // is no further work to do. return Ok(NotReady); } } next.set_waiter(maybe_strong.unwrap()); } debug!(" + poll_permit -- pre-CAS; next = {:?}", next); debug_assert_ne!(curr.0, 0); debug_assert_ne!(next.0, 0); match next.compare_exchange(&self.state, curr, AcqRel, Acquire) { Ok(_) => { debug!(" + poll_permit -- CAS ok"); match curr.waiter() { Some(prev_waiter) => { let waiter = maybe_strong.unwrap(); // Finish pushing unsafe { prev_waiter.as_ref().next.store(waiter.as_ptr(), Release); } debug!(" + poll_permit -- waiter pushed"); return Ok(NotReady); } None => { debug!(" + poll_permit -- permit acquired"); undo_strong!(); return Ok(Ready(())); } } } Err(actual) => { curr = actual; } } } } /// Close the semaphore. This prevents the semaphore from issuing new /// permits and notifies all pending waiters. pub fn close(&self) { debug!("+ Semaphore::close"); // Acquire the `rx_lock`, setting the "closed" flag on the lock. let prev = self.rx_lock.fetch_or(1, AcqRel); debug!(" + close -- rx_lock.fetch_add(1)"); if prev != 0 { debug!("+ close -- locked; prev = {}", prev); // Another thread has the lock and will be responsible for notifying // pending waiters. return; } self.add_permits_locked(0, true); } /// Add `n` new permits to the semaphore. pub fn add_permits(&self, n: usize) { debug!(" + add_permits; n = {}", n); if n == 0 { return; } // TODO: Handle overflow. A panic is not sufficient, the process must // abort. let prev = self.rx_lock.fetch_add(n << 1, AcqRel); debug!(" + add_permits; rx_lock.fetch_add(n << 1); n = {}", n); if prev != 0 { debug!(" + add_permits -- locked; prev = {}", prev); // Another thread has the lock and will be responsible for notifying // pending waiters. return; } self.add_permits_locked(n, false); } fn add_permits_locked(&self, mut rem: usize, mut closed: bool) { while rem > 0 || closed { debug!( " + add_permits_locked -- iter; rem = {}; closed = {:?}", rem, closed ); if closed { SemState::fetch_set_closed(&self.state, AcqRel); } // Release the permits and notify self.add_permits_locked2(rem, closed); let n = rem << 1; let actual = if closed { let actual = self.rx_lock.fetch_sub(n | 1, AcqRel); debug!( " + add_permits_locked; rx_lock.fetch_sub(n | 1); n = {}; actual={}", n, actual ); closed = false; actual } else { let actual = self.rx_lock.fetch_sub(n, AcqRel); debug!( " + add_permits_locked; rx_lock.fetch_sub(n); n = {}; actual={}", n, actual ); closed = actual & 1 == 1; actual }; rem = (actual >> 1) - rem; } debug!(" + add_permits; done"); } /// Release a specific amount of permits to the semaphore /// /// This function is called by `add_permits` after the add lock has been /// acquired. fn add_permits_locked2(&self, mut n: usize, closed: bool) { while n > 0 || closed { let waiter = match self.pop(n, closed) { Some(waiter) => waiter, None => { return; } }; debug!(" + release_n -- notify"); if waiter.notify(closed) { n = n.saturating_sub(1); debug!(" + release_n -- dec"); } } } /// Pop a waiter /// /// `rem` represents the remaining number of times the caller will pop. If /// there are no more waiters to pop, `rem` is used to set the available /// permits. fn pop(&self, rem: usize, closed: bool) -> Option> { debug!(" + pop; rem = {}", rem); 'outer: loop { unsafe { let mut head = self.head.with(|head| *head); let mut next_ptr = head.as_ref().next.load(Acquire); let stub = self.stub(); if head == stub { debug!(" + pop; head == stub"); let next = match NonNull::new(next_ptr) { Some(next) => next, None => { // This loop is not part of the standard intrusive mpsc // channel algorithm. This is where we atomically pop // the last task and add `rem` to the remaining capacity. // // This modification to the pop algorithm works because, // at this point, we have not done any work (only done // reading). We have a *pretty* good idea that there is // no concurrent pusher. // // The capacity is then atomically added by doing an // AcqRel CAS on `state`. The `state` cell is the // linchpin of the algorithm. // // By successfully CASing `head` w/ AcqRel, we ensure // that, if any thread was racing and entered a push, we // see that and abort pop, retrying as it is // "inconsistent". let mut curr = SemState::load(&self.state, Acquire); loop { if curr.has_waiter(&self.stub) { // Inconsistent debug!(" + pop; inconsistent 1"); yield_now(); continue 'outer; } // When closing the semaphore, nodes are popped // with `rem == 0`. In this case, we are not // adding permits, but notifying waiters of the // semaphore's closed state. if rem == 0 { debug_assert!(curr.is_closed(), "state = {:?}", curr); return None; } let mut next = curr; next.release_permits(rem, &self.stub); match next.compare_exchange(&self.state, curr, AcqRel, Acquire) { Ok(_) => return None, Err(actual) => { curr = actual; } } } } }; debug!(" + pop; got next waiter"); self.head.with_mut(|head| *head = next); head = next; next_ptr = next.as_ref().next.load(Acquire); } if let Some(next) = NonNull::new(next_ptr) { self.head.with_mut(|head| *head = next); return Some(Arc::from_raw(head.as_ptr())); } let state = SemState::load(&self.state, Acquire); // This must always be a pointer as the wait list is not empty. let tail = state.waiter().unwrap(); if tail != head { // Inconsistent debug!(" + pop; inconsistent 2"); yield_now(); continue 'outer; } self.push_stub(closed); next_ptr = head.as_ref().next.load(Acquire); if let Some(next) = NonNull::new(next_ptr) { self.head.with_mut(|head| *head = next); return Some(Arc::from_raw(head.as_ptr())); } // Inconsistent state, loop debug!(" + pop; inconsistent 3"); yield_now(); } } } unsafe fn push_stub(&self, closed: bool) { let stub = self.stub(); // Set the next pointer. This does not require an atomic operation as // this node is not accessible. The write will be flushed with the next // operation stub.as_ref().next.store(ptr::null_mut(), Relaxed); // Update the tail to point to the new node. We need to see the previous // node in order to update the next pointer as well as release `task` // to any other threads calling `push`. let prev = SemState::new_ptr(stub, closed).swap(&self.state, AcqRel); debug_assert_eq!(closed, prev.is_closed()); // The stub is only pushed when there are pending tasks. Because of // this, the state must *always* be in pointer mode. let prev = prev.waiter().unwrap(); // We don't want the *existing* pointer to be a stub. debug_assert_ne!(prev, stub); // Release `task` to the consume end. prev.as_ref().next.store(stub.as_ptr(), Release); } fn stub(&self) -> NonNull { unsafe { NonNull::new_unchecked(&*self.stub as *const _ as *mut _) } } } impl fmt::Debug for Semaphore { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("Semaphore") .field("state", &SemState::load(&self.state, Relaxed)) .field("head", &self.head.with(|ptr| ptr)) .field("rx_lock", &self.rx_lock.load(Relaxed)) .field("stub", &self.stub) .finish() } } unsafe impl Send for Semaphore {} unsafe impl Sync for Semaphore {} // ===== impl Permit ===== impl Permit { /// Create a new `Permit`. /// /// The permit begins in the "unacquired" state. /// /// # Examples /// /// ``` /// use tokio_sync::semaphore::Permit; /// /// let permit = Permit::new(); /// assert!(!permit.is_acquired()); /// ``` pub fn new() -> Permit { Permit { waiter: None, state: PermitState::Idle, } } /// Returns true if the permit has been acquired pub fn is_acquired(&self) -> bool { self.state == PermitState::Acquired } /// Try to acquire the permit. If no permits are available, the current task /// is notified once a new permit becomes available. pub fn poll_acquire(&mut self, semaphore: &Semaphore) -> Poll<(), AcquireError> { use futures::Async::*; match self.state { PermitState::Idle => {} PermitState::Waiting => { let waiter = self.waiter.as_ref().unwrap(); if waiter.acquire()? { self.state = PermitState::Acquired; return Ok(Ready(())); } else { return Ok(NotReady); } } PermitState::Acquired => { return Ok(Ready(())); } } match semaphore.poll_permit(Some(self))? { Ready(v) => { self.state = PermitState::Acquired; Ok(Ready(v)) } NotReady => { self.state = PermitState::Waiting; Ok(NotReady) } } } /// Try to acquire the permit. pub fn try_acquire(&mut self, semaphore: &Semaphore) -> Result<(), TryAcquireError> { use futures::Async::*; match self.state { PermitState::Idle => {} PermitState::Waiting => { let waiter = self.waiter.as_ref().unwrap(); if waiter.acquire2().map_err(to_try_acquire)? { self.state = PermitState::Acquired; return Ok(()); } else { return Err(TryAcquireError::no_permits()); } } PermitState::Acquired => { return Ok(()); } } match semaphore.poll_permit(None).map_err(to_try_acquire)? { Ready(()) => { self.state = PermitState::Acquired; Ok(()) } NotReady => Err(TryAcquireError::no_permits()), } } /// Release a permit back to the semaphore pub fn release(&mut self, semaphore: &Semaphore) { if self.forget2() { semaphore.add_permits(1); } } /// Forget the permit **without** releasing it back to the semaphore. /// /// After calling `forget`, `poll_acquire` is able to acquire new permit /// from the sempahore. /// /// Repeatedly calling `forget` without associated calls to `add_permit` /// will result in the semaphore losing all permits. pub fn forget(&mut self) { self.forget2(); } /// Returns `true` if the permit was acquired fn forget2(&mut self) -> bool { match self.state { PermitState::Idle => false, PermitState::Waiting => { let ret = self.waiter.as_ref().unwrap().cancel_interest(); self.state = PermitState::Idle; ret } PermitState::Acquired => { self.state = PermitState::Idle; true } } } } // ===== impl AcquireError ==== impl AcquireError { fn closed() -> AcquireError { AcquireError(()) } } fn to_try_acquire(_: AcquireError) -> TryAcquireError { TryAcquireError::closed() } impl fmt::Display for AcquireError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for AcquireError { fn description(&self) -> &str { "semaphore closed" } } // ===== impl TryAcquireError ===== impl TryAcquireError { fn closed() -> TryAcquireError { TryAcquireError { kind: ErrorKind::Closed, } } fn no_permits() -> TryAcquireError { TryAcquireError { kind: ErrorKind::NoPermits, } } /// Returns true if the error was caused by a closed semaphore. pub fn is_closed(&self) -> bool { match self.kind { ErrorKind::Closed => true, _ => false, } } /// Returns true if the error was caused by calling `try_acquire` on a /// semaphore with no available permits. pub fn is_no_permits(&self) -> bool { match self.kind { ErrorKind::NoPermits => true, _ => false, } } } impl fmt::Display for TryAcquireError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for TryAcquireError { fn description(&self) -> &str { match self.kind { ErrorKind::Closed => "semaphore closed", ErrorKind::NoPermits => "no permits available", } } } // ===== impl WaiterNode ===== impl WaiterNode { fn new() -> WaiterNode { WaiterNode { state: AtomicUsize::new(NodeState::new().to_usize()), task: AtomicTask::new(), next: AtomicPtr::new(ptr::null_mut()), } } fn acquire(&self) -> Result { if self.acquire2()? { return Ok(true); } self.task.register(); self.acquire2() } fn acquire2(&self) -> Result { use self::NodeState::*; match Idle.compare_exchange(&self.state, Assigned, AcqRel, Acquire) { Ok(_) => Ok(true), Err(Closed) => Err(AcquireError::closed()), Err(_) => Ok(false), } } fn register(&self) { self.task.register() } /// Returns `true` if the permit has been acquired fn cancel_interest(&self) -> bool { use self::NodeState::*; match Queued.compare_exchange(&self.state, QueuedWaiting, AcqRel, Acquire) { // Successfully removed interest from the queued node. The permit // has not been assigned to the node. Ok(_) => false, // The semaphore has been closed, there is no further action to // take. Err(Closed) => false, // The permit has been assigned. It must be acquired in order to // be released back to the semaphore. Err(Assigned) => { match self.acquire2() { Ok(true) => true, // Not a reachable state Ok(false) => panic!(), // The semaphore has been closed, no further action to take. Err(_) => false, } } Err(state) => panic!("unexpected state = {:?}", state), } } /// Transition the state to `QueuedWaiting`. /// /// This step can only happen from `Queued` or from `Idle`. /// /// Returns `true` if transitioning into a queued state. fn to_queued_waiting(&self) -> bool { use self::NodeState::*; let mut curr = NodeState::load(&self.state, Acquire); loop { debug_assert!(curr == Idle || curr == Queued, "actual = {:?}", curr); let next = QueuedWaiting; match next.compare_exchange(&self.state, curr, AcqRel, Acquire) { Ok(_) => { if curr.is_queued() { return false; } else { // Transitioned to queued, reset next pointer self.next.store(ptr::null_mut(), Relaxed); return true; } } Err(actual) => { curr = actual; } } } } /// Notify the waiter /// /// Returns `true` if the waiter accepts the notification fn notify(&self, closed: bool) -> bool { use self::NodeState::*; // Assume QueuedWaiting state let mut curr = QueuedWaiting; loop { let next = match curr { Queued => Idle, QueuedWaiting => { if closed { Closed } else { Assigned } } actual => panic!("actual = {:?}", actual), }; match next.compare_exchange(&self.state, curr, AcqRel, Acquire) { Ok(_) => match curr { QueuedWaiting => { debug!(" + notify -- task notified"); self.task.notify(); return true; } other => { debug!(" + notify -- not notified; state = {:?}", other); return false; } }, Err(actual) => curr = actual, } } } fn revert_to_idle(&self) { use self::NodeState::Idle; // There are no other handles to the node NodeState::store(&self.state, Idle, Relaxed); } fn into_non_null(arc: Arc) -> NonNull { let ptr = Arc::into_raw(arc); unsafe { NonNull::new_unchecked(ptr as *mut _) } } } // ===== impl State ===== /// Flag differentiating between available permits and waiter pointers. /// /// If we assume pointers are properly aligned, then the least significant bit /// will always be zero. So, we use that bit to track if the value represents a /// number. const NUM_FLAG: usize = 0b01; const CLOSED_FLAG: usize = 0b10; const MAX_PERMITS: usize = usize::MAX >> NUM_SHIFT; /// When representing "numbers", the state has to be shifted this much (to get /// rid of the flag bit). const NUM_SHIFT: usize = 2; impl SemState { /// Returns a new default `State` value. fn new(permits: usize, stub: &WaiterNode) -> SemState { assert!(permits <= MAX_PERMITS); if permits > 0 { SemState((permits << NUM_SHIFT) | NUM_FLAG) } else { SemState(stub as *const _ as usize) } } /// Returns a `State` tracking `ptr` as the tail of the queue. fn new_ptr(tail: NonNull, closed: bool) -> SemState { let mut val = tail.as_ptr() as usize; if closed { val |= CLOSED_FLAG; } SemState(val) } /// Returns the amount of remaining capacity fn available_permits(&self) -> usize { if !self.has_available_permits() { return 0; } self.0 >> NUM_SHIFT } /// Returns true if the state has permits that can be claimed by a waiter. fn has_available_permits(&self) -> bool { self.0 & NUM_FLAG == NUM_FLAG } fn has_waiter(&self, stub: &WaiterNode) -> bool { !self.has_available_permits() && !self.is_stub(stub) } /// Try to acquire a permit /// /// # Return /// /// Returns `true` if the permit was acquired, `false` otherwise. If `false` /// is returned, it can be assumed that `State` represents the head pointer /// in the mpsc channel. fn acquire_permit(&mut self, stub: &WaiterNode) -> bool { if !self.has_available_permits() { return false; } debug_assert!(self.waiter().is_none()); self.0 -= 1 << NUM_SHIFT; if self.0 == NUM_FLAG { // Set the state to the stub pointer. self.0 = stub as *const _ as usize; } true } /// Release permits /// /// Returns `true` if the permits were accepted. fn release_permits(&mut self, permits: usize, stub: &WaiterNode) { debug_assert!(permits > 0); if self.is_stub(stub) { self.0 = (permits << NUM_SHIFT) | NUM_FLAG | (self.0 & CLOSED_FLAG); return; } debug_assert!(self.has_available_permits()); self.0 += permits << NUM_SHIFT; } fn is_waiter(&self) -> bool { self.0 & NUM_FLAG == 0 } /// Returns the waiter, if one is set. fn waiter(&self) -> Option> { if self.is_waiter() { let waiter = NonNull::new(self.as_ptr()).expect("null pointer stored"); Some(waiter) } else { None } } /// Assumes `self` represents a pointer fn as_ptr(&self) -> *mut WaiterNode { (self.0 & !CLOSED_FLAG) as *mut WaiterNode } /// Set to a pointer to a waiter. /// /// This can only be done from the full state. fn set_waiter(&mut self, waiter: NonNull) { let waiter = waiter.as_ptr() as usize; debug_assert!(waiter & NUM_FLAG == 0); debug_assert!(!self.is_closed()); self.0 = waiter; } fn is_stub(&self, stub: &WaiterNode) -> bool { self.as_ptr() as usize == stub as *const _ as usize } /// Load the state from an AtomicUsize. fn load(cell: &AtomicUsize, ordering: Ordering) -> SemState { let value = cell.load(ordering); debug!(" + SemState::load; value = {}", value); SemState(value) } /// Swap the values fn swap(&self, cell: &AtomicUsize, ordering: Ordering) -> SemState { let prev = SemState(cell.swap(self.to_usize(), ordering)); debug_assert_eq!(prev.is_closed(), self.is_closed()); prev } /// Compare and exchange the current value into the provided cell fn compare_exchange( &self, cell: &AtomicUsize, prev: SemState, success: Ordering, failure: Ordering, ) -> Result { debug_assert_eq!(prev.is_closed(), self.is_closed()); let res = cell.compare_exchange(prev.to_usize(), self.to_usize(), success, failure); debug!( " + SemState::compare_exchange; prev = {}; next = {}; result = {:?}", prev.to_usize(), self.to_usize(), res ); res.map(SemState).map_err(SemState) } fn fetch_set_closed(cell: &AtomicUsize, ordering: Ordering) -> SemState { let value = cell.fetch_or(CLOSED_FLAG, ordering); SemState(value) } fn is_closed(&self) -> bool { self.0 & CLOSED_FLAG == CLOSED_FLAG } /// Converts the state into a `usize` representation. fn to_usize(&self) -> usize { self.0 } } impl fmt::Debug for SemState { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { let mut fmt = fmt.debug_struct("SemState"); if self.is_waiter() { fmt.field("state", &""); } else { fmt.field("permits", &self.available_permits()); } fmt.finish() } } // ===== impl NodeState ===== impl NodeState { fn new() -> NodeState { NodeState::Idle } fn from_usize(value: usize) -> NodeState { use self::NodeState::*; match value { 0 => Idle, 1 => Queued, 2 => QueuedWaiting, 3 => Assigned, 4 => Closed, _ => panic!(), } } fn load(cell: &AtomicUsize, ordering: Ordering) -> NodeState { NodeState::from_usize(cell.load(ordering)) } /// Store a value fn store(cell: &AtomicUsize, value: NodeState, ordering: Ordering) { cell.store(value.to_usize(), ordering); } fn compare_exchange( &self, cell: &AtomicUsize, prev: NodeState, success: Ordering, failure: Ordering, ) -> Result { cell.compare_exchange(prev.to_usize(), self.to_usize(), success, failure) .map(NodeState::from_usize) .map_err(NodeState::from_usize) } /// Returns `true` if `self` represents a queued state. fn is_queued(&self) -> bool { use self::NodeState::*; match *self { Queued | QueuedWaiting => true, _ => false, } } fn to_usize(&self) -> usize { *self as usize } } tokio-sync-0.1.7/src/task/atomic_task.rs010064400007650000024000000323051354770560000164440ustar0000000000000000use loom::{ futures::task::{self, Task}, sync::atomic::AtomicUsize, sync::CausalCell, }; use std::fmt; use std::sync::atomic::Ordering::{AcqRel, Acquire, Release}; /// A synchronization primitive for task notification. /// /// `AtomicTask` will coordinate concurrent notifications with the consumer /// potentially "updating" the underlying task to notify. This is useful in /// scenarios where a computation completes in another thread and wants to /// notify the consumer, but the consumer is in the process of being migrated to /// a new logical task. /// /// Consumers should call `register` before checking the result of a computation /// and producers should call `notify` after producing the computation (this /// differs from the usual `thread::park` pattern). It is also permitted for /// `notify` to be called **before** `register`. This results in a no-op. /// /// A single `AtomicTask` may be reused for any number of calls to `register` or /// `notify`. /// /// `AtomicTask` does not provide any memory ordering guarantees, as such the /// user should use caution and use other synchronization primitives to guard /// the result of the underlying computation. pub struct AtomicTask { state: AtomicUsize, task: CausalCell>, } // `AtomicTask` is a multi-consumer, single-producer transfer cell. The cell // stores a `Task` value produced by calls to `register` and many threads can // race to take the task (to notify it) by calling `notify. // // If a new `Task` instance is produced by calling `register` before an existing // one is consumed, then the existing one is overwritten. // // While `AtomicTask` is single-producer, the implementation ensures memory // safety. In the event of concurrent calls to `register`, there will be a // single winner whose task will get stored in the cell. The losers will not // have their tasks notified. As such, callers should ensure to add // synchronization to calls to `register`. // // The implementation uses a single `AtomicUsize` value to coordinate access to // the `Task` cell. There are two bits that are operated on independently. These // are represented by `REGISTERING` and `NOTIFYING`. // // The `REGISTERING` bit is set when a producer enters the critical section. The // `NOTIFYING` bit is set when a consumer enters the critical section. Neither // bit being set is represented by `WAITING`. // // A thread obtains an exclusive lock on the task cell by transitioning the // state from `WAITING` to `REGISTERING` or `NOTIFYING`, depending on the // operation the thread wishes to perform. When this transition is made, it is // guaranteed that no other thread will access the task cell. // // # Registering // // On a call to `register`, an attempt to transition the state from WAITING to // REGISTERING is made. On success, the caller obtains a lock on the task cell. // // If the lock is obtained, then the thread sets the task cell to the task // provided as an argument. Then it attempts to transition the state back from // `REGISTERING` -> `WAITING`. // // If this transition is successful, then the registering process is complete // and the next call to `notify` will observe the task. // // If the transition fails, then there was a concurrent call to `notify` that // was unable to access the task cell (due to the registering thread holding the // lock). To handle this, the registering thread removes the task it just set // from the cell and calls `notify` on it. This call to notify represents the // attempt to notify by the other thread (that set the `NOTIFYING` bit). The // state is then transitioned from `REGISTERING | NOTIFYING` back to `WAITING`. // This transition must succeed because, at this point, the state cannot be // transitioned by another thread. // // # Notifying // // On a call to `notify`, an attempt to transition the state from `WAITING` to // `NOTIFYING` is made. On success, the caller obtains a lock on the task cell. // // If the lock is obtained, then the thread takes ownership of the current value // in teh task cell, and calls `notify` on it. The state is then transitioned // back to `WAITING`. This transition must succeed as, at this point, the state // cannot be transitioned by another thread. // // If the thread is unable to obtain the lock, the `NOTIFYING` bit is still. // This is because it has either been set by the current thread but the previous // value included the `REGISTERING` bit **or** a concurrent thread is in the // `NOTIFYING` critical section. Either way, no action must be taken. // // If the current thread is the only concurrent call to `notify` and another // thread is in the `register` critical section, when the other thread **exits** // the `register` critical section, it will observe the `NOTIFYING` bit and // handle the notify itself. // // If another thread is in the `notify` critical section, then it will handle // notifying the task. // // # A potential race (is safely handled). // // Imagine the following situation: // // * Thread A obtains the `notify` lock and notifies a task. // // * Before thread A releases the `notify` lock, the notified task is scheduled. // // * Thread B attempts to notify the task. In theory this should result in the // task being notified, but it cannot because thread A still holds the notify // lock. // // This case is handled by requiring users of `AtomicTask` to call `register` // **before** attempting to observe the application state change that resulted // in the task being notified. The notifiers also change the application state // before calling notify. // // Because of this, the task will do one of two things. // // 1) Observe the application state change that Thread B is notifying on. In // this case, it is OK for Thread B's notification to be lost. // // 2) Call register before attempting to observe the application state. Since // Thread A still holds the `notify` lock, the call to `register` will result // in the task notifying itself and get scheduled again. /// Idle state const WAITING: usize = 0; /// A new task value is being registered with the `AtomicTask` cell. const REGISTERING: usize = 0b01; /// The task currently registered with the `AtomicTask` cell is being notified. const NOTIFYING: usize = 0b10; impl AtomicTask { /// Create an `AtomicTask` initialized with the given `Task` pub fn new() -> AtomicTask { AtomicTask { state: AtomicUsize::new(WAITING), task: CausalCell::new(None), } } /// Registers the current task to be notified on calls to `notify`. /// /// This is the same as calling `register_task` with `task::current()`. pub fn register(&self) { self.do_register(CurrentTask); } /// Registers the provided task to be notified on calls to `notify`. /// /// The new task will take place of any previous tasks that were registered /// by previous calls to `register`. Any calls to `notify` that happen after /// a call to `register` (as defined by the memory ordering rules), will /// notify the `register` caller's task. /// /// It is safe to call `register` with multiple other threads concurrently /// calling `notify`. This will result in the `register` caller's current /// task being notified once. /// /// This function is safe to call concurrently, but this is generally a bad /// idea. Concurrent calls to `register` will attempt to register different /// tasks to be notified. One of the callers will win and have its task set, /// but there is no guarantee as to which caller will succeed. pub fn register_task(&self, task: Task) { self.do_register(ExactTask(task)); } fn do_register(&self, reg: R) where R: Register, { debug!(" + register_task"); match self.state.compare_and_swap(WAITING, REGISTERING, Acquire) { WAITING => { unsafe { // Locked acquired, update the waker cell self.task.with_mut(|t| reg.register(&mut *t)); // Release the lock. If the state transitioned to include // the `NOTIFYING` bit, this means that a notify has been // called concurrently, so we have to remove the task and // notify it.` // // Start by assuming that the state is `REGISTERING` as this // is what we jut set it to. let res = self .state .compare_exchange(REGISTERING, WAITING, AcqRel, Acquire); match res { Ok(_) => {} Err(actual) => { // This branch can only be reached if a // concurrent thread called `notify`. In this // case, `actual` **must** be `REGISTERING | // `NOTIFYING`. debug_assert_eq!(actual, REGISTERING | NOTIFYING); // Take the task to notify once the atomic operation has // completed. let notify = self.task.with_mut(|t| (*t).take()).unwrap(); // Just swap, because no one could change state // while state == `Registering | `Waking` self.state.swap(WAITING, AcqRel); // The atomic swap was complete, now // notify the task and return. notify.notify(); } } } } NOTIFYING => { // Currently in the process of notifying the task, i.e., // `notify` is currently being called on the old task handle. // So, we call notify on the new task handle reg.notify(); } state => { // In this case, a concurrent thread is holding the // "registering" lock. This probably indicates a bug in the // caller's code as racing to call `register` doesn't make much // sense. // // We just want to maintain memory safety. It is ok to drop the // call to `register`. debug_assert!(state == REGISTERING || state == REGISTERING | NOTIFYING); } } } /// Notifies the task that last called `register`. /// /// If `register` has not been called yet, then this does nothing. pub fn notify(&self) { debug!(" + notify"); if let Some(task) = self.take_task() { task.notify(); } } /// Attempts to take the `Task` value out of the `AtomicTask` with the /// intention that the caller will notify the task later. pub fn take_task(&self) -> Option { debug!(" + take_task"); // AcqRel ordering is used in order to acquire the value of the `task` // cell as well as to establish a `release` ordering with whatever // memory the `AtomicTask` is associated with. match self.state.fetch_or(NOTIFYING, AcqRel) { WAITING => { debug!(" + WAITING"); // The notifying lock has been acquired. let task = unsafe { self.task.with_mut(|t| (*t).take()) }; // Release the lock self.state.fetch_and(!NOTIFYING, Release); debug!(" + Done taking"); task } state => { debug!(" + state = {:?}", state); // There is a concurrent thread currently updating the // associated task. // // Nothing more to do as the `NOTIFYING` bit has been set. It // doesn't matter if there are concurrent registering threads or // not. // debug_assert!( state == REGISTERING || state == REGISTERING | NOTIFYING || state == NOTIFYING ); None } } } } impl Default for AtomicTask { fn default() -> Self { AtomicTask::new() } } impl fmt::Debug for AtomicTask { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { write!(fmt, "AtomicTask") } } unsafe impl Send for AtomicTask {} unsafe impl Sync for AtomicTask {} trait Register { fn register(self, slot: &mut Option); fn notify(self); } struct CurrentTask; impl Register for CurrentTask { fn register(self, slot: &mut Option) { let should_update = (&*slot) .as_ref() .map(|prev| !prev.will_notify_current()) .unwrap_or(true); if should_update { *slot = Some(task::current()); } } fn notify(self) { task::current().notify(); } } struct ExactTask(Task); impl Register for ExactTask { fn register(self, slot: &mut Option) { // When calling register_task with an exact task, it doesn't matter // if the previous task would have notified current. We *always* want // to save that exact task. *slot = Some(self.0); } fn notify(self) { self.0.notify(); } } tokio-sync-0.1.7/src/task/mod.rs010064400007650000024000000001501354770560000147160ustar0000000000000000//! Thread-safe task notification primitives. mod atomic_task; pub use self::atomic_task::AtomicTask; tokio-sync-0.1.7/src/watch.rs010064400007650000024000000262101354770560000143100ustar0000000000000000//! A single-producer, multi-consumer channel that only retains the *last* sent //! value. //! //! This channel is useful for watching for changes to a value from multiple //! points in the code base, for example, changes to configuration values. //! //! # Usage //! //! [`channel`] returns a [`Sender`] / [`Receiver`] pair. These are the producer //! and sender halves of the channel. The channel is created with an initial //! value. `Receiver::poll` will always be ready upon creation and will yield //! either this initial value or the latest value that has been sent by //! `Sender`. //! //! Calls to [`Receiver::poll`] and [`Receiver::poll_ref`] will always yield //! the latest value. //! //! # Examples //! //! ``` //! # extern crate futures; //! extern crate tokio; //! //! use tokio::prelude::*; //! use tokio::sync::watch; //! //! # tokio::run(futures::future::lazy(|| { //! let (mut tx, rx) = watch::channel("hello"); //! //! tokio::spawn(rx.for_each(|value| { //! println!("received = {:?}", value); //! Ok(()) //! }).map_err(|_| ())); //! //! tx.broadcast("world").unwrap(); //! # Ok(()) //! # })); //! ``` //! //! # Closing //! //! [`Sender::poll_close`] allows the producer to detect when all [`Sender`] //! handles have been dropped. This indicates that there is no further interest //! in the values being produced and work can be stopped. //! //! # Thread safety //! //! Both [`Sender`] and [`Receiver`] are thread safe. They can be moved to other //! threads and can be used in a concurrent environment. Clones of [`Receiver`] //! handles may be moved to separate threads and also used concurrently. //! //! [`Sender`]: struct.Sender.html //! [`Receiver`]: struct.Receiver.html //! [`channel`]: fn.channel.html //! [`Sender::poll_close`]: struct.Sender.html#method.poll_close //! [`Receiver::poll`]: struct.Receiver.html#method.poll //! [`Receiver::poll_ref`]: struct.Receiver.html#method.poll_ref use fnv::FnvHashMap; use futures::task::AtomicTask; use futures::{Async, AsyncSink, Poll, Sink, StartSend, Stream}; use std::ops; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; use std::sync::{Arc, Mutex, RwLock, RwLockReadGuard, Weak}; /// Receives values from the associated `Sender`. /// /// Instances are created by the [`channel`](fn.channel.html) function. #[derive(Debug)] pub struct Receiver { /// Pointer to the shared state shared: Arc>, /// Pointer to the watcher's internal state inner: Arc, /// Watcher ID. id: u64, /// Last observed version ver: usize, } /// Sends values to the associated `Receiver`. /// /// Instances are created by the [`channel`](fn.channel.html) function. #[derive(Debug)] pub struct Sender { shared: Weak>, } /// Returns a reference to the inner value /// /// Outstanding borrows hold a read lock on the inner value. This means that /// long lived borrows could cause the produce half to block. It is recommended /// to keep the borrow as short lived as possible. #[derive(Debug)] pub struct Ref<'a, T: 'a> { inner: RwLockReadGuard<'a, T>, } pub mod error { //! Watch error types use std::fmt; /// Error produced when receiving a value fails. #[derive(Debug)] pub struct RecvError { pub(crate) _p: (), } /// Error produced when sending a value fails. #[derive(Debug)] pub struct SendError { pub(crate) inner: T, } // ===== impl RecvError ===== impl fmt::Display for RecvError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for RecvError { fn description(&self) -> &str { "channel closed" } } // ===== impl SendError ===== impl fmt::Display for SendError { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { use std::error::Error; write!(fmt, "{}", self.description()) } } impl ::std::error::Error for SendError { fn description(&self) -> &str { "channel closed" } } } #[derive(Debug)] struct Shared { /// The most recent value value: RwLock, /// The current version /// /// The lowest bit represents a "closed" state. The rest of the bits /// represent the current version. version: AtomicUsize, /// All watchers watchers: Mutex, /// Task to notify when all watchers drop cancel: AtomicTask, } #[derive(Debug)] struct Watchers { next_id: u64, watchers: FnvHashMap>, } #[derive(Debug)] struct WatchInner { task: AtomicTask, } const CLOSED: usize = 1; /// Create a new watch channel, returning the "send" and "receive" handles. /// /// All values sent by `Sender` will become visible to the `Receiver` handles. /// Only the last value sent is made available to the `Receiver` half. All /// intermediate values are dropped. /// /// # Examples /// /// ``` /// # extern crate futures; /// extern crate tokio; /// /// use tokio::prelude::*; /// use tokio::sync::watch; /// /// # tokio::run(futures::future::lazy(|| { /// let (mut tx, rx) = watch::channel("hello"); /// /// tokio::spawn(rx.for_each(|value| { /// println!("received = {:?}", value); /// Ok(()) /// }).map_err(|_| ())); /// /// tx.broadcast("world").unwrap(); /// # Ok(()) /// # })); /// ``` pub fn channel(init: T) -> (Sender, Receiver) { const INIT_ID: u64 = 0; let inner = Arc::new(WatchInner::new()); // Insert the watcher let mut watchers = FnvHashMap::with_capacity_and_hasher(0, Default::default()); watchers.insert(INIT_ID, inner.clone()); let shared = Arc::new(Shared { value: RwLock::new(init), version: AtomicUsize::new(2), watchers: Mutex::new(Watchers { next_id: INIT_ID + 1, watchers, }), cancel: AtomicTask::new(), }); let tx = Sender { shared: Arc::downgrade(&shared), }; let rx = Receiver { shared, inner, id: INIT_ID, ver: 0, }; (tx, rx) } impl Receiver { /// Returns a reference to the most recently sent value /// /// Outstanding borrows hold a read lock. This means that long lived borrows /// could cause the send half to block. It is recommended to keep the borrow /// as short lived as possible. /// /// # Examples /// /// ``` /// # extern crate tokio; /// # use tokio::sync::watch; /// let (_, rx) = watch::channel("hello"); /// assert_eq!(*rx.get_ref(), "hello"); /// ``` pub fn get_ref(&self) -> Ref { let inner = self.shared.value.read().unwrap(); Ref { inner } } /// Attempts to receive the latest value sent via the channel. /// /// If a new, unobserved, value has been sent, a reference to it is /// returned. If no new value has been sent, then `NotReady` is returned and /// the current task is notified once a new value is sent. /// /// Only the **most recent** value is returned. If the receiver is falling /// behind the sender, intermediate values are dropped. pub fn poll_ref(&mut self) -> Poll>, error::RecvError> { // Make sure the task is up to date self.inner.task.register(); let state = self.shared.version.load(SeqCst); let version = state & !CLOSED; if version != self.ver { // Track the latest version self.ver = version; let inner = self.shared.value.read().unwrap(); return Ok(Some(Ref { inner }).into()); } if CLOSED == state & CLOSED { // The `Store` handle has been dropped. return Ok(None.into()); } Ok(Async::NotReady) } } impl Stream for Receiver { type Item = T; type Error = error::RecvError; fn poll(&mut self) -> Poll, error::RecvError> { let item = try_ready!(self.poll_ref()); Ok(Async::Ready(item.map(|v_ref| v_ref.clone()))) } } impl Clone for Receiver { fn clone(&self) -> Self { let inner = Arc::new(WatchInner::new()); let shared = self.shared.clone(); let id = { let mut watchers = shared.watchers.lock().unwrap(); let id = watchers.next_id; watchers.next_id += 1; watchers.watchers.insert(id, inner.clone()); id }; let ver = self.ver; Receiver { shared: shared, inner, id, ver, } } } impl Drop for Receiver { fn drop(&mut self) { let mut watchers = self.shared.watchers.lock().unwrap(); watchers.watchers.remove(&self.id); } } impl WatchInner { fn new() -> Self { WatchInner { task: AtomicTask::new(), } } } impl Sender { /// Broadcast a new value via the channel, notifying all receivers. pub fn broadcast(&mut self, value: T) -> Result<(), error::SendError> { let shared = match self.shared.upgrade() { Some(shared) => shared, // All `Watch` handles have been canceled None => return Err(error::SendError { inner: value }), }; // Replace the value { let mut lock = shared.value.write().unwrap(); *lock = value; } // Update the version. 2 is used so that the CLOSED bit is not set. shared.version.fetch_add(2, SeqCst); // Notify all watchers notify_all(&*shared); // Return the old value Ok(()) } /// Returns `Ready` when all receivers have dropped. /// /// This allows the producer to get notified when interest in the produced /// values is canceled and immediately stop doing work. pub fn poll_close(&mut self) -> Poll<(), ()> { match self.shared.upgrade() { Some(shared) => { shared.cancel.register(); Ok(Async::NotReady) } None => Ok(Async::Ready(())), } } } impl Sink for Sender { type SinkItem = T; type SinkError = error::SendError; fn start_send(&mut self, item: T) -> StartSend> { let _ = self.broadcast(item)?; Ok(AsyncSink::Ready) } fn poll_complete(&mut self) -> Poll<(), error::SendError> { Ok(().into()) } } /// Notify all watchers of a change fn notify_all(shared: &Shared) { let watchers = shared.watchers.lock().unwrap(); for watcher in watchers.watchers.values() { // Notify the task watcher.task.notify(); } } impl Drop for Sender { fn drop(&mut self) { if let Some(shared) = self.shared.upgrade() { shared.version.fetch_or(CLOSED, SeqCst); notify_all(&*shared); } } } // ===== impl Ref ===== impl<'a, T: 'a> ops::Deref for Ref<'a, T> { type Target = T; fn deref(&self) -> &T { self.inner.deref() } } // ===== impl Shared ===== impl Drop for Shared { fn drop(&mut self) { self.cancel.notify(); } } tokio-sync-0.1.7/tests/atomic_task.rs010064400007650000024000000027501354770560000160560ustar0000000000000000extern crate futures; extern crate tokio_mock_task; extern crate tokio_sync; use futures::task::{self, Task}; use tokio_mock_task::*; use tokio_sync::task::AtomicTask; trait AssertSend: Send {} trait AssertSync: Send {} impl AssertSend for AtomicTask {} impl AssertSync for AtomicTask {} impl AssertSend for Task {} impl AssertSync for Task {} #[test] fn register_task() { // AtomicTask::register_task should *always* register the // arbitrary task. let atomic = AtomicTask::new(); let mut mock1 = MockTask::new(); let mut mock2 = MockTask::new(); // Register once... mock1.enter(|| atomic.register()); // Grab the actual 2nd task from the mock... let task2 = mock2.enter(task::current); // Now register the 2nd task, even though in the context where // the first task would be considered 'current'... { // Need a block to grab a reference, so that we only move // task2 into the closure, not the AtomicTask... let atomic = &atomic; mock1.enter(move || { atomic.register_task(task2); }); } // Just proving that they haven't been notified yet... assert!(!mock1.is_notified(), "mock1 shouldn't be notified yet"); assert!(!mock2.is_notified(), "mock2 shouldn't be notified yet"); // Now trigger the notify, and ensure it was task2 atomic.notify(); assert!(!mock1.is_notified(), "mock1 shouldn't be notified"); assert!(mock2.is_notified(), "mock2 should be notified"); } tokio-sync-0.1.7/tests/errors.rs010064400007650000024000000013001354770560000150620ustar0000000000000000extern crate tokio_sync; fn is_error() {} #[test] fn mpsc_error_bound() { use tokio_sync::mpsc::error; is_error::(); is_error::(); is_error::>(); is_error::(); is_error::(); is_error::>(); } #[test] fn oneshot_error_bound() { use tokio_sync::oneshot::error; is_error::(); is_error::(); } #[test] fn watch_error_bound() { use tokio_sync::watch::error; is_error::(); is_error::>(); } tokio-sync-0.1.7/tests/fuzz_atomic_task.rs010064400007650000024000000021611354770560000171300ustar0000000000000000#![deny(warnings)] extern crate futures; #[macro_use] extern crate loom; #[allow(dead_code)] #[path = "../src/task/atomic_task.rs"] mod atomic_task; use atomic_task::AtomicTask; use loom::futures::block_on; use loom::sync::atomic::AtomicUsize; use loom::thread; use futures::future::poll_fn; use futures::Async; use std::sync::atomic::Ordering::Relaxed; use std::sync::Arc; struct Chan { num: AtomicUsize, task: AtomicTask, } #[test] fn basic_notification() { const NUM_NOTIFY: usize = 2; loom::fuzz(|| { let chan = Arc::new(Chan { num: AtomicUsize::new(0), task: AtomicTask::new(), }); for _ in 0..NUM_NOTIFY { let chan = chan.clone(); thread::spawn(move || { chan.num.fetch_add(1, Relaxed); chan.task.notify(); }); } block_on(poll_fn(move || { chan.task.register(); if NUM_NOTIFY == chan.num.load(Relaxed) { return Ok(Async::Ready(())); } Ok::<_, ()>(Async::NotReady) })) .unwrap(); }); } tokio-sync-0.1.7/tests/fuzz_list.rs010064400007650000024000000027001354770560000156040ustar0000000000000000extern crate futures; #[macro_use] extern crate loom; macro_rules! if_fuzz { ($($t:tt)*) => { $($t)* } } #[path = "../src/mpsc/list.rs"] #[allow(warnings)] mod list; #[path = "../src/mpsc/block.rs"] #[allow(warnings)] mod block; const BLOCK_CAP: usize = 2; use loom::thread; use std::sync::Arc; #[test] fn smoke() { use block::Read::*; const NUM_TX: usize = 2; const NUM_MSG: usize = 2; loom::fuzz(|| { let (tx, mut rx) = list::channel(); let tx = Arc::new(tx); for th in 0..NUM_TX { let tx = tx.clone(); thread::spawn(move || { for i in 0..NUM_MSG { tx.push((th, i)); } debug!(" + tx thread done"); }); } let mut next = vec![0; NUM_TX]; loop { debug!(" + rx.pop()"); match rx.pop(&tx) { Some(Value((th, v))) => { debug!(" + pop() -> Some(Value({}))", v); assert_eq!(v, next[th]); next[th] += 1; if next.iter().all(|&i| i == NUM_MSG) { break; } } Some(Closed) => { panic!(); } None => { debug!(" + pop() -> None"); loom::yield_now(); } } } }); } tokio-sync-0.1.7/tests/fuzz_mpsc.rs010064400007650000024000000013301354770560000155710ustar0000000000000000extern crate futures; #[macro_use] extern crate loom; macro_rules! if_fuzz { ($($t:tt)*) => { $($t)* } } #[path = "../src/mpsc/mod.rs"] #[allow(warnings)] mod mpsc; #[path = "../src/semaphore.rs"] #[allow(warnings)] mod semaphore; use futures::{future::poll_fn, Stream}; use loom::futures::block_on; use loom::thread; #[test] fn closing_tx() { loom::fuzz(|| { let (mut tx, mut rx) = mpsc::channel(16); thread::spawn(move || { tx.try_send(()).unwrap(); drop(tx); }); let v = block_on(poll_fn(|| rx.poll())).unwrap(); assert!(v.is_some()); let v = block_on(poll_fn(|| rx.poll())).unwrap(); assert!(v.is_none()); }); } tokio-sync-0.1.7/tests/fuzz_oneshot.rs010064400007650000024000000040011354770560000163040ustar0000000000000000extern crate futures; extern crate loom; #[path = "../src/oneshot.rs"] #[allow(warnings)] mod oneshot; use futures::{Async, Future}; use loom::futures::block_on; use loom::thread; #[test] fn smoke() { loom::fuzz(|| { let (tx, rx) = oneshot::channel(); thread::spawn(move || { tx.send(1).unwrap(); }); let value = block_on(rx).unwrap(); assert_eq!(1, value); }); } #[test] fn changing_rx_task() { loom::fuzz(|| { let (tx, mut rx) = oneshot::channel(); thread::spawn(move || { tx.send(1).unwrap(); }); let rx = thread::spawn(move || { let t1 = block_on(futures::future::poll_fn(|| Ok::<_, ()>(rx.poll().into()))).unwrap(); match t1 { Ok(Async::Ready(value)) => { // ok assert_eq!(1, value); None } Ok(Async::NotReady) => Some(rx), Err(_) => unreachable!(), } }) .join() .unwrap(); if let Some(rx) = rx { // Previous task parked, use a new task... let value = block_on(rx).unwrap(); assert_eq!(1, value); } }); } #[test] fn changing_tx_task() { loom::fuzz(|| { let (mut tx, rx) = oneshot::channel::(); thread::spawn(move || { drop(rx); }); let tx = thread::spawn(move || { let t1 = block_on(futures::future::poll_fn(|| { Ok::<_, ()>(tx.poll_close().into()) })) .unwrap(); match t1 { Ok(Async::Ready(())) => None, Ok(Async::NotReady) => Some(tx), Err(_) => unreachable!(), } }) .join() .unwrap(); if let Some(mut tx) = tx { // Previous task parked, use a new task... block_on(futures::future::poll_fn(move || tx.poll_close())).unwrap(); } }); } tokio-sync-0.1.7/tests/fuzz_semaphore.rs010064400007650000024000000070021354770560000166140ustar0000000000000000#[macro_use] extern crate futures; #[macro_use] extern crate loom; #[path = "../src/semaphore.rs"] #[allow(warnings)] mod semaphore; use semaphore::*; use futures::{future, Async, Future, Poll}; use loom::futures::block_on; use loom::thread; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; use std::sync::Arc; #[test] fn basic_usage() { const NUM: usize = 2; struct Actor { waiter: Permit, shared: Arc, } struct Shared { semaphore: Semaphore, active: AtomicUsize, } impl Future for Actor { type Item = (); type Error = (); fn poll(&mut self) -> Poll<(), ()> { try_ready!(self .waiter .poll_acquire(&self.shared.semaphore) .map_err(|_| ())); let actual = self.shared.active.fetch_add(1, SeqCst); assert!(actual <= NUM - 1); let actual = self.shared.active.fetch_sub(1, SeqCst); assert!(actual <= NUM); self.waiter.release(&self.shared.semaphore); Ok(Async::Ready(())) } } loom::fuzz(|| { let shared = Arc::new(Shared { semaphore: Semaphore::new(NUM), active: AtomicUsize::new(0), }); for _ in 0..NUM { let shared = shared.clone(); thread::spawn(move || { block_on(Actor { waiter: Permit::new(), shared, }) .unwrap(); }); } block_on(Actor { waiter: Permit::new(), shared, }) .unwrap(); }); } #[test] fn release() { loom::fuzz(|| { let semaphore = Arc::new(Semaphore::new(1)); { let semaphore = semaphore.clone(); thread::spawn(move || { let mut permit = Permit::new(); block_on(future::lazy(|| { permit.poll_acquire(&semaphore).unwrap(); Ok::<_, ()>(()) })) .unwrap(); permit.release(&semaphore); }); } let mut permit = Permit::new(); block_on(future::poll_fn(|| permit.poll_acquire(&semaphore))).unwrap(); permit.release(&semaphore); }); } #[test] fn basic_closing() { const NUM: usize = 2; loom::fuzz(|| { let semaphore = Arc::new(Semaphore::new(1)); for _ in 0..NUM { let semaphore = semaphore.clone(); thread::spawn(move || { let mut permit = Permit::new(); for _ in 0..2 { block_on(future::poll_fn(|| { permit.poll_acquire(&semaphore).map_err(|_| ()) }))?; permit.release(&semaphore); } Ok::<(), ()>(()) }); } semaphore.close(); }); } #[test] fn concurrent_close() { const NUM: usize = 3; loom::fuzz(|| { let semaphore = Arc::new(Semaphore::new(1)); for _ in 0..NUM { let semaphore = semaphore.clone(); thread::spawn(move || { let mut permit = Permit::new(); block_on(future::poll_fn(|| { permit.poll_acquire(&semaphore).map_err(|_| ()) }))?; permit.release(&semaphore); semaphore.close(); Ok::<(), ()>(()) }); } }); } tokio-sync-0.1.7/tests/lock.rs010064400007650000024000000026651354770560000145150ustar0000000000000000extern crate futures; extern crate tokio_mock_task; extern crate tokio_sync; use tokio_mock_task::*; use tokio_sync::lock::Lock; macro_rules! assert_ready { ($e:expr) => {{ match $e { futures::Async::Ready(v) => v, futures::Async::NotReady => panic!("not ready"), } }}; } macro_rules! assert_not_ready { ($e:expr) => {{ match $e { futures::Async::NotReady => {} futures::Async::Ready(v) => panic!("ready; value = {:?}", v), } }}; } #[test] fn straight_execution() { let mut l = Lock::new(100); // We can immediately acquire the lock and take the value let mut g = assert_ready!(l.poll_lock()); assert_eq!(&*g, &100); *g = 99; drop(g); let mut g = assert_ready!(l.poll_lock()); assert_eq!(&*g, &99); *g = 98; drop(g); let mut g = assert_ready!(l.poll_lock()); assert_eq!(&*g, &98); // We can continue to access the guard even if the lock is dropped drop(l); *g = 97; assert_eq!(&*g, &97); } #[test] fn readiness() { let mut task = MockTask::new(); let mut l = Lock::new(100); let g = assert_ready!(l.poll_lock()); // We can't now acquire the lease since it's already held in g task.enter(|| { assert_not_ready!(l.poll_lock()); }); // But once g unlocks, we can acquire it drop(g); assert!(task.is_notified()); assert_ready!(l.poll_lock()); } tokio-sync-0.1.7/tests/mpsc.rs010064400007650000024000000201171354770560000145170ustar0000000000000000extern crate futures; extern crate tokio_mock_task; extern crate tokio_sync; use tokio_mock_task::*; use tokio_sync::mpsc; use futures::prelude::*; use std::sync::Arc; use std::thread; trait AssertSend: Send {} impl AssertSend for mpsc::Sender {} impl AssertSend for mpsc::Receiver {} macro_rules! assert_ready { ($e:expr) => {{ match $e { Ok(futures::Async::Ready(v)) => v, Ok(_) => panic!("not ready"), Err(e) => panic!("error = {:?}", e), } }}; } macro_rules! assert_not_ready { ($e:expr) => {{ match $e { Ok(futures::Async::NotReady) => {} Ok(futures::Async::Ready(v)) => panic!("ready; value = {:?}", v), Err(e) => panic!("error = {:?}", e), } }}; } #[test] fn send_recv_with_buffer() { let (mut tx, mut rx) = mpsc::channel::(16); // Using poll_ready / try_send assert_ready!(tx.poll_ready()); tx.try_send(1).unwrap(); // Without poll_ready tx.try_send(2).unwrap(); // Sink API assert!(tx.start_send(3).unwrap().is_ready()); assert_ready!(tx.poll_complete()); assert_ready!(tx.close()); drop(tx); let val = assert_ready!(rx.poll()); assert_eq!(val, Some(1)); let val = assert_ready!(rx.poll()); assert_eq!(val, Some(2)); let val = assert_ready!(rx.poll()); assert_eq!(val, Some(3)); let val = assert_ready!(rx.poll()); assert!(val.is_none()); } #[test] fn start_send_past_cap() { let (mut tx1, mut rx) = mpsc::channel(1); let mut tx2 = tx1.clone(); let mut task1 = MockTask::new(); let mut task2 = MockTask::new(); let res = tx1.start_send(()).unwrap(); assert!(res.is_ready()); task1.enter(|| { let res = tx1.start_send(()).unwrap(); assert!(!res.is_ready()); }); task2.enter(|| { assert_not_ready!(tx2.poll_ready()); }); drop(tx1); let val = assert_ready!(rx.poll()); assert!(val.is_some()); assert!(task2.is_notified()); assert!(!task1.is_notified()); drop(tx2); let val = assert_ready!(rx.poll()); assert!(val.is_none()); } #[test] #[should_panic] fn buffer_gteq_one() { mpsc::channel::(0); } #[test] fn send_recv_unbounded() { let (mut tx, mut rx) = mpsc::unbounded_channel::(); // Using `try_send` tx.try_send(1).unwrap(); // Using `Sink` API assert!(tx.start_send(2).unwrap().is_ready()); assert_ready!(tx.poll_complete()); let val = assert_ready!(rx.poll()); assert_eq!(val, Some(1)); let val = assert_ready!(rx.poll()); assert_eq!(val, Some(2)); assert_ready!(tx.poll_complete()); assert_ready!(tx.close()); drop(tx); let val = assert_ready!(rx.poll()); assert!(val.is_none()); } #[test] fn no_t_bounds_buffer() { struct NoImpls; let (tx, mut rx) = mpsc::channel(100); // sender should be Debug even though T isn't Debug println!("{:?}", tx); // same with Receiver println!("{:?}", rx); // and sender should be Clone even though T isn't Clone assert!(tx.clone().try_send(NoImpls).is_ok()); assert!(assert_ready!(rx.poll()).is_some()); } #[test] fn no_t_bounds_unbounded() { struct NoImpls; let (tx, mut rx) = mpsc::unbounded_channel(); // sender should be Debug even though T isn't Debug println!("{:?}", tx); // same with Receiver println!("{:?}", rx); // and sender should be Clone even though T isn't Clone assert!(tx.clone().try_send(NoImpls).is_ok()); assert!(assert_ready!(rx.poll()).is_some()); } #[test] fn send_recv_buffer_limited() { let (mut tx, mut rx) = mpsc::channel::(1); let mut task = MockTask::new(); // Run on a task context task.enter(|| { assert!(tx.poll_complete().unwrap().is_ready()); assert!(tx.poll_ready().unwrap().is_ready()); // Send first message let res = tx.start_send(1).unwrap(); assert!(is_ready(&res)); assert!(tx.poll_ready().unwrap().is_not_ready()); // Send second message let res = tx.start_send(2).unwrap(); assert!(!is_ready(&res)); // Take the value assert_eq!(rx.poll().unwrap(), Async::Ready(Some(1))); assert!(tx.poll_ready().unwrap().is_ready()); let res = tx.start_send(2).unwrap(); assert!(is_ready(&res)); assert!(tx.poll_ready().unwrap().is_not_ready()); // Take the value assert_eq!(rx.poll().unwrap(), Async::Ready(Some(2))); assert!(tx.poll_ready().unwrap().is_ready()); }); } #[test] fn send_shared_recv() { let (tx1, rx) = mpsc::channel::(16); let tx2 = tx1.clone(); let mut rx = rx.wait(); tx1.send(1).wait().unwrap(); assert_eq!(rx.next().unwrap().unwrap(), 1); tx2.send(2).wait().unwrap(); assert_eq!(rx.next().unwrap().unwrap(), 2); } #[test] fn send_recv_threads() { let (tx, rx) = mpsc::channel::(16); let mut rx = rx.wait(); thread::spawn(move || { tx.send(1).wait().unwrap(); }); assert_eq!(rx.next().unwrap().unwrap(), 1); } #[test] fn recv_close_gets_none_idle() { let (mut tx, mut rx) = mpsc::channel::(10); let mut task = MockTask::new(); rx.close(); task.enter(|| { let val = assert_ready!(rx.poll()); assert!(val.is_none()); assert!(tx.poll_ready().is_err()); }); } #[test] fn recv_close_gets_none_reserved() { let (mut tx1, mut rx) = mpsc::channel::(1); let mut tx2 = tx1.clone(); assert_ready!(tx1.poll_ready()); let mut task = MockTask::new(); task.enter(|| { assert_not_ready!(tx2.poll_ready()); }); rx.close(); assert!(task.is_notified()); task.enter(|| { assert!(tx2.poll_ready().is_err()); assert_not_ready!(rx.poll()); }); assert!(!task.is_notified()); assert!(tx1.try_send(123).is_ok()); assert!(task.is_notified()); task.enter(|| { let v = assert_ready!(rx.poll()); assert_eq!(v, Some(123)); let v = assert_ready!(rx.poll()); assert!(v.is_none()); }); } #[test] fn tx_close_gets_none() { let (_, mut rx) = mpsc::channel::(10); let mut task = MockTask::new(); // Run on a task context task.enter(|| { let v = assert_ready!(rx.poll()); assert!(v.is_none()); }); } fn is_ready(res: &AsyncSink) -> bool { match *res { AsyncSink::Ready => true, _ => false, } } #[test] fn try_send_fail() { let (mut tx, rx) = mpsc::channel(1); let mut rx = rx.wait(); tx.try_send("hello").unwrap(); // This should fail assert!(tx.try_send("fail").unwrap_err().is_full()); assert_eq!(rx.next().unwrap().unwrap(), "hello"); tx.try_send("goodbye").unwrap(); drop(tx); assert_eq!(rx.next().unwrap().unwrap(), "goodbye"); assert!(rx.next().is_none()); } #[test] fn drop_tx_with_permit_releases_permit() { // poll_ready reserves capacity, ensure that the capacity is released if tx // is dropped w/o sending a value. let (mut tx1, _rx) = mpsc::channel::(1); let mut tx2 = tx1.clone(); let mut task = MockTask::new(); assert_ready!(tx1.poll_ready()); task.enter(|| { assert_not_ready!(tx2.poll_ready()); }); drop(tx1); assert!(task.is_notified()); assert_ready!(tx2.poll_ready()); } #[test] fn dropping_rx_closes_channel() { let (mut tx, rx) = mpsc::channel(100); let msg = Arc::new(()); tx.try_send(msg.clone()).unwrap(); drop(rx); assert!(tx.poll_ready().is_err()); assert_eq!(1, Arc::strong_count(&msg)); } #[test] fn dropping_rx_closes_channel_for_try() { let (mut tx, rx) = mpsc::channel(100); let msg = Arc::new(()); tx.try_send(msg.clone()).unwrap(); drop(rx); assert!(tx.try_send(msg.clone()).unwrap_err().is_closed()); assert_eq!(1, Arc::strong_count(&msg)); } #[test] fn unconsumed_messages_are_dropped() { let msg = Arc::new(()); let (mut tx, rx) = mpsc::channel(100); tx.try_send(msg.clone()).unwrap(); assert_eq!(2, Arc::strong_count(&msg)); drop((tx, rx)); assert_eq!(1, Arc::strong_count(&msg)); } tokio-sync-0.1.7/tests/oneshot.rs010064400007650000024000000123061354770560000152350ustar0000000000000000extern crate futures; extern crate tokio_mock_task; extern crate tokio_sync; use tokio_mock_task::*; use tokio_sync::oneshot; use futures::prelude::*; macro_rules! assert_ready { ($e:expr) => {{ match $e { Ok(futures::Async::Ready(v)) => v, Ok(_) => panic!("not ready"), Err(e) => panic!("error = {:?}", e), } }}; } macro_rules! assert_not_ready { ($e:expr) => {{ match $e { Ok(futures::Async::NotReady) => {} Ok(futures::Async::Ready(v)) => panic!("ready; value = {:?}", v), Err(e) => panic!("error = {:?}", e), } }}; } trait AssertSend: Send {} impl AssertSend for oneshot::Sender {} impl AssertSend for oneshot::Receiver {} #[test] fn send_recv() { let (tx, mut rx) = oneshot::channel(); let mut task = MockTask::new(); task.enter(|| { assert_not_ready!(rx.poll()); }); assert!(tx.send(1).is_ok()); assert!(task.is_notified()); let val = assert_ready!(rx.poll()); assert_eq!(val, 1); } #[test] fn close_tx() { let (tx, mut rx) = oneshot::channel::(); let mut task = MockTask::new(); task.enter(|| { assert_not_ready!(rx.poll()); }); drop(tx); assert!(task.is_notified()); assert!(rx.poll().is_err()); } #[test] fn close_rx() { // First, without checking poll_close() // let (tx, _) = oneshot::channel(); assert!(tx.send(1).is_err()); // Second, via poll_close(); let (mut tx, rx) = oneshot::channel(); let mut task = MockTask::new(); task.enter(|| assert_not_ready!(tx.poll_close())); drop(rx); assert!(task.is_notified()); assert!(tx.is_closed()); assert_ready!(tx.poll_close()); assert!(tx.send(1).is_err()); } #[test] fn explicit_close_poll() { // First, with message sent let (tx, mut rx) = oneshot::channel(); assert!(tx.send(1).is_ok()); rx.close(); let value = assert_ready!(rx.poll()); assert_eq!(value, 1); println!("~~~~~~~~~ TWO ~~~~~~~~~~"); // Second, without the message sent let (mut tx, mut rx) = oneshot::channel::(); let mut task = MockTask::new(); task.enter(|| assert_not_ready!(tx.poll_close())); rx.close(); assert!(task.is_notified()); assert!(tx.is_closed()); assert_ready!(tx.poll_close()); assert!(tx.send(1).is_err()); assert!(rx.poll().is_err()); // Again, but without sending the value this time let (mut tx, mut rx) = oneshot::channel::(); let mut task = MockTask::new(); task.enter(|| assert_not_ready!(tx.poll_close())); rx.close(); assert!(task.is_notified()); assert!(tx.is_closed()); assert_ready!(tx.poll_close()); assert!(rx.poll().is_err()); } #[test] fn explicit_close_try_recv() { // First, with message sent let (tx, mut rx) = oneshot::channel(); assert!(tx.send(1).is_ok()); rx.close(); assert_eq!(rx.try_recv().unwrap(), 1); println!("~~~~~~~~~ TWO ~~~~~~~~~~"); // Second, without the message sent let (mut tx, mut rx) = oneshot::channel::(); let mut task = MockTask::new(); task.enter(|| assert_not_ready!(tx.poll_close())); rx.close(); assert!(task.is_notified()); assert!(tx.is_closed()); assert_ready!(tx.poll_close()); assert!(rx.try_recv().is_err()); } #[test] #[should_panic] fn close_try_recv_poll() { let (_tx, mut rx) = oneshot::channel::(); let mut task = MockTask::new(); rx.close(); assert!(rx.try_recv().is_err()); task.enter(|| { let _ = rx.poll(); }); } #[test] fn drops_tasks() { let (mut tx, mut rx) = oneshot::channel::(); let mut tx_task = MockTask::new(); let mut rx_task = MockTask::new(); tx_task.enter(|| { assert_not_ready!(tx.poll_close()); }); rx_task.enter(|| { assert_not_ready!(rx.poll()); }); drop(tx); drop(rx); assert_eq!(1, tx_task.notifier_ref_count()); assert_eq!(1, rx_task.notifier_ref_count()); } #[test] fn receiver_changes_task() { let (tx, mut rx) = oneshot::channel(); let mut task1 = MockTask::new(); let mut task2 = MockTask::new(); task1.enter(|| { assert_not_ready!(rx.poll()); }); assert_eq!(2, task1.notifier_ref_count()); assert_eq!(1, task2.notifier_ref_count()); task2.enter(|| { assert_not_ready!(rx.poll()); }); assert_eq!(1, task1.notifier_ref_count()); assert_eq!(2, task2.notifier_ref_count()); tx.send(1).unwrap(); assert!(!task1.is_notified()); assert!(task2.is_notified()); assert_ready!(rx.poll()); } #[test] fn sender_changes_task() { let (mut tx, rx) = oneshot::channel::(); let mut task1 = MockTask::new(); let mut task2 = MockTask::new(); task1.enter(|| { assert_not_ready!(tx.poll_close()); }); assert_eq!(2, task1.notifier_ref_count()); assert_eq!(1, task2.notifier_ref_count()); task2.enter(|| { assert_not_ready!(tx.poll_close()); }); assert_eq!(1, task1.notifier_ref_count()); assert_eq!(2, task2.notifier_ref_count()); drop(rx); assert!(!task1.is_notified()); assert!(task2.is_notified()); assert_ready!(tx.poll_close()); } tokio-sync-0.1.7/tests/semaphore.rs010064400007650000024000000075711354770560000155510ustar0000000000000000extern crate futures; extern crate tokio_mock_task; extern crate tokio_sync; use tokio_mock_task::*; use tokio_sync::semaphore::{Permit, Semaphore}; macro_rules! assert_ready { ($e:expr) => {{ match $e { Ok(futures::Async::Ready(v)) => v, Ok(_) => panic!("not ready"), Err(e) => panic!("error = {:?}", e), } }}; } macro_rules! assert_not_ready { ($e:expr) => {{ match $e { Ok(futures::Async::NotReady) => {} Ok(futures::Async::Ready(v)) => panic!("ready; value = {:?}", v), Err(e) => panic!("error = {:?}", e), } }}; } #[test] fn available_permits() { let s = Semaphore::new(100); assert_eq!(s.available_permits(), 100); // Polling for a permit succeeds immediately let mut permit = Permit::new(); assert!(!permit.is_acquired()); assert_ready!(permit.poll_acquire(&s)); assert_eq!(s.available_permits(), 99); assert!(permit.is_acquired()); // Polling again on the same waiter does not claim a new permit assert_ready!(permit.poll_acquire(&s)); assert_eq!(s.available_permits(), 99); assert!(permit.is_acquired()); } #[test] fn unavailable_permits() { let s = Semaphore::new(1); let mut permit_1 = Permit::new(); let mut permit_2 = Permit::new(); // Acquire the first permit assert_ready!(permit_1.poll_acquire(&s)); assert_eq!(s.available_permits(), 0); let mut task = MockTask::new(); task.enter(|| { // Try to acquire the second permit assert_not_ready!(permit_2.poll_acquire(&s)); }); permit_1.release(&s); assert_eq!(s.available_permits(), 0); assert!(task.is_notified()); assert_ready!(permit_2.poll_acquire(&s)); permit_2.release(&s); assert_eq!(s.available_permits(), 1); } #[test] fn zero_permits() { let s = Semaphore::new(0); assert_eq!(s.available_permits(), 0); let mut permit = Permit::new(); let mut task = MockTask::new(); // Try to acquire the permit task.enter(|| { assert_not_ready!(permit.poll_acquire(&s)); }); s.add_permits(1); assert!(task.is_notified()); assert_ready!(permit.poll_acquire(&s)); } #[test] #[should_panic] fn validates_max_permits() { use std::usize; Semaphore::new((usize::MAX >> 2) + 1); } #[test] fn close_semaphore_prevents_acquire() { let s = Semaphore::new(1); s.close(); assert_eq!(1, s.available_permits()); let mut permit = Permit::new(); assert!(permit.poll_acquire(&s).is_err()); assert_eq!(1, s.available_permits()); } #[test] fn close_semaphore_notifies_permit1() { let s = Semaphore::new(0); let mut permit = Permit::new(); let mut task = MockTask::new(); task.enter(|| { assert_not_ready!(permit.poll_acquire(&s)); }); s.close(); assert!(task.is_notified()); assert!(permit.poll_acquire(&s).is_err()); } #[test] fn close_semaphore_notifies_permit2() { let s = Semaphore::new(2); let mut permit1 = Permit::new(); let mut permit2 = Permit::new(); let mut permit3 = Permit::new(); let mut permit4 = Permit::new(); // Acquire a couple of permits assert_ready!(permit1.poll_acquire(&s)); assert_ready!(permit2.poll_acquire(&s)); let mut task1 = MockTask::new(); let mut task2 = MockTask::new(); task1.enter(|| { assert_not_ready!(permit3.poll_acquire(&s)); }); task2.enter(|| { assert_not_ready!(permit4.poll_acquire(&s)); }); s.close(); assert!(task1.is_notified()); assert!(task2.is_notified()); assert!(permit3.poll_acquire(&s).is_err()); assert!(permit4.poll_acquire(&s).is_err()); assert_eq!(0, s.available_permits()); permit1.release(&s); assert_eq!(1, s.available_permits()); assert!(permit1.poll_acquire(&s).is_err()); permit2.release(&s); assert_eq!(2, s.available_permits()); } tokio-sync-0.1.7/tests/watch.rs010064400007650000024000000116121354770560000146630ustar0000000000000000extern crate futures; extern crate tokio_mock_task; extern crate tokio_sync; use tokio_mock_task::*; use tokio_sync::watch; macro_rules! assert_ready { ($e:expr) => {{ match $e { Ok(futures::Async::Ready(v)) => v, Ok(_) => panic!("not ready"), Err(e) => panic!("error = {:?}", e), } }}; } macro_rules! assert_not_ready { ($e:expr) => {{ match $e { Ok(futures::Async::NotReady) => {} Ok(futures::Async::Ready(v)) => panic!("ready; value = {:?}", v), Err(e) => panic!("error = {:?}", e), } }}; } #[test] fn single_rx() { let (mut tx, mut rx) = watch::channel("one"); let mut task = MockTask::new(); task.enter(|| { let v = assert_ready!(rx.poll_ref()).unwrap(); assert_eq!(*v, "one"); }); task.enter(|| assert_not_ready!(rx.poll_ref())); assert!(!task.is_notified()); tx.broadcast("two").unwrap(); assert!(task.is_notified()); task.enter(|| { let v = assert_ready!(rx.poll_ref()).unwrap(); assert_eq!(*v, "two"); }); task.enter(|| assert_not_ready!(rx.poll_ref())); drop(tx); assert!(task.is_notified()); task.enter(|| { let res = assert_ready!(rx.poll_ref()); assert!(res.is_none()); }); } #[test] fn stream_impl() { use futures::Stream; let (mut tx, mut rx) = watch::channel("one"); let mut task = MockTask::new(); task.enter(|| { let v = assert_ready!(rx.poll()).unwrap(); assert_eq!(v, "one"); }); task.enter(|| assert_not_ready!(rx.poll())); assert!(!task.is_notified()); tx.broadcast("two").unwrap(); assert!(task.is_notified()); task.enter(|| { let v = assert_ready!(rx.poll()).unwrap(); assert_eq!(v, "two"); }); task.enter(|| assert_not_ready!(rx.poll())); drop(tx); assert!(task.is_notified()); task.enter(|| { let res = assert_ready!(rx.poll()); assert!(res.is_none()); }); } #[test] fn multi_rx() { let (mut tx, mut rx1) = watch::channel("one"); let mut rx2 = rx1.clone(); let mut task1 = MockTask::new(); let mut task2 = MockTask::new(); task1.enter(|| { let res = assert_ready!(rx1.poll_ref()); assert_eq!(*res.unwrap(), "one"); }); task2.enter(|| { let res = assert_ready!(rx2.poll_ref()); assert_eq!(*res.unwrap(), "one"); }); tx.broadcast("two").unwrap(); assert!(task1.is_notified()); assert!(task2.is_notified()); task1.enter(|| { let res = assert_ready!(rx1.poll_ref()); assert_eq!(*res.unwrap(), "two"); }); tx.broadcast("three").unwrap(); assert!(task1.is_notified()); assert!(task2.is_notified()); task1.enter(|| { let res = assert_ready!(rx1.poll_ref()); assert_eq!(*res.unwrap(), "three"); }); task2.enter(|| { let res = assert_ready!(rx2.poll_ref()); assert_eq!(*res.unwrap(), "three"); }); tx.broadcast("four").unwrap(); task1.enter(|| { let res = assert_ready!(rx1.poll_ref()); assert_eq!(*res.unwrap(), "four"); }); drop(tx); task1.enter(|| { let res = assert_ready!(rx1.poll_ref()); assert!(res.is_none()); }); task2.enter(|| { let res = assert_ready!(rx2.poll_ref()); assert_eq!(*res.unwrap(), "four"); }); task2.enter(|| { let res = assert_ready!(rx2.poll_ref()); assert!(res.is_none()); }); } #[test] fn rx_observes_final_value() { // Initial value let (tx, mut rx) = watch::channel("one"); let mut task = MockTask::new(); drop(tx); task.enter(|| { let res = assert_ready!(rx.poll_ref()); assert!(res.is_some()); assert_eq!(*res.unwrap(), "one"); }); task.enter(|| { let res = assert_ready!(rx.poll_ref()); assert!(res.is_none()); }); // Sending a value let (mut tx, mut rx) = watch::channel("one"); let mut task = MockTask::new(); tx.broadcast("two").unwrap(); task.enter(|| { let res = assert_ready!(rx.poll_ref()); assert!(res.is_some()); assert_eq!(*res.unwrap(), "two"); }); task.enter(|| assert_not_ready!(rx.poll_ref())); tx.broadcast("three").unwrap(); drop(tx); assert!(task.is_notified()); task.enter(|| { let res = assert_ready!(rx.poll_ref()); assert!(res.is_some()); assert_eq!(*res.unwrap(), "three"); }); task.enter(|| { let res = assert_ready!(rx.poll_ref()); assert!(res.is_none()); }); } #[test] fn poll_close() { let (mut tx, rx) = watch::channel("one"); let mut task = MockTask::new(); task.enter(|| assert_not_ready!(tx.poll_close())); drop(rx); assert!(task.is_notified()); task.enter(|| assert_ready!(tx.poll_close())); assert!(tx.broadcast("two").is_err()); } tokio-sync-0.1.7/.cargo_vcs_info.json0000644000000001120000000000000131710ustar00{ "git": { "sha1": "da186a785999eb5234ac0aad0b987bb128d52f69" } }