Skip to content

Commit 54974b9

Browse files
Move opinions about encoding from communication to timely. (#597)
* Introduce Bytesable trait * Use Bytesable for serialization * Remove Message from Allocate::pipeline * Remove communication dependence on Message * Move opinions about encoding from communication to timely * Move opinions about bincode into pacts * Rebase and respond to fix-ups
1 parent 229c114 commit 54974b9

File tree

21 files changed

+398
-226
lines changed

21 files changed

+398
-226
lines changed

communication/Cargo.toml

-1
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@ default = ["getopts"]
1818

1919
[dependencies]
2020
getopts = { version = "0.2.21", optional = true }
21-
bincode = { version = "1.0" }
2221
byteorder = "1.5"
2322
serde = { version = "1.0", features = ["derive"] }
2423
timely_bytes = { path = "../bytes", version = "0.12" }

communication/examples/comm_hello.rs

+23-4
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,24 @@
1-
use std::ops::Deref;
2-
use timely_communication::{Message, Allocate};
1+
use timely_communication::{Allocate, Bytesable};
2+
3+
/// A wrapper that indicates the serialization/deserialization strategy.
4+
pub struct Message {
5+
/// Text contents.
6+
pub payload: String,
7+
}
8+
9+
impl Bytesable for Message {
10+
fn from_bytes(bytes: timely_bytes::arc::Bytes) -> Self {
11+
Message { payload: std::str::from_utf8(&bytes[..]).unwrap().to_string() }
12+
}
13+
14+
fn length_in_bytes(&self) -> usize {
15+
self.payload.len()
16+
}
17+
18+
fn into_bytes<W: ::std::io::Write>(&self, writer: &mut W) {
19+
writer.write_all(self.payload.as_bytes()).unwrap();
20+
}
21+
}
322

423
fn main() {
524

@@ -14,7 +33,7 @@ fn main() {
1433

1534
// send typed data along each channel
1635
for i in 0 .. allocator.peers() {
17-
senders[i].send(Message::from_typed(format!("hello, {}", i)));
36+
senders[i].send(Message { payload: format!("hello, {}", i)});
1837
senders[i].done();
1938
}
2039

@@ -26,7 +45,7 @@ fn main() {
2645
allocator.receive();
2746

2847
if let Some(message) = receiver.recv() {
29-
println!("worker {}: received: <{}>", allocator.index(), message.deref());
48+
println!("worker {}: received: <{}>", allocator.index(), message.payload);
3049
received += 1;
3150
}
3251

communication/src/allocator/generic.rs

+4-4
Original file line numberDiff line numberDiff line change
@@ -8,11 +8,11 @@ use std::cell::RefCell;
88

99
use crate::allocator::thread::ThreadBuilder;
1010
use crate::allocator::process::ProcessBuilder as TypedProcessBuilder;
11-
use crate::allocator::{Allocate, AllocateBuilder, Thread, Process};
11+
use crate::allocator::{Allocate, AllocateBuilder, Exchangeable, Thread, Process};
1212
use crate::allocator::zero_copy::allocator_process::{ProcessBuilder, ProcessAllocator};
1313
use crate::allocator::zero_copy::allocator::{TcpBuilder, TcpAllocator};
1414

15-
use crate::{Push, Pull, Data, Message};
15+
use crate::{Push, Pull};
1616

1717
/// Enumerates known implementors of `Allocate`.
1818
/// Passes trait method calls on to members.
@@ -47,7 +47,7 @@ impl Generic {
4747
}
4848
}
4949
/// Constructs several send endpoints and one receive endpoint.
50-
fn allocate<T: Data>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<Message<T>>>>, Box<dyn Pull<Message<T>>>) {
50+
fn allocate<T: Exchangeable>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<T>>>, Box<dyn Pull<T>>) {
5151
match self {
5252
Generic::Thread(t) => t.allocate(identifier),
5353
Generic::Process(p) => p.allocate(identifier),
@@ -86,7 +86,7 @@ impl Generic {
8686
impl Allocate for Generic {
8787
fn index(&self) -> usize { self.index() }
8888
fn peers(&self) -> usize { self.peers() }
89-
fn allocate<T: Data>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<Message<T>>>>, Box<dyn Pull<Message<T>>>) {
89+
fn allocate<T: Exchangeable>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<T>>>, Box<dyn Pull<T>>) {
9090
self.allocate(identifier)
9191
}
9292

communication/src/allocator/mod.rs

+10-4
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@ pub mod counters;
1717

1818
pub mod zero_copy;
1919

20-
use crate::{Data, Push, Pull, Message};
20+
use crate::{Bytesable, Push, Pull};
2121

2222
/// A proto-allocator, which implements `Send` and can be completed with `build`.
2323
///
@@ -32,6 +32,12 @@ pub trait AllocateBuilder : Send {
3232
fn build(self) -> Self::Allocator;
3333
}
3434

35+
use std::any::Any;
36+
37+
/// A type that can be sent along an allocated channel.
38+
pub trait Exchangeable : Send+Any+Bytesable { }
39+
impl<T: Send+Any+Bytesable> Exchangeable for T { }
40+
3541
/// A type capable of allocating channels.
3642
///
3743
/// There is some feature creep, in that this contains several convenience methods about the nature
@@ -42,7 +48,7 @@ pub trait Allocate {
4248
/// The number of workers in the communication group.
4349
fn peers(&self) -> usize;
4450
/// Constructs several send endpoints and one receive endpoint.
45-
fn allocate<T: Data>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<Message<T>>>>, Box<dyn Pull<Message<T>>>);
51+
fn allocate<T: Exchangeable>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<T>>>, Box<dyn Pull<T>>);
4652
/// A shared queue of communication events with channel identifier.
4753
///
4854
/// It is expected that users of the channel allocator will regularly
@@ -85,8 +91,8 @@ pub trait Allocate {
8591
/// By default, this method uses the thread-local channel constructor
8692
/// based on a shared `VecDeque` which updates the event queue.
8793
fn pipeline<T: 'static>(&mut self, identifier: usize) ->
88-
(thread::ThreadPusher<Message<T>>,
89-
thread::ThreadPuller<Message<T>>)
94+
(thread::ThreadPusher<T>,
95+
thread::ThreadPuller<T>)
9096
{
9197
thread::Thread::new_from(identifier, self.events().clone())
9298
}

communication/src/allocator/process.rs

+6-6
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,7 @@ use crossbeam_channel::{Sender, Receiver};
1010

1111
use crate::allocator::thread::{ThreadBuilder};
1212
use crate::allocator::{Allocate, AllocateBuilder, Thread};
13-
use crate::{Push, Pull, Message};
13+
use crate::{Push, Pull};
1414
use crate::buzzer::Buzzer;
1515

1616
/// An allocator for inter-thread, intra-process communication
@@ -110,7 +110,7 @@ impl Process {
110110
impl Allocate for Process {
111111
fn index(&self) -> usize { self.index }
112112
fn peers(&self) -> usize { self.peers }
113-
fn allocate<T: Any+Send+Sync+'static>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<Message<T>>>>, Box<dyn Pull<Message<T>>>) {
113+
fn allocate<T: Any+Send>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<T>>>, Box<dyn Pull<T>>) {
114114

115115
// this is race-y global initialisation of all channels for all workers, performed by the
116116
// first worker that enters this critical section
@@ -126,7 +126,7 @@ impl Allocate for Process {
126126
let mut pushers = Vec::with_capacity(self.peers);
127127
let mut pullers = Vec::with_capacity(self.peers);
128128
for buzzer in self.buzzers.iter() {
129-
let (s, r): (Sender<Message<T>>, Receiver<Message<T>>) = crossbeam_channel::unbounded();
129+
let (s, r): (Sender<T>, Receiver<T>) = crossbeam_channel::unbounded();
130130
// TODO: the buzzer in the pusher may be redundant, because we need to buzz post-counter.
131131
pushers.push((Pusher { target: s }, buzzer.clone()));
132132
pullers.push(Puller { source: r, current: None });
@@ -142,7 +142,7 @@ impl Allocate for Process {
142142

143143
let vector =
144144
entry
145-
.downcast_mut::<Vec<Option<(Vec<(Pusher<Message<T>>, Buzzer)>, Puller<Message<T>>)>>>()
145+
.downcast_mut::<Vec<Option<(Vec<(Pusher<T>, Buzzer)>, Puller<T>)>>>()
146146
.expect("failed to correctly cast channel");
147147

148148
let (sends, recv) =
@@ -166,10 +166,10 @@ impl Allocate for Process {
166166
sends.into_iter()
167167
.zip(self.counters_send.iter())
168168
.map(|((s,b), sender)| CountPusher::new(s, identifier, sender.clone(), b))
169-
.map(|s| Box::new(s) as Box<dyn Push<super::Message<T>>>)
169+
.map(|s| Box::new(s) as Box<dyn Push<T>>)
170170
.collect::<Vec<_>>();
171171

172-
let recv = Box::new(CountPuller::new(recv, identifier, self.inner.events().clone())) as Box<dyn Pull<super::Message<T>>>;
172+
let recv = Box::new(CountPuller::new(recv, identifier, self.inner.events().clone())) as Box<dyn Pull<T>>;
173173

174174
(sends, recv)
175175
}

communication/src/allocator/thread.rs

+4-4
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,7 @@ use std::collections::VecDeque;
88
use crate::allocator::{Allocate, AllocateBuilder};
99
use crate::allocator::counters::Pusher as CountPusher;
1010
use crate::allocator::counters::Puller as CountPuller;
11-
use crate::{Push, Pull, Message};
11+
use crate::{Push, Pull};
1212

1313
/// Builder for single-threaded allocator.
1414
pub struct ThreadBuilder;
@@ -28,7 +28,7 @@ pub struct Thread {
2828
impl Allocate for Thread {
2929
fn index(&self) -> usize { 0 }
3030
fn peers(&self) -> usize { 1 }
31-
fn allocate<T: 'static>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<Message<T>>>>, Box<dyn Pull<Message<T>>>) {
31+
fn allocate<T: 'static>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<T>>>, Box<dyn Pull<T>>) {
3232
let (pusher, puller) = Thread::new_from(identifier, self.events.clone());
3333
(vec![Box::new(pusher)], Box::new(puller))
3434
}
@@ -62,9 +62,9 @@ impl Thread {
6262

6363
/// Creates a new thread-local channel from an identifier and shared counts.
6464
pub fn new_from<T: 'static>(identifier: usize, events: Rc<RefCell<Vec<usize>>>)
65-
-> (ThreadPusher<Message<T>>, ThreadPuller<Message<T>>)
65+
-> (ThreadPusher<T>, ThreadPuller<T>)
6666
{
67-
let shared = Rc::new(RefCell::new((VecDeque::<Message<T>>::new(), VecDeque::<Message<T>>::new())));
67+
let shared = Rc::new(RefCell::new((VecDeque::<T>::new(), VecDeque::<T>::new())));
6868
let pusher = Pusher { target: shared.clone() };
6969
let pusher = CountPusher::new(pusher, identifier, events.clone());
7070
let puller = Puller { source: shared, current: None };

communication/src/allocator/zero_copy/allocator.rs

+4-4
Original file line numberDiff line numberDiff line change
@@ -8,8 +8,8 @@ use timely_bytes::arc::Bytes;
88

99
use crate::networking::MessageHeader;
1010

11-
use crate::{Allocate, Message, Data, Push, Pull};
12-
use crate::allocator::AllocateBuilder;
11+
use crate::{Allocate, Push, Pull};
12+
use crate::allocator::{AllocateBuilder, Exchangeable};
1313
use crate::allocator::canary::Canary;
1414

1515
use super::bytes_exchange::{BytesPull, SendEndpoint, MergeQueue};
@@ -135,7 +135,7 @@ pub struct TcpAllocator<A: Allocate> {
135135
impl<A: Allocate> Allocate for TcpAllocator<A> {
136136
fn index(&self) -> usize { self.index }
137137
fn peers(&self) -> usize { self.peers }
138-
fn allocate<T: Data>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<Message<T>>>>, Box<dyn Pull<Message<T>>>) {
138+
fn allocate<T: Exchangeable>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<T>>>, Box<dyn Pull<T>>) {
139139

140140
// Assume and enforce in-order identifier allocation.
141141
if let Some(bound) = self.channel_id_bound {
@@ -144,7 +144,7 @@ impl<A: Allocate> Allocate for TcpAllocator<A> {
144144
self.channel_id_bound = Some(identifier);
145145

146146
// Result list of boxed pushers.
147-
let mut pushes = Vec::<Box<dyn Push<Message<T>>>>::new();
147+
let mut pushes = Vec::<Box<dyn Push<T>>>::new();
148148

149149
// Inner exchange allocations.
150150
let inner_peers = self.inner.peers();

communication/src/allocator/zero_copy/allocator_process.rs

+4-4
Original file line numberDiff line numberDiff line change
@@ -9,8 +9,8 @@ use timely_bytes::arc::Bytes;
99

1010
use crate::networking::MessageHeader;
1111

12-
use crate::{Allocate, Message, Data, Push, Pull};
13-
use crate::allocator::{AllocateBuilder};
12+
use crate::{Allocate, Push, Pull};
13+
use crate::allocator::{AllocateBuilder, Exchangeable};
1414
use crate::allocator::canary::Canary;
1515

1616
use super::bytes_exchange::{BytesPull, SendEndpoint, MergeQueue};
@@ -119,15 +119,15 @@ pub struct ProcessAllocator {
119119
impl Allocate for ProcessAllocator {
120120
fn index(&self) -> usize { self.index }
121121
fn peers(&self) -> usize { self.peers }
122-
fn allocate<T: Data>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<Message<T>>>>, Box<dyn Pull<Message<T>>>) {
122+
fn allocate<T: Exchangeable>(&mut self, identifier: usize) -> (Vec<Box<dyn Push<T>>>, Box<dyn Pull<T>>) {
123123

124124
// Assume and enforce in-order identifier allocation.
125125
if let Some(bound) = self.channel_id_bound {
126126
assert!(bound < identifier);
127127
}
128128
self.channel_id_bound = Some(identifier);
129129

130-
let mut pushes = Vec::<Box<dyn Push<Message<T>>>>::with_capacity(self.peers());
130+
let mut pushes = Vec::<Box<dyn Push<T>>>::with_capacity(self.peers());
131131

132132
for target_index in 0 .. self.peers() {
133133

communication/src/allocator/zero_copy/push_pull.rs

+15-17
Original file line numberDiff line numberDiff line change
@@ -8,9 +8,7 @@ use timely_bytes::arc::Bytes;
88

99
use crate::allocator::canary::Canary;
1010
use crate::networking::MessageHeader;
11-
12-
use crate::{Data, Push, Pull};
13-
use crate::allocator::Message;
11+
use crate::{Bytesable, Push, Pull};
1412

1513
use super::bytes_exchange::{BytesPush, SendEndpoint};
1614

@@ -35,9 +33,9 @@ impl<T, P: BytesPush> Pusher<T, P> {
3533
}
3634
}
3735

38-
impl<T:Data, P: BytesPush> Push<Message<T>> for Pusher<T, P> {
36+
impl<T: Bytesable, P: BytesPush> Push<T> for Pusher<T, P> {
3937
#[inline]
40-
fn push(&mut self, element: &mut Option<Message<T>>) {
38+
fn push(&mut self, element: &mut Option<T>) {
4139
if let Some(ref mut element) = *element {
4240

4341
// determine byte lengths and build header.
@@ -68,11 +66,11 @@ impl<T:Data, P: BytesPush> Push<Message<T>> for Pusher<T, P> {
6866
/// allocation.
6967
pub struct Puller<T> {
7068
_canary: Canary,
71-
current: Option<Message<T>>,
69+
current: Option<T>,
7270
receiver: Rc<RefCell<VecDeque<Bytes>>>, // source of serialized buffers
7371
}
7472

75-
impl<T:Data> Puller<T> {
73+
impl<T: Bytesable> Puller<T> {
7674
/// Creates a new `Puller` instance from a shared queue.
7775
pub fn new(receiver: Rc<RefCell<VecDeque<Bytes>>>, _canary: Canary) -> Puller<T> {
7876
Puller {
@@ -83,14 +81,14 @@ impl<T:Data> Puller<T> {
8381
}
8482
}
8583

86-
impl<T:Data> Pull<Message<T>> for Puller<T> {
84+
impl<T: Bytesable> Pull<T> for Puller<T> {
8785
#[inline]
88-
fn pull(&mut self) -> &mut Option<Message<T>> {
86+
fn pull(&mut self) -> &mut Option<T> {
8987
self.current =
9088
self.receiver
9189
.borrow_mut()
9290
.pop_front()
93-
.map(Message::from_bytes);
91+
.map(T::from_bytes);
9492

9593
&mut self.current
9694
}
@@ -103,15 +101,15 @@ impl<T:Data> Pull<Message<T>> for Puller<T> {
103101
/// like the `bytes` crate (../bytes/) which provides an exclusive view of a shared
104102
/// allocation.
105103
pub struct PullerInner<T> {
106-
inner: Box<dyn Pull<Message<T>>>, // inner pullable (e.g. intra-process typed queue)
104+
inner: Box<dyn Pull<T>>, // inner pullable (e.g. intra-process typed queue)
107105
_canary: Canary,
108-
current: Option<Message<T>>,
106+
current: Option<T>,
109107
receiver: Rc<RefCell<VecDeque<Bytes>>>, // source of serialized buffers
110108
}
111109

112-
impl<T:Data> PullerInner<T> {
110+
impl<T: Bytesable> PullerInner<T> {
113111
/// Creates a new `PullerInner` instance from a shared queue.
114-
pub fn new(inner: Box<dyn Pull<Message<T>>>, receiver: Rc<RefCell<VecDeque<Bytes>>>, _canary: Canary) -> Self {
112+
pub fn new(inner: Box<dyn Pull<T>>, receiver: Rc<RefCell<VecDeque<Bytes>>>, _canary: Canary) -> Self {
115113
PullerInner {
116114
inner,
117115
_canary,
@@ -121,9 +119,9 @@ impl<T:Data> PullerInner<T> {
121119
}
122120
}
123121

124-
impl<T:Data> Pull<Message<T>> for PullerInner<T> {
122+
impl<T: Bytesable> Pull<T> for PullerInner<T> {
125123
#[inline]
126-
fn pull(&mut self) -> &mut Option<Message<T>> {
124+
fn pull(&mut self) -> &mut Option<T> {
127125

128126
let inner = self.inner.pull();
129127
if inner.is_some() {
@@ -134,7 +132,7 @@ impl<T:Data> Pull<Message<T>> for PullerInner<T> {
134132
self.receiver
135133
.borrow_mut()
136134
.pop_front()
137-
.map(Message::from_bytes);
135+
.map(T::from_bytes);
138136

139137
&mut self.current
140138
}

0 commit comments

Comments
 (0)