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
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

//! Rehydration of storage hosts.
//!
//! Rehydration is the process of bringing a crashed `storaged` process back
//! up to date. The [`RehydratingStorageClient`] records all commands it
//! observes in a minimal form. If it observes a send or receive failure while
//! communicating with the underlying client, it will reconnect the client and
//! replay the command stream.

use std::collections::{BTreeMap, HashMap};
use std::sync::Arc;
use std::time::Duration;

use anyhow::anyhow;
use differential_dataflow::lattice::Lattice;
use futures::Stream;
use timely::progress::{Antichain, Timestamp};
use timely::PartialOrder;
use tokio::select;
use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender};
use tokio::sync::Mutex;
use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::warn;

use mz_build_info::BuildInfo;
use mz_ore::retry::Retry;
use mz_ore::task::{AbortOnDropHandle, JoinHandleExt};
use mz_persist_client::cache::PersistClientCache;
use mz_persist_types::Codec64;
use mz_repr::{Diff, GlobalId};
use mz_service::client::GenericClient;

use crate::controller::ResumptionFrontierCalculator;
use crate::protocol::client::{
    CreateSinkCommand, CreateSourceCommand, StorageClient, StorageCommand, StorageGrpcClient,
    StorageResponse,
};
use crate::types::sources::SourceData;

/// A storage client that replays the command stream on failure.
///
/// See the [module documentation](self) for details.
#[derive(Debug)]
pub struct RehydratingStorageClient<T> {
    command_tx: UnboundedSender<StorageCommand<T>>,
    response_rx: UnboundedReceiverStream<StorageResponse<T>>,
    _task: AbortOnDropHandle<()>,
}

impl<T> RehydratingStorageClient<T>
where
    T: Timestamp + Lattice + Codec64,
    StorageGrpcClient: StorageClient<T>,
{
    /// Creates a `RehydratingStorageClient` for a storage host with the given
    /// network address.
    pub fn new(
        addr: String,
        build_info: &'static BuildInfo,
        persist: Arc<Mutex<PersistClientCache>>,
    ) -> 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,
            sources: BTreeMap::new(),
            sinks: BTreeMap::new(),
            uppers: HashMap::new(),
            initialized: false,
            persist,
        };
        let task = mz_ore::task::spawn(|| "rehydration", async move { task.run().await });
        RehydratingStorageClient {
            command_tx,
            response_rx: UnboundedReceiverStream::new(response_rx),
            _task: task.abort_on_drop(),
        }
    }

    /// Sends a command to the underlying client.
    pub fn send(&mut self, cmd: StorageCommand<T>) {
        self.command_tx
            .send(cmd)
            .expect("rehydration task should not drop first");
    }

    /// Returns a stream that produces responses from the underlying client.
    pub fn response_stream(&mut self) -> impl Stream<Item = StorageResponse<T>> + '_ {
        &mut self.response_rx
    }
}

/// A task that manages rehydration.
struct RehydrationTask<T> {
    /// The network address of the storage host.
    addr: String,
    /// The build information for this process.
    build_info: &'static BuildInfo,
    /// A channel upon which commands intended for the storage host are delivered.
    command_rx: UnboundedReceiver<StorageCommand<T>>,
    /// A channel upon which responses from the storage host are delivered.
    response_tx: UnboundedSender<StorageResponse<T>>,
    /// The sources that have been observed.
    sources: BTreeMap<GlobalId, CreateSourceCommand<T>>,
    /// The exports that have been observed.
    sinks: BTreeMap<GlobalId, CreateSinkCommand<T>>,
    /// The upper frontier information received.
    uppers: HashMap<GlobalId, Antichain<T>>,
    /// Set to `true` once [`StorageCommand::InitializationComplete`] has been
    /// observed.
    initialized: bool,
    /// A handle to Persist
    persist: Arc<Mutex<PersistClientCache>>,
}

enum RehydrationTaskState {
    /// The storage host should be (re)hydrated.
    Rehydrate,
    /// Communication with the storage host is live. Commands and responses should
    /// be forwarded until an error occurs.
    Pump { client: StorageGrpcClient },
    /// The caller has asked us to shut down communication with this storage
    /// host.
    Done,
}

