timely_communication/allocator/
process.rs1use std::rc::Rc;
4use std::cell::RefCell;
5use std::sync::{Arc, Mutex};
6use std::any::Any;
7use std::time::Duration;
8use std::collections::{HashMap, VecDeque};
9use crossbeam_channel::{Sender, Receiver};
10
11use crate::allocator::thread::{ThreadBuilder};
12use crate::allocator::{Allocate, AllocateBuilder, Event, Thread};
13use crate::{Push, Pull, Message};
14use crate::buzzer::Buzzer;
15
16pub struct ProcessBuilder {
18 inner: ThreadBuilder,
19 index: usize,
20 peers: usize,
21 channels: Arc<Mutex<HashMap<usize, Box<dyn Any+Send>>>>,
23
24 buzzers_send: Vec<Sender<Buzzer>>,
26 buzzers_recv: Vec<Receiver<Buzzer>>,
27
28 counters_send: Vec<Sender<(usize, Event)>>,
29 counters_recv: Receiver<(usize, Event)>,
30}
31
32impl AllocateBuilder for ProcessBuilder {
33 type Allocator = Process;
34 fn build(self) -> Self::Allocator {
35
36 for worker in self.buzzers_send.iter() {
38 let buzzer = Buzzer::new();
39 worker.send(buzzer).expect("Failed to send buzzer");
40 }
41 let mut buzzers = Vec::with_capacity(self.buzzers_recv.len());
42 for worker in self.buzzers_recv.iter() {
43 buzzers.push(worker.recv().expect("Failed to recv buzzer"));
44 }
45
46 Process {
47 inner: self.inner.build(),
48 index: self.index,
49 peers: self.peers,
50 channels: self.channels,
51 buzzers,
52 counters_send: self.counters_send,
53 counters_recv: self.counters_recv,
54 }
55 }
56}
57
58pub struct Process {
60 inner: Thread,
61 index: usize,
62 peers: usize,
63 channels: Arc<Mutex<HashMap<usize, Box<dyn Any+Send>>>>,
65 buzzers: Vec<Buzzer>,
66 counters_send: Vec<Sender<(usize, Event)>>,
67 counters_recv: Receiver<(usize, Event)>,
68}
69
70impl Process {
71 pub fn inner(&mut self) -> &mut Thread { &mut self.inner }
73 pub fn new_vector(peers: usize) -> Vec<ProcessBuilder> {
75
76 let mut counters_send = Vec::with_capacity(peers);
77 let mut counters_recv = Vec::with_capacity(peers);
78 for _ in 0 .. peers {
79 let (send, recv) = crossbeam_channel::unbounded();
80 counters_send.push(send);
81 counters_recv.push(recv);
82 }
83
84 let channels = Arc::new(Mutex::new(HashMap::with_capacity(peers)));
85
86 let (buzzers_send, buzzers_recv) = crate::promise_futures(peers, peers);
88
89 counters_recv
90 .into_iter()
91 .zip(buzzers_send.into_iter())
92 .zip(buzzers_recv.into_iter())
93 .enumerate()
94 .map(|(index, ((recv, bsend), brecv))| {
95 ProcessBuilder {
96 inner: ThreadBuilder,
97 index,
98 peers,
99 buzzers_send: bsend,
100 buzzers_recv: brecv,
101 channels: channels.clone(),
102 counters_send: counters_send.clone(),
103 counters_recv: recv,
104 }
105 })
106 .collect()
107 }
108}
109
110impl Allocate for Process {
111 fn index(&self) -> usize { self.index }
112 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>>>) {
114
115 let mut channels = self.channels.lock().expect("mutex error?");
120
121 let (sends, recv, empty) = {
122
123 let entry = channels.entry(identifier).or_insert_with(|| {
125
126 let mut pushers = Vec::with_capacity(self.peers);
127 let mut pullers = Vec::with_capacity(self.peers);
128 for buzzer in self.buzzers.iter() {
129 let (s, r): (Sender<Message<T>>, Receiver<Message<T>>) = crossbeam_channel::unbounded();
130 pushers.push((Pusher { target: s }, buzzer.clone()));
132 pullers.push(Puller { source: r, current: None });
133 }
134
135 let mut to_box = Vec::with_capacity(pullers.len());
136 for recv in pullers.into_iter() {
137 to_box.push(Some((pushers.clone(), recv)));
138 }
139
140 Box::new(to_box)
141 });
142
143 let vector =
144 entry
145 .downcast_mut::<Vec<Option<(Vec<(Pusher<Message<T>>, Buzzer)>, Puller<Message<T>>)>>>()
146 .expect("failed to correctly cast channel");
147
148 let (sends, recv) =
149 vector[self.index]
150 .take()
151 .expect("channel already consumed");
152
153 let empty = vector.iter().all(|x| x.is_none());
154
155 (sends, recv, empty)
156 };
157
158 if empty { channels.remove(&identifier); }
161
162 use crate::allocator::counters::ArcPusher as CountPusher;
163 use crate::allocator::counters::Puller as CountPuller;
164
165 let sends =
166 sends.into_iter()
167 .zip(self.counters_send.iter())
168 .map(|((s,b), sender)| CountPusher::new(s, identifier, sender.clone(), b))
169 .map(|s| Box::new(s) as Box<dyn Push<super::Message<T>>>)
170 .collect::<Vec<_>>();
171
172 let recv = Box::new(CountPuller::new(recv, identifier, self.inner.events().clone())) as Box<dyn Pull<super::Message<T>>>;
173
174 (sends, recv)
175 }
176
177 fn events(&self) -> &Rc<RefCell<VecDeque<(usize, Event)>>> {
178 self.inner.events()
179 }
180
181 fn await_events(&self, duration: Option<Duration>) {
182 self.inner.await_events(duration);
183 }
184
185 fn receive(&mut self) {
186 let mut events = self.inner.events().borrow_mut();
187 while let Ok((index, event)) = self.counters_recv.try_recv() {
188 events.push_back((index, event));
189 }
190 }
191}
192
193struct Pusher<T> {
195 target: Sender<T>,
196}
197
198impl<T> Clone for Pusher<T> {
199 fn clone(&self) -> Self {
200 Self {
201 target: self.target.clone(),
202 }
203 }
204}
205
206impl<T> Push<T> for Pusher<T> {
207 #[inline] fn push(&mut self, element: &mut Option<T>) {
208 if let Some(element) = element.take() {
209 let _ = self.target.send(element);
212 }
213 }
214}
215
216struct Puller<T> {
218 current: Option<T>,
219 source: Receiver<T>,
220}
221
222impl<T> Pull<T> for Puller<T> {
223 #[inline]
224 fn pull(&mut self) -> &mut Option<T> {
225 self.current = self.source.try_recv().ok();
226 &mut self.current
227 }
228}