krafka 0.10.0

A pure Rust, async-native Apache Kafka client
Documentation
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
//! Secure connection with TLS and SASL support.
//!
//! This module provides authenticated connections to Kafka brokers.

use std::time::Duration;

use crate::auth::{
    AuthConfig, AwsMskIamCredentialProvider, ChannelBinding, MskIamAuthenticator, OAuthBearerToken,
    OAuthBearerTokenProvider, PlainCredentials, SaslMechanism, ScramClient, ScramMechanism,
    SecurityProtocol, TlsConfig,
};
use crate::error::{KrafkaError, Result};
use zeroize::Zeroizing;

use super::connection::ConnectionConfig;

/// Extended connection config with authentication.
#[derive(Debug, Clone)]
pub struct SecureConnectionConfig {
    /// Base connection config.
    pub connection: ConnectionConfig,
    /// Authentication config.
    pub auth: AuthConfig,
}

impl Default for SecureConnectionConfig {
    fn default() -> Self {
        Self {
            connection: ConnectionConfig::default(),
            auth: AuthConfig::plaintext(),
        }
    }
}

impl SecureConnectionConfig {
    /// Create a new secure connection config builder.
    pub fn builder() -> SecureConnectionConfigBuilder {
        SecureConnectionConfigBuilder::default()
    }
}

/// Builder for SecureConnectionConfig.
#[must_use = "builders do nothing until .build() is called"]
#[derive(Debug, Default)]
pub struct SecureConnectionConfigBuilder {
    connection: ConnectionConfig,
    auth: AuthConfig,
}

impl SecureConnectionConfigBuilder {
    /// Set connection timeout.
    pub fn connect_timeout(mut self, timeout: Duration) -> Self {
        self.connection.connect_timeout = timeout;
        self
    }

    /// Set request timeout.
    pub fn request_timeout(mut self, timeout: Duration) -> Self {
        self.connection.request_timeout = timeout;
        self
    }

    /// Set client ID.
    pub fn client_id(mut self, client_id: impl Into<String>) -> Self {
        self.connection.client_id = client_id.into();
        self
    }

    /// Set TCP nodelay.
    pub fn nodelay(mut self, nodelay: bool) -> Self {
        self.connection.nodelay = nodelay;
        self
    }

    /// Set authentication config.
    pub fn auth(mut self, auth: AuthConfig) -> Self {
        self.auth = auth;
        self
    }

    /// Configure SASL/PLAIN authentication.
    pub fn sasl_plain(
        mut self,
        username: impl Into<String>,
        password: impl Into<String>,
    ) -> crate::Result<Self> {
        self.auth = AuthConfig::sasl_plain(username, password)?;
        Ok(self)
    }

    /// Configure SASL/SCRAM-SHA-256 authentication.
    pub fn sasl_scram_sha256(
        mut self,
        username: impl Into<String>,
        password: impl Into<String>,
    ) -> Self {
        self.auth = AuthConfig::sasl_scram_sha256(username, password);
        self
    }

    /// Configure SASL/SCRAM-SHA-512 authentication.
    pub fn sasl_scram_sha512(
        mut self,
        username: impl Into<String>,
        password: impl Into<String>,
    ) -> Self {
        self.auth = AuthConfig::sasl_scram_sha512(username, password);
        self
    }

    /// Configure AWS MSK IAM authentication.
    pub fn aws_msk_iam(
        mut self,
        access_key_id: impl Into<String>,
        secret_access_key: impl Into<String>,
        region: impl Into<String>,
    ) -> Self {
        self.auth = AuthConfig::aws_msk_iam(access_key_id, secret_access_key, region);
        self
    }

    /// Configure AWS MSK IAM authentication with a credential provider.
    ///
    /// The provider is called on every new broker connection, ensuring
    /// credentials are always fresh. Recommended for temporary credentials.
    pub fn aws_msk_iam_provider(
        mut self,
        provider: impl AwsMskIamCredentialProvider + 'static,
    ) -> Self {
        self.auth = AuthConfig::aws_msk_iam_provider(provider);
        self
    }

