hyperi-rustlib 2.6.0

Opinionated Rust framework for high-throughput data pipelines at PB scale. Auto-wiring config, logging, metrics, tracing, health, and graceful shutdown — built from many years of production infrastructure experience.
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
// Project:   hyperi-rustlib
// File:      src/transport/redis_transport.rs
// Purpose:   Redis/Valkey Streams transport
// Language:  Rust
//
// License:   FSL-1.1-ALv2
// Copyright: (c) 2026 HYPERI PTY LIMITED

//! # Redis Streams Transport
//!
//! Lightweight pub/sub transport using Redis (or Valkey) Streams.
//! Uses `XADD` for send, `XREADGROUP` for receive, and `XACK` for commit.
//!
//! ## Send
//!
//! Appends payload bytes to a named stream via `XADD`. Optionally caps
//! the stream length with `MAXLEN ~` for approximate trimming.
//!
//! ## Receive
//!
//! Reads from a consumer group via `XREADGROUP` with blocking. Creates
//! the consumer group on first use if it does not exist.
//!
//! ## Commit
//!
//! Acknowledges processed entries via `XACK` so they are not re-delivered
//! to other consumers in the same group.
//!
//! ## Example
//!
//! ```rust,ignore
//! use hyperi_rustlib::transport::redis_transport::{RedisTransport, RedisTransportConfig};
//!
//! let config = RedisTransportConfig {
//!     stream: Some("events".into()),
//!     ..Default::default()
//! };
//! let transport = RedisTransport::new(&config).await?;
//! transport.send("events", b"{\"msg\":\"hello\"}").await;
//! ```

use super::error::{TransportError, TransportResult};
use super::traits::{CommitToken, TransportBase, TransportReceiver, TransportSender};
use super::types::{Message, PayloadFormat, SendResult};
use redis::AsyncCommands;
use redis::streams::{StreamMaxlen, StreamReadOptions, StreamReadReply};
use serde::{Deserialize, Serialize};
use std::sync::Arc;
use std::sync::atomic::{AtomicBool, Ordering};
use tokio::sync::Mutex;

/// Commit token for Redis Streams transport.
///
/// Contains the stream name and entry ID needed for `XACK`.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct RedisToken {
    /// Stream name the entry belongs to.
    pub stream: Arc<str>,
    /// Redis stream entry ID (e.g. "1711432800000-0").
    pub entry_id: String,
}

impl CommitToken for RedisToken {}

impl std::fmt::Display for RedisToken {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        write!(f, "redis:{}:{}", self.stream, self.entry_id)
    }
}

fn default_url() -> String {
    "redis://127.0.0.1:6379".into()
}

fn default_group() -> String {
    "dfe".into()
}

fn default_consumer() -> String {
    "consumer-1".into()
}

fn default_block_ms() -> usize {
    5000
}

/// Configuration for Redis/Valkey Streams transport.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct RedisTransportConfig {
    /// Redis/Valkey connection URL.
    ///
    /// Supports `redis://`, `rediss://` (TLS), and `unix://` schemes.
    /// Default: `"redis://127.0.0.1:6379"`.
    #[serde(default = "default_url")]
    pub url: String,

    /// Stream name for send/receive.
    ///
    /// Used as default when key is empty in `send()`.
    #[serde(default)]
    pub stream: Option<String>,

    /// Consumer group name. Default: `"dfe"`.
    #[serde(default = "default_group")]
    pub group: String,

    /// Consumer name within group. Default: hostname or `"consumer-1"`.
    #[serde(default = "default_consumer")]
    pub consumer: String,

    /// Maximum stream length (approximate via `MAXLEN ~`).
    ///
    /// `None` means unlimited growth.
    #[serde(default)]
    pub max_stream_len: Option<usize>,

    /// Block timeout in milliseconds for `XREADGROUP`. Default: 5000.
    #[serde(default = "default_block_ms")]
    pub block_ms: usize,

    /// Inbound message filters (applied on recv before caller sees messages).
    #[serde(default)]
    pub filters_in: Vec<super::filter::FilterRule>,

    /// Outbound message filters (applied on send before transport dispatches).
    #[serde(default)]
    pub filters_out: Vec<super::filter::FilterRule>,
}

