rollup merge of #20273: alexcrichton/second-pass-comm
Conflicts: src/doc/guide.md src/libcollections/bit.rs src/libcollections/btree/node.rs src/libcollections/slice.rs src/libcore/ops.rs src/libcore/prelude.rs src/librand/rand_impls.rs src/librustc/middle/check_match.rs src/librustc/middle/infer/region_inference/mod.rs src/librustc_driver/lib.rs src/librustdoc/test.rs src/libstd/bitflags.rs src/libstd/io/comm_adapters.rs src/libstd/io/mem.rs src/libstd/io/mod.rs src/libstd/io/net/pipe.rs src/libstd/io/net/tcp.rs src/libstd/io/net/udp.rs src/libstd/io/pipe.rs src/libstd/io/process.rs src/libstd/io/stdio.rs src/libstd/io/timer.rs src/libstd/io/util.rs src/libstd/macros.rs src/libstd/os.rs src/libstd/path/posix.rs src/libstd/path/windows.rs src/libstd/prelude/v1.rs src/libstd/rand/mod.rs src/libstd/rand/os.rs src/libstd/sync/barrier.rs src/libstd/sync/condvar.rs src/libstd/sync/future.rs src/libstd/sync/mpsc/mod.rs src/libstd/sync/mpsc/mpsc_queue.rs src/libstd/sync/mpsc/select.rs src/libstd/sync/mpsc/spsc_queue.rs src/libstd/sync/mutex.rs src/libstd/sync/once.rs src/libstd/sync/rwlock.rs src/libstd/sync/semaphore.rs src/libstd/sync/task_pool.rs src/libstd/sys/common/helper_thread.rs src/libstd/sys/unix/process.rs src/libstd/sys/unix/timer.rs src/libstd/sys/windows/c.rs src/libstd/sys/windows/timer.rs src/libstd/sys/windows/tty.rs src/libstd/thread.rs src/libstd/thread_local/mod.rs src/libstd/thread_local/scoped.rs src/libtest/lib.rs src/test/auxiliary/cci_capture_clause.rs src/test/bench/shootout-reverse-complement.rs src/test/bench/shootout-spectralnorm.rs src/test/compile-fail/array-old-syntax-2.rs src/test/compile-fail/bind-by-move-no-guards.rs src/test/compile-fail/builtin-superkinds-self-type.rs src/test/compile-fail/comm-not-freeze-receiver.rs src/test/compile-fail/comm-not-freeze.rs src/test/compile-fail/issue-12041.rs src/test/compile-fail/unsendable-class.rs src/test/run-pass/builtin-superkinds-capabilities-transitive.rs src/test/run-pass/builtin-superkinds-capabilities-xc.rs src/test/run-pass/builtin-superkinds-capabilities.rs src/test/run-pass/builtin-superkinds-self-type.rs src/test/run-pass/capturing-logging.rs src/test/run-pass/closure-bounds-can-capture-chan.rs src/test/run-pass/comm.rs src/test/run-pass/core-run-destroy.rs src/test/run-pass/drop-trait-enum.rs src/test/run-pass/hashmap-memory.rs src/test/run-pass/issue-13494.rs src/test/run-pass/issue-3609.rs src/test/run-pass/issue-4446.rs src/test/run-pass/issue-4448.rs src/test/run-pass/issue-8827.rs src/test/run-pass/issue-9396.rs src/test/run-pass/ivec-tag.rs src/test/run-pass/rust-log-filter.rs src/test/run-pass/send-resource.rs src/test/run-pass/send-type-inference.rs src/test/run-pass/sendable-class.rs src/test/run-pass/spawn-types.rs src/test/run-pass/task-comm-0.rs src/test/run-pass/task-comm-10.rs src/test/run-pass/task-comm-11.rs src/test/run-pass/task-comm-13.rs src/test/run-pass/task-comm-14.rs src/test/run-pass/task-comm-15.rs src/test/run-pass/task-comm-16.rs src/test/run-pass/task-comm-3.rs src/test/run-pass/task-comm-4.rs src/test/run-pass/task-comm-5.rs src/test/run-pass/task-comm-6.rs src/test/run-pass/task-comm-7.rs src/test/run-pass/task-comm-9.rs src/test/run-pass/task-comm-chan-nil.rs src/test/run-pass/task-spawn-move-and-copy.rs src/test/run-pass/task-stderr.rs src/test/run-pass/tcp-accept-stress.rs src/test/run-pass/tcp-connect-timeouts.rs src/test/run-pass/tempfile.rs src/test/run-pass/trait-bounds-in-arc.rs src/test/run-pass/trivial-message.rs src/test/run-pass/unique-send-2.rs src/test/run-pass/unique-send.rs src/test/run-pass/unwind-resource.rs
This commit is contained in:
@@ -92,7 +92,7 @@ mod tests {
|
||||
use prelude::v1::*;
|
||||
|
||||
use sync::{Arc, Barrier};
|
||||
use comm::{channel, Empty};
|
||||
use sync::mpsc::{channel, TryRecvError};
|
||||
use thread::Thread;
|
||||
|
||||
#[test]
|
||||
@@ -105,21 +105,21 @@ mod tests {
|
||||
let tx = tx.clone();
|
||||
Thread::spawn(move|| {
|
||||
c.wait();
|
||||
tx.send(true);
|
||||
tx.send(true).unwrap();
|
||||
}).detach();
|
||||
}
|
||||
|
||||
// At this point, all spawned tasks should be blocked,
|
||||
// so we shouldn't get anything from the port
|
||||
assert!(match rx.try_recv() {
|
||||
Err(Empty) => true,
|
||||
Err(TryRecvError::Empty) => true,
|
||||
_ => false,
|
||||
});
|
||||
|
||||
barrier.wait();
|
||||
// Now, the barrier is cleared and we should get data.
|
||||
for _ in range(0u, 9) {
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -281,8 +281,8 @@ impl StaticCondvar {
|
||||
mod tests {
|
||||
use prelude::v1::*;
|
||||
|
||||
use comm::channel;
|
||||
use super::{StaticCondvar, CONDVAR_INIT};
|
||||
use sync::mpsc::channel;
|
||||
use sync::{StaticMutex, MUTEX_INIT, Condvar, Mutex, Arc};
|
||||
use thread::Thread;
|
||||
use time::Duration;
|
||||
@@ -331,25 +331,25 @@ mod tests {
|
||||
let mut cnt = lock.lock().unwrap();
|
||||
*cnt += 1;
|
||||
if *cnt == N {
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
}
|
||||
while *cnt != 0 {
|
||||
cnt = cond.wait(cnt).unwrap();
|
||||
}
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
}).detach();
|
||||
}
|
||||
drop(tx);
|
||||
|
||||
let &(ref lock, ref cond) = &*data;
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
let mut cnt = lock.lock().unwrap();
|
||||
*cnt = 0;
|
||||
cond.notify_all();
|
||||
drop(cnt);
|
||||
|
||||
for _ in range(0, N) {
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -28,7 +28,7 @@ use core::prelude::*;
|
||||
use core::mem::replace;
|
||||
|
||||
use self::FutureState::*;
|
||||
use comm::{Receiver, channel};
|
||||
use sync::mpsc::{Receiver, channel};
|
||||
use thunk::{Thunk};
|
||||
use thread::Thread;
|
||||
|
||||
@@ -122,8 +122,8 @@ impl<A:Send> Future<A> {
|
||||
* waiting for the result to be received on the port.
|
||||
*/
|
||||
|
||||
Future::from_fn(move|:| {
|
||||
rx.recv()
|
||||
Future::from_fn(move |:| {
|
||||
rx.recv().unwrap()
|
||||
})
|
||||
}
|
||||
|
||||
@@ -141,7 +141,7 @@ impl<A:Send> Future<A> {
|
||||
|
||||
Thread::spawn(move |:| {
|
||||
// Don't panic if the other end has hung up
|
||||
let _ = tx.send_opt(blk());
|
||||
let _ = tx.send(blk());
|
||||
}).detach();
|
||||
|
||||
Future::from_receiver(rx)
|
||||
@@ -151,7 +151,7 @@ impl<A:Send> Future<A> {
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use prelude::v1::*;
|
||||
use comm::channel;
|
||||
use sync::mpsc::channel;
|
||||
use sync::Future;
|
||||
use thread::Thread;
|
||||
|
||||
@@ -164,7 +164,7 @@ mod test {
|
||||
#[test]
|
||||
fn test_from_receiver() {
|
||||
let (tx, rx) = channel();
|
||||
tx.send("whale".to_string());
|
||||
tx.send("whale".to_string()).unwrap();
|
||||
let mut f = Future::from_receiver(rx);
|
||||
assert_eq!(f.get(), "whale");
|
||||
}
|
||||
@@ -184,7 +184,7 @@ mod test {
|
||||
#[test]
|
||||
fn test_interface_unwrap() {
|
||||
let f = Future::from_value("fail".to_string());
|
||||
assert_eq!(f.unwrap(), "fail");
|
||||
assert_eq!(f.into_inner(), "fail");
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -213,8 +213,8 @@ mod test {
|
||||
let f = Future::spawn(move|| { expected });
|
||||
let _t = Thread::spawn(move|| {
|
||||
let mut f = f;
|
||||
tx.send(f.get());
|
||||
tx.send(f.get()).unwrap();
|
||||
});
|
||||
assert_eq!(rx.recv(), expected);
|
||||
assert_eq!(rx.recv().unwrap(), expected);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -33,6 +33,8 @@ pub use self::future::Future;
|
||||
pub use self::task_pool::TaskPool;
|
||||
|
||||
pub mod atomic;
|
||||
pub mod mpsc;
|
||||
|
||||
mod barrier;
|
||||
mod condvar;
|
||||
mod future;
|
||||
|
||||
87
src/libstd/sync/mpsc/blocking.rs
Normal file
87
src/libstd/sync/mpsc/blocking.rs
Normal file
@@ -0,0 +1,87 @@
|
||||
// Copyright 2014 The Rust Project Developers. See the COPYRIGHT
|
||||
// file at the top-level directory of this distribution and at
|
||||
// http://rust-lang.org/COPYRIGHT.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
|
||||
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
|
||||
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
|
||||
// option. This file may not be copied, modified, or distributed
|
||||
// except according to those terms.
|
||||
|
||||
//! Generic support for building blocking abstractions.
|
||||
|
||||
use thread::Thread;
|
||||
use sync::atomic::{AtomicBool, INIT_ATOMIC_BOOL, Ordering};
|
||||
use sync::Arc;
|
||||
use kinds::{Sync, Send};
|
||||
use kinds::marker::{NoSend, NoSync};
|
||||
use mem;
|
||||
use clone::Clone;
|
||||
|
||||
struct Inner {
|
||||
thread: Thread,
|
||||
woken: AtomicBool,
|
||||
}
|
||||
|
||||
unsafe impl Send for Inner {}
|
||||
unsafe impl Sync for Inner {}
|
||||
|
||||
#[deriving(Clone)]
|
||||
pub struct SignalToken {
|
||||
inner: Arc<Inner>,
|
||||
}
|
||||
|
||||
pub struct WaitToken {
|
||||
inner: Arc<Inner>,
|
||||
no_send: NoSend,
|
||||
no_sync: NoSync,
|
||||
}
|
||||
|
||||
pub fn tokens() -> (WaitToken, SignalToken) {
|
||||
let inner = Arc::new(Inner {
|
||||
thread: Thread::current(),
|
||||
woken: INIT_ATOMIC_BOOL,
|
||||
});
|
||||
let wait_token = WaitToken {
|
||||
inner: inner.clone(),
|
||||
no_send: NoSend,
|
||||
no_sync: NoSync,
|
||||
};
|
||||
let signal_token = SignalToken {
|
||||
inner: inner
|
||||
};
|
||||
(wait_token, signal_token)
|
||||
}
|
||||
|
||||
impl SignalToken {
|
||||
pub fn signal(&self) -> bool {
|
||||
let wake = !self.inner.woken.compare_and_swap(false, true, Ordering::SeqCst);
|
||||
if wake {
|
||||
self.inner.thread.unpark();
|
||||
}
|
||||
wake
|
||||
}
|
||||
|
||||
/// Convert to an unsafe uint value. Useful for storing in a pipe's state
|
||||
/// flag.
|
||||
#[inline]
|
||||
pub unsafe fn cast_to_uint(self) -> uint {
|
||||
mem::transmute(self.inner)
|
||||
}
|
||||
|
||||
/// Convert from an unsafe uint value. Useful for retrieving a pipe's state
|
||||
/// flag.
|
||||
#[inline]
|
||||
pub unsafe fn cast_from_uint(signal_ptr: uint) -> SignalToken {
|
||||
SignalToken { inner: mem::transmute(signal_ptr) }
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
impl WaitToken {
|
||||
pub fn wait(self) {
|
||||
while !self.inner.woken.load(Ordering::SeqCst) {
|
||||
Thread::park()
|
||||
}
|
||||
}
|
||||
}
|
||||
2079
src/libstd/sync/mpsc/mod.rs
Normal file
2079
src/libstd/sync/mpsc/mod.rs
Normal file
File diff suppressed because it is too large
Load Diff
205
src/libstd/sync/mpsc/mpsc_queue.rs
Normal file
205
src/libstd/sync/mpsc/mpsc_queue.rs
Normal file
@@ -0,0 +1,205 @@
|
||||
/* 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 tasks, 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.
|
||||
|
||||
#![experimental]
|
||||
|
||||
// http://www.1024cores.net/home/lock-free-algorithms
|
||||
// /queues/non-intrusive-mpsc-node-based-queue
|
||||
|
||||
pub use self::PopResult::*;
|
||||
|
||||
use core::prelude::*;
|
||||
|
||||
use alloc::boxed::Box;
|
||||
use core::mem;
|
||||
use core::cell::UnsafeCell;
|
||||
|
||||
use sync::atomic::{AtomicPtr, Release, Acquire, AcqRel, Relaxed};
|
||||
|
||||
/// 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,
|
||||
}
|
||||
|
||||
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).
|
||||
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> {
|
||||
mem::transmute(box Node {
|
||||
next: AtomicPtr::new(0 as *mut Node<T>),
|
||||
value: v,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Send> 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, AcqRel);
|
||||
(*prev).next.store(n, 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 pre-empted 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.
|
||||
pub fn pop(&self) -> PopResult<T> {
|
||||
unsafe {
|
||||
let tail = *self.tail.get();
|
||||
let next = (*tail).next.load(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();
|
||||
let _: Box<Node<T>> = mem::transmute(tail);
|
||||
return Data(ret);
|
||||
}
|
||||
|
||||
if self.head.load(Acquire) == tail {Empty} else {Inconsistent}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[unsafe_destructor]
|
||||
impl<T: Send> Drop for Queue<T> {
|
||||
fn drop(&mut self) {
|
||||
unsafe {
|
||||
let mut cur = *self.tail.get();
|
||||
while !cur.is_null() {
|
||||
let next = (*cur).next.load(Relaxed);
|
||||
let _: Box<Node<T>> = mem::transmute(cur);
|
||||
cur = next;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use prelude::v1::*;
|
||||
|
||||
use sync::mpsc::channel;
|
||||
use super::{Queue, Data, Empty, Inconsistent};
|
||||
use sync::Arc;
|
||||
use thread::Thread;
|
||||
|
||||
#[test]
|
||||
fn test_full() {
|
||||
let q = Queue::new();
|
||||
q.push(box 1i);
|
||||
q.push(box 2i);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test() {
|
||||
let nthreads = 8u;
|
||||
let nmsgs = 1000u;
|
||||
let q = Queue::new();
|
||||
match q.pop() {
|
||||
Empty => {}
|
||||
Inconsistent | Data(..) => panic!()
|
||||
}
|
||||
let (tx, rx) = channel();
|
||||
let q = Arc::new(q);
|
||||
|
||||
for _ in range(0, nthreads) {
|
||||
let tx = tx.clone();
|
||||
let q = q.clone();
|
||||
Thread::spawn(move|| {
|
||||
for i in range(0, nmsgs) {
|
||||
q.push(i);
|
||||
}
|
||||
tx.send(()).unwrap();
|
||||
}).detach();
|
||||
}
|
||||
|
||||
let mut i = 0u;
|
||||
while i < nthreads * nmsgs {
|
||||
match q.pop() {
|
||||
Empty | Inconsistent => {},
|
||||
Data(_) => { i += 1 }
|
||||
}
|
||||
}
|
||||
drop(tx);
|
||||
for _ in range(0, nthreads) {
|
||||
rx.recv().unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
375
src/libstd/sync/mpsc/oneshot.rs
Normal file
375
src/libstd/sync/mpsc/oneshot.rs
Normal file
@@ -0,0 +1,375 @@
|
||||
// Copyright 2014 The Rust Project Developers. See the COPYRIGHT
|
||||
// file at the top-level directory of this distribution and at
|
||||
// http://rust-lang.org/COPYRIGHT.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
|
||||
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
|
||||
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
|
||||
// option. This file may not be copied, modified, or distributed
|
||||
// except according to those terms.
|
||||
|
||||
/// Oneshot channels/ports
|
||||
///
|
||||
/// This is the initial flavor of channels/ports used for comm module. This is
|
||||
/// an optimization for the one-use case of a channel. The major optimization of
|
||||
/// this type is to have one and exactly one allocation when the chan/port pair
|
||||
/// is created.
|
||||
///
|
||||
/// Another possible optimization would be to not use an Arc box because
|
||||
/// in theory we know when the shared packet can be deallocated (no real need
|
||||
/// for the atomic reference counting), but I was having trouble how to destroy
|
||||
/// the data early in a drop of a Port.
|
||||
///
|
||||
/// # Implementation
|
||||
///
|
||||
/// Oneshots are implemented around one atomic uint variable. This variable
|
||||
/// indicates both the state of the port/chan but also contains any tasks
|
||||
/// blocked on the port. All atomic operations happen on this one word.
|
||||
///
|
||||
/// In order to upgrade a oneshot channel, an upgrade is considered a disconnect
|
||||
/// on behalf of the channel side of things (it can be mentally thought of as
|
||||
/// consuming the port). This upgrade is then also stored in the shared packet.
|
||||
/// The one caveat to consider is that when a port sees a disconnected channel
|
||||
/// it must check for data because there is no "data plus upgrade" state.
|
||||
|
||||
pub use self::Failure::*;
|
||||
pub use self::UpgradeResult::*;
|
||||
pub use self::SelectionResult::*;
|
||||
use self::MyUpgrade::*;
|
||||
|
||||
use core::prelude::*;
|
||||
|
||||
use sync::mpsc::Receiver;
|
||||
use sync::mpsc::blocking::{mod, SignalToken};
|
||||
use core::mem;
|
||||
use sync::atomic;
|
||||
|
||||
// Various states you can find a port in.
|
||||
const EMPTY: uint = 0; // initial state: no data, no blocked reciever
|
||||
const DATA: uint = 1; // data ready for receiver to take
|
||||
const DISCONNECTED: uint = 2; // channel is disconnected OR upgraded
|
||||
// Any other value represents a pointer to a SignalToken value. The
|
||||
// protocol ensures that when the state moves *to* a pointer,
|
||||
// ownership of the token is given to the packet, and when the state
|
||||
// moves *from* a pointer, ownership of the token is transferred to
|
||||
// whoever changed the state.
|
||||
|
||||
pub struct Packet<T> {
|
||||
// Internal state of the chan/port pair (stores the blocked task as well)
|
||||
state: atomic::AtomicUint,
|
||||
// One-shot data slot location
|
||||
data: Option<T>,
|
||||
// when used for the second time, a oneshot channel must be upgraded, and
|
||||
// this contains the slot for the upgrade
|
||||
upgrade: MyUpgrade<T>,
|
||||
}
|
||||
|
||||
pub enum Failure<T> {
|
||||
Empty,
|
||||
Disconnected,
|
||||
Upgraded(Receiver<T>),
|
||||
}
|
||||
|
||||
pub enum UpgradeResult {
|
||||
UpSuccess,
|
||||
UpDisconnected,
|
||||
UpWoke(SignalToken),
|
||||
}
|
||||
|
||||
pub enum SelectionResult<T> {
|
||||
SelCanceled,
|
||||
SelUpgraded(SignalToken, Receiver<T>),
|
||||
SelSuccess,
|
||||
}
|
||||
|
||||
enum MyUpgrade<T> {
|
||||
NothingSent,
|
||||
SendUsed,
|
||||
GoUp(Receiver<T>),
|
||||
}
|
||||
|
||||
impl<T: Send> Packet<T> {
|
||||
pub fn new() -> Packet<T> {
|
||||
Packet {
|
||||
data: None,
|
||||
upgrade: NothingSent,
|
||||
state: atomic::AtomicUint::new(EMPTY),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn send(&mut self, t: T) -> Result<(), T> {
|
||||
// Sanity check
|
||||
match self.upgrade {
|
||||
NothingSent => {}
|
||||
_ => panic!("sending on a oneshot that's already sent on "),
|
||||
}
|
||||
assert!(self.data.is_none());
|
||||
self.data = Some(t);
|
||||
self.upgrade = SendUsed;
|
||||
|
||||
match self.state.swap(DATA, atomic::SeqCst) {
|
||||
// Sent the data, no one was waiting
|
||||
EMPTY => Ok(()),
|
||||
|
||||
// Couldn't send the data, the port hung up first. Return the data
|
||||
// back up the stack.
|
||||
DISCONNECTED => {
|
||||
Err(self.data.take().unwrap())
|
||||
}
|
||||
|
||||
// Not possible, these are one-use channels
|
||||
DATA => unreachable!(),
|
||||
|
||||
// There is a thread waiting on the other end. We leave the 'DATA'
|
||||
// state inside so it'll pick it up on the other end.
|
||||
ptr => unsafe {
|
||||
SignalToken::cast_from_uint(ptr).signal();
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Just tests whether this channel has been sent on or not, this is only
|
||||
// safe to use from the sender.
|
||||
pub fn sent(&self) -> bool {
|
||||
match self.upgrade {
|
||||
NothingSent => false,
|
||||
_ => true,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn recv(&mut self) -> Result<T, Failure<T>> {
|
||||
// Attempt to not block the task (it's a little expensive). If it looks
|
||||
// like we're not empty, then immediately go through to `try_recv`.
|
||||
if self.state.load(atomic::SeqCst) == EMPTY {
|
||||
let (wait_token, signal_token) = blocking::tokens();
|
||||
let ptr = unsafe { signal_token.cast_to_uint() };
|
||||
|
||||
// race with senders to enter the blocking state
|
||||
if self.state.compare_and_swap(EMPTY, ptr, atomic::SeqCst) == EMPTY {
|
||||
wait_token.wait();
|
||||
debug_assert!(self.state.load(atomic::SeqCst) != EMPTY);
|
||||
} else {
|
||||
// drop the signal token, since we never blocked
|
||||
drop(unsafe { SignalToken::cast_from_uint(ptr) });
|
||||
}
|
||||
}
|
||||
|
||||
self.try_recv()
|
||||
}
|
||||
|
||||
pub fn try_recv(&mut self) -> Result<T, Failure<T>> {
|
||||
match self.state.load(atomic::SeqCst) {
|
||||
EMPTY => Err(Empty),
|
||||
|
||||
// We saw some data on the channel, but the channel can be used
|
||||
// again to send us an upgrade. As a result, we need to re-insert
|
||||
// into the channel that there's no data available (otherwise we'll
|
||||
// just see DATA next time). This is done as a cmpxchg because if
|
||||
// the state changes under our feet we'd rather just see that state
|
||||
// change.
|
||||
DATA => {
|
||||
self.state.compare_and_swap(DATA, EMPTY, atomic::SeqCst);
|
||||
match self.data.take() {
|
||||
Some(data) => Ok(data),
|
||||
None => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
// There's no guarantee that we receive before an upgrade happens,
|
||||
// and an upgrade flags the channel as disconnected, so when we see
|
||||
// this we first need to check if there's data available and *then*
|
||||
// we go through and process the upgrade.
|
||||
DISCONNECTED => {
|
||||
match self.data.take() {
|
||||
Some(data) => Ok(data),
|
||||
None => {
|
||||
match mem::replace(&mut self.upgrade, SendUsed) {
|
||||
SendUsed | NothingSent => Err(Disconnected),
|
||||
GoUp(upgrade) => Err(Upgraded(upgrade))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// We are the sole receiver; there cannot be a blocking
|
||||
// receiver already.
|
||||
_ => unreachable!()
|
||||
}
|
||||
}
|
||||
|
||||
// Returns whether the upgrade was completed. If the upgrade wasn't
|
||||
// completed, then the port couldn't get sent to the other half (it will
|
||||
// never receive it).
|
||||
pub fn upgrade(&mut self, up: Receiver<T>) -> UpgradeResult {
|
||||
let prev = match self.upgrade {
|
||||
NothingSent => NothingSent,
|
||||
SendUsed => SendUsed,
|
||||
_ => panic!("upgrading again"),
|
||||
};
|
||||
self.upgrade = GoUp(up);
|
||||
|
||||
match self.state.swap(DISCONNECTED, atomic::SeqCst) {
|
||||
// If the channel is empty or has data on it, then we're good to go.
|
||||
// Senders will check the data before the upgrade (in case we
|
||||
// plastered over the DATA state).
|
||||
DATA | EMPTY => UpSuccess,
|
||||
|
||||
// If the other end is already disconnected, then we failed the
|
||||
// upgrade. Be sure to trash the port we were given.
|
||||
DISCONNECTED => { self.upgrade = prev; UpDisconnected }
|
||||
|
||||
// If someone's waiting, we gotta wake them up
|
||||
ptr => UpWoke(unsafe { SignalToken::cast_from_uint(ptr) })
|
||||
}
|
||||
}
|
||||
|
||||
pub fn drop_chan(&mut self) {
|
||||
match self.state.swap(DISCONNECTED, atomic::SeqCst) {
|
||||
DATA | DISCONNECTED | EMPTY => {}
|
||||
|
||||
// If someone's waiting, we gotta wake them up
|
||||
ptr => unsafe {
|
||||
SignalToken::cast_from_uint(ptr).signal();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn drop_port(&mut self) {
|
||||
match self.state.swap(DISCONNECTED, atomic::SeqCst) {
|
||||
// An empty channel has nothing to do, and a remotely disconnected
|
||||
// channel also has nothing to do b/c we're about to run the drop
|
||||
// glue
|
||||
DISCONNECTED | EMPTY => {}
|
||||
|
||||
// There's data on the channel, so make sure we destroy it promptly.
|
||||
// This is why not using an arc is a little difficult (need the box
|
||||
// to stay valid while we take the data).
|
||||
DATA => { self.data.take().unwrap(); }
|
||||
|
||||
// We're the only ones that can block on this port
|
||||
_ => unreachable!()
|
||||
}
|
||||
}
|
||||
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
// select implementation
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
// If Ok, the value is whether this port has data, if Err, then the upgraded
|
||||
// port needs to be checked instead of this one.
|
||||
pub fn can_recv(&mut self) -> Result<bool, Receiver<T>> {
|
||||
match self.state.load(atomic::SeqCst) {
|
||||
EMPTY => Ok(false), // Welp, we tried
|
||||
DATA => Ok(true), // we have some un-acquired data
|
||||
DISCONNECTED if self.data.is_some() => Ok(true), // we have data
|
||||
DISCONNECTED => {
|
||||
match mem::replace(&mut self.upgrade, SendUsed) {
|
||||
// The other end sent us an upgrade, so we need to
|
||||
// propagate upwards whether the upgrade can receive
|
||||
// data
|
||||
GoUp(upgrade) => Err(upgrade),
|
||||
|
||||
// If the other end disconnected without sending an
|
||||
// upgrade, then we have data to receive (the channel is
|
||||
// disconnected).
|
||||
up => { self.upgrade = up; Ok(true) }
|
||||
}
|
||||
}
|
||||
_ => unreachable!(), // we're the "one blocker"
|
||||
}
|
||||
}
|
||||
|
||||
// Attempts to start selection on this port. This can either succeed, fail
|
||||
// because there is data, or fail because there is an upgrade pending.
|
||||
pub fn start_selection(&mut self, token: SignalToken) -> SelectionResult<T> {
|
||||
let ptr = unsafe { token.cast_to_uint() };
|
||||
match self.state.compare_and_swap(EMPTY, ptr, atomic::SeqCst) {
|
||||
EMPTY => SelSuccess,
|
||||
DATA => {
|
||||
drop(unsafe { SignalToken::cast_from_uint(ptr) });
|
||||
SelCanceled
|
||||
}
|
||||
DISCONNECTED if self.data.is_some() => {
|
||||
drop(unsafe { SignalToken::cast_from_uint(ptr) });
|
||||
SelCanceled
|
||||
}
|
||||
DISCONNECTED => {
|
||||
match mem::replace(&mut self.upgrade, SendUsed) {
|
||||
// The other end sent us an upgrade, so we need to
|
||||
// propagate upwards whether the upgrade can receive
|
||||
// data
|
||||
GoUp(upgrade) => {
|
||||
SelUpgraded(unsafe { SignalToken::cast_from_uint(ptr) }, upgrade)
|
||||
}
|
||||
|
||||
// If the other end disconnected without sending an
|
||||
// upgrade, then we have data to receive (the channel is
|
||||
// disconnected).
|
||||
up => {
|
||||
self.upgrade = up;
|
||||
drop(unsafe { SignalToken::cast_from_uint(ptr) });
|
||||
SelCanceled
|
||||
}
|
||||
}
|
||||
}
|
||||
_ => unreachable!(), // we're the "one blocker"
|
||||
}
|
||||
}
|
||||
|
||||
// Remove a previous selecting task from this port. This ensures that the
|
||||
// blocked task will no longer be visible to any other threads.
|
||||
//
|
||||
// The return value indicates whether there's data on this port.
|
||||
pub fn abort_selection(&mut self) -> Result<bool, Receiver<T>> {
|
||||
let state = match self.state.load(atomic::SeqCst) {
|
||||
// Each of these states means that no further activity will happen
|
||||
// with regard to abortion selection
|
||||
s @ EMPTY |
|
||||
s @ DATA |
|
||||
s @ DISCONNECTED => s,
|
||||
|
||||
// If we've got a blocked task, then use an atomic to gain ownership
|
||||
// of it (may fail)
|
||||
ptr => self.state.compare_and_swap(ptr, EMPTY, atomic::SeqCst)
|
||||
};
|
||||
|
||||
// Now that we've got ownership of our state, figure out what to do
|
||||
// about it.
|
||||
match state {
|
||||
EMPTY => unreachable!(),
|
||||
// our task used for select was stolen
|
||||
DATA => Ok(true),
|
||||
|
||||
// If the other end has hung up, then we have complete ownership
|
||||
// of the port. First, check if there was data waiting for us. This
|
||||
// is possible if the other end sent something and then hung up.
|
||||
//
|
||||
// We then need to check to see if there was an upgrade requested,
|
||||
// and if so, the upgraded port needs to have its selection aborted.
|
||||
DISCONNECTED => {
|
||||
if self.data.is_some() {
|
||||
Ok(true)
|
||||
} else {
|
||||
match mem::replace(&mut self.upgrade, SendUsed) {
|
||||
GoUp(port) => Err(port),
|
||||
_ => Ok(true),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// We woke ourselves up from select.
|
||||
ptr => unsafe {
|
||||
drop(SignalToken::cast_from_uint(ptr));
|
||||
Ok(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[unsafe_destructor]
|
||||
impl<T: Send> Drop for Packet<T> {
|
||||
fn drop(&mut self) {
|
||||
assert_eq!(self.state.load(atomic::SeqCst), DISCONNECTED);
|
||||
}
|
||||
}
|
||||
747
src/libstd/sync/mpsc/select.rs
Normal file
747
src/libstd/sync/mpsc/select.rs
Normal file
@@ -0,0 +1,747 @@
|
||||
// Copyright 2013-2014 The Rust Project Developers. See the COPYRIGHT
|
||||
// file at the top-level directory of this distribution and at
|
||||
// http://rust-lang.org/COPYRIGHT.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
|
||||
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
|
||||
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
|
||||
// option. This file may not be copied, modified, or distributed
|
||||
// except according to those terms.
|
||||
|
||||
//! Selection over an array of receivers
|
||||
//!
|
||||
//! This module contains the implementation machinery necessary for selecting
|
||||
//! over a number of receivers. One large goal of this module is to provide an
|
||||
//! efficient interface to selecting over any receiver of any type.
|
||||
//!
|
||||
//! This is achieved through an architecture of a "receiver set" in which
|
||||
//! receivers are added to a set and then the entire set is waited on at once.
|
||||
//! The set can be waited on multiple times to prevent re-adding each receiver
|
||||
//! to the set.
|
||||
//!
|
||||
//! Usage of this module is currently encouraged to go through the use of the
|
||||
//! `select!` macro. This macro allows naturally binding of variables to the
|
||||
//! received values of receivers in a much more natural syntax then usage of the
|
||||
//! `Select` structure directly.
|
||||
//!
|
||||
//! # Example
|
||||
//!
|
||||
//! ```rust
|
||||
//! use std::sync::mpsc::channel;
|
||||
//!
|
||||
//! let (tx1, rx1) = channel();
|
||||
//! let (tx2, rx2) = channel();
|
||||
//!
|
||||
//! tx1.send(1i).unwrap();
|
||||
//! tx2.send(2i).unwrap();
|
||||
//!
|
||||
//! select! {
|
||||
//! val = rx1.recv() => {
|
||||
//! assert_eq!(val.unwrap(), 1i);
|
||||
//! },
|
||||
//! val = rx2.recv() => {
|
||||
//! assert_eq!(val.unwrap(), 2i);
|
||||
//! }
|
||||
//! }
|
||||
//! ```
|
||||
|
||||
#![allow(dead_code)]
|
||||
#![experimental = "This implementation, while likely sufficient, is unsafe and \
|
||||
likely to be error prone. At some point in the future this \
|
||||
module will likely be replaced, and it is currently \
|
||||
unknown how much API breakage that will cause. The ability \
|
||||
to select over a number of channels will remain forever, \
|
||||
but no guarantees beyond this are being made"]
|
||||
|
||||
|
||||
use core::prelude::*;
|
||||
|
||||
use core::cell::Cell;
|
||||
use core::kinds::marker;
|
||||
use core::mem;
|
||||
use core::uint;
|
||||
|
||||
use sync::mpsc::{Receiver, RecvError};
|
||||
use sync::mpsc::blocking::{mod, SignalToken};
|
||||
|
||||
/// The "receiver set" of the select interface. This structure is used to manage
|
||||
/// a set of receivers which are being selected over.
|
||||
pub struct Select {
|
||||
head: *mut Handle<'static, ()>,
|
||||
tail: *mut Handle<'static, ()>,
|
||||
next_id: Cell<uint>,
|
||||
marker1: marker::NoSend,
|
||||
}
|
||||
|
||||
/// A handle to a receiver which is currently a member of a `Select` set of
|
||||
/// receivers. This handle is used to keep the receiver in the set as well as
|
||||
/// interact with the underlying receiver.
|
||||
pub struct Handle<'rx, T:'rx> {
|
||||
/// The ID of this handle, used to compare against the return value of
|
||||
/// `Select::wait()`
|
||||
id: uint,
|
||||
selector: &'rx Select,
|
||||
next: *mut Handle<'static, ()>,
|
||||
prev: *mut Handle<'static, ()>,
|
||||
added: bool,
|
||||
packet: &'rx (Packet+'rx),
|
||||
|
||||
// due to our fun transmutes, we be sure to place this at the end. (nothing
|
||||
// previous relies on T)
|
||||
rx: &'rx Receiver<T>,
|
||||
}
|
||||
|
||||
struct Packets { cur: *mut Handle<'static, ()> }
|
||||
|
||||
#[doc(hidden)]
|
||||
#[deriving(PartialEq)]
|
||||
pub enum StartResult {
|
||||
Installed,
|
||||
Abort,
|
||||
}
|
||||
|
||||
#[doc(hidden)]
|
||||
pub trait Packet {
|
||||
fn can_recv(&self) -> bool;
|
||||
fn start_selection(&self, token: SignalToken) -> StartResult;
|
||||
fn abort_selection(&self) -> bool;
|
||||
}
|
||||
|
||||
impl Select {
|
||||
/// Creates a new selection structure. This set is initially empty and
|
||||
/// `wait` will panic!() if called.
|
||||
///
|
||||
/// Usage of this struct directly can sometimes be burdensome, and usage is
|
||||
/// rather much easier through the `select!` macro.
|
||||
pub fn new() -> Select {
|
||||
Select {
|
||||
marker1: marker::NoSend,
|
||||
head: 0 as *mut Handle<'static, ()>,
|
||||
tail: 0 as *mut Handle<'static, ()>,
|
||||
next_id: Cell::new(1),
|
||||
}
|
||||
}
|
||||
|
||||
/// Creates a new handle into this receiver set for a new receiver. Note
|
||||
/// that this does *not* add the receiver to the receiver set, for that you
|
||||
/// must call the `add` method on the handle itself.
|
||||
pub fn handle<'a, T: Send>(&'a self, rx: &'a Receiver<T>) -> Handle<'a, T> {
|
||||
let id = self.next_id.get();
|
||||
self.next_id.set(id + 1);
|
||||
Handle {
|
||||
id: id,
|
||||
selector: self,
|
||||
next: 0 as *mut Handle<'static, ()>,
|
||||
prev: 0 as *mut Handle<'static, ()>,
|
||||
added: false,
|
||||
rx: rx,
|
||||
packet: rx,
|
||||
}
|
||||
}
|
||||
|
||||
/// Waits for an event on this receiver set. The returned value is *not* an
|
||||
/// index, but rather an id. This id can be queried against any active
|
||||
/// `Handle` structures (each one has an `id` method). The handle with
|
||||
/// the matching `id` will have some sort of event available on it. The
|
||||
/// event could either be that data is available or the corresponding
|
||||
/// channel has been closed.
|
||||
pub fn wait(&self) -> uint {
|
||||
self.wait2(true)
|
||||
}
|
||||
|
||||
/// Helper method for skipping the preflight checks during testing
|
||||
fn wait2(&self, do_preflight_checks: bool) -> uint {
|
||||
// Note that this is currently an inefficient implementation. We in
|
||||
// theory have knowledge about all receivers in the set ahead of time,
|
||||
// so this method shouldn't really have to iterate over all of them yet
|
||||
// again. The idea with this "receiver set" interface is to get the
|
||||
// interface right this time around, and later this implementation can
|
||||
// be optimized.
|
||||
//
|
||||
// This implementation can be summarized by:
|
||||
//
|
||||
// fn select(receivers) {
|
||||
// if any receiver ready { return ready index }
|
||||
// deschedule {
|
||||
// block on all receivers
|
||||
// }
|
||||
// unblock on all receivers
|
||||
// return ready index
|
||||
// }
|
||||
//
|
||||
// Most notably, the iterations over all of the receivers shouldn't be
|
||||
// necessary.
|
||||
unsafe {
|
||||
// Stage 1: preflight checks. Look for any packets ready to receive
|
||||
if do_preflight_checks {
|
||||
for handle in self.iter() {
|
||||
if (*handle).packet.can_recv() {
|
||||
return (*handle).id();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Stage 2: begin the blocking process
|
||||
//
|
||||
// Create a number of signal tokens, and install each one
|
||||
// sequentially until one fails. If one fails, then abort the
|
||||
// selection on the already-installed tokens.
|
||||
let (wait_token, signal_token) = blocking::tokens();
|
||||
for (i, handle) in self.iter().enumerate() {
|
||||
match (*handle).packet.start_selection(signal_token.clone()) {
|
||||
StartResult::Installed => {}
|
||||
StartResult::Abort => {
|
||||
// Go back and abort the already-begun selections
|
||||
for handle in self.iter().take(i) {
|
||||
(*handle).packet.abort_selection();
|
||||
}
|
||||
return (*handle).id;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Stage 3: no messages available, actually block
|
||||
wait_token.wait();
|
||||
|
||||
// Stage 4: there *must* be message available; find it.
|
||||
//
|
||||
// Abort the selection process on each receiver. If the abort
|
||||
// process returns `true`, then that means that the receiver is
|
||||
// ready to receive some data. Note that this also means that the
|
||||
// receiver may have yet to have fully read the `to_wake` field and
|
||||
// woken us up (although the wakeup is guaranteed to fail).
|
||||
//
|
||||
// This situation happens in the window of where a sender invokes
|
||||
// increment(), sees -1, and then decides to wake up the task. After
|
||||
// all this is done, the sending thread will set `selecting` to
|
||||
// `false`. Until this is done, we cannot return. If we were to
|
||||
// return, then a sender could wake up a receiver which has gone
|
||||
// back to sleep after this call to `select`.
|
||||
//
|
||||
// Note that it is a "fairly small window" in which an increment()
|
||||
// views that it should wake a thread up until the `selecting` bit
|
||||
// is set to false. For now, the implementation currently just spins
|
||||
// in a yield loop. This is very distasteful, but this
|
||||
// implementation is already nowhere near what it should ideally be.
|
||||
// A rewrite should focus on avoiding a yield loop, and for now this
|
||||
// implementation is tying us over to a more efficient "don't
|
||||
// iterate over everything every time" implementation.
|
||||
let mut ready_id = uint::MAX;
|
||||
for handle in self.iter() {
|
||||
if (*handle).packet.abort_selection() {
|
||||
ready_id = (*handle).id;
|
||||
}
|
||||
}
|
||||
|
||||
// We must have found a ready receiver
|
||||
assert!(ready_id != uint::MAX);
|
||||
return ready_id;
|
||||
}
|
||||
}
|
||||
|
||||
fn iter(&self) -> Packets { Packets { cur: self.head } }
|
||||
}
|
||||
|
||||
impl<'rx, T: Send> Handle<'rx, T> {
|
||||
/// Retrieve the id of this handle.
|
||||
#[inline]
|
||||
pub fn id(&self) -> uint { self.id }
|
||||
|
||||
/// Block to receive a value on the underlying receiver, returning `Some` on
|
||||
/// success or `None` if the channel disconnects. This function has the same
|
||||
/// semantics as `Receiver.recv`
|
||||
pub fn recv(&mut self) -> Result<T, RecvError> { self.rx.recv() }
|
||||
|
||||
/// Adds this handle to the receiver set that the handle was created from. This
|
||||
/// method can be called multiple times, but it has no effect if `add` was
|
||||
/// called previously.
|
||||
///
|
||||
/// This method is unsafe because it requires that the `Handle` is not moved
|
||||
/// while it is added to the `Select` set.
|
||||
pub unsafe fn add(&mut self) {
|
||||
if self.added { return }
|
||||
let selector: &mut Select = mem::transmute(&*self.selector);
|
||||
let me: *mut Handle<'static, ()> = mem::transmute(&*self);
|
||||
|
||||
if selector.head.is_null() {
|
||||
selector.head = me;
|
||||
selector.tail = me;
|
||||
} else {
|
||||
(*me).prev = selector.tail;
|
||||
assert!((*me).next.is_null());
|
||||
(*selector.tail).next = me;
|
||||
selector.tail = me;
|
||||
}
|
||||
self.added = true;
|
||||
}
|
||||
|
||||
/// Removes this handle from the `Select` set. This method is unsafe because
|
||||
/// it has no guarantee that the `Handle` was not moved since `add` was
|
||||
/// called.
|
||||
pub unsafe fn remove(&mut self) {
|
||||
if !self.added { return }
|
||||
|
||||
let selector: &mut Select = mem::transmute(&*self.selector);
|
||||
let me: *mut Handle<'static, ()> = mem::transmute(&*self);
|
||||
|
||||
if self.prev.is_null() {
|
||||
assert_eq!(selector.head, me);
|
||||
selector.head = self.next;
|
||||
} else {
|
||||
(*self.prev).next = self.next;
|
||||
}
|
||||
if self.next.is_null() {
|
||||
assert_eq!(selector.tail, me);
|
||||
selector.tail = self.prev;
|
||||
} else {
|
||||
(*self.next).prev = self.prev;
|
||||
}
|
||||
|
||||
self.next = 0 as *mut Handle<'static, ()>;
|
||||
self.prev = 0 as *mut Handle<'static, ()>;
|
||||
|
||||
self.added = false;
|
||||
}
|
||||
}
|
||||
|
||||
#[unsafe_destructor]
|
||||
impl Drop for Select {
|
||||
fn drop(&mut self) {
|
||||
assert!(self.head.is_null());
|
||||
assert!(self.tail.is_null());
|
||||
}
|
||||
}
|
||||
|
||||
#[unsafe_destructor]
|
||||
impl<'rx, T: Send> Drop for Handle<'rx, T> {
|
||||
fn drop(&mut self) {
|
||||
unsafe { self.remove() }
|
||||
}
|
||||
}
|
||||
|
||||
impl Iterator<*mut Handle<'static, ()>> for Packets {
|
||||
fn next(&mut self) -> Option<*mut Handle<'static, ()>> {
|
||||
if self.cur.is_null() {
|
||||
None
|
||||
} else {
|
||||
let ret = Some(self.cur);
|
||||
unsafe { self.cur = (*self.cur).next; }
|
||||
ret
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
#[allow(unused_imports)]
|
||||
mod test {
|
||||
use prelude::v1::*;
|
||||
|
||||
use thread::Thread;
|
||||
use super::*;
|
||||
use sync::mpsc::*;
|
||||
|
||||
// Don't use the libstd version so we can pull in the right Select structure
|
||||
// (std::comm points at the wrong one)
|
||||
macro_rules! select {
|
||||
(
|
||||
$($name:pat = $rx:ident.$meth:ident() => $code:expr),+
|
||||
) => ({
|
||||
let sel = Select::new();
|
||||
$( let mut $rx = sel.handle(&$rx); )+
|
||||
unsafe {
|
||||
$( $rx.add(); )+
|
||||
}
|
||||
let ret = sel.wait();
|
||||
$( if ret == $rx.id() { let $name = $rx.$meth(); $code } else )+
|
||||
{ unreachable!() }
|
||||
})
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn smoke() {
|
||||
let (tx1, rx1) = channel::<int>();
|
||||
let (tx2, rx2) = channel::<int>();
|
||||
tx1.send(1).unwrap();
|
||||
select! {
|
||||
foo = rx1.recv() => { assert_eq!(foo.unwrap(), 1); },
|
||||
_bar = rx2.recv() => { panic!() }
|
||||
}
|
||||
tx2.send(2).unwrap();
|
||||
select! {
|
||||
_foo = rx1.recv() => { panic!() },
|
||||
bar = rx2.recv() => { assert_eq!(bar.unwrap(), 2) }
|
||||
}
|
||||
drop(tx1);
|
||||
select! {
|
||||
foo = rx1.recv() => { assert!(foo.is_err()); },
|
||||
_bar = rx2.recv() => { panic!() }
|
||||
}
|
||||
drop(tx2);
|
||||
select! {
|
||||
bar = rx2.recv() => { assert!(bar.is_err()); }
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn smoke2() {
|
||||
let (_tx1, rx1) = channel::<int>();
|
||||
let (_tx2, rx2) = channel::<int>();
|
||||
let (_tx3, rx3) = channel::<int>();
|
||||
let (_tx4, rx4) = channel::<int>();
|
||||
let (tx5, rx5) = channel::<int>();
|
||||
tx5.send(4).unwrap();
|
||||
select! {
|
||||
_foo = rx1.recv() => { panic!("1") },
|
||||
_foo = rx2.recv() => { panic!("2") },
|
||||
_foo = rx3.recv() => { panic!("3") },
|
||||
_foo = rx4.recv() => { panic!("4") },
|
||||
foo = rx5.recv() => { assert_eq!(foo.unwrap(), 4); }
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn closed() {
|
||||
let (_tx1, rx1) = channel::<int>();
|
||||
let (tx2, rx2) = channel::<int>();
|
||||
drop(tx2);
|
||||
|
||||
select! {
|
||||
_a1 = rx1.recv() => { panic!() },
|
||||
a2 = rx2.recv() => { assert!(a2.is_err()); }
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn unblocks() {
|
||||
let (tx1, rx1) = channel::<int>();
|
||||
let (_tx2, rx2) = channel::<int>();
|
||||
let (tx3, rx3) = channel::<int>();
|
||||
|
||||
let _t = Thread::spawn(move|| {
|
||||
for _ in range(0u, 20) { Thread::yield_now(); }
|
||||
tx1.send(1).unwrap();
|
||||
rx3.recv().unwrap();
|
||||
for _ in range(0u, 20) { Thread::yield_now(); }
|
||||
});
|
||||
|
||||
select! {
|
||||
a = rx1.recv() => { assert_eq!(a.unwrap(), 1); },
|
||||
_b = rx2.recv() => { panic!() }
|
||||
}
|
||||
tx3.send(1).unwrap();
|
||||
select! {
|
||||
a = rx1.recv() => { assert!(a.is_err()) },
|
||||
_b = rx2.recv() => { panic!() }
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn both_ready() {
|
||||
let (tx1, rx1) = channel::<int>();
|
||||
let (tx2, rx2) = channel::<int>();
|
||||
let (tx3, rx3) = channel::<()>();
|
||||
|
||||
let _t = Thread::spawn(move|| {
|
||||
for _ in range(0u, 20) { Thread::yield_now(); }
|
||||
tx1.send(1).unwrap();
|
||||
tx2.send(2).unwrap();
|
||||
rx3.recv().unwrap();
|
||||
});
|
||||
|
||||
select! {
|
||||
a = rx1.recv() => { assert_eq!(a.unwrap(), 1); },
|
||||
a = rx2.recv() => { assert_eq!(a.unwrap(), 2); }
|
||||
}
|
||||
select! {
|
||||
a = rx1.recv() => { assert_eq!(a.unwrap(), 1); },
|
||||
a = rx2.recv() => { assert_eq!(a.unwrap(), 2); }
|
||||
}
|
||||
assert_eq!(rx1.try_recv(), Err(TryRecvError::Empty));
|
||||
assert_eq!(rx2.try_recv(), Err(TryRecvError::Empty));
|
||||
tx3.send(()).unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn stress() {
|
||||
static AMT: int = 10000;
|
||||
let (tx1, rx1) = channel::<int>();
|
||||
let (tx2, rx2) = channel::<int>();
|
||||
let (tx3, rx3) = channel::<()>();
|
||||
|
||||
let _t = Thread::spawn(move|| {
|
||||
for i in range(0, AMT) {
|
||||
if i % 2 == 0 {
|
||||
tx1.send(i).unwrap();
|
||||
} else {
|
||||
tx2.send(i).unwrap();
|
||||
}
|
||||
rx3.recv().unwrap();
|
||||
}
|
||||
});
|
||||
|
||||
for i in range(0, AMT) {
|
||||
select! {
|
||||
i1 = rx1.recv() => { assert!(i % 2 == 0 && i == i1.unwrap()); },
|
||||
i2 = rx2.recv() => { assert!(i % 2 == 1 && i == i2.unwrap()); }
|
||||
}
|
||||
tx3.send(()).unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn cloning() {
|
||||
let (tx1, rx1) = channel::<int>();
|
||||
let (_tx2, rx2) = channel::<int>();
|
||||
let (tx3, rx3) = channel::<()>();
|
||||
|
||||
let _t = Thread::spawn(move|| {
|
||||
rx3.recv().unwrap();
|
||||
tx1.clone();
|
||||
assert_eq!(rx3.try_recv(), Err(TryRecvError::Empty));
|
||||
tx1.send(2).unwrap();
|
||||
rx3.recv().unwrap();
|
||||
});
|
||||
|
||||
tx3.send(()).unwrap();
|
||||
select! {
|
||||
_i1 = rx1.recv() => {},
|
||||
_i2 = rx2.recv() => panic!()
|
||||
}
|
||||
tx3.send(()).unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn cloning2() {
|
||||
let (tx1, rx1) = channel::<int>();
|
||||
let (_tx2, rx2) = channel::<int>();
|
||||
let (tx3, rx3) = channel::<()>();
|
||||
|
||||
let _t = Thread::spawn(move|| {
|
||||
rx3.recv().unwrap();
|
||||
tx1.clone();
|
||||
assert_eq!(rx3.try_recv(), Err(TryRecvError::Empty));
|
||||
tx1.send(2).unwrap();
|
||||
rx3.recv().unwrap();
|
||||
});
|
||||
|
||||
tx3.send(()).unwrap();
|
||||
select! {
|
||||
_i1 = rx1.recv() => {},
|
||||
_i2 = rx2.recv() => panic!()
|
||||
}
|
||||
tx3.send(()).unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn cloning3() {
|
||||
let (tx1, rx1) = channel::<()>();
|
||||
let (tx2, rx2) = channel::<()>();
|
||||
let (tx3, rx3) = channel::<()>();
|
||||
let _t = Thread::spawn(move|| {
|
||||
let s = Select::new();
|
||||
let mut h1 = s.handle(&rx1);
|
||||
let mut h2 = s.handle(&rx2);
|
||||
unsafe { h2.add(); }
|
||||
unsafe { h1.add(); }
|
||||
assert_eq!(s.wait(), h2.id);
|
||||
tx3.send(()).unwrap();
|
||||
});
|
||||
|
||||
for _ in range(0u, 1000) { Thread::yield_now(); }
|
||||
drop(tx1.clone());
|
||||
tx2.send(()).unwrap();
|
||||
rx3.recv().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight1() {
|
||||
let (tx, rx) = channel();
|
||||
tx.send(()).unwrap();
|
||||
select! {
|
||||
_n = rx.recv() => {}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight2() {
|
||||
let (tx, rx) = channel();
|
||||
tx.send(()).unwrap();
|
||||
tx.send(()).unwrap();
|
||||
select! {
|
||||
_n = rx.recv() => {}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight3() {
|
||||
let (tx, rx) = channel();
|
||||
drop(tx.clone());
|
||||
tx.send(()).unwrap();
|
||||
select! {
|
||||
_n = rx.recv() => {}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight4() {
|
||||
let (tx, rx) = channel();
|
||||
tx.send(()).unwrap();
|
||||
let s = Select::new();
|
||||
let mut h = s.handle(&rx);
|
||||
unsafe { h.add(); }
|
||||
assert_eq!(s.wait2(false), h.id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight5() {
|
||||
let (tx, rx) = channel();
|
||||
tx.send(()).unwrap();
|
||||
tx.send(()).unwrap();
|
||||
let s = Select::new();
|
||||
let mut h = s.handle(&rx);
|
||||
unsafe { h.add(); }
|
||||
assert_eq!(s.wait2(false), h.id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight6() {
|
||||
let (tx, rx) = channel();
|
||||
drop(tx.clone());
|
||||
tx.send(()).unwrap();
|
||||
let s = Select::new();
|
||||
let mut h = s.handle(&rx);
|
||||
unsafe { h.add(); }
|
||||
assert_eq!(s.wait2(false), h.id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight7() {
|
||||
let (tx, rx) = channel::<()>();
|
||||
drop(tx);
|
||||
let s = Select::new();
|
||||
let mut h = s.handle(&rx);
|
||||
unsafe { h.add(); }
|
||||
assert_eq!(s.wait2(false), h.id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight8() {
|
||||
let (tx, rx) = channel();
|
||||
tx.send(()).unwrap();
|
||||
drop(tx);
|
||||
rx.recv().unwrap();
|
||||
let s = Select::new();
|
||||
let mut h = s.handle(&rx);
|
||||
unsafe { h.add(); }
|
||||
assert_eq!(s.wait2(false), h.id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn preflight9() {
|
||||
let (tx, rx) = channel();
|
||||
drop(tx.clone());
|
||||
tx.send(()).unwrap();
|
||||
drop(tx);
|
||||
rx.recv().unwrap();
|
||||
let s = Select::new();
|
||||
let mut h = s.handle(&rx);
|
||||
unsafe { h.add(); }
|
||||
assert_eq!(s.wait2(false), h.id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn oneshot_data_waiting() {
|
||||
let (tx1, rx1) = channel();
|
||||
let (tx2, rx2) = channel();
|
||||
let _t = Thread::spawn(move|| {
|
||||
select! {
|
||||
_n = rx1.recv() => {}
|
||||
}
|
||||
tx2.send(()).unwrap();
|
||||
});
|
||||
|
||||
for _ in range(0u, 100) { Thread::yield_now() }
|
||||
tx1.send(()).unwrap();
|
||||
rx2.recv().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn stream_data_waiting() {
|
||||
let (tx1, rx1) = channel();
|
||||
let (tx2, rx2) = channel();
|
||||
tx1.send(()).unwrap();
|
||||
tx1.send(()).unwrap();
|
||||
rx1.recv().unwrap();
|
||||
rx1.recv().unwrap();
|
||||
let _t = Thread::spawn(move|| {
|
||||
select! {
|
||||
_n = rx1.recv() => {}
|
||||
}
|
||||
tx2.send(()).unwrap();
|
||||
});
|
||||
|
||||
for _ in range(0u, 100) { Thread::yield_now() }
|
||||
tx1.send(()).unwrap();
|
||||
rx2.recv().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn shared_data_waiting() {
|
||||
let (tx1, rx1) = channel();
|
||||
let (tx2, rx2) = channel();
|
||||
drop(tx1.clone());
|
||||
tx1.send(()).unwrap();
|
||||
rx1.recv().unwrap();
|
||||
let _t = Thread::spawn(move|| {
|
||||
select! {
|
||||
_n = rx1.recv() => {}
|
||||
}
|
||||
tx2.send(()).unwrap();
|
||||
});
|
||||
|
||||
for _ in range(0u, 100) { Thread::yield_now() }
|
||||
tx1.send(()).unwrap();
|
||||
rx2.recv().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn sync1() {
|
||||
let (tx, rx) = sync_channel::<int>(1);
|
||||
tx.send(1).unwrap();
|
||||
select! {
|
||||
n = rx.recv() => { assert_eq!(n.unwrap(), 1); }
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn sync2() {
|
||||
let (tx, rx) = sync_channel::<int>(0);
|
||||
let _t = Thread::spawn(move|| {
|
||||
for _ in range(0u, 100) { Thread::yield_now() }
|
||||
tx.send(1).unwrap();
|
||||
});
|
||||
select! {
|
||||
n = rx.recv() => { assert_eq!(n.unwrap(), 1); }
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn sync3() {
|
||||
let (tx1, rx1) = sync_channel::<int>(0);
|
||||
let (tx2, rx2): (Sender<int>, Receiver<int>) = channel();
|
||||
let _t = Thread::spawn(move|| { tx1.send(1).unwrap(); });
|
||||
let _t = Thread::spawn(move|| { tx2.send(2).unwrap(); });
|
||||
select! {
|
||||
n = rx1.recv() => {
|
||||
let n = n.unwrap();
|
||||
assert_eq!(n, 1);
|
||||
assert_eq!(rx2.recv().unwrap(), 2);
|
||||
},
|
||||
n = rx2.recv() => {
|
||||
let n = n.unwrap();
|
||||
assert_eq!(n, 2);
|
||||
assert_eq!(rx1.recv().unwrap(), 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
486
src/libstd/sync/mpsc/shared.rs
Normal file
486
src/libstd/sync/mpsc/shared.rs
Normal file
@@ -0,0 +1,486 @@
|
||||
// Copyright 2014 The Rust Project Developers. See the COPYRIGHT
|
||||
// file at the top-level directory of this distribution and at
|
||||
// http://rust-lang.org/COPYRIGHT.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
|
||||
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
|
||||
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
|
||||
// option. This file may not be copied, modified, or distributed
|
||||
// except according to those terms.
|
||||
|
||||
/// Shared channels
|
||||
///
|
||||
/// This is the flavor of channels which are not necessarily optimized for any
|
||||
/// particular use case, but are the most general in how they are used. Shared
|
||||
/// channels are cloneable allowing for multiple senders.
|
||||
///
|
||||
/// High level implementation details can be found in the comment of the parent
|
||||
/// module. You'll also note that the implementation of the shared and stream
|
||||
/// channels are quite similar, and this is no coincidence!
|
||||
|
||||
pub use self::Failure::*;
|
||||
|
||||
use core::prelude::*;
|
||||
|
||||
use core::cmp;
|
||||
use core::int;
|
||||
|
||||
use sync::{atomic, Mutex, MutexGuard};
|
||||
use sync::mpsc::mpsc_queue as mpsc;
|
||||
use sync::mpsc::blocking::{mod, SignalToken};
|
||||
use sync::mpsc::select::StartResult;
|
||||
use sync::mpsc::select::StartResult::*;
|
||||
use thread::Thread;
|
||||
|
||||
const DISCONNECTED: int = int::MIN;
|
||||
const FUDGE: int = 1024;
|
||||
#[cfg(test)]
|
||||
const MAX_STEALS: int = 5;
|
||||
#[cfg(not(test))]
|
||||
const MAX_STEALS: int = 1 << 20;
|
||||
|
||||
pub struct Packet<T> {
|
||||
queue: mpsc::Queue<T>,
|
||||
cnt: atomic::AtomicInt, // How many items are on this channel
|
||||
steals: int, // How many times has a port received without blocking?
|
||||
to_wake: atomic::AtomicUint, // SignalToken for wake up
|
||||
|
||||
// The number of channels which are currently using this packet.
|
||||
channels: atomic::AtomicInt,
|
||||
|
||||
// See the discussion in Port::drop and the channel send methods for what
|
||||
// these are used for
|
||||
port_dropped: atomic::AtomicBool,
|
||||
sender_drain: atomic::AtomicInt,
|
||||
|
||||
// this lock protects various portions of this implementation during
|
||||
// select()
|
||||
select_lock: Mutex<()>,
|
||||
}
|
||||
|
||||
pub enum Failure {
|
||||
Empty,
|
||||
Disconnected,
|
||||
}
|
||||
|
||||
impl<T: Send> Packet<T> {
|
||||
// Creation of a packet *must* be followed by a call to postinit_lock
|
||||
// and later by inherit_blocker
|
||||
pub fn new() -> Packet<T> {
|
||||
let p = Packet {
|
||||
queue: mpsc::Queue::new(),
|
||||
cnt: atomic::AtomicInt::new(0),
|
||||
steals: 0,
|
||||
to_wake: atomic::AtomicUint::new(0),
|
||||
channels: atomic::AtomicInt::new(2),
|
||||
port_dropped: atomic::AtomicBool::new(false),
|
||||
sender_drain: atomic::AtomicInt::new(0),
|
||||
select_lock: Mutex::new(()),
|
||||
};
|
||||
return p;
|
||||
}
|
||||
|
||||
// This function should be used after newly created Packet
|
||||
// was wrapped with an Arc
|
||||
// In other case mutex data will be duplicated while cloning
|
||||
// and that could cause problems on platforms where it is
|
||||
// represented by opaque data structure
|
||||
pub fn postinit_lock(&self) -> MutexGuard<()> {
|
||||
self.select_lock.lock().unwrap()
|
||||
}
|
||||
|
||||
// This function is used at the creation of a shared packet to inherit a
|
||||
// previously blocked task. This is done to prevent spurious wakeups of
|
||||
// tasks in select().
|
||||
//
|
||||
// This can only be called at channel-creation time
|
||||
pub fn inherit_blocker(&mut self,
|
||||
token: Option<SignalToken>,
|
||||
guard: MutexGuard<()>) {
|
||||
token.map(|token| {
|
||||
assert_eq!(self.cnt.load(atomic::SeqCst), 0);
|
||||
assert_eq!(self.to_wake.load(atomic::SeqCst), 0);
|
||||
self.to_wake.store(unsafe { token.cast_to_uint() }, atomic::SeqCst);
|
||||
self.cnt.store(-1, atomic::SeqCst);
|
||||
|
||||
// This store is a little sketchy. What's happening here is that
|
||||
// we're transferring a blocker from a oneshot or stream channel to
|
||||
// this shared channel. In doing so, we never spuriously wake them
|
||||
// up and rather only wake them up at the appropriate time. This
|
||||
// implementation of shared channels assumes that any blocking
|
||||
// recv() will undo the increment of steals performed in try_recv()
|
||||
// once the recv is complete. This thread that we're inheriting,
|
||||
// however, is not in the middle of recv. Hence, the first time we
|
||||
// wake them up, they're going to wake up from their old port, move
|
||||
// on to the upgraded port, and then call the block recv() function.
|
||||
//
|
||||
// When calling this function, they'll find there's data immediately
|
||||
// available, counting it as a steal. This in fact wasn't a steal
|
||||
// because we appropriately blocked them waiting for data.
|
||||
//
|
||||
// To offset this bad increment, we initially set the steal count to
|
||||
// -1. You'll find some special code in abort_selection() as well to
|
||||
// ensure that this -1 steal count doesn't escape too far.
|
||||
self.steals = -1;
|
||||
});
|
||||
|
||||
// When the shared packet is constructed, we grabbed this lock. The
|
||||
// purpose of this lock is to ensure that abort_selection() doesn't
|
||||
// interfere with this method. After we unlock this lock, we're
|
||||
// signifying that we're done modifying self.cnt and self.to_wake and
|
||||
// the port is ready for the world to continue using it.
|
||||
drop(guard);
|
||||
}
|
||||
|
||||
pub fn send(&mut self, t: T) -> Result<(), T> {
|
||||
// See Port::drop for what's going on
|
||||
if self.port_dropped.load(atomic::SeqCst) { return Err(t) }
|
||||
|
||||
// Note that the multiple sender case is a little trickier
|
||||
// semantically than the single sender case. The logic for
|
||||
// incrementing is "add and if disconnected store disconnected".
|
||||
// This could end up leading some senders to believe that there
|
||||
// wasn't a disconnect if in fact there was a disconnect. This means
|
||||
// that while one thread is attempting to re-store the disconnected
|
||||
// states, other threads could walk through merrily incrementing
|
||||
// this very-negative disconnected count. To prevent senders from
|
||||
// spuriously attempting to send when the channels is actually
|
||||
// disconnected, the count has a ranged check here.
|
||||
//
|
||||
// This is also done for another reason. Remember that the return
|
||||
// value of this function is:
|
||||
//
|
||||
// `true` == the data *may* be received, this essentially has no
|
||||
// meaning
|
||||
// `false` == the data will *never* be received, this has a lot of
|
||||
// meaning
|
||||
//
|
||||
// In the SPSC case, we have a check of 'queue.is_empty()' to see
|
||||
// whether the data was actually received, but this same condition
|
||||
// means nothing in a multi-producer context. As a result, this
|
||||
// preflight check serves as the definitive "this will never be
|
||||
// received". Once we get beyond this check, we have permanently
|
||||
// entered the realm of "this may be received"
|
||||
if self.cnt.load(atomic::SeqCst) < DISCONNECTED + FUDGE {
|
||||
return Err(t)
|
||||
}
|
||||
|
||||
self.queue.push(t);
|
||||
match self.cnt.fetch_add(1, atomic::SeqCst) {
|
||||
-1 => {
|
||||
self.take_to_wake().signal();
|
||||
}
|
||||
|
||||
// In this case, we have possibly failed to send our data, and
|
||||
// we need to consider re-popping the data in order to fully
|
||||
// destroy it. We must arbitrate among the multiple senders,
|
||||
// however, because the queues that we're using are
|
||||
// single-consumer queues. In order to do this, all exiting
|
||||
// pushers will use an atomic count in order to count those
|
||||
// flowing through. Pushers who see 0 are required to drain as
|
||||
// much as possible, and then can only exit when they are the
|
||||
// only pusher (otherwise they must try again).
|
||||
n if n < DISCONNECTED + FUDGE => {
|
||||
// see the comment in 'try' for a shared channel for why this
|
||||
// window of "not disconnected" is ok.
|
||||
self.cnt.store(DISCONNECTED, atomic::SeqCst);
|
||||
|
||||
if self.sender_drain.fetch_add(1, atomic::SeqCst) == 0 {
|
||||
loop {
|
||||
// drain the queue, for info on the thread yield see the
|
||||
// discussion in try_recv
|
||||
loop {
|
||||
match self.queue.pop() {
|
||||
mpsc::Data(..) => {}
|
||||
mpsc::Empty => break,
|
||||
mpsc::Inconsistent => Thread::yield_now(),
|
||||
}
|
||||
}
|
||||
// maybe we're done, if we're not the last ones
|
||||
// here, then we need to go try again.
|
||||
if self.sender_drain.fetch_sub(1, atomic::SeqCst) == 1 {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// At this point, there may still be data on the queue,
|
||||
// but only if the count hasn't been incremented and
|
||||
// some other sender hasn't finished pushing data just
|
||||
// yet. That sender in question will drain its own data.
|
||||
}
|
||||
}
|
||||
|
||||
// Can't make any assumptions about this case like in the SPSC case.
|
||||
_ => {}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn recv(&mut self) -> Result<T, Failure> {
|
||||
// This code is essentially the exact same as that found in the stream
|
||||
// case (see stream.rs)
|
||||
match self.try_recv() {
|
||||
Err(Empty) => {}
|
||||
data => return data,
|
||||
}
|
||||
|
||||
let (wait_token, signal_token) = blocking::tokens();
|
||||
if self.decrement(signal_token) == Installed {
|
||||
wait_token.wait()
|
||||
}
|
||||
|
||||
match self.try_recv() {
|
||||
data @ Ok(..) => { self.steals -= 1; data }
|
||||
data => data,
|
||||
}
|
||||
}
|
||||
|
||||
// Essentially the exact same thing as the stream decrement function.
|
||||
// Returns true if blocking should proceed.
|
||||
fn decrement(&mut self, token: SignalToken) -> StartResult {
|
||||
assert_eq!(self.to_wake.load(atomic::SeqCst), 0);
|
||||
let ptr = unsafe { token.cast_to_uint() };
|
||||
self.to_wake.store(ptr, atomic::SeqCst);
|
||||
|
||||
let steals = self.steals;
|
||||
self.steals = 0;
|
||||
|
||||
match self.cnt.fetch_sub(1 + steals, atomic::SeqCst) {
|
||||
DISCONNECTED => { self.cnt.store(DISCONNECTED, atomic::SeqCst); }
|
||||
// If we factor in our steals and notice that the channel has no
|
||||
// data, we successfully sleep
|
||||
n => {
|
||||
assert!(n >= 0);
|
||||
if n - steals <= 0 { return Installed }
|
||||
}
|
||||
}
|
||||
|
||||
self.to_wake.store(0, atomic::SeqCst);
|
||||
drop(unsafe { SignalToken::cast_from_uint(ptr) });
|
||||
Abort
|
||||
}
|
||||
|
||||
pub fn try_recv(&mut self) -> Result<T, Failure> {
|
||||
let ret = match self.queue.pop() {
|
||||
mpsc::Data(t) => Some(t),
|
||||
mpsc::Empty => None,
|
||||
|
||||
// This is a bit of an interesting case. The channel is reported as
|
||||
// having data available, but our pop() has failed due to the queue
|
||||
// being in an inconsistent state. This means that there is some
|
||||
// pusher somewhere which has yet to complete, but we are guaranteed
|
||||
// that a pop will eventually succeed. In this case, we spin in a
|
||||
// yield loop because the remote sender should finish their enqueue
|
||||
// operation "very quickly".
|
||||
//
|
||||
// Avoiding this yield loop would require a different queue
|
||||
// abstraction which provides the guarantee that after M pushes have
|
||||
// succeeded, at least M pops will succeed. The current queues
|
||||
// guarantee that if there are N active pushes, you can pop N times
|
||||
// once all N have finished.
|
||||
mpsc::Inconsistent => {
|
||||
let data;
|
||||
loop {
|
||||
Thread::yield_now();
|
||||
match self.queue.pop() {
|
||||
mpsc::Data(t) => { data = t; break }
|
||||
mpsc::Empty => panic!("inconsistent => empty"),
|
||||
mpsc::Inconsistent => {}
|
||||
}
|
||||
}
|
||||
Some(data)
|
||||
}
|
||||
};
|
||||
match ret {
|
||||
// See the discussion in the stream implementation for why we
|
||||
// might decrement steals.
|
||||
Some(data) => {
|
||||
if self.steals > MAX_STEALS {
|
||||
match self.cnt.swap(0, atomic::SeqCst) {
|
||||
DISCONNECTED => {
|
||||
self.cnt.store(DISCONNECTED, atomic::SeqCst);
|
||||
}
|
||||
n => {
|
||||
let m = cmp::min(n, self.steals);
|
||||
self.steals -= m;
|
||||
self.bump(n - m);
|
||||
}
|
||||
}
|
||||
assert!(self.steals >= 0);
|
||||
}
|
||||
self.steals += 1;
|
||||
Ok(data)
|
||||
}
|
||||
|
||||
// See the discussion in the stream implementation for why we try
|
||||
// again.
|
||||
None => {
|
||||
match self.cnt.load(atomic::SeqCst) {
|
||||
n if n != DISCONNECTED => Err(Empty),
|
||||
_ => {
|
||||
match self.queue.pop() {
|
||||
mpsc::Data(t) => Ok(t),
|
||||
mpsc::Empty => Err(Disconnected),
|
||||
// with no senders, an inconsistency is impossible.
|
||||
mpsc::Inconsistent => unreachable!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Prepares this shared packet for a channel clone, essentially just bumping
|
||||
// a refcount.
|
||||
pub fn clone_chan(&mut self) {
|
||||
self.channels.fetch_add(1, atomic::SeqCst);
|
||||
}
|
||||
|
||||
// Decrement the reference count on a channel. This is called whenever a
|
||||
// Chan is dropped and may end up waking up a receiver. It's the receiver's
|
||||
// responsibility on the other end to figure out that we've disconnected.
|
||||
pub fn drop_chan(&mut self) {
|
||||
match self.channels.fetch_sub(1, atomic::SeqCst) {
|
||||
1 => {}
|
||||
n if n > 1 => return,
|
||||
n => panic!("bad number of channels left {}", n),
|
||||
}
|
||||
|
||||
match self.cnt.swap(DISCONNECTED, atomic::SeqCst) {
|
||||
-1 => { self.take_to_wake().signal(); }
|
||||
DISCONNECTED => {}
|
||||
n => { assert!(n >= 0); }
|
||||
}
|
||||
}
|
||||
|
||||
// See the long discussion inside of stream.rs for why the queue is drained,
|
||||
// and why it is done in this fashion.
|
||||
pub fn drop_port(&mut self) {
|
||||
self.port_dropped.store(true, atomic::SeqCst);
|
||||
let mut steals = self.steals;
|
||||
while {
|
||||
let cnt = self.cnt.compare_and_swap(steals, DISCONNECTED, atomic::SeqCst);
|
||||
cnt != DISCONNECTED && cnt != steals
|
||||
} {
|
||||
// See the discussion in 'try_recv' for why we yield
|
||||
// control of this thread.
|
||||
loop {
|
||||
match self.queue.pop() {
|
||||
mpsc::Data(..) => { steals += 1; }
|
||||
mpsc::Empty | mpsc::Inconsistent => break,
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Consumes ownership of the 'to_wake' field.
|
||||
fn take_to_wake(&mut self) -> SignalToken {
|
||||
let ptr = self.to_wake.load(atomic::SeqCst);
|
||||
self.to_wake.store(0, atomic::SeqCst);
|
||||
assert!(ptr != 0);
|
||||
unsafe { SignalToken::cast_from_uint(ptr) }
|
||||
}
|
||||
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
// select implementation
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
// Helper function for select, tests whether this port can receive without
|
||||
// blocking (obviously not an atomic decision).
|
||||
//
|
||||
// This is different than the stream version because there's no need to peek
|
||||
// at the queue, we can just look at the local count.
|
||||
pub fn can_recv(&mut self) -> bool {
|
||||
let cnt = self.cnt.load(atomic::SeqCst);
|
||||
cnt == DISCONNECTED || cnt - self.steals > 0
|
||||
}
|
||||
|
||||
// increment the count on the channel (used for selection)
|
||||
fn bump(&mut self, amt: int) -> int {
|
||||
match self.cnt.fetch_add(amt, atomic::SeqCst) {
|
||||
DISCONNECTED => {
|
||||
self.cnt.store(DISCONNECTED, atomic::SeqCst);
|
||||
DISCONNECTED
|
||||
}
|
||||
n => n
|
||||
}
|
||||
}
|
||||
|
||||
// Inserts the signal token for selection on this port, returning true if
|
||||
// blocking should proceed.
|
||||
//
|
||||
// The code here is the same as in stream.rs, except that it doesn't need to
|
||||
// peek at the channel to see if an upgrade is pending.
|
||||
pub fn start_selection(&mut self, token: SignalToken) -> StartResult {
|
||||
match self.decrement(token) {
|
||||
Installed => Installed,
|
||||
Abort => {
|
||||
let prev = self.bump(1);
|
||||
assert!(prev == DISCONNECTED || prev >= 0);
|
||||
Abort
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Cancels a previous task waiting on this port, returning whether there's
|
||||
// data on the port.
|
||||
//
|
||||
// This is similar to the stream implementation (hence fewer comments), but
|
||||
// uses a different value for the "steals" variable.
|
||||
pub fn abort_selection(&mut self, _was_upgrade: bool) -> bool {
|
||||
// Before we do anything else, we bounce on this lock. The reason for
|
||||
// doing this is to ensure that any upgrade-in-progress is gone and
|
||||
// done with. Without this bounce, we can race with inherit_blocker
|
||||
// about looking at and dealing with to_wake. Once we have acquired the
|
||||
// lock, we are guaranteed that inherit_blocker is done.
|
||||
{
|
||||
let _guard = self.select_lock.lock().unwrap();
|
||||
}
|
||||
|
||||
// Like the stream implementation, we want to make sure that the count
|
||||
// on the channel goes non-negative. We don't know how negative the
|
||||
// stream currently is, so instead of using a steal value of 1, we load
|
||||
// the channel count and figure out what we should do to make it
|
||||
// positive.
|
||||
let steals = {
|
||||
let cnt = self.cnt.load(atomic::SeqCst);
|
||||
if cnt < 0 && cnt != DISCONNECTED {-cnt} else {0}
|
||||
};
|
||||
let prev = self.bump(steals + 1);
|
||||
|
||||
if prev == DISCONNECTED {
|
||||
assert_eq!(self.to_wake.load(atomic::SeqCst), 0);
|
||||
true
|
||||
} else {
|
||||
let cur = prev + steals + 1;
|
||||
assert!(cur >= 0);
|
||||
if prev < 0 {
|
||||
drop(self.take_to_wake());
|
||||
} else {
|
||||
while self.to_wake.load(atomic::SeqCst) != 0 {
|
||||
Thread::yield_now();
|
||||
}
|
||||
}
|
||||
// if the number of steals is -1, it was the pre-emptive -1 steal
|
||||
// count from when we inherited a blocker. This is fine because
|
||||
// we're just going to overwrite it with a real value.
|
||||
assert!(self.steals == 0 || self.steals == -1);
|
||||
self.steals = steals;
|
||||
prev >= 0
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[unsafe_destructor]
|
||||
impl<T: Send> Drop for Packet<T> {
|
||||
fn drop(&mut self) {
|
||||
// Note that this load is not only an assert for correctness about
|
||||
// disconnection, but also a proper fence before the read of
|
||||
// `to_wake`, so this assert cannot be removed with also removing
|
||||
// the `to_wake` assert.
|
||||
assert_eq!(self.cnt.load(atomic::SeqCst), DISCONNECTED);
|
||||
assert_eq!(self.to_wake.load(atomic::SeqCst), 0);
|
||||
assert_eq!(self.channels.load(atomic::SeqCst), 0);
|
||||
}
|
||||
}
|
||||
343
src/libstd/sync/mpsc/spsc_queue.rs
Normal file
343
src/libstd/sync/mpsc/spsc_queue.rs
Normal file
@@ -0,0 +1,343 @@
|
||||
/* 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.
|
||||
*/
|
||||
|
||||
// http://www.1024cores.net/home/lock-free-algorithms/queues/unbounded-spsc-queue
|
||||
|
||||
//! A single-producer single-consumer concurrent queue
|
||||
//!
|
||||
//! This module contains the implementation of an SPSC queue which can be used
|
||||
//! concurrently between two tasks. This data structure is safe to use and
|
||||
//! enforces the semantics that there is one pusher and one popper.
|
||||
|
||||
#![experimental]
|
||||
|
||||
use core::prelude::*;
|
||||
|
||||
use alloc::boxed::Box;
|
||||
use core::mem;
|
||||
use core::cell::UnsafeCell;
|
||||
|
||||
use sync::atomic::{AtomicPtr, Relaxed, AtomicUint, Acquire, Release};
|
||||
|
||||
// Node within the linked list queue of messages to send
|
||||
struct Node<T> {
|
||||
// FIXME: this could be an uninitialized T if we're careful enough, and
|
||||
// that would reduce memory usage (and be a bit faster).
|
||||
// is it worth it?
|
||||
value: Option<T>, // nullable for re-use of nodes
|
||||
next: AtomicPtr<Node<T>>, // next node in the queue
|
||||
}
|
||||
|
||||
/// The single-producer single-consumer queue. This structure is not cloneable,
|
||||
/// but it can be safely shared in an Arc if it is guaranteed that there
|
||||
/// is only one popper and one pusher touching the queue at any one point in
|
||||
/// time.
|
||||
pub struct Queue<T> {
|
||||
// consumer fields
|
||||
tail: UnsafeCell<*mut Node<T>>, // where to pop from
|
||||
tail_prev: AtomicPtr<Node<T>>, // where to pop from
|
||||
|
||||
// producer fields
|
||||
head: UnsafeCell<*mut Node<T>>, // where to push to
|
||||
first: UnsafeCell<*mut Node<T>>, // where to get new nodes from
|
||||
tail_copy: UnsafeCell<*mut Node<T>>, // between first/tail
|
||||
|
||||
// Cache maintenance fields. Additions and subtractions are stored
|
||||
// separately in order to allow them to use nonatomic addition/subtraction.
|
||||
cache_bound: uint,
|
||||
cache_additions: AtomicUint,
|
||||
cache_subtractions: AtomicUint,
|
||||
}
|
||||
|
||||
unsafe impl<T: Send> Send for Queue<T> { }
|
||||
|
||||
unsafe impl<T: Send> Sync for Queue<T> { }
|
||||
|
||||
impl<T: Send> Node<T> {
|
||||
fn new() -> *mut Node<T> {
|
||||
unsafe {
|
||||
mem::transmute(box Node {
|
||||
value: None,
|
||||
next: AtomicPtr::new(0 as *mut Node<T>),
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Send> Queue<T> {
|
||||
/// Creates a new queue.
|
||||
///
|
||||
/// This is unsafe as the type system doesn't enforce a single
|
||||
/// consumer-producer relationship. It also allows the consumer to `pop`
|
||||
/// items while there is a `peek` active due to all methods having a
|
||||
/// non-mutable receiver.
|
||||
///
|
||||
/// # Arguments
|
||||
///
|
||||
/// * `bound` - This queue implementation is implemented with a linked
|
||||
/// list, and this means that a push is always a malloc. In
|
||||
/// order to amortize this cost, an internal cache of nodes is
|
||||
/// maintained to prevent a malloc from always being
|
||||
/// necessary. This bound is the limit on the size of the
|
||||
/// cache (if desired). If the value is 0, then the cache has
|
||||
/// no bound. Otherwise, the cache will never grow larger than
|
||||
/// `bound` (although the queue itself could be much larger.
|
||||
pub unsafe fn new(bound: uint) -> Queue<T> {
|
||||
let n1 = Node::new();
|
||||
let n2 = Node::new();
|
||||
(*n1).next.store(n2, Relaxed);
|
||||
Queue {
|
||||
tail: UnsafeCell::new(n2),
|
||||
tail_prev: AtomicPtr::new(n1),
|
||||
head: UnsafeCell::new(n2),
|
||||
first: UnsafeCell::new(n1),
|
||||
tail_copy: UnsafeCell::new(n1),
|
||||
cache_bound: bound,
|
||||
cache_additions: AtomicUint::new(0),
|
||||
cache_subtractions: AtomicUint::new(0),
|
||||
}
|
||||
}
|
||||
|
||||
/// Pushes a new value onto this queue. Note that to use this function
|
||||
/// safely, it must be externally guaranteed that there is only one pusher.
|
||||
pub fn push(&self, t: T) {
|
||||
unsafe {
|
||||
// Acquire a node (which either uses a cached one or allocates a new
|
||||
// one), and then append this to the 'head' node.
|
||||
let n = self.alloc();
|
||||
assert!((*n).value.is_none());
|
||||
(*n).value = Some(t);
|
||||
(*n).next.store(0 as *mut Node<T>, Relaxed);
|
||||
(**self.head.get()).next.store(n, Release);
|
||||
*self.head.get() = n;
|
||||
}
|
||||
}
|
||||
|
||||
unsafe fn alloc(&self) -> *mut Node<T> {
|
||||
// First try to see if we can consume the 'first' node for our uses.
|
||||
// We try to avoid as many atomic instructions as possible here, so
|
||||
// the addition to cache_subtractions is not atomic (plus we're the
|
||||
// only one subtracting from the cache).
|
||||
if *self.first.get() != *self.tail_copy.get() {
|
||||
if self.cache_bound > 0 {
|
||||
let b = self.cache_subtractions.load(Relaxed);
|
||||
self.cache_subtractions.store(b + 1, Relaxed);
|
||||
}
|
||||
let ret = *self.first.get();
|
||||
*self.first.get() = (*ret).next.load(Relaxed);
|
||||
return ret;
|
||||
}
|
||||
// If the above fails, then update our copy of the tail and try
|
||||
// again.
|
||||
*self.tail_copy.get() = self.tail_prev.load(Acquire);
|
||||
if *self.first.get() != *self.tail_copy.get() {
|
||||
if self.cache_bound > 0 {
|
||||
let b = self.cache_subtractions.load(Relaxed);
|
||||
self.cache_subtractions.store(b + 1, Relaxed);
|
||||
}
|
||||
let ret = *self.first.get();
|
||||
*self.first.get() = (*ret).next.load(Relaxed);
|
||||
return ret;
|
||||
}
|
||||
// If all of that fails, then we have to allocate a new node
|
||||
// (there's nothing in the node cache).
|
||||
Node::new()
|
||||
}
|
||||
|
||||
/// Attempts to pop a value from this queue. Remember that to use this type
|
||||
/// safely you must ensure that there is only one popper at a time.
|
||||
pub fn pop(&self) -> Option<T> {
|
||||
unsafe {
|
||||
// The `tail` node is not actually a used node, but rather a
|
||||
// sentinel from where we should start popping from. Hence, look at
|
||||
// tail's next field and see if we can use it. If we do a pop, then
|
||||
// the current tail node is a candidate for going into the cache.
|
||||
let tail = *self.tail.get();
|
||||
let next = (*tail).next.load(Acquire);
|
||||
if next.is_null() { return None }
|
||||
assert!((*next).value.is_some());
|
||||
let ret = (*next).value.take();
|
||||
|
||||
*self.tail.get() = next;
|
||||
if self.cache_bound == 0 {
|
||||
self.tail_prev.store(tail, Release);
|
||||
} else {
|
||||
// FIXME: this is dubious with overflow.
|
||||
let additions = self.cache_additions.load(Relaxed);
|
||||
let subtractions = self.cache_subtractions.load(Relaxed);
|
||||
let size = additions - subtractions;
|
||||
|
||||
if size < self.cache_bound {
|
||||
self.tail_prev.store(tail, Release);
|
||||
self.cache_additions.store(additions + 1, Relaxed);
|
||||
} else {
|
||||
(*self.tail_prev.load(Relaxed)).next.store(next, Relaxed);
|
||||
// We have successfully erased all references to 'tail', so
|
||||
// now we can safely drop it.
|
||||
let _: Box<Node<T>> = mem::transmute(tail);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
|
||||
/// Attempts to peek at the head of the queue, returning `None` if the queue
|
||||
/// has no data currently
|
||||
///
|
||||
/// # Warning
|
||||
/// The reference returned is invalid if it is not used before the consumer
|
||||
/// pops the value off the queue. If the producer then pushes another value
|
||||
/// onto the queue, it will overwrite the value pointed to by the reference.
|
||||
pub fn peek<'a>(&'a self) -> Option<&'a mut T> {
|
||||
// This is essentially the same as above with all the popping bits
|
||||
// stripped out.
|
||||
unsafe {
|
||||
let tail = *self.tail.get();
|
||||
let next = (*tail).next.load(Acquire);
|
||||
if next.is_null() { return None }
|
||||
return (*next).value.as_mut();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[unsafe_destructor]
|
||||
impl<T: Send> Drop for Queue<T> {
|
||||
fn drop(&mut self) {
|
||||
unsafe {
|
||||
let mut cur = *self.first.get();
|
||||
while !cur.is_null() {
|
||||
let next = (*cur).next.load(Relaxed);
|
||||
let _n: Box<Node<T>> = mem::transmute(cur);
|
||||
cur = next;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use prelude::v1::*;
|
||||
|
||||
use sync::Arc;
|
||||
use super::Queue;
|
||||
use thread::Thread;
|
||||
use sync::mpsc::channel;
|
||||
|
||||
#[test]
|
||||
fn smoke() {
|
||||
unsafe {
|
||||
let queue = Queue::new(0);
|
||||
queue.push(1i);
|
||||
queue.push(2);
|
||||
assert_eq!(queue.pop(), Some(1i));
|
||||
assert_eq!(queue.pop(), Some(2));
|
||||
assert_eq!(queue.pop(), None);
|
||||
queue.push(3);
|
||||
queue.push(4);
|
||||
assert_eq!(queue.pop(), Some(3));
|
||||
assert_eq!(queue.pop(), Some(4));
|
||||
assert_eq!(queue.pop(), None);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn peek() {
|
||||
unsafe {
|
||||
let queue = Queue::new(0);
|
||||
queue.push(vec![1i]);
|
||||
|
||||
// Ensure the borrowchecker works
|
||||
match queue.peek() {
|
||||
Some(vec) => match vec.as_slice() {
|
||||
// Note that `pop` is not allowed here due to borrow
|
||||
[1] => {}
|
||||
_ => return
|
||||
},
|
||||
None => unreachable!()
|
||||
}
|
||||
|
||||
queue.pop();
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn drop_full() {
|
||||
unsafe {
|
||||
let q = Queue::new(0);
|
||||
q.push(box 1i);
|
||||
q.push(box 2i);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn smoke_bound() {
|
||||
unsafe {
|
||||
let q = Queue::new(0);
|
||||
q.push(1i);
|
||||
q.push(2);
|
||||
assert_eq!(q.pop(), Some(1));
|
||||
assert_eq!(q.pop(), Some(2));
|
||||
assert_eq!(q.pop(), None);
|
||||
q.push(3);
|
||||
q.push(4);
|
||||
assert_eq!(q.pop(), Some(3));
|
||||
assert_eq!(q.pop(), Some(4));
|
||||
assert_eq!(q.pop(), None);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn stress() {
|
||||
unsafe {
|
||||
stress_bound(0);
|
||||
stress_bound(1);
|
||||
}
|
||||
|
||||
unsafe fn stress_bound(bound: uint) {
|
||||
let q = Arc::new(Queue::new(bound));
|
||||
|
||||
let (tx, rx) = channel();
|
||||
let q2 = q.clone();
|
||||
let _t = Thread::spawn(move|| {
|
||||
for _ in range(0u, 100000) {
|
||||
loop {
|
||||
match q2.pop() {
|
||||
Some(1i) => break,
|
||||
Some(_) => panic!(),
|
||||
None => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
tx.send(()).unwrap();
|
||||
});
|
||||
for _ in range(0i, 100000) {
|
||||
q.push(1);
|
||||
}
|
||||
rx.recv().unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
484
src/libstd/sync/mpsc/stream.rs
Normal file
484
src/libstd/sync/mpsc/stream.rs
Normal file
@@ -0,0 +1,484 @@
|
||||
// Copyright 2014 The Rust Project Developers. See the COPYRIGHT
|
||||
// file at the top-level directory of this distribution and at
|
||||
// http://rust-lang.org/COPYRIGHT.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
|
||||
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
|
||||
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
|
||||
// option. This file may not be copied, modified, or distributed
|
||||
// except according to those terms.
|
||||
|
||||
/// Stream channels
|
||||
///
|
||||
/// This is the flavor of channels which are optimized for one sender and one
|
||||
/// receiver. The sender will be upgraded to a shared channel if the channel is
|
||||
/// cloned.
|
||||
///
|
||||
/// High level implementation details can be found in the comment of the parent
|
||||
/// module.
|
||||
|
||||
pub use self::Failure::*;
|
||||
pub use self::UpgradeResult::*;
|
||||
pub use self::SelectionResult::*;
|
||||
use self::Message::*;
|
||||
|
||||
use core::prelude::*;
|
||||
|
||||
use core::cmp;
|
||||
use core::int;
|
||||
use thread::Thread;
|
||||
|
||||
use sync::mpsc::blocking::{mod, SignalToken};
|
||||
use sync::mpsc::spsc_queue as spsc;
|
||||
use sync::mpsc::Receiver;
|
||||
use sync::atomic;
|
||||
|
||||
const DISCONNECTED: int = int::MIN;
|
||||
#[cfg(test)]
|
||||
const MAX_STEALS: int = 5;
|
||||
#[cfg(not(test))]
|
||||
const MAX_STEALS: int = 1 << 20;
|
||||
|
||||
pub struct Packet<T> {
|
||||
queue: spsc::Queue<Message<T>>, // internal queue for all message
|
||||
|
||||
cnt: atomic::AtomicInt, // How many items are on this channel
|
||||
steals: int, // How many times has a port received without blocking?
|
||||
to_wake: atomic::AtomicUint, // SignalToken for the blocked thread to wake up
|
||||
|
||||
port_dropped: atomic::AtomicBool, // flag if the channel has been destroyed.
|
||||
}
|
||||
|
||||
pub enum Failure<T> {
|
||||
Empty,
|
||||
Disconnected,
|
||||
Upgraded(Receiver<T>),
|
||||
}
|
||||
|
||||
pub enum UpgradeResult {
|
||||
UpSuccess,
|
||||
UpDisconnected,
|
||||
UpWoke(SignalToken),
|
||||
}
|
||||
|
||||
pub enum SelectionResult<T> {
|
||||
SelSuccess,
|
||||
SelCanceled,
|
||||
SelUpgraded(SignalToken, Receiver<T>),
|
||||
}
|
||||
|
||||
// Any message could contain an "upgrade request" to a new shared port, so the
|
||||
// internal queue it's a queue of T, but rather Message<T>
|
||||
enum Message<T> {
|
||||
Data(T),
|
||||
GoUp(Receiver<T>),
|
||||
}
|
||||
|
||||
impl<T: Send> Packet<T> {
|
||||
pub fn new() -> Packet<T> {
|
||||
Packet {
|
||||
queue: unsafe { spsc::Queue::new(128) },
|
||||
|
||||
cnt: atomic::AtomicInt::new(0),
|
||||
steals: 0,
|
||||
to_wake: atomic::AtomicUint::new(0),
|
||||
|
||||
port_dropped: atomic::AtomicBool::new(false),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn send(&mut self, t: T) -> Result<(), T> {
|
||||
// If the other port has deterministically gone away, then definitely
|
||||
// must return the data back up the stack. Otherwise, the data is
|
||||
// considered as being sent.
|
||||
if self.port_dropped.load(atomic::SeqCst) { return Err(t) }
|
||||
|
||||
match self.do_send(Data(t)) {
|
||||
UpSuccess | UpDisconnected => {},
|
||||
UpWoke(token) => { token.signal(); }
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn upgrade(&mut self, up: Receiver<T>) -> UpgradeResult {
|
||||
// If the port has gone away, then there's no need to proceed any
|
||||
// further.
|
||||
if self.port_dropped.load(atomic::SeqCst) { return UpDisconnected }
|
||||
|
||||
self.do_send(GoUp(up))
|
||||
}
|
||||
|
||||
fn do_send(&mut self, t: Message<T>) -> UpgradeResult {
|
||||
self.queue.push(t);
|
||||
match self.cnt.fetch_add(1, atomic::SeqCst) {
|
||||
// As described in the mod's doc comment, -1 == wakeup
|
||||
-1 => UpWoke(self.take_to_wake()),
|
||||
// As as described before, SPSC queues must be >= -2
|
||||
-2 => UpSuccess,
|
||||
|
||||
// Be sure to preserve the disconnected state, and the return value
|
||||
// in this case is going to be whether our data was received or not.
|
||||
// This manifests itself on whether we have an empty queue or not.
|
||||
//
|
||||
// Primarily, are required to drain the queue here because the port
|
||||
// will never remove this data. We can only have at most one item to
|
||||
// drain (the port drains the rest).
|
||||
DISCONNECTED => {
|
||||
self.cnt.store(DISCONNECTED, atomic::SeqCst);
|
||||
let first = self.queue.pop();
|
||||
let second = self.queue.pop();
|
||||
assert!(second.is_none());
|
||||
|
||||
match first {
|
||||
Some(..) => UpSuccess, // we failed to send the data
|
||||
None => UpDisconnected, // we successfully sent data
|
||||
}
|
||||
}
|
||||
|
||||
// Otherwise we just sent some data on a non-waiting queue, so just
|
||||
// make sure the world is sane and carry on!
|
||||
n => { assert!(n >= 0); UpSuccess }
|
||||
}
|
||||
}
|
||||
|
||||
// Consumes ownership of the 'to_wake' field.
|
||||
fn take_to_wake(&mut self) -> SignalToken {
|
||||
let ptr = self.to_wake.load(atomic::SeqCst);
|
||||
self.to_wake.store(0, atomic::SeqCst);
|
||||
assert!(ptr != 0);
|
||||
unsafe { SignalToken::cast_from_uint(ptr) }
|
||||
}
|
||||
|
||||
// Decrements the count on the channel for a sleeper, returning the sleeper
|
||||
// back if it shouldn't sleep. Note that this is the location where we take
|
||||
// steals into account.
|
||||
fn decrement(&mut self, token: SignalToken) -> Result<(), SignalToken> {
|
||||
assert_eq!(self.to_wake.load(atomic::SeqCst), 0);
|
||||
let ptr = unsafe { token.cast_to_uint() };
|
||||
self.to_wake.store(ptr, atomic::SeqCst);
|
||||
|
||||
let steals = self.steals;
|
||||
self.steals = 0;
|
||||
|
||||
match self.cnt.fetch_sub(1 + steals, atomic::SeqCst) {
|
||||
DISCONNECTED => { self.cnt.store(DISCONNECTED, atomic::SeqCst); }
|
||||
// If we factor in our steals and notice that the channel has no
|
||||
// data, we successfully sleep
|
||||
n => {
|
||||
assert!(n >= 0);
|
||||
if n - steals <= 0 { return Ok(()) }
|
||||
}
|
||||
}
|
||||
|
||||
self.to_wake.store(0, atomic::SeqCst);
|
||||
Err(unsafe { SignalToken::cast_from_uint(ptr) })
|
||||
}
|
||||
|
||||
pub fn recv(&mut self) -> Result<T, Failure<T>> {
|
||||
// Optimistic preflight check (scheduling is expensive).
|
||||
match self.try_recv() {
|
||||
Err(Empty) => {}
|
||||
data => return data,
|
||||
}
|
||||
|
||||
// Welp, our channel has no data. Deschedule the current task and
|
||||
// initiate the blocking protocol.
|
||||
let (wait_token, signal_token) = blocking::tokens();
|
||||
if self.decrement(signal_token).is_ok() {
|
||||
wait_token.wait()
|
||||
}
|
||||
|
||||
match self.try_recv() {
|
||||
// Messages which actually popped from the queue shouldn't count as
|
||||
// a steal, so offset the decrement here (we already have our
|
||||
// "steal" factored into the channel count above).
|
||||
data @ Ok(..) |
|
||||
data @ Err(Upgraded(..)) => {
|
||||
self.steals -= 1;
|
||||
data
|
||||
}
|
||||
|
||||
data => data,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn try_recv(&mut self) -> Result<T, Failure<T>> {
|
||||
match self.queue.pop() {
|
||||
// If we stole some data, record to that effect (this will be
|
||||
// factored into cnt later on).
|
||||
//
|
||||
// Note that we don't allow steals to grow without bound in order to
|
||||
// prevent eventual overflow of either steals or cnt as an overflow
|
||||
// would have catastrophic results. Sometimes, steals > cnt, but
|
||||
// other times cnt > steals, so we don't know the relation between
|
||||
// steals and cnt. This code path is executed only rarely, so we do
|
||||
// a pretty slow operation, of swapping 0 into cnt, taking steals
|
||||
// down as much as possible (without going negative), and then
|
||||
// adding back in whatever we couldn't factor into steals.
|
||||
Some(data) => {
|
||||
if self.steals > MAX_STEALS {
|
||||
match self.cnt.swap(0, atomic::SeqCst) {
|
||||
DISCONNECTED => {
|
||||
self.cnt.store(DISCONNECTED, atomic::SeqCst);
|
||||
}
|
||||
n => {
|
||||
let m = cmp::min(n, self.steals);
|
||||
self.steals -= m;
|
||||
self.bump(n - m);
|
||||
}
|
||||
}
|
||||
assert!(self.steals >= 0);
|
||||
}
|
||||
self.steals += 1;
|
||||
match data {
|
||||
Data(t) => Ok(t),
|
||||
GoUp(up) => Err(Upgraded(up)),
|
||||
}
|
||||
}
|
||||
|
||||
None => {
|
||||
match self.cnt.load(atomic::SeqCst) {
|
||||
n if n != DISCONNECTED => Err(Empty),
|
||||
|
||||
// This is a little bit of a tricky case. We failed to pop
|
||||
// data above, and then we have viewed that the channel is
|
||||
// disconnected. In this window more data could have been
|
||||
// sent on the channel. It doesn't really make sense to
|
||||
// return that the channel is disconnected when there's
|
||||
// actually data on it, so be extra sure there's no data by
|
||||
// popping one more time.
|
||||
//
|
||||
// We can ignore steals because the other end is
|
||||
// disconnected and we'll never need to really factor in our
|
||||
// steals again.
|
||||
_ => {
|
||||
match self.queue.pop() {
|
||||
Some(Data(t)) => Ok(t),
|
||||
Some(GoUp(up)) => Err(Upgraded(up)),
|
||||
None => Err(Disconnected),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn drop_chan(&mut self) {
|
||||
// Dropping a channel is pretty simple, we just flag it as disconnected
|
||||
// and then wakeup a blocker if there is one.
|
||||
match self.cnt.swap(DISCONNECTED, atomic::SeqCst) {
|
||||
-1 => { self.take_to_wake().signal(); }
|
||||
DISCONNECTED => {}
|
||||
n => { assert!(n >= 0); }
|
||||
}
|
||||
}
|
||||
|
||||
pub fn drop_port(&mut self) {
|
||||
// Dropping a port seems like a fairly trivial thing. In theory all we
|
||||
// need to do is flag that we're disconnected and then everything else
|
||||
// can take over (we don't have anyone to wake up).
|
||||
//
|
||||
// The catch for Ports is that we want to drop the entire contents of
|
||||
// the queue. There are multiple reasons for having this property, the
|
||||
// largest of which is that if another chan is waiting in this channel
|
||||
// (but not received yet), then waiting on that port will cause a
|
||||
// deadlock.
|
||||
//
|
||||
// So if we accept that we must now destroy the entire contents of the
|
||||
// queue, this code may make a bit more sense. The tricky part is that
|
||||
// we can't let any in-flight sends go un-dropped, we have to make sure
|
||||
// *everything* is dropped and nothing new will come onto the channel.
|
||||
|
||||
// The first thing we do is set a flag saying that we're done for. All
|
||||
// sends are gated on this flag, so we're immediately guaranteed that
|
||||
// there are a bounded number of active sends that we'll have to deal
|
||||
// with.
|
||||
self.port_dropped.store(true, atomic::SeqCst);
|
||||
|
||||
// Now that we're guaranteed to deal with a bounded number of senders,
|
||||
// we need to drain the queue. This draining process happens atomically
|
||||
// with respect to the "count" of the channel. If the count is nonzero
|
||||
// (with steals taken into account), then there must be data on the
|
||||
// channel. In this case we drain everything and then try again. We will
|
||||
// continue to fail while active senders send data while we're dropping
|
||||
// data, but eventually we're guaranteed to break out of this loop
|
||||
// (because there is a bounded number of senders).
|
||||
let mut steals = self.steals;
|
||||
while {
|
||||
let cnt = self.cnt.compare_and_swap(
|
||||
steals, DISCONNECTED, atomic::SeqCst);
|
||||
cnt != DISCONNECTED && cnt != steals
|
||||
} {
|
||||
loop {
|
||||
match self.queue.pop() {
|
||||
Some(..) => { steals += 1; }
|
||||
None => break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// At this point in time, we have gated all future senders from sending,
|
||||
// and we have flagged the channel as being disconnected. The senders
|
||||
// still have some responsibility, however, because some sends may not
|
||||
// complete until after we flag the disconnection. There are more
|
||||
// details in the sending methods that see DISCONNECTED
|
||||
}
|
||||
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
// select implementation
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
// Tests to see whether this port can receive without blocking. If Ok is
|
||||
// returned, then that's the answer. If Err is returned, then the returned
|
||||
// port needs to be queried instead (an upgrade happened)
|
||||
pub fn can_recv(&mut self) -> Result<bool, Receiver<T>> {
|
||||
// We peek at the queue to see if there's anything on it, and we use
|
||||
// this return value to determine if we should pop from the queue and
|
||||
// upgrade this channel immediately. If it looks like we've got an
|
||||
// upgrade pending, then go through the whole recv rigamarole to update
|
||||
// the internal state.
|
||||
match self.queue.peek() {
|
||||
Some(&GoUp(..)) => {
|
||||
match self.recv() {
|
||||
Err(Upgraded(port)) => Err(port),
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
Some(..) => Ok(true),
|
||||
None => Ok(false)
|
||||
}
|
||||
}
|
||||
|
||||
// increment the count on the channel (used for selection)
|
||||
fn bump(&mut self, amt: int) -> int {
|
||||
match self.cnt.fetch_add(amt, atomic::SeqCst) {
|
||||
DISCONNECTED => {
|
||||
self.cnt.store(DISCONNECTED, atomic::SeqCst);
|
||||
DISCONNECTED
|
||||
}
|
||||
n => n
|
||||
}
|
||||
}
|
||||
|
||||
// Attempts to start selecting on this port. Like a oneshot, this can fail
|
||||
// immediately because of an upgrade.
|
||||
pub fn start_selection(&mut self, token: SignalToken) -> SelectionResult<T> {
|
||||
match self.decrement(token) {
|
||||
Ok(()) => SelSuccess,
|
||||
Err(token) => {
|
||||
let ret = match self.queue.peek() {
|
||||
Some(&GoUp(..)) => {
|
||||
match self.queue.pop() {
|
||||
Some(GoUp(port)) => SelUpgraded(token, port),
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
Some(..) => SelCanceled,
|
||||
None => SelCanceled,
|
||||
};
|
||||
// Undo our decrement above, and we should be guaranteed that the
|
||||
// previous value is positive because we're not going to sleep
|
||||
let prev = self.bump(1);
|
||||
assert!(prev == DISCONNECTED || prev >= 0);
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Removes a previous task from being blocked in this port
|
||||
pub fn abort_selection(&mut self,
|
||||
was_upgrade: bool) -> Result<bool, Receiver<T>> {
|
||||
// If we're aborting selection after upgrading from a oneshot, then
|
||||
// we're guarantee that no one is waiting. The only way that we could
|
||||
// have seen the upgrade is if data was actually sent on the channel
|
||||
// half again. For us, this means that there is guaranteed to be data on
|
||||
// this channel. Furthermore, we're guaranteed that there was no
|
||||
// start_selection previously, so there's no need to modify `self.cnt`
|
||||
// at all.
|
||||
//
|
||||
// Hence, because of these invariants, we immediately return `Ok(true)`.
|
||||
// Note that the data may not actually be sent on the channel just yet.
|
||||
// The other end could have flagged the upgrade but not sent data to
|
||||
// this end. This is fine because we know it's a small bounded windows
|
||||
// of time until the data is actually sent.
|
||||
if was_upgrade {
|
||||
assert_eq!(self.steals, 0);
|
||||
assert_eq!(self.to_wake.load(atomic::SeqCst), 0);
|
||||
return Ok(true)
|
||||
}
|
||||
|
||||
// We want to make sure that the count on the channel goes non-negative,
|
||||
// and in the stream case we can have at most one steal, so just assume
|
||||
// that we had one steal.
|
||||
let steals = 1;
|
||||
let prev = self.bump(steals + 1);
|
||||
|
||||
// If we were previously disconnected, then we know for sure that there
|
||||
// is no task in to_wake, so just keep going
|
||||
let has_data = if prev == DISCONNECTED {
|
||||
assert_eq!(self.to_wake.load(atomic::SeqCst), 0);
|
||||
true // there is data, that data is that we're disconnected
|
||||
} else {
|
||||
let cur = prev + steals + 1;
|
||||
assert!(cur >= 0);
|
||||
|
||||
// If the previous count was negative, then we just made things go
|
||||
// positive, hence we passed the -1 boundary and we're responsible
|
||||
// for removing the to_wake() field and trashing it.
|
||||
//
|
||||
// If the previous count was positive then we're in a tougher
|
||||
// situation. A possible race is that a sender just incremented
|
||||
// through -1 (meaning it's going to try to wake a task up), but it
|
||||
// hasn't yet read the to_wake. In order to prevent a future recv()
|
||||
// from waking up too early (this sender picking up the plastered
|
||||
// over to_wake), we spin loop here waiting for to_wake to be 0.
|
||||
// Note that this entire select() implementation needs an overhaul,
|
||||
// and this is *not* the worst part of it, so this is not done as a
|
||||
// final solution but rather out of necessity for now to get
|
||||
// something working.
|
||||
if prev < 0 {
|
||||
drop(self.take_to_wake());
|
||||
} else {
|
||||
while self.to_wake.load(atomic::SeqCst) != 0 {
|
||||
Thread::yield_now();
|
||||
}
|
||||
}
|
||||
assert_eq!(self.steals, 0);
|
||||
self.steals = steals;
|
||||
|
||||
// if we were previously positive, then there's surely data to
|
||||
// receive
|
||||
prev >= 0
|
||||
};
|
||||
|
||||
// Now that we've determined that this queue "has data", we peek at the
|
||||
// queue to see if the data is an upgrade or not. If it's an upgrade,
|
||||
// then we need to destroy this port and abort selection on the
|
||||
// upgraded port.
|
||||
if has_data {
|
||||
match self.queue.peek() {
|
||||
Some(&GoUp(..)) => {
|
||||
match self.queue.pop() {
|
||||
Some(GoUp(port)) => Err(port),
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
_ => Ok(true),
|
||||
}
|
||||
} else {
|
||||
Ok(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[unsafe_destructor]
|
||||
impl<T: Send> Drop for Packet<T> {
|
||||
fn drop(&mut self) {
|
||||
// Note that this load is not only an assert for correctness about
|
||||
// disconnection, but also a proper fence before the read of
|
||||
// `to_wake`, so this assert cannot be removed with also removing
|
||||
// the `to_wake` assert.
|
||||
assert_eq!(self.cnt.load(atomic::SeqCst), DISCONNECTED);
|
||||
assert_eq!(self.to_wake.load(atomic::SeqCst), 0);
|
||||
}
|
||||
}
|
||||
483
src/libstd/sync/mpsc/sync.rs
Normal file
483
src/libstd/sync/mpsc/sync.rs
Normal file
@@ -0,0 +1,483 @@
|
||||
// Copyright 2014 The Rust Project Developers. See the COPYRIGHT
|
||||
// file at the top-level directory of this distribution and at
|
||||
// http://rust-lang.org/COPYRIGHT.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
|
||||
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
|
||||
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
|
||||
// option. This file may not be copied, modified, or distributed
|
||||
// except according to those terms.
|
||||
|
||||
/// Synchronous channels/ports
|
||||
///
|
||||
/// This channel implementation differs significantly from the asynchronous
|
||||
/// implementations found next to it (oneshot/stream/share). This is an
|
||||
/// implementation of a synchronous, bounded buffer channel.
|
||||
///
|
||||
/// Each channel is created with some amount of backing buffer, and sends will
|
||||
/// *block* until buffer space becomes available. A buffer size of 0 is valid,
|
||||
/// which means that every successful send is paired with a successful recv.
|
||||
///
|
||||
/// This flavor of channels defines a new `send_opt` method for channels which
|
||||
/// is the method by which a message is sent but the task does not panic if it
|
||||
/// cannot be delivered.
|
||||
///
|
||||
/// Another major difference is that send() will *always* return back the data
|
||||
/// if it couldn't be sent. This is because it is deterministically known when
|
||||
/// the data is received and when it is not received.
|
||||
///
|
||||
/// Implementation-wise, it can all be summed up with "use a mutex plus some
|
||||
/// logic". The mutex used here is an OS native mutex, meaning that no user code
|
||||
/// is run inside of the mutex (to prevent context switching). This
|
||||
/// implementation shares almost all code for the buffered and unbuffered cases
|
||||
/// of a synchronous channel. There are a few branches for the unbuffered case,
|
||||
/// but they're mostly just relevant to blocking senders.
|
||||
|
||||
use core::prelude::*;
|
||||
|
||||
pub use self::Failure::*;
|
||||
use self::Blocker::*;
|
||||
|
||||
use vec::Vec;
|
||||
use core::mem;
|
||||
|
||||
use sync::{atomic, Mutex, MutexGuard};
|
||||
use sync::mpsc::blocking::{mod, WaitToken, SignalToken};
|
||||
use sync::mpsc::select::StartResult::{mod, Installed, Abort};
|
||||
|
||||
pub struct Packet<T> {
|
||||
/// Only field outside of the mutex. Just done for kicks, but mainly because
|
||||
/// the other shared channel already had the code implemented
|
||||
channels: atomic::AtomicUint,
|
||||
|
||||
lock: Mutex<State<T>>,
|
||||
}
|
||||
|
||||
unsafe impl<T:Send> Send for Packet<T> { }
|
||||
|
||||
unsafe impl<T:Send> Sync for Packet<T> { }
|
||||
|
||||
struct State<T> {
|
||||
disconnected: bool, // Is the channel disconnected yet?
|
||||
queue: Queue, // queue of senders waiting to send data
|
||||
blocker: Blocker, // currently blocked task on this channel
|
||||
buf: Buffer<T>, // storage for buffered messages
|
||||
cap: uint, // capacity of this channel
|
||||
|
||||
/// A curious flag used to indicate whether a sender failed or succeeded in
|
||||
/// blocking. This is used to transmit information back to the task that it
|
||||
/// must dequeue its message from the buffer because it was not received.
|
||||
/// This is only relevant in the 0-buffer case. This obviously cannot be
|
||||
/// safely constructed, but it's guaranteed to always have a valid pointer
|
||||
/// value.
|
||||
canceled: Option<&'static mut bool>,
|
||||
}
|
||||
|
||||
unsafe impl<T: Send> Send for State<T> {}
|
||||
|
||||
/// Possible flavors of threads who can be blocked on this channel.
|
||||
enum Blocker {
|
||||
BlockedSender(SignalToken),
|
||||
BlockedReceiver(SignalToken),
|
||||
NoneBlocked
|
||||
}
|
||||
|
||||
/// Simple queue for threading tasks together. Nodes are stack-allocated, so
|
||||
/// this structure is not safe at all
|
||||
struct Queue {
|
||||
head: *mut Node,
|
||||
tail: *mut Node,
|
||||
}
|
||||
|
||||
struct Node {
|
||||
token: Option<SignalToken>,
|
||||
next: *mut Node,
|
||||
}
|
||||
|
||||
unsafe impl Send for Node {}
|
||||
|
||||
/// A simple ring-buffer
|
||||
struct Buffer<T> {
|
||||
buf: Vec<Option<T>>,
|
||||
start: uint,
|
||||
size: uint,
|
||||
}
|
||||
|
||||
#[deriving(Show)]
|
||||
pub enum Failure {
|
||||
Empty,
|
||||
Disconnected,
|
||||
}
|
||||
|
||||
/// Atomically blocks the current thread, placing it into `slot`, unlocking `lock`
|
||||
/// in the meantime. This re-locks the mutex upon returning.
|
||||
fn wait<'a, 'b, T: Send>(lock: &'a Mutex<State<T>>,
|
||||
mut guard: MutexGuard<'b, State<T>>,
|
||||
f: fn(SignalToken) -> Blocker)
|
||||
-> MutexGuard<'a, State<T>>
|
||||
{
|
||||
let (wait_token, signal_token) = blocking::tokens();
|
||||
match mem::replace(&mut guard.blocker, f(signal_token)) {
|
||||
NoneBlocked => {}
|
||||
_ => unreachable!(),
|
||||
}
|
||||
drop(guard); // unlock
|
||||
wait_token.wait(); // block
|
||||
lock.lock().unwrap() // relock
|
||||
}
|
||||
|
||||
/// Wakes up a thread, dropping the lock at the correct time
|
||||
fn wakeup<T>(token: SignalToken, guard: MutexGuard<State<T>>) {
|
||||
// We need to be careful to wake up the waiting task *outside* of the mutex
|
||||
// in case it incurs a context switch.
|
||||
drop(guard);
|
||||
token.signal();
|
||||
}
|
||||
|
||||
impl<T: Send> Packet<T> {
|
||||
pub fn new(cap: uint) -> Packet<T> {
|
||||
Packet {
|
||||
channels: atomic::AtomicUint::new(1),
|
||||
lock: Mutex::new(State {
|
||||
disconnected: false,
|
||||
blocker: NoneBlocked,
|
||||
cap: cap,
|
||||
canceled: None,
|
||||
queue: Queue {
|
||||
head: 0 as *mut Node,
|
||||
tail: 0 as *mut Node,
|
||||
},
|
||||
buf: Buffer {
|
||||
buf: range(0, cap + if cap == 0 {1} else {0}).map(|_| None).collect(),
|
||||
start: 0,
|
||||
size: 0,
|
||||
},
|
||||
}),
|
||||
}
|
||||
}
|
||||
|
||||
// wait until a send slot is available, returning locked access to
|
||||
// the channel state.
|
||||
fn acquire_send_slot(&self) -> MutexGuard<State<T>> {
|
||||
let mut node = Node { token: None, next: 0 as *mut Node };
|
||||
loop {
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
// are we ready to go?
|
||||
if guard.disconnected || guard.buf.size() < guard.buf.cap() {
|
||||
return guard;
|
||||
}
|
||||
// no room; actually block
|
||||
let wait_token = guard.queue.enqueue(&mut node);
|
||||
drop(guard);
|
||||
wait_token.wait();
|
||||
}
|
||||
}
|
||||
|
||||
pub fn send(&self, t: T) -> Result<(), T> {
|
||||
let mut guard = self.acquire_send_slot();
|
||||
if guard.disconnected { return Err(t) }
|
||||
guard.buf.enqueue(t);
|
||||
|
||||
match mem::replace(&mut guard.blocker, NoneBlocked) {
|
||||
// if our capacity is 0, then we need to wait for a receiver to be
|
||||
// available to take our data. After waiting, we check again to make
|
||||
// sure the port didn't go away in the meantime. If it did, we need
|
||||
// to hand back our data.
|
||||
NoneBlocked if guard.cap == 0 => {
|
||||
let mut canceled = false;
|
||||
assert!(guard.canceled.is_none());
|
||||
guard.canceled = Some(unsafe { mem::transmute(&mut canceled) });
|
||||
let mut guard = wait(&self.lock, guard, BlockedSender);
|
||||
if canceled {Err(guard.buf.dequeue())} else {Ok(())}
|
||||
}
|
||||
|
||||
// success, we buffered some data
|
||||
NoneBlocked => Ok(()),
|
||||
|
||||
// success, someone's about to receive our buffered data.
|
||||
BlockedReceiver(token) => { wakeup(token, guard); Ok(()) }
|
||||
|
||||
BlockedSender(..) => panic!("lolwut"),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn try_send(&self, t: T) -> Result<(), super::TrySendError<T>> {
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
if guard.disconnected {
|
||||
Err(super::TrySendError::Disconnected(t))
|
||||
} else if guard.buf.size() == guard.buf.cap() {
|
||||
Err(super::TrySendError::Full(t))
|
||||
} else if guard.cap == 0 {
|
||||
// With capacity 0, even though we have buffer space we can't
|
||||
// transfer the data unless there's a receiver waiting.
|
||||
match mem::replace(&mut guard.blocker, NoneBlocked) {
|
||||
NoneBlocked => Err(super::TrySendError::Full(t)),
|
||||
BlockedSender(..) => unreachable!(),
|
||||
BlockedReceiver(token) => {
|
||||
guard.buf.enqueue(t);
|
||||
wakeup(token, guard);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// If the buffer has some space and the capacity isn't 0, then we
|
||||
// just enqueue the data for later retrieval, ensuring to wake up
|
||||
// any blocked receiver if there is one.
|
||||
assert!(guard.buf.size() < guard.buf.cap());
|
||||
guard.buf.enqueue(t);
|
||||
match mem::replace(&mut guard.blocker, NoneBlocked) {
|
||||
BlockedReceiver(token) => wakeup(token, guard),
|
||||
NoneBlocked => {}
|
||||
BlockedSender(..) => unreachable!(),
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
// Receives a message from this channel
|
||||
//
|
||||
// When reading this, remember that there can only ever be one receiver at
|
||||
// time.
|
||||
pub fn recv(&self) -> Result<T, ()> {
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
|
||||
// Wait for the buffer to have something in it. No need for a while loop
|
||||
// because we're the only receiver.
|
||||
let mut waited = false;
|
||||
if !guard.disconnected && guard.buf.size() == 0 {
|
||||
guard = wait(&self.lock, guard, BlockedReceiver);
|
||||
waited = true;
|
||||
}
|
||||
if guard.disconnected && guard.buf.size() == 0 { return Err(()) }
|
||||
|
||||
// Pick up the data, wake up our neighbors, and carry on
|
||||
assert!(guard.buf.size() > 0);
|
||||
let ret = guard.buf.dequeue();
|
||||
self.wakeup_senders(waited, guard);
|
||||
return Ok(ret);
|
||||
}
|
||||
|
||||
pub fn try_recv(&self) -> Result<T, Failure> {
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
|
||||
// Easy cases first
|
||||
if guard.disconnected { return Err(Disconnected) }
|
||||
if guard.buf.size() == 0 { return Err(Empty) }
|
||||
|
||||
// Be sure to wake up neighbors
|
||||
let ret = Ok(guard.buf.dequeue());
|
||||
self.wakeup_senders(false, guard);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
// Wake up pending senders after some data has been received
|
||||
//
|
||||
// * `waited` - flag if the receiver blocked to receive some data, or if it
|
||||
// just picked up some data on the way out
|
||||
// * `guard` - the lock guard that is held over this channel's lock
|
||||
fn wakeup_senders(&self, waited: bool, mut guard: MutexGuard<State<T>>) {
|
||||
let pending_sender1: Option<SignalToken> = guard.queue.dequeue();
|
||||
|
||||
// If this is a no-buffer channel (cap == 0), then if we didn't wait we
|
||||
// need to ACK the sender. If we waited, then the sender waking us up
|
||||
// was already the ACK.
|
||||
let pending_sender2 = if guard.cap == 0 && !waited {
|
||||
match mem::replace(&mut guard.blocker, NoneBlocked) {
|
||||
NoneBlocked => None,
|
||||
BlockedReceiver(..) => unreachable!(),
|
||||
BlockedSender(token) => {
|
||||
guard.canceled.take();
|
||||
Some(token)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
None
|
||||
};
|
||||
mem::drop(guard);
|
||||
|
||||
// only outside of the lock do we wake up the pending tasks
|
||||
pending_sender1.map(|t| t.signal());
|
||||
pending_sender2.map(|t| t.signal());
|
||||
}
|
||||
|
||||
// Prepares this shared packet for a channel clone, essentially just bumping
|
||||
// a refcount.
|
||||
pub fn clone_chan(&self) {
|
||||
self.channels.fetch_add(1, atomic::SeqCst);
|
||||
}
|
||||
|
||||
pub fn drop_chan(&self) {
|
||||
// Only flag the channel as disconnected if we're the last channel
|
||||
match self.channels.fetch_sub(1, atomic::SeqCst) {
|
||||
1 => {}
|
||||
_ => return
|
||||
}
|
||||
|
||||
// Not much to do other than wake up a receiver if one's there
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
if guard.disconnected { return }
|
||||
guard.disconnected = true;
|
||||
match mem::replace(&mut guard.blocker, NoneBlocked) {
|
||||
NoneBlocked => {}
|
||||
BlockedSender(..) => unreachable!(),
|
||||
BlockedReceiver(token) => wakeup(token, guard),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn drop_port(&self) {
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
|
||||
if guard.disconnected { return }
|
||||
guard.disconnected = true;
|
||||
|
||||
// If the capacity is 0, then the sender may want its data back after
|
||||
// we're disconnected. Otherwise it's now our responsibility to destroy
|
||||
// the buffered data. As with many other portions of this code, this
|
||||
// needs to be careful to destroy the data *outside* of the lock to
|
||||
// prevent deadlock.
|
||||
let _data = if guard.cap != 0 {
|
||||
mem::replace(&mut guard.buf.buf, Vec::new())
|
||||
} else {
|
||||
Vec::new()
|
||||
};
|
||||
let mut queue = mem::replace(&mut guard.queue, Queue {
|
||||
head: 0 as *mut Node,
|
||||
tail: 0 as *mut Node,
|
||||
});
|
||||
|
||||
let waiter = match mem::replace(&mut guard.blocker, NoneBlocked) {
|
||||
NoneBlocked => None,
|
||||
BlockedSender(token) => {
|
||||
*guard.canceled.take().unwrap() = true;
|
||||
Some(token)
|
||||
}
|
||||
BlockedReceiver(..) => unreachable!(),
|
||||
};
|
||||
mem::drop(guard);
|
||||
|
||||
loop {
|
||||
match queue.dequeue() {
|
||||
Some(token) => { token.signal(); }
|
||||
None => break,
|
||||
}
|
||||
}
|
||||
waiter.map(|t| t.signal());
|
||||
}
|
||||
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
// select implementation
|
||||
////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
// If Ok, the value is whether this port has data, if Err, then the upgraded
|
||||
// port needs to be checked instead of this one.
|
||||
pub fn can_recv(&self) -> bool {
|
||||
let guard = self.lock.lock().unwrap();
|
||||
guard.disconnected || guard.buf.size() > 0
|
||||
}
|
||||
|
||||
// Attempts to start selection on this port. This can either succeed or fail
|
||||
// because there is data waiting.
|
||||
pub fn start_selection(&self, token: SignalToken) -> StartResult {
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
if guard.disconnected || guard.buf.size() > 0 {
|
||||
Abort
|
||||
} else {
|
||||
match mem::replace(&mut guard.blocker, BlockedReceiver(token)) {
|
||||
NoneBlocked => {}
|
||||
BlockedSender(..) => unreachable!(),
|
||||
BlockedReceiver(..) => unreachable!(),
|
||||
}
|
||||
Installed
|
||||
}
|
||||
}
|
||||
|
||||
// Remove a previous selecting task from this port. This ensures that the
|
||||
// blocked task will no longer be visible to any other threads.
|
||||
//
|
||||
// The return value indicates whether there's data on this port.
|
||||
pub fn abort_selection(&self) -> bool {
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
match mem::replace(&mut guard.blocker, NoneBlocked) {
|
||||
NoneBlocked => true,
|
||||
BlockedSender(token) => {
|
||||
guard.blocker = BlockedSender(token);
|
||||
true
|
||||
}
|
||||
BlockedReceiver(token) => { drop(token); false }
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[unsafe_destructor]
|
||||
impl<T: Send> Drop for Packet<T> {
|
||||
fn drop(&mut self) {
|
||||
assert_eq!(self.channels.load(atomic::SeqCst), 0);
|
||||
let mut guard = self.lock.lock().unwrap();
|
||||
assert!(guard.queue.dequeue().is_none());
|
||||
assert!(guard.canceled.is_none());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
////////////////////////////////////////////////////////////////////////////////
|
||||
// Buffer, a simple ring buffer backed by Vec<T>
|
||||
////////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
impl<T> Buffer<T> {
|
||||
fn enqueue(&mut self, t: T) {
|
||||
let pos = (self.start + self.size) % self.buf.len();
|
||||
self.size += 1;
|
||||
let prev = mem::replace(&mut self.buf[pos], Some(t));
|
||||
assert!(prev.is_none());
|
||||
}
|
||||
|
||||
fn dequeue(&mut self) -> T {
|
||||
let start = self.start;
|
||||
self.size -= 1;
|
||||
self.start = (self.start + 1) % self.buf.len();
|
||||
self.buf[start].take().unwrap()
|
||||
}
|
||||
|
||||
fn size(&self) -> uint { self.size }
|
||||
fn cap(&self) -> uint { self.buf.len() }
|
||||
}
|
||||
|
||||
////////////////////////////////////////////////////////////////////////////////
|
||||
// Queue, a simple queue to enqueue tasks with (stack-allocated nodes)
|
||||
////////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
impl Queue {
|
||||
fn enqueue(&mut self, node: &mut Node) -> WaitToken {
|
||||
let (wait_token, signal_token) = blocking::tokens();
|
||||
node.token = Some(signal_token);
|
||||
node.next = 0 as *mut Node;
|
||||
|
||||
if self.tail.is_null() {
|
||||
self.head = node as *mut Node;
|
||||
self.tail = node as *mut Node;
|
||||
} else {
|
||||
unsafe {
|
||||
(*self.tail).next = node as *mut Node;
|
||||
self.tail = node as *mut Node;
|
||||
}
|
||||
}
|
||||
|
||||
wait_token
|
||||
}
|
||||
|
||||
fn dequeue(&mut self) -> Option<SignalToken> {
|
||||
if self.head.is_null() {
|
||||
return None
|
||||
}
|
||||
let node = self.head;
|
||||
self.head = unsafe { (*node).next };
|
||||
if self.head.is_null() {
|
||||
self.tail = 0 as *mut Node;
|
||||
}
|
||||
unsafe {
|
||||
(*node).next = 0 as *mut Node;
|
||||
Some((*node).token.take().unwrap())
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -48,7 +48,7 @@ use sys_common::mutex as sys;
|
||||
/// ```rust
|
||||
/// use std::sync::{Arc, Mutex};
|
||||
/// use std::thread::Thread;
|
||||
/// use std::comm::channel;
|
||||
/// use std::sync::mpsc::channel;
|
||||
///
|
||||
/// const N: uint = 10;
|
||||
///
|
||||
@@ -72,13 +72,13 @@ use sys_common::mutex as sys;
|
||||
/// let mut data = data.lock().unwrap();
|
||||
/// *data += 1;
|
||||
/// if *data == N {
|
||||
/// tx.send(());
|
||||
/// tx.send(()).unwrap();
|
||||
/// }
|
||||
/// // the lock is unlocked here when `data` goes out of scope.
|
||||
/// }).detach();
|
||||
/// }
|
||||
///
|
||||
/// rx.recv();
|
||||
/// rx.recv().unwrap();
|
||||
/// ```
|
||||
///
|
||||
/// To recover from a poisoned mutex:
|
||||
@@ -325,7 +325,7 @@ pub fn guard_poison<'a, T>(guard: &MutexGuard<'a, T>) -> &'a poison::Flag {
|
||||
mod test {
|
||||
use prelude::v1::*;
|
||||
|
||||
use comm::channel;
|
||||
use sync::mpsc::channel;
|
||||
use sync::{Arc, Mutex, StaticMutex, MUTEX_INIT, Condvar};
|
||||
use thread::Thread;
|
||||
|
||||
@@ -370,14 +370,14 @@ mod test {
|
||||
let (tx, rx) = channel();
|
||||
for _ in range(0, K) {
|
||||
let tx2 = tx.clone();
|
||||
Thread::spawn(move|| { inc(); tx2.send(()); }).detach();
|
||||
Thread::spawn(move|| { inc(); tx2.send(()).unwrap(); }).detach();
|
||||
let tx2 = tx.clone();
|
||||
Thread::spawn(move|| { inc(); tx2.send(()); }).detach();
|
||||
Thread::spawn(move|| { inc(); tx2.send(()).unwrap(); }).detach();
|
||||
}
|
||||
|
||||
drop(tx);
|
||||
for _ in range(0, 2 * K) {
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
}
|
||||
assert_eq!(unsafe {CNT}, J * K * 2);
|
||||
unsafe {
|
||||
@@ -398,7 +398,7 @@ mod test {
|
||||
let (tx, rx) = channel();
|
||||
let _t = Thread::spawn(move|| {
|
||||
// wait until parent gets in
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
let &(ref lock, ref cvar) = &*packet2.0;
|
||||
let mut lock = lock.lock().unwrap();
|
||||
*lock = true;
|
||||
@@ -407,7 +407,7 @@ mod test {
|
||||
|
||||
let &(ref lock, ref cvar) = &*packet.0;
|
||||
let mut lock = lock.lock().unwrap();
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
assert!(!*lock);
|
||||
while !*lock {
|
||||
lock = cvar.wait(lock).unwrap();
|
||||
@@ -421,7 +421,7 @@ mod test {
|
||||
let (tx, rx) = channel();
|
||||
|
||||
let _t = Thread::spawn(move || -> () {
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
let &(ref lock, ref cvar) = &*packet2.0;
|
||||
let _g = lock.lock().unwrap();
|
||||
cvar.notify_one();
|
||||
@@ -431,7 +431,7 @@ mod test {
|
||||
|
||||
let &(ref lock, ref cvar) = &*packet.0;
|
||||
let mut lock = lock.lock().unwrap();
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
while *lock == 1 {
|
||||
match cvar.wait(lock) {
|
||||
Ok(l) => {
|
||||
@@ -465,9 +465,9 @@ mod test {
|
||||
let lock = arc2.lock().unwrap();
|
||||
let lock2 = lock.lock().unwrap();
|
||||
assert_eq!(*lock2, 1);
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
});
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
||||
@@ -126,7 +126,7 @@ mod test {
|
||||
|
||||
use thread::Thread;
|
||||
use super::{ONCE_INIT, Once};
|
||||
use comm::channel;
|
||||
use sync::mpsc::channel;
|
||||
|
||||
#[test]
|
||||
fn smoke_once() {
|
||||
@@ -155,7 +155,7 @@ mod test {
|
||||
});
|
||||
assert!(run);
|
||||
}
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
}).detach();
|
||||
}
|
||||
|
||||
@@ -168,7 +168,7 @@ mod test {
|
||||
}
|
||||
|
||||
for _ in range(0u, 10) {
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -359,7 +359,7 @@ mod tests {
|
||||
use prelude::v1::*;
|
||||
|
||||
use rand::{mod, Rng};
|
||||
use comm::channel;
|
||||
use sync::mpsc::channel;
|
||||
use thread::Thread;
|
||||
use sync::{Arc, RWLock, StaticRWLock, RWLOCK_INIT};
|
||||
|
||||
@@ -404,7 +404,7 @@ mod tests {
|
||||
}).detach();
|
||||
}
|
||||
drop(tx);
|
||||
let _ = rx.recv_opt();
|
||||
let _ = rx.recv();
|
||||
unsafe { R.destroy(); }
|
||||
}
|
||||
|
||||
@@ -467,7 +467,7 @@ mod tests {
|
||||
Thread::yield_now();
|
||||
*lock = tmp + 1;
|
||||
}
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
}).detach();
|
||||
|
||||
// Readers try to catch the writer in the act
|
||||
@@ -486,7 +486,7 @@ mod tests {
|
||||
}
|
||||
|
||||
// Wait for writer to finish
|
||||
rx.recv();
|
||||
rx.recv().unwrap();
|
||||
let lock = arc.read().unwrap();
|
||||
assert_eq!(*lock, 10);
|
||||
}
|
||||
|
||||
@@ -108,7 +108,7 @@ mod tests {
|
||||
|
||||
use sync::Arc;
|
||||
use super::Semaphore;
|
||||
use comm::channel;
|
||||
use sync::mpsc::channel;
|
||||
use thread::Thread;
|
||||
|
||||
#[test]
|
||||
@@ -143,7 +143,7 @@ mod tests {
|
||||
let s2 = s.clone();
|
||||
let _t = Thread::spawn(move|| {
|
||||
s2.acquire();
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
});
|
||||
s.release();
|
||||
let _ = rx.recv();
|
||||
@@ -157,7 +157,7 @@ mod tests {
|
||||
let _ = rx.recv();
|
||||
});
|
||||
s.acquire();
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -171,11 +171,11 @@ mod tests {
|
||||
let _t = Thread::spawn(move|| {
|
||||
let _g = s2.access();
|
||||
let _ = rx2.recv();
|
||||
tx1.send(());
|
||||
tx1.send(()).unwrap();
|
||||
});
|
||||
let _g = s.access();
|
||||
tx2.send(());
|
||||
let _ = rx1.recv();
|
||||
tx2.send(()).unwrap();
|
||||
rx1.recv().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -186,12 +186,12 @@ mod tests {
|
||||
{
|
||||
let _g = s.access();
|
||||
Thread::spawn(move|| {
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
drop(s2.access());
|
||||
tx.send(());
|
||||
tx.send(()).unwrap();
|
||||
}).detach();
|
||||
rx.recv(); // wait for child to come alive
|
||||
rx.recv().unwrap(); // wait for child to come alive
|
||||
}
|
||||
rx.recv(); // wait for child to be done
|
||||
rx.recv().unwrap(); // wait for child to be done
|
||||
}
|
||||
}
|
||||
|
||||
@@ -12,9 +12,9 @@
|
||||
|
||||
use core::prelude::*;
|
||||
|
||||
use thread::Thread;
|
||||
use comm::{channel, Sender, Receiver};
|
||||
use sync::{Arc, Mutex};
|
||||
use sync::mpsc::{channel, Sender, Receiver};
|
||||
use thread::Thread;
|
||||
use thunk::Thunk;
|
||||
|
||||
struct Sentinel<'a> {
|
||||
@@ -55,7 +55,7 @@ impl<'a> Drop for Sentinel<'a> {
|
||||
/// ```rust
|
||||
/// use std::sync::TaskPool;
|
||||
/// use std::iter::AdditiveIterator;
|
||||
/// use std::comm::channel;
|
||||
/// use std::sync::mpsc::channel;
|
||||
///
|
||||
/// let pool = TaskPool::new(4u);
|
||||
///
|
||||
@@ -63,7 +63,7 @@ impl<'a> Drop for Sentinel<'a> {
|
||||
/// for _ in range(0, 8u) {
|
||||
/// let tx = tx.clone();
|
||||
/// pool.execute(move|| {
|
||||
/// tx.send(1u);
|
||||
/// tx.send(1u).unwrap();
|
||||
/// });
|
||||
/// }
|
||||
///
|
||||
@@ -101,7 +101,7 @@ impl TaskPool {
|
||||
pub fn execute<F>(&self, job: F)
|
||||
where F : FnOnce(), F : Send
|
||||
{
|
||||
self.jobs.send(Thunk::new(job));
|
||||
self.jobs.send(Thunk::new(job)).unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -115,7 +115,7 @@ fn spawn_in_pool(jobs: Arc<Mutex<Receiver<Thunk>>>) {
|
||||
// Only lock jobs for the time it takes
|
||||
// to get a job, not run it.
|
||||
let lock = jobs.lock().unwrap();
|
||||
lock.recv_opt()
|
||||
lock.recv()
|
||||
};
|
||||
|
||||
match message {
|
||||
@@ -134,7 +134,7 @@ fn spawn_in_pool(jobs: Arc<Mutex<Receiver<Thunk>>>) {
|
||||
mod test {
|
||||
use prelude::v1::*;
|
||||
use super::*;
|
||||
use comm::channel;
|
||||
use sync::mpsc::channel;
|
||||
|
||||
const TEST_TASKS: uint = 4u;
|
||||
|
||||
@@ -148,7 +148,7 @@ mod test {
|
||||
for _ in range(0, TEST_TASKS) {
|
||||
let tx = tx.clone();
|
||||
pool.execute(move|| {
|
||||
tx.send(1u);
|
||||
tx.send(1u).unwrap();
|
||||
});
|
||||
}
|
||||
|
||||
@@ -177,7 +177,7 @@ mod test {
|
||||
for _ in range(0, TEST_TASKS) {
|
||||
let tx = tx.clone();
|
||||
pool.execute(move|| {
|
||||
tx.send(1u);
|
||||
tx.send(1u).unwrap();
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user