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:
Carl Lerche 2019-01-22 11:37:26 -08:00 committed by GitHub
parent 91f20e33a4
commit 13083153aa
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
43 changed files with 5145 additions and 2384 deletions

View File

@ -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

View File

@ -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

View File

@ -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 }

View File

@ -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
View 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,
};

View File

@ -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.

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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
}

View File

@ -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;
}
}
}
}

View File

@ -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()
}
}

View File

@ -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()
}

View File

@ -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());
}

View File

@ -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();
}

View File

@ -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());
}
}

View File

@ -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
View 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
View 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();
}
}
}

View 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
View 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
View 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();
}

View 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)
}
}

View 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
View 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
View 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()
}
}

View 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;

View File

@ -0,0 +1 @@

View File

View 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
View 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

File diff suppressed because it is too large Load Diff

View 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 {}

View File

@ -0,0 +1,5 @@
//! Thread-safe task notification primitives.
mod atomic_task;
pub use self::atomic_task::AtomicTask;

View 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 {}

View 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();
});
}

View 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();
}
}
}
});
}

View 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);
});
}

View 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
View 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
View 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());
}

View 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());
}