mirror of
https://github.com/tokio-rs/tokio.git
synced 2025-09-28 12:10:37 +00:00
Introduce tokio-sync crate containing synchronization primitives. (#839)
Introduce a tokio-sync crate containing useful synchronization primitives for programs written using Tokio. The initial release contains: * An mpsc channel * A oneshot channel * A semaphore implementation * An `AtomicTask` primitive. The `oneshot` and `mpsc` channels are new implementations providing improved performance characteristics. In some benchmarks, the new mpsc channel shows up to 7x improvement over the version provided by the `futures` crate. Unfortunately, the `oneshot` implementation only provides a slight performance improvement as it is mostly limited by the `futures` 0.1 task system. Once updated to the `std` version of `Future` (currently nightly only), much greater performance improvements should be achievable by `oneshot`. Additionally, he implementations provided here are checked using [Loom](http://github.com/carllerche/loom/), which provides greater confidence of correctness.
This commit is contained in:
parent
91f20e33a4
commit
13083153aa
@ -11,6 +11,7 @@ install:
|
||||
- rustup-init.exe -y --default-host %TARGET%
|
||||
- set PATH=%PATH%;C:\Users\appveyor\.cargo\bin
|
||||
- set RUST_BACKTRACE=1
|
||||
- set LOOM_MAX_DURATION=10
|
||||
|
||||
- rustc -V
|
||||
- cargo -V
|
||||
|
@ -101,6 +101,8 @@ script: |
|
||||
cargo check --all --exclude tokio-tls --target $TARGET
|
||||
cargo check --tests --all --exclude tokio-tls --target $TARGET
|
||||
else
|
||||
# Limit the execution time of loom tests.
|
||||
export LOOM_MAX_DURATION=10
|
||||
cargo test --all --no-fail-fast
|
||||
cargo doc --all
|
||||
fi
|
||||
|
@ -26,7 +26,6 @@ members = [
|
||||
"./",
|
||||
"tokio-async-await",
|
||||
"tokio-buf",
|
||||
"tokio-channel",
|
||||
"tokio-codec",
|
||||
"tokio-current-thread",
|
||||
"tokio-executor",
|
||||
@ -34,6 +33,7 @@ members = [
|
||||
"tokio-io",
|
||||
"tokio-reactor",
|
||||
"tokio-signal",
|
||||
"tokio-sync",
|
||||
"tokio-threadpool",
|
||||
"tokio-timer",
|
||||
"tokio-tcp",
|
||||
@ -49,6 +49,7 @@ default = [
|
||||
"io",
|
||||
"reactor",
|
||||
"rt-full",
|
||||
"sync",
|
||||
"tcp",
|
||||
"timer",
|
||||
"udp",
|
||||
@ -67,6 +68,7 @@ rt-full = [
|
||||
"tokio-executor",
|
||||
"tokio-threadpool",
|
||||
]
|
||||
sync = ["tokio-sync"]
|
||||
tcp = ["tokio-tcp"]
|
||||
timer = ["tokio-timer"]
|
||||
udp = ["tokio-udp"]
|
||||
@ -95,6 +97,7 @@ tokio-fs = { version = "0.1.3", path = "tokio-fs", optional = true }
|
||||
tokio-io = { version = "0.1.6", path = "tokio-io", optional = true }
|
||||
tokio-executor = { version = "0.1.5", path = "tokio-executor", optional = true }
|
||||
tokio-reactor = { version = "0.1.1", path = "tokio-reactor", optional = true }
|
||||
tokio-sync = { version = "0.1.0", path = "tokio-sync", optional = true }
|
||||
tokio-threadpool = { version = "0.1.8", path = "tokio-threadpool", optional = true }
|
||||
tokio-tcp = { version = "0.1.0", path = "tokio-tcp", optional = true }
|
||||
tokio-udp = { version = "0.1.0", path = "tokio-udp", optional = true }
|
||||
|
@ -100,6 +100,8 @@ extern crate tokio_fs;
|
||||
extern crate tokio_reactor;
|
||||
#[cfg(feature = "rt-full")]
|
||||
extern crate tokio_threadpool;
|
||||
#[cfg(feature = "sync")]
|
||||
extern crate tokio_sync;
|
||||
#[cfg(feature = "timer")]
|
||||
extern crate tokio_timer;
|
||||
#[cfg(feature = "tcp")]
|
||||
@ -126,6 +128,8 @@ pub mod net;
|
||||
pub mod prelude;
|
||||
#[cfg(feature = "reactor")]
|
||||
pub mod reactor;
|
||||
#[cfg(feature = "sync")]
|
||||
pub mod sync;
|
||||
#[cfg(feature = "timer")]
|
||||
pub mod timer;
|
||||
pub mod util;
|
||||
|
16
src/sync.rs
Normal file
16
src/sync.rs
Normal file
@ -0,0 +1,16 @@
|
||||
//! Future-aware synchronization
|
||||
//!
|
||||
//! This module is enabled with the **`sync`** feature flag.
|
||||
//!
|
||||
//! Tasks sometimes need to communicate with each other. This module contains
|
||||
//! two basic abstractions for doing so:
|
||||
//!
|
||||
//! - [oneshot](oneshot/index.html), a way of sending a single value
|
||||
//! from one task to another.
|
||||
//! - [mpsc](mpsc/index.html), a multi-producer, single-consumer channel for
|
||||
//! sending values between tasks.
|
||||
|
||||
pub use tokio_sync::{
|
||||
mpsc,
|
||||
oneshot,
|
||||
};
|
@ -1,51 +0,0 @@
|
||||
Copyright (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.
|
||||
|
||||
Copyright (c) 2016 futures-rs 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.
|
@ -1,14 +0,0 @@
|
||||
#![doc(html_root_url = "https://docs.rs/tokio-channel/0.1.0")]
|
||||
#![deny(missing_docs, warnings, missing_debug_implementations)]
|
||||
|
||||
//! Asynchronous channels.
|
||||
//!
|
||||
//! This crate provides channels that can be used to communicate between
|
||||
//! asynchronous tasks.
|
||||
|
||||
extern crate futures;
|
||||
|
||||
pub mod mpsc;
|
||||
pub mod oneshot;
|
||||
|
||||
mod lock;
|
@ -1,105 +0,0 @@
|
||||
//! A "mutex" which only supports `try_lock`
|
||||
//!
|
||||
//! As a futures library the eventual call to an event loop should be the only
|
||||
//! thing that ever blocks, so this is assisted with a fast user-space
|
||||
//! implementation of a lock that can only have a `try_lock` operation.
|
||||
|
||||
use std::cell::UnsafeCell;
|
||||
use std::ops::{Deref, DerefMut};
|
||||
use std::sync::atomic::Ordering::SeqCst;
|
||||
use std::sync::atomic::AtomicBool;
|
||||
|
||||
/// A "mutex" around a value, similar to `std::sync::Mutex<T>`.
|
||||
///
|
||||
/// This lock only supports the `try_lock` operation, however, and does not
|
||||
/// implement poisoning.
|
||||
#[derive(Debug)]
|
||||
pub struct Lock<T> {
|
||||
locked: AtomicBool,
|
||||
data: UnsafeCell<T>,
|
||||
}
|
||||
|
||||
/// Sentinel representing an acquired lock through which the data can be
|
||||
/// accessed.
|
||||
pub struct TryLock<'a, T: 'a> {
|
||||
__ptr: &'a Lock<T>,
|
||||
}
|
||||
|
||||
// The `Lock` structure is basically just a `Mutex<T>`, and these two impls are
|
||||
// intended to mirror the standard library's corresponding impls for `Mutex<T>`.
|
||||
//
|
||||
// If a `T` is sendable across threads, so is the lock, and `T` must be sendable
|
||||
// across threads to be `Sync` because it allows mutable access from multiple
|
||||
// threads.
|
||||
unsafe impl<T: Send> Send for Lock<T> {}
|
||||
unsafe impl<T: Send> Sync for Lock<T> {}
|
||||
|
||||
impl<T> Lock<T> {
|
||||
/// Creates a new lock around the given value.
|
||||
pub fn new(t: T) -> Lock<T> {
|
||||
Lock {
|
||||
locked: AtomicBool::new(false),
|
||||
data: UnsafeCell::new(t),
|
||||
}
|
||||
}
|
||||
|
||||
/// Attempts to acquire this lock, returning whether the lock was acquired or
|
||||
/// not.
|
||||
///
|
||||
/// If `Some` is returned then the data this lock protects can be accessed
|
||||
/// through the sentinel. This sentinel allows both mutable and immutable
|
||||
/// access.
|
||||
///
|
||||
/// If `None` is returned then the lock is already locked, either elsewhere
|
||||
/// on this thread or on another thread.
|
||||
pub fn try_lock(&self) -> Option<TryLock<T>> {
|
||||
if !self.locked.swap(true, SeqCst) {
|
||||
Some(TryLock { __ptr: self })
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, T> Deref for TryLock<'a, T> {
|
||||
type Target = T;
|
||||
fn deref(&self) -> &T {
|
||||
// The existence of `TryLock` represents that we own the lock, so we
|
||||
// can safely access the data here.
|
||||
unsafe { &*self.__ptr.data.get() }
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, T> DerefMut for TryLock<'a, T> {
|
||||
fn deref_mut(&mut self) -> &mut T {
|
||||
// The existence of `TryLock` represents that we own the lock, so we
|
||||
// can safely access the data here.
|
||||
//
|
||||
// Additionally, we're the *only* `TryLock` in existence so mutable
|
||||
// access should be ok.
|
||||
unsafe { &mut *self.__ptr.data.get() }
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, T> Drop for TryLock<'a, T> {
|
||||
fn drop(&mut self) {
|
||||
self.__ptr.locked.store(false, SeqCst);
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::Lock;
|
||||
|
||||
#[test]
|
||||
fn smoke() {
|
||||
let a = Lock::new(1);
|
||||
let mut a1 = a.try_lock().unwrap();
|
||||
assert!(a.try_lock().is_none());
|
||||
assert_eq!(*a1, 1);
|
||||
*a1 = 2;
|
||||
drop(a1);
|
||||
assert_eq!(*a.try_lock().unwrap(), 2);
|
||||
assert_eq!(*a.try_lock().unwrap(), 2);
|
||||
}
|
||||
}
|
@ -1,989 +0,0 @@
|
||||
//! A multi-producer, single-consumer, futures-aware, FIFO queue with back pressure.
|
||||
//!
|
||||
//! A channel can be used as a communication primitive between tasks running on
|
||||
//! `futures-rs` executors. Channel creation provides `Receiver` and `Sender`
|
||||
//! handles. `Receiver` implements `Stream` and allows a task to read values
|
||||
//! out of the channel. If there is no message to read from the channel, the
|
||||
//! current task will be notified when a new value is sent. `Sender` implements
|
||||
//! the `Sink` trait and allows a task to send messages into the channel. If
|
||||
//! the channel is at capacity, then send will be rejected and the task will be
|
||||
//! notified when additional capacity is available.
|
||||
//!
|
||||
//! # 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, `Sender::poll` will return `Ok(Ready(None))`.
|
||||
//!
|
||||
//! If the receiver handle is dropped, then messages can no longer be read out
|
||||
//! of the channel. In this case, a `send` will result in an error.
|
||||
//!
|
||||
//! # Clean Shutdown
|
||||
//!
|
||||
//! If the `Receiver` is simply dropped, then it is possible for there to be
|
||||
//! messages still in the channel that will not be processed. As such, it is
|
||||
//! usually desirable to perform a "clean" shutdown. To do this, the receiver
|
||||
//! will first call `close`, which will prevent any further messages to be sent
|
||||
//! into the channel. Then, the receiver consumes the channel to completion, at
|
||||
//! which point the receiver can be dropped.
|
||||
|
||||
// At the core, the channel uses an atomic FIFO queue for message passing. This
|
||||
// queue is used as the primary coordination primitive. In order to enforce
|
||||
// capacity limits and handle back pressure, a secondary FIFO queue is used to
|
||||
// send parked task handles.
|
||||
//
|
||||
// The general idea is that the channel is created with a `buffer` size of `n`.
|
||||
// The channel capacity is `n + num-senders`. Each sender gets one "guaranteed"
|
||||
// slot to hold a message. This allows `Sender` to know for a fact that a send
|
||||
// will succeed *before* starting to do the actual work of sending the value.
|
||||
// Since most of this work is lock-free, once the work starts, it is impossible
|
||||
// to safely revert.
|
||||
//
|
||||
// If the sender is unable to process a send operation, then the current
|
||||
// task is parked and the handle is sent on the parked task queue.
|
||||
//
|
||||
// Note that the implementation guarantees that the channel capacity will never
|
||||
// exceed the configured limit, however there is no *strict* guarantee that the
|
||||
// receiver will wake up a parked task *immediately* when a slot becomes
|
||||
// available. However, it will almost always unpark a task when a slot becomes
|
||||
// available and it is *guaranteed* that a sender will be unparked when the
|
||||
// message that caused the sender to become parked is read out of the channel.
|
||||
//
|
||||
// The steps for sending a message are roughly:
|
||||
//
|
||||
// 1) Increment the channel message count
|
||||
// 2) If the channel is at capacity, push the task handle onto the wait queue
|
||||
// 3) Push the message onto the message queue.
|
||||
//
|
||||
// The steps for receiving a message are roughly:
|
||||
//
|
||||
// 1) Pop a message from the message queue
|
||||
// 2) Pop a task handle from the wait queue
|
||||
// 3) Decrement the channel message count.
|
||||
//
|
||||
// It's important for the order of operations on lock-free structures to happen
|
||||
// in reverse order between the sender and receiver. This makes the message
|
||||
// queue the primary coordination structure and establishes the necessary
|
||||
// happens-before semantics required for the acquire / release semantics used
|
||||
// by the queue structure.
|
||||
|
||||
|
||||
|
||||
use mpsc::queue::{Queue, PopResult};
|
||||
|
||||
use futures::task::{self, Task};
|
||||
use futures::{Async, AsyncSink, Poll, StartSend, Sink, Stream};
|
||||
|
||||
use std::fmt;
|
||||
use std::error::Error;
|
||||
use std::any::Any;
|
||||
use std::sync::atomic::AtomicUsize;
|
||||
use std::sync::atomic::Ordering::SeqCst;
|
||||
use std::sync::{Arc, Mutex};
|
||||
use std::thread;
|
||||
use std::usize;
|
||||
|
||||
mod queue;
|
||||
|
||||
/// The transmission end of a channel which is used to send values.
|
||||
///
|
||||
/// This is created by the `channel` method.
|
||||
#[derive(Debug)]
|
||||
pub struct Sender<T> {
|
||||
// Channel state shared between the sender and receiver.
|
||||
inner: Arc<Inner<T>>,
|
||||
|
||||
// Handle to the task that is blocked on this sender. This handle is sent
|
||||
// to the receiver half in order to be notified when the sender becomes
|
||||
// unblocked.
|
||||
sender_task: Arc<Mutex<SenderTask>>,
|
||||
|
||||
// True if the sender might be blocked. This is an optimization to avoid
|
||||
// having to lock the mutex most of the time.
|
||||
maybe_parked: bool,
|
||||
}
|
||||
|
||||
/// The transmission end of a channel which is used to send values.
|
||||
///
|
||||
/// This is created by the `unbounded` method.
|
||||
#[derive(Debug)]
|
||||
pub struct UnboundedSender<T>(Sender<T>);
|
||||
|
||||
trait AssertKinds: Send + Sync + Clone {}
|
||||
impl AssertKinds for UnboundedSender<u32> {}
|
||||
|
||||
|
||||
/// The receiving end of a channel which implements the `Stream` trait.
|
||||
///
|
||||
/// This is a concrete implementation of a stream which can be used to represent
|
||||
/// a stream of values being computed elsewhere. This is created by the
|
||||
/// `channel` method.
|
||||
#[derive(Debug)]
|
||||
pub struct Receiver<T> {
|
||||
inner: Arc<Inner<T>>,
|
||||
}
|
||||
|
||||
/// Error type for sending, used when the receiving end of a channel is
|
||||
/// dropped
|
||||
#[derive(Clone, PartialEq, Eq)]
|
||||
pub struct SendError<T>(T);
|
||||
|
||||
/// Error type returned from `try_send`
|
||||
#[derive(Clone, PartialEq, Eq)]
|
||||
pub struct TrySendError<T> {
|
||||
kind: TrySendErrorKind<T>,
|
||||
}
|
||||
|
||||
#[derive(Clone, PartialEq, Eq)]
|
||||
enum TrySendErrorKind<T> {
|
||||
Full(T),
|
||||
Disconnected(T),
|
||||
}
|
||||
|
||||
impl<T> fmt::Debug for SendError<T> {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
|
||||
fmt.debug_tuple("SendError")
|
||||
.field(&"...")
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> fmt::Display for SendError<T> {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
|
||||
write!(fmt, "send failed because receiver is gone")
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Any> Error for SendError<T>
|
||||
{
|
||||
fn description(&self) -> &str {
|
||||
"send failed because receiver is gone"
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> SendError<T> {
|
||||
/// Returns the message that was attempted to be sent but failed.
|
||||
pub fn into_inner(self) -> T {
|
||||
self.0
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> fmt::Debug for TrySendError<T> {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
|
||||
fmt.debug_tuple("TrySendError")
|
||||
.field(&"...")
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> fmt::Display for TrySendError<T> {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
|
||||
if self.is_full() {
|
||||
write!(fmt, "send failed because channel is full")
|
||||
} else {
|
||||
write!(fmt, "send failed because receiver is gone")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Any> Error for TrySendError<T> {
|
||||
fn description(&self) -> &str {
|
||||
if self.is_full() {
|
||||
"send failed because channel is full"
|
||||
} else {
|
||||
"send failed because receiver is gone"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> TrySendError<T> {
|
||||
/// Returns true if this error is a result of the channel being full
|
||||
pub fn is_full(&self) -> bool {
|
||||
use self::TrySendErrorKind::*;
|
||||
|
||||
match self.kind {
|
||||
Full(_) => true,
|
||||
_ => false,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if this error is a result of the receiver being dropped
|
||||
pub fn is_disconnected(&self) -> bool {
|
||||
use self::TrySendErrorKind::*;
|
||||
|
||||
match self.kind {
|
||||
Disconnected(_) => true,
|
||||
_ => false,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the message that was attempted to be sent but failed.
|
||||
pub fn into_inner(self) -> T {
|
||||
use self::TrySendErrorKind::*;
|
||||
|
||||
match self.kind {
|
||||
Full(v) | Disconnected(v) => v,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct Inner<T> {
|
||||
// Max buffer size of the channel. If `None` then the channel is unbounded.
|
||||
buffer: Option<usize>,
|
||||
|
||||
// Internal channel state. Consists of the number of messages stored in the
|
||||
// channel as well as a flag signalling that the channel is closed.
|
||||
state: AtomicUsize,
|
||||
|
||||
// Atomic, FIFO queue used to send messages to the receiver
|
||||
message_queue: Queue<Option<T>>,
|
||||
|
||||
// Atomic, FIFO queue used to send parked task handles to the receiver.
|
||||
parked_queue: Queue<Arc<Mutex<SenderTask>>>,
|
||||
|
||||
// Number of senders in existence
|
||||
num_senders: AtomicUsize,
|
||||
|
||||
// Handle to the receiver's task.
|
||||
recv_task: Mutex<ReceiverTask>,
|
||||
}
|
||||
|
||||
// Struct representation of `Inner::state`.
|
||||
#[derive(Debug, Clone, Copy)]
|
||||
struct State {
|
||||
// `true` when the channel is open
|
||||
is_open: bool,
|
||||
|
||||
// Number of messages in the channel
|
||||
num_messages: usize,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct ReceiverTask {
|
||||
unparked: bool,
|
||||
task: Option<Task>,
|
||||
}
|
||||
|
||||
// Returned from Receiver::try_park()
|
||||
enum TryPark {
|
||||
Parked,
|
||||
Closed,
|
||||
NotEmpty,
|
||||
}
|
||||
|
||||
// The `is_open` flag is stored in the left-most bit of `Inner::state`
|
||||
const OPEN_MASK: usize = usize::MAX - (usize::MAX >> 1);
|
||||
|
||||
// When a new channel is created, it is created in the open state with no
|
||||
// pending messages.
|
||||
const INIT_STATE: usize = OPEN_MASK;
|
||||
|
||||
// The maximum number of messages that a channel can track is `usize::MAX >> 1`
|
||||
const MAX_CAPACITY: usize = !(OPEN_MASK);
|
||||
|
||||
// The maximum requested buffer size must be less than the maximum capacity of
|
||||
// a channel. This is because each sender gets a guaranteed slot.
|
||||
const MAX_BUFFER: usize = MAX_CAPACITY >> 1;
|
||||
|
||||
// Sent to the consumer to wake up blocked producers
|
||||
#[derive(Debug)]
|
||||
struct SenderTask {
|
||||
task: Option<Task>,
|
||||
is_parked: bool,
|
||||
}
|
||||
|
||||
impl SenderTask {
|
||||
fn new() -> Self {
|
||||
SenderTask {
|
||||
task: None,
|
||||
is_parked: false,
|
||||
}
|
||||
}
|
||||
|
||||
fn notify(&mut self) {
|
||||
self.is_parked = false;
|
||||
|
||||
if let Some(task) = self.task.take() {
|
||||
task.notify();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Creates an in-memory channel implementation of the `Stream` trait with
|
||||
/// bounded capacity.
|
||||
///
|
||||
/// This method creates a concrete implementation of the `Stream` trait which
|
||||
/// can be used to send values across threads in a streaming fashion. This
|
||||
/// channel is unique in that it implements back pressure to ensure that the
|
||||
/// sender never outpaces the receiver. The channel capacity is equal to
|
||||
/// `buffer + num-senders`. In other words, each sender gets a guaranteed slot
|
||||
/// in the channel capacity, and on top of that there are `buffer` "first come,
|
||||
/// first serve" slots available to all senders.
|
||||
///
|
||||
/// The `Receiver` returned implements the `Stream` trait and has access to any
|
||||
/// number of the associated combinators for transforming the result.
|
||||
pub fn channel<T>(buffer: usize) -> (Sender<T>, Receiver<T>) {
|
||||
// Check that the requested buffer size does not exceed the maximum buffer
|
||||
// size permitted by the system.
|
||||
assert!(buffer < MAX_BUFFER, "requested buffer size too large");
|
||||
channel2(Some(buffer))
|
||||
}
|
||||
|
||||
/// Creates an in-memory channel implementation of the `Stream` trait with
|
||||
/// unbounded capacity.
|
||||
///
|
||||
/// This method creates a concrete implementation of the `Stream` trait which
|
||||
/// can be used to send values across threads in a streaming fashion. 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 buffered
|
||||
/// internally.
|
||||
///
|
||||
/// **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<T>() -> (UnboundedSender<T>, Receiver<T>) {
|
||||
let (tx, rx) = channel2(None);
|
||||
(UnboundedSender(tx), rx)
|
||||
}
|
||||
|
||||
fn channel2<T>(buffer: Option<usize>) -> (Sender<T>, Receiver<T>) {
|
||||
let inner = Arc::new(Inner {
|
||||
buffer: buffer,
|
||||
state: AtomicUsize::new(INIT_STATE),
|
||||
message_queue: Queue::new(),
|
||||
parked_queue: Queue::new(),
|
||||
num_senders: AtomicUsize::new(1),
|
||||
recv_task: Mutex::new(ReceiverTask {
|
||||
unparked: false,
|
||||
task: None,
|
||||
}),
|
||||
});
|
||||
|
||||
let tx = Sender {
|
||||
inner: inner.clone(),
|
||||
sender_task: Arc::new(Mutex::new(SenderTask::new())),
|
||||
maybe_parked: false,
|
||||
};
|
||||
|
||||
let rx = Receiver {
|
||||
inner: inner,
|
||||
};
|
||||
|
||||
(tx, rx)
|
||||
}
|
||||
|
||||
/*
|
||||
*
|
||||
* ===== impl Sender =====
|
||||
*
|
||||
*/
|
||||
|
||||
impl<T> Sender<T> {
|
||||
/// Attempts to send a message on this `Sender<T>` without blocking.
|
||||
///
|
||||
/// This function, unlike `start_send`, is safe to call whether it's being
|
||||
/// called on a task or not. Note that this function, however, will *not*
|
||||
/// attempt to block the current task if the message cannot be sent.
|
||||
///
|
||||
/// It is not recommended to call this function from inside of a future,
|
||||
/// only from an external thread where you've otherwise arranged to be
|
||||
/// notified when the channel is no longer full.
|
||||
pub fn try_send(&mut self, msg: T) -> Result<(), TrySendError<T>> {
|
||||
// If the sender is currently blocked, reject the message
|
||||
if !self.poll_unparked(false).is_ready() {
|
||||
return Err(TrySendError {
|
||||
kind: TrySendErrorKind::Full(msg),
|
||||
});
|
||||
}
|
||||
|
||||
// The channel has capacity to accept the message, so send it
|
||||
self.do_send(Some(msg), false)
|
||||
.map_err(|SendError(v)| {
|
||||
TrySendError {
|
||||
kind: TrySendErrorKind::Disconnected(v),
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// Do the send without failing
|
||||
// None means close
|
||||
fn do_send(&mut self, msg: Option<T>, do_park: bool) -> Result<(), SendError<T>> {
|
||||
// First, increment the number of messages contained by the channel.
|
||||
// This operation will also atomically determine if the sender task
|
||||
// should be parked.
|
||||
//
|
||||
// None is returned in the case that the channel has been closed by the
|
||||
// receiver. This happens when `Receiver::close` is called or the
|
||||
// receiver is dropped.
|
||||
let park_self = match self.inc_num_messages(msg.is_none()) {
|
||||
Some(park_self) => park_self,
|
||||
None => {
|
||||
// The receiver has closed the channel. Only abort if actually
|
||||
// sending a message. It is important that the stream
|
||||
// termination (None) is always sent. This technically means
|
||||
// that it is possible for the queue to contain the following
|
||||
// number of messages:
|
||||
//
|
||||
// num-senders + buffer + 1
|
||||
//
|
||||
if let Some(msg) = msg {
|
||||
return Err(SendError(msg));
|
||||
} else {
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// If the channel has reached capacity, then the sender task needs to
|
||||
// be parked. This will send the task handle on the parked task queue.
|
||||
//
|
||||
// However, when `do_send` is called while dropping the `Sender`,
|
||||
// `task::current()` can't be called safely. In this case, in order to
|
||||
// maintain internal consistency, a blank message is pushed onto the
|
||||
// parked task queue.
|
||||
if park_self {
|
||||
self.park(do_park);
|
||||
}
|
||||
|
||||
self.queue_push_and_signal(msg);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// Do the send without parking current task.
|
||||
//
|
||||
// To be called from unbounded sender.
|
||||
fn do_send_nb(&self, msg: T) -> Result<(), SendError<T>> {
|
||||
match self.inc_num_messages(false) {
|
||||
Some(park_self) => assert!(!park_self),
|
||||
None => return Err(SendError(msg)),
|
||||
};
|
||||
|
||||
self.queue_push_and_signal(Some(msg));
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// Push message to the queue and signal to the receiver
|
||||
fn queue_push_and_signal(&self, msg: Option<T>) {
|
||||
// Push the message onto the message queue
|
||||
self.inner.message_queue.push(msg);
|
||||
|
||||
// Signal to the receiver that a message has been enqueued. If the
|
||||
// receiver is parked, this will unpark the task.
|
||||
self.signal();
|
||||
}
|
||||
|
||||
// Increment the number of queued messages. Returns if the sender should
|
||||
// block.
|
||||
fn inc_num_messages(&self, close: bool) -> Option<bool> {
|
||||
let mut curr = self.inner.state.load(SeqCst);
|
||||
|
||||
loop {
|
||||
let mut state = decode_state(curr);
|
||||
|
||||
// The receiver end closed the channel.
|
||||
if !state.is_open {
|
||||
return None;
|
||||
}
|
||||
|
||||
// This probably is never hit? Odds are the process will run out of
|
||||
// memory first. It may be worth to return something else in this
|
||||
// case?
|
||||
assert!(state.num_messages < MAX_CAPACITY, "buffer space exhausted; \
|
||||
sending this messages would overflow the state");
|
||||
|
||||
state.num_messages += 1;
|
||||
|
||||
// The channel is closed by all sender handles being dropped.
|
||||
if close {
|
||||
state.is_open = false;
|
||||
}
|
||||
|
||||
let next = encode_state(&state);
|
||||
match self.inner.state.compare_exchange(curr, next, SeqCst, SeqCst) {
|
||||
Ok(_) => {
|
||||
// Block if the current number of pending messages has exceeded
|
||||
// the configured buffer size
|
||||
let park_self = match self.inner.buffer {
|
||||
Some(buffer) => state.num_messages > buffer,
|
||||
None => false,
|
||||
};
|
||||
|
||||
return Some(park_self)
|
||||
}
|
||||
Err(actual) => curr = actual,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Signal to the receiver task that a message has been enqueued
|
||||
fn signal(&self) {
|
||||
// TODO
|
||||
// This logic can probably be improved by guarding the lock with an
|
||||
// atomic.
|
||||
//
|
||||
// Do this step first so that the lock is dropped when
|
||||
// `unpark` is called
|
||||
let task = {
|
||||
let mut recv_task = self.inner.recv_task.lock().unwrap();
|
||||
|
||||
// If the receiver has already been unparked, then there is nothing
|
||||
// more to do
|
||||
if recv_task.unparked {
|
||||
return;
|
||||
}
|
||||
|
||||
// Setting this flag enables the receiving end to detect that
|
||||
// an unpark event happened in order to avoid unnecessarily
|
||||
// parking.
|
||||
recv_task.unparked = true;
|
||||
recv_task.task.take()
|
||||
};
|
||||
|
||||
if let Some(task) = task {
|
||||
task.notify();
|
||||
}
|
||||
}
|
||||
|
||||
fn park(&mut self, can_park: bool) {
|
||||
// TODO: clean up internal state if the task::current will fail
|
||||
|
||||
let task = if can_park {
|
||||
Some(task::current())
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
{
|
||||
let mut sender = self.sender_task.lock().unwrap();
|
||||
sender.task = task;
|
||||
sender.is_parked = true;
|
||||
}
|
||||
|
||||
// Send handle over queue
|
||||
let t = self.sender_task.clone();
|
||||
self.inner.parked_queue.push(t);
|
||||
|
||||
// Check to make sure we weren't closed after we sent our task on the
|
||||
// queue
|
||||
let state = decode_state(self.inner.state.load(SeqCst));
|
||||
self.maybe_parked = state.is_open;
|
||||
}
|
||||
|
||||
/// Polls the channel to determine if there is guaranteed to be capacity to send at least one
|
||||
/// item without waiting.
|
||||
///
|
||||
/// Returns `Ok(Async::Ready(_))` if there is sufficient capacity, or returns
|
||||
/// `Ok(Async::NotReady)` if the channel is not guaranteed to have capacity. Returns
|
||||
/// `Err(SendError(_))` if the receiver has been dropped.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// This method will panic if called from outside the context of a task or future.
|
||||
pub fn poll_ready(&mut self) -> Poll<(), SendError<()>> {
|
||||
let state = decode_state(self.inner.state.load(SeqCst));
|
||||
if !state.is_open {
|
||||
return Err(SendError(()));
|
||||
}
|
||||
|
||||
Ok(self.poll_unparked(true))
|
||||
}
|
||||
|
||||
fn poll_unparked(&mut self, do_park: bool) -> Async<()> {
|
||||
// First check the `maybe_parked` variable. This avoids acquiring the
|
||||
// lock in most cases
|
||||
if self.maybe_parked {
|
||||
// Get a lock on the task handle
|
||||
let mut task = self.sender_task.lock().unwrap();
|
||||
|
||||
if !task.is_parked {
|
||||
self.maybe_parked = false;
|
||||
return Async::Ready(())
|
||||
}
|
||||
|
||||
// At this point, an unpark request is pending, so there will be an
|
||||
// unpark sometime in the future. We just need to make sure that
|
||||
// the correct task will be notified.
|
||||
//
|
||||
// Update the task in case the `Sender` has been moved to another
|
||||
// task
|
||||
task.task = if do_park {
|
||||
Some(task::current())
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
Async::NotReady
|
||||
} else {
|
||||
Async::Ready(())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Sink for Sender<T> {
|
||||
type SinkItem = T;
|
||||
type SinkError = SendError<T>;
|
||||
|
||||
fn start_send(&mut self, msg: T) -> StartSend<T, SendError<T>> {
|
||||
// If the sender is currently blocked, reject the message before doing
|
||||
// any work.
|
||||
if !self.poll_unparked(true).is_ready() {
|
||||
return Ok(AsyncSink::NotReady(msg));
|
||||
}
|
||||
|
||||
// The channel has capacity to accept the message, so send it.
|
||||
self.do_send(Some(msg), true)?;
|
||||
|
||||
Ok(AsyncSink::Ready)
|
||||
}
|
||||
|
||||
fn poll_complete(&mut self) -> Poll<(), SendError<T>> {
|
||||
Ok(Async::Ready(()))
|
||||
}
|
||||
|
||||
fn close(&mut self) -> Poll<(), SendError<T>> {
|
||||
Ok(Async::Ready(()))
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> UnboundedSender<T> {
|
||||
/// Sends the provided message along this channel.
|
||||
///
|
||||
/// This is an unbounded sender, so this function differs from `Sink::send`
|
||||
/// by ensuring the return type reflects that the channel is always ready to
|
||||
/// receive messages.
|
||||
#[deprecated(note = "renamed to `unbounded_send`")]
|
||||
#[doc(hidden)]
|
||||
pub fn send(&self, msg: T) -> Result<(), SendError<T>> {
|
||||
self.unbounded_send(msg)
|
||||
}
|
||||
|
||||
/// Sends the provided message along this channel.
|
||||
///
|
||||
/// This is an unbounded sender, so this function differs from `Sink::send`
|
||||
/// by ensuring the return type reflects that the channel is always ready to
|
||||
/// receive messages.
|
||||
pub fn unbounded_send(&self, msg: T) -> Result<(), SendError<T>> {
|
||||
self.0.do_send_nb(msg)
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Sink for UnboundedSender<T> {
|
||||
type SinkItem = T;
|
||||
type SinkError = SendError<T>;
|
||||
|
||||
fn start_send(&mut self, msg: T) -> StartSend<T, SendError<T>> {
|
||||
self.0.start_send(msg)
|
||||
}
|
||||
|
||||
fn poll_complete(&mut self) -> Poll<(), SendError<T>> {
|
||||
self.0.poll_complete()
|
||||
}
|
||||
|
||||
fn close(&mut self) -> Poll<(), SendError<T>> {
|
||||
Ok(Async::Ready(()))
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, T> Sink for &'a UnboundedSender<T> {
|
||||
type SinkItem = T;
|
||||
type SinkError = SendError<T>;
|
||||
|
||||
fn start_send(&mut self, msg: T) -> StartSend<T, SendError<T>> {
|
||||
self.0.do_send_nb(msg)?;
|
||||
Ok(AsyncSink::Ready)
|
||||
}
|
||||
|
||||
fn poll_complete(&mut self) -> Poll<(), SendError<T>> {
|
||||
Ok(Async::Ready(()))
|
||||
}
|
||||
|
||||
fn close(&mut self) -> Poll<(), SendError<T>> {
|
||||
Ok(Async::Ready(()))
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Clone for UnboundedSender<T> {
|
||||
fn clone(&self) -> UnboundedSender<T> {
|
||||
UnboundedSender(self.0.clone())
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
impl<T> Clone for Sender<T> {
|
||||
fn clone(&self) -> Sender<T> {
|
||||
// Since this atomic op isn't actually guarding any memory and we don't
|
||||
// care about any orderings besides the ordering on the single atomic
|
||||
// variable, a relaxed ordering is acceptable.
|
||||
let mut curr = self.inner.num_senders.load(SeqCst);
|
||||
|
||||
loop {
|
||||
// If the maximum number of senders has been reached, then fail
|
||||
if curr == self.inner.max_senders() {
|
||||
panic!("cannot clone `Sender` -- too many outstanding senders");
|
||||
}
|
||||
|
||||
debug_assert!(curr < self.inner.max_senders());
|
||||
|
||||
let next = curr + 1;
|
||||
let actual = self.inner.num_senders.compare_and_swap(curr, next, SeqCst);
|
||||
|
||||
// The ABA problem doesn't matter here. We only care that the
|
||||
// number of senders never exceeds the maximum.
|
||||
if actual == curr {
|
||||
return Sender {
|
||||
inner: self.inner.clone(),
|
||||
sender_task: Arc::new(Mutex::new(SenderTask::new())),
|
||||
maybe_parked: false,
|
||||
};
|
||||
}
|
||||
|
||||
curr = actual;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Drop for Sender<T> {
|
||||
fn drop(&mut self) {
|
||||
// Ordering between variables don't matter here
|
||||
let prev = self.inner.num_senders.fetch_sub(1, SeqCst);
|
||||
|
||||
if prev == 1 {
|
||||
let _ = self.do_send(None, false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
*
|
||||
* ===== impl Receiver =====
|
||||
*
|
||||
*/
|
||||
|
||||
impl<T> Receiver<T> {
|
||||
/// Closes the receiving half
|
||||
///
|
||||
/// 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) {
|
||||
let mut curr = self.inner.state.load(SeqCst);
|
||||
|
||||
loop {
|
||||
let mut state = decode_state(curr);
|
||||
|
||||
if !state.is_open {
|
||||
break
|
||||
}
|
||||
|
||||
state.is_open = false;
|
||||
|
||||
let next = encode_state(&state);
|
||||
match self.inner.state.compare_exchange(curr, next, SeqCst, SeqCst) {
|
||||
Ok(_) => break,
|
||||
Err(actual) => curr = actual,
|
||||
}
|
||||
}
|
||||
|
||||
// Wake up any threads waiting as they'll see that we've closed the
|
||||
// channel and will continue on their merry way.
|
||||
loop {
|
||||
match unsafe { self.inner.parked_queue.pop() } {
|
||||
PopResult::Data(task) => {
|
||||
task.lock().unwrap().notify();
|
||||
}
|
||||
PopResult::Empty => break,
|
||||
PopResult::Inconsistent => thread::yield_now(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn next_message(&mut self) -> Async<Option<T>> {
|
||||
// Pop off a message
|
||||
loop {
|
||||
match unsafe { self.inner.message_queue.pop() } {
|
||||
PopResult::Data(msg) => {
|
||||
return Async::Ready(msg);
|
||||
}
|
||||
PopResult::Empty => {
|
||||
// The queue is empty, return NotReady
|
||||
return Async::NotReady;
|
||||
}
|
||||
PopResult::Inconsistent => {
|
||||
// Inconsistent means that there will be a message to pop
|
||||
// in a short time. This branch can only be reached if
|
||||
// values are being produced from another thread, so there
|
||||
// are a few ways that we can deal with this:
|
||||
//
|
||||
// 1) Spin
|
||||
// 2) thread::yield_now()
|
||||
// 3) task::current().unwrap() & return NotReady
|
||||
//
|
||||
// For now, thread::yield_now() is used, but it would
|
||||
// probably be better to spin a few times then yield.
|
||||
thread::yield_now();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Unpark a single task handle if there is one pending in the parked queue
|
||||
fn unpark_one(&mut self) {
|
||||
loop {
|
||||
match unsafe { self.inner.parked_queue.pop() } {
|
||||
PopResult::Data(task) => {
|
||||
task.lock().unwrap().notify();
|
||||
return;
|
||||
}
|
||||
PopResult::Empty => {
|
||||
// Queue empty, no task to wake up.
|
||||
return;
|
||||
}
|
||||
PopResult::Inconsistent => {
|
||||
// Same as above
|
||||
thread::yield_now();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Try to park the receiver task
|
||||
fn try_park(&self) -> TryPark {
|
||||
let curr = self.inner.state.load(SeqCst);
|
||||
let state = decode_state(curr);
|
||||
|
||||
// If the channel is closed, then there is no need to park.
|
||||
if !state.is_open && state.num_messages == 0 {
|
||||
return TryPark::Closed;
|
||||
}
|
||||
|
||||
// First, track the task in the `recv_task` slot
|
||||
let mut recv_task = self.inner.recv_task.lock().unwrap();
|
||||
|
||||
if recv_task.unparked {
|
||||
// Consume the `unpark` signal without actually parking
|
||||
recv_task.unparked = false;
|
||||
return TryPark::NotEmpty;
|
||||
}
|
||||
|
||||
recv_task.task = Some(task::current());
|
||||
TryPark::Parked
|
||||
}
|
||||
|
||||
fn dec_num_messages(&self) {
|
||||
let mut curr = self.inner.state.load(SeqCst);
|
||||
|
||||
loop {
|
||||
let mut state = decode_state(curr);
|
||||
|
||||
state.num_messages -= 1;
|
||||
|
||||
let next = encode_state(&state);
|
||||
match self.inner.state.compare_exchange(curr, next, SeqCst, SeqCst) {
|
||||
Ok(_) => break,
|
||||
Err(actual) => curr = actual,
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Stream for Receiver<T> {
|
||||
type Item = T;
|
||||
type Error = ();
|
||||
|
||||
fn poll(&mut self) -> Poll<Option<T>, ()> {
|
||||
loop {
|
||||
// Try to read a message off of the message queue.
|
||||
let msg = match self.next_message() {
|
||||
Async::Ready(msg) => msg,
|
||||
Async::NotReady => {
|
||||
// There are no messages to read, in this case, attempt to
|
||||
// park. The act of parking will verify that the channel is
|
||||
// still empty after the park operation has completed.
|
||||
match self.try_park() {
|
||||
TryPark::Parked => {
|
||||
// The task was parked, and the channel is still
|
||||
// empty, return NotReady.
|
||||
return Ok(Async::NotReady);
|
||||
}
|
||||
TryPark::Closed => {
|
||||
// The channel is closed, there will be no further
|
||||
// messages.
|
||||
return Ok(Async::Ready(None));
|
||||
}
|
||||
TryPark::NotEmpty => {
|
||||
// A message has been sent while attempting to
|
||||
// park. Loop again, the next iteration is
|
||||
// guaranteed to get the message.
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// If there are any parked task handles in the parked queue, pop
|
||||
// one and unpark it.
|
||||
self.unpark_one();
|
||||
|
||||
// Decrement number of messages
|
||||
self.dec_num_messages();
|
||||
|
||||
// Return the message
|
||||
return Ok(Async::Ready(msg));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Drop for Receiver<T> {
|
||||
fn drop(&mut self) {
|
||||
// Drain the channel of all pending messages
|
||||
self.close();
|
||||
while self.next_message().is_ready() {
|
||||
// ...
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
*
|
||||
* ===== impl Inner =====
|
||||
*
|
||||
*/
|
||||
|
||||
impl<T> Inner<T> {
|
||||
// The return value is such that the total number of messages that can be
|
||||
// enqueued into the channel will never exceed MAX_CAPACITY
|
||||
fn max_senders(&self) -> usize {
|
||||
match self.buffer {
|
||||
Some(buffer) => MAX_CAPACITY - buffer,
|
||||
None => MAX_BUFFER,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
unsafe impl<T: Send> Send for Inner<T> {}
|
||||
unsafe impl<T: Send> Sync for Inner<T> {}
|
||||
|
||||
/*
|
||||
*
|
||||
* ===== Helpers =====
|
||||
*
|
||||
*/
|
||||
|
||||
fn decode_state(num: usize) -> State {
|
||||
State {
|
||||
is_open: num & OPEN_MASK == OPEN_MASK,
|
||||
num_messages: num & MAX_CAPACITY,
|
||||
}
|
||||
}
|
||||
|
||||
fn encode_state(state: &State) -> usize {
|
||||
let mut num = state.num_messages;
|
||||
|
||||
if state.is_open {
|
||||
num |= OPEN_MASK;
|
||||
}
|
||||
|
||||
num
|
||||
}
|
@ -1,151 +0,0 @@
|
||||
/* Copyright (c) 2010-2011 Dmitry Vyukov. All rights reserved.
|
||||
* Redistribution and use in source and binary forms, with or without
|
||||
* modification, are permitted provided that the following conditions are met:
|
||||
*
|
||||
* 1. Redistributions of source code must retain the above copyright notice,
|
||||
* this list of conditions and the following disclaimer.
|
||||
*
|
||||
* 2. Redistributions in binary form must reproduce the above copyright
|
||||
* notice, this list of conditions and the following disclaimer in the
|
||||
* documentation and/or other materials provided with the distribution.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY DMITRY VYUKOV "AS IS" AND ANY EXPRESS OR IMPLIED
|
||||
* WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
|
||||
* MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
|
||||
* SHALL DMITRY VYUKOV OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
|
||||
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
|
||||
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
|
||||
* LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE
|
||||
* OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
|
||||
* ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
*
|
||||
* The views and conclusions contained in the software and documentation are
|
||||
* those of the authors and should not be interpreted as representing official
|
||||
* policies, either expressed or implied, of Dmitry Vyukov.
|
||||
*/
|
||||
|
||||
//! A mostly lock-free multi-producer, single consumer queue.
|
||||
//!
|
||||
//! This module contains an implementation of a concurrent MPSC queue. This
|
||||
//! queue can be used to share data between threads, and is also used as the
|
||||
//! building block of channels in rust.
|
||||
//!
|
||||
//! Note that the current implementation of this queue has a caveat of the `pop`
|
||||
//! method, and see the method for more information about it. Due to this
|
||||
//! caveat, this queue may not be appropriate for all use-cases.
|
||||
|
||||
// http://www.1024cores.net/home/lock-free-algorithms
|
||||
// /queues/non-intrusive-mpsc-node-based-queue
|
||||
|
||||
// NOTE: this implementation is lifted from the standard library and only
|
||||
// slightly modified
|
||||
|
||||
pub use self::PopResult::*;
|
||||
use std::prelude::v1::*;
|
||||
|
||||
use std::cell::UnsafeCell;
|
||||
use std::ptr;
|
||||
use std::sync::atomic::{AtomicPtr, Ordering};
|
||||
|
||||
/// A result of the `pop` function.
|
||||
pub enum PopResult<T> {
|
||||
/// Some data has been popped
|
||||
Data(T),
|
||||
/// The queue is empty
|
||||
Empty,
|
||||
/// The queue is in an inconsistent state. Popping data should succeed, but
|
||||
/// some pushers have yet to make enough progress in order allow a pop to
|
||||
/// succeed. It is recommended that a pop() occur "in the near future" in
|
||||
/// order to see if the sender has made progress or not
|
||||
Inconsistent,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct Node<T> {
|
||||
next: AtomicPtr<Node<T>>,
|
||||
value: Option<T>,
|
||||
}
|
||||
|
||||
/// The multi-producer single-consumer structure. This is not cloneable, but it
|
||||
/// may be safely shared so long as it is guaranteed that there is only one
|
||||
/// popper at a time (many pushers are allowed).
|
||||
#[derive(Debug)]
|
||||
pub struct Queue<T> {
|
||||
head: AtomicPtr<Node<T>>,
|
||||
tail: UnsafeCell<*mut Node<T>>,
|
||||
}
|
||||
|
||||
unsafe impl<T: Send> Send for Queue<T> { }
|
||||
unsafe impl<T: Send> Sync for Queue<T> { }
|
||||
|
||||
impl<T> Node<T> {
|
||||
unsafe fn new(v: Option<T>) -> *mut Node<T> {
|
||||
Box::into_raw(Box::new(Node {
|
||||
next: AtomicPtr::new(ptr::null_mut()),
|
||||
value: v,
|
||||
}))
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Queue<T> {
|
||||
/// Creates a new queue that is safe to share among multiple producers and
|
||||
/// one consumer.
|
||||
pub fn new() -> Queue<T> {
|
||||
let stub = unsafe { Node::new(None) };
|
||||
Queue {
|
||||
head: AtomicPtr::new(stub),
|
||||
tail: UnsafeCell::new(stub),
|
||||
}
|
||||
}
|
||||
|
||||
/// Pushes a new value onto this queue.
|
||||
pub fn push(&self, t: T) {
|
||||
unsafe {
|
||||
let n = Node::new(Some(t));
|
||||
let prev = self.head.swap(n, Ordering::AcqRel);
|
||||
(*prev).next.store(n, Ordering::Release);
|
||||
}
|
||||
}
|
||||
|
||||
/// Pops some data from this queue.
|
||||
///
|
||||
/// Note that the current implementation means that this function cannot
|
||||
/// return `Option<T>`. It is possible for this queue to be in an
|
||||
/// inconsistent state where many pushes have succeeded and completely
|
||||
/// finished, but pops cannot return `Some(t)`. This inconsistent state
|
||||
/// happens when a pusher is preempted at an inopportune moment.
|
||||
///
|
||||
/// This inconsistent state means that this queue does indeed have data, but
|
||||
/// it does not currently have access to it at this time.
|
||||
///
|
||||
/// This function is unsafe because only one thread can call it at a time.
|
||||
pub unsafe fn pop(&self) -> PopResult<T> {
|
||||
let tail = *self.tail.get();
|
||||
let next = (*tail).next.load(Ordering::Acquire);
|
||||
|
||||
if !next.is_null() {
|
||||
*self.tail.get() = next;
|
||||
assert!((*tail).value.is_none());
|
||||
assert!((*next).value.is_some());
|
||||
let ret = (*next).value.take().unwrap();
|
||||
drop(Box::from_raw(tail));
|
||||
return Data(ret);
|
||||
}
|
||||
|
||||
if self.head.load(Ordering::Acquire) == tail {Empty} else {Inconsistent}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Drop for Queue<T> {
|
||||
fn drop(&mut self) {
|
||||
unsafe {
|
||||
let mut cur = *self.tail.get();
|
||||
while !cur.is_null() {
|
||||
let next = (*cur).next.load(Ordering::Relaxed);
|
||||
drop(Box::from_raw(cur));
|
||||
cur = next;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -1,426 +0,0 @@
|
||||
//! A one-shot, futures-aware channel
|
||||
|
||||
use lock::Lock;
|
||||
|
||||
use futures::{Future, Poll, Async};
|
||||
use futures::task::{self, Task};
|
||||
|
||||
use std::sync::Arc;
|
||||
use std::sync::atomic::AtomicBool;
|
||||
use std::sync::atomic::Ordering::SeqCst;
|
||||
use std::error::Error;
|
||||
use std::fmt;
|
||||
|
||||
/// A future representing the completion of a computation happening elsewhere in
|
||||
/// memory.
|
||||
///
|
||||
/// This is created by the `oneshot::channel` function.
|
||||
#[must_use = "futures do nothing unless polled"]
|
||||
#[derive(Debug)]
|
||||
pub struct Receiver<T> {
|
||||
inner: Arc<Inner<T>>,
|
||||
}
|
||||
|
||||
/// Represents the completion half of a oneshot through which the result of a
|
||||
/// computation is signaled.
|
||||
///
|
||||
/// This is created by the `oneshot::channel` function.
|
||||
#[derive(Debug)]
|
||||
pub struct Sender<T> {
|
||||
inner: Arc<Inner<T>>,
|
||||
}
|
||||
|
||||
/// Internal state of the `Receiver`/`Sender` pair above. This is all used as
|
||||
/// the internal synchronization between the two for send/recv operations.
|
||||
#[derive(Debug)]
|
||||
struct Inner<T> {
|
||||
/// Indicates whether this oneshot is complete yet. This is filled in both
|
||||
/// by `Sender::drop` and by `Receiver::drop`, and both sides interpret it
|
||||
/// appropriately.
|
||||
///
|
||||
/// For `Receiver`, if this is `true`, then it's guaranteed that `data` is
|
||||
/// unlocked and ready to be inspected.
|
||||
///
|
||||
/// For `Sender` if this is `true` then the oneshot has gone away and it
|
||||
/// can return ready from `poll_cancel`.
|
||||
complete: AtomicBool,
|
||||
|
||||
/// The actual data being transferred as part of this `Receiver`. This is
|
||||
/// filled in by `Sender::complete` and read by `Receiver::poll`.
|
||||
///
|
||||
/// Note that this is protected by `Lock`, but it is in theory safe to
|
||||
/// replace with an `UnsafeCell` as it's actually protected by `complete`
|
||||
/// above. I wouldn't recommend doing this, however, unless someone is
|
||||
/// supremely confident in the various atomic orderings here and there.
|
||||
data: Lock<Option<T>>,
|
||||
|
||||
/// Field to store the task which is blocked in `Receiver::poll`.
|
||||
///
|
||||
/// This is filled in when a oneshot is polled but not ready yet. Note that
|
||||
/// the `Lock` here, unlike in `data` above, is important to resolve races.
|
||||
/// Both the `Receiver` and the `Sender` halves understand that if they
|
||||
/// can't acquire the lock then some important interference is happening.
|
||||
rx_task: Lock<Option<Task>>,
|
||||
|
||||
/// Like `rx_task` above, except for the task blocked in
|
||||
/// `Sender::poll_cancel`. Additionally, `Lock` cannot be `UnsafeCell`.
|
||||
tx_task: Lock<Option<Task>>,
|
||||
}
|
||||
|
||||
/// Creates a new futures-aware, one-shot channel.
|
||||
///
|
||||
/// This function is similar to Rust's channels found in the standard library.
|
||||
/// Two halves are returned, the first of which is a `Sender` handle, used to
|
||||
/// signal the end of a computation and provide its value. The second half is a
|
||||
/// `Receiver` which implements the `Future` trait, resolving to the value that
|
||||
/// was given to the `Sender` handle.
|
||||
///
|
||||
/// Each half can be separately owned and sent across threads/tasks.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
/// extern crate tokio_channel;
|
||||
/// extern crate futures;
|
||||
///
|
||||
/// use tokio_channel::oneshot;
|
||||
/// use futures::*;
|
||||
/// use std::thread;
|
||||
///
|
||||
/// # fn main() {
|
||||
/// let (p, c) = oneshot::channel::<i32>();
|
||||
///
|
||||
/// thread::spawn(|| {
|
||||
/// c.map(|i| {
|
||||
/// println!("got: {}", i);
|
||||
/// }).wait();
|
||||
/// });
|
||||
///
|
||||
/// p.send(3).unwrap();
|
||||
/// # }
|
||||
/// ```
|
||||
pub fn channel<T>() -> (Sender<T>, Receiver<T>) {
|
||||
let inner = Arc::new(Inner::new());
|
||||
let receiver = Receiver {
|
||||
inner: inner.clone(),
|
||||
};
|
||||
let sender = Sender {
|
||||
inner: inner,
|
||||
};
|
||||
(sender, receiver)
|
||||
}
|
||||
|
||||
impl<T> Inner<T> {
|
||||
fn new() -> Inner<T> {
|
||||
Inner {
|
||||
complete: AtomicBool::new(false),
|
||||
data: Lock::new(None),
|
||||
rx_task: Lock::new(None),
|
||||
tx_task: Lock::new(None),
|
||||
}
|
||||
}
|
||||
|
||||
fn send(&self, t: T) -> Result<(), T> {
|
||||
if self.complete.load(SeqCst) {
|
||||
return Err(t)
|
||||
}
|
||||
|
||||
// Note that this lock acquisition may fail if the receiver
|
||||
// is closed and sets the `complete` flag to true, whereupon
|
||||
// the receiver may call `poll()`.
|
||||
if let Some(mut slot) = self.data.try_lock() {
|
||||
assert!(slot.is_none());
|
||||
*slot = Some(t);
|
||||
drop(slot);
|
||||
|
||||
// If the receiver called `close()` between the check at the
|
||||
// start of the function, and the lock being released, then
|
||||
// the receiver may not be around to receive it, so try to
|
||||
// pull it back out.
|
||||
if self.complete.load(SeqCst) {
|
||||
// If lock acquisition fails, then receiver is actually
|
||||
// receiving it, so we're good.
|
||||
if let Some(mut slot) = self.data.try_lock() {
|
||||
if let Some(t) = slot.take() {
|
||||
return Err(t);
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
} else {
|
||||
// Must have been closed
|
||||
Err(t)
|
||||
}
|
||||
}
|
||||
|
||||
fn poll_cancel(&self) -> Poll<(), ()> {
|
||||
// Fast path up first, just read the flag and see if our other half is
|
||||
// gone. This flag is set both in our destructor and the oneshot
|
||||
// destructor, but our destructor hasn't run yet so if it's set then the
|
||||
// oneshot is gone.
|
||||
if self.complete.load(SeqCst) {
|
||||
return Ok(Async::Ready(()))
|
||||
}
|
||||
|
||||
// If our other half is not gone then we need to park our current task
|
||||
// and move it into the `notify_cancel` slot to get notified when it's
|
||||
// actually gone.
|
||||
//
|
||||
// If `try_lock` fails, then the `Receiver` is in the process of using
|
||||
// it, so we can deduce that it's now in the process of going away and
|
||||
// hence we're canceled. If it succeeds then we just store our handle.
|
||||
//
|
||||
// Crucially we then check `oneshot_gone` *again* before we return.
|
||||
// While we were storing our handle inside `notify_cancel` the `Receiver`
|
||||
// may have been dropped. The first thing it does is set the flag, and
|
||||
// if it fails to acquire the lock it assumes that we'll see the flag
|
||||
// later on. So... we then try to see the flag later on!
|
||||
let handle = task::current();
|
||||
match self.tx_task.try_lock() {
|
||||
Some(mut p) => *p = Some(handle),
|
||||
None => return Ok(Async::Ready(())),
|
||||
}
|
||||
if self.complete.load(SeqCst) {
|
||||
Ok(Async::Ready(()))
|
||||
} else {
|
||||
Ok(Async::NotReady)
|
||||
}
|
||||
}
|
||||
|
||||
fn is_canceled(&self) -> bool {
|
||||
self.complete.load(SeqCst)
|
||||
}
|
||||
|
||||
fn drop_tx(&self) {
|
||||
// Flag that we're a completed `Sender` and try to wake up a receiver.
|
||||
// Whether or not we actually stored any data will get picked up and
|
||||
// translated to either an item or cancellation.
|
||||
//
|
||||
// Note that if we fail to acquire the `rx_task` lock then that means
|
||||
// we're in one of two situations:
|
||||
//
|
||||
// 1. The receiver is trying to block in `poll`
|
||||
// 2. The receiver is being dropped
|
||||
//
|
||||
// In the first case it'll check the `complete` flag after it's done
|
||||
// blocking to see if it succeeded. In the latter case we don't need to
|
||||
// wake up anyone anyway. So in both cases it's ok to ignore the `None`
|
||||
// case of `try_lock` and bail out.
|
||||
//
|
||||
// The first case crucially depends on `Lock` using `SeqCst` ordering
|
||||
// under the hood. If it instead used `Release` / `Acquire` ordering,
|
||||
// then it would not necessarily synchronize with `inner.complete`
|
||||
// and deadlock might be possible, as was observed in
|
||||
// https://github.com/rust-lang-nursery/futures-rs/pull/219.
|
||||
self.complete.store(true, SeqCst);
|
||||
if let Some(mut slot) = self.rx_task.try_lock() {
|
||||
if let Some(task) = slot.take() {
|
||||
drop(slot);
|
||||
task.notify();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn close_rx(&self) {
|
||||
// Flag our completion and then attempt to wake up the sender if it's
|
||||
// blocked. See comments in `drop` below for more info
|
||||
self.complete.store(true, SeqCst);
|
||||
if let Some(mut handle) = self.tx_task.try_lock() {
|
||||
if let Some(task) = handle.take() {
|
||||
drop(handle);
|
||||
task.notify()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn recv(&self) -> Poll<T, Canceled> {
|
||||
let mut done = false;
|
||||
|
||||
// Check to see if some data has arrived. If it hasn't then we need to
|
||||
// block our task.
|
||||
//
|
||||
// Note that the acquisition of the `rx_task` lock might fail below, but
|
||||
// the only situation where this can happen is during `Sender::drop`
|
||||
// when we are indeed completed already. If that's happening then we
|
||||
// know we're completed so keep going.
|
||||
if self.complete.load(SeqCst) {
|
||||
done = true;
|
||||
} else {
|
||||
let task = task::current();
|
||||
match self.rx_task.try_lock() {
|
||||
Some(mut slot) => *slot = Some(task),
|
||||
None => done = true,
|
||||
}
|
||||
}
|
||||
|
||||
// If we're `done` via one of the paths above, then look at the data and
|
||||
// figure out what the answer is. If, however, we stored `rx_task`
|
||||
// successfully above we need to check again if we're completed in case
|
||||
// a message was sent while `rx_task` was locked and couldn't notify us
|
||||
// otherwise.
|
||||
//
|
||||
// If we're not done, and we're not complete, though, then we've
|
||||
// successfully blocked our task and we return `NotReady`.
|
||||
if done || self.complete.load(SeqCst) {
|
||||
// If taking the lock fails, the sender will realise that the we're
|
||||
// `done` when it checks the `complete` flag on the way out, and will
|
||||
// treat the send as a failure.
|
||||
if let Some(mut slot) = self.data.try_lock() {
|
||||
if let Some(data) = slot.take() {
|
||||
return Ok(data.into());
|
||||
}
|
||||
}
|
||||
Err(Canceled)
|
||||
} else {
|
||||
Ok(Async::NotReady)
|
||||
}
|
||||
}
|
||||
|
||||
fn drop_rx(&self) {
|
||||
// Indicate to the `Sender` that we're done, so any future calls to
|
||||
// `poll_cancel` are weeded out.
|
||||
self.complete.store(true, SeqCst);
|
||||
|
||||
// If we've blocked a task then there's no need for it to stick around,
|
||||
// so we need to drop it. If this lock acquisition fails, though, then
|
||||
// it's just because our `Sender` is trying to take the task, so we
|
||||
// let them take care of that.
|
||||
if let Some(mut slot) = self.rx_task.try_lock() {
|
||||
let task = slot.take();
|
||||
drop(slot);
|
||||
drop(task);
|
||||
}
|
||||
|
||||
// Finally, if our `Sender` wants to get notified of us going away, it
|
||||
// would have stored something in `tx_task`. Here we try to peel that
|
||||
// out and unpark it.
|
||||
//
|
||||
// Note that the `try_lock` here may fail, but only if the `Sender` is
|
||||
// in the process of filling in the task. If that happens then we
|
||||
// already flagged `complete` and they'll pick that up above.
|
||||
if let Some(mut handle) = self.tx_task.try_lock() {
|
||||
if let Some(task) = handle.take() {
|
||||
drop(handle);
|
||||
task.notify()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Sender<T> {
|
||||
#[deprecated(note = "renamed to `send`", since = "0.1.11")]
|
||||
#[doc(hidden)]
|
||||
#[cfg(feature = "with-deprecated")]
|
||||
pub fn complete(self, t: T) {
|
||||
drop(self.send(t));
|
||||
}
|
||||
|
||||
/// Completes this oneshot with a successful result.
|
||||
///
|
||||
/// This function will consume `self` and indicate to the other end, the
|
||||
/// `Receiver`, that the value provided is the result of the computation this
|
||||
/// represents.
|
||||
///
|
||||
/// If the value is successfully enqueued for the remote end to receive,
|
||||
/// then `Ok(())` is returned. If the receiving end was deallocated before
|
||||
/// this function was called, however, then `Err` is returned with the value
|
||||
/// provided.
|
||||
pub fn send(self, t: T) -> Result<(), T> {
|
||||
self.inner.send(t)
|
||||
}
|
||||
|
||||
/// Polls this `Sender` half to detect whether the `Receiver` this has
|
||||
/// paired with has gone away.
|
||||
///
|
||||
/// This function can be used to learn about when the `Receiver` (consumer)
|
||||
/// half has gone away and nothing will be able to receive a message sent
|
||||
/// from `send`.
|
||||
///
|
||||
/// If `Ready` is returned then it means that the `Receiver` has disappeared
|
||||
/// and the result this `Sender` would otherwise produce should no longer
|
||||
/// be produced.
|
||||
///
|
||||
/// If `NotReady` is returned then the `Receiver` is still alive and may be
|
||||
/// able to receive a message if sent. The current task, however, is
|
||||
/// scheduled to receive a notification if the corresponding `Receiver` goes
|
||||
/// away.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// Like `Future::poll`, this function will panic if it's not called from
|
||||
/// within the context of a task. In other words, this should only ever be
|
||||
/// called from inside another future.
|
||||
///
|
||||
/// If you're calling this function from a context that does not have a
|
||||
/// task, then you can use the `is_canceled` API instead.
|
||||
pub fn poll_cancel(&mut self) -> Poll<(), ()> {
|
||||
self.inner.poll_cancel()
|
||||
}
|
||||
|
||||
/// Tests to see whether this `Sender`'s corresponding `Receiver`
|
||||
/// has gone away.
|
||||
///
|
||||
/// This function can be used to learn about when the `Receiver` (consumer)
|
||||
/// half has gone away and nothing will be able to receive a message sent
|
||||
/// from `send`.
|
||||
///
|
||||
/// Note that this function is intended to *not* be used in the context of a
|
||||
/// future. If you're implementing a future you probably want to call the
|
||||
/// `poll_cancel` function which will block the current task if the
|
||||
/// cancellation hasn't happened yet. This can be useful when working on a
|
||||
/// non-futures related thread, though, which would otherwise panic if
|
||||
/// `poll_cancel` were called.
|
||||
pub fn is_canceled(&self) -> bool {
|
||||
self.inner.is_canceled()
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Drop for Sender<T> {
|
||||
fn drop(&mut self) {
|
||||
self.inner.drop_tx()
|
||||
}
|
||||
}
|
||||
|
||||
/// Error returned from a `Receiver<T>` whenever the corresponding `Sender<T>`
|
||||
/// is dropped.
|
||||
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
|
||||
pub struct Canceled;
|
||||
|
||||
impl fmt::Display for Canceled {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
|
||||
write!(fmt, "oneshot canceled")
|
||||
}
|
||||
}
|
||||
|
||||
impl Error for Canceled {
|
||||
fn description(&self) -> &str {
|
||||
"oneshot canceled"
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Receiver<T> {
|
||||
/// Gracefully close this receiver, preventing sending any future messages.
|
||||
///
|
||||
/// Any `send` operation which happens after this method returns is
|
||||
/// guaranteed to fail. Once this method is called the normal `poll` method
|
||||
/// can be used to determine whether a message was actually sent or not. If
|
||||
/// `Canceled` is returned from `poll` then no message was sent.
|
||||
pub fn close(&mut self) {
|
||||
self.inner.close_rx()
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Future for Receiver<T> {
|
||||
type Item = T;
|
||||
type Error = Canceled;
|
||||
|
||||
fn poll(&mut self) -> Poll<T, Canceled> {
|
||||
self.inner.recv()
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Drop for Receiver<T> {
|
||||
fn drop(&mut self) {
|
||||
self.inner.drop_rx()
|
||||
}
|
||||
}
|
@ -1,22 +0,0 @@
|
||||
extern crate tokio_channel;
|
||||
extern crate futures;
|
||||
|
||||
|
||||
use tokio_channel::mpsc::*;
|
||||
use futures::prelude::*;
|
||||
use std::thread;
|
||||
|
||||
#[test]
|
||||
fn smoke() {
|
||||
let (mut sender, receiver) = channel(1);
|
||||
|
||||
let t = thread::spawn(move ||{
|
||||
while let Ok(s) = sender.send(42).wait() {
|
||||
sender = s;
|
||||
}
|
||||
});
|
||||
|
||||
receiver.take(3).for_each(|_| Ok(())).wait().unwrap();
|
||||
|
||||
t.join().unwrap()
|
||||
}
|
@ -1,481 +0,0 @@
|
||||
extern crate tokio_channel;
|
||||
#[macro_use]
|
||||
extern crate futures;
|
||||
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
use tokio_channel::mpsc;
|
||||
use tokio_channel::oneshot;
|
||||
|
||||
use futures::prelude::*;
|
||||
use futures::future::lazy;
|
||||
|
||||
use std::thread;
|
||||
use std::sync::{Arc, Mutex};
|
||||
use std::sync::atomic::{AtomicUsize, Ordering};
|
||||
|
||||
trait AssertSend: Send {}
|
||||
impl AssertSend for mpsc::Sender<i32> {}
|
||||
impl AssertSend for mpsc::Receiver<i32> {}
|
||||
|
||||
#[test]
|
||||
fn send_recv() {
|
||||
let (tx, rx) = mpsc::channel::<i32>(16);
|
||||
let mut rx = rx.wait();
|
||||
|
||||
tx.send(1).wait().unwrap();
|
||||
|
||||
assert_eq!(rx.next().unwrap(), Ok(1));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn send_recv_no_buffer() {
|
||||
let (mut tx, mut rx) = mpsc::channel::<i32>(0);
|
||||
|
||||
// Run on a task context
|
||||
lazy(move || {
|
||||
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());
|
||||
|
||||
Ok::<(), ()>(())
|
||||
}).wait().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn send_shared_recv() {
|
||||
let (tx1, rx) = mpsc::channel::<i32>(16);
|
||||
let tx2 = tx1.clone();
|
||||
let mut rx = rx.wait();
|
||||
|
||||
tx1.send(1).wait().unwrap();
|
||||
assert_eq!(rx.next().unwrap(), Ok(1));
|
||||
|
||||
tx2.send(2).wait().unwrap();
|
||||
assert_eq!(rx.next().unwrap(), Ok(2));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn send_recv_threads() {
|
||||
let (tx, rx) = mpsc::channel::<i32>(16);
|
||||
let mut rx = rx.wait();
|
||||
|
||||
thread::spawn(move|| {
|
||||
tx.send(1).wait().unwrap();
|
||||
});
|
||||
|
||||
assert_eq!(rx.next().unwrap(), Ok(1));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn send_recv_threads_no_capacity() {
|
||||
let (tx, rx) = mpsc::channel::<i32>(0);
|
||||
let mut rx = rx.wait();
|
||||
|
||||
let (readytx, readyrx) = mpsc::channel::<()>(2);
|
||||
let mut readyrx = readyrx.wait();
|
||||
let t = thread::spawn(move|| {
|
||||
let readytx = readytx.sink_map_err(|_| panic!());
|
||||
let (a, b) = tx.send(1).join(readytx.send(())).wait().unwrap();
|
||||
a.send(2).join(b.send(())).wait().unwrap();
|
||||
});
|
||||
|
||||
drop(readyrx.next().unwrap());
|
||||
assert_eq!(rx.next().unwrap(), Ok(1));
|
||||
drop(readyrx.next().unwrap());
|
||||
assert_eq!(rx.next().unwrap(), Ok(2));
|
||||
|
||||
t.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn recv_close_gets_none() {
|
||||
let (mut tx, mut rx) = mpsc::channel::<i32>(10);
|
||||
|
||||
// Run on a task context
|
||||
lazy(move || {
|
||||
rx.close();
|
||||
|
||||
assert_eq!(rx.poll(), Ok(Async::Ready(None)));
|
||||
assert!(tx.poll_ready().is_err());
|
||||
|
||||
drop(tx);
|
||||
|
||||
Ok::<(), ()>(())
|
||||
}).wait().unwrap();
|
||||
}
|
||||
|
||||
|
||||
#[test]
|
||||
fn tx_close_gets_none() {
|
||||
let (_, mut rx) = mpsc::channel::<i32>(10);
|
||||
|
||||
// Run on a task context
|
||||
lazy(move || {
|
||||
assert_eq!(rx.poll(), Ok(Async::Ready(None)));
|
||||
assert_eq!(rx.poll(), Ok(Async::Ready(None)));
|
||||
|
||||
Ok::<(), ()>(())
|
||||
}).wait().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn stress_shared_unbounded() {
|
||||
const AMT: u32 = 10000;
|
||||
const NTHREADS: u32 = 8;
|
||||
let (tx, rx) = mpsc::unbounded::<i32>();
|
||||
let mut rx = rx.wait();
|
||||
|
||||
let t = thread::spawn(move|| {
|
||||
for _ in 0..AMT * NTHREADS {
|
||||
assert_eq!(rx.next().unwrap(), Ok(1));
|
||||
}
|
||||
|
||||
if rx.next().is_some() {
|
||||
panic!();
|
||||
}
|
||||
});
|
||||
|
||||
for _ in 0..NTHREADS {
|
||||
let tx = tx.clone();
|
||||
|
||||
thread::spawn(move|| {
|
||||
for _ in 0..AMT {
|
||||
tx.unbounded_send(1).unwrap();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
drop(tx);
|
||||
|
||||
t.join().ok().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn stress_shared_bounded_hard() {
|
||||
const AMT: u32 = 10000;
|
||||
const NTHREADS: u32 = 8;
|
||||
let (tx, rx) = mpsc::channel::<i32>(0);
|
||||
let mut rx = rx.wait();
|
||||
|
||||
let t = thread::spawn(move|| {
|
||||
for _ in 0..AMT * NTHREADS {
|
||||
assert_eq!(rx.next().unwrap(), Ok(1));
|
||||
}
|
||||
|
||||
if rx.next().is_some() {
|
||||
panic!();
|
||||
}
|
||||
});
|
||||
|
||||
for _ in 0..NTHREADS {
|
||||
let mut tx = tx.clone();
|
||||
|
||||
thread::spawn(move|| {
|
||||
for _ in 0..AMT {
|
||||
tx = tx.send(1).wait().unwrap();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
drop(tx);
|
||||
|
||||
t.join().ok().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn stress_receiver_multi_task_bounded_hard() {
|
||||
const AMT: usize = 10_000;
|
||||
const NTHREADS: u32 = 2;
|
||||
|
||||
let (mut tx, rx) = mpsc::channel::<usize>(0);
|
||||
let rx = Arc::new(Mutex::new(Some(rx)));
|
||||
let n = Arc::new(AtomicUsize::new(0));
|
||||
|
||||
let mut th = vec![];
|
||||
|
||||
for _ in 0..NTHREADS {
|
||||
let rx = rx.clone();
|
||||
let n = n.clone();
|
||||
|
||||
let t = thread::spawn(move || {
|
||||
let mut i = 0;
|
||||
|
||||
loop {
|
||||
i += 1;
|
||||
let mut lock = rx.lock().ok().unwrap();
|
||||
|
||||
match lock.take() {
|
||||
Some(mut rx) => {
|
||||
if i % 5 == 0 {
|
||||
let (item, rest) = rx.into_future().wait().ok().unwrap();
|
||||
|
||||
if item.is_none() {
|
||||
break;
|
||||
}
|
||||
|
||||
n.fetch_add(1, Ordering::Relaxed);
|
||||
*lock = Some(rest);
|
||||
} else {
|
||||
// Just poll
|
||||
let n = n.clone();
|
||||
let r = lazy(move || {
|
||||
let r = match rx.poll().unwrap() {
|
||||
Async::Ready(Some(_)) => {
|
||||
n.fetch_add(1, Ordering::Relaxed);
|
||||
*lock = Some(rx);
|
||||
false
|
||||
}
|
||||
Async::Ready(None) => {
|
||||
true
|
||||
}
|
||||
Async::NotReady => {
|
||||
*lock = Some(rx);
|
||||
false
|
||||
}
|
||||
};
|
||||
|
||||
Ok::<bool, ()>(r)
|
||||
}).wait().unwrap();
|
||||
|
||||
if r {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
None => break,
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
th.push(t);
|
||||
}
|
||||
|
||||
for i in 0..AMT {
|
||||
tx = tx.send(i).wait().unwrap();
|
||||
}
|
||||
|
||||
drop(tx);
|
||||
|
||||
for t in th {
|
||||
t.join().unwrap();
|
||||
}
|
||||
|
||||
assert_eq!(AMT, n.load(Ordering::Relaxed));
|
||||
}
|
||||
|
||||
/// Stress test that receiver properly receives all the messages
|
||||
/// after sender dropped.
|
||||
#[test]
|
||||
fn stress_drop_sender() {
|
||||
fn list() -> Box<Stream<Item=i32, Error=u32>> {
|
||||
let (tx, rx) = mpsc::channel(1);
|
||||
tx.send(Ok(1))
|
||||
.and_then(|tx| tx.send(Ok(2)))
|
||||
.and_then(|tx| tx.send(Ok(3)))
|
||||
.forget();
|
||||
Box::new(rx.then(|r| r.unwrap()))
|
||||
}
|
||||
|
||||
for _ in 0..10000 {
|
||||
assert_eq!(list().wait().collect::<Result<Vec<_>, _>>(),
|
||||
Ok(vec![1, 2, 3]));
|
||||
}
|
||||
}
|
||||
|
||||
/// Stress test that after receiver dropped,
|
||||
/// no messages are lost.
|
||||
fn stress_close_receiver_iter() {
|
||||
let (tx, rx) = mpsc::unbounded();
|
||||
let (unwritten_tx, unwritten_rx) = std::sync::mpsc::channel();
|
||||
let th = thread::spawn(move || {
|
||||
for i in 1.. {
|
||||
if let Err(_) = tx.unbounded_send(i) {
|
||||
unwritten_tx.send(i).expect("unwritten_tx");
|
||||
return;
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
let mut rx = rx.wait();
|
||||
|
||||
// Read one message to make sure thread effectively started
|
||||
assert_eq!(Some(Ok(1)), rx.next());
|
||||
|
||||
rx.get_mut().close();
|
||||
|
||||
for i in 2.. {
|
||||
match rx.next() {
|
||||
Some(Ok(r)) => assert!(i == r),
|
||||
Some(Err(_)) => unreachable!(),
|
||||
None => {
|
||||
let unwritten = unwritten_rx.recv().expect("unwritten_rx");
|
||||
assert_eq!(unwritten, i);
|
||||
th.join().unwrap();
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn stress_close_receiver() {
|
||||
for _ in 0..10000 {
|
||||
stress_close_receiver_iter();
|
||||
}
|
||||
}
|
||||
|
||||
/// Tests that after `poll_ready` indicates capacity a channel can always send without waiting.
|
||||
#[test]
|
||||
fn stress_poll_ready() {
|
||||
// A task which checks channel capacity using poll_ready, and pushes items onto the channel when
|
||||
// ready.
|
||||
struct SenderTask {
|
||||
sender: mpsc::Sender<u32>,
|
||||
count: u32,
|
||||
}
|
||||
impl Future for SenderTask {
|
||||
type Item = ();
|
||||
type Error = ();
|
||||
fn poll(&mut self) -> Poll<(), ()> {
|
||||
// In a loop, check if the channel is ready. If so, push an item onto the channel
|
||||
// (asserting that it doesn't attempt to block).
|
||||
while self.count > 0 {
|
||||
try_ready!(self.sender.poll_ready().map_err(|_| ()));
|
||||
assert!(self.sender.start_send(self.count).unwrap().is_ready());
|
||||
self.count -= 1;
|
||||
}
|
||||
Ok(Async::Ready(()))
|
||||
}
|
||||
}
|
||||
|
||||
const AMT: u32 = 1000;
|
||||
const NTHREADS: u32 = 8;
|
||||
|
||||
/// Run a stress test using the specified channel capacity.
|
||||
fn stress(capacity: usize) {
|
||||
let (tx, rx) = mpsc::channel(capacity);
|
||||
let mut threads = Vec::new();
|
||||
for _ in 0..NTHREADS {
|
||||
let sender = tx.clone();
|
||||
threads.push(thread::spawn(move || {
|
||||
SenderTask {
|
||||
sender: sender,
|
||||
count: AMT,
|
||||
}.wait()
|
||||
}));
|
||||
}
|
||||
drop(tx);
|
||||
|
||||
let mut rx = rx.wait();
|
||||
for _ in 0..AMT * NTHREADS {
|
||||
assert!(rx.next().is_some());
|
||||
}
|
||||
|
||||
assert!(rx.next().is_none());
|
||||
|
||||
for thread in threads {
|
||||
thread.join().unwrap().unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
stress(0);
|
||||
stress(1);
|
||||
stress(8);
|
||||
stress(16);
|
||||
}
|
||||
|
||||
fn is_ready<T>(res: &AsyncSink<T>) -> bool {
|
||||
match *res {
|
||||
AsyncSink::Ready => true,
|
||||
_ => false,
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn try_send_1() {
|
||||
const N: usize = 3000;
|
||||
let (mut tx, rx) = mpsc::channel(0);
|
||||
|
||||
let t = thread::spawn(move || {
|
||||
for i in 0..N {
|
||||
loop {
|
||||
if tx.try_send(i).is_ok() {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
for (i, j) in rx.wait().enumerate() {
|
||||
assert_eq!(i, j.unwrap());
|
||||
}
|
||||
t.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn try_send_2() {
|
||||
let (mut tx, rx) = mpsc::channel(0);
|
||||
|
||||
tx.try_send("hello").unwrap();
|
||||
|
||||
let (readytx, readyrx) = oneshot::channel::<()>();
|
||||
|
||||
let th = thread::spawn(|| {
|
||||
lazy(|| {
|
||||
assert!(tx.start_send("fail").unwrap().is_not_ready());
|
||||
Ok::<_, ()>(())
|
||||
}).wait().unwrap();
|
||||
|
||||
drop(readytx);
|
||||
tx.send("goodbye").wait().unwrap();
|
||||
});
|
||||
|
||||
let mut rx = rx.wait();
|
||||
|
||||
drop(readyrx.wait());
|
||||
assert_eq!(rx.next(), Some(Ok("hello")));
|
||||
assert_eq!(rx.next(), Some(Ok("goodbye")));
|
||||
assert!(rx.next().is_none());
|
||||
|
||||
th.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn try_send_fail() {
|
||||
let (mut tx, rx) = mpsc::channel(0);
|
||||
let mut rx = rx.wait();
|
||||
|
||||
tx.try_send("hello").unwrap();
|
||||
|
||||
// This should fail
|
||||
assert!(tx.try_send("fail").is_err());
|
||||
|
||||
assert_eq!(rx.next(), Some(Ok("hello")));
|
||||
|
||||
tx.try_send("goodbye").unwrap();
|
||||
drop(tx);
|
||||
|
||||
assert_eq!(rx.next(), Some(Ok("goodbye")));
|
||||
assert!(rx.next().is_none());
|
||||
}
|
@ -1,124 +0,0 @@
|
||||
extern crate tokio_channel;
|
||||
extern crate futures;
|
||||
|
||||
mod support;
|
||||
use support::*;
|
||||
|
||||
use tokio_channel::oneshot::*;
|
||||
|
||||
use futures::prelude::*;
|
||||
use futures::future::{lazy, ok};
|
||||
|
||||
use std::sync::mpsc;
|
||||
use std::thread;
|
||||
|
||||
#[test]
|
||||
fn smoke_poll() {
|
||||
let (mut tx, rx) = channel::<u32>();
|
||||
|
||||
lazy(|| {
|
||||
assert!(tx.poll_cancel().unwrap().is_not_ready());
|
||||
assert!(tx.poll_cancel().unwrap().is_not_ready());
|
||||
drop(rx);
|
||||
assert!(tx.poll_cancel().unwrap().is_ready());
|
||||
assert!(tx.poll_cancel().unwrap().is_ready());
|
||||
ok::<(), ()>(())
|
||||
}).wait().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn cancel_notifies() {
|
||||
let (tx, rx) = channel::<u32>();
|
||||
let (tx2, rx2) = mpsc::channel();
|
||||
|
||||
WaitForCancel { tx: tx }.then(move |v| tx2.send(v)).forget();
|
||||
drop(rx);
|
||||
rx2.recv().unwrap().unwrap();
|
||||
}
|
||||
|
||||
struct WaitForCancel {
|
||||
tx: Sender<u32>,
|
||||
}
|
||||
|
||||
impl Future for WaitForCancel {
|
||||
type Item = ();
|
||||
type Error = ();
|
||||
|
||||
fn poll(&mut self) -> Poll<(), ()> {
|
||||
self.tx.poll_cancel()
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn cancel_lots() {
|
||||
let (tx, rx) = mpsc::channel::<(Sender<_>, mpsc::Sender<_>)>();
|
||||
let t = thread::spawn(move || {
|
||||
for (tx, tx2) in rx {
|
||||
WaitForCancel { tx: tx }.then(move |v| tx2.send(v)).forget();
|
||||
}
|
||||
|
||||
});
|
||||
|
||||
for _ in 0..20000 {
|
||||
let (otx, orx) = channel::<u32>();
|
||||
let (tx2, rx2) = mpsc::channel();
|
||||
tx.send((otx, tx2)).unwrap();
|
||||
drop(orx);
|
||||
rx2.recv().unwrap().unwrap();
|
||||
}
|
||||
drop(tx);
|
||||
|
||||
t.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn close() {
|
||||
let (mut tx, mut rx) = channel::<u32>();
|
||||
rx.close();
|
||||
assert!(rx.poll().is_err());
|
||||
assert!(tx.poll_cancel().unwrap().is_ready());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn close_wakes() {
|
||||
let (tx, mut rx) = channel::<u32>();
|
||||
let (tx2, rx2) = mpsc::channel();
|
||||
let t = thread::spawn(move || {
|
||||
rx.close();
|
||||
rx2.recv().unwrap();
|
||||
});
|
||||
WaitForCancel { tx: tx }.wait().unwrap();
|
||||
tx2.send(()).unwrap();
|
||||
t.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn is_canceled() {
|
||||
let (tx, rx) = channel::<u32>();
|
||||
assert!(!tx.is_canceled());
|
||||
drop(rx);
|
||||
assert!(tx.is_canceled());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn cancel_sends() {
|
||||
let (tx, rx) = mpsc::channel::<Sender<_>>();
|
||||
let t = thread::spawn(move || {
|
||||
for otx in rx {
|
||||
let _ = otx.send(42);
|
||||
}
|
||||
});
|
||||
|
||||
for _ in 0..20000 {
|
||||
let (otx, mut orx) = channel::<u32>();
|
||||
tx.send(otx).unwrap();
|
||||
|
||||
orx.close();
|
||||
// Not necessary to wrap in a task because the implementation of oneshot
|
||||
// never calls `task::current()` if the channel has been closed already.
|
||||
let _ = orx.poll();
|
||||
}
|
||||
|
||||
drop(tx);
|
||||
t.join().unwrap();
|
||||
}
|
@ -1,16 +0,0 @@
|
||||
use futures::Future;
|
||||
|
||||
pub trait ForgetExt {
|
||||
fn forget(self);
|
||||
}
|
||||
|
||||
impl<F> ForgetExt for F
|
||||
where F: Future + Sized + Send + 'static,
|
||||
F::Item: Send,
|
||||
F::Error: Send
|
||||
{
|
||||
fn forget(self) {
|
||||
use std::thread;
|
||||
thread::spawn(|| self.wait());
|
||||
}
|
||||
}
|
@ -1,8 +1,9 @@
|
||||
[package]
|
||||
name = "tokio-channel"
|
||||
name = "tokio-sync"
|
||||
|
||||
# When releasing to crates.io:
|
||||
# - Update html_root_url.
|
||||
# - Update doc url
|
||||
# - Update CHANGELOG.md.
|
||||
# - Create "v0.1.x" git tag.
|
||||
version = "0.1.0"
|
||||
@ -10,11 +11,17 @@ authors = ["Carl Lerche <me@carllerche.com>"]
|
||||
license = "MIT"
|
||||
repository = "https://github.com/tokio-rs/tokio"
|
||||
homepage = "https://tokio.rs"
|
||||
documentation = "https://docs.rs/tokio-channel/0.1.0"
|
||||
documentation = "https://docs.rs/tokio-sync/0.1.0/tokio_sync"
|
||||
description = """
|
||||
Channels for asynchronous communication using Tokio.
|
||||
Synchronization utilities.
|
||||
"""
|
||||
categories = ["asynchronous"]
|
||||
|
||||
[dependencies]
|
||||
futures = "0.1.23"
|
||||
futures = "0.1.19"
|
||||
|
||||
[dev-dependencies]
|
||||
env_logger = { version = "0.5", default-features = false }
|
||||
tokio = { path = ".." }
|
||||
tokio-mock-task = "0.1.1"
|
||||
loom = { version = "0.1.0", features = ["futures"] }
|
25
tokio-sync/LICENSE
Normal file
25
tokio-sync/LICENSE
Normal file
@ -0,0 +1,25 @@
|
||||
Copyright (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.
|
280
tokio-sync/benches/mpsc.rs
Normal file
280
tokio-sync/benches/mpsc.rs
Normal file
@ -0,0 +1,280 @@
|
||||
#![feature(test)]
|
||||
|
||||
extern crate tokio_sync;
|
||||
extern crate futures;
|
||||
extern crate test;
|
||||
|
||||
mod tokio {
|
||||
use tokio_sync::mpsc::*;
|
||||
use futures::{Async, Stream, Sink};
|
||||
use test::{self, Bencher};
|
||||
use std::thread;
|
||||
|
||||
#[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 unbounded_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 unbounded_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::<Sender<i32>>();
|
||||
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::<i32>(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 {
|
||||
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::<Sender<i32>>();
|
||||
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::<i32>(1);
|
||||
|
||||
for th in &txs {
|
||||
th.send(tx.clone()).unwrap();
|
||||
}
|
||||
|
||||
drop(tx);
|
||||
|
||||
let rx = rx.wait()
|
||||
.take(THREADS * ITERS);
|
||||
|
||||
for v in rx {
|
||||
test::black_box(v);
|
||||
}
|
||||
});
|
||||
|
||||
drop(txs);
|
||||
|
||||
for th in threads {
|
||||
th.join().unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
mod legacy {
|
||||
use futures::{Async, Stream, Sink};
|
||||
use futures::sync::mpsc::*;
|
||||
use test::{self, Bencher};
|
||||
use std::thread;
|
||||
|
||||
#[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 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_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::<Sender<i32>>();
|
||||
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::<i32>(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 {
|
||||
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::<Sender<i32>>();
|
||||
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::<i32>(1);
|
||||
|
||||
for th in &txs {
|
||||
th.send(tx.clone()).unwrap();
|
||||
}
|
||||
|
||||
drop(tx);
|
||||
|
||||
let rx = rx.wait()
|
||||
.take(THREADS * ITERS);
|
||||
|
||||
for v in rx {
|
||||
test::black_box(v);
|
||||
}
|
||||
});
|
||||
|
||||
drop(txs);
|
||||
|
||||
for th in threads {
|
||||
th.join().unwrap();
|
||||
}
|
||||
}
|
||||
}
|
213
tokio-sync/benches/oneshot.rs
Normal file
213
tokio-sync/benches/oneshot.rs
Normal file
@ -0,0 +1,213 @@
|
||||
#![feature(test)]
|
||||
|
||||
extern crate tokio_sync;
|
||||
extern crate futures;
|
||||
extern crate test;
|
||||
|
||||
mod tokio {
|
||||
use futures::{future, Async, Future};
|
||||
use tokio_sync::oneshot;
|
||||
use test::{Bencher};
|
||||
|
||||
#[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<F: Future>(mut f: F) -> Result<F::Item, F::Error> {
|
||||
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::{future, Async, Future};
|
||||
use futures::sync::oneshot;
|
||||
use test::{Bencher};
|
||||
|
||||
#[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<F: Future>(mut f: F) -> Result<F::Item, F::Error> {
|
||||
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();
|
||||
}
|
||||
}
|
29
tokio-sync/src/lib.rs
Normal file
29
tokio-sync/src/lib.rs
Normal file
@ -0,0 +1,29 @@
|
||||
#![doc(html_root_url = "https://docs.rs/tokio-sync/0.1.0")]
|
||||
#![deny(missing_debug_implementations, missing_docs, unreachable_pub)]
|
||||
#![cfg_attr(test, deny(warnings))]
|
||||
|
||||
//! Asynchronous synchronization primitives.
|
||||
//!
|
||||
//! This crate provides primitives for synchronizing asynchronous tasks.
|
||||
|
||||
extern crate futures;
|
||||
|
||||
macro_rules! debug {
|
||||
($($t:tt)*) => {
|
||||
if false {
|
||||
println!($($t)*);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
macro_rules! if_fuzz {
|
||||
($($t:tt)*) => {{
|
||||
if false { $($t)* }
|
||||
}}
|
||||
}
|
||||
|
||||
mod loom;
|
||||
pub mod oneshot;
|
||||
pub mod mpsc;
|
||||
pub mod semaphore;
|
||||
pub mod task;
|
36
tokio-sync/src/loom.rs
Normal file
36
tokio-sync/src/loom.rs
Normal file
@ -0,0 +1,36 @@
|
||||
pub(crate) mod futures {
|
||||
pub(crate) use futures::task;
|
||||
pub(crate) use ::task::AtomicTask;
|
||||
}
|
||||
|
||||
pub(crate) mod sync {
|
||||
pub(crate) use std::sync::atomic;
|
||||
|
||||
use std::cell::UnsafeCell;
|
||||
|
||||
pub(crate) struct CausalCell<T>(UnsafeCell<T>);
|
||||
|
||||
impl<T> CausalCell<T> {
|
||||
pub(crate) fn new(data: T) -> CausalCell<T> {
|
||||
CausalCell(UnsafeCell::new(data))
|
||||
}
|
||||
|
||||
pub(crate) fn with<F, R>(&self, f: F) -> R
|
||||
where
|
||||
F: FnOnce(*const T) -> R,
|
||||
{
|
||||
f(self.0.get())
|
||||
}
|
||||
|
||||
pub(crate) fn with_mut<F, R>(&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();
|
||||
}
|
395
tokio-sync/src/mpsc/block.rs
Normal file
395
tokio-sync/src/mpsc/block.rs
Normal file
@ -0,0 +1,395 @@
|
||||
use loom::{
|
||||
self,
|
||||
sync::CausalCell,
|
||||
sync::atomic::{
|
||||
AtomicPtr,
|
||||
AtomicUsize,
|
||||
},
|
||||
};
|
||||
|
||||
use std::mem::{self, ManuallyDrop};
|
||||
use std::ops;
|
||||
use std::ptr::{self, NonNull};
|
||||
use std::sync::atomic::Ordering::{self, Acquire, Release, AcqRel};
|
||||
|
||||
/// A block in a linked list.
|
||||
///
|
||||
/// Each block in the list can hold up to `BLOCK_CAP` messages.
|
||||
pub(crate) struct Block<T> {
|
||||
/// 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<Block<T>>,
|
||||
|
||||
/// 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<usize>,
|
||||
|
||||
/// 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<T>,
|
||||
}
|
||||
|
||||
pub(crate) enum Read<T> {
|
||||
Value(T),
|
||||
Closed,
|
||||
}
|
||||
|
||||
struct Values<T>([CausalCell<ManuallyDrop<T>>; 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<T> Block<T> {
|
||||
pub(crate) fn new(start_index: usize) -> Block<T> {
|
||||
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<Read<T>> {
|
||||
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<usize> {
|
||||
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<NonNull<Block<T>>> {
|
||||
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<Block<T>>, ordering: Ordering)
|
||||
-> Result<(), NonNull<Block<T>>>
|
||||
{
|
||||
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<Block<T>> {
|
||||
// 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<T> Values<T> {
|
||||
unsafe fn uninitialized() -> Values<T> {
|
||||
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<T> ops::Index<usize> for Values<T> {
|
||||
type Output = CausalCell<ManuallyDrop<T>>;
|
||||
|
||||
fn index(&self, index: usize) -> &Self::Output {
|
||||
self.0.index(index)
|
||||
}
|
||||
}
|
243
tokio-sync/src/mpsc/bounded.rs
Normal file
243
tokio-sync/src/mpsc/bounded.rs
Normal file
@ -0,0 +1,243 @@
|
||||
use 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.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct Sender<T> {
|
||||
chan: chan::Tx<T, Semaphore>,
|
||||
}
|
||||
|
||||
/// Receive values from the associated `Sender`.
|
||||
///
|
||||
/// Instances are created by the [`channel`](fn.channel.html) function.
|
||||
#[derive(Debug)]
|
||||
pub struct Receiver<T> {
|
||||
/// The channel receiver
|
||||
chan: chan::Rx<T, Semaphore>,
|
||||
}
|
||||
|
||||
/// Error returned by the `Sender`.
|
||||
#[derive(Debug)]
|
||||
pub struct SendError(());
|
||||
|
||||
/// Error returned by `Sender::try_send`.
|
||||
#[derive(Debug)]
|
||||
pub struct TrySendError<T> {
|
||||
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<Item = (), Error = ()> + 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<T>(buffer: usize) -> (Sender<T>, Receiver<T>) {
|
||||
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<T> Receiver<T> {
|
||||
pub(crate) fn new(chan: chan::Rx<T, Semaphore>) -> Receiver<T> {
|
||||
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<T> Stream for Receiver<T> {
|
||||
type Item = T;
|
||||
type Error = RecvError;
|
||||
|
||||
fn poll(&mut self) -> Poll<Option<T>, Self::Error> {
|
||||
self.chan.recv()
|
||||
.map_err(|_| RecvError(()))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
impl<T> Sender<T> {
|
||||
pub(crate) fn new(chan: chan::Tx<T, Semaphore>) -> Sender<T> {
|
||||
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<T>> {
|
||||
self.chan.try_send(message)?;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Sink for Sender<T> {
|
||||
type SinkItem = T;
|
||||
type SinkError = SendError;
|
||||
|
||||
fn start_send(&mut self, msg: T) -> StartSend<T, Self::SinkError> {
|
||||
use futures::AsyncSink;
|
||||
use futures::Async::*;
|
||||
|
||||
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<T: fmt::Debug> fmt::Display for TrySendError<T> {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
|
||||
use std::error::Error;
|
||||
write!(fmt, "{}", self.description())
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: fmt::Debug> ::std::error::Error for TrySendError<T> {
|
||||
fn description(&self) -> &str {
|
||||
match self.kind {
|
||||
ErrorKind::Closed => "channel closed",
|
||||
ErrorKind::NoCapacity => "no available capacity",
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> From<(T, chan::TrySendError)> for TrySendError<T> {
|
||||
fn from((value, err): (T, chan::TrySendError)) -> TrySendError<T> {
|
||||
TrySendError {
|
||||
value,
|
||||
kind: match err {
|
||||
chan::TrySendError::Closed => ErrorKind::Closed,
|
||||
chan::TrySendError::NoPermits => ErrorKind::NoCapacity,
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
402
tokio-sync/src/mpsc/chan.rs
Normal file
402
tokio-sync/src/mpsc/chan.rs
Normal file
@ -0,0 +1,402 @@
|
||||
use super::list;
|
||||
use futures::Poll;
|
||||
use futures::task::AtomicTask;
|
||||
|
||||
use std::cell::UnsafeCell;
|
||||
use std::process;
|
||||
use std::sync::Arc;
|
||||
use std::sync::atomic::AtomicUsize;
|
||||
use std::sync::atomic::Ordering::{AcqRel, Relaxed};
|
||||
|
||||
/// Channel sender
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct Tx<T, S: Semaphore> {
|
||||
inner: Arc<Chan<T, S>>,
|
||||
permit: S::Permit,
|
||||
}
|
||||
|
||||
/// Channel receiver
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct Rx<T, S: Semaphore> {
|
||||
inner: Arc<Chan<T, S>>,
|
||||
}
|
||||
|
||||
#[derive(Debug, Eq, PartialEq)]
|
||||
pub(crate) enum TrySendError {
|
||||
Closed,
|
||||
NoPermits,
|
||||
}
|
||||
|
||||
pub(crate) trait Semaphore: Sync {
|
||||
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);
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct Chan<T, S> {
|
||||
/// Handle to the push half of the lock-free list.
|
||||
tx: list::Tx<T>,
|
||||
|
||||
/// 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: UnsafeCell<RxFields<T>>,
|
||||
}
|
||||
|
||||
/// Fields only accessed by `Rx` handle.
|
||||
#[derive(Debug)]
|
||||
struct RxFields<T> {
|
||||
/// Channel receiver. This field is only accessed by the `Receiver` type.
|
||||
list: list::Rx<T>,
|
||||
|
||||
/// `true` if `Rx::close` is called.
|
||||
rx_closed: bool,
|
||||
}
|
||||
|
||||
unsafe impl<T: Send, S: Send> Send for Chan<T, S> {}
|
||||
unsafe impl<T: Send, S: Sync> Sync for Chan<T, S> {}
|
||||
|
||||
pub(crate) fn channel<T, S>(semaphore: S) -> (Tx<T, S>, Rx<T, S>)
|
||||
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: UnsafeCell::new(RxFields {
|
||||
list: rx,
|
||||
rx_closed: false,
|
||||
}),
|
||||
});
|
||||
|
||||
(Tx::new(chan.clone()), Rx::new(chan))
|
||||
}
|
||||
|
||||
// ===== impl Tx =====
|
||||
|
||||
impl<T, S> Tx<T, S>
|
||||
where
|
||||
S: Semaphore,
|
||||
{
|
||||
fn new(chan: Arc<Chan<T, S>>) -> Tx<T, S> {
|
||||
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<T, S> Clone for Tx<T, S>
|
||||
where
|
||||
S: Semaphore,
|
||||
{
|
||||
fn clone(&self) -> Tx<T, S> {
|
||||
// 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<T, S> Drop for Tx<T, S>
|
||||
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<T, S> Rx<T, S>
|
||||
where
|
||||
S: Semaphore,
|
||||
{
|
||||
fn new(chan: Arc<Chan<T, S>>) -> Rx<T, S> {
|
||||
Rx { inner: chan }
|
||||
}
|
||||
|
||||
pub(crate) fn close(&mut self) {
|
||||
let rx_fields = unsafe { &mut *self.inner.rx_fields.get() };
|
||||
|
||||
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<Option<T>, ()> {
|
||||
use super::block::Read::*;
|
||||
use futures::Async::*;
|
||||
|
||||
let rx_fields = unsafe { &mut *self.inner.rx_fields.get() };
|
||||
|
||||
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<T, S> Drop for Rx<T, S>
|
||||
where
|
||||
S: Semaphore,
|
||||
{
|
||||
fn drop(&mut self) {
|
||||
use super::block::Read::Value;
|
||||
|
||||
self.close();
|
||||
|
||||
let rx_fields = unsafe { &mut *self.inner.rx_fields.get() };
|
||||
|
||||
while let Some(Value(_)) = rx_fields.list.pop(&self.inner.tx) {
|
||||
self.inner.semaphore.add_permit();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// ===== impl Chan =====
|
||||
|
||||
impl<T, S> Drop for Chan<T, S> {
|
||||
fn drop(&mut self) {
|
||||
use super::block::Read::Value;
|
||||
|
||||
let rx_fields = unsafe { &mut *self.rx_fields.get() };
|
||||
|
||||
while let Some(Value(_)) = rx_fields.list.pop(&self.tx) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
use semaphore::TryAcquireError;
|
||||
|
||||
impl From<TryAcquireError> 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) {
|
||||
if permit.is_acquired() {
|
||||
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);
|
||||
}
|
||||
}
|
329
tokio-sync/src/mpsc/list.rs
Normal file
329
tokio-sync/src/mpsc/list.rs
Normal file
@ -0,0 +1,329 @@
|
||||
//! A concurrent, lock-free, FIFO list.
|
||||
|
||||
use super::block::{self, Block};
|
||||
|
||||
use loom::{
|
||||
self,
|
||||
sync::atomic::{AtomicUsize, AtomicPtr},
|
||||
};
|
||||
|
||||
use std::fmt;
|
||||
use std::ptr::NonNull;
|
||||
use std::sync::atomic::Ordering::{Acquire, Release, AcqRel, Relaxed};
|
||||
|
||||
/// List queue transmit handle
|
||||
pub(crate) struct Tx<T> {
|
||||
/// Tail in the `Block` mpmc list.
|
||||
block_tail: AtomicPtr<Block<T>>,
|
||||
|
||||
/// 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<T> {
|
||||
/// Pointer to the block being processed
|
||||
head: NonNull<Block<T>>,
|
||||
|
||||
/// Next slot index to process
|
||||
index: usize,
|
||||
|
||||
/// Pointer to the next block pending release
|
||||
free_head: NonNull<Block<T>>,
|
||||
}
|
||||
|
||||
pub(crate) fn channel<T>() -> (Tx<T>, Rx<T>) {
|
||||
// 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<T> Tx<T> {
|
||||
/// 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 `free_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<Block<T>> {
|
||||
// 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<Block<T>>) {
|
||||
// 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 {
|
||||
let _ = Box::from_raw(block.as_ptr());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: fmt::Debug> fmt::Debug for Tx<T> {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
|
||||
use std::sync::atomic::Ordering::Relaxed;
|
||||
|
||||
fmt.debug_struct("Tx")
|
||||
.field("block_tail", &self.block_tail.load(Relaxed))
|
||||
.field("tail_position", &self.tail_position.load(Relaxed))
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Rx<T> {
|
||||
/// Pop the next value off the queue
|
||||
pub(crate) fn pop(&mut self, tx: &Tx<T>) -> Option<block::Read<T>> {
|
||||
// Advance `head`, if needed
|
||||
if !self.try_advancing_head() {
|
||||
debug!("+ !self.try_advancing_head() -> false");
|
||||
return None;
|
||||
}
|
||||
|
||||
self.free_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 free_blocks(&mut self, tx: &Tx<T>) {
|
||||
debug!("+ free_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 `free_blocks` routine trails the `recv`
|
||||
// routine. Any memory accessed by `free_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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: fmt::Debug> fmt::Debug for Rx<T> {
|
||||
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()
|
||||
}
|
||||
}
|
79
tokio-sync/src/mpsc/mod.rs
Normal file
79
tokio-sync/src/mpsc/mod.rs
Normal file
@ -0,0 +1,79 @@
|
||||
//! 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::{
|
||||
SendError,
|
||||
TrySendError,
|
||||
RecvError,
|
||||
};
|
||||
|
||||
pub use super::unbounded::{
|
||||
UnboundedSendError,
|
||||
UnboundedTrySendError,
|
||||
UnboundedRecvError,
|
||||
};
|
||||
}
|
||||
|
||||
/// 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;
|
1
tokio-sync/src/mpsc/rx.rs
Normal file
1
tokio-sync/src/mpsc/rx.rs
Normal file
@ -0,0 +1 @@
|
||||
|
0
tokio-sync/src/mpsc/tx.rs
Normal file
0
tokio-sync/src/mpsc/tx.rs
Normal file
157
tokio-sync/src/mpsc/unbounded.rs
Normal file
157
tokio-sync/src/mpsc/unbounded.rs
Normal file
@ -0,0 +1,157 @@
|
||||
use super::chan;
|
||||
|
||||
use loom::sync::atomic::AtomicUsize;
|
||||
use futures::{Poll, Sink, StartSend, Stream};
|
||||
|
||||
use std::fmt;
|
||||
|
||||
/// Send values to the associated `UnboundedReceiver`.
|
||||
///
|
||||
/// Instances are created by the
|
||||
/// [`unbounded_channel`](fn.unbounded_channel.html) function.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct UnboundedSender<T> {
|
||||
chan: chan::Tx<T, Semaphore>,
|
||||
}
|
||||
|
||||
/// Receive values from the associated `UnboundedSender`.
|
||||
///
|
||||
/// Instances are created by the
|
||||
/// [`unbounded_channel`](fn.unbounded_channel.html) function.
|
||||
#[derive(Debug)]
|
||||
pub struct UnboundedReceiver<T> {
|
||||
/// The channel receiver
|
||||
chan: chan::Rx<T, Semaphore>,
|
||||
}
|
||||
|
||||
/// Error returned by the `UnboundedSender`.
|
||||
#[derive(Debug)]
|
||||
pub struct UnboundedSendError(());
|
||||
|
||||
/// Error returned by `UnboundedSender::try_send`.
|
||||
#[derive(Debug)]
|
||||
pub struct UnboundedTrySendError<T>(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<T>() -> (UnboundedSender<T>, UnboundedReceiver<T>) {
|
||||
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<T> UnboundedReceiver<T> {
|
||||
pub(crate) fn new(chan: chan::Rx<T, Semaphore>) -> UnboundedReceiver<T> {
|
||||
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<T> Stream for UnboundedReceiver<T> {
|
||||
type Item = T;
|
||||
type Error = UnboundedRecvError;
|
||||
|
||||
fn poll(&mut self) -> Poll<Option<T>, Self::Error> {
|
||||
self.chan.recv()
|
||||
.map_err(|_| UnboundedRecvError(()))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
impl<T> UnboundedSender<T> {
|
||||
pub(crate) fn new(chan: chan::Tx<T, Semaphore>) -> UnboundedSender<T> {
|
||||
UnboundedSender { chan }
|
||||
}
|
||||
|
||||
/// Attempts to send a message on this `UnboundedSender` without blocking.
|
||||
pub fn try_send(&mut self, message: T)
|
||||
-> Result<(), UnboundedTrySendError<T>>
|
||||
{
|
||||
self.chan.try_send(message)?;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Sink for UnboundedSender<T> {
|
||||
type SinkItem = T;
|
||||
type SinkError = UnboundedSendError;
|
||||
|
||||
fn start_send(&mut self, msg: T) -> StartSend<T, Self::SinkError> {
|
||||
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<T: fmt::Debug> fmt::Display for UnboundedTrySendError<T> {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
|
||||
use std::error::Error;
|
||||
write!(fmt, "{}", self.description())
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: fmt::Debug> ::std::error::Error for UnboundedTrySendError<T> {
|
||||
fn description(&self) -> &str {
|
||||
"channel closed"
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> From<(T, chan::TrySendError)> for UnboundedTrySendError<T> {
|
||||
fn from((value, err): (T, chan::TrySendError)) -> UnboundedTrySendError<T> {
|
||||
assert_eq!(chan::TrySendError::Closed, err);
|
||||
UnboundedTrySendError(value)
|
||||
}
|
||||
}
|
509
tokio-sync/src/oneshot.rs
Normal file
509
tokio-sync/src/oneshot.rs
Normal file
@ -0,0 +1,509 @@
|
||||
//! A channel for sending a single message between asynchronous tasks.
|
||||
|
||||
use loom::{
|
||||
futures::task::{self, Task},
|
||||
sync::CausalCell,
|
||||
sync::atomic::AtomicUsize,
|
||||
};
|
||||
|
||||
use futures::{Async, Future, Poll};
|
||||
|
||||
use std::fmt;
|
||||
use std::mem::{self, ManuallyDrop};
|
||||
use std::sync::Arc;
|
||||
use std::sync::atomic::Ordering::{self, Acquire, AcqRel};
|
||||
|
||||
/// Sends a value to the associated `Receiver`.
|
||||
///
|
||||
/// Instances are created by the [`channel`](fn.channel.html) function.
|
||||
#[derive(Debug)]
|
||||
pub struct Sender<T> {
|
||||
inner: Option<Arc<Inner<T>>>,
|
||||
}
|
||||
|
||||
/// Receive a value from the associated `Sender`.
|
||||
///
|
||||
/// Instances are created by the [`channel`](fn.channel.html) function.
|
||||
#[derive(Debug)]
|
||||
pub struct Receiver<T> {
|
||||
inner: Option<Arc<Inner<T>>>,
|
||||
}
|
||||
|
||||
pub mod error {
|
||||
//! Oneshot error types
|
||||
|
||||
/// 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) ());
|
||||
}
|
||||
|
||||
use self::error::*;
|
||||
|
||||
struct Inner<T> {
|
||||
/// 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<Option<T>>,
|
||||
|
||||
/// The task to notify when the receiver drops without consuming the value.
|
||||
tx_task: CausalCell<ManuallyDrop<Task>>,
|
||||
|
||||
/// The task to notify when the value is sent.
|
||||
rx_task: CausalCell<ManuallyDrop<Task>>,
|
||||
}
|
||||
|
||||
#[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::<i32>();
|
||||
///
|
||||
/// # 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<T>() -> (Sender<T>, Receiver<T>) {
|
||||
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<T> Sender<T> {
|
||||
/// 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 tx_task = unsafe { inner.tx_task() };
|
||||
|
||||
if !tx_task.will_notify_current() {
|
||||
state = State::unset_tx_task(&inner.state);
|
||||
|
||||
if state.is_closed() {
|
||||
return Ok(Async::Ready(()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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<T> Drop for Sender<T> {
|
||||
fn drop(&mut self) {
|
||||
if let Some(inner) = self.inner.as_ref() {
|
||||
inner.complete();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Receiver<T> {
|
||||
/// 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<T, TryRecvError> {
|
||||
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<T> Drop for Receiver<T> {
|
||||
fn drop(&mut self) {
|
||||
if let Some(inner) = self.inner.as_ref() {
|
||||
inner.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Future for Receiver<T> {
|
||||
type Item = T;
|
||||
type Error = RecvError;
|
||||
|
||||
fn poll(&mut self) -> Poll<T, RecvError> {
|
||||
use futures::Async::{Ready, NotReady};
|
||||
|
||||
// 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<T> Inner<T> {
|
||||
fn complete(&self) -> bool {
|
||||
let prev = State::set_complete(&self.state);
|
||||
|
||||
if prev.is_closed() {
|
||||
return false;
|
||||
}
|
||||
|
||||
if prev.is_rx_task_set() {
|
||||
let rx_task = unsafe { self.rx_task() };
|
||||
rx_task.notify();
|
||||
}
|
||||
|
||||
true
|
||||
}
|
||||
|
||||
fn poll_recv(&self) -> Poll<T, RecvError> {
|
||||
use futures::Async::{Ready, NotReady};
|
||||
|
||||
// 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 rx_task = unsafe { self.rx_task() };
|
||||
|
||||
// Check if the task is still the same
|
||||
if !rx_task.will_notify_current() {
|
||||
// Unset the task
|
||||
state = State::unset_rx_task(&self.state);
|
||||
|
||||
if state.is_complete() {
|
||||
return match unsafe { self.consume_value() } {
|
||||
Some(value) => Ok(Ready(value)),
|
||||
None => Err(RecvError(())),
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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() {
|
||||
let tx_task = unsafe { self.tx_task() };
|
||||
tx_task.notify();
|
||||
}
|
||||
}
|
||||
|
||||
/// Consume the value. This function does not check `state`.
|
||||
unsafe fn consume_value(&self) -> Option<T> {
|
||||
self.value.with_mut(|ptr| {
|
||||
(*ptr).take()
|
||||
})
|
||||
}
|
||||
|
||||
unsafe fn rx_task(&self) -> &Task {
|
||||
&*self.rx_task.with(|ptr| ptr)
|
||||
}
|
||||
|
||||
unsafe fn set_rx_task(&self) {
|
||||
self.rx_task.with_mut(|ptr| {
|
||||
*ptr = ManuallyDrop::new(task::current())
|
||||
});
|
||||
}
|
||||
|
||||
unsafe fn tx_task(&self) -> &Task {
|
||||
&*self.tx_task.with(|ptr| ptr)
|
||||
}
|
||||
|
||||
unsafe fn set_tx_task(&self) {
|
||||
self.tx_task.with_mut(|ptr| {
|
||||
*ptr = ManuallyDrop::new(task::current())
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
unsafe impl<T: Send> Send for Inner<T> {}
|
||||
unsafe impl<T: Send> Sync for Inner<T> {}
|
||||
|
||||
impl<T> Drop for Inner<T> {
|
||||
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<T: fmt::Debug> fmt::Debug for Inner<T> {
|
||||
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()
|
||||
}
|
||||
}
|
1116
tokio-sync/src/semaphore.rs
Normal file
1116
tokio-sync/src/semaphore.rs
Normal file
File diff suppressed because it is too large
Load Diff
286
tokio-sync/src/task/atomic_task.rs
Normal file
286
tokio-sync/src/task/atomic_task.rs
Normal file
@ -0,0 +1,286 @@
|
||||
use ::loom::{
|
||||
futures::task::{self, Task},
|
||||
sync::CausalCell,
|
||||
sync::atomic::AtomicUsize,
|
||||
};
|
||||
|
||||
use std::fmt;
|
||||
use std::sync::atomic::Ordering::{Acquire, Release, AcqRel};
|
||||
|
||||
/// 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<Option<Task>>,
|
||||
}
|
||||
|
||||
// `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.register_task(task::current());
|
||||
}
|
||||
|
||||
/// 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) {
|
||||
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| *t = Some(task));
|
||||
|
||||
// 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
|
||||
task.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");
|
||||
// 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 notifying");
|
||||
|
||||
if let Some(task) = task {
|
||||
task.notify();
|
||||
}
|
||||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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 {}
|
5
tokio-sync/src/task/mod.rs
Normal file
5
tokio-sync/src/task/mod.rs
Normal file
@ -0,0 +1,5 @@
|
||||
//! Thread-safe task notification primitives.
|
||||
|
||||
mod atomic_task;
|
||||
|
||||
pub use self::atomic_task::AtomicTask;
|
14
tokio-sync/tests/atomic_task.rs
Normal file
14
tokio-sync/tests/atomic_task.rs
Normal file
@ -0,0 +1,14 @@
|
||||
extern crate futures;
|
||||
extern crate tokio_sync;
|
||||
|
||||
use futures::task::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 {}
|
54
tokio-sync/tests/fuzz_atomic_task.rs
Normal file
54
tokio-sync/tests/fuzz_atomic_task.rs
Normal file
@ -0,0 +1,54 @@
|
||||
extern crate futures;
|
||||
#[macro_use]
|
||||
extern crate loom;
|
||||
|
||||
#[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::Async;
|
||||
use futures::future::poll_fn;
|
||||
|
||||
use std::sync::Arc;
|
||||
use std::sync::atomic::Ordering::Relaxed;
|
||||
|
||||
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();
|
||||
});
|
||||
}
|
71
tokio-sync/tests/fuzz_list.rs
Normal file
71
tokio-sync/tests/fuzz_list.rs
Normal file
@ -0,0 +1,71 @@
|
||||
extern 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();
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
23
tokio-sync/tests/fuzz_oneshot.rs
Normal file
23
tokio-sync/tests/fuzz_oneshot.rs
Normal file
@ -0,0 +1,23 @@
|
||||
extern crate futures;
|
||||
extern crate loom;
|
||||
|
||||
#[path = "../src/oneshot.rs"]
|
||||
#[allow(warnings)]
|
||||
mod oneshot;
|
||||
|
||||
use loom::thread;
|
||||
use loom::futures::block_on;
|
||||
|
||||
#[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);
|
||||
});
|
||||
}
|
134
tokio-sync/tests/fuzz_semaphore.rs
Normal file
134
tokio-sync/tests/fuzz_semaphore.rs
Normal file
@ -0,0 +1,134 @@
|
||||
#[macro_use]
|
||||
extern crate futures;
|
||||
#[macro_use]
|
||||
extern crate loom;
|
||||
|
||||
#[path = "../src/semaphore.rs"]
|
||||
#[allow(warnings)]
|
||||
mod semaphore;
|
||||
|
||||
use semaphore::*;
|
||||
|
||||
use futures::{future, Future, Async, Poll};
|
||||
use loom::thread;
|
||||
use loom::futures::block_on;
|
||||
|
||||
use std::sync::Arc;
|
||||
use std::sync::atomic::AtomicUsize;
|
||||
use std::sync::atomic::Ordering::SeqCst;
|
||||
|
||||
#[test]
|
||||
fn basic_usage() {
|
||||
const NUM: usize = 2;
|
||||
|
||||
struct Actor {
|
||||
waiter: Permit,
|
||||
shared: Arc<Shared>,
|
||||
}
|
||||
|
||||
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 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::<(), ()>(())
|
||||
});
|
||||
}
|
||||
});
|
||||
}
|
288
tokio-sync/tests/mpsc.rs
Normal file
288
tokio-sync/tests/mpsc.rs
Normal file
@ -0,0 +1,288 @@
|
||||
extern crate futures;
|
||||
extern crate tokio_mock_task;
|
||||
extern crate tokio_sync;
|
||||
|
||||
use tokio_sync::mpsc;
|
||||
use tokio_mock_task::*;
|
||||
|
||||
use futures::prelude::*;
|
||||
|
||||
use std::thread;
|
||||
use std::sync::Arc;
|
||||
|
||||
trait AssertSend: Send {}
|
||||
impl AssertSend for mpsc::Sender<i32> {}
|
||||
impl AssertSend for mpsc::Receiver<i32> {}
|
||||
|
||||
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::<i32>(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 send_recv_unbounded() {
|
||||
let (mut tx, mut rx) = mpsc::unbounded_channel::<i32>();
|
||||
|
||||
// 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 send_recv_buffer_limited() {
|
||||
let (mut tx, mut rx) = mpsc::channel::<i32>(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::<i32>(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::<i32>(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::<i32>(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::<i32>(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::<i32>(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<T>(res: &AsyncSink<T>) -> 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").is_err());
|
||||
|
||||
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::<i32>(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 unconsumed_messagers_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));
|
||||
}
|
244
tokio-sync/tests/oneshot.rs
Normal file
244
tokio-sync/tests/oneshot.rs
Normal file
@ -0,0 +1,244 @@
|
||||
extern crate futures;
|
||||
extern crate tokio_mock_task;
|
||||
extern crate tokio_sync;
|
||||
|
||||
use tokio_sync::oneshot;
|
||||
use tokio_mock_task::*;
|
||||
|
||||
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<i32> {}
|
||||
impl AssertSend for oneshot::Receiver<i32> {}
|
||||
|
||||
#[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::<i32>();
|
||||
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::<i32>();
|
||||
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::<i32>();
|
||||
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::<i32>();
|
||||
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::<i32>();
|
||||
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::<i32>();
|
||||
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());
|
||||
});
|
||||
|
||||
task2.enter(|| {
|
||||
assert_not_ready!(rx.poll());
|
||||
});
|
||||
|
||||
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::<i32>();
|
||||
|
||||
let mut task1 = MockTask::new();
|
||||
let mut task2 = MockTask::new();
|
||||
|
||||
task1.enter(|| {
|
||||
assert_not_ready!(tx.poll_close());
|
||||
});
|
||||
|
||||
task2.enter(|| {
|
||||
assert_not_ready!(tx.poll_close());
|
||||
});
|
||||
|
||||
drop(rx);
|
||||
|
||||
assert!(!task1.is_notified());
|
||||
assert!(task2.is_notified());
|
||||
|
||||
assert_ready!(tx.poll_close());
|
||||
}
|
174
tokio-sync/tests/semaphore.rs
Normal file
174
tokio-sync/tests/semaphore.rs
Normal file
@ -0,0 +1,174 @@
|
||||
extern crate futures;
|
||||
extern crate tokio_mock_task;
|
||||
extern crate tokio_sync;
|
||||
|
||||
use tokio_sync::semaphore::{Semaphore, Permit};
|
||||
use tokio_mock_task::*;
|
||||
|
||||
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());
|
||||
}
|
Loading…
x
Reference in New Issue
Block a user