drasi-lib 0.6.0

Embedded Drasi for in-process data change processing using continuous queries
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
// Copyright 2025 The Drasi Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use crate::profiling::ProfilingMetadata;
use drasi_core::models::SourceChange;
use serde::{Deserialize, Serialize};
use std::collections::HashMap;
use std::sync::atomic::AtomicU64;
use std::sync::Arc;
use tokio::sync::{broadcast, mpsc};

/// Trait for types that have a timestamp, required for priority queue ordering
pub trait Timestamped {
    fn timestamp(&self) -> chrono::DateTime<chrono::Utc>;
}

/// Type of Drasi component
///
/// Used for identifying component types in events and monitoring.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Hash)]
pub enum ComponentType {
    Source,
    Query,
    Reaction,
    BootstrapProvider,
    IdentityProvider,
}

/// Execution status of a Drasi component
///
/// `ComponentStatus` represents the current lifecycle state of sources, queries, and reactions.
/// Components transition through these states during their lifecycle, from creation through
/// execution to shutdown.
///
/// # Status Lifecycle
///
/// A typical component lifecycle follows this progression:
///
/// ```text
/// Added → Starting → Running → Stopping → Stopped
///              ↓                              ↓
///            Error                         Removed
/// ```
///
/// # Status Values
///
/// - **Added**: Component has been registered in the graph but not yet started
/// - **Starting**: Component is initializing (connecting to resources, loading data, etc.)
/// - **Running**: Component is actively processing (ingesting, querying, or delivering)
/// - **Stopping**: Component is shutting down gracefully
/// - **Stopped**: Component is not running (stopped after previously running)
/// - **Removed**: Component has been removed from the graph
/// - **Error**: Component encountered an error and cannot continue (see error_message)
///
/// # Usage
///
/// Status is available through runtime information methods on [`DrasiLib`](crate::DrasiLib):
///
/// - [`get_source_status()`](crate::DrasiLib::get_source_status)
/// - [`get_query_status()`](crate::DrasiLib::get_query_status)
/// - [`get_reaction_status()`](crate::DrasiLib::get_reaction_status)
///
/// And through runtime info structs:
///
/// - [`SourceRuntime`](crate::SourceRuntime)
/// - [`QueryRuntime`](crate::QueryRuntime)
/// - [`ReactionRuntime`](crate::ReactionRuntime)
///
/// # Examples
///
/// ## Monitoring Component Status
///
/// ```no_run
/// use drasi_lib::{DrasiLib, ComponentStatus};
///
/// # async fn example() -> Result<(), Box<dyn std::error::Error>> {
/// let core = DrasiLib::builder().with_id("my-server").build().await?;
/// core.start().await?;
///
/// // Check source status
/// let source_status = core.get_source_status("orders_db").await?;
/// match source_status {
///     ComponentStatus::Running => println!("Source is running"),
///     ComponentStatus::Error => println!("Source has errors"),
///     ComponentStatus::Starting => println!("Source is starting up"),
///     _ => println!("Source status: {:?}", source_status),
/// }
///
/// // Get detailed info with status
/// let source_info = core.get_source_info("orders_db").await?;
/// if source_info.status == ComponentStatus::Error {
///     if let Some(error) = source_info.error_message {
///         eprintln!("Error: {}", error);
///     }
/// }
/// # Ok(())
/// # }
/// ```
///
/// ## Waiting for Component to Start
///
/// ```no_run
/// use drasi_lib::{DrasiLib, ComponentStatus};
/// use tokio::time::{sleep, Duration};
///
/// # async fn example() -> Result<(), Box<dyn std::error::Error>> {
/// let core = DrasiLib::builder().with_id("my-server").build().await?;
/// core.start_source("orders_db").await?;
///
/// // Poll until source is running
/// loop {
///     let status = core.get_source_status("orders_db").await?;
///     match status {
///         ComponentStatus::Running => break,
///         ComponentStatus::Error => return Err("Source failed to start".into()),
///         _ => sleep(Duration::from_millis(100)).await,
///     }
/// }
/// println!("Source is now running");
/// # Ok(())
/// # }
/// ```
///
/// ## Checking All Components
///
/// ```no_run
/// use drasi_lib::{DrasiLib, ComponentStatus};
///
/// # async fn example() -> Result<(), Box<dyn std::error::Error>> {
/// let core = DrasiLib::builder().with_id("my-server").build().await?;
/// core.start().await?;
///
/// // Check all sources
/// let sources = core.list_sources().await?;
/// for (id, status) in sources {
///     println!("Source {}: {:?}", id, status);
/// }
///
/// // Check all queries
/// let queries = core.list_queries().await?;
/// for (id, status) in queries {
///     println!("Query {}: {:?}", id, status);
/// }
///
/// // Check all reactions
/// let reactions = core.list_reactions().await?;
/// for (id, status) in reactions {
///     println!("Reaction {}: {:?}", id, status);
/// }
/// # Ok(())
/// # }
/// ```
#[derive(Debug, Clone, Copy, Serialize, Deserialize, PartialEq, Eq, Hash)]
pub enum ComponentStatus {
    /// Component has been registered in the graph but not yet started.
    Added,
    Starting,
    Running,
    Stopping,
    Stopped,
    /// Component has been removed from the graph.
    Removed,
    Reconfiguring,
    Error,
}

