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
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.

//! This module provides a TCP-based boundary.

use serde::{Deserialize, Serialize};
use timely::dataflow::operators::capture::EventCore;
use timely::logging::WorkerIdentifier;
use tokio_serde::formats::Bincode;
use tokio_util::codec::LengthDelimitedCodec;

use mz_dataflow_types::DataflowError;
use mz_dataflow_types::SourceInstanceId;

/// Type alias for a source subscription including a source worker.
pub type SubscriptionId = (SourceInstanceId, WorkerIdentifier);

pub mod server {
    //! TCP boundary server
    use super::SubscriptionId;
    use super::{length_delimited_codec, Command, Framed, Response};
    use crate::boundary::StorageCapture;
    use differential_dataflow::Collection;
    use futures::{SinkExt, TryStreamExt};
    use std::any::Any;
    use std::collections::{HashMap, HashSet};
    use std::marker::{Send, Sync};
    use std::sync::{Arc, Weak};
    use timely::dataflow::operators::capture::{EventCore, EventPusherCore};
    use timely::dataflow::operators::Capture;
    use timely::dataflow::Scope;
    use tokio::io::{AsyncRead, AsyncWrite};
    use tokio::net::{TcpListener, TcpStream, ToSocketAddrs};
    use tokio::select;
    use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender};
    use tokio::sync::Mutex;
    use tokio::task::JoinHandle;
    use tokio_serde::formats::Bincode;
    use tracing::{debug, info, warn};

    use mz_dataflow_types::DataflowError;
    use mz_dataflow_types::SourceInstanceRequest;
    use mz_repr::{Diff, Row, Timestamp};

    /// A framed connection from the server's perspective.
    type FramedServer<C> = Framed<C, Command, Response>;

    /// Constructs a framed connection for the server.
    fn framed_server<C>(conn: C) -> FramedServer<C>
    where
        C: AsyncRead + AsyncWrite,
    {
        tokio_serde::Framed::new(
            tokio_util::codec::Framed::new(conn, length_delimited_codec()),
            Bincode::default(),
        )
    }

    /// Unique client identifier, only used internally
    type ClientId = u64;

    /// Shared state between main server loop and client-specific task.
    #[derive(Debug, Default)]
    struct Shared {
        subscriptions: HashMap<SubscriptionId, SourceState>,
        channels: HashMap<ClientId, UnboundedSender<Response>>,
    }

    /// Shared per-subscription state
    #[derive(Debug, Default)]
    struct SourceState {
        /// Pending responses stashed before client registered.
        stash: Vec<Response>,
        /// Client, once it registered
        client_id: Option<ClientId>,
        /// Tokens to drop to terminate source.
        token: Option<Arc<dyn Any + Send + Sync>>,
    }

    /// Response from Timely workers to network handler.
    #[derive(Debug)]
    enum WorkerResponse {
        /// Announce the presence of a captured source.
        Announce {
            subscription_id: SubscriptionId,
            token: Arc<dyn Any + Send + Sync>,
        },
        /// Data from a source
        Response(Response),
    }

    /// Alias for a commonly used type for sending source instantiation requests.
    type SourceRequestSender = tokio::sync::mpsc::UnboundedSender<SourceInstanceRequest>;

    /// Start the boundary server listening for connections on `addr`.
    ///
    /// Returns a handle implementing [StorageCapture] and a join handle to await termination.
    pub async fn serve<A: ToSocketAddrs + std::fmt::Display>(
        addr: A,
        storage_workers: usize,
    ) -> std::io::Result<(
        TcpEventLinkHandle,
        tokio::sync::mpsc::UnboundedReceiver<SourceInstanceRequest>,
        JoinHandle<std::io::Result<()>>,
    )> {
        let (worker_tx, worker_rx) = unbounded_channel();

        let (request_tx, request_rx) = tokio::sync::mpsc::unbounded_channel();

        info!("About to bind to {addr}");
        let listener = TcpListener::bind(addr).await?;
        info!(
            "listening for storage connection on {}...",
            listener.local_addr()?
        );

        let thread = mz_ore::task::spawn(
            || "storage server",
            accept(listener, worker_rx, request_tx, storage_workers),
        );

        Ok((TcpEventLinkHandle { worker_tx }, request_rx, thread))
    }

    /// Accept connections on `listener` and listening for data on `worker_rx`.
    async fn accept(
        listener: TcpListener,
        mut worker_rx: UnboundedReceiver<WorkerResponse>,
        render_requests: SourceRequestSender,
        storage_workers: usize,
    ) -> std::io::Result<()> {
        debug!("server listening {listener:?}");

        let shared = Default::default();
        let state = Arc::new(Mutex::new(shared));

        let client_id = std::sync::atomic::AtomicU64::default();

        loop {
            select! {
                _ = async {
                    loop {
                        match listener.accept().await {
                            Ok((socket, addr)) => {
                                let id = client_id.fetch_add(1, std::sync::atomic::Ordering::SeqCst);
                                let render_requests = render_requests.clone();
                                debug!("Accepting client {id} on {addr}...");
                                let state = Arc::clone(&state);
                                mz_ore::task::spawn(|| "client loop", async move {
                                    handle_compute(id, Arc::clone(&state), socket, render_requests.clone(), storage_workers).await
                                });
                            }
                            Err(e) => {
                                warn!("Failed to accept connection: {e}");
                            }
                        }
                    }
                } => {}
                worker_response = worker_rx.recv() => match worker_response {
                    Some(WorkerResponse::Announce{subscription_id, token}) => {
                        debug!("Subscription announced: {subscription_id:?}");
                        let mut state = state.lock().await;
                        let subscription = state.subscriptions.entry(subscription_id).or_default();
                        subscription.token = Some(token);
                    }
                    Some(WorkerResponse::Response(response)) => {
                        let mut state = state.lock().await;
                        if let Some(subscription) = state.subscriptions.get_mut(&response.subscription_id) {
                            if let Some(client_id) = subscription.client_id {
                                if let Some(channel) = state.channels.get_mut(&client_id){
                                    let _ = channel.send(response);
                                }
                            } else {
                                subscription.stash.push(response);
                            }
                        }
                    }
                    None => return Ok(()),
                },
            }
        }
    }

    /// Handle the server side of a compute client connection in `socket`, identified by
    /// `client_id`. `state` is a handle to the shared state.
    ///
    /// This function calls into the inner part and upon client termination cleans up state.
    async fn handle_compute(
        client_id: ClientId,
        state: Arc<Mutex<Shared>>,
        socket: TcpStream,
        render_requests: SourceRequestSender,
        storage_workers: usize,
    ) -> std::io::Result<()> {
        let mut source_ids = HashSet::new();
        let result = handle_compute_inner(
            client_id,
            Arc::clone(&state),
            socket,
            &mut source_ids,
            render_requests,
            storage_workers,
        )
        .await;
        let mut state = state.lock().await;
        for source_id in source_ids {
            state.subscriptions.remove(&source_id);
        }
        state.channels.remove(&client_id);
        result
    }

    /// Inner portion to handle a compute client.
    async fn handle_compute_inner(
        client_id: ClientId,
        state: Arc<Mutex<Shared>>,
        mut socket: TcpStream,
        active_source_ids: &mut HashSet<SubscriptionId>,
        render_requests: SourceRequestSender,
        storage_workers: usize,
    ) -> std::io::Result<()> {
        /// Announce storage configuration
        use tokio_byteorder::{AsyncWriteBytesExt, NetworkEndian};
        socket
            .write_u64::<NetworkEndian>(storage_workers as u64)
            .await?;

        let mut connection = framed_server(socket);

        let (client_tx, mut client_rx) = unbounded_channel();
        state.lock().await.channels.insert(client_id, client_tx);

        loop {
            select! {
                cmd = connection.try_next() => match cmd? {
                    Some(Command::Subscribe(subscription_id, request)) => {
                        debug!("Subscribe client {client_id} to {subscription_id:?}");
                        let _ = render_requests.send(request);
                        let new = active_source_ids.insert(subscription_id);
                        assert!(new, "Duplicate key: {subscription_id:?}");
                        let stashed = {
                            let mut state = state.lock().await;
                            let subscription = state.subscriptions.entry(subscription_id).or_default();
                            assert!(subscription.client_id.is_none());
                            subscription.client_id = Some(client_id);
                            std::mem::take(&mut subscription.stash)
                        };
                        for stashed in stashed {
                            connection.send(stashed).await?;
                        }
                    }
                    Some(Command::Unsubscribe(subscription_id)) => {
                        debug!("Unsubscribe client {client_id} from {subscription_id:?}");
                        let mut state = state.lock().await;
                        state.subscriptions.remove(&subscription_id);
                        let removed = active_source_ids.remove(&subscription_id);
                        assert!(removed, "Unknown key: {subscription_id:?}");
                    }
                    None => {
                        return Ok(());
                    }
                },
                Some(response) = client_rx.recv() => connection.send(response).await?,
            }
        }
    }

    /// A handle to the boundary service. Implements [StorageCapture] to capture sources.
    #[derive(Clone, Debug)]
    pub struct TcpEventLinkHandle {
        worker_tx: UnboundedSender<WorkerResponse>,
    }

    impl StorageCapture for TcpEventLinkHandle {
        fn capture<G: Scope<Timestamp = Timestamp>>(
            &mut self,
            id: mz_repr::GlobalId,
            ok: Collection<G, Row, Diff>,
            err: Collection<G, DataflowError, Diff>,
            token: Arc<dyn Any + Send + Sync>,
            _name: &str,
            dataflow_id: uuid::Uuid,
        ) {
            let subscription_id = ((dataflow_id, id), ok.inner.scope().index());

            // Announce that we're capturing data, with the source ID and a token. Once the token
            // is dropped, we drop the `token` to terminate the source.
            self.worker_tx
                .send(WorkerResponse::Announce {
                    subscription_id,
                    token: Arc::clone(&token),
                })
                .unwrap();

            ok.inner.capture_into(UnboundedEventPusher::new(
                self.worker_tx.clone(),
                &token,
                move |event| {
                    WorkerResponse::Response(Response {
                        subscription_id,
                        data: Ok(event),
                    })
                },
            ));
            err.inner.capture_into(UnboundedEventPusher::new(
                self.worker_tx.clone(),
                &token,
                move |event| {
                    WorkerResponse::Response(Response {
                        subscription_id,
                        data: Err(event),
                    })
                },
            ));
        }
    }

    /// Helper struct to capture data into a sender and drop tokens once dropped itself.
    struct UnboundedEventPusher<R, F> {
        /// The sender to pass all data to.
        sender: UnboundedSender<R>,
        /// A function to convert the input data into something the sender can transport.
        convert: F,
        /// A weak reference to a token that e.g. the network layer can drop.
        client_token: Weak<dyn Any + Send + Sync>,
    }

    impl<R, F> UnboundedEventPusher<R, F> {
        /// Construct a new pusher. It'll retain a weak reference to `client_token`.
        fn new(
            sender: UnboundedSender<R>,
            client_token: &Arc<dyn Any + Send + Sync>,
            convert: F,
        ) -> Self {
            Self {
                sender,
                client_token: Arc::downgrade(client_token),
                convert,
            }
        }
    }

    impl<T, D, R, F: Fn(EventCore<T, D>) -> R> EventPusherCore<T, D> for UnboundedEventPusher<R, F> {
        fn push(&mut self, event: EventCore<T, D>) {
            if self.client_token.upgrade().is_some() {
                // Loss of the receiving end means we need to do nothing,
                // so we ignore errors
                // TODO(guswynn): can we handle failure better here?
                let _ = self.sender.send((self.convert)(event));
            }
        }
    }
}

