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
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
// 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.

//! Connection types.

use std::borrow::Cow;
use std::collections::{BTreeMap, BTreeSet};
use std::net::SocketAddr;
use std::sync::Arc;

use anyhow::{anyhow, bail, Context};
use itertools::Itertools;
use mz_ccsr::tls::{Certificate, Identity};
use mz_cloud_resources::{vpc_endpoint_host, AwsExternalIdPrefix, CloudResourceReader};
use mz_dyncfg::ConfigSet;
use mz_kafka_util::client::{
    BrokerAddr, BrokerRewrite, MzClientContext, MzKafkaError, TunnelConfig, TunnelingClientContext,
};
use mz_ore::assert_none;
use mz_ore::error::ErrorExt;
use mz_ore::future::{InTask, OreFutureExt};
use mz_ore::netio::resolve_address;
use mz_ore::num::NonNeg;
use mz_postgres_util::tunnel::PostgresFlavor;
use mz_proto::tokio_postgres::any_ssl_mode;
use mz_proto::{IntoRustIfSome, ProtoType, RustType, TryFromProtoError};
use mz_repr::url::any_url;
use mz_repr::{CatalogItemId, GlobalId};
use mz_secrets::SecretsReader;
use mz_ssh_util::keys::SshKeyPair;
use mz_ssh_util::tunnel::SshTunnelConfig;
use mz_ssh_util::tunnel_manager::{ManagedSshTunnelHandle, SshTunnelManager};
use mz_tls_util::Pkcs12Archive;
use mz_tracing::CloneableEnvFilter;
use proptest::strategy::Strategy;
use proptest_derive::Arbitrary;
use rdkafka::config::FromClientConfigAndContext;
use rdkafka::consumer::{BaseConsumer, Consumer};
use rdkafka::ClientContext;
use regex::Regex;
use serde::{Deserialize, Deserializer, Serialize};
use tokio::net;
use tokio::runtime::Handle;
use tokio_postgres::config::SslMode;
use tracing::{debug, warn};
use url::Url;

use crate::configuration::StorageConfiguration;
use crate::connections::aws::{
    AwsConnection, AwsConnectionReference, AwsConnectionValidationError,
};
use crate::connections::string_or_secret::StringOrSecret;
use crate::controller::AlterError;
use crate::dyncfgs::{
    ENFORCE_EXTERNAL_ADDRESSES, KAFKA_CLIENT_ID_ENRICHMENT_RULES,
    KAFKA_DEFAULT_AWS_PRIVATELINK_ENDPOINT_IDENTIFICATION_ALGORITHM,
};
use crate::errors::{ContextCreationError, CsrConnectError};
use crate::AlterCompatible;

pub mod aws;
pub mod inline;
pub mod string_or_secret;

include!(concat!(env!("OUT_DIR"), "/mz_storage_types.connections.rs"));

/// An extension trait for [`SecretsReader`]
#[async_trait::async_trait]
trait SecretsReaderExt {
    /// `SecretsReader::read`, but optionally run in a task.
    async fn read_in_task_if(
        &self,
        in_task: InTask,
        id: CatalogItemId,
    ) -> Result<Vec<u8>, anyhow::Error>;

    /// `SecretsReader::read_string`, but optionally run in a task.
    async fn read_string_in_task_if(
        &self,
        in_task: InTask,
        id: CatalogItemId,
    ) -> Result<String, anyhow::Error>;
}

#[async_trait::async_trait]
impl SecretsReaderExt for Arc<dyn SecretsReader> {
    async fn read_in_task_if(
        &self,
        in_task: InTask,
        id: CatalogItemId,
    ) -> Result<Vec<u8>, anyhow::Error> {
        let sr = Arc::clone(self);
        async move { sr.read(id).await }
            .run_in_task_if(in_task, || "secrets_reader_read".to_string())
            .await
    }
    async fn read_string_in_task_if(
        &self,
        in_task: InTask,
        id: CatalogItemId,
    ) -> Result<String, anyhow::Error> {
        let sr = Arc::clone(self);
        async move { sr.read_string(id).await }
            .run_in_task_if(in_task, || "secrets_reader_read".to_string())
            .await
    }
}

/// Extra context to pass through when instantiating a connection for a source
/// or sink.
///
/// Should be kept cheaply cloneable.
#[derive(Debug, Clone)]
pub struct ConnectionContext {
    /// An opaque identifier for the environment in which this process is
    /// running.
    ///
    /// The storage layer is intentionally unaware of the structure within this
    /// identifier. Higher layers of the stack can make use of that structure,
    /// but the storage layer should be oblivious to it.
    pub environment_id: String,
    /// The level for librdkafka's logs.
    pub librdkafka_log_level: tracing::Level,
    /// A prefix for an external ID to use for all AWS AssumeRole operations.
    pub aws_external_id_prefix: Option<AwsExternalIdPrefix>,
    /// The ARN for a Materialize-controlled role to assume before assuming
    /// a customer's requested role for an AWS connection.
    pub aws_connection_role_arn: Option<String>,
    /// A secrets reader.
    pub secrets_reader: Arc<dyn SecretsReader>,
    /// A cloud resource reader, if supported in this configuration.
    pub cloud_resource_reader: Option<Arc<dyn CloudResourceReader>>,
    /// A manager for SSH tunnels.
    pub ssh_tunnel_manager: SshTunnelManager,
}

impl ConnectionContext {
    /// Constructs a new connection context from command line arguments.
    ///
    /// **WARNING:** it is critical for security that the `aws_external_id` be
    /// provided by the operator of the Materialize service (i.e., via a CLI
    /// argument or environment variable) and not the end user of Materialize
    /// (e.g., via a configuration option in a SQL statement). See
    /// [`AwsExternalIdPrefix`] for details.
    pub fn from_cli_args(
        environment_id: String,
        startup_log_level: &CloneableEnvFilter,
        aws_external_id_prefix: Option<AwsExternalIdPrefix>,
        aws_connection_role_arn: Option<String>,
        secrets_reader: Arc<dyn SecretsReader>,
        cloud_resource_reader: Option<Arc<dyn CloudResourceReader>>,
    ) -> ConnectionContext {
        ConnectionContext {
            environment_id,
            librdkafka_log_level: mz_ore::tracing::crate_level(
                &startup_log_level.clone().into(),
                "librdkafka",
            ),
            aws_external_id_prefix,
            aws_connection_role_arn,
            secrets_reader,
            cloud_resource_reader,
            ssh_tunnel_manager: SshTunnelManager::default(),
        }
    }

    /// Constructs a new connection context for usage in tests.
    pub fn for_tests(secrets_reader: Arc<dyn SecretsReader>) -> ConnectionContext {
        ConnectionContext {
            environment_id: "test-environment-id".into(),
            librdkafka_log_level: tracing::Level::INFO,
            aws_external_id_prefix: Some(
                AwsExternalIdPrefix::new_from_cli_argument_or_environment_variable(
                    "test-aws-external-id-prefix",
                ),
            ),
            aws_connection_role_arn: Some(
                "arn:aws:iam::123456789000:role/MaterializeConnection".into(),
            ),
            secrets_reader,
            cloud_resource_reader: None,
            ssh_tunnel_manager: SshTunnelManager::default(),
        }
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub enum Connection<C: ConnectionAccess = InlinedConnection> {
    Kafka(KafkaConnection<C>),
    Csr(CsrConnection<C>),
    Postgres(PostgresConnection<C>),
    Ssh(SshConnection),
    Aws(AwsConnection),
    AwsPrivatelink(AwsPrivatelinkConnection),
    MySql(MySqlConnection<C>),
}

impl<R: ConnectionResolver> IntoInlineConnection<Connection, R>
    for Connection<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> Connection {
        match self {
            Connection::Kafka(kafka) => Connection::Kafka(kafka.into_inline_connection(r)),
            Connection::Csr(csr) => Connection::Csr(csr.into_inline_connection(r)),
            Connection::Postgres(pg) => Connection::Postgres(pg.into_inline_connection(r)),
            Connection::Ssh(ssh) => Connection::Ssh(ssh),
            Connection::Aws(aws) => Connection::Aws(aws),
            Connection::AwsPrivatelink(awspl) => Connection::AwsPrivatelink(awspl),
            Connection::MySql(mysql) => Connection::MySql(mysql.into_inline_connection(r)),
        }
    }
}

impl<C: ConnectionAccess> Connection<C> {
    /// Whether this connection should be validated by default on creation.
    pub fn validate_by_default(&self) -> bool {
        match self {
            Connection::Kafka(conn) => conn.validate_by_default(),
            Connection::Csr(conn) => conn.validate_by_default(),
            Connection::Postgres(conn) => conn.validate_by_default(),
            Connection::Ssh(conn) => conn.validate_by_default(),
            Connection::Aws(conn) => conn.validate_by_default(),
            Connection::AwsPrivatelink(conn) => conn.validate_by_default(),
            Connection::MySql(conn) => conn.validate_by_default(),
        }
    }
}

impl Connection<InlinedConnection> {
    /// Validates this connection by attempting to connect to the upstream system.
    pub async fn validate(
        &self,
        id: CatalogItemId,
        storage_configuration: &StorageConfiguration,
    ) -> Result<(), ConnectionValidationError> {
        match self {
            Connection::Kafka(conn) => conn.validate(id, storage_configuration).await?,
            Connection::Csr(conn) => conn.validate(id, storage_configuration).await?,
            Connection::Postgres(conn) => conn.validate(id, storage_configuration).await?,
            Connection::Ssh(conn) => conn.validate(id, storage_configuration).await?,
            Connection::Aws(conn) => conn.validate(id, storage_configuration).await?,
            Connection::AwsPrivatelink(conn) => conn.validate(id, storage_configuration).await?,
            Connection::MySql(conn) => conn.validate(id, storage_configuration).await?,
        }
        Ok(())
    }