    /// Configure SASL/OAUTHBEARER authentication with a static token.
    ///
    /// For automatic token refresh, use [`sasl_oauthbearer_provider()`](Self::sasl_oauthbearer_provider).
    /// For SASL extensions, use [`sasl_oauthbearer_token()`](Self::sasl_oauthbearer_token).
    pub fn sasl_oauthbearer(mut self, token: impl Into<String>) -> Self {
        self.auth = AuthConfig::sasl_oauthbearer(token);
        self
    }

    /// Configure SASL/OAUTHBEARER authentication with a pre-built token.
    pub fn sasl_oauthbearer_token(mut self, token: OAuthBearerToken) -> Self {
        self.auth = AuthConfig::sasl_oauthbearer_token(token);
        self
    }

    /// Configure SASL/OAUTHBEARER authentication with an async token provider.
    ///
    /// The provider is called on every new broker connection, ensuring
    /// tokens are always fresh.
    pub fn sasl_oauthbearer_provider(
        mut self,
        provider: impl OAuthBearerTokenProvider + 'static,
    ) -> Self {
        self.auth = AuthConfig::sasl_oauthbearer_provider(provider);
        self
    }

    /// Configure TLS with default settings.
    pub fn tls(mut self, tls_config: TlsConfig) -> Self {
        self.auth.tls_config = Some(tls_config);
        if self.auth.security_protocol == SecurityProtocol::Plaintext {
            self.auth.security_protocol = SecurityProtocol::Ssl;
        } else if self.auth.security_protocol == SecurityProtocol::SaslPlaintext {
            self.auth.security_protocol = SecurityProtocol::SaslSsl;
        }
        self
    }

    /// Build the config.
    pub fn build(self) -> SecureConnectionConfig {
        SecureConnectionConfig {
            connection: self.connection,
            auth: self.auth,
        }
    }
}

/// Response from processing a SASL challenge.
#[non_exhaustive]
#[derive(Debug)]
pub enum ChallengeResponse {
    /// Send these bytes and continue the handshake.
    ///
    /// Wrapped in [`Zeroizing`] because SCRAM challenge responses contain
    /// proof data derived from the password.
    Continue(Zeroizing<Vec<u8>>),
    /// Send these bytes to satisfy a protocol requirement (e.g., the
    /// OAuthBearer `\x01` failure-ack per RFC 7628 §3.2.3), then fail
    /// with the given auth error. The caller must **not** attempt to read
    /// a broker response after sending — the server may close immediately.
    AckThenFail {
        /// Bytes the protocol requires the client to send before giving up.
        ack: Vec<u8>,
        /// The underlying authentication error.
        error: KrafkaError,
    },
    /// Authentication step complete, no response to send.
    Done,
}

/// SASL authenticator for handling authentication handshakes.
pub struct SaslAuthenticator {
    mechanism: SaslMechanism,
    plain_credentials: Option<PlainCredentials>,
    scram_client: Option<ScramClient>,
    msk_iam_authenticator: Option<MskIamAuthenticator>,
    msk_iam_complete: bool,
    oauthbearer_token: Option<OAuthBearerToken>,
    oauthbearer_complete: bool,
}

