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

//! Data definition language (DDL) utilities for CONNECTION objects.

use std::collections::{BTreeMap, BTreeSet};

use array_concat::concat_arrays;
use itertools::Itertools;
use maplit::btreemap;
use mz_ore::num::NonNeg;
use mz_ore::str::StrExt;
use mz_postgres_util::tunnel::PostgresFlavor;
use mz_repr::GlobalId;
use mz_sql_parser::ast::display::AstDisplay;
use mz_sql_parser::ast::ConnectionOptionName::*;
use mz_sql_parser::ast::{
    ConnectionDefaultAwsPrivatelink, ConnectionOption, ConnectionOptionName, CreateConnectionType,
    KafkaBroker, KafkaBrokerAwsPrivatelinkOption, KafkaBrokerAwsPrivatelinkOptionName,
    KafkaBrokerTunnel,
};
use mz_storage_types::connections::aws::{
    AwsAssumeRole, AwsAuth, AwsConnection, AwsConnectionReference, AwsCredentials,
};
use mz_storage_types::connections::inline::ReferencedConnection;
use mz_storage_types::connections::string_or_secret::StringOrSecret;
use mz_storage_types::connections::{
    AwsPrivatelink, AwsPrivatelinkConnection, CsrConnection, CsrConnectionHttpAuth,
    KafkaConnection, KafkaSaslConfig, KafkaTlsConfig, KafkaTopicOptions, MySqlConnection,
    MySqlSslMode, PostgresConnection, SshConnection, SshTunnel, TlsIdentity, Tunnel,
};

use crate::names::Aug;
use crate::plan::statement::{Connection, ResolvedItemName};
use crate::plan::with_options::{self, TryFromValue};
use crate::plan::{PlanError, StatementContext};
use crate::session::vars::{self, ENABLE_AWS_MSK_IAM_AUTH};

generate_extracted_config!(
    ConnectionOption,
    (AccessKeyId, StringOrSecret),
    (AssumeRoleArn, String),
    (AssumeRoleSessionName, String),
    (AvailabilityZones, Vec<String>),
    (AwsConnection, with_options::Object),
    (AwsPrivatelink, ConnectionDefaultAwsPrivatelink<Aug>),
    // (AwsPrivatelink, with_options::Object),
    (Broker, Vec<KafkaBroker<Aug>>),
    (Brokers, Vec<KafkaBroker<Aug>>),
    (Database, String),
    (Endpoint, String),
    (Host, String),
    (Password, with_options::Secret),
    (Port, u16),
    (ProgressTopic, String),
    (ProgressTopicReplicationFactor, i32),
    (Region, String),
    (SaslMechanisms, String),
    (SaslPassword, with_options::Secret),
    (SaslUsername, StringOrSecret),
    (SecretAccessKey, with_options::Secret),
    (SecurityProtocol, String),
    (ServiceName, String),
    (SshTunnel, with_options::Object),
    (SslCertificate, StringOrSecret),
    (SslCertificateAuthority, StringOrSecret),
    (SslKey, with_options::Secret),
    (SslMode, String),
    (SessionToken, StringOrSecret),
    (Url, String),
    (User, StringOrSecret)
);

generate_extracted_config!(
    KafkaBrokerAwsPrivatelinkOption,
    (AvailabilityZone, String),
    (Port, u16)
);

/// Options which cannot be changed using ALTER CONNECTION.
pub(crate) const INALTERABLE_OPTIONS: &[ConnectionOptionName] =
    &[ProgressTopic, ProgressTopicReplicationFactor];

/// Options of which only one may be specified.
pub(crate) const MUTUALLY_EXCLUSIVE_SETS: &[&[ConnectionOptionName]] = &[&[Broker, Brokers]];

