diff --git a/Cargo.toml b/Cargo.toml index ba1cf61ed..06686a588 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,10 +16,10 @@ license = "MIT" #build = "booktests.rs" [dependencies] -abomonation = "0.5" +abomonation = { git = "https://github.com/frankmcsherry/abomonation" } abomonation_derive = "0.3" -timely_communication = "0.6" -byteorder="1" +bytes = { path = "./bytes" } +timely_communication = { path = "./communication"} time="0.1.34" [dev-dependencies] diff --git a/bytes/src/lib.rs b/bytes/src/lib.rs index 3a3a0cfd9..451550d00 100644 --- a/bytes/src/lib.rs +++ b/bytes/src/lib.rs @@ -25,7 +25,7 @@ //! drop(shared2); //! drop(shared3); //! -//! if let Ok(bytes) = shared4.try_recover() { +//! if let Ok(bytes) = shared4.try_recover::>() { //! assert_eq!(bytes[200..1024].to_vec(), [1u8;824].to_vec()); //! assert_eq!(bytes[60..100].to_vec(), [2u8;40].to_vec()); //! assert_eq!(bytes[100..200].to_vec(), [3u8;100].to_vec()); @@ -42,9 +42,10 @@ pub mod rc { use std::ops::{Deref, DerefMut}; use std::rc::Rc; + use std::any::Any; /// A thread-local byte buffer backed by a shared allocation. - pub struct Bytes> { + pub struct Bytes { /// Pointer to the start of this slice (not the allocation). ptr: *mut u8, /// Length of this slice. @@ -54,20 +55,24 @@ pub mod rc { /// Importantly, this is unavailable for as long as the struct exists, which may /// prevent shared access to ptr[0 .. len]. I'm not sure I understand Rust's rules /// enough to make a strong statement about this. - sequestered: Rc, + sequestered: Rc>, } - impl> Bytes { + impl Bytes { /// Create a new instance from a byte allocation. - pub fn from(bytes: B) -> Bytes { + pub fn from(bytes: B) -> Bytes where B: DerefMut+'static { - let mut rc = Rc::new(bytes); + let mut boxed = Box::new(bytes) as Box; + + let ptr = boxed.downcast_mut::().unwrap().as_mut_ptr(); + let len = boxed.downcast_ref::().unwrap().len(); + let sequestered = Rc::new(boxed); Bytes { - ptr: Rc::get_mut(&mut rc).unwrap().as_mut_ptr(), - len: rc.len(), - sequestered: rc, + ptr, + len, + sequestered, } } @@ -77,7 +82,7 @@ pub mod rc { /// /// This method uses an `unsafe` region to advance the pointer by `index`. It first /// tests `index` against `self.len`, which should ensure that the offset is in-bounds. - pub fn extract_to(&mut self, index: usize) -> Bytes { + pub fn extract_to(&mut self, index: usize) -> Bytes { assert!(index <= self.len); @@ -97,9 +102,9 @@ pub mod rc { /// /// This method either results in the underlying storage if it is uniquely held, or the /// input `Bytes` if it is not uniquely held. - pub fn try_recover(self) -> Result> { + pub fn try_recover(self) -> Result where B: DerefMut+'static { match Rc::try_unwrap(self.sequestered) { - Ok(bytes) => Ok(bytes), + Ok(bytes) => Ok(*bytes.downcast::().unwrap()), Err(rc) => Err(Bytes { ptr: self.ptr, len: self.len, @@ -109,14 +114,14 @@ pub mod rc { } } - impl> Deref for Bytes { + impl Deref for Bytes { type Target = [u8]; fn deref(&self) -> &[u8] { unsafe { ::std::slice::from_raw_parts(self.ptr, self.len) } } } - impl> DerefMut for Bytes { + impl DerefMut for Bytes { fn deref_mut(&mut self) -> &mut [u8] { unsafe { ::std::slice::from_raw_parts_mut(self.ptr, self.len) } } @@ -128,9 +133,10 @@ pub mod arc { use std::ops::{Deref, DerefMut}; use std::sync::Arc; + use std::any::Any; /// A thread-safe byte buffer backed by a shared allocation. - pub struct Bytes> { + pub struct Bytes { /// Pointer to the start of this slice (not the allocation). ptr: *mut u8, /// Length of this slice. @@ -140,20 +146,26 @@ pub mod arc { /// Importantly, this is unavailable for as long as the struct exists, which may /// prevent shared access to ptr[0 .. len]. I'm not sure I understand Rust's rules /// enough to make a strong statement about this. - sequestered: Arc, + sequestered: Arc>, } - impl> Bytes { + unsafe impl Send for Bytes { } + + impl Bytes { /// Create a new instance from a byte allocation. - pub fn from(bytes: B) -> Bytes { + pub fn from(bytes: B) -> Bytes where B : DerefMut+'static { - let mut arc = Arc::new(bytes); + let mut boxed = Box::new(bytes) as Box; + + let ptr = boxed.downcast_mut::().unwrap().as_mut_ptr(); + let len = boxed.downcast_ref::().unwrap().len(); + let sequestered = Arc::new(boxed); Bytes { - ptr: Arc::get_mut(&mut arc).unwrap().as_mut_ptr(), - len: arc.len(), - sequestered: arc, + ptr, + len, + sequestered, } } @@ -163,7 +175,7 @@ pub mod arc { /// /// This method uses an `unsafe` region to advance the pointer by `index`. It first /// tests `index` against `self.len`, which should ensure that the offset is in-bounds. - pub fn extract_to(&mut self, index: usize) -> Bytes { + pub fn extract_to(&mut self, index: usize) -> Bytes { assert!(index <= self.len); @@ -183,9 +195,28 @@ pub mod arc { /// /// This method either results in the underlying storage if it is uniquely held, or the /// input `Bytes` if it is not uniquely held. - pub fn try_recover(self) -> Result> { + /// + /// #Examples + /// + /// ``` + /// use bytes::arc::Bytes; + /// + /// let bytes = vec![0u8; 1024]; + /// let mut shared1 = Bytes::from(bytes); + /// let mut shared2 = shared1.extract_to(100); + /// let mut shared3 = shared1.extract_to(100); + /// let mut shared4 = shared2.extract_to(60); + /// + /// drop(shared1); + /// drop(shared2); + /// drop(shared4); + /// let recovered = shared3.try_recover::>().ok().expect("recovery failed"); + /// assert!(recovered.len() == 1024); + /// ``` + pub fn try_recover(self) -> Result where B: DerefMut+'static { + // println!("Trying recovery; strong count: {:?}", Arc::strong_count(&self.sequestered)); match Arc::try_unwrap(self.sequestered) { - Ok(bytes) => Ok(bytes), + Ok(bytes) => Ok(*bytes.downcast::().unwrap()), Err(arc) => Err(Bytes { ptr: self.ptr, len: self.len, @@ -193,18 +224,86 @@ pub mod arc { }), } } + + /// Regenerates the Bytes if it is uniquely held. + /// + /// If uniquely held, this method recovers the initial pointer and length + /// of the sequestered allocation and re-initialized the Bytes. The return + /// value indicates whether this occurred. + /// + /// #Examples + /// + /// ``` + /// use bytes::arc::Bytes; + /// + /// let bytes = vec![0u8; 1024]; + /// let mut shared1 = Bytes::from(bytes); + /// let mut shared2 = shared1.extract_to(100); + /// let mut shared3 = shared1.extract_to(100); + /// let mut shared4 = shared2.extract_to(60); + /// + /// drop(shared1); + /// drop(shared2); + /// drop(shared4); + /// assert!(shared3.try_regenerate::>()); + /// assert!(shared3.len() == 1024); + /// ``` + pub fn try_regenerate(&mut self) -> bool where B: DerefMut+'static { + if let Some(boxed) = Arc::get_mut(&mut self.sequestered) { + let downcast = boxed.downcast_mut::().expect("Downcast failed"); + self.ptr = downcast.as_mut_ptr(); + self.len = downcast.len(); + true + } + else { + false + } + } + + /// Attempts to merge adjacent slices from the same allocation. + /// + /// If the merge succeeds then `other.len` is added to `self` and the result is `Ok(())`. + /// If the merge fails self is unmodified and the result is `Err(other)`, returning the + /// bytes supplied as input. + /// + /// #Examples + /// + /// ``` + /// use bytes::arc::Bytes; + /// + /// let bytes = vec![0u8; 1024]; + /// let mut shared1 = Bytes::from(bytes); + /// let mut shared2 = shared1.extract_to(100); + /// let mut shared3 = shared1.extract_to(100); + /// let mut shared4 = shared2.extract_to(60); + /// + /// // memory in slabs [4, 2, 3, 1]: merge back in arbitrary order. + /// shared2.try_merge(shared3).ok().expect("Failed to merge 2 and 3"); + /// shared2.try_merge(shared1).ok().expect("Failed to merge 23 and 1"); + /// shared4.try_merge(shared2).ok().expect("Failed to merge 4 and 231"); + /// ``` + pub fn try_merge(&mut self, other: Bytes) -> Result<(), Bytes> { + use ::std::sync::Arc; + if Arc::ptr_eq(&self.sequestered, &other.sequestered) && ::std::ptr::eq(unsafe { self.ptr.offset(self.len as isize) }, other.ptr) { + self.len += other.len; + Ok(()) + } + else { + Err(other) + } + } } - impl> Deref for Bytes { + impl Deref for Bytes { type Target = [u8]; fn deref(&self) -> &[u8] { unsafe { ::std::slice::from_raw_parts(self.ptr, self.len) } } } - impl> DerefMut for Bytes { + impl DerefMut for Bytes { fn deref_mut(&mut self) -> &mut [u8] { unsafe { ::std::slice::from_raw_parts_mut(self.ptr, self.len) } } } -} +} \ No newline at end of file diff --git a/communication/Cargo.toml b/communication/Cargo.toml index 0e375b6cb..381e8eda7 100644 --- a/communication/Cargo.toml +++ b/communication/Cargo.toml @@ -17,11 +17,12 @@ default=["arg_parse"] arg_parse=["getopts"] [dependencies] -byteorder="1" +#byteorder="1" getopts={version="0.2.14", optional=true} time="0.1.34" -abomonation = "0.5" +abomonation = { git = "https://github.com/frankmcsherry/abomonation" } abomonation_derive = "0.3" +bytes = { path = "../bytes" } [profile.release] opt-level = 3 diff --git a/communication/examples/hello.rs b/communication/examples/hello.rs index 3c6919935..ef051142e 100644 --- a/communication/examples/hello.rs +++ b/communication/examples/hello.rs @@ -1,5 +1,8 @@ extern crate timely_communication; +use std::ops::Deref; +use timely_communication::Message; + fn main() { // extract the configuration from user-supplied arguments, initialize the computation. @@ -14,7 +17,7 @@ fn main() { // send typed data along each channel for i in 0 .. allocator.peers() { - senders[i].send(format!("hello, {}", i)); + senders[i].send(Message::from_typed(format!("hello, {}", i))); senders[i].done(); } @@ -22,10 +25,15 @@ fn main() { // we have to count down ourselves. let mut received = 0; while received < allocator.peers() { + + allocator.pre_work(); + if let Some(message) = receiver.recv() { - println!("worker {}: received: <{}>", allocator.index(), message); + println!("worker {}: received: <{}>", allocator.index(), message.deref()); received += 1; } + + allocator.post_work(); } allocator.index() diff --git a/communication/src/allocator/binary.rs b/communication/src/allocator/binary.rs index cb5cee117..52bb5062b 100644 --- a/communication/src/allocator/binary.rs +++ b/communication/src/allocator/binary.rs @@ -1,8 +1,8 @@ use std::sync::mpsc::{Sender, Receiver, channel}; use std::sync::Arc; -use {Allocate, Data, Push, Pull, Serialize}; -use allocator::Process; +use {Allocate, Data, Push, Pull}; +use allocator::{Message, Process}; use networking::MessageHeader; // A communicator intended for binary channels (networking, pipes, shared memory) @@ -26,8 +26,8 @@ impl Binary { impl Allocate for Binary { fn index(&self) -> usize { self.index } fn peers(&self) -> usize { self.peers } - fn allocate(&mut self) -> (Vec>>, Box>, Option) { - let mut pushers: Vec>> = Vec::new(); + fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { + let mut pushers: Vec>>> = Vec::new(); // we'll need process-local channels as well (no self-loop binary connection in this design; perhaps should allow) let inner_peers = self.inner.peers(); @@ -73,7 +73,7 @@ impl Allocate for Binary { sender: false, remote: None, }); - let pullable = Box::new(Puller::new(inner_recv, recv, logger)); + let pullable = Box::new(Puller::::new(inner_recv, recv, logger)) as Box>>; self.allocated += 1; @@ -99,18 +99,25 @@ impl Pusher { } } -impl Push for Pusher { - #[inline] fn push(&mut self, element: &mut Option) { +impl Push> for Pusher { + #[inline] fn push(&mut self, element: &mut Option>) { if let Some(ref mut element) = *element { + self.log_sender.when_enabled(|l| l.log(::logging::CommsEvent::Serialization(::logging::SerializationEvent { seq_no: Some(self.header.seqno), is_start: true, }))); + let mut bytes = Vec::new(); - ::into_bytes(element, &mut bytes); + element.into_bytes(&mut bytes); + // match element { + // Message::Binary(b) => bytes.extend(b.as_bytes().iter().cloned()), + // Message::Typed(t) => t.into_bytes(&mut bytes), + // }; let mut header = self.header; header.length = bytes.len(); self.sender.send((header, bytes)).ok(); // TODO : should be unwrap()? + self.log_sender.when_enabled(|l| l.log(::logging::CommsEvent::Serialization(::logging::SerializationEvent { seq_no: Some(self.header.seqno), is_start: true, @@ -121,37 +128,40 @@ impl Push for Pusher { } struct Puller { - inner: Box>, // inner pullable (e.g. intra-process typed queue) - current: Option, + inner: Box>>, // inner pullable (e.g. intra-process typed queue) + current: Option>, receiver: Receiver>, // source of serialized buffers log_sender: ::logging::CommsLogger, } impl Puller { - fn new(inner: Box>, receiver: Receiver>, log_sender: ::logging::CommsLogger) -> Puller { + fn new(inner: Box>>, receiver: Receiver>, log_sender: ::logging::CommsLogger) -> Puller { Puller { inner: inner, receiver: receiver, current: None, log_sender: log_sender } } } -impl Pull for Puller { +impl Pull> for Puller { #[inline] - fn pull(&mut self) -> &mut Option { + fn pull(&mut self) -> &mut Option> { let inner = self.inner.pull(); let log_sender = &self.log_sender; if inner.is_some() { inner } else { - self.current = self.receiver.try_recv().ok().map(|mut bytes| { + self.current = self.receiver.try_recv().ok().map(|bytes| { log_sender.when_enabled(|l| l.log( ::logging::CommsEvent::Serialization(::logging::SerializationEvent { seq_no: None, is_start: true, }))); - let result = ::from_bytes(&mut bytes); + + let bytes = ::bytes::arc::Bytes::from(bytes); + log_sender.when_enabled(|l| l.log( ::logging::CommsEvent::Serialization(::logging::SerializationEvent { seq_no: None, is_start: false, }))); - result + + unsafe { Message::from_bytes(bytes) } }); &mut self.current } diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index cfc01e115..708b3d2ab 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -3,40 +3,69 @@ //! This type is useful in settings where it is difficult to write code generic in `A: Allocate`, //! for example closures whose type arguments must be specified. -use allocator::{Allocate, Thread, Process, Binary}; +use allocator::{Allocate, AllocateBuilder, Message, Thread, Process}; +use allocator::zero_copy::allocator_process::{ProcessBuilder, ProcessAllocator}; +use allocator::zero_copy::allocator::{TcpBuilder, TcpAllocator}; + use {Push, Pull, Data}; /// Enumerates known implementors of `Allocate`. /// Passes trait method calls on to members. pub enum Generic { + /// Intra-thread allocator. Thread(Thread), + /// Inter-thread, intra-process allocator. Process(Process), - Binary(Binary), + /// Inter-thread, intra-process serializing allocator. + ProcessBinary(ProcessAllocator), + /// Inter-process allocator. + ZeroCopy(TcpAllocator), } impl Generic { /// The index of the worker out of `(0..self.peers())`. pub fn index(&self) -> usize { - match *self { - Generic::Thread(ref t) => t.index(), - Generic::Process(ref p) => p.index(), - Generic::Binary(ref b) => b.index(), + match self { + &Generic::Thread(ref t) => t.index(), + &Generic::Process(ref p) => p.index(), + &Generic::ProcessBinary(ref pb) => pb.index(), + &Generic::ZeroCopy(ref z) => z.index(), } } /// The number of workers. pub fn peers(&self) -> usize { - match *self { - Generic::Thread(ref t) => t.peers(), - Generic::Process(ref p) => p.peers(), - Generic::Binary(ref b) => b.peers(), + match self { + &Generic::Thread(ref t) => t.peers(), + &Generic::Process(ref p) => p.peers(), + &Generic::ProcessBinary(ref pb) => pb.peers(), + &Generic::ZeroCopy(ref z) => z.peers(), } } /// Constructs several send endpoints and one receive endpoint. - pub fn allocate(&mut self) -> (Vec>>, Box>, Option) { - match *self { - Generic::Thread(ref mut t) => t.allocate(), - Generic::Process(ref mut p) => p.allocate(), - Generic::Binary(ref mut b) => b.allocate(), + pub fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { + match self { + &mut Generic::Thread(ref mut t) => t.allocate(), + &mut Generic::Process(ref mut p) => p.allocate(), + &mut Generic::ProcessBinary(ref mut pb) => pb.allocate(), + &mut Generic::ZeroCopy(ref mut z) => z.allocate(), + } + } + /// Perform work before scheduling operators. + pub fn pre_work(&mut self) { + match self { + &mut Generic::Thread(ref mut t) => t.pre_work(), + &mut Generic::Process(ref mut p) => p.pre_work(), + &mut Generic::ProcessBinary(ref mut pb) => pb.pre_work(), + &mut Generic::ZeroCopy(ref mut z) => z.pre_work(), + } + } + /// Perform work after scheduling operators. + pub fn post_work(&mut self) { + match self { + &mut Generic::Thread(ref mut t) => t.post_work(), + &mut Generic::Process(ref mut p) => p.post_work(), + &mut Generic::ProcessBinary(ref mut pb) => pb.post_work(), + &mut Generic::ZeroCopy(ref mut z) => z.post_work(), } } } @@ -44,7 +73,39 @@ impl Generic { impl Allocate for Generic { fn index(&self) -> usize { self.index() } fn peers(&self) -> usize { self.peers() } - fn allocate(&mut self) -> (Vec>>, Box>, Option) { + fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { self.allocate() } + + fn pre_work(&mut self) { self.pre_work(); } + fn post_work(&mut self) { self.post_work(); } +} + + +/// Enumerations of constructable implementors of `Allocate`. +/// +/// The builder variants are meant to be `Send`, so that they can be moved across threads, +/// whereas the allocator they construct may not. As an example, the `ProcessBinary` type +/// contains `Rc` wrapped state, and so cannot itself be moved across threads. +pub enum GenericBuilder { + /// Builder for `Thread` allocator. + Thread(Thread), + /// Builder for `Process` allocator. + Process(Process), + /// Builder for `ProcessBinary` allocator. + ProcessBinary(ProcessBuilder), + /// Builder for `ZeroCopy` allocator. + ZeroCopy(TcpBuilder), +} + +impl AllocateBuilder for GenericBuilder { + type Allocator = Generic; + fn build(self) -> Generic { + match self { + GenericBuilder::Thread(t) => Generic::Thread(t), + GenericBuilder::Process(p) => Generic::Process(p), + GenericBuilder::ProcessBinary(pb) => Generic::ProcessBinary(pb.build()), + GenericBuilder::ZeroCopy(z) => Generic::ZeroCopy(z.build()), + } + } } diff --git a/communication/src/allocator/mod.rs b/communication/src/allocator/mod.rs index 3fed04758..1a8c46c4e 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -1,25 +1,44 @@ -//! Types and traits for the allocation of channels between threads, process, and computers. +//! Types and traits for the allocation of channels. pub use self::thread::Thread; pub use self::process::Process; -pub use self::binary::Binary; -pub use self::generic::Generic; +// pub use self::binary::Binary; +pub use self::generic::{Generic, GenericBuilder}; pub mod thread; pub mod process; -pub mod binary; +// pub mod binary; pub mod generic; -use {Data, Push, Pull}; +pub mod zero_copy; -// The Communicator trait presents the interface a worker has to the outside world. -// The worker can see its index, the total number of peers, and acquire channels to and from the other workers. -// There is an assumption that each worker performs the same channel allocation logic; things go wrong otherwise. +use {Data, Push, Pull, Message}; + +/// A proto-allocator, which implements `Send` and can be completed with `build`. +/// +/// This trait exists because some allocators contain non-Send elements, like `Rc` wrappers for +/// shared state. As such, what we actually need to create to initialize a computation are builders, +/// which we can then spawn in new threads each of which then construct their actual allocator. +pub trait AllocateBuilder : Send { + /// The type of built allocator. + type Allocator: Allocate; + /// Builds allocator, consumes self. + fn build(self) -> Self::Allocator; +} + +/// A type capable of allocating channels. +/// +/// There is some feature creep, in that this contains several convenience methods about the nature +/// of the allocated channels, and maintenance methods to ensure that they move records around. pub trait Allocate { /// The index of the worker out of `(0..self.peers())`. fn index(&self) -> usize; /// The number of workers. fn peers(&self) -> usize; /// Constructs several send endpoints and one receive endpoint. - fn allocate(&mut self) -> (Vec>>, Box>, Option); + fn allocate(&mut self) -> (Vec>>>, Box>>, Option); + /// Work performed before scheduling dataflows. + fn pre_work(&mut self) { } + /// Work performed after scheduling dataflows. + fn post_work(&mut self) { } } diff --git a/communication/src/allocator/process.rs b/communication/src/allocator/process.rs index ac3b79814..2068edc1a 100644 --- a/communication/src/allocator/process.rs +++ b/communication/src/allocator/process.rs @@ -1,11 +1,13 @@ +//! Typed inter-thread, intra-process channels. + use std::sync::{Arc, Mutex}; use std::any::Any; use std::sync::mpsc::{Sender, Receiver, channel}; -use allocator::{Allocate, Thread}; +use allocator::{Allocate, AllocateBuilder, Message, Thread}; use {Push, Pull}; -// A specific Communicator for inter-thread intra-process communication +/// An allocater for inter-thread, intra-process communication pub struct Process { inner: Thread, // inner Thread index: usize, // number out of peers @@ -15,7 +17,9 @@ pub struct Process { } impl Process { + /// Access the wrapped inner allocator. pub fn inner<'a>(&'a mut self) -> &'a mut Thread { &mut self.inner } + /// Allocate a list of connected intra-process allocators. pub fn new_vector(count: usize) -> Vec { let channels = Arc::new(Mutex::new(Vec::new())); (0 .. count).map(|index| Process { @@ -31,7 +35,7 @@ impl Process { impl Allocate for Process { fn index(&self) -> usize { self.index } fn peers(&self) -> usize { self.peers } - fn allocate(&mut self) -> (Vec>>, Box>, Option) { + fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { // ensure exclusive access to shared list of channels let mut channels = self.channels.lock().ok().expect("mutex error?"); @@ -41,7 +45,7 @@ impl Allocate for Process { let mut pushers = Vec::new(); let mut pullers = Vec::new(); for _ in 0..self.peers { - let (s, r): (Sender, Receiver) = channel(); + let (s, r): (Sender>, Receiver>) = channel(); pushers.push(Pusher { target: s }); pullers.push(Puller { source: r, current: None }); } @@ -54,25 +58,29 @@ impl Allocate for Process { channels.push(Box::new(to_box)); } + let vector = + channels[self.allocated] + .downcast_mut::<(Vec>>, Puller>)>>)>() + .expect("failed to correctly cast channel"); - if let Some(ref mut vector) = channels[self.allocated].downcast_mut::<(Vec>, Puller)>>)>() { - if let Some((send, recv)) = vector[self.index].take() { - self.allocated += 1; - let mut temp = Vec::new(); - for s in send.into_iter() { temp.push(Box::new(s) as Box>); } - return (temp, Box::new(recv) as Box>, None) - } - else { - panic!("channel already consumed"); - } - } - else { - panic!("failed to correctly cast channel"); - } + let (send, recv) = + vector[self.index] + .take() + .expect("channel already consumed"); + + self.allocated += 1; + let mut temp = Vec::new(); + for s in send.into_iter() { temp.push(Box::new(s) as Box>>); } + (temp, Box::new(recv) as Box>>, None) } } -// an observer wrapping a Rust channel +impl AllocateBuilder for Process { + type Allocator = Self; + fn build(self) -> Self { self } +} + +/// The push half of an intra-process channel. struct Pusher { target: Sender, } @@ -91,6 +99,7 @@ impl Push for Pusher { } } +/// The pull half of an intra-process channel. struct Puller { current: Option, source: Receiver, diff --git a/communication/src/allocator/thread.rs b/communication/src/allocator/thread.rs index 05729f0c6..19f3b3db2 100644 --- a/communication/src/allocator/thread.rs +++ b/communication/src/allocator/thread.rs @@ -1,23 +1,31 @@ +//! Intra-thread communication. + use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; -use allocator::Allocate; +use allocator::{Allocate, AllocateBuilder, Message}; use {Push, Pull}; -// The simplest communicator remains worker-local and just queues sent messages. +/// An allocator for intra-thread communication. pub struct Thread; impl Allocate for Thread { fn index(&self) -> usize { 0 } fn peers(&self) -> usize { 1 } - fn allocate(&mut self) -> (Vec>>, Box>, Option) { + fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { let (pusher, puller) = Thread::new(); (vec![Box::new(pusher)], Box::new(puller), None) } } +impl AllocateBuilder for Thread { + type Allocator = Self; + fn build(self) -> Self { self } +} + impl Thread { + /// Allocates a new pusher and puller pair. pub fn new() -> (Pusher, Puller) { let shared = Rc::new(RefCell::new((VecDeque::::new(), VecDeque::::new()))); (Pusher { target: shared.clone() }, Puller { source: shared, current: None }) @@ -25,7 +33,7 @@ impl Thread { } -// an observer wrapping a Rust channel +/// The push half of an intra-thread channel. pub struct Pusher { target: Rc, VecDeque)>>, } @@ -41,6 +49,7 @@ impl Push for Pusher { } } +/// The pull half of an intra-thread channel. pub struct Puller { current: Option, source: Rc, VecDeque)>>, @@ -50,12 +59,12 @@ impl Pull for Puller { #[inline(always)] fn pull(&mut self) -> &mut Option { let mut borrow = self.source.borrow_mut(); - if let Some(element) = self.current.take() { - // TODO : Arbitrary constant. - if borrow.1.len() < 16 { - borrow.1.push_back(element); - } - } + // if let Some(element) = self.current.take() { + // // TODO : Arbitrary constant. + // if borrow.1.len() < 16 { + // borrow.1.push_back(element); + // } + // } self.current = borrow.0.pop_front(); &mut self.current } diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs new file mode 100644 index 000000000..5a427a0f1 --- /dev/null +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -0,0 +1,219 @@ +//! Zero-copy allocator based on TCP. +use std::rc::Rc; +use std::cell::RefCell; +use std::collections::VecDeque; +// use std::sync::mpsc::{channel, Sender, Receiver}; + +use bytes::arc::Bytes; + +use networking::MessageHeader; + +use {Allocate, Data, Push, Pull}; +use allocator::{Message, Process}; + +use super::bytes_exchange::{BytesPull, SendEndpoint, MergeQueue, Signal}; +use super::push_pull::{Pusher, PullerInner}; + +/// Builds an instance of a TcpAllocator. +/// +/// Builders are required because some of the state in a `TcpAllocator` cannot be sent between +/// threads (specifically, the `Rc>` local channels). So, we must package up the state +/// shared between threads here, and then provide a method that will instantiate the non-movable +/// members once in the destination thread. +pub struct TcpBuilder { + inner: A, + index: usize, // number out of peers + peers: usize, // number of peer allocators. + sends: Vec, // for pushing bytes at remote processes. + recvs: Vec, // for pulling bytes from remote processes. + signal: Signal, +} + +/// Creates a vector of builders, sharing appropriate state. +pub fn new_vector( + my_process: usize, + threads: usize, + processes: usize) +// -> (Vec>, Vec>, Vec>) { +-> (Vec>, Vec<(Vec, Signal)>, Vec>) { + + // The results are a vector of builders, as well as the necessary shared state to build each + // of the send and receive communication threads, respectively. + + let worker_signals: Vec = (0 .. threads).map(|_| Signal::new()).collect(); + let network_signals: Vec = (0 .. processes-1).map(|_| Signal::new()).collect(); + + let worker_to_network: Vec> = (0 .. threads).map(|_| (0 .. processes-1).map(|p| MergeQueue::new(network_signals[p].clone())).collect()).collect(); + let network_to_worker: Vec> = (0 .. processes-1).map(|_| (0 .. threads).map(|t| MergeQueue::new(worker_signals[t].clone())).collect()).collect(); + + let worker_from_network: Vec> = (0 .. threads).map(|t| (0 .. processes-1).map(|p| network_to_worker[p][t].clone()).collect()).collect(); + let network_from_worker: Vec> = (0 .. processes-1).map(|p| (0 .. threads).map(|t| worker_to_network[t][p].clone()).collect()).collect(); + + let builders = + Process::new_vector(threads) + .into_iter() + .zip(worker_signals) + .zip(worker_to_network) + .zip(worker_from_network) + .enumerate() + .map(|(index, (((inner, signal), sends), recvs))| { + TcpBuilder { + inner, + index: my_process * threads + index, + peers: threads * processes, + sends, + recvs, + signal, + }}) + .collect(); + + let sends = network_from_worker.into_iter().zip(network_signals).collect(); + + (builders, sends, network_to_worker) +} + +impl TcpBuilder { + + /// Builds a `TcpAllocator`, instantiating `Rc>` elements. + pub fn build(self) -> TcpAllocator { + + let mut sends = Vec::new(); + for send in self.sends.into_iter() { + let sendpoint = SendEndpoint::new(send); + sends.push(Rc::new(RefCell::new(sendpoint))); + } + + TcpAllocator { + inner: self.inner, + index: self.index, + peers: self.peers, + allocated: 0, + _signal: self.signal, + staged: Vec::new(), + sends, + recvs: self.recvs, + to_local: Vec::new(), + } + } +} + +/// A TCP-based allocator for inter-process communication. +pub struct TcpAllocator { + + inner: A, // A non-serialized inner allocator for process-local peers. + + index: usize, // number out of peers + peers: usize, // number of peer allocators (for typed channel allocation). + allocated: usize, // indicates how many channels have been allocated (locally). + + _signal: Signal, + + staged: Vec, + + // sending, receiving, and responding to binary buffers. + sends: Vec>>>, // sends[x] -> goes to process x. + recvs: Vec, // recvs[x] <- from process x?. + to_local: Vec>>>, // to worker-local typed pullers. +} + +impl Allocate for TcpAllocator { + fn index(&self) -> usize { self.index } + fn peers(&self) -> usize { self.peers } + fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { + + let channel_id = self.allocated; + self.allocated += 1; + + // Result list of boxed pushers. + let mut pushes = Vec::>>>::new(); + + // Inner exchange allocations. + let inner_peers = self.inner.peers(); + let (mut inner_sends, inner_recv, _) = self.inner.allocate(); + + for target_index in 0 .. self.peers() { + + // TODO: crappy place to hardcode this rule. + let mut process_id = target_index / inner_peers; + + if process_id == self.index / inner_peers { + pushes.push(inner_sends.remove(0)); + } + else { + // message header template. + let header = MessageHeader { + channel: channel_id, + source: self.index, + target: target_index, + length: 0, + seqno: 0, + }; + + // create, box, and stash new process_binary pusher. + if process_id > self.index / inner_peers { process_id -= 1; } + pushes.push(Box::new(Pusher::new(header, self.sends[process_id].clone()))); + } + } + + while self.to_local.len() <= channel_id { + self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); + } + + let puller = Box::new(PullerInner::new(inner_recv, self.to_local[channel_id].clone())); + + (pushes, puller, None) + } + + // Perform preparatory work, most likely reading binary buffers from self.recv. + #[inline(never)] + fn pre_work(&mut self) { + + for recv in self.recvs.iter_mut() { + recv.drain_into(&mut self.staged); + } + + for mut bytes in self.staged.drain(..) { + + // We expect that `bytes` contains an integral number of messages. + // No splitting occurs across allocations. + while bytes.len() > 0 { + + if let Some(header) = MessageHeader::try_read(&mut bytes[..]) { + + // Get the header and payload, ditch the header. + let mut peel = bytes.extract_to(header.required_bytes()); + let _ = peel.extract_to(40); + + // Ensure that a queue exists. + // We may receive data before allocating, and shouldn't block. + while self.to_local.len() <= header.channel { + self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); + } + + // Introduce the binary slice into the operator input queue. + self.to_local[header.channel].borrow_mut().push_back(peel); + } + else { + println!("failed to read full header!"); + } + } + } + } + + // Perform postparatory work, most likely sending un-full binary buffers. + fn post_work(&mut self) { + // Publish outgoing byte ledgers. + for send in self.sends.iter_mut() { + send.borrow_mut().publish(); + } + + // OPTIONAL: Tattle on channels sitting on borrowed data. + // OPTIONAL: Perhaps copy borrowed data into owned allocation. + // for index in 0 .. self.to_local.len() { + // let len = self.to_local[index].borrow_mut().len(); + // if len > 0 { + // eprintln!("Warning: worker {}, undrained channel[{}].len() = {}", self.index, index, len); + // } + // } + } +} \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/allocator_process.rs b/communication/src/allocator/zero_copy/allocator_process.rs new file mode 100644 index 000000000..f7bc4ac81 --- /dev/null +++ b/communication/src/allocator/zero_copy/allocator_process.rs @@ -0,0 +1,194 @@ +//! Zero-copy allocator for intra-process serialized communication. + +use std::rc::Rc; +use std::cell::RefCell; +use std::collections::VecDeque; + +use bytes::arc::Bytes; + +use networking::MessageHeader; + +use {Allocate, Data, Push, Pull}; +use allocator::Message; + +use super::bytes_exchange::{BytesPull, SendEndpoint, MergeQueue, Signal}; + +use super::push_pull::{Pusher, Puller}; + +/// Builds an instance of a ProcessAllocator. +/// +/// Builders are required because some of the state in a `ProcessAllocator` cannot be sent between +/// threads (specifically, the `Rc>` local channels). So, we must package up the state +/// shared between threads here, and then provide a method that will instantiate the non-movable +/// members once in the destination thread. +pub struct ProcessBuilder { + index: usize, // number out of peers + peers: usize, // number of peer allocators. + sends: Vec, // for pushing bytes at remote processes. + recvs: Vec, // for pulling bytes from remote processes. + signal: Signal, +} + +impl ProcessBuilder { + /// Creates a vector of builders, sharing appropriate state. + /// + /// This method requires access to a byte exchanger, from which it mints channels. + pub fn new_vector(count: usize) -> Vec { + + let signals: Vec = (0 .. count).map(|_| Signal::new()).collect(); + + let mut sends = Vec::new(); + let mut recvs = Vec::new(); + for _ in 0 .. count { sends.push(Vec::new()); } + for _ in 0 .. count { recvs.push(Vec::new()); } + + for source in 0 .. count { + for target in 0 .. count { + let send = MergeQueue::new(signals[target].clone()); + let recv = send.clone(); + sends[source].push(send); + recvs[target].push(recv); + } + } + + sends.into_iter() + .zip(recvs) + .zip(signals) + .enumerate() + .map(|(index, ((sends, recvs), signal))| + ProcessBuilder { + index, + peers: count, + sends, + recvs, + signal, + } + ) + .collect() + } + + /// Builds a `ProcessAllocator`, instantiating `Rc>` elements. + pub fn build(self) -> ProcessAllocator { + + let mut sends = Vec::new(); + for send in self.sends.into_iter() { + let sendpoint = SendEndpoint::new(send); + sends.push(Rc::new(RefCell::new(sendpoint))); + } + + ProcessAllocator { + // inner: self.inner, + index: self.index, + peers: self.peers, + allocated: 0, + staged: Vec::new(), + sends, + recvs: self.recvs, + to_local: Vec::new(), + _signal: self.signal, + } + } +} + +/// A serializing allocator for inter-thread intra-process communication. +pub struct ProcessAllocator { + + index: usize, // number out of peers + peers: usize, // number of peer allocators (for typed channel allocation). + allocated: usize, // indicates how many channels have been allocated (locally). + + _signal: Signal, + // sending, receiving, and responding to binary buffers. + staged: Vec, + sends: Vec>>>, // sends[x] -> goes to process x. + recvs: Vec, // recvs[x] <- from process x?. + to_local: Vec>>>, // to worker-local typed pullers. +} + +impl Allocate for ProcessAllocator { + fn index(&self) -> usize { self.index } + fn peers(&self) -> usize { self.peers } + fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { + + let channel_id = self.allocated; + self.allocated += 1; + + let mut pushes = Vec::>>>::new(); + + for target_index in 0 .. self.peers() { + + // message header template. + let header = MessageHeader { + channel: channel_id, + source: self.index, + target: target_index, + length: 0, + seqno: 0, + }; + + // create, box, and stash new process_binary pusher. + pushes.push(Box::new(Pusher::new(header, self.sends[target_index].clone()))); + } + + while self.to_local.len() <= channel_id { + self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); + } + + let puller = Box::new(Puller::new(self.to_local[channel_id].clone())); + + (pushes, puller, None) + } + + // Perform preparatory work, most likely reading binary buffers from self.recv. + #[inline(never)] + fn pre_work(&mut self) { + + for recv in self.recvs.iter_mut() { + recv.drain_into(&mut self.staged); + } + + for mut bytes in self.staged.drain(..) { + + // We expect that `bytes` contains an integral number of messages. + // No splitting occurs across allocations. + while bytes.len() > 0 { + + if let Some(header) = MessageHeader::try_read(&mut bytes[..]) { + + // Get the header and payload, ditch the header. + let mut peel = bytes.extract_to(header.required_bytes()); + let _ = peel.extract_to(40); + + // Ensure that a queue exists. + // We may receive data before allocating, and shouldn't block. + while self.to_local.len() <= header.channel { + self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); + } + + // Introduce the binary slice into the operator input queue. + self.to_local[header.channel].borrow_mut().push_back(peel); + } + else { + println!("failed to read full header!"); + } + } + } + } + + // Perform postparatory work, most likely sending un-full binary buffers. + fn post_work(&mut self) { + // Publish outgoing byte ledgers. + for send in self.sends.iter_mut() { + send.borrow_mut().publish(); + } + + // OPTIONAL: Tattle on channels sitting on borrowed data. + // OPTIONAL: Perhaps copy borrowed data into owned allocation. + // for index in 0 .. self.to_local.len() { + // let len = self.to_local[index].borrow_mut().len(); + // if len > 0 { + // eprintln!("Warning: worker {}, undrained channel[{}].len() = {}", self.index, index, len); + // } + // } + } +} \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs new file mode 100644 index 000000000..8c6a6f1ed --- /dev/null +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -0,0 +1,194 @@ +//! Types and traits for sharing `Bytes`. + +use std::thread::Thread; +use std::sync::{Arc, Mutex, RwLock}; +use std::collections::VecDeque; + +use bytes::arc::Bytes; +use super::bytes_slab::BytesSlab; + +/// A target for `Bytes`. +pub trait BytesPush { + // /// Pushes bytes at the instance. + // fn push(&mut self, bytes: Bytes); + /// Pushes many bytes at the instance. + fn extend>(&mut self, iter: I); +} +/// A source for `Bytes`. +pub trait BytesPull { + // /// Pulls bytes from the instance. + // fn pull(&mut self) -> Option; + /// Drains many bytes from the instance. + fn drain_into(&mut self, vec: &mut Vec); +} + +/// A signal appropriate to wake a single thread. +/// +/// Internally this type uses thread parking and unparking, where the first thread to call +/// `wait` is registered as the thread to wake. Other threads that call `wait` will just be +/// parked without registering themselves, which would probably be a bug (of theirs). +#[derive(Clone)] +pub struct Signal { + thread: Arc>>, +} + +impl Signal { + /// Creates a new signal. + pub fn new() -> Self { + Signal { thread: Arc::new(RwLock::new(None)) } + } + /// Blocks unless or until ping is called. + pub fn wait(&self) { + // It is important not to block on the first call; doing so would fail to unblock + // from pings before the first call to wait. This may appear as a spurious wake-up, + // and ideally the caller is prepared for that. + if self.thread.read().expect("failed to read thread").is_none() { + *self.thread.write().expect("failed to set thread") = Some(::std::thread::current()) + } + else { + ::std::thread::park(); + } + } + /// Unblocks the current or next call to wait. + pub fn ping(&self) { + if let Some(thread) = self.thread.read().expect("failed to read thread").as_ref() { + thread.unpark(); + } + } +} + +use std::sync::atomic::{AtomicBool, Ordering}; +/// Who knows. +#[derive(Clone)] +pub struct MergeQueue { + queue: Arc>>, // queue of bytes. + dirty: Signal, // indicates whether there may be data present. + panic: Arc, +} + +impl MergeQueue { + /// Allocates a new queue with an associated signal. + pub fn new(signal: Signal) -> Self { + MergeQueue { + queue: Arc::new(Mutex::new(VecDeque::new())), + dirty: signal, + panic: Arc::new(AtomicBool::new(false)), + } + } + /// Indicates that all input handles to the queue have dropped. + pub fn is_complete(&self) -> bool { + if self.panic.load(Ordering::SeqCst) { panic!("MergeQueue poisoned."); } + Arc::strong_count(&self.queue) == 1 + } +} + +impl BytesPush for MergeQueue { + fn extend>(&mut self, iterator: I) { + + if self.panic.load(Ordering::SeqCst) { panic!("MergeQueue poisoned."); } + + // should lock once, extend; shouldn't re-lock. + let mut queue = self.queue.lock().expect("Failed to lock queue"); + let mut iterator = iterator.into_iter(); + if let Some(bytes) = iterator.next() { + let mut tail = if let Some(mut tail) = queue.pop_back() { + if let Err(bytes) = tail.try_merge(bytes) { + queue.push_back(::std::mem::replace(&mut tail, bytes)); + } + tail + } + else { + self.dirty.ping(); // only signal from empty to non-empty. + bytes + }; + + for bytes in iterator { + if let Err(bytes) = tail.try_merge(bytes) { + queue.push_back(::std::mem::replace(&mut tail, bytes)); + } + } + queue.push_back(tail); + } + } +} + +impl BytesPull for MergeQueue { + fn drain_into(&mut self, vec: &mut Vec) { + if self.panic.load(Ordering::SeqCst) { panic!("MergeQueue poisoned."); } + let mut queue = self.queue.lock().expect("unable to lock mutex"); + vec.extend(queue.drain(..)); + } +} + +// We want to ping in the drop because a channel closing can unblock a thread waiting on +// the next bit of data to show up. +impl Drop for MergeQueue { + fn drop(&mut self) { + // Propagate panic information, to distinguish between clean and unclean shutdown. + if ::std::thread::panicking() { + self.panic.store(true, Ordering::SeqCst); + } + else { + // TODO: Perhaps this aggressive ordering can relax orderings elsewhere. + if self.panic.load(Ordering::SeqCst) { panic!("MergeQueue poisoned."); } + } + // Drop the queue before pinging. + self.queue = Arc::new(Mutex::new(VecDeque::new())); + self.dirty.ping(); + } +} + + +/// A `BytesPush` wrapper which stages writes. +pub struct SendEndpoint { + send: P, + buffer: BytesSlab, +} + +impl SendEndpoint