impl SaslAuthenticator {
    /// Create a new SASL authenticator from auth config.
    ///
    /// Returns `Ok(None)` when `auth` has no SASL mechanism configured (plaintext
    /// connections). Returns `Err` for any configuration error, including:
    /// - GSSAPI/Kerberos (not supported in the pure-Rust build)
    /// - Missing SCRAM credentials
    /// - Missing/unresolved OAuthBearer token
    ///
    /// # Arguments
    ///
    /// * `auth` - The authentication configuration
    /// * `channel_binding` - Channel binding data for SCRAM mechanisms; pass
    ///   [`ChannelBinding::TlsServerEndPoint`] when authenticating over TLS
    ///
    /// For OAUTHBEARER with a token provider, call
    /// [`AuthConfig::resolve_provider_to_token()`] first and pass the
    /// resolved config. Provider-based configs without a resolved token
    /// return an error.
    ///
    /// For MSK IAM, you must provide the broker host after creation using `set_msk_host()`.
    pub fn new(auth: &AuthConfig, channel_binding: ChannelBinding) -> Result<Option<Self>> {
        let Some(mechanism) = auth.sasl_mechanism.as_ref() else {
            // No SASL mechanism — plaintext connection, no authenticator needed.
            return Ok(None);
        };

        match mechanism {
            SaslMechanism::Plain => Ok(Some(Self {
                mechanism: SaslMechanism::Plain,
                plain_credentials: auth.plain_credentials.clone(),
                scram_client: None,
                msk_iam_authenticator: None,
                msk_iam_complete: false,
                oauthbearer_token: None,
                oauthbearer_complete: false,
            })),
            SaslMechanism::ScramSha256 => {
                let creds = auth
                    .scram_credentials
                    .as_ref()
                    .ok_or_else(|| KrafkaError::auth("SCRAM-SHA-256 credentials not configured"))?;
                Ok(Some(Self {
                    mechanism: SaslMechanism::ScramSha256,
                    plain_credentials: None,
                    scram_client: Some(ScramClient::new(
                        &creds.username,
                        &creds.password,
                        ScramMechanism::Sha256,
                        channel_binding,
                    )),
                    msk_iam_authenticator: None,
                    msk_iam_complete: false,
                    oauthbearer_token: None,
                    oauthbearer_complete: false,
                }))
            }
            SaslMechanism::ScramSha512 => {
                let creds = auth
                    .scram_credentials
                    .as_ref()
                    .ok_or_else(|| KrafkaError::auth("SCRAM-SHA-512 credentials not configured"))?;
                Ok(Some(Self {
                    mechanism: SaslMechanism::ScramSha512,
                    plain_credentials: None,
                    scram_client: Some(ScramClient::new(
                        &creds.username,
                        &creds.password,
                        ScramMechanism::Sha512,
                        channel_binding,
                    )),
                    msk_iam_authenticator: None,
                    msk_iam_complete: false,
                    oauthbearer_token: None,
                    oauthbearer_complete: false,
                }))
            }
            SaslMechanism::AwsMskIam => {
                // MSK IAM requires the broker host to be set later
                Ok(Some(Self {
                    mechanism: SaslMechanism::AwsMskIam,
                    plain_credentials: None,
                    scram_client: None,
                    msk_iam_authenticator: None,
                    msk_iam_complete: false,
                    oauthbearer_token: None,
                    oauthbearer_complete: false,
                }))
            }
            SaslMechanism::OAuthBearer => {
                let token = auth.oauthbearer_token.as_ref().cloned().ok_or_else(|| {
                    KrafkaError::auth(
                        "OAUTHBEARER mechanism requires an OAuth bearer token; \
                         if using a token provider, call resolve_provider_to_token() first",
                    )
                })?;
                Ok(Some(Self {
                    mechanism: SaslMechanism::OAuthBearer,
                    plain_credentials: None,
                    scram_client: None,
                    msk_iam_authenticator: None,
                    msk_iam_complete: false,
                    oauthbearer_token: Some(token),
                    oauthbearer_complete: false,
                }))
            }
            SaslMechanism::Gssapi => Err(KrafkaError::auth(
                "SASL/GSSAPI (Kerberos) is not available in the pure-Rust build; \
                 use OAUTHBEARER for token-based authentication or SCRAM-SHA-256/512 \
                 for password-based authentication",
            )),
        }
    }

    /// Create a new SASL authenticator for MSK IAM with the broker host.
    ///
    /// # Errors
    ///
    /// Returns `Err` if MSK IAM signing payload creation fails.
    /// Returns `Ok(None)` if the mechanism is not MSK IAM or credentials are missing.
    pub fn new_msk_iam(
        auth: &AuthConfig,
        host: &str,
        clock_offset_secs: i64,
    ) -> Result<Option<Self>> {
        if !matches!(auth.sasl_mechanism, Some(SaslMechanism::AwsMskIam)) {
            return Ok(None);
        }

        let Some(creds) = auth.aws_msk_iam_credentials.as_ref() else {
            return Ok(None);
        };
        let authenticator =
            MskIamAuthenticator::new_with_clock_offset(creds, host, clock_offset_secs)?;

        Ok(Some(Self {
            mechanism: SaslMechanism::AwsMskIam,
            plain_credentials: None,
            scram_client: None,
            msk_iam_authenticator: Some(authenticator),
            msk_iam_complete: false,
            oauthbearer_token: None,
            oauthbearer_complete: false,
        }))
    }

