laminar-core 0.26.0

Core streaming engine for LaminarDB - operators, checkpoint barriers, and streaming primitives
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
//! Source — entry point for data into a streaming pipeline.

use std::sync::atomic::{AtomicI64, AtomicU64, Ordering};
use std::sync::{Arc, OnceLock};
use std::time::Duration;

use arrow::array::RecordBatch;
use arrow::datatypes::SchemaRef;

use super::channel::{channel_with_config, Producer};
use super::config::SourceConfig;
use super::error::{StreamingError, TryPushError};
use super::sink::Sink;

/// Trait for types that can be streamed through a Source.
pub trait Record: Clone + Send + Sized + 'static {
    /// Returns the Arrow schema for this record type.
    fn schema() -> SchemaRef;

    /// Converts this record to an Arrow `RecordBatch`.
    ///
    /// The batch will contain a single row with this record's data.
    fn to_record_batch(&self) -> RecordBatch;

    /// Returns the event time for this record, if applicable.
    ///
    /// Event time is used for watermark generation and window assignment.
    /// Returns `None` if the record doesn't have an event time.
    fn event_time(&self) -> Option<i64> {
        None
    }

    /// Converts a batch of records to an Arrow `RecordBatch`.
    ///
    /// The default implementation converts each record individually and concatenates them.
    /// Derived implementations can override this to optimize allocation and copying.
    fn to_record_batch_from_iter<I>(records: I) -> RecordBatch
    where
        I: IntoIterator<Item = Self>,
    {
        let batches: Vec<RecordBatch> = records.into_iter().map(|r| r.to_record_batch()).collect();
        if batches.is_empty() {
            return RecordBatch::new_empty(Self::schema());
        }
        arrow::compute::concat_batches(&Self::schema(), &batches)
            .unwrap_or_else(|_| RecordBatch::new_empty(Self::schema()))
    }
}

/// Internal message type that wraps records and control signals.
#[derive(Clone)]
pub(crate) enum SourceMessage<T> {
    /// A data record.
    Record(T),

    /// A batch of Arrow records.
    Batch(RecordBatch),
}

/// Shared state for watermark tracking.
struct SourceWatermark {
    /// Current watermark value.
    /// Atomically updated to support multi-producer scenarios.
    /// Wrapped in `Arc` so the checkpoint manager can read it without locking.
    current: Arc<AtomicI64>,
}

impl SourceWatermark {
    fn new() -> Self {
        Self {
            current: Arc::new(AtomicI64::new(i64::MIN)),
        }
    }

    fn from_arc(arc: Arc<AtomicI64>) -> Self {
        Self { current: arc }
    }

    fn update(&self, timestamp: i64) {
        // Only advance watermark, never go backwards
        let mut current = self.current.load(Ordering::Acquire);
        while timestamp > current {
            match self.current.compare_exchange_weak(
                current,
                timestamp,
                Ordering::AcqRel,
                Ordering::Acquire,
            ) {
                Ok(_) => break,
                Err(actual) => current = actual,
            }
        }
    }

    fn get(&self) -> i64 {
        self.current.load(Ordering::Acquire)
    }

    fn arc(&self) -> Arc<AtomicI64> {
        Arc::clone(&self.current)
    }
}

/// Shared state for a Source/Sink pair.
struct SourceInner<T: Record> {
    /// Channel producer for sending records.
    producer: Producer<SourceMessage<T>>,

    /// Watermark state.
    watermark: SourceWatermark,

    /// Schema for type validation.
    schema: SchemaRef,

    /// Source name (for debugging/metrics).
    name: Option<String>,

    /// Monotonic sequence counter, incremented on each successful push.
    /// Wrapped in `Arc` so the checkpoint manager can read it without locking.
    sequence: Arc<AtomicU64>,

    /// Event-time column name set via programmatic API.
    /// Read once at pipeline startup, not on the hot path.
    event_time_column: OnceLock<String>,

