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
// 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.

//! Code to spin up communication mesh for a compute replica.
//!
//! The startup protocol is as follows:
//! The compute controller in `environmentd`, after having connected to all the
//! `clusterd` processes in a replica, sends each of them a `CreateTimely` command
//! containing an epoch value (which is the same across all copies of the command).
//! The meaning of this value is irrelevant,
//! as long as it is totally ordered and
//! increases monotonically (including across `environmentd` restarts)
//!
//! In the past, we've seen issues caused by `environmentd`'s replica connections
//! flapping repeatedly and causing several instances of the startup code to spin up
//! in short succession (or even simultaneously) in response to different `CreateTimely`
//! commands, causing mass confusion among the processes
//! and possible crash loops. To avoid this, we do not allow processes to connect to each
//! other unless they are responding to a `CreateTimely` command with the same epoch value.
//! If a process discovers the existence of a peer with a lower epoch value, it ignores it,
//! and if it discovers one with a higher epoch value, it aborts the connection.
//! Such a process is guaranteed to eventually hear about the higher epoch value
//! (and, thus, successfully connect to its peers), since
//! `environmentd` sends `CreateTimely` commands to all processes in a replica.
//!
//! Concretely, each process awaits connections from its peers with higher indices,
//! and initiates connections to those with lower indices. Having established
//! a TCP connection, they exchange epochs, to enable the logic described above.

use std::any::Any;
use std::cmp::Ordering;
use std::fmt::Display;
use std::time::Duration;

use anyhow::Context;
use futures::stream::FuturesUnordered;
use futures::FutureExt;
use futures::StreamExt;
use timely::communication::allocator::zero_copy::initialize::initialize_networking_from_sockets;
use timely::communication::allocator::GenericBuilder;
use tracing::{debug, info, warn};

use mz_compute_client::protocol::command::ComputeStartupEpoch;
use mz_ore::cast::CastFrom;
use mz_ore::netio::{Listener, Stream};

/// Creates communication mesh from cluster config
pub async fn initialize_networking(
    workers: usize,
    process: usize,
    addresses: Vec<String>,
    epoch: ComputeStartupEpoch,
) -> Result<(Vec<GenericBuilder>, Box<dyn Any + Send>), anyhow::Error> {
    info!(
        process = process,
        "initializing network for multi-process timely instance, with {} processes for epoch number {epoch}",
        addresses.len()
    );
    let sockets = create_sockets(addresses, u64::cast_from(process), epoch)
        .await
        .context("failed to set up timely sockets")?;

    if sockets
        .iter()
        .filter_map(|s| s.as_ref())
        .all(|s| s.is_tcp())
    {
        let sockets = sockets
            .into_iter()
            .map(|s| s.map(|s| s.unwrap_tcp().into_std()).transpose())
            .collect::<Result<Vec<_>, _>>()
            .map_err(anyhow::Error::from)
            .context("failed to get standard sockets from tokio sockets")?;
        initialize_networking_inner(sockets, process, workers)
    } else if sockets
        .iter()
        .filter_map(|s| s.as_ref())
        .all(|s| s.is_unix())
    {
        let sockets = sockets
            .into_iter()
            .map(|s| s.map(|s| s.unwrap_unix().into_std()).transpose())
            .collect::<Result<Vec<_>, _>>()
            .map_err(anyhow::Error::from)
            .context("failed to get standard sockets from tokio sockets")?;
        initialize_networking_inner(sockets, process, workers)
    } else {
        anyhow::bail!("cannot mix TCP and Unix streams");
    }
}

fn initialize_networking_inner<S>(
    sockets: Vec<Option<S>>,
    process: usize,
    workers: usize,
) -> Result<(Vec<GenericBuilder>, Box<dyn Any + Send>), anyhow::Error>
where
    S: timely::communication::allocator::zero_copy::stream::Stream + 'static,
{
    for s in &sockets {
        if let Some(s) = s {
            s.set_nonblocking(false)
                .context("failed to set socket to non-blocking")?;
        }
    }

    match initialize_networking_from_sockets(sockets, process, workers, Box::new(|_| None)) {
        Ok((stuff, guard)) => {
            info!(process = process, "successfully initialized network");
            Ok((
                stuff.into_iter().map(GenericBuilder::ZeroCopy).collect(),
                Box::new(guard),
            ))
        }
        Err(err) => {
            warn!(process, "failed to initialize network: {err}");
            Err(anyhow::Error::from(err).context("failed to initialize networking from sockets"))
        }
    }
}