/// A source change event with metadata for dispatching to queries.
#[derive(Debug, Clone)]
pub struct SourceChangeEvent {
    pub source_id: String,
    pub change: SourceChange,
    pub timestamp: chrono::DateTime<chrono::Utc>,
}

/// Control events from sources for query coordination
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub enum SourceControl {
    /// Query subscription control event
    Subscription {
        query_id: String,
        query_node_id: String,
        node_labels: Vec<String>,
        rel_labels: Vec<String>,
        operation: ControlOperation,
    },
    /// Signal from FutureQueueSource that one or more future items are due.
    FuturesDue,
}

/// Control operation types
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub enum ControlOperation {
    Insert,
    Update,
    Delete,
}

/// Unified event envelope carrying both data changes and control messages
#[derive(Debug, Clone)]
pub enum SourceEvent {
    /// Data change event from source
    Change(SourceChange),
    /// Control event for query coordination
    Control(SourceControl),
}

/// Wrapper for source events with metadata
#[derive(Debug, Clone)]
pub struct SourceEventWrapper {
    pub source_id: String,
    pub event: SourceEvent,
    pub timestamp: chrono::DateTime<chrono::Utc>,
    /// Optional profiling metadata for performance tracking
    pub profiling: Option<ProfilingMetadata>,
    /// Monotonic, replayable sequence number stamped by the source.
    /// `None` for volatile sources that don't support replay.
    /// When present, must be strictly increasing per source.
    pub sequence: Option<u64>,
}

impl SourceEventWrapper {
    /// Create a new SourceEventWrapper without profiling
    pub fn new(
        source_id: String,
        event: SourceEvent,
        timestamp: chrono::DateTime<chrono::Utc>,
    ) -> Self {
        Self {
            source_id,
            event,
            timestamp,
            profiling: None,
            sequence: None,
        }
    }

    /// Create a new SourceEventWrapper with profiling metadata
    pub fn with_profiling(
        source_id: String,
        event: SourceEvent,
        timestamp: chrono::DateTime<chrono::Utc>,
        profiling: ProfilingMetadata,
    ) -> Self {
        Self {
            source_id,
            event,
            timestamp,
            profiling: Some(profiling),
            sequence: None,
        }
    }

    /// Create a new SourceEventWrapper with a sequence number (and optional profiling)
    pub fn with_sequence(
        source_id: String,
        event: SourceEvent,
        timestamp: chrono::DateTime<chrono::Utc>,
        sequence: u64,
        profiling: Option<ProfilingMetadata>,
    ) -> Self {
        Self {
            source_id,
            event,
            timestamp,
            profiling,
            sequence: Some(sequence),
        }
    }

    /// Consume this wrapper and return its components.
    /// This enables zero-copy extraction when the wrapper has sole ownership.
    pub fn into_parts(
        self,
    ) -> (
        String,
        SourceEvent,
        chrono::DateTime<chrono::Utc>,
        Option<ProfilingMetadata>,
        Option<u64>,
    ) {
        (
            self.source_id,
            self.event,
            self.timestamp,
            self.profiling,
            self.sequence,
        )
    }

    /// Try to extract components from an Arc<SourceEventWrapper>.
    /// Uses Arc::try_unwrap to avoid cloning when we have sole ownership.
    /// Returns Ok with owned components if sole owner, Err with Arc back if shared.
    ///
    /// This enables zero-copy in Channel dispatch mode (single consumer per event)
    /// while still working correctly in Broadcast mode (cloning required).
    pub fn try_unwrap_arc(
        arc_self: Arc<Self>,
    ) -> Result<
        (
            String,
            SourceEvent,
            chrono::DateTime<chrono::Utc>,
            Option<ProfilingMetadata>,
            Option<u64>,
        ),
        Arc<Self>,
    > {
        Arc::try_unwrap(arc_self).map(|wrapper| wrapper.into_parts())
    }
}