    /// Set the broker host for MSK IAM authentication.
    ///
    /// Must be called before `initial_response()` for MSK IAM.
    ///
    /// # Errors
    ///
    /// Returns an error if the mechanism is MSK IAM but credentials are
    /// missing (e.g. `resolve_msk_iam_provider()` was not called) or if
    /// MSK IAM signing payload creation fails.
    pub fn set_msk_host(
        &mut self,
        auth: &AuthConfig,
        host: &str,
        clock_offset_secs: i64,
    ) -> Result<()> {
        if self.mechanism == SaslMechanism::AwsMskIam {
            let creds = auth.aws_msk_iam_credentials.as_ref().ok_or_else(|| {
                KrafkaError::auth(
                    "AWS MSK IAM mechanism selected but no credentials available; \
                     if using a credential provider, ensure resolve_msk_iam_provider() \
                     is called before creating the authenticator",
                )
            })?;
            self.msk_iam_authenticator = Some(MskIamAuthenticator::new_with_clock_offset(
                creds,
                host,
                clock_offset_secs,
            )?);
        }
        Ok(())
    }

    /// Get the mechanism name for SASL handshake.
    pub fn mechanism_name(&self) -> &str {
        match self.mechanism {
            SaslMechanism::Plain => "PLAIN",
            SaslMechanism::ScramSha256 => "SCRAM-SHA-256",
            SaslMechanism::ScramSha512 => "SCRAM-SHA-512",
            SaslMechanism::AwsMskIam => "AWS_MSK_IAM",
            SaslMechanism::OAuthBearer => "OAUTHBEARER",
            SaslMechanism::Gssapi => "GSSAPI",
        }
    }

    /// Get the initial authentication bytes.
    ///
    /// For PLAIN mechanism, the returned bytes are wrapped in `Zeroizing`
    /// to ensure the password is erased from memory after being sent on the wire.
    ///
    /// Returns an error if the OAuthBearer token is expired or too close to expiry.
    pub fn initial_response(&mut self) -> Result<Zeroizing<Vec<u8>>> {
        match self.mechanism {
            SaslMechanism::Plain => Ok(self
                .plain_credentials
                .as_ref()
                .map(|c| c.to_auth_bytes())
                .unwrap_or_default()),
            SaslMechanism::ScramSha256 | SaslMechanism::ScramSha512 => Ok(Zeroizing::new(
                self.scram_client
                    .as_mut()
                    .map(|c| c.client_first_message())
                    .unwrap_or_default(),
            )),
            SaslMechanism::AwsMskIam => Ok(Zeroizing::new(
                self.msk_iam_authenticator
                    .as_ref()
                    .map(|a| a.create_auth_payload())
                    .unwrap_or_default(),
            )),
            SaslMechanism::OAuthBearer => {
                if let Some(token) = &self.oauthbearer_token {
                    if token.needs_refresh() {
                        return Err(KrafkaError::auth(
                            "OAuthBearer token is expired or too close to expiry; obtain a fresh token before connecting",
                        ));
                    }
                    Ok(Zeroizing::new(token.to_gs2_initial_response()))
                } else {
                    Ok(Zeroizing::new(Vec::new()))
                }
            }
            SaslMechanism::Gssapi => Ok(Zeroizing::new(Vec::new())),
        }
    }