impl Default for RedisTransportConfig {
    fn default() -> Self {
        Self {
            url: default_url(),
            stream: None,
            group: default_group(),
            consumer: default_consumer(),
            max_stream_len: None,
            block_ms: default_block_ms(),
            filters_in: Vec::new(),
            filters_out: Vec::new(),
        }
    }
}

impl RedisTransportConfig {
    /// Load from the config cascade under the `transport.redis` key.
    #[must_use]
    pub fn from_cascade() -> Self {
        #[cfg(feature = "config")]
        {
            if let Some(cfg) = crate::config::try_get()
                && let Ok(tc) = cfg.unmarshal_key_registered::<Self>("transport.redis")
            {
                return tc;
            }
        }
        Self::default()
    }
}

/// Redis/Valkey Streams transport.
///
/// Supports both send (`XADD`) and receive (`XREADGROUP`) operations.
/// Works with both Redis and Valkey (same wire protocol).
pub struct RedisTransport {
    conn: Mutex<redis::aio::MultiplexedConnection>,
    config: RedisTransportConfig,
    closed: Arc<AtomicBool>,
    /// Whether the consumer group has been ensured for a given stream.
    group_created: Mutex<std::collections::HashSet<String>>,
    /// Transport-level message filter engine.
    filter_engine: super::filter::TransportFilterEngine,
    /// Buffer for messages staged to DLQ by inbound filters.
    /// Drained by `take_filtered_dlq_entries()`.
    filtered_dlq_buffer: parking_lot::Mutex<Vec<super::filter::FilteredDlqEntry>>,
}

impl RedisTransport {
    /// Create a new Redis Streams transport.
    ///
    /// Connects to the Redis server and prepares for stream operations.
    /// The consumer group is created lazily on first `recv()` call.
    ///
    /// # Errors
    ///
    /// Returns error if the URL is invalid or connection fails.
    pub async fn new(config: &RedisTransportConfig) -> TransportResult<Self> {
        let client = redis::Client::open(config.url.as_str()).map_err(|e| {
            TransportError::Config(format!("invalid Redis URL '{}': {e}", config.url))
        })?;

        let conn = client
            .get_multiplexed_async_connection()
            .await
            .map_err(|e| {
                TransportError::Connection(format!(
                    "failed to connect to Redis at '{}': {e}",
                    config.url
                ))
            })?;

        #[cfg(feature = "logger")]
        tracing::info!(
            url = %config.url,
            stream = ?config.stream,
            group = %config.group,
            "Redis transport opened"
        );

        let filter_engine = super::filter::TransportFilterEngine::new(
            &config.filters_in,
            &config.filters_out,
            &crate::transport::filter::TransportFilterTierConfig::default(),
        )?;

        let closed = Arc::new(AtomicBool::new(false));

        #[cfg(feature = "health")]
        {
            let h = Arc::clone(&closed);
            crate::health::HealthRegistry::register("transport:redis", move || {
                if h.load(Ordering::Relaxed) {
                    crate::health::HealthStatus::Unhealthy
                } else {
                    crate::health::HealthStatus::Healthy
                }
            });
        }

        Ok(Self {
            conn: Mutex::new(conn),
            config: config.clone(),
            closed,
            group_created: Mutex::new(std::collections::HashSet::new()),
            filter_engine,
            filtered_dlq_buffer: parking_lot::Mutex::new(Vec::new()),
        })
    }

    /// Resolve the stream name: use `key` if non-empty, else fall back to config.
    fn resolve_stream<'a>(&'a self, key: &'a str) -> Result<&'a str, TransportError> {
        if !key.is_empty() {
            return Ok(key);
        }
        self.config.stream.as_deref().ok_or_else(|| {
            TransportError::Config(
                "no stream name: key is empty and config.stream is not set".into(),
            )
        })
    }

    /// Ensure the consumer group exists for the given stream.
    ///
    /// Uses `XGROUP CREATE ... MKSTREAM` so the stream is created if absent.
    /// Idempotent: tracks which streams have been initialised and only
    /// issues the command once per stream per transport instance.
    async fn ensure_group(&self, stream: &str) -> TransportResult<()> {
        {
            let created = self.group_created.lock().await;
            if created.contains(stream) {
                return Ok(());
            }
        }

        let mut conn = self.conn.lock().await;
        let result: redis::RedisResult<()> = conn
            .xgroup_create_mkstream(stream, &self.config.group, "0")
            .await;

        match result {
            Ok(()) => {}
            Err(e) => {
                // "BUSYGROUP Consumer Group name already exists" is not an error
                let msg = e.to_string();
                if !msg.contains("BUSYGROUP") {
                    return Err(TransportError::Connection(format!(
                        "failed to create consumer group '{}' on stream '{stream}': {e}",
                        self.config.group
                    )));
                }
            }
        }

        self.group_created.lock().await.insert(stream.to_string());
        Ok(())
    }
}

