krafka 0.7.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
//! SASL/OAUTHBEARER authentication (RFC 7628, KIP-255).
//!
//! Implements the SASL/OAUTHBEARER mechanism for Kafka. The client sends an
//! OAuth 2.0 bearer token in the GS2 framing format, and the server validates
//! the token against its configured OAuth/OIDC provider.
//!
//! # Token format (RFC 7628)
//!
//! The initial client response uses the GS2 framing:
//!
//! ```text
//! n,,\x01auth=Bearer <token>\x01\x01
//! ```
//!
//! Optional SASL extensions can be appended before the terminator:
//!
//! ```text
//! n,,\x01auth=Bearer <token>\x01key1=value1\x01key2=value2\x01\x01
//! ```
//!
//! # Usage
//!
//! ```rust,ignore
//! use krafka::auth::{AuthConfig, OAuthBearerToken};
//!
//! // Static token
//! let config = AuthConfig::sasl_oauthbearer("my-jwt-token");
//!
//! // Token with SASL extensions (e.g., Confluent Cloud)
//! let token = OAuthBearerToken::new("my-jwt-token")
//!     .with_extension("logicalCluster", "lkc-123")
//!     .with_extension("identityPoolId", "pool-456");
//! let config = AuthConfig::sasl_oauthbearer_token(token);
//!
//! // Automatic token refresh via provider (recommended for production)
//! let config = AuthConfig::sasl_oauthbearer_provider(|| async {
//!     let jwt = my_oauth_client.get_access_token().await?;
//!     Ok(OAuthBearerToken::new(jwt))
//! });
//! ```

use std::collections::BTreeMap;
use std::fmt;
use std::future::Future;
use std::pin::Pin;
use std::sync::Arc;
use std::time::{SystemTime, UNIX_EPOCH};

use zeroize::{Zeroize, ZeroizeOnDrop};

use crate::error::{KrafkaError, Result};

const OAUTHBEARER_EXPIRY_SKEW_MARGIN_MS: i64 = 30_000;

fn current_epoch_ms() -> i64 {
    let now_u128 = SystemTime::now()
        .duration_since(UNIX_EPOCH)
        .unwrap_or_default()
        .as_millis();
    i64::try_from(now_u128).unwrap_or(i64::MAX)
}

/// Trait for providing fresh OAuth 2.0 bearer tokens on each broker connection.
///
/// Implement this to integrate with your OAuth/OIDC provider. The provider is
/// called on every new broker connection (including automatic reconnections),
/// ensuring tokens are always fresh.
///
/// # Examples
///
/// ```rust,ignore
/// use krafka::auth::{OAuthBearerToken, OAuthBearerTokenProvider};
/// use krafka::error::Result;
/// use std::future::Future;
/// use std::pin::Pin;
///
/// struct MyProvider { /* OAuth client */ }
///
/// impl OAuthBearerTokenProvider for MyProvider {
///     fn provide_token(&self) -> Pin<Box<dyn Future<Output = Result<OAuthBearerToken>> + Send + '_>> {
///         Box::pin(async move {
///             // Fetch a fresh token from your OAuth server
///             let jwt = my_oauth_client.get_access_token().await?;
///             Ok(OAuthBearerToken::new(jwt))
///         })
///     }
/// }
/// ```
pub trait OAuthBearerTokenProvider: Send + Sync {
    /// Fetch a fresh OAuth 2.0 bearer token.
    ///
    /// Called on every new broker connection. Implementations should handle
    /// token caching and refresh internally if desired.
    fn provide_token(&self) -> Pin<Box<dyn Future<Output = Result<OAuthBearerToken>> + Send + '_>>;
}