/// Creates socket connections from a list of host addresses.
///
/// The item at index i in the resulting vec, is a Some(TcpSocket) to process i, except
/// for item `my_index` which is None (no socket to self).
async fn create_sockets(
    addresses: Vec<String>,
    my_index: u64,
    my_epoch: ComputeStartupEpoch,
) -> Result<Vec<Option<Stream>>, anyhow::Error> {
    let my_index_uz = usize::cast_from(my_index);
    assert!(my_index_uz < addresses.len());
    let n_peers = addresses.len() - 1;
    let mut results: Vec<_> = (0..addresses.len()).map(|_| None).collect();

    let my_address = &addresses[my_index_uz];
    let listener = loop {
        let mut tries = 0;
        match Listener::bind(my_address.clone()).await {
            Ok(ok) => break ok,
            Err(e) => {
                warn!("failed to listen on address {my_address}: {e}");
                tries += 1;
                if tries == 10 {
                    return Err(e.into());
                }
                tokio::time::sleep(Duration::from_secs(1)).await;
            }
        }
    };

    struct ConnectionEstablished {
        peer_index: u64,
        stream: Stream,
    }

    let mut futs = FuturesUnordered::new();
    for i in 0..my_index {
        let address = addresses[usize::cast_from(i)].clone();
        futs.push(
            async move {
                start_connection(address, my_index, my_epoch)
                    .await
                    .map(move |stream| ConnectionEstablished {
                        peer_index: i,
                        stream,
                    })
            }
            .boxed(),
        );
    }

    futs.push({
        let f = async {
            await_connection(&listener, my_index, my_epoch)
                .await
                .map(|(stream, peer_index)| ConnectionEstablished { peer_index, stream })
        }
        .boxed();
        f
    });

    while results.iter().filter(|maybe| maybe.is_some()).count() != n_peers {
        let ConnectionEstablished { peer_index, stream } = futs
            .next()
            .await
            .expect("we should always at least have a listener task")?;

        let from_listener = match my_index.cmp(&peer_index) {
            Ordering::Less => true,
            Ordering::Equal => panic!("someone claimed to be us"),
            Ordering::Greater => false,
        };

        if from_listener {
            futs.push({
                let f = async {
                    await_connection(&listener, my_index, my_epoch)
                        .await
                        .map(|(stream, peer_index)| ConnectionEstablished { peer_index, stream })
                }
                .boxed();
                f
            });
        }

        let old = std::mem::replace(&mut results[usize::cast_from(peer_index)], Some(stream));
        if old.is_some() {
            panic!("connected to peer {peer_index} multiple times");
        }
    }

    Ok(results)
}

#[derive(Debug)]
/// This task can never successfully boot, since
/// a peer has seen a higher epoch from `environmentd`.
pub struct EpochMismatch {
    /// The epoch we know about
    mine: ComputeStartupEpoch,
    /// The higher epoch from our peer
    peer: ComputeStartupEpoch,
}

impl Display for EpochMismatch {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        let EpochMismatch { mine, peer } = self;
        write!(f, "epoch mismatch: ours was {mine}; the peer's was {peer}")
    }
}

impl std::error::Error for EpochMismatch {}

async fn start_connection(
    address: String,
    my_index: u64,
    my_epoch: ComputeStartupEpoch,
) -> Result<Stream, anyhow::Error> {
    loop {
        info!(
            process = my_index,
            "Attempting to connect to process at {address}"
        );

        match Stream::connect(&address).await {
            Ok(mut s) => {
                if let Stream::Tcp(tcp) = &s {
                    tcp.set_nodelay(true)?;
                }
                use tokio::io::AsyncWriteExt;

                s.write_all(&my_index.to_be_bytes()).await?;

                s.write_all(&my_epoch.to_bytes()).await?;

                let mut buffer = [0u8; 16];
                use tokio::io::AsyncReadExt;
                s.read_exact(&mut buffer).await?;
                let peer_epoch = ComputeStartupEpoch::from_bytes(buffer);
                debug!("start: received peer epoch {peer_epoch}");

                match my_epoch.cmp(&peer_epoch) {
                    Ordering::Less => {
                        return Err(EpochMismatch {
                            mine: my_epoch,
                            peer: peer_epoch,
                        }
                        .into());
                    }
                    Ordering::Greater => {
                        warn!(
                            process = my_index,
                            "peer at address {address} gave older epoch ({peer_epoch}) than ours ({my_epoch})"
                        );
                    }
                    Ordering::Equal => return Ok(s),
                }
            }
            Err(err) => {
                info!(
                    process = my_index,
                    "error connecting to process at {address}: {err}; will retry"
                );
            }
        }
        tokio::time::sleep(Duration::from_secs(1)).await;
    }
}

async fn await_connection(
    listener: &Listener,
    my_index: u64, // only for logging
    my_epoch: ComputeStartupEpoch,
) -> Result<(Stream, u64), anyhow::Error> {
    loop {
        info!(process = my_index, "awaiting connection from peer");
        let mut s = listener.accept().await?.0;
        info!(process = my_index, "accepted connection from peer");
        if let Stream::Tcp(tcp) = &s {
            tcp.set_nodelay(true)?;
        }

        let mut buffer = [0u8; 16];
        use tokio::io::AsyncReadExt;

        s.read_exact(&mut buffer[0..8]).await?;
        let peer_index = u64::from_be_bytes((&buffer[0..8]).try_into().unwrap());
        debug!("await: received peer index {peer_index}");

        s.read_exact(&mut buffer).await?;
        let peer_epoch = ComputeStartupEpoch::from_bytes(buffer);
        debug!("await: received peer epoch {peer_epoch}");

        use tokio::io::AsyncWriteExt;
        s.write_all(&my_epoch.to_bytes()[..]).await?;

        match my_epoch.cmp(&peer_epoch) {
            Ordering::Less => {
                return Err(EpochMismatch {
                    mine: my_epoch,
                    peer: peer_epoch,
                }
                .into());
            }
            Ordering::Greater => {
                warn!(
                    process = my_index,
                    "peer {peer_index} gave older epoch ({peer_epoch}) than ours ({my_epoch})"
                );
            }
            Ordering::Equal => return Ok((s, peer_index)),
        }
    }
}