// Implement Timestamped for SourceEventWrapper for use in generic priority queue
impl Timestamped for SourceEventWrapper {
    fn timestamp(&self) -> chrono::DateTime<chrono::Utc> {
        self.timestamp
    }
}

/// Arc-wrapped SourceEventWrapper for zero-copy distribution
pub type ArcSourceEvent = Arc<SourceEventWrapper>;

/// Bootstrap event wrapper for dedicated bootstrap channels
#[derive(Debug, Clone)]
pub struct BootstrapEvent {
    pub source_id: String,
    pub change: SourceChange,
    pub timestamp: chrono::DateTime<chrono::Utc>,
    pub sequence: u64,
}

/// Subscription request from Query to Source
#[derive(Debug, Clone)]
pub struct SubscriptionRequest {
    pub query_id: String,
    pub source_id: String,
    pub enable_bootstrap: bool,
    pub node_labels: Vec<String>,
    pub relation_labels: Vec<String>,
}

/// Subscription response from Source to Query
pub struct SubscriptionResponse {
    pub query_id: String,
    pub source_id: String,
    pub receiver: Box<dyn super::ChangeReceiver<SourceEventWrapper>>,
    pub bootstrap_receiver: Option<BootstrapEventReceiver>,
    /// Shared handle for the query to report its last durably-processed sequence position.
    /// Created by replay-capable sources when `request_position_handle` is true.
    /// The query writes to this atomically after each commit; the source reads the
    /// minimum across all subscribers to advance its upstream cursor.
    /// Sources should initialize this to `u64::MAX` (meaning "no position confirmed yet").
    pub position_handle: Option<Arc<AtomicU64>>,
}

/// Subscription response from Query to Reaction
pub struct QuerySubscriptionResponse {
    pub query_id: String,
    pub receiver: Box<dyn super::ChangeReceiver<QueryResult>>,
}

/// Typed result diff emitted by continuous queries.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
#[serde(tag = "type")]
pub enum ResultDiff {
    #[serde(rename = "ADD")]
    Add { data: serde_json::Value },
    #[serde(rename = "DELETE")]
    Delete { data: serde_json::Value },
    #[serde(rename = "UPDATE")]
    Update {
        data: serde_json::Value,
        before: serde_json::Value,
        after: serde_json::Value,
        #[serde(skip_serializing_if = "Option::is_none")]
        grouping_keys: Option<Vec<String>>,
    },
    #[serde(rename = "aggregation")]
    Aggregation {
        before: Option<serde_json::Value>,
        after: serde_json::Value,
    },
    #[serde(rename = "noop")]
    Noop,
}

/// Result emitted by a continuous query when data changes.
///
/// Contains the diff (added, updated, deleted rows) plus metadata and
/// optional profiling information. Dispatched to reactions via the priority queue.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct QueryResult {
    pub query_id: String,
    pub timestamp: chrono::DateTime<chrono::Utc>,
    pub results: Vec<ResultDiff>,
    pub metadata: HashMap<String, serde_json::Value>,
    /// Optional profiling metadata for performance tracking
    #[serde(skip_serializing_if = "Option::is_none")]
    pub profiling: Option<ProfilingMetadata>,
}

impl QueryResult {
    /// Create a new QueryResult without profiling
    pub fn new(
        query_id: String,
        timestamp: chrono::DateTime<chrono::Utc>,
        results: Vec<ResultDiff>,
        metadata: HashMap<String, serde_json::Value>,
    ) -> Self {
        Self {
            query_id,
            timestamp,
            results,
            metadata,
            profiling: None,
        }
    }

    /// Create a new QueryResult with profiling metadata
    pub fn with_profiling(
        query_id: String,
        timestamp: chrono::DateTime<chrono::Utc>,
        results: Vec<ResultDiff>,
        metadata: HashMap<String, serde_json::Value>,
        profiling: ProfilingMetadata,
    ) -> Self {
        Self {
            query_id,
            timestamp,
            results,
            metadata,
            profiling: Some(profiling),
        }
    }
}

// Implement Timestamped for QueryResult for use in generic priority queue
impl Timestamped for QueryResult {
    fn timestamp(&self) -> chrono::DateTime<chrono::Utc> {
        self.timestamp
    }
}

/// Arc-wrapped QueryResult for zero-copy distribution
pub type ArcQueryResult = Arc<QueryResult>;

