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
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
use std::collections::{BTreeMap, HashMap};
use std::time::Duration;
use anyhow::anyhow;
use differential_dataflow::lattice::Lattice;
use futures::Stream;
use timely::progress::frontier::MutableAntichain;
use timely::progress::{Antichain, Timestamp};
use tokio::select;
use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender};
use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::warn;
use mz_build_info::BuildInfo;
use mz_ore::retry::Retry;
use mz_repr::GlobalId;
use mz_service::client::GenericClient;
use crate::client::{
IngestSourceCommand, StorageClient, StorageCommand, StorageGrpcClient, StorageResponse,
};
#[derive(Debug)]
pub struct RehydratingStorageClient<T> {
command_tx: UnboundedSender<StorageCommand<T>>,
response_rx: UnboundedReceiverStream<StorageResponse<T>>,
}
impl<T> RehydratingStorageClient<T>
where
T: Timestamp + Lattice,
StorageGrpcClient: StorageClient<T>,
{
pub fn new(addr: String, build_info: &'static BuildInfo) -> RehydratingStorageClient<T> {
let (command_tx, command_rx) = unbounded_channel();
let (response_tx, response_rx) = unbounded_channel();
let mut task = RehydrationTask {
addr,
build_info,
command_rx,
response_tx,
ingestions: BTreeMap::new(),
uppers: HashMap::new(),
};
mz_ore::task::spawn(|| "rehydration", async move { task.run().await });
RehydratingStorageClient {
command_tx,
response_rx: UnboundedReceiverStream::new(response_rx),
}
}
pub fn send(&mut self, cmd: StorageCommand<T>) {
self.command_tx
.send(cmd)
.expect("rehydration task should not drop first");
}
pub fn response_stream(&mut self) -> impl Stream<Item = StorageResponse<T>> + '_ {
&mut self.response_rx
}
}
struct RehydrationTask<T> {
addr: String,
build_info: &'static BuildInfo,
command_rx: UnboundedReceiver<StorageCommand<T>>,
response_tx: UnboundedSender<StorageResponse<T>>,
ingestions: BTreeMap<GlobalId, IngestSourceCommand<T>>,
uppers: HashMap<GlobalId, (Antichain<T>, MutableAntichain<T>)>,
}
enum RehydrationTaskState {
Rehydrate,
Pump { client: StorageGrpcClient },
Done,
}
impl<T> RehydrationTask<T>
where
T: Timestamp + Lattice,
StorageGrpcClient: StorageClient<T>,
{
async fn run(&mut self) {
let mut state = RehydrationTaskState::Rehydrate;
loop {
state = match state {
RehydrationTaskState::Rehydrate => self.step_rehydrate().await,
RehydrationTaskState::Pump { client } => self.step_pump(client).await,
RehydrationTaskState::Done => break,
}
}
}
async fn step_rehydrate(&mut self) -> RehydrationTaskState {
for (_id, (_, frontiers)) in self.uppers.iter_mut() {
*frontiers = MutableAntichain::new_bottom(T::minimum());
}
let client = Retry::default()
.clamp_backoff(Duration::from_secs(32))
.retry_async(|_| {
let addr = self.addr.clone();
let version = self.build_info.semver_version();
async move {
match StorageGrpcClient::connect(addr, version).await {
Ok(client) => Ok(client),
Err(e) => {
warn!("error connecting to storage host, retrying: {e}");
Err(e)
}
}
}
})
.await
.expect("retry retries forever");
self.send_command(
client,
StorageCommand::IngestSources(self.ingestions.values().cloned().collect()),
)
.await
}
async fn step_pump(&mut self, mut client: StorageGrpcClient) -> RehydrationTaskState {
select! {
command = self.command_rx.recv() => match command {
None => RehydrationTaskState::Done,
Some(command) => {
self.absorb_command(&command);
self.send_command(client, command).await
}
},
response = client.recv() => {
let response = match response.transpose() {
None => {
Err(anyhow!("storage host unexpectedly gracefully terminated connection"))
}
Some(response) => response,
};
self.send_response(client, response).await
}
}
}
async fn send_command(
&mut self,
mut client: StorageGrpcClient,
command: StorageCommand<T>,
) -> RehydrationTaskState {
match client.send(command).await {
Ok(()) => RehydrationTaskState::Pump { client },
Err(e) => self.send_response(client, Err(e)).await,
}
}
async fn send_response(
&mut self,
client: StorageGrpcClient,
response: Result<StorageResponse<T>, anyhow::Error>,
) -> RehydrationTaskState {
match response {
Ok(response) => {
if let Some(response) = self.absorb_response(response) {
if self.response_tx.send(response).is_err() {
RehydrationTaskState::Done
} else {
RehydrationTaskState::Pump { client }
}
} else {
RehydrationTaskState::Pump { client }
}
}
Err(e) => {
warn!("storage host produced error, reconnecting: {e}");
RehydrationTaskState::Rehydrate
}
}
}
fn absorb_command(&mut self, command: &StorageCommand<T>) {
match command {
StorageCommand::IngestSources(ingestions) => {
for ingestion in ingestions {
self.ingestions.insert(ingestion.id, ingestion.clone());
self.uppers.insert(
ingestion.id,
(
Antichain::from_elem(T::minimum()),
MutableAntichain::new_bottom(T::minimum()),
),
);
}
}
StorageCommand::AllowCompaction(frontiers) => {
for (id, frontier) in frontiers {
if frontier.is_empty() {
self.ingestions.remove(id);
self.uppers.remove(id);
}
}
}
}
}
fn absorb_response(&mut self, response: StorageResponse<T>) -> Option<StorageResponse<T>> {
match response {
StorageResponse::FrontierUppers(mut list) => {
for (id, changes) in list.iter_mut() {
if let Some((reported, tracked)) = self.uppers.get_mut(id) {
tracked.update_iter(changes.drain());
changes.extend(reported.iter().map(|t| (t.clone(), -1)));
reported.clear();
for (time1, _neg_one) in changes.iter() {
for time2 in tracked.frontier().iter() {
reported.insert(time1.join(time2));
}
}
changes.extend(reported.iter().map(|t| (t.clone(), 1)));
changes.compact();
} else {
panic!("RehydratingStorageClient received FrontierUppers response for absent identifier {id}");
}
}
if !list.is_empty() {
Some(StorageResponse::FrontierUppers(list))
} else {
None
}
}
}
}
}