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
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
// 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-based implementations of Persist PubSub client and server.

use std::collections::btree_map::Entry;
use std::collections::BTreeMap;
use std::fmt::{Debug, Formatter};
use std::net::SocketAddr;
use std::pin::Pin;
use std::str::FromStr;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::{Arc, Mutex, RwLock, Weak};
use std::time::{Duration, Instant, SystemTime};

use anyhow::{anyhow, Error};
use async_trait::async_trait;
use bytes::Bytes;
use futures::Stream;
use mz_dyncfg::Config;
use mz_ore::cast::CastFrom;
use mz_ore::collections::{HashMap, HashSet};
use mz_ore::metrics::MetricsRegistry;
use mz_ore::retry::RetryResult;
use mz_ore::task::JoinHandle;
use mz_persist::location::VersionedData;
use mz_proto::{ProtoType, RustType};
use prost::Message;
use tokio::sync::mpsc::error::TrySendError;
use tokio::sync::mpsc::Sender;
use tokio_stream::wrappers::errors::BroadcastStreamRecvError;
use tokio_stream::wrappers::{BroadcastStream, ReceiverStream};
use tokio_stream::StreamExt;
use tonic::metadata::{AsciiMetadataKey, AsciiMetadataValue, MetadataMap};
use tonic::transport::Endpoint;
use tonic::{Extensions, Request, Response, Status, Streaming};
use tracing::{debug, error, info, info_span, warn, Instrument};

use crate::cache::{DynState, StateCache};
use crate::cfg::PersistConfig;
use crate::internal::metrics::{PubSubClientCallMetrics, PubSubServerMetrics};
use crate::internal::service::proto_persist_pub_sub_client::ProtoPersistPubSubClient;
use crate::internal::service::proto_persist_pub_sub_server::ProtoPersistPubSubServer;
use crate::internal::service::{
    proto_persist_pub_sub_server, proto_pub_sub_message, ProtoPubSubMessage, ProtoPushDiff,
    ProtoSubscribe, ProtoUnsubscribe,
};
use crate::metrics::Metrics;
use crate::ShardId;

/// Determines whether PubSub clients should connect to the PubSub server.
pub(crate) const PUBSUB_CLIENT_ENABLED: Config<bool> = Config::new(
    "persist_pubsub_client_enabled",
    true,
    "Whether to connect to the Persist PubSub service.",
);

/// For connected clients, determines whether to push state diffs to the PubSub
/// server. For the server, determines whether to broadcast state diffs to
/// subscribed clients.
pub(crate) const PUBSUB_PUSH_DIFF_ENABLED: Config<bool> = Config::new(
    "persist_pubsub_push_diff_enabled",
    true,
    "Whether to push state diffs to Persist PubSub.",
);

/// For connected clients, determines whether to push state diffs to the PubSub
/// server. For the server, determines whether to broadcast state diffs to
/// subscribed clients.
pub(crate) const PUBSUB_SAME_PROCESS_DELEGATE_ENABLED: Config<bool> = Config::new(
    "persist_pubsub_same_process_delegate_enabled",
    true,
    "Whether to push state diffs to Persist PubSub on the same process.",
);

/// Top-level Trait to create a PubSubClient.
///
/// Returns a [PubSubClientConnection] with a [PubSubSender] for issuing RPCs to the PubSub
/// server, and a [PubSubReceiver] that receives messages, such as state diffs.
pub trait PersistPubSubClient {
    /// Receive handles with which to push and subscribe to diffs.
    fn connect(
        pubsub_config: PersistPubSubClientConfig,
        metrics: Arc<Metrics>,
    ) -> PubSubClientConnection;
}

/// Wrapper type for a matching [PubSubSender] and [PubSubReceiver] client pair.
#[derive(Debug)]
pub struct PubSubClientConnection {
    /// The sender client to Persist PubSub.
    pub sender: Arc<dyn PubSubSender>,
    /// The receiver client to Persist PubSub.
    pub receiver: Box<dyn PubSubReceiver>,
}

impl PubSubClientConnection {
    /// Creates a new [PubSubClientConnection] from a matching [PubSubSender] and [PubSubReceiver].
    pub fn new(sender: Arc<dyn PubSubSender>, receiver: Box<dyn PubSubReceiver>) -> Self {
        Self { sender, receiver }
    }

    /// Creates a no-op [PubSubClientConnection] that neither sends nor receives messages.
    pub fn noop() -> Self {
        Self {
            sender: Arc::new(NoopPubSubSender),
            receiver: Box::new(futures::stream::empty()),
        }
    }
}

/// The public send-side client to Persist PubSub.
pub trait PubSubSender: std::fmt::Debug + Send + Sync {
    /// Push a diff to subscribers.
    fn push_diff(&self, shard_id: &ShardId, diff: &VersionedData);

    /// Subscribe the corresponding [PubSubReceiver] to diffs for the given shard.
    ///
    /// Returns a token that, when dropped, will unsubscribe the client from the
    /// shard.
    ///
    /// If the client is already subscribed to the shard, repeated calls will make
    /// no further calls to the server and instead return clones of the `Arc<ShardSubscriptionToken>`.
    fn subscribe(self: Arc<Self>, shard_id: &ShardId) -> Arc<ShardSubscriptionToken>;
}

/// The internal send-side client trait to Persist PubSub, responsible for issuing RPCs
/// to the PubSub service. This trait is separated out from [PubSubSender] to keep the
/// client implementations straightforward, while offering a more ergonomic public API
/// in [PubSubSender].
trait PubSubSenderInternal: Debug + Send + Sync {
    /// Push a diff to subscribers.
    fn push_diff(&self, shard_id: &ShardId, diff: &VersionedData);

    /// Subscribe the corresponding [PubSubReceiver] to diffs for the given shard.
    ///
    /// This call is idempotent and is a no-op for an already subscribed shard.
    fn subscribe(&self, shard_id: &ShardId);

    /// Unsubscribe the corresponding [PubSubReceiver] from diffs for the given shard.
    ///
    /// This call is idempotent and is a no-op for already unsubscribed shards.
    fn unsubscribe(&self, shard_id: &ShardId);
}

/// The receive-side client to Persist PubSub.
///
/// Returns diffs (and maybe in the future, blobs) for any shards subscribed to
/// by the corresponding `PubSubSender`.
pub trait PubSubReceiver:
    Stream<Item = ProtoPubSubMessage> + Send + Unpin + std::fmt::Debug
{
}

impl<T> PubSubReceiver for T where
    T: Stream<Item = ProtoPubSubMessage> + Send + Unpin + std::fmt::Debug
{
}

/// A token corresponding to a subscription to diffs for a particular shard.
///
/// When dropped, the client that originated the token will be unsubscribed
/// from further diffs to the shard.
pub struct ShardSubscriptionToken {
    pub(crate) shard_id: ShardId,
    sender: Arc<dyn PubSubSenderInternal>,
}

impl Debug for ShardSubscriptionToken {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        let ShardSubscriptionToken {
            shard_id,
            sender: _sender,
        } = self;
        write!(f, "ShardSubscriptionToken({})", shard_id)
    }
}

impl Drop for ShardSubscriptionToken {
    fn drop(&mut self) {
        self.sender.unsubscribe(&self.shard_id);
    }
}

/// A gRPC metadata key to indicate the caller id of a client.
pub const PERSIST_PUBSUB_CALLER_KEY: &str = "persist-pubsub-caller-id";

/// Client configuration for connecting to a remote PubSub server.
#[derive(Debug)]
pub struct PersistPubSubClientConfig {
    /// Connection address for the pubsub server, e.g. `http://localhost:6879`
    pub url: String,
    /// A caller ID for the client. Used for debugging.
    pub caller_id: String,
    /// A copy of [PersistConfig]
    pub persist_cfg: PersistConfig,
}