pub(super) fn validate_options_per_connection_type(
    t: CreateConnectionType,
    mut options: BTreeSet<ConnectionOptionName>,
) -> Result<(), PlanError> {
    use mz_sql_parser::ast::ConnectionOptionName::*;
    let permitted_options = match t {
        CreateConnectionType::Aws => [
            AccessKeyId,
            SecretAccessKey,
            SessionToken,
            Endpoint,
            Region,
            AssumeRoleArn,
            AssumeRoleSessionName,
        ]
        .as_slice(),
        CreateConnectionType::AwsPrivatelink => &[AvailabilityZones, Port, ServiceName],
        CreateConnectionType::Csr => &[
            AwsPrivatelink,
            Password,
            Port,
            SshTunnel,
            SslCertificate,
            SslCertificateAuthority,
            SslKey,
            Url,
            User,
        ],
        CreateConnectionType::Kafka => &[
            AwsConnection,
            Broker,
            Brokers,
            ProgressTopic,
            ProgressTopicReplicationFactor,
            AwsPrivatelink,
            SshTunnel,
            SslKey,
            SslCertificate,
            SslCertificateAuthority,
            SaslMechanisms,
            SaslUsername,
            SaslPassword,
            SecurityProtocol,
        ],
        CreateConnectionType::Postgres | CreateConnectionType::Yugabyte => &[
            AwsPrivatelink,
            Database,
            Host,
            Password,
            Port,
            SshTunnel,
            SslCertificate,
            SslCertificateAuthority,
            SslKey,
            SslMode,
            User,
        ],
        CreateConnectionType::Ssh => &[Host, Port, User],
        CreateConnectionType::MySql => &[
            AwsPrivatelink,
            Host,
            Password,
            Port,
            SshTunnel,
            SslCertificate,
            SslCertificateAuthority,
            SslKey,
            SslMode,
            User,
        ],
    };

    for o in permitted_options {
        options.remove(o);
    }

    if !options.is_empty() {
        sql_bail!(
            "{} connections do not support {} values",
            t,
            options.iter().join(", ")
        )
    }

    Ok(())
}

impl ConnectionOptionExtracted {
    pub(super) fn ensure_only_valid_options(
        &self,
        t: CreateConnectionType,
    ) -> Result<(), PlanError> {
        validate_options_per_connection_type(t, self.seen.clone())
    }