/// Blanket impl: any `Fn() -> Future<Output = Result<OAuthBearerToken>>` is a provider.
///
/// The `'static` bound on `Fut` is required because the trait method signature
/// uses an anonymous lifetime (`+ '_`), and the compiler cannot prove the
/// future outlives `&self` without it. In practice this is not restrictive:
/// closures that own their captured state (the common case) produce `'static`
/// futures. For borrowing patterns, implement `OAuthBearerTokenProvider`
/// directly.
impl<F, Fut> OAuthBearerTokenProvider for F
where
    F: Fn() -> Fut + Send + Sync,
    Fut: Future<Output = Result<OAuthBearerToken>> + Send + 'static,
{
    fn provide_token(&self) -> Pin<Box<dyn Future<Output = Result<OAuthBearerToken>> + Send + '_>> {
        Box::pin(self())
    }
}

/// Handle wrapping an [`Arc<dyn OAuthBearerTokenProvider>`].
///
/// This wrapper provides `Clone` and `Debug` so it can be stored in
/// [`AuthConfig`](super::AuthConfig) without requiring implementors to
/// derive those traits.
#[derive(Clone)]
pub struct OAuthBearerTokenProviderHandle(Arc<dyn OAuthBearerTokenProvider>);

impl OAuthBearerTokenProviderHandle {
    /// Create a new handle wrapping the given provider.
    pub fn new(provider: impl OAuthBearerTokenProvider + 'static) -> Self {
        Self(Arc::new(provider))
    }

    /// Fetch a fresh token from the wrapped provider.
    pub async fn provide_token(&self) -> Result<OAuthBearerToken> {
        self.0.provide_token().await
    }
}

impl fmt::Debug for OAuthBearerTokenProviderHandle {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        f.write_str("[OAuthBearerTokenProvider]")
    }
}

/// OAuth 2.0 bearer token for SASL/OAUTHBEARER authentication.
///
/// Implements the SASL/OAUTHBEARER mechanism as defined in RFC 7628 and KIP-255.
/// The token value is zeroized from memory on drop.
///
/// # Examples
///
/// ```rust
/// use krafka::auth::OAuthBearerToken;
///
/// // Simple token
/// let token = OAuthBearerToken::new("eyJhbGciOiJSUzI1NiIsInR5cCI6IkpXVCJ9...");
///
/// // Token with SASL extensions
/// let token = OAuthBearerToken::new("eyJhbGciOiJSUzI1NiIsInR5cCI6IkpXVCJ9...")
///     .with_extension("logicalCluster", "lkc-abc123")
///     .with_extension("identityPoolId", "pool-xyz789");
/// ```
#[derive(Clone, Zeroize, ZeroizeOnDrop)]
pub struct OAuthBearerToken {
    /// The OAuth 2.0 bearer token value (zeroized on drop).
    token_value: String,
    /// Optional SASL extensions (key=value pairs). Not secrets; skipped for zeroize.
    /// Uses `BTreeMap` for deterministic ordering.
    #[zeroize(skip)]
    extensions: BTreeMap<String, String>,
    /// Optional token expiry as milliseconds since the Unix epoch.
    ///
    /// When set, the client validates the token is not expired before sending
    /// it to the broker— preventing wasted authentication round-trips
    /// and potential infinite retry loops with stale tokens.
    #[zeroize(skip)]
    lifetime_ms: Option<i64>,
}

impl OAuthBearerToken {
    /// Create a new OAuth bearer token.
    ///
    /// The `token_value` should be a valid OAuth 2.0 access token, typically a JWT.
    ///
    /// # Example
    ///
    /// ```rust
    /// use krafka::auth::OAuthBearerToken;
    /// let token = OAuthBearerToken::new("my-jwt-token");
    /// ```
    pub fn new(token_value: impl Into<String>) -> Self {
        Self {
            token_value: token_value.into(),
            extensions: BTreeMap::new(),
            lifetime_ms: None,
        }
    }

    /// Add a SASL extension key-value pair.
    ///
    /// Extensions are sent as part of the initial OAUTHBEARER message and can be
    /// used for additional authentication context. For example, Confluent Cloud
    /// uses `logicalCluster` and `identityPoolId` extensions.
    ///
    /// # Example
    ///
    /// ```rust
    /// use krafka::auth::OAuthBearerToken;
    /// let token = OAuthBearerToken::new("my-jwt-token")
    ///     .with_extension("logicalCluster", "lkc-abc123");
    /// ```
    pub fn with_extension(mut self, key: impl Into<String>, value: impl Into<String>) -> Self {
        self.extensions.insert(key.into(), value.into());
        self
    }