    pub fn unwrap_kafka(self) -> <InlinedConnection as ConnectionAccess>::Kafka {
        match self {
            Self::Kafka(conn) => conn,
            o => unreachable!("{o:?} is not a Kafka connection"),
        }
    }

    pub fn unwrap_pg(self) -> <InlinedConnection as ConnectionAccess>::Pg {
        match self {
            Self::Postgres(conn) => conn,
            o => unreachable!("{o:?} is not a Postgres connection"),
        }
    }

    pub fn unwrap_mysql(self) -> <InlinedConnection as ConnectionAccess>::MySql {
        match self {
            Self::MySql(conn) => conn,
            o => unreachable!("{o:?} is not a MySQL connection"),
        }
    }

    pub fn unwrap_aws(self) -> <InlinedConnection as ConnectionAccess>::Aws {
        match self {
            Self::Aws(conn) => conn,
            o => unreachable!("{o:?} is not an AWS connection"),
        }
    }

    pub fn unwrap_ssh(self) -> <InlinedConnection as ConnectionAccess>::Ssh {
        match self {
            Self::Ssh(conn) => conn,
            o => unreachable!("{o:?} is not an SSH connection"),
        }
    }

    pub fn unwrap_csr(self) -> <InlinedConnection as ConnectionAccess>::Csr {
        match self {
            Self::Csr(conn) => conn,
            o => unreachable!("{o:?} is not a Kafka connection"),
        }
    }
}

/// An error returned by [`Connection::validate`].
#[derive(thiserror::Error, Debug)]
pub enum ConnectionValidationError {
    #[error(transparent)]
    Aws(#[from] AwsConnectionValidationError),
    #[error("{}", .0.display_with_causes())]
    Other(#[from] anyhow::Error),
}

impl ConnectionValidationError {
    /// Reports additional details about the error, if any are available.
    pub fn detail(&self) -> Option<String> {
        match self {
            ConnectionValidationError::Aws(e) => e.detail(),
            ConnectionValidationError::Other(_) => None,
        }
    }

    /// Reports a hint for the user about how the error could be fixed.
    pub fn hint(&self) -> Option<String> {
        match self {
            ConnectionValidationError::Aws(e) => e.hint(),
            ConnectionValidationError::Other(_) => None,
        }
    }
}

impl<C: ConnectionAccess> AlterCompatible for Connection<C> {
    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        match (self, other) {
            (Self::Aws(s), Self::Aws(o)) => s.alter_compatible(id, o),
            (Self::AwsPrivatelink(s), Self::AwsPrivatelink(o)) => s.alter_compatible(id, o),
            (Self::Ssh(s), Self::Ssh(o)) => s.alter_compatible(id, o),
            (Self::Csr(s), Self::Csr(o)) => s.alter_compatible(id, o),
            (Self::Kafka(s), Self::Kafka(o)) => s.alter_compatible(id, o),
            (Self::Postgres(s), Self::Postgres(o)) => s.alter_compatible(id, o),
            (Self::MySql(s), Self::MySql(o)) => s.alter_compatible(id, o),
            _ => {
                tracing::warn!(
                    "Connection incompatible:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );
                Err(AlterError { id })
            }
        }
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct AwsPrivatelinkConnection {
    pub service_name: String,
    pub availability_zones: Vec<String>,
}

impl AlterCompatible for AwsPrivatelinkConnection {
    fn alter_compatible(&self, _id: GlobalId, _other: &Self) -> Result<(), AlterError> {
        // Every element of the AwsPrivatelinkConnection connection is configurable.
        Ok(())
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct KafkaTlsConfig {
    pub identity: Option<TlsIdentity>,
    pub root_cert: Option<StringOrSecret>,
}

#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, Arbitrary)]
pub struct KafkaSaslConfig<C: ConnectionAccess = InlinedConnection> {
    pub mechanism: String,
    pub username: StringOrSecret,
    pub password: Option<CatalogItemId>,
    pub aws: Option<AwsConnectionReference<C>>,
}

impl<R: ConnectionResolver> IntoInlineConnection<KafkaSaslConfig, R>
    for KafkaSaslConfig<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> KafkaSaslConfig {
        KafkaSaslConfig {
            mechanism: self.mechanism,
            username: self.username,
            password: self.password,
            aws: self.aws.map(|aws| aws.into_inline_connection(&r)),
        }
    }
}

/// Specifies a Kafka broker in a [`KafkaConnection`].
#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct KafkaBroker<C: ConnectionAccess = InlinedConnection> {
    /// The address of the Kafka broker.
    pub address: String,
    /// An optional tunnel to use when connecting to the broker.
    pub tunnel: Tunnel<C>,
}

impl<R: ConnectionResolver> IntoInlineConnection<KafkaBroker, R>
    for KafkaBroker<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> KafkaBroker {
        let KafkaBroker { address, tunnel } = self;
        KafkaBroker {
            address,
            tunnel: tunnel.into_inline_connection(r),
        }
    }
}

impl RustType<ProtoKafkaBroker> for KafkaBroker {
    fn into_proto(&self) -> ProtoKafkaBroker {
        ProtoKafkaBroker {
            address: self.address.into_proto(),
            tunnel: Some(self.tunnel.into_proto()),
        }
    }

    fn from_proto(proto: ProtoKafkaBroker) -> Result<Self, TryFromProtoError> {
        Ok(KafkaBroker {
            address: proto.address.into_rust()?,
            tunnel: proto
                .tunnel
                .into_rust_if_some("ProtoKafkaConnection::tunnel")?,
        })
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, Default)]
pub struct KafkaTopicOptions {
    /// The replication factor for the topic.
    /// If `None`, the broker default will be used.
    pub replication_factor: Option<NonNeg<i32>>,
    /// The number of partitions to create.
    /// If `None`, the broker default will be used.
    pub partition_count: Option<NonNeg<i32>>,
    /// The initial configuration parameters for the topic.
    pub topic_config: BTreeMap<String, String>,
}

impl RustType<ProtoKafkaTopicOptions> for KafkaTopicOptions {
    fn into_proto(&self) -> ProtoKafkaTopicOptions {
        ProtoKafkaTopicOptions {
            replication_factor: self.replication_factor.map(|f| *f),
            partition_count: self.partition_count.map(|f| *f),
            topic_config: self.topic_config.clone(),
        }
    }

    fn from_proto(proto: ProtoKafkaTopicOptions) -> Result<Self, TryFromProtoError> {
        Ok(KafkaTopicOptions {
            replication_factor: proto.replication_factor.map(NonNeg::try_from).transpose()?,
            partition_count: proto.partition_count.map(NonNeg::try_from).transpose()?,
            topic_config: proto.topic_config,
        })
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct KafkaConnection<C: ConnectionAccess = InlinedConnection> {
    pub brokers: Vec<KafkaBroker<C>>,
    /// A tunnel through which to route traffic,
    /// that can be overridden for individual brokers
    /// in `brokers`.
    pub default_tunnel: Tunnel<C>,
    pub progress_topic: Option<String>,
    pub progress_topic_options: KafkaTopicOptions,
    pub options: BTreeMap<String, StringOrSecret>,
    pub tls: Option<KafkaTlsConfig>,
    pub sasl: Option<KafkaSaslConfig<C>>,
}

impl<R: ConnectionResolver> IntoInlineConnection<KafkaConnection, R>
    for KafkaConnection<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> KafkaConnection {
        let KafkaConnection {
            brokers,
            progress_topic,
            progress_topic_options,
            default_tunnel,
            options,
            tls,
            sasl,
        } = self;

        let brokers = brokers
            .into_iter()
            .map(|broker| broker.into_inline_connection(&r))
            .collect();

        KafkaConnection {
            brokers,
            progress_topic,
            progress_topic_options,
            default_tunnel: default_tunnel.into_inline_connection(&r),
            options,
            tls,
            sasl: sasl.map(|sasl| sasl.into_inline_connection(&r)),
        }
    }
}

impl<C: ConnectionAccess> KafkaConnection<C> {
    /// Returns the name of the progress topic to use for the connection.
    ///
    /// The caller is responsible for providing the connection ID as it is not
    /// known to `KafkaConnection`.
    pub fn progress_topic(
        &self,
        connection_context: &ConnectionContext,
        connection_id: CatalogItemId,
    ) -> Cow<str> {
        if let Some(progress_topic) = &self.progress_topic {
            Cow::Borrowed(progress_topic)
        } else {
            Cow::Owned(format!(
                "_materialize-progress-{}-{}",
                connection_context.environment_id, connection_id,
            ))
        }
    }

    fn validate_by_default(&self) -> bool {
        true
    }
}

impl KafkaConnection {
    /// Generates a string that can be used as the base for a configuration ID
    /// (e.g., `client.id`, `group.id`, `transactional.id`) for a Kafka source
    /// or sink.
    pub fn id_base(
        connection_context: &ConnectionContext,
        connection_id: CatalogItemId,
        object_id: GlobalId,
    ) -> String {
        format!(
            "materialize-{}-{}-{}",
            connection_context.environment_id, connection_id, object_id,
        )
    }

    /// Enriches the provided `client_id` according to any enrichment rules in
    /// the `kafka_client_id_enrichment_rules` configuration parameter.
    pub fn enrich_client_id(&self, configs: &ConfigSet, client_id: &mut String) {
        #[derive(Debug, Deserialize)]
        struct EnrichmentRule {
            #[serde(deserialize_with = "deserialize_regex")]
            pattern: Regex,
            payload: String,
        }

        fn deserialize_regex<'de, D>(deserializer: D) -> Result<Regex, D::Error>
        where
            D: Deserializer<'de>,
        {
            let buf = String::deserialize(deserializer)?;
            Regex::new(&buf).map_err(serde::de::Error::custom)
        }

        let rules = KAFKA_CLIENT_ID_ENRICHMENT_RULES.get(configs);
        let rules = match serde_json::from_value::<Vec<EnrichmentRule>>(rules) {
            Ok(rules) => rules,
            Err(e) => {
                warn!(%e, "failed to decode kafka_client_id_enrichment_rules");
                return;
            }
        };

        // Check every rule against every broker. Rules are matched in the order
        // that they are specified. It is usually a configuration error if
        // multiple rules match the same list of Kafka brokers, but we
        // nonetheless want to provide well defined semantics.
        debug!(?self.brokers, "evaluating client ID enrichment rules");
        for rule in rules {
            let is_match = self
                .brokers
                .iter()
                .any(|b| rule.pattern.is_match(&b.address));
            debug!(?rule, is_match, "evaluated client ID enrichment rule");
            if is_match {
                client_id.push('-');
                client_id.push_str(&rule.payload);
            }
        }
    }

    /// Creates a Kafka client for the connection.
    pub async fn create_with_context<C, T>(
        &self,
        storage_configuration: &StorageConfiguration,
        context: C,
        extra_options: &BTreeMap<&str, String>,
        in_task: InTask,
    ) -> Result<T, ContextCreationError>
    where
        C: ClientContext,
        T: FromClientConfigAndContext<TunnelingClientContext<C>>,
    {
        let mut options = self.options.clone();

        // Ensure that Kafka topics are *not* automatically created when
        // consuming, producing, or fetching metadata for a topic. This ensures
        // that we don't accidentally create topics with the wrong number of
        // partitions.
        options.insert("allow.auto.create.topics".into(), "false".into());

        let brokers = match &self.default_tunnel {
            Tunnel::AwsPrivatelink(t) => {
                assert!(&self.brokers.is_empty());

                let algo = KAFKA_DEFAULT_AWS_PRIVATELINK_ENDPOINT_IDENTIFICATION_ALGORITHM
                    .get(storage_configuration.config_set());
                options.insert("ssl.endpoint.identification.algorithm".into(), algo.into());

                // When using a default privatelink tunnel broker/brokers cannot be specified
                // instead the tunnel connection_id and port are used for the initial connection.
                format!(
                    "{}:{}",
                    vpc_endpoint_host(
                        t.connection_id,
                        None, // Default tunnel does not support availability zones.
                    ),
                    t.port.unwrap_or(9092)
                )
            }
            _ => self.brokers.iter().map(|b| &b.address).join(","),
        };
        options.insert("bootstrap.servers".into(), brokers.into());
        let security_protocol = match (self.tls.is_some(), self.sasl.is_some()) {
            (false, false) => "PLAINTEXT",
            (true, false) => "SSL",
            (false, true) => "SASL_PLAINTEXT",
            (true, true) => "SASL_SSL",
        };
        options.insert("security.protocol".into(), security_protocol.into());
        if let Some(tls) = &self.tls {
            if let Some(root_cert) = &tls.root_cert {
                options.insert("ssl.ca.pem".into(), root_cert.clone());
            }
            if let Some(identity) = &tls.identity {
                options.insert("ssl.key.pem".into(), StringOrSecret::Secret(identity.key));
                options.insert("ssl.certificate.pem".into(), identity.cert.clone());
            }
        }
        if let Some(sasl) = &self.sasl {
            options.insert("sasl.mechanisms".into(), (&sasl.mechanism).into());
            options.insert("sasl.username".into(), sasl.username.clone());
            if let Some(password) = sasl.password {
                options.insert("sasl.password".into(), StringOrSecret::Secret(password));
            }
        }

        let mut config = mz_kafka_util::client::create_new_client_config(
            storage_configuration
                .connection_context
                .librdkafka_log_level,
            storage_configuration.parameters.kafka_timeout_config,
        );
        for (k, v) in options {
            config.set(
                k,
                v.get_string(
                    in_task,
                    &storage_configuration.connection_context.secrets_reader,
                )
                .await
                .context("reading kafka secret")?,
            );
        }
        for (k, v) in extra_options {
            config.set(*k, v);
        }

        let aws_config = match self.sasl.as_ref().and_then(|sasl| sasl.aws.as_ref()) {
            None => None,
            Some(aws) => Some(
                aws.connection
                    .load_sdk_config(
                        &storage_configuration.connection_context,
                        aws.connection_id,
                        in_task,
                    )
                    .await?,
            ),
        };

        // TODO(roshan): Implement enforcement of external address validation once
        // rdkafka client has been updated to support providing multiple resolved
        // addresses for brokers
        let mut context = TunnelingClientContext::new(
            context,
            Handle::current(),
            storage_configuration
                .connection_context
                .ssh_tunnel_manager
                .clone(),
            storage_configuration.parameters.ssh_timeout_config,
            aws_config,
            in_task,
        );

        match &self.default_tunnel {
            Tunnel::Direct => {
                // By default, don't offer a default override for broker address lookup.
            }
            Tunnel::AwsPrivatelink(pl) => {
                context.set_default_tunnel(TunnelConfig::StaticHost(vpc_endpoint_host(
                    pl.connection_id,
                    None, // Default tunnel does not support availability zones.
                )));
            }
            Tunnel::Ssh(ssh_tunnel) => {
                let secret = storage_configuration
                    .connection_context
                    .secrets_reader
                    .read_in_task_if(in_task, ssh_tunnel.connection_id)
                    .await?;
                let key_pair = SshKeyPair::from_bytes(&secret)?;

                // Ensure any ssh-bastion address we connect to is resolved to an external address.
                let resolved = resolve_address(
                    &ssh_tunnel.connection.host,
                    ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
                )
                .await?;
                context.set_default_tunnel(TunnelConfig::Ssh(SshTunnelConfig {
                    host: resolved
                        .iter()
                        .map(|a| a.to_string())
                        .collect::<BTreeSet<_>>(),
                    port: ssh_tunnel.connection.port,
                    user: ssh_tunnel.connection.user.clone(),
                    key_pair,
                }));
            }
        }

        for broker in &self.brokers {
            let mut addr_parts = broker.address.splitn(2, ':');
            let addr = BrokerAddr {
                host: addr_parts
                    .next()
                    .context("BROKER is not address:port")?
                    .into(),
                port: addr_parts
                    .next()
                    .unwrap_or("9092")
                    .parse()
                    .context("parsing BROKER port")?,
            };
            match &broker.tunnel {
                Tunnel::Direct => {
                    // By default, don't override broker address lookup.
                    //
                    // N.B.
                    //
                    // We _could_ pre-setup the default ssh tunnel for all known brokers here, but
                    // we avoid doing because:
                    // - Its not necessary.
                    // - Not doing so makes it easier to test the `FailedDefaultSshTunnel` path
                    // in the `TunnelingClientContext`.
                }
                Tunnel::AwsPrivatelink(aws_privatelink) => {
                    let host = mz_cloud_resources::vpc_endpoint_host(
                        aws_privatelink.connection_id,
                        aws_privatelink.availability_zone.as_deref(),
                    );
                    let port = aws_privatelink.port;
                    context.add_broker_rewrite(
                        addr,
                        BrokerRewrite {
                            host: host.clone(),
                            port,
                        },
                    );
                }
                Tunnel::Ssh(ssh_tunnel) => {
                    // Ensure any SSH bastion address we connect to is resolved to an external address.
                    let ssh_host_resolved = resolve_address(
                        &ssh_tunnel.connection.host,
                        ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
                    )
                    .await?;
                    context
                        .add_ssh_tunnel(
                            addr,
                            SshTunnelConfig {
                                host: ssh_host_resolved
                                    .iter()
                                    .map(|a| a.to_string())
                                    .collect::<BTreeSet<_>>(),
                                port: ssh_tunnel.connection.port,
                                user: ssh_tunnel.connection.user.clone(),
                                key_pair: SshKeyPair::from_bytes(
                                    &storage_configuration
                                        .connection_context
                                        .secrets_reader
                                        .read_in_task_if(in_task, ssh_tunnel.connection_id)
                                        .await?,
                                )?,
                            },
                        )
                        .await
                        .map_err(ContextCreationError::Ssh)?;
                }
            }
        }

        Ok(config.create_with_context(context)?)
    }

    async fn validate(
        &self,
        _id: CatalogItemId,
        storage_configuration: &StorageConfiguration,
    ) -> Result<(), anyhow::Error> {
        let (context, error_rx) = MzClientContext::with_errors();
        let consumer: BaseConsumer<_> = self
            .create_with_context(
                storage_configuration,
                context,
                &BTreeMap::new(),
                // We are in a normal tokio context during validation, already.
                InTask::No,
            )
            .await?;
        let consumer = Arc::new(consumer);

        let timeout = storage_configuration
            .parameters
            .kafka_timeout_config
            .fetch_metadata_timeout;

        // librdkafka doesn't expose an API for determining whether a connection to
        // the Kafka cluster has been successfully established. So we make a
        // metadata request, though we don't care about the results, so that we can
        // report any errors making that request. If the request succeeds, we know
        // we were able to contact at least one broker, and that's a good proxy for
        // being able to contact all the brokers in the cluster.
        //
        // The downside of this approach is it produces a generic error message like
        // "metadata fetch error" with no additional details. The real networking
        // error is buried in the librdkafka logs, which are not visible to users.
        let result = mz_ore::task::spawn_blocking(|| "kafka_get_metadata", {
            let consumer = Arc::clone(&consumer);
            move || consumer.fetch_metadata(None, timeout)
        })
        .await?;
        match result {
            Ok(_) => Ok(()),
            // The error returned by `fetch_metadata` does not provide any details which makes for
            // a crappy user facing error message. For this reason we attempt to grab a better
            // error message from the client context, which should contain any error logs emitted
            // by librdkafka, and fallback to the generic error if there is nothing there.
            Err(err) => {
                // Multiple errors might have been logged during this validation but some are more
                // relevant than others. Specifically, we prefer non-internal errors over internal
                // errors since those give much more useful information to the users.
                let main_err = error_rx.try_iter().reduce(|cur, new| match cur {
                    MzKafkaError::Internal(_) => new,
                    _ => cur,
                });

                // Don't drop the consumer until after we've drained the errors
                // channel. Dropping the consumer can introduce spurious errors.
                // See database-issues#7432.
                drop(consumer);

                match main_err {
                    Some(err) => Err(err.into()),
                    None => Err(err.into()),
                }
            }
        }
    }
}

impl<C: ConnectionAccess> AlterCompatible for KafkaConnection<C> {
    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        let KafkaConnection {
            brokers: _,
            default_tunnel: _,
            progress_topic,
            progress_topic_options,
            options: _,
            tls: _,
            sasl: _,
        } = self;

        let compatibility_checks = [
            (progress_topic == &other.progress_topic, "progress_topic"),
            (
                progress_topic_options == &other.progress_topic_options,
                "progress_topic_options",
            ),
        ];

        for (compatible, field) in compatibility_checks {
            if !compatible {
                tracing::warn!(
                    "KafkaConnection incompatible at {field}:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );

                return Err(AlterError { id });
            }
        }

        Ok(())
    }
}

impl RustType<ProtoKafkaConnectionTlsConfig> for KafkaTlsConfig {
    fn into_proto(&self) -> ProtoKafkaConnectionTlsConfig {
        ProtoKafkaConnectionTlsConfig {
            identity: self.identity.into_proto(),
            root_cert: self.root_cert.into_proto(),
        }
    }

    fn from_proto(proto: ProtoKafkaConnectionTlsConfig) -> Result<Self, TryFromProtoError> {
        Ok(KafkaTlsConfig {
            root_cert: proto.root_cert.into_rust()?,
            identity: proto.identity.into_rust()?,
        })
    }
}

impl RustType<ProtoKafkaConnectionSaslConfig> for KafkaSaslConfig {
    fn into_proto(&self) -> ProtoKafkaConnectionSaslConfig {
        ProtoKafkaConnectionSaslConfig {
            mechanism: self.mechanism.into_proto(),
            username: Some(self.username.into_proto()),
            password: self.password.into_proto(),
            aws: self.aws.into_proto(),
        }
    }

    fn from_proto(proto: ProtoKafkaConnectionSaslConfig) -> Result<Self, TryFromProtoError> {
        Ok(KafkaSaslConfig {
            mechanism: proto.mechanism,
            username: proto
                .username
                .into_rust_if_some("ProtoKafkaConnectionSaslConfig::username")?,
            password: proto.password.into_rust()?,
            aws: proto.aws.into_rust()?,
        })
    }
}

impl RustType<ProtoKafkaConnection> for KafkaConnection {
    fn into_proto(&self) -> ProtoKafkaConnection {
        ProtoKafkaConnection {
            brokers: self.brokers.into_proto(),
            default_tunnel: Some(self.default_tunnel.into_proto()),
            progress_topic: self.progress_topic.into_proto(),
            progress_topic_options: Some(self.progress_topic_options.into_proto()),
            options: self
                .options
                .iter()
                .map(|(k, v)| (k.clone(), v.into_proto()))
                .collect(),
            tls: self.tls.into_proto(),
            sasl: self.sasl.into_proto(),
        }
    }

    fn from_proto(proto: ProtoKafkaConnection) -> Result<Self, TryFromProtoError> {
        Ok(KafkaConnection {
            brokers: proto.brokers.into_rust()?,
            default_tunnel: proto
                .default_tunnel
                .into_rust_if_some("ProtoKafkaConnection::default_tunnel")?,
            progress_topic: proto.progress_topic,
            progress_topic_options: match proto.progress_topic_options {
                Some(progress_topic_options) => progress_topic_options.into_rust()?,
                None => Default::default(),
            },
            options: proto
                .options
                .into_iter()
                .map(|(k, v)| StringOrSecret::from_proto(v).map(|v| (k, v)))
                .collect::<Result<_, _>>()?,
            tls: proto.tls.into_rust()?,
            sasl: proto.sasl.into_rust()?,
        })
    }
}

/// A connection to a Confluent Schema Registry.
#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, Arbitrary)]
pub struct CsrConnection<C: ConnectionAccess = InlinedConnection> {
    /// The URL of the schema registry.
    #[proptest(strategy = "any_url()")]
    pub url: Url,
    /// Trusted root TLS certificate in PEM format.
    pub tls_root_cert: Option<StringOrSecret>,
    /// An optional TLS client certificate for authentication with the schema
    /// registry.
    pub tls_identity: Option<TlsIdentity>,
    /// Optional HTTP authentication credentials for the schema registry.
    pub http_auth: Option<CsrConnectionHttpAuth>,
    /// A tunnel through which to route traffic.
    pub tunnel: Tunnel<C>,
}

impl<R: ConnectionResolver> IntoInlineConnection<CsrConnection, R>
    for CsrConnection<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> CsrConnection {
        let CsrConnection {
            url,
            tls_root_cert,
            tls_identity,
            http_auth,
            tunnel,
        } = self;
        CsrConnection {
            url,
            tls_root_cert,
            tls_identity,
            http_auth,
            tunnel: tunnel.into_inline_connection(r),
        }
    }
}

impl<C: ConnectionAccess> CsrConnection<C> {
    fn validate_by_default(&self) -> bool {
        true
    }
}

impl CsrConnection {
    /// Constructs a schema registry client from the connection.
    pub async fn connect(
        &self,
        storage_configuration: &StorageConfiguration,
        in_task: InTask,
    ) -> Result<mz_ccsr::Client, CsrConnectError> {
        let mut client_config = mz_ccsr::ClientConfig::new(self.url.clone());
        if let Some(root_cert) = &self.tls_root_cert {
            let root_cert = root_cert
                .get_string(
                    in_task,
                    &storage_configuration.connection_context.secrets_reader,
                )
                .await?;
            let root_cert = Certificate::from_pem(root_cert.as_bytes())?;
            client_config = client_config.add_root_certificate(root_cert);
        }

        if let Some(tls_identity) = &self.tls_identity {
            let key = &storage_configuration
                .connection_context
                .secrets_reader
                .read_string_in_task_if(in_task, tls_identity.key)
                .await?;
            let cert = tls_identity
                .cert
                .get_string(
                    in_task,
                    &storage_configuration.connection_context.secrets_reader,
                )
                .await?;
            let ident = Identity::from_pem(key.as_bytes(), cert.as_bytes())?;
            client_config = client_config.identity(ident);
        }

        if let Some(http_auth) = &self.http_auth {
            let username = http_auth
                .username
                .get_string(
                    in_task,
                    &storage_configuration.connection_context.secrets_reader,
                )
                .await?;
            let password = match http_auth.password {
                None => None,
                Some(password) => Some(
                    storage_configuration
                        .connection_context
                        .secrets_reader
                        .read_string_in_task_if(in_task, password)
                        .await?,
                ),
            };
            client_config = client_config.auth(username, password);
        }

        // `net::lookup_host` requires a port but the port will be ignored when
        // passed to `resolve_to_addrs`. We use a dummy port that will be easy
        // to spot in the logs to make it obvious if some component downstream
        // incorrectly starts using this port.
        const DUMMY_PORT: u16 = 11111;

        // TODO: use types to enforce that the URL has a string hostname.
        let host = self
            .url
            .host_str()
            .ok_or_else(|| anyhow!("url missing host"))?;
        match &self.tunnel {
            Tunnel::Direct => {
                // Ensure any host we connect to is resolved to an external address.
                let resolved = resolve_address(
                    host,
                    ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
                )
                .await?;
                client_config = client_config.resolve_to_addrs(
                    host,
                    &resolved
                        .iter()
                        .map(|addr| SocketAddr::new(*addr, DUMMY_PORT))
                        .collect::<Vec<_>>(),
                )
            }
            Tunnel::Ssh(ssh_tunnel) => {
                let ssh_tunnel = ssh_tunnel
                    .connect(
                        storage_configuration,
                        host,
                        // Default to the default http port, but this
                        // could default to 8081...
                        self.url.port().unwrap_or(80),
                        in_task,
                    )
                    .await
                    .map_err(CsrConnectError::Ssh)?;

                // Carefully inject the SSH tunnel into the client
                // configuration. This is delicate because we need TLS
                // verification to continue to use the remote hostname rather
                // than the tunnel hostname.

                client_config = client_config
                    // `resolve_to_addrs` allows us to rewrite the hostname
                    // at the DNS level, which means the TCP connection is
                    // correctly routed through the tunnel, but TLS verification
                    // is still performed against the remote hostname.
                    // Unfortunately the port here is ignored...
                    .resolve_to_addrs(
                        host,
                        &[SocketAddr::new(ssh_tunnel.local_addr().ip(), DUMMY_PORT)],
                    )
                    // ...so we also dynamically rewrite the URL to use the
                    // current port for the SSH tunnel.
                    //
                    // WARNING: this is brittle, because we only dynamically
                    // update the client configuration with the tunnel *port*,
                    // and not the hostname This works fine in practice, because
                    // only the SSH tunnel port will change if the tunnel fails
                    // and has to be restarted (the hostname is always
                    // 127.0.0.1)--but this is an an implementation detail of
                    // the SSH tunnel code that we're relying on.
                    .dynamic_url({
                        let remote_url = self.url.clone();
                        move || {
                            let mut url = remote_url.clone();
                            url.set_port(Some(ssh_tunnel.local_addr().port()))
                                .expect("cannot fail");
                            url
                        }
                    });
            }
            Tunnel::AwsPrivatelink(connection) => {
                assert_none!(connection.port);

                let privatelink_host = mz_cloud_resources::vpc_endpoint_host(
                    connection.connection_id,
                    connection.availability_zone.as_deref(),
                );
                let addrs: Vec<_> = net::lookup_host((privatelink_host, DUMMY_PORT))
                    .await
                    .context("resolving PrivateLink host")?
                    .collect();
                client_config = client_config.resolve_to_addrs(host, &addrs)
            }
        }

        Ok(client_config.build()?)
    }

    async fn validate(
        &self,
        _id: CatalogItemId,
        storage_configuration: &StorageConfiguration,
    ) -> Result<(), anyhow::Error> {
        let client = self
            .connect(
                storage_configuration,
                // We are in a normal tokio context during validation, already.
                InTask::No,
            )
            .await?;
        client.list_subjects().await?;
        Ok(())
    }
}

impl RustType<ProtoCsrConnection> for CsrConnection {
    fn into_proto(&self) -> ProtoCsrConnection {
        ProtoCsrConnection {
            url: Some(self.url.into_proto()),
            tls_root_cert: self.tls_root_cert.into_proto(),
            tls_identity: self.tls_identity.into_proto(),
            http_auth: self.http_auth.into_proto(),
            tunnel: Some(self.tunnel.into_proto()),
        }
    }

    fn from_proto(proto: ProtoCsrConnection) -> Result<Self, TryFromProtoError> {
        Ok(CsrConnection {
            url: proto.url.into_rust_if_some("ProtoCsrConnection::url")?,
            tls_root_cert: proto.tls_root_cert.into_rust()?,
            tls_identity: proto.tls_identity.into_rust()?,
            http_auth: proto.http_auth.into_rust()?,
            tunnel: proto
                .tunnel
                .into_rust_if_some("ProtoCsrConnection::tunnel")?,
        })
    }
}

impl<C: ConnectionAccess> AlterCompatible for CsrConnection<C> {
    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        let CsrConnection {
            tunnel,
            // All non-tunnel fields may change
            url: _,
            tls_root_cert: _,
            tls_identity: _,
            http_auth: _,
        } = self;

        let compatibility_checks = [(tunnel.alter_compatible(id, &other.tunnel).is_ok(), "tunnel")];

        for (compatible, field) in compatibility_checks {
            if !compatible {
                tracing::warn!(
                    "CsrConnection incompatible at {field}:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );

                return Err(AlterError { id });
            }
        }
        Ok(())
    }
}

/// A TLS key pair used for client identity.
#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct TlsIdentity {
    /// The client's TLS public certificate in PEM format.
    pub cert: StringOrSecret,
    /// The ID of the secret containing the client's TLS private key in PEM
    /// format.
    pub key: CatalogItemId,
}

impl RustType<ProtoTlsIdentity> for TlsIdentity {
    fn into_proto(&self) -> ProtoTlsIdentity {
        ProtoTlsIdentity {
            cert: Some(self.cert.into_proto()),
            key: Some(self.key.into_proto()),
        }
    }

    fn from_proto(proto: ProtoTlsIdentity) -> Result<Self, TryFromProtoError> {
        Ok(TlsIdentity {
            cert: proto.cert.into_rust_if_some("ProtoTlsIdentity::cert")?,
            key: proto.key.into_rust_if_some("ProtoTlsIdentity::key")?,
        })
    }
}

/// HTTP authentication credentials in a [`CsrConnection`].
#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct CsrConnectionHttpAuth {
    /// The username.
    pub username: StringOrSecret,
    /// The ID of the secret containing the password, if any.
    pub password: Option<CatalogItemId>,
}

impl RustType<ProtoCsrConnectionHttpAuth> for CsrConnectionHttpAuth {
    fn into_proto(&self) -> ProtoCsrConnectionHttpAuth {
        ProtoCsrConnectionHttpAuth {
            username: Some(self.username.into_proto()),
            password: self.password.into_proto(),
        }
    }