    pub fn try_into_connection(
        self,
        scx: &StatementContext,
        connection_type: CreateConnectionType,
    ) -> Result<Connection<ReferencedConnection>, PlanError> {
        self.ensure_only_valid_options(connection_type)?;

        let connection: Connection<ReferencedConnection> = match connection_type {
            CreateConnectionType::Aws => {
                let credentials = match (
                    self.access_key_id,
                    self.secret_access_key,
                    self.session_token,
                ) {
                    (Some(access_key_id), Some(secret_access_key), session_token) => {
                        Some(AwsCredentials {
                            access_key_id,
                            secret_access_key: secret_access_key.into(),
                            session_token,
                        })
                    }
                    (None, None, None) => None,
                    _ => {
                        sql_bail!("must specify both ACCESS KEY ID and SECRET ACCESS KEY with optional SESSION TOKEN");
                    }
                };

                let assume_role = match (self.assume_role_arn, self.assume_role_session_name) {
                    (Some(arn), session_name) => Some(AwsAssumeRole { arn, session_name }),
                    (None, Some(_)) => {
                        sql_bail!(
                            "must specify ASSUME ROLE ARN with optional ASSUME ROLE SESSION NAME"
                        );
                    }
                    _ => None,
                };

                let auth = match (credentials, assume_role) {
                    (None, None) => sql_bail!("must specify either ASSUME ROLE ARN or ACCESS KEY ID and SECRET ACCESS KEY"),
                    (Some(credentials), None) => AwsAuth::Credentials(credentials),
                    (None, Some(assume_role)) => AwsAuth::AssumeRole(assume_role),
                    (Some(_), Some(_)) => {
                        sql_bail!("cannot specify both ACCESS KEY ID and ASSUME ROLE ARN");
                    }
                };

                Connection::Aws(AwsConnection {
                    auth,
                    endpoint: match self.endpoint {
                        // TODO(benesch): this should not treat an empty endpoint as equivalent to a `NULL`
                        // endpoint, but making that change now would break testdrive. AWS connections are
                        // all behind feature flags mode right now, so no particular urgency to correct
                        // this.
                        Some(endpoint) if !endpoint.is_empty() => Some(endpoint),
                        _ => None,
                    },
                    region: self.region,
                })
            }
            CreateConnectionType::AwsPrivatelink => {
                let connection = AwsPrivatelinkConnection {
                    service_name: self
                        .service_name
                        .ok_or_else(|| sql_err!("SERVICE NAME option is required"))?,
                    availability_zones: self
                        .availability_zones
                        .ok_or_else(|| sql_err!("AVAILABILITY ZONES option is required"))?,
                };
                if let Some(supported_azs) = scx.catalog.aws_privatelink_availability_zones() {
                    let mut unique_azs: BTreeSet<String> = BTreeSet::new();
                    let mut duplicate_azs: BTreeSet<String> = BTreeSet::new();
                    // Validate each AZ is supported
                    for connection_az in &connection.availability_zones {
                        if unique_azs.contains(connection_az) {
                            duplicate_azs.insert(connection_az.to_string());
                        } else {
                            unique_azs.insert(connection_az.to_string());
                        }
                        if !supported_azs.contains(connection_az) {
                            return Err(PlanError::InvalidPrivatelinkAvailabilityZone {
                                name: connection_az.to_string(),
                                supported_azs,
                            });
                        }
                    }
                    if duplicate_azs.len() > 0 {
                        return Err(PlanError::DuplicatePrivatelinkAvailabilityZone {
                            duplicate_azs,
                        });
                    }
                }
                Connection::AwsPrivatelink(connection)
            }
            CreateConnectionType::Kafka => {
                let (tls, sasl) = plan_kafka_security(scx, &self)?;

                Connection::Kafka(KafkaConnection {
                    brokers: self.get_brokers(scx)?,
                    default_tunnel: scx
                        .build_tunnel_definition(self.ssh_tunnel, self.aws_privatelink)?,
                    progress_topic: self.progress_topic,
                    progress_topic_options: KafkaTopicOptions {
                        // We only allow configuring the progress topic replication factor for now.
                        // For correctness, the partition count MUST be one and for performance the compaction
                        // policy MUST be enabled.
                        partition_count: Some(NonNeg::try_from(1).expect("1 is positive")),
                        replication_factor: self.progress_topic_replication_factor.map(|val| {
                            if val <= 0 {
                                Err(sql_err!("invalid CONNECTION: PROGRESS TOPIC REPLICATION FACTOR must be greater than 0"))?
                            }
                            NonNeg::try_from(val).map_err(|e| sql_err!("{e}"))
                        }).transpose()?,
                        topic_config: btreemap! {
                            "cleanup.policy".to_string() => "compact".to_string(),
                        },
                    },
                    options: BTreeMap::new(),
                    tls,
                    sasl,
                })
            }
            CreateConnectionType::Csr => {
                let url: reqwest::Url = match self.url {
                    Some(url) => url
                        .parse()
                        .map_err(|e| sql_err!("parsing schema registry url: {e}"))?,
                    None => sql_bail!("invalid CONNECTION: must specify URL"),
                };
                let _ = url
                    .host_str()
                    .ok_or_else(|| sql_err!("invalid CONNECTION: URL must specify domain name"))?;
                if url.path() != "/" {
                    sql_bail!("invalid CONNECTION: URL must have an empty path");
                }
                let cert = self.ssl_certificate;
                let key = self.ssl_key.map(|secret| secret.into());
                let tls_identity = match (cert, key) {
                    (None, None) => None,
                    (Some(cert), Some(key)) => Some(TlsIdentity { cert, key }),
                    _ => sql_bail!(
                        "invalid CONNECTION: reading from SSL-auth Confluent Schema Registry requires both SSL KEY and SSL CERTIFICATE"
                    ),
                };
                let http_auth = self.user.map(|username| CsrConnectionHttpAuth {
                    username,
                    password: self.password.map(|secret| secret.into()),
                });

                // TODO we should move to self.port being unsupported if aws_privatelink is some, see <https://github.com/MaterializeInc/materialize/issues/24712#issuecomment-1925443977>
                if let Some(privatelink) = self.aws_privatelink.as_ref() {
                    if privatelink.port.is_some() {
                        sql_bail!("invalid CONNECTION: PORT in AWS PRIVATELINK is only supported for kafka")
                    }
                }
                let tunnel = scx.build_tunnel_definition(self.ssh_tunnel, self.aws_privatelink)?;

                Connection::Csr(CsrConnection {
                    url,
                    tls_root_cert: self.ssl_certificate_authority,
                    tls_identity,
                    http_auth,
                    tunnel,
                })
            }
            CreateConnectionType::Postgres | CreateConnectionType::Yugabyte => {
                if matches!(connection_type, CreateConnectionType::Yugabyte) {
                    scx.require_feature_flag(&vars::ENABLE_YUGABYTE_CONNECTION)?;
                }
                let cert = self.ssl_certificate;
                let key = self.ssl_key.map(|secret| secret.into());
                let tls_identity = match (cert, key) {
                    (None, None) => None,
                    (Some(cert), Some(key)) => Some(TlsIdentity { cert, key }),
                    _ => sql_bail!(
                        "invalid CONNECTION: both SSL KEY and SSL CERTIFICATE are required"
                    ),
                };
                let tls_mode = match self.ssl_mode.as_ref().map(|m| m.as_str()) {
                    None | Some("disable") => tokio_postgres::config::SslMode::Disable,
                    // "prefer" intentionally omitted because it has dubious security
                    // properties.
                    Some("require") | Some("required") => tokio_postgres::config::SslMode::Require,
                    Some("verify_ca") | Some("verify-ca") => {
                        tokio_postgres::config::SslMode::VerifyCa
                    }
                    Some("verify_full") | Some("verify-full") => {
                        tokio_postgres::config::SslMode::VerifyFull
                    }
                    Some(m) => sql_bail!("invalid CONNECTION: unknown SSL MODE {}", m.quoted()),
                };

                // TODO we should move to self.port being unsupported if aws_privatelink is some, see <https://github.com/MaterializeInc/materialize/issues/24712#issuecomment-1925443977>
                if let Some(privatelink) = self.aws_privatelink.as_ref() {
                    if privatelink.port.is_some() {
                        sql_bail!("invalid CONNECTION: PORT in AWS PRIVATELINK is only supported for kafka")
                    }
                }
                let tunnel = scx.build_tunnel_definition(self.ssh_tunnel, self.aws_privatelink)?;

                Connection::Postgres(PostgresConnection {
                    database: self
                        .database
                        .ok_or_else(|| sql_err!("DATABASE option is required"))?,
                    password: self.password.map(|password| password.into()),
                    host: self
                        .host
                        .ok_or_else(|| sql_err!("HOST option is required"))?,
                    port: self.port.unwrap_or(5432_u16),
                    tunnel,
                    tls_mode,
                    tls_root_cert: self.ssl_certificate_authority,
                    tls_identity,
                    user: self
                        .user
                        .ok_or_else(|| sql_err!("USER option is required"))?,
                    flavor: match connection_type {
                        CreateConnectionType::Postgres => PostgresFlavor::Vanilla,
                        CreateConnectionType::Yugabyte => PostgresFlavor::Yugabyte,
                        _ => unreachable!(),
                    },
                })
            }
            CreateConnectionType::Ssh => Connection::Ssh(SshConnection {
                host: self
                    .host
                    .ok_or_else(|| sql_err!("HOST option is required"))?,
                port: self.port.unwrap_or(22_u16),
                user: match self
                    .user
                    .ok_or_else(|| sql_err!("USER option is required"))?
                {
                    StringOrSecret::String(user) => user,
                    StringOrSecret::Secret(_) => {
                        sql_bail!("SSH connections do not support supplying USER value as SECRET")
                    }
                },
            }),
            CreateConnectionType::MySql => {
                let cert = self.ssl_certificate;
                let key = self.ssl_key.map(|secret| secret.into());
                let tls_identity = match (cert, key) {
                    (None, None) => None,
                    (Some(cert), Some(key)) => Some(TlsIdentity { cert, key }),
                    _ => sql_bail!(
                        "invalid CONNECTION: both SSL KEY and SSL CERTIFICATE are required"
                    ),
                };
                // Accepts the same SSL Mode values as the MySQL Client
                // https://dev.mysql.com/doc/refman/8.0/en/connection-options.html#option_general_ssl-mode
                let tls_mode = match self
                    .ssl_mode
                    .map(|f| f.to_uppercase())
                    .as_ref()
                    .map(|m| m.as_str())
                {
                    None | Some("DISABLED") => MySqlSslMode::Disabled,
                    // "preferred" intentionally omitted because it has dubious security
                    // properties.
                    Some("REQUIRED") | Some("REQUIRE") => MySqlSslMode::Required,
                    Some("VERIFY_CA") | Some("VERIFY-CA") => MySqlSslMode::VerifyCa,
                    Some("VERIFY_IDENTITY") | Some("VERIFY-IDENTITY") => {
                        MySqlSslMode::VerifyIdentity
                    }
                    Some(m) => sql_bail!("invalid CONNECTION: unknown SSL MODE {}", m.quoted()),
                };

                // TODO we should move to self.port being unsupported if aws_privatelink is some, see <https://github.com/MaterializeInc/materialize/issues/24712#issuecomment-1925443977>
                if let Some(privatelink) = self.aws_privatelink.as_ref() {
                    if privatelink.port.is_some() {
                        sql_bail!("invalid CONNECTION: PORT in AWS PRIVATELINK is only supported for kafka")
                    }
                }
                let tunnel = scx.build_tunnel_definition(self.ssh_tunnel, self.aws_privatelink)?;

                Connection::MySql(MySqlConnection {
                    password: self.password.map(|password| password.into()),
                    host: self
                        .host
                        .ok_or_else(|| sql_err!("HOST option is required"))?,
                    port: self.port.unwrap_or(3306_u16),
                    tunnel,
                    tls_mode,
                    tls_root_cert: self.ssl_certificate_authority,
                    tls_identity,
                    user: self
                        .user
                        .ok_or_else(|| sql_err!("USER option is required"))?,
                })
            }
        };

        Ok(connection)
    }