    /// Max out-of-orderness bound, paired with `event_time_column`.
    /// Read once at pipeline startup, not on the hot path.
    max_out_of_orderness: OnceLock<Duration>,
}

/// A streaming data source. Cloneable for multi-producer use.
pub struct Source<T: Record> {
    inner: Arc<SourceInner<T>>,
}

impl<T: Record> Source<T> {
    /// Creates a new Source/Sink pair.
    pub(crate) fn new(config: SourceConfig) -> (Self, Sink<T>) {
        let channel_config = config.channel;
        let (producer, consumer) = channel_with_config::<SourceMessage<T>>(&channel_config);

        let schema = T::schema();

        let inner = Arc::new(SourceInner {
            producer,
            watermark: SourceWatermark::new(),
            schema: schema.clone(),
            name: config.name,
            sequence: Arc::new(AtomicU64::new(0)),
            event_time_column: OnceLock::new(),
            max_out_of_orderness: OnceLock::new(),
        });

        let source = Self { inner };
        let sink = Sink::new(consumer, schema);

        (source, sink)
    }

    /// Pushes a record. Non-blocking — returns `ChannelFull` if the buffer is full.
    ///
    /// # Errors
    ///
    /// Returns `StreamingError::ChannelFull` if the buffer is full or the sink was dropped.
    pub fn push(&self, record: T) -> Result<(), StreamingError> {
        if let Some(event_time) = record.event_time() {
            self.inner.watermark.update(event_time);
        }

        self.inner
            .producer
            .push(SourceMessage::Record(record))
            .map_err(|_| StreamingError::ChannelFull)?;

        self.inner.sequence.fetch_add(1, Ordering::Relaxed);
        Ok(())
    }

    /// Pushes a record, returning it on failure.
    ///
    /// # Errors
    ///
    /// Returns `TryPushError` containing the record if the channel is full.
    pub fn try_push(&self, record: T) -> Result<(), TryPushError<T>> {
        if let Some(event_time) = record.event_time() {
            self.inner.watermark.update(event_time);
        }

        self.inner
            .producer
            .push(SourceMessage::Record(record))
            .map_err(|msg| match msg {
                SourceMessage::Record(r) => TryPushError {
                    value: r,
                    error: StreamingError::ChannelFull,
                },
                SourceMessage::Batch(_) => unreachable!("only Record is pushed here"),
            })?;

        self.inner.sequence.fetch_add(1, Ordering::Relaxed);
        Ok(())
    }

    /// Pushes multiple records (cloned). Stops at the first failure.
    pub fn push_batch(&self, records: &[T]) -> usize
    where
        T: Clone,
    {
        self.push_batch_drain(records.iter().cloned())
    }

    /// Pushes records from an iterator, consuming them (zero-clone).
    /// Stops at the first failure. Returns the number pushed.
    pub fn push_batch_drain<I>(&self, records: I) -> usize
    where
        I: IntoIterator<Item = T>,
    {
        let mut count = 0;
        for record in records {
            if self.push(record).is_err() {
                break;
            }
            count += 1;
        }
        count
    }

    /// Pushes an Arrow `RecordBatch` directly.
    ///
    /// This is more efficient than pushing individual records when you
    /// already have data in Arrow format.
    ///
    /// # Errors
    ///
    /// Returns `StreamingError::SchemaMismatch` if the batch schema doesn't match.
    /// Returns `StreamingError::ChannelClosed` if the sink has been dropped.
    pub fn push_arrow(&self, batch: RecordBatch) -> Result<(), StreamingError> {
        // Validate schema matches (skip for type-erased sources with empty schema)
        if !self.inner.schema.fields().is_empty() && batch.schema() != self.inner.schema {
            return Err(StreamingError::SchemaMismatch {
                expected: self
                    .inner
                    .schema
                    .fields()
                    .iter()
                    .map(|f| f.name().clone())
                    .collect(),
                actual: batch
                    .schema()
                    .fields()
                    .iter()
                    .map(|f| f.name().clone())
                    .collect(),
            });
        }

        self.inner
            .producer
            .push(SourceMessage::Batch(batch))
            .map_err(|_| StreamingError::ChannelFull)?;

        self.inner.sequence.fetch_add(1, Ordering::Relaxed);
        Ok(())
    }