impl TransportBase for RedisTransport {
    async fn close(&self) -> TransportResult<()> {
        self.closed.store(true, Ordering::Relaxed);
        Ok(())
    }

    fn is_healthy(&self) -> bool {
        !self.closed.load(Ordering::Relaxed)
    }

    fn name(&self) -> &'static str {
        "redis"
    }
}

impl TransportSender for RedisTransport {
    async fn send(&self, key: &str, payload: &[u8]) -> SendResult {
        if self.closed.load(Ordering::Relaxed) {
            return SendResult::Fatal(TransportError::Closed);
        }

        // Outbound filter check
        if self.filter_engine.has_outbound_filters() {
            match self.filter_engine.apply_outbound(payload) {
                super::filter::FilterDisposition::Pass => {}
                super::filter::FilterDisposition::Drop => return SendResult::Ok,
                super::filter::FilterDisposition::Dlq => return SendResult::FilteredDlq,
            }
        }

        let stream = match self.resolve_stream(key) {
            Ok(s) => s.to_string(),
            Err(e) => return SendResult::Fatal(e),
        };

        let mut conn = self.conn.lock().await;

        let result: redis::RedisResult<String> = if let Some(max_len) = self.config.max_stream_len {
            conn.xadd_maxlen(
                &stream,
                StreamMaxlen::Approx(max_len),
                "*",
                &[("payload", payload)],
            )
            .await
        } else {
            conn.xadd(&stream, "*", &[("payload", payload)]).await
        };

        match result {
            Ok(_entry_id) => {
                #[cfg(feature = "logger")]
                tracing::debug!(stream = %stream, "Redis transport: XADD sent");

                #[cfg(feature = "metrics")]
                metrics::counter!("dfe_transport_sent_total", "transport" => "redis").increment(1);

                SendResult::Ok
            }
            Err(e) => {
                #[cfg(feature = "logger")]
                tracing::warn!(error = %e, stream = %stream, "Redis transport: XADD error");

                SendResult::Fatal(TransportError::Send(format!(
                    "XADD to stream '{stream}' failed: {e}"
                )))
            }
        }
    }
}

impl TransportReceiver for RedisTransport {
    type Token = RedisToken;

    async fn recv(&self, max: usize) -> TransportResult<Vec<Message<Self::Token>>> {
        if self.closed.load(Ordering::Relaxed) {
            return Err(TransportError::Closed);
        }

        let stream_name = self
            .config
            .stream
            .as_deref()
            .ok_or_else(|| TransportError::Config("config.stream must be set for recv()".into()))?
            .to_string();

        self.ensure_group(&stream_name).await?;

        let opts = StreamReadOptions::default()
            .group(&self.config.group, &self.config.consumer)
            .count(max)
            .block(self.config.block_ms);

        let mut conn = self.conn.lock().await;

        // ">" means only new (undelivered) messages
        let reply: StreamReadReply = conn
            .xread_options(&[&stream_name], &[">"], &opts)
            .await
            .map_err(|e| {
                #[cfg(feature = "logger")]
                tracing::warn!(error = %e, stream = %stream_name, "Redis transport: XREADGROUP error");

                TransportError::Recv(format!("XREADGROUP on stream '{stream_name}' failed: {e}"))
            })?;

        let stream_arc: Arc<str> = Arc::from(stream_name.as_str());
        let mut messages = Vec::new();

        for stream_key in &reply.keys {
            for stream_id in &stream_key.ids {
                // Extract the "payload" field from the entry
                let payload_bytes: Option<Vec<u8>> = stream_id
                    .map
                    .get("payload")
                    .and_then(|v| redis::from_redis_value(v.clone()).ok());

                let payload = payload_bytes.unwrap_or_default();
                let format = PayloadFormat::detect(&payload);
                let timestamp_ms = parse_entry_timestamp(&stream_id.id);

                messages.push(Message {
                    key: Some(Arc::clone(&stream_arc)),
                    payload,
                    token: RedisToken {
                        stream: Arc::clone(&stream_arc),
                        entry_id: stream_id.id.clone(),
                    },
                    timestamp_ms,
                    format,
                });
            }
        }

        // Apply inbound filters: drop messages, stage DLQ entries
        if self.filter_engine.has_inbound_filters() {
            let mut staged_dlq: Vec<super::filter::FilteredDlqEntry> = Vec::new();
            messages.retain(|msg| match self.filter_engine.apply_inbound(&msg.payload) {
                super::filter::FilterDisposition::Pass => true,
                super::filter::FilterDisposition::Drop => false,
                super::filter::FilterDisposition::Dlq => {
                    staged_dlq.push(super::filter::FilteredDlqEntry {
                        payload: msg.payload.clone(),
                        key: msg.key.clone(),
                        reason: "transport filter".to_string(),
                    });
                    false
                }
            });
            if !staged_dlq.is_empty() {
                self.filtered_dlq_buffer.lock().extend(staged_dlq);
            }
        }

        #[cfg(feature = "logger")]
        if !messages.is_empty() {
            tracing::debug!(
                messages = messages.len(),
                "Redis transport: XREADGROUP received"
            );
        }

        #[cfg(feature = "metrics")]
        if !messages.is_empty() {
            metrics::counter!("dfe_transport_received_total", "transport" => "redis")
                .increment(messages.len() as u64);
        }

        Ok(messages)
    }