    /// Process a challenge response from the broker.
    ///
    /// Returns a [`ChallengeResponse`] indicating what the caller should do next:
    /// - [`Continue`](ChallengeResponse::Continue) — send the bytes, read the next challenge.
    /// - [`AckThenFail`](ChallengeResponse::AckThenFail) — send the ack bytes, then
    ///   return the error **without** reading a response (the server may close immediately).
    /// - [`Done`](ChallengeResponse::Done) — handshake complete, nothing to send.
    pub async fn process_challenge(&mut self, challenge: &[u8]) -> Result<ChallengeResponse> {
        match self.mechanism {
            SaslMechanism::Plain => {
                // PLAIN has no challenge-response, just initial auth
                Ok(ChallengeResponse::Done)
            }
            SaslMechanism::ScramSha256 | SaslMechanism::ScramSha512 => {
                let scram = self
                    .scram_client
                    .as_mut()
                    .ok_or_else(|| KrafkaError::auth("SCRAM client not initialized"))?;

                // Process based on current state
                match scram.state() {
                    crate::auth::ScramState::WaitingServerFirst => {
                        let response = scram.process_server_first(challenge).await?;
                        Ok(ChallengeResponse::Continue(Zeroizing::new(response)))
                    }
                    crate::auth::ScramState::WaitingServerFinal => {
                        scram.verify_server_final(challenge)?;
                        Ok(ChallengeResponse::Done)
                    }
                    _ => Err(KrafkaError::auth("Unexpected SCRAM state")),
                }
            }
            SaslMechanism::AwsMskIam => {
                // MSK IAM authentication is complete after the server accepts the signed payload
                // The server sends back a success response (which may be empty)
                self.msk_iam_complete = true;
                Ok(ChallengeResponse::Done)
            }
            SaslMechanism::OAuthBearer => {
                // Process server response after initial GS2 token message
                let token = self
                    .oauthbearer_token
                    .as_ref()
                    .ok_or_else(|| KrafkaError::auth("OAuthBearer token not configured"))?;

                match token.process_server_response(challenge) {
                    Ok(()) => {
                        self.oauthbearer_complete = true;
                        Ok(ChallengeResponse::Done)
                    }
                    Err(e) => {
                        // RFC 7628 §3.2.3: client MUST send a single \x01 byte to
                        // acknowledge the server's error before closing the connection.
                        // Return AckThenFail so the caller can send the ack and
                        // propagate the real auth error deterministically.
                        Ok(ChallengeResponse::AckThenFail {
                            ack: vec![0x01],
                            error: e,
                        })
                    }
                }
            }
            SaslMechanism::Gssapi => Err(KrafkaError::auth(
                "SASL/GSSAPI (Kerberos) is not available in the pure-Rust build",
            )),
        }
    }

    /// Check if authentication is complete.
    pub fn is_complete(&self) -> bool {
        match self.mechanism {
            SaslMechanism::Plain => true, // PLAIN completes after initial response
            SaslMechanism::ScramSha256 | SaslMechanism::ScramSha512 => self
                .scram_client
                .as_ref()
                .is_some_and(|c| *c.state() == crate::auth::ScramState::Complete),
            SaslMechanism::AwsMskIam => self.msk_iam_complete,
            SaslMechanism::OAuthBearer => self.oauthbearer_complete,
            SaslMechanism::Gssapi => false,
        }
    }
}

#[cfg(test)]
#[allow(clippy::unwrap_used, clippy::expect_used, clippy::panic)]
mod tests {
    use super::*;

    #[test]
    fn test_secure_connection_config_default() {
        let config = SecureConnectionConfig::default();
        assert_eq!(config.auth.security_protocol, SecurityProtocol::Plaintext);
        assert!(!config.auth.requires_tls());
        assert!(!config.auth.requires_sasl());
    }

    #[test]
    fn test_secure_connection_config_builder() {
        let config = SecureConnectionConfig::builder()
            .client_id("test-client")
            .connect_timeout(Duration::from_secs(5))
            .sasl_plain("user", "pass")
            .unwrap()
            .build();

        assert_eq!(config.connection.client_id, "test-client");
        assert_eq!(config.connection.connect_timeout, Duration::from_secs(5));
        assert!(config.auth.requires_sasl());
    }

    #[test]
    fn test_secure_connection_config_with_tls() {
        let config = SecureConnectionConfig::builder()
            .tls(TlsConfig::new())
            .build();

        assert!(config.auth.requires_tls());
    }

    #[test]
    fn test_sasl_authenticator_plain() {
        let auth = AuthConfig::sasl_plain("user", "pass").unwrap();
        let mut authenticator = SaslAuthenticator::new(&auth, ChannelBinding::None)
            .unwrap()
            .unwrap();

        assert_eq!(authenticator.mechanism_name(), "PLAIN");

        let initial = authenticator.initial_response().unwrap();
        assert_eq!(&*initial, b"\0user\0pass");
        assert!(authenticator.is_complete());
    }

    #[test]
    fn test_sasl_authenticator_scram() {
        let auth = AuthConfig::sasl_scram_sha256("user", "pass");
        let mut authenticator = SaslAuthenticator::new(&auth, ChannelBinding::None)
            .unwrap()
            .unwrap();

        assert_eq!(authenticator.mechanism_name(), "SCRAM-SHA-256");

        let initial = authenticator.initial_response().unwrap();
        assert!(initial.starts_with(b"n,,n=user,r="));
        assert!(!authenticator.is_complete());
    }