    fn from_proto(proto: ProtoCsrConnectionHttpAuth) -> Result<Self, TryFromProtoError> {
        Ok(CsrConnectionHttpAuth {
            username: proto
                .username
                .into_rust_if_some("ProtoCsrConnectionHttpAuth::username")?,
            password: proto.password.into_rust()?,
        })
    }
}

/// A connection to a PostgreSQL server.
#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, Arbitrary)]
pub struct PostgresConnection<C: ConnectionAccess = InlinedConnection> {
    /// The hostname of the server.
    pub host: String,
    /// The port of the server.
    pub port: u16,
    /// The name of the database to connect to.
    pub database: String,
    /// The username to authenticate as.
    pub user: StringOrSecret,
    /// An optional password for authentication.
    pub password: Option<CatalogItemId>,
    /// A tunnel through which to route traffic.
    pub tunnel: Tunnel<C>,
    /// Whether to use TLS for encryption, authentication, or both.
    #[proptest(strategy = "any_ssl_mode()")]
    pub tls_mode: SslMode,
    /// An optional root TLS certificate in PEM format, to verify the server's
    /// identity.
    pub tls_root_cert: Option<StringOrSecret>,
    /// An optional TLS client certificate for authentication.
    pub tls_identity: Option<TlsIdentity>,
    /// The kind of postgres server we are connecting to. This can be vanilla, for a normal
    /// postgres server or some other system that is pg compatible, like Yugabyte, Aurora, etc.
    pub flavor: PostgresFlavor,
}

impl<R: ConnectionResolver> IntoInlineConnection<PostgresConnection, R>
    for PostgresConnection<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> PostgresConnection {
        let PostgresConnection {
            host,
            port,
            database,
            user,
            password,
            tunnel,
            tls_mode,
            tls_root_cert,
            tls_identity,
            flavor,
        } = self;

        PostgresConnection {
            host,
            port,
            database,
            user,
            password,
            tunnel: tunnel.into_inline_connection(r),
            tls_mode,
            tls_root_cert,
            tls_identity,
            flavor,
        }
    }
}

impl<C: ConnectionAccess> PostgresConnection<C> {
    fn validate_by_default(&self) -> bool {
        true
    }
}

impl PostgresConnection<InlinedConnection> {
    pub async fn config(
        &self,
        secrets_reader: &Arc<dyn mz_secrets::SecretsReader>,
        storage_configuration: &StorageConfiguration,
        in_task: InTask,
    ) -> Result<mz_postgres_util::Config, anyhow::Error> {
        let params = &storage_configuration.parameters;

        let mut config = tokio_postgres::Config::new();
        config
            .host(&self.host)
            .port(self.port)
            .dbname(&self.database)
            .user(&self.user.get_string(in_task, secrets_reader).await?)
            .ssl_mode(self.tls_mode);
        if let Some(password) = self.password {
            let password = secrets_reader
                .read_string_in_task_if(in_task, password)
                .await?;
            config.password(password);
        }
        if let Some(tls_root_cert) = &self.tls_root_cert {
            let tls_root_cert = tls_root_cert.get_string(in_task, secrets_reader).await?;
            config.ssl_root_cert(tls_root_cert.as_bytes());
        }
        if let Some(tls_identity) = &self.tls_identity {
            let cert = tls_identity
                .cert
                .get_string(in_task, secrets_reader)
                .await?;
            let key = secrets_reader
                .read_string_in_task_if(in_task, tls_identity.key)
                .await?;
            config.ssl_cert(cert.as_bytes()).ssl_key(key.as_bytes());
        }

        if let Some(connect_timeout) = params.pg_source_connect_timeout {
            config.connect_timeout(connect_timeout);
        }
        if let Some(keepalives_retries) = params.pg_source_tcp_keepalives_retries {
            config.keepalives_retries(keepalives_retries);
        }
        if let Some(keepalives_idle) = params.pg_source_tcp_keepalives_idle {
            config.keepalives_idle(keepalives_idle);
        }
        if let Some(keepalives_interval) = params.pg_source_tcp_keepalives_interval {
            config.keepalives_interval(keepalives_interval);
        }
        if let Some(tcp_user_timeout) = params.pg_source_tcp_user_timeout {
            config.tcp_user_timeout(tcp_user_timeout);
        }

        let mut options = vec![];
        if let Some(wal_sender_timeout) = params.pg_source_wal_sender_timeout {
            options.push(format!(
                "--wal_sender_timeout={}",
                wal_sender_timeout.as_millis()
            ));
        };
        if params.pg_source_tcp_configure_server {
            if let Some(keepalives_retries) = params.pg_source_tcp_keepalives_retries {
                options.push(format!("--tcp_keepalives_count={}", keepalives_retries));
            }
            if let Some(keepalives_idle) = params.pg_source_tcp_keepalives_idle {
                options.push(format!(
                    "--tcp_keepalives_idle={}",
                    keepalives_idle.as_secs()
                ));
            }
            if let Some(keepalives_interval) = params.pg_source_tcp_keepalives_interval {
                options.push(format!(
                    "--tcp_keepalives_interval={}",
                    keepalives_interval.as_secs()
                ));
            }
            if let Some(tcp_user_timeout) = params.pg_source_tcp_user_timeout {
                options.push(format!(
                    "--tcp_user_timeout={}",
                    tcp_user_timeout.as_millis()
                ));
            }
        }
        config.options(options.join(" ").as_str());

        let tunnel = match &self.tunnel {
            Tunnel::Direct => {
                // Ensure any host we connect to is resolved to an external address.
                let resolved = resolve_address(
                    &self.host,
                    ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
                )
                .await?;
                mz_postgres_util::TunnelConfig::Direct {
                    resolved_ips: Some(resolved),
                }
            }
            Tunnel::Ssh(SshTunnel {
                connection_id,
                connection,
            }) => {
                let secret = secrets_reader
                    .read_in_task_if(in_task, *connection_id)
                    .await?;
                let key_pair = SshKeyPair::from_bytes(&secret)?;
                // Ensure any ssh-bastion host we connect to is resolved to an external address.
                let resolved = resolve_address(
                    &connection.host,
                    ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
                )
                .await?;
                mz_postgres_util::TunnelConfig::Ssh {
                    config: SshTunnelConfig {
                        host: resolved
                            .iter()
                            .map(|a| a.to_string())
                            .collect::<BTreeSet<_>>(),
                        port: connection.port,
                        user: connection.user.clone(),
                        key_pair,
                    },
                }
            }
            Tunnel::AwsPrivatelink(connection) => {
                assert_none!(connection.port);
                mz_postgres_util::TunnelConfig::AwsPrivatelink {
                    connection_id: connection.connection_id,
                }
            }
        };

        Ok(mz_postgres_util::Config::new(
            config,
            tunnel,
            params.ssh_timeout_config,
            in_task,
        )?)
    }

