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

//! gRPC transport for the [client](crate::client) module.

use async_stream::stream;
use async_trait::async_trait;
use futures::future;
use futures::stream::{Stream, StreamExt, TryStreamExt};
use http::uri::PathAndQuery;
use mz_ore::netio::{Listener, SocketAddr, SocketAddrType};
use mz_proto::{ProtoType, RustType};
use once_cell::sync::Lazy;
use semver::Version;
use std::fmt::{self, Debug};
use std::pin::Pin;
use std::sync::Arc;
use tokio::net::UnixStream;
use tokio::select;
use tokio::sync::mpsc::{self, UnboundedSender};
use tokio::sync::{oneshot, Mutex};
use tokio_stream::wrappers::UnboundedReceiverStream;
use tonic::body::BoxBody;
use tonic::codegen::InterceptedService;
use tonic::metadata::{AsciiMetadataKey, AsciiMetadataValue};
use tonic::service::Interceptor;
use tonic::transport::{Body, Channel, Endpoint, NamedService, Server};
use tonic::{IntoStreamingRequest, Request, Response, Status, Streaming};
use tower::Service;
use tracing::{debug, error, info};

use crate::client::{GenericClient, Partitionable, Partitioned};
use crate::codec::{StatCodec, StatsCollector};
use crate::params::GrpcClientParameters;

include!(concat!(env!("OUT_DIR"), "/mz_service.params.rs"));

// Use with generated servers Server::new(Svc).max_decoding_message_size
pub const MAX_GRPC_MESSAGE_SIZE: usize = usize::MAX;

pub type ResponseStream<PR> = Pin<Box<dyn Stream<Item = Result<PR, Status>> + Send>>;

pub type ClientTransport = InterceptedService<Channel, VersionAttachInterceptor>;

/// Types that we send and receive over a service endpoint.
pub trait ProtoServiceTypes: Debug + Clone + Send {
    type PC: prost::Message + Clone + 'static;
    type PR: prost::Message + Clone + Default + 'static;
    type STATS: StatsCollector<Self::PC, Self::PR> + 'static;
    const URL: &'static str;
}

/// A client to a remote dataflow server using gRPC and protobuf based
/// communication.
///
/// The client opens a connection using the proto client stubs that are
/// generated by tonic from a service definition. When the client is connected,
/// it will call automatically the only RPC defined in the service description,
/// encapsulated by the `BidiProtoClient` trait. This trait bound is not on the
/// `Client` type parameter here, but it IS on the impl blocks. Bidirectional
/// protobuf RPC sets up two streams that persist after the RPC has returned: A
/// Request (Command) stream (for us, backed by a unbounded mpsc queue) going
/// from this instance to the server and a response stream coming back
/// (represented directly as a `Streaming<Response>` instance). The recv and send
/// functions interact with the two mpsc channels or the streaming instance
/// respectively.
#[derive(Debug)]
pub struct GrpcClient<G>
where
    G: ProtoServiceTypes,
{
    /// The sender for commands.
    tx: UnboundedSender<G::PC>,
    /// The receiver for responses.
    rx: Streaming<G::PR>,
}

impl<G> GrpcClient<G>
where
    G: ProtoServiceTypes,
{
    /// Connects to the server at the given address, announcing the specified
    /// client version.
    pub async fn connect(
        addr: String,
        version: Version,
        metrics: G::STATS,
        params: &GrpcClientParameters,
    ) -> Result<Self, anyhow::Error> {
        debug!("GrpcClient {}: Attempt to connect", addr);

        let channel = match SocketAddrType::guess(&addr) {
            SocketAddrType::Inet => {
                let mut endpoint = Endpoint::new(format!("http://{}", addr))?;
                if let Some(connect_timeout) = params.connect_timeout {
                    endpoint = endpoint.connect_timeout(connect_timeout);
                }
                if let Some(keep_alive_timeout) = params.http2_keep_alive_timeout {
                    endpoint = endpoint.keep_alive_timeout(keep_alive_timeout);
                }
                if let Some(keep_alive_interval) = params.http2_keep_alive_interval {
                    endpoint = endpoint.http2_keep_alive_interval(keep_alive_interval);
                }
                endpoint.connect().await?
            }
            SocketAddrType::Unix => {
                let addr = addr.clone();
                Endpoint::from_static("http://localhost") // URI is ignored
                    .connect_with_connector(tower::service_fn(move |_| {
                        UnixStream::connect(addr.clone())
                    }))
                    .await?
            }
        };
        let service = InterceptedService::new(channel, VersionAttachInterceptor::new(version));
        let mut client = BidiProtoClient::new(service, G::URL, metrics);
        let (tx, rx) = mpsc::unbounded_channel();
        let rx = client
            .establish_bidi_stream(UnboundedReceiverStream::new(rx))
            .await?
            .into_inner();
        info!("GrpcClient {}: connected", &addr);
        Ok(GrpcClient { tx, rx })
    }

    /// Like [`GrpcClient::connect`], but for multiple partitioned servers.
    pub async fn connect_partitioned<C, R>(
        dests: Vec<(String, G::STATS)>,
        version: Version,
        params: &GrpcClientParameters,
    ) -> Result<Partitioned<Self, C, R>, anyhow::Error>
    where
        (C, R): Partitionable<C, R>,
    {
        let clients = future::try_join_all(
            dests
                .into_iter()
                .map(|(addr, metrics)| Self::connect(addr, version.clone(), metrics, params)),
        )
        .await?;
        Ok(Partitioned::new(clients))
    }
}