    #[tokio::test]
    async fn test_sasl_authenticator_msk_iam() {
        let auth = AuthConfig::aws_msk_iam("AKIAIOSFODNN7EXAMPLE", "secret", "us-east-1");
        let mut authenticator =
            SaslAuthenticator::new_msk_iam(&auth, "broker.kafka.us-east-1.amazonaws.com", 0)
                .unwrap()
                .unwrap();

        assert_eq!(authenticator.mechanism_name(), "AWS_MSK_IAM");

        let initial = authenticator.initial_response().unwrap();
        let payload_str = String::from_utf8(initial.to_vec()).unwrap();

        // Verify JSON payload structure
        assert!(payload_str.contains("\"version\":\"2020_10_22\""));
        assert!(payload_str.contains("\"host\":\"broker.kafka.us-east-1.amazonaws.com\""));
        assert!(payload_str.contains("\"action\":\"kafka-cluster:Connect\""));
        assert!(payload_str.contains("\"x-amz-signature\":"));

        // Not complete until server responds
        assert!(!authenticator.is_complete());

        // Process empty challenge (server acceptance)
        authenticator.process_challenge(&[]).await.unwrap();
        assert!(authenticator.is_complete());
    }

    #[test]
    fn test_secure_connection_config_builder_msk_iam() {
        let config = SecureConnectionConfig::builder()
            .aws_msk_iam("AKID", "secret", "us-east-1")
            .build();

        assert!(config.auth.requires_tls());
        assert!(config.auth.requires_sasl());
        assert_eq!(config.auth.sasl_mechanism, Some(SaslMechanism::AwsMskIam));
    }

    #[tokio::test]
    async fn test_sasl_authenticator_oauthbearer() {
        let auth = AuthConfig::sasl_oauthbearer("my-jwt-token");
        let mut authenticator = SaslAuthenticator::new(&auth, ChannelBinding::None)
            .unwrap()
            .unwrap();

        assert_eq!(authenticator.mechanism_name(), "OAUTHBEARER");

        let initial = authenticator.initial_response().unwrap();
        assert_eq!(&*initial, b"n,,\x01auth=Bearer my-jwt-token\x01\x01");

        // Not complete until server responds
        assert!(!authenticator.is_complete());

        // Process empty challenge (server acceptance)
        authenticator.process_challenge(&[]).await.unwrap();
        assert!(authenticator.is_complete());
    }

    #[test]
    fn test_sasl_authenticator_oauthbearer_with_extensions() {
        let token = OAuthBearerToken::new("tok").with_extension("logicalCluster", "lkc-123");
        let auth = AuthConfig::sasl_oauthbearer_token(token);
        let mut authenticator = SaslAuthenticator::new(&auth, ChannelBinding::None)
            .unwrap()
            .unwrap();

        let initial = authenticator.initial_response().unwrap();
        let initial_str = String::from_utf8_lossy(&initial);
        assert!(initial_str.starts_with("n,,\x01auth=Bearer tok"));
        assert!(initial_str.contains("logicalCluster=lkc-123"));
        assert!(initial_str.ends_with("\x01\x01"));
    }