    async fn validate(
        &self,
        _id: CatalogItemId,
        storage_configuration: &StorageConfiguration,
    ) -> Result<(), anyhow::Error> {
        let config = self
            .config(
                &storage_configuration.connection_context.secrets_reader,
                storage_configuration,
                // We are in a normal tokio context during validation, already.
                InTask::No,
            )
            .await?;
        let client = config
            .connect(
                "connection validation",
                &storage_configuration.connection_context.ssh_tunnel_manager,
            )
            .await?;
        use PostgresFlavor::*;
        match (client.server_flavor(), &self.flavor) {
            (Vanilla, Yugabyte) => bail!("Expected to find PostgreSQL server, found Yugabyte."),
            (Yugabyte, Vanilla) => bail!("Expected to find Yugabyte server, found PostgreSQL."),
            (Vanilla, Vanilla) | (Yugabyte, Yugabyte) => {}
        }
        Ok(())
    }
}

impl<C: ConnectionAccess> AlterCompatible for PostgresConnection<C> {
    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        let PostgresConnection {
            tunnel,
            flavor,
            // All non-tunnel options may change arbitrarily
            host: _,
            port: _,
            database: _,
            user: _,
            password: _,
            tls_mode: _,
            tls_root_cert: _,
            tls_identity: _,
        } = self;