impl<T> RehydrationTask<T>
where
    T: Timestamp + Lattice + Codec64,
    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 {
        // Reconnect to the storage host.
        let client = Retry::default()
            .clamp_backoff(Duration::from_secs(1))
            .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");

        for ingest in self.sources.values_mut() {
            let mut persist_clients = self.persist.lock().await;
            let mut state = ingest
                .description
                .initialize_state(&mut persist_clients)
                .await;
            let resume_upper = ingest
                .description
                .calculate_resumption_frontier(&mut state)
                .await;
            ingest.resume_upper = resume_upper;
        }

        for export in self.sinks.values_mut() {
            let mut persist_clients = self.persist.lock().await;
            let persist_client = persist_clients
                .open(
                    export
                        .description
                        .from_storage_metadata
                        .persist_location
                        .clone(),
                )
                .await
                .expect("error creating persist client");
            let from_read_handle = persist_client
                .open_reader::<SourceData, (), T, Diff>(
                    export.description.from_storage_metadata.data_shard,
                )
                .await
                .expect("from collection disappeared");

            let cached_as_of = &export.description.as_of;
            // The controller has the dependency recorded in it's `exported_collections` so this
            // should not change at least until the sink is started up (because the storage
            // controller will not downgrade the source's since).
            let from_since = from_read_handle.since();
            export.description.as_of = cached_as_of.maybe_fast_forward(from_since);
        }

        // Rehydrate all commands.
        let mut commands = vec![
            StorageCommand::CreateSources(self.sources.values().cloned().collect()),
            StorageCommand::CreateSinks(self.sinks.values().cloned().collect()),
        ];
        if self.initialized {
            commands.push(StorageCommand::InitializationComplete)
        }
        self.send_commands(client, commands).await
    }

    async fn step_pump(&mut self, mut client: StorageGrpcClient) -> RehydrationTaskState {
        select! {
            // Command from controller to forward to storage host.
            command = self.command_rx.recv() => match command {
                None => RehydrationTaskState::Done,
                Some(command) => {
                    self.absorb_command(&command);
                    self.send_commands(client, vec![command]).await
                }
            },
            // Response from storage host to forward to controller.
            response = client.recv() => {
                let response = match response.transpose() {
                    None => {
                        // In the future, if a storage host politely hangs up,
                        // we might want to take it as a signal that a new
                        // controller has taken over. For now we just try to
                        // reconnect.
                        Err(anyhow!("storage host unexpectedly gracefully terminated connection"))
                    }
                    Some(response) => response,
                };

                self.send_response(client, response)
            }
        }
    }

    async fn send_commands(
        &mut self,
        mut client: StorageGrpcClient,
        commands: impl IntoIterator<Item = StorageCommand<T>>,
    ) -> RehydrationTaskState {
        for command in commands {
            if let Err(e) = client.send(command).await {
                return self.send_response(client, Err(e));
            }
        }
        RehydrationTaskState::Pump { client }
    }

    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::InitializationComplete => self.initialized = true,
            StorageCommand::CreateSources(ingestions) => {
                for ingestion in ingestions {
                    self.sources.insert(ingestion.id, ingestion.clone());
                    // Initialize the uppers we are tracking
                    for &export_id in ingestion.description.source_exports.keys() {
                        self.uppers
                            .insert(export_id, Antichain::from_elem(T::minimum()));
                    }
                }
            }
            StorageCommand::CreateSinks(exports) => {
                for export in exports {
                    self.sinks.insert(export.id, export.clone());
                    // Initialize the uppers we are tracking
                    self.uppers
                        .insert(export.id, Antichain::from_elem(T::minimum()));
                }
            }
            StorageCommand::AllowCompaction(frontiers) => {
                for (id, frontier) in frontiers {
                    if frontier.is_empty() {
                        self.sources.remove(id);
                        self.uppers.remove(id);
                    }
                }
            }
        }
    }

    fn absorb_response(&mut self, response: StorageResponse<T>) -> Option<StorageResponse<T>> {
        match response {
            StorageResponse::FrontierUppers(list) => {
                let mut new_uppers = Vec::new();

                for (id, new_upper) in list {
                    if let Some(reported) = self.uppers.get_mut(&id) {
                        if PartialOrder::less_than(reported, &new_upper) {
                            reported.clone_from(&new_upper);
                            new_uppers.push((id, new_upper));
                        }
                    } else {
                        // It can happen during source shutdown that we remove
                        // the tracked upper from our state but a
                        // `FrontierUppers` response is still on the wire.
                        //
                        // This is very fine to ignore, especially now that
                        // these upper updates are plain `Antichains`, and not
                        // `ChangeBatches` where we need to be extra careful
                        // about not messing up our state.
                        tracing::info!("RehydratingStorageClient received FrontierUppers response {new_upper:?} for absent identifier {id}");
                    }
                }
                if !new_uppers.is_empty() {
                    Some(StorageResponse::FrontierUppers(new_uppers))
                } else {
                    None
                }
            }
        }
    }
}