#[async_trait]
impl<G, C, R> GenericClient<C, R> for GrpcClient<G>
where
    C: RustType<G::PC> + Send + Sync + 'static,
    R: RustType<G::PR> + Send + Sync + 'static,
    G: ProtoServiceTypes,
{
    async fn send(&mut self, cmd: C) -> Result<(), anyhow::Error> {
        self.tx.send(cmd.into_proto())?;
        Ok(())
    }

    async fn recv(&mut self) -> Result<Option<R>, anyhow::Error> {
        match self.rx.try_next().await? {
            None => Ok(None),
            Some(response) => Ok(Some(response.into_rust()?)),
        }
    }
}

/// Encapsulates the core functionality of a tonic gRPC client for a service
/// that exposes a single bidirectional RPC stream.
///
/// The client calls back into the StatsCollector on each command send and
/// response receive.
///
/// See the documentation on [`GrpcClient`] for details.
pub struct BidiProtoClient<PC, PR, S>
where
    PC: prost::Message + 'static,
    PR: Default + prost::Message + 'static,
    S: StatsCollector<PC, PR>,
{
    inner: tonic::client::Grpc<ClientTransport>,
    path: &'static str,
    codec: StatCodec<PC, PR, S>,
}

impl<PC, PR, S> BidiProtoClient<PC, PR, S>
where
    PC: Clone + prost::Message + 'static,
    PR: Clone + Default + prost::Message + 'static,
    S: StatsCollector<PC, PR> + 'static,
{
    fn new(inner: ClientTransport, path: &'static str, stats_collector: S) -> Self
    where
        Self: Sized,
    {
        let inner = tonic::client::Grpc::new(inner)
            .max_decoding_message_size(MAX_GRPC_MESSAGE_SIZE)
            .max_encoding_message_size(MAX_GRPC_MESSAGE_SIZE);
        let codec = StatCodec::new(stats_collector);
        BidiProtoClient { inner, path, codec }
    }

    async fn establish_bidi_stream(
        &mut self,
        rx: UnboundedReceiverStream<PC>,
    ) -> Result<Response<Streaming<PR>>, Status> {
        self.inner.ready().await.map_err(|e| {
            tonic::Status::new(
                tonic::Code::Unknown,
                format!("Service was not ready: {}", e),
            )
        })?;
        let path = PathAndQuery::from_static(self.path);
        self.inner
            .streaming(rx.into_streaming_request(), path, self.codec.clone())
            .await
    }
}

/// A gRPC server that stitches a gRPC service with a single bidirectional
/// stream to a [`GenericClient`].
///
/// It is the counterpart of [`GrpcClient`].
///
/// To use, implement the tonic-generated `ProtoService` trait for this type.
/// The implementation of the bidirectional stream method should call
/// [`GrpcServer::forward_bidi_stream`] to stitch the bidirectional stream to
/// the client underlying this server.
pub struct GrpcServer<F> {
    state: Arc<GrpcServerState<F>>,
}

struct GrpcServerState<F> {
    cancel_tx: Mutex<oneshot::Sender<()>>,
    client_builder: F,
}

impl<F, G> GrpcServer<F>
where
    F: Fn() -> G + Send + Sync + 'static,
{
    /// Starts the server, listening for gRPC connections on `listen_addr`.
    ///
    /// The trait bounds on `S` are intimidating, but it is the return type of
    /// `service_builder`, which is a function that
    /// turns a `GrpcServer<ProtoCommandType, ProtoResponseType>` into a
    /// [`Service`] that represents a gRPC server. This is always encapsulated
    /// by the tonic-generated `ProtoServer::new` method for a specific Protobuf
    /// service.
    pub async fn serve<S, Fs>(
        listen_addr: SocketAddr,
        version: Version,
        client_builder: F,
        service_builder: Fs,
    ) -> Result<(), anyhow::Error>
    where
        S: Service<
                http::Request<Body>,
                Response = http::Response<BoxBody>,
                Error = std::convert::Infallible,
            > + NamedService
            + Clone
            + Send
            + 'static,
        S::Future: Send + 'static,
        Fs: FnOnce(Self) -> S + Send + 'static,
    {
        let (cancel_tx, _cancel_rx) = oneshot::channel();
        let state = GrpcServerState {
            cancel_tx: Mutex::new(cancel_tx),
            client_builder,
        };
        let server = Self {
            state: Arc::new(state),
        };
        let service = InterceptedService::new(
            service_builder(server),
            VersionCheckExactInterceptor::new(version),
        );

        info!("Starting to listen on {}", listen_addr);
        let listener = Listener::bind(listen_addr).await?;

        Server::builder()
            .add_service(service)
            .serve_with_incoming(listener)
            .await?;
        Ok(())
    }

    /// Handles a bidirectional stream request by forwarding commands to and
    /// responses from the server's underlying client.
    ///
    /// Call this method from the implementation of the tonic-generated
    /// `ProtoService`.
    pub async fn forward_bidi_stream<C, R, PC, PR>(
        &self,
        request: Request<Streaming<PC>>,
    ) -> Result<Response<ResponseStream<PR>>, Status>
    where
        G: GenericClient<C, R> + 'static,
        C: RustType<PC> + Send + Sync + 'static + fmt::Debug,
        R: RustType<PR> + Send + Sync + 'static + fmt::Debug,
        PC: fmt::Debug + Send + Sync + 'static,
        PR: fmt::Debug + Send + Sync + 'static,
    {
        info!("GrpcServer: remote client connected");

        // Install our cancellation token. This may drop an existing
        // cancellation token. We're allowed to run until someone else drops our
        // cancellation token.
        //
        // TODO(benesch): rather than blindly dropping the existing cancellation
        // token, we should check epochs, and only drop the existing connection
        // if it is at a lower epoch.
        // See: https://github.com/MaterializeInc/materialize/issues/13377
        let (cancel_tx, mut cancel_rx) = oneshot::channel();
        *self.state.cancel_tx.lock().await = cancel_tx;

        // Construct a new client and forward commands and responses until
        // canceled.
        let mut request = request.into_inner();
        let state = Arc::clone(&self.state);
        let response = stream! {
            let mut client = (state.client_builder)();
            loop {
                select! {
                    command = request.next() => {
                        let command = match command {
                            None => break,
                            Some(Ok(command)) => command,
                            Some(Err(e)) => {
                                error!("error handling client: {e}");
                                break;
                            }
                        };
                        let command = match command.into_rust() {
                            Ok(command) => command,
                            Err(e) => {
                                error!("error converting command from protobuf: {}", e);
                                break;
                            }
                        };
                        if let Err(e) = client.send(command).await {
                            yield Err(Status::unknown(e.to_string()));
                        }
                    }
                    response = client.recv() => {
                        match response {
                            Ok(Some(response)) => yield Ok(response.into_proto()),
                            Ok(None) => break,
                            Err(e) => yield Err(Status::unknown(e.to_string())),
                        }
                    }
                    _ = &mut cancel_rx => break,
                }
            }
            info!("GrpcServer: remote client disconnected");
        };
        Ok(Response::new(Box::pin(response)))
    }
}

static VERSION_METADATA_KEY: Lazy<AsciiMetadataKey> =
    Lazy::new(|| AsciiMetadataKey::from_static("x-mz-version"));

/// A gRPC interceptor that attaches a version as metadata to each request.
#[derive(Debug, Clone)]
pub struct VersionAttachInterceptor {
    version: AsciiMetadataValue,
}

impl VersionAttachInterceptor {
    fn new(version: Version) -> VersionAttachInterceptor {
        VersionAttachInterceptor {
            version: version
                .to_string()
                .try_into()
                .expect("semver versions are valid metadata values"),
        }
    }
}

impl Interceptor for VersionAttachInterceptor {
    fn call(&mut self, mut request: Request<()>) -> Result<Request<()>, Status> {
        request
            .metadata_mut()
            .insert(VERSION_METADATA_KEY.clone(), self.version.clone());
        Ok(request)
    }
}

/// A gRPC interceptor that ensures the version attached to the request by the
/// `VersionAttachInterceptor` exactly matches the expected version.
#[derive(Debug, Clone)]
struct VersionCheckExactInterceptor {
    version: AsciiMetadataValue,
}

impl VersionCheckExactInterceptor {
    fn new(version: Version) -> VersionCheckExactInterceptor {
        VersionCheckExactInterceptor {
            version: version
                .to_string()
                .try_into()
                .expect("semver versions are valid metadata values"),
        }
    }
}

impl Interceptor for VersionCheckExactInterceptor {
    fn call(&mut self, request: Request<()>) -> Result<Request<()>, Status> {
        match request.metadata().get(&*VERSION_METADATA_KEY) {
            None => Err(Status::permission_denied(
                "request missing version metadata",
            )),
            Some(version) if version == self.version => Ok(request),
            Some(version) => Err(Status::permission_denied(format!(
                "request version {:?} but {:?} required",
                version, self.version
            ))),
        }
    }
}