/// A [PersistPubSubClient] implementation backed by gRPC.
///
/// Returns a [PubSubClientConnection] backed by channels that submit and receive
/// messages to and from a long-lived bidirectional gRPC stream. The gRPC stream
/// will be transparently reestablished if the connection is lost.
#[derive(Debug)]
pub struct GrpcPubSubClient;

impl GrpcPubSubClient {
    async fn reconnect_to_server_forever(
        send_requests: tokio::sync::broadcast::Sender<ProtoPubSubMessage>,
        receiver_input: &tokio::sync::mpsc::Sender<ProtoPubSubMessage>,
        sender: Arc<SubscriptionTrackingSender>,
        metadata: MetadataMap,
        config: PersistPubSubClientConfig,
        metrics: Arc<Metrics>,
    ) {
        // Once enabled, the PubSub server cannot be disabled or otherwise
        // reconfigured. So we wait for at least one configuration sync to
        // complete. This gives `environmentd` at least one chance to update
        // PubSub configuration parameters. See database-issues#7168 for details.
        config.persist_cfg.configs_synced_once().await;

        let mut is_first_connection_attempt = true;
        loop {
            metrics.pubsub_client.grpc_connection.connected.set(0);

            if !PUBSUB_CLIENT_ENABLED.get(&config.persist_cfg) {
                tokio::time::sleep(Duration::from_secs(5)).await;
                continue;
            }

            // add a bit of backoff when reconnecting after some network/server failure
            if is_first_connection_attempt {
                is_first_connection_attempt = false;
            } else {
                tokio::time::sleep(config.persist_cfg.pubsub_reconnect_backoff).await;
            }

            info!("Connecting to Persist PubSub: {}", config.url);
            let client = mz_ore::retry::Retry::default()
                .clamp_backoff(config.persist_cfg.pubsub_connect_max_backoff)
                .retry_async(|_| async {
                    metrics
                        .pubsub_client
                        .grpc_connection
                        .connect_call_attempt_count
                        .inc();
                    let endpoint = match Endpoint::from_str(&config.url) {
                        Ok(endpoint) => endpoint,
                        Err(err) => return RetryResult::FatalErr(err),
                    };
                    ProtoPersistPubSubClient::connect(
                        endpoint
                            .connect_timeout(config.persist_cfg.pubsub_connect_attempt_timeout)
                            .timeout(config.persist_cfg.pubsub_request_timeout),
                    )
                    .await
                    .into()
                })
                .await;

            let mut client = match client {
                Ok(client) => client,
                Err(err) => {
                    error!("fatal error connecting to persist pubsub: {:?}", err);
                    return;
                }
            };

            metrics
                .pubsub_client
                .grpc_connection
                .connection_established_count
                .inc();
            metrics.pubsub_client.grpc_connection.connected.set(1);

            info!("Connected to Persist PubSub: {}", config.url);

            let mut broadcast = BroadcastStream::new(send_requests.subscribe());
            let broadcast_errors = metrics
                .pubsub_client
                .grpc_connection
                .broadcast_recv_lagged_count
                .clone();
            let pubsub_request = Request::from_parts(
                metadata.clone(),
                Extensions::default(),
                async_stream::stream! {
                    while let Some(message) = broadcast.next().await {
                        debug!("sending pubsub message: {:?}", message);
                        match message {
                            Ok(message) => yield message,
                            Err(BroadcastStreamRecvError::Lagged(i)) => {
                                broadcast_errors.inc_by(i);
                            }
                        }
                    }
                },
            );

            let responses = match client.pub_sub(pubsub_request).await {
                Ok(response) => response.into_inner(),
                Err(err) => {
                    warn!("pub_sub rpc error: {:?}", err);
                    continue;
                }
            };

            // shard subscriptions are tracked by connection on the server, so if our
            // gRPC stream is ever swapped out, we must inform the server which shards
            // our client intended to be subscribed to.
            sender.reconnect();

            let stream_completed = GrpcPubSubClient::consume_grpc_stream(
                responses,
                receiver_input,
                &config,
                metrics.as_ref(),
            )
            .await;

            match stream_completed {
                // common case: reconnect due to some transient error
                Ok(_) => continue,
                // uncommon case: we should stop connecting to the PubSub server entirely.
                // in practice, we should only see this during shut down.
                Err(err) => {
                    warn!("shutting down connection loop to Persist PubSub: {}", err);
                    return;
                }
            }
        }
    }

    async fn consume_grpc_stream(
        mut responses: Streaming<ProtoPubSubMessage>,
        receiver_input: &Sender<ProtoPubSubMessage>,
        config: &PersistPubSubClientConfig,
        metrics: &Metrics,
    ) -> Result<(), Error> {
        loop {
            if !PUBSUB_CLIENT_ENABLED.get(&config.persist_cfg) {
                return Ok(());
            }

            debug!("awaiting next pubsub response");
            match responses.next().await {
                Some(Ok(message)) => {
                    debug!("received pubsub message: {:?}", message);
                    match receiver_input.send(message).await {
                        Ok(_) => {}
                        // if the receiver has dropped, we can drop our
                        // no-longer-needed grpc connection entirely.
                        Err(err) => {
                            return Err(anyhow!("closing pubsub grpc client connection: {}", err));
                        }
                    }
                }
                Some(Err(err)) => {
                    metrics.pubsub_client.grpc_connection.grpc_error_count.inc();
                    warn!("pubsub client error: {:?}", err);
                    return Ok(());
                }
                None => return Ok(()),
            }
        }
    }
}

impl PersistPubSubClient for GrpcPubSubClient {
    fn connect(config: PersistPubSubClientConfig, metrics: Arc<Metrics>) -> PubSubClientConnection {
        // Create a stable channel for our client to transmit message into our gRPC stream. We use a
        // broadcast to allow us to create new Receivers on demand, in case the underlying gRPC stream
        // is swapped out (e.g. due to connection failure). It is expected that only 1 Receiver is
        // ever active at a given time.
        let (send_requests, _) =
            tokio::sync::broadcast::channel(config.persist_cfg.pubsub_client_sender_channel_size);
        // Create a stable channel to receive messages from our gRPC stream. The input end lives inside
        // a task that continuously reads from the active gRPC stream, decoupling the `PubSubReceiver`
        // from the lifetime of a specific gRPC connection.
        let (receiver_input, receiver_output) =
            tokio::sync::mpsc::channel(config.persist_cfg.pubsub_client_receiver_channel_size);

        let sender = Arc::new(SubscriptionTrackingSender::new(Arc::new(
            GrpcPubSubSender {
                metrics: Arc::clone(&metrics),
                requests: send_requests.clone(),
            },
        )));
        let pubsub_sender = Arc::clone(&sender);
        mz_ore::task::spawn(
            || "persist::rpc::client::connection".to_string(),
            async move {
                let mut metadata = MetadataMap::new();
                metadata.insert(
                    AsciiMetadataKey::from_static(PERSIST_PUBSUB_CALLER_KEY),
                    AsciiMetadataValue::try_from(&config.caller_id)
                        .unwrap_or_else(|_| AsciiMetadataValue::from_static("unknown")),
                );

                GrpcPubSubClient::reconnect_to_server_forever(
                    send_requests,
                    &receiver_input,
                    pubsub_sender,
                    metadata,
                    config,
                    metrics,
                )
                .await;
            },
        );

        PubSubClientConnection {
            sender,
            receiver: Box::new(ReceiverStream::new(receiver_output)),
        }
    }
}

/// An internal, gRPC-backed implementation of [PubSubSender].
struct GrpcPubSubSender {
    metrics: Arc<Metrics>,
    requests: tokio::sync::broadcast::Sender<ProtoPubSubMessage>,
}

impl Debug for GrpcPubSubSender {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        let GrpcPubSubSender {
            metrics: _metrics,
            requests: _requests,
        } = self;