    /// Set the token expiry time as milliseconds since the Unix epoch.
    ///
    /// When set, the client validates the token is not expired before sending
    /// it to the broker and rejects tokens in the final 30 seconds before
    /// expiry to avoid clock-skew races during SASL/OAUTHBEARER handshakes.
    ///
    /// # Example
    ///
    /// ```rust
    /// use krafka::auth::OAuthBearerToken;
    /// let token = OAuthBearerToken::new("my-jwt-token")
    ///     .with_lifetime_ms(1700000000000); // expires at this epoch-ms
    /// ```
    pub fn with_lifetime_ms(mut self, lifetime_ms: i64) -> Self {
        self.lifetime_ms = Some(lifetime_ms);
        self
    }

    /// Returns the token expiry time in milliseconds since the Unix epoch, if set.
    pub fn lifetime_ms(&self) -> Option<i64> {
        self.lifetime_ms
    }

    /// Returns `true` if the token has a known expiry and that expiry is in the past.
    pub fn is_expired(&self) -> bool {
        self.lifetime_ms
            .is_some_and(|lifetime_ms| current_epoch_ms() >= lifetime_ms)
    }

    /// Returns `true` if the token should be refreshed before starting a new
    /// SASL handshake.
    ///
    /// Tokens in their final 30 seconds are treated as stale even if their
    /// expiry timestamp is still in the future. This mirrors the common Kafka
    /// client practice of refreshing before the edge of expiry so broker/client
    /// clock skew does not cause avoidable authentication failures.
    pub fn needs_refresh(&self) -> bool {
        self.lifetime_ms.is_some_and(|lifetime_ms| {
            current_epoch_ms() >= lifetime_ms.saturating_sub(OAUTHBEARER_EXPIRY_SKEW_MARGIN_MS)
        })
    }

    /// Build the initial client response in GS2 framing format (RFC 7628).
    ///
    /// Format: `n,,\x01auth=Bearer <token>[\x01key=value]*\x01\x01`
    pub(crate) fn to_gs2_initial_response(&self) -> Vec<u8> {
        // Pre-calculate capacity to avoid reallocations
        let mut capacity = 3 + 1 + 12 + self.token_value.len() + 2; // n,,\x01auth=Bearer \x01\x01
        for (k, v) in &self.extensions {
            capacity += 1 + k.len() + 1 + v.len(); // \x01key=value
        }

        let mut response = Vec::with_capacity(capacity);

        // GS2 header: no channel binding, no authorization identity
        response.extend_from_slice(b"n,,");

        // Authorization: Bearer <token>
        response.push(0x01);
        response.extend_from_slice(b"auth=Bearer ");
        response.extend_from_slice(self.token_value.as_bytes());

        // SASL extensions
        for (key, value) in &self.extensions {
            response.push(0x01);
            response.extend_from_slice(key.as_bytes());
            response.push(b'=');
            response.extend_from_slice(value.as_bytes());
        }

        // Terminator
        response.push(0x01);
        response.push(0x01);

        response
    }

    /// Process the server's response after the initial client message.
    ///
    /// On success, the server sends an empty response. On failure, the server
    /// sends a JSON error payload with status, scope, and OpenID configuration.
    ///
    /// Returns `Ok(())` on success or an error with the server's message on failure.
    pub(crate) fn process_server_response(&self, challenge: &[u8]) -> Result<()> {
        // Empty challenge means the server accepted the token
        if challenge.is_empty() {
            return Ok(());
        }

        // Server sent an error — parse the JSON error response
        let error_msg = String::from_utf8_lossy(challenge);
        Err(KrafkaError::auth(format!(
            "OAUTHBEARER authentication failed: {error_msg}"
        )))
    }
}