    /// Emits a watermark timestamp.
    ///
    /// Watermarks signal that no events with timestamps less than or equal
    /// to this value will arrive in the future. This enables window triggers
    /// and garbage collection.
    ///
    /// Watermarks are monotonically increasing - if a lower timestamp is
    /// passed, it will be ignored.
    pub fn watermark(&self, timestamp: i64) {
        // The shared atomic is the authoritative watermark: the pipeline's
        // watermark UDF, late-row filter, and checkpoint registration all
        // read it via `watermark_atomic()`. Subscribers receive data only,
        // so there is no in-band watermark message to emit.
        self.inner.watermark.update(timestamp);
    }

    /// Returns the current watermark value.
    #[must_use]
    pub fn current_watermark(&self) -> i64 {
        self.inner.watermark.get()
    }

    /// Returns the schema for this source.
    #[must_use]
    pub fn schema(&self) -> SchemaRef {
        Arc::clone(&self.inner.schema)
    }

    /// Returns the source name, if configured.
    #[must_use]
    pub fn name(&self) -> Option<&str> {
        self.inner.name.as_deref()
    }

    /// Returns true if the sink has been dropped.
    #[must_use]
    pub fn is_closed(&self) -> bool {
        self.inner.producer.is_closed()
    }

    /// Returns the number of pending items in the buffer.
    #[must_use]
    pub fn pending(&self) -> usize {
        self.inner.producer.len()
    }

    /// Returns the buffer capacity.
    #[must_use]
    pub fn capacity(&self) -> usize {
        self.inner.producer.capacity()
    }

    /// Returns the current sequence number (total successful pushes).
    #[must_use]
    pub fn sequence(&self) -> u64 {
        self.inner.sequence.load(Ordering::Acquire)
    }

    /// Returns the shared sequence counter for checkpoint registration.
    #[must_use]
    pub fn sequence_counter(&self) -> Arc<AtomicU64> {
        Arc::clone(&self.inner.sequence)
    }

    /// Returns the shared watermark atomic for checkpoint registration.
    #[must_use]
    pub fn watermark_atomic(&self) -> Arc<AtomicI64> {
        self.inner.watermark.arc()
    }

    /// Declare which column in the source data represents event time.
    ///
    /// When set, `source.watermark()` enables late-row filtering
    /// without a SQL `WATERMARK FOR` clause.
    ///
    /// Only the first call takes effect; subsequent calls are silently ignored.
    pub fn set_event_time_column(&self, column: &str) {
        let _ = self.inner.event_time_column.set(column.to_owned());
    }

    /// Returns the configured event-time column, if any.
    #[must_use]
    pub fn event_time_column(&self) -> Option<String> {
        self.inner.event_time_column.get().cloned()
    }

    /// Set the max out-of-orderness bound for watermark generation.
    ///
    /// Only the first call takes effect; subsequent calls are silently ignored.
    pub fn set_max_out_of_orderness(&self, dur: Duration) {
        let _ = self.inner.max_out_of_orderness.set(dur);
    }

    /// Returns the configured max out-of-orderness, if any.
    #[must_use]
    pub fn max_out_of_orderness(&self) -> Option<Duration> {
        self.inner.max_out_of_orderness.get().copied()
    }
}