        write!(f, "GrpcPubSubSender")
    }
}

impl GrpcPubSubSender {
    fn send(&self, message: proto_pub_sub_message::Message, metrics: &PubSubClientCallMetrics) {
        let now = SystemTime::now()
            .duration_since(SystemTime::UNIX_EPOCH)
            .expect("failed to get millis since epoch");

        let message = ProtoPubSubMessage {
            timestamp: Some(now.into_proto()),
            message: Some(message),
        };
        let size = message.encoded_len();

        match self.requests.send(message) {
            Ok(_) => {
                metrics.succeeded.inc();
                metrics.bytes_sent.inc_by(u64::cast_from(size));
            }
            Err(err) => {
                metrics.failed.inc();
                debug!("error sending client message: {}", err);
            }
        }
    }
}

impl PubSubSenderInternal for GrpcPubSubSender {
    fn push_diff(&self, shard_id: &ShardId, diff: &VersionedData) {
        self.send(
            proto_pub_sub_message::Message::PushDiff(ProtoPushDiff {
                shard_id: shard_id.into_proto(),
                seqno: diff.seqno.into_proto(),
                diff: diff.data.clone(),
            }),
            &self.metrics.pubsub_client.sender.push,
        )
    }

    fn subscribe(&self, shard_id: &ShardId) {
        self.send(
            proto_pub_sub_message::Message::Subscribe(ProtoSubscribe {
                shard_id: shard_id.into_proto(),
            }),
            &self.metrics.pubsub_client.sender.subscribe,
        )
    }

    fn unsubscribe(&self, shard_id: &ShardId) {
        self.send(
            proto_pub_sub_message::Message::Unsubscribe(ProtoUnsubscribe {
                shard_id: shard_id.into_proto(),
            }),
            &self.metrics.pubsub_client.sender.unsubscribe,
        )
    }
}

/// An wrapper for a [PubSubSenderInternal] that implements [PubSubSender]
/// by maintaining a map of active shard subscriptions to their tokens.
#[derive(Debug)]
struct SubscriptionTrackingSender {
    delegate: Arc<dyn PubSubSenderInternal>,
    subscribes: Arc<Mutex<BTreeMap<ShardId, Weak<ShardSubscriptionToken>>>>,
}

impl SubscriptionTrackingSender {
    fn new(sender: Arc<dyn PubSubSenderInternal>) -> Self {
        Self {
            delegate: sender,
            subscribes: Default::default(),
        }
    }

    fn reconnect(&self) {
        let mut subscribes = self.subscribes.lock().expect("lock");
        subscribes.retain(|shard_id, token| {
            if token.upgrade().is_none() {
                false
            } else {
                debug!("reconnecting to: {}", shard_id);
                self.delegate.subscribe(shard_id);
                true
            }
        })
    }
}

impl PubSubSender for SubscriptionTrackingSender {
    fn push_diff(&self, shard_id: &ShardId, diff: &VersionedData) {
        self.delegate.push_diff(shard_id, diff)
    }

    fn subscribe(self: Arc<Self>, shard_id: &ShardId) -> Arc<ShardSubscriptionToken> {
        let mut subscribes = self.subscribes.lock().expect("lock");
        if let Some(token) = subscribes.get(shard_id) {
            match token.upgrade() {
                None => assert!(subscribes.remove(shard_id).is_some()),
                Some(token) => {
                    return Arc::clone(&token);
                }
            }
        }

        let pubsub_sender = Arc::clone(&self.delegate);
        let token = Arc::new(ShardSubscriptionToken {
            shard_id: *shard_id,
            sender: pubsub_sender,
        });

        assert!(subscribes
            .insert(*shard_id, Arc::downgrade(&token))
            .is_none());

        self.delegate.subscribe(shard_id);

        token
    }
}

/// A wrapper intended to provide client-side metrics for a connection
/// that communicates directly with the server state, such as one created
/// by [PersistGrpcPubSubServer::new_same_process_connection].
#[derive(Debug)]
pub struct MetricsSameProcessPubSubSender {
    delegate_subscribe: bool,
    metrics: Arc<Metrics>,
    delegate: Arc<dyn PubSubSender>,
}

impl MetricsSameProcessPubSubSender {
    /// Returns a new [MetricsSameProcessPubSubSender], wrapping the given
    /// `Arc<dyn PubSubSender>`'s calls to provide client-side metrics.
    pub fn new(
        cfg: &PersistConfig,
        pubsub_sender: Arc<dyn PubSubSender>,
        metrics: Arc<Metrics>,
    ) -> Self {
        Self {
            delegate_subscribe: PUBSUB_SAME_PROCESS_DELEGATE_ENABLED.get(cfg),
            delegate: pubsub_sender,
            metrics,
        }
    }
}

impl PubSubSender for MetricsSameProcessPubSubSender {
    fn push_diff(&self, shard_id: &ShardId, diff: &VersionedData) {
        self.delegate.push_diff(shard_id, diff);
        self.metrics.pubsub_client.sender.push.succeeded.inc();
    }

    fn subscribe(self: Arc<Self>, shard_id: &ShardId) -> Arc<ShardSubscriptionToken> {
        if self.delegate_subscribe {
            let delegate = Arc::clone(&self.delegate);
            delegate.subscribe(shard_id)
        } else {
            // Create a no-op token that does not subscribe nor unsubscribe.
            // This is ideal for single-process persist setups, since the sender and
            // receiver should already share a state cache... but if the diffs are
            // generated remotely but applied on the server, this may cause us to fall
            // back to polling consensus.
            Arc::new(ShardSubscriptionToken {
                shard_id: *shard_id,
                sender: Arc::new(NoopPubSubSender),
            })
        }
    }
}

#[derive(Debug)]
pub(crate) struct NoopPubSubSender;

impl PubSubSenderInternal for NoopPubSubSender {
    fn push_diff(&self, _shard_id: &ShardId, _diff: &VersionedData) {}
    fn subscribe(&self, _shard_id: &ShardId) {}
    fn unsubscribe(&self, _shard_id: &ShardId) {}
}

impl PubSubSender for NoopPubSubSender {
    fn push_diff(&self, _shard_id: &ShardId, _diff: &VersionedData) {}

    fn subscribe(self: Arc<Self>, shard_id: &ShardId) -> Arc<ShardSubscriptionToken> {
        Arc::new(ShardSubscriptionToken {
            shard_id: *shard_id,
            sender: self,
        })
    }
}