pub mod client {
    //! TCP boundary client
    use crate::boundary::tcp_boundary::{length_delimited_codec, Command, Framed, Response};
    use crate::boundary::ComputeReplay;
    use differential_dataflow::{AsCollection, Collection};
    use futures::{Sink, SinkExt, TryStreamExt};
    use mz_timely_util::activator::{ActivatorTrait, ArcActivator};
    use mz_timely_util::replay::MzReplay;
    use std::any::Any;
    use std::collections::HashMap;
    use std::rc::Rc;
    use std::time::Duration;
    use timely::dataflow::operators::capture::event::EventIteratorCore;
    use timely::dataflow::operators::capture::EventCore;
    use timely::dataflow::Scope;
    use timely::logging::WorkerIdentifier;
    use tokio::io::{AsyncRead, AsyncWrite};
    use tokio::net::{TcpStream, ToSocketAddrs};
    use tokio::select;
    use tokio::sync::mpsc::error::TryRecvError;
    use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender};
    use tokio::sync::oneshot::Sender;
    use tokio::task::JoinHandle;
    use tokio::time;
    use tokio_serde::formats::Bincode;
    use tracing::{debug, info, warn};

    use mz_dataflow_types::{DataflowError, SourceInstanceId};
    use mz_repr::{Diff, Row, Timestamp};

    /// A framed connection from the client's perspective.
    type FramedClient<C> = Framed<C, Response, Command>;

    /// Constructs a framed connection for the client.
    fn framed_client<C>(conn: C) -> FramedClient<C>
    where
        C: AsyncRead + AsyncWrite,
    {
        tokio_serde::Framed::new(
            tokio_util::codec::Framed::new(conn, length_delimited_codec()),
            Bincode::default(),
        )
    }

    /// A handle to the storage client. Implements [ComputeReplay].
    #[derive(Debug, Clone)]
    pub struct TcpEventLinkClientHandle {
        announce_tx: UnboundedSender<Announce>,
        storage_workers: usize,
    }

    /// State per worker and source.
    #[derive(Debug)]
    struct WorkerState<T, D, A: ActivatorTrait> {
        sender: UnboundedSender<EventCore<T, D>>,
        activator: Option<A>,
    }

    impl<T, D, A: ActivatorTrait> WorkerState<T, D, A> {
        /// Construct a new state object with the provided `sender` and no activator.
        fn new(sender: UnboundedSender<EventCore<T, D>>) -> Self {
            Self {
                sender,
                activator: None,
            }
        }

        /// Register an `activator` to wake the Timely operator.
        fn register(&mut self, activator: A) {
            self.activator = Some(activator);
        }

        /// Send data at the channel and activate if the activator is set.
        fn send_and_activate(&mut self, event: EventCore<T, D>) {
            let res = self.sender.send(event);
            if res.is_err() {
                warn!("Receiver hung up");
            }
            if let Some(activator) = &mut self.activator {
                activator.activate();
            }
        }
    }

    #[derive(Debug, Default)]
    struct SubscriptionState<T, R> {
        ok_state: HashMap<WorkerIdentifier, WorkerState<T, Vec<(Row, T, R)>, ArcActivator>>,
        err_state:
            HashMap<WorkerIdentifier, WorkerState<T, Vec<(DataflowError, T, R)>, ArcActivator>>,
    }

    #[derive(Debug)]
    struct ClientState {
        subscriptions:
            HashMap<SourceInstanceId, SubscriptionState<mz_repr::Timestamp, mz_repr::Diff>>,
        workers: usize,
    }

    impl ClientState {
        fn new(workers: usize) -> Self {
            Self {
                workers,
                subscriptions: Default::default(),
            }
        }

        async fn handle_announce<C: Sink<Command, Error = std::io::Error> + Unpin>(
            &mut self,
            client: &mut C,
            announce: Announce,
        ) -> std::io::Result<()> {
            match announce {
                Announce::Register {
                    source_id,
                    worker,
                    request,
                    storage_workers,
                    ok_tx,
                    ok_activator,
                    err_activator,
                    err_tx,
                } => {
                    let state = self.subscriptions.entry(source_id).or_default();
                    state.ok_state.insert(worker, WorkerState::new(ok_tx));
                    state.err_state.insert(worker, WorkerState::new(err_tx));
                    let worker_state = state.ok_state.get_mut(&worker).unwrap();
                    worker_state.register(ok_activator);
                    let worker_state = state.err_state.get_mut(&worker).unwrap();
                    worker_state.register(err_activator);

                    // Send subscription command to server
                    debug!("Subscribing to {source_id:?}");
                    client
                        .send_all(&mut futures::stream::iter(storage_workers.into_iter().map(
                            |storage_worker| {
                                Ok(Command::Subscribe(
                                    (source_id, storage_worker),
                                    request.clone(),
                                ))
                            },
                        )))
                        .await?;
                }
                Announce::Drop(source_id, worker, storage_workers) => {
                    // Announce to unsubscribe from the source.
                    if let Some(state) = self.subscriptions.get_mut(&source_id) {
                        state.ok_state.remove(&worker);
                        state.err_state.remove(&worker);
                        debug!("Unsubscribing from {source_id:?}");
                        client
                            .send_all(&mut futures::stream::iter(storage_workers.into_iter().map(
                                |storage_worker| {
                                    Ok(Command::Unsubscribe((source_id, storage_worker)))
                                },
                            )))
                            .await?;
                        let cleanup = state.ok_state.is_empty();
                        if cleanup {
                            self.subscriptions.remove(&source_id);
                        }
                    }
                }
            }
            Ok(())
        }

        async fn handle_response(
            &mut self,
            Response {
                subscription_id,
                data,
            }: Response,
        ) {
            if let Some(state) = self.subscriptions.get_mut(&subscription_id.0) {
                let worker = subscription_id.1 % self.workers;
                match data {
                    Ok(event) => {
                        // We might have dropped the local worker already but still receive data
                        if let Some(channel) = state.ok_state.get_mut(&worker) {
                            channel.send_and_activate(event);
                        }
                    }
                    Err(event) => {
                        // We might have dropped the local worker already but still receive data
                        if let Some(channel) = state.err_state.get_mut(&worker) {
                            channel.send_and_activate(event);
                        }
                    }
                }
            }
        }
    }

    /// Connect to a storage boundary server. Returns a handle to replay sources and a join handle
    /// to await termination.
    pub async fn connect<A: ToSocketAddrs + std::fmt::Debug>(
        addr: A,
        workers: usize,
    ) -> std::io::Result<(TcpEventLinkClientHandle, JoinHandle<std::io::Result<()>>)> {
        let (announce_tx, announce_rx) = unbounded_channel();
        info!("About to connect to {addr:?}");
        let mut stream = TcpStream::connect(&addr).await;
        while let Err(e) = stream {
            tracing::warn!("Connect error: {e}; reconnecting");
            time::sleep(Duration::from_secs(1)).await;
            stream = TcpStream::connect(&addr).await;
        }
        info!("Connected to storage server");

        let (storage_workers_tx, storage_workers_rx) = tokio::sync::oneshot::channel();

        let thread = mz_ore::task::spawn(
            || "storage client",
            run_client(stream.unwrap(), announce_rx, storage_workers_tx, workers),
        );

        let storage_workers = storage_workers_rx
            .await
            .map_err(|err| std::io::Error::new(std::io::ErrorKind::Other, err))?;

        Ok((
            TcpEventLinkClientHandle {
                announce_tx,
                storage_workers,
            },
            thread,
        ))
    }

    /// Communicate data and subscriptions on `stream`, receiving local replay notifications on
    /// `announce_rx`. `workers` is the number of local Timely workers.
    async fn run_client(
        mut stream: TcpStream,
        mut announce_rx: UnboundedReceiver<Announce>,
        storage_workers_tx: Sender<usize>,
        workers: usize,
    ) -> std::io::Result<()> {
        debug!("client: connected to server");

        /// Retrieve storage configuration
        use tokio_byteorder::{AsyncReadBytesExt, NetworkEndian};
        let storage_workers = stream.read_u64::<NetworkEndian>().await?;
        let _ = storage_workers_tx.send(storage_workers as usize);

        let mut client = framed_client(stream);

        let mut state = ClientState::new(workers);

        loop {
            select! {
                response = client.try_next() => match response? {
                    Some(response) => state.handle_response(response).await,
                    None => break,
                },
                announce = announce_rx.recv() => match announce {
                    Some(announce) => state.handle_announce(&mut client, announce).await?,
                    None => break,
                }
            }
        }

        Ok(())
    }

    /// Announcement protocol from a Timely worker to the storage client.
    #[derive(Debug)]
    enum Announce {
        /// Provide activators for the source
        Register {
            source_id: SourceInstanceId,
            worker: WorkerIdentifier,
            request: mz_dataflow_types::SourceInstanceRequest,
            storage_workers: Vec<WorkerIdentifier>,
            ok_activator: ArcActivator,
            err_activator: ArcActivator,
            ok_tx: UnboundedSender<EventCore<Timestamp, Vec<(Row, Timestamp, Diff)>>>,
            err_tx: UnboundedSender<EventCore<Timestamp, Vec<(DataflowError, Timestamp, Diff)>>>,
        },
        /// Replayer dropped
        Drop(SourceInstanceId, WorkerIdentifier, Vec<WorkerIdentifier>),
    }

    impl ComputeReplay for TcpEventLinkClientHandle {
        fn replay<G: Scope<Timestamp = Timestamp>>(
            &mut self,
            scope: &mut G,
            name: &str,
            request: mz_dataflow_types::SourceInstanceRequest,
        ) -> (
            Collection<G, Row, Diff>,
            Collection<G, DataflowError, Diff>,
            Rc<dyn Any>,
        ) {
            let source_id = request.unique_id();
            // Create a channel to receive worker/replay-specific data channels
            let (ok_tx, ok_rx) = unbounded_channel();
            let (err_tx, err_rx) = unbounded_channel();
            let ok_activator = ArcActivator::new(format!("{name}-ok-activator"), 1);
            let err_activator = ArcActivator::new(format!("{name}-err-activator"), 1);

            let storage_workers = (0..self.storage_workers)
                .into_iter()
                .map(|x| scope.index() + x * scope.peers())
                .filter(|x| *x < self.storage_workers)
                .collect::<Vec<_>>();

            // Register with the storage client
            let register = Announce::Register {
                source_id,
                worker: scope.index(),
                request,
                storage_workers: storage_workers.clone(),
                ok_tx,
                err_tx,
                ok_activator: ok_activator.clone(),
                err_activator: err_activator.clone(),
            };
            self.announce_tx.send(register).unwrap();

            // Construct activators and replay data
            let mut ok_rx = Some(ok_rx);
            let (ok, ok_tok) = storage_workers
                .iter()
                .map(|_| {
                    UnboundedEventPuller::new(ok_rx.take().unwrap_or_else(|| unbounded_channel().1))
                })
                .mz_replay(scope, &format!("{name}-ok"), Duration::MAX, ok_activator);
            let ok = ok.as_collection();
            let mut err_rx = Some(err_rx);
            let (err, err_tok) = storage_workers
                .iter()
                .map(|_| {
                    UnboundedEventPuller::new(
                        err_rx.take().unwrap_or_else(|| unbounded_channel().1),
                    )
                })
                .mz_replay(scope, &format!("{name}-err"), Duration::MAX, err_activator);
            let err = err.as_collection();

            // Construct token to unsubscribe from source
            let token = Rc::new(DropReplay {
                announce_tx: self.announce_tx.clone(),
                message: Some(Announce::Drop(source_id, scope.index(), storage_workers)),
            });

            (ok, err, Rc::new((token, ok_tok, err_tok)))
        }
    }

    /// Utility to send a message on drop.
    struct DropReplay {
        /// Channel where to send the message
        announce_tx: UnboundedSender<Announce>,
        /// Pre-defined message to send
        message: Option<Announce>,
    }

    impl Drop for DropReplay {
        fn drop(&mut self) {
            if let Some(message) = self.message.take() {
                // Ignore errors on send as it indicates the client is no longer running
                let _ = self.announce_tx.send(message);
            }
        }
    }

    /// Puller reading from a `receiver` channel.
    struct UnboundedEventPuller<D> {
        /// The receiver to read from.
        receiver: UnboundedReceiver<D>,
        /// Current element
        element: Option<D>,
    }

    impl<D> UnboundedEventPuller<D> {
        /// Construct a new puller reading from `receiver`.
        fn new(receiver: UnboundedReceiver<D>) -> Self {
            Self {
                receiver,
                element: None,
            }
        }
    }

    impl<T, D> EventIteratorCore<T, D> for UnboundedEventPuller<EventCore<T, D>> {
        fn next(&mut self) -> Option<&EventCore<T, D>> {
            match self.receiver.try_recv() {
                Ok(element) => {
                    self.element = Some(element);
                    self.element.as_ref()
                }
                Err(TryRecvError::Empty) | Err(TryRecvError::Disconnected) => {
                    self.element.take();
                    None
                }
            }
        }
    }
}

