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
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
// 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 replicas.
//!
//! Rehydration is the process of bringing a crashed storage replica 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::btree_map::Entry;
use std::collections::BTreeMap;
use std::num::NonZeroI64;
use std::time::Duration;

use anyhow::anyhow;
use differential_dataflow::lattice::Lattice;
use futures::{Stream, StreamExt};
use mz_build_info::BuildInfo;
use mz_cluster_client::client::{ClusterReplicaLocation, ClusterStartupEpoch, TimelyConfig};
use mz_ore::retry::Retry;
use mz_ore::task::{AbortOnDropHandle, JoinHandleExt};
use mz_persist_types::Codec64;
use mz_repr::GlobalId;
use mz_service::client::{GenericClient, Partitioned};
use mz_service::params::GrpcClientParameters;
use mz_storage_types::parameters::StorageParameters;
use timely::progress::{Antichain, Timestamp};
use timely::PartialOrder;
use tokio::select;
use tokio::sync::mpsc::error::TryRecvError;
use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender};
use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::warn;

use crate::client::{
    CreateSinkCommand, RunIngestionCommand, StorageClient, StorageCommand, StorageGrpcClient,
    StorageResponse,
};
use crate::metrics::RehydratingStorageClientMetrics;

/// 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<RehydrationCommand<T>>,
    response_rx: UnboundedReceiverStream<StorageResponse<T>>,
    _task: AbortOnDropHandle<()>,
}

type PartitionedClient<T> = Partitioned<StorageGrpcClient, StorageCommand<T>, StorageResponse<T>>;

impl<T> RehydratingStorageClient<T>
where
    T: Timestamp + Lattice + Codec64,
    StorageGrpcClient: StorageClient<T>,
{
    /// Creates a `RehydratingStorageClient` that is not yet connected to
    /// a storage replica.
    pub fn new(
        build_info: &'static BuildInfo,
        metrics: RehydratingStorageClientMetrics,
        envd_epoch: NonZeroI64,
        grpc_client_params: GrpcClientParameters,
        variable_length_row_encoding: bool,
    ) -> RehydratingStorageClient<T> {
        let (command_tx, command_rx) = unbounded_channel();
        let (response_tx, response_rx) = unbounded_channel();
        let mut task = RehydrationTask {
            build_info,
            command_rx,
            response_tx,
            sources: BTreeMap::new(),
            sinks: BTreeMap::new(),
            uppers: BTreeMap::new(),
            sinces: BTreeMap::new(),
            initialized: false,
            current_epoch: ClusterStartupEpoch::new(envd_epoch, 0),
            config: Default::default(),
            metrics,
            grpc_client_params,
            variable_length_row_encoding,
        };
        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(),
        }
    }

    /// Connects to the storage replica at the specified network address.
    pub fn connect(&mut self, location: ClusterReplicaLocation) {
        self.command_tx
            .send(RehydrationCommand::Connect { location })
            .expect("rehydration task should not drop first");
    }

    /// Reset the connection.
    pub fn reset(&mut self) {
        self.command_tx
            .send(RehydrationCommand::Reset)
            .expect("rehydration task should not drop first");
    }

    /// Sends a command to the underlying client.
    pub fn send(&mut self, cmd: StorageCommand<T>) {
        self.command_tx
            .send(RehydrationCommand::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
    }
}

#[derive(Debug, Clone)]
enum RehydrationCommand<T> {
    /// (Re)connect to a storage replica.
    Connect {
        /// The location of the (singular) replica we are going to connect to.
        location: ClusterReplicaLocation,
    },
    /// Send the contained storage command to the replica.
    Send(StorageCommand<T>),
    /// Reset the task to it's beginning state, as if
    /// no `Connect` command has ever been received.
    Reset,
}

/// A task that manages rehydration.
struct RehydrationTask<T> {
    /// The build information for this process.
    build_info: &'static BuildInfo,
    /// A channel upon which commands intended for the storage replica are
    /// delivered.
    command_rx: UnboundedReceiver<RehydrationCommand<T>>,
    /// A channel upon which responses from the storage replica are delivered.
    response_tx: UnboundedSender<StorageResponse<T>>,
    /// The sources that have been observed.
    sources: BTreeMap<GlobalId, RunIngestionCommand>,
    /// The exports that have been observed.
    sinks: BTreeMap<GlobalId, CreateSinkCommand<T>>,
    /// The upper frontier information received.
    uppers: BTreeMap<GlobalId, Antichain<T>>,
    /// The since frontiers that have been observed.
    sinces: BTreeMap<GlobalId, Antichain<T>>,
    /// Set to `true` once [`StorageCommand::InitializationComplete`] has been
    /// observed.
    initialized: bool,
    /// The current epoch for the replica we are connecting to.
    current_epoch: ClusterStartupEpoch,
    /// Storage configuration that has been observed.
    config: StorageParameters,
    /// Prometheus metrics
    metrics: RehydratingStorageClientMetrics,
    /// gRPC client parameters.
    grpc_client_params: GrpcClientParameters,
    /// Whether to enable variable-length row encoding.
    variable_length_row_encoding: bool,
}