impl<T: Record> Clone for Source<T> {
    fn clone(&self) -> Self {
        let producer = self.inner.producer.clone();
        let event_time_col = self.inner.event_time_column.get().cloned();
        let event_time_column = OnceLock::new();
        if let Some(col) = event_time_col {
            let _ = event_time_column.set(col);
        }
        let max_ooo = self.inner.max_out_of_orderness.get().copied();
        let max_out_of_orderness = OnceLock::new();
        if let Some(dur) = max_ooo {
            let _ = max_out_of_orderness.set(dur);
        }
        Self {
            inner: Arc::new(SourceInner {
                producer,
                watermark: SourceWatermark::from_arc(self.inner.watermark.arc()),
                schema: Arc::clone(&self.inner.schema),
                name: self.inner.name.clone(),
                sequence: Arc::clone(&self.inner.sequence),
                event_time_column,
                max_out_of_orderness,
            }),
        }
    }
}

impl<T: Record + std::fmt::Debug> std::fmt::Debug for Source<T> {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("Source")
            .field("name", &self.inner.name)
            .field("pending", &self.pending())
            .field("capacity", &self.capacity())
            .field("watermark", &self.current_watermark())
            .finish()
    }
}

/// Creates a new Source/Sink pair with the given buffer size.
#[must_use]
pub fn create<T: Record>(buffer_size: usize) -> (Source<T>, Sink<T>) {
    Source::new(SourceConfig::with_buffer_size(buffer_size))
}

/// Creates a new Source/Sink pair with custom configuration.
#[must_use]
pub fn create_with_config<T: Record>(config: SourceConfig) -> (Source<T>, Sink<T>) {
    Source::new(config)
}

#[cfg(test)]
mod tests {
    use super::*;
    use arrow::array::{Float64Array, Int64Array, StringArray};
    use arrow::datatypes::{DataType, Field, Schema};
    use std::sync::Arc;

    // Test record type
    #[derive(Clone, Debug)]
    struct TestEvent {
        id: i64,
        value: f64,
        timestamp: i64,
    }

    impl Record for TestEvent {
        fn schema() -> SchemaRef {
            Arc::new(Schema::new(vec![
                Field::new("id", DataType::Int64, false),
                Field::new("value", DataType::Float64, false),
                Field::new("timestamp", DataType::Int64, false),
            ]))
        }

        fn to_record_batch(&self) -> RecordBatch {
            RecordBatch::try_new(
                Self::schema(),
                vec![
                    Arc::new(Int64Array::from(vec![self.id])),
                    Arc::new(Float64Array::from(vec![self.value])),
                    Arc::new(Int64Array::from(vec![self.timestamp])),
                ],
            )
            .unwrap()
        }

        fn event_time(&self) -> Option<i64> {
            Some(self.timestamp)
        }
    }

    #[tokio::test]
    async fn test_create_source_sink() {
        let (source, _sink) = create::<TestEvent>(1024);

        assert!(!source.is_closed());
        assert_eq!(source.pending(), 0);
    }

    #[tokio::test]
    async fn test_push_single() {
        let (source, _sink) = create::<TestEvent>(16);

        let event = TestEvent {
            id: 1,
            value: 42.0,
            timestamp: 1000,
        };

        assert!(source.push(event).is_ok());
        assert_eq!(source.pending(), 1);
    }

    #[tokio::test]
    async fn test_try_push() {
        let (source, _sink) = create::<TestEvent>(16);

        let event = TestEvent {
            id: 1,
            value: 42.0,
            timestamp: 1000,
        };

        assert!(source.try_push(event).is_ok());
    }

    #[tokio::test]
    async fn test_push_batch() {
        let (source, _sink) = create::<TestEvent>(16);

        let events = vec![
            TestEvent {
                id: 1,
                value: 1.0,
                timestamp: 1000,
            },
            TestEvent {
                id: 2,
                value: 2.0,
                timestamp: 2000,
            },
            TestEvent {
                id: 3,
                value: 3.0,
                timestamp: 3000,
            },
        ];

        let count = source.push_batch(&events);
        assert_eq!(count, 3);
        assert_eq!(source.pending(), 3);
    }