/// Spawns a Tokio task that consumes a [PubSubReceiver], applying its diffs to a [StateCache].
pub(crate) fn subscribe_state_cache_to_pubsub(
    cache: Arc<StateCache>,
    mut pubsub_receiver: Box<dyn PubSubReceiver>,
) -> JoinHandle<()> {
    let mut state_refs: HashMap<ShardId, Weak<dyn DynState>> = HashMap::new();
    let receiver_metrics = cache.metrics.pubsub_client.receiver.clone();

    mz_ore::task::spawn(
        || "persist::rpc::client::state_cache_diff_apply",
        async move {
            while let Some(msg) = pubsub_receiver.next().await {
                match msg.message {
                    Some(proto_pub_sub_message::Message::PushDiff(diff)) => {
                        receiver_metrics.push_received.inc();
                        let shard_id = diff.shard_id.into_rust().expect("valid shard id");
                        let diff = VersionedData {
                            seqno: diff.seqno.into_rust().expect("valid SeqNo"),
                            data: diff.diff,
                        };
                        debug!(
                            "applying pubsub diff {} {} {}",
                            shard_id,
                            diff.seqno,
                            diff.data.len()
                        );

                        let mut pushed_diff = false;
                        if let Some(state_ref) = state_refs.get(&shard_id) {
                            // common case: we have a reference to the shard state already
                            // and can apply our diff directly.
                            if let Some(state) = state_ref.upgrade() {
                                state.push_diff(diff.clone());
                                pushed_diff = true;
                                receiver_metrics.state_pushed_diff_fast_path.inc();
                            }
                        }

                        if !pushed_diff {
                            // uncommon case: we either don't have a reference yet, or ours
                            // is out-of-date (e.g. the shard was dropped and then re-added
                            // to StateCache). here we'll fetch the latest, try to apply the
                            // diff again, and update our local reference.
                            let state_ref = cache.get_state_weak(&shard_id);
                            match state_ref {
                                None => {
                                    state_refs.remove(&shard_id);
                                }
                                Some(state_ref) => {
                                    if let Some(state) = state_ref.upgrade() {
                                        state.push_diff(diff);
                                        pushed_diff = true;
                                        state_refs.insert(shard_id, state_ref);
                                    } else {
                                        state_refs.remove(&shard_id);
                                    }
                                }
                            }

                            if pushed_diff {
                                receiver_metrics.state_pushed_diff_slow_path_succeeded.inc();
                            } else {
                                receiver_metrics.state_pushed_diff_slow_path_failed.inc();
                            }
                        }

                        if let Some(send_timestamp) = msg.timestamp {
                            let send_timestamp =
                                send_timestamp.into_rust().expect("valid timestamp");
                            let now = SystemTime::now()
                                .duration_since(SystemTime::UNIX_EPOCH)
                                .expect("failed to get millis since epoch");
                            receiver_metrics
                                .approx_diff_latency_seconds
                                .observe((now.saturating_sub(send_timestamp)).as_secs_f64());
                        }
                    }
                    ref msg @ None | ref msg @ Some(_) => {
                        warn!("pubsub client received unexpected message: {:?}", msg);
                        receiver_metrics.unknown_message_received.inc();
                    }
                }
            }
        },
    )
}

/// Internal state of a PubSub server implementation.
#[derive(Debug)]
pub(crate) struct PubSubState {
    /// Assigns a unique ID to each incoming connection.
    connection_id_counter: AtomicUsize,
    /// Maintains a mapping of `ShardId --> [ConnectionId -> Tx]`.
    shard_subscribers:
        Arc<RwLock<BTreeMap<ShardId, BTreeMap<usize, Sender<Result<ProtoPubSubMessage, Status>>>>>>,
    /// Active connections.
    connections: Arc<RwLock<HashSet<usize>>>,
    /// Server-side metrics.
    metrics: Arc<PubSubServerMetrics>,
}

impl PubSubState {
    fn new_connection(
        self: Arc<Self>,
        notifier: Sender<Result<ProtoPubSubMessage, Status>>,
    ) -> PubSubConnection {
        let connection_id = self.connection_id_counter.fetch_add(1, Ordering::SeqCst);
        {
            debug!("inserting connid: {}", connection_id);
            let mut connections = self.connections.write().expect("lock");
            assert!(connections.insert(connection_id));
        }

        self.metrics.active_connections.inc();
        PubSubConnection {
            connection_id,
            notifier,
            state: self,
        }
    }

    fn remove_connection(&self, connection_id: usize) {
        let now = Instant::now();

        {
            debug!("removing connid: {}", connection_id);
            let mut connections = self.connections.write().expect("lock");
            assert!(
                connections.remove(&connection_id),
                "unknown connection id: {}",
                connection_id
            );
        }

        {
            let mut subscribers = self.shard_subscribers.write().expect("lock poisoned");
            subscribers.retain(|_shard, connections_for_shard| {
                connections_for_shard.remove(&connection_id);
                !connections_for_shard.is_empty()
            });
        }

        self.metrics
            .connection_cleanup_seconds
            .inc_by(now.elapsed().as_secs_f64());
        self.metrics.active_connections.dec();
    }

    fn push_diff(&self, connection_id: usize, shard_id: &ShardId, data: &VersionedData) {
        let now = Instant::now();
        self.metrics.push_call_count.inc();

        assert!(
            self.connections
                .read()
                .expect("lock")
                .contains(&connection_id),
            "unknown connection id: {}",
            connection_id
        );

        let subscribers = self.shard_subscribers.read().expect("lock poisoned");
        if let Some(subscribed_connections) = subscribers.get(shard_id) {
            let mut num_sent = 0;
            let mut data_size = 0;

            for (subscribed_conn_id, tx) in subscribed_connections {
                // skip sending the diff back to the original sender
                if *subscribed_conn_id == connection_id {
                    continue;
                }
                debug!(
                    "server forwarding req to {} conns {} {} {}",
                    subscribed_conn_id,
                    &shard_id,
                    data.seqno,
                    data.data.len()
                );
                let req = ProtoPubSubMessage {
                    timestamp: Some(
                        SystemTime::now()
                            .duration_since(SystemTime::UNIX_EPOCH)
                            .expect("failed to get millis since epoch")
                            .into_proto(),
                    ),
                    message: Some(proto_pub_sub_message::Message::PushDiff(ProtoPushDiff {
                        seqno: data.seqno.into_proto(),
                        shard_id: shard_id.to_string(),
                        diff: Bytes::clone(&data.data),
                    })),
                };
                data_size = req.encoded_len();
                match tx.try_send(Ok(req)) {
                    Ok(_) => {
                        num_sent += 1;
                    }
                    Err(TrySendError::Full(_)) => {
                        self.metrics.broadcasted_diff_dropped_channel_full.inc();
                    }
                    Err(TrySendError::Closed(_)) => {}
                };
            }

            self.metrics.broadcasted_diff_count.inc_by(num_sent);
            self.metrics
                .broadcasted_diff_bytes
                .inc_by(num_sent * u64::cast_from(data_size));
        }

        self.metrics
            .push_seconds
            .inc_by(now.elapsed().as_secs_f64());
    }

    fn subscribe(
        &self,
        connection_id: usize,
        notifier: Sender<Result<ProtoPubSubMessage, Status>>,
        shard_id: &ShardId,
    ) {
        let now = Instant::now();
        self.metrics.subscribe_call_count.inc();

        assert!(
            self.connections
                .read()
                .expect("lock")
                .contains(&connection_id),
            "unknown connection id: {}",
            connection_id
        );

        {
            let mut subscribed_shards = self.shard_subscribers.write().expect("lock poisoned");
            subscribed_shards
                .entry(*shard_id)
                .or_default()
                .insert(connection_id, notifier);
        }

        self.metrics
            .subscribe_seconds
            .inc_by(now.elapsed().as_secs_f64());
    }

    fn unsubscribe(&self, connection_id: usize, shard_id: &ShardId) {
        let now = Instant::now();
        self.metrics.unsubscribe_call_count.inc();

        assert!(
            self.connections
                .read()
                .expect("lock")
                .contains(&connection_id),
            "unknown connection id: {}",
            connection_id
        );

        {
            let mut subscribed_shards = self.shard_subscribers.write().expect("lock poisoned");
            if let Entry::Occupied(mut entry) = subscribed_shards.entry(*shard_id) {
                let subscribed_connections = entry.get_mut();
                subscribed_connections.remove(&connection_id);

                if subscribed_connections.is_empty() {
                    entry.remove_entry();
                }
            }
        }

        self.metrics
            .unsubscribe_seconds
            .inc_by(now.elapsed().as_secs_f64());
    }

    #[cfg(test)]
    fn new_for_test() -> Self {
        Self {
            connection_id_counter: AtomicUsize::new(0),
            shard_subscribers: Default::default(),
            connections: Default::default(),
            metrics: Arc::new(PubSubServerMetrics::new(&MetricsRegistry::new())),
        }
    }

    #[cfg(test)]
    fn active_connections(&self) -> HashSet<usize> {
        self.connections.read().expect("lock").clone()
    }