        let compatibility_checks = [
            (tunnel.alter_compatible(id, &other.tunnel).is_ok(), "tunnel"),
            (flavor == &other.flavor, "flavor"),
        ];

        for (compatible, field) in compatibility_checks {
            if !compatible {
                tracing::warn!(
                    "PostgresConnection incompatible at {field}:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );

                return Err(AlterError { id });
            }
        }
        Ok(())
    }
}

impl RustType<ProtoPostgresConnection> for PostgresConnection {
    fn into_proto(&self) -> ProtoPostgresConnection {
        ProtoPostgresConnection {
            host: self.host.into_proto(),
            port: self.port.into_proto(),
            database: self.database.into_proto(),
            user: Some(self.user.into_proto()),
            password: self.password.into_proto(),
            tls_mode: Some(self.tls_mode.into_proto()),
            tls_root_cert: self.tls_root_cert.into_proto(),
            tls_identity: self.tls_identity.into_proto(),
            tunnel: Some(self.tunnel.into_proto()),
            flavor: Some(self.flavor.into_proto()),
        }
    }

    fn from_proto(proto: ProtoPostgresConnection) -> Result<Self, TryFromProtoError> {
        Ok(PostgresConnection {
            host: proto.host,
            port: proto.port.into_rust()?,
            database: proto.database,
            user: proto
                .user
                .into_rust_if_some("ProtoPostgresConnection::user")?,
            password: proto.password.into_rust()?,
            tunnel: proto
                .tunnel
                .into_rust_if_some("ProtoPostgresConnection::tunnel")?,
            tls_mode: proto
                .tls_mode
                .into_rust_if_some("ProtoPostgresConnection::tls_mode")?,
            tls_root_cert: proto.tls_root_cert.into_rust()?,
            tls_identity: proto.tls_identity.into_rust()?,
            flavor: proto
                .flavor
                .into_rust_if_some("ProtoPostgresConnection::flavor")?,
        })
    }
}

/// Specifies how to tunnel a connection.
#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub enum Tunnel<C: ConnectionAccess = InlinedConnection> {
    /// No tunneling.
    Direct,
    /// Via the specified SSH tunnel connection.
    Ssh(SshTunnel<C>),
    /// Via the specified AWS PrivateLink connection.
    AwsPrivatelink(AwsPrivatelink),
}