    #[tokio::test]
    async fn test_push_arrow() {
        let (source, _sink) = create::<TestEvent>(16);

        let batch = RecordBatch::try_new(
            TestEvent::schema(),
            vec![
                Arc::new(Int64Array::from(vec![1, 2, 3])),
                Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0])),
                Arc::new(Int64Array::from(vec![1000, 2000, 3000])),
            ],
        )
        .unwrap();

        assert!(source.push_arrow(batch).is_ok());
    }

    #[tokio::test]
    async fn test_push_arrow_schema_mismatch() {
        let (source, _sink) = create::<TestEvent>(16);

        // Create batch with different schema
        let wrong_schema = Arc::new(Schema::new(vec![Field::new(
            "wrong",
            DataType::Utf8,
            false,
        )]));

        let batch = RecordBatch::try_new(
            wrong_schema,
            vec![Arc::new(StringArray::from(vec!["test"]))],
        )
        .unwrap();

        let result = source.push_arrow(batch);
        assert!(matches!(result, Err(StreamingError::SchemaMismatch { .. })));
    }

    #[tokio::test]
    async fn test_watermark() {
        let (source, _sink) = create::<TestEvent>(16);

        assert_eq!(source.current_watermark(), i64::MIN);

        source.watermark(1000);
        assert_eq!(source.current_watermark(), 1000);

        source.watermark(2000);
        assert_eq!(source.current_watermark(), 2000);

        // Watermark should not go backwards
        source.watermark(1500);
        assert_eq!(source.current_watermark(), 2000);
    }

    #[tokio::test]
    async fn test_watermark_from_event_time() {
        let (source, _sink) = create::<TestEvent>(16);

        let event = TestEvent {
            id: 1,
            value: 42.0,
            timestamp: 5000,
        };

        source.push(event).unwrap();

        // Watermark should be updated from event time
        assert_eq!(source.current_watermark(), 5000);
    }

    #[tokio::test]
    async fn test_clone_multi_producer() {
        let (source, sink) = create::<TestEvent>(16);
        let source2 = source.clone();
        let mut sub = sink.subscribe(); // subscribe before push

        source
            .push(TestEvent {
                id: 1,
                value: 1.0,
                timestamp: 1000,
            })
            .unwrap();
        source2
            .push(TestEvent {
                id: 2,
                value: 2.0,
                timestamp: 2000,
            })
            .unwrap();

        tokio::time::sleep(std::time::Duration::from_millis(10)).await;
        assert!(sub.poll().is_some());
        assert!(sub.poll().is_some());
    }

    #[tokio::test]
    async fn test_schema() {
        let (source, _sink) = create::<TestEvent>(16);

        let schema = source.schema();
        assert_eq!(schema.fields().len(), 3);
        assert_eq!(schema.field(0).name(), "id");
        assert_eq!(schema.field(1).name(), "value");
        assert_eq!(schema.field(2).name(), "timestamp");
    }

    #[tokio::test]
    async fn test_named_source() {
        let config = SourceConfig::named("my_source");
        let (source, _sink) = create_with_config::<TestEvent>(config);

        assert_eq!(source.name(), Some("my_source"));
    }

    #[tokio::test]
    async fn test_debug_format() {
        let (source, _sink) = create::<TestEvent>(16);

        let debug = format!("{source:?}");
        assert!(debug.contains("Source"));
    }

    #[tokio::test]
    async fn test_set_event_time_column() {
        let (source, _sink) = create::<TestEvent>(16);

        assert!(source.event_time_column().is_none());

        source.set_event_time_column("timestamp");
        assert_eq!(source.event_time_column(), Some("timestamp".to_string()));
    }

    #[tokio::test]
    async fn test_event_time_column_preserved_on_clone() {
        let (source, _sink) = create::<TestEvent>(16);
        source.set_event_time_column("ts");

        let source2 = source.clone();
        assert_eq!(source2.event_time_column(), Some("ts".to_string()));
    }
}