    #[cfg(test)]
    fn subscriptions(&self, connection_id: usize) -> HashSet<ShardId> {
        let mut shards = HashSet::new();

        let subscribers = self.shard_subscribers.read().expect("lock");
        for (shard, subscribed_connections) in subscribers.iter() {
            if subscribed_connections.contains_key(&connection_id) {
                shards.insert(*shard);
            }
        }

        shards
    }

    #[cfg(test)]
    fn shard_subscription_counts(&self) -> mz_ore::collections::HashMap<ShardId, usize> {
        let mut shards = mz_ore::collections::HashMap::new();

        let subscribers = self.shard_subscribers.read().expect("lock");
        for (shard, subscribed_connections) in subscribers.iter() {
            shards.insert(*shard, subscribed_connections.len());
        }

        shards
    }
}

/// A gRPC-based implementation of a Persist PubSub server.
#[derive(Debug)]
pub struct PersistGrpcPubSubServer {
    cfg: PersistConfig,
    state: Arc<PubSubState>,
}

impl PersistGrpcPubSubServer {
    /// Creates a new [PersistGrpcPubSubServer].
    pub fn new(cfg: &PersistConfig, metrics_registry: &MetricsRegistry) -> Self {
        let metrics = PubSubServerMetrics::new(metrics_registry);
        let state = Arc::new(PubSubState {
            connection_id_counter: AtomicUsize::new(0),
            shard_subscribers: Default::default(),
            connections: Default::default(),
            metrics: Arc::new(metrics),
        });

        PersistGrpcPubSubServer {
            cfg: cfg.clone(),
            state,
        }
    }

    /// Creates a connection to [PersistGrpcPubSubServer] that is directly connected
    /// to the server state. Calls into this connection do not go over the network
    /// nor require message serde.
    pub fn new_same_process_connection(&self) -> PubSubClientConnection {
        let (tx, rx) = tokio::sync::mpsc::channel(self.cfg.pubsub_client_receiver_channel_size);
        let sender: Arc<dyn PubSubSender> = Arc::new(SubscriptionTrackingSender::new(Arc::new(
            Arc::clone(&self.state).new_connection(tx),
        )));

        PubSubClientConnection {
            sender,
            receiver: Box::new(
                ReceiverStream::new(rx).map(|x| x.expect("cannot receive grpc errors locally")),
            ),
        }
    }

    /// Starts the gRPC server. Consumes `self` and runs until the task is cancelled.
    pub async fn serve(self, listen_addr: SocketAddr) -> Result<(), anyhow::Error> {
        // Increase the default message decoding limit to avoid unnecessary panics
        tonic::transport::Server::builder()
            .add_service(ProtoPersistPubSubServer::new(self).max_decoding_message_size(usize::MAX))
            .serve(listen_addr)
            .await?;
        Ok(())
    }

    /// Starts the gRPC server with the given listener stream.
    /// Consumes `self` and runs until the task is cancelled.
    pub async fn serve_with_stream(
        self,
        listener: tokio_stream::wrappers::TcpListenerStream,
    ) -> Result<(), anyhow::Error> {
        tonic::transport::Server::builder()
            .add_service(ProtoPersistPubSubServer::new(self))
            .serve_with_incoming(listener)
            .await?;
        Ok(())
    }
}

#[async_trait]
impl proto_persist_pub_sub_server::ProtoPersistPubSub for PersistGrpcPubSubServer {
    type PubSubStream = Pin<Box<dyn Stream<Item = Result<ProtoPubSubMessage, Status>> + Send>>;

    #[mz_ore::instrument(name = "persist::rpc::server", level = "info")]
    async fn pub_sub(
        &self,
        request: Request<Streaming<ProtoPubSubMessage>>,
    ) -> Result<Response<Self::PubSubStream>, Status> {
        let caller_id = request
            .metadata()
            .get(AsciiMetadataKey::from_static(PERSIST_PUBSUB_CALLER_KEY))
            .map(|key| key.to_str().ok())
            .flatten()
            .map(|key| key.to_string())
            .unwrap_or_else(|| "unknown".to_string());
        info!("Received Persist PubSub connection from: {:?}", caller_id);

        let mut in_stream = request.into_inner();
        let (tx, rx) = tokio::sync::mpsc::channel(self.cfg.pubsub_server_connection_channel_size);

        let caller = caller_id.clone();
        let cfg = Arc::clone(&self.cfg.configs);
        let server_state = Arc::clone(&self.state);
        // this spawn here to cleanup after connection error / disconnect, otherwise the stream
        // would not be polled after the connection drops. in our case, we want to clear the
        // connection and its subscriptions from our shared state when it drops.
        let connection_span = info_span!("connection", caller_id);
        mz_ore::task::spawn(
            || format!("persist_pubsub_connection({})", caller),
            async move {
                let connection = server_state.new_connection(tx);
                while let Some(result) = in_stream.next().await {
                    let req = match result {
                        Ok(req) => req,
                        Err(err) => {
                            warn!("pubsub connection err: {}", err);
                            break;
                        }
                    };

                    match req.message {
                        None => {
                            warn!("received empty message from: {}", caller_id);
                        }
                        Some(proto_pub_sub_message::Message::PushDiff(req)) => {
                            let shard_id = req.shard_id.parse().expect("valid shard id");
                            let diff = VersionedData {
                                seqno: req.seqno.into_rust().expect("valid seqno"),
                                data: req.diff.clone(),
                            };
                            if PUBSUB_PUSH_DIFF_ENABLED.get(&cfg) {
                                connection.push_diff(&shard_id, &diff);
                            }
                        }
                        Some(proto_pub_sub_message::Message::Subscribe(diff)) => {
                            let shard_id = diff.shard_id.parse().expect("valid shard id");
                            connection.subscribe(&shard_id);
                        }
                        Some(proto_pub_sub_message::Message::Unsubscribe(diff)) => {
                            let shard_id = diff.shard_id.parse().expect("valid shard id");
                            connection.unsubscribe(&shard_id);
                        }
                    }
                }

                info!("Persist PubSub connection ended: {:?}", caller_id);
            }
            .instrument(connection_span),
        );

        let out_stream: Self::PubSubStream = Box::pin(ReceiverStream::new(rx));
        Ok(Response::new(out_stream))
    }
}

/// An active connection managed by [PubSubState].
///
/// When dropped, removes itself from [PubSubState], clearing all of its subscriptions.
#[derive(Debug)]
pub(crate) struct PubSubConnection {
    connection_id: usize,
    notifier: Sender<Result<ProtoPubSubMessage, Status>>,
    state: Arc<PubSubState>,
}

impl PubSubSenderInternal for PubSubConnection {
    fn push_diff(&self, shard_id: &ShardId, diff: &VersionedData) {
        self.state.push_diff(self.connection_id, shard_id, diff)
    }

    fn subscribe(&self, shard_id: &ShardId) {
        self.state
            .subscribe(self.connection_id, self.notifier.clone(), shard_id)
    }

    fn unsubscribe(&self, shard_id: &ShardId) {
        self.state.unsubscribe(self.connection_id, shard_id)
    }
}

impl Drop for PubSubConnection {
    fn drop(&mut self) {
        self.state.remove_connection(self.connection_id)
    }
}

#[cfg(test)]
mod pubsub_state {
    use std::str::FromStr;
    use std::sync::Arc;
    use std::sync::LazyLock;

    use bytes::Bytes;
    use mz_ore::collections::HashSet;
    use mz_persist::location::{SeqNo, VersionedData};
    use mz_proto::RustType;
    use tokio::sync::mpsc::error::TryRecvError;
    use tokio::sync::mpsc::Receiver;
    use tonic::Status;

    use crate::internal::service::proto_pub_sub_message::Message;
    use crate::internal::service::ProtoPubSubMessage;
    use crate::rpc::{PubSubSenderInternal, PubSubState};
    use crate::ShardId;