/// A command sent from the storage client to the storage server.
#[derive(Clone, Debug, Serialize, Deserialize)]
enum Command {
    /// Subscribe the client to `source_id`.
    Subscribe(SubscriptionId, mz_dataflow_types::SourceInstanceRequest),
    /// Unsubscribe the client from `source_id`.
    Unsubscribe(SubscriptionId),
}

/// Data provided to the storage client from the storage server upon successful registration.
///
/// Each message has a source_id to identify the subscription, a worker that produced the data,
/// and the data itself. When replaying, the mapping of storage worker to compute worker needs
/// (at least) to be static.
#[derive(Clone, Debug, Serialize, Deserialize)]
struct Response {
    subscription_id: SubscriptionId,
    /// Contents of the Ok-collection
    data: Result<
        EventCore<mz_repr::Timestamp, Vec<(mz_repr::Row, mz_repr::Timestamp, mz_repr::Diff)>>,
        EventCore<mz_repr::Timestamp, Vec<(DataflowError, mz_repr::Timestamp, mz_repr::Diff)>>,
    >,
}

/// A framed connection to a storage server.
pub type Framed<C, T, U> =
    tokio_serde::Framed<tokio_util::codec::Framed<C, LengthDelimitedCodec>, T, U, Bincode<T, U>>;

fn length_delimited_codec() -> LengthDelimitedCodec {
    // NOTE(benesch): using an unlimited maximum frame length is problematic
    // because Tokio never shrinks its buffer. Sending or receiving one large
    // message of size N means the client will hold on to a buffer of size
    // N forever. We should investigate alternative transport protocols that
    // do not have this limitation.
    let mut codec = LengthDelimitedCodec::new();
    codec.set_max_frame_length(usize::MAX);
    codec
}