From 33b40a3b744c9935b911d10f586f8e48971fa3e8 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 20 Feb 2018 16:22:53 +0100 Subject: [PATCH 01/43] dataplane mockp --- Cargo.toml | 3 +- communication/src/allocator/generic.rs | 40 +++ communication/src/allocator/mod.rs | 6 +- communication/src/allocator/process_binary.rs | 262 ++++++++++++++++++ communication/src/initialize.rs | 28 +- communication/src/lib.rs | 11 +- communication/src/networking.rs | 8 +- src/dataflow/channels/message.rs | 15 +- src/dataflow/scopes/root.rs | 6 +- 9 files changed, 357 insertions(+), 22 deletions(-) create mode 100644 communication/src/allocator/process_binary.rs diff --git a/Cargo.toml b/Cargo.toml index 9ad6b066f..339cf4a86 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -18,7 +18,8 @@ license = "MIT" [dependencies] abomonation = "0.5" abomonation_derive = "0.3" -timely_communication = "0.5" +#timely_communication = "0.5" +timely_communication = { path = "./communication"} byteorder="0.4.2" time="0.1.34" diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index e2d3331c2..7ccc9eff4 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -4,6 +4,7 @@ //! for example closures whose type arguments must be specified. use allocator::{Allocate, Thread, Process, Binary}; +use allocator::process_binary::{ProcessBinary, ProcessBinaryBuilder}; use {Push, Pull, Data}; /// Enumerates known implementors of `Allocate`. @@ -12,6 +13,7 @@ pub enum Generic { Thread(Thread), Process(Process), Binary(Binary), + ProcessBinary(ProcessBinary), } impl Generic { @@ -21,6 +23,7 @@ impl Generic { &Generic::Thread(ref t) => t.index(), &Generic::Process(ref p) => p.index(), &Generic::Binary(ref b) => b.index(), + &Generic::ProcessBinary(ref pb) => pb.index(), } } /// The number of workers. @@ -29,6 +32,7 @@ impl Generic { &Generic::Thread(ref t) => t.peers(), &Generic::Process(ref p) => p.peers(), &Generic::Binary(ref b) => b.peers(), + &Generic::ProcessBinary(ref pb) => pb.peers(), } } /// Constructs several send endpoints and one receive endpoint. @@ -37,6 +41,18 @@ impl Generic { &mut Generic::Thread(ref mut t) => t.allocate(), &mut Generic::Process(ref mut p) => p.allocate(), &mut Generic::Binary(ref mut b) => b.allocate(), + &mut Generic::ProcessBinary(ref mut pb) => pb.allocate(), + } + } + + pub fn pre_work(&mut self) { + if let &mut Generic::ProcessBinary(ref mut pb) = self { + pb.pre_work(); + } + } + pub fn post_work(&mut self) { + if let &mut Generic::ProcessBinary(ref mut pb) = self { + pb.post_work(); } } } @@ -47,4 +63,28 @@ impl Allocate for Generic { 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(); } +} + + +/// Enumerates known implementors of `Allocate`. +/// Passes trait method calls on to members. +pub enum GenericBuilder { + Thread(Thread), + Process(Process), + Binary(Binary), + ProcessBinary(ProcessBinaryBuilder), +} + +impl GenericBuilder { + pub fn build(self) -> Generic { + match self { + GenericBuilder::Thread(t) => Generic::Thread(t), + GenericBuilder::Process(p) => Generic::Process(p), + GenericBuilder::Binary(b) => Generic::Binary(b), + GenericBuilder::ProcessBinary(pb) => Generic::ProcessBinary(pb.build()), + } + } } diff --git a/communication/src/allocator/mod.rs b/communication/src/allocator/mod.rs index 3fed04758..8840d5d36 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -3,12 +3,13 @@ pub use self::thread::Thread; pub use self::process::Process; pub use self::binary::Binary; -pub use self::generic::Generic; +pub use self::generic::{Generic, GenericBuilder}; pub mod thread; pub mod process; pub mod binary; pub mod generic; +pub mod process_binary; use {Data, Push, Pull}; @@ -22,4 +23,7 @@ pub trait Allocate { fn peers(&self) -> usize; /// Constructs several send endpoints and one receive endpoint. fn allocate(&mut self) -> (Vec>>, Box>, Option); + + fn pre_work(&mut self) { } + fn post_work(&mut self) { } } diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs new file mode 100644 index 000000000..477ed87c3 --- /dev/null +++ b/communication/src/allocator/process_binary.rs @@ -0,0 +1,262 @@ + +use std::rc::Rc; +use std::cell::RefCell; +use std::sync::mpsc::{Sender, Receiver, channel}; +use std::collections::VecDeque; + +use networking::MessageHeader; + +use {Allocate, Data, Push, Pull, Serialize}; +// use allocator::{Allocate, Thread}; +// use {Push, Pull}; + +pub struct ProcessBinaryBuilder { + index: usize, // number out of peers + peers: usize, // number of peer allocators (for typed channel allocation). + recv: Receiver>, // from all other workers. + sends: Vec>>, // to each other worker. +} + +impl ProcessBinaryBuilder { + + pub fn new_vector(count: usize) -> Vec { + + let mut sends = Vec::new(); + let mut recvs = Vec::new(); + + for _index in 0 .. count { + let (send, recv) = channel(); + sends.push(send); + recvs.push(recv); + } + + let mut result = Vec::new(); + for (index, recv) in recvs.drain(..).enumerate() { + result.push(ProcessBinaryBuilder { + index, + peers: count, + sends: sends.clone(), + recv, + }); + } + + result + } + + pub fn build(self) -> ProcessBinary { + let mut shared = Vec::new(); + for send in self.sends.into_iter() { + shared.push(Rc::new(RefCell::new(SharedByteBuffer::new(send)))); + } + + ProcessBinary { + index: self.index, + peers: self.peers, + allocated: 0, + sends: shared, + recv: self.recv, + to_local: Vec::new(), + } + } +} + +// A specific Communicator for inter-thread intra-process communication +pub struct ProcessBinary { + index: usize, // number out of peers + peers: usize, // number of peer allocators (for typed channel allocation). + allocated: usize, // indicates how many have been allocated (locally). + sends: Vec>>, // channels[x] -> goes to worker x. + recv: Receiver>, // from all other workers. + to_local: Vec>>>>,// to worker-local typed pullers. +} + +impl Allocate for ProcessBinary { + 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. + fn pre_work(&mut self) { + + while let Ok(bytes) = self.recv.try_recv() { + + // we are guaranteed that `bytes` contains exactly an integral number of messages. + // no splitting occurs across allocations. + + let mut slice = &bytes[..]; + while let Some(header) = MessageHeader::try_read(&mut slice) { + let h_len = header.length as usize; // length in bytes + let to_push = slice[..h_len].to_vec(); + slice = &slice[h_len..]; + + while self.to_local.len() <= header.channel { + self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); + } + + self.to_local[header.channel].borrow_mut().push_back(to_push); + } + + assert_eq!(slice.len(), 0); + } + } + + // Perform postparatory work, most likely sending incomplete binary buffers. + fn post_work(&mut self) { + for send in self.sends.iter_mut() { + send.borrow_mut().flush(); + } + } +} + +struct SharedByteBuffer { + sender: Sender>, // channels for each destination worker. + buffer: Vec, // working space for each destination worker. + length: usize, // currently occupied prefix bytes. + capacity: usize, // default capacity. +} + +impl SharedByteBuffer { + + // Allocates a new SharedByteBuffer with an indicated default capacity. + fn new(sender: Sender>) -> Self { + SharedByteBuffer { + sender, + buffer: vec![0; 1 << 10], + length: 0, + capacity: 1 << 10, + } + } + + // Retrieve a writeable buffer of `size` bytes. + // + // This may result in the current working buffer being sent and a new buffer + // being acquired or allocated. + fn reserve(&mut self, size: usize) -> &mut [u8] { + + if self.length + size > self.buffer.len() { + + // if we have filled our current buffer, double the capacity of the next buffer. + // current rule of thumb: double up to 1 << 20 bytes, stop doubling then. + if self.capacity < (1 << 20) { + self.capacity *= 2; + } + + let capacity = ::std::cmp::max(self.capacity, size.next_power_of_two()); + let new_buffer = vec![0; capacity]; + let mut old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + old_buffer.truncate(self.length); + if old_buffer.len() > 0 { + self.sender.send(old_buffer).expect("Reserve: failed to send."); + } + self.length = 0; + } + + let offset = self.length; + self.length += size; + &mut self.buffer[offset..][..size] + } + + // Push out all pending byte buffer sends. + fn flush(&mut self) { + if self.length > 0 { + // reduce the capacity by a factor of two, for kicks. + if self.capacity > 1 { self.capacity /= 2; } + + let new_buffer = vec![0; self.capacity]; + let mut old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + old_buffer.truncate(self.length); + if old_buffer.len() > 0 { + self.sender.send(old_buffer).expect("Flush: failed to send."); + } + self.length = 0; + } + } +} + + +struct Pusher { + header: MessageHeader, + sender: Rc>, + phantom: ::std::marker::PhantomData, +} + +impl Pusher { + 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 element_length = element.length_in_bytes(); + let mut header = self.header; + self.header.seqno += 1; + header.length = element_length; + + // acquire byte buffer and write header, element. + let mut borrow = self.sender.borrow_mut(); + let mut bytes = borrow.reserve(header.required_bytes()); + assert_eq!(bytes.len(), header.required_bytes()); + // println!("allocated {} bytes for {}", bytes.len(), header.required_bytes()); + header.write_to(&mut bytes).expect("failed to write header!"); + element.into_bytes(&mut bytes); + assert_eq!(bytes.len(), 0); + } + } +} + +struct Puller { + current: Option, + receiver: Rc>>>, // source of serialized buffers +} +impl Puller { + 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(|mut bytes| ::from_bytes(&mut bytes)); + &mut self.current + } +} \ No newline at end of file diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index 1efd32110..b8647617e 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -5,7 +5,8 @@ use std::io::BufRead; use getopts; use std::sync::Arc; -use allocator::{Thread, Process, Generic}; +use allocator::{Thread, Process, Generic, GenericBuilder}; +use allocator::process_binary::ProcessBinaryBuilder; use networking::initialize_networking; /// Possible configurations for the communication infrastructure. @@ -72,13 +73,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,String> { match config { - Configuration::Thread => Ok(vec![Generic::Thread(Thread)]), - Configuration::Process(threads) => Ok(Process::new_vector(threads).into_iter().map(|x| Generic::Process(x)).collect()), + Configuration::Thread => { + // Ok(vec![GenericBuilder::Thread(Thread)]) + 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()) + Ok(ProcessBinaryBuilder::new_vector(threads).into_iter().map(|x| GenericBuilder::ProcessBinary(x)).collect()) + }, Configuration::Cluster(threads, process, addresses, report) => { if let Ok(stuff) = initialize_networking(addresses, process, threads, report, logger) { - Ok(stuff.into_iter().map(|x| Generic::Binary(x)).collect()) + Ok(stuff.into_iter().map(|x| GenericBuilder::Binary(x)).collect()) } else { Err("failed to initialize networking".to_owned()) @@ -151,7 +160,7 @@ 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> { @@ -159,12 +168,13 @@ pub fn initializeT+Send+Sync+'static>( let logic = Arc::new(func); let mut guards = Vec::new(); - for allocator in allocators.into_iter() { + for (index, builder) in allocators.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)))); } diff --git a/communication/src/lib.rs b/communication/src/lib.rs index 957c2d63e..b3053420e 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -82,7 +82,7 @@ mod drain; pub mod logging; use std::any::Any; -use abomonation::{Abomonation, encode, decode}; +use abomonation::{Abomonation, encode, decode, measure}; pub use allocator::Generic as Allocator; pub use allocator::Allocate; @@ -98,9 +98,11 @@ impl Data for T { } /// /// A default implementation is provided for any `T: Abomonation+Clone`. pub trait Serialize { + /// Number of bytes required for the binary representation of the object. + fn length_in_bytes(&self) -> usize; /// 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); + fn into_bytes(&mut self, &mut W); /// 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; @@ -109,7 +111,10 @@ pub trait Serialize { // 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) { + fn length_in_bytes(&self) -> usize { + measure(self) + } + fn into_bytes(&mut self, bytes: &mut W) { // NOTE: `unwrap` should be ok, as Rust docs say writes to `Vec` do not fail. unsafe { encode(self, bytes).unwrap(); } } diff --git a/communication/src/networking.rs b/communication/src/networking.rs index 8b0ee41aa..f40f158ec 100644 --- a/communication/src/networking.rs +++ b/communication/src/networking.rs @@ -31,7 +31,7 @@ pub struct MessageHeader { impl MessageHeader { // returns a header when there is enough supporting data - fn try_read(bytes: &mut &[u8]) -> Option { + pub fn try_read(bytes: &mut &[u8]) -> Option { if bytes.len() > size_of::() { // capture original in case we need to rewind let original = *bytes; @@ -61,7 +61,7 @@ impl MessageHeader { else { None } } - fn write_to(&self, writer: &mut W) -> Result<()> { + pub 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)); @@ -69,6 +69,10 @@ impl MessageHeader { try!(writer.write_u64::(self.seqno as u64)); Ok(()) } + + pub fn required_bytes(&self) -> usize { + 40 + self.length + } } // structure in charge of receiving data from a Reader, for example the network diff --git a/src/dataflow/channels/message.rs b/src/dataflow/channels/message.rs index b1ef3ddbe..c5f75ad30 100644 --- a/src/dataflow/channels/message.rs +++ b/src/dataflow/channels/message.rs @@ -36,12 +36,17 @@ impl Message { // Implementation required to get different behavior out of communication fabric. impl Serialize for Message { + + #[inline] + fn length_in_bytes(&self) -> usize { + measure(&self.time) + measure(&self.from) + measure(&self.seq) + measure(self.data.deref()) + } + #[inline] - fn into_bytes(&mut self, bytes: &mut Vec) { + fn into_bytes(&mut self, bytes: &mut W) { - // 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); + // // 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()); // Almost like serializing `self`, except `self.data` is special. unsafe { encode(&self.time, bytes).unwrap(); } @@ -74,7 +79,7 @@ impl Serialize for Message { pub enum Content { /// A serialized representation of data. /// - /// This representation may be efficiently observed as shared references, + /// 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. diff --git a/src/dataflow/scopes/root.rs b/src/dataflow/scopes/root.rs index 87fa67cc7..ff430358c 100644 --- a/src/dataflow/scopes/root.rs +++ b/src/dataflow/scopes/root.rs @@ -39,10 +39,12 @@ impl Root { /// Performs one step of the computation. /// - /// A step gives each dataflow operator a chance to run, and is the + /// A step gives each dataflow operator a chance to run, and is the /// main way to ensure that a computation procedes. 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(); @@ -54,6 +56,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. From 96748caff700d8b8a1f9ef3155ba2ba659097613 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 22 Feb 2018 05:09:24 +0100 Subject: [PATCH 02/43] ByteExchange added --- communication/src/allocator/process_binary.rs | 206 ++++++++++++------ communication/src/networking.rs | 3 + examples/pingpong.rs | 7 +- src/dataflow/channels/message.rs | 1 - src/dataflow/channels/pushers/exchange.rs | 6 +- src/dataflow/channels/pushers/tee.rs | 22 +- src/dataflow/operators/input.rs | 13 +- 7 files changed, 169 insertions(+), 89 deletions(-) diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index 477ed87c3..840d22ad7 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -7,14 +7,69 @@ use std::collections::VecDeque; use networking::MessageHeader; use {Allocate, Data, Push, Pull, Serialize}; -// use allocator::{Allocate, Thread}; -// use {Push, Pull}; + +/// A unidirectional bulk byte channel +/// +/// This channel means to accept large owned `Vec` buffers and hand them off to another +/// thread, who consumes them and returns emptied allocations. The intent here is that the +/// sender can notice when the channel is not empty and can consolidate writes to a large +/// buffer until that changes. +pub struct ByteExchange { + send: Sender>, + recv: Receiver>, + balance: usize, // number sent minus number received back. +} + +impl ByteExchange { + + pub fn new_pair() -> (Self, Self) { + + let (send1, recv1) = channel(); + let (send2, recv2) = channel(); + + let result1 = ByteExchange { send: send1, recv: recv2, balance: 0, }; + let result2 = ByteExchange { send: send2, recv: recv1, balance: 0, }; + + (result1, result2) + } + + #[inline(always)] + pub fn queue_length(&self) -> usize { + self.balance + } + pub fn send(&mut self, bytes: Vec) { + // println!("sending: {:?}", bytes.len()); + // non-empty bytes are expected to return. + if bytes.len() > 0 { + // println!("incrementing balance: {:?}", self.balance); + self.balance += 1; + } + self.send + .send(bytes) + .expect("failed to send!"); + } + pub fn recv(&mut self) -> Option> { + if let Ok(bytes) = self.recv.try_recv() { + // println!("recving: {:?}", bytes.len()); + // empty bytes are returend buffers. + if bytes.len() == 0 { + assert!(self.balance > 0); + // println!("decrementing balance: {:?}", self.balance); + self.balance -= 1; + } + Some(bytes) + } + else { + None + } + } +} pub struct ProcessBinaryBuilder { index: usize, // number out of peers peers: usize, // number of peer allocators (for typed channel allocation). - recv: Receiver>, // from all other workers. - sends: Vec>>, // to each other worker. + sends: Vec, // with each other worker (for pushing bytes) + recvs: Vec, // with each other worker (for pulling bytes) } impl ProcessBinaryBuilder { @@ -23,21 +78,25 @@ impl ProcessBinaryBuilder { let mut sends = Vec::new(); let mut recvs = Vec::new(); - - for _index in 0 .. count { - let (send, recv) = channel(); - sends.push(send); - recvs.push(recv); + 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, recv) = ByteExchange::new_pair(); + sends[source].push(send); + recvs[target].push(recv); + } } let mut result = Vec::new(); - for (index, recv) in recvs.drain(..).enumerate() { + for (index, (sends, recvs)) in sends.drain(..).zip(recvs.drain(..)).enumerate() { result.push(ProcessBinaryBuilder { index, peers: count, - sends: sends.clone(), - recv, - }); + sends, + recvs, + }) } result @@ -54,7 +113,7 @@ impl ProcessBinaryBuilder { peers: self.peers, allocated: 0, sends: shared, - recv: self.recv, + recvs: self.recvs, to_local: Vec::new(), } } @@ -66,7 +125,7 @@ pub struct ProcessBinary { peers: usize, // number of peer allocators (for typed channel allocation). allocated: usize, // indicates how many have been allocated (locally). sends: Vec>>, // channels[x] -> goes to worker x. - recv: Receiver>, // from all other workers. + recvs: Vec, // from all other workers. to_local: Vec>>>>,// to worker-local typed pullers. } @@ -104,28 +163,36 @@ impl Allocate for ProcessBinary { (pushes, puller, None) } - // Perform preparatory work, most likely reading binary buffers from self.recv. + // Perform preparatory work, most likely reading binary buffers from self.recv. + #[inline(never)] fn pre_work(&mut self) { - while let Ok(bytes) = self.recv.try_recv() { + for recv in self.recvs.iter_mut() { + + while let Some(mut bytes) = recv.recv() { - // we are guaranteed that `bytes` contains exactly an integral number of messages. - // no splitting occurs across allocations. + // we are guaranteed that `bytes` contains exactly an integral number of messages. + // no splitting occurs across allocations. - let mut slice = &bytes[..]; - while let Some(header) = MessageHeader::try_read(&mut slice) { - let h_len = header.length as usize; // length in bytes - let to_push = slice[..h_len].to_vec(); - slice = &slice[h_len..]; + { + let mut slice = &bytes[..]; + while let Some(header) = MessageHeader::try_read(&mut slice) { + let h_len = header.length as usize; // length in bytes + let to_push = slice[..h_len].to_vec(); + slice = &slice[h_len..]; - while self.to_local.len() <= header.channel { - self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); + while self.to_local.len() <= header.channel { + self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); + } + + self.to_local[header.channel].borrow_mut().push_back(to_push); + } + assert_eq!(slice.len(), 0); } - self.to_local[header.channel].borrow_mut().push_back(to_push); + bytes.clear(); + recv.send(bytes); } - - assert_eq!(slice.len(), 0); } } @@ -138,66 +205,78 @@ impl Allocate for ProcessBinary { } struct SharedByteBuffer { - sender: Sender>, // channels for each destination worker. - buffer: Vec, // working space for each destination worker. - length: usize, // currently occupied prefix bytes. - capacity: usize, // default capacity. + sender: ByteExchange, // channels for each destination worker. + buffer: Vec, // working space for each destination worker. + stash: Vec>, } impl SharedByteBuffer { // Allocates a new SharedByteBuffer with an indicated default capacity. - fn new(sender: Sender>) -> Self { + fn new(sender: ByteExchange) -> Self { SharedByteBuffer { sender, - buffer: vec![0; 1 << 10], - length: 0, - capacity: 1 << 10, + buffer: Vec::with_capacity(1 << 20), + stash: Vec::new(), } } - // Retrieve a writeable buffer of `size` bytes. + // Retrieve a writeable buffer with at least `size` bytes available capacity. // // This may result in the current working buffer being sent and a new buffer // being acquired or allocated. - fn reserve(&mut self, size: usize) -> &mut [u8] { + fn reserve(&mut self, size: usize) -> &mut Vec { - if self.length + size > self.buffer.len() { + if self.buffer.len() + size > self.buffer.capacity() { - // if we have filled our current buffer, double the capacity of the next buffer. - // current rule of thumb: double up to 1 << 20 bytes, stop doubling then. - if self.capacity < (1 << 20) { - self.capacity *= 2; + // if we need more space than we expect from our byte exchange, ... + let new_buffer = if size > (1 << 20) { + Vec::with_capacity(size) } + else { + if self.stash.len() > 0 { self.stash.pop().unwrap() } + else { + self.sender.recv().unwrap_or_else(|| { println!("allocating"); Vec::with_capacity(1 << 20) }) + } + }; - let capacity = ::std::cmp::max(self.capacity, size.next_power_of_two()); - let new_buffer = vec![0; capacity]; - let mut old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); - old_buffer.truncate(self.length); + assert!(new_buffer.is_empty()); + assert!(new_buffer.capacity() >= size); + + let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); if old_buffer.len() > 0 { - self.sender.send(old_buffer).expect("Reserve: failed to send."); + self.sender.send(old_buffer); } - self.length = 0; } - let offset = self.length; - self.length += size; - &mut self.buffer[offset..][..size] + &mut self.buffer } // Push out all pending byte buffer sends. fn flush(&mut self) { - if self.length > 0 { - // reduce the capacity by a factor of two, for kicks. - if self.capacity > 1 { self.capacity /= 2; } + if self.buffer.len() > 0 { - let new_buffer = vec![0; self.capacity]; - let mut old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); - old_buffer.truncate(self.length); - if old_buffer.len() > 0 { - self.sender.send(old_buffer).expect("Flush: failed to send."); + while let Some(bytes) = self.sender.recv() { + self.stash.push(bytes); + } + + // only ship the buffer if the recipient has consumed everything we've sent them. + // otherwise, wait for the first flush call when they have (they should eventually). + if self.sender.queue_length() == 0 { + let new_buffer = if self.stash.len() > 0 { + self.stash.pop().unwrap() + } + else { + self.sender.recv().unwrap_or_else(|| { println!("allocating"); Vec::with_capacity(1 << 20) }) + }; + let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + if old_buffer.len() > 0 { + self.sender.send(old_buffer); + } + } + else { + // println!("work to send, but queue length: {:?}", self.sender.queue_length()); } - self.length = 0; } } } @@ -233,11 +312,8 @@ impl Push for Pusher { // acquire byte buffer and write header, element. let mut borrow = self.sender.borrow_mut(); let mut bytes = borrow.reserve(header.required_bytes()); - assert_eq!(bytes.len(), header.required_bytes()); - // println!("allocated {} bytes for {}", bytes.len(), header.required_bytes()); header.write_to(&mut bytes).expect("failed to write header!"); element.into_bytes(&mut bytes); - assert_eq!(bytes.len(), 0); } } } diff --git a/communication/src/networking.rs b/communication/src/networking.rs index f40f158ec..a46d838df 100644 --- a/communication/src/networking.rs +++ b/communication/src/networking.rs @@ -31,6 +31,7 @@ pub struct MessageHeader { impl MessageHeader { // returns a header when there is enough supporting data + #[inline(always)] pub fn try_read(bytes: &mut &[u8]) -> Option { if bytes.len() > size_of::() { // capture original in case we need to rewind @@ -61,6 +62,7 @@ impl MessageHeader { else { None } } + #[inline(always)] pub fn write_to(&self, writer: &mut W) -> Result<()> { try!(writer.write_u64::(self.channel as u64)); try!(writer.write_u64::(self.source as u64)); @@ -70,6 +72,7 @@ impl MessageHeader { Ok(()) } + #[inline(always)] pub fn required_bytes(&self) -> usize { 40 + self.length } 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/src/dataflow/channels/message.rs b/src/dataflow/channels/message.rs index c5f75ad30..c569e3117 100644 --- a/src/dataflow/channels/message.rs +++ b/src/dataflow/channels/message.rs @@ -204,7 +204,6 @@ impl DerefMut for Content { 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())) } diff --git a/src/dataflow/channels/pushers/exchange.rs b/src/dataflow/channels/pushers/exchange.rs index 687770273..82ccc2909 100644 --- a/src/dataflow/channels/pushers/exchange.rs +++ b/src/dataflow/channels/pushers/exchange.rs @@ -1,4 +1,4 @@ -//! The exchange pattern distributes pushed data between many target pushees. +//! The exchange pattern distributes pushed data between many target pushees. use {Push, Data}; use dataflow::channels::Content; @@ -38,7 +38,7 @@ impl)>, H: Fn(&T, &D)->u64> Exchange)>, H: Fn(&T, &D)->u64> Push<(T, Content)> for Exchange { - #[inline] + #[inline(never)] fn push(&mut self, message: &mut Option<(T, Content)>) { // if only one pusher, no exchange if self.pushers.len() == 1 { @@ -72,7 +72,7 @@ impl)>, H: Fn(& // self.flush(index); // } // } - + } } // as a last resort, use mod (%) diff --git a/src/dataflow/channels/pushers/tee.rs b/src/dataflow/channels/pushers/tee.rs index 311069afe..379d2b1f5 100644 --- a/src/dataflow/channels/pushers/tee.rs +++ b/src/dataflow/channels/pushers/tee.rs @@ -17,24 +17,22 @@ pub struct Tee { impl Push<(T, Content)> for Tee { #[inline] fn push(&mut self, message: &mut Option<(T, Content)>) { + let mut pushers = self.shared.borrow_mut(); 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]); - } + for index in 1..pushers.len() { + self.buffer.extend_from_slice(data); + Content::push_at(&mut self.buffer, (*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); + } } } diff --git a/src/dataflow/operators/input.rs b/src/dataflow/operators/input.rs index 663a6b274..4ae588ae6 100644 --- a/src/dataflow/operators/input.rs +++ b/src/dataflow/operators/input.rs @@ -66,7 +66,7 @@ pub trait Input<'a, A: Allocate, T: Timestamp> { /// Create a new stream from a supplied interactive handle. /// /// This method creates a new timely stream whose data are supplied interactively through the `handle` - /// argument. Each handle may be used multiple times (or not at all), and will clone data as appropriate + /// argument. Each handle may be used multiple times (or not at all), and will clone data as appropriate /// if it as attached to more than one stream. /// /// #Examples @@ -121,7 +121,7 @@ impl<'a, A: Allocate, T: Timestamp> Input<'a, A, T> for Child<'a, Root, T> { copies: copies, }); - Stream::new(Source { index: index, port: 0 }, registrar, self.clone()) + Stream::new(Source { index: index, port: 0 }, registrar, self.clone()) } } @@ -229,7 +229,7 @@ impl Handle { /// } /// }); /// ``` - pub fn to_stream<'a, A: Allocate>(&mut self, scope: &mut Child<'a, Root, T>) -> Stream, T>, D> + pub fn to_stream<'a, A: Allocate>(&mut self, scope: &mut Child<'a, Root, T>) -> Stream, T>, D> where T: Ord { scope.input_from(self) } @@ -252,16 +252,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()); + 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()); + debug_assert!(self.buffer1.is_empty()); } } self.buffer1.clear(); @@ -281,7 +282,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() == Content::::default_length()); self.buffer1.push(data); if self.buffer1.len() == self.buffer1.capacity() { self.flush(); From 8330cb8c6537508a71a2c14772674d64292017d5 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 22 Feb 2018 05:24:21 +0100 Subject: [PATCH 03/43] don't panic on shutdown returning empty buffers --- communication/src/allocator/process_binary.rs | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index 840d22ad7..c6e30763b 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -38,23 +38,24 @@ impl ByteExchange { self.balance } pub fn send(&mut self, bytes: Vec) { - // println!("sending: {:?}", bytes.len()); // non-empty bytes are expected to return. if bytes.len() > 0 { - // println!("incrementing balance: {:?}", self.balance); self.balance += 1; + self.send + .send(bytes) + .expect("failed to send byte buffer!"); + } + else { + // failing to return a byte buffer because the other side has hung up + // should not be a fatal event. + let _ = self.send.send(bytes); } - self.send - .send(bytes) - .expect("failed to send!"); } pub fn recv(&mut self) -> Option> { if let Ok(bytes) = self.recv.try_recv() { - // println!("recving: {:?}", bytes.len()); // empty bytes are returend buffers. if bytes.len() == 0 { assert!(self.balance > 0); - // println!("decrementing balance: {:?}", self.balance); self.balance -= 1; } Some(bytes) From 666a710bd4c2cbe12d6a4324a15e73259fe7c94b Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 22 Feb 2018 19:23:35 +0100 Subject: [PATCH 04/43] updates --- communication/src/allocator/process_binary.rs | 7 ++++++- src/dataflow/channels/pushers/exchange.rs | 19 +++++++++---------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index c6e30763b..53a6667cd 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -333,7 +333,12 @@ impl Pull for Puller { #[inline] fn pull(&mut self) -> &mut Option { - self.current = self.receiver.borrow_mut().pop_front().map(|mut bytes| ::from_bytes(&mut bytes)); + self.current = + self.receiver + .borrow_mut() + .pop_front() + .map(|mut bytes| ::from_bytes(&mut bytes)); + &mut self.current } } \ No newline at end of file diff --git a/src/dataflow/channels/pushers/exchange.rs b/src/dataflow/channels/pushers/exchange.rs index 82ccc2909..6274c1c10 100644 --- a/src/dataflow/channels/pushers/exchange.rs +++ b/src/dataflow/channels/pushers/exchange.rs @@ -60,19 +60,18 @@ impl)>, H: Fn(& let mask = (self.pushers.len() - 1) as u64; for datum in data.drain(..) { let index = (((self.hash_func)(time, &datum)) & mask) as usize; + // self.buffers[index].push(datum); + // if self.buffers[index].len() == self.buffers[index].capacity() { + // self.flush(index); + // } - self.buffers[index].push(datum); - if self.buffers[index].len() == self.buffers[index].capacity() { - self.flush(index); + unsafe { + self.buffers.get_unchecked_mut(index).push(datum); + if self.buffers.get_unchecked(index).len() == self.buffers.get_unchecked(index).capacity() { + self.flush(index); + } } - // unsafe { - // self.buffers.get_unchecked_mut(index).push(datum); - // if self.buffers.get_unchecked(index).len() == self.buffers.get_unchecked(index).capacity() { - // self.flush(index); - // } - // } - } } // as a last resort, use mod (%) From f895f28965ac0a4b4b0d82b45b7c5b931f1a25d1 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 7 May 2018 15:40:05 +0200 Subject: [PATCH 05/43] default size --- communication/src/allocator/process_binary.rs | 65 ++++++++++++------- 1 file changed, 41 insertions(+), 24 deletions(-) diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index 53a6667cd..3491d4d0e 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -43,7 +43,7 @@ impl ByteExchange { self.balance += 1; self.send .send(bytes) - .expect("failed to send byte buffer!"); + .expect("ByteExchange::send(): failed to send byte buffer!"); } else { // failing to return a byte buffer because the other side has hung up @@ -53,7 +53,7 @@ impl ByteExchange { } pub fn recv(&mut self) -> Option> { if let Ok(bytes) = self.recv.try_recv() { - // empty bytes are returend buffers. + // empty bytes are returned buffers. if bytes.len() == 0 { assert!(self.balance > 0); self.balance -= 1; @@ -206,54 +206,76 @@ impl Allocate for ProcessBinary { } struct SharedByteBuffer { - sender: ByteExchange, // channels for each destination worker. - buffer: Vec, // working space for each destination worker. - stash: Vec>, + sender: ByteExchange, // channels for each destination worker. + buffer: Vec, // working space for each destination worker. + stash: Vec>, // spare buffers + default_size: usize, } impl SharedByteBuffer { // Allocates a new SharedByteBuffer with an indicated default capacity. fn new(sender: ByteExchange) -> Self { + let default_size = 1 << 20; SharedByteBuffer { sender, - buffer: Vec::with_capacity(1 << 20), + buffer: Vec::with_capacity(default_size), stash: Vec::new(), + default_size, } } - // Retrieve a writeable buffer with at least `size` bytes available capacity. - // - // This may result in the current working buffer being sent and a new buffer - // being acquired or allocated. + /// Acquires a fresh buffer. + /// + /// The fresh buffer is acquired first from the local stash, then from the + /// return channel, and lastly allocated if no other buffers are available. + fn get_buffer(&mut self) -> Vec { + if self.stash.len() > 0 { + self.stash.pop().unwrap() + } + else { + self.sender.recv().unwrap_or_else(|| { println!("allocating"); Vec::with_capacity(self.default_size) }) + } + } + + /// Retrieve a writeable buffer with at least `size` bytes available capacity. + /// + /// This may result in the current working buffer being sent and a new buffer + /// being acquired or allocated. fn reserve(&mut self, size: usize) -> &mut Vec { if self.buffer.len() + size > self.buffer.capacity() { // if we need more space than we expect from our byte exchange, ... - let new_buffer = if size > (1 << 20) { + let new_buffer = if size > self.default_size { Vec::with_capacity(size) } else { - if self.stash.len() > 0 { self.stash.pop().unwrap() } - else { - self.sender.recv().unwrap_or_else(|| { println!("allocating"); Vec::with_capacity(1 << 20) }) - } + self.get_buffer() }; - assert!(new_buffer.is_empty()); - assert!(new_buffer.capacity() >= size); + debug_assert!(new_buffer.is_empty()); + debug_assert!(new_buffer.capacity() >= size); let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); if old_buffer.len() > 0 { self.sender.send(old_buffer); } + else { + // We should stash `old_buffer`, if the right size. + if old_buffer.capacity() == self.default_size { + self.stash.push(old_buffer); + } + } } &mut self.buffer } - // Push out all pending byte buffer sends. + /// Push out all pending byte buffer sends. + /// + /// This has the effect of emptying `self.buffer`, most likely by sending + /// it through `self.sender`. fn flush(&mut self) { if self.buffer.len() > 0 { @@ -264,12 +286,7 @@ impl SharedByteBuffer { // only ship the buffer if the recipient has consumed everything we've sent them. // otherwise, wait for the first flush call when they have (they should eventually). if self.sender.queue_length() == 0 { - let new_buffer = if self.stash.len() > 0 { - self.stash.pop().unwrap() - } - else { - self.sender.recv().unwrap_or_else(|| { println!("allocating"); Vec::with_capacity(1 << 20) }) - }; + let new_buffer = self.get_buffer(); let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); if old_buffer.len() > 0 { self.sender.send(old_buffer); From f7574fe99e7996c376cb00c6c55215f4705bc796 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 10 May 2018 12:19:55 +0200 Subject: [PATCH 06/43] remove vestigial drain --- communication/src/drain.rs | 91 --------------------------------- communication/src/lib.rs | 1 - communication/src/networking.rs | 3 +- 3 files changed, 1 insertion(+), 94 deletions(-) delete mode 100644 communication/src/drain.rs 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/lib.rs b/communication/src/lib.rs index 8338bb303..d2f79559a 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -79,7 +79,6 @@ extern crate time; pub mod allocator; mod networking; pub mod initialize; -mod drain; pub mod logging; use std::any::Any; diff --git a/communication/src/networking.rs b/communication/src/networking.rs index 025d412aa..f8bf4f415 100644 --- a/communication/src/networking.rs +++ b/communication/src/networking.rs @@ -13,7 +13,6 @@ use std::time::Duration; 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. @@ -178,7 +177,7 @@ impl BinarySender { stash.push((header, buffer)); } - for (header, mut buffer) in stash.drain_temp() { + for (header, mut buffer) in stash.drain(..) { assert!(header.length == buffer.len()); self.log_sender.when_enabled(|l| l.log(::logging::CommsEvent::Communication(::logging::CommunicationEvent { is_send: true, From 2850a710a6febfddfaed78d8cfa84510526e9cef Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Fri, 11 May 2018 17:12:29 +0200 Subject: [PATCH 07/43] new trait --- bytes/src/lib.rs | 20 +- communication/src/allocator/generic.rs | 25 +- communication/src/allocator/process_binary.rs | 324 +++++++++--------- 3 files changed, 192 insertions(+), 177 deletions(-) diff --git a/bytes/src/lib.rs b/bytes/src/lib.rs index 3a3a0cfd9..39992fe28 100644 --- a/bytes/src/lib.rs +++ b/bytes/src/lib.rs @@ -44,7 +44,7 @@ pub mod rc { use std::rc::Rc; /// 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. @@ -57,10 +57,10 @@ pub mod 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 { let mut rc = Rc::new(bytes); @@ -109,14 +109,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) } } @@ -130,7 +130,7 @@ pub mod arc { use std::sync::Arc; /// 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. @@ -143,10 +143,10 @@ pub mod arc { sequestered: Arc, } - 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 { let mut arc = Arc::new(bytes); @@ -195,14 +195,14 @@ pub mod arc { } } - 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) } } diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index 7ccc9eff4..33dacdd59 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -13,7 +13,7 @@ pub enum Generic { Thread(Thread), Process(Process), Binary(Binary), - ProcessBinary(ProcessBinary), + ProcessBinary(ProcessBinary<::allocator::process_binary::vec::VecBytesExchange>), } impl Generic { @@ -46,13 +46,19 @@ impl Generic { } pub fn pre_work(&mut self) { - if let &mut Generic::ProcessBinary(ref mut pb) = self { - pb.pre_work(); + match self { + &mut Generic::Thread(ref mut t) => t.pre_work(), + &mut Generic::Process(ref mut p) => p.pre_work(), + &mut Generic::Binary(ref mut b) => b.pre_work(), + &mut Generic::ProcessBinary(ref mut pb) => pb.pre_work(), } } pub fn post_work(&mut self) { - if let &mut Generic::ProcessBinary(ref mut pb) = self { - pb.post_work(); + match self { + &mut Generic::Thread(ref mut t) => t.post_work(), + &mut Generic::Process(ref mut p) => p.post_work(), + &mut Generic::Binary(ref mut b) => b.post_work(), + &mut Generic::ProcessBinary(ref mut pb) => pb.post_work(), } } } @@ -69,13 +75,16 @@ impl Allocate for Generic { } -/// Enumerates known implementors of `Allocate`. -/// Passes trait method calls on to members. +/// 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 { Thread(Thread), Process(Process), Binary(Binary), - ProcessBinary(ProcessBinaryBuilder), + ProcessBinary(ProcessBinaryBuilder<::allocator::process_binary::vec::VecBytesExchange>), } impl GenericBuilder { diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index 3491d4d0e..106e7bd50 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -1,81 +1,172 @@ use std::rc::Rc; use std::cell::RefCell; -use std::sync::mpsc::{Sender, Receiver, channel}; use std::collections::VecDeque; +use std::ops::DerefMut; use networking::MessageHeader; use {Allocate, Data, Push, Pull, Serialize}; -/// A unidirectional bulk byte channel -/// -/// This channel means to accept large owned `Vec` buffers and hand them off to another -/// thread, who consumes them and returns emptied allocations. The intent here is that the -/// sender can notice when the channel is not empty and can consolidate writes to a large -/// buffer until that changes. -pub struct ByteExchange { - send: Sender>, - recv: Receiver>, - balance: usize, // number sent minus number received back. +/// Reports the remaining capacity. +pub trait Available: ::std::io::Write { + /// Reports the remaining capacity. + /// + /// The instance should be able to accept `self.available()` bytes written at it + /// without complaint, and can have any (correct) behavior it likes on additional + /// bytes written. + fn available(&self) -> usize; } -impl ByteExchange { +impl Available for Vec { + fn available(&self) -> usize { self.capacity() - self.len() } +} - pub fn new_pair() -> (Self, Self) { +pub trait BytesExchange { + type Send: BytesSendEndpoint+'static; + type Recv: BytesRecvEndpoint+'static; + fn new() -> (Self::Send, Self::Recv); +} - let (send1, recv1) = channel(); - let (send2, recv2) = channel(); +pub trait BytesSendEndpoint { + type SendBuffer: Available; + fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; + fn publish(&mut self); +} - let result1 = ByteExchange { send: send1, recv: recv2, balance: 0, }; - let result2 = ByteExchange { send: send2, recv: recv1, balance: 0, }; +pub trait BytesRecvEndpoint { + type RecvBuffer: DerefMut; + fn receive(&mut self) -> Option; + fn recycle(&mut self, buffer: Self::RecvBuffer); +} - (result1, result2) +pub mod vec { + + use std::sync::mpsc::{Sender, Receiver, channel}; + + use super::{BytesExchange, BytesSendEndpoint, BytesRecvEndpoint}; + + pub struct VecSendEndpoint { + send: Sender>, // send full vectors + recv: Receiver>, // recv empty vectors + balance: usize, // #sent - #recv. + + buffer: Vec, + stash: Vec>, // spare buffers + default_size: usize, } - #[inline(always)] - pub fn queue_length(&self) -> usize { - self.balance + impl VecSendEndpoint { + /// Drains `self.recv` of empty buffers, stashes them. + fn drain_recv(&mut self) { + while let Ok(bytes) = self.recv.try_recv() { + self.balance -= 1; + if bytes.capacity() == self.default_size { + self.stash.push(bytes); + } + } + } + /// Moves `self.buffer` into `self.send`, replaces with empty buffer. + fn send_buffer(&mut self) { + let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); + if buffer.len() > 0 { + self.send.send(buffer).expect("VecSendEndpoint::send_buffer(): failed to send buffer"); + self.balance += 1; + } + else { + if buffer.capacity() == self.default_size { + self.stash.push(buffer); + } + } + } } - pub fn send(&mut self, bytes: Vec) { - // non-empty bytes are expected to return. - if bytes.len() > 0 { - self.balance += 1; - self.send - .send(bytes) - .expect("ByteExchange::send(): failed to send byte buffer!"); + + impl BytesSendEndpoint for VecSendEndpoint { + + type SendBuffer = Vec; + + fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { + if self.buffer.capacity() < capacity + self.buffer.len() { + self.send_buffer(); + if capacity > self.default_size { + self.buffer = Vec::with_capacity(capacity); + } + else { + self.drain_recv(); + self.buffer = self.stash.pop().unwrap_or_else(|| Vec::with_capacity(self.default_size)) + } + } + + &mut self.buffer } - else { - // failing to return a byte buffer because the other side has hung up - // should not be a fatal event. - let _ = self.send.send(bytes); + + fn publish(&mut self) { + self.drain_recv(); + if self.balance == 0 { + self.send_buffer(); + } } } - pub fn recv(&mut self) -> Option> { - if let Ok(bytes) = self.recv.try_recv() { - // empty bytes are returned buffers. - if bytes.len() == 0 { - assert!(self.balance > 0); - self.balance -= 1; + + pub struct VecRecvEndpoint { + recv: Receiver>, // recv full vectors + send: Sender>, // send empty vectors + } + + impl BytesRecvEndpoint for VecRecvEndpoint { + type RecvBuffer = Vec; + fn receive(&mut self) -> Option { + if let Ok(bytes) = self.recv.try_recv() { + Some(bytes) } - Some(bytes) + else { None } } - else { - None + fn recycle(&mut self, mut buffer: Self::RecvBuffer) { + buffer.clear(); + // other side hanging up shouldn't cause panic. + let _ = self.send.send(buffer); + } + } + + pub struct VecBytesExchange; + + impl BytesExchange for VecBytesExchange { + type Send = VecSendEndpoint; + type Recv = VecRecvEndpoint; + fn new() -> (Self::Send, Self::Recv) { + + let (send1, recv1) = channel(); + let (send2, recv2) = channel(); + + let result1 = VecSendEndpoint { + send: send1, + recv: recv2, + balance: 0, + buffer: Vec::new(), + stash: Vec::new(), + default_size: 1 << 20, + }; + let result2 = VecRecvEndpoint { + send: send2, + recv: recv1, + }; + + (result1, result2) } } } -pub struct ProcessBinaryBuilder { - index: usize, // number out of peers - peers: usize, // number of peer allocators (for typed channel allocation). - sends: Vec, // with each other worker (for pushing bytes) - recvs: Vec, // with each other worker (for pulling bytes) + +pub struct ProcessBinaryBuilder { + index: usize, // number out of peers + peers: usize, // number of peer allocators (for typed channel allocation). + sends: Vec, // with each other worker (for pushing bytes) + recvs: Vec, // with each other worker (for pulling bytes) } -impl ProcessBinaryBuilder { +impl ProcessBinaryBuilder { - pub fn new_vector(count: usize) -> Vec { + pub fn new_vector(count: usize) -> Vec> { let mut sends = Vec::new(); let mut recvs = Vec::new(); @@ -84,7 +175,7 @@ impl ProcessBinaryBuilder { for source in 0 .. count { for target in 0 .. count { - let (send, recv) = ByteExchange::new_pair(); + let (send, recv) = BE::new(); sends[source].push(send); recvs[target].push(recv); } @@ -103,10 +194,10 @@ impl ProcessBinaryBuilder { result } - pub fn build(self) -> ProcessBinary { + pub fn build(self) -> ProcessBinary { let mut shared = Vec::new(); for send in self.sends.into_iter() { - shared.push(Rc::new(RefCell::new(SharedByteBuffer::new(send)))); + shared.push(Rc::new(RefCell::new(send))); } ProcessBinary { @@ -121,16 +212,16 @@ impl ProcessBinaryBuilder { } // A specific Communicator for inter-thread intra-process communication -pub struct ProcessBinary { +pub struct ProcessBinary { index: usize, // number out of peers peers: usize, // number of peer allocators (for typed channel allocation). allocated: usize, // indicates how many have been allocated (locally). - sends: Vec>>, // channels[x] -> goes to worker x. - recvs: Vec, // from all other workers. + sends: Vec>>, // channels[x] -> goes to worker x. + recvs: Vec, // from all other workers. to_local: Vec>>>>,// to worker-local typed pullers. } -impl Allocate for ProcessBinary { +impl Allocate for ProcessBinary { fn index(&self) -> usize { self.index } fn peers(&self) -> usize { self.peers } fn allocate(&mut self) -> (Vec>>, Box>, Option) { @@ -170,7 +261,7 @@ impl Allocate for ProcessBinary { for recv in self.recvs.iter_mut() { - while let Some(mut bytes) = recv.recv() { + while let Some(mut bytes) = recv.receive() { // we are guaranteed that `bytes` contains exactly an integral number of messages. // no splitting occurs across allocations. @@ -191,8 +282,7 @@ impl Allocate for ProcessBinary { assert_eq!(slice.len(), 0); } - bytes.clear(); - recv.send(bytes); + recv.recycle(bytes); } } } @@ -200,114 +290,24 @@ impl Allocate for ProcessBinary { // Perform postparatory work, most likely sending incomplete binary buffers. fn post_work(&mut self) { for send in self.sends.iter_mut() { - send.borrow_mut().flush(); + send.borrow_mut().publish(); } } } -struct SharedByteBuffer { - sender: ByteExchange, // channels for each destination worker. - buffer: Vec, // working space for each destination worker. - stash: Vec>, // spare buffers - default_size: usize, -} - -impl SharedByteBuffer { - - // Allocates a new SharedByteBuffer with an indicated default capacity. - fn new(sender: ByteExchange) -> Self { - let default_size = 1 << 20; - SharedByteBuffer { - sender, - buffer: Vec::with_capacity(default_size), - stash: Vec::new(), - default_size, - } - } - - /// Acquires a fresh buffer. - /// - /// The fresh buffer is acquired first from the local stash, then from the - /// return channel, and lastly allocated if no other buffers are available. - fn get_buffer(&mut self) -> Vec { - if self.stash.len() > 0 { - self.stash.pop().unwrap() - } - else { - self.sender.recv().unwrap_or_else(|| { println!("allocating"); Vec::with_capacity(self.default_size) }) - } - } - - /// Retrieve a writeable buffer with at least `size` bytes available capacity. - /// - /// This may result in the current working buffer being sent and a new buffer - /// being acquired or allocated. - fn reserve(&mut self, size: usize) -> &mut Vec { - - if self.buffer.len() + size > self.buffer.capacity() { - - // if we need more space than we expect from our byte exchange, ... - let new_buffer = if size > self.default_size { - Vec::with_capacity(size) - } - else { - self.get_buffer() - }; - - debug_assert!(new_buffer.is_empty()); - debug_assert!(new_buffer.capacity() >= size); - - let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); - if old_buffer.len() > 0 { - self.sender.send(old_buffer); - } - else { - // We should stash `old_buffer`, if the right size. - if old_buffer.capacity() == self.default_size { - self.stash.push(old_buffer); - } - } - } - - &mut self.buffer - } - - /// Push out all pending byte buffer sends. - /// - /// This has the effect of emptying `self.buffer`, most likely by sending - /// it through `self.sender`. - fn flush(&mut self) { - if self.buffer.len() > 0 { - - while let Some(bytes) = self.sender.recv() { - self.stash.push(bytes); - } - - // only ship the buffer if the recipient has consumed everything we've sent them. - // otherwise, wait for the first flush call when they have (they should eventually). - if self.sender.queue_length() == 0 { - let new_buffer = self.get_buffer(); - let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); - if old_buffer.len() > 0 { - self.sender.send(old_buffer); - } - } - else { - // println!("work to send, but queue length: {:?}", self.sender.queue_length()); - } - } - } -} - - -struct Pusher { +/// 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. +struct Pusher { header: MessageHeader, - sender: Rc>, + sender: Rc>, phantom: ::std::marker::PhantomData, } -impl Pusher { - pub fn new(header: MessageHeader, sender: Rc>) -> Pusher { +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, @@ -316,7 +316,7 @@ impl Pusher { } } -impl Push for Pusher { +impl Push for Pusher { #[inline] fn push(&mut self, element: &mut Option) { if let Some(ref mut element) = *element { @@ -336,6 +336,12 @@ impl Push for Pusher { } } +/// 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. struct Puller { current: Option, receiver: Rc>>>, // source of serialized buffers From 2bc8ab841bb20ccb350bded311aaa4ae795781be Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 14 May 2018 16:59:51 +0200 Subject: [PATCH 08/43] bytes update --- bytes/src/lib.rs | 67 +++++++++++++++++++++++++++++------------------- 1 file changed, 40 insertions(+), 27 deletions(-) diff --git a/bytes/src/lib.rs b/bytes/src/lib.rs index 39992fe28..b59899c07 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 where B: DerefMut { + 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 where B : DerefMut { + pub fn from(bytes: B) -> Bytes where B : DerefMut+'static { + + let mut boxed = Box::new(bytes) as Box; - let mut arc = Arc::new(bytes); + 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,10 @@ 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> { + 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, @@ -195,14 +208,14 @@ pub mod arc { } } - 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) } } From 7e1929b3fe123a27c00baf974f397b19ba5a92e9 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 15 May 2018 10:37:13 +0200 Subject: [PATCH 09/43] communication update --- communication/Cargo.toml | 1 + communication/examples/hello.rs | 5 + communication/src/allocator/binary.rs | 6 +- communication/src/allocator/process_binary.rs | 110 +++++++++++------- communication/src/allocator/thread.rs | 12 +- communication/src/initialize.rs | 2 + communication/src/lib.rs | 16 ++- 7 files changed, 97 insertions(+), 55 deletions(-) diff --git a/communication/Cargo.toml b/communication/Cargo.toml index 07677c179..a63024706 100644 --- a/communication/Cargo.toml +++ b/communication/Cargo.toml @@ -22,6 +22,7 @@ getopts={version="0.2.14", optional=true} time="0.1.34" abomonation = "0.5" abomonation_derive = "0.3" +bytes = { path = "../bytes" } [[examples]] features = "arg_parse" diff --git a/communication/examples/hello.rs b/communication/examples/hello.rs index 3c6919935..01fa2e14e 100644 --- a/communication/examples/hello.rs +++ b/communication/examples/hello.rs @@ -22,10 +22,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); received += 1; } + + allocator.post_work(); } allocator.index() diff --git a/communication/src/allocator/binary.rs b/communication/src/allocator/binary.rs index cb5cee117..d453394f8 100644 --- a/communication/src/allocator/binary.rs +++ b/communication/src/allocator/binary.rs @@ -139,13 +139,15 @@ impl Pull for Puller { 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); + let result = ::from_bytes(bytes); log_sender.when_enabled(|l| l.log( ::logging::CommsEvent::Serialization(::logging::SerializationEvent { seq_no: None, diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index 106e7bd50..013ff841c 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -4,37 +4,25 @@ use std::cell::RefCell; use std::collections::VecDeque; use std::ops::DerefMut; +use bytes::arc::Bytes; + use networking::MessageHeader; use {Allocate, Data, Push, Pull, Serialize}; -/// Reports the remaining capacity. -pub trait Available: ::std::io::Write { - /// Reports the remaining capacity. - /// - /// The instance should be able to accept `self.available()` bytes written at it - /// without complaint, and can have any (correct) behavior it likes on additional - /// bytes written. - fn available(&self) -> usize; -} - -impl Available for Vec { - fn available(&self) -> usize { self.capacity() - self.len() } -} - pub trait BytesExchange { - type Send: BytesSendEndpoint+'static; - type Recv: BytesRecvEndpoint+'static; + type Send: SendEndpoint+'static; + type Recv: RecvEndpoint+'static; fn new() -> (Self::Send, Self::Recv); } -pub trait BytesSendEndpoint { - type SendBuffer: Available; +pub trait SendEndpoint { + type SendBuffer: ::std::io::Write; fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; fn publish(&mut self); } -pub trait BytesRecvEndpoint { +pub trait RecvEndpoint { type RecvBuffer: DerefMut; fn receive(&mut self) -> Option; fn recycle(&mut self, buffer: Self::RecvBuffer); @@ -44,7 +32,7 @@ pub mod vec { use std::sync::mpsc::{Sender, Receiver, channel}; - use super::{BytesExchange, BytesSendEndpoint, BytesRecvEndpoint}; + use super::{BytesExchange, SendEndpoint, RecvEndpoint}; pub struct VecSendEndpoint { send: Sender>, // send full vectors @@ -68,6 +56,9 @@ pub mod vec { } /// Moves `self.buffer` into `self.send`, replaces with empty buffer. fn send_buffer(&mut self) { + + // println!("sending buffer of {:?} bytes", self.buffer.len()); + let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); if buffer.len() > 0 { self.send.send(buffer).expect("VecSendEndpoint::send_buffer(): failed to send buffer"); @@ -81,11 +72,14 @@ pub mod vec { } } - impl BytesSendEndpoint for VecSendEndpoint { + impl SendEndpoint for VecSendEndpoint { type SendBuffer = Vec; fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { + + // println!("reserving {:?} bytes", capacity); + if self.buffer.capacity() < capacity + self.buffer.len() { self.send_buffer(); if capacity > self.default_size { @@ -113,7 +107,7 @@ pub mod vec { send: Sender>, // send empty vectors } - impl BytesRecvEndpoint for VecRecvEndpoint { + impl RecvEndpoint for VecRecvEndpoint { type RecvBuffer = Vec; fn receive(&mut self) -> Option { if let Ok(bytes) = self.recv.try_recv() { @@ -207,6 +201,7 @@ impl ProcessBinaryBuilder { sends: shared, recvs: self.recvs, to_local: Vec::new(), + in_progress: Vec::new(), } } } @@ -218,7 +213,9 @@ pub struct ProcessBinary { allocated: usize, // indicates how many have been allocated (locally). sends: Vec>>, // channels[x] -> goes to worker x. recvs: Vec, // from all other workers. - to_local: Vec>>>>,// to worker-local typed pullers. + to_local: Vec>>>, // to worker-local typed pullers. + + in_progress: Vec>, } impl Allocate for ProcessBinary { @@ -250,7 +247,7 @@ impl Allocate for ProcessBinary { self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); } - let puller = Box::new(Puller::new(self.to_local[channel_id].clone())); + let puller = Box::new(Puller::new(self.to_local[channel_id].clone(), channel_id)); (pushes, puller, None) } @@ -259,39 +256,69 @@ impl Allocate for ProcessBinary { #[inline(never)] fn pre_work(&mut self) { - for recv in self.recvs.iter_mut() { + let recvs_len = self.recvs.len(); + for (peer, recv) in self.recvs.iter_mut().enumerate() { while let Some(mut bytes) = recv.receive() { + let mut bytes = Bytes::from(bytes); + // we are guaranteed that `bytes` contains exactly an integral number of messages. // no splitting occurs across allocations. + while bytes.len() > 0 { + + if let Some(header) = MessageHeader::try_read(&mut &bytes[..]) { - { - let mut slice = &bytes[..]; - while let Some(header) = MessageHeader::try_read(&mut slice) { - let h_len = header.length as usize; // length in bytes - let to_push = slice[..h_len].to_vec(); - slice = &slice[h_len..]; + // Get the header and payload, ditch the header. + let mut peel = bytes.extract_to(header.required_bytes()); + peel.extract_to(40); while self.to_local.len() <= header.channel { self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); } - self.to_local[header.channel].borrow_mut().push_back(to_push); + self.to_local[header.channel].borrow_mut().push_back(peel); + } + else { + println!("failed to read full header!"); } - assert_eq!(slice.len(), 0); } - recv.recycle(bytes); + assert!(peer < recvs_len); + self.in_progress.push(Some((bytes, peer))); } } } // Perform postparatory work, most likely sending incomplete binary buffers. fn post_work(&mut self) { + + // Publish outgoing byte ledgers. for send in self.sends.iter_mut() { send.borrow_mut().publish(); } + + for index in 0 .. self.to_local.len() { + let len = self.to_local[index].borrow_mut().len(); + if len > 0 { + println!("Warning: worker {}, undrained channel[{}].len() = {}", self.index, index, len); + } + } + + // Recycle completely processed byte buffers. + for index in 0 .. self.in_progress.len() { + if let Some((bytes, peer)) = self.in_progress[index].take() { + match bytes.try_recover::<::RecvBuffer>() { + Ok(vec) => { + self.recvs[peer].recycle(vec); + } + Err(bytes) => { + self.in_progress[index] = Some((bytes, peer)); + } + } + } + } + self.in_progress.retain(|x| x.is_some()); } } @@ -299,13 +326,13 @@ impl Allocate for ProcessBinary { /// /// This pusher has a fixed MessageHeader, and access to a SharedByteBuffer which it uses to /// acquire buffers for serialization. -struct Pusher { +struct Pusher { header: MessageHeader, sender: Rc>, phantom: ::std::marker::PhantomData, } -impl Pusher { +impl Pusher { /// Creates a new `Pusher` from a header and shared byte buffer. pub fn new(header: MessageHeader, sender: Rc>) -> Pusher { Pusher { @@ -316,7 +343,7 @@ impl Pusher { } } -impl Push for Pusher { +impl Push for Pusher { #[inline] fn push(&mut self, element: &mut Option) { if let Some(ref mut element) = *element { @@ -343,12 +370,13 @@ impl Push for Pusher { /// like the `bytes` crate (../bytes/) which provides an exclusive view of a shared /// allocation. struct Puller { + channel: usize, current: Option, - receiver: Rc>>>, // source of serialized buffers + receiver: Rc>>, // source of serialized buffers } impl Puller { - fn new(receiver: Rc>>>) -> Puller { - Puller { current: None, receiver } + fn new(receiver: Rc>>, channel: usize) -> Puller { + Puller { channel, current: None, receiver } } } @@ -360,7 +388,7 @@ impl Pull for Puller { self.receiver .borrow_mut() .pop_front() - .map(|mut bytes| ::from_bytes(&mut bytes)); + .map(|bytes| ::from_bytes(bytes)); &mut self.current } diff --git a/communication/src/allocator/thread.rs b/communication/src/allocator/thread.rs index 05729f0c6..5cb4d1e08 100644 --- a/communication/src/allocator/thread.rs +++ b/communication/src/allocator/thread.rs @@ -50,12 +50,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/initialize.rs b/communication/src/initialize.rs index 6855714b8..b8efa536f 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -128,10 +128,12 @@ fn create_allocators(config: Configuration, logger: LogBuilder) -> Result 0 { +/// allocator.pre_work(); /// if let Some(message) = receiver.recv() { /// println!("worker {}: received: <{}>", allocator.index(), message); /// expecting -= 1; /// } +/// allocator.post_work(); /// } /// /// // optionally, return something diff --git a/communication/src/lib.rs b/communication/src/lib.rs index d2f79559a..5678ee097 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -37,10 +37,13 @@ //! // 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); //! expecting -= 1; //! } +//! allocator.post_work(); //! } //! //! // optionally, return something @@ -76,6 +79,8 @@ extern crate abomonation; #[macro_use] extern crate abomonation_derive; extern crate time; +extern crate bytes; + pub mod allocator; mod networking; pub mod initialize; @@ -105,7 +110,7 @@ pub trait Serialize { fn into_bytes(&mut self, &mut W); /// 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; + fn from_bytes(bytes::arc::Bytes) -> Self; } // NOTE : this should be unsafe, because these methods are. @@ -114,12 +119,11 @@ impl Serialize for T { fn length_in_bytes(&self) -> usize { measure(self) } - fn into_bytes(&mut self, bytes: &mut W) { - // NOTE: `unwrap` should be ok, as Rust docs say writes to `Vec` do not fail. - unsafe { encode(self, bytes).unwrap(); } + fn into_bytes(&mut self, writer: &mut W) { + unsafe { encode(self, writer).expect("abomonation failed"); } } - fn from_bytes(bytes: &mut Vec) -> Self { - (* unsafe { decode::(bytes) }.unwrap().0).clone() + fn from_bytes(mut bytes: bytes::arc::Bytes) -> Self { + (* unsafe { decode::(&mut bytes) }.expect("de-abomonation failed").0).clone() } } From f0bf2fd2c3bebc79f06e85aef7c10c334cfa1927 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 15 May 2018 10:46:58 +0200 Subject: [PATCH 10/43] bytes oriented --- Cargo.toml | 1 + src/dataflow/channels/message.rs | 23 ++++++++++++++--------- src/lib.rs | 1 + src/progress/nested/subgraph.rs | 19 ++++++++++++------- 4 files changed, 28 insertions(+), 16 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index f95b88bc3..a9c279672 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -22,6 +22,7 @@ abomonation_derive = "0.3" timely_communication = { path = "./communication"} byteorder="1" time="0.1.34" +bytes = { path = "./bytes" } [dev-dependencies] timely_sort="0.1.6" diff --git a/src/dataflow/channels/message.rs b/src/dataflow/channels/message.rs index 7957f5ca1..5b76a93fb 100644 --- a/src/dataflow/channels/message.rs +++ b/src/dataflow/channels/message.rs @@ -56,36 +56,41 @@ impl Serialize for Message { 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()); + fn from_bytes(mut bytes: ::bytes::arc::Bytes) -> Self { 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 (t,r) = unsafe { decode::(&mut bytes) }.expect("failed to get time"); + let (&f,r) = unsafe { decode::(r) }.expect("failed to get from"); + let (&s,r) = unsafe { decode::(r) }.expect("failed to get seq"); 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(); + let length = unsafe { decode::>(&mut bytes[offset..]) }.expect("failed to deser Vec").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), + /// + /// TODO: We use an `arc::Bytes` here because `Content` needs to be `Send`. + /// This requirement seems artificial, and should probably be fixed. + Bytes(::bytes::arc::Bytes, usize, usize), /// Typed data, which may be efficiently mutated or claimed for ownership. Typed(Vec), } +impl Clone for Content { + fn clone(&self) -> Self { + unimplemented!() + } +} // 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. diff --git a/src/lib.rs b/src/lib.rs index 6cb8cd412..0db43d6fc 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -62,6 +62,7 @@ extern crate abomonation; 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}; diff --git a/src/progress/nested/subgraph.rs b/src/progress/nested/subgraph.rs index 9d7dcd798..6b4ede36f 100644 --- a/src/progress/nested/subgraph.rs +++ b/src/progress/nested/subgraph.rs @@ -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. @@ -838,12 +843,12 @@ impl PerOperatorState { internal_activity } - else { - // Active operators should always be scheduled, and should re-assert their activity if - // they want to be scheduled again. If we are here, it is because the operator declined - // to express activity explicitly. - false - } + // else { + // // Active operators should always be scheduled, and should re-assert their activity if + // // they want to be scheduled again. If we are here, it is because the operator declined + // // to express activity explicitly. + // false + // } } else { From 20064ddcdb8d08b635ca14ccbd3b09fa340806ee Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 28 May 2018 13:21:59 +0200 Subject: [PATCH 11/43] communication revamp --- Cargo.toml | 2 +- communication/Cargo.toml | 2 +- communication/examples/hello.rs | 7 +- communication/src/allocator/binary.rs | 39 ++- communication/src/allocator/generic.rs | 6 +- communication/src/allocator/mod.rs | 143 ++++++++- communication/src/allocator/process.rs | 10 +- communication/src/allocator/process_binary.rs | 28 +- communication/src/allocator/thread.rs | 4 +- communication/src/initialize.rs | 8 +- communication/src/lib.rs | 47 +-- examples/bfs.rs | 4 +- examples/broadcast.rs | 29 -- examples/hashjoin.rs | 9 +- examples/pagerank.rs | 9 +- examples/wordcount.rs | 2 +- src/dataflow/channels/mod.rs | 57 +++- src/dataflow/channels/pact.rs | 289 +++++++++--------- src/dataflow/channels/pullers/counter.rs | 18 +- src/dataflow/channels/pushers/buffer.rs | 45 +-- src/dataflow/channels/pushers/counter.rs | 21 +- src/dataflow/channels/pushers/exchange.rs | 21 +- src/dataflow/channels/pushers/tee.rs | 22 +- .../operators/aggregation/aggregate.rs | 5 +- .../operators/aggregation/state_machine.rs | 9 +- src/dataflow/operators/broadcast.rs | 2 +- src/dataflow/operators/capture/capture.rs | 13 +- src/dataflow/operators/concat.rs | 13 +- src/dataflow/operators/count.rs | 8 +- src/dataflow/operators/delay.rs | 29 +- src/dataflow/operators/enterleave.rs | 40 +-- src/dataflow/operators/exchange.rs | 50 +-- src/dataflow/operators/feedback.rs | 15 +- src/dataflow/operators/filter.rs | 8 +- src/dataflow/operators/generic/binary.rs | 22 +- src/dataflow/operators/generic/handles.rs | 54 ++-- src/dataflow/operators/generic/notificator.rs | 14 +- src/dataflow/operators/generic/operator.rs | 26 +- src/dataflow/operators/generic/unary.rs | 12 +- src/dataflow/operators/input.rs | 16 +- src/dataflow/operators/inspect.rs | 12 +- src/dataflow/operators/map.rs | 14 +- src/dataflow/operators/mod.rs | 4 +- src/dataflow/operators/partition.rs | 6 +- src/dataflow/operators/probe.rs | 13 +- src/dataflow/operators/reclock.rs | 4 +- src/dataflow/operators/to_stream.rs | 4 +- src/dataflow/scopes/child.rs | 4 +- src/dataflow/scopes/root.rs | 4 +- src/dataflow/stream.rs | 6 +- src/progress/broadcast.rs | 53 +++- 51 files changed, 785 insertions(+), 497 deletions(-) delete mode 100644 examples/broadcast.rs diff --git a/Cargo.toml b/Cargo.toml index a9c279672..5e8f3086b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,7 +16,7 @@ license = "MIT" #build = "booktests.rs" [dependencies] -abomonation = "0.5" +abomonation = { git = "https://github.com/frankmcsherry/abomonation" } abomonation_derive = "0.3" #timely_communication = "0.5" timely_communication = { path = "./communication"} diff --git a/communication/Cargo.toml b/communication/Cargo.toml index a63024706..59c372752 100644 --- a/communication/Cargo.toml +++ b/communication/Cargo.toml @@ -20,7 +20,7 @@ arg_parse=["getopts"] 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" } diff --git a/communication/examples/hello.rs b/communication/examples/hello.rs index 01fa2e14e..67f495188 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::allocator::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(); } @@ -26,7 +29,7 @@ fn main() { allocator.pre_work(); if let Some(message) = receiver.recv() { - println!("worker {}: received: <{}>", allocator.index(), message); + println!("worker {}: received: <{}>", allocator.index(), message.deref()); received += 1; } diff --git a/communication/src/allocator/binary.rs b/communication/src/allocator/binary.rs index d453394f8..5edcb8cd1 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,20 +128,20 @@ 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 } @@ -147,13 +154,15 @@ impl Pull for Puller { }))); let bytes = ::bytes::arc::Bytes::from(bytes); - let result = ::from_bytes(bytes); + // let result = ::from_bytes(bytes); log_sender.when_enabled(|l| l.log( ::logging::CommsEvent::Serialization(::logging::SerializationEvent { seq_no: None, is_start: false, }))); - result + + // Message::Typed(result) + Message::from_bytes(bytes) }); &mut self.current } diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index 33dacdd59..8afdcd1df 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -3,7 +3,7 @@ //! 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, Message, Thread, Process, Binary}; use allocator::process_binary::{ProcessBinary, ProcessBinaryBuilder}; use {Push, Pull, Data}; @@ -36,7 +36,7 @@ impl Generic { } } /// Constructs several send endpoints and one receive endpoint. - pub fn allocate(&mut self) -> (Vec>>, Box>, Option) { + 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(), @@ -66,7 +66,7 @@ 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() } diff --git a/communication/src/allocator/mod.rs b/communication/src/allocator/mod.rs index 8840d5d36..5f3ab194e 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -5,6 +5,9 @@ pub use self::process::Process; pub use self::binary::Binary; pub use self::generic::{Generic, GenericBuilder}; +use bytes::arc::Bytes; +use abomonation::{Abomonation, abomonated::Abomonated, encode, measure}; + pub mod thread; pub mod process; pub mod binary; @@ -13,6 +16,143 @@ pub mod process_binary; use {Data, Push, Pull}; +/// Possible returned representations from a channel. +enum TypedOrBinary { + /// Binary representation. + Binary(Abomonated), + /// Rust typed instance. + Typed(T), +} + +pub enum RefOrMut<'a, T> where T: 'a { + Ref(&'a T), + 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 + } +} + +pub struct Message { + payload: TypedOrBinary, +} + +impl Message { + pub fn from_typed(typed: T) -> Self { + Message { payload: TypedOrBinary::Typed(typed) } + } + pub fn if_typed(self) -> Option { + match self.payload { + TypedOrBinary::Binary(_) => None, + TypedOrBinary::Typed(typed) => Some(typed), + } + } + pub fn if_mut(&mut self) -> Option<&mut T> { + match &mut self.payload { + TypedOrBinary::Binary(_) => None, + TypedOrBinary::Typed(typed) => Some(typed), + } + } +} + +impl Message { + pub fn from_bytes(bytes: Bytes) -> Self { + + unsafe { + let abomonated = Abomonated::new(bytes).expect("Abomonated::new() failed."); + Message { payload: TypedOrBinary::Binary(abomonated) } + } + + } + + 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) }, + } + } + + fn length_in_bytes(&self) -> usize { + match &self.payload { + TypedOrBinary::Binary(bytes) => { bytes.as_bytes().len() }, + TypedOrBinary::Typed(typed) => { measure(typed) }, + } + } + 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, + } + } + + 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!() + } + } + +} + // 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. @@ -22,7 +162,8 @@ pub trait Allocate { /// 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); + fn allocate(&mut self) -> (Vec>>>, Box>>, Option); fn pre_work(&mut self) { } fn post_work(&mut self) { } diff --git a/communication/src/allocator/process.rs b/communication/src/allocator/process.rs index ac3b79814..0a92ee29b 100644 --- a/communication/src/allocator/process.rs +++ b/communication/src/allocator/process.rs @@ -2,7 +2,7 @@ use std::sync::{Arc, Mutex}; use std::any::Any; use std::sync::mpsc::{Sender, Receiver, channel}; -use allocator::{Allocate, Thread}; +use allocator::{Allocate, Message, Thread}; use {Push, Pull}; // A specific Communicator for inter-thread intra-process communication @@ -31,7 +31,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?"); @@ -55,12 +55,12 @@ impl Allocate for Process { } - if let Some(ref mut vector) = channels[self.allocated].downcast_mut::<(Vec>, Puller)>>)>() { + 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) + 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"); diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index 013ff841c..0a7c39db2 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -8,7 +8,8 @@ use bytes::arc::Bytes; use networking::MessageHeader; -use {Allocate, Data, Push, Pull, Serialize}; +use allocator::Message; +use {Allocate, Data, Push, Pull}; pub trait BytesExchange { type Send: SendEndpoint+'static; @@ -221,12 +222,12 @@ pub struct ProcessBinary { impl Allocate for ProcessBinary { 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) { let channel_id = self.allocated; self.allocated += 1; - let mut pushes = Vec::>>::new(); + let mut pushes = Vec::>>>::new(); for target_index in 0 .. self.peers() { @@ -326,13 +327,13 @@ impl Allocate for ProcessBinary { /// /// This pusher has a fixed MessageHeader, and access to a SharedByteBuffer which it uses to /// acquire buffers for serialization. -struct Pusher { +struct Pusher { header: MessageHeader, sender: Rc>, phantom: ::std::marker::PhantomData, } -impl Pusher { +impl Pusher { /// Creates a new `Pusher` from a header and shared byte buffer. pub fn new(header: MessageHeader, sender: Rc>) -> Pusher { Pusher { @@ -343,22 +344,23 @@ impl Pusher { } } -impl Push for Pusher { +impl Push> for Pusher { #[inline] - fn push(&mut self, element: &mut Option) { + fn push(&mut self, element: &mut Option>) { if let Some(ref mut element) = *element { // determine byte lengths and build header. - let element_length = element.length_in_bytes(); let mut header = self.header; self.header.seqno += 1; - header.length = element_length; + header.length = element.length_in_bytes(); // acquire byte buffer and write header, element. let mut borrow = self.sender.borrow_mut(); let mut bytes = borrow.reserve(header.required_bytes()); header.write_to(&mut bytes).expect("failed to write header!"); + element.into_bytes(&mut bytes); + } } } @@ -371,7 +373,7 @@ impl Push for Pusher { /// allocation. struct Puller { channel: usize, - current: Option, + current: Option>, receiver: Rc>>, // source of serialized buffers } impl Puller { @@ -380,15 +382,15 @@ impl Puller { } } -impl Pull for Puller { +impl Pull> for Puller { #[inline] - fn pull(&mut self) -> &mut Option { + fn pull(&mut self) -> &mut Option> { self.current = self.receiver .borrow_mut() .pop_front() - .map(|bytes| ::from_bytes(bytes)); + .map(|bytes| Message::from_bytes(bytes)); &mut self.current } diff --git a/communication/src/allocator/thread.rs b/communication/src/allocator/thread.rs index 5cb4d1e08..66443c143 100644 --- a/communication/src/allocator/thread.rs +++ b/communication/src/allocator/thread.rs @@ -2,7 +2,7 @@ use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; -use allocator::Allocate; +use allocator::{Allocate, Message}; use {Push, Pull}; @@ -11,7 +11,7 @@ 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) } diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index b8efa536f..4e361e31b 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -121,8 +121,9 @@ fn create_allocators(config: Configuration, logger: LogBuilder) -> Result Result 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(); diff --git a/communication/src/lib.rs b/communication/src/lib.rs index 5678ee097..47c03aae9 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -30,8 +30,9 @@ //! 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::allocator::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. @@ -40,7 +41,8 @@ //! //! 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(); @@ -72,6 +74,8 @@ //! result: Ok(1) //! ``` +// #![forbid(missing_docs)] + #[cfg(feature = "arg_parse")] extern crate getopts; extern crate byteorder; @@ -87,45 +91,16 @@ pub mod initialize; pub mod logging; use std::any::Any; -use abomonation::{Abomonation, encode, decode, measure}; +use abomonation::Abomonation; pub use allocator::Generic as Allocator; pub use allocator::Allocate; pub use initialize::{initialize, Configuration, WorkerGuards}; +pub use allocator::Message as CommMessage; /// 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 { - /// Number of bytes required for the binary representation of the object. - fn length_in_bytes(&self) -> usize; - /// 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 W); - /// 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(bytes::arc::Bytes) -> 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 length_in_bytes(&self) -> usize { - measure(self) - } - fn into_bytes(&mut self, writer: &mut W) { - unsafe { encode(self, writer).expect("abomonation failed"); } - } - fn from_bytes(mut bytes: bytes::arc::Bytes) -> Self { - (* unsafe { decode::(&mut bytes) }.expect("de-abomonation failed").0).clone() - } -} +pub trait Data : Send+Any+Abomonation+'static { } +impl Data for T { } /// Pushing elements of type `T`. pub trait Push { diff --git a/examples/bfs.rs b/examples/bfs.rs index fb7b3c1de..7dab4cc86 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/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/mod.rs b/src/dataflow/channels/mod.rs index 191623b95..a3c4822c5 100644 --- a/src/dataflow/channels/mod.rs +++ b/src/dataflow/channels/mod.rs @@ -1,13 +1,62 @@ //! Structured communication between timely dataflow operators. -pub use self::message::Message; -pub use self::message::Content; +use timely_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; +// /// 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 = ::timely_communication::allocator::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..63c1d5915 100644 --- a/src/dataflow/channels/pact.rs +++ b/src/dataflow/channels/pact.rs @@ -13,9 +13,10 @@ 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 timely_communication::CommMessage; use dataflow::channels::pushers::Exchange as ExchangePusher; -use dataflow::channels::{Message, Content}; +use super::{Bundle, Message}; use logging::Logger; @@ -24,9 +25,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 +35,14 @@ 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 = ThreadPusher>>; + type Puller = ThreadPuller>>; 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::>(); + // (Pusher::new(pusher, allocator.index(), allocator.index(), identifier, None, logging.clone()), + // Puller::new(puller, allocator.index(), identifier, None, logging.clone())) + (pusher, puller) } } @@ -62,142 +62,143 @@ 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>>>; - 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())) - } -} - -/// 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>>>; + 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::>(); - (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>> { - pusher: P, - channel: usize, - comm_channel: Option, - counter: usize, - source: usize, - target: usize, - phantom: ::std::marker::PhantomData<(T, D)>, - logging: Logger, -} -impl>> Pusher { - /// Allocates a new pusher. - pub fn new(pusher: P, source: usize, target: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { - Pusher { - pusher, - channel, - comm_channel, - counter: 0, - source, - target, - phantom: ::std::marker::PhantomData, - logging, - } + // 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())) + (Box::new(ExchangePusher::new(senders, move |_, d| (self.hash_func)(d))), receiver) } } -impl>> Push<(T, Content)> for Pusher { - #[inline(always)] - fn push(&mut self, pair: &mut Option<(T, Content)>) { - if let Some((time, data)) = pair.take() { - - let length = 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, - channel: self.channel, - comm_channel: self.comm_channel, - source: self.source, - target: self.target, - seq_no: counter, - length, - }))); - - // Log something about (index, counter, time?, length?); - } - else { self.pusher.done(); } - } -} - -/// Wraps a `Message` puller to provide a `Pull<(T, Content)>`. -pub struct Puller>> { - puller: P, - current: Option<(T, Content)>, - channel: usize, - comm_channel: Option, - counter: usize, - index: usize, - logging: Logger, -} -impl>> Puller { - /// Allocates a new `Puller`. - pub fn new(puller: P, index: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { - Puller { - puller, - channel, - comm_channel, - current: None, - counter: 0, - index, - logging, - } - } -} - -impl>> Pull<(T, Content)> for Puller { - #[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; - - ::std::mem::swap(&mut previous, self.puller.pull()); - - if let Some(message) = previous.as_ref() { - - 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(), - }))); - } - - self.current = previous.map(|message| (message.time, message.data)); - &mut self.current - } -} +// /// 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>> { +// pusher: P, +// channel: usize, +// comm_channel: Option, +// counter: usize, +// source: usize, +// target: usize, +// phantom: ::std::marker::PhantomData<(T, D)>, +// logging: Logger, +// } +// impl>> Pusher { +// /// Allocates a new pusher. +// pub fn new(pusher: P, source: usize, target: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { +// Pusher { +// pusher, +// channel, +// comm_channel, +// counter: 0, +// source, +// target, +// phantom: ::std::marker::PhantomData, +// logging, +// } +// } +// } + +// impl>> Push<(T, Content)> for Pusher { +// #[inline(always)] +// fn push(&mut self, pair: &mut Option<(T, Content)>) { +// if let Some((time, data)) = pair.take() { + +// let length = 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, +// channel: self.channel, +// comm_channel: self.comm_channel, +// source: self.source, +// target: self.target, +// seq_no: counter, +// length, +// }))); + +// // Log something about (index, counter, time?, length?); +// } +// else { self.pusher.done(); } +// } +// } + +// /// Wraps a `Message` puller to provide a `Pull<(T, Content)>`. +// pub struct Puller>> { +// puller: P, +// current: Option>, +// channel: usize, +// comm_channel: Option, +// counter: usize, +// index: usize, +// logging: Logger, +// } +// impl>> Puller { +// /// Allocates a new `Puller`. +// pub fn new(puller: P, index: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { +// Puller { +// puller, +// channel, +// comm_channel, +// current: None, +// counter: 0, +// index, +// logging, +// } +// } +// } + +// impl>> Pull<(T, Content)> for Puller { +// #[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; + +// ::std::mem::swap(&mut previous, self.puller.pull()); + +// if let Some(message) = previous.as_ref() { + +// 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(), +// }))); +// } + +// self.current = previous.map(|message| (message.time, message.data)); +// &mut self.current +// } +// } diff --git a/src/dataflow/channels/pullers/counter.rs b/src/dataflow/channels/pullers/counter.rs index b1b333908..76bad2a31 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; /// 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..d96051a05 100644 --- a/src/dataflow/channels/pushers/buffer.rs +++ b/src/dataflow/channels/pushers/buffer.rs @@ -1,7 +1,7 @@ //! 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; @@ -10,19 +10,19 @@ use timely_communication::Push; /// /// 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,19 +72,22 @@ 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)); + let time = self.time.as_ref().expect("Buffer::give_vec(): time is None.").clone(); + let data = ::std::mem::replace(vector, Vec::new()); - self.pusher.push(&mut message); - if let Some((_, data)) = message { - *content = data; + let mut bundle = Some(Bundle::from_typed(Message::new(time, data, 0, 0))); + + self.pusher.push(&mut bundle); + if let Some(message) = bundle { + if let Some(message) = message.if_typed() { + *vector = message.data; + } } } } @@ -95,11 +98,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 +121,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 +137,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 +152,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..daed38063 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 dataflow::channels::Bundle; use 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 8b3254e6f..3d2c65ac4 100644 --- a/src/dataflow/channels/pushers/exchange.rs +++ b/src/dataflow/channels/pushers/exchange.rs @@ -1,24 +1,23 @@ //! The exchange pattern distributes pushed data between many target pushees. use {Push, Data}; -use dataflow::channels::Content; -use abomonation::Abomonation; +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

{ /// Moves `self.buffer` into `self.send`, replaces with empty buffer. fn send_buffer(&mut self) { @@ -56,15 +90,12 @@ impl BytesSendEndpoint { } /// Allocates a new `BytesSendEndpoint` from a shared queue. - pub fn new(queue: SharedQueueSend) -> Self { - BytesSendEndpoint { + pub fn new(queue: P) -> Self { + SendEndpoint { send: queue, buffer: BytesSlab::new(20), } } -} - -impl BytesSendEndpoint { pub fn make_valid(&mut self, bytes: usize) { self.buffer.make_valid(bytes); @@ -86,26 +117,26 @@ impl BytesSendEndpoint { } } -impl Drop for BytesSendEndpoint { +impl Drop for SendEndpoint

{ fn drop(&mut self) { self.send_buffer(); } } -pub struct BytesRecvEndpoint { - recv: SharedQueueRecv, -} +// pub struct RecvEndpoint { +// recv: PSharedQueueRecv, +// } -impl BytesRecvEndpoint { - pub fn new(queue: SharedQueueRecv) -> Self { - BytesRecvEndpoint { recv: queue } - } -} +// impl BytesRecvEndpoint { +// pub fn new(queue: SharedQueueRecv) -> Self { +// BytesRecvEndpoint { recv: queue } +// } +// } -impl RecvEndpoint for BytesRecvEndpoint { - type RecvBuffer = Bytes; - fn receive(&mut self) -> Option { - self.recv.pop() - } -} \ No newline at end of file +// impl RecvEndpoint for BytesRecvEndpoint { +// type RecvBuffer = Bytes; +// fn receive(&mut self) -> Option { +// self.recv.pop() +// } +// } \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/bytes_slab.rs b/communication/src/allocator/zero_copy/bytes_slab.rs index 382fa2914..3000e83a9 100644 --- a/communication/src/allocator/zero_copy/bytes_slab.rs +++ b/communication/src/allocator/zero_copy/bytes_slab.rs @@ -20,10 +20,6 @@ impl BytesSlab { } } - // pub fn is_full(&self) -> bool { self.valid == self.buffer.len() } - - // pub fn bytes(&mut self) -> &mut Bytes { &mut self.buffer } - pub fn empty(&mut self) -> &mut [u8] { &mut self.buffer[self.valid..] } @@ -69,10 +65,6 @@ impl BytesSlab { 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); - if !(self.buffer.len() == (1 << self.shift)) { - println!("len: {:?}, expected: {:?}", self.buffer.len(), (1 << self.shift)); - } - self.buffer[.. self.valid].copy_from_slice(&old_buffer[.. self.valid]); self.in_progress.push(Some(old_buffer)); } diff --git a/communication/src/allocator/zero_copy/mod.rs b/communication/src/allocator/zero_copy/mod.rs index a4ba53d77..70ff6d2ff 100644 --- a/communication/src/allocator/zero_copy/mod.rs +++ b/communication/src/allocator/zero_copy/mod.rs @@ -1,9 +1,7 @@ pub mod bytes_slab; -pub mod shared_queue; pub mod bytes_exchange; pub mod binary; pub mod allocator; +pub mod allocator_process; pub mod initialize; - -// pub use self::shared_queue::SharedQueue; -pub use self::bytes_exchange::{BytesExchange, BytesSendEndpoint, BytesRecvEndpoint}; \ No newline at end of file +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..51735ff86 --- /dev/null +++ b/communication/src/allocator/zero_copy/push_pull.rs @@ -0,0 +1,133 @@ +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 { + 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 { + 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/initialize.rs b/communication/src/initialize.rs index c6bafd1f1..cd5c6f394 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -12,6 +12,7 @@ use std::any::Any; use allocator::{Thread, Process, Generic, GenericBuilder}; // use allocator::process_binary::ProcessBinaryBuilder; // use networking::initialize_networking; +use allocator::zero_copy::allocator_process::ProcessBuilder; use allocator::zero_copy::initialize::initialize_networking; /// Possible configurations for the communication infrastructure. @@ -86,8 +87,8 @@ fn create_allocators(config: Configuration, logger: LogBuilder) -> Result<(Vec { - Ok((Process::new_vector(threads).into_iter().map(|x| GenericBuilder::Process(x)).collect(), Box::new(()))) - // Ok(ProcessBinaryBuilder::new_vector(threads).into_iter().map(|x| GenericBuilder::ProcessBinary(x)).collect()) + // 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, guard)) = initialize_networking(addresses, process, threads, report, logger) { diff --git a/src/lib.rs b/src/lib.rs index 8bb2c9932..050268cfc 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -92,9 +92,4 @@ 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 { } +impl ExchangeData for T { } \ No newline at end of file From 20c2e38d01b555258419c8429388ad6c379208f4 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 25 Aug 2018 19:29:02 +0200 Subject: [PATCH 32/43] tidy --- communication/src/allocator/process_binary.rs | 417 --------- .../src/allocator/process_binary_neu.rs | 789 ------------------ .../src/allocator/zero_copy/bytes_exchange.rs | 63 +- .../src/allocator/zero_copy/shared_queue.rs | 48 -- 4 files changed, 7 insertions(+), 1310 deletions(-) delete mode 100644 communication/src/allocator/process_binary.rs delete mode 100644 communication/src/allocator/process_binary_neu.rs delete mode 100644 communication/src/allocator/zero_copy/shared_queue.rs diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs deleted file mode 100644 index c0aa3018d..000000000 --- a/communication/src/allocator/process_binary.rs +++ /dev/null @@ -1,417 +0,0 @@ - -use std::rc::Rc; -use std::cell::RefCell; -use std::collections::VecDeque; -use std::ops::DerefMut; - -use bytes::arc::Bytes; - -use networking::MessageHeader; - -use allocator::Message; -use {Allocate, Data, Push, Pull}; - -pub trait BytesExchange { - type Send: SendEndpoint+'static; - type Recv: RecvEndpoint+'static; - fn new() -> (Self::Send, Self::Recv); -} - -pub trait SendEndpoint { - type SendBuffer: ::std::io::Write; - fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; - fn publish(&mut self); -} - -pub trait RecvEndpoint { - type RecvBuffer: DerefMut; - fn receive(&mut self) -> Option; - fn recycle(&mut self, buffer: Self::RecvBuffer); -} - -pub mod vec { - - use std::sync::{Arc, Mutex}; - use std::sync::mpsc::{Sender, Receiver, channel}; - use std::collections::VecDeque; - - use super::{BytesExchange, SendEndpoint, RecvEndpoint}; - - pub struct VecSendEndpoint { - // in_flight: Arc, - // send: Sender>, // send full vectors - send: Arc>>>, - recv: Receiver>, // recv empty vectors - balance: usize, // #sent - #recv. - - buffer: Vec, - stash: Vec>, // spare buffers - default_size: usize, - } - - impl VecSendEndpoint { - /// Drains `self.recv` of empty buffers, stashes them. - fn drain_recv(&mut self) { - while let Ok(bytes) = self.recv.try_recv() { - self.balance -= 1; - if bytes.capacity() == self.default_size { - self.stash.push(bytes); - } - } - } - /// Moves `self.buffer` into `self.send`, replaces with empty buffer. - fn send_buffer(&mut self) { - - // println!("sending buffer of {:?} bytes", self.buffer.len()); - - let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); - if buffer.len() > 0 { - if let Ok(mut lock) = self.send.lock() { - lock.push_back(buffer); - self.balance += 1; - } - else { - panic!("unable to acquire lock"); - } - // self.send.send(buffer).expect("VecSendEndpoint::send_buffer(): failed to send buffer"); - } - else { - if buffer.capacity() == self.default_size { - self.stash.push(buffer); - } - } - } - } - - impl SendEndpoint for VecSendEndpoint { - - type SendBuffer = Vec; - - fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { - - // println!("reserving {:?} bytes", capacity); - - if self.buffer.capacity() < capacity + self.buffer.len() { - self.send_buffer(); - if capacity > self.default_size { - self.buffer = Vec::with_capacity(capacity); - } - else { - self.drain_recv(); - self.buffer = self.stash.pop().unwrap_or_else(|| Vec::with_capacity(self.default_size)) - } - } - - &mut self.buffer - } - - fn publish(&mut self) { - self.drain_recv(); - if self.send.lock().map(|queue| queue.is_empty()).expect("Failed to lock mutex") { - // if self.balance == 0 { - self.send_buffer(); - } - } - } - - pub struct VecRecvEndpoint { - recv: Arc>>>, - // recv: Receiver>, // recv full vectors - send: Sender>, // send empty vectors - } - - impl RecvEndpoint for VecRecvEndpoint { - type RecvBuffer = Vec; - fn receive(&mut self) -> Option { - if let Ok(mut lock) = self.recv.lock() { - lock.pop_front() - } - else { - panic!("Failed to lock mutex"); - } - // if let Ok(bytes) = self.recv.try_recv() { - // Some(bytes) - // } - // else { None } - } - fn recycle(&mut self, mut buffer: Self::RecvBuffer) { - buffer.clear(); - // other side hanging up shouldn't cause panic. - let _ = self.send.send(buffer); - } - } - - pub struct VecBytesExchange; - - impl BytesExchange for VecBytesExchange { - type Send = VecSendEndpoint; - type Recv = VecRecvEndpoint; - fn new() -> (Self::Send, Self::Recv) { - - let send1 = Arc::new(Mutex::new(VecDeque::new())); - let recv1 = send1.clone(); - - // let (send1, recv1) = channel(); - let (send2, recv2) = channel(); - - let result1 = VecSendEndpoint { - send: send1, - recv: recv2, - balance: 0, - buffer: Vec::new(), - stash: Vec::new(), - default_size: 1 << 20, - }; - let result2 = VecRecvEndpoint { - send: send2, - recv: recv1, - }; - - (result1, result2) - } - } -} - - -pub struct ProcessBinaryBuilder { - index: usize, // number out of peers - peers: usize, // number of peer allocators (for typed channel allocation). - sends: Vec, // with each other worker (for pushing bytes) - recvs: Vec, // with each other worker (for pulling bytes) -} - -impl ProcessBinaryBuilder { - - pub fn new_vector(count: usize) -> Vec> { - - 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, recv) = BE::new(); - sends[source].push(send); - recvs[target].push(recv); - } - } - - let mut result = Vec::new(); - for (index, (sends, recvs)) in sends.drain(..).zip(recvs.drain(..)).enumerate() { - result.push(ProcessBinaryBuilder { - index, - peers: count, - sends, - recvs, - }) - } - - result - } - - pub fn build(self) -> ProcessBinary { - let mut shared = Vec::new(); - for send in self.sends.into_iter() { - shared.push(Rc::new(RefCell::new(send))); - } - - ProcessBinary { - index: self.index, - peers: self.peers, - allocated: 0, - sends: shared, - recvs: self.recvs, - to_local: Vec::new(), - in_progress: Vec::new(), - } - } -} - -// A specific Communicator for inter-thread intra-process communication -pub struct ProcessBinary { - index: usize, // number out of peers - peers: usize, // number of peer allocators (for typed channel allocation). - allocated: usize, // indicates how many have been allocated (locally). - sends: Vec>>, // channels[x] -> goes to worker x. - recvs: Vec, // from all other workers. - to_local: Vec>>>, // to worker-local typed pullers. - - in_progress: Vec>, -} - -impl Allocate for ProcessBinary { - 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) { - - let recvs_len = self.recvs.len(); - for (peer, recv) in self.recvs.iter_mut().enumerate() { - - while let Some(mut bytes) = recv.receive() { - - let mut bytes = Bytes::from(bytes); - - // we are guaranteed that `bytes` contains exactly 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()); - peel.extract_to(40); - - while self.to_local.len() <= header.channel { - self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); - } - - self.to_local[header.channel].borrow_mut().push_back(peel); - } - else { - println!("failed to read full header!"); - } - } - - assert!(peer < recvs_len); - self.in_progress.push(Some((bytes, peer))); - } - } - } - - // Perform postparatory work, most likely sending incomplete binary buffers. - fn post_work(&mut self) { - - // Publish outgoing byte ledgers. - for send in self.sends.iter_mut() { - send.borrow_mut().publish(); - } - - for index in 0 .. self.to_local.len() { - let len = self.to_local[index].borrow_mut().len(); - if len > 0 { - println!("Warning: worker {}, undrained channel[{}].len() = {}", self.index, index, len); - } - } - - // Recycle completely processed byte buffers. - for index in 0 .. self.in_progress.len() { - if let Some((bytes, peer)) = self.in_progress[index].take() { - match bytes.try_recover::<::RecvBuffer>() { - Ok(vec) => { - self.recvs[peer].recycle(vec); - } - Err(bytes) => { - self.in_progress[index] = Some((bytes, peer)); - } - } - } - } - self.in_progress.retain(|x| x.is_some()); - } -} - -/// 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. -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(); - - // acquire byte buffer and write header, element. - let mut borrow = self.sender.borrow_mut(); - let mut bytes = borrow.reserve(header.required_bytes()); - header.write_to(&mut bytes).expect("failed to write header!"); - - element.into_bytes(&mut 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. -struct Puller { - current: Option>, - receiver: Rc>>, // source of serialized buffers -} -impl Puller { - 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 - } -} \ No newline at end of file diff --git a/communication/src/allocator/process_binary_neu.rs b/communication/src/allocator/process_binary_neu.rs deleted file mode 100644 index 4c45a415f..000000000 --- a/communication/src/allocator/process_binary_neu.rs +++ /dev/null @@ -1,789 +0,0 @@ -use std::rc::Rc; -use std::cell::RefCell; -use std::collections::VecDeque; -use std::ops::DerefMut; - -use bytes::arc::Bytes; - -use networking::MessageHeader; - -use {Allocate, Data, Push, Pull}; -use allocator::{Message, Process}; - -use std::sync::{Arc, Mutex}; - -pub struct SharedQueue { - queue: Arc>> -} - -impl SharedQueue { - pub fn push(&mut self, bytes: T) { self.queue.lock().expect("unable to lock shared queue").push_back(bytes) } - pub fn pop(&mut self) -> Option { self.queue.lock().expect("unable to lock shared queue").pop_front() } - pub fn drain_into(&mut self, dest: &mut Vec) { let mut lock = self.queue.lock().expect("unable to lock shared queue"); dest.extend(lock.drain(..)); } - pub fn is_empty(&self) -> bool { self.queue.lock().expect("unable to lock shared queue").is_empty() } - pub fn is_done(&self) -> bool { Arc::strong_count(&self.queue) == 1 } - pub fn new() -> Self { SharedQueue { queue: Arc::new(Mutex::new(VecDeque::new())) } } -} - -impl Clone for SharedQueue { - fn clone(&self) -> Self { - SharedQueue { queue: self.queue.clone() } - } -} - -/// A type that can allocate send and receive endpoints for byte exchanges. -/// -/// The `BytesExchange` intent is that one can abstractly define mechanisms for exchanging -/// bytes between various entities. In some cases this may be between worker threads within -/// a process, in other cases it may be between worker threads and remote processes. At the -/// moment the cardinalities of remote endpoints requires some context and interpretation. -pub trait BytesExchange { - /// The type of the send endpoint. - type Send: SendEndpoint+'static; - /// The type of the receive endpoint. - type Recv: RecvEndpoint+'static; - /// Allocates endpoint pairs for a specified worker. - /// - /// Importantly, the Send side may share state to coalesce the buffering and - /// transmission of records. That is why there are `Rc>` things there. - fn next(&mut self) -> Option<(Vec, Vec)>; -} - -/// A type that can provide and publish writeable binary buffers. -pub trait SendEndpoint { - /// The type of the writeable binary buffer. - type SendBuffer: ::std::io::Write; - /// Provides a writeable buffer of the requested capacity. - fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; - /// Indicates that it is now appropriate to publish the buffer. - fn publish(&mut self); -} - -/// A type that can provide readable binary buffers. -pub trait RecvEndpoint { - type RecvBuffer: DerefMut; - /// Provides a readable buffer. - fn receive(&mut self) -> Option; -} - -pub mod common { - - use bytes::arc::Bytes; - use super::{SendEndpoint, RecvEndpoint, SharedQueue}; - - pub struct VecSendEndpoint { - send: SharedQueue, - in_progress: Vec>, - buffer: Vec, - stash: Vec>, - default_size: usize, - } - - impl VecSendEndpoint { - /// Attempts to recover in-use buffers once uniquely owned. - fn harvest_shared(&mut self) { - for shared in self.in_progress.iter_mut() { - if let Some(bytes) = shared.take() { - match bytes.try_recover::>() { - Ok(vec) => { self.stash.push(vec); }, - Err(bytes) => { *shared = Some(bytes); }, - } - } - } - self.in_progress.retain(|x| x.is_some()); - } - - /// Moves `self.buffer` into `self.send`, replaces with empty buffer. - fn send_buffer(&mut self) { - - let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); - let buffer_len = buffer.len(); - if buffer_len > 0 { - - let mut bytes = Bytes::from(buffer); - let to_send = bytes.extract_to(buffer_len); - - self.send.push(to_send); - self.in_progress.push(Some(bytes)); - } - else { - if buffer.capacity() == self.default_size { - self.stash.push(buffer); - } - } - } - - /// Allocates a new `VecSendEndpoint` from a shared queue. - pub fn new(queue: SharedQueue) -> Self { - VecSendEndpoint { - send: queue, - in_progress: Vec::new(), - buffer: Vec::new(), - stash: Vec::new(), - default_size: 1 << 20, - } - } - } - - impl SendEndpoint for VecSendEndpoint { - - type SendBuffer = Vec; - - fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { - - // If we don't have enough capacity in `self.buffer`... - if self.buffer.capacity() < capacity + self.buffer.len() { - self.send_buffer(); - if capacity > self.default_size { - self.buffer = Vec::with_capacity(capacity); - } - else { - if self.stash.is_empty() { - // Attempt to recover shared buffers. - self.harvest_shared(); - } - self.buffer = self.stash.pop().unwrap_or_else(|| Vec::with_capacity(self.default_size)) - } - } - - &mut self.buffer - } - - fn publish(&mut self) { - self.harvest_shared(); - if self.send.is_empty() { - self.send_buffer(); - } - } - } - - pub struct VecRecvEndpoint { - recv: SharedQueue, - } - - - impl VecRecvEndpoint { - pub fn new(queue: SharedQueue) -> Self { - VecRecvEndpoint { recv: queue } - } - } - - impl RecvEndpoint for VecRecvEndpoint { - type RecvBuffer = Bytes; - fn receive(&mut self) -> Option { - self.recv.pop() - } - } -} - -/// Byte exchange mechanisms which use shared memory queues. -pub mod local { - - use bytes::arc::Bytes; - - use super::{BytesExchange, SharedQueue}; - use super::common::{VecSendEndpoint, VecRecvEndpoint}; - - pub struct LocalBytesExchange { - // forward[i][j] contains a shared queue for data from i to j. - forward: Vec>>, - counter: usize, - } - - impl LocalBytesExchange { - fn new(workers: usize) -> LocalBytesExchange { - - let mut forward = Vec::new(); - for _source in 0 .. workers { - let mut temp_forward = Vec::new(); - for _target in 0 .. workers { - temp_forward.push(SharedQueue::new()); - } - forward.push(temp_forward); - } - - LocalBytesExchange { - forward, - counter: 0, - } - } - } - - impl BytesExchange for LocalBytesExchange { - type Send = VecSendEndpoint; - type Recv = VecRecvEndpoint; - fn next(&mut self) -> Option<(Vec, Vec)> { - - if self.counter < self.forward.len() { - - let mut sends = Vec::with_capacity(self.forward.len()); - - for forward in self.forward[self.counter].iter() { - sends.push(VecSendEndpoint::new(forward.clone())); - } - - let mut recvs = Vec::with_capacity(self.forward.len()); - - for forward in self.forward.iter() { - recvs.push(VecRecvEndpoint::new(forward[self.counter].clone())); - } - - self.counter += 1; - - Some((sends, recvs)) - } - else { None } - } - } -} - -/// A BytesExchange implementation using communication threads and TCP connections. -pub mod tcp { - - use std::io::{Read, Write}; - - use bytes::arc::Bytes; - - use networking::MessageHeader; - use super::{BytesExchange, SharedQueue}; - use super::common::{VecSendEndpoint, VecRecvEndpoint}; - - /// Allocates pairs of byte exchanges for remote workers. - pub struct TcpBytesExchange { - /// Forward[i,j]: from worker i to process j. - forward: Vec>>, - /// Reverse[i,j]: to process i from worker j. - reverse: Vec>>, - } - - impl BytesExchange for TcpBytesExchange { - - type Send = VecSendEndpoint; - type Recv = VecRecvEndpoint; - - // Returns two vectors of length #processes - 1. - // The first contains destinations to send to remote processes, - // The second contains sources to receive from remote processes. - fn next(&mut self) -> Option<(Vec, Vec)> { - - if !self.forward.is_empty() && !self.reverse.is_empty() { - Some(( - self.forward.remove(0).into_iter().map(|x| VecSendEndpoint::new(x)).collect(), - self.reverse.remove(0).into_iter().map(|x| VecRecvEndpoint::new(x)).collect(), - )) - } - else { - None - } - } - } - - impl TcpBytesExchange { - pub fn new(forward: Vec>>, reverse: Vec>>) -> Self { - TcpBytesExchange { - forward, - reverse, - } - } - } - - // Allocates local and remote queue pairs, respectively. - pub fn allocate_queue_pairs(local: usize, remote: usize) -> (Vec>>, Vec>>) { - - // type annotations necessary despite return signature because ... Rust. - let local_to_remote: Vec> = (0 .. local).map(|_| (0 .. remote).map(|_| SharedQueue::new()).collect()).collect(); - let remote_to_local: Vec> = (0 .. remote).map(|r| (0 .. local).map(|l| local_to_remote[l][r].clone()).collect()).collect(); - - (local_to_remote, remote_to_local) - } - - /// Receives serialized data from a `Read`, for example the network. - /// - /// The `BinaryReceiver` repeatedly reads binary data from its reader into - /// a binary Bytes slice which can be broken off and handed to recipients as - /// messages become complete. - pub struct BinaryReceiver { - - worker_offset: usize, - - reader: R, // the generic reader. - buffer: Bytes, // current working buffer. - length: usize, // consumed buffer elements. - targets: Vec>, // to process-local workers. - log_sender: ::logging::CommsLogger, // logging stuffs. - - in_progress: Vec>, // buffers shared with workers. - stash: Vec>, // reclaimed and resuable buffers. - size: usize, // current buffer allocation size. - } - - impl BinaryReceiver { - - pub fn new( - reader: R, - targets: Vec>, - worker_offset: usize, - log_sender: ::logging::CommsLogger) -> BinaryReceiver { - BinaryReceiver { - reader, - targets, - log_sender, - buffer: Bytes::from(vec![0u8; 1 << 20]), - length: 0, - in_progress: Vec::new(), - stash: Vec::new(), - size: 1 << 20, - worker_offset, - } - } - - // Retire `self.buffer` and acquire a new buffer of at least `self.size` bytes. - fn refresh_buffer(&mut self) { - - if self.stash.is_empty() { - for shared in self.in_progress.iter_mut() { - if let Some(bytes) = shared.take() { - match bytes.try_recover::>() { - Ok(vec) => { self.stash.push(vec); }, - Err(bytes) => { *shared = Some(bytes); }, - } - } - } - self.in_progress.retain(|x| x.is_some()); - } - - let self_size = self.size; - self.stash.retain(|x| x.capacity() == self_size); - - - let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; 1 << self.size]); - let new_buffer = Bytes::from(new_buffer); - let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); - - self.buffer[.. self.length].copy_from_slice(&old_buffer[.. self.length]); - - self.in_progress.push(Some(old_buffer)); - } - - pub fn recv_loop(&mut self) { - - // 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. - - loop { - - // Attempt to read some more bytes into self.buffer. - self.length += self.reader.read(&mut self.buffer[self.length ..]).unwrap_or(0); - - // Consume complete messages from the front of self.buffer. - while let Some(header) = MessageHeader::try_read(&mut &self.buffer[.. self.length]) { - // TODO: Consolidate message sequences sent to the same worker. - let peeled_bytes = header.required_bytes(); - let bytes = self.buffer.extract_to(peeled_bytes); - self.length -= peeled_bytes; - self.targets[header.target - self.worker_offset].push(bytes); - } - - // If our buffer is full we should copy it to a new buffer. - if self.length == self.buffer.len() { - // If full and not complete, we must increase the size. - if self.length == self.size { - self.size *= 2; - } - self.refresh_buffer(); - } - } - } - } - - // structure in charge of sending data to a Writer, for example the network. - pub struct BinarySender { - writer: W, - sources: Vec>, - log_sender: ::logging::CommsLogger, - } - - impl BinarySender { - pub fn new(writer: W, sources: Vec>, log_sender: ::logging::CommsLogger) -> BinarySender { - BinarySender { writer, sources, log_sender } - } - - pub fn send_loop(&mut self) { - - let mut stash = Vec::new(); - while !self.sources.is_empty() { - - for source in self.sources.iter_mut() { - source.drain_into(&mut stash); - } - - // If we got zero data, check that everyone is still alive. - if stash.is_empty() { - self.sources.retain(|x| !x.is_done()); - self.writer.flush().expect("Failed to flush writer."); - } - - for bytes in stash.drain(..) { - self.writer.write_all(&bytes[..]).expect("Write failure in send_loop."); - } - - } - } - } -} - -/// Builds an instance of a ProcessBinary. -/// -/// Builders are required because some of the state in a `ProcessBinary` 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 ProcessBinaryBuilder { - inner: Process, - index: usize, // number out of peers - peers: usize, // number of peer allocators (for typed channel allocation). - sends: Vec, // for pushing bytes at each other process. - recvs: Vec, // for pulling bytes from each other process. -} - -impl ProcessBinaryBuilder { - /// 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( - mut byte_exchanger: BE, - my_process: usize, - threads: usize, - processes: usize) -> Vec> { - - Process::new_vector(threads) - .into_iter() - .enumerate() - .map(|(index, inner)| { - let (sends, recvs) = byte_exchanger.next().unwrap(); - ProcessBinaryBuilder { - inner, - index: my_process * threads + index, - peers: threads * processes, - sends, - recvs, - }}) - .collect() - } - - /// Builds a `ProcessBinary`, instantiating `Rc>` elements. - pub fn build(self) -> ProcessBinary { - - let mut sends = Vec::new(); - for send in self.sends.into_iter() { - sends.push(Rc::new(RefCell::new(send))); - } - - ProcessBinary { - inner: self.inner, - index: self.index, - peers: self.peers, - allocated: 0, - sends, - recvs: self.recvs, - to_local: Vec::new(), - } - } -} - -// A specific Communicator for inter-thread intra-process communication -pub struct ProcessBinary { - - inner: Process, // 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). - - // 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 ProcessBinary { - 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 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. - 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(Puller::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() { - - while let Some(mut bytes) = recv.receive() { - - // TODO: We could wrap `bytes` in a bytes::rc::Bytes, - // which could reduce `Arc` overhead, if it hurts. - // This new `Arc` should be local/uncontended, though. - let mut bytes = Bytes::from(bytes); - - // 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); - // } - // } - } -} - -/// 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>> { - - let processes = addresses.len(); - - use networking::create_sockets; - let mut results = create_sockets(addresses, my_index, noisy)?; - - // Send and recv connections between local workers and remote processes. - let (local_send, remote_recv) = tcp::allocate_queue_pairs(threads, results.len() - 1); - let (local_recv, remote_send) = tcp::allocate_queue_pairs(threads, results.len() - 1); - - let mut remote_recv_iter = remote_recv.into_iter(); - let mut remote_send_iter = remote_send.into_iter(); - - // 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 = remote_recv_iter.next().unwrap(); - let remote_send = remote_send_iter.next().unwrap(); - - { - let log_sender = log_sender.clone(); - let stream = stream.try_clone()?; - // start senders and receivers associated with this stream - 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), - }); - - let stream = ::std::io::BufWriter::with_capacity(1 << 20, stream); - tcp::BinarySender::new(stream, remote_recv, log_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 = - ::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), - }); - tcp::BinaryReceiver::new(stream, remote_send, threads * my_index, log_sender) - .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 byte_exchange = tcp::TcpBytesExchange::new(local_send, local_recv); - let builders = ProcessBinaryBuilder::new_vector(byte_exchange, my_index, threads, processes); - - Ok(builders) -} - - -/// 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. -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(); - - // acquire byte buffer and write header, element. - let mut borrow = self.sender.borrow_mut(); - let mut bytes = borrow.reserve(header.required_bytes()); - header.write_to(&mut bytes).expect("failed to write header!"); - - element.into_bytes(&mut 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. -struct Puller { - inner: Box>>, // inner pullable (e.g. intra-process typed queue) - current: Option>, - receiver: Rc>>, // source of serialized buffers -} - -impl Puller { - fn new(inner: Box>>, receiver: Rc>>) -> Puller { - Puller { - inner, - current: None, - receiver, - } - } -} - -impl Pull> for Puller { - #[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/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index 12efaac38..a0807ab36 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -1,53 +1,21 @@ use bytes::arc::Bytes; use super::bytes_slab::BytesSlab; -// /// A type that can allocate send and receive endpoints for byte exchanges. -// /// -// /// The `BytesExchange` intent is that one can abstractly define mechanisms for exchanging -// /// bytes between various entities. In some cases this may be between worker threads within -// /// a process, in other cases it may be between worker threads and remote processes. At the -// /// moment the cardinalities of remote endpoints requires some context and interpretation. -// pub trait BytesExchange { -// /// The type of the send endpoint. -// type Send: SendEndpoint+'static; -// /// The type of the receive endpoint. -// type Recv: RecvEndpoint+'static; -// /// Allocates endpoint pairs for a specified worker. -// /// -// /// Importantly, the Send side may share state to coalesce the buffering and -// /// transmission of records. That is why there are `Rc>` things there. -// fn next(&mut self) -> Option<(Vec, Vec)>; -// } - -// /// A type that can provide and publish writeable binary buffers. -// pub trait SendEndpoint { -// /// The type of the writeable binary buffer. -// type SendBuffer: ::std::io::Write; -// /// Provides a writeable buffer of the requested capacity. -// fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; -// /// Indicates that it is now appropriate to publish the buffer. -// fn publish(&mut self); -// } - -// /// A type that can provide readable binary buffers. -// pub trait RecvEndpoint { -// type RecvBuffer: DerefMut; -// /// Provides a readable buffer. -// fn receive(&mut self) -> Option; -// } - +/// A target for `Bytes`. pub trait BytesPush { fn push(&mut self, bytes: Bytes); } +/// A source for `Bytes`. pub trait BytesPull { fn pull(&mut self) -> Option; } // std::sync::mpsc implementations. -impl BytesPush for ::std::sync::mpsc::Sender { +use ::std::sync::mpsc::{Sender, Receiver}; +impl BytesPush for Sender { fn push(&mut self, bytes: Bytes) { self.send(bytes) .expect("unable to send Bytes"); } } -impl BytesPull for ::std::sync::mpsc::Receiver { +impl BytesPull for Receiver { fn pull(&mut self) -> Option { self.try_recv() .ok() @@ -73,6 +41,7 @@ impl BytesPull for Arc>> { } } +/// A `BytesPush` wrapper which stages writes. pub struct SendEndpoint { send: P, buffer: BytesSlab, @@ -121,22 +90,4 @@ impl Drop for SendEndpoint

{ fn drop(&mut self) { self.send_buffer(); } -} - -// pub struct RecvEndpoint { -// recv: PSharedQueueRecv, -// } - - -// impl BytesRecvEndpoint { -// pub fn new(queue: SharedQueueRecv) -> Self { -// BytesRecvEndpoint { recv: queue } -// } -// } - -// impl RecvEndpoint for BytesRecvEndpoint { -// type RecvBuffer = Bytes; -// fn receive(&mut self) -> Option { -// self.recv.pop() -// } -// } \ No newline at end of file +} \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/shared_queue.rs b/communication/src/allocator/zero_copy/shared_queue.rs deleted file mode 100644 index 7a3500a0d..000000000 --- a/communication/src/allocator/zero_copy/shared_queue.rs +++ /dev/null @@ -1,48 +0,0 @@ -use std::sync::mpsc::{Sender, Receiver}; - -// pub struct SharedQueue { -// queue: Arc<(Mutex>, CondVar)>, -// } - -// impl SharedQueue { -// pub fn push(&mut self, item: T) { -// let lock = queue.0.lock(); -// lock.push_back(item); -// queue.1.notify_all(); -// } - -// pub fn pop(&mut self) -> Option { - -// } -// } - -pub struct SharedQueueSend { - queue: Sender, -} - -impl SharedQueueSend { - pub fn push(&mut self, item: T) { - self.queue - .send(item) - .expect("unable to lock shared queue"); - } - pub fn is_empty(&self) -> bool { - unimplemented!() - } - pub fn from(queue: Sender) -> Self { - SharedQueueSend { - queue, - } - } -} - -pub struct SharedQueueRecv { - queue: Receiver, -} - -impl SharedQueueRecv { - pub fn pop(&mut self) -> Option { - self.queue.try_recv().ok() - } - pub fn from(queue: Receiver) -> Self { SharedQueueRecv { queue } } -} \ No newline at end of file From b37bb3eea11b9eaee7ab2f4227a8463ee5e2bafd Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 25 Aug 2018 21:41:39 +0200 Subject: [PATCH 33/43] sizing bug in bytesslab --- communication/src/allocator/zero_copy/bytes_slab.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/communication/src/allocator/zero_copy/bytes_slab.rs b/communication/src/allocator/zero_copy/bytes_slab.rs index 3000e83a9..9396f8264 100644 --- a/communication/src/allocator/zero_copy/bytes_slab.rs +++ b/communication/src/allocator/zero_copy/bytes_slab.rs @@ -60,6 +60,7 @@ impl BytesSlab { } } self.in_progress.retain(|x| x.is_some()); + self.stash.retain(|x| x.len() == 1 << self.shift); } let new_buffer = self.stash.pop().unwrap_or_else(|| Bytes::from(vec![0; 1 << self.shift].into_boxed_slice())); From 06a36ebfbdb8a930994094ba6af49bf7046530a4 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 25 Aug 2018 21:45:45 +0200 Subject: [PATCH 34/43] sizing bug in bytesslab --- communication/src/allocator/zero_copy/bytes_slab.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/communication/src/allocator/zero_copy/bytes_slab.rs b/communication/src/allocator/zero_copy/bytes_slab.rs index 9396f8264..2bab83724 100644 --- a/communication/src/allocator/zero_copy/bytes_slab.rs +++ b/communication/src/allocator/zero_copy/bytes_slab.rs @@ -60,7 +60,8 @@ impl BytesSlab { } } self.in_progress.retain(|x| x.is_some()); - self.stash.retain(|x| x.len() == 1 << self.shift); + let self_shift = self.shift; + self.stash.retain(|x| x.len() == 1 << self_shift); } let new_buffer = self.stash.pop().unwrap_or_else(|| Bytes::from(vec![0; 1 << self.shift].into_boxed_slice())); From afb5b0431b234b75017dba61230df1f7f4528f28 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 27 Aug 2018 16:08:08 +0200 Subject: [PATCH 35/43] use prior default process allocator --- communication/src/initialize.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index cd5c6f394..c25edac0c 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -87,8 +87,8 @@ fn create_allocators(config: Configuration, logger: LogBuilder) -> Result<(Vec { - // 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(()))) + 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, guard)) = initialize_networking(addresses, process, threads, report, logger) { From 8124a46b2b0d7055e2552ff9b0eeaa4309124e64 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 28 Aug 2018 19:28:25 +0200 Subject: [PATCH 36/43] documenting, reorganization, perf bugs --- Cargo.toml | 4 +- communication/Cargo.toml | 2 +- communication/src/allocator/generic.rs | 40 +-- communication/src/allocator/mod.rs | 182 ++-------- communication/src/allocator/process.rs | 9 +- communication/src/allocator/thread.rs | 8 +- .../src/allocator/zero_copy/allocator.rs | 100 +++--- .../allocator/zero_copy/allocator_process.rs | 6 +- .../src/allocator/zero_copy/bytes_exchange.rs | 19 +- .../src/allocator/zero_copy/bytes_slab.rs | 73 ++-- .../src/allocator/zero_copy/initialize.rs | 19 +- communication/src/allocator/zero_copy/mod.rs | 8 +- .../src/allocator/zero_copy/push_pull.rs | 4 + .../allocator/zero_copy/{binary.rs => tcp.rs} | 4 +- communication/src/initialize.rs | 6 +- communication/src/lib.rs | 8 +- communication/src/logging.rs | 25 +- communication/src/message.rs | 158 ++++++++ communication/src/networking.rs | 338 ++---------------- src/dataflow/channels/message.rs | 226 ------------ src/dataflow/channels/mod.rs | 6 +- src/dataflow/channels/pact.rs | 8 +- src/dataflow/channels/pushers/buffer.rs | 2 +- src/dataflow/channels/pushers/tee.rs | 2 +- src/dataflow/operators/broadcast.rs | 4 +- src/dataflow/operators/capture/capture.rs | 2 +- src/dataflow/operators/count.rs | 2 +- src/dataflow/operators/generic/handles.rs | 3 +- src/dataflow/operators/input.rs | 2 +- src/dataflow/operators/probe.rs | 2 +- src/dataflow/scopes/child.rs | 3 +- src/dataflow/scopes/mod.rs | 2 +- src/dataflow/scopes/root.rs | 3 +- src/execute.rs | 2 +- src/lib.rs | 7 +- src/logging.rs | 20 +- src/progress/broadcast.rs | 10 +- src/progress/frontier.rs | 9 + src/progress/nested/reachability.rs | 3 +- src/progress/nested/subgraph.rs | 2 +- 40 files changed, 468 insertions(+), 865 deletions(-) rename communication/src/allocator/zero_copy/{binary.rs => tcp.rs} (98%) create mode 100644 communication/src/message.rs delete mode 100644 src/dataflow/channels/message.rs diff --git a/Cargo.toml b/Cargo.toml index ceee16e59..06686a588 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -18,11 +18,9 @@ license = "MIT" [dependencies] abomonation = { git = "https://github.com/frankmcsherry/abomonation" } abomonation_derive = "0.3" -#timely_communication = "0.6" +bytes = { path = "./bytes" } timely_communication = { path = "./communication"} -byteorder="1" time="0.1.34" -bytes = { path = "./bytes" } [dev-dependencies] timely_sort="0.1.6" diff --git a/communication/Cargo.toml b/communication/Cargo.toml index 36b0e142a..381e8eda7 100644 --- a/communication/Cargo.toml +++ b/communication/Cargo.toml @@ -17,7 +17,7 @@ default=["arg_parse"] arg_parse=["getopts"] [dependencies] -byteorder="1" +#byteorder="1" getopts={version="0.2.14", optional=true} time="0.1.34" abomonation = { git = "https://github.com/frankmcsherry/abomonation" } diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index b762672cc..708b3d2ab 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -3,26 +3,23 @@ //! 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, Message, Thread, Process, Binary}; -// use allocator::process_binary::{ProcessBinary, ProcessBinaryBuilder}; - -use allocator::zero_copy::allocator_process::ProcessAllocator; -use allocator::zero_copy::allocator_process::ProcessBuilder; - -use allocator::zero_copy::allocator::TcpAllocator as ZeroCopyAllocator; -use allocator::zero_copy::allocator::TcpBuilder as ZeroCopyBuilder; -// use allocator::zero_copy::binary::TcpBytesExchange; +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), - ZeroCopy(ZeroCopyAllocator), + /// Inter-process allocator. + ZeroCopy(TcpAllocator), } impl Generic { @@ -31,7 +28,6 @@ impl Generic { match self { &Generic::Thread(ref t) => t.index(), &Generic::Process(ref p) => p.index(), - &Generic::Binary(ref b) => b.index(), &Generic::ProcessBinary(ref pb) => pb.index(), &Generic::ZeroCopy(ref z) => z.index(), } @@ -41,7 +37,6 @@ impl Generic { match self { &Generic::Thread(ref t) => t.peers(), &Generic::Process(ref p) => p.peers(), - &Generic::Binary(ref b) => b.peers(), &Generic::ProcessBinary(ref pb) => pb.peers(), &Generic::ZeroCopy(ref z) => z.peers(), } @@ -51,26 +46,24 @@ impl Generic { match self { &mut Generic::Thread(ref mut t) => t.allocate(), &mut Generic::Process(ref mut p) => p.allocate(), - &mut Generic::Binary(ref mut b) => b.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::Binary(ref mut b) => b.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::Binary(ref mut b) => b.post_work(), &mut Generic::ProcessBinary(ref mut pb) => pb.post_work(), &mut Generic::ZeroCopy(ref mut z) => z.post_work(), } @@ -95,19 +88,22 @@ impl Allocate for Generic { /// 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), - Binary(Binary), + /// Builder for `ProcessBinary` allocator. ProcessBinary(ProcessBuilder), - ZeroCopy(ZeroCopyBuilder), + /// Builder for `ZeroCopy` allocator. + ZeroCopy(TcpBuilder), } -impl GenericBuilder { - pub fn build(self) -> Generic { +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::Binary(b) => Generic::Binary(b), 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 412ef437c..1a8c46c4e 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -1,179 +1,35 @@ -//! 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::binary::Binary; pub use self::generic::{Generic, GenericBuilder}; -use bytes::arc::Bytes; -use abomonation::{Abomonation, abomonated::Abomonated, encode, measure}; - pub mod thread; pub mod process; -pub mod binary; +// pub mod binary; pub mod generic; pub mod zero_copy; -use {Data, Push, Pull}; - -/// 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. - 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`. - 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!() - } - } +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; } -// 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. +/// 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; @@ -181,6 +37,8 @@ pub trait Allocate { fn peers(&self) -> usize; /// Constructs several send endpoints and one receive endpoint. 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 7468d089b..a9cadafea 100644 --- a/communication/src/allocator/process.rs +++ b/communication/src/allocator/process.rs @@ -1,3 +1,5 @@ +//! Typed inter-thread, intra-process channels. + use std::sync::{Arc, Mutex}; use std::any::Any; use std::sync::mpsc::{Sender, Receiver, channel}; @@ -5,7 +7,7 @@ use std::sync::mpsc::{Sender, Receiver, channel}; use allocator::{Allocate, 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 { @@ -71,7 +75,7 @@ impl Allocate for Process { } } -// an observer wrapping a Rust channel +/// The push half of an intra-process channel. struct Pusher { target: Sender, } @@ -90,6 +94,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 66443c143..8724efa4c 100644 --- a/communication/src/allocator/thread.rs +++ b/communication/src/allocator/thread.rs @@ -1,3 +1,5 @@ +//! Intra-thread communication. + use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; @@ -6,7 +8,7 @@ use allocator::{Allocate, 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 } @@ -18,6 +20,7 @@ impl Allocate for Thread { } 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 +28,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 +44,7 @@ impl Push for Pusher { } } +/// The pull half of an intra-thread channel. pub struct Puller { current: Option, source: Rc, VecDeque)>>, diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index b1489237c..cd4a5cc55 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -1,3 +1,4 @@ +//! Zero-copy allocator based on TCP. use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; @@ -19,60 +20,61 @@ use super::push_pull::{Pusher, PullerInner}; /// 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: Process, +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: Receiver, // for pulling bytes from remote processes. } -impl TcpBuilder { - /// 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( - my_process: usize, - threads: usize, - processes: usize) -> (Vec, Vec>, Vec>) { - - let mut l2r_send = Vec::new(); - let mut l2r_recv = Vec::new(); - let mut r2l_send = Vec::new(); - let mut r2l_recv = Vec::new(); - - for _ in 0 .. threads { - let (send, recv) = channel(); - r2l_send.push(send); - r2l_recv.push(recv); - } - - for _ in 0 .. processes - 1 { - let (send, recv) = channel(); - l2r_send.push(send); - l2r_recv.push(recv); - } +/// 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( + my_process: usize, + threads: usize, + processes: usize) -> (Vec>, Vec>, Vec>) { + + let mut l2r_send = Vec::new(); + let mut l2r_recv = Vec::new(); + let mut r2l_send = Vec::new(); + let mut r2l_recv = Vec::new(); + + for _ in 0 .. threads { + let (send, recv) = channel(); + r2l_send.push(send); + r2l_recv.push(recv); + } - let builders = - Process::new_vector(threads) - .into_iter() - .zip(r2l_recv.into_iter()) - .enumerate() - .map(|(index, (inner, recvs))| { - TcpBuilder { - inner, - index: my_process * threads + index, - peers: threads * processes, - sends: l2r_send.clone(), - recvs, - }}) - .collect(); - - (builders, l2r_recv, r2l_send) + for _ in 0 .. processes - 1 { + let (send, recv) = channel(); + l2r_send.push(send); + l2r_recv.push(recv); } + let builders = + Process::new_vector(threads) + .into_iter() + .zip(r2l_recv.into_iter()) + .enumerate() + .map(|(index, (inner, recvs))| { + TcpBuilder { + inner, + index: my_process * threads + index, + peers: threads * processes, + sends: l2r_send.clone(), + recvs, + }}) + .collect(); + + (builders, l2r_recv, r2l_send) +} + +impl TcpBuilder { + /// Builds a `TcpAllocator`, instantiating `Rc>` elements. - pub fn build(self) -> TcpAllocator { + pub fn build(self) -> TcpAllocator { let mut sends = Vec::new(); for send in self.sends.into_iter() { @@ -92,10 +94,10 @@ impl TcpBuilder { } } -// A specific Communicator for inter-thread intra-process communication -pub struct TcpAllocator { +/// A TCP-based allocator for inter-process communication. +pub struct TcpAllocator { - inner: Process, // A non-serialized inner allocator for process-local peers. + 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). @@ -107,7 +109,7 @@ pub struct TcpAllocator { to_local: Vec>>>, // to worker-local typed pullers. } -impl Allocate for TcpAllocator { +impl Allocate for TcpAllocator { fn index(&self) -> usize { self.index } fn peers(&self) -> usize { self.peers } fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { @@ -165,7 +167,7 @@ impl Allocate for TcpAllocator { // No splitting occurs across allocations. while bytes.len() > 0 { - if let Some(header) = MessageHeader::try_read(&mut &bytes[..]) { + 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()); diff --git a/communication/src/allocator/zero_copy/allocator_process.rs b/communication/src/allocator/zero_copy/allocator_process.rs index 11e8d3495..80e08a523 100644 --- a/communication/src/allocator/zero_copy/allocator_process.rs +++ b/communication/src/allocator/zero_copy/allocator_process.rs @@ -1,3 +1,5 @@ +//! Zero-copy allocator for intra-process serialized communication. + use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; @@ -81,7 +83,7 @@ impl ProcessBuilder { } } -// A specific Communicator for inter-thread intra-process communication +/// A serializing allocator for inter-thread intra-process communication. pub struct ProcessAllocator { index: usize, // number out of peers @@ -139,7 +141,7 @@ impl Allocate for ProcessAllocator { // No splitting occurs across allocations. while bytes.len() > 0 { - if let Some(header) = MessageHeader::try_read(&mut &bytes[..]) { + 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()); diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index a0807ab36..5a233a992 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -1,10 +1,18 @@ +//! Types and traits for sharing `Bytes`. + use bytes::arc::Bytes; use super::bytes_slab::BytesSlab; /// A target for `Bytes`. -pub trait BytesPush { fn push(&mut self, bytes: Bytes); } +pub trait BytesPush { + /// Pushes bytes at the instance. + fn push(&mut self, bytes: Bytes); +} /// A source for `Bytes`. -pub trait BytesPull { fn pull(&mut self) -> Option; } +pub trait BytesPull { + /// Pulls bytes from the instance. + fn pull(&mut self) -> Option; +} // std::sync::mpsc implementations. use ::std::sync::mpsc::{Sender, Receiver}; @@ -65,11 +73,14 @@ impl SendEndpoint

{ 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 { @@ -80,7 +91,7 @@ impl SendEndpoint

{ 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(); } diff --git a/communication/src/allocator/zero_copy/bytes_slab.rs b/communication/src/allocator/zero_copy/bytes_slab.rs index 2bab83724..2054f88c5 100644 --- a/communication/src/allocator/zero_copy/bytes_slab.rs +++ b/communication/src/allocator/zero_copy/bytes_slab.rs @@ -1,5 +1,12 @@ +//! 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. @@ -9,7 +16,7 @@ pub struct BytesSlab { } 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()), @@ -19,55 +26,63 @@ impl BytesSlab { valid: 0, } } - + /// The empty region of the slab. pub fn empty(&mut self) -> &mut [u8] { &mut self.buffer[self.valid..] } - - pub fn valid(&mut self) -> &[u8] { + /// 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) } - // Retire `self.buffer` and acquire a new buffer of at least `self.size` 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) { - // Increase allocation if insufficient. - while self.valid + capacity > (1 << self.shift) { - self.shift += 1; - self.stash.clear(); - } + if self.empty().len() < capacity { - // 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); + // Increase allocation if copy would be insufficient. + while self.valid + capacity > (1 << self.shift) { + self.shift += 1; + self.stash.clear(); + } + + // 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 self_shift = self.shift; + self.stash.retain(|x| x.len() == 1 << self_shift); } - self.in_progress.retain(|x| x.is_some()); - let self_shift = self.shift; - self.stash.retain(|x| x.len() == 1 << self_shift); - } - 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); + 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)); - 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 index 052f93645..79d805c18 100644 --- a/communication/src/allocator/zero_copy/initialize.rs +++ b/communication/src/allocator/zero_copy/initialize.rs @@ -1,9 +1,15 @@ +//! Network initialization. use std::sync::Arc; - -use super::binary::{send_loop, recv_loop}; -use super::allocator::TcpBuilder; - +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<()>>, @@ -29,14 +35,13 @@ pub fn initialize_networking( threads: usize, noisy: bool, log_sender: Arc::logging::CommsLogger+Send+Sync>) --> ::std::io::Result<(Vec, CommsGuard)> { +-> ::std::io::Result<(Vec>, CommsGuard)> { let processes = addresses.len(); - use networking::create_sockets; let mut results = create_sockets(addresses, my_index, noisy)?; - let (builders, remote_recvs, remote_sends) = TcpBuilder::new_vector(my_index, threads, processes); + let (builders, remote_recvs, remote_sends) = new_vector(my_index, threads, processes); let mut remote_recv_iter = remote_recvs.into_iter(); let mut send_guards = Vec::new(); diff --git a/communication/src/allocator/zero_copy/mod.rs b/communication/src/allocator/zero_copy/mod.rs index 70ff6d2ff..e4bf9e396 100644 --- a/communication/src/allocator/zero_copy/mod.rs +++ b/communication/src/allocator/zero_copy/mod.rs @@ -1,6 +1,12 @@ +//! 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 binary; +pub mod tcp; pub mod allocator; pub mod allocator_process; pub mod initialize; diff --git a/communication/src/allocator/zero_copy/push_pull.rs b/communication/src/allocator/zero_copy/push_pull.rs index 51735ff86..5fe65667f 100644 --- a/communication/src/allocator/zero_copy/push_pull.rs +++ b/communication/src/allocator/zero_copy/push_pull.rs @@ -1,3 +1,5 @@ +//! Push and Pull implementations wrapping serialized data. + use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; @@ -69,6 +71,7 @@ pub struct Puller { } impl Puller { + /// Creates a new `Puller` instance from a shared queue. pub fn new(receiver: Rc>>) -> Puller { Puller { current: None, @@ -103,6 +106,7 @@ pub struct PullerInner { } impl PullerInner { + /// Creates a new `PullerInner` instance from a shared queue. pub fn new(inner: Box>>, receiver: Rc>>) -> Self { PullerInner { inner, diff --git a/communication/src/allocator/zero_copy/binary.rs b/communication/src/allocator/zero_copy/tcp.rs similarity index 98% rename from communication/src/allocator/zero_copy/binary.rs rename to communication/src/allocator/zero_copy/tcp.rs index a7cc1af02..21a33af49 100644 --- a/communication/src/allocator/zero_copy/binary.rs +++ b/communication/src/allocator/zero_copy/tcp.rs @@ -1,3 +1,5 @@ +//! + use std::io::{Read, Write}; use std::sync::mpsc::{Sender, Receiver}; use std::net::TcpStream; @@ -50,7 +52,7 @@ pub fn recv_loop( buffer.make_valid(read); // Consume complete messages from the front of self.buffer. - while let Some(header) = MessageHeader::try_read(&mut buffer.valid()) { + while let Some(header) = MessageHeader::try_read(buffer.valid()) { // TODO: Consolidate message sequences sent to the same worker? let peeled_bytes = header.required_bytes(); diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index c25edac0c..145b8c5a4 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -9,10 +9,8 @@ use std::sync::Arc; use std::any::Any; -use allocator::{Thread, Process, Generic, GenericBuilder}; -// use allocator::process_binary::ProcessBinaryBuilder; -// use networking::initialize_networking; -use allocator::zero_copy::allocator_process::ProcessBuilder; +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. diff --git a/communication/src/lib.rs b/communication/src/lib.rs index 873b1c82d..8b65959f6 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -74,11 +74,10 @@ //! result: Ok(1) //! ``` -// #![forbid(missing_docs)] +#![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; @@ -86,9 +85,10 @@ extern crate time; extern crate bytes; pub mod allocator; -mod networking; +pub mod networking; pub mod initialize; pub mod logging; +pub mod message; use std::any::Any; @@ -97,7 +97,7 @@ use abomonation::Abomonation; pub use allocator::Generic as Allocator; pub use allocator::Allocate; pub use initialize::{initialize, Configuration, WorkerGuards}; -pub use allocator::Message as CommMessage; +pub use message::Message; /// A composite trait for types that may be used with channels. pub trait Data : Send+Any+Abomonation+'static { } 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 9cfe84ef6..50436240c 100644 --- a/communication/src/networking.rs +++ b/communication/src/networking.rs @@ -1,322 +1,60 @@ //! 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}; - -// 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 + /// Returns a header when there is enough supporting data #[inline(always)] - pub 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 } + pub fn try_read(bytes: &mut [u8]) -> Option { + unsafe { decode::(bytes) } + .and_then(|(header, remaining)| { + if remaining.len() > header.length { + Some(header.clone()) + } + else { + None + } + }) } + /// Writes the header as binary data. #[inline(always)] - pub 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(()) + pub fn write_to(&self, writer: &mut W) -> ::std::io::Result<()> { + unsafe { encode(self, writer) } } + /// The number of bytes required for the header and data. #[inline(always)] pub fn required_bytes(&self) -> usize { - 40 + self.length + ::std::mem::size_of::() + self.length } } -// 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..]; - } - - 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]; - } - 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, - } - } - - 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(..) { - 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 - } - } -} - -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); } - - // 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 mut results = create_sockets(addresses, my_index, noisy)?; - - 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), - }); - BinarySender::new(BufWriter::with_capacity(1 << 20, stream), sender_channels_r,log_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), - }); - BinaryReceiver::new(stream, reader_channels_r, log_sender) - .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) -} - - +/// Creates socket connections from a list of host addresses. pub fn create_sockets(addresses: Vec, my_index: usize, noisy: bool) -> Result>> { let hosts1 = Arc::new(addresses); @@ -336,8 +74,8 @@ pub fn create_sockets(addresses: Vec, my_index: usize, noisy: bool) -> R } -// 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; @@ -345,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"); - 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; @@ -361,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 = listener.accept()?.0; stream.set_nodelay(true).expect("set_nodelay call failed"); - let identifier = 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/src/dataflow/channels/message.rs b/src/dataflow/channels/message.rs deleted file mode 100644 index 5b76a93fb..000000000 --- a/src/dataflow/channels/message.rs +++ /dev/null @@ -1,226 +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 length_in_bytes(&self) -> usize { - measure(&self.time) + measure(&self.from) + measure(&self.seq) + measure(self.data.deref()) - } - - #[inline] - fn into_bytes(&mut self, bytes: &mut W) { - - // // 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()); - - // 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(mut bytes: ::bytes::arc::Bytes) -> Self { - let x_len = bytes.len(); - let (time, from, seq, offset) = { - let (t,r) = unsafe { decode::(&mut bytes) }.expect("failed to get time"); - let (&f,r) = unsafe { decode::(r) }.expect("failed to get from"); - let (&s,r) = unsafe { decode::(r) }.expect("failed to get seq"); - 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..]) }.expect("failed to deser Vec").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. -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. - /// - /// TODO: We use an `arc::Bytes` here because `Content` needs to be `Send`. - /// This requirement seems artificial, and should probably be fixed. - Bytes(::bytes::arc::Bytes, usize, usize), - /// Typed data, which may be efficiently mutated or claimed for ownership. - Typed(Vec), -} - -impl Clone for Content { - fn clone(&self) -> Self { - unimplemented!() - } -} -// 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)) }; - // 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 a3c4822c5..5d7ee9790 100644 --- a/src/dataflow/channels/mod.rs +++ b/src/dataflow/channels/mod.rs @@ -1,18 +1,16 @@ //! Structured communication between timely dataflow operators. -use timely_communication::Push; +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 = ::timely_communication::allocator::Message>; +pub type Bundle = ::communication::Message>; /// A serializable representation of timestamped data. #[derive(Clone, Abomonation)] diff --git a/src/dataflow/channels/pact.rs b/src/dataflow/channels/pact.rs index 46a7b3929..70e18ed8a 100644 --- a/src/dataflow/channels/pact.rs +++ b/src/dataflow/channels/pact.rs @@ -9,10 +9,10 @@ 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 super::{Bundle, Message}; diff --git a/src/dataflow/channels/pushers/buffer.rs b/src/dataflow/channels/pushers/buffer.rs index 75ec69106..af6ad4ca0 100644 --- a/src/dataflow/channels/pushers/buffer.rs +++ b/src/dataflow/channels/pushers/buffer.rs @@ -4,7 +4,7 @@ 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. /// diff --git a/src/dataflow/channels/pushers/tee.rs b/src/dataflow/channels/pushers/tee.rs index 682d038e9..437eeafce 100644 --- a/src/dataflow/channels/pushers/tee.rs +++ b/src/dataflow/channels/pushers/tee.rs @@ -6,7 +6,7 @@ use std::cell::RefCell; 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 { diff --git a/src/dataflow/operators/broadcast.rs b/src/dataflow/operators/broadcast.rs index 7f14de681..123af5c14 100644 --- a/src/dataflow/operators/broadcast.rs +++ b/src/dataflow/operators/broadcast.rs @@ -1,6 +1,6 @@ //! Broadcast records to all workers. -use timely_communication::Pull; +use communication::Pull; use ::ExchangeData; use progress::nested::subgraph::{Source, Target}; @@ -87,7 +87,7 @@ impl Operate for BroadcastOperator { let mut vec = Vec::new(); while let Some(bundle) = self.input.next() { - use timely_communication::allocator::RefOrMut; + use communication::message::RefOrMut; match bundle.as_ref_or_mut() { RefOrMut::Ref(bundle) => { diff --git a/src/dataflow/operators/capture/capture.rs b/src/dataflow/operators/capture/capture.rs index 667029ea6..62c33ef36 100644 --- a/src/dataflow/operators/capture/capture.rs +++ b/src/dataflow/operators/capture/capture.rs @@ -140,7 +140,7 @@ impl Capture for Stream { }, move |consumed, _internal, _external| { - use timely_communication::allocator::RefOrMut; + use communication::message::RefOrMut; // turn each received message into an event. let mut borrow = event_pusher2.borrow_mut(); diff --git a/src/dataflow/operators/count.rs b/src/dataflow/operators/count.rs index 357a9d323..320451645 100644 --- a/src/dataflow/operators/count.rs +++ b/src/dataflow/operators/count.rs @@ -1,7 +1,7 @@ //! Counts the number of records at each time. use std::collections::HashMap; -use timely_communication::allocator::RefOrMut; +use communication::message::RefOrMut; use Data; use dataflow::channels::pact::Pipeline; diff --git a/src/dataflow/operators/generic/handles.rs b/src/dataflow/operators/generic/handles.rs index 8270fb911..1d729779d 100644 --- a/src/dataflow/operators/generic/handles.rs +++ b/src/dataflow/operators/generic/handles.rs @@ -5,7 +5,6 @@ use std::rc::Rc; use std::cell::RefCell; -// use std::borrow::Cow; use ::Data; use progress::Timestamp; @@ -15,7 +14,7 @@ use dataflow::channels::pullers::Counter as PullCounter; use dataflow::channels::pushers::Counter as PushCounter; use dataflow::channels::pushers::buffer::{Buffer, Session}; use dataflow::channels::Bundle; -use timely_communication::{Push, Pull, allocator::RefOrMut}; +use communication::{Push, Pull, message::RefOrMut}; use logging::Logger; use dataflow::operators::CapabilityRef; diff --git a/src/dataflow/operators/input.rs b/src/dataflow/operators/input.rs index 9cec262bf..267a6ecad 100644 --- a/src/dataflow/operators/input.rs +++ b/src/dataflow/operators/input.rs @@ -11,7 +11,7 @@ use progress::ChangeBatch; use progress::timestamp::RootTimestamp; use progress::nested::product::Product; -use timely_communication::Allocate; +use communication::Allocate; use Data; use communication::Push; use dataflow::channels::Message; diff --git a/src/dataflow/operators/probe.rs b/src/dataflow/operators/probe.rs index e5f48fb42..bd9130e93 100644 --- a/src/dataflow/operators/probe.rs +++ b/src/dataflow/operators/probe.rs @@ -112,7 +112,7 @@ impl Probe for Stream { started = true; } - use timely_communication::allocator::RefOrMut; + use communication::message::RefOrMut; while let Some(message) = input.next() { let (time, data) = match message.as_ref_or_mut() { diff --git a/src/dataflow/scopes/child.rs b/src/dataflow/scopes/child.rs index 6f2d48892..3e18ac682 100644 --- a/src/dataflow/scopes/child.rs +++ b/src/dataflow/scopes/child.rs @@ -5,7 +5,6 @@ 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 communication::{Allocate, Data, Push, Pull}; use logging::Logger; @@ -87,7 +86,7 @@ impl<'a, G: ScopeParent, T: Timestamp> Scope for Child<'a, G, T> { } } -use timely_communication::allocator::Message; +use communication::Message; impl<'a, G: ScopeParent, T: Timestamp> Allocate for Child<'a, G, T> { fn index(&self) -> usize { self.parent.index() } 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 13e91e607..7749ea7ef 100644 --- a/src/dataflow/scopes/root.rs +++ b/src/dataflow/scopes/root.rs @@ -8,7 +8,6 @@ use std::any::Any; use progress::timestamp::RootTimestamp; use progress::{Timestamp, Operate, SubgraphBuilder}; use logging::Logger; -// use timely_communication::{Allocate, Data}; use communication::{Allocate, Data, Push, Pull}; use super::{ScopeParent, Child}; @@ -132,7 +131,7 @@ impl ScopeParent for Root { } } -use timely_communication::allocator::Message; +use communication::Message; impl Allocate for Root { fn index(&self) -> usize { self.allocator.borrow().index() } 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 050268cfc..99d0a09a9 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -55,13 +55,12 @@ //! 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}; @@ -91,5 +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 { } \ No newline at end of file +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 9ce1fc21b..904973571 100644 --- a/src/progress/broadcast.rs +++ b/src/progress/broadcast.rs @@ -1,15 +1,9 @@ //! Broadcasts progress information among workers. -use timely_communication::allocator::Message; - -use progress::Timestamp; -use progress::ChangeBatch; -use timely_communication::Allocate; -use communication::{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 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 eef329ae4..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; From 83a3806c1392cf373df6a742c43cb6eb7d55818e Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 28 Aug 2018 20:02:42 +0200 Subject: [PATCH 37/43] choice of inequality is important --- communication/src/networking.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/communication/src/networking.rs b/communication/src/networking.rs index 50436240c..ebfde5690 100644 --- a/communication/src/networking.rs +++ b/communication/src/networking.rs @@ -32,7 +32,7 @@ impl MessageHeader { pub fn try_read(bytes: &mut [u8]) -> Option { unsafe { decode::(bytes) } .and_then(|(header, remaining)| { - if remaining.len() > header.length { + if remaining.len() >= header.length { Some(header.clone()) } else { From db5a8569c57215055d51fe6e18d79268d6febefc Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Wed, 29 Aug 2018 10:12:42 +0200 Subject: [PATCH 38/43] stop eager posting --- communication/src/allocator/zero_copy/bytes_exchange.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index 5a233a992..696a23c90 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -78,7 +78,7 @@ impl SendEndpoint

{ /// 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(); + // self.send_buffer(); } /// Acquires a prefix of `self.empty()` of length at least `capacity`. pub fn reserve(&mut self, capacity: usize) -> &mut [u8] { From 400665ced5c11b9ee662979afe53975c59f72fc3 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 30 Aug 2018 08:18:24 +0200 Subject: [PATCH 39/43] add try_merge functionality --- bytes/src/lib.rs | 71 +++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 70 insertions(+), 1 deletion(-) diff --git a/bytes/src/lib.rs b/bytes/src/lib.rs index cc563bc99..451550d00 100644 --- a/bytes/src/lib.rs +++ b/bytes/src/lib.rs @@ -195,6 +195,24 @@ 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. + /// + /// #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) { @@ -212,6 +230,24 @@ pub mod arc { /// 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"); @@ -223,6 +259,39 @@ pub mod arc { 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 { @@ -237,4 +306,4 @@ pub mod arc { unsafe { ::std::slice::from_raw_parts_mut(self.ptr, self.len) } } } -} +} \ No newline at end of file From aa7dab42ae6c89a786ef4877ed2ec0b4363e9645 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 30 Aug 2018 08:18:56 +0200 Subject: [PATCH 40/43] merge_queue --- .../src/allocator/zero_copy/allocator.rs | 71 ++++--- .../allocator/zero_copy/allocator_process.rs | 40 ++-- .../src/allocator/zero_copy/bytes_exchange.rs | 177 +++++++++++++++--- .../src/allocator/zero_copy/initialize.rs | 12 +- communication/src/allocator/zero_copy/tcp.rs | 56 ++++-- 5 files changed, 264 insertions(+), 92 deletions(-) diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index cd4a5cc55..da4dd2bc6 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -2,7 +2,7 @@ use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; -use std::sync::mpsc::{channel, Sender, Receiver}; +// use std::sync::mpsc::{channel, Sender, Receiver}; use bytes::arc::Bytes; @@ -11,7 +11,7 @@ use networking::MessageHeader; use {Allocate, Data, Push, Pull}; use allocator::{Message, Process}; -use super::bytes_exchange::{BytesPull, SendEndpoint}; +use super::bytes_exchange::{BytesPull, SendEndpoint, MergeQueue, Signal}; use super::push_pull::{Pusher, PullerInner}; /// Builds an instance of a TcpAllocator. @@ -22,53 +22,54 @@ use super::push_pull::{Pusher, PullerInner}; /// 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: Receiver, // for pulling bytes from remote processes. + 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. -/// -/// This method requires access to a byte exchanger, from which it mints channels. pub fn new_vector( my_process: usize, threads: usize, - processes: usize) -> (Vec>, Vec>, Vec>) { + processes: usize) +// -> (Vec>, Vec>, Vec>) { +-> (Vec>, Vec<(Vec, Signal)>, Vec>) { - let mut l2r_send = Vec::new(); - let mut l2r_recv = Vec::new(); - let mut r2l_send = Vec::new(); - let mut r2l_recv = Vec::new(); + // 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. - for _ in 0 .. threads { - let (send, recv) = channel(); - r2l_send.push(send); - r2l_recv.push(recv); - } + let worker_signals: Vec = (0 .. threads).map(|_| Signal::new()).collect(); + let network_signals: Vec = (0 .. processes-1).map(|_| Signal::new()).collect(); - for _ in 0 .. processes - 1 { - let (send, recv) = channel(); - l2r_send.push(send); - l2r_recv.push(recv); - } + 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(r2l_recv.into_iter()) + .zip(worker_signals) + .zip(worker_to_network) + .zip(worker_from_network) .enumerate() - .map(|(index, (inner, recvs))| { + .map(|(index, (((inner, signal), sends), recvs))| { TcpBuilder { inner, index: my_process * threads + index, peers: threads * processes, - sends: l2r_send.clone(), + sends, recvs, + signal, }}) .collect(); - (builders, l2r_recv, r2l_send) + let sends = network_from_worker.into_iter().zip(network_signals).collect(); + + (builders, sends, network_to_worker) } impl TcpBuilder { @@ -87,6 +88,8 @@ impl TcpBuilder { index: self.index, peers: self.peers, allocated: 0, + signal: self.signal, + staged: Vec::new(), sends, recvs: self.recvs, to_local: Vec::new(), @@ -103,9 +106,13 @@ pub struct TcpAllocator { 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: Receiver, // recvs[x] <- from process x?. + sends: Vec>>>, // sends[x] -> goes to process x. + recvs: Vec, // recvs[x] <- from process x?. to_local: Vec>>>, // to worker-local typed pullers. } @@ -161,7 +168,11 @@ impl Allocate for TcpAllocator { #[inline(never)] fn pre_work(&mut self) { - while let Some(mut bytes) = self.recvs.pull() { + 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. diff --git a/communication/src/allocator/zero_copy/allocator_process.rs b/communication/src/allocator/zero_copy/allocator_process.rs index 80e08a523..189a4733a 100644 --- a/communication/src/allocator/zero_copy/allocator_process.rs +++ b/communication/src/allocator/zero_copy/allocator_process.rs @@ -76,6 +76,7 @@ impl ProcessBuilder { index: self.index, peers: self.peers, allocated: 0, + staged: Vec::new(), sends, recvs: self.recvs, to_local: Vec::new(), @@ -91,6 +92,7 @@ pub struct ProcessAllocator { allocated: usize, // indicates how many channels have been allocated (locally). // 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. @@ -135,30 +137,32 @@ impl Allocate for ProcessAllocator { fn pre_work(&mut self) { for recv in self.recvs.iter_mut() { - while let Some(mut bytes) = recv.pull() { + recv.drain_into(&mut self.staged); + } - // We expect that `bytes` contains an integral number of messages. - // No splitting occurs across allocations. - while bytes.len() > 0 { + for mut bytes in self.staged.drain(..) { - if let Some(header) = MessageHeader::try_read(&mut bytes[..]) { + // We expect that `bytes` contains an integral number of messages. + // No splitting occurs across allocations. + while bytes.len() > 0 { - // Get the header and payload, ditch the header. - let mut peel = bytes.extract_to(header.required_bytes()); - let _ = peel.extract_to(40); + if let Some(header) = MessageHeader::try_read(&mut bytes[..]) { - // 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()))); - } + // Get the header and payload, ditch the header. + let mut peel = bytes.extract_to(header.required_bytes()); + let _ = peel.extract_to(40); - // 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!"); + // 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!"); } } } diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index 696a23c90..9ffc80634 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -5,50 +5,185 @@ use super::bytes_slab::BytesSlab; /// A target for `Bytes`. pub trait BytesPush { - /// Pushes bytes at the instance. - fn push(&mut self, bytes: Bytes); + // /// 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; + // /// Pulls bytes from the instance. + // fn pull(&mut self) -> Option; + /// Drains many bytes from the instance. + fn drain_into(&mut self, vec: &mut Vec); } // std::sync::mpsc implementations. use ::std::sync::mpsc::{Sender, Receiver}; impl BytesPush for Sender { - fn push(&mut self, bytes: Bytes) { - self.send(bytes) - .expect("unable to send Bytes"); + fn extend>(&mut self, iterator: I) { + for bytes in iterator { + self.send(bytes) + .expect("unable to send Bytes"); + } } } impl BytesPull for Receiver { - fn pull(&mut self) -> Option { - self.try_recv() - .ok() + fn drain_into(&mut self, vec: &mut Vec) { + while let Some(item) = self.try_recv().ok() { + vec.push(item); + } } } // Arc>> implementations. -use ::std::sync::{Arc, Mutex}; +use ::std::sync::{Arc, Mutex, Condvar}; use ::std::collections::VecDeque; impl BytesPush for Arc>> { - fn push(&mut self, bytes: Bytes) { + fn extend>(&mut self, iter: I) { self.lock() .expect("unable to lock mutex") - .push_back(bytes); + .extend(iter); } } impl BytesPull for Arc>> { - fn pull(&mut self) -> Option { - self.lock() - .expect("unable to lock mutex") - .pop_front() + fn drain_into(&mut self, vec: &mut Vec) { + let mut queue = self.lock().expect("unable to lock mutex"); + vec.extend(queue.drain(..)); + } +} + +use std::sync::RwLock; +use std::thread::Thread; +/// 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(); + } + } +} + +// /// A signal which +// #[derive(Clone)] +// pub struct Signal { +// thing: Arc<(Mutex, Condvar)>, +// } + +// impl Signal { +// /// Allocates a new signal. +// pub fn new() -> Self { +// Signal { thing: Arc::new((Mutex::new(false), Condvar::new())) } +// } +// /// Blocks until the signal is set, and then unsets the signal. +// pub fn wait(&self) { +// let mut signaled = self.thing.0.lock().expect("Failed to lock mutex."); +// while !*signaled { +// signaled = self.thing.1.wait(signaled).expect("Failed to wait."); +// } +// *signaled = false; +// } +// /// Sets the signal, unblocking a waiting thread. +// pub fn ping(&self) { +// *self.thing.0.lock().expect("Failed to lock mutex.") = true; +// self.thing.1.notify_one(); +// } +// } + +/// Who knows. +#[derive(Clone)] +pub struct MergeQueue { + queue: Arc>>, // queue of bytes. + dirty: Signal, // indicates whether there may be data present. +} + +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, + } + } + /// Indicates that all input handles to the queue have dropped. + pub fn is_complete(&self) -> bool { + Arc::strong_count(&self.queue) == 1 + } +} + +impl BytesPush for MergeQueue { + fn extend>(&mut self, iterator: I) { + // 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) { + 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) { + // 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, @@ -59,10 +194,9 @@ 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.push(self.buffer.extract(valid_len)); + self.send.extend(Some(self.buffer.extract(valid_len))); } } @@ -78,7 +212,7 @@ impl SendEndpoint

{ /// 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(); + self.send_buffer(); } /// Acquires a prefix of `self.empty()` of length at least `capacity`. pub fn reserve(&mut self, capacity: usize) -> &mut [u8] { @@ -101,4 +235,5 @@ impl Drop for SendEndpoint

{ fn drop(&mut self) { self.send_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 index 79d805c18..b99605aa7 100644 --- a/communication/src/allocator/zero_copy/initialize.rs +++ b/communication/src/allocator/zero_copy/initialize.rs @@ -43,6 +43,7 @@ pub fn initialize_networking( 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(); @@ -52,7 +53,7 @@ pub fn initialize_networking( if let Some(stream) = results[index].take() { - let remote_recv = remote_recv_iter.next().unwrap(); + let (remote_recv, signal) = remote_recv_iter.next().unwrap(); { let log_sender = log_sender.clone(); @@ -68,15 +69,16 @@ pub fn initialize_networking( remote: Some(index), }); - let stream = ::std::io::BufWriter::with_capacity(1 << 20, stream); - send_loop(stream, remote_recv, log_sender); + 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 remote_sends = remote_sends.clone(); let log_sender = log_sender.clone(); let stream = stream.try_clone()?; let join_guard = @@ -88,7 +90,7 @@ pub fn initialize_networking( sender: false, remote: Some(index), }); - recv_loop(stream, remote_sends, threads * my_index, log_sender); + recv_loop(stream, remote_send, threads * my_index, log_sender); })?; recv_guards.push(join_guard); diff --git a/communication/src/allocator/zero_copy/tcp.rs b/communication/src/allocator/zero_copy/tcp.rs index 21a33af49..3d81036c2 100644 --- a/communication/src/allocator/zero_copy/tcp.rs +++ b/communication/src/allocator/zero_copy/tcp.rs @@ -1,14 +1,12 @@ //! use std::io::{Read, Write}; -use std::sync::mpsc::{Sender, Receiver}; use std::net::TcpStream; -use bytes::arc::Bytes; - use networking::MessageHeader; use super::bytes_slab::BytesSlab; +use super::bytes_exchange::{MergeQueue, Signal}; /// Repeatedly reads from a TcpStream and carves out messages. /// @@ -18,12 +16,18 @@ use super::bytes_slab::BytesSlab; /// take down the computation and cause the failures to cascade. pub fn recv_loop( mut reader: TcpStream, - targets: Vec>, + 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. @@ -59,9 +63,7 @@ pub fn recv_loop( let bytes = buffer.extract(peeled_bytes); if header.length > 0 { - targets[header.target - worker_offset] - .send(bytes) - .expect("Worker queue unavailable in recv_loop"); + stageds[header.target - worker_offset].push(bytes); } else { // Shutting down; confirm absence of subsequent data. @@ -75,6 +77,12 @@ pub fn recv_loop( } } } + + // 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"); } @@ -84,25 +92,37 @@ pub fn recv_loop( /// 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( - mut writer: ::std::io::BufWriter, - source: Receiver, + // 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 let Ok(bytes) = source.recv() { - stash.push(bytes); - while let Ok(bytes) = source.try_recv() { - stash.push(bytes); + while !sources.is_empty() { + + for source in sources.iter_mut() { + use allocator::zero_copy::bytes_exchange::BytesPull; + source.drain_into(&mut stash); } - // TODO: Could do scatter/gather write here. - for bytes in stash.drain(..) { - writer.write_all(&bytes[..]).expect("Write failure in send_loop."); + 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."); + } } - // TODO: Could delay this until a self.source.recv() would block. - writer.flush().expect("Failed to flush writer."); } // Write final zero-length header. From a002ffa560542355b49a7eaeeefeadd5771fd4b9 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Fri, 31 Aug 2018 11:12:06 +0200 Subject: [PATCH 41/43] drop in-progress on resize --- communication/src/allocator/zero_copy/bytes_slab.rs | 6 ++---- communication/src/allocator/zero_copy/tcp.rs | 1 + 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/communication/src/allocator/zero_copy/bytes_slab.rs b/communication/src/allocator/zero_copy/bytes_slab.rs index 2054f88c5..979f4c7d0 100644 --- a/communication/src/allocator/zero_copy/bytes_slab.rs +++ b/communication/src/allocator/zero_copy/bytes_slab.rs @@ -57,7 +57,8 @@ impl BytesSlab { // Increase allocation if copy would be insufficient. while self.valid + capacity > (1 << self.shift) { self.shift += 1; - self.stash.clear(); + self.stash.clear(); // clear wrongly sized buffers. + self.in_progress.clear(); // clear wrongly sized buffers. } // Attempt to reclaim shared slices. @@ -73,8 +74,6 @@ impl BytesSlab { } } self.in_progress.retain(|x| x.is_some()); - let self_shift = self.shift; - self.stash.retain(|x| x.len() == 1 << self_shift); } let new_buffer = self.stash.pop().unwrap_or_else(|| Bytes::from(vec![0; 1 << self.shift].into_boxed_slice())); @@ -82,7 +81,6 @@ impl BytesSlab { 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/tcp.rs b/communication/src/allocator/zero_copy/tcp.rs index 3d81036c2..25ea14eb3 100644 --- a/communication/src/allocator/zero_copy/tcp.rs +++ b/communication/src/allocator/zero_copy/tcp.rs @@ -104,6 +104,7 @@ pub fn send_loop( 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); From 88cfb72d62ea4a55abf6ea6d965ced543f640207 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 3 Sep 2018 10:52:03 +0200 Subject: [PATCH 42/43] panic propagation --- .../src/allocator/zero_copy/bytes_exchange.rs | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index 9ffc80634..f014761d9 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -118,11 +118,13 @@ impl Signal { // } // } +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 { @@ -131,16 +133,21 @@ impl MergeQueue { 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(); @@ -168,6 +175,7 @@ impl BytesPush for MergeQueue { 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(..)); } @@ -177,6 +185,14 @@ impl BytesPull for MergeQueue { // 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(); From 03b44d61c3fee57a5eca8a81c491b6467bd74c77 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 4 Sep 2018 14:17:48 +0200 Subject: [PATCH 43/43] public fns, process serialization --- communication/examples/hello.rs | 2 +- communication/src/allocator/process.rs | 7 +- communication/src/allocator/thread.rs | 7 +- .../src/allocator/zero_copy/allocator.rs | 4 +- .../allocator/zero_copy/allocator_process.rs | 49 +++++++------ .../src/allocator/zero_copy/bytes_exchange.rs | 69 ++----------------- communication/src/allocator/zero_copy/tcp.rs | 4 +- communication/src/initialize.rs | 67 ++++++++++++++++-- communication/src/lib.rs | 4 +- 9 files changed, 114 insertions(+), 99 deletions(-) diff --git a/communication/examples/hello.rs b/communication/examples/hello.rs index 67f495188..ef051142e 100644 --- a/communication/examples/hello.rs +++ b/communication/examples/hello.rs @@ -1,7 +1,7 @@ extern crate timely_communication; use std::ops::Deref; -use timely_communication::allocator::Message; +use timely_communication::Message; fn main() { diff --git a/communication/src/allocator/process.rs b/communication/src/allocator/process.rs index a9cadafea..2068edc1a 100644 --- a/communication/src/allocator/process.rs +++ b/communication/src/allocator/process.rs @@ -4,7 +4,7 @@ use std::sync::{Arc, Mutex}; use std::any::Any; use std::sync::mpsc::{Sender, Receiver, channel}; -use allocator::{Allocate, Message, Thread}; +use allocator::{Allocate, AllocateBuilder, Message, Thread}; use {Push, Pull}; /// An allocater for inter-thread, intra-process communication @@ -75,6 +75,11 @@ impl Allocate for Process { } } +impl AllocateBuilder for Process { + type Allocator = Self; + fn build(self) -> Self { self } +} + /// The push half of an intra-process channel. struct Pusher { target: Sender, diff --git a/communication/src/allocator/thread.rs b/communication/src/allocator/thread.rs index 8724efa4c..19f3b3db2 100644 --- a/communication/src/allocator/thread.rs +++ b/communication/src/allocator/thread.rs @@ -4,7 +4,7 @@ use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; -use allocator::{Allocate, Message}; +use allocator::{Allocate, AllocateBuilder, Message}; use {Push, Pull}; @@ -19,6 +19,11 @@ impl Allocate for Thread { } } +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) { diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index da4dd2bc6..5a427a0f1 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -88,7 +88,7 @@ impl TcpBuilder { index: self.index, peers: self.peers, allocated: 0, - signal: self.signal, + _signal: self.signal, staged: Vec::new(), sends, recvs: self.recvs, @@ -106,7 +106,7 @@ pub struct TcpAllocator { peers: usize, // number of peer allocators (for typed channel allocation). allocated: usize, // indicates how many channels have been allocated (locally). - signal: Signal, + _signal: Signal, staged: Vec, diff --git a/communication/src/allocator/zero_copy/allocator_process.rs b/communication/src/allocator/zero_copy/allocator_process.rs index 189a4733a..f7bc4ac81 100644 --- a/communication/src/allocator/zero_copy/allocator_process.rs +++ b/communication/src/allocator/zero_copy/allocator_process.rs @@ -3,7 +3,6 @@ use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; -use std::sync::{Arc, Mutex}; use bytes::arc::Bytes; @@ -12,7 +11,7 @@ use networking::MessageHeader; use {Allocate, Data, Push, Pull}; use allocator::Message; -use super::bytes_exchange::{BytesPull, SendEndpoint}; +use super::bytes_exchange::{BytesPull, SendEndpoint, MergeQueue, Signal}; use super::push_pull::{Pusher, Puller}; @@ -23,10 +22,11 @@ use super::push_pull::{Pusher, Puller}; /// 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. + 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 { @@ -35,6 +35,8 @@ impl ProcessBuilder { /// 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()); } @@ -42,24 +44,27 @@ impl ProcessBuilder { for source in 0 .. count { for target in 0 .. count { - let send = Arc::new(Mutex::new(VecDeque::new())); + let send = MergeQueue::new(signals[target].clone()); let recv = send.clone(); sends[source].push(send); recvs[target].push(recv); } } - let mut result = Vec::new(); - for (index, (sends, recvs)) in sends.drain(..).zip(recvs.drain(..)).enumerate() { - result.push(ProcessBuilder { - index, - peers: count, - sends, - recvs, - }) - } - - result + 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. @@ -80,6 +85,7 @@ impl ProcessBuilder { sends, recvs: self.recvs, to_local: Vec::new(), + _signal: self.signal, } } } @@ -91,11 +97,12 @@ pub struct ProcessAllocator { 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. + 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 { diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index f014761d9..8c6a6f1ed 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -1,5 +1,9 @@ //! 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; @@ -18,45 +22,6 @@ pub trait BytesPull { fn drain_into(&mut self, vec: &mut Vec); } -// std::sync::mpsc implementations. -use ::std::sync::mpsc::{Sender, Receiver}; -impl BytesPush for Sender { - fn extend>(&mut self, iterator: I) { - for bytes in iterator { - self.send(bytes) - .expect("unable to send Bytes"); - } - } -} - -impl BytesPull for Receiver { - fn drain_into(&mut self, vec: &mut Vec) { - while let Some(item) = self.try_recv().ok() { - vec.push(item); - } - } -} - -// Arc>> implementations. -use ::std::sync::{Arc, Mutex, Condvar}; -use ::std::collections::VecDeque; -impl BytesPush for Arc>> { - fn extend>(&mut self, iter: I) { - self.lock() - .expect("unable to lock mutex") - .extend(iter); - } -} - -impl BytesPull for Arc>> { - fn drain_into(&mut self, vec: &mut Vec) { - let mut queue = self.lock().expect("unable to lock mutex"); - vec.extend(queue.drain(..)); - } -} - -use std::sync::RwLock; -use std::thread::Thread; /// A signal appropriate to wake a single thread. /// /// Internally this type uses thread parking and unparking, where the first thread to call @@ -92,32 +57,6 @@ impl Signal { } } -// /// A signal which -// #[derive(Clone)] -// pub struct Signal { -// thing: Arc<(Mutex, Condvar)>, -// } - -// impl Signal { -// /// Allocates a new signal. -// pub fn new() -> Self { -// Signal { thing: Arc::new((Mutex::new(false), Condvar::new())) } -// } -// /// Blocks until the signal is set, and then unsets the signal. -// pub fn wait(&self) { -// let mut signaled = self.thing.0.lock().expect("Failed to lock mutex."); -// while !*signaled { -// signaled = self.thing.1.wait(signaled).expect("Failed to wait."); -// } -// *signaled = false; -// } -// /// Sets the signal, unblocking a waiting thread. -// pub fn ping(&self) { -// *self.thing.0.lock().expect("Failed to lock mutex.") = true; -// self.thing.1.notify_one(); -// } -// } - use std::sync::atomic::{AtomicBool, Ordering}; /// Who knows. #[derive(Clone)] diff --git a/communication/src/allocator/zero_copy/tcp.rs b/communication/src/allocator/zero_copy/tcp.rs index 25ea14eb3..e76e018ed 100644 --- a/communication/src/allocator/zero_copy/tcp.rs +++ b/communication/src/allocator/zero_copy/tcp.rs @@ -18,7 +18,7 @@ pub fn recv_loop( mut reader: TcpStream, mut targets: Vec, worker_offset: usize, - log_sender: ::logging::CommsLogger) + _log_sender: ::logging::CommsLogger) { let mut buffer = BytesSlab::new(20); @@ -96,7 +96,7 @@ pub fn send_loop( writer: TcpStream, mut sources: Vec, signal: Signal, - log_sender: ::logging::CommsLogger) + _log_sender: ::logging::CommsLogger) { let mut writer = ::std::io::BufWriter::with_capacity(1 << 16, writer); diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index 145b8c5a4..5e8fa9378 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -123,7 +123,7 @@ fn create_allocators(config: Configuration, logger: LogBuilder) -> Result<(VecT+Send+Sync+'static>( log_sender: LogBuilder, func: F, ) -> Result,String> { - let (allocators, others) = try!(create_allocators(config, log_sender)); - let logic = Arc::new(func); + initialize_from(allocators, others, func) +} +/// 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 (index, builder) in allocators.into_iter().enumerate() { + for (index, builder) in builders.into_iter().enumerate() { let clone = logic.clone(); guards.push(try!(thread::Builder::new() .name(format!("worker thread {}", index)) diff --git a/communication/src/lib.rs b/communication/src/lib.rs index 8b65959f6..8ef80d051 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -30,7 +30,7 @@ //! let (mut senders, mut receiver, _) = allocator.allocate(); //! //! // send typed data along each channel -//! use timely_communication::allocator::Message; +//! use timely_communication::Message; //! senders[0].send(Message::from_typed(format!("hello, {}", 0))); //! senders[1].send(Message::from_typed(format!("hello, {}", 1))); //! @@ -96,7 +96,7 @@ 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.

, 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 +30,24 @@ impl)>, H: Fn(&T, &D)->u64> Exchange)>, H: Fn(&T, &D)->u64> Push<(T, Content)> for Exchange { +impl>, H: Fn(&T, &D)->u64> Push> for Exchange { #[inline(never)] - fn push(&mut self, message: &mut Option<(T, Content)>) { + 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 e71c5e15e..682d038e9 100644 --- a/src/dataflow/channels/pushers/tee.rs +++ b/src/dataflow/channels/pushers/tee.rs @@ -3,25 +3,25 @@ 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; /// 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)>) { + fn push(&mut self, message: &mut Option>) { let mut pushers = self.shared.borrow_mut(); - if let Some((ref time, ref mut data)) = *message { + if let Some(message) = message { for index in 1..pushers.len() { - self.buffer.extend_from_slice(data); - Content::push_at(&mut self.buffer, (*time).clone(), &mut pushers[index-1]); + self.buffer.extend_from_slice(&message.data); + Message::push_at(&mut self.buffer, message.time.clone(), &mut pushers[index-1]); } } else { @@ -41,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(), }; @@ -60,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 af779dfc0..c9b4a89f3 100644 --- a/src/dataflow/operators/aggregation/aggregate.rs +++ b/src/dataflow/operators/aggregation/aggregate.rs @@ -80,13 +80,14 @@ impl Aggregate for >(&self, fold: F, emit: G, 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 ea2c67527..970328650 100644 --- a/src/dataflow/operators/aggregation/state_machine.rs +++ b/src/dataflow/operators/aggregation/state_machine.rs @@ -66,19 +66,24 @@ impl StateMachine f let mut pending = 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| { // 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/broadcast.rs b/src/dataflow/operators/broadcast.rs index f49cde1ac..f05c72dcd 100644 --- a/src/dataflow/operators/broadcast.rs +++ b/src/dataflow/operators/broadcast.rs @@ -12,7 +12,7 @@ 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::{Pusher, Puller}; /// Broadcast records to all workers. pub trait Broadcast { diff --git a/src/dataflow/operators/capture/capture.rs b/src/dataflow/operators/capture/capture.rs index 7bafbc3ec..667029ea6 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 timely_communication::allocator::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 a3cda2f9d..63bcfbe1e 100644 --- a/src/dataflow/operators/concat.rs +++ b/src/dataflow/operators/concat.rs @@ -26,9 +26,16 @@ pub trait Concat { impl Concat for Stream { fn concat(&self, other: &Stream) -> Stream { - self.binary_stream(other, Pipeline, Pipeline, "Concat", |input1, input2, output| { - input1.for_each(|time, data| { output.session(&time).give_content(data); }); - input2.for_each(|time, data| { output.session(&time).give_content(data); }); + let mut vector = Vec::new(); + self.binary_stream(other, Pipeline, Pipeline, "Concat", move |input1, input2, output| { + input1.for_each(|time, data| { + data.swap(&mut vector); + output.session(&time).give_vec(&mut vector); + }); + input2.for_each(|time, 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 a55d79a37..a88eba1cb 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 timely_communication::allocator::RefOrMut; + use Data; use dataflow::channels::pact::Pipeline; use dataflow::{Stream, Scope}; use dataflow::operators::generic::unary::Unary; -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)+'static>(&self, default: A, logic: F) -> Stream; + fn accumulate>)+'static>(&self, default: A, logic: F) -> Stream; /// Counts the number of records observed at each time. /// /// #Examples @@ -53,7 +53,7 @@ pub trait Accumulate { } impl Accumulate for Stream { - fn accumulate)+'static>(&self, default: A, logic: F) -> Stream { + fn accumulate>)+'static>(&self, default: A, logic: F) -> 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 0c80aff7d..bcad35682 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::unary::Unary; @@ -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 delayG::Timestamp+'static>(&self, func: F) -> 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_batchG::Timestamp+'static>(&self, func: F) -> 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..ee9d1048d 100644 --- a/src/dataflow/operators/enterleave.rs +++ b/src/dataflow/operators/enterleave.rs @@ -28,7 +28,7 @@ use progress::nested::subgraph::{Source, Target}; use progress::nested::product::Product; use {Data, 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 +136,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 +159,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 ae477ca4d..5424c09d8 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::unary::Unary; use progress::timestamp::Timestamp; @@ -23,36 +23,38 @@ pub trait Exchange { /// ``` fn exchangeu64+'static>(&self, route: F) -> 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_tsu64+'static>(&self, route: F) -> 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_tsu64+'static>(&self, route: F) -> Self; } impl, D: ExchangeData> Exchange for Stream { fn exchangeu64+'static>(&self, route: F) -> Stream { - self.unary_stream(ExchangePact::new(route), "Exchange", |input, output| { + let mut vector = Vec::new(); + self.unary_stream(ExchangePact::new(route), "Exchange", 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); }); }) } - fn exchange_tsu64+'static>(&self, route: F) -> Stream { - self.unary_stream(TimeExchangePact::new(route), "Exchange", |input, output| { - input.for_each(|time, data| { - output.session(&time).give_content(data); - }); - }) - } + // fn exchange_tsu64+'static>(&self, route: F) -> Stream { + // self.unary_stream(TimeExchangePact::new(route), "Exchange", |input, output| { + // input.for_each(|time, data| { + // output.session(&time).give_content(data); + // }); + // }) + // } } diff --git a/src/dataflow/operators/feedback.rs b/src/dataflow/operators/feedback.rs index 00634b237..3cbe7c89e 100644 --- a/src/dataflow/operators/feedback.rs +++ b/src/dataflow/operators/feedback.rs @@ -13,7 +13,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 +78,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 7c4416343..cc2cceb80 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 filterbool+'static>(&self, predicate: L) -> Stream { + let mut vector = Vec::new(); self.unary_stream(Pipeline, "Filter", 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 316f035e2..a28a5534e 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); /// }); /// }); /// }); @@ -61,13 +66,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..8270fb911 100644 --- a/src/dataflow/operators/generic/handles.rs +++ b/src/dataflow/operators/generic/handles.rs @@ -5,14 +5,17 @@ use std::rc::Rc; use std::cell::RefCell; +// use std::borrow::Cow; + +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 timely_communication::{Push, Pull, allocator::RefOrMut}; use logging::Logger; use dataflow::operators::CapabilityRef; @@ -20,30 +23,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 +70,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 +89,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 +102,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 +119,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 +136,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 +155,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 +179,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 +201,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 +211,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 dcb4b6d7d..9125615c2 100644 --- a/src/dataflow/operators/generic/operator.rs +++ b/src/dataflow/operators/generic/operator.rs @@ -36,12 +36,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,12 +81,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); /// } /// } /// }); @@ -114,13 +120,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| { @@ -170,15 +180,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 5a560e2ff..dfc5bd65f 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); /// }); /// }); /// }); @@ -50,10 +52,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 61ec0043d..1fffa05ff 100644 --- a/src/dataflow/operators/input.rs +++ b/src/dataflow/operators/input.rs @@ -13,7 +13,7 @@ use progress::nested::product::Product; use timely_communication::Allocate; use {Data, Push}; -use dataflow::channels::Content; +use dataflow::channels::{Bundle, Message}; use dataflow::channels::pushers::{Tee, Counter}; use dataflow::{Stream, Scope}; @@ -197,8 +197,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(), } } @@ -257,11 +257,11 @@ impl Handle { 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]); + 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]); + Message::push_at(&mut self.buffer1, self.now_at.clone(), &mut self.pushers[index]); debug_assert!(self.buffer1.is_empty()); } } @@ -282,7 +282,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.buffer1.capacity() == Content::::default_length()); + // assert!(self.buffer1.capacity() == Message::::default_length()); self.buffer1.push(data); if self.buffer1.len() == self.buffer1.capacity() { self.flush(); @@ -302,11 +302,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 0866a3e11..c6c674f95 100644 --- a/src/dataflow/operators/inspect.rs +++ b/src/dataflow/operators/inspect.rs @@ -36,19 +36,23 @@ pub trait Inspect { impl Inspect for Stream { fn inspect(&self, mut func: F) -> Stream { + let mut vector = Vec::new(); self.unary_stream(Pipeline, "Inspect", move |input, output| { input.for_each(|time, data| { - for datum in data.iter() { func(datum); } - output.session(&time).give_content(data); + data.swap(&mut vector); + for datum in vector.iter() { func(datum); } + output.session(&time).give_vec(&mut vector); }); }) } fn inspect_batch(&self, mut func: F) -> Stream { + let mut vector = Vec::new(); self.unary_stream(Pipeline, "InspectBatch", 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 e1982f975..cedc8650b 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 mapD2+'static>(&self, logic: L) -> Stream { + let mut vector = Vec::new(); self.unary_stream(Pipeline, "Map", 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: L) -> Stream { + let mut vector = Vec::new(); self.unary_stream(Pipeline, "MapInPlace", 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_mapI+'static>(&self, logic: L) -> Stream where I::Item: Data { + let mut vector = Vec::new(); self.unary_stream(Pipeline, "FlatMap", 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/mod.rs b/src/dataflow/operators/mod.rs index b87e857e8..3b2e79c86 100644 --- a/src/dataflow/operators/mod.rs +++ b/src/dataflow/operators/mod.rs @@ -20,7 +20,7 @@ pub use self::inspect::Inspect; pub use self::filter::Filter; pub use self::delay::Delay; pub use self::exchange::Exchange; -pub use self::broadcast::Broadcast; +// pub use self::broadcast::Broadcast; pub use self::probe::Probe; pub use self::to_stream::ToStream; pub use self::capture::Capture; @@ -42,7 +42,7 @@ pub mod inspect; pub mod filter; pub mod delay; pub mod exchange; -pub mod broadcast; +// pub mod broadcast; pub mod probe; pub mod to_stream; pub mod capture; 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..1df70a077 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 timely_communication::allocator::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(); diff --git a/src/dataflow/operators/reclock.rs b/src/dataflow/operators/reclock.rs index 1e12d32ac..0c199b966 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..4d21a1535 100644 --- a/src/dataflow/scopes/child.rs +++ b/src/dataflow/scopes/child.rs @@ -87,10 +87,12 @@ impl<'a, G: ScopeParent, T: Timestamp> Scope for Child<'a, G, T> { } } +use timely_communication::allocator::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/root.rs b/src/dataflow/scopes/root.rs index 907dcb77b..292039e1d 100644 --- a/src/dataflow/scopes/root.rs +++ b/src/dataflow/scopes/root.rs @@ -132,10 +132,12 @@ impl ScopeParent for Root { } } +use timely_communication::allocator::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..8d10ef4f2 100644 --- a/src/dataflow/stream.rs +++ b/src/dataflow/stream.rs @@ -9,7 +9,7 @@ use progress::nested::subgraph::{Source, Target}; use 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/progress/broadcast.rs b/src/progress/broadcast.rs index 69ad124a9..12d8e1192 100644 --- a/src/progress/broadcast.rs +++ b/src/progress/broadcast.rs @@ -1,19 +1,24 @@ //! Broadcasts progress information among workers. +use timely_communication::allocator::Message; + use progress::Timestamp; use progress::ChangeBatch; use timely_communication::Allocate; use {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 +43,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 +76,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 +107,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 From 8030b457a171a204cac8937f2293af70401fb640 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 28 May 2018 14:56:16 +0200 Subject: [PATCH 12/43] use Message::push_at --- src/dataflow/channels/pushers/buffer.rs | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/src/dataflow/channels/pushers/buffer.rs b/src/dataflow/channels/pushers/buffer.rs index d96051a05..75ec69106 100644 --- a/src/dataflow/channels/pushers/buffer.rs +++ b/src/dataflow/channels/pushers/buffer.rs @@ -79,16 +79,7 @@ impl>> Buffer where T: Eq+Clone { } let time = self.time.as_ref().expect("Buffer::give_vec(): time is None.").clone(); - let data = ::std::mem::replace(vector, Vec::new()); - - let mut bundle = Some(Bundle::from_typed(Message::new(time, data, 0, 0))); - - self.pusher.push(&mut bundle); - if let Some(message) = bundle { - if let Some(message) = message.if_typed() { - *vector = message.data; - } - } + Message::push_at(vector, time, &mut self.pusher); } } From 0a9c7d6e55ce660d8467680a9d5b03911a8ebfd8 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 29 May 2018 09:45:24 +0200 Subject: [PATCH 13/43] no clue --- communication/src/allocator/binary.rs | 5 ++- communication/src/allocator/mod.rs | 35 +++++++++++++------ communication/src/allocator/process_binary.rs | 2 +- src/dataflow/operators/input.rs | 2 +- 4 files changed, 29 insertions(+), 15 deletions(-) diff --git a/communication/src/allocator/binary.rs b/communication/src/allocator/binary.rs index 5edcb8cd1..52bb5062b 100644 --- a/communication/src/allocator/binary.rs +++ b/communication/src/allocator/binary.rs @@ -154,15 +154,14 @@ impl Pull> for Puller { }))); let bytes = ::bytes::arc::Bytes::from(bytes); - // let result = ::from_bytes(bytes); + log_sender.when_enabled(|l| l.log( ::logging::CommsEvent::Serialization(::logging::SerializationEvent { seq_no: None, is_start: false, }))); - // Message::Typed(result) - Message::from_bytes(bytes) + unsafe { Message::from_bytes(bytes) } }); &mut self.current } diff --git a/communication/src/allocator/mod.rs b/communication/src/allocator/mod.rs index 5f3ab194e..e575a31a5 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -24,8 +24,11 @@ enum TypedOrBinary { 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), } @@ -58,20 +61,24 @@ impl<'a, T: Clone+'a> RefOrMut<'a, T> { } } +/// 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, @@ -81,15 +88,23 @@ impl Message { } impl Message { - pub fn from_bytes(bytes: Bytes) -> Self { - - unsafe { - let abomonated = Abomonated::new(bytes).expect("Abomonated::new() failed."); - Message { payload: TypedOrBinary::Binary(abomonated) } - } - + /// 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) }, @@ -97,12 +112,14 @@ impl Message { } } + /// The number of bytes required to serialize the data. 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`. fn into_bytes(&self, writer: &mut W) { match &self.payload { TypedOrBinary::Binary(bytes) => { @@ -134,7 +151,7 @@ impl Message { 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 { @@ -162,9 +179,7 @@ pub trait Allocate { /// 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); - fn pre_work(&mut self) { } fn post_work(&mut self) { } } diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index 0a7c39db2..5e4d7d565 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -390,7 +390,7 @@ impl Pull> for Puller { self.receiver .borrow_mut() .pop_front() - .map(|bytes| Message::from_bytes(bytes)); + .map(|bytes| unsafe { Message::from_bytes(bytes) }); &mut self.current } diff --git a/src/dataflow/operators/input.rs b/src/dataflow/operators/input.rs index 1fffa05ff..db8e70c56 100644 --- a/src/dataflow/operators/input.rs +++ b/src/dataflow/operators/input.rs @@ -13,7 +13,7 @@ use progress::nested::product::Product; use timely_communication::Allocate; use {Data, Push}; -use dataflow::channels::{Bundle, Message}; +use dataflow::channels::Message; use dataflow::channels::pushers::{Tee, Counter}; use dataflow::{Stream, Scope}; From b314103d3d5fba957d5f5aa92d8122ee4445ae76 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 12 Jun 2018 14:54:03 +0200 Subject: [PATCH 14/43] merge, examples fixed --- examples/bfs.rs | 2 +- examples/unionfind.rs | 47 ++++++++++++---------- src/dataflow/operators/branch.rs | 11 +++-- src/dataflow/operators/generic/operator.rs | 17 +++++--- tests/barrier.rs | 2 +- 5 files changed, 46 insertions(+), 33 deletions(-) diff --git a/examples/bfs.rs b/examples/bfs.rs index 7dab4cc86..f993f9bd1 100644 --- a/examples/bfs.rs +++ b/examples/bfs.rs @@ -10,7 +10,7 @@ use timely_sort::{RadixSorter, RadixSorterBase}; use timely_sort::LSBRadixSorter as Sorter; use timely::dataflow::operators::{ToStream, Concat, LoopVariable, ConnectLoop}; -use timely::dataflow::operators::generic::binary::Binary; +use timely::dataflow::operators::generic::operator::Binary; use timely::dataflow::channels::pact::Exchange; fn main() { diff --git a/examples/unionfind.rs b/examples/unionfind.rs index ee4ce4ae0..9ab1b571f 100644 --- a/examples/unionfind.rs +++ b/examples/unionfind.rs @@ -9,7 +9,7 @@ use rand::{Rng, SeedableRng, StdRng}; use timely::dataflow::*; use timely::dataflow::operators::{Input, Exchange, Probe}; -use timely::dataflow::operators::generic::unary::Unary; +use timely::dataflow::operators::generic::operators::Operator; use timely::dataflow::channels::pact::Pipeline; fn main() { @@ -60,33 +60,36 @@ trait UnionFind { impl UnionFind for Stream { fn union_find(&self) -> Stream { - let mut roots = vec![]; // u32 works, and is smaller than uint/u64 - let mut ranks = vec![]; // u8 should be large enough (n < 2^256) + self.unary(Pipeline, "UnionFind", |_,_| { - self.unary_stream(Pipeline, "UnionFind", move |input, output| { + let mut roots = vec![]; // u32 works, and is smaller than uint/u64 + let mut ranks = vec![]; // u8 should be large enough (n < 2^256) - while let Some((time, data)) = input.next() { + move |input, output| { - let mut session = output.session(&time); - for &(mut x, mut y) in data.iter() { + while let Some((time, data)) = input.next() { - // grow arrays if required. - let m = ::std::cmp::max(x, y); - for i in roots.len() .. (m + 1) { - roots.push(i); - ranks.push(0); - } + let mut session = output.session(&time); + for &(mut x, mut y) in data.iter() { - // look up roots for `x` and `y`. - while x != roots[x] { x = roots[x]; } - while y != roots[y] { y = roots[y]; } + // grow arrays if required. + let m = ::std::cmp::max(x, y); + for i in roots.len() .. (m + 1) { + roots.push(i); + ranks.push(0); + } - if x != y { - session.give((x, y)); - match ranks[x].cmp(&ranks[y]) { - Ordering::Less => { roots[x] = y }, - Ordering::Greater => { roots[y] = x }, - Ordering::Equal => { roots[y] = x; ranks[x] += 1 }, + // look up roots for `x` and `y`. + while x != roots[x] { x = roots[x]; } + while y != roots[y] { y = roots[y]; } + + if x != y { + session.give((x, y)); + match ranks[x].cmp(&ranks[y]) { + Ordering::Less => { roots[x] = y }, + Ordering::Greater => { roots[y] = x }, + Ordering::Equal => { roots[y] = x; ranks[x] += 1 }, + } } } } 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/generic/operator.rs b/src/dataflow/operators/generic/operator.rs index 218f65908..b10a66ad8 100644 --- a/src/dataflow/operators/generic/operator.rs +++ b/src/dataflow/operators/generic/operator.rs @@ -81,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| { @@ -215,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| { diff --git a/tests/barrier.rs b/tests/barrier.rs index 209abf92c..954e6b630 100644 --- a/tests/barrier.rs +++ b/tests/barrier.rs @@ -5,7 +5,7 @@ use timely::dataflow::channels::pact::Pipeline; use timely::progress::timestamp::RootTimestamp; use timely::dataflow::operators::{LoopVariable, ConnectLoop}; -use timely::dataflow::operators::generic::unary::Unary; +use timely::dataflow::operators::generic::operator::Operator; use timely_communication::Configuration; From 8875539a0b1094f61367e76ee246abf4df224e28 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 26 Jun 2018 13:55:54 +0200 Subject: [PATCH 15/43] re-enable logging --- src/dataflow/channels/pact.rs | 208 ++++++++++++++++------------------ 1 file changed, 99 insertions(+), 109 deletions(-) diff --git a/src/dataflow/channels/pact.rs b/src/dataflow/channels/pact.rs index 63c1d5915..46a7b3929 100644 --- a/src/dataflow/channels/pact.rs +++ b/src/dataflow/channels/pact.rs @@ -13,7 +13,6 @@ 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 timely_communication::CommMessage; use dataflow::channels::pushers::Exchange as ExchangePusher; use super::{Bundle, Message}; @@ -35,14 +34,13 @@ pub trait ParallelizationContract { /// A direct connection pub struct Pipeline; impl ParallelizationContract for Pipeline { - type Pusher = ThreadPusher>>; - type Puller = ThreadPuller>>; + 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())) - (pusher, puller) + (LogPusher::new(pusher, allocator.index(), allocator.index(), identifier, None, logging.clone()), + LogPuller::new(puller, allocator.index(), identifier, None, logging.clone())) } } @@ -62,13 +60,12 @@ 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 = Box>>>; + 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())) - (Box::new(ExchangePusher::new(senders, move |_, d| (self.hash_func)(d))), receiver) + 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()))) } } @@ -95,110 +92,103 @@ implu64+'st // } -// /// Wraps a `Message` pusher to provide a `Push<(T, Content)>`. -// pub struct Pusher>> { -// pusher: P, -// channel: usize, -// comm_channel: Option, -// counter: usize, -// source: usize, -// target: usize, -// phantom: ::std::marker::PhantomData<(T, D)>, -// logging: Logger, -// } -// impl>> Pusher { -// /// Allocates a new pusher. -// pub fn new(pusher: P, source: usize, target: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { -// Pusher { -// pusher, -// channel, -// comm_channel, -// counter: 0, -// source, -// target, -// phantom: ::std::marker::PhantomData, -// logging, -// } -// } -// } +/// Wraps a `Message` pusher to provide a `Push<(T, Content)>`. +pub struct LogPusher>> { + pusher: P, + channel: usize, + comm_channel: Option, + counter: usize, + source: usize, + target: usize, + phantom: ::std::marker::PhantomData<(T, D)>, + logging: Logger, +} +impl>> LogPusher { + /// Allocates a new pusher. + pub fn new(pusher: P, source: usize, target: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { + LogPusher { + pusher, + channel, + comm_channel, + counter: 0, + source, + target, + phantom: ::std::marker::PhantomData, + logging, + } + } +} -// impl>> Push<(T, Content)> for Pusher { -// #[inline(always)] -// fn push(&mut self, pair: &mut Option<(T, Content)>) { -// if let Some((time, data)) = pair.take() { +impl>> Push> for LogPusher { + #[inline(always)] + fn push(&mut self, pair: &mut Option>) { + + if let Some(bundle) = pair { + let length = bundle.data.len(); + let counter = self.counter; + self.counter += 1; + + self.logging.when_enabled(|l| l.log(::logging::TimelyEvent::Messages(::logging::MessagesEvent { + is_send: true, + channel: self.channel, + comm_channel: self.comm_channel, + source: self.source, + target: self.target, + seq_no: counter, + length, + }))); + } -// let length = data.len(); + self.pusher.push(pair); + } +} -// let counter = self.counter; +/// Wraps a `Message` puller to provide a `Pull<(T, Content)>`. +pub struct LogPuller>> { + puller: P, + channel: usize, + comm_channel: Option, + index: usize, + phantom: ::std::marker::PhantomData<(T, D)>, + logging: Logger, +} +impl>> LogPuller { + /// Allocates a new `Puller`. + pub fn new(puller: P, index: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { + LogPuller { + puller, + channel, + comm_channel, + index, + phantom: ::std::marker::PhantomData, + logging, + } + } +} -// 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)); +impl>> Pull> for LogPuller { + #[inline(always)] + fn pull(&mut self) -> &mut Option> { -// self.logging.when_enabled(|l| l.log(::logging::TimelyEvent::Messages(::logging::MessagesEvent { -// is_send: true, -// channel: self.channel, -// comm_channel: self.comm_channel, -// source: self.source, -// target: self.target, -// seq_no: counter, -// length, -// }))); + let result = self.puller.pull(); -// // Log something about (index, counter, time?, length?); -// } -// else { self.pusher.done(); } -// } -// } + if let Some(bundle) = result { -// /// Wraps a `Message` puller to provide a `Pull<(T, Content)>`. -// pub struct Puller>> { -// puller: P, -// current: Option>, -// channel: usize, -// comm_channel: Option, -// counter: usize, -// index: usize, -// logging: Logger, -// } -// impl>> Puller { -// /// Allocates a new `Puller`. -// pub fn new(puller: P, index: usize, channel: usize, comm_channel: Option, logging: Logger) -> Self { -// Puller { -// puller, -// channel, -// comm_channel, -// current: None, -// counter: 0, -// index, -// logging, -// } -// } -// } + let channel = self.channel; + let comm_channel = self.comm_channel; + let target = self.index; -// impl>> Pull<(T, Content)> for Puller { -// #[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; - -// ::std::mem::swap(&mut previous, self.puller.pull()); - -// if let Some(message) = previous.as_ref() { - -// 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(), -// }))); -// } + self.logging.when_enabled(|l| l.log(::logging::TimelyEvent::Messages(::logging::MessagesEvent { + is_send: false, + 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 + } +} From e832f76fb035c100e8777562a895d015f0cdf056 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 26 Jun 2018 15:14:09 +0200 Subject: [PATCH 16/43] reintroduce broadcast --- src/dataflow/operators/broadcast.rs | 26 +++++++++++++++++++------- src/dataflow/operators/mod.rs | 4 ++-- 2 files changed, 21 insertions(+), 9 deletions(-) diff --git a/src/dataflow/operators/broadcast.rs b/src/dataflow/operators/broadcast.rs index f05c72dcd..7f14de681 100644 --- a/src/dataflow/operators/broadcast.rs +++ b/src/dataflow/operators/broadcast.rs @@ -7,12 +7,12 @@ 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 timely_communication::allocator::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/mod.rs b/src/dataflow/operators/mod.rs index 69ac906bf..bde9a9e80 100644 --- a/src/dataflow/operators/mod.rs +++ b/src/dataflow/operators/mod.rs @@ -20,7 +20,7 @@ pub use self::inspect::Inspect; pub use self::filter::Filter; pub use self::delay::Delay; pub use self::exchange::Exchange; -// pub use self::broadcast::Broadcast; +pub use self::broadcast::Broadcast; pub use self::probe::Probe; pub use self::to_stream::ToStream; pub use self::capture::Capture; @@ -44,7 +44,7 @@ pub mod inspect; pub mod filter; pub mod delay; pub mod exchange; -// pub mod broadcast; +pub mod broadcast; pub mod probe; pub mod to_stream; pub mod capture; From 42c47e95dd27d0aabc5551b27877bea449897bed Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Wed, 27 Jun 2018 12:56:56 +0200 Subject: [PATCH 17/43] push more --- communication/src/allocator/process_binary.rs | 39 ++++++++++++++----- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index 5e4d7d565..db59e1cc9 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -31,12 +31,16 @@ pub trait RecvEndpoint { pub mod vec { + use std::sync::{Arc, Mutex}; use std::sync::mpsc::{Sender, Receiver, channel}; + use std::collections::VecDeque; use super::{BytesExchange, SendEndpoint, RecvEndpoint}; pub struct VecSendEndpoint { - send: Sender>, // send full vectors + // in_flight: Arc, + // send: Sender>, // send full vectors + send: Arc>>>, recv: Receiver>, // recv empty vectors balance: usize, // #sent - #recv. @@ -62,8 +66,14 @@ pub mod vec { let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); if buffer.len() > 0 { - self.send.send(buffer).expect("VecSendEndpoint::send_buffer(): failed to send buffer"); - self.balance += 1; + if let Ok(mut lock) = self.send.lock() { + lock.push_back(buffer); + self.balance += 1; + } + else { + panic!("unable to acquire lock"); + } + // self.send.send(buffer).expect("VecSendEndpoint::send_buffer(): failed to send buffer"); } else { if buffer.capacity() == self.default_size { @@ -97,24 +107,32 @@ pub mod vec { fn publish(&mut self) { self.drain_recv(); - if self.balance == 0 { + if self.send.lock().map(|queue| queue.is_empty()).expect("Failed to lock mutex") { + // if self.balance == 0 { self.send_buffer(); } } } pub struct VecRecvEndpoint { - recv: Receiver>, // recv full vectors + recv: Arc>>>, + // recv: Receiver>, // recv full vectors send: Sender>, // send empty vectors } impl RecvEndpoint for VecRecvEndpoint { type RecvBuffer = Vec; fn receive(&mut self) -> Option { - if let Ok(bytes) = self.recv.try_recv() { - Some(bytes) + if let Ok(mut lock) = self.recv.lock() { + lock.pop_front() } - else { None } + else { + panic!("Failed to lock mutex"); + } + // if let Ok(bytes) = self.recv.try_recv() { + // Some(bytes) + // } + // else { None } } fn recycle(&mut self, mut buffer: Self::RecvBuffer) { buffer.clear(); @@ -130,7 +148,10 @@ pub mod vec { type Recv = VecRecvEndpoint; fn new() -> (Self::Send, Self::Recv) { - let (send1, recv1) = channel(); + let send1 = Arc::new(Mutex::new(VecDeque::new())); + let recv1 = send1.clone(); + + // let (send1, recv1) = channel(); let (send2, recv2) = channel(); let result1 = VecSendEndpoint { From 1e3173878180ad096d696ba79b63cc4029436ad8 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 28 Jun 2018 11:06:05 +0200 Subject: [PATCH 18/43] re-enable execution filtering --- src/progress/nested/subgraph.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/progress/nested/subgraph.rs b/src/progress/nested/subgraph.rs index 6b4ede36f..eef329ae4 100644 --- a/src/progress/nested/subgraph.rs +++ b/src/progress/nested/subgraph.rs @@ -300,7 +300,7 @@ impl Operate for Subgraph>>, frontier: &mut [ChangeBatch]) { + fn set_external_summary(&mut self, _summaries: Vec>>, frontier: &mut [ChangeBatch]) { // We must first translate `summaries` to summaries in the subgraph's timestamp type. // Each of these summaries correspond to dropping the inner timestamp coordinate and replacing @@ -773,7 +773,7 @@ impl PerOperatorState { // 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 + if any_progress_updates || _was_recently_active || _outstanding_messages || _held_capabilities { let self_id = self.id; // avoid capturing `self` in logging closures. @@ -843,12 +843,12 @@ impl PerOperatorState { internal_activity } - // else { - // // Active operators should always be scheduled, and should re-assert their activity if - // // they want to be scheduled again. If we are here, it is because the operator declined - // // to express activity explicitly. - // false - // } + else { + // Active operators should always be scheduled, and should re-assert their activity if + // they want to be scheduled again. If we are here, it is because the operator declined + // to express activity explicitly. + false + } } else { From 0ccc5f7854f2072edea227ff0458d6b8bc02d129 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 28 Jun 2018 15:04:53 +0200 Subject: [PATCH 19/43] re-introduce thread process --- communication/src/allocator/process.rs | 29 +++++++++---------- communication/src/allocator/process_binary.rs | 7 ++--- communication/src/initialize.rs | 8 ++--- 3 files changed, 21 insertions(+), 23 deletions(-) diff --git a/communication/src/allocator/process.rs b/communication/src/allocator/process.rs index 0a92ee29b..7468d089b 100644 --- a/communication/src/allocator/process.rs +++ b/communication/src/allocator/process.rs @@ -41,7 +41,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,21 +54,20 @@ 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) } } diff --git a/communication/src/allocator/process_binary.rs b/communication/src/allocator/process_binary.rs index db59e1cc9..c0aa3018d 100644 --- a/communication/src/allocator/process_binary.rs +++ b/communication/src/allocator/process_binary.rs @@ -269,7 +269,7 @@ impl Allocate for ProcessBinary { self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); } - let puller = Box::new(Puller::new(self.to_local[channel_id].clone(), channel_id)); + let puller = Box::new(Puller::new(self.to_local[channel_id].clone())); (pushes, puller, None) } @@ -393,13 +393,12 @@ impl Push> for Pusher { /// like the `bytes` crate (../bytes/) which provides an exclusive view of a shared /// allocation. struct Puller { - channel: usize, current: Option>, receiver: Rc>>, // source of serialized buffers } impl Puller { - fn new(receiver: Rc>>, channel: usize) -> Puller { - Puller { channel, current: None, receiver } + fn new(receiver: Rc>>) -> Puller { + Puller { current: None, receiver } } } diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index 4e361e31b..a0d6e60bd 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -79,12 +79,12 @@ type LogBuilder = Arc::logging::CommsLogger+Send+Syn fn create_allocators(config: Configuration, logger: LogBuilder) -> Result,String> { match config { Configuration::Thread => { - // Ok(vec![GenericBuilder::Thread(Thread)]) - Ok(ProcessBinaryBuilder::new_vector(1).into_iter().map(|x| GenericBuilder::ProcessBinary(x)).collect()) + Ok(vec![GenericBuilder::Thread(Thread)]) + // 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()) - Ok(ProcessBinaryBuilder::new_vector(threads).into_iter().map(|x| GenericBuilder::ProcessBinary(x)).collect()) + Ok(Process::new_vector(threads).into_iter().map(|x| GenericBuilder::Process(x)).collect()) + // Ok(ProcessBinaryBuilder::new_vector(threads).into_iter().map(|x| GenericBuilder::ProcessBinary(x)).collect()) }, Configuration::Cluster(threads, process, addresses, report) => { if let Ok(stuff) = initialize_networking(addresses, process, threads, report, logger) { From 667bcc67bd390a6da7c23f630eaa910d5ac8bffe Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 26 Jul 2018 20:52:31 +0200 Subject: [PATCH 20/43] new tcp beginnings --- communication/src/allocator/mod.rs | 1 + .../src/allocator/process_binary_neu.rs | 717 ++++++++++++++++++ 2 files changed, 718 insertions(+) create mode 100644 communication/src/allocator/process_binary_neu.rs diff --git a/communication/src/allocator/mod.rs b/communication/src/allocator/mod.rs index e575a31a5..40690e49f 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -13,6 +13,7 @@ pub mod process; pub mod binary; pub mod generic; pub mod process_binary; +pub mod process_binary_neu; use {Data, Push, Pull}; diff --git a/communication/src/allocator/process_binary_neu.rs b/communication/src/allocator/process_binary_neu.rs new file mode 100644 index 000000000..c23addfe5 --- /dev/null +++ b/communication/src/allocator/process_binary_neu.rs @@ -0,0 +1,717 @@ +use std::rc::Rc; +use std::cell::RefCell; +use std::collections::VecDeque; +use std::ops::DerefMut; + +use bytes::arc::Bytes; + +use networking::MessageHeader; + +use {Allocate, Data, Push, Pull}; +use allocator::{Message, Process}; + +use std::sync::{Arc, Mutex}; + +pub struct SharedQueue { + queue: Arc>> +} + +impl SharedQueue { + pub fn push(&mut self, bytes: T) { self.queue.lock().expect("unable to lock shared queue").push_back(bytes) } + pub fn pop(&mut self) -> Option { self.queue.lock().expect("unable to lock shared queue").pop_front() } + pub fn is_empty(&self) -> bool { self.queue.lock().expect("unable to lock shared queue").is_empty() } + pub fn is_done(&self) -> bool { Arc::strong_count(&self.queue) == 1 } + pub fn new() -> Self { SharedQueue { queue: Arc::new(Mutex::new(VecDeque::new())) } } +} + +impl Clone for SharedQueue { + fn clone(&self) -> Self { + SharedQueue { queue: self.queue.clone() } + } +} + +/// A type that can allocate send and receive endpoints for byte exchanges. +/// +/// The `BytesExchange` intent is that one can abstractly define mechanisms for exchanging +/// bytes between various entities. In some cases this may be between worker threads within +/// a process, in other cases it may be between worker threads and remote processes. At the +/// moment the cardinalities of remote endpoints requires some context and interpretation. +pub trait BytesExchange { + /// The type of the send endpoint. + type Send: SendEndpoint+'static; + /// The type of the receive endpoint. + type Recv: RecvEndpoint+'static; + /// Allocates endpoint pairs for a specified worker. + /// + /// Importantly, the Send side may share state to coalesce the buffering and + /// transmission of records. That is why there are `Rc>` things there. + fn new(&mut self, worker: usize) -> (Vec, Vec); +} + +/// A type that can provide and publish writeable binary buffers. +pub trait SendEndpoint { + /// The type of the writeable binary buffer. + type SendBuffer: ::std::io::Write; + /// Provides a writeable buffer of the requested capacity. + fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; + /// Indicates that it is now appropriate to publish the buffer. + fn publish(&mut self); +} + +/// A type that can provide readable binary buffers. +pub trait RecvEndpoint { + type RecvBuffer: DerefMut; + /// Provides a readable buffer. + fn receive(&mut self) -> Option; +} + +pub mod common { + + use bytes::arc::Bytes; + use super::{SendEndpoint, RecvEndpoint, SharedQueue}; + + pub struct VecSendEndpoint { + send: SharedQueue, + in_progress: Vec>, + buffer: Vec, + stash: Vec>, + default_size: usize, + } + + impl VecSendEndpoint { + /// Attempts to recover in-use buffers once uniquely owned. + fn harvest_shared(&mut self) { + for shared in self.in_progress.iter_mut() { + if let Some(bytes) = shared.take() { + match bytes.try_recover::>() { + Ok(vec) => { self.stash.push(vec); }, + Err(bytes) => { *shared = Some(bytes); }, + } + } + } + self.in_progress.retain(|x| x.is_some()); + } + + /// Moves `self.buffer` into `self.send`, replaces with empty buffer. + fn send_buffer(&mut self) { + + let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); + let buffer_len = buffer.len(); + if buffer_len > 0 { + + let mut bytes = Bytes::from(buffer); + let to_send = bytes.extract_to(buffer_len); + + self.send.push(to_send); + self.in_progress.push(Some(bytes)); + } + else { + if buffer.capacity() == self.default_size { + self.stash.push(buffer); + } + } + } + + /// Allocates a new `VecSendEndpoint` from a shared queue. + pub fn new(queue: SharedQueue) -> Self { + VecSendEndpoint { + send: queue, + in_progress: Vec::new(), + buffer: Vec::new(), + stash: Vec::new(), + default_size: 1 << 20, + } + } + } + + impl SendEndpoint for VecSendEndpoint { + + type SendBuffer = Vec; + + fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { + + // If we don't have enough capacity in `self.buffer`... + if self.buffer.capacity() < capacity + self.buffer.len() { + self.send_buffer(); + if capacity > self.default_size { + self.buffer = Vec::with_capacity(capacity); + } + else { + if self.stash.is_empty() { + // Attempt to recover shared buffers. + self.harvest_shared(); + } + self.buffer = self.stash.pop().unwrap_or_else(|| Vec::with_capacity(self.default_size)) + } + } + + &mut self.buffer + } + + fn publish(&mut self) { + self.harvest_shared(); + if self.send.is_empty() { + self.send_buffer(); + } + } + } + + pub struct VecRecvEndpoint { + recv: SharedQueue, + } + + + impl VecRecvEndpoint { + pub fn new(queue: SharedQueue) -> Self { + VecRecvEndpoint { recv: queue } + } + } + + impl RecvEndpoint for VecRecvEndpoint { + type RecvBuffer = Bytes; + fn receive(&mut self) -> Option { + self.recv.pop() + } + } +} + +/// Byte exchange mechanisms which use shared memory queues. +pub mod local { + + use bytes::arc::Bytes; + + use super::{BytesExchange, SharedQueue}; + use super::common::{VecSendEndpoint, VecRecvEndpoint}; + + pub struct LocalBytesExchange { + // forward[i][j] contains a shared queue for data from i to j. + forward: Vec>>, + counter: usize, + } + + impl LocalBytesExchange { + fn new(workers: usize) -> LocalBytesExchange { + + let mut forward = Vec::new(); + for _source in 0 .. workers { + let mut temp_forward = Vec::new(); + for _target in 0 .. workers { + temp_forward.push(SharedQueue::new()); + } + forward.push(temp_forward); + } + + LocalBytesExchange { + forward, + counter: 0, + } + } + } + + impl BytesExchange for LocalBytesExchange { + type Send = VecSendEndpoint; + type Recv = VecRecvEndpoint; + fn new(&mut self, worker: usize) -> (Vec, Vec) { + + let mut sends = Vec::with_capacity(self.forward.len()); + + for forward in self.forward[self.counter].iter() { + sends.push(VecSendEndpoint::new(forward.clone())); + } + + let mut recvs = Vec::with_capacity(self.forward.len()); + + for forward in self.forward.iter() { + recvs.push(VecRecvEndpoint::new(forward[self.counter].clone())); + } + + self.counter += 1; + + (sends, recvs) + } + } +} + +/// A BytesExchange implementation using communication threads and TCP connections. +pub mod tcp { + + use std::io::{Read, Write}; + + use bytes::arc::Bytes; + + use networking::MessageHeader; + use super::{BytesExchange, SharedQueue}; + use super::common::{VecSendEndpoint, VecRecvEndpoint}; + + /// Allocates pairs of byte exchanges for remote workers. + pub struct TcpBytesExchange { + /// Forward[i,j]: from worker i to process j. + forward: Vec>>, + /// Reverse[i,j]: to process i from worker j. + reverse: Vec>>, + counter: usize, + } + + impl BytesExchange for TcpBytesExchange { + + type Send = VecSendEndpoint; + type Recv = VecRecvEndpoint; + + // Returns two vectors of length #processes - 1. + // The first contains destinations to send to remote processes, + // The second contains sources to receive from remote processes. + fn new(&mut self, worker: usize) -> (Vec, Vec) { + + let mut sends = Vec::with_capacity(self.forward.len()); + for queue in self.forward[self.counter].iter() { + sends.push(VecSendEndpoint::new(queue.clone())); + } + + let mut recvs = Vec::with_capacity(self.forward.len()); + for queue in self.reverse[self.counter].iter() { + recvs.push(VecRecvEndpoint::new(queue.clone())); + } + + self.counter += 1; + + (sends, recvs) + } + } + + impl TcpBytesExchange { + fn new() -> Self { + unimplemented!() + } + } + + // Allocates local and remote queue pairs, respectively. + fn allocate_queue_pairs(local: usize, remote: usize) -> (Vec>>, Vec>>) { + + // type annotations necessary despite return signature because ... Rust. + let local_to_remote: Vec> = (0 .. local).map(|_| (0 .. remote).map(|_| SharedQueue::new()).collect()).collect(); + let remote_to_local: Vec> = (0 .. remote).map(|r| (0 .. local).map(|l| local_to_remote[l][r].clone()).collect()).collect(); + + (local_to_remote, remote_to_local) + } + + /// Receives serialized data from a `Read`, for example the network. + /// + /// The `BinaryReceiver` repeatedly reads binary data from its reader into + /// a binary Bytes slice which can be broken off and handed to recipients as + /// messages become complete. + struct BinaryReceiver { + + worker_offset: usize, + + reader: R, // the generic reader. + buffer: Bytes, // current working buffer. + length: usize, // consumed buffer elements. + targets: Vec>, // to process-local workers. + log_sender: ::logging::CommsLogger, // logging stuffs. + + in_progress: Vec>, // buffers shared with workers. + stash: Vec>, // reclaimed and resuable buffers. + size: usize, // current buffer allocation size. + } + + impl BinaryReceiver { + + fn new( + reader: R, + targets: Vec>, + worker_offset: usize, + log_sender: ::logging::CommsLogger) -> BinaryReceiver { + BinaryReceiver { + reader, + targets, + log_sender, + buffer: Bytes::from(vec![0u8; 1 << 20]), + length: 0, + in_progress: Vec::new(), + stash: Vec::new(), + size: 1 << 20, + worker_offset, + } + } + + // Retire `self.buffer` and acquire a new buffer of at least `self.size` bytes. + fn refresh_buffer(&mut self) { + + if self.stash.is_empty() { + for shared in self.in_progress.iter_mut() { + if let Some(bytes) = shared.take() { + match bytes.try_recover::>() { + Ok(vec) => { self.stash.push(vec); }, + Err(bytes) => { *shared = Some(bytes); }, + } + } + } + self.in_progress.retain(|x| x.is_some()); + } + + let self_size = self.size; + self.stash.retain(|x| x.capacity() == self_size); + + + let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; 1 << self.size]); + let new_buffer = Bytes::from(new_buffer); + let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + + self.buffer[.. self.length].copy_from_slice(&old_buffer[.. self.length]); + + self.in_progress.push(Some(old_buffer)); + } + + fn recv_loop(&mut self) { + + // 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. + + loop { + + // If our buffer is full we should copy it to a new buffer. + if self.length == self.buffer.len() { + // If full and not complete, we must increase the size. + if self.length == self.size { + self.size *= 2; + } + self.refresh_buffer(); + } + + // Attempt to read some more bytes into self.buffer. + self.length = self.reader.read(&mut self.buffer[self.length ..]).unwrap_or(0); + + // Consume complete messages from the front of self.buffer. + while let Some(header) = MessageHeader::try_read(&mut &self.buffer[.. self.length]) { + + 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, + }) + ) + ); + + // TODO: Consolidate message sequences sent to the same worker. + let bytes = self.buffer.extract_to(header.required_bytes()); + self.targets[header.target - self.worker_offset].push(bytes); + } + } + } + } + + // structure in charge of sending data to a Writer, for example the network + struct BinarySender { + writer: W, + sources: Vec>, + log_sender: ::logging::CommsLogger, + } + + impl BinarySender { + fn new(writer: W, sources: Vec>, log_sender: ::logging::CommsLogger) -> BinarySender { + BinarySender { writer, sources, log_sender } + } + + fn send_loop(&mut self) { + + let mut stash = Vec::new(); + + // TODO: The previous TCP code exited cleanly when all inputs were dropped. + // Here we don't have inputs that drop, but perhaps we can instead + // notice when the other side has hung up (the `Arc` in `SharedQueue`). + + while !self.sources.is_empty() { + + for source in self.sources.iter_mut() { + while let Some(bytes) = source.pop() { + stash.push(bytes); + } + } + + // If we got zero data, check that everyone is still alive. + if stash.is_empty() { + self.sources.retain(|x| !x.is_done()); + } + + for bytes in stash.drain(..) { + self.writer.write_all(&bytes[..]); + } + + self.writer.flush().unwrap(); // <-- because writer is buffered + } + } + } +} + +/// Builds an instance of a ProcessBinary. +/// +/// Builders are required because some of the state in a `ProcessBinary` 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 ProcessBinaryBuilder { + inner: Process, + index: usize, // number out of peers + peers: usize, // number of peer allocators (for typed channel allocation). + sends: Vec, // for pushing bytes at each other process. + recvs: Vec, // for pulling bytes from each other process. +} + +impl ProcessBinaryBuilder { + /// 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( + mut byte_exchanger: BE, + my_process: usize, + threads: usize, + processes: usize) -> Vec> { + + Process::new_vector(threads) + .into_iter() + .enumerate() + .map(|(index, inner)| { + let (sends, recvs) = byte_exchanger.new(my_process * threads + index); + ProcessBinaryBuilder { + inner, + index: my_process * threads + index, + peers: threads * processes, + sends, + recvs, + }}) + .collect() + } + + /// Builds a `ProcessBinary`, instantiating `Rc>` elements. + pub fn build(self) -> ProcessBinary { + + let mut sends = Vec::new(); + for send in self.sends.into_iter() { + sends.push(Rc::new(RefCell::new(send))); + } + + ProcessBinary { + inner: self.inner, + index: self.index, + peers: self.peers, + allocated: 0, + sends, + recvs: self.recvs, + to_local: Vec::new(), + } + } +} + +// A specific Communicator for inter-thread intra-process communication +pub struct ProcessBinary { + + inner: Process, // 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). + + // 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 ProcessBinary { + 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 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. + 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(Puller::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() { + + while let Some(mut bytes) = recv.receive() { + + // TODO: We could wrap `bytes` in a bytes::rc::Bytes, + // which could reduce `Arc` overhead, if it hurts. + // This new `Arc` should be local/uncontended, though. + let mut bytes = Bytes::from(bytes); + + // 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); + // } + // } + } +} + +/// 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. +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(); + + // acquire byte buffer and write header, element. + let mut borrow = self.sender.borrow_mut(); + let mut bytes = borrow.reserve(header.required_bytes()); + header.write_to(&mut bytes).expect("failed to write header!"); + + element.into_bytes(&mut 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. +struct Puller { + inner: Box>>, // inner pullable (e.g. intra-process typed queue) + current: Option>, + receiver: Rc>>, // source of serialized buffers +} + +impl Puller { + fn new(inner: Box>>, receiver: Rc>>) -> Puller { + Puller { + inner, + current: None, + receiver, + } + } +} + +impl Pull> for Puller { + #[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 From ae34b8f4d4b7ff75a1e0cd366054385f39c52777 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Fri, 27 Jul 2018 09:00:50 +0200 Subject: [PATCH 21/43] compiling checkpoint --- .../src/allocator/process_binary_neu.rs | 212 ++++++++++++------ communication/src/initialize.rs | 2 +- communication/src/networking.rs | 86 +++---- 3 files changed, 189 insertions(+), 111 deletions(-) diff --git a/communication/src/allocator/process_binary_neu.rs b/communication/src/allocator/process_binary_neu.rs index c23addfe5..4c45a415f 100644 --- a/communication/src/allocator/process_binary_neu.rs +++ b/communication/src/allocator/process_binary_neu.rs @@ -19,6 +19,7 @@ pub struct SharedQueue { impl SharedQueue { pub fn push(&mut self, bytes: T) { self.queue.lock().expect("unable to lock shared queue").push_back(bytes) } pub fn pop(&mut self) -> Option { self.queue.lock().expect("unable to lock shared queue").pop_front() } + pub fn drain_into(&mut self, dest: &mut Vec) { let mut lock = self.queue.lock().expect("unable to lock shared queue"); dest.extend(lock.drain(..)); } pub fn is_empty(&self) -> bool { self.queue.lock().expect("unable to lock shared queue").is_empty() } pub fn is_done(&self) -> bool { Arc::strong_count(&self.queue) == 1 } pub fn new() -> Self { SharedQueue { queue: Arc::new(Mutex::new(VecDeque::new())) } } @@ -45,7 +46,7 @@ pub trait BytesExchange { /// /// Importantly, the Send side may share state to coalesce the buffering and /// transmission of records. That is why there are `Rc>` things there. - fn new(&mut self, worker: usize) -> (Vec, Vec); + fn next(&mut self) -> Option<(Vec, Vec)>; } /// A type that can provide and publish writeable binary buffers. @@ -211,23 +212,27 @@ pub mod local { impl BytesExchange for LocalBytesExchange { type Send = VecSendEndpoint; type Recv = VecRecvEndpoint; - fn new(&mut self, worker: usize) -> (Vec, Vec) { + fn next(&mut self) -> Option<(Vec, Vec)> { - let mut sends = Vec::with_capacity(self.forward.len()); + if self.counter < self.forward.len() { - for forward in self.forward[self.counter].iter() { - sends.push(VecSendEndpoint::new(forward.clone())); - } + let mut sends = Vec::with_capacity(self.forward.len()); - let mut recvs = Vec::with_capacity(self.forward.len()); + for forward in self.forward[self.counter].iter() { + sends.push(VecSendEndpoint::new(forward.clone())); + } - for forward in self.forward.iter() { - recvs.push(VecRecvEndpoint::new(forward[self.counter].clone())); - } + let mut recvs = Vec::with_capacity(self.forward.len()); - self.counter += 1; + for forward in self.forward.iter() { + recvs.push(VecRecvEndpoint::new(forward[self.counter].clone())); + } + + self.counter += 1; - (sends, recvs) + Some((sends, recvs)) + } + else { None } } } } @@ -249,7 +254,6 @@ pub mod tcp { forward: Vec>>, /// Reverse[i,j]: to process i from worker j. reverse: Vec>>, - counter: usize, } impl BytesExchange for TcpBytesExchange { @@ -260,32 +264,31 @@ pub mod tcp { // Returns two vectors of length #processes - 1. // The first contains destinations to send to remote processes, // The second contains sources to receive from remote processes. - fn new(&mut self, worker: usize) -> (Vec, Vec) { + fn next(&mut self) -> Option<(Vec, Vec)> { - let mut sends = Vec::with_capacity(self.forward.len()); - for queue in self.forward[self.counter].iter() { - sends.push(VecSendEndpoint::new(queue.clone())); + if !self.forward.is_empty() && !self.reverse.is_empty() { + Some(( + self.forward.remove(0).into_iter().map(|x| VecSendEndpoint::new(x)).collect(), + self.reverse.remove(0).into_iter().map(|x| VecRecvEndpoint::new(x)).collect(), + )) } - - let mut recvs = Vec::with_capacity(self.forward.len()); - for queue in self.reverse[self.counter].iter() { - recvs.push(VecRecvEndpoint::new(queue.clone())); + else { + None } - - self.counter += 1; - - (sends, recvs) } } impl TcpBytesExchange { - fn new() -> Self { - unimplemented!() + pub fn new(forward: Vec>>, reverse: Vec>>) -> Self { + TcpBytesExchange { + forward, + reverse, + } } } // Allocates local and remote queue pairs, respectively. - fn allocate_queue_pairs(local: usize, remote: usize) -> (Vec>>, Vec>>) { + pub fn allocate_queue_pairs(local: usize, remote: usize) -> (Vec>>, Vec>>) { // type annotations necessary despite return signature because ... Rust. let local_to_remote: Vec> = (0 .. local).map(|_| (0 .. remote).map(|_| SharedQueue::new()).collect()).collect(); @@ -299,7 +302,7 @@ pub mod tcp { /// The `BinaryReceiver` repeatedly reads binary data from its reader into /// a binary Bytes slice which can be broken off and handed to recipients as /// messages become complete. - struct BinaryReceiver { + pub struct BinaryReceiver { worker_offset: usize, @@ -316,7 +319,7 @@ pub mod tcp { impl BinaryReceiver { - fn new( + pub fn new( reader: R, targets: Vec>, worker_offset: usize, @@ -362,7 +365,7 @@ pub mod tcp { self.in_progress.push(Some(old_buffer)); } - fn recv_loop(&mut self) { + pub fn recv_loop(&mut self) { // Each loop iteration adds to `self.Bytes` and consumes all complete messages. // At the start of each iteration, `self.buffer[..self.length]` represents valid @@ -374,6 +377,18 @@ pub mod tcp { loop { + // Attempt to read some more bytes into self.buffer. + self.length += self.reader.read(&mut self.buffer[self.length ..]).unwrap_or(0); + + // Consume complete messages from the front of self.buffer. + while let Some(header) = MessageHeader::try_read(&mut &self.buffer[.. self.length]) { + // TODO: Consolidate message sequences sent to the same worker. + let peeled_bytes = header.required_bytes(); + let bytes = self.buffer.extract_to(peeled_bytes); + self.length -= peeled_bytes; + self.targets[header.target - self.worker_offset].push(bytes); + } + // If our buffer is full we should copy it to a new buffer. if self.length == self.buffer.len() { // If full and not complete, we must increase the size. @@ -382,72 +397,41 @@ pub mod tcp { } self.refresh_buffer(); } - - // Attempt to read some more bytes into self.buffer. - self.length = self.reader.read(&mut self.buffer[self.length ..]).unwrap_or(0); - - // Consume complete messages from the front of self.buffer. - while let Some(header) = MessageHeader::try_read(&mut &self.buffer[.. self.length]) { - - 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, - }) - ) - ); - - // TODO: Consolidate message sequences sent to the same worker. - let bytes = self.buffer.extract_to(header.required_bytes()); - self.targets[header.target - self.worker_offset].push(bytes); - } } } } - // structure in charge of sending data to a Writer, for example the network - struct BinarySender { + // structure in charge of sending data to a Writer, for example the network. + pub struct BinarySender { writer: W, sources: Vec>, log_sender: ::logging::CommsLogger, } impl BinarySender { - fn new(writer: W, sources: Vec>, log_sender: ::logging::CommsLogger) -> BinarySender { + pub fn new(writer: W, sources: Vec>, log_sender: ::logging::CommsLogger) -> BinarySender { BinarySender { writer, sources, log_sender } } - fn send_loop(&mut self) { + pub fn send_loop(&mut self) { let mut stash = Vec::new(); - - // TODO: The previous TCP code exited cleanly when all inputs were dropped. - // Here we don't have inputs that drop, but perhaps we can instead - // notice when the other side has hung up (the `Arc` in `SharedQueue`). - while !self.sources.is_empty() { for source in self.sources.iter_mut() { - while let Some(bytes) = source.pop() { - stash.push(bytes); - } + source.drain_into(&mut stash); } // If we got zero data, check that everyone is still alive. if stash.is_empty() { self.sources.retain(|x| !x.is_done()); + self.writer.flush().expect("Failed to flush writer."); } for bytes in stash.drain(..) { - self.writer.write_all(&bytes[..]); + self.writer.write_all(&bytes[..]).expect("Write failure in send_loop."); } - self.writer.flush().unwrap(); // <-- because writer is buffered } } } @@ -481,7 +465,7 @@ impl ProcessBinaryBuilder { .into_iter() .enumerate() .map(|(index, inner)| { - let (sends, recvs) = byte_exchanger.new(my_process * threads + index); + let (sends, recvs) = byte_exchanger.next().unwrap(); ProcessBinaryBuilder { inner, index: my_process * threads + index, @@ -632,6 +616,94 @@ impl Allocate for ProcessBinary { } } +/// 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>> { + + let processes = addresses.len(); + + use networking::create_sockets; + let mut results = create_sockets(addresses, my_index, noisy)?; + + // Send and recv connections between local workers and remote processes. + let (local_send, remote_recv) = tcp::allocate_queue_pairs(threads, results.len() - 1); + let (local_recv, remote_send) = tcp::allocate_queue_pairs(threads, results.len() - 1); + + let mut remote_recv_iter = remote_recv.into_iter(); + let mut remote_send_iter = remote_send.into_iter(); + + // 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 = remote_recv_iter.next().unwrap(); + let remote_send = remote_send_iter.next().unwrap(); + + { + let log_sender = log_sender.clone(); + let stream = stream.try_clone()?; + // start senders and receivers associated with this stream + 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), + }); + + let stream = ::std::io::BufWriter::with_capacity(1 << 20, stream); + tcp::BinarySender::new(stream, remote_recv, log_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 = + ::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), + }); + tcp::BinaryReceiver::new(stream, remote_send, threads * my_index, log_sender) + .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 byte_exchange = tcp::TcpBytesExchange::new(local_send, local_recv); + let builders = ProcessBinaryBuilder::new_vector(byte_exchange, my_index, threads, processes); + + Ok(builders) +} + + /// 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 diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index a0d6e60bd..e7ea86946 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -8,7 +8,7 @@ use getopts; use std::sync::Arc; use allocator::{Thread, Process, Generic, GenericBuilder}; -use allocator::process_binary::ProcessBinaryBuilder; +// use allocator::process_binary::ProcessBinaryBuilder; use networking::initialize_networking; /// Possible configurations for the communication infrastructure. diff --git a/communication/src/networking.rs b/communication/src/networking.rs index 83367f1b4..479815113 100644 --- a/communication/src/networking.rs +++ b/communication/src/networking.rs @@ -234,19 +234,7 @@ 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(); - - let start_task = thread::spawn(move || start_connections(hosts1, my_index, noisy)); - 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 results = create_sockets(addresses, my_index, noisy)?; 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) @@ -266,18 +254,17 @@ pub fn initialize_networking( 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() - })?; + 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), + }); + BinarySender::new(BufWriter::with_capacity(1 << 20, stream), sender_channels_r,log_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 @@ -289,18 +276,17 @@ pub fn initialize_networking( 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() - })?; + 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), + }); + BinaryReceiver::new(stream, reader_channels_r, log_sender) + .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 @@ -330,6 +316,26 @@ pub fn initialize_networking( Ok(results) } + +pub fn create_sockets(addresses: Vec, my_index: usize, noisy: bool) -> Result>> { + + let hosts1 = Arc::new(addresses); + let hosts2 = hosts1.clone(); + + let start_task = thread::spawn(move || start_connections(hosts1, my_index, noisy)); + 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) } + + Ok(results) +} + + // result contains connections [0, my_index - 1]. fn start_connections(addresses: Arc>, my_index: usize, noisy: bool) -> Result>> { let mut results: Vec<_> = (0..my_index).map(|_| None).collect(); @@ -339,7 +345,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)); + stream.write_u64::(my_index as u64)?; results[index as usize] = Some(stream); if noisy { println!("worker {}:\tconnection to worker {}", my_index, index); } connected = true; @@ -361,9 +367,9 @@ fn await_connections(addresses: Arc>, my_index: usize, noisy: bool) 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 identifier = stream.read_u64::()? as usize; results[identifier - my_index - 1] = Some(stream); if noisy { println!("worker {}:\tconnection from worker {}", my_index, identifier); } } From 1ae27f13903190c546bcac4f545de40a51e8ac03 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Fri, 27 Jul 2018 09:29:33 +0200 Subject: [PATCH 22/43] reorganization --- communication/src/allocator/mod.rs | 4 +- .../src/allocator/zero_copy/allocator.rs | 276 ++++++++++++++++++ .../src/allocator/zero_copy/binary.rs | 195 +++++++++++++ .../src/allocator/zero_copy/bytes_exchange.rs | 143 +++++++++ .../src/allocator/zero_copy/initialize.rs | 92 ++++++ communication/src/allocator/zero_copy/mod.rs | 8 + .../src/allocator/zero_copy/shared_queue.rs | 21 ++ 7 files changed, 738 insertions(+), 1 deletion(-) create mode 100644 communication/src/allocator/zero_copy/allocator.rs create mode 100644 communication/src/allocator/zero_copy/binary.rs create mode 100644 communication/src/allocator/zero_copy/bytes_exchange.rs create mode 100644 communication/src/allocator/zero_copy/initialize.rs create mode 100644 communication/src/allocator/zero_copy/mod.rs create mode 100644 communication/src/allocator/zero_copy/shared_queue.rs diff --git a/communication/src/allocator/mod.rs b/communication/src/allocator/mod.rs index 40690e49f..88505e323 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -13,7 +13,9 @@ pub mod process; pub mod binary; pub mod generic; pub mod process_binary; -pub mod process_binary_neu; +// pub mod process_binary_neu; + +pub mod zero_copy; use {Data, Push, Pull}; diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs new file mode 100644 index 000000000..6cc09baf8 --- /dev/null +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -0,0 +1,276 @@ +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, Process}; + +use super::{BytesExchange}; +use super::bytes_exchange::{SendEndpoint, RecvEndpoint}; + +/// Builds an instance of a ProcessBinary. +/// +/// Builders are required because some of the state in a `ProcessBinary` 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 ProcessBinaryBuilder { + inner: Process, + index: usize, // number out of peers + peers: usize, // number of peer allocators (for typed channel allocation). + sends: Vec, // for pushing bytes at each other process. + recvs: Vec, // for pulling bytes from each other process. +} + +impl ProcessBinaryBuilder { + /// 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( + mut byte_exchanger: BE, + my_process: usize, + threads: usize, + processes: usize) -> Vec> { + + Process::new_vector(threads) + .into_iter() + .enumerate() + .map(|(index, inner)| { + let (sends, recvs) = byte_exchanger.next().unwrap(); + ProcessBinaryBuilder { + inner, + index: my_process * threads + index, + peers: threads * processes, + sends, + recvs, + }}) + .collect() + } + + /// Builds a `ProcessBinary`, instantiating `Rc>` elements. + pub fn build(self) -> ProcessBinary { + + let mut sends = Vec::new(); + for send in self.sends.into_iter() { + sends.push(Rc::new(RefCell::new(send))); + } + + ProcessBinary { + inner: self.inner, + index: self.index, + peers: self.peers, + allocated: 0, + sends, + recvs: self.recvs, + to_local: Vec::new(), + } + } +} + +// A specific Communicator for inter-thread intra-process communication +pub struct ProcessBinary { + + inner: Process, // 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). + + // 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 ProcessBinary { + 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 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. + 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(Puller::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() { + + while let Some(mut bytes) = recv.receive() { + + // TODO: We could wrap `bytes` in a bytes::rc::Bytes, + // which could reduce `Arc` overhead, if it hurts. + // This new `Arc` should be local/uncontended, though. + let mut bytes = Bytes::from(bytes); + + // 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); + // } + // } + } +} + +/// 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. +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(); + + // acquire byte buffer and write header, element. + let mut borrow = self.sender.borrow_mut(); + let mut bytes = borrow.reserve(header.required_bytes()); + header.write_to(&mut bytes).expect("failed to write header!"); + + element.into_bytes(&mut 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. +struct Puller { + inner: Box>>, // inner pullable (e.g. intra-process typed queue) + current: Option>, + receiver: Rc>>, // source of serialized buffers +} + +impl Puller { + fn new(inner: Box>>, receiver: Rc>>) -> Puller { + Puller { + inner, + current: None, + receiver, + } + } +} + +impl Pull> for Puller { + #[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/binary.rs b/communication/src/allocator/zero_copy/binary.rs new file mode 100644 index 000000000..2751fbdab --- /dev/null +++ b/communication/src/allocator/zero_copy/binary.rs @@ -0,0 +1,195 @@ +use std::io::{Read, Write}; + +use bytes::arc::Bytes; + +use networking::MessageHeader; +use super::{BytesExchange, SharedQueue}; +use super::{BytesSendEndpoint, BytesRecvEndpoint}; + +/// Allocates pairs of byte exchanges for remote workers. +pub struct TcpBytesExchange { + /// Forward[i,j]: from worker i to process j. + forward: Vec>>, + /// Reverse[i,j]: to process i from worker j. + reverse: Vec>>, +} + +impl BytesExchange for TcpBytesExchange { + + type Send = BytesSendEndpoint; + type Recv = BytesRecvEndpoint; + + // Returns two vectors of length #processes - 1. + // The first contains destinations to send to remote processes, + // The second contains sources to receive from remote processes. + fn next(&mut self) -> Option<(Vec, Vec)> { + + if !self.forward.is_empty() && !self.reverse.is_empty() { + Some(( + self.forward.remove(0).into_iter().map(|x| BytesSendEndpoint::new(x)).collect(), + self.reverse.remove(0).into_iter().map(|x| BytesRecvEndpoint::new(x)).collect(), + )) + } + else { + None + } + } +} + +impl TcpBytesExchange { + pub fn new(forward: Vec>>, reverse: Vec>>) -> Self { + TcpBytesExchange { + forward, + reverse, + } + } +} + +// Allocates local and remote queue pairs, respectively. +pub fn allocate_queue_pairs(local: usize, remote: usize) -> (Vec>>, Vec>>) { + + // type annotations necessary despite return signature because ... Rust. + let local_to_remote: Vec> = (0 .. local).map(|_| (0 .. remote).map(|_| SharedQueue::new()).collect()).collect(); + let remote_to_local: Vec> = (0 .. remote).map(|r| (0 .. local).map(|l| local_to_remote[l][r].clone()).collect()).collect(); + + (local_to_remote, remote_to_local) +} + +/// Receives serialized data from a `Read`, for example the network. +/// +/// The `BinaryReceiver` repeatedly reads binary data from its reader into +/// a binary Bytes slice which can be broken off and handed to recipients as +/// messages become complete. +pub struct BinaryReceiver { + + worker_offset: usize, + + reader: R, // the generic reader. + buffer: Bytes, // current working buffer. + length: usize, // consumed buffer elements. + targets: Vec>, // to process-local workers. + log_sender: ::logging::CommsLogger, // logging stuffs. + + in_progress: Vec>, // buffers shared with workers. + stash: Vec>, // reclaimed and resuable buffers. + size: usize, // current buffer allocation size. +} + +impl BinaryReceiver { + + pub fn new( + reader: R, + targets: Vec>, + worker_offset: usize, + log_sender: ::logging::CommsLogger) -> BinaryReceiver { + BinaryReceiver { + reader, + targets, + log_sender, + buffer: Bytes::from(vec![0u8; 1 << 20]), + length: 0, + in_progress: Vec::new(), + stash: Vec::new(), + size: 1 << 20, + worker_offset, + } + } + + // Retire `self.buffer` and acquire a new buffer of at least `self.size` bytes. + fn refresh_buffer(&mut self) { + + if self.stash.is_empty() { + for shared in self.in_progress.iter_mut() { + if let Some(bytes) = shared.take() { + match bytes.try_recover::>() { + Ok(vec) => { self.stash.push(vec); }, + Err(bytes) => { *shared = Some(bytes); }, + } + } + } + self.in_progress.retain(|x| x.is_some()); + } + + let self_size = self.size; + self.stash.retain(|x| x.capacity() == self_size); + + + let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; 1 << self.size]); + let new_buffer = Bytes::from(new_buffer); + let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + + self.buffer[.. self.length].copy_from_slice(&old_buffer[.. self.length]); + + self.in_progress.push(Some(old_buffer)); + } + + pub fn recv_loop(&mut self) { + + // 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. + + loop { + + // Attempt to read some more bytes into self.buffer. + self.length += self.reader.read(&mut self.buffer[self.length ..]).unwrap_or(0); + + // Consume complete messages from the front of self.buffer. + while let Some(header) = MessageHeader::try_read(&mut &self.buffer[.. self.length]) { + // TODO: Consolidate message sequences sent to the same worker. + let peeled_bytes = header.required_bytes(); + let bytes = self.buffer.extract_to(peeled_bytes); + self.length -= peeled_bytes; + self.targets[header.target - self.worker_offset].push(bytes); + } + + // If our buffer is full we should copy it to a new buffer. + if self.length == self.buffer.len() { + // If full and not complete, we must increase the size. + if self.length == self.size { + self.size *= 2; + } + self.refresh_buffer(); + } + } + } +} + +// structure in charge of sending data to a Writer, for example the network. +pub struct BinarySender { + writer: W, + sources: Vec>, + log_sender: ::logging::CommsLogger, +} + +impl BinarySender { + pub fn new(writer: W, sources: Vec>, log_sender: ::logging::CommsLogger) -> BinarySender { + BinarySender { writer, sources, log_sender } + } + + pub fn send_loop(&mut self) { + + let mut stash = Vec::new(); + while !self.sources.is_empty() { + + for source in self.sources.iter_mut() { + source.drain_into(&mut stash); + } + + // If we got zero data, check that everyone is still alive. + if stash.is_empty() { + self.sources.retain(|x| !x.is_done()); + self.writer.flush().expect("Failed to flush writer."); + } + + for bytes in stash.drain(..) { + self.writer.write_all(&bytes[..]).expect("Write failure in send_loop."); + } + + } + } +} \ 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..b2a7640f0 --- /dev/null +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -0,0 +1,143 @@ +use std::ops::DerefMut; +use bytes::arc::Bytes; + +use super::SharedQueue; + +/// A type that can allocate send and receive endpoints for byte exchanges. +/// +/// The `BytesExchange` intent is that one can abstractly define mechanisms for exchanging +/// bytes between various entities. In some cases this may be between worker threads within +/// a process, in other cases it may be between worker threads and remote processes. At the +/// moment the cardinalities of remote endpoints requires some context and interpretation. +pub trait BytesExchange { + /// The type of the send endpoint. + type Send: SendEndpoint+'static; + /// The type of the receive endpoint. + type Recv: RecvEndpoint+'static; + /// Allocates endpoint pairs for a specified worker. + /// + /// Importantly, the Send side may share state to coalesce the buffering and + /// transmission of records. That is why there are `Rc>` things there. + fn next(&mut self) -> Option<(Vec, Vec)>; +} + +/// A type that can provide and publish writeable binary buffers. +pub trait SendEndpoint { + /// The type of the writeable binary buffer. + type SendBuffer: ::std::io::Write; + /// Provides a writeable buffer of the requested capacity. + fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; + /// Indicates that it is now appropriate to publish the buffer. + fn publish(&mut self); +} + +/// A type that can provide readable binary buffers. +pub trait RecvEndpoint { + type RecvBuffer: DerefMut; + /// Provides a readable buffer. + fn receive(&mut self) -> Option; +} + +pub struct BytesSendEndpoint { + send: SharedQueue, + in_progress: Vec>, + buffer: Vec, + stash: Vec>, + default_size: usize, +} + +impl BytesSendEndpoint { + /// Attempts to recover in-use buffers once uniquely owned. + fn harvest_shared(&mut self) { + for shared in self.in_progress.iter_mut() { + if let Some(bytes) = shared.take() { + match bytes.try_recover::>() { + Ok(vec) => { self.stash.push(vec); }, + Err(bytes) => { *shared = Some(bytes); }, + } + } + } + self.in_progress.retain(|x| x.is_some()); + } + + /// Moves `self.buffer` into `self.send`, replaces with empty buffer. + fn send_buffer(&mut self) { + + let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); + let buffer_len = buffer.len(); + if buffer_len > 0 { + + let mut bytes = Bytes::from(buffer); + let to_send = bytes.extract_to(buffer_len); + + self.send.push(to_send); + self.in_progress.push(Some(bytes)); + } + else { + if buffer.capacity() == self.default_size { + self.stash.push(buffer); + } + } + } + + /// Allocates a new `BytesSendEndpoint` from a shared queue. + pub fn new(queue: SharedQueue) -> Self { + BytesSendEndpoint { + send: queue, + in_progress: Vec::new(), + buffer: Vec::new(), + stash: Vec::new(), + default_size: 1 << 20, + } + } +} + +impl SendEndpoint for BytesSendEndpoint { + + type SendBuffer = Vec; + + fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { + + // If we don't have enough capacity in `self.buffer`... + if self.buffer.capacity() < capacity + self.buffer.len() { + self.send_buffer(); + if capacity > self.default_size { + self.buffer = Vec::with_capacity(capacity); + } + else { + if self.stash.is_empty() { + // Attempt to recover shared buffers. + self.harvest_shared(); + } + self.buffer = self.stash.pop().unwrap_or_else(|| Vec::with_capacity(self.default_size)) + } + } + + &mut self.buffer + } + + fn publish(&mut self) { + self.harvest_shared(); + if self.send.is_empty() { + self.send_buffer(); + } + } +} + +pub struct BytesRecvEndpoint { + recv: SharedQueue, +} + + +impl BytesRecvEndpoint { + pub fn new(queue: SharedQueue) -> Self { + BytesRecvEndpoint { recv: queue } + } +} + +impl RecvEndpoint for BytesRecvEndpoint { + type RecvBuffer = Bytes; + fn receive(&mut self) -> Option { + self.recv.pop() + } +} \ 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..9c3e7065d --- /dev/null +++ b/communication/src/allocator/zero_copy/initialize.rs @@ -0,0 +1,92 @@ + +use std::sync::Arc; + +use super::binary::{allocate_queue_pairs, BinarySender, BinaryReceiver, TcpBytesExchange}; +use super::allocator::ProcessBinaryBuilder; + +/// 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>> { + + let processes = addresses.len(); + + use networking::create_sockets; + let mut results = create_sockets(addresses, my_index, noisy)?; + + // Send and recv connections between local workers and remote processes. + let (local_send, remote_recv) = allocate_queue_pairs(threads, results.len() - 1); + let (local_recv, remote_send) = allocate_queue_pairs(threads, results.len() - 1); + + let mut remote_recv_iter = remote_recv.into_iter(); + let mut remote_send_iter = remote_send.into_iter(); + + // 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 = remote_recv_iter.next().unwrap(); + let remote_send = remote_send_iter.next().unwrap(); + + { + let log_sender = log_sender.clone(); + let stream = stream.try_clone()?; + // start senders and receivers associated with this stream + 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), + }); + + let stream = ::std::io::BufWriter::with_capacity(1 << 20, stream); + BinarySender::new(stream, remote_recv, log_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 = + ::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), + }); + BinaryReceiver::new(stream, remote_send, threads * my_index, log_sender) + .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 byte_exchange = TcpBytesExchange::new(local_send, local_recv); + let builders = ProcessBinaryBuilder::new_vector(byte_exchange, my_index, threads, processes); + + Ok(builders) +} \ 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..14d9d0fd4 --- /dev/null +++ b/communication/src/allocator/zero_copy/mod.rs @@ -0,0 +1,8 @@ +pub mod shared_queue; +pub mod bytes_exchange; +pub mod binary; +pub mod allocator; +pub mod initialize; + +pub use self::shared_queue::SharedQueue; +pub use self::bytes_exchange::{BytesExchange, BytesSendEndpoint, BytesRecvEndpoint}; \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/shared_queue.rs b/communication/src/allocator/zero_copy/shared_queue.rs new file mode 100644 index 000000000..2fe818004 --- /dev/null +++ b/communication/src/allocator/zero_copy/shared_queue.rs @@ -0,0 +1,21 @@ +use std::sync::{Arc, Mutex}; +use std::collections::VecDeque; + +pub struct SharedQueue { + queue: Arc>> +} + +impl SharedQueue { + pub fn push(&mut self, bytes: T) { self.queue.lock().expect("unable to lock shared queue").push_back(bytes) } + pub fn pop(&mut self) -> Option { self.queue.lock().expect("unable to lock shared queue").pop_front() } + pub fn drain_into(&mut self, dest: &mut Vec) { let mut lock = self.queue.lock().expect("unable to lock shared queue"); dest.extend(lock.drain(..)); } + pub fn is_empty(&self) -> bool { self.queue.lock().expect("unable to lock shared queue").is_empty() } + pub fn is_done(&self) -> bool { Arc::strong_count(&self.queue) == 1 } + pub fn new() -> Self { SharedQueue { queue: Arc::new(Mutex::new(VecDeque::new())) } } +} + +impl Clone for SharedQueue { + fn clone(&self) -> Self { + SharedQueue { queue: self.queue.clone() } + } +} From b09e61ba2a63bcc68537f482a80f0c7f0d4fcb5f Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 31 Jul 2018 10:26:18 +0200 Subject: [PATCH 23/43] hello example works --- communication/src/allocator/generic.rs | 13 +++++++++++++ communication/src/allocator/zero_copy/allocator.rs | 3 ++- communication/src/initialize.rs | 5 +++-- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index 8afdcd1df..7975446c7 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -5,6 +5,11 @@ use allocator::{Allocate, Message, Thread, Process, Binary}; use allocator::process_binary::{ProcessBinary, ProcessBinaryBuilder}; + +use allocator::zero_copy::allocator::ProcessBinary as ZeroCopyAllocator; +use allocator::zero_copy::allocator::ProcessBinaryBuilder as ZeroCopyBuilder; +use allocator::zero_copy::binary::TcpBytesExchange; + use {Push, Pull, Data}; /// Enumerates known implementors of `Allocate`. @@ -14,6 +19,7 @@ pub enum Generic { Process(Process), Binary(Binary), ProcessBinary(ProcessBinary<::allocator::process_binary::vec::VecBytesExchange>), + ZeroCopy(ZeroCopyAllocator), } impl Generic { @@ -24,6 +30,7 @@ impl Generic { &Generic::Process(ref p) => p.index(), &Generic::Binary(ref b) => b.index(), &Generic::ProcessBinary(ref pb) => pb.index(), + &Generic::ZeroCopy(ref z) => z.index(), } } /// The number of workers. @@ -33,6 +40,7 @@ impl Generic { &Generic::Process(ref p) => p.peers(), &Generic::Binary(ref b) => b.peers(), &Generic::ProcessBinary(ref pb) => pb.peers(), + &Generic::ZeroCopy(ref z) => z.peers(), } } /// Constructs several send endpoints and one receive endpoint. @@ -42,6 +50,7 @@ impl Generic { &mut Generic::Process(ref mut p) => p.allocate(), &mut Generic::Binary(ref mut b) => b.allocate(), &mut Generic::ProcessBinary(ref mut pb) => pb.allocate(), + &mut Generic::ZeroCopy(ref mut z) => z.allocate(), } } @@ -51,6 +60,7 @@ impl Generic { &mut Generic::Process(ref mut p) => p.pre_work(), &mut Generic::Binary(ref mut b) => b.pre_work(), &mut Generic::ProcessBinary(ref mut pb) => pb.pre_work(), + &mut Generic::ZeroCopy(ref mut z) => z.pre_work(), } } pub fn post_work(&mut self) { @@ -59,6 +69,7 @@ impl Generic { &mut Generic::Process(ref mut p) => p.post_work(), &mut Generic::Binary(ref mut b) => b.post_work(), &mut Generic::ProcessBinary(ref mut pb) => pb.post_work(), + &mut Generic::ZeroCopy(ref mut z) => z.post_work(), } } } @@ -85,6 +96,7 @@ pub enum GenericBuilder { Process(Process), Binary(Binary), ProcessBinary(ProcessBinaryBuilder<::allocator::process_binary::vec::VecBytesExchange>), + ZeroCopy(ZeroCopyBuilder), } impl GenericBuilder { @@ -94,6 +106,7 @@ impl GenericBuilder { GenericBuilder::Process(p) => Generic::Process(p), GenericBuilder::Binary(b) => Generic::Binary(b), GenericBuilder::ProcessBinary(pb) => Generic::ProcessBinary(pb.build()), + GenericBuilder::ZeroCopy(z) => Generic::ZeroCopy(z.build()), } } } diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index 6cc09baf8..c419df9ae 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -104,7 +104,7 @@ impl Allocate for ProcessBinary { for target_index in 0 .. self.peers() { // TODO: crappy place to hardcode this rule. - let process_id = target_index / inner_peers; + let mut process_id = target_index / inner_peers; if process_id == self.index / inner_peers { pushes.push(inner_sends.remove(0)); @@ -120,6 +120,7 @@ impl Allocate for ProcessBinary { }; // 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()))); } } diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index e7ea86946..3f9e9b128 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -9,7 +9,8 @@ use std::sync::Arc; use allocator::{Thread, Process, Generic, GenericBuilder}; // use allocator::process_binary::ProcessBinaryBuilder; -use networking::initialize_networking; +// use networking::initialize_networking; +use allocator::zero_copy::initialize::initialize_networking; /// Possible configurations for the communication infrastructure. pub enum Configuration { @@ -88,7 +89,7 @@ fn create_allocators(config: Configuration, logger: LogBuilder) -> Result { if let Ok(stuff) = initialize_networking(addresses, process, threads, report, logger) { - Ok(stuff.into_iter().map(|x| GenericBuilder::Binary(x)).collect()) + Ok(stuff.into_iter().map(|x| GenericBuilder::ZeroCopy(x)).collect()) } else { Err("failed to initialize networking".to_owned()) From b39dff211a3757e95d3f45f3f718492c523a44c6 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 31 Jul 2018 19:06:05 +0200 Subject: [PATCH 24/43] seems to work --- .../src/allocator/zero_copy/allocator.rs | 5 +++++ communication/src/allocator/zero_copy/binary.rs | 15 +++++++++++---- .../src/allocator/zero_copy/bytes_exchange.rs | 7 ++++++- communication/src/lib.rs | 1 + 4 files changed, 23 insertions(+), 5 deletions(-) diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index c419df9ae..c90a1d09b 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -271,6 +271,11 @@ impl Pull> for Puller { .pop_front() .map(|bytes| unsafe { Message::from_bytes(bytes) }); + // use std::ops::Deref; + // if let Some(ref mut x) = self.current { + // println!("decoded: {:?}", x.as_ref_or_mut().deref()); + // } + &mut self.current } } diff --git a/communication/src/allocator/zero_copy/binary.rs b/communication/src/allocator/zero_copy/binary.rs index 2751fbdab..1544d92ad 100644 --- a/communication/src/allocator/zero_copy/binary.rs +++ b/communication/src/allocator/zero_copy/binary.rs @@ -102,7 +102,7 @@ impl BinaryReceiver { for shared in self.in_progress.iter_mut() { if let Some(bytes) = shared.take() { match bytes.try_recover::>() { - Ok(vec) => { self.stash.push(vec); }, + Ok(mut vec) => { self.stash.push(vec); }, Err(bytes) => { *shared = Some(bytes); }, } } @@ -113,11 +113,13 @@ impl BinaryReceiver { let self_size = self.size; self.stash.retain(|x| x.capacity() == self_size); - - let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; 1 << self.size]); + let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; self.size]); let new_buffer = Bytes::from(new_buffer); let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + debug_assert!(self.length <= old_buffer.len()); + debug_assert!(self.length <= self.buffer.len()); + self.buffer[.. self.length].copy_from_slice(&old_buffer[.. self.length]); self.in_progress.push(Some(old_buffer)); @@ -136,7 +138,9 @@ impl BinaryReceiver { loop { // Attempt to read some more bytes into self.buffer. - self.length += self.reader.read(&mut self.buffer[self.length ..]).unwrap_or(0); + let read = self.reader.read(&mut self.buffer[self.length ..]).unwrap_or(0); + // if read > 0 { println!("read from reader: {:?} bytes", read); } + self.length += read; // Consume complete messages from the front of self.buffer. while let Some(header) = MessageHeader::try_read(&mut &self.buffer[.. self.length]) { @@ -144,6 +148,7 @@ impl BinaryReceiver { let peeled_bytes = header.required_bytes(); let bytes = self.buffer.extract_to(peeled_bytes); self.length -= peeled_bytes; + // println!("decoded message with {:?} bytes", bytes.len()); self.targets[header.target - self.worker_offset].push(bytes); } @@ -152,6 +157,7 @@ impl BinaryReceiver { // If full and not complete, we must increase the size. if self.length == self.size { self.size *= 2; + println!("new size: {:?}", self.size); } self.refresh_buffer(); } @@ -187,6 +193,7 @@ impl BinarySender { } for bytes in stash.drain(..) { + // println!("sending {} bytes", bytes.len()); self.writer.write_all(&bytes[..]).expect("Write failure in send_loop."); } diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index b2a7640f0..a490b780b 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -52,7 +52,7 @@ impl BytesSendEndpoint { for shared in self.in_progress.iter_mut() { if let Some(bytes) = shared.take() { match bytes.try_recover::>() { - Ok(vec) => { self.stash.push(vec); }, + Ok(mut vec) => { vec.clear(); self.stash.push(vec); }, Err(bytes) => { *shared = Some(bytes); }, } } @@ -98,6 +98,8 @@ impl SendEndpoint for BytesSendEndpoint { fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { + // println!("reserving {:?} bytes", capacity); + // If we don't have enough capacity in `self.buffer`... if self.buffer.capacity() < capacity + self.buffer.len() { self.send_buffer(); @@ -121,6 +123,9 @@ impl SendEndpoint for BytesSendEndpoint { if self.send.is_empty() { self.send_buffer(); } + else { + // println!("delaying publication!"); + } } } diff --git a/communication/src/lib.rs b/communication/src/lib.rs index 47c03aae9..873b1c82d 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -91,6 +91,7 @@ pub mod initialize; pub mod logging; use std::any::Any; + use abomonation::Abomonation; pub use allocator::Generic as Allocator; From 6cfff5a11ac0427ed569818f64b84f5eb87d4f70 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 2 Aug 2018 19:23:25 +0200 Subject: [PATCH 25/43] working, better perf --- communication/src/allocator/generic.rs | 10 +- .../src/allocator/zero_copy/allocator.rs | 128 +++++++++------- .../src/allocator/zero_copy/binary.rs | 145 ++++++++---------- .../src/allocator/zero_copy/bytes_exchange.rs | 31 ++-- .../src/allocator/zero_copy/initialize.rs | 56 ++++--- communication/src/allocator/zero_copy/mod.rs | 2 +- .../src/allocator/zero_copy/shared_queue.rs | 43 ++++-- communication/src/initialize.rs | 22 +-- 8 files changed, 233 insertions(+), 204 deletions(-) diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index 7975446c7..8bacd138f 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -6,9 +6,9 @@ use allocator::{Allocate, Message, Thread, Process, Binary}; use allocator::process_binary::{ProcessBinary, ProcessBinaryBuilder}; -use allocator::zero_copy::allocator::ProcessBinary as ZeroCopyAllocator; -use allocator::zero_copy::allocator::ProcessBinaryBuilder as ZeroCopyBuilder; -use allocator::zero_copy::binary::TcpBytesExchange; +use allocator::zero_copy::allocator::TcpAllocator as ZeroCopyAllocator; +use allocator::zero_copy::allocator::TcpBuilder as ZeroCopyBuilder; +// use allocator::zero_copy::binary::TcpBytesExchange; use {Push, Pull, Data}; @@ -19,7 +19,7 @@ pub enum Generic { Process(Process), Binary(Binary), ProcessBinary(ProcessBinary<::allocator::process_binary::vec::VecBytesExchange>), - ZeroCopy(ZeroCopyAllocator), + ZeroCopy(ZeroCopyAllocator), } impl Generic { @@ -96,7 +96,7 @@ pub enum GenericBuilder { Process(Process), Binary(Binary), ProcessBinary(ProcessBinaryBuilder<::allocator::process_binary::vec::VecBytesExchange>), - ZeroCopy(ZeroCopyBuilder), + ZeroCopy(ZeroCopyBuilder), } impl GenericBuilder { diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index c90a1d09b..f5f63fad3 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -1,6 +1,7 @@ use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; +use std::sync::{Arc, mpsc::{channel, Sender, Receiver}}; use bytes::arc::Bytes; @@ -9,57 +10,82 @@ use networking::MessageHeader; use {Allocate, Data, Push, Pull}; use allocator::{Message, Process}; -use super::{BytesExchange}; -use super::bytes_exchange::{SendEndpoint, RecvEndpoint}; +use super::bytes_exchange::SendEndpoint; +use super::bytes_exchange::BytesSendEndpoint; +use super::shared_queue::SharedQueueSend; -/// Builds an instance of a ProcessBinary. +/// Builds an instance of a TcpAllocator. /// -/// Builders are required because some of the state in a `ProcessBinary` cannot be sent between +/// 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 ProcessBinaryBuilder { +pub struct TcpBuilder { inner: Process, - index: usize, // number out of peers - peers: usize, // number of peer allocators (for typed channel allocation). - sends: Vec, // for pushing bytes at each other process. - recvs: Vec, // for pulling bytes from each other process. + index: usize, // number out of peers + peers: usize, // number of peer allocators. + sends: Vec)>>, // for pushing bytes at remote processes. + recvs: Receiver, // for pulling bytes from remote processes. } -impl ProcessBinaryBuilder { +impl TcpBuilder { /// 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( - mut byte_exchanger: BE, my_process: usize, threads: usize, - processes: usize) -> Vec> { + processes: usize) -> (Vec, Vec)>>, Vec>) { + let mut l2r_send = Vec::new(); + let mut l2r_recv = Vec::new(); + let mut r2l_send = Vec::new(); + let mut r2l_recv = Vec::new(); + + for _ in 0 .. threads { + let (send, recv) = channel(); + r2l_send.push(send); + r2l_recv.push(recv); + } + + for _ in 0 .. processes - 1 { + let (send, recv) = channel(); + l2r_send.push(send); + l2r_recv.push(recv); + } + + // let mut to_comms = (0 .. processes).map(|_| channel()).collect::>(); + // let mut from_comms = (0 .. threads).map(|_| channel()).collect::>(); + + let builders = Process::new_vector(threads) .into_iter() + .zip(r2l_recv.into_iter()) .enumerate() - .map(|(index, inner)| { - let (sends, recvs) = byte_exchanger.next().unwrap(); - ProcessBinaryBuilder { + .map(|(index, (inner, recvs))| { + TcpBuilder { inner, index: my_process * threads + index, peers: threads * processes, - sends, + sends: l2r_send.clone(), recvs, }}) - .collect() + .collect(); + + (builders, l2r_recv, r2l_send) } - /// Builds a `ProcessBinary`, instantiating `Rc>` elements. - pub fn build(self) -> ProcessBinary { + /// Builds a `TcpAllocator`, instantiating `Rc>` elements. + pub fn build(self) -> TcpAllocator { let mut sends = Vec::new(); for send in self.sends.into_iter() { - sends.push(Rc::new(RefCell::new(send))); + let shared = SharedQueueSend::from(send); + let sendpoint = BytesSendEndpoint::new(shared); + sends.push(Rc::new(RefCell::new(sendpoint))); } - ProcessBinary { + TcpAllocator { inner: self.inner, index: self.index, peers: self.peers, @@ -72,7 +98,7 @@ impl ProcessBinaryBuilder { } // A specific Communicator for inter-thread intra-process communication -pub struct ProcessBinary { +pub struct TcpAllocator { inner: Process, // A non-serialized inner allocator for process-local peers. @@ -81,12 +107,12 @@ pub struct ProcessBinary { allocated: usize, // indicates how many channels have been allocated (locally). // sending, receiving, and responding to binary buffers. - sends: Vec>>, // sends[x] -> goes to process x. - recvs: Vec, // recvs[x] <- from process x?. + sends: Vec>>, // sends[x] -> goes to process x. + recvs: Receiver, // recvs[x] <- from process x?. to_local: Vec>>>, // to worker-local typed pullers. } -impl Allocate for ProcessBinary { +impl Allocate for TcpAllocator { fn index(&self) -> usize { self.index } fn peers(&self) -> usize { self.peers } fn allocate(&mut self) -> (Vec>>>, Box>>, Option) { @@ -138,37 +164,34 @@ impl Allocate for ProcessBinary { #[inline(never)] fn pre_work(&mut self) { - for recv in self.recvs.iter_mut() { + while let Ok(bytes) = self.recvs.try_recv() { - while let Some(mut bytes) = recv.receive() { + // TODO: We could wrap `bytes` in a bytes::rc::Bytes, + // which could reduce `Arc` overhead, if it hurts. + // This new `Arc` should be local/uncontended, though. + let mut bytes = Bytes::from(bytes); - // TODO: We could wrap `bytes` in a bytes::rc::Bytes, - // which could reduce `Arc` overhead, if it hurts. - // This new `Arc` should be local/uncontended, though. - let mut bytes = Bytes::from(bytes); + // We expect that `bytes` contains an integral number of messages. + // No splitting occurs across allocations. + while bytes.len() > 0 { - // 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[..]) { - 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); - // 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!"); + // 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!"); } } } @@ -271,11 +294,6 @@ impl Pull> for Puller { .pop_front() .map(|bytes| unsafe { Message::from_bytes(bytes) }); - // use std::ops::Deref; - // if let Some(ref mut x) = self.current { - // println!("decoded: {:?}", x.as_ref_or_mut().deref()); - // } - &mut self.current } } diff --git a/communication/src/allocator/zero_copy/binary.rs b/communication/src/allocator/zero_copy/binary.rs index 1544d92ad..f48769bc6 100644 --- a/communication/src/allocator/zero_copy/binary.rs +++ b/communication/src/allocator/zero_copy/binary.rs @@ -1,73 +1,25 @@ use std::io::{Read, Write}; +use std::sync::{Arc, mpsc::{Sender, Receiver}}; +use std::net::TcpStream; + use bytes::arc::Bytes; use networking::MessageHeader; -use super::{BytesExchange, SharedQueue}; -use super::{BytesSendEndpoint, BytesRecvEndpoint}; - -/// Allocates pairs of byte exchanges for remote workers. -pub struct TcpBytesExchange { - /// Forward[i,j]: from worker i to process j. - forward: Vec>>, - /// Reverse[i,j]: to process i from worker j. - reverse: Vec>>, -} - -impl BytesExchange for TcpBytesExchange { - - type Send = BytesSendEndpoint; - type Recv = BytesRecvEndpoint; - - // Returns two vectors of length #processes - 1. - // The first contains destinations to send to remote processes, - // The second contains sources to receive from remote processes. - fn next(&mut self) -> Option<(Vec, Vec)> { - - if !self.forward.is_empty() && !self.reverse.is_empty() { - Some(( - self.forward.remove(0).into_iter().map(|x| BytesSendEndpoint::new(x)).collect(), - self.reverse.remove(0).into_iter().map(|x| BytesRecvEndpoint::new(x)).collect(), - )) - } - else { - None - } - } -} - -impl TcpBytesExchange { - pub fn new(forward: Vec>>, reverse: Vec>>) -> Self { - TcpBytesExchange { - forward, - reverse, - } - } -} - -// Allocates local and remote queue pairs, respectively. -pub fn allocate_queue_pairs(local: usize, remote: usize) -> (Vec>>, Vec>>) { - - // type annotations necessary despite return signature because ... Rust. - let local_to_remote: Vec> = (0 .. local).map(|_| (0 .. remote).map(|_| SharedQueue::new()).collect()).collect(); - let remote_to_local: Vec> = (0 .. remote).map(|r| (0 .. local).map(|l| local_to_remote[l][r].clone()).collect()).collect(); - - (local_to_remote, remote_to_local) -} /// Receives serialized data from a `Read`, for example the network. /// /// The `BinaryReceiver` repeatedly reads binary data from its reader into /// a binary Bytes slice which can be broken off and handed to recipients as /// messages become complete. -pub struct BinaryReceiver { +pub struct BinaryReceiver { worker_offset: usize, - reader: R, // the generic reader. + reader: TcpStream, // the generic reader. buffer: Bytes, // current working buffer. length: usize, // consumed buffer elements. - targets: Vec>, // to process-local workers. + targets: Vec>, // to process-local workers. log_sender: ::logging::CommsLogger, // logging stuffs. in_progress: Vec>, // buffers shared with workers. @@ -75,13 +27,13 @@ pub struct BinaryReceiver { size: usize, // current buffer allocation size. } -impl BinaryReceiver { +impl BinaryReceiver { pub fn new( - reader: R, - targets: Vec>, + reader: TcpStream, + targets: Vec>, worker_offset: usize, - log_sender: ::logging::CommsLogger) -> BinaryReceiver { + log_sender: ::logging::CommsLogger) -> BinaryReceiver { BinaryReceiver { reader, targets, @@ -135,21 +87,36 @@ impl BinaryReceiver { // 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. - loop { + let mut active = true; + let mut count = 0; + + while active { // Attempt to read some more bytes into self.buffer. - let read = self.reader.read(&mut self.buffer[self.length ..]).unwrap_or(0); - // if read > 0 { println!("read from reader: {:?} bytes", read); } + let read = match self.reader.read(&mut self.buffer[self.length ..]) { + Ok(n) => n, + Err(x) => { + // We don't expect this, as socket closure results in Ok(0) reads. + println!("Error: {:?}", x); + 0 + }, + }; + + active = read > 0; + self.length += read; // Consume complete messages from the front of self.buffer. while let Some(header) = MessageHeader::try_read(&mut &self.buffer[.. self.length]) { + // TODO: Consolidate message sequences sent to the same worker. let peeled_bytes = header.required_bytes(); let bytes = self.buffer.extract_to(peeled_bytes); self.length -= peeled_bytes; - // println!("decoded message with {:?} bytes", bytes.len()); - self.targets[header.target - self.worker_offset].push(bytes); + + self.targets[header.target - self.worker_offset] + .send(bytes) + .expect("Worker queue unavailable in recv_loop"); } // If our buffer is full we should copy it to a new buffer. @@ -157,46 +124,62 @@ impl BinaryReceiver { // If full and not complete, we must increase the size. if self.length == self.size { self.size *= 2; - println!("new size: {:?}", self.size); } self.refresh_buffer(); } } + + // println!("RECVER EXITING"); } } +// impl Drop for BinaryReceiver { +// fn drop(&mut self) { +// self.reader.shutdown(::std::net::Shutdown::Read).expect("Read shutdown failed"); +// } +// } + // structure in charge of sending data to a Writer, for example the network. -pub struct BinarySender { - writer: W, - sources: Vec>, +pub struct BinarySender { + writer: ::std::io::BufWriter, + source: Receiver<(Bytes, Arc<()>)>, log_sender: ::logging::CommsLogger, } -impl BinarySender { - pub fn new(writer: W, sources: Vec>, log_sender: ::logging::CommsLogger) -> BinarySender { - BinarySender { writer, sources, log_sender } +impl BinarySender { + pub fn new(writer: ::std::io::BufWriter, source: Receiver<(Bytes, Arc<()>)>, log_sender: ::logging::CommsLogger) -> BinarySender { + BinarySender { writer, source, log_sender } } pub fn send_loop(&mut self) { + let mut count = 0; let mut stash = Vec::new(); - while !self.sources.is_empty() { - - for source in self.sources.iter_mut() { - source.drain_into(&mut stash); - } - // If we got zero data, check that everyone is still alive. - if stash.is_empty() { - self.sources.retain(|x| !x.is_done()); - self.writer.flush().expect("Failed to flush writer."); + while let Ok((bytes, _count)) = self.source.recv() { + stash.push(bytes); + while let Ok((bytes, _count)) = self.source.try_recv() { + stash.push(bytes); } + // TODO: Could do scatter/gather write here. for bytes in stash.drain(..) { - // println!("sending {} bytes", bytes.len()); + count += bytes.len(); + // println!("Sending bytes: {:?}", bytes.len()); self.writer.write_all(&bytes[..]).expect("Write failure in send_loop."); + // println!("Sent bytes: {:?}", count); } - + // TODO: Could delay this until a self.source.recv() would block. + self.writer.flush().expect("Failed to flush writer."); } + + // println!("SENDER EXITING"); + } +} + +impl Drop for BinarySender { + fn drop(&mut self) { + self.writer.flush().expect("Failed to flush writer."); + self.writer.get_mut().shutdown(::std::net::Shutdown::Write).expect("Write shutdown failed"); } } \ 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 index a490b780b..c0ee8cac5 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -1,7 +1,7 @@ use std::ops::DerefMut; use bytes::arc::Bytes; -use super::SharedQueue; +use super::shared_queue::{SharedQueueSend, SharedQueueRecv}; /// A type that can allocate send and receive endpoints for byte exchanges. /// @@ -39,7 +39,7 @@ pub trait RecvEndpoint { } pub struct BytesSendEndpoint { - send: SharedQueue, + send: SharedQueueSend, in_progress: Vec>, buffer: Vec, stash: Vec>, @@ -63,25 +63,20 @@ impl BytesSendEndpoint { /// Moves `self.buffer` into `self.send`, replaces with empty buffer. fn send_buffer(&mut self) { - let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); - let buffer_len = buffer.len(); - if buffer_len > 0 { + if self.buffer.len() > 0 { + let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); + let buffer_len = buffer.len(); let mut bytes = Bytes::from(buffer); let to_send = bytes.extract_to(buffer_len); self.send.push(to_send); self.in_progress.push(Some(bytes)); } - else { - if buffer.capacity() == self.default_size { - self.stash.push(buffer); - } - } } /// Allocates a new `BytesSendEndpoint` from a shared queue. - pub fn new(queue: SharedQueue) -> Self { + pub fn new(queue: SharedQueueSend) -> Self { BytesSendEndpoint { send: queue, in_progress: Vec::new(), @@ -99,6 +94,9 @@ impl SendEndpoint for BytesSendEndpoint { fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { // println!("reserving {:?} bytes", capacity); + if self.send.is_empty() { + self.send_buffer(); + } // If we don't have enough capacity in `self.buffer`... if self.buffer.capacity() < capacity + self.buffer.len() { @@ -129,13 +127,20 @@ impl SendEndpoint for BytesSendEndpoint { } } +impl Drop for BytesSendEndpoint { + fn drop(&mut self) { + self.send_buffer(); + assert!(self.buffer.is_empty()); + } +} + pub struct BytesRecvEndpoint { - recv: SharedQueue, + recv: SharedQueueRecv, } impl BytesRecvEndpoint { - pub fn new(queue: SharedQueue) -> Self { + pub fn new(queue: SharedQueueRecv) -> Self { BytesRecvEndpoint { recv: queue } } } diff --git a/communication/src/allocator/zero_copy/initialize.rs b/communication/src/allocator/zero_copy/initialize.rs index 9c3e7065d..92178ce0d 100644 --- a/communication/src/allocator/zero_copy/initialize.rs +++ b/communication/src/allocator/zero_copy/initialize.rs @@ -1,8 +1,26 @@ use std::sync::Arc; -use super::binary::{allocate_queue_pairs, BinarySender, BinaryReceiver, TcpBytesExchange}; -use super::allocator::ProcessBinaryBuilder; +use super::binary::{BinarySender, BinaryReceiver}; +use super::allocator::TcpBuilder; + +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( @@ -11,19 +29,18 @@ pub fn initialize_networking( threads: usize, noisy: bool, log_sender: Arc::logging::CommsLogger+Send+Sync>) --> ::std::io::Result>> { +-> ::std::io::Result<(Vec, CommsGuard)> { let processes = addresses.len(); use networking::create_sockets; let mut results = create_sockets(addresses, my_index, noisy)?; - // Send and recv connections between local workers and remote processes. - let (local_send, remote_recv) = allocate_queue_pairs(threads, results.len() - 1); - let (local_recv, remote_send) = allocate_queue_pairs(threads, results.len() - 1); + let (builders, remote_recvs, remote_sends) = TcpBuilder::new_vector(my_index, threads, processes); + let mut remote_recv_iter = remote_recvs.into_iter(); - let mut remote_recv_iter = remote_recv.into_iter(); - let mut remote_send_iter = remote_send.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() { @@ -31,12 +48,10 @@ pub fn initialize_networking( if let Some(stream) = results[index].take() { let remote_recv = remote_recv_iter.next().unwrap(); - let remote_send = remote_send_iter.next().unwrap(); { let log_sender = log_sender.clone(); let stream = stream.try_clone()?; - // start senders and receivers associated with this stream let join_guard = ::std::thread::Builder::new() .name(format!("send thread {}", index)) @@ -53,16 +68,14 @@ pub fn initialize_networking( .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); + send_guards.push(join_guard); } { + let remote_sends = remote_sends.clone(); let log_sender = log_sender.clone(); let stream = stream.try_clone()?; - let _join_guard = + let join_guard = ::std::thread::Builder::new() .name(format!("recv thread {}", index)) .spawn(move || { @@ -71,22 +84,15 @@ pub fn initialize_networking( sender: false, remote: Some(index), }); - BinaryReceiver::new(stream, remote_send, threads * my_index, log_sender) + BinaryReceiver::new(stream, remote_sends, threads * my_index, log_sender) .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. + recv_guards.push(join_guard); } } } - let byte_exchange = TcpBytesExchange::new(local_send, local_recv); - let builders = ProcessBinaryBuilder::new_vector(byte_exchange, my_index, threads, processes); - - Ok(builders) + 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 index 14d9d0fd4..d9f7d34e4 100644 --- a/communication/src/allocator/zero_copy/mod.rs +++ b/communication/src/allocator/zero_copy/mod.rs @@ -4,5 +4,5 @@ pub mod binary; pub mod allocator; pub mod initialize; -pub use self::shared_queue::SharedQueue; +// pub use self::shared_queue::SharedQueue; pub use self::bytes_exchange::{BytesExchange, BytesSendEndpoint, BytesRecvEndpoint}; \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/shared_queue.rs b/communication/src/allocator/zero_copy/shared_queue.rs index 2fe818004..f4cde670b 100644 --- a/communication/src/allocator/zero_copy/shared_queue.rs +++ b/communication/src/allocator/zero_copy/shared_queue.rs @@ -1,21 +1,34 @@ -use std::sync::{Arc, Mutex}; -use std::collections::VecDeque; +use std::sync::{Arc, mpsc::{Sender, Receiver}}; -pub struct SharedQueue { - queue: Arc>> +pub struct SharedQueueSend { + queue: Sender<(T, Arc<()>)>, + count: Arc<()>, } -impl SharedQueue { - pub fn push(&mut self, bytes: T) { self.queue.lock().expect("unable to lock shared queue").push_back(bytes) } - pub fn pop(&mut self) -> Option { self.queue.lock().expect("unable to lock shared queue").pop_front() } - pub fn drain_into(&mut self, dest: &mut Vec) { let mut lock = self.queue.lock().expect("unable to lock shared queue"); dest.extend(lock.drain(..)); } - pub fn is_empty(&self) -> bool { self.queue.lock().expect("unable to lock shared queue").is_empty() } - pub fn is_done(&self) -> bool { Arc::strong_count(&self.queue) == 1 } - pub fn new() -> Self { SharedQueue { queue: Arc::new(Mutex::new(VecDeque::new())) } } +impl SharedQueueSend { + pub fn push(&mut self, item: T) { + self.queue + .send((item, self.count.clone())) + .expect("unable to lock shared queue"); + } + pub fn is_empty(&self) -> bool { + Arc::strong_count(&self.count) == 1 + } + pub fn from(queue: Sender<(T, Arc<()>)>) -> Self { + SharedQueueSend { + queue, + count: Arc::new(()), + } + } } -impl Clone for SharedQueue { - fn clone(&self) -> Self { - SharedQueue { queue: self.queue.clone() } - } +pub struct SharedQueueRecv { + queue: Receiver<(T, Arc<()>)>, } + +impl SharedQueueRecv { + pub fn pop(&mut self) -> Option { + self.queue.try_recv().ok().map(|(item, _count)| item) + } + pub fn from(queue: Receiver<(T, Arc<()>)>) -> Self { SharedQueueRecv { queue } } +} \ No newline at end of file diff --git a/communication/src/initialize.rs b/communication/src/initialize.rs index 3f9e9b128..c6bafd1f1 100644 --- a/communication/src/initialize.rs +++ b/communication/src/initialize.rs @@ -7,6 +7,8 @@ use std::io::BufRead; use getopts; use std::sync::Arc; +use std::any::Any; + use allocator::{Thread, Process, Generic, GenericBuilder}; // use allocator::process_binary::ProcessBinaryBuilder; // use networking::initialize_networking; @@ -77,19 +79,19 @@ impl Configuration { type LogBuilder = Arc::logging::CommsLogger+Send+Sync>; -fn create_allocators(config: Configuration, logger: LogBuilder) -> Result,String> { +fn create_allocators(config: Configuration, logger: LogBuilder) -> Result<(Vec, Box),String> { match config { Configuration::Thread => { - Ok(vec![GenericBuilder::Thread(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()) + Ok((Process::new_vector(threads).into_iter().map(|x| GenericBuilder::Process(x)).collect(), Box::new(()))) // Ok(ProcessBinaryBuilder::new_vector(threads).into_iter().map(|x| GenericBuilder::ProcessBinary(x)).collect()) }, Configuration::Cluster(threads, process, addresses, report) => { - if let Ok(stuff) = initialize_networking(addresses, process, threads, report, logger) { - Ok(stuff.into_iter().map(|x| GenericBuilder::ZeroCopy(x)).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()) @@ -170,7 +172,7 @@ pub fn initializeT+Send+Sync+'static>( func: F, ) -> Result,String> { - let allocators = try!(create_allocators(config, log_sender)); + let (allocators, others) = try!(create_allocators(config, log_sender)); let logic = Arc::new(func); let mut guards = Vec::new(); @@ -185,12 +187,13 @@ pub fn initializeT+Send+Sync+'static>( .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 { @@ -205,7 +208,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"); } } From 372752e6370ddbbc0b0f203d08d4d79ffecb531a Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Fri, 3 Aug 2018 11:44:42 +0200 Subject: [PATCH 26/43] more reliable performance --- .../src/allocator/zero_copy/allocator.rs | 23 ++-- .../src/allocator/zero_copy/binary.rs | 107 ++++++++------- .../src/allocator/zero_copy/bytes_exchange.rs | 128 ++++++++++-------- .../src/allocator/zero_copy/bytes_slab.rs | 82 +++++++++++ communication/src/allocator/zero_copy/mod.rs | 1 + 5 files changed, 227 insertions(+), 114 deletions(-) create mode 100644 communication/src/allocator/zero_copy/bytes_slab.rs diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index f5f63fad3..c2557fcc9 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -219,15 +219,15 @@ impl Allocate for TcpAllocator { /// /// This pusher has a fixed MessageHeader, and access to a SharedByteBuffer which it uses to /// acquire buffers for serialization. -struct Pusher { +struct Pusher { header: MessageHeader, - sender: Rc>, + sender: Rc>, phantom: ::std::marker::PhantomData, } -impl Pusher { +impl Pusher { /// Creates a new `Pusher` from a header and shared byte buffer. - pub fn new(header: MessageHeader, sender: Rc>) -> Pusher { + pub fn new(header: MessageHeader, sender: Rc>) -> Pusher { Pusher { header: header, sender: sender, @@ -236,7 +236,7 @@ impl Pusher { } } -impl Push> for Pusher { +impl Push> for Pusher { #[inline] fn push(&mut self, element: &mut Option>) { if let Some(ref mut element) = *element { @@ -248,11 +248,14 @@ impl Push> for Pusher { // acquire byte buffer and write header, element. let mut borrow = self.sender.borrow_mut(); - let mut bytes = borrow.reserve(header.required_bytes()); - header.write_to(&mut bytes).expect("failed to write header!"); - - element.into_bytes(&mut bytes); - + { + 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()); } } } diff --git a/communication/src/allocator/zero_copy/binary.rs b/communication/src/allocator/zero_copy/binary.rs index f48769bc6..d5447b352 100644 --- a/communication/src/allocator/zero_copy/binary.rs +++ b/communication/src/allocator/zero_copy/binary.rs @@ -7,6 +7,8 @@ use bytes::arc::Bytes; use networking::MessageHeader; +use super::bytes_slab::BytesSlab; + /// Receives serialized data from a `Read`, for example the network. /// /// The `BinaryReceiver` repeatedly reads binary data from its reader into @@ -17,14 +19,16 @@ pub struct BinaryReceiver { worker_offset: usize, reader: TcpStream, // the generic reader. - buffer: Bytes, // current working buffer. - length: usize, // consumed buffer elements. targets: Vec>, // to process-local workers. log_sender: ::logging::CommsLogger, // logging stuffs. - in_progress: Vec>, // buffers shared with workers. - stash: Vec>, // reclaimed and resuable buffers. - size: usize, // current buffer allocation size. + buffer: BytesSlab, + + // buffer: Bytes, // current working buffer. + // length: usize, // consumed buffer elements. + // in_progress: Vec>, // buffers shared with workers. + // stash: Vec>, // reclaimed and resuable buffers. + // size: usize, // current buffer allocation size. } impl BinaryReceiver { @@ -38,44 +42,45 @@ impl BinaryReceiver { reader, targets, log_sender, - buffer: Bytes::from(vec![0u8; 1 << 20]), - length: 0, - in_progress: Vec::new(), - stash: Vec::new(), - size: 1 << 20, worker_offset, + buffer: BytesSlab::new(20), + // buffer: Bytes::from(vec![0u8; 1 << 20]), + // length: 0, + // in_progress: Vec::new(), + // stash: Vec::new(), + // size: 1 << 20, } } - // Retire `self.buffer` and acquire a new buffer of at least `self.size` bytes. - fn refresh_buffer(&mut self) { - - if self.stash.is_empty() { - for shared in self.in_progress.iter_mut() { - if let Some(bytes) = shared.take() { - match bytes.try_recover::>() { - Ok(mut vec) => { self.stash.push(vec); }, - Err(bytes) => { *shared = Some(bytes); }, - } - } - } - self.in_progress.retain(|x| x.is_some()); - } + // // Retire `self.buffer` and acquire a new buffer of at least `self.size` bytes. + // fn refresh_buffer(&mut self) { - let self_size = self.size; - self.stash.retain(|x| x.capacity() == self_size); + // if self.stash.is_empty() { + // for shared in self.in_progress.iter_mut() { + // if let Some(bytes) = shared.take() { + // match bytes.try_recover::>() { + // Ok(mut vec) => { self.stash.push(vec); }, + // Err(bytes) => { *shared = Some(bytes); }, + // } + // } + // } + // self.in_progress.retain(|x| x.is_some()); + // } - let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; self.size]); - let new_buffer = Bytes::from(new_buffer); - let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + // let self_size = self.size; + // self.stash.retain(|x| x.capacity() == self_size); - debug_assert!(self.length <= old_buffer.len()); - debug_assert!(self.length <= self.buffer.len()); + // let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; self.size]); + // let new_buffer = Bytes::from(new_buffer); + // let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); - self.buffer[.. self.length].copy_from_slice(&old_buffer[.. self.length]); + // debug_assert!(self.length <= old_buffer.len()); + // debug_assert!(self.length <= self.buffer.len()); - self.in_progress.push(Some(old_buffer)); - } + // self.buffer[.. self.length].copy_from_slice(&old_buffer[.. self.length]); + + // self.in_progress.push(Some(old_buffer)); + // } pub fn recv_loop(&mut self) { @@ -92,8 +97,12 @@ impl BinaryReceiver { while active { + self.buffer.ensure_capacity(1); + + assert!(!self.buffer.empty().is_empty()); + // Attempt to read some more bytes into self.buffer. - let read = match self.reader.read(&mut self.buffer[self.length ..]) { + let read = match self.reader.read(&mut self.buffer.empty()) { Ok(n) => n, Err(x) => { // We don't expect this, as socket closure results in Ok(0) reads. @@ -102,31 +111,33 @@ impl BinaryReceiver { }, }; - active = read > 0; + // println!("read bytes: {:?}", read); - self.length += read; + active = read > 0; + self.buffer.make_valid(read); + // self.length += read; // Consume complete messages from the front of self.buffer. - while let Some(header) = MessageHeader::try_read(&mut &self.buffer[.. self.length]) { + while let Some(header) = MessageHeader::try_read(&mut self.buffer.valid()) { // TODO: Consolidate message sequences sent to the same worker. let peeled_bytes = header.required_bytes(); - let bytes = self.buffer.extract_to(peeled_bytes); - self.length -= peeled_bytes; + let bytes = self.buffer.extract(peeled_bytes); + // self.length -= peeled_bytes; self.targets[header.target - self.worker_offset] .send(bytes) .expect("Worker queue unavailable in recv_loop"); } - // If our buffer is full we should copy it to a new buffer. - if self.length == self.buffer.len() { - // If full and not complete, we must increase the size. - if self.length == self.size { - self.size *= 2; - } - self.refresh_buffer(); - } + // // If our buffer is full we should copy it to a new buffer. + // if self.length == self.buffer.len() { + // // If full and not complete, we must increase the size. + // if self.length == self.size { + // self.size *= 2; + // } + // self.refresh_buffer(); + // } } // println!("RECVER EXITING"); diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index c0ee8cac5..812eff638 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -2,6 +2,7 @@ use std::ops::DerefMut; use bytes::arc::Bytes; use super::shared_queue::{SharedQueueSend, SharedQueueRecv}; +use super::bytes_slab::BytesSlab; /// A type that can allocate send and receive endpoints for byte exchanges. /// @@ -40,38 +41,42 @@ pub trait RecvEndpoint { pub struct BytesSendEndpoint { send: SharedQueueSend, - in_progress: Vec>, - buffer: Vec, - stash: Vec>, - default_size: usize, + buffer: BytesSlab, + // in_progress: Vec>, + // buffer: Vec, + // stash: Vec>, + // default_size: usize, } impl BytesSendEndpoint { - /// Attempts to recover in-use buffers once uniquely owned. - fn harvest_shared(&mut self) { - for shared in self.in_progress.iter_mut() { - if let Some(bytes) = shared.take() { - match bytes.try_recover::>() { - Ok(mut vec) => { vec.clear(); self.stash.push(vec); }, - Err(bytes) => { *shared = Some(bytes); }, - } - } - } - self.in_progress.retain(|x| x.is_some()); - } + // /// Attempts to recover in-use buffers once uniquely owned. + // fn harvest_shared(&mut self) { + // for shared in self.in_progress.iter_mut() { + // if let Some(bytes) = shared.take() { + // match bytes.try_recover::>() { + // Ok(mut vec) => { vec.clear(); self.stash.push(vec); }, + // Err(bytes) => { *shared = Some(bytes); }, + // } + // } + // } + // self.in_progress.retain(|x| x.is_some()); + // } /// Moves `self.buffer` into `self.send`, replaces with empty buffer. fn send_buffer(&mut self) { - if self.buffer.len() > 0 { + let valid_len = self.buffer.valid().len(); + if valid_len > 0 { + + let to_send = self.buffer.extract(valid_len); - let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); - let buffer_len = buffer.len(); - let mut bytes = Bytes::from(buffer); - let to_send = bytes.extract_to(buffer_len); + // let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); + // let buffer_len = buffer.len(); + // let mut bytes = Bytes::from(buffer); + // let to_send = bytes.extract_to(buffer_len); self.send.push(to_send); - self.in_progress.push(Some(bytes)); + // self.in_progress.push(Some(bytes)); } } @@ -79,58 +84,69 @@ impl BytesSendEndpoint { pub fn new(queue: SharedQueueSend) -> Self { BytesSendEndpoint { send: queue, - in_progress: Vec::new(), - buffer: Vec::new(), - stash: Vec::new(), - default_size: 1 << 20, + buffer: BytesSlab::new(20), + // in_progress: Vec::new(), + // buffer: Vec::new(), + // stash: Vec::new(), + // default_size: 1 << 20, } } } -impl SendEndpoint for BytesSendEndpoint { - - type SendBuffer = Vec; - - fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer { +impl BytesSendEndpoint { - // println!("reserving {:?} bytes", capacity); - if self.send.is_empty() { - self.send_buffer(); - } + pub fn make_valid(&mut self, bytes: usize) { + self.buffer.make_valid(bytes); + self.send_buffer(); + } + pub fn reserve(&mut self, capacity: usize) -> &mut [u8] { - // If we don't have enough capacity in `self.buffer`... - if self.buffer.capacity() < capacity + self.buffer.len() { + if self.buffer.empty().len() < capacity { self.send_buffer(); - if capacity > self.default_size { - self.buffer = Vec::with_capacity(capacity); - } - else { - if self.stash.is_empty() { - // Attempt to recover shared buffers. - self.harvest_shared(); - } - self.buffer = self.stash.pop().unwrap_or_else(|| Vec::with_capacity(self.default_size)) - } + self.buffer.ensure_capacity(capacity); } - &mut self.buffer + assert!(self.buffer.empty().len() >= capacity); + + // // println!("reserving {:?} bytes", capacity); + // // if self.send.is_empty() { + // // self.send_buffer(); + // // } + + // // If we don't have enough capacity in `self.buffer`... + // if self.buffer.capacity() < capacity + self.buffer.len() { + // self.send_buffer(); + // if capacity > self.default_size { + // self.buffer = Vec::with_capacity(capacity); + // } + // else { + // if self.stash.is_empty() { + // // Attempt to recover shared buffers. + // self.harvest_shared(); + // } + // self.buffer = self.stash.pop().unwrap_or_else(|| Vec::with_capacity(self.default_size)) + // } + // } + + // self.buffer.make_valid(capacity); + self.buffer.empty() } - fn publish(&mut self) { - self.harvest_shared(); - if self.send.is_empty() { + pub fn publish(&mut self) { + // self.harvest_shared(); + // if self.send.is_empty() { self.send_buffer(); - } - else { - // println!("delaying publication!"); - } + // } + // else { + // // println!("delaying publication!"); + // } } } impl Drop for BytesSendEndpoint { fn drop(&mut self) { self.send_buffer(); - assert!(self.buffer.is_empty()); + // assert!(self.buffer.is_empty()); } } 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..dd156c528 --- /dev/null +++ b/communication/src/allocator/zero_copy/bytes_slab.rs @@ -0,0 +1,82 @@ +use bytes::arc::Bytes; + +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 { + + 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, + } + } + + // pub fn is_full(&self) -> bool { self.valid == self.buffer.len() } + + // pub fn bytes(&mut self) -> &mut Bytes { &mut self.buffer } + + pub fn empty(&mut self) -> &mut [u8] { + &mut self.buffer[self.valid..] + } + + pub fn valid(&mut self) -> &[u8] { + &mut self.buffer[..self.valid] + } + + pub fn make_valid(&mut self, bytes: usize) { + self.valid += bytes; + } + + pub fn extract(&mut self, bytes: usize) -> Bytes { + debug_assert!(bytes <= self.valid); + self.valid -= bytes; + self.buffer.extract_to(bytes) + } + + // Retire `self.buffer` and acquire a new buffer of at least `self.size` bytes. + pub fn ensure_capacity(&mut self, capacity: usize) { + + // Increase allocation if insufficient. + while self.valid + capacity > (1 << self.shift) { + self.shift += 1; + self.stash.clear(); + } + + // Attempt to reclaim shared slices. + if self.stash.is_empty() { + for shared in self.in_progress.iter_mut() { + if let Some(bytes) = shared.take() { + match bytes.try_recover::>() { + Ok(mut vec) => { + if vec.len() == (1 << self.shift) { + self.stash.push(vec); + } + }, + Err(bytes) => { *shared = Some(bytes); }, + } + } + } + self.in_progress.retain(|x| x.is_some()); + } + + let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; 1 << self.shift].into_boxed_slice()); + let new_buffer = Bytes::from(new_buffer); + let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); + + if !(self.buffer.len() == (1 << self.shift)) { + println!("len: {:?}, expected: {:?}", self.buffer.len(), (1 << self.shift)); + } + + 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/mod.rs b/communication/src/allocator/zero_copy/mod.rs index d9f7d34e4..a4ba53d77 100644 --- a/communication/src/allocator/zero_copy/mod.rs +++ b/communication/src/allocator/zero_copy/mod.rs @@ -1,3 +1,4 @@ +pub mod bytes_slab; pub mod shared_queue; pub mod bytes_exchange; pub mod binary; From 62629f6261243f27a568d30ce9cd3a1bccb65a52 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 13 Aug 2018 08:28:07 +0200 Subject: [PATCH 27/43] unknown --- bytes/src/lib.rs | 17 +++++ .../src/allocator/zero_copy/allocator.rs | 10 +-- .../src/allocator/zero_copy/binary.rs | 72 ++----------------- .../src/allocator/zero_copy/bytes_exchange.rs | 62 +--------------- .../src/allocator/zero_copy/bytes_slab.rs | 19 +++-- .../src/allocator/zero_copy/shared_queue.rs | 34 ++++++--- 6 files changed, 59 insertions(+), 155 deletions(-) diff --git a/bytes/src/lib.rs b/bytes/src/lib.rs index b59899c07..cc563bc99 100644 --- a/bytes/src/lib.rs +++ b/bytes/src/lib.rs @@ -206,6 +206,23 @@ 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. + 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 + } + } } impl Deref for Bytes { diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index c2557fcc9..0bbafc056 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -1,7 +1,7 @@ use std::rc::Rc; use std::cell::RefCell; use std::collections::VecDeque; -use std::sync::{Arc, mpsc::{channel, Sender, Receiver}}; +use std::sync::mpsc::{channel, Sender, Receiver}; use bytes::arc::Bytes; @@ -10,7 +10,6 @@ use networking::MessageHeader; use {Allocate, Data, Push, Pull}; use allocator::{Message, Process}; -use super::bytes_exchange::SendEndpoint; use super::bytes_exchange::BytesSendEndpoint; use super::shared_queue::SharedQueueSend; @@ -24,7 +23,7 @@ pub struct TcpBuilder { inner: Process, index: usize, // number out of peers peers: usize, // number of peer allocators. - sends: Vec)>>, // for pushing bytes at remote processes. + sends: Vec>, // for pushing bytes at remote processes. recvs: Receiver, // for pulling bytes from remote processes. } @@ -35,7 +34,7 @@ impl TcpBuilder { pub fn new_vector( my_process: usize, threads: usize, - processes: usize) -> (Vec, Vec)>>, Vec>) { + processes: usize) -> (Vec, Vec>, Vec>) { let mut l2r_send = Vec::new(); let mut l2r_recv = Vec::new(); @@ -54,9 +53,6 @@ impl TcpBuilder { l2r_recv.push(recv); } - // let mut to_comms = (0 .. processes).map(|_| channel()).collect::>(); - // let mut from_comms = (0 .. threads).map(|_| channel()).collect::>(); - let builders = Process::new_vector(threads) .into_iter() diff --git a/communication/src/allocator/zero_copy/binary.rs b/communication/src/allocator/zero_copy/binary.rs index d5447b352..535ecab52 100644 --- a/communication/src/allocator/zero_copy/binary.rs +++ b/communication/src/allocator/zero_copy/binary.rs @@ -1,8 +1,7 @@ use std::io::{Read, Write}; -use std::sync::{Arc, mpsc::{Sender, Receiver}}; +use std::sync::mpsc::{Sender, Receiver}; use std::net::TcpStream; - use bytes::arc::Bytes; use networking::MessageHeader; @@ -23,12 +22,6 @@ pub struct BinaryReceiver { log_sender: ::logging::CommsLogger, // logging stuffs. buffer: BytesSlab, - - // buffer: Bytes, // current working buffer. - // length: usize, // consumed buffer elements. - // in_progress: Vec>, // buffers shared with workers. - // stash: Vec>, // reclaimed and resuable buffers. - // size: usize, // current buffer allocation size. } impl BinaryReceiver { @@ -44,44 +37,9 @@ impl BinaryReceiver { log_sender, worker_offset, buffer: BytesSlab::new(20), - // buffer: Bytes::from(vec![0u8; 1 << 20]), - // length: 0, - // in_progress: Vec::new(), - // stash: Vec::new(), - // size: 1 << 20, } } - // // Retire `self.buffer` and acquire a new buffer of at least `self.size` bytes. - // fn refresh_buffer(&mut self) { - - // if self.stash.is_empty() { - // for shared in self.in_progress.iter_mut() { - // if let Some(bytes) = shared.take() { - // match bytes.try_recover::>() { - // Ok(mut vec) => { self.stash.push(vec); }, - // Err(bytes) => { *shared = Some(bytes); }, - // } - // } - // } - // self.in_progress.retain(|x| x.is_some()); - // } - - // let self_size = self.size; - // self.stash.retain(|x| x.capacity() == self_size); - - // let new_buffer = self.stash.pop().unwrap_or_else(|| vec![0; self.size]); - // let new_buffer = Bytes::from(new_buffer); - // let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); - - // debug_assert!(self.length <= old_buffer.len()); - // debug_assert!(self.length <= self.buffer.len()); - - // self.buffer[.. self.length].copy_from_slice(&old_buffer[.. self.length]); - - // self.in_progress.push(Some(old_buffer)); - // } - pub fn recv_loop(&mut self) { // Each loop iteration adds to `self.Bytes` and consumes all complete messages. @@ -93,8 +51,6 @@ impl BinaryReceiver { // can be recovered once all readers have read what they need to. let mut active = true; - let mut count = 0; - while active { self.buffer.ensure_capacity(1); @@ -111,11 +67,8 @@ impl BinaryReceiver { }, }; - // println!("read bytes: {:?}", read); - active = read > 0; self.buffer.make_valid(read); - // self.length += read; // Consume complete messages from the front of self.buffer. while let Some(header) = MessageHeader::try_read(&mut self.buffer.valid()) { @@ -123,23 +76,12 @@ impl BinaryReceiver { // TODO: Consolidate message sequences sent to the same worker. let peeled_bytes = header.required_bytes(); let bytes = self.buffer.extract(peeled_bytes); - // self.length -= peeled_bytes; self.targets[header.target - self.worker_offset] .send(bytes) .expect("Worker queue unavailable in recv_loop"); } - - // // If our buffer is full we should copy it to a new buffer. - // if self.length == self.buffer.len() { - // // If full and not complete, we must increase the size. - // if self.length == self.size { - // self.size *= 2; - // } - // self.refresh_buffer(); - // } } - // println!("RECVER EXITING"); } } @@ -153,32 +95,28 @@ impl BinaryReceiver { // structure in charge of sending data to a Writer, for example the network. pub struct BinarySender { writer: ::std::io::BufWriter, - source: Receiver<(Bytes, Arc<()>)>, + source: Receiver, log_sender: ::logging::CommsLogger, } impl BinarySender { - pub fn new(writer: ::std::io::BufWriter, source: Receiver<(Bytes, Arc<()>)>, log_sender: ::logging::CommsLogger) -> BinarySender { + pub fn new(writer: ::std::io::BufWriter, source: Receiver, log_sender: ::logging::CommsLogger) -> BinarySender { BinarySender { writer, source, log_sender } } pub fn send_loop(&mut self) { - let mut count = 0; let mut stash = Vec::new(); - while let Ok((bytes, _count)) = self.source.recv() { + while let Ok(bytes) = self.source.recv() { stash.push(bytes); - while let Ok((bytes, _count)) = self.source.try_recv() { + while let Ok(bytes) = self.source.try_recv() { stash.push(bytes); } // TODO: Could do scatter/gather write here. for bytes in stash.drain(..) { - count += bytes.len(); - // println!("Sending bytes: {:?}", bytes.len()); self.writer.write_all(&bytes[..]).expect("Write failure in send_loop."); - // println!("Sent bytes: {:?}", count); } // TODO: Could delay this until a self.source.recv() would block. self.writer.flush().expect("Failed to flush writer."); diff --git a/communication/src/allocator/zero_copy/bytes_exchange.rs b/communication/src/allocator/zero_copy/bytes_exchange.rs index 812eff638..7b60acc47 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -42,41 +42,16 @@ pub trait RecvEndpoint { pub struct BytesSendEndpoint { send: SharedQueueSend, buffer: BytesSlab, - // in_progress: Vec>, - // buffer: Vec, - // stash: Vec>, - // default_size: usize, } impl BytesSendEndpoint { - // /// Attempts to recover in-use buffers once uniquely owned. - // fn harvest_shared(&mut self) { - // for shared in self.in_progress.iter_mut() { - // if let Some(bytes) = shared.take() { - // match bytes.try_recover::>() { - // Ok(mut vec) => { vec.clear(); self.stash.push(vec); }, - // Err(bytes) => { *shared = Some(bytes); }, - // } - // } - // } - // self.in_progress.retain(|x| x.is_some()); - // } /// 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 { - - let to_send = self.buffer.extract(valid_len); - - // let buffer = ::std::mem::replace(&mut self.buffer, Vec::new()); - // let buffer_len = buffer.len(); - // let mut bytes = Bytes::from(buffer); - // let to_send = bytes.extract_to(buffer_len); - - self.send.push(to_send); - // self.in_progress.push(Some(bytes)); + self.send.push(self.buffer.extract(valid_len)); } } @@ -85,10 +60,6 @@ impl BytesSendEndpoint { BytesSendEndpoint { send: queue, buffer: BytesSlab::new(20), - // in_progress: Vec::new(), - // buffer: Vec::new(), - // stash: Vec::new(), - // default_size: 1 << 20, } } } @@ -107,46 +78,17 @@ impl BytesSendEndpoint { } assert!(self.buffer.empty().len() >= capacity); - - // // println!("reserving {:?} bytes", capacity); - // // if self.send.is_empty() { - // // self.send_buffer(); - // // } - - // // If we don't have enough capacity in `self.buffer`... - // if self.buffer.capacity() < capacity + self.buffer.len() { - // self.send_buffer(); - // if capacity > self.default_size { - // self.buffer = Vec::with_capacity(capacity); - // } - // else { - // if self.stash.is_empty() { - // // Attempt to recover shared buffers. - // self.harvest_shared(); - // } - // self.buffer = self.stash.pop().unwrap_or_else(|| Vec::with_capacity(self.default_size)) - // } - // } - - // self.buffer.make_valid(capacity); self.buffer.empty() } pub fn publish(&mut self) { - // self.harvest_shared(); - // if self.send.is_empty() { - self.send_buffer(); - // } - // else { - // // println!("delaying publication!"); - // } + self.send_buffer(); } } impl Drop for BytesSendEndpoint { fn drop(&mut self) { self.send_buffer(); - // assert!(self.buffer.is_empty()); } } diff --git a/communication/src/allocator/zero_copy/bytes_slab.rs b/communication/src/allocator/zero_copy/bytes_slab.rs index dd156c528..382fa2914 100644 --- a/communication/src/allocator/zero_copy/bytes_slab.rs +++ b/communication/src/allocator/zero_copy/bytes_slab.rs @@ -3,7 +3,7 @@ use bytes::arc::Bytes; pub struct BytesSlab { buffer: Bytes, // current working buffer. in_progress: Vec>, // buffers shared with workers. - stash: Vec>, // reclaimed and resuable buffers. + stash: Vec, // reclaimed and resuable buffers. shift: usize, // current buffer allocation size. valid: usize, // buffer[..valid] are valid bytes. } @@ -54,22 +54,19 @@ impl BytesSlab { // Attempt to reclaim shared slices. if self.stash.is_empty() { for shared in self.in_progress.iter_mut() { - if let Some(bytes) = shared.take() { - match bytes.try_recover::>() { - Ok(mut vec) => { - if vec.len() == (1 << self.shift) { - self.stash.push(vec); - } - }, - Err(bytes) => { *shared = Some(bytes); }, + 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(|| vec![0; 1 << self.shift].into_boxed_slice()); - let new_buffer = Bytes::from(new_buffer); + 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); if !(self.buffer.len() == (1 << self.shift)) { diff --git a/communication/src/allocator/zero_copy/shared_queue.rs b/communication/src/allocator/zero_copy/shared_queue.rs index f4cde670b..7a3500a0d 100644 --- a/communication/src/allocator/zero_copy/shared_queue.rs +++ b/communication/src/allocator/zero_copy/shared_queue.rs @@ -1,34 +1,48 @@ -use std::sync::{Arc, mpsc::{Sender, Receiver}}; +use std::sync::mpsc::{Sender, Receiver}; + +// pub struct SharedQueue { +// queue: Arc<(Mutex>, CondVar)>, +// } + +// impl SharedQueue { +// pub fn push(&mut self, item: T) { +// let lock = queue.0.lock(); +// lock.push_back(item); +// queue.1.notify_all(); +// } + +// pub fn pop(&mut self) -> Option { + +// } +// } pub struct SharedQueueSend { - queue: Sender<(T, Arc<()>)>, - count: Arc<()>, + queue: Sender, } impl SharedQueueSend { pub fn push(&mut self, item: T) { self.queue - .send((item, self.count.clone())) + .send(item) .expect("unable to lock shared queue"); } pub fn is_empty(&self) -> bool { - Arc::strong_count(&self.count) == 1 + unimplemented!() } - pub fn from(queue: Sender<(T, Arc<()>)>) -> Self { + pub fn from(queue: Sender) -> Self { SharedQueueSend { queue, - count: Arc::new(()), } } } pub struct SharedQueueRecv { - queue: Receiver<(T, Arc<()>)>, + queue: Receiver, } impl SharedQueueRecv { pub fn pop(&mut self) -> Option { - self.queue.try_recv().ok().map(|(item, _count)| item) + self.queue.try_recv().ok() } - pub fn from(queue: Receiver<(T, Arc<()>)>) -> Self { SharedQueueRecv { queue } } + pub fn from(queue: Receiver) -> Self { SharedQueueRecv { queue } } } \ No newline at end of file From 3392352194270d1ddbd8d9608050b19c8c9f00ad Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 13 Aug 2018 16:11:14 +0200 Subject: [PATCH 28/43] end with empty message --- .../src/allocator/zero_copy/allocator.rs | 1 + .../src/allocator/zero_copy/binary.rs | 205 ++++++++---------- .../src/allocator/zero_copy/initialize.rs | 8 +- communication/src/networking.rs | 2 +- 4 files changed, 100 insertions(+), 116 deletions(-) diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index 0bbafc056..9812d05a6 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -241,6 +241,7 @@ impl Push> for Pusher { 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(); diff --git a/communication/src/allocator/zero_copy/binary.rs b/communication/src/allocator/zero_copy/binary.rs index 535ecab52..a7cc1af02 100644 --- a/communication/src/allocator/zero_copy/binary.rs +++ b/communication/src/allocator/zero_copy/binary.rs @@ -8,127 +8,112 @@ use networking::MessageHeader; use super::bytes_slab::BytesSlab; -/// Receives serialized data from a `Read`, for example the network. +/// Repeatedly reads from a TcpStream and carves out messages. /// -/// The `BinaryReceiver` repeatedly reads binary data from its reader into -/// a binary Bytes slice which can be broken off and handed to recipients as -/// messages become complete. -pub struct BinaryReceiver { - - worker_offset: usize, - - reader: TcpStream, // the generic reader. - targets: Vec>, // to process-local workers. - log_sender: ::logging::CommsLogger, // logging stuffs. - - buffer: BytesSlab, -} - -impl BinaryReceiver { - - pub fn new( - reader: TcpStream, - targets: Vec>, - worker_offset: usize, - log_sender: ::logging::CommsLogger) -> BinaryReceiver { - BinaryReceiver { - reader, - targets, - log_sender, - worker_offset, - buffer: BytesSlab::new(20), - } - } - - pub fn recv_loop(&mut self) { - - // 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 { - - self.buffer.ensure_capacity(1); - - assert!(!self.buffer.empty().is_empty()); - - // Attempt to read some more bytes into self.buffer. - let read = match self.reader.read(&mut self.buffer.empty()) { - Ok(n) => n, - Err(x) => { - // We don't expect this, as socket closure results in Ok(0) reads. - println!("Error: {:?}", x); - 0 - }, - }; - - active = read > 0; - self.buffer.make_valid(read); - - // Consume complete messages from the front of self.buffer. - while let Some(header) = MessageHeader::try_read(&mut self.buffer.valid()) { - - // TODO: Consolidate message sequences sent to the same worker. - let peeled_bytes = header.required_bytes(); - let bytes = self.buffer.extract(peeled_bytes); - - self.targets[header.target - self.worker_offset] +/// 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, + targets: Vec>, + worker_offset: usize, + log_sender: ::logging::CommsLogger) +{ + let mut buffer = BytesSlab::new(20); + + // 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(&mut 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 { + targets[header.target - worker_offset] .send(bytes) .expect("Worker queue unavailable in recv_loop"); } + 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."); + } + } } - // println!("RECVER EXITING"); } + // println!("RECVER EXITING"); } -// impl Drop for BinaryReceiver { -// fn drop(&mut self) { -// self.reader.shutdown(::std::net::Shutdown::Read).expect("Read shutdown failed"); -// } -// } - -// structure in charge of sending data to a Writer, for example the network. -pub struct BinarySender { - writer: ::std::io::BufWriter, - source: Receiver, - log_sender: ::logging::CommsLogger, -} - -impl BinarySender { - pub fn new(writer: ::std::io::BufWriter, source: Receiver, log_sender: ::logging::CommsLogger) -> BinarySender { - BinarySender { writer, source, log_sender } - } - - pub fn send_loop(&mut self) { - - let mut stash = Vec::new(); - - while let Ok(bytes) = self.source.recv() { +/// 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( + mut writer: ::std::io::BufWriter, + source: Receiver, + log_sender: ::logging::CommsLogger) +{ + + let mut stash = Vec::new(); + + while let Ok(bytes) = source.recv() { + stash.push(bytes); + while let Ok(bytes) = source.try_recv() { stash.push(bytes); - while let Ok(bytes) = self.source.try_recv() { - stash.push(bytes); - } - - // TODO: Could do scatter/gather write here. - for bytes in stash.drain(..) { - self.writer.write_all(&bytes[..]).expect("Write failure in send_loop."); - } - // TODO: Could delay this until a self.source.recv() would block. - self.writer.flush().expect("Failed to flush writer."); } - // println!("SENDER EXITING"); + // TODO: Could do scatter/gather write here. + for bytes in stash.drain(..) { + writer.write_all(&bytes[..]).expect("Write failure in send_loop."); + } + // TODO: Could delay this until a self.source.recv() would block. + writer.flush().expect("Failed to flush writer."); } -} -impl Drop for BinarySender { - fn drop(&mut self) { - self.writer.flush().expect("Failed to flush writer."); - self.writer.get_mut().shutdown(::std::net::Shutdown::Write).expect("Write shutdown failed"); - } + // 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/allocator/zero_copy/initialize.rs b/communication/src/allocator/zero_copy/initialize.rs index 92178ce0d..052f93645 100644 --- a/communication/src/allocator/zero_copy/initialize.rs +++ b/communication/src/allocator/zero_copy/initialize.rs @@ -1,7 +1,7 @@ use std::sync::Arc; -use super::binary::{BinarySender, BinaryReceiver}; +use super::binary::{send_loop, recv_loop}; use super::allocator::TcpBuilder; pub struct CommsGuard { @@ -64,8 +64,7 @@ pub fn initialize_networking( }); let stream = ::std::io::BufWriter::with_capacity(1 << 20, stream); - BinarySender::new(stream, remote_recv, log_sender) - .send_loop() + send_loop(stream, remote_recv, log_sender); })?; send_guards.push(join_guard); @@ -84,8 +83,7 @@ pub fn initialize_networking( sender: false, remote: Some(index), }); - BinaryReceiver::new(stream, remote_sends, threads * my_index, log_sender) - .recv_loop() + recv_loop(stream, remote_sends, threads * my_index, log_sender); })?; recv_guards.push(join_guard); diff --git a/communication/src/networking.rs b/communication/src/networking.rs index 479815113..9cfe84ef6 100644 --- a/communication/src/networking.rs +++ b/communication/src/networking.rs @@ -32,7 +32,7 @@ impl MessageHeader { // returns a header when there is enough supporting data #[inline(always)] pub fn try_read(bytes: &mut &[u8]) -> Option { - if bytes.len() > size_of::() { + if bytes.len() >= size_of::() { // capture original in case we need to rewind let original = *bytes; From faf245c354e906d579c1561a39fa2e23eea3c8e2 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 23 Aug 2018 19:26:17 +0200 Subject: [PATCH 29/43] re-export communication --- src/dataflow/channels/pullers/counter.rs | 2 +- src/dataflow/channels/pushers/counter.rs | 2 +- src/dataflow/channels/pushers/exchange.rs | 3 ++- src/dataflow/operators/enterleave.rs | 3 ++- src/dataflow/operators/feedback.rs | 3 ++- src/dataflow/operators/input.rs | 3 ++- src/dataflow/scopes/child.rs | 4 ++-- src/dataflow/scopes/root.rs | 4 ++-- src/dataflow/stream.rs | 2 +- src/lib.rs | 6 +++++- src/progress/broadcast.rs | 2 +- 11 files changed, 21 insertions(+), 13 deletions(-) diff --git a/src/dataflow/channels/pullers/counter.rs b/src/dataflow/channels/pullers/counter.rs index 76bad2a31..5458ab7e0 100644 --- a/src/dataflow/channels/pullers/counter.rs +++ b/src/dataflow/channels/pullers/counter.rs @@ -5,7 +5,7 @@ use std::cell::RefCell; 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>> { diff --git a/src/dataflow/channels/pushers/counter.rs b/src/dataflow/channels/pushers/counter.rs index daed38063..0d9c8bbef 100644 --- a/src/dataflow/channels/pushers/counter.rs +++ b/src/dataflow/channels/pushers/counter.rs @@ -5,7 +5,7 @@ use std::cell::RefCell; use progress::ChangeBatch; use dataflow::channels::Bundle; -use Push; +use communication::Push; /// A wrapper which updates shared `produced` based on the number of records pushed. pub struct Counter>> { diff --git a/src/dataflow/channels/pushers/exchange.rs b/src/dataflow/channels/pushers/exchange.rs index 3d2c65ac4..575d5a74e 100644 --- a/src/dataflow/channels/pushers/exchange.rs +++ b/src/dataflow/channels/pushers/exchange.rs @@ -1,6 +1,7 @@ //! The exchange pattern distributes pushed data between many target pushees. -use {Push, Data}; +use Data; +use communication::Push; use dataflow::channels::{Bundle, Message}; // TODO : Software write combining diff --git a/src/dataflow/operators/enterleave.rs b/src/dataflow/operators/enterleave.rs index ee9d1048d..725f6c479 100644 --- a/src/dataflow/operators/enterleave.rs +++ b/src/dataflow/operators/enterleave.rs @@ -26,7 +26,8 @@ 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::{Bundle, Message}; diff --git a/src/dataflow/operators/feedback.rs b/src/dataflow/operators/feedback.rs index 3cbe7c89e..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; diff --git a/src/dataflow/operators/input.rs b/src/dataflow/operators/input.rs index db8e70c56..9cec262bf 100644 --- a/src/dataflow/operators/input.rs +++ b/src/dataflow/operators/input.rs @@ -12,7 +12,8 @@ use progress::timestamp::RootTimestamp; use progress::nested::product::Product; use timely_communication::Allocate; -use {Data, Push}; +use Data; +use communication::Push; use dataflow::channels::Message; use dataflow::channels::pushers::{Tee, Counter}; diff --git a/src/dataflow/scopes/child.rs b/src/dataflow/scopes/child.rs index 4d21a1535..6f2d48892 100644 --- a/src/dataflow/scopes/child.rs +++ b/src/dataflow/scopes/child.rs @@ -5,8 +5,8 @@ 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 timely_communication::{Allocate, Data}; +use communication::{Allocate, Data, Push, Pull}; use logging::Logger; use super::{ScopeParent, Scope}; diff --git a/src/dataflow/scopes/root.rs b/src/dataflow/scopes/root.rs index 292039e1d..13e91e607 100644 --- a/src/dataflow/scopes/root.rs +++ b/src/dataflow/scopes/root.rs @@ -8,8 +8,8 @@ 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 timely_communication::{Allocate, Data}; +use communication::{Allocate, Data, Push, Pull}; use super::{ScopeParent, Child}; diff --git a/src/dataflow/stream.rs b/src/dataflow/stream.rs index 8d10ef4f2..223917a8e 100644 --- a/src/dataflow/stream.rs +++ b/src/dataflow/stream.rs @@ -6,7 +6,7 @@ use progress::nested::subgraph::{Source, Target}; -use Push; +use communication::Push; use dataflow::Scope; use dataflow::channels::pushers::tee::TeeHelper; use dataflow::channels::Bundle; diff --git a/src/lib.rs b/src/lib.rs index 0db43d6fc..237219532 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -65,9 +65,13 @@ 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; +/// Re-export of the `timely_communication` crate. +pub mod communication { + pub use timely_communication::*; +} + pub mod progress; pub mod dataflow; pub mod execute; diff --git a/src/progress/broadcast.rs b/src/progress/broadcast.rs index 12d8e1192..9ce1fc21b 100644 --- a/src/progress/broadcast.rs +++ b/src/progress/broadcast.rs @@ -5,7 +5,7 @@ use timely_communication::allocator::Message; use progress::Timestamp; use progress::ChangeBatch; use timely_communication::Allocate; -use {Push, Pull}; +use communication::{Push, Pull}; use logging::Logger; From 8f29c3f42ba109c18983458d5a1dc96c3a967574 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 23 Aug 2018 19:42:13 +0200 Subject: [PATCH 30/43] oops --- src/dataflow/operators/probe.rs | 2 +- src/lib.rs | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/dataflow/operators/probe.rs b/src/dataflow/operators/probe.rs index 1df70a077..e5f48fb42 100644 --- a/src/dataflow/operators/probe.rs +++ b/src/dataflow/operators/probe.rs @@ -181,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/lib.rs b/src/lib.rs index 237219532..8bb2c9932 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -67,6 +67,8 @@ extern crate bytes; pub use execute::{execute, execute_logging, execute_from_args, execute_from_args_logging, example}; pub use order::PartialOrder; +pub use timely_communication::Configuration; + /// Re-export of the `timely_communication` crate. pub mod communication { pub use timely_communication::*; From 45b98f17474f5b6a6b8c74ee707340c6cddb1a82 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Fri, 24 Aug 2018 08:22:37 +0200 Subject: [PATCH 31/43] zerocopy process allocator --- communication/src/allocator/generic.rs | 9 +- communication/src/allocator/mod.rs | 2 - .../src/allocator/zero_copy/allocator.rs | 106 +--------- .../allocator/zero_copy/allocator_process.rs | 181 ++++++++++++++++++ .../src/allocator/zero_copy/bytes_exchange.rs | 141 ++++++++------ .../src/allocator/zero_copy/bytes_slab.rs | 8 - communication/src/allocator/zero_copy/mod.rs | 6 +- .../src/allocator/zero_copy/push_pull.rs | 133 +++++++++++++ communication/src/initialize.rs | 5 +- src/lib.rs | 7 +- 10 files changed, 418 insertions(+), 180 deletions(-) create mode 100644 communication/src/allocator/zero_copy/allocator_process.rs create mode 100644 communication/src/allocator/zero_copy/push_pull.rs diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index 8bacd138f..b762672cc 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -4,7 +4,10 @@ //! for example closures whose type arguments must be specified. use allocator::{Allocate, Message, Thread, Process, Binary}; -use allocator::process_binary::{ProcessBinary, ProcessBinaryBuilder}; +// use allocator::process_binary::{ProcessBinary, ProcessBinaryBuilder}; + +use allocator::zero_copy::allocator_process::ProcessAllocator; +use allocator::zero_copy::allocator_process::ProcessBuilder; use allocator::zero_copy::allocator::TcpAllocator as ZeroCopyAllocator; use allocator::zero_copy::allocator::TcpBuilder as ZeroCopyBuilder; @@ -18,7 +21,7 @@ pub enum Generic { Thread(Thread), Process(Process), Binary(Binary), - ProcessBinary(ProcessBinary<::allocator::process_binary::vec::VecBytesExchange>), + ProcessBinary(ProcessAllocator), ZeroCopy(ZeroCopyAllocator), } @@ -95,7 +98,7 @@ pub enum GenericBuilder { Thread(Thread), Process(Process), Binary(Binary), - ProcessBinary(ProcessBinaryBuilder<::allocator::process_binary::vec::VecBytesExchange>), + ProcessBinary(ProcessBuilder), ZeroCopy(ZeroCopyBuilder), } diff --git a/communication/src/allocator/mod.rs b/communication/src/allocator/mod.rs index 88505e323..412ef437c 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -12,8 +12,6 @@ pub mod thread; pub mod process; pub mod binary; pub mod generic; -pub mod process_binary; -// pub mod process_binary_neu; pub mod zero_copy; diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index 9812d05a6..b1489237c 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -10,8 +10,8 @@ use networking::MessageHeader; use {Allocate, Data, Push, Pull}; use allocator::{Message, Process}; -use super::bytes_exchange::BytesSendEndpoint; -use super::shared_queue::SharedQueueSend; +use super::bytes_exchange::{BytesPull, SendEndpoint}; +use super::push_pull::{Pusher, PullerInner}; /// Builds an instance of a TcpAllocator. /// @@ -76,8 +76,7 @@ impl TcpBuilder { let mut sends = Vec::new(); for send in self.sends.into_iter() { - let shared = SharedQueueSend::from(send); - let sendpoint = BytesSendEndpoint::new(shared); + let sendpoint = SendEndpoint::new(send); sends.push(Rc::new(RefCell::new(sendpoint))); } @@ -103,7 +102,7 @@ pub struct TcpAllocator { allocated: usize, // indicates how many channels have been allocated (locally). // sending, receiving, and responding to binary buffers. - sends: Vec>>, // sends[x] -> goes to process x. + sends: Vec>>>>, // sends[x] -> goes to process x. recvs: Receiver, // recvs[x] <- from process x?. to_local: Vec>>>, // to worker-local typed pullers. } @@ -151,7 +150,7 @@ impl Allocate for TcpAllocator { self.to_local.push(Rc::new(RefCell::new(VecDeque::new()))); } - let puller = Box::new(Puller::new(inner_recv, self.to_local[channel_id].clone())); + let puller = Box::new(PullerInner::new(inner_recv, self.to_local[channel_id].clone())); (pushes, puller, None) } @@ -160,12 +159,7 @@ impl Allocate for TcpAllocator { #[inline(never)] fn pre_work(&mut self) { - while let Ok(bytes) = self.recvs.try_recv() { - - // TODO: We could wrap `bytes` in a bytes::rc::Bytes, - // which could reduce `Arc` overhead, if it hurts. - // This new `Arc` should be local/uncontended, though. - let mut bytes = Bytes::from(bytes); + while let Some(mut bytes) = self.recvs.pull() { // We expect that `bytes` contains an integral number of messages. // No splitting occurs across allocations. @@ -209,92 +203,4 @@ impl Allocate for TcpAllocator { // } // } } -} - -/// 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. -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. -struct Puller { - inner: Box>>, // inner pullable (e.g. intra-process typed queue) - current: Option>, - receiver: Rc>>, // source of serialized buffers -} - -impl Puller { - fn new(inner: Box>>, receiver: Rc>>) -> Puller { - Puller { - inner, - current: None, - receiver, - } - } -} - -impl Pull> for Puller { - #[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/allocator_process.rs b/communication/src/allocator/zero_copy/allocator_process.rs new file mode 100644 index 000000000..11e8d3495 --- /dev/null +++ b/communication/src/allocator/zero_copy/allocator_process.rs @@ -0,0 +1,181 @@ +use std::rc::Rc; +use std::cell::RefCell; +use std::collections::VecDeque; +use std::sync::{Arc, Mutex}; + +use bytes::arc::Bytes; + +use networking::MessageHeader; + +use {Allocate, Data, Push, Pull}; +use allocator::Message; + +use super::bytes_exchange::{BytesPull, SendEndpoint}; + +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. +} + +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 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 = Arc::new(Mutex::new(VecDeque::new())); + let recv = send.clone(); + sends[source].push(send); + recvs[target].push(recv); + } + } + + let mut result = Vec::new(); + for (index, (sends, recvs)) in sends.drain(..).zip(recvs.drain(..)).enumerate() { + result.push(ProcessBuilder { + index, + peers: count, + sends, + recvs, + }) + } + + result + } + + /// 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, + sends, + recvs: self.recvs, + to_local: Vec::new(), + } + } +} + +// A specific Communicator 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). + + // 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 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() { + while let Some(mut bytes) = recv.pull() { + + // 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 index 7b60acc47..12efaac38 100644 --- a/communication/src/allocator/zero_copy/bytes_exchange.rs +++ b/communication/src/allocator/zero_copy/bytes_exchange.rs @@ -1,50 +1,84 @@ -use std::ops::DerefMut; use bytes::arc::Bytes; - -use super::shared_queue::{SharedQueueSend, SharedQueueRecv}; use super::bytes_slab::BytesSlab; -/// A type that can allocate send and receive endpoints for byte exchanges. -/// -/// The `BytesExchange` intent is that one can abstractly define mechanisms for exchanging -/// bytes between various entities. In some cases this may be between worker threads within -/// a process, in other cases it may be between worker threads and remote processes. At the -/// moment the cardinalities of remote endpoints requires some context and interpretation. -pub trait BytesExchange { - /// The type of the send endpoint. - type Send: SendEndpoint+'static; - /// The type of the receive endpoint. - type Recv: RecvEndpoint+'static; - /// Allocates endpoint pairs for a specified worker. - /// - /// Importantly, the Send side may share state to coalesce the buffering and - /// transmission of records. That is why there are `Rc>` things there. - fn next(&mut self) -> Option<(Vec, Vec)>; +// /// A type that can allocate send and receive endpoints for byte exchanges. +// /// +// /// The `BytesExchange` intent is that one can abstractly define mechanisms for exchanging +// /// bytes between various entities. In some cases this may be between worker threads within +// /// a process, in other cases it may be between worker threads and remote processes. At the +// /// moment the cardinalities of remote endpoints requires some context and interpretation. +// pub trait BytesExchange { +// /// The type of the send endpoint. +// type Send: SendEndpoint+'static; +// /// The type of the receive endpoint. +// type Recv: RecvEndpoint+'static; +// /// Allocates endpoint pairs for a specified worker. +// /// +// /// Importantly, the Send side may share state to coalesce the buffering and +// /// transmission of records. That is why there are `Rc>` things there. +// fn next(&mut self) -> Option<(Vec, Vec)>; +// } + +// /// A type that can provide and publish writeable binary buffers. +// pub trait SendEndpoint { +// /// The type of the writeable binary buffer. +// type SendBuffer: ::std::io::Write; +// /// Provides a writeable buffer of the requested capacity. +// fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; +// /// Indicates that it is now appropriate to publish the buffer. +// fn publish(&mut self); +// } + +// /// A type that can provide readable binary buffers. +// pub trait RecvEndpoint { +// type RecvBuffer: DerefMut; +// /// Provides a readable buffer. +// fn receive(&mut self) -> Option; +// } + +pub trait BytesPush { fn push(&mut self, bytes: Bytes); } +pub trait BytesPull { fn pull(&mut self) -> Option; } + +// std::sync::mpsc implementations. +impl BytesPush for ::std::sync::mpsc::Sender { + fn push(&mut self, bytes: Bytes) { + self.send(bytes) + .expect("unable to send Bytes"); + } } -/// A type that can provide and publish writeable binary buffers. -pub trait SendEndpoint { - /// The type of the writeable binary buffer. - type SendBuffer: ::std::io::Write; - /// Provides a writeable buffer of the requested capacity. - fn reserve(&mut self, capacity: usize) -> &mut Self::SendBuffer; - /// Indicates that it is now appropriate to publish the buffer. - fn publish(&mut self); +impl BytesPull for ::std::sync::mpsc::Receiver { + fn pull(&mut self) -> Option { + self.try_recv() + .ok() + } } -/// A type that can provide readable binary buffers. -pub trait RecvEndpoint { - type RecvBuffer: DerefMut; - /// Provides a readable buffer. - fn receive(&mut self) -> Option; +// Arc>> implementations. +use ::std::sync::{Arc, Mutex}; +use ::std::collections::VecDeque; +impl BytesPush for Arc>> { + fn push(&mut self, bytes: Bytes) { + self.lock() + .expect("unable to lock mutex") + .push_back(bytes); + } +} + +impl BytesPull for Arc>> { + fn pull(&mut self) -> Option { + self.lock() + .expect("unable to lock mutex") + .pop_front() + } } -pub struct BytesSendEndpoint { - send: SharedQueueSend, +pub struct SendEndpoint { + send: P, buffer: BytesSlab, } -impl BytesSendEndpoint { +impl SendEndpoint