enum RehydrationTaskState<T: Timestamp + Lattice> {
    /// Wait for the address of the storage replica to connect to.
    AwaitAddress,
    /// The storage replica should be (re)hydrated.
    Rehydrate {
        /// The location of the storage replica.
        location: ClusterReplicaLocation,
    },
    /// Communication with the storage replica is live. Commands and responses
    /// should be forwarded until an error occurs.
    Pump {
        /// The location of the storage replica.
        location: ClusterReplicaLocation,
        /// The connected client for the replica.
        client: PartitionedClient<T>,
    },
    /// The caller has asked us to shut down communication with this storage
    /// cluster.
    Done,
}

impl<T> RehydrationTask<T>
where
    T: Timestamp + Lattice + Codec64,
    StorageGrpcClient: StorageClient<T>,
{
    async fn run(&mut self) {
        let mut state = RehydrationTaskState::AwaitAddress;
        loop {
            state = match state {
                RehydrationTaskState::AwaitAddress => self.step_await_address().await,
                RehydrationTaskState::Rehydrate { location } => self.step_rehydrate(location).await,
                RehydrationTaskState::Pump { location, client } => {
                    self.step_pump(location, client).await
                }
                RehydrationTaskState::Done => break,
            }
        }
    }

    async fn step_await_address(&mut self) -> RehydrationTaskState<T> {
        loop {
            match self.command_rx.recv().await {
                None => break RehydrationTaskState::Done,
                Some(RehydrationCommand::Connect { location }) => {
                    break RehydrationTaskState::Rehydrate { location }
                }
                Some(RehydrationCommand::Send(command)) => {
                    self.absorb_command(&command);
                }
                Some(RehydrationCommand::Reset) => {}
            }
        }
    }

    async fn step_rehydrate(
        &mut self,
        location: ClusterReplicaLocation,
    ) -> RehydrationTaskState<T> {
        // Reconnect to the storage replica.
        let stream = Retry::default()
            .clamp_backoff(Duration::from_secs(1))
            .into_retry_stream();
        tokio::pin!(stream);

        // TODO(guswynn): cluster-unification: share this code with compute, by consolidating
        // on use of `ReplicaTask`.
        let (client, timely_command) = loop {
            let state = stream.next().await.expect("infinite stream");
            // Drain any pending commands, in case we've been told to connect
            // to a new storage replica.
            loop {
                match self.command_rx.try_recv() {
                    Ok(RehydrationCommand::Connect { location }) => {
                        return RehydrationTaskState::Rehydrate { location };
                    }
                    Ok(RehydrationCommand::Send(command)) => {
                        self.absorb_command(&command);
                    }
                    Ok(RehydrationCommand::Reset) => return RehydrationTaskState::AwaitAddress,
                    Err(TryRecvError::Disconnected) => return RehydrationTaskState::Done,
                    Err(TryRecvError::Empty) => break,
                }
            }

            let timely_config = TimelyConfig {
                workers: location.workers,
                // Overridden by the storage `PartitionedState` implementation.
                process: 0,
                addresses: location.dataflow_addrs.clone(),
                // This value is not currently used by storage, so we just choose
                // some identifiable value.
                //
                // TODO(guswynn): cluster-unification: ensure this is cleaned up when
                // the compute and storage command streams are merged.
                idle_arrangement_merge_effort: 1337,
                variable_length_row_encoding: self.variable_length_row_encoding,
            };
            let dests = location
                .ctl_addrs
                .clone()
                .into_iter()
                .map(|addr| (addr, self.metrics.clone()))
                .collect();
            let version = self.build_info.semver_version();
            let client =
                StorageGrpcClient::connect_partitioned(dests, version, &self.grpc_client_params)
                    .await;

            let client = match client {
                Ok(client) => client,
                Err(e) => {
                    if state.i >= mz_service::retry::INFO_MIN_RETRIES {
                        tracing::info!(
                            "error connecting to {:?} for storage, retrying in {:?}: {e}",
                            location,
                            state.next_backoff.unwrap()
                        );
                    } else {
                        tracing::debug!(
                            "error connecting to {:?} for storage, retrying in {:?}: {e}",
                            location,
                            state.next_backoff.unwrap()
                        );
                    }
                    continue;
                }
            };

            // The first epoch we actually send to the cluster will be `1`, just like compute.
            let new_epoch = ClusterStartupEpoch::new(
                self.current_epoch.envd(),
                self.current_epoch.replica() + 1,
            );
            self.current_epoch = new_epoch;
            let timely_command = StorageCommand::CreateTimely {
                config: timely_config,
                epoch: new_epoch,
            };

            break (client, timely_command);
        };

        // Rehydrate all commands.
        let mut commands = vec![
            timely_command,
            StorageCommand::UpdateConfiguration(self.config.clone()),
            StorageCommand::RunIngestions(self.sources.values().cloned().collect()),
            StorageCommand::CreateSinks(self.sinks.values().cloned().collect()),
            StorageCommand::AllowCompaction(
                self.sinces
                    .iter()
                    .map(|(id, since)| (*id, since.clone()))
                    .collect(),
            ),
        ];
        if self.initialized {
            commands.push(StorageCommand::InitializationComplete)
        }
        self.send_commands(location, client, commands).await
    }

    async fn step_pump(
        &mut self,
        location: ClusterReplicaLocation,
        mut client: PartitionedClient<T>,
    ) -> RehydrationTaskState<T> {
        select! {
            // Command from controller to forward to storage cluster.
            command = self.command_rx.recv() => match command {
                None => RehydrationTaskState::Done,
                Some(RehydrationCommand::Connect { location }) => RehydrationTaskState::Rehydrate { location },
                Some(RehydrationCommand::Send(command)) => {
                    self.absorb_command(&command);
                    self.send_commands(location, client, vec![command]).await
                }
                Some(RehydrationCommand::Reset) => {
                    RehydrationTaskState::AwaitAddress
                }
            },
            // Response from storage cluster to forward to controller.
            response = client.recv() => {
                let response = match response.transpose() {
                    None => {
                        // In the future, if a storage cluster 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 cluster unexpectedly gracefully terminated connection"))
                    }
                    Some(response) => response,
                };

                self.send_response(location, client, response)
            }
        }
    }

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

    fn send_response(
        &mut self,
        location: ClusterReplicaLocation,
        client: PartitionedClient<T>,
        response: Result<StorageResponse<T>, anyhow::Error>,
    ) -> RehydrationTaskState<T> {
        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 { location, client }
                    }
                } else {
                    RehydrationTaskState::Pump { location, client }
                }
            }
            Err(e) => {
                warn!("storage cluster produced error, reconnecting: {e}");
                RehydrationTaskState::Rehydrate { location }
            }
        }
    }

    fn absorb_command(&mut self, command: &StorageCommand<T>) {
        match command {
            StorageCommand::CreateTimely { .. } => {
                // We assume these are ordered correctly
            }
            StorageCommand::InitializationComplete => self.initialized = true,
            StorageCommand::UpdateConfiguration(params) => {
                self.config.update(params.clone());
            }
            StorageCommand::RunIngestions(ingestions) => {
                for ingestion in ingestions {
                    let prev = self.sources.insert(ingestion.id, ingestion.clone());
                    assert!(
                        prev.is_some() == ingestion.update,
                        "can only and must update source if RunIngestion is update"
                    );

                    for id in ingestion.description.subsource_ids() {
                        match self.uppers.entry(id) {
                            Entry::Occupied(_) => {
                                assert!(ingestion.update, "tried to re-insert frontier for {}", id)
                            }
                            Entry::Vacant(v) => {
                                v.insert(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) => {
                // Remember for rehydration!
                self.sinces.extend(frontiers.iter().cloned());

                for (id, frontier) in frontiers {
                    match self.sinks.get_mut(id) {
                        Some(export) => {
                            export.description.as_of.downgrade(frontier);
                        }
                        None if self.sources.contains_key(id) => continue,
                        None => panic!("AllowCompaction command for non-existent {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 {
                    let reported = match self.uppers.get_mut(&id) {
                        Some(reported) => reported,
                        None => panic!("Reference to absent collection: {id}"),
                    };
                    if PartialOrder::less_than(reported, &new_upper) {
                        reported.clone_from(&new_upper);
                        new_uppers.push((id, new_upper));
                    }
                }
                if !new_uppers.is_empty() {
                    Some(StorageResponse::FrontierUppers(new_uppers))
                } else {
                    None
                }
            }
            StorageResponse::DroppedIds(dropped_ids) => {
                tracing::debug!("dropped IDs: {:?}", dropped_ids);

                for id in dropped_ids.iter() {
                    self.sources.remove(id);
                    self.sinks.remove(id);
                    self.uppers.remove(id);
                    self.sinces.remove(id);
                }
                Some(StorageResponse::DroppedIds(dropped_ids))
            }
            StorageResponse::StatisticsUpdates(source_stats, sink_stats) => {
                // Just forward it along.
                Some(StorageResponse::StatisticsUpdates(source_stats, sink_stats))
            }
        }
    }
}