impl<R: ConnectionResolver> IntoInlineConnection<Tunnel, R> for Tunnel<ReferencedConnection> {
    fn into_inline_connection(self, r: R) -> Tunnel {
        match self {
            Tunnel::Direct => Tunnel::Direct,
            Tunnel::Ssh(ssh) => Tunnel::Ssh(ssh.into_inline_connection(r)),
            Tunnel::AwsPrivatelink(awspl) => Tunnel::AwsPrivatelink(awspl),
        }
    }
}

impl RustType<ProtoTunnel> for Tunnel<InlinedConnection> {
    fn into_proto(&self) -> ProtoTunnel {
        use proto_tunnel::Tunnel as ProtoTunnelField;
        ProtoTunnel {
            tunnel: Some(match &self {
                Tunnel::Direct => ProtoTunnelField::Direct(()),
                Tunnel::Ssh(ssh) => ProtoTunnelField::Ssh(ssh.into_proto()),
                Tunnel::AwsPrivatelink(aws) => ProtoTunnelField::AwsPrivatelink(aws.into_proto()),
            }),
        }
    }

    fn from_proto(proto: ProtoTunnel) -> Result<Self, TryFromProtoError> {
        use proto_tunnel::Tunnel as ProtoTunnelField;
        Ok(match proto.tunnel {
            None => return Err(TryFromProtoError::missing_field("ProtoTunnel::tunnel")),
            Some(ProtoTunnelField::Direct(())) => Tunnel::Direct,
            Some(ProtoTunnelField::Ssh(ssh)) => Tunnel::Ssh(ssh.into_rust()?),
            Some(ProtoTunnelField::AwsPrivatelink(aws)) => Tunnel::AwsPrivatelink(aws.into_rust()?),
        })
    }
}

impl<C: ConnectionAccess> AlterCompatible for Tunnel<C> {
    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        let compatible = match (self, other) {
            (Self::Ssh(s), Self::Ssh(o)) => s.alter_compatible(id, o).is_ok(),
            (s, o) => s == o,
        };