impl fmt::Debug for OAuthBearerToken {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        f.debug_struct("OAuthBearerToken")
            .field("token_value", &"[REDACTED]")
            .field("extensions", &self.extensions)
            .field("lifetime_ms", &self.lifetime_ms)
            .finish()
    }
}

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

    #[test]
    fn test_oauthbearer_token_basic() {
        let token = OAuthBearerToken::new("my-jwt-token");
        let response = token.to_gs2_initial_response();

        // Verify GS2 format: n,,\x01auth=Bearer my-jwt-token\x01\x01
        let expected = b"n,,\x01auth=Bearer my-jwt-token\x01\x01";
        assert_eq!(response, expected);
    }

    #[test]
    fn test_oauthbearer_token_with_single_extension() {
        let token = OAuthBearerToken::new("my-token").with_extension("logicalCluster", "lkc-123");
        let response = token.to_gs2_initial_response();
        let response_str = String::from_utf8_lossy(&response);

        assert!(response_str.starts_with("n,,\x01auth=Bearer my-token"));
        assert!(response_str.contains("\x01logicalCluster=lkc-123"));
        assert!(response_str.ends_with("\x01\x01"));
    }

    #[test]
    fn test_oauthbearer_token_with_multiple_extensions() {
        let token = OAuthBearerToken::new("tok")
            .with_extension("ext1", "val1")
            .with_extension("ext2", "val2");
        let response = token.to_gs2_initial_response();
        let response_str = String::from_utf8_lossy(&response);

        assert!(response_str.starts_with("n,,\x01auth=Bearer tok"));
        assert!(response_str.contains("ext1=val1"));
        assert!(response_str.contains("ext2=val2"));
        assert!(response_str.ends_with("\x01\x01"));
    }

    #[test]
    fn test_oauthbearer_debug_redacts_token() {
        let token = OAuthBearerToken::new("secret-token-value");
        let debug = format!("{token:?}");
        assert!(!debug.contains("secret-token-value"));
        assert!(debug.contains("[REDACTED]"));
    }

    #[test]
    fn test_oauthbearer_server_response_success_empty() {
        let token = OAuthBearerToken::new("tok");
        assert!(token.process_server_response(b"").is_ok());
    }

    #[test]
    fn test_oauthbearer_server_response_error_json() {
        let token = OAuthBearerToken::new("tok");
        let error_json = br#"{"status":"invalid_token","scope":"openid"}"#;
        let result = token.process_server_response(error_json);
        assert!(result.is_err());
        let err = result.unwrap_err().to_string();
        assert!(err.contains("invalid_token"));
    }

    #[test]
    fn test_oauthbearer_gs2_format_compliance() {
        // Verify exact RFC 7628 wire format
        let token = OAuthBearerToken::new("eyJhbGciOiJSUzI1NiJ9.eyJzdWIiOiJhbGljZSJ9.sig");
        let response = token.to_gs2_initial_response();

        // Must start with GS2 header
        assert_eq!(&response[..3], b"n,,");
        // Followed by \x01
        assert_eq!(response[3], 0x01);
        // auth=Bearer prefix
        assert_eq!(&response[4..16], b"auth=Bearer ");
        // Ends with double \x01
        let len = response.len();
        assert_eq!(response[len - 2], 0x01);
        assert_eq!(response[len - 1], 0x01);
    }

    #[test]
    fn test_oauthbearer_empty_token_produces_valid_gs2() {
        // Edge case: empty token value
        let token = OAuthBearerToken::new("");
        let response = token.to_gs2_initial_response();
        assert_eq!(response, b"n,,\x01auth=Bearer \x01\x01");
    }

    #[test]
    fn test_oauthbearer_token_clone() {
        let token = OAuthBearerToken::new("tok").with_extension("k", "v");
        let cloned = token.clone();
        assert_eq!(
            cloned.to_gs2_initial_response(),
            token.to_gs2_initial_response()
        );
    }

    #[tokio::test]
    async fn test_token_provider_closure_impl() {
        let provider = || async { Ok(OAuthBearerToken::new("from-closure")) };
        let token = provider.provide_token().await.unwrap();
        assert_eq!(
            token.to_gs2_initial_response(),
            OAuthBearerToken::new("from-closure").to_gs2_initial_response()
        );
    }

    #[tokio::test]
    async fn test_token_provider_handle() {
        let handle = OAuthBearerTokenProviderHandle::new(|| async {
            Ok(OAuthBearerToken::new("handle-token"))
        });
        let token = handle.provide_token().await.unwrap();
        assert_eq!(
            token.to_gs2_initial_response(),
            OAuthBearerToken::new("handle-token").to_gs2_initial_response()
        );
    }

    #[test]
    fn test_token_provider_handle_clone() {
        let handle =
            OAuthBearerTokenProviderHandle::new(|| async { Ok(OAuthBearerToken::new("tok")) });
        let cloned = handle.clone();
        // Both point to the same Arc
        assert!(Arc::ptr_eq(&handle.0, &cloned.0));
    }

    #[test]
    fn test_token_provider_handle_debug_no_secrets() {
        let handle = OAuthBearerTokenProviderHandle::new(|| async {
            Ok(OAuthBearerToken::new("super-secret"))
        });
        let debug = format!("{handle:?}");
        assert_eq!(debug, "[OAuthBearerTokenProvider]");
        assert!(!debug.contains("super-secret"));
    }

    #[tokio::test]
    async fn test_token_provider_error_propagation() {
        let handle = OAuthBearerTokenProviderHandle::new(|| async {
            Err(KrafkaError::auth("token expired"))
        });
        let result = handle.provide_token().await;
        assert!(result.is_err());
        assert!(result.unwrap_err().to_string().contains("token expired"));
    }

    #[tokio::test]
    async fn test_token_provider_struct_impl() {
        struct StaticProvider {
            token: String,
        }
        impl OAuthBearerTokenProvider for StaticProvider {
            fn provide_token(
                &self,
            ) -> Pin<Box<dyn Future<Output = Result<OAuthBearerToken>> + Send + '_>> {
                let token = self.token.clone();
                Box::pin(async move { Ok(OAuthBearerToken::new(token)) })
            }
        }

        let provider = StaticProvider {
            token: "struct-token".to_string(),
        };
        let handle = OAuthBearerTokenProviderHandle::new(provider);
        let token = handle.provide_token().await.unwrap();
        assert_eq!(
            token.to_gs2_initial_response(),
            OAuthBearerToken::new("struct-token").to_gs2_initial_response()
        );
    }

    #[test]
    fn test_oauthbearer_token_not_expired_without_lifetime() {
        let token = OAuthBearerToken::new("tok");
        assert!(!token.is_expired());
        assert!(token.lifetime_ms().is_none());
    }

    #[test]
    fn test_oauthbearer_token_not_expired_future_lifetime() {
        // Set expiry 1 hour in the future
        let future_ms = SystemTime::now()
            .duration_since(UNIX_EPOCH)
            .unwrap()
            .as_millis() as i64
            + 3_600_000;
        let token = OAuthBearerToken::new("tok").with_lifetime_ms(future_ms);
        assert!(!token.is_expired());
        assert_eq!(token.lifetime_ms(), Some(future_ms));
    }

    #[test]
    fn test_oauthbearer_token_expired_past_lifetime() {
        // Set expiry 1 hour in the past
        let past_ms = SystemTime::now()
            .duration_since(UNIX_EPOCH)
            .unwrap()
            .as_millis() as i64
            - 3_600_000;
        let token = OAuthBearerToken::new("tok").with_lifetime_ms(past_ms);
        assert!(token.is_expired());
    }

    #[test]
    fn test_oauthbearer_token_needs_refresh_near_expiry() {
        let near_future_ms = current_epoch_ms() + 10_000;
        let token = OAuthBearerToken::new("tok").with_lifetime_ms(near_future_ms);

        assert!(!token.is_expired());
        assert!(token.needs_refresh());
    }

    #[test]
    fn test_oauthbearer_token_does_not_need_refresh_with_safe_margin() {
        let future_ms = current_epoch_ms() + 60_000;
        let token = OAuthBearerToken::new("tok").with_lifetime_ms(future_ms);

        assert!(!token.needs_refresh());
    }
}