    const SHARD_ID_0: LazyLock<ShardId> =
        LazyLock::new(|| ShardId::from_str("s00000000-0000-0000-0000-000000000000").unwrap());
    const SHARD_ID_1: LazyLock<ShardId> =
        LazyLock::new(|| ShardId::from_str("s11111111-1111-1111-1111-111111111111").unwrap());

    const VERSIONED_DATA_0: VersionedData = VersionedData {
        seqno: SeqNo(0),
        data: Bytes::from_static(&[0, 1, 2, 3]),
    };

    const VERSIONED_DATA_1: VersionedData = VersionedData {
        seqno: SeqNo(1),
        data: Bytes::from_static(&[4, 5, 6, 7]),
    };

    #[mz_ore::test]
    #[should_panic(expected = "unknown connection id: 100")]
    fn test_zero_connections_push_diff() {
        let state = Arc::new(PubSubState::new_for_test());
        state.push_diff(100, &SHARD_ID_0, &VERSIONED_DATA_0);
    }

    #[mz_ore::test]
    #[should_panic(expected = "unknown connection id: 100")]
    fn test_zero_connections_subscribe() {
        let state = Arc::new(PubSubState::new_for_test());
        let (tx, _) = tokio::sync::mpsc::channel(100);
        state.subscribe(100, tx, &SHARD_ID_0);
    }

    #[mz_ore::test]
    #[should_panic(expected = "unknown connection id: 100")]
    fn test_zero_connections_unsubscribe() {
        let state = Arc::new(PubSubState::new_for_test());
        state.unsubscribe(100, &SHARD_ID_0);
    }

    #[mz_ore::test]
    #[should_panic(expected = "unknown connection id: 100")]
    fn test_zero_connections_remove() {
        let state = Arc::new(PubSubState::new_for_test());
        state.remove_connection(100)
    }

    #[mz_ore::test]
    fn test_single_connection() {
        let state = Arc::new(PubSubState::new_for_test());

        let (tx, mut rx) = tokio::sync::mpsc::channel(100);
        let connection = Arc::clone(&state).new_connection(tx);

        assert_eq!(
            state.active_connections(),
            HashSet::from([connection.connection_id])
        );

        // no messages should have been broadcasted yet
        assert!(matches!(rx.try_recv(), Err(TryRecvError::Empty)));

        connection.push_diff(
            &SHARD_ID_0,
            &VersionedData {
                seqno: SeqNo::minimum(),
                data: Bytes::new(),
            },
        );

        // server should not broadcast a message back to originating client
        assert!(matches!(rx.try_recv(), Err(TryRecvError::Empty)));

        // a connection can subscribe to a shard
        connection.subscribe(&SHARD_ID_0);
        assert_eq!(
            state.subscriptions(connection.connection_id),
            HashSet::from([SHARD_ID_0.clone()])
        );

        // a connection can unsubscribe
        connection.unsubscribe(&SHARD_ID_0);
        assert!(state.subscriptions(connection.connection_id).is_empty());

        // a connection can subscribe to many shards
        connection.subscribe(&SHARD_ID_0);
        connection.subscribe(&SHARD_ID_1);
        assert_eq!(
            state.subscriptions(connection.connection_id),
            HashSet::from([*SHARD_ID_0, *SHARD_ID_1])
        );

        // and to a single shard many times idempotently
        connection.subscribe(&SHARD_ID_0);
        connection.subscribe(&SHARD_ID_0);
        assert_eq!(
            state.subscriptions(connection.connection_id),
            HashSet::from([*SHARD_ID_0, *SHARD_ID_1])
        );

        // dropping the connection should unsubscribe all shards and unregister the connection
        let connection_id = connection.connection_id;
        drop(connection);
        assert!(state.subscriptions(connection_id).is_empty());
        assert!(state.active_connections().is_empty());
    }

    #[mz_ore::test]
    fn test_many_connection() {
        let state = Arc::new(PubSubState::new_for_test());

        let (tx1, mut rx1) = tokio::sync::mpsc::channel(100);
        let conn1 = Arc::clone(&state).new_connection(tx1);

        let (tx2, mut rx2) = tokio::sync::mpsc::channel(100);
        let conn2 = Arc::clone(&state).new_connection(tx2);

        let (tx3, mut rx3) = tokio::sync::mpsc::channel(100);
        let conn3 = Arc::clone(&state).new_connection(tx3);

        conn1.subscribe(&SHARD_ID_0);
        conn2.subscribe(&SHARD_ID_0);
        conn2.subscribe(&SHARD_ID_1);

        assert_eq!(
            state.active_connections(),
            HashSet::from([
                conn1.connection_id,
                conn2.connection_id,
                conn3.connection_id
            ])
        );

        // broadcast a diff to a shard subscribed to by several connections
        conn3.push_diff(&SHARD_ID_0, &VERSIONED_DATA_0);
        assert_push(&mut rx1, &SHARD_ID_0, &VERSIONED_DATA_0);
        assert_push(&mut rx2, &SHARD_ID_0, &VERSIONED_DATA_0);
        assert!(matches!(rx3.try_recv(), Err(TryRecvError::Empty)));

        // broadcast a diff shared by publisher. it should not receive the diff back.
        conn1.push_diff(&SHARD_ID_0, &VERSIONED_DATA_0);
        assert!(matches!(rx1.try_recv(), Err(TryRecvError::Empty)));
        assert_push(&mut rx2, &SHARD_ID_0, &VERSIONED_DATA_0);
        assert!(matches!(rx3.try_recv(), Err(TryRecvError::Empty)));

        // broadcast a diff to a shard subscribed to by one connection
        conn3.push_diff(&SHARD_ID_1, &VERSIONED_DATA_1);
        assert!(matches!(rx1.try_recv(), Err(TryRecvError::Empty)));
        assert_push(&mut rx2, &SHARD_ID_1, &VERSIONED_DATA_1);
        assert!(matches!(rx3.try_recv(), Err(TryRecvError::Empty)));

        // broadcast a diff to a shard subscribed to by no connections
        conn2.unsubscribe(&SHARD_ID_1);
        conn3.push_diff(&SHARD_ID_1, &VERSIONED_DATA_1);
        assert!(matches!(rx1.try_recv(), Err(TryRecvError::Empty)));
        assert!(matches!(rx2.try_recv(), Err(TryRecvError::Empty)));
        assert!(matches!(rx3.try_recv(), Err(TryRecvError::Empty)));

        // dropping connections unsubscribes them
        let conn1_id = conn1.connection_id;
        drop(conn1);
        conn3.push_diff(&SHARD_ID_0, &VERSIONED_DATA_0);
        assert!(matches!(rx1.try_recv(), Err(TryRecvError::Disconnected)));
        assert_push(&mut rx2, &SHARD_ID_0, &VERSIONED_DATA_0);
        assert!(matches!(rx3.try_recv(), Err(TryRecvError::Empty)));

        assert!(state.subscriptions(conn1_id).is_empty());
        assert_eq!(
            state.subscriptions(conn2.connection_id),
            HashSet::from([*SHARD_ID_0])
        );
        assert_eq!(state.subscriptions(conn3.connection_id), HashSet::new());
        assert_eq!(
            state.active_connections(),
            HashSet::from([conn2.connection_id, conn3.connection_id])
        );
    }

    fn assert_push(
        rx: &mut Receiver<Result<ProtoPubSubMessage, Status>>,
        shard: &ShardId,
        data: &VersionedData,
    ) {
        let message = rx
            .try_recv()
            .expect("message in channel")
            .expect("pubsub")
            .message
            .expect("proto contains message");
        match message {
            Message::PushDiff(x) => {
                assert_eq!(x.shard_id, shard.into_proto());
                assert_eq!(x.seqno, data.seqno.into_proto());
                assert_eq!(x.diff, data.data);
            }
            Message::Subscribe(_) | Message::Unsubscribe(_) => panic!("unexpected message type"),
        };
    }
}