        if !compatible {
            tracing::warn!(
                "Tunnel incompatible:\nself:\n{:#?}\n\nother\n{:#?}",
                self,
                other
            );

            return Err(AlterError { id });
        }

        Ok(())
    }
}

/// Specifies which MySQL SSL Mode to use:
/// <https://dev.mysql.com/doc/refman/8.0/en/connection-options.html#option_general_ssl-mode>
/// This is not available as an enum in the mysql-async crate, so we define our own.
#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub enum MySqlSslMode {
    Disabled,
    Required,
    VerifyCa,
    VerifyIdentity,
}

impl RustType<i32> for MySqlSslMode {
    fn into_proto(&self) -> i32 {
        match self {
            MySqlSslMode::Disabled => ProtoMySqlSslMode::Disabled.into(),
            MySqlSslMode::Required => ProtoMySqlSslMode::Required.into(),
            MySqlSslMode::VerifyCa => ProtoMySqlSslMode::VerifyCa.into(),
            MySqlSslMode::VerifyIdentity => ProtoMySqlSslMode::VerifyIdentity.into(),
        }
    }

    fn from_proto(proto: i32) -> Result<Self, TryFromProtoError> {
        Ok(match ProtoMySqlSslMode::try_from(proto) {
            Ok(ProtoMySqlSslMode::Disabled) => MySqlSslMode::Disabled,
            Ok(ProtoMySqlSslMode::Required) => MySqlSslMode::Required,
            Ok(ProtoMySqlSslMode::VerifyCa) => MySqlSslMode::VerifyCa,
            Ok(ProtoMySqlSslMode::VerifyIdentity) => MySqlSslMode::VerifyIdentity,
            Err(_) => {
                return Err(TryFromProtoError::UnknownEnumVariant(
                    "tls_mode".to_string(),
                ))
            }
        })
    }
}

pub fn any_mysql_ssl_mode() -> impl Strategy<Value = MySqlSslMode> {
    proptest::sample::select(vec![
        MySqlSslMode::Disabled,
        MySqlSslMode::Required,
        MySqlSslMode::VerifyCa,
        MySqlSslMode::VerifyIdentity,
    ])
}

/// A connection to a MySQL server.
#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, Arbitrary)]
pub struct MySqlConnection<C: ConnectionAccess = InlinedConnection> {
    /// The hostname of the server.
    pub host: String,
    /// The port of the server.
    pub port: u16,
    /// The username to authenticate as.
    pub user: StringOrSecret,
    /// An optional password for authentication.
    pub password: Option<CatalogItemId>,
    /// A tunnel through which to route traffic.
    pub tunnel: Tunnel<C>,
    /// Whether to use TLS for encryption, verify the server's certificate, and identity.
    #[proptest(strategy = "any_mysql_ssl_mode()")]
    pub tls_mode: MySqlSslMode,
    /// An optional root TLS certificate in PEM format, to verify the server's
    /// identity.
    pub tls_root_cert: Option<StringOrSecret>,
    /// An optional TLS client certificate for authentication.
    pub tls_identity: Option<TlsIdentity>,
}

impl<R: ConnectionResolver> IntoInlineConnection<MySqlConnection, R>
    for MySqlConnection<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> MySqlConnection {
        let MySqlConnection {
            host,
            port,
            user,
            password,
            tunnel,
            tls_mode,
            tls_root_cert,
            tls_identity,
        } = self;

        MySqlConnection {
            host,
            port,
            user,
            password,
            tunnel: tunnel.into_inline_connection(r),
            tls_mode,
            tls_root_cert,
            tls_identity,
        }
    }
}

impl<C: ConnectionAccess> MySqlConnection<C> {
    fn validate_by_default(&self) -> bool {
        true
    }
}

impl MySqlConnection<InlinedConnection> {
    pub async fn config(
        &self,
        secrets_reader: &Arc<dyn mz_secrets::SecretsReader>,
        storage_configuration: &StorageConfiguration,
        in_task: InTask,
    ) -> Result<mz_mysql_util::Config, anyhow::Error> {
        // TODO(roshan): Set appropriate connection timeouts
        let mut opts = mysql_async::OptsBuilder::default()
            .ip_or_hostname(&self.host)
            .tcp_port(self.port)
            .user(Some(&self.user.get_string(in_task, secrets_reader).await?));

        if let Some(password) = self.password {
            let password = secrets_reader
                .read_string_in_task_if(in_task, password)
                .await?;
            opts = opts.pass(Some(password));
        }

        // Our `MySqlSslMode` enum matches the official MySQL Client `--ssl-mode` parameter values
        // which uses opt-in security features (SSL, CA verification, & Identity verification).
        // The mysql_async crate `SslOpts` struct uses an opt-out mechanism for each of these, so
        // we need to appropriately disable features to match the intent of each enum value.
        let mut ssl_opts = match self.tls_mode {
            MySqlSslMode::Disabled => None,
            MySqlSslMode::Required => Some(
                mysql_async::SslOpts::default()
                    .with_danger_accept_invalid_certs(true)
                    .with_danger_skip_domain_validation(true),
            ),
            MySqlSslMode::VerifyCa => {
                Some(mysql_async::SslOpts::default().with_danger_skip_domain_validation(true))
            }
            MySqlSslMode::VerifyIdentity => Some(mysql_async::SslOpts::default()),
        };

        if matches!(
            self.tls_mode,
            MySqlSslMode::VerifyCa | MySqlSslMode::VerifyIdentity
        ) {
            if let Some(tls_root_cert) = &self.tls_root_cert {
                let tls_root_cert = tls_root_cert.get_string(in_task, secrets_reader).await?;
                ssl_opts = ssl_opts.map(|opts| {
                    opts.with_root_certs(vec![tls_root_cert.as_bytes().to_vec().into()])
                });
            }
        }

        if let Some(identity) = &self.tls_identity {
            let key = secrets_reader
                .read_string_in_task_if(in_task, identity.key)
                .await?;
            let cert = identity.cert.get_string(in_task, secrets_reader).await?;
            let Pkcs12Archive { der, pass } =
                mz_tls_util::pkcs12der_from_pem(key.as_bytes(), cert.as_bytes())?;

            // Add client identity to SSLOpts
            ssl_opts = ssl_opts.map(|opts| {
                opts.with_client_identity(Some(
                    mysql_async::ClientIdentity::new(der.into()).with_password(pass),
                ))
            });
        }

        opts = opts.ssl_opts(ssl_opts);

        let tunnel = match &self.tunnel {
            Tunnel::Direct => {
                // Ensure any host we connect to is resolved to an external address.
                let resolved = resolve_address(
                    &self.host,
                    ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
                )
                .await?;
                mz_mysql_util::TunnelConfig::Direct {
                    resolved_ips: Some(resolved),
                }
            }
            Tunnel::Ssh(SshTunnel {
                connection_id,
                connection,
            }) => {
                let secret = secrets_reader
                    .read_in_task_if(in_task, *connection_id)
                    .await?;
                let key_pair = SshKeyPair::from_bytes(&secret)?;
                // Ensure any ssh-bastion host we connect to is resolved to an external address.
                let resolved = resolve_address(
                    &connection.host,
                    ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
                )
                .await?;
                mz_mysql_util::TunnelConfig::Ssh {
                    config: SshTunnelConfig {
                        host: resolved
                            .iter()
                            .map(|a| a.to_string())
                            .collect::<BTreeSet<_>>(),
                        port: connection.port,
                        user: connection.user.clone(),
                        key_pair,
                    },
                }
            }
            Tunnel::AwsPrivatelink(connection) => {
                assert_none!(connection.port);
                mz_mysql_util::TunnelConfig::AwsPrivatelink {
                    connection_id: connection.connection_id,
                }
            }
        };

        opts = storage_configuration
            .parameters
            .mysql_source_timeouts
            .apply_to_opts(opts)?;

        Ok(mz_mysql_util::Config::new(
            opts.into(),
            tunnel,
            storage_configuration.parameters.ssh_timeout_config,
            in_task,
        ))
    }

    async fn validate(
        &self,
        _id: CatalogItemId,
        storage_configuration: &StorageConfiguration,
    ) -> Result<(), anyhow::Error> {
        let config = self
            .config(
                &storage_configuration.connection_context.secrets_reader,
                storage_configuration,
                // We are in a normal tokio context during validation, already.
                InTask::No,
            )
            .await?;
        let conn = config
            .connect(
                "connection validation",
                &storage_configuration.connection_context.ssh_tunnel_manager,
            )
            .await?;
        conn.disconnect().await?;
        Ok(())
    }
}

impl RustType<ProtoMySqlConnection> for MySqlConnection {
    fn into_proto(&self) -> ProtoMySqlConnection {
        ProtoMySqlConnection {
            host: self.host.into_proto(),
            port: self.port.into_proto(),
            user: Some(self.user.into_proto()),
            password: self.password.into_proto(),
            tls_mode: self.tls_mode.into_proto(),
            tls_root_cert: self.tls_root_cert.into_proto(),
            tls_identity: self.tls_identity.into_proto(),
            tunnel: Some(self.tunnel.into_proto()),
        }
    }