/// Lifecycle event emitted when a component's status changes.
///
/// Broadcast via the component event channel to all subscribers.
/// Used for monitoring, logging, and reactive lifecycle coordination.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct ComponentEvent {
    pub component_id: String,
    pub component_type: ComponentType,
    pub status: ComponentStatus,
    pub timestamp: chrono::DateTime<chrono::Utc>,
    pub message: Option<String>,
}

/// Control messages for component lifecycle management.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub enum ControlMessage {
    Start(String),
    Stop(String),
    Status(String),
    Shutdown,
}

pub type ComponentEventBroadcastSender = broadcast::Sender<ComponentEvent>;
pub type ComponentEventBroadcastReceiver = broadcast::Receiver<ComponentEvent>;
/// Backward-compatible mpsc channel types used by host-sdk plugin callbacks.
/// New code should use `ComponentUpdateSender` from `component_graph` instead.
pub type ComponentEventSender = mpsc::Sender<ComponentEvent>;
pub type ComponentEventReceiver = mpsc::Receiver<ComponentEvent>;
pub type ControlMessageReceiver = mpsc::Receiver<ControlMessage>;
pub type ControlMessageSender = mpsc::Sender<ControlMessage>;

// Broadcast channel types for zero-copy event distribution
pub type SourceBroadcastSender = broadcast::Sender<ArcSourceEvent>;
pub type SourceBroadcastReceiver = broadcast::Receiver<ArcSourceEvent>;

// Broadcast channel types for zero-copy query result distribution
pub type QueryResultBroadcastSender = broadcast::Sender<ArcQueryResult>;
pub type QueryResultBroadcastReceiver = broadcast::Receiver<ArcQueryResult>;

// Bootstrap channel types for dedicated bootstrap data delivery
pub type BootstrapEventSender = mpsc::Sender<BootstrapEvent>;
pub type BootstrapEventReceiver = mpsc::Receiver<BootstrapEvent>;

/// Control signals for coordination
#[derive(Debug, Clone, Serialize, Deserialize)]
pub enum ControlSignal {
    /// Query has entered running state
    Running { query_id: String },
    /// Query has stopped
    Stopped { query_id: String },
    /// Query has been deleted
    Deleted { query_id: String },
}

/// Wrapper for control signals with metadata
#[derive(Debug, Clone)]
pub struct ControlSignalWrapper {
    pub signal: ControlSignal,
    pub timestamp: chrono::DateTime<chrono::Utc>,
    pub sequence_number: Option<u64>,
}

impl ControlSignalWrapper {
    pub fn new(signal: ControlSignal) -> Self {
        Self {
            signal,
            timestamp: chrono::Utc::now(),
            sequence_number: None,
        }
    }

    pub fn with_sequence(signal: ControlSignal, sequence_number: u64) -> Self {
        Self {
            signal,
            timestamp: chrono::Utc::now(),
            sequence_number: Some(sequence_number),
        }
    }
}

pub type ControlSignalReceiver = mpsc::Receiver<ControlSignalWrapper>;
pub type ControlSignalSender = mpsc::Sender<ControlSignalWrapper>;

pub struct EventChannels {
    pub _control_tx: ControlMessageSender,
    pub control_signal_tx: ControlSignalSender,
}

pub struct EventReceivers {
    pub _control_rx: ControlMessageReceiver,
    pub control_signal_rx: ControlSignalReceiver,
}