{ + + /// Moves `self.buffer` into `self.send`, replaces with empty buffer. + fn send_buffer(&mut self) { + let valid_len = self.buffer.valid().len(); + if valid_len > 0 { + self.send.extend(Some(self.buffer.extract(valid_len))); + } + } + + /// Allocates a new `BytesSendEndpoint` from a shared queue. + pub fn new(queue: P) -> Self { + SendEndpoint { + send: queue, + buffer: BytesSlab::new(20), + } + } + /// Makes the next `bytes` bytes valid. + /// + /// The current implementation also sends the bytes, to ensure early visibility. + pub fn make_valid(&mut self, bytes: usize) { + self.buffer.make_valid(bytes); + self.send_buffer(); + } + /// Acquires a prefix of `self.empty()` of length at least `capacity`. + pub fn reserve(&mut self, capacity: usize) -> &mut [u8] { + + if self.buffer.empty().len() < capacity { + self.send_buffer(); + self.buffer.ensure_capacity(capacity); + } + + assert!(self.buffer.empty().len() >= capacity); + self.buffer.empty() + } + /// Marks all written data as valid, makes visible. + pub fn publish(&mut self) { + self.send_buffer(); + } +} + +impl Drop for SendEndpoint

{ + fn drop(&mut self) { + self.send_buffer(); + } +} + diff --git a/communication/src/allocator/zero_copy/bytes_slab.rs b/communication/src/allocator/zero_copy/bytes_slab.rs new file mode 100644 index 000000000..979f4c7d0 --- /dev/null +++ b/communication/src/allocator/zero_copy/bytes_slab.rs @@ -0,0 +1,86 @@ +//! A large binary allocation for writing and sharing. + +use bytes::arc::Bytes; + +/// A large binary allocation for writing and sharing. +/// +/// A bytes slab wraps a `Bytes` and maintains a valid (written) length, and supports writing after +/// this valid length, and extracting `Bytes` up to this valid length. Extracted bytes are enqueued +/// and checked for uniqueness in order to recycle them (once all shared references are dropped). +pub struct BytesSlab { + buffer: Bytes, // current working buffer. + in_progress: Vec>, // buffers shared with workers. + stash: Vec, // reclaimed and resuable buffers. + shift: usize, // current buffer allocation size. + valid: usize, // buffer[..valid] are valid bytes. +} + +impl BytesSlab { + /// Allocates a new `BytesSlab` with an initial size determined by a shift. + pub fn new(shift: usize) -> Self { + BytesSlab { + buffer: Bytes::from(vec![0u8; 1 << shift].into_boxed_slice()), + in_progress: Vec::new(), + stash: Vec::new(), + shift, + valid: 0, + } + } + /// The empty region of the slab. + pub fn empty(&mut self) -> &mut [u8] { + &mut self.buffer[self.valid..] + } + /// The valid region of the slab. + pub fn valid(&mut self) -> &mut [u8] { + &mut self.buffer[..self.valid] + } + /// Marks the next `bytes` bytes as valid. + pub fn make_valid(&mut self, bytes: usize) { + self.valid += bytes; + } + /// Extracts the first `bytes` valid bytes. + pub fn extract(&mut self, bytes: usize) -> Bytes { + debug_assert!(bytes <= self.valid); + self.valid -= bytes; + self.buffer.extract_to(bytes) + } + + /// Ensures that `self.empty().len()` is at least `capacity`. + /// + /// This method may retire the current buffer if it does not have enough space, in which case + /// it will copy any remaining contents into a new buffer. If this would not create enough free + /// space, the shift is increased until it is sufficient. + pub fn ensure_capacity(&mut self, capacity: usize) { + + if self.empty().len() < capacity { + + // Increase allocation if copy would be insufficient. + while self.valid + capacity > (1 << self.shift) { + self.shift += 1; + self.stash.clear(); // clear wrongly sized buffers. + self.in_progress.clear(); // clear wrongly sized buffers. + } + + // Attempt to reclaim shared slices. + if self.stash.is_empty() { + for shared in self.in_progress.iter_mut() { + if let Some(mut bytes) = shared.take() { + if bytes.try_regenerate::>() { + self.stash.push(bytes); + } + else { + *shared = Some(bytes); + } + } + } + self.in_progress.retain(|x| x.is_some()); + } + + let new_buffer = self.stash.pop().unwrap_or_else(|| Bytes::from(vec![0; 1 << self.shift].into_boxed_slice())); + let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + + self.buffer[.. self.valid].copy_from_slice(&old_buffer[.. self.valid]); + self.in_progress.push(Some(old_buffer)); + } + } +} \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/initialize.rs b/communication/src/allocator/zero_copy/initialize.rs new file mode 100644 index 000000000..b99605aa7 --- /dev/null +++ b/communication/src/allocator/zero_copy/initialize.rs @@ -0,0 +1,103 @@ +//! Network initialization. + +use std::sync::Arc; +use allocator::Process; +use networking::create_sockets; +use super::tcp::{send_loop, recv_loop}; +use super::allocator::{TcpBuilder, new_vector}; + +/// Join handles for send and receive threads. +/// +/// On drop, the guard joins with each of the threads to ensure that they complete +/// cleanly and send all necessary data. +pub struct CommsGuard { + send_guards: Vec<::std::thread::JoinHandle<()>>, + recv_guards: Vec<::std::thread::JoinHandle<()>>, +} + +impl Drop for CommsGuard { + fn drop(&mut self) { + for handle in self.send_guards.drain(..) { + handle.join().expect("Send thread panic"); + } + // println!("SEND THREADS JOINED"); + for handle in self.recv_guards.drain(..) { + handle.join().expect("Recv thread panic"); + } + // println!("RECV THREADS JOINED"); + } +} + +/// Initializes network connections +pub fn initialize_networking( + addresses: Vec, + my_index: usize, + threads: usize, + noisy: bool, + log_sender: Arc::logging::CommsLogger+Send+Sync>) +-> ::std::io::Result<(Vec>, CommsGuard)> { + + let processes = addresses.len(); + + let mut results = create_sockets(addresses, my_index, noisy)?; + + let (builders, remote_recvs, remote_sends) = new_vector(my_index, threads, processes); + let mut remote_recv_iter = remote_recvs.into_iter(); + let mut remote_send_iter = remote_sends.into_iter(); + + let mut send_guards = Vec::new(); + let mut recv_guards = Vec::new(); + + // for each process, if a stream exists (i.e. not local) ... + for index in 0..results.len() { + + if let Some(stream) = results[index].take() { + + let (remote_recv, signal) = remote_recv_iter.next().unwrap(); + + { + let log_sender = log_sender.clone(); + let stream = stream.try_clone()?; + let join_guard = + ::std::thread::Builder::new() + .name(format!("send thread {}", index)) + .spawn(move || { + + let log_sender = log_sender(::logging::CommsSetup { + process: my_index, + sender: true, + remote: Some(index), + }); + + send_loop(stream, remote_recv, signal, log_sender); + })?; + + send_guards.push(join_guard); + } + + let remote_send = remote_send_iter.next().unwrap(); + + { + // let remote_sends = remote_sends.clone(); + let log_sender = log_sender.clone(); + let stream = stream.try_clone()?; + let join_guard = + ::std::thread::Builder::new() + .name(format!("recv thread {}", index)) + .spawn(move || { + let log_sender = log_sender(::logging::CommsSetup { + process: my_index, + sender: false, + remote: Some(index), + }); + recv_loop(stream, remote_send, threads * my_index, log_sender); + })?; + + recv_guards.push(join_guard); + } + + } + } + + Ok((builders, CommsGuard { send_guards, recv_guards })) +} \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/mod.rs b/communication/src/allocator/zero_copy/mod.rs new file mode 100644 index 000000000..e4bf9e396 --- /dev/null +++ b/communication/src/allocator/zero_copy/mod.rs @@ -0,0 +1,13 @@ +//! Allocators based on serialized data which avoid copies. +//! +//! These allocators are based on `Abomonation` serialization, and its ability to deserialized +//! typed Rust data in-place. They surface references to data, often ultimately referencing the +//! raw binary data they initial received. + +pub mod bytes_slab; +pub mod bytes_exchange; +pub mod tcp; +pub mod allocator; +pub mod allocator_process; +pub mod initialize; +pub mod push_pull; \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/push_pull.rs b/communication/src/allocator/zero_copy/push_pull.rs new file mode 100644 index 000000000..5fe65667f --- /dev/null +++ b/communication/src/allocator/zero_copy/push_pull.rs @@ -0,0 +1,137 @@ +//! Push and Pull implementations wrapping serialized data. + +use std::rc::Rc; +use std::cell::RefCell; +use std::collections::VecDeque; + +use bytes::arc::Bytes; + +use networking::MessageHeader; + +use {Data, Push, Pull}; +use allocator::Message; + +use super::bytes_exchange::{BytesPush, SendEndpoint}; + +/// An adapter into which one may push elements of type `T`. +/// +/// This pusher has a fixed MessageHeader, and access to a SharedByteBuffer which it uses to +/// acquire buffers for serialization. +pub struct Pusher { + header: MessageHeader, + sender: Rc>>, + phantom: ::std::marker::PhantomData, +} + +impl Pusher { + /// Creates a new `Pusher` from a header and shared byte buffer. + pub fn new(header: MessageHeader, sender: Rc>>) -> Pusher { + Pusher { + header: header, + sender: sender, + phantom: ::std::marker::PhantomData, + } + } +} + +impl Push> for Pusher { + #[inline] + fn push(&mut self, element: &mut Option>) { + if let Some(ref mut element) = *element { + + // determine byte lengths and build header. + let mut header = self.header; + self.header.seqno += 1; + header.length = element.length_in_bytes(); + assert!(header.length > 0); + + // acquire byte buffer and write header, element. + let mut borrow = self.sender.borrow_mut(); + { + let mut bytes = borrow.reserve(header.required_bytes()); + assert!(bytes.len() >= header.required_bytes()); + let mut writer = &mut bytes; + header.write_to(writer).expect("failed to write header!"); + element.into_bytes(writer); + } + borrow.make_valid(header.required_bytes()); + } + } +} + +/// An adapter from which one can pull elements of type `T`. +/// +/// This type is very simple, and just consumes owned `Vec` allocations. It is +/// not the most efficient thing possible, which would probably instead be something +/// like the `bytes` crate (../bytes/) which provides an exclusive view of a shared +/// allocation. +pub struct Puller { + current: Option>, + receiver: Rc>>, // source of serialized buffers +} + +impl Puller { + /// Creates a new `Puller` instance from a shared queue. + pub fn new(receiver: Rc>>) -> Puller { + Puller { + current: None, + receiver, + } + } +} + +impl Pull> for Puller { + #[inline] + fn pull(&mut self) -> &mut Option> { + self.current = + self.receiver + .borrow_mut() + .pop_front() + .map(|bytes| unsafe { Message::from_bytes(bytes) }); + + &mut self.current + } +} + +/// An adapter from which one can pull elements of type `T`. +/// +/// This type is very simple, and just consumes owned `Vec` allocations. It is +/// not the most efficient thing possible, which would probably instead be something +/// like the `bytes` crate (../bytes/) which provides an exclusive view of a shared +/// allocation. +pub struct PullerInner { + inner: Box>>, // inner pullable (e.g. intra-process typed queue) + current: Option>, + receiver: Rc>>, // source of serialized buffers +} + +impl PullerInner { + /// Creates a new `PullerInner` instance from a shared queue. + pub fn new(inner: Box>>, receiver: Rc>>) -> Self { + PullerInner { + inner, + current: None, + receiver, + } + } +} + +impl Pull> for PullerInner { + #[inline] + fn pull(&mut self) -> &mut Option> { + + let inner = self.inner.pull(); + if inner.is_some() { + inner + } + else { + self.current = + self.receiver + .borrow_mut() + .pop_front() + .map(|bytes| unsafe { Message::from_bytes(bytes) }); + + &mut self.current + } + } +} \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/tcp.rs b/communication/src/allocator/zero_copy/tcp.rs new file mode 100644 index 000000000..e76e018ed --- /dev/null +++ b/communication/src/allocator/zero_copy/tcp.rs @@ -0,0 +1,142 @@ +//! + +use std::io::{Read, Write}; +use std::net::TcpStream; + +use networking::MessageHeader; + +use super::bytes_slab::BytesSlab; +use super::bytes_exchange::{MergeQueue, Signal}; + +/// Repeatedly reads from a TcpStream and carves out messages. +/// +/// The intended communication pattern is a sequence of (header, message)^* for valid +/// messages, followed by a header for a zero length message indicating the end of stream. +/// If the stream ends without being shut down, the receive thread panics in an attempt to +/// take down the computation and cause the failures to cascade. +pub fn recv_loop( + mut reader: TcpStream, + mut targets: Vec, + worker_offset: usize, + _log_sender: ::logging::CommsLogger) +{ + let mut buffer = BytesSlab::new(20); + + // Where we stash Bytes before handing them off. + let mut stageds = Vec::with_capacity(targets.len()); + for _ in 0 .. targets.len() { + stageds.push(Vec::new()); + } + + // Each loop iteration adds to `self.Bytes` and consumes all complete messages. + // At the start of each iteration, `self.buffer[..self.length]` represents valid + // data, and the remaining capacity is available for reading from the reader. + // + // Once the buffer fills, we need to copy uncomplete messages to a new shared + // allocation and place the existing Bytes into `self.in_progress`, so that it + // can be recovered once all readers have read what they need to. + let mut active = true; + while active { + + buffer.ensure_capacity(1); + + assert!(!buffer.empty().is_empty()); + + // Attempt to read some more bytes into self.buffer. + let read = match reader.read(&mut buffer.empty()) { + Ok(n) => n, + Err(x) => { + // We don't expect this, as socket closure results in Ok(0) reads. + println!("Error: {:?}", x); + 0 + }, + }; + + assert!(read > 0); + buffer.make_valid(read); + + // Consume complete messages from the front of self.buffer. + while let Some(header) = MessageHeader::try_read(buffer.valid()) { + + // TODO: Consolidate message sequences sent to the same worker? + let peeled_bytes = header.required_bytes(); + let bytes = buffer.extract(peeled_bytes); + + if header.length > 0 { + stageds[header.target - worker_offset].push(bytes); + } + else { + // Shutting down; confirm absence of subsequent data. + active = false; + if !buffer.valid().is_empty() { + panic!("Clean shutdown followed by data."); + } + buffer.ensure_capacity(1); + if reader.read(&mut buffer.empty()).expect("read failure") > 0 { + panic!("Clean shutdown followed by data."); + } + } + } + + // Pass bytes along to targets. + for (index, staged) in stageds.iter_mut().enumerate() { + use allocator::zero_copy::bytes_exchange::BytesPush; + targets[index].extend(staged.drain(..)); + } + } + // println!("RECVER EXITING"); +} + +/// Repeatedly sends messages into a TcpStream. +/// +/// The intended communication pattern is a sequence of (header, message)^* for valid +/// messages, followed by a header for a zero length message indicating the end of stream. +pub fn send_loop( + // TODO: Maybe we don't need BufWriter with consolidation in writes. + writer: TcpStream, + mut sources: Vec, + signal: Signal, + _log_sender: ::logging::CommsLogger) +{ + + let mut writer = ::std::io::BufWriter::with_capacity(1 << 16, writer); + let mut stash = Vec::new(); + + while !sources.is_empty() { + + // TODO: Round-robin better, to release resources fairly when overloaded. + for source in sources.iter_mut() { + use allocator::zero_copy::bytes_exchange::BytesPull; + source.drain_into(&mut stash); + } + + if stash.is_empty() { + // No evidence of records to read, but sources not yet empty (at start of loop). + // We are going to flush our writer (to move buffered data) and wait on a signal. + // We could get awoken by more data, a channel closing, or spuriously perhaps. + writer.flush().expect("Failed to flush writer."); + signal.wait(); + sources.retain(|source| !source.is_complete()); + } + else { + // TODO: Could do scatter/gather write here. + for bytes in stash.drain(..) { + writer.write_all(&bytes[..]).expect("Write failure in send_loop."); + } + } + } + + // Write final zero-length header. + // Would be better with meaningful metadata, but as this stream merges many + // workers it isn't clear that there is anything specific to write here. + let header = MessageHeader { + channel: 0, + source: 0, + target: 0, + length: 0, + seqno: 0, + }; + header.write_to(&mut writer).expect("Failed to write header!"); + writer.flush().expect("Failed to flush writer."); + writer.get_mut().shutdown(::std::net::Shutdown::Write).expect("Write shutdown failed"); +} \ No newline at end of file diff --git a/communication/src/drain.rs b/communication/src/drain.rs deleted file mode 100644 index e52f7991f..000000000 --- a/communication/src/drain.rs +++ /dev/null @@ -1,91 +0,0 @@ -//! Placeholder `Drain` implementation for `Vec` until it lands in stable Rust. - -use std::slice; -use std::ptr; - -pub trait DrainExt { - fn drain_temp(&mut self) -> Drain; -} - -impl DrainExt for Vec { - fn drain_temp(&mut self) -> Drain { - // Memory safety - // - // When the Drain is first created, it shortens the length of - // the source vector to make sure no uninitialized or moved-from elements - // are accessible at all if the Drain's destructor never gets to run. - // - // Drain will ptr::read out the values to remove. - // When finished, remaining tail of the vec is copied back to cover - // the hole, and the vector length is restored to the new length. - // - let len = self.len(); - let start = 0; - let end = len; - assert!(start <= end); - assert!(end <= len); - - unsafe { - // set self.vec length's to start, to be safe in case Drain is leaked - self.set_len(start); - // Use the borrow in the IterMut to indicate borrowing behavior of the - // whole Drain iterator (like &mut T). - let range_slice = slice::from_raw_parts_mut( - self.as_mut_ptr().offset(start as isize), - end - start); - Drain { - tail_start: end, - tail_len: len - end, - iter: range_slice.iter_mut(), - vec: self as *mut _, - } - } - } -} - -pub struct Drain<'a, T: 'a> { - /// Index of tail to preserve - tail_start: usize, - /// Length of tail - tail_len: usize, - /// Current remaining range to remove - iter: slice::IterMut<'a, T>, - vec: *mut Vec, -} - -impl<'a, T> Iterator for Drain<'a, T> { - type Item = T; - - #[inline] - fn next(&mut self) -> Option { - self.iter.next().map(|elt| - unsafe { - ptr::read(elt as *const _) - } - ) - } - - fn size_hint(&self) -> (usize, Option) { - self.iter.size_hint() - } -} - -impl<'a, T> Drop for Drain<'a, T> { - fn drop(&mut self) { - // exhaust self first - while let Some(_) = self.next() { } - - if self.tail_len > 0 { - unsafe { - let source_vec = &mut *self.vec; - // memmove back untouched tail, update to new length - let start = source_vec.len(); - let tail = self.tail_start; - let src = source_vec.as_ptr().offset(tail as isize); - let dst = source_vec.as_mut_ptr().offset(start as isize); - ptr::copy(src, dst, self.tail_len); - source_vec.set_len(start + self.tail_len); - } - } - } -} diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index 2db29c251..5e8fa9378 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -7,8 +7,11 @@ use std::io::BufRead; use getopts; use std::sync::Arc; -use allocator::{Thread, Process, Generic}; -use networking::initialize_networking; +use std::any::Any; + +use allocator::{AllocateBuilder, Thread, Process, Generic, GenericBuilder}; +// use allocator::zero_copy::allocator_process::ProcessBuilder; +use allocator::zero_copy::initialize::initialize_networking; /// Possible configurations for the communication infrastructure. pub enum Configuration { @@ -73,13 +76,21 @@ impl Configuration { } } -fn create_allocators(config: Configuration, logger: Arc::logging::CommsLogger+Send+Sync>) -> Result,String> { +type LogBuilder = Arc::logging::CommsLogger+Send+Sync>; + +fn create_allocators(config: Configuration, logger: LogBuilder) -> Result<(Vec, Box),String> { match config { - Configuration::Thread => Ok(vec![Generic::Thread(Thread)]), - Configuration::Process(threads) => Ok(Process::new_vector(threads).into_iter().map(Generic::Process).collect()), + Configuration::Thread => { + Ok((vec![GenericBuilder::Thread(Thread)], Box::new(()))) + // Ok(ProcessBinaryBuilder::new_vector(1).into_iter().map(|x| GenericBuilder::ProcessBinary(x)).collect()) + }, + Configuration::Process(threads) => { + Ok((Process::new_vector(threads).into_iter().map(|x| GenericBuilder::Process(x)).collect(), Box::new(()))) + // Ok((ProcessBuilder::new_vector(threads).into_iter().map(|x| GenericBuilder::ProcessBinary(x)).collect(), Box::new(()))) + }, Configuration::Cluster(threads, process, addresses, report) => { - if let Ok(stuff) = initialize_networking(addresses, process, threads, report, logger) { - Ok(stuff.into_iter().map(Generic::Binary).collect()) + if let Ok((stuff, guard)) = initialize_networking(addresses, process, threads, report, logger) { + Ok((stuff.into_iter().map(|x| GenericBuilder::ZeroCopy(x)).collect(), Box::new(guard))) } else { Err("failed to initialize networking".to_owned()) @@ -112,17 +123,21 @@ fn create_allocators(config: Configuration, logger: Arc 0 { +/// allocator.pre_work(); /// if let Some(message) = receiver.recv() { -/// println!("worker {}: received: <{}>", allocator.index(), message); +/// use std::ops::Deref; +/// println!("worker {}: received: <{}>", allocator.index(), message.deref()); /// expecting -= 1; /// } +/// allocator.post_work(); /// } /// /// // optionally, return something @@ -152,30 +167,91 @@ fn create_allocators(config: Configuration, logger: ArcT+Send+Sync+'static>( config: Configuration, - log_sender: Arc::logging::CommsLogger+Send+Sync>, + log_sender: LogBuilder, func: F, ) -> Result,String> { + let (allocators, others) = try!(create_allocators(config, log_sender)); + initialize_from(allocators, others, func) +} - let allocators = try!(create_allocators(config, log_sender)); +/// Initializes computation and runs a distributed computation. +/// +/// This version of `initialize` allows you to explicitly specify the allocators that +/// you want to use, by providing an explicit list of allocator builders. +/// +/// #Examples +/// ``` +/// use timely_communication::Allocate; +/// // configure for two threads, just one process. +/// let builders = timely_communication::allocator::process::Process::new_vector(2); +/// +/// // initializes communication, spawns workers +/// let guards = timely_communication::initialize_from(builders, Box::new(()), |mut allocator| { +/// println!("worker {} started", allocator.index()); +/// +/// // allocates pair of senders list and one receiver. +/// let (mut senders, mut receiver, _) = allocator.allocate(); +/// +/// // send typed data along each channel +/// use timely_communication::Message; +/// senders[0].send(Message::from_typed(format!("hello, {}", 0))); +/// senders[1].send(Message::from_typed(format!("hello, {}", 1))); +/// +/// // no support for termination notification, +/// // we have to count down ourselves. +/// let mut expecting = 2; +/// while expecting > 0 { +/// allocator.pre_work(); +/// if let Some(message) = receiver.recv() { +/// use std::ops::Deref; +/// println!("worker {}: received: <{}>", allocator.index(), message.deref()); +/// expecting -= 1; +/// } +/// allocator.post_work(); +/// } +/// +/// // optionally, return something +/// allocator.index() +/// }); +/// +/// // computation runs until guards are joined or dropped. +/// if let Ok(guards) = guards { +/// for guard in guards.join() { +/// println!("result: {:?}", guard); +/// } +/// } +/// else { println!("error in computation"); } +/// ``` +pub fn initialize_from( + builders: Vec, + others: Box, + func: F, +) -> Result,String> +where + A: AllocateBuilder+'static, + T: Send+'static, + F: Fn(::Allocator)->T+Send+Sync+'static +{ let logic = Arc::new(func); - let mut guards = Vec::new(); - for allocator in allocators.into_iter() { + for (index, builder) in builders.into_iter().enumerate() { let clone = logic.clone(); guards.push(try!(thread::Builder::new() - .name(format!("worker thread {}", allocator.index())) + .name(format!("worker thread {}", index)) .spawn(move || { - (*clone)(allocator) + let communicator = builder.build(); + (*clone)(communicator) }) .map_err(|e| format!("{:?}", e)))); } - Ok(WorkerGuards { guards: guards }) + Ok(WorkerGuards { guards, others }) } /// Maintains `JoinHandle`s for worker threads. pub struct WorkerGuards { - guards: Vec<::std::thread::JoinHandle> + guards: Vec<::std::thread::JoinHandle>, + others: Box, } impl WorkerGuards { @@ -190,7 +266,8 @@ impl WorkerGuards { impl Drop for WorkerGuards { fn drop(&mut self) { for guard in self.guards.drain(..) { - guard.join().unwrap(); + guard.join().expect("Worker panic"); } + // println!("WORKER THREADS JOINED"); } } diff --git a/communication/src/lib.rs b/communication/src/lib.rs index 42380c146..8ef80d051 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -30,17 +30,22 @@ //! let (mut senders, mut receiver, _) = allocator.allocate(); //! //! // send typed data along each channel -//! senders[0].send(format!("hello, {}", 0)); -//! senders[1].send(format!("hello, {}", 1)); +//! use timely_communication::Message; +//! senders[0].send(Message::from_typed(format!("hello, {}", 0))); +//! senders[1].send(Message::from_typed(format!("hello, {}", 1))); //! //! // no support for termination notification, //! // we have to count down ourselves. //! let mut expecting = 2; //! while expecting > 0 { +//! +//! allocator.pre_work(); //! if let Some(message) = receiver.recv() { -//! println!("worker {}: received: <{}>", allocator.index(), message); +//! use std::ops::Deref; +//! println!("worker {}: received: <{}>", allocator.index(), message.deref()); //! expecting -= 1; //! } +//! allocator.post_work(); //! } //! //! // optionally, return something @@ -69,55 +74,34 @@ //! result: Ok(1) //! ``` +#![forbid(missing_docs)] + #[cfg(feature = "arg_parse")] extern crate getopts; -extern crate byteorder; extern crate abomonation; #[macro_use] extern crate abomonation_derive; extern crate time; +extern crate bytes; + pub mod allocator; -mod networking; +pub mod networking; pub mod initialize; -mod drain; pub mod logging; +pub mod message; use std::any::Any; -use abomonation::{Abomonation, encode, decode}; + +use abomonation::Abomonation; pub use allocator::Generic as Allocator; pub use allocator::Allocate; -pub use initialize::{initialize, Configuration, WorkerGuards}; +pub use initialize::{initialize, initialize_from, Configuration, WorkerGuards}; +pub use message::Message; /// A composite trait for types that may be used with channels. -pub trait Data : Send+Any+Serialize+'static { } -impl Data for T { } - -/// Conversions to and from `Vec`. -/// -/// A type must implement this trait to move along the channels produced by an `A: Allocate`. -/// -/// A default implementation is provided for any `T: Abomonation+Clone`. -pub trait Serialize { - /// Append the binary representation of `self` to a vector of bytes. The `&mut self` argument - /// may be mutated, but the second argument should only be appended to. - fn into_bytes(&mut self, &mut Vec); - /// Recover an instance of Self from its binary representation. The `&mut Vec` argument may - /// be taken with `mem::replace` if it is needed. - fn from_bytes(&mut Vec) -> Self; -} - -// NOTE : this should be unsafe, because these methods are. -// NOTE : figure this out later. don't use for serious things. -impl Serialize for T { - fn into_bytes(&mut self, bytes: &mut Vec) { - // NOTE: `unwrap` should be ok, as Rust docs say writes to `Vec` do not fail. - unsafe { encode(self, bytes).unwrap(); } - } - fn from_bytes(bytes: &mut Vec) -> Self { - (* unsafe { decode::(bytes) }.unwrap().0).clone() - } -} +pub trait Data : Send+Any+Abomonation+'static { } +impl Data for T { } /// Pushing elements of type `T`. pub trait Push { diff --git a/communication/src/logging.rs b/communication/src/logging.rs index 43b9a0eee..9c5aa1e07 100644 --- a/communication/src/logging.rs +++ b/communication/src/logging.rs @@ -1,3 +1,5 @@ +//! Types and traits related to logging. + use std::rc::Rc; use std::cell::RefCell; @@ -21,11 +23,15 @@ pub fn get_precise_time_ns() -> u64 { const BUFFERING_LOGGER_CAPACITY: usize = 1024; +/// Either log data or and end marker. pub enum LoggerBatch { + /// Log data. Logs(Vec<(u64, S, L)>), + /// End of log marker. End, } +/// An active buffering logger. pub struct ActiveBufferingLogger { setup: S, buffer: Vec<(u64, S, L)>, @@ -33,6 +39,7 @@ pub struct ActiveBufferingLogger { } impl ActiveBufferingLogger { + /// Adds an element to the log. pub fn log(&mut self, l: L) { let ts = get_precise_time_ns(); self.buffer.push((ts, self.setup.clone(), l)); @@ -47,11 +54,13 @@ impl ActiveBufferingLogger { } } +/// A possibly inactive buffering logger. pub struct BufferingLogger { target: Option>>, } impl BufferingLogger { + /// Creates a new active buffering logger. pub fn new(setup: S, pusher: Box)->()>) -> Self { BufferingLogger { target: Some(RefCell::new(ActiveBufferingLogger { @@ -61,19 +70,19 @@ impl BufferingLogger { })), } } - + /// Creates a new inactive buffering logger. pub fn new_inactive() -> Rc> { Rc::new(BufferingLogger { target: None, }) } - + /// Invokes the closure if active, and ignores it if not active. pub fn when_enabled)->()>(&self, f: F) { if let Some(ref logger) = self.target { f(&mut *logger.borrow_mut()) } } - + /// Flushes the logs. pub fn flush(&self) { if let Some(ref logger) = self.target { logger.borrow_mut().flush(); @@ -94,13 +103,18 @@ impl Drop for BufferingLogger { /// A log writer for a communication thread. pub type CommsLogger = Rc>; +/// Configuration information about a communication thread. #[derive(Abomonation, Debug, PartialEq, Eq, Hash, Clone, Copy)] pub struct CommsSetup { + /// True when this is a send thread (or the receive thread). pub sender: bool, + /// The process id of the thread. pub process: usize, + /// The remote process id. pub remote: Option, } +/// A communication event, observing a message. #[derive(Abomonation, Debug, Clone)] pub struct CommunicationEvent { /// true for send event, false for receive event @@ -118,13 +132,18 @@ pub struct CommunicationEvent { #[derive(Abomonation, Debug, Clone)] /// Serialization pub struct SerializationEvent { + /// The sequence number of the message. pub seq_no: Option, + /// True when this is the start of serialization. pub is_start: bool, } +/// The types of communication events. #[derive(Debug, Clone, Abomonation)] pub enum CommsEvent { + /// A communication event. /* 0 */ Communication(CommunicationEvent), + /// A serialization event. /* 1 */ Serialization(SerializationEvent), } diff --git a/communication/src/message.rs b/communication/src/message.rs new file mode 100644 index 000000000..a972c9182 --- /dev/null +++ b/communication/src/message.rs @@ -0,0 +1,158 @@ +//! Types wrapping typed data. + +use bytes::arc::Bytes; +use abomonation::{Abomonation, abomonated::Abomonated, encode, measure}; + +/// Possible returned representations from a channel. +enum TypedOrBinary { + /// Binary representation. + Binary(Abomonated), + /// Rust typed instance. + Typed(T), +} + +/// Either an immutable or mutable reference. +pub enum RefOrMut<'a, T> where T: 'a { + /// An immutable reference. + Ref(&'a T), + /// A mutable reference. + Mut(&'a mut T), +} + +impl<'a, T: 'a> ::std::ops::Deref for RefOrMut<'a, T> { + type Target = T; + fn deref(&self) -> &Self::Target { + match self { + RefOrMut::Ref(reference) => reference, + RefOrMut::Mut(reference) => reference, + } + } +} + +impl<'a, T: Clone+'a> RefOrMut<'a, T> { + /// Extracts the contents of `self`, either by cloning or swapping. + /// + /// This consumes `self` because its contents are now in an unknown state. + pub fn swap<'b>(self, element: &'b mut T) { + match self { + RefOrMut::Ref(reference) => element.clone_from(reference), + RefOrMut::Mut(reference) => ::std::mem::swap(reference, element), + }; + } + /// Extracts the contents of `self`, either by cloning or swapping. + /// + /// This consumes `self` because its contents are now in an unknown state. + pub fn replace(self, mut element: T) -> T { + self.swap(&mut element); + element + } +} + +/// A wrapped message which may be either typed or binary data. +pub struct Message { + payload: TypedOrBinary, +} + +impl Message { + /// Wrap a typed item as a message. + pub fn from_typed(typed: T) -> Self { + Message { payload: TypedOrBinary::Typed(typed) } + } + /// Destructures and returns any typed data. + pub fn if_typed(self) -> Option { + match self.payload { + TypedOrBinary::Binary(_) => None, + TypedOrBinary::Typed(typed) => Some(typed), + } + } + /// Returns a mutable reference, if typed. + pub fn if_mut(&mut self) -> Option<&mut T> { + match &mut self.payload { + TypedOrBinary::Binary(_) => None, + TypedOrBinary::Typed(typed) => Some(typed), + } + } +} + +impl Message { + /// Wrap bytes as a message. + /// + /// #Safety + /// + /// This method is unsafe, in that `Abomonated::new()` is unsafe: it presumes that + /// the binary data can be safely decoded, which is unsafe for e.g. UTF8 data and + /// enumerations (perhaps among many other types). + pub unsafe fn from_bytes(bytes: Bytes) -> Self { + let abomonated = Abomonated::new(bytes).expect("Abomonated::new() failed."); + Message { payload: TypedOrBinary::Binary(abomonated) } + } + + /// Returns an immutable or mutable typed reference. + /// + /// This method returns a mutable reference if the underlying data are typed Rust + /// instances, which admit mutation, and it returns an immutable reference if the + /// data are serialized binary data. + pub fn as_ref_or_mut(&mut self) -> RefOrMut { + match &mut self.payload { + TypedOrBinary::Binary(bytes) => { RefOrMut::Ref(bytes) }, + TypedOrBinary::Typed(typed) => { RefOrMut::Mut(typed) }, + } + } + + /// The number of bytes required to serialize the data. + pub fn length_in_bytes(&self) -> usize { + match &self.payload { + TypedOrBinary::Binary(bytes) => { bytes.as_bytes().len() }, + TypedOrBinary::Typed(typed) => { measure(typed) }, + } + } + /// Writes the binary representation into `writer`. + pub fn into_bytes(&self, writer: &mut W) { + match &self.payload { + TypedOrBinary::Binary(bytes) => { + writer.write_all(bytes.as_bytes()).expect("Message::into_bytes(): write_all failed."); + }, + TypedOrBinary::Typed(typed) => { + unsafe { encode(typed, writer).expect("Message::into_bytes(): Abomonation::encode failed"); } + }, + } + } +} + +impl ::std::ops::Deref for Message { + type Target = T; + fn deref(&self) -> &Self::Target { + // TODO: In principle we have aready decoded, but let's go again + match &self.payload { + TypedOrBinary::Binary(bytes) => { bytes }, + TypedOrBinary::Typed(typed) => { typed }, + } + } +} + +impl Message { + /// Produces a typed instance of the wrapped element. + pub fn into_typed(self) -> T { + match self.payload { + TypedOrBinary::Binary(bytes) => bytes.clone(), + TypedOrBinary::Typed(instance) => instance, + } + } + /// Ensures the message is typed data and returns a mutable reference to it. + pub fn as_mut(&mut self) -> &mut T { + let mut decoded = None; + if let TypedOrBinary::Binary(bytes) = &mut self.payload { + decoded = Some(bytes.clone()); + } + if let Some(decoded) = decoded { + self.payload = TypedOrBinary::Typed(decoded); + } + if let TypedOrBinary::Typed(typed) = &mut self.payload { + typed + } + else { + unreachable!() + } + } + +} \ No newline at end of file diff --git a/communication/src/networking.rs b/communication/src/networking.rs index 5668684d5..ebfde5690 100644 --- a/communication/src/networking.rs +++ b/communication/src/networking.rs @@ -1,233 +1,62 @@ //! Networking code for sending and receiving fixed size `Vec` between machines. -use std::io::{Read, Write, Result, BufWriter}; +use std::io::{Read, Result}; use std::net::{TcpListener, TcpStream}; -use std::mem::size_of; use std::sync::Arc; -use std::sync::mpsc::{Sender, Receiver, channel}; +// use std::sync::mpsc::{Sender, Receiver, channel}; use std::thread; use std::thread::sleep; use std::time::Duration; -// TODO : Would be nice to remove this dependence -use byteorder::{LittleEndian, ReadBytesExt, WriteBytesExt}; - -use allocator::{Process, Binary}; -use drain::DrainExt; - -// TODO : Much of this only relates to BinaryWriter/BinaryReader based communication, not networking. -// TODO : Could be moved somewhere less networking-specific. +use abomonation::{encode, decode}; /// Framing data for each `Vec` transmission, indicating a typed channel, the source and /// destination workers, and the length in bytes. -#[derive(Copy, Clone)] +#[derive(Copy, Clone, Abomonation)] pub struct MessageHeader { - pub channel: usize, // index of channel - pub source: usize, // index of worker sending message - pub target: usize, // index of worker receiving message - pub length: usize, // number of bytes in message - pub seqno: usize, // sequence number + /// index of channel. + pub channel: usize, + /// index of worker sending message. + pub source: usize, + /// index of worker receiving message. + pub target: usize, + /// number of bytes in message. + pub length: usize, + /// sequence number. + pub seqno: usize, } impl MessageHeader { - // returns a header when there is enough supporting data - fn try_read(bytes: &mut &[u8]) -> Option { - if bytes.len() > size_of::() { - // capture original in case we need to rewind - let original = *bytes; - - // unclear what order struct initializers run in, so ... - let channel = bytes.read_u64::().unwrap() as usize; - let source = bytes.read_u64::().unwrap() as usize; - let target = bytes.read_u64::().unwrap() as usize; - let length = bytes.read_u64::().unwrap() as usize; - let seqno = bytes.read_u64::().unwrap() as usize; - - if bytes.len() >= length { - Some(MessageHeader { - channel: channel, - source: source, - target: target, - length: length, - seqno: seqno, - }) - } - else { - // rewind the reader - *bytes = original; - None - } - } - else { None } - } - - fn write_to(&self, writer: &mut W) -> Result<()> { - try!(writer.write_u64::(self.channel as u64)); - try!(writer.write_u64::(self.source as u64)); - try!(writer.write_u64::(self.target as u64)); - try!(writer.write_u64::(self.length as u64)); - try!(writer.write_u64::(self.seqno as u64)); - Ok(()) - } -} - -// structure in charge of receiving data from a Reader, for example the network -struct BinaryReceiver { - reader: R, // the generic reader - buffer: Vec, // current working buffer - length: usize, - targets: Switchboard>>, - log_sender: ::logging::CommsLogger, -} - -impl BinaryReceiver { - fn new( - reader: R, - channels: Receiver<((usize, - usize), - Sender>)>, - log_sender: ::logging::CommsLogger) -> BinaryReceiver { - BinaryReceiver { - reader: reader, - buffer: vec![0u8; 1 << 20], - length: 0, - targets: Switchboard::new(channels), - log_sender: log_sender, - } - } - - fn recv_loop(&mut self) { - loop { - - // if we've mostly filled our buffer and still can't read a whole message from it, - // we'll need more space / to read more at once. let's double the buffer! - if self.length >= self.buffer.len() / 2 { - self.buffer.extend(::std::iter::repeat(0u8).take(self.length)); - } - - // attempt to read some more bytes into our buffer - let read = self.reader.read(&mut self.buffer[self.length..]).unwrap_or(0); - self.length += read; - - let remaining = { - let mut slice = &self.buffer[..self.length]; - while let Some(header) = MessageHeader::try_read(&mut slice) { - self.log_sender.when_enabled(|l| l.log(::logging::CommsEvent::Communication(::logging::CommunicationEvent { - is_send: false, - comm_channel: header.channel, - source: header.source, - target: header.target, - seqno: header.seqno, - }))); - let h_len = header.length as usize; // length in bytes - let target = &mut self.targets.ensure(header.target, header.channel); - target.send(slice[..h_len].to_vec()).unwrap(); - slice = &slice[h_len..]; + /// Returns a header when there is enough supporting data + #[inline(always)] + pub fn try_read(bytes: &mut [u8]) -> Option { + unsafe { decode::(bytes) } + .and_then(|(header, remaining)| { + if remaining.len() >= header.length { + Some(header.clone()) } - - slice.len() - }; - - // we consumed bytes, must shift to beginning. - // this should optimize to copy_overlapping; - // would just do that if it weren't unsafe =/ - if remaining < self.length { - let offset = self.length - remaining; - for index in 0..remaining { - self.buffer[index] = self.buffer[index + offset]; + else { + None } - self.length = remaining; - } - } + }) } -} -// structure in charge of sending data to a Writer, for example the network -struct BinarySender { - writer: W, - sources: Receiver<(MessageHeader, Vec)>, - log_sender: ::logging::CommsLogger, -} - -impl BinarySender { - fn new(writer: W, sources: Receiver<(MessageHeader, Vec)>, log_sender: ::logging::CommsLogger) -> BinarySender { - BinarySender { - writer: writer, - sources: sources, - log_sender: log_sender, - } + /// Writes the header as binary data. + #[inline(always)] + pub fn write_to(&self, writer: &mut W) -> ::std::io::Result<()> { + unsafe { encode(self, writer) } } - fn send_loop(&mut self) { - let mut stash = Vec::new(); - - // This blocks until there is data to receive, and should return an `Err(_)` once all matching `Sender` instances - // have been dropped. This should allow the method to return, and the send thread to cleanly exit on shutdown. - while let Ok((header, buffer)) = self.sources.recv() { - - stash.push((header, buffer)); - - // collect any additional outstanding data to send - while let Ok((header, buffer)) = self.sources.try_recv() { - stash.push((header, buffer)); - } - - for (header, mut buffer) in stash.drain_temp() { - assert!(header.length == buffer.len()); - self.log_sender.when_enabled(|l| l.log(::logging::CommsEvent::Communication(::logging::CommunicationEvent { - is_send: true, - comm_channel: header.channel, - source: header.source, - target: header.target, - seqno: header.seqno, - }))); - header.write_to(&mut self.writer).unwrap(); - self.writer.write_all(&buffer[..]).unwrap(); - buffer.clear(); - } - - self.writer.flush().unwrap(); // <-- because writer is buffered - } + /// The number of bytes required for the header and data. + #[inline(always)] + pub fn required_bytes(&self) -> usize { + ::std::mem::size_of::() + self.length } } -struct Switchboard { - source: Receiver<((usize, usize), T)>, - buffer: Vec>>, -} - -impl Switchboard { - pub fn new(source: Receiver<((usize, usize), T)>) -> Switchboard { - Switchboard { - source: source, - buffer: Vec::new(), - } - } - - pub fn ensure(&mut self, a: usize, b: usize) -> &mut T { - - // ensure a, b, c are indexable - while self.buffer.len() <= a { self.buffer.push(Vec::new()); } - while self.buffer[a].len() <= b { self.buffer[a].push(None); } +/// Creates socket connections from a list of host addresses. +pub fn create_sockets(addresses: Vec, my_index: usize, noisy: bool) -> Result>> { - // repeatedly pull instructions until a, b, c exists. - while self.buffer[a][b].is_none() { - let ((x, y), s) = self.source.recv().unwrap(); - while self.buffer.len() <= x { self.buffer.push(Vec::new()); } - while self.buffer[x].len() <= y { self.buffer[x].push(None); } - self.buffer[x][y] = Some(s); - } - - // we've just ensured that this is not None - self.buffer[a][b].as_mut().unwrap() - } -} - -/// Initializes network connections -pub fn initialize_networking( - addresses: Vec, my_index: usize, threads: usize, noisy: bool, log_sender: Arc::logging::CommsLogger+Send+Sync>) -> Result> { - - let processes = addresses.len(); let hosts1 = Arc::new(addresses); let hosts2 = hosts1.clone(); @@ -235,97 +64,18 @@ pub fn initialize_networking( let await_task = thread::spawn(move || await_connections(hosts2, my_index, noisy)); let mut results = start_task.join().unwrap()?; - results.push(None); let to_extend = await_task.join().unwrap()?; results.extend(to_extend.into_iter()); if noisy { println!("worker {}:\tinitialization complete", my_index) } - let mut readers = Vec::new(); // handles to the BinaryReceivers (to present new channels) - let mut senders = Vec::new(); // destinations for serialized data (to send serialized data) - - // for each process, if a stream exists (i.e. not local) ... - for index in 0..results.len() { - if let Some(stream) = results[index].take() { - - let (reader_channels_s, reader_channels_r) = channel(); - let (sender_channels_s, sender_channels_r) = channel(); - - readers.push(reader_channels_s); - senders.push(sender_channels_s); - - { - let log_sender = log_sender.clone(); - let stream = stream.try_clone()?; - // start senders and receivers associated with this stream - let join_guard = - thread::Builder::new().name(format!("send thread {}", index)) - .spawn(move || { - let log_sender = log_sender(::logging::CommsSetup { - process: my_index, - sender: true, - remote: Some(index), - }); - let mut sender = BinarySender::new(BufWriter::with_capacity(1 << 20, stream), - sender_channels_r, - log_sender); - sender.send_loop() - })?; - - // Forget the guard, so that the send thread is not detached from the main thread. - // This ensures that main thread awaits the completion of the send thread, and all - // of its transmissions, before exiting and potentially stranding other workers. - ::std::mem::forget(join_guard); - } - - { - let log_sender = log_sender.clone(); - let stream = stream.try_clone()?; - let _join_guard = - thread::Builder::new().name(format!("recv thread {}", index)) - .spawn(move || { - let log_sender = log_sender(::logging::CommsSetup { - process: my_index, - sender: false, - remote: Some(index), - }); - let mut recver = BinaryReceiver::new(stream, - reader_channels_r, - log_sender); - recver.recv_loop() - })?; - - // We do not mem::forget the join_guard here, because we deem there to be no harm - // in closing the process and abandoning the receiver thread. All worker threads - // will have exited, and we don't expect that continuing to read has a benefit. - // We could introduce a "shutdown" message into the "protocol" which would confirm - // a clear conclusion to the interaction. - } - - } - } - - let proc_comms = Process::new_vector(threads); - - let mut results = Vec::new(); - for (index, proc_comm) in proc_comms.into_iter().enumerate() { - results.push(Binary { - inner: proc_comm, - index: my_index * threads + index, - peers: threads * processes, - allocated: 0, - readers: readers.clone(), - senders: senders.clone(), - log_sender: log_sender.clone(), - }); - } - Ok(results) } -// result contains connections [0, my_index - 1]. -fn start_connections(addresses: Arc>, my_index: usize, noisy: bool) -> Result>> { + +/// Result contains connections [0, my_index - 1]. +pub fn start_connections(addresses: Arc>, my_index: usize, noisy: bool) -> Result>> { let mut results: Vec<_> = (0..my_index).map(|_| None).collect(); for index in 0..my_index { let mut connected = false; @@ -333,7 +83,7 @@ fn start_connections(addresses: Arc>, my_index: usize, noisy: bool) match TcpStream::connect(&addresses[index][..]) { Ok(mut stream) => { stream.set_nodelay(true).expect("set_nodelay call failed"); - try!(stream.write_u64::(my_index as u64)); + unsafe { encode(&(my_index as u64), &mut stream) }.expect("failed to encode/send worker index"); results[index as usize] = Some(stream); if noisy { println!("worker {}:\tconnection to worker {}", my_index, index); } connected = true; @@ -349,15 +99,17 @@ fn start_connections(addresses: Arc>, my_index: usize, noisy: bool) Ok(results) } -// result contains connections [my_index + 1, addresses.len() - 1]. -fn await_connections(addresses: Arc>, my_index: usize, noisy: bool) -> Result>> { +/// Result contains connections [my_index + 1, addresses.len() - 1]. +pub fn await_connections(addresses: Arc>, my_index: usize, noisy: bool) -> Result>> { let mut results: Vec<_> = (0..(addresses.len() - my_index - 1)).map(|_| None).collect(); let listener = try!(TcpListener::bind(&addresses[my_index][..])); for _ in (my_index + 1) .. addresses.len() { - let mut stream = try!(listener.accept()).0; + let mut stream = listener.accept()?.0; stream.set_nodelay(true).expect("set_nodelay call failed"); - let identifier = try!(stream.read_u64::()) as usize; + let mut buffer = [0u8;8]; + stream.read_exact(&mut buffer).expect("failed to read worker index"); + let identifier = unsafe { decode::(&mut buffer) }.expect("failed to decode worker index").0.clone() as usize; results[identifier - my_index - 1] = Some(stream); if noisy { println!("worker {}:\tconnection from worker {}", my_index, identifier); } } diff --git a/examples/bfs.rs b/examples/bfs.rs index 50a65d26c..80cde4c93 100644 --- a/examples/bfs.rs +++ b/examples/bfs.rs @@ -64,7 +64,7 @@ fn main() { // receive edges, start to sort them input1.for_each(|time, data| { notify.notify_at(time.retain()); - edge_list.push(data.replace_with(Vec::new())); + edge_list.push(data.replace(Vec::new())); }); // receive (node, worker) pairs, note any new ones. @@ -74,7 +74,7 @@ fn main() { notify.notify_at(time.retain()); Vec::new() }) - .push(data.replace_with(Vec::new())); + .push(data.replace(Vec::new())); }); notify.for_each(|time, _num, _notify| { diff --git a/examples/broadcast.rs b/examples/broadcast.rs deleted file mode 100644 index 4a2ae288a..000000000 --- a/examples/broadcast.rs +++ /dev/null @@ -1,29 +0,0 @@ -extern crate timely; - -use timely::dataflow::operators::*; - -fn main() { - timely::execute_from_args(std::env::args().skip(1), move |worker| { - let index = worker.index(); - let peers = worker.peers(); - - let mut input = worker.dataflow::(|scope| { - - let (input, stream) = scope.new_input(); - - stream - .broadcast() - .inspect(move |x| println!("{} -> {:?}", index, x)); - - input - }); - - for round in 0u64..10 { - if (round as usize) % peers == index { - input.send(round); - } - input.advance_to(round + 1); - worker.step(); - } - }).unwrap(); -} diff --git a/examples/hashjoin.rs b/examples/hashjoin.rs index c97c3b970..27be5eece 100644 --- a/examples/hashjoin.rs +++ b/examples/hashjoin.rs @@ -40,12 +40,16 @@ fn main() { let mut map1 = HashMap::>::new(); let mut map2 = HashMap::>::new(); + let mut vector1 = Vec::new(); + let mut vector2 = Vec::new(); + move |input1, input2, output| { // Drain first input, check second map, update first map. input1.for_each(|time, data| { + data.swap(&mut vector1); let mut session = output.session(&time); - for (key, val1) in data.drain(..) { + for (key, val1) in vector1.drain(..) { if let Some(values) = map2.get(&key) { for val2 in values.iter() { session.give((val1.clone(), val2.clone())); @@ -58,8 +62,9 @@ fn main() { // Drain second input, check first map, update second map. input2.for_each(|time, data| { + data.swap(&mut vector2); let mut session = output.session(&time); - for (key, val2) in data.drain(..) { + for (key, val2) in vector2.drain(..) { if let Some(values) = map1.get(&key) { for val1 in values.iter() { session.give((val1.clone(), val2.clone())); diff --git a/examples/pagerank.rs b/examples/pagerank.rs index be2412b27..1a4670789 100644 --- a/examples/pagerank.rs +++ b/examples/pagerank.rs @@ -50,18 +50,23 @@ fn main() { let mut diffs = Vec::new(); // for received but un-acted upon deltas. let mut delta = Vec::new(); + let mut edge_vec = Vec::new(); + let mut rank_vec = Vec::new(); + let timer = ::std::time::Instant::now(); move |input1, input2, output| { // hold on to edge changes until it is time. input1.for_each(|time, data| { - edge_stash.entry(time.retain()).or_insert(Vec::new()).extend(data.drain(..)); + data.swap(&mut edge_vec); + edge_stash.entry(time.retain()).or_insert(Vec::new()).extend(edge_vec.drain(..)); }); // hold on to rank changes until it is time. input2.for_each(|time, data| { - rank_stash.entry(time.retain()).or_insert(Vec::new()).extend(data.drain(..)); + data.swap(&mut rank_vec); + rank_stash.entry(time.retain()).or_insert(Vec::new()).extend(rank_vec.drain(..)); }); let frontiers = &[input1.frontier(), input2.frontier()]; diff --git a/examples/pingpong.rs b/examples/pingpong.rs index 800f6cce9..50c722439 100644 --- a/examples/pingpong.rs +++ b/examples/pingpong.rs @@ -5,13 +5,16 @@ use timely::dataflow::operators::*; fn main() { let iterations = std::env::args().nth(1).unwrap().parse::().unwrap(); + let elements = std::env::args().nth(2).unwrap().parse::().unwrap(); // initializes and runs a timely dataflow - timely::execute_from_args(std::env::args().skip(2), move |worker| { + timely::execute_from_args(std::env::args().skip(3), move |worker| { let index = worker.index(); + let peers = worker.peers(); worker.dataflow(move |scope| { let (helper, cycle) = scope.loop_variable(iterations, 1); - (0..1).take(if index == 0 { 1 } else { 0 }) + (0 .. elements) + .filter(move |&x| (x as usize) % peers == index) .to_stream(scope) .concat(&cycle) .exchange(|&x| x) diff --git a/examples/wordcount.rs b/examples/wordcount.rs index 63a387b16..f039c8344 100644 --- a/examples/wordcount.rs +++ b/examples/wordcount.rs @@ -33,7 +33,7 @@ fn main() { while let Some((time, data)) = input.next() { queues.entry(time.retain()) .or_insert(Vec::new()) - .push(data.take()); + .push(data.replace(Vec::new())); } for (key, val) in queues.iter_mut() { diff --git a/src/dataflow/channels/message.rs b/src/dataflow/channels/message.rs deleted file mode 100644 index 9d535b21f..000000000 --- a/src/dataflow/channels/message.rs +++ /dev/null @@ -1,217 +0,0 @@ -//! Core type for communicating a collection of `D: Data` records. -//! -//! `Message` is meant to be treated as a `Vec`, with the caveat that it may wrap either -//! typed `Vec` data or binary `Vec` data that have not yet been deserialized. The type -//! implements `Deref` and `DerefMut` with `Target = Vec`, whose implementations accommodate -//! the possibly serialized representation. -use timely_communication::{Serialize, Push}; -use std::ops::{Deref, DerefMut}; -use abomonation::{Abomonation, encode, decode, measure}; - -/// A serializable representation of timestamped data. -#[derive(Clone)] -pub struct Message { - /// The timestamp associated with the message. - pub time: T, - /// The data in the message. - pub data: Content, - /// The source worker. - pub from: usize, - /// A sequence number for this worker-to-worker stream. - pub seq: usize, -} - -impl Message { - /// Allocates a new message from a time, content, source worker id, and sequence number. - #[inline] - pub fn new(time: T, data: Content, from: usize, seq: usize) -> Message { - Message { - time, - data, - from, - seq, - } - } -} - -// Implementation required to get different behavior out of communication fabric. -impl Serialize for Message { - #[inline] - fn into_bytes(&mut self, bytes: &mut Vec) { - - // Reserve the minimal number of bytes to prevent the need to resize. - let bytes_needed = measure(&self.time) + measure(&self.from) + measure(&self.seq) + measure(self.data.deref()); - bytes.reserve(bytes_needed); - - // Almost like serializing `self`, except `self.data` is special. - unsafe { encode(&self.time, bytes).unwrap(); } - unsafe { encode(&self.from, bytes).unwrap(); } - unsafe { encode(&self.seq, bytes).unwrap(); } - let vec: &Vec = self.data.deref(); - unsafe { encode(vec, bytes).unwrap(); } - } - #[inline] - fn from_bytes(bytes: &mut Vec) -> Self { - // This method *steals* `bytes` and avoids allocation and copying. - let mut bytes = ::std::mem::replace(bytes, Vec::new()); - let x_len = bytes.len(); - let (time, from, seq, offset) = { - let (t,r) = unsafe { decode::(&mut bytes) }.unwrap(); - let (&f,r) = unsafe { decode::(r) }.unwrap(); - let (&s,r) = unsafe { decode::(r) }.unwrap(); - let o = x_len - r.len(); - ((*t).clone(), f, s, o) - }; - - // The call to `decode` should mean we can freely dereference. - let length = unsafe { decode::>(&mut bytes[offset..]) }.unwrap().0.len(); - Message::new(time, Content::Bytes(bytes, offset, length), from, seq) - } -} - -/// A batch of data, represented either as serialized bytes or typed Rust objects. -#[derive(Clone)] -pub enum Content { - /// A serialized representation of data. - /// - /// This representation may be efficiently observed as shared references, - /// but may only more expensively be converted into typed data. - Bytes(Vec, usize, usize), - /// Typed data, which may be efficiently mutated or claimed for ownership. - Typed(Vec), -} - -// ALLOC : This Drop implementation gets *very* angry if we drop allocated data. -// ALLOC : It probably shouldn't be used in practice, but should help track down who is being -// ALLOC : bad about respecting allocated memory. -// impl Drop for Message { -// match self.contents { -// Content::Bytes(bytes, _, _) => { assert!(bytes.capacity() == 0); } -// Content::Typed(typed) => { assert!(typed.capacity() == 0); } -// } -// } - -impl Content { - /// Gives ownership of the content, leaving an empty vector behind. - pub fn take(&mut self) -> Content { - ::std::mem::replace(self, Content::Typed(Vec::new())) - } - - /// Default number of elements in a typed allocated message. This could vary as a function of - /// `std::mem::size_of::()`, so is left as a method rather than a constant. - #[inline] - pub fn default_length() -> usize { 1024 } - - /// The length of the underlying typed vector. - /// - /// The length is tracked without needing to deserialize the data, so that this method can be - /// called even for `D` that do not implement `Serializable`. - #[inline] - pub fn len(&self) -> usize { - match *self { - Content::Bytes(_, _, length) => length, - Content::Typed(ref data) => data.len(), - } - } - - /// Constructs a `Message` from typed data, replacing its argument with `Vec::new()`. - #[inline] - pub fn from_typed(typed: &mut Vec) -> Content { - Content::Typed(::std::mem::replace(typed, Vec::new())) - } - - /// Returns the typed vector, cleared, or a Vec::new() if the data are binary (and drops them - /// on the floor, I guess! Ouch. - /// ALLOC : dropping of binary data. likely called only by persons who pushed typed data on, - /// ALLOC : so perhaps not all that common. Could put a panic! here just for fun! :D - /// ALLOC : casual dropping of contents of `data`, which might have allocated memory. - #[inline] - pub fn into_typed(self) -> Vec { - match self { - Content::Bytes(_,_,_) => Vec::new(), - Content::Typed(mut data) => { data.clear(); data }, - } - } - - /// Pushes `buffer` into `pusher`, ensuring that `buffer` remains valid once returned. - #[inline(always)] - pub fn push_at)>>(buffer: &mut Vec, time: T, pusher: &mut P) { - - let data = Content::from_typed(buffer); - let mut message = Some((time, data)); - - pusher.push(&mut message); - - if let Some((_, Content::Typed(mut typed))) = message { - typed.clear(); - *buffer = typed; - } - else { - // println!("re-allocating (nothing returned)"); - *buffer = Vec::with_capacity(Content::::default_length()); - } - - // TODO : Assert failing, but not sure if is bug when deser can make arbitrary lengths - // TODO : in clone. Revisit! - // assert!(buffer.capacity() == Content::::default_length()); - if buffer.capacity() != Content::::default_length() { - // println!("re-allocating (wrong size)"); - *buffer = Vec::with_capacity(Content::::default_length()); - } - } -} - -impl Content { - /// Swaps the contents with another vector. - /// - /// This method is a convenient way to take ownership of the underlying data without - /// needing to import the `DerefMut` trait and write horrible gunk. - #[inline] - pub fn replace_with(&mut self, other: Vec) -> Vec { - ::std::mem::replace(self.deref_mut(), other) - } -} - - -impl Deref for Content { - type Target = Vec; - #[inline] - fn deref(&self) -> &Vec { - match *self { - Content::Bytes(ref bytes, offset, _length) => { - // verify wasn't actually safe, it turns out... - unsafe { ::std::mem::transmute(bytes.get_unchecked(offset)) } - }, - Content::Typed(ref data) => data, - } - } -} - -// TODO : Rather than .clone() the decoded data, we should try and re-rig serialization so that the -// TODO : underlying byte array can just be handed to Vec::from_raw_parts, cloning any owned data. -// TODO : I think we would need to make sure that the byte array had the right alignment, so that -// TODO : when the Vec is eventually dropped we don't de-allocate the wrong number of bytes. -// TODO : This requires mucking with the Abomonation code, as it doesn't currently let you step in -// TODO : and skip copying the 24 byte Vec struct first. We'd also have to bake in the typed length -// TODO : somewhere outside of this serialized hunk of data. -impl DerefMut for Content { - #[inline] - fn deref_mut(&mut self) -> &mut Vec { - let value = if let Content::Bytes(ref mut bytes, offset, _length) = *self { - let data: &Vec = unsafe { ::std::mem::transmute(bytes.get_unchecked(offset)) }; - // let (data, _) = verify::>(&bytes[offset..]).unwrap(); - // ALLOC : clone() will allocate a Vec and maybe more. - Some(Content::Typed((*data).clone())) - } - else { None }; - - if let Some(contents) = value { - *self = contents; - } - - if let Content::Typed(ref mut data) = *self { - data - } - else { unreachable!() } - } -} diff --git a/src/dataflow/channels/mod.rs b/src/dataflow/channels/mod.rs index 191623b95..5d7ee9790 100644 --- a/src/dataflow/channels/mod.rs +++ b/src/dataflow/channels/mod.rs @@ -1,13 +1,60 @@ //! Structured communication between timely dataflow operators. -pub use self::message::Message; -pub use self::message::Content; +use communication::Push; /// A collection of types that may be pushed at. pub mod pushers; /// A collection of types that may be pulled from. pub mod pullers; -/// Types relating to batching of timestamped records. -pub mod message; /// Parallelization contracts, describing how data must be exchanged between operators. pub mod pact; + +/// The input to and output from timely dataflow communication channels. +pub type Bundle = ::communication::Message>; + +/// A serializable representation of timestamped data. +#[derive(Clone, Abomonation)] +pub struct Message { + /// The timestamp associated with the message. + pub time: T, + /// The data in the message. + pub data: Vec, + /// The source worker. + pub from: usize, + /// A sequence number for this worker-to-worker stream. + pub seq: usize, +} + +impl Message { + /// Default buffer size. + pub fn default_length() -> usize { + 1024 + } + + /// Creates a new message instance from arguments. + pub fn new(time: T, data: Vec, from: usize, seq: usize) -> Self { + Message { time, data, from, seq } + } + + /// Forms a message, and pushes contents at `pusher`. + #[inline(always)] + pub fn push_at>>(buffer: &mut Vec, time: T, pusher: &mut P) { + + let data = ::std::mem::replace(buffer, Vec::new()); + let message = Message::new(time, data, 0, 0); + let mut bundle = Some(Bundle::from_typed(message)); + + pusher.push(&mut bundle); + + if let Some(message) = bundle { + if let Some(message) = message.if_typed() { + *buffer = message.data; + buffer.clear(); + } + } + + // TODO: Unclear we always want this here. + if buffer.capacity() != Self::default_length() { + *buffer = Vec::with_capacity(Self::default_length()); + } + }} \ No newline at end of file diff --git a/src/dataflow/channels/pact.rs b/src/dataflow/channels/pact.rs index 216cea82c..70e18ed8a 100644 --- a/src/dataflow/channels/pact.rs +++ b/src/dataflow/channels/pact.rs @@ -9,13 +9,13 @@ use std::marker::PhantomData; -use timely_communication::{Allocate, Push, Pull, Data}; -use timely_communication::allocator::Thread; -use timely_communication::allocator::thread::Pusher as ThreadPusher; -use timely_communication::allocator::thread::Puller as ThreadPuller; +use communication::{Allocate, Push, Pull, Data}; +use communication::allocator::Thread; +use communication::allocator::thread::Pusher as ThreadPusher; +use communication::allocator::thread::Puller as ThreadPuller; use dataflow::channels::pushers::Exchange as ExchangePusher; -use dataflow::channels::{Message, Content}; +use super::{Bundle, Message}; use logging::Logger; @@ -24,9 +24,9 @@ use abomonation::Abomonation; /// A `ParallelizationContract` allocates paired `Push` and `Pull` implementors. pub trait ParallelizationContract { /// Type implementing `Push` produced by this pact. - type Pusher: Push<(T, Content)>+'static; + type Pusher: Push>+'static; /// Type implementing `Pull` produced by this pact. - type Puller: Pull<(T, Content)>+'static; + type Puller: Pull>+'static; /// Allocates a matched pair of push and pull endpoints implementing the pact. fn connect(self, allocator: &mut A, identifier: usize, logging: Logger) -> (Self::Pusher, Self::Puller); } @@ -34,15 +34,13 @@ pub trait ParallelizationContract { /// A direct connection pub struct Pipeline; impl ParallelizationContract for Pipeline { - // TODO: These two could mention types in communication::thread, but they are currently private. - type Pusher = Pusher>>; - type Puller = Puller>>; + type Pusher = LogPusher>>; + type Puller = LogPuller>>; fn connect(self, allocator: &mut A, identifier: usize, logging: Logger) -> (Self::Pusher, Self::Puller) { // ignore `&mut A` and use thread allocator - let (pusher, puller) = Thread::new::>(); - - (Pusher::new(pusher, allocator.index(), allocator.index(), identifier, None, logging.clone()), - Puller::new(puller, allocator.index(), identifier, None, logging.clone())) + let (pusher, puller) = Thread::new::>(); + (LogPusher::new(pusher, allocator.index(), allocator.index(), identifier, None, logging.clone()), + LogPuller::new(puller, allocator.index(), identifier, None, logging.clone())) } } @@ -62,40 +60,40 @@ implu64> Exchange { implu64+'static> ParallelizationContract for Exchange { // TODO: The closure in the type prevents us from naming it. // Could specialize `ExchangePusher` to a time-free version. - type Pusher = Box)>>; - type Puller = Puller>>>; + type Pusher = Box>>; + type Puller = Box>>; fn connect(self, allocator: &mut A, identifier: usize, logging: Logger) -> (Self::Pusher, Self::Puller) { let (senders, receiver, channel_id) = allocator.allocate::>(); - let senders = senders.into_iter().enumerate().map(|(i,x)| Pusher::new(x, allocator.index(), i, identifier, channel_id, logging.clone())).collect::>(); - (Box::new(ExchangePusher::new(senders, move |_, d| (self.hash_func)(d))), Puller::new(receiver, allocator.index(), identifier, channel_id, logging.clone())) + let senders = senders.into_iter().enumerate().map(|(i,x)| LogPusher::new(x, allocator.index(), i, identifier, channel_id, logging.clone())).collect::>(); + (Box::new(ExchangePusher::new(senders, move |_, d| (self.hash_func)(d))), Box::new(LogPuller::new(receiver, allocator.index(), identifier, channel_id, logging.clone()))) } } -/// An exchange between multiple observers by time and data -pub struct TimeExchangeu64+'static> { hash_func: F, phantom: PhantomData<(T, D)>, } -implu64> TimeExchange { - /// Allocates a new `TimeExchange` pact from a distribution function. - pub fn new(func: F) -> TimeExchange { - TimeExchange { - hash_func: func, - phantom: PhantomData, - } - } -} - -implu64+'static> ParallelizationContract for TimeExchange { - type Pusher = ExchangePusher>>>, F>; - type Puller = Puller>>>; - fn connect(self, allocator: &mut A, identifier: usize, logging: Logger) -> (Self::Pusher, Self::Puller) { - let (senders, receiver, channel_id) = allocator.allocate::>(); - let senders = senders.into_iter().enumerate().map(|(i,x)| Pusher::new(x, allocator.index(), i, identifier, channel_id, logging.clone())).collect::>(); - (ExchangePusher::new(senders, self.hash_func), Puller::new(receiver, allocator.index(), identifier, channel_id, logging.clone())) - } -} +// /// An exchange between multiple observers by time and data +// pub struct TimeExchangeu64+'static> { hash_func: F, phantom: PhantomData<(T, D)>, } +// implu64> TimeExchange { +// /// Allocates a new `TimeExchange` pact from a distribution function. +// pub fn new(func: F) -> TimeExchange { +// TimeExchange { +// hash_func: func, +// phantom: PhantomData, +// } +// } +// } + +// implu64+'static> ParallelizationContract for TimeExchange { +// type Pusher = ExchangePusher>>>>, F>; +// type Puller = Puller>>>>; +// fn connect(self, allocator: &mut A, identifier: usize, logging: Logger) -> (Self::Pusher, Self::Puller) { +// let (senders, receiver, channel_id) = allocator.allocate::>(); +// let senders = senders.into_iter().enumerate().map(|(i,x)| Pusher::new(x, allocator.index(), i, identifier, channel_id, logging.clone())).collect::>(); +// (ExchangePusher::new(senders, self.hash_func), Puller::new(receiver, allocator.index(), identifier, channel_id, logging.clone())) +// } +// } /// Wraps a `Message` pusher to provide a `Push<(T, Content)>`. -pub struct Pusher>> { +pub struct LogPusher>> { pusher: P, channel: usize, comm_channel: Option, @@ -105,10 +103,10 @@ pub struct Pusher>> { phantom: ::std::marker::PhantomData<(T, D)>, logging: Logger, } -impl>> Pusher { +impl>> LogPusher { /// Allocates a new pusher. pub fn new(pusher: P, source: usize, target: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { - Pusher { + LogPusher { pusher, channel, comm_channel, @@ -121,19 +119,14 @@ impl>> Pusher { } } -impl>> Push<(T, Content)> for Pusher { +impl>> Push> for LogPusher { #[inline(always)] - fn push(&mut self, pair: &mut Option<(T, Content)>) { - if let Some((time, data)) = pair.take() { - - let length = data.len(); + fn push(&mut self, pair: &mut Option>) { + if let Some(bundle) = pair { + let length = bundle.data.len(); let counter = self.counter; - - let mut message = Some(Message::new(time, data, self.source, self.counter)); self.counter += 1; - self.pusher.push(&mut message); - *pair = message.map(|x| (x.time, x.data)); self.logging.when_enabled(|l| l.log(::logging::TimelyEvent::Messages(::logging::MessagesEvent { is_send: true, @@ -144,60 +137,58 @@ impl>> Push<(T, Content)> for Pusher { seq_no: counter, length, }))); - - // Log something about (index, counter, time?, length?); } - else { self.pusher.done(); } + + self.pusher.push(pair); } } /// Wraps a `Message` puller to provide a `Pull<(T, Content)>`. -pub struct Puller>> { +pub struct LogPuller>> { puller: P, - current: Option<(T, Content)>, channel: usize, comm_channel: Option, - counter: usize, index: usize, + phantom: ::std::marker::PhantomData<(T, D)>, logging: Logger, } -impl>> Puller { +impl>> LogPuller { /// Allocates a new `Puller`. pub fn new(puller: P, index: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { - Puller { + LogPuller { puller, channel, comm_channel, - current: None, - counter: 0, index, + phantom: ::std::marker::PhantomData, logging, } } } -impl>> Pull<(T, Content)> for Puller { +impl>> Pull> for LogPuller { #[inline(always)] - fn pull(&mut self) -> &mut Option<(T, Content)> { - let mut previous = self.current.take().map(|(time, data)| Message::new(time, data, self.index, self.counter)); - self.counter += 1; + fn pull(&mut self) -> &mut Option> { + + let result = self.puller.pull(); - ::std::mem::swap(&mut previous, self.puller.pull()); + if let Some(bundle) = result { - if let Some(message) = previous.as_ref() { + let channel = self.channel; + let comm_channel = self.comm_channel; + let target = self.index; self.logging.when_enabled(|l| l.log(::logging::TimelyEvent::Messages(::logging::MessagesEvent { is_send: false, - channel: self.channel, - comm_channel: self.comm_channel, - source: message.from, - target: self.index, - seq_no: message.seq, - length: message.data.len(), + channel, + comm_channel, + source: bundle.from, + target, + seq_no: bundle.seq, + length: bundle.data.len(), }))); } - self.current = previous.map(|message| (message.time, message.data)); - &mut self.current + result } } diff --git a/src/dataflow/channels/pullers/counter.rs b/src/dataflow/channels/pullers/counter.rs index b1b333908..5458ab7e0 100644 --- a/src/dataflow/channels/pullers/counter.rs +++ b/src/dataflow/channels/pullers/counter.rs @@ -3,25 +3,25 @@ use std::rc::Rc; use std::cell::RefCell; -use dataflow::channels::Content; +use dataflow::channels::Bundle; use progress::ChangeBatch; -use Pull; +use communication::Pull; /// A wrapper which accounts records pulled past in a shared count map. -pub struct Counter)>> { +pub struct Counter>> { pullable: P, consumed: Rc>>, phantom: ::std::marker::PhantomData, } -impl)>> Counter { +impl>> Counter { /// Retrieves the next timestamp and batch of data. #[inline] - pub fn next(&mut self) -> Option<(&T, &mut Content)> { - if let Some((ref time, ref mut data)) = *self.pullable.pull() { - if data.len() > 0 { - self.consumed.borrow_mut().update(time.clone(), data.len() as i64); - Some((time, data)) + pub fn next(&mut self) -> Option<&mut Bundle> { + if let Some(message) = self.pullable.pull() { + if message.data.len() > 0 { + self.consumed.borrow_mut().update(message.time.clone(), message.data.len() as i64); + Some(message) } else { None } } @@ -29,7 +29,7 @@ impl)>> Counter { } } -impl)>> Counter { +impl>> Counter { /// Allocates a new `Counter` from a boxed puller. pub fn new(pullable: P) -> Self { Counter { diff --git a/src/dataflow/channels/pushers/buffer.rs b/src/dataflow/channels/pushers/buffer.rs index 4611cf6c5..af6ad4ca0 100644 --- a/src/dataflow/channels/pushers/buffer.rs +++ b/src/dataflow/channels/pushers/buffer.rs @@ -1,28 +1,28 @@ //! Buffering and session mechanisms to provide the appearance of record-at-a-time sending, //! with the performance of batched sends. -use dataflow::channels::Content; +use dataflow::channels::{Bundle, Message}; use progress::Timestamp; use dataflow::operators::Capability; -use timely_communication::Push; +use communication::Push; /// Buffers data sent at the same time, for efficient communication. /// /// The `Buffer` type should be used by calling `session` with a time, which checks whether /// data must be flushed and creates a `Session` object which allows sending at the given time. -pub struct Buffer)>> { +pub struct Buffer>> { time: Option, // the currently open time, if it is open buffer: Vec, // a buffer for records, to send at self.time pusher: P, } -impl)>> Buffer where T: Eq+Clone { +impl>> Buffer where T: Eq+Clone { /// Creates a new `Buffer`. pub fn new(pusher: P) -> Buffer { Buffer { time: None, - buffer: Vec::with_capacity(Content::::default_length()), + buffer: Vec::with_capacity(Message::::default_length()), pusher, } } @@ -58,7 +58,7 @@ impl)>> Buffer where T: Eq+Clone { fn flush(&mut self) { if !self.buffer.is_empty() { let time = self.time.as_ref().unwrap().clone(); - Content::push_at(&mut self.buffer, time, &mut self.pusher); + Message::push_at(&mut self.buffer, time, &mut self.pusher); } } @@ -72,20 +72,14 @@ impl)>> Buffer where T: Eq+Clone { } // Gives an entire message at a specific time. - fn give_content(&mut self, content: &mut Content) { + fn give_vec(&mut self, vector: &mut Vec) { // flush to ensure fifo-ness if !self.buffer.is_empty() { self.flush(); } - let time = self.time.as_ref().unwrap().clone(); - let data = ::std::mem::replace(content, Content::Typed(Vec::new())); - let mut message = Some((time, data)); - - self.pusher.push(&mut message); - if let Some((_, data)) = message { - *content = data; - } + let time = self.time.as_ref().expect("Buffer::give_vec(): time is None.").clone(); + Message::push_at(vector, time, &mut self.pusher); } } @@ -95,11 +89,11 @@ impl)>> Buffer where T: Eq+Clone { /// The `Session` struct provides the user-facing interface to an operator output, namely /// the `Buffer` type. A `Session` wraps a session of output at a specified time, and /// avoids what would otherwise be a constant cost of checking timestamp equality. -pub struct Session<'a, T, D, P: Push<(T, Content)>+'a> where T: Eq+Clone+'a, D: 'a { +pub struct Session<'a, T, D, P: Push>+'a> where T: Eq+Clone+'a, D: 'a { buffer: &'a mut Buffer, } -impl<'a, T, D, P: Push<(T, Content)>+'a> Session<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { +impl<'a, T, D, P: Push>+'a> Session<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { /// Provides one record at the time specified by the `Session`. #[inline(always)] pub fn give(&mut self, data: D) { @@ -118,15 +112,15 @@ impl<'a, T, D, P: Push<(T, Content)>+'a> Session<'a, T, D, P> where T: Eq+Cl /// often be more efficient to re-use this memory rather than have timely allocate /// new backing memory. #[inline(always)] - pub fn give_content(&mut self, message: &mut Content) { + pub fn give_vec(&mut self, message: &mut Vec) { if message.len() > 0 { - self.buffer.give_content(message); + self.buffer.give_vec(message); } } } /// A session which will flush itself when dropped. -pub struct AutoflushSession<'a, T: Timestamp, D, P: Push<(T, Content)>+'a> where +pub struct AutoflushSession<'a, T: Timestamp, D, P: Push>+'a> where T: Eq+Clone+'a, D: 'a { /// A reference to the underlying buffer. buffer: &'a mut Buffer, @@ -134,7 +128,7 @@ pub struct AutoflushSession<'a, T: Timestamp, D, P: Push<(T, Content)>+'a> wh _capability: Capability, } -impl<'a, T: Timestamp, D, P: Push<(T, Content)>+'a> AutoflushSession<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { +impl<'a, T: Timestamp, D, P: Push>+'a> AutoflushSession<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { /// Transmits a single record. #[inline(always)] pub fn give(&mut self, data: D) { @@ -149,14 +143,14 @@ impl<'a, T: Timestamp, D, P: Push<(T, Content)>+'a> AutoflushSession<'a, T, D } /// Transmits a pre-packed batch of data. #[inline(always)] - pub fn give_content(&mut self, message: &mut Content) { + pub fn give_content(&mut self, message: &mut Vec) { if message.len() > 0 { - self.buffer.give_content(message); + self.buffer.give_vec(message); } } } -impl<'a, T: Timestamp, D, P: Push<(T, Content)>+'a> Drop for AutoflushSession<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { +impl<'a, T: Timestamp, D, P: Push>+'a> Drop for AutoflushSession<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { fn drop(&mut self) { self.buffer.cease(); } diff --git a/src/dataflow/channels/pushers/counter.rs b/src/dataflow/channels/pushers/counter.rs index e119e6356..0d9c8bbef 100644 --- a/src/dataflow/channels/pushers/counter.rs +++ b/src/dataflow/channels/pushers/counter.rs @@ -4,21 +4,21 @@ use std::rc::Rc; use std::cell::RefCell; use progress::ChangeBatch; -use dataflow::channels::Content; -use Push; +use dataflow::channels::Bundle; +use communication::Push; /// A wrapper which updates shared `produced` based on the number of records pushed. -pub struct Counter)>> { +pub struct Counter>> { pushee: P, produced: Rc>>, phantom: ::std::marker::PhantomData, } -impl)>> Push<(T, Content)> for Counter where T : Eq+Clone+'static { +impl Push> for Counter where T : Ord+Clone+'static, P: Push> { #[inline(always)] - fn push(&mut self, message: &mut Option<(T, Content)>) { - if let Some((ref time, ref data)) = *message { - self.produced.borrow_mut().update(time.clone(), data.len() as i64); + fn push(&mut self, message: &mut Option>) { + if let Some(message) = message { + self.produced.borrow_mut().update(message.time.clone(), message.data.len() as i64); } // only propagate `None` if dirty (indicates flush) @@ -28,7 +28,7 @@ impl)>> Push<(T, Content)> for Counter)>> Counter where T : Ord+Clone+'static { +impl>> Counter where T : Ord+Clone+'static { /// Allocates a new `Counter` from a pushee and shared counts. pub fn new(pushee: P) -> Counter { Counter { @@ -42,11 +42,4 @@ impl)>> Counter where T : Ord+Clone+'stati pub fn produced(&self) -> &Rc>> { &self.produced } - // /// Extracts shared counts into `updates`. - // /// - // /// It is unclear why this method exists at the same time the counts are shared. - // /// Perhaps this should be investigated, and only one pattern used. Seriously. - // #[inline] pub fn pull_progress(&mut self, updates: &mut ChangeBatch) { - // self.counts.borrow_mut().drain_into(updates); - // } } diff --git a/src/dataflow/channels/pushers/exchange.rs b/src/dataflow/channels/pushers/exchange.rs index f862e7501..575d5a74e 100644 --- a/src/dataflow/channels/pushers/exchange.rs +++ b/src/dataflow/channels/pushers/exchange.rs @@ -1,24 +1,24 @@ //! The exchange pattern distributes pushed data between many target pushees. -use {Push, Data}; -use dataflow::channels::Content; -use abomonation::Abomonation; +use Data; +use communication::Push; +use dataflow::channels::{Bundle, Message}; // TODO : Software write combining /// Distributes records among target pushees according to a distribution function. -pub struct Exchange)>, H: Fn(&T, &D) -> u64> { +pub struct Exchange>, H: Fn(&T, &D) -> u64> { pushers: Vec

, buffers: Vec>, current: Option, hash_func: H, } -impl)>, H: Fn(&T, &D)->u64> Exchange { +impl>, H: Fn(&T, &D)->u64> Exchange { /// Allocates a new `Exchange` from a supplied set of pushers and a distribution function. pub fn new(pushers: Vec

, key: H) -> Exchange { let mut buffers = vec![]; for _ in 0..pushers.len() { - buffers.push(Vec::with_capacity(Content::::default_length())); + buffers.push(Vec::with_capacity(Message::::default_length())); } Exchange { pushers, @@ -31,20 +31,24 @@ impl)>, H: Fn(&T, &D)->u64> Exchange)>, H: Fn(&T, &D)->u64> Push<(T, Content)> for Exchange { - #[inline] - fn push(&mut self, message: &mut Option<(T, Content)>) { +impl>, H: Fn(&T, &D)->u64> Push> for Exchange { + #[inline(never)] + fn push(&mut self, message: &mut Option>) { // if only one pusher, no exchange if self.pushers.len() == 1 { self.pushers[0].push(message); } - else if let Some((ref time, ref mut data)) = *message { + else if let Some(message) = message { + + let message = message.as_mut(); + let time = &message.time; + let data = &mut message.data; // if the time isn't right, flush everything. if self.current.as_ref().map_or(false, |x| x != time) { diff --git a/src/dataflow/channels/pushers/tee.rs b/src/dataflow/channels/pushers/tee.rs index 816f08e38..437eeafce 100644 --- a/src/dataflow/channels/pushers/tee.rs +++ b/src/dataflow/channels/pushers/tee.rs @@ -3,38 +3,36 @@ use std::rc::Rc; use std::cell::RefCell; -use dataflow::channels::Content; -use abomonation::Abomonation; +use Data; +use dataflow::channels::{Bundle, Message}; -use timely_communication::Push; +use communication::Push; /// Wraps a shared list of `Box` to forward pushes to. Owned by `Stream`. pub struct Tee { buffer: Vec, - shared: Rc)>>>>>, + shared: Rc>>>>>, } -impl Push<(T, Content)> for Tee { +impl Push> for Tee { #[inline] - fn push(&mut self, message: &mut Option<(T, Content)>) { - if let Some((ref time, ref mut data)) = *message { - let mut pushers = self.shared.borrow_mut(); - for index in 0..pushers.len() { - if index < pushers.len() - 1 { - // TODO : was `push_all`, but is now `extend`, slow. - self.buffer.extend_from_slice(data); - Content::push_at(&mut self.buffer, (*time).clone(), &mut pushers[index]); - } - else { - Content::push_at(data, (*time).clone(), &mut pushers[index]); - } + fn push(&mut self, message: &mut Option>) { + let mut pushers = self.shared.borrow_mut(); + if let Some(message) = message { + for index in 1..pushers.len() { + self.buffer.extend_from_slice(&message.data); + Message::push_at(&mut self.buffer, message.time.clone(), &mut pushers[index-1]); } } else { - for pusher in self.shared.borrow_mut().iter_mut() { - pusher.push(&mut None); + for index in 1..pushers.len() { + pushers[index-1].push(&mut None); } } + if pushers.len() > 0 { + let last = pushers.len() - 1; + pushers[last].push(message); + } } } @@ -43,7 +41,7 @@ impl Tee { pub fn new() -> (Tee, TeeHelper) { let shared = Rc::new(RefCell::new(Vec::new())); let port = Tee { - buffer: Vec::with_capacity(Content::::default_length()), + buffer: Vec::with_capacity(Message::::default_length()), shared: shared.clone(), }; @@ -62,12 +60,12 @@ impl Clone for Tee { /// A shared list of `Box` used to add `Push` implementors. pub struct TeeHelper { - shared: Rc)>>>>> + shared: Rc>>>>> } impl TeeHelper { /// Adds a new `Push` implementor to the list of recipients shared with a `Stream`. - pub fn add_pusher)>+'static>(&self, pusher: P) { + pub fn add_pusher>+'static>(&self, pusher: P) { self.shared.borrow_mut().push(Box::new(pusher)); } } diff --git a/src/dataflow/operators/aggregation/aggregate.rs b/src/dataflow/operators/aggregation/aggregate.rs index 686e8fa0d..42c215561 100644 --- a/src/dataflow/operators/aggregation/aggregate.rs +++ b/src/dataflow/operators/aggregation/aggregate.rs @@ -76,13 +76,14 @@ impl Aggregate for hash: H) -> Stream where S::Timestamp: Eq { let mut aggregates = HashMap::new(); - + let mut vector = Vec::new(); self.unary_notify(Exchange::new(move |&(ref k, _)| hash(k)), "Aggregate", vec![], move |input, output, notificator| { // read each input, fold into aggregates input.for_each(|time, data| { + data.swap(&mut vector); let agg_time = aggregates.entry(time.time().clone()).or_insert_with(HashMap::new); - for (key, val) in data.drain(..) { + for (key, val) in vector.drain(..) { let agg = agg_time.entry(key.clone()).or_insert_with(Default::default); fold(&key, val, agg); } diff --git a/src/dataflow/operators/aggregation/state_machine.rs b/src/dataflow/operators/aggregation/state_machine.rs index 11e6fddb2..246b606cb 100644 --- a/src/dataflow/operators/aggregation/state_machine.rs +++ b/src/dataflow/operators/aggregation/state_machine.rs @@ -66,6 +66,8 @@ impl StateMachine f let mut pending: HashMap<_, Vec<(K, V)>> = HashMap::new(); // times -> (keys -> state) let mut states = HashMap::new(); // keys -> state + let mut vector = Vec::new(); + self.unary_notify(Exchange::new(move |&(ref k, _)| hash(k)), "StateMachine", vec![], move |input, output, notificator| { // go through each time with data, process each (key, val) pair. @@ -85,15 +87,18 @@ impl StateMachine f // stash each input and request a notification when ready input.for_each(|time, data| { + + data.swap(&mut vector); + // stash if not time yet if notificator.frontier(0).less_than(time.time()) { - pending.entry(time.time().clone()).or_insert_with(Vec::new).extend(data.drain(..)); + pending.entry(time.time().clone()).or_insert_with(Vec::new).extend(vector.drain(..)); notificator.notify_at(time.retain()); } else { // else we can process immediately let mut session = output.session(&time); - for (key, val) in data.drain(..) { + for (key, val) in vector.drain(..) { let (remove, output) = { let state = states.entry(key.clone()).or_insert_with(Default::default); fold(&key, val, state) diff --git a/src/dataflow/operators/branch.rs b/src/dataflow/operators/branch.rs index 9944cba72..b94094352 100644 --- a/src/dataflow/operators/branch.rs +++ b/src/dataflow/operators/branch.rs @@ -46,14 +46,16 @@ impl Branch for Stream { let (mut output2, stream2) = builder.new_output(); builder.build(move |_| { + let mut vector = Vec::new(); move |_frontiers| { let mut output1_handle = output1.activate(); let mut output2_handle = output2.activate(); input.for_each(|time, data| { + data.swap(&mut vector); let mut out1 = output1_handle.session(&time); let mut out2 = output2_handle.session(&time); - for datum in data.drain(..) { + for datum in vector.drain(..) { if condition(&time.time(), &datum) { out2.give(datum); } else { @@ -74,7 +76,7 @@ pub trait BranchWhen { /// For each time, the supplied closure is called. If it returns true, /// the records for that will be sent to the second returned stream, otherwise /// they will be sent to the first. - /// + /// /// #Examples /// ``` /// use timely::dataflow::operators::{ToStream, BranchWhen, Inspect, Delay}; @@ -108,17 +110,20 @@ impl BranchWhen for Stream { let (mut output2, stream2) = builder.new_output(); builder.build(move |_| { + + let mut vector = Vec::new(); move |_frontiers| { let mut output1_handle = output1.activate(); let mut output2_handle = output2.activate(); input.for_each(|time, data| { + data.swap(&mut vector); let mut out = if condition(&time.time()) { output2_handle.session(&time) } else { output1_handle.session(&time) }; - out.give_content(data); + out.give_vec(&mut vector); }); } }); diff --git a/src/dataflow/operators/broadcast.rs b/src/dataflow/operators/broadcast.rs index f49cde1ac..123af5c14 100644 --- a/src/dataflow/operators/broadcast.rs +++ b/src/dataflow/operators/broadcast.rs @@ -1,18 +1,18 @@ //! Broadcast records to all workers. -use timely_communication::Pull; +use communication::Pull; use ::ExchangeData; use progress::nested::subgraph::{Source, Target}; use dataflow::{Stream, Scope}; use progress::ChangeBatch; use progress::{Timestamp, Operate, Antichain}; -use dataflow::channels::{Message}; +use dataflow::channels::{Message, Bundle}; use dataflow::channels::pushers::Counter as PushCounter; use dataflow::channels::pushers::buffer::Buffer as PushBuffer; use dataflow::channels::pushers::Tee; use dataflow::channels::pullers::Counter as PullCounter; -use dataflow::channels::pact::{Pusher, Puller}; +use dataflow::channels::pact::{LogPusher, LogPuller}; /// Broadcast records to all workers. pub trait Broadcast { @@ -42,7 +42,7 @@ impl Broadcast for Stream { assert_eq!(pushers.len(), scope.peers()); - let receiver = Puller::new(puller, scope.index(), channel_id, comm_chan, scope.logging()); + let receiver = LogPuller::new(puller, scope.index(), channel_id, comm_chan, scope.logging()); let operator = BroadcastOperator { index: scope.index(), @@ -54,7 +54,7 @@ impl Broadcast for Stream { let operator_index = scope.add_operator(operator); for (i, pusher) in pushers.into_iter().enumerate() { - let sender = Pusher::new(pusher, scope.index(), i, channel_id, comm_chan, scope.logging()); + let sender = LogPusher::new(pusher, scope.index(), i, channel_id, comm_chan, scope.logging()); self.connect_to(Target { index: operator_index, port: i }, sender, channel_id); } @@ -65,7 +65,7 @@ impl Broadcast for Stream { struct BroadcastOperator { index: usize, peers: usize, - input: PullCounter>>>>, + input: PullCounter>>>>, output: PushBuffer>>, } @@ -84,8 +84,20 @@ impl Operate for BroadcastOperator { _internal: &mut [ChangeBatch], produced: &mut [ChangeBatch]) -> bool { - while let Some((time, data)) = self.input.next() { - self.output.session(time).give_content(data); + let mut vec = Vec::new(); + while let Some(bundle) = self.input.next() { + + use communication::message::RefOrMut; + + match bundle.as_ref_or_mut() { + RefOrMut::Ref(bundle) => { + RefOrMut::Ref(&bundle.data).swap(&mut vec); + self.output.session(&bundle.time).give_vec(&mut vec); + }, + RefOrMut::Mut(bundle) => { + self.output.session(&bundle.time).give_vec(&mut bundle.data); + }, + } } self.output.cease(); self.input.consumed().borrow_mut().drain_into(&mut consumed[self.index]); diff --git a/src/dataflow/operators/capture/capture.rs b/src/dataflow/operators/capture/capture.rs index 7bafbc3ec..62c33ef36 100644 --- a/src/dataflow/operators/capture/capture.rs +++ b/src/dataflow/operators/capture/capture.rs @@ -7,7 +7,6 @@ use std::rc::Rc; use std::cell::RefCell; -use std::ops::DerefMut; use ::Data; use dataflow::{Scope, Stream}; @@ -140,10 +139,18 @@ impl Capture for Stream { } }, move |consumed, _internal, _external| { + + use communication::message::RefOrMut; + // turn each received message into an event. let mut borrow = event_pusher2.borrow_mut(); - while let Some((time, data)) = input.next() { - borrow.push(Event::Messages(time.clone(), data.deref_mut().clone())); + while let Some(message) = input.next() { + let (time, data) = match message.as_ref_or_mut() { + RefOrMut::Ref(reference) => (&reference.time, RefOrMut::Ref(&reference.data)), + RefOrMut::Mut(reference) => (&reference.time, RefOrMut::Mut(&mut reference.data)), + }; + let vector = data.replace(Vec::new()); + borrow.push(Event::Messages(time.clone(), vector)); } input.consumed().borrow_mut().drain_into(&mut consumed[0]); false diff --git a/src/dataflow/operators/concat.rs b/src/dataflow/operators/concat.rs index a5eef3e36..1c7a1f77f 100644 --- a/src/dataflow/operators/concat.rs +++ b/src/dataflow/operators/concat.rs @@ -65,11 +65,14 @@ impl Concatenate for G { // build an operator that plays out all input data. builder.build(move |_capability| { + + let mut vector = Vec::new(); move |_frontier| { let mut output = output.activate(); for handle in handles.iter_mut() { handle.for_each(|time, data| { - output.session(&time).give_content(data); + data.swap(&mut vector); + output.session(&time).give_vec(&mut vector); }) } } diff --git a/src/dataflow/operators/count.rs b/src/dataflow/operators/count.rs index cb050cac2..320451645 100644 --- a/src/dataflow/operators/count.rs +++ b/src/dataflow/operators/count.rs @@ -1,13 +1,13 @@ //! Counts the number of records at each time. use std::collections::HashMap; +use communication::message::RefOrMut; + use Data; use dataflow::channels::pact::Pipeline; use dataflow::{Stream, Scope}; use dataflow::operators::generic::operator::Operator; -use dataflow::channels::message::Content; - /// Accumulates records within a timestamp. pub trait Accumulate { /// Accumulates records within a timestamp. @@ -28,7 +28,7 @@ pub trait Accumulate { /// let extracted = captured.extract(); /// assert_eq!(extracted, vec![(RootTimestamp::new(0), vec![45])]); /// ``` - fn accumulate(&self, default: A, logic: impl Fn(&mut A, &mut Content)+'static) -> Stream; + fn accumulate(&self, default: A, logic: impl Fn(&mut A, RefOrMut>)+'static) -> Stream; /// Counts the number of records observed at each time. /// /// #Examples @@ -53,7 +53,7 @@ pub trait Accumulate { } impl Accumulate for Stream { - fn accumulate(&self, default: A, logic: impl Fn(&mut A, &mut Content)+'static) -> Stream { + fn accumulate(&self, default: A, logic: impl Fn(&mut A, RefOrMut>)+'static) -> Stream { let mut accums = HashMap::new(); self.unary_notify(Pipeline, "Accumulate", vec![], move |input, output, notificator| { diff --git a/src/dataflow/operators/delay.rs b/src/dataflow/operators/delay.rs index 1d9d6720f..4cc0aaa2b 100644 --- a/src/dataflow/operators/delay.rs +++ b/src/dataflow/operators/delay.rs @@ -1,12 +1,10 @@ //! Operators acting on timestamps to logically delay records use std::collections::HashMap; -use std::ops::DerefMut; use Data; use order::PartialOrder; use dataflow::channels::pact::Pipeline; -use dataflow::channels::Content; use dataflow::{Stream, Scope}; use dataflow::operators::generic::operator::Operator; @@ -24,18 +22,16 @@ pub trait Delay { /// and delays each element `i` to time `RootTimestamp(i)`. /// /// ``` - /// use timely::dataflow::operators::{ToStream, Delay}; - /// use timely::dataflow::operators::generic::unary::Unary; + /// use timely::dataflow::operators::{ToStream, Delay, Operator}; /// use timely::dataflow::channels::pact::Pipeline; /// use timely::progress::timestamp::RootTimestamp; /// /// timely::example(|scope| { /// (0..10).to_stream(scope) /// .delay(|data, time| RootTimestamp::new(*data)) - /// .unary_stream(Pipeline, "example", |input, output| { + /// .sink(Pipeline, "example", |input| { /// input.for_each(|time, data| { /// println!("data at time: {:?}", time); - /// output.session(&time).give_content(data); /// }); /// }); /// }); @@ -54,18 +50,16 @@ pub trait Delay { /// and delays each batch (there is just one) to time `RootTimestamp(1)`. /// /// ``` - /// use timely::dataflow::operators::{ToStream, Delay}; - /// use timely::dataflow::operators::generic::unary::Unary; + /// use timely::dataflow::operators::{ToStream, Delay, Operator}; /// use timely::dataflow::channels::pact::Pipeline; /// use timely::progress::timestamp::RootTimestamp; /// /// timely::example(|scope| { /// (0..10).to_stream(scope) /// .delay_batch(|time| RootTimestamp::new(time.inner + 1)) - /// .unary_stream(Pipeline, "example", |input, output| { + /// .sink(Pipeline, "example", |input| { /// input.for_each(|time, data| { /// println!("data at time: {:?}", time); - /// output.session(&time).give_content(data); /// }); /// }); /// }); @@ -76,9 +70,11 @@ pub trait Delay { impl Delay for Stream { fn delay(&self, func: impl Fn(&D, &G::Timestamp)->G::Timestamp+'static) -> Stream { let mut elements = HashMap::new(); + let mut vector = Vec::new(); self.unary_notify(Pipeline, "Delay", vec![], move |input, output, notificator| { input.for_each(|time, data| { - for datum in data.drain(..) { + data.swap(&mut vector); + for datum in vector.drain(..) { let new_time = func(&datum, &time); assert!(time.time().less_equal(&new_time)); elements.entry(new_time.clone()) @@ -97,28 +93,21 @@ impl Delay for Stream { } fn delay_batch(&self, func: impl Fn(&G::Timestamp)->G::Timestamp+'static) -> Stream { - let mut stash = Vec::new(); let mut elements = HashMap::new(); self.unary_notify(Pipeline, "Delay", vec![], move |input, output, notificator| { input.for_each(|time, data| { let new_time = func(&time); assert!(time.time().less_equal(&new_time)); - let spare = stash.pop().unwrap_or_else(Vec::new); - let data = ::std::mem::replace(data.deref_mut(), spare); - elements.entry(new_time.clone()) .or_insert_with(|| { notificator.notify_at(time.delayed(&new_time)); Vec::new() }) - .push(data); + .push(data.replace(Vec::new())); }); // for each available notification, send corresponding set notificator.for_each(|time,_,_| { if let Some(mut datas) = elements.remove(&time) { for mut data in datas.drain(..) { - let mut message = Content::from_typed(&mut data); - output.session(&time).give_content(&mut message); - let buffer = message.into_typed(); - if buffer.capacity() == Content::::default_length() { stash.push(buffer); } + output.session(&time).give_vec(&mut data); } } }); diff --git a/src/dataflow/operators/enterleave.rs b/src/dataflow/operators/enterleave.rs index a61196f72..725f6c479 100644 --- a/src/dataflow/operators/enterleave.rs +++ b/src/dataflow/operators/enterleave.rs @@ -26,9 +26,10 @@ use std::marker::PhantomData; use progress::Timestamp; use progress::nested::subgraph::{Source, Target}; use progress::nested::product::Product; -use {Data, Push}; +use Data; +use communication::Push; use dataflow::channels::pushers::{Counter, Tee}; -use dataflow::channels::Content; +use dataflow::channels::{Bundle, Message}; use dataflow::{Stream, Scope, ScopeParent}; use dataflow::scopes::Child; @@ -136,14 +137,17 @@ struct IngressNub { targets: Counter, TData, Tee, TData>>, } -impl Push<(TOuter, Content)> for IngressNub { - fn push(&mut self, message: &mut Option<(TOuter, Content)>) { - if let Some((ref time, ref mut data)) = *message { - let content = ::std::mem::replace(data, Content::Typed(Vec::new())); - let mut message = Some((Product::new(time.clone(), Default::default()), content)); - self.targets.push(&mut message); - if let Some((_, content)) = message { - *data = content; +impl Push> for IngressNub { + fn push(&mut self, message: &mut Option>) { + if let Some(message) = message { + let outer_message = message.as_mut(); + let data = ::std::mem::replace(&mut outer_message.data, Vec::new()); + let mut inner_message = Some(Bundle::from_typed(Message::new(Product::new(outer_message.time.clone(), Default::default()), data, 0, 0))); + self.targets.push(&mut inner_message); + if let Some(inner_message) = inner_message { + if let Some(inner_message) = inner_message.if_typed() { + outer_message.data = inner_message.data; + } } } else { self.targets.done(); } @@ -156,15 +160,18 @@ struct EgressNub { phantom: PhantomData, } -impl Push<(Product, Content)> for EgressNub +impl Push, TData>> for EgressNub where TOuter: Timestamp, TInner: Timestamp, TData: Data { - fn push(&mut self, message: &mut Option<(Product, Content)>) { - if let Some((ref time, ref mut data)) = *message { - let content = ::std::mem::replace(data, Content::Typed(Vec::new())); - let mut message = Some((time.outer.clone(), content)); - self.targets.push(&mut message); - if let Some((_, content)) = message { - *data = content; + fn push(&mut self, message: &mut Option, TData>>) { + if let Some(message) = message { + let inner_message = message.as_mut(); + let data = ::std::mem::replace(&mut inner_message.data, Vec::new()); + let mut outer_message = Some(Bundle::from_typed(Message::new(inner_message.time.outer.clone(), data, 0, 0))); + self.targets.push(&mut outer_message); + if let Some(outer_message) = outer_message { + if let Some(outer_message) = outer_message.if_typed() { + inner_message.data = outer_message.data; + } } } else { self.targets.done(); } diff --git a/src/dataflow/operators/exchange.rs b/src/dataflow/operators/exchange.rs index d97246746..6996404b7 100644 --- a/src/dataflow/operators/exchange.rs +++ b/src/dataflow/operators/exchange.rs @@ -2,7 +2,7 @@ use ::ExchangeData; use dataflow::channels::pact::Exchange as ExchangePact; -use dataflow::channels::pact::TimeExchange as TimeExchangePact; +// use dataflow::channels::pact::TimeExchange as TimeExchangePact; use dataflow::{Stream, Scope}; use dataflow::operators::generic::operator::Operator; use progress::timestamp::Timestamp; @@ -22,36 +22,15 @@ pub trait Exchange { /// }); /// ``` fn exchange(&self, route: impl Fn(&D)->u64+'static) -> Self; - - /// Exchange records by time so that all records whose time and data - /// evaluate to the same `route` are at the same worker. - /// - /// #Examples - /// ``` - /// use timely::dataflow::operators::{ToStream, Exchange, Inspect}; - /// - /// timely::example(|scope| { - /// (0..10).to_stream(scope) - /// .exchange_ts(|&t, &x| t.inner & 1 ^ x) - /// .inspect(|x| println!("seen: {:?}", x)); - /// }); - /// ``` - fn exchange_ts(&self, route: impl Fn(&T, &D)->u64+'static) -> Self; } impl, D: ExchangeData> Exchange for Stream { fn exchange(&self, route: impl Fn(&D)->u64+'static) -> Stream { - self.unary(ExchangePact::new(route), "Exchange", |_,_| |input, output| { - input.for_each(|time, data| { - output.session(&time).give_content(data); - }); - }) - } - - fn exchange_ts(&self, route: impl Fn(&T, &D)->u64+'static) -> Stream { - self.unary(TimeExchangePact::new(route), "Exchange", |_,_| |input, output| { + let mut vector = Vec::new(); + self.unary(ExchangePact::new(route), "Exchange", move |_,_| move |input, output| { input.for_each(|time, data| { - output.session(&time).give_content(data); + data.swap(&mut vector); + output.session(&time).give_vec(&mut vector); }); }) } diff --git a/src/dataflow/operators/feedback.rs b/src/dataflow/operators/feedback.rs index 00634b237..72a1fd0ff 100644 --- a/src/dataflow/operators/feedback.rs +++ b/src/dataflow/operators/feedback.rs @@ -3,7 +3,8 @@ use std::rc::Rc; use std::cell::RefCell; -use {Data, Push}; +use Data; +use communication::Push; use progress::{Timestamp, Operate, PathSummary}; use progress::frontier::Antichain; @@ -13,7 +14,7 @@ use progress::ChangeBatch; use progress::nested::product::Product; use progress::nested::Summary::Local; -use dataflow::channels::Content; +use dataflow::channels::Bundle; use dataflow::channels::pushers::{Counter, Tee}; use dataflow::{Stream, Scope, ScopeParent}; @@ -78,13 +79,14 @@ struct Observer { targets: Counter, D, Tee, D>>, } -impl Push<(Product, Content)> for Observer { +impl Push, D>> for Observer { #[inline] - fn push(&mut self, message: &mut Option<(Product, Content)>) { - let active = if let Some((ref mut time, _)) = *message { - if let Some(new_time) = self.summary.results_in(&time.inner) { - time.inner = new_time; - time.inner.less_equal(&self.limit) + fn push(&mut self, message: &mut Option, D>>) { + let active = if let Some(message) = message { + let message = message.as_mut(); + if let Some(new_time) = self.summary.results_in(&message.time.inner) { + message.time.inner = new_time; + message.time.inner.less_equal(&self.limit) } else { false diff --git a/src/dataflow/operators/filter.rs b/src/dataflow/operators/filter.rs index d78510c0d..aceaaa418 100644 --- a/src/dataflow/operators/filter.rs +++ b/src/dataflow/operators/filter.rs @@ -24,11 +24,13 @@ pub trait Filter { impl Filter for Stream { fn filter(&self, predicate: impl Fn(&D)->bool+'static) -> Stream { + let mut vector = Vec::new(); self.unary(Pipeline, "Filter", move |_,_| move |input, output| { input.for_each(|time, data| { - data.retain(|x| predicate(x)); - if data.len() > 0 { - output.session(&time).give_content(data); + data.swap(&mut vector); + vector.retain(|x| predicate(x)); + if vector.len() > 0 { + output.session(&time).give_vec(&mut vector); } }); }) diff --git a/src/dataflow/operators/generic/binary.rs b/src/dataflow/operators/generic/binary.rs index c04b00fe2..52d9d1e31 100644 --- a/src/dataflow/operators/generic/binary.rs +++ b/src/dataflow/operators/generic/binary.rs @@ -27,12 +27,17 @@ pub trait Binary { /// let stream1 = (0..10).to_stream(scope); /// let stream2 = (0..10).to_stream(scope); /// - /// stream1.binary_stream(&stream2, Pipeline, Pipeline, "example", |input1, input2, output| { + /// let mut vector1 = Vec::new(); + /// let mut vector2 = Vec::new(); + /// + /// stream1.binary_stream(&stream2, Pipeline, Pipeline, "example", move |input1, input2, output| { /// input1.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector1); + /// output.session(&time).give_vec(&mut vector1); /// }); /// input2.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector2); + /// output.session(&time).give_vec(&mut vector2); /// }); /// }); /// }); @@ -62,13 +67,18 @@ pub trait Binary { /// let stream1 = (0..10).to_stream(scope); /// let stream2 = (0..10).to_stream(scope); /// - /// stream1.binary_notify(&stream2, Pipeline, Pipeline, "example", Vec::new(), |input1, input2, output, notificator| { + /// let mut vector1 = Vec::new(); + /// let mut vector2 = Vec::new(); + /// + /// stream1.binary_notify(&stream2, Pipeline, Pipeline, "example", Vec::new(), move |input1, input2, output, notificator| { /// input1.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector1); + /// output.session(&time).give_vec(&mut vector1); /// notificator.notify_at(time.retain()); /// }); /// input2.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector2); + /// output.session(&time).give_vec(&mut vector2); /// notificator.notify_at(time.retain()); /// }); /// notificator.for_each(|time,_count,_notificator| { diff --git a/src/dataflow/operators/generic/handles.rs b/src/dataflow/operators/generic/handles.rs index f54641756..1d729779d 100644 --- a/src/dataflow/operators/generic/handles.rs +++ b/src/dataflow/operators/generic/handles.rs @@ -5,14 +5,16 @@ use std::rc::Rc; use std::cell::RefCell; + +use ::Data; use progress::Timestamp; use progress::ChangeBatch; use progress::frontier::MutableAntichain; use dataflow::channels::pullers::Counter as PullCounter; use dataflow::channels::pushers::Counter as PushCounter; use dataflow::channels::pushers::buffer::{Buffer, Session}; -use dataflow::channels::Content; -use timely_communication::{Push, Pull}; +use dataflow::channels::Bundle; +use communication::{Push, Pull, message::RefOrMut}; use logging::Logger; use dataflow::operators::CapabilityRef; @@ -20,30 +22,37 @@ use dataflow::operators::capability::mint_ref as mint_capability_ref; use dataflow::operators::capability::CapabilityTrait; /// Handle to an operator's input stream. -pub struct InputHandle)>> { +pub struct InputHandle>> { pull_counter: PullCounter, internal: Rc>>, logging: Logger, } /// Handle to an operator's input stream and frontier. -pub struct FrontieredInputHandle<'a, T: Timestamp, D: 'a, P: Pull<(T, Content)>+'a> { +pub struct FrontieredInputHandle<'a, T: Timestamp, D: 'a, P: Pull>+'a> { /// The underlying input handle. pub handle: &'a mut InputHandle, /// The frontier as reported by timely progress tracking. pub frontier: &'a MutableAntichain, } -impl<'a, T: Timestamp, D, P: Pull<(T, Content)>> InputHandle { +impl<'a, T: Timestamp, D: Data, P: Pull>> InputHandle { /// Reads the next input buffer (at some timestamp `t`) and a corresponding capability for `t`. /// The timestamp `t` of the input buffer can be retrieved by invoking `.time()` on the capability. /// Returns `None` when there's no more data available. #[inline(always)] - pub fn next(&mut self) -> Option<(CapabilityRef, &mut Content)> { + pub fn next(&mut self) -> Option<(CapabilityRef, RefOrMut>)> { let internal = &mut self.internal; - self.pull_counter.next().map(|(time, content)| { - (mint_capability_ref(time, internal.clone()), content) + self.pull_counter.next().map(|bundle| { + match bundle.as_ref_or_mut() { + RefOrMut::Ref(bundle) => { + (mint_capability_ref(&bundle.time, internal.clone()), RefOrMut::Ref(&bundle.data)) + }, + RefOrMut::Mut(bundle) => { + (mint_capability_ref(&bundle.time, internal.clone()), RefOrMut::Mut(&mut bundle.data)) + }, + } }) } @@ -60,13 +69,13 @@ impl<'a, T: Timestamp, D, P: Pull<(T, Content)>> InputHandle { /// (0..10).to_stream(scope) /// .unary_stream(Pipeline, "example", |input, output| { /// input.for_each(|cap, data| { - /// output.session(&cap).give_content(data); + /// output.session(&cap).give_vec(&mut data.replace(Vec::new())); /// }); /// }); /// }); /// ``` #[inline] - pub fn for_each, &mut Content)>(&mut self, mut logic: F) { + pub fn for_each, RefOrMut>)>(&mut self, mut logic: F) { let logging = self.logging.clone(); while let Some((cap, data)) = self.next() { logging.when_enabled(|l| l.log(::logging::TimelyEvent::GuardedMessage( @@ -79,7 +88,7 @@ impl<'a, T: Timestamp, D, P: Pull<(T, Content)>> InputHandle { } -impl<'a, T: Timestamp, D, P: Pull<(T, Content)>+'a> FrontieredInputHandle<'a, T, D, P> { +impl<'a, T: Timestamp, D: Data, P: Pull>+'a> FrontieredInputHandle<'a, T, D, P> { /// Allocate a new frontiered input handle. pub fn new(handle: &'a mut InputHandle, frontier: &'a MutableAntichain) -> Self { FrontieredInputHandle { @@ -92,7 +101,7 @@ impl<'a, T: Timestamp, D, P: Pull<(T, Content)>+'a> FrontieredInputHandle<'a, /// The timestamp `t` of the input buffer can be retrieved by invoking `.time()` on the capability. /// Returns `None` when there's no more data available. #[inline(always)] - pub fn next(&mut self) -> Option<(CapabilityRef, &mut Content)> { + pub fn next(&mut self) -> Option<(CapabilityRef, RefOrMut>)> { self.handle.next() } @@ -109,13 +118,13 @@ impl<'a, T: Timestamp, D, P: Pull<(T, Content)>+'a> FrontieredInputHandle<'a, /// (0..10).to_stream(scope) /// .unary_stream(Pipeline, "example", |input, output| { /// input.for_each(|cap, data| { - /// output.session(&cap).give_content(data); + /// output.session(&cap).give_vec(&mut data.replace(Vec::new())); /// }); /// }); /// }); /// ``` #[inline] - pub fn for_each, &mut Content)>(&mut self, logic: F) { + pub fn for_each, RefOrMut>)>(&mut self, logic: F) { self.handle.for_each(logic) } @@ -126,13 +135,13 @@ impl<'a, T: Timestamp, D, P: Pull<(T, Content)>+'a> FrontieredInputHandle<'a, } } -pub fn _access_pull_counter)>>(input: &mut InputHandle) -> &mut PullCounter { +pub fn _access_pull_counter>>(input: &mut InputHandle) -> &mut PullCounter { &mut input.pull_counter } /// Constructs an input handle. /// Declared separately so that it can be kept private when `InputHandle` is re-exported. -pub fn new_input_handle)>>(pull_counter: PullCounter, internal: Rc>>, logging: Logger) -> InputHandle { +pub fn new_input_handle>>(pull_counter: PullCounter, internal: Rc>>, logging: Logger) -> InputHandle { InputHandle { pull_counter, internal, @@ -145,11 +154,11 @@ pub fn new_input_handle)>>(pull_counter: /// An `OutputWrapper` exists to prevent anyone from using the wrapped buffer in any way other /// than with an `OutputHandle`, whose methods ensure that capabilities are used and that the /// pusher is flushed (via the `cease` method) once it is no longer used. -pub struct OutputWrapper)>> { +pub struct OutputWrapper>> { push_buffer: Buffer> } -impl)>> OutputWrapper { +impl>> OutputWrapper { /// Creates a new output wrapper from a push buffer. pub fn new(buffer: Buffer>) -> Self { OutputWrapper { @@ -169,11 +178,11 @@ impl)>> OutputWrapper { /// Handle to an operator's output stream. -pub struct OutputHandle<'a, T: Timestamp, D: 'a, P: Push<(T, Content)>+'a> { +pub struct OutputHandle<'a, T: Timestamp, D: 'a, P: Push>+'a> { push_buffer: &'a mut Buffer>, } -impl<'a, T: Timestamp, D, P: Push<(T, Content)>> OutputHandle<'a, T, D, P> { +impl<'a, T: Timestamp, D, P: Push>> OutputHandle<'a, T, D, P> { /// Obtains a session that can send data at the timestamp associated with capability `cap`. /// /// In order to send data at a future timestamp, obtain a capability for the new timestamp @@ -191,7 +200,7 @@ impl<'a, T: Timestamp, D, P: Push<(T, Content)>> OutputHandle<'a, T, D, P> { /// input.for_each(|cap, data| { /// let mut time = cap.time().clone(); /// time.inner += 1; - /// output.session(&cap.delayed(&time)).give_content(data); + /// output.session(&cap.delayed(&time)).give_vec(&mut data.replace(Vec::new())); /// }); /// }); /// }); @@ -201,7 +210,7 @@ impl<'a, T: Timestamp, D, P: Push<(T, Content)>> OutputHandle<'a, T, D, P> { } } -impl<'a, T: Timestamp, D, P: Push<(T, Content)>> Drop for OutputHandle<'a, T, D, P> { +impl<'a, T: Timestamp, D, P: Push>> Drop for OutputHandle<'a, T, D, P> { fn drop(&mut self) { self.push_buffer.cease(); } diff --git a/src/dataflow/operators/generic/notificator.rs b/src/dataflow/operators/generic/notificator.rs index 324bfafb5..13e56f046 100644 --- a/src/dataflow/operators/generic/notificator.rs +++ b/src/dataflow/operators/generic/notificator.rs @@ -58,7 +58,7 @@ impl<'a, T: Timestamp> Notificator<'a, T> { /// (0..10).to_stream(scope) /// .unary_notify(Pipeline, "example", Vec::new(), |input, output, notificator| { /// input.for_each(|cap, data| { - /// output.session(&cap).give_content(data); + /// output.session(&cap).give_vec(&mut data.replace(Vec::new())); /// let mut time = cap.time().clone(); /// time.inner += 1; /// notificator.notify_at(cap.delayed(&time)); @@ -199,13 +199,17 @@ fn notificator_delivers_notifications_in_topo_order() { /// in1.binary_frontier(&in2, Pipeline, Pipeline, "example", |mut _default_cap, _info| { /// let mut notificator = FrontierNotificator::new(); /// let mut stash = HashMap::new(); +/// let mut vector1 = Vec::new(); +/// let mut vector2 = Vec::new(); /// move |input1, input2, output| { /// while let Some((time, data)) = input1.next() { -/// stash.entry(time.time().clone()).or_insert(Vec::new()).extend(data.drain(..)); +/// data.swap(&mut vector1); +/// stash.entry(time.time().clone()).or_insert(Vec::new()).extend(vector1.drain(..)); /// notificator.notify_at(time.retain()); /// } /// while let Some((time, data)) = input2.next() { -/// stash.entry(time.time().clone()).or_insert(Vec::new()).extend(data.drain(..)); +/// data.swap(&mut vector2); +/// stash.entry(time.time().clone()).or_insert(Vec::new()).extend(vector2.drain(..)); /// notificator.notify_at(time.retain()); /// } /// notificator.for_each(&[input1.frontier(), input2.frontier()], |time, _| { @@ -269,7 +273,7 @@ impl FrontierNotificator { /// let mut notificator = FrontierNotificator::new(); /// move |input, output| { /// input.for_each(|cap, data| { - /// output.session(&cap).give_content(data); + /// output.session(&cap).give_vec(&mut data.replace(Vec::new())); /// let mut time = cap.time().clone(); /// time.inner += 1; /// notificator.notify_at(cap.delayed(&time)); @@ -386,7 +390,7 @@ impl FrontierNotificator { /// let mut notificator = FrontierNotificator::new(); /// move |input, output| { /// input.for_each(|cap, data| { - /// output.session(&cap).give_content(data); + /// output.session(&cap).give_vec(&mut data.replace(Vec::new())); /// let mut time = cap.time().clone(); /// time.inner += 1; /// notificator.notify_at(cap.delayed(&time)); diff --git a/src/dataflow/operators/generic/operator.rs b/src/dataflow/operators/generic/operator.rs index 3433f71bc..b10a66ad8 100644 --- a/src/dataflow/operators/generic/operator.rs +++ b/src/dataflow/operators/generic/operator.rs @@ -38,12 +38,16 @@ pub trait Operator { /// let mut cap = Some(default_cap.delayed(&RootTimestamp::new(12))); /// let mut notificator = FrontierNotificator::new(); /// let mut stash = HashMap::new(); + /// let mut vector = Vec::new(); /// move |input, output| { /// if let Some(ref c) = cap.take() { /// output.session(&c).give(12); /// } /// while let Some((time, data)) = input.next() { - /// stash.entry(time.time().clone()).or_insert(Vec::new()); + /// data.swap(&mut vector); + /// stash.entry(time.time().clone()) + /// .or_insert(Vec::new()) + /// .extend(vector.drain(..)); /// } /// notificator.for_each(&[input.frontier()], |time, _not| { /// if let Some(mut vec) = stash.remove(time.time()) { @@ -77,10 +81,12 @@ pub trait Operator { /// /// fn main() { /// timely::example(|scope| { + /// let mut vector = Vec::new(); /// (0u64..10).to_stream(scope) - /// .unary_notify(Pipeline, "example", None, |input, output, notificator| { + /// .unary_notify(Pipeline, "example", None, move |input, output, notificator| { /// input.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector); + /// output.session(&time).give_vec(&mut vector); /// notificator.notify_at(time.retain()); /// }); /// notificator.for_each(|time, _cnt, _not| { @@ -113,12 +119,14 @@ pub trait Operator { /// (0u64..10).to_stream(scope) /// .unary(Pipeline, "example", |default_cap, _info| { /// let mut cap = Some(default_cap.delayed(&RootTimestamp::new(12))); + /// let mut vector = Vec::new(); /// move |input, output| { /// if let Some(ref c) = cap.take() { /// output.session(&c).give(100); /// } /// while let Some((time, data)) = input.next() { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector); + /// output.session(&time).give_vec(&mut vector); /// } /// } /// }); @@ -150,13 +158,17 @@ pub trait Operator { /// in1.binary_frontier(&in2, Pipeline, Pipeline, "example", |mut _default_cap, _info| { /// let mut notificator = FrontierNotificator::new(); /// let mut stash = HashMap::new(); + /// let mut vector1 = Vec::new(); + /// let mut vector2 = Vec::new(); /// move |input1, input2, output| { /// while let Some((time, data)) = input1.next() { - /// stash.entry(time.time().clone()).or_insert(Vec::new()).extend(data.drain(..)); + /// data.swap(&mut vector1); + /// stash.entry(time.time().clone()).or_insert(Vec::new()).extend(vector1.drain(..)); /// notificator.notify_at(time.retain()); /// } /// while let Some((time, data)) = input2.next() { - /// stash.entry(time.time().clone()).or_insert(Vec::new()).extend(data.drain(..)); + /// data.swap(&mut vector2); + /// stash.entry(time.time().clone()).or_insert(Vec::new()).extend(vector2.drain(..)); /// notificator.notify_at(time.retain()); /// } /// notificator.for_each(&[input1.frontier(), input2.frontier()], |time, _not| { @@ -205,14 +217,17 @@ pub trait Operator { /// let (in1_handle, in1) = scope.new_input(); /// let (in2_handle, in2) = scope.new_input(); /// - /// - /// in1.binary_notify(&in2, Pipeline, Pipeline, "example", None, |input1, input2, output, notificator| { + /// let mut vector1 = Vec::new(); + /// let mut vector2 = Vec::new(); + /// in1.binary_notify(&in2, Pipeline, Pipeline, "example", None, move |input1, input2, output, notificator| { /// input1.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector1); + /// output.session(&time).give_vec(&mut vector1); /// notificator.notify_at(time.retain()); /// }); /// input2.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector2); + /// output.session(&time).give_vec(&mut vector2); /// notificator.notify_at(time.retain()); /// }); /// notificator.for_each(|time, _cnt, _not| { @@ -258,15 +273,19 @@ pub trait Operator { /// (0u64..10).to_stream(scope) /// .binary(&stream2, Pipeline, Pipeline, "example", |default_cap, _info| { /// let mut cap = Some(default_cap.delayed(&RootTimestamp::new(12))); + /// let mut vector1 = Vec::new(); + /// let mut vector2 = Vec::new(); /// move |input1, input2, output| { /// if let Some(ref c) = cap.take() { /// output.session(&c).give(100); /// } /// while let Some((time, data)) = input1.next() { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector1); + /// output.session(&time).give_vec(&mut vector1); /// } /// while let Some((time, data)) = input2.next() { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector2); + /// output.session(&time).give_vec(&mut vector2); /// } /// } /// }).inspect(|x| println!("{:?}", x)); diff --git a/src/dataflow/operators/generic/unary.rs b/src/dataflow/operators/generic/unary.rs index 3600f8467..e36130cb8 100644 --- a/src/dataflow/operators/generic/unary.rs +++ b/src/dataflow/operators/generic/unary.rs @@ -24,10 +24,12 @@ pub trait Unary { /// use timely::dataflow::channels::pact::Pipeline; /// /// timely::example(|scope| { + /// let mut vector = Vec::new(); /// (0..10).to_stream(scope) - /// .unary_stream(Pipeline, "example", |input, output| { + /// .unary_stream(Pipeline, "example", move |input, output| { /// input.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector); + /// output.session(&time).give_vec(&mut vector); /// }); /// }); /// }); @@ -51,10 +53,12 @@ pub trait Unary { /// use timely::dataflow::channels::pact::Pipeline; /// /// timely::example(|scope| { + /// let mut vector = Vec::new(); /// (0..10).to_stream(scope) - /// .unary_notify(Pipeline, "example", Vec::new(), |input, output, notificator| { + /// .unary_notify(Pipeline, "example", Vec::new(), move |input, output, notificator| { /// input.for_each(|time, data| { - /// output.session(&time).give_content(data); + /// data.swap(&mut vector); + /// output.session(&time).give_vec(&mut vector); /// notificator.notify_at(time.retain()); /// }); /// notificator.for_each(|time,_,_| { diff --git a/src/dataflow/operators/input.rs b/src/dataflow/operators/input.rs index 7ed0edb7e..267a6ecad 100644 --- a/src/dataflow/operators/input.rs +++ b/src/dataflow/operators/input.rs @@ -11,9 +11,10 @@ use progress::ChangeBatch; use progress::timestamp::RootTimestamp; use progress::nested::product::Product; -use timely_communication::Allocate; -use {Data, Push}; -use dataflow::channels::Content; +use communication::Allocate; +use Data; +use communication::Push; +use dataflow::channels::Message; use dataflow::channels::pushers::{Tee, Counter}; use dataflow::{Stream, Scope}; @@ -197,8 +198,8 @@ impl Handle { Handle { progress: Vec::new(), pushers: Vec::new(), - buffer1: Vec::with_capacity(Content::::default_length()), - buffer2: Vec::with_capacity(Content::::default_length()), + buffer1: Vec::with_capacity(Message::::default_length()), + buffer2: Vec::with_capacity(Message::::default_length()), now_at: Default::default(), } } @@ -252,16 +253,17 @@ impl Handle { } // flushes our buffer at each of the destinations. there can be more than one; clone if needed. + #[inline(never)] fn flush(&mut self) { for index in 0 .. self.pushers.len() { if index < self.pushers.len() - 1 { self.buffer2.extend_from_slice(&self.buffer1[..]); - Content::push_at(&mut self.buffer2, self.now_at.clone(), &mut self.pushers[index]); - assert!(self.buffer2.is_empty()); + Message::push_at(&mut self.buffer2, self.now_at.clone(), &mut self.pushers[index]); + debug_assert!(self.buffer2.is_empty()); } else { - Content::push_at(&mut self.buffer1, self.now_at.clone(), &mut self.pushers[index]); - assert!(self.buffer1.is_empty()); + Message::push_at(&mut self.buffer1, self.now_at.clone(), &mut self.pushers[index]); + debug_assert!(self.buffer1.is_empty()); } } self.buffer1.clear(); @@ -281,7 +283,7 @@ impl Handle { #[inline(always)] /// Sends one record into the corresponding timely dataflow `Stream`, at the current epoch. pub fn send(&mut self, data: D) { - // assert!(self.buffer.capacity() == Content::::default_length()); + // assert!(self.buffer1.capacity() == Message::::default_length()); self.buffer1.push(data); if self.buffer1.len() == self.buffer1.capacity() { self.flush(); @@ -301,11 +303,11 @@ impl Handle { for index in 0 .. self.pushers.len() { if index < self.pushers.len() - 1 { self.buffer2.extend_from_slice(&buffer[..]); - Content::push_at(&mut self.buffer2, self.now_at.clone(), &mut self.pushers[index]); + Message::push_at(&mut self.buffer2, self.now_at.clone(), &mut self.pushers[index]); assert!(self.buffer2.is_empty()); } else { - Content::push_at(buffer, self.now_at.clone(), &mut self.pushers[index]); + Message::push_at(buffer, self.now_at.clone(), &mut self.pushers[index]); assert!(buffer.is_empty()); } } diff --git a/src/dataflow/operators/inspect.rs b/src/dataflow/operators/inspect.rs index 29c8bc6cd..9f44c3f14 100644 --- a/src/dataflow/operators/inspect.rs +++ b/src/dataflow/operators/inspect.rs @@ -58,11 +58,14 @@ pub trait Inspect { } impl Inspect for Stream { + fn inspect_batch(&self, mut func: impl FnMut(&G::Timestamp, &[D])+'static) -> Stream { + let mut vector = Vec::new(); self.unary(Pipeline, "InspectBatch", move |_,_| move |input, output| { input.for_each(|time, data| { - func(&time, &data[..]); - output.session(&time).give_content(data); + data.swap(&mut vector); + func(&time, &vector[..]); + output.session(&time).give_vec(&mut vector); }); }) } diff --git a/src/dataflow/operators/map.rs b/src/dataflow/operators/map.rs index c045be4d0..2306535b4 100644 --- a/src/dataflow/operators/map.rs +++ b/src/dataflow/operators/map.rs @@ -50,17 +50,21 @@ pub trait Map { impl Map for Stream { fn map(&self, logic: impl Fn(D)->D2+'static) -> Stream { + let mut vector = Vec::new(); self.unary(Pipeline, "Map", move |_,_| move |input, output| { input.for_each(|time, data| { - output.session(&time).give_iterator(data.drain(..).map(|x| logic(x))); + data.swap(&mut vector); + output.session(&time).give_iterator(vector.drain(..).map(|x| logic(x))); }); }) } fn map_in_place(&self, logic: impl Fn(&mut D)+'static) -> Stream { + let mut vector = Vec::new(); self.unary(Pipeline, "MapInPlace", move |_,_| move |input, output| { input.for_each(|time, data| { - for datum in data.iter_mut() { logic(datum); } - output.session(&time).give_content(data); + data.swap(&mut vector); + for datum in vector.iter_mut() { logic(datum); } + output.session(&time).give_vec(&mut vector); }) }) } @@ -68,9 +72,11 @@ impl Map for Stream { // TODO : number of elements from the iterator. This would allow iterators that produce many // TODO : records without taking arbitrarily long and arbitrarily much memory. fn flat_map(&self, logic: impl Fn(D)->I+'static) -> Stream where I::Item: Data { + let mut vector = Vec::new(); self.unary(Pipeline, "FlatMap", move |_,_| move |input, output| { input.for_each(|time, data| { - output.session(&time).give_iterator(data.drain(..).flat_map(|x| logic(x).into_iter())); + data.swap(&mut vector); + output.session(&time).give_iterator(vector.drain(..).flat_map(|x| logic(x).into_iter())); }); }) } diff --git a/src/dataflow/operators/partition.rs b/src/dataflow/operators/partition.rs index 307b0c056..e01fa78d7 100644 --- a/src/dataflow/operators/partition.rs +++ b/src/dataflow/operators/partition.rs @@ -88,8 +88,12 @@ impl(u64, D2)> Operate for Operato _internal: &mut [ChangeBatch], produced: &mut [ChangeBatch]) -> bool { - while let Some((time, data)) = self.input.next() { + while let Some(message) = self.input.next() { + let outputs = self.outputs.iter_mut(); + let mut message = message.as_mut(); + let time = &message.time; + let data = &mut message.data; // TODO : This results in small sends for many parts, as sessions does the buffering let mut sessions: Vec<_> = outputs.map(|x| x.session(time)).collect(); diff --git a/src/dataflow/operators/probe.rs b/src/dataflow/operators/probe.rs index 152acbe3d..bd9130e93 100644 --- a/src/dataflow/operators/probe.rs +++ b/src/dataflow/operators/probe.rs @@ -99,6 +99,8 @@ impl Probe for Stream { let frontier = handle.frontier.clone(); let mut started = false; + let mut vector = Vec::new(); + builder.build( move |changes| { frontier.borrow_mut().update_iter(changes[0].drain()); @@ -110,8 +112,15 @@ impl Probe for Stream { started = true; } - while let Some((time, data)) = input.next() { - output.session(time).give_content(data); + use communication::message::RefOrMut; + + while let Some(message) = input.next() { + let (time, data) = match message.as_ref_or_mut() { + RefOrMut::Ref(reference) => (&reference.time, RefOrMut::Ref(&reference.data)), + RefOrMut::Mut(reference) => (&reference.time, RefOrMut::Mut(&mut reference.data)), + }; + data.swap(&mut vector); + output.session(time).give_vec(&mut vector); } output.cease(); @@ -172,7 +181,7 @@ impl Clone for Handle { #[cfg(test)] mod tests { - use ::Configuration; + use ::communication::Configuration; use ::progress::timestamp::RootTimestamp; use dataflow::operators::{Input, Probe}; diff --git a/src/dataflow/operators/reclock.rs b/src/dataflow/operators/reclock.rs index 111551e0e..a7d56f0f0 100644 --- a/src/dataflow/operators/reclock.rs +++ b/src/dataflow/operators/reclock.rs @@ -58,7 +58,7 @@ impl Reclock for Stream { // stash each data input with its timestamp. input1.for_each(|cap, data| { - stash.push((cap.time().clone(), data.take())); + stash.push((cap.time().clone(), data.replace(Vec::new()))); }); // request notification at time, to flush stash. @@ -71,7 +71,7 @@ impl Reclock for Stream { let mut session = output.session(&cap); for &mut (ref t, ref mut data) in &mut stash { if t.less_equal(cap.time()) { - session.give_content(data); + session.give_vec(data); } } stash.retain(|x| !x.0.less_equal(cap.time())); diff --git a/src/dataflow/operators/to_stream.rs b/src/dataflow/operators/to_stream.rs index 831443096..21d6f8a3a 100644 --- a/src/dataflow/operators/to_stream.rs +++ b/src/dataflow/operators/to_stream.rs @@ -3,7 +3,7 @@ use progress::Timestamp; use Data; -use dataflow::channels::Content; +use dataflow::channels::Message; use dataflow::operators::generic::operator::source; use dataflow::{Stream, Scope}; @@ -40,7 +40,7 @@ impl ToStream for I where I:: if let Some(element) = iterator.next() { let mut session = output.session(capability.as_ref().unwrap()); session.give(element); - for element in iterator.by_ref().take((256 * Content::::default_length()) - 1) { + for element in iterator.by_ref().take((256 * Message::::default_length()) - 1) { session.give(element); } } diff --git a/src/dataflow/scopes/child.rs b/src/dataflow/scopes/child.rs index 1df516b97..3e18ac682 100644 --- a/src/dataflow/scopes/child.rs +++ b/src/dataflow/scopes/child.rs @@ -5,8 +5,7 @@ use std::cell::RefCell; use progress::{Timestamp, Operate, SubgraphBuilder}; use progress::nested::{Source, Target}; use progress::nested::product::Product; -use timely_communication::{Allocate, Data}; -use {Push, Pull}; +use communication::{Allocate, Data, Push, Pull}; use logging::Logger; use super::{ScopeParent, Scope}; @@ -87,10 +86,12 @@ impl<'a, G: ScopeParent, T: Timestamp> Scope for Child<'a, G, T> { } } +use communication::Message; + impl<'a, G: ScopeParent, T: Timestamp> Allocate for Child<'a, G, T> { fn index(&self) -> usize { self.parent.index() } fn peers(&self) -> usize { self.parent.peers() } - fn allocate(&mut self) -> (Vec>>, Box>, Option) { + fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { self.parent.allocate() } } diff --git a/src/dataflow/scopes/mod.rs b/src/dataflow/scopes/mod.rs index 9ff02ade6..cacd089ab 100644 --- a/src/dataflow/scopes/mod.rs +++ b/src/dataflow/scopes/mod.rs @@ -3,7 +3,7 @@ use progress::{Timestamp, Operate}; use progress::nested::{Source, Target}; use logging::Logger; -use timely_communication::Allocate; +use communication::Allocate; pub mod root; pub mod child; diff --git a/src/dataflow/scopes/root.rs b/src/dataflow/scopes/root.rs index d1527f2f4..7749ea7ef 100644 --- a/src/dataflow/scopes/root.rs +++ b/src/dataflow/scopes/root.rs @@ -8,8 +8,7 @@ use std::any::Any; use progress::timestamp::RootTimestamp; use progress::{Timestamp, Operate, SubgraphBuilder}; use logging::Logger; -use timely_communication::{Allocate, Data}; -use {Push, Pull}; +use communication::{Allocate, Data, Push, Pull}; use super::{ScopeParent, Child}; @@ -41,6 +40,8 @@ impl Root { /// main way to ensure that a computation proceeds. pub fn step(&mut self) -> bool { + self.allocator.borrow_mut().pre_work(); + let mut active = false; for dataflow in self.dataflows.borrow_mut().iter_mut() { let sub_active = dataflow.step(); @@ -52,6 +53,8 @@ impl Root { // TODO(andreal) do we want to flush logs here? + self.allocator.borrow_mut().post_work(); + active } /// Calls `self.step()` as long as `func` evaluates to true. @@ -128,10 +131,12 @@ impl ScopeParent for Root { } } +use communication::Message; + impl Allocate for Root { fn index(&self) -> usize { self.allocator.borrow().index() } fn peers(&self) -> usize { self.allocator.borrow().peers() } - fn allocate(&mut self) -> (Vec>>, Box>, Option) { + fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { self.allocator.borrow_mut().allocate() } } diff --git a/src/dataflow/stream.rs b/src/dataflow/stream.rs index 976fad327..223917a8e 100644 --- a/src/dataflow/stream.rs +++ b/src/dataflow/stream.rs @@ -6,10 +6,10 @@ use progress::nested::subgraph::{Source, Target}; -use Push; +use communication::Push; use dataflow::Scope; use dataflow::channels::pushers::tee::TeeHelper; -use dataflow::channels::Content; +use dataflow::channels::Bundle; // use dataflow::scopes::root::loggers::CHANNELS_Q; @@ -23,7 +23,7 @@ pub struct Stream { name: Source, /// The `Scope` containing the stream. scope: S, - /// Maintains a list of Push<(T, Content)> interested in the stream's output. + /// Maintains a list of Push> interested in the stream's output. ports: TeeHelper, } @@ -32,7 +32,7 @@ impl Stream { /// /// The destination is described both by a `Target`, for progress tracking information, and a `P: Push` where the /// records should actually be sent. The identifier is unique to the edge and is used only for logging purposes. - pub fn connect_to)>+'static>(&self, target: Target, pusher: P, identifier: usize) { + pub fn connect_to>+'static>(&self, target: Target, pusher: P, identifier: usize) { let logging = self.scope().logging(); logging.when_enabled(|l| l.log(::logging::TimelyEvent::Channels(::logging::ChannelsEvent { diff --git a/src/execute.rs b/src/execute.rs index 92e1cdfe7..f365138d1 100644 --- a/src/execute.rs +++ b/src/execute.rs @@ -1,6 +1,6 @@ //! Starts a timely dataflow execution from configuration information and per-worker logic. -use timely_communication::{initialize, Configuration, Allocator, WorkerGuards}; +use communication::{initialize, Configuration, Allocator, WorkerGuards}; use dataflow::scopes::{Root, Child}; use logging::LoggerConfig; diff --git a/src/lib.rs b/src/lib.rs index 6cb8cd412..99d0a09a9 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -55,18 +55,24 @@ //! We then introduce input at increasing rounds, indicate the advance to the system (promising //! that we will introduce no more input at prior rounds), and step the computation. -#![deny(missing_docs)] +#![forbid(missing_docs)] extern crate abomonation; #[macro_use] extern crate abomonation_derive; extern crate timely_communication; extern crate time; -extern crate byteorder; +extern crate bytes; pub use execute::{execute, execute_logging, execute_from_args, execute_from_args_logging, example}; -pub use timely_communication::{Allocate, Push, Pull, Configuration}; pub use order::PartialOrder; +pub use timely_communication::Configuration; + +/// Re-export of the `timely_communication` crate. +pub mod communication { + pub use timely_communication::*; +} + pub mod progress; pub mod dataflow; pub mod execute; @@ -84,10 +90,5 @@ impl Data for T { } /// /// The `ExchangeData` trait extends `Data` with any requirements imposed by the `timely_communication` /// `Data` trait, which describes requirements for communication along channels. -pub trait ExchangeData: Data + timely_communication::Data { } -impl ExchangeData for T { } - - -// /// A composite trait for types usable in timely dataflow. -// pub trait Data: timely_communication::Data + abomonation::Abomonation { } -// impl Data for T { } +pub trait ExchangeData: Data + communication::Data { } +impl ExchangeData for T { } \ No newline at end of file diff --git a/src/logging.rs b/src/logging.rs index cbac6d508..e2dfd5daf 100644 --- a/src/logging.rs +++ b/src/logging.rs @@ -11,7 +11,7 @@ use ::progress::frontier::MutableAntichain; use dataflow::operators::capture::{Event, EventPusher}; -use timely_communication::logging::{BufferingLogger, LoggerBatch, CommsEvent, CommsSetup}; +use communication::logging::{BufferingLogger, LoggerBatch, CommsEvent, CommsSetup}; type LogMessage = (u64, TimelySetup, TimelyEvent); type CommsMessage = (u64, CommsSetup, CommsEvent); @@ -64,11 +64,11 @@ impl LoggerConfig { use std::net::TcpStream; use std::collections::HashMap; - use ::timely_communication::logging::BufferingLogger; + use ::communication::logging::BufferingLogger; use ::dataflow::operators::capture::EventWriter; let timely_stream = Mutex::new(HashMap::::new()); - let comm_stream = Mutex::new(HashMap::<::timely_communication::logging::CommsSetup, TcpStream>::new()); + let comm_stream = Mutex::new(HashMap::<::communication::logging::CommsSetup, TcpStream>::new()); ::logging::LoggerConfig { timely_logging: match ::std::env::var("TIMELY_WORKER_LOG_ADDR") { @@ -97,7 +97,7 @@ impl LoggerConfig { communication_logging: match ::std::env::var("TIMELY_COMM_LOG_ADDR") { Ok(addr) => { eprintln!("enabled COMM logging to {}", addr); - Arc::new(move |events_setup: ::timely_communication::logging::CommsSetup| { + Arc::new(move |events_setup: ::communication::logging::CommsSetup| { let send = comm_stream @@ -362,18 +362,28 @@ pub struct InputEvent { #[derive(Debug, Clone, Abomonation)] /// An event in a timely worker -#[allow(missing_docs)] pub enum TimelyEvent { + /// Operator creation. /* 0 */ Operates(OperatesEvent), + /// Channel creation. /* 1 */ Channels(ChannelsEvent), + /// Progress message send or receive. /* 2 */ Progress(ProgressEvent), + /// Progress propagation (reasoning). /* 3 */ PushProgress(PushProgressEvent), + /// Message send or receive. /* 4 */ Messages(MessagesEvent), + /// Operator start or stop. /* 5 */ Schedule(ScheduleEvent), + /// No clue. /* 6 */ Application(ApplicationEvent), + /// Per-message computation. /* 7 */ GuardedMessage(GuardedMessageEvent), + /// Per-notification computation. /* 8 */ GuardedProgress(GuardedProgressEvent), + /// Communication channel event. /* 9 */ CommChannels(CommChannelsEvent), + /// Input event. /* 10 */ Input(InputEvent), } diff --git a/src/progress/broadcast.rs b/src/progress/broadcast.rs index 69ad124a9..904973571 100644 --- a/src/progress/broadcast.rs +++ b/src/progress/broadcast.rs @@ -1,19 +1,18 @@ //! Broadcasts progress information among workers. -use progress::Timestamp; -use progress::ChangeBatch; -use timely_communication::Allocate; -use {Push, Pull}; +use progress::{ChangeBatch, Timestamp}; +use communication::{Allocate, Message, Push, Pull}; use logging::Logger; /// A list of progress updates corresponding to `((child_scope, [in/out]_port, timestamp), delta)` pub type ProgressVec = Vec<((usize, usize, T), i64)>; /// A progress update message consisting of source worker id, sequence number and lists of /// message and internal updates -pub type ProgressMsg = (usize, usize, ProgressVec, ProgressVec); +pub type ProgressMsg = Message<(usize, usize, ProgressVec, ProgressVec)>; /// Manages broadcasting of progress updates to and receiving updates from workers. pub struct Progcaster { + to_push: Option>, pushers: Vec>>>, puller: Box>>, /// Source worker index @@ -38,9 +37,16 @@ impl Progcaster { }))); let worker = allocator.index(); let addr = path.clone(); - Progcaster { pushers, puller, source: worker, - counter: 0, addr, comm_channel: chan, - logging } + Progcaster { + to_push: None, + pushers, + puller, + source: worker, + counter: 0, + addr, + comm_channel: chan, + logging, + } } /// Sends and receives progress updates, broadcasting the contents of `messages` and `internal`, @@ -64,9 +70,28 @@ impl Progcaster { }))); for pusher in self.pushers.iter_mut() { + + // Attempt to re-use allocations, if possible. + if let Some(tuple) = &mut self.to_push { + let tuple = tuple.as_mut(); + tuple.0 = self.source; + tuple.1 = self.counter; + tuple.2.clear(); tuple.2.extend(messages.iter().cloned()); + tuple.3.clear(); tuple.3.extend(internal.iter().cloned()); + } + // If we don't have an allocation ... + if self.to_push.is_none() { + self.to_push = Some(Message::from_typed(( + self.source, + self.counter, + messages.clone().into_inner(), + internal.clone().into_inner(), + ))); + } + // TODO: This should probably use a broadcast channel, or somehow serialize only once. // It really shouldn't be doing all of this cloning, that's for sure. - pusher.push(&mut Some((self.source, self.counter, messages.clone().into_inner(), internal.clone().into_inner()))); + pusher.push(&mut self.to_push); } self.counter += 1; @@ -76,14 +101,20 @@ impl Progcaster { } // TODO : Could take ownership, and recycle / reuse for next broadcast ... - while let Some((ref source, ref counter, ref mut recv_messages, ref mut recv_internal)) = *self.puller.pull() { + // while let Some((ref source, ref counter, ref mut recv_messages, ref mut recv_internal)) = *self.puller.pull() { + while let Some(message) = self.puller.pull() { + + let source = message.0; + let counter = message.1; + let recv_messages = &message.2; + let recv_internal = &message.3; let comm_channel = self.comm_channel; let addr = &mut self.addr; self.logging.when_enabled(|l| l.log(::logging::TimelyEvent::Progress(::logging::ProgressEvent { is_send: false, - source: *source, - seq_no: *counter, + source: source, + seq_no: counter, comm_channel, addr: addr.clone(), // TODO: fill with additional data diff --git a/src/progress/frontier.rs b/src/progress/frontier.rs index 9d95c0683..3d8902e2b 100644 --- a/src/progress/frontier.rs +++ b/src/progress/frontier.rs @@ -352,6 +352,15 @@ impl MutableAntichain { } self.frontier_temp.clear(); } + + /// Reports the count for a queried time. + pub fn count_for(&self, query_time: &T) -> i64 { + self.updates + .iter() + .filter(|td| td.0.eq(query_time)) + .map(|td| td.1) + .sum() + } } /// A wrapper for elements of an antichain. diff --git a/src/progress/nested/reachability.rs b/src/progress/nested/reachability.rs index 9162a73bb..0ec76dace 100644 --- a/src/progress/nested/reachability.rs +++ b/src/progress/nested/reachability.rs @@ -485,7 +485,6 @@ impl Tracker { } } - /// Adds the path summary `summary` to `target` and returns true iff a change occurred. fn add_summary(vector: &mut Vec<(Target, Antichain)>, target: Target, summary: S) -> bool { for &mut (ref t, ref mut antichain) in vector.iter_mut() { @@ -493,4 +492,4 @@ fn add_summary(vector: &mut Vec<(Target, Antichain)>, tar } vector.push((target, Antichain::from_elem(summary))); true -} +} \ No newline at end of file diff --git a/src/progress/nested/subgraph.rs b/src/progress/nested/subgraph.rs index f53e24a26..cf6c0c0b3 100644 --- a/src/progress/nested/subgraph.rs +++ b/src/progress/nested/subgraph.rs @@ -4,7 +4,7 @@ use std::rc::Rc; use std::cell::RefCell; use std::default::Default; -use timely_communication::Allocate; +use communication::Allocate; use logging::Logger; @@ -769,7 +769,12 @@ impl PerOperatorState { let _outstanding_messages = _outstanding_messages.iter().any(|chain| !chain.is_empty()); let _held_capabilities = internal_capabilities.iter().any(|chain| !chain.is_empty()); - if any_progress_updates || _was_recently_active || _outstanding_messages || _held_capabilities { + // TODO: This is reasonable, in principle, but `_outstanding_messages` determined from pointstamps + // alone leaves us in a weird state should progress messages get blocked by non-execution of + // e.g. the exchange operator in the exchange.rs example. + + if any_progress_updates || _was_recently_active || _outstanding_messages || _held_capabilities + { let self_id = self.id; // avoid capturing `self` in logging closures.