    fn take_filtered_dlq_entries(&self) -> Vec<super::filter::FilteredDlqEntry> {
        std::mem::take(&mut *self.filtered_dlq_buffer.lock())
    }

    async fn commit(&self, tokens: &[Self::Token]) -> TransportResult<()> {
        if tokens.is_empty() {
            return Ok(());
        }

        // Group tokens by stream name for batch XACK
        let mut by_stream: std::collections::HashMap<&str, Vec<&str>> =
            std::collections::HashMap::new();
        for token in tokens {
            by_stream
                .entry(&token.stream)
                .or_default()
                .push(&token.entry_id);
        }

        let mut conn = self.conn.lock().await;

        for (stream, ids) in &by_stream {
            let id_refs: &[&str] = ids;
            let _acked: i32 = conn
                .xack(*stream, &self.config.group, id_refs)
                .await
                .map_err(|e| {
                    #[cfg(feature = "logger")]
                    tracing::warn!(error = %e, stream = %stream, "Redis transport: XACK error");

                    TransportError::Commit(format!("XACK on stream '{stream}' failed: {e}"))
                })?;
        }

        #[cfg(feature = "logger")]
        tracing::debug!(count = tokens.len(), "Redis transport: XACK committed");

        Ok(())
    }
}

/// Parse millisecond timestamp from a Redis stream entry ID.
///
/// Entry IDs have the format `<millisecondsTime>-<sequenceNumber>`.
fn parse_entry_timestamp(entry_id: &str) -> Option<i64> {
    entry_id
        .split_once('-')
        .and_then(|(ms_str, _)| ms_str.parse::<i64>().ok())
}

#[cfg(test)]
mod tests {
    use super::*;

    #[test]
    fn token_display() {
        let token = RedisToken {
            stream: Arc::from("my_stream"),
            entry_id: "1711432800000-0".into(),
        };
        assert_eq!(format!("{token}"), "redis:my_stream:1711432800000-0");
    }

    #[test]
    fn token_clone() {
        let token = RedisToken {
            stream: Arc::from("s1"),
            entry_id: "100-0".into(),
        };
        let cloned = token.clone();
        assert_eq!(token, cloned);
    }

    #[test]
    fn config_defaults() {
        let config = RedisTransportConfig::default();
        assert_eq!(config.url, "redis://127.0.0.1:6379");
        assert!(config.stream.is_none());
        assert_eq!(config.group, "dfe");
        assert!(config.max_stream_len.is_none());
        assert_eq!(config.block_ms, 5000);
    }