#[cfg(test)]
mod grpc {
    use std::net::{Ipv4Addr, SocketAddr, SocketAddrV4};
    use std::str::FromStr;
    use std::sync::Arc;
    use std::time::{Duration, Instant};

    use bytes::Bytes;
    use futures_util::FutureExt;
    use mz_dyncfg::ConfigUpdates;
    use mz_ore::assert_none;
    use mz_ore::collections::HashMap;
    use mz_ore::metrics::MetricsRegistry;
    use mz_persist::location::{SeqNo, VersionedData};
    use mz_proto::RustType;
    use std::sync::LazyLock;
    use tokio::net::TcpListener;
    use tokio_stream::wrappers::TcpListenerStream;
    use tokio_stream::StreamExt;

    use crate::cfg::PersistConfig;
    use crate::internal::service::proto_pub_sub_message::Message;
    use crate::internal::service::ProtoPubSubMessage;
    use crate::metrics::Metrics;
    use crate::rpc::{
        GrpcPubSubClient, PersistGrpcPubSubServer, PersistPubSubClient, PersistPubSubClientConfig,
        PubSubState, PUBSUB_CLIENT_ENABLED,
    };
    use crate::ShardId;

    static SHARD_ID_0: LazyLock<ShardId> =
        LazyLock::new(|| ShardId::from_str("s00000000-0000-0000-0000-000000000000").unwrap());
    static SHARD_ID_1: LazyLock<ShardId> =
        LazyLock::new(|| ShardId::from_str("s11111111-1111-1111-1111-111111111111").unwrap());
    const VERSIONED_DATA_0: VersionedData = VersionedData {
        seqno: SeqNo(0),
        data: Bytes::from_static(&[0, 1, 2, 3]),
    };
    const VERSIONED_DATA_1: VersionedData = VersionedData {
        seqno: SeqNo(1),
        data: Bytes::from_static(&[4, 5, 6, 7]),
    };

    const CONNECT_TIMEOUT: Duration = Duration::from_secs(10);
    const SUBSCRIPTIONS_TIMEOUT: Duration = Duration::from_secs(3);
    const SERVER_SHUTDOWN_TIMEOUT: Duration = Duration::from_secs(2);

    // NB: we use separate runtimes for client and server throughout these tests to cleanly drop
    // ALL tasks (including spawned child tasks) associated with one end of a connection, to most
    // closely model an actual disconnect.