    #[tokio::test]
    async fn test_sasl_authenticator_oauthbearer_server_error() {
        let auth = AuthConfig::sasl_oauthbearer("bad-token");
        let mut authenticator = SaslAuthenticator::new(&auth, ChannelBinding::None)
            .unwrap()
            .unwrap();
        let _ = authenticator.initial_response().unwrap();

        // Server error returns AckThenFail: the \x01 byte and the auth error together.
        let result = authenticator
            .process_challenge(br#"{"status":"invalid_token"}"#)
            .await
            .unwrap();
        match result {
            ChallengeResponse::AckThenFail { ack, error } => {
                assert_eq!(ack, vec![0x01]);
                assert!(error.to_string().contains("invalid_token"));
            }
            other => panic!("expected AckThenFail, got {other:?}"),
        }

        // Authentication was not completed successfully.
        assert!(!authenticator.is_complete());
    }

    #[test]
    fn test_sasl_authenticator_oauthbearer_missing_token() {
        // Create an AuthConfig with OAuthBearer mechanism but no token
        let auth = AuthConfig {
            security_protocol: SecurityProtocol::SaslPlaintext,
            sasl_mechanism: Some(SaslMechanism::OAuthBearer),
            oauthbearer_token: None,
            ..Default::default()
        };
        assert!(SaslAuthenticator::new(&auth, ChannelBinding::None).is_err());
    }

    #[test]
    fn test_sasl_authenticator_gssapi_fails_gracefully() {
        let auth = AuthConfig {
            security_protocol: SecurityProtocol::SaslPlaintext,
            sasl_mechanism: Some(SaslMechanism::Gssapi),
            ..Default::default()
        };
        assert!(SaslAuthenticator::new(&auth, ChannelBinding::None).is_err());
    }

    #[test]
    fn test_secure_connection_config_builder_oauthbearer() {
        let config = SecureConnectionConfig::builder()
            .sasl_oauthbearer("my-token")
            .build();

        assert!(config.auth.requires_sasl());
        assert_eq!(config.auth.sasl_mechanism, Some(SaslMechanism::OAuthBearer));
        assert!(config.auth.oauthbearer_token.is_some());
    }

    #[test]
    fn test_secure_connection_config_builder_oauthbearer_token() {
        let token = OAuthBearerToken::new("tok").with_extension("key", "val");
        let config = SecureConnectionConfig::builder()
            .sasl_oauthbearer_token(token)
            .build();

        assert!(config.auth.requires_sasl());
        assert_eq!(config.auth.sasl_mechanism, Some(SaslMechanism::OAuthBearer));
    }

    #[test]
    fn test_secure_connection_config_builder_oauthbearer_provider() {
        let config = SecureConnectionConfig::builder()
            .sasl_oauthbearer_provider(|| async { Ok(OAuthBearerToken::new("provider-token")) })
            .build();

        assert!(config.auth.requires_sasl());
        assert_eq!(config.auth.sasl_mechanism, Some(SaslMechanism::OAuthBearer));
        assert!(config.auth.oauthbearer_provider.is_some());
        assert!(config.auth.oauthbearer_token.is_none());
    }

    #[test]
    fn test_sasl_authenticator_oauthbearer_expired_token_rejected() {
        use std::time::{SystemTime, UNIX_EPOCH};

        // Token expired 1 hour ago
        let past_ms = SystemTime::now()
            .duration_since(UNIX_EPOCH)
            .unwrap()
            .as_millis() as i64
            - 3_600_000;
        let token = OAuthBearerToken::new("expired-jwt").with_lifetime_ms(past_ms);
        let auth = AuthConfig::sasl_oauthbearer_token(token);
        let mut authenticator = SaslAuthenticator::new(&auth, ChannelBinding::None)
            .unwrap()
            .unwrap();

        let result = authenticator.initial_response();
        assert!(result.is_err());
        assert!(result.unwrap_err().to_string().contains("expired"));
    }

    #[test]
    fn test_sasl_authenticator_oauthbearer_valid_token_accepted() {
        use std::time::{SystemTime, UNIX_EPOCH};

        // Token expires 1 hour from now
        let future_ms = SystemTime::now()
            .duration_since(UNIX_EPOCH)
            .unwrap()
            .as_millis() as i64
            + 3_600_000;
        let token = OAuthBearerToken::new("valid-jwt").with_lifetime_ms(future_ms);
        let auth = AuthConfig::sasl_oauthbearer_token(token);
        let mut authenticator = SaslAuthenticator::new(&auth, ChannelBinding::None)
            .unwrap()
            .unwrap();

        let result = authenticator.initial_response();
        assert!(result.is_ok());
    }

    #[test]
    fn test_sasl_authenticator_oauthbearer_near_expiry_token_rejected() {
        use std::time::{SystemTime, UNIX_EPOCH};

        let near_future_ms = SystemTime::now()
            .duration_since(UNIX_EPOCH)
            .unwrap()
            .as_millis() as i64
            + 10_000;
        let token = OAuthBearerToken::new("near-expiry-jwt").with_lifetime_ms(near_future_ms);
        let auth = AuthConfig::sasl_oauthbearer_token(token);
        let mut authenticator = SaslAuthenticator::new(&auth, ChannelBinding::None)
            .unwrap()
            .unwrap();

        let result = authenticator.initial_response();
        assert!(result.is_err());
        assert!(
            result
                .unwrap_err()
                .to_string()
                .contains("too close to expiry")
        );
    }
}