    pub fn get_brokers(
        &self,
        scx: &StatementContext,
    ) -> Result<Vec<mz_storage_types::connections::KafkaBroker<ReferencedConnection>>, PlanError>
    {
        let mut brokers = match (&self.broker, &self.brokers, &self.aws_privatelink) {
            (Some(v), None, None) => v.to_vec(),
            (None, Some(v), None) => v.to_vec(),
            (None, None, Some(_)) => vec![],
            (None, None, None) => {
                sql_bail!("invalid CONNECTION: must set one of BROKER, BROKERS, or AWS PRIVATELINK")
            }
            _ => sql_bail!(
                "invalid CONNECTION: can only set one of BROKER, BROKERS, or AWS PRIVATELINK"
            ),
        };

        // NOTE: we allow broker configurations to be mixed and matched. If/when we support
        // a top-level `SSH TUNNEL` configuration, we will need additional assertions.

        let mut out = vec![];
        for broker in &mut brokers {
            if broker.address.contains(',') {
                sql_bail!("invalid CONNECTION: cannot specify multiple Kafka broker addresses in one string.\n\n
Instead, specify BROKERS using multiple strings, e.g. BROKERS ('kafka:9092', 'kafka:9093')");
            }

            let tunnel = match &broker.tunnel {
                KafkaBrokerTunnel::Direct => Tunnel::Direct,
                KafkaBrokerTunnel::AwsPrivatelink(aws_privatelink) => {
                    let KafkaBrokerAwsPrivatelinkOptionExtracted {
                        availability_zone,
                        port,
                        seen: _,
                    } = KafkaBrokerAwsPrivatelinkOptionExtracted::try_from(
                        aws_privatelink.options.clone(),
                    )?;

                    let id = match &aws_privatelink.connection {
                        ResolvedItemName::Item { id, .. } => id,
                        _ => sql_bail!(
                            "internal error: Kafka PrivateLink connection was not resolved"
                        ),
                    };
                    let entry = scx.catalog.get_item(id);
                    match entry.connection()? {
                        Connection::AwsPrivatelink(connection) => {
                            if let Some(az) = &availability_zone {
                                if !connection.availability_zones.contains(az) {
                                    sql_bail!("AWS PrivateLink availability zone {} does not match any of the \
                                      availability zones on the AWS PrivateLink connection {}",
                                      az.quoted(),
                                        scx.catalog.resolve_full_name(entry.name()).to_string().quoted())
                                }
                            }
                            Tunnel::AwsPrivatelink(AwsPrivatelink {
                                connection_id: *id,
                                availability_zone,
                                port,
                            })
                        }
                        _ => {
                            sql_bail!("{} is not an AWS PRIVATELINK connection", entry.name().item)
                        }
                    }
                }
                KafkaBrokerTunnel::SshTunnel(ssh) => {
                    let id = match &ssh {
                        ResolvedItemName::Item { id, .. } => id,
                        _ => sql_bail!(
                            "internal error: Kafka SSH tunnel connection was not resolved"
                        ),
                    };
                    let ssh_tunnel = scx.catalog.get_item(id);
                    match ssh_tunnel.connection()? {
                        Connection::Ssh(_connection) => Tunnel::Ssh(SshTunnel {
                            connection_id: *id,
                            connection: *id,
                        }),
                        _ => {
                            sql_bail!("{} is not an SSH connection", ssh_tunnel.name().item)
                        }
                    }
                }
            };

            out.push(mz_storage_types::connections::KafkaBroker {
                address: broker.address.clone(),
                tunnel,
            });
        }

        Ok(out)
    }
}

fn plan_kafka_security(
    scx: &StatementContext,
    v: &ConnectionOptionExtracted,
) -> Result<
    (
        Option<KafkaTlsConfig>,
        Option<KafkaSaslConfig<ReferencedConnection>>,
    ),
    PlanError,
> {
    const SASL_CONFIGS: [ConnectionOptionName; 4] = [
        ConnectionOptionName::AwsConnection,
        ConnectionOptionName::SaslMechanisms,
        ConnectionOptionName::SaslUsername,
        ConnectionOptionName::SaslPassword,
    ];

    const ALL_CONFIGS: [ConnectionOptionName; 7] = concat_arrays!(
        [
            ConnectionOptionName::SslKey,
            ConnectionOptionName::SslCertificate,
            ConnectionOptionName::SslCertificateAuthority,
        ],
        SASL_CONFIGS
    );

    enum SecurityProtocol {
        Plaintext,
        Ssl,
        SaslPlaintext,
        SaslSsl,
    }

    let security_protocol = v.security_protocol.as_ref().map(|v| v.to_uppercase());
    let security_protocol = match security_protocol.as_deref() {
        Some("PLAINTEXT") => SecurityProtocol::Plaintext,
        Some("SSL") => SecurityProtocol::Ssl,
        Some("SASL_PLAINTEXT") => SecurityProtocol::SaslPlaintext,
        Some("SASL_SSL") => SecurityProtocol::SaslSsl,
        Some(p) => sql_bail!("unknown security protocol: {}", p),
        // To be secure by default, if no security protocol is explicitly
        // specified, we always choose one of the SSL-enabled protocols, using
        // the presence of any SASL options to guide us between them. Users must
        // explicitly choose a plaintext mechanism if that's what they want.
        None if SASL_CONFIGS.iter().any(|c| v.seen.contains(c)) => SecurityProtocol::SaslSsl,
        None => SecurityProtocol::Ssl,
    };

    let mut outstanding = ALL_CONFIGS
        .into_iter()
        .filter(|c| v.seen.contains(c))
        .collect::<BTreeSet<ConnectionOptionName>>();

    let tls = match security_protocol {
        SecurityProtocol::Ssl | SecurityProtocol::SaslSsl => {
            outstanding.remove(&ConnectionOptionName::SslCertificate);
            let identity = match &v.ssl_certificate {
                None => None,
                Some(cert) => {
                    outstanding.remove(&ConnectionOptionName::SslKey);
                    let Some(key) = &v.ssl_key else {
                        sql_bail!("SSL KEY must be specified with SSL CERTIFICATE");
                    };
                    Some(TlsIdentity {
                        cert: cert.clone(),
                        key: (*key).into(),
                    })
                }
            };
            outstanding.remove(&ConnectionOptionName::SslCertificateAuthority);
            Some(KafkaTlsConfig {
                identity,
                root_cert: v.ssl_certificate_authority.clone(),
            })
        }
        _ => None,
    };

    let sasl = match security_protocol {
        SecurityProtocol::SaslPlaintext | SecurityProtocol::SaslSsl => {
            outstanding.remove(&ConnectionOptionName::AwsConnection);
            match &v.aws_connection {
                Some(id) => {
                    scx.require_feature_flag(&ENABLE_AWS_MSK_IAM_AUTH)?;
                    let id = GlobalId::from(id);
                    let item = scx.catalog.get_item(&id);
                    let aws = match item.connection()? {
                        Connection::Aws(_) => AwsConnectionReference {
                            connection_id: id,
                            connection: id,
                        },
                        _ => sql_bail!("{} is not an AWS connection", item.name().item),
                    };
                    Some(KafkaSaslConfig {
                        mechanism: "OAUTHBEARER".into(),
                        username: "".into(),
                        password: None,
                        aws: Some(aws),
                    })
                }
                None => {
                    outstanding.remove(&ConnectionOptionName::SaslMechanisms);
                    outstanding.remove(&ConnectionOptionName::SaslUsername);
                    outstanding.remove(&ConnectionOptionName::SaslPassword);
                    // TODO(benesch): support a less confusing `SASL MECHANISM`
                    // alias, as only a single mechanism that can be specified.
                    let Some(mechanism) = &v.sasl_mechanisms else {
                        sql_bail!("SASL MECHANISMS must be specified");
                    };
                    let Some(username) = &v.sasl_username else {
                        sql_bail!("SASL USERNAME must be specified");
                    };
                    let Some(password) = &v.sasl_password else {
                        sql_bail!("SASL PASSWORD must be specified");
                    };
                    Some(KafkaSaslConfig {
                        // librdkafka requires SASL mechanisms to be upper case (PLAIN,
                        // SCRAM-SHA-256). For usability, we automatically uppercase the
                        // mechanism that user provides. This avoids a frustrating
                        // interaction with identifier case folding. Consider `SASL
                        // MECHANISMS = PLAIN`. Identifier case folding results in a
                        // SASL mechanism of `plain` (note the lowercase), which
                        // Materialize previously rejected with an error of "SASL
                        // mechanism must be uppercase." This was deeply frustarting for
                        // users who were not familiar with identifier case folding
                        // rules. See #22205.
                        mechanism: mechanism.to_uppercase(),
                        username: username.clone(),
                        password: Some((*password).into()),
                        aws: None,
                    })
                }
            }
        }
        _ => None,
    };

    if let Some(outstanding) = outstanding.first() {
        sql_bail!("option {outstanding} not supported with this configuration");
    }

    Ok((tls, sasl))
}