    #[test]
    fn config_deserialise_minimal() {
        let yaml = r"
url: redis://myhost:6380
stream: events
";
        let config: RedisTransportConfig = serde_yaml_ng::from_str(yaml).unwrap();
        assert_eq!(config.url, "redis://myhost:6380");
        assert_eq!(config.stream.as_deref(), Some("events"));
        // Defaults should be applied
        assert_eq!(config.group, "dfe");
        assert_eq!(config.block_ms, 5000);
    }

    #[test]
    fn config_deserialise_full() {
        let yaml = r"
url: rediss://secure.redis.io:6380
stream: audit_log
group: my_group
consumer: worker-3
max_stream_len: 100000
block_ms: 2000
";
        let config: RedisTransportConfig = serde_yaml_ng::from_str(yaml).unwrap();
        assert_eq!(config.url, "rediss://secure.redis.io:6380");
        assert_eq!(config.stream.as_deref(), Some("audit_log"));
        assert_eq!(config.group, "my_group");
        assert_eq!(config.consumer, "worker-3");
        assert_eq!(config.max_stream_len, Some(100_000));
        assert_eq!(config.block_ms, 2000);
    }

    #[test]
    fn parse_entry_timestamp_valid() {
        assert_eq!(
            parse_entry_timestamp("1711432800000-0"),
            Some(1_711_432_800_000)
        );
        assert_eq!(parse_entry_timestamp("0-0"), Some(0));
    }

    #[test]
    fn parse_entry_timestamp_invalid() {
        assert_eq!(parse_entry_timestamp("not-a-number"), None);
        assert_eq!(parse_entry_timestamp(""), None);
    }

    #[test]
    fn resolve_stream_uses_key_when_non_empty() {
        let config = RedisTransportConfig {
            stream: Some("default_stream".into()),
            ..Default::default()
        };
        // Cannot call resolve_stream without a transport instance, so test
        // the logic inline: non-empty key takes precedence.
        let key = "override_stream";
        let resolved = if key.is_empty() {
            config.stream.as_deref().unwrap_or("")
        } else {
            key
        };
        assert_eq!(resolved, "override_stream");
    }

    #[test]
    fn resolve_stream_falls_back_to_config() {
        let config = RedisTransportConfig {
            stream: Some("default_stream".into()),
            ..Default::default()
        };
        let key = "";
        let resolved = if key.is_empty() {
            config.stream.as_deref().unwrap_or("")
        } else {
            key
        };
        assert_eq!(resolved, "default_stream");
    }

    // Integration test: requires a running Redis instance.
    // Run with: REDIS_URL=redis://localhost:6379 cargo nextest run redis_integration
    #[tokio::test]
    async fn redis_integration_xadd_xreadgroup_xack() {
        let Ok(url) = std::env::var("REDIS_URL") else {
            eprintln!("Skipping: REDIS_URL not set");
            return;
        };

        let stream = format!("test_stream_{}", chrono::Utc::now().timestamp_millis());
        let group = "test_group";
        let consumer = "test_consumer";

        let config = RedisTransportConfig {
            url: url.clone(),
            stream: Some(stream.clone()),
            group: group.into(),
            consumer: consumer.into(),
            max_stream_len: Some(1000),
            block_ms: 1000,
            ..Default::default()
        };

        let transport = RedisTransport::new(&config).await.unwrap();

        // Send two messages
        let r1 = transport.send("", b"{\"n\":1}").await;
        assert!(r1.is_ok(), "first send should succeed");

        let r2 = transport.send("", b"{\"n\":2}").await;
        assert!(r2.is_ok(), "second send should succeed");

        // Receive messages
        let messages = transport.recv(10).await.unwrap();
        assert_eq!(messages.len(), 2, "should receive 2 messages");
        assert_eq!(messages[0].payload, b"{\"n\":1}");
        assert_eq!(messages[1].payload, b"{\"n\":2}");

        // Commit (XACK)
        let tokens: Vec<_> = messages.iter().map(|m| m.token.clone()).collect();
        transport.commit(&tokens).await.unwrap();

        // After commit, no new messages should be available
        let more = transport.recv(10).await.unwrap();
        assert!(more.is_empty(), "no more messages after commit");

        // Clean up: delete the test stream
        let mut conn = transport.conn.lock().await;
        let _: redis::RedisResult<()> =
            redis::cmd("DEL").arg(&stream).query_async(&mut *conn).await;

        transport.close().await.unwrap();
        assert!(!transport.is_healthy());
    }
}