    #[mz_ore::test]
    #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `socket` on OS `linux`
    fn grpc_server() {
        let metrics = Arc::new(Metrics::new(
            &test_persist_config(),
            &MetricsRegistry::new(),
        ));
        let server_runtime = tokio::runtime::Runtime::new().expect("server runtime");
        let client_runtime = tokio::runtime::Runtime::new().expect("client runtime");

        // start the server
        let (addr, tcp_listener_stream) = server_runtime.block_on(new_tcp_listener());
        let server_state = server_runtime.block_on(spawn_server(tcp_listener_stream));

        // start a client.
        {
            let _guard = client_runtime.enter();
            mz_ore::task::spawn(|| "client".to_string(), async move {
                let client = GrpcPubSubClient::connect(
                    PersistPubSubClientConfig {
                        url: format!("http://{}", addr),
                        caller_id: "client".to_string(),
                        persist_cfg: test_persist_config(),
                    },
                    metrics,
                );
                let _token = client.sender.subscribe(&SHARD_ID_0);
                tokio::time::sleep(Duration::MAX).await;
            });
        }

        // wait until the client is connected and subscribed
        server_runtime.block_on(async {
            poll_until_true(CONNECT_TIMEOUT, || {
                server_state.active_connections().len() == 1
            })
            .await;
            poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
                server_state.shard_subscription_counts() == HashMap::from([(*SHARD_ID_0, 1)])
            })
            .await
        });

        // drop the client
        client_runtime.shutdown_timeout(SERVER_SHUTDOWN_TIMEOUT);

        // server should notice the client dropping and clean up its state
        server_runtime.block_on(async {
            poll_until_true(CONNECT_TIMEOUT, || {
                server_state.active_connections().is_empty()
            })
            .await;
            poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
                server_state.shard_subscription_counts() == HashMap::new()
            })
            .await
        });
    }

    #[mz_ore::test]
    #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `socket` on OS `linux`
    fn grpc_client_sender_reconnects() {
        let metrics = Arc::new(Metrics::new(
            &test_persist_config(),
            &MetricsRegistry::new(),
        ));
        let server_runtime = tokio::runtime::Runtime::new().expect("server runtime");
        let client_runtime = tokio::runtime::Runtime::new().expect("client runtime");
        let (addr, tcp_listener_stream) = server_runtime.block_on(new_tcp_listener());

        // start a client
        let client = client_runtime.block_on(async {
            GrpcPubSubClient::connect(
                PersistPubSubClientConfig {
                    url: format!("http://{}", addr),
                    caller_id: "client".to_string(),
                    persist_cfg: test_persist_config(),
                },
                metrics,
            )
        });

        // we can subscribe before connecting to the pubsub server
        let _token = Arc::clone(&client.sender).subscribe(&SHARD_ID_0);
        // we can subscribe and unsubscribe before connecting to the pubsub server
        let _token_2 = Arc::clone(&client.sender).subscribe(&SHARD_ID_1);
        drop(_token_2);

        // create the server after the client is up
        let server_state = server_runtime.block_on(spawn_server(tcp_listener_stream));

        server_runtime.block_on(async {
            // client connects automatically once the server is up
            poll_until_true(CONNECT_TIMEOUT, || {
                server_state.active_connections().len() == 1
            })
            .await;

            // client rehydrated its subscriptions. notably, only includes the shard that
            // still has an active token
            poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
                server_state.shard_subscription_counts() == HashMap::from([(*SHARD_ID_0, 1)])
            })
            .await;
        });

        // kill the server
        server_runtime.shutdown_timeout(SERVER_SHUTDOWN_TIMEOUT);

        // client can still send requests without error
        let _token_2 = Arc::clone(&client.sender).subscribe(&SHARD_ID_1);

        // create a new server
        let server_runtime = tokio::runtime::Runtime::new().expect("server runtime");
        let tcp_listener_stream = server_runtime.block_on(async {
            TcpListenerStream::new(
                TcpListener::bind(addr)
                    .await
                    .expect("can bind to previous addr"),
            )
        });
        let server_state = server_runtime.block_on(spawn_server(tcp_listener_stream));

        server_runtime.block_on(async {
            // client automatically reconnects to new server
            poll_until_true(CONNECT_TIMEOUT, || {
                server_state.active_connections().len() == 1
            })
            .await;

            // and rehydrates its subscriptions, including the new one that was sent
            // while the server was unavailable.
            poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
                server_state.shard_subscription_counts()
                    == HashMap::from([(*SHARD_ID_0, 1), (*SHARD_ID_1, 1)])
            })
            .await;
        });
    }

    #[mz_ore::test(tokio::test(flavor = "multi_thread"))]
    #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `socket` on OS `linux`
    async fn grpc_client_sender_subscription_tokens() {
        let metrics = Arc::new(Metrics::new(
            &test_persist_config(),
            &MetricsRegistry::new(),
        ));

        let (addr, tcp_listener_stream) = new_tcp_listener().await;
        let server_state = spawn_server(tcp_listener_stream).await;

        let client = GrpcPubSubClient::connect(
            PersistPubSubClientConfig {
                url: format!("http://{}", addr),
                caller_id: "client".to_string(),
                persist_cfg: test_persist_config(),
            },
            metrics,
        );

        // our client connects
        poll_until_true(CONNECT_TIMEOUT, || {
            server_state.active_connections().len() == 1
        })
        .await;

        // we can subscribe to a shard, receiving back a token
        let token = Arc::clone(&client.sender).subscribe(&SHARD_ID_0);
        poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
            server_state.shard_subscription_counts() == HashMap::from([(*SHARD_ID_0, 1)])
        })
        .await;

        // dropping the token will unsubscribe our client
        drop(token);
        poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
            server_state.shard_subscription_counts() == HashMap::new()
        })
        .await;

        // we can resubscribe to a shard
        let token = Arc::clone(&client.sender).subscribe(&SHARD_ID_0);
        poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
            server_state.shard_subscription_counts() == HashMap::from([(*SHARD_ID_0, 1)])
        })
        .await;

        // we can subscribe many times idempotently, receiving back Arcs to the same token
        let token2 = Arc::clone(&client.sender).subscribe(&SHARD_ID_0);
        let token3 = Arc::clone(&client.sender).subscribe(&SHARD_ID_0);
        assert_eq!(Arc::strong_count(&token), 3);
        poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
            server_state.shard_subscription_counts() == HashMap::from([(*SHARD_ID_0, 1)])
        })
        .await;

        // dropping all of the tokens will unsubscribe the shard
        drop(token);
        drop(token2);
        drop(token3);
        poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
            server_state.shard_subscription_counts() == HashMap::new()
        })
        .await;

        // we can subscribe to many shards
        let _token0 = Arc::clone(&client.sender).subscribe(&SHARD_ID_0);
        let _token1 = Arc::clone(&client.sender).subscribe(&SHARD_ID_1);
        poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
            server_state.shard_subscription_counts()
                == HashMap::from([(*SHARD_ID_0, 1), (*SHARD_ID_1, 1)])
        })
        .await;
    }

    #[mz_ore::test]
    #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `socket` on OS `linux`
    fn grpc_client_receiver() {
        let metrics = Arc::new(Metrics::new(
            &PersistConfig::new_for_tests(),
            &MetricsRegistry::new(),
        ));
        let server_runtime = tokio::runtime::Runtime::new().expect("server runtime");
        let client_runtime = tokio::runtime::Runtime::new().expect("client runtime");
        let (addr, tcp_listener_stream) = server_runtime.block_on(new_tcp_listener());

        // create two clients, so we can test that broadcast messages are received by the other
        let mut client_1 = client_runtime.block_on(async {
            GrpcPubSubClient::connect(
                PersistPubSubClientConfig {
                    url: format!("http://{}", addr),
                    caller_id: "client_1".to_string(),
                    persist_cfg: test_persist_config(),
                },
                Arc::clone(&metrics),
            )
        });
        let mut client_2 = client_runtime.block_on(async {
            GrpcPubSubClient::connect(
                PersistPubSubClientConfig {
                    url: format!("http://{}", addr),
                    caller_id: "client_2".to_string(),
                    persist_cfg: test_persist_config(),
                },
                metrics,
            )
        });

        // we can check our receiver output before connecting to the server.
        // these calls are race-y, since there's no guarantee on the time it
        // would take for a message to be received were one to have been sent,
        // but, better than nothing?
        assert_none!(client_1.receiver.next().now_or_never());
        assert_none!(client_2.receiver.next().now_or_never());

        // start the server
        let server_state = server_runtime.block_on(spawn_server(tcp_listener_stream));

        // wait until both clients are connected
        server_runtime.block_on(poll_until_true(CONNECT_TIMEOUT, || {
            server_state.active_connections().len() == 2
        }));

        // no messages have been broadcast yet
        assert_none!(client_1.receiver.next().now_or_never());
        assert_none!(client_2.receiver.next().now_or_never());

        // subscribe and send a diff
        let _token_client_1 = Arc::clone(&client_1.sender).subscribe(&SHARD_ID_0);
        let _token_client_2 = Arc::clone(&client_2.sender).subscribe(&SHARD_ID_0);
        server_runtime.block_on(poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
            server_state.shard_subscription_counts() == HashMap::from([(*SHARD_ID_0, 2)])
        }));

        // the subscriber non-sender client receives the diff
        client_1.sender.push_diff(&SHARD_ID_0, &VERSIONED_DATA_1);
        assert_none!(client_1.receiver.next().now_or_never());
        client_runtime.block_on(async {
            assert_push(
                client_2.receiver.next().await.expect("has diff"),
                &SHARD_ID_0,
                &VERSIONED_DATA_1,
            )
        });

        // kill the server
        server_runtime.shutdown_timeout(SERVER_SHUTDOWN_TIMEOUT);

        // receivers can still be polled without error
        assert_none!(client_1.receiver.next().now_or_never());
        assert_none!(client_2.receiver.next().now_or_never());

        // create a new server
        let server_runtime = tokio::runtime::Runtime::new().expect("server runtime");
        let tcp_listener_stream = server_runtime.block_on(async {
            TcpListenerStream::new(
                TcpListener::bind(addr)
                    .await
                    .expect("can bind to previous addr"),
            )
        });
        let server_state = server_runtime.block_on(spawn_server(tcp_listener_stream));

        // client automatically reconnects to new server and rehydrates subscriptions
        server_runtime.block_on(async {
            poll_until_true(CONNECT_TIMEOUT, || {
                server_state.active_connections().len() == 2
            })
            .await;
            poll_until_true(SUBSCRIPTIONS_TIMEOUT, || {
                server_state.shard_subscription_counts() == HashMap::from([(*SHARD_ID_0, 2)])
            })
            .await;
        });

        // pushing and receiving diffs works as expected.
        // this time we'll push from the other client.
        client_2.sender.push_diff(&SHARD_ID_0, &VERSIONED_DATA_0);
        client_runtime.block_on(async {
            assert_push(
                client_1.receiver.next().await.expect("has diff"),
                &SHARD_ID_0,
                &VERSIONED_DATA_0,
            )
        });
        assert_none!(client_2.receiver.next().now_or_never());
    }

    async fn new_tcp_listener() -> (SocketAddr, TcpListenerStream) {
        let addr = SocketAddr::V4(SocketAddrV4::new(Ipv4Addr::LOCALHOST, 0));
        let tcp_listener = TcpListener::bind(addr).await.expect("tcp listener");

        (
            tcp_listener.local_addr().expect("bound to local address"),
            TcpListenerStream::new(tcp_listener),
        )
    }

    #[allow(clippy::unused_async)]
    async fn spawn_server(tcp_listener_stream: TcpListenerStream) -> Arc<PubSubState> {
        let server = PersistGrpcPubSubServer::new(&test_persist_config(), &MetricsRegistry::new());
        let server_state = Arc::clone(&server.state);

        let _server_task = mz_ore::task::spawn(|| "server".to_string(), async move {
            server.serve_with_stream(tcp_listener_stream).await
        });
        server_state
    }

    async fn poll_until_true<F>(timeout: Duration, f: F)
    where
        F: Fn() -> bool,
    {
        let now = Instant::now();
        loop {
            if f() {
                return;
            }

            if now.elapsed() > timeout {
                panic!("timed out");
            }

            tokio::time::sleep(Duration::from_millis(1)).await;
        }
    }

    fn assert_push(message: ProtoPubSubMessage, shard: &ShardId, data: &VersionedData) {
        let message = message.message.expect("proto contains message");
        match message {
            Message::PushDiff(x) => {
                assert_eq!(x.shard_id, shard.into_proto());
                assert_eq!(x.seqno, data.seqno.into_proto());
                assert_eq!(x.diff, data.data);
            }
            Message::Subscribe(_) | Message::Unsubscribe(_) => panic!("unexpected message type"),
        };
    }

    fn test_persist_config() -> PersistConfig {
        let mut cfg = PersistConfig::new_for_tests();
        cfg.pubsub_reconnect_backoff = Duration::ZERO;

        let mut updates = ConfigUpdates::default();
        updates.add(&PUBSUB_CLIENT_ENABLED, true);
        cfg.apply_from(&updates);

        cfg
    }
}