impl EventChannels {
    pub fn new() -> (Self, EventReceivers) {
        let (control_tx, control_rx) = mpsc::channel(100);
        let (control_signal_tx, control_signal_rx) = mpsc::channel(100);

        let channels = Self {
            _control_tx: control_tx,
            control_signal_tx,
        };

        let receivers = EventReceivers {
            _control_rx: control_rx,
            control_signal_rx,
        };

        (channels, receivers)
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use drasi_core::models::{Element, ElementMetadata, ElementReference, SourceChange};

    fn create_test_source_change() -> SourceChange {
        let element = Element::Node {
            metadata: ElementMetadata {
                reference: ElementReference::new("TestSource", "test-node-1"),
                labels: vec!["TestLabel".into()].into(),
                effective_from: 1000,
            },
            properties: Default::default(),
        };
        SourceChange::Insert { element }
    }

    #[test]
    fn test_source_event_wrapper_into_parts() {
        let change = create_test_source_change();
        let wrapper = SourceEventWrapper::new(
            "test-source".to_string(),
            SourceEvent::Change(change),
            chrono::Utc::now(),
        );

        let (source_id, event, _timestamp, profiling, _sequence) = wrapper.into_parts();

        assert_eq!(source_id, "test-source");
        assert!(matches!(event, SourceEvent::Change(_)));
        assert!(profiling.is_none());
    }

    #[test]
    fn test_try_unwrap_arc_sole_owner() {
        let change = create_test_source_change();
        let wrapper = SourceEventWrapper::new(
            "test-source".to_string(),
            SourceEvent::Change(change),
            chrono::Utc::now(),
        );
        let arc = Arc::new(wrapper);

        // With sole ownership, try_unwrap_arc should succeed
        let result = SourceEventWrapper::try_unwrap_arc(arc);
        assert!(result.is_ok());

        let (source_id, event, _timestamp, _profiling, _sequence) = result.unwrap();
        assert_eq!(source_id, "test-source");
        assert!(matches!(event, SourceEvent::Change(_)));
    }

    #[test]
    fn test_try_unwrap_arc_shared() {
        let change = create_test_source_change();
        let wrapper = SourceEventWrapper::new(
            "test-source".to_string(),
            SourceEvent::Change(change),
            chrono::Utc::now(),
        );
        let arc = Arc::new(wrapper);
        let _arc2 = arc.clone(); // Create another reference

        // With shared ownership, try_unwrap_arc should fail and return the Arc
        let result = SourceEventWrapper::try_unwrap_arc(arc);
        assert!(result.is_err());

        // The returned Arc should still be valid
        let returned_arc = result.unwrap_err();
        assert_eq!(returned_arc.source_id, "test-source");
    }

    #[test]
    fn test_zero_copy_extraction_path() {
        // Simulate the zero-copy extraction path used in query processing
        let change = create_test_source_change();
        let wrapper = SourceEventWrapper::new(
            "test-source".to_string(),
            SourceEvent::Change(change),
            chrono::Utc::now(),
        );
        let arc = Arc::new(wrapper);

        // This is the zero-copy path - when we have sole ownership
        let (source_id, event, _timestamp, _profiling, _sequence) =
            match SourceEventWrapper::try_unwrap_arc(arc) {
                Ok(parts) => parts,
                Err(arc) => {
                    // Fallback to cloning (would be needed in broadcast mode)
                    (
                        arc.source_id.clone(),
                        arc.event.clone(),
                        arc.timestamp,
                        arc.profiling.clone(),
                        arc.sequence,
                    )
                }
            };

        // Extract SourceChange from owned event (no clone!)
        let source_change = match event {
            SourceEvent::Change(change) => Some(change),
            _ => None,
        };

        assert_eq!(source_id, "test-source");
        assert!(source_change.is_some());
    }

    #[test]
    fn test_source_event_wrapper_with_sequence() {
        let change = create_test_source_change();
        let wrapper = SourceEventWrapper::with_sequence(
            "test-source".to_string(),
            SourceEvent::Change(change),
            chrono::Utc::now(),
            42,
            None,
        );
        assert_eq!(wrapper.sequence, Some(42));
        assert!(wrapper.profiling.is_none());

        let (_source_id, _event, _timestamp, _profiling, sequence) = wrapper.into_parts();
        assert_eq!(sequence, Some(42));
    }

    #[test]
    fn test_source_event_wrapper_new_has_no_sequence() {
        let change = create_test_source_change();
        let wrapper = SourceEventWrapper::new(
            "test-source".to_string(),
            SourceEvent::Change(change),
            chrono::Utc::now(),
        );
        assert!(wrapper.sequence.is_none());
    }

    #[test]
    fn test_subscription_response_with_position_handle() {
        use std::sync::atomic::{AtomicU64, Ordering};
        use std::sync::Arc;

        let handle = Arc::new(AtomicU64::new(u64::MAX));
        assert_eq!(handle.load(Ordering::Relaxed), u64::MAX);

        // Verify the handle can be cloned and read (simulates source reading query's position)
        let handle_clone = handle.clone();
        handle.store(500, Ordering::Relaxed);
        assert_eq!(handle_clone.load(Ordering::Relaxed), 500);
    }

    #[test]
    fn test_subscription_settings_with_resume_from() {
        use std::collections::HashSet;
        let settings = crate::config::SourceSubscriptionSettings {
            source_id: "test-source".to_string(),
            enable_bootstrap: false,
            query_id: "test-query".to_string(),
            nodes: HashSet::new(),
            relations: HashSet::new(),
            resume_from: Some(500),
            request_position_handle: true,
        };
        assert_eq!(settings.resume_from, Some(500));
        assert!(settings.request_position_handle);
    }
}