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