    fn from_proto(proto: ProtoMySqlConnection) -> Result<Self, TryFromProtoError> {
        Ok(MySqlConnection {
            host: proto.host,
            port: proto.port.into_rust()?,
            user: proto.user.into_rust_if_some("ProtoMySqlConnection::user")?,
            password: proto.password.into_rust()?,
            tunnel: proto
                .tunnel
                .into_rust_if_some("ProtoMySqlConnection::tunnel")?,
            tls_mode: proto.tls_mode.into_rust()?,
            tls_root_cert: proto.tls_root_cert.into_rust()?,
            tls_identity: proto.tls_identity.into_rust()?,
        })
    }
}

impl<C: ConnectionAccess> AlterCompatible for MySqlConnection<C> {
    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        let MySqlConnection {
            tunnel,
            // All non-tunnel options may change arbitrarily
            host: _,
            port: _,
            user: _,
            password: _,
            tls_mode: _,
            tls_root_cert: _,
            tls_identity: _,
        } = self;

        let compatibility_checks = [(tunnel.alter_compatible(id, &other.tunnel).is_ok(), "tunnel")];

        for (compatible, field) in compatibility_checks {
            if !compatible {
                tracing::warn!(
                    "MySqlConnection incompatible at {field}:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );

                return Err(AlterError { id });
            }
        }
        Ok(())
    }
}

/// A connection to an SSH tunnel.
#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct SshConnection {
    pub host: String,
    pub port: u16,
    pub user: String,
}

use self::inline::{
    ConnectionAccess, ConnectionResolver, InlinedConnection, IntoInlineConnection,
    ReferencedConnection,
};

impl RustType<ProtoSshConnection> for SshConnection {
    fn into_proto(&self) -> ProtoSshConnection {
        ProtoSshConnection {
            host: self.host.into_proto(),
            port: self.port.into_proto(),
            user: self.user.into_proto(),
        }
    }

    fn from_proto(proto: ProtoSshConnection) -> Result<Self, TryFromProtoError> {
        Ok(SshConnection {
            host: proto.host,
            port: proto.port.into_rust()?,
            user: proto.user,
        })
    }
}

impl AlterCompatible for SshConnection {
    fn alter_compatible(&self, _id: GlobalId, _other: &Self) -> Result<(), AlterError> {
        // Every element of the SSH connection is configurable.
        Ok(())
    }
}

/// Specifies an AWS PrivateLink service for a [`Tunnel`].
#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct AwsPrivatelink {
    /// The ID of the connection to the AWS PrivateLink service.
    pub connection_id: CatalogItemId,
    // The availability zone to use when connecting to the AWS PrivateLink service.
    pub availability_zone: Option<String>,
    /// The port to use when connecting to the AWS PrivateLink service, if
    /// different from the port in [`KafkaBroker::address`].
    pub port: Option<u16>,
}

impl RustType<ProtoAwsPrivatelink> for AwsPrivatelink {
    fn into_proto(&self) -> ProtoAwsPrivatelink {
        ProtoAwsPrivatelink {
            connection_id: Some(self.connection_id.into_proto()),
            availability_zone: self.availability_zone.into_proto(),
            port: self.port.into_proto(),
        }
    }

    fn from_proto(proto: ProtoAwsPrivatelink) -> Result<Self, TryFromProtoError> {
        Ok(AwsPrivatelink {
            connection_id: proto
                .connection_id
                .into_rust_if_some("ProtoAwsPrivatelink::connection_id")?,
            availability_zone: proto.availability_zone.into_rust()?,
            port: proto.port.into_rust()?,
        })
    }
}

impl AlterCompatible for AwsPrivatelink {
    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        let AwsPrivatelink {
            connection_id,
            availability_zone: _,
            port: _,
        } = self;

        let compatibility_checks = [(connection_id == &other.connection_id, "connection_id")];

        for (compatible, field) in compatibility_checks {
            if !compatible {
                tracing::warn!(
                    "AwsPrivatelink incompatible at {field}:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );

                return Err(AlterError { id });
            }
        }

        Ok(())
    }
}

/// Specifies an SSH tunnel connection.
#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct SshTunnel<C: ConnectionAccess = InlinedConnection> {
    /// id of the ssh connection
    pub connection_id: CatalogItemId,
    /// ssh connection object
    pub connection: C::Ssh,
}

impl<R: ConnectionResolver> IntoInlineConnection<SshTunnel, R> for SshTunnel<ReferencedConnection> {
    fn into_inline_connection(self, r: R) -> SshTunnel {
        let SshTunnel {
            connection,
            connection_id,
        } = self;

        SshTunnel {
            connection: r.resolve_connection(connection).unwrap_ssh(),
            connection_id,
        }
    }
}

impl RustType<ProtoSshTunnel> for SshTunnel<InlinedConnection> {
    fn into_proto(&self) -> ProtoSshTunnel {
        ProtoSshTunnel {
            connection_id: Some(self.connection_id.into_proto()),
            connection: Some(self.connection.into_proto()),
        }
    }

    fn from_proto(proto: ProtoSshTunnel) -> Result<Self, TryFromProtoError> {
        Ok(SshTunnel {
            connection_id: proto
                .connection_id
                .into_rust_if_some("ProtoSshTunnel::connection_id")?,
            connection: proto
                .connection
                .into_rust_if_some("ProtoSshTunnel::connection")?,
        })
    }
}

impl SshTunnel<InlinedConnection> {
    /// Like [`SshTunnelConfig::connect`], but the SSH key is loaded from a
    /// secret.
    async fn connect(
        &self,
        storage_configuration: &StorageConfiguration,
        remote_host: &str,
        remote_port: u16,
        in_task: InTask,
    ) -> Result<ManagedSshTunnelHandle, anyhow::Error> {
        // Ensure any ssh-bastion host we connect to is resolved to an external address.
        let resolved = resolve_address(
            &self.connection.host,
            ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
        )
        .await?;
        storage_configuration
            .connection_context
            .ssh_tunnel_manager
            .connect(
                SshTunnelConfig {
                    host: resolved
                        .iter()
                        .map(|a| a.to_string())
                        .collect::<BTreeSet<_>>(),
                    port: self.connection.port,
                    user: self.connection.user.clone(),
                    key_pair: SshKeyPair::from_bytes(
                        &storage_configuration
                            .connection_context
                            .secrets_reader
                            .read_in_task_if(in_task, self.connection_id)
                            .await?,
                    )?,
                },
                remote_host,
                remote_port,
                storage_configuration.parameters.ssh_timeout_config,
                in_task,
            )
            .await
    }
}

impl<C: ConnectionAccess> AlterCompatible for SshTunnel<C> {
    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        let SshTunnel {
            connection_id,
            connection,
        } = self;

        let compatibility_checks = [
            (connection_id == &other.connection_id, "connection_id"),
            (
                connection.alter_compatible(id, &other.connection).is_ok(),
                "connection",
            ),
        ];

        for (compatible, field) in compatibility_checks {
            if !compatible {
                tracing::warn!(
                    "SshTunnel incompatible at {field}:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );

                return Err(AlterError { id });
            }
        }

        Ok(())
    }
}

impl SshConnection {
    #[allow(clippy::unused_async)]
    async fn validate(
        &self,
        id: CatalogItemId,
        storage_configuration: &StorageConfiguration,
    ) -> Result<(), anyhow::Error> {
        let secret = storage_configuration
            .connection_context
            .secrets_reader
            .read_in_task_if(
                // We are in a normal tokio context during validation, already.
                InTask::No,
                id,
            )
            .await?;
        let key_pair = SshKeyPair::from_bytes(&secret)?;

        // Ensure any ssh-bastion host we connect to is resolved to an external address.
        let resolved = resolve_address(
            &self.host,
            ENFORCE_EXTERNAL_ADDRESSES.get(storage_configuration.config_set()),
        )
        .await?;

        let config = SshTunnelConfig {
            host: resolved
                .iter()
                .map(|a| a.to_string())
                .collect::<BTreeSet<_>>(),
            port: self.port,
            user: self.user.clone(),
            key_pair,
        };
        // Note that we do NOT use the `SshTunnelManager` here, as we want to validate that we
        // can actually create a new connection to the ssh bastion, without tunneling.
        config
            .validate(storage_configuration.parameters.ssh_timeout_config)
            .await
    }

    fn validate_by_default(&self) -> bool {
        false
    }
}

impl AwsPrivatelinkConnection {
    #[allow(clippy::unused_async)]
    async fn validate(
        &self,
        id: CatalogItemId,
        storage_configuration: &StorageConfiguration,
    ) -> Result<(), anyhow::Error> {
        let Some(ref cloud_resource_reader) = storage_configuration
            .connection_context
            .cloud_resource_reader
        else {
            return Err(anyhow!("AWS PrivateLink connections are unsupported"));
        };

        // No need to optionally run this in a task, as we are just validating from envd.
        let status = cloud_resource_reader.read(id).await?;

        let availability = status
            .conditions
            .as_ref()
            .and_then(|conditions| conditions.iter().find(|c| c.type_ == "Available"));

        match availability {
            Some(condition) if condition.status == "True" => Ok(()),
            Some(condition) => Err(anyhow!("{}", condition.message)),
            None => Err(anyhow!("Endpoint availability is unknown")),
        }
    }

    fn validate_by_default(&self) -> bool {
        false
    }
}