scouter-dataframe 0.25.0

DataFusion client for long-term storage of scouter data
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
use crate::error::TraceEngineError;
use crate::parquet::tracing::catalog::TraceCatalogProvider;
use crate::parquet::tracing::engine::{TableCommand, TraceSpanDBEngine};
use crate::parquet::tracing::queries::TraceQueries;
use crate::storage::ObjectStore;
use datafusion::prelude::SessionContext;
use scouter_settings::ObjectStorageSettings;
use scouter_types::TraceSpanRecord;
use std::sync::Arc;
use tokio::sync::mpsc;
use tokio::time::{interval, Duration};
use tracing::{debug, info};

const FLUSH_INTERVAL_SECS: u64 = 5;

/// Global singleton for the TraceSpanService.
///
/// Initialized via `init_trace_span_service()` in server setup.
/// Consumer workers call `get_trace_span_service()` to obtain the Arc for writing.
/// Uses `RwLock<Option<...>>` so tests can re-initialize with a fresh service.
static TRACE_SPAN_SERVICE: std::sync::RwLock<Option<Arc<TraceSpanService>>> =
    std::sync::RwLock::new(None);

/// Initialize the global `TraceSpanService`.
///
/// If a previous service exists (e.g. in test re-initialization), it is signaled to
/// shut down before being replaced.
pub async fn init_trace_span_service(
    storage_settings: &ObjectStorageSettings,
    compaction_interval_hours: u64,
    flush_interval_secs: Option<u64>,
    retention_days: Option<u32>,
    refresh_interval_secs: u64,
) -> Result<Arc<TraceSpanService>, TraceEngineError> {
    // Shut down any existing service before replacing
    let old_service = {
        let guard = TRACE_SPAN_SERVICE.read().unwrap();
        guard.clone()
    };
    if let Some(old) = old_service {
        info!("Shutting down previous TraceSpanService before re-initialization");
        old.signal_shutdown().await;
    }

    let service = Arc::new(
        TraceSpanService::new(
            storage_settings,
            compaction_interval_hours,
            flush_interval_secs,
            retention_days,
            refresh_interval_secs,
        )
        .await?,
    );

    {
        let mut guard = TRACE_SPAN_SERVICE.write().unwrap();
        *guard = Some(service.clone());
    }

    info!("TraceSpanService global singleton initialized");
    Ok(service)
}

/// Retrieve the global `TraceSpanService` initialized during startup.
///
/// Returns `None` if called before `init_trace_span_service()`.
pub fn get_trace_span_service() -> Option<Arc<TraceSpanService>> {
    TRACE_SPAN_SERVICE.read().unwrap().clone()
}

pub struct TraceSpanService {
    engine_tx: mpsc::Sender<TableCommand>,
    span_tx: mpsc::Sender<Vec<TraceSpanRecord>>,
    shutdown_tx: mpsc::Sender<()>,
    engine_handle: tokio::task::JoinHandle<()>,
    buffer_handle: tokio::task::JoinHandle<()>,
    pub query_service: TraceQueries,
    /// Shared SessionContext — passed to TraceSummaryService so both engines share the same
    /// context (and thus the same UDF registrations and object-store bindings).
    pub ctx: Arc<SessionContext>,
    /// Shared catalog — passed to TraceSummaryService so summary engine can call
    /// `catalog.swap()` for atomic `TableProvider` updates.
    pub catalog: Arc<TraceCatalogProvider>,
    /// Shared ObjectStore — passed to TraceSummaryService so both engines use the same
    /// CachingStore instance, preventing stale reads on cloud backends (GCS/S3).
    pub object_store: ObjectStore,
}

impl TraceSpanService {
    /// Create a new `TraceSpanService` with the given storage settings and start the engine and buffering actors.
    /// The buffering actor will flush spans to storage when the buffer reaches capacity or after a time interval.
    /// # Arguments
    /// * `storage_settings` - Configuration for object storage where trace spans will be persisted.
    /// * `compaction_interval_hours` - How often the engine should perform compaction
    ///   (merging small files into larger ones). Longer intervals reduce write amplification
    ///   but may increase read latency.
    /// * `flush_interval_secs` - Optional interval in seconds for flushing the buffer to storage
    /// * `refresh_interval_secs` - How often to refresh the in-memory Delta table snapshot from shared object storage. Useful for multi-pod deployments to pick up new commits.
    pub async fn new(
        storage_settings: &ObjectStorageSettings,
        compaction_interval_hours: u64,
        flush_interval_secs: Option<u64>,
        retention_days: Option<u32>,
        refresh_interval_secs: u64,
    ) -> Result<Self, TraceEngineError> {
        let buffer_size = storage_settings.trace_buffer_size();
        let engine = TraceSpanDBEngine::new(storage_settings).await?;

        info!(
            "TraceSpanService initialized with buffer_size: {}",
            buffer_size
        );

        let ctx = engine.ctx();
        let catalog = engine.catalog.clone();
        let object_store = engine.object_store.clone();
        let (engine_tx, engine_handle) = engine.start_actor(
            compaction_interval_hours,
            retention_days,
            refresh_interval_secs,
        );
        let (span_tx, span_rx) = mpsc::channel::<Vec<TraceSpanRecord>>(100);
        let (shutdown_tx, shutdown_rx) = mpsc::channel::<()>(1);

        let buffer_handle = Self::start_buffering_actor(
            engine_tx.clone(),
            span_rx,
            shutdown_rx,
            flush_interval_secs,
            buffer_size,
        );

        Ok(TraceSpanService {
            engine_tx,
            span_tx,
            shutdown_tx,
            engine_handle,
            buffer_handle,
            query_service: TraceQueries::new(ctx.clone()),
            ctx,
            catalog,
            object_store,
        })
    }

    fn start_buffering_actor(
        engine_tx: mpsc::Sender<TableCommand>,
        mut span_rx: mpsc::Receiver<Vec<TraceSpanRecord>>,
        mut shutdown_rx: mpsc::Receiver<()>,
        flush_interval_secs: Option<u64>,
        buffer_size: usize,
    ) -> tokio::task::JoinHandle<()> {
        tokio::spawn(async move {
            let mut buffer: Vec<TraceSpanRecord> = Vec::with_capacity(buffer_size);
            let mut flush_ticker = interval(Duration::from_secs(
                flush_interval_secs.unwrap_or(FLUSH_INTERVAL_SECS),
            ));
            flush_ticker.tick().await;

            loop {
                tokio::select! {
                    Some(spans) = span_rx.recv() => {
                        buffer.extend(spans);
                        if buffer.len() >= buffer_size {
                            Self::flush_buffer(&engine_tx, &mut buffer).await;
                        }
                    }
                    _ = flush_ticker.tick() => {
                        if !buffer.is_empty() {
                            info!("Flushing spans buffer with {} spans", buffer.len());
                            Self::flush_buffer(&engine_tx, &mut buffer).await;
                        }
                    }
                    _ = shutdown_rx.recv() => {
                        info!("Buffer actor received shutdown signal");
                        if !buffer.is_empty() {
                            info!("Flushing final {} spans before shutdown", buffer.len());
                            Self::flush_buffer(&engine_tx, &mut buffer).await;
                        }
                        break;
                    }
                }
            }

            info!("Buffering actor shutting down");
        })
    }

    async fn flush_buffer(
        engine_tx: &mpsc::Sender<TableCommand>,
        buffer: &mut Vec<TraceSpanRecord>,
    ) {
        if buffer.is_empty() {
            return;
        }

        let capacity = buffer.capacity();
        let spans_to_write = std::mem::replace(buffer, Vec::with_capacity(capacity));
        let span_count = spans_to_write.len();

        debug!("Sending write command to engine for {} spans", span_count);

        let (tx, rx) = tokio::sync::oneshot::channel();

        if let Err(e) = engine_tx
            .send(TableCommand::Write {
                spans: spans_to_write,
                respond_to: tx,
            })
            .await
        {
            tracing::error!("Failed to send write command: {}", e);
            return;
        }

        match rx.await {
            Ok(Ok(())) => info!("Successfully flushed {} spans", span_count),
            Ok(Err(e)) => tracing::error!("Write failed: {}", e),
            Err(e) => tracing::error!("Failed to receive write response: {}", e),
        }
    }

    /// Send spans to the buffering actor for async write to Delta Lake.
    /// # Arguments
    /// * `spans` - A batch of `TraceSpanRecord` to write. The buffering actor will flush to storage
    ///   when the buffer reaches capacity or after a time interval.
    pub async fn write_spans(&self, spans: Vec<TraceSpanRecord>) -> Result<(), TraceEngineError> {
        self.span_tx
            .send(spans)
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;
        Ok(())
    }

    /// Write spans directly to the engine actor, bypassing the buffer.
    ///
    /// Unlike `write_spans()`, this method sends a single large batch as one
    /// Delta Lake commit and awaits the result. Use for bulk seeding/migration
    /// where you need deterministic commit boundaries and maximum throughput.
    /// This is used in benchmarks and stress tests to simulate high-volume writes without caching effects
    pub async fn write_spans_direct(
        &self,
        spans: Vec<TraceSpanRecord>,
    ) -> Result<(), TraceEngineError> {
        let (tx, rx) = tokio::sync::oneshot::channel();
        self.engine_tx
            .send(TableCommand::Write {
                spans,
                respond_to: tx,
            })
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;
        rx.await.map_err(|_| TraceEngineError::ChannelClosed)?
    }

    // Admin operations — these can be called directly or via the control table for single-writer cron-style execution across
    // Primarily used in tests and benches. In Production, we default to the control table approach to ensure only worker runs them at a time, but the direct methods remain available for manual invocation when needed.
    pub async fn optimize(&self) -> Result<(), TraceEngineError> {
        let (tx, rx) = tokio::sync::oneshot::channel();

        self.engine_tx
            .send(TableCommand::Optimize { respond_to: tx })
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;

        rx.await.map_err(|_| TraceEngineError::ChannelClosed)?
    }

    pub async fn vacuum(&self, retention_hours: u64) -> Result<(), TraceEngineError> {
        let (tx, rx) = tokio::sync::oneshot::channel();

        self.engine_tx
            .send(TableCommand::Vacuum {
                retention_hours,
                respond_to: tx,
            })
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;

        rx.await.map_err(|_| TraceEngineError::ChannelClosed)?
    }

    /// Delete all spans with a `partition_date` older than `retention_days` ago,
    /// then run VACUUM to physically reclaim disk space.
    ///
    /// Call order matters: DELETE marks files as unreferenced in the Delta log;
    /// VACUUM then removes the orphaned Parquet files from storage.
    pub async fn expire(&self, retention_days: u32) -> Result<(), TraceEngineError> {
        let cutoff_date =
            (chrono::Utc::now() - chrono::Duration::days(retention_days as i64)).date_naive();

        // Step 1: DELETE WHERE partition_date < cutoff (marks rows removed in Delta log)
        let (tx, rx) = tokio::sync::oneshot::channel();
        self.engine_tx
            .send(TableCommand::Expire {
                cutoff_date,
                respond_to: tx,
            })
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;
        rx.await.map_err(|_| TraceEngineError::ChannelClosed)??;

        // Step 2: VACUUM — physically deletes orphaned files from storage.
        // retention_hours=0 with enforce_retention_duration=false removes all
        // post-DELETE orphans immediately.
        self.vacuum(0).await
    }

    /// Signal shutdown without consuming `self` — safe to call from `Arc<TraceSpanService>`.
    ///
    /// Sends the shutdown signal to the buffering actor and engine actor.
    /// Callers that own `self` should prefer `shutdown()` to await full drain.
    pub async fn signal_shutdown(&self) {
        info!("TraceSpanService signaling shutdown");
        let _ = self.shutdown_tx.send(()).await;
        let _ = self.engine_tx.send(TableCommand::Shutdown).await;
    }

    pub async fn shutdown(self) -> Result<(), TraceEngineError> {
        info!("TraceSpanService shutting down");

        let _ = self.shutdown_tx.send(()).await;

        if let Err(e) = self.buffer_handle.await {
            tracing::error!("Buffer handle error: {}", e);
        }

        self.engine_tx
            .send(TableCommand::Shutdown)
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;

        if let Err(e) = self.engine_handle.await {
            tracing::error!("Engine handle error: {}", e);
        }

        info!("TraceSpanService shutdown complete");
        Ok(())
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::parquet::tracing::queries::{
        date_lit, ts_lit, PARTITION_DATE_COL, SPAN_TABLE_NAME, START_TIME_COL,
    };
    use arrow_array::Array;
    use chrono::Utc;
    use datafusion::logical_expr::col;
    use scouter_mocks::generate_trace_with_spans;
    use scouter_settings::ObjectStorageSettings;
    use scouter_types::sql::TraceSpan;
    use scouter_types::{Attribute, SpanId, TraceId, TraceSpanRecord};
    use serde_json::Value;
    use tracing_subscriber;

    fn cleanup() {
        let _ = tracing_subscriber::fmt()
            .with_max_level(tracing::Level::INFO)
            .try_init();

        let storage_settings = ObjectStorageSettings::default();
        let current_dir = std::env::current_dir().unwrap();
        let storage_path = current_dir.join(storage_settings.storage_root());
        if storage_path.exists() {
            let _ = std::fs::remove_dir_all(storage_path);
        }
    }

    /// Build a deterministic `TraceSpanRecord` with the given IDs and attributes.
    fn make_span(
        trace_id: &TraceId,
        span_id: SpanId,
        parent_span_id: Option<SpanId>,
        service_name: &str,
        span_name: &str,
        attributes: Vec<Attribute>,
    ) -> TraceSpanRecord {
        let now = Utc::now();
        TraceSpanRecord {
            created_at: now,
            trace_id: *trace_id,
            span_id,
            parent_span_id,
            flags: 1,
            trace_state: String::new(),
            scope_name: "test.scope".to_string(),
            scope_version: None,
            span_name: span_name.to_string(),
            span_kind: "INTERNAL".to_string(),
            start_time: now,
            end_time: now + chrono::Duration::milliseconds(100),
            duration_ms: 100,
            status_code: 0,
            status_message: "OK".to_string(),
            attributes,
            events: vec![],
            links: vec![],
            label: None,
            input: Value::Null,
            output: Value::Null,
            service_name: service_name.to_string(),
            resource_attributes: vec![],
        }
    }

    #[tokio::test]
    async fn test_service_initialization() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;
        service.shutdown().await?;
        cleanup();
        Ok(())
    }

    #[tokio::test]
    async fn test_dataframe_trace_write_single_batch() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;

        let (_trace_record, spans, _tags) = generate_trace_with_spans(3, 0);
        info!("Test: writing {} spans", spans.len());

        let first_trace_id = spans.first().unwrap().trace_id;
        service.write_spans(spans).await?;

        info!("Test: waiting for flush");
        tokio::time::sleep(Duration::from_secs(5)).await;

        let trace_id_bytes = first_trace_id.as_bytes();
        let result_spans: Vec<TraceSpan> = service
            .query_service
            .get_trace_spans(Some(trace_id_bytes.as_slice()), None, None, None, None)
            .await?;

        assert!(
            !result_spans.is_empty(),
            "Expected at least 1 span but got 0"
        );

        service.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// Verify that `build_span_tree` returns spans in DFS (depth-first) order with correct
    /// depth, path, and root_span_id fields — matching what the Postgres recursive CTE produced.
    #[tokio::test]
    async fn test_span_tree_sort_order() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;

        // Build a deterministic tree: root → child → grandchild
        let trace_id = TraceId::from_bytes([1u8; 16]);
        let root_span_id = SpanId::from_bytes([1u8; 8]);
        let child_span_id = SpanId::from_bytes([2u8; 8]);
        let grandchild_span_id = SpanId::from_bytes([3u8; 8]);

        let root = make_span(
            &trace_id,
            root_span_id.clone(),
            None,
            "svc",
            "root_op",
            vec![],
        );
        let child = make_span(
            &trace_id,
            child_span_id.clone(),
            Some(root_span_id.clone()),
            "svc",
            "child_op",
            vec![],
        );
        let grandchild = make_span(
            &trace_id,
            grandchild_span_id.clone(),
            Some(child_span_id.clone()),
            "svc",
            "grandchild_op",
            vec![],
        );

        service.write_spans(vec![root, child, grandchild]).await?;
        tokio::time::sleep(Duration::from_secs(4)).await;

        let spans: Vec<TraceSpan> = service
            .query_service
            .get_trace_spans(Some(trace_id.as_bytes().as_slice()), None, None, None, None)
            .await?;

        assert_eq!(spans.len(), 3, "Expected 3 spans");

        // DFS order: root(0), child(1), grandchild(2)
        let by_order: Vec<&TraceSpan> = {
            let mut v: Vec<&TraceSpan> = spans.iter().collect();
            v.sort_by_key(|s| s.span_order);
            v
        };

        assert_eq!(
            by_order[0].span_name, "root_op",
            "span_order=0 should be root"
        );
        assert_eq!(by_order[0].depth, 0);
        assert_eq!(by_order[0].path.len(), 1);

        assert_eq!(
            by_order[1].span_name, "child_op",
            "span_order=1 should be child"
        );
        assert_eq!(by_order[1].depth, 1);
        assert_eq!(by_order[1].path.len(), 2);

        assert_eq!(
            by_order[2].span_name, "grandchild_op",
            "span_order=2 should be grandchild"
        );
        assert_eq!(by_order[2].depth, 2);
        assert_eq!(by_order[2].path.len(), 3);

        // All spans should share the same root_span_id
        let root_sid = root_span_id.to_hex();
        for span in &spans {
            assert_eq!(
                span.root_span_id, root_sid,
                "root_span_id mismatch for {}",
                span.span_name
            );
        }

        service.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// Verify `get_trace_metrics` returns time-bucketed aggregate rows.
    #[tokio::test]
    async fn test_trace_metrics_basic() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;

        let (_record, spans, _tags) = generate_trace_with_spans(5, 0);
        service.write_spans(spans).await?;
        tokio::time::sleep(Duration::from_secs(4)).await;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);

        let metrics = service
            .query_service
            .get_trace_metrics(None, start, end, "hour", None, None)
            .await?;

        assert!(!metrics.is_empty(), "Expected at least one metric bucket");
        assert!(metrics[0].trace_count > 0, "Expected non-zero trace count");

        service.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// Verify `get_trace_metrics` with a service_name filter excludes other services.
    #[tokio::test]
    async fn test_trace_metrics_service_filter() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;

        // Write spans for two distinct services using deterministic IDs
        let trace_a = TraceId::from_bytes([10u8; 16]);
        let trace_b = TraceId::from_bytes([20u8; 16]);

        let span_a = make_span(
            &trace_a,
            SpanId::from_bytes([10u8; 8]),
            None,
            "service_alpha",
            "op_a",
            vec![],
        );
        let span_b = make_span(
            &trace_b,
            SpanId::from_bytes([20u8; 8]),
            None,
            "service_beta",
            "op_b",
            vec![],
        );

        service.write_spans(vec![span_a, span_b]).await?;
        tokio::time::sleep(Duration::from_secs(4)).await;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);

        // Filter to service_alpha only
        let metrics_alpha = service
            .query_service
            .get_trace_metrics(Some("service_alpha"), start, end, "hour", None, None)
            .await?;

        // Filter to service_beta only
        let metrics_beta = service
            .query_service
            .get_trace_metrics(Some("service_beta"), start, end, "hour", None, None)
            .await?;

        let alpha_count: i64 = metrics_alpha.iter().map(|m| m.trace_count).sum();
        let beta_count: i64 = metrics_beta.iter().map(|m| m.trace_count).sum();

        assert!(alpha_count > 0, "Expected non-zero count for service_alpha");
        assert!(beta_count > 0, "Expected non-zero count for service_beta");

        // Querying with a non-existent service returns nothing
        let metrics_none = service
            .query_service
            .get_trace_metrics(Some("nonexistent_svc"), start, end, "hour", None, None)
            .await?;
        assert!(
            metrics_none.is_empty(),
            "Expected no buckets for nonexistent service"
        );

        service.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// Verify all three DataFusion filter layers are wired correctly by inspecting the
    /// physical plan produced for a partitioned time-window query.
    ///
    /// Layer 1 — partition pruning: `partition_date` appears in the plan because Delta Lake
    /// pushes partition column filters to directory enumeration before reading any files.
    ///
    /// Layer 2 — row-group stats: `start_time` appears because typed Timestamp literals
    /// enable Parquet min/max pruning across row groups.
    ///
    /// Layer 3 — bloom filter: querying a nonexistent `trace_id` within a tight window
    /// returns 0 rows; bloom filters discard row groups instantly without page-level scanning.
    #[tokio::test]
    async fn test_query_plan_shows_filter_layers() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;

        // Write a known batch directly so it is immediately queryable
        // Use distinct byte values that don't collide with other tests.
        let trace_id = TraceId::from_bytes([0xAA_u8; 16]);
        let root_id = SpanId::from_bytes([0xAA_u8; 8]);
        let child_id = SpanId::from_bytes([0xBB_u8; 8]);
        let spans = vec![
            make_span(&trace_id, root_id.clone(), None, "svc-a", "root-op", vec![]),
            make_span(
                &trace_id,
                child_id.clone(),
                Some(root_id),
                "svc-a",
                "child-op",
                vec![],
            ),
        ];
        service.write_spans_direct(spans).await?;

        // ── Layers 1 & 2: inspect physical plan ──────────────────────────────────
        let now = Utc::now();
        let start = now - chrono::Duration::hours(1);
        let end = now + chrono::Duration::hours(1);

        // Build the same DataFrame the query path would produce internally
        let df = service
            .ctx
            .table(SPAN_TABLE_NAME)
            .await
            .map_err(TraceEngineError::DatafusionError)?;

        // Partition filter — pushed to directory enumeration (Layer 1)
        let df = df
            .filter(
                col(PARTITION_DATE_COL)
                    .gt_eq(date_lit(&start))
                    .and(col(PARTITION_DATE_COL).lt_eq(date_lit(&end))),
            )
            .map_err(TraceEngineError::DatafusionError)?;

        // Timestamp filter — enables row-group min/max pruning (Layer 2)
        let df = df
            .filter(
                col(START_TIME_COL)
                    .gt_eq(ts_lit(&start))
                    .and(col(START_TIME_COL).lt(ts_lit(&end))),
            )
            .map_err(TraceEngineError::DatafusionError)?;

        // Collect the physical plan as a string via EXPLAIN
        let explain_df = df
            .explain(false, false)
            .map_err(TraceEngineError::DatafusionError)?;
        let batches = explain_df
            .collect()
            .await
            .map_err(TraceEngineError::DatafusionError)?;

        let plan_text: String = batches
            .iter()
            .flat_map(|b| {
                let plan_col = b.column_by_name("plan").unwrap();
                let arr =
                    arrow::compute::cast(plan_col, &arrow::datatypes::DataType::Utf8).unwrap();
                let s = arr
                    .as_any()
                    .downcast_ref::<arrow::array::StringArray>()
                    .unwrap();
                (0..s.len())
                    .map(|i| s.value(i).to_string())
                    .collect::<Vec<_>>()
            })
            .collect::<Vec<_>>()
            .join("\n");

        // Layer 1: partition filter must appear in plan
        assert!(
            plan_text.contains("partition_date"),
            "Partition filter not found in physical plan:\n{plan_text}"
        );
        // Layer 2: start_time row-group filter must appear in plan
        assert!(
            plan_text.contains("start_time"),
            "Row-group time filter not found in physical plan:\n{plan_text}"
        );

        // ── Layer 3: bloom filter — behavioral proof ──────────────────────────────
        // A nonexistent trace_id with bloom filters enabled returns 0 rows.
        // Without bloom filters, DataFusion would scan every row group to confirm absence.
        let fake_id = TraceId::from_bytes([0xFF_u8; 16]);
        let result = service
            .query_service
            .get_trace_spans(
                Some(fake_id.as_bytes()),
                None,
                Some(&start),
                Some(&end),
                None,
            )
            .await?;
        assert!(
            result.is_empty(),
            "Expected 0 spans for nonexistent trace_id"
        );

        service.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// Verify `get_trace_metrics` with attribute_filters narrows results to matching spans.
    #[tokio::test]
    async fn test_trace_metrics_with_attribute_filter() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;

        let trace_kafka = TraceId::from_bytes([30u8; 16]);
        let trace_http = TraceId::from_bytes([40u8; 16]);

        // span with component:kafka attribute
        let span_kafka = make_span(
            &trace_kafka,
            SpanId::from_bytes([30u8; 8]),
            None,
            "my_service",
            "kafka_consumer",
            vec![Attribute {
                key: "component".to_string(),
                value: Value::String("kafka".to_string()),
            }],
        );
        // span with component:http attribute (should NOT match kafka filter)
        let span_http = make_span(
            &trace_http,
            SpanId::from_bytes([40u8; 8]),
            None,
            "my_service",
            "http_handler",
            vec![Attribute {
                key: "component".to_string(),
                value: Value::String("http".to_string()),
            }],
        );

        service.write_spans(vec![span_kafka, span_http]).await?;
        tokio::time::sleep(Duration::from_secs(4)).await;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);
        let kafka_filter = vec!["component:kafka".to_string()];

        // With filter, only kafka trace should appear
        let filtered = service
            .query_service
            .get_trace_metrics(None, start, end, "hour", Some(&kafka_filter), None)
            .await?;

        let filtered_count: i64 = filtered.iter().map(|m| m.trace_count).sum();
        assert!(
            filtered_count > 0,
            "Expected non-zero count with kafka attribute filter"
        );

        // Without filter, both traces appear
        let unfiltered = service
            .query_service
            .get_trace_metrics(None, start, end, "hour", None, None)
            .await?;
        let unfiltered_count: i64 = unfiltered.iter().map(|m| m.trace_count).sum();
        assert!(
            unfiltered_count >= filtered_count,
            "Unfiltered count ({}) should be >= filtered count ({})",
            unfiltered_count,
            filtered_count
        );

        service.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// Simulate a 2-pod deployment: writer pod commits, reader pod picks it up via refresh.
    ///
    /// Two `TraceSpanService` instances share the same local storage directory (same as
    /// sharing GCS/S3 in production). The writer commits spans directly; the reader's refresh
    /// ticker (1s interval) picks up the new Delta log entry and re-registers the SessionContext.
    ///
    /// Note: we use `TraceSpanService::new()` directly (not `init_trace_span_service()`) because
    /// the global singleton pattern shuts down the previous service on re-init, which would
    /// kill the writer before the reader can see its data.
    #[tokio::test]
    async fn test_distributed_refresh() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();

        // "Writer pod" — standard refresh interval (won't need to refresh since it's the writer)
        let writer = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;

        // "Reader pod" — 1s refresh interval for fast test turnaround
        let reader = TraceSpanService::new(&storage_settings, 24, Some(2), None, 1).await?;

        // Write spans via the writer using a deterministic trace ID
        let trace_id = TraceId::from_bytes([0xDD_u8; 16]);
        let span = make_span(
            &trace_id,
            SpanId::from_bytes([0xDD_u8; 8]),
            None,
            "distributed-svc",
            "test-op",
            vec![],
        );
        writer.write_spans_direct(vec![span]).await?;

        // Wait for the reader's refresh ticker to fire (1s interval + margin)
        tokio::time::sleep(Duration::from_secs(3)).await;

        // Reader should now see the data written by the writer
        let results = reader
            .query_service
            .get_trace_spans(Some(trace_id.as_bytes()), None, None, None, None)
            .await?;

        assert!(
            !results.is_empty(),
            "Reader pod should see spans written by writer pod after refresh"
        );

        writer.shutdown().await?;
        reader.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// Regression test for stale DataFusion session state after Delta Lake writes.
    ///
    /// Before the fix, `SessionContext` cached file metadata from the first write and
    /// subsequent writes were invisible to queries — the session held a stale snapshot
    /// of the Delta log. The fix calls `update_datafusion_session()` after each write
    /// to re-register the table provider with the latest Delta log state.
    #[tokio::test]
    async fn test_span_write_visibility_across_multiple_writes() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);

        // Write batch #1 (2 spans)
        let trace1 = TraceId::from_bytes([0xB0; 16]);
        let spans1 = vec![
            make_span(
                &trace1,
                SpanId::from_bytes([0xB0; 8]),
                None,
                "svc_vis",
                "op1",
                vec![],
            ),
            make_span(
                &trace1,
                SpanId::from_bytes([0xB1; 8]),
                Some(SpanId::from_bytes([0xB0; 8])),
                "svc_vis",
                "op2",
                vec![],
            ),
        ];
        service.write_spans_direct(spans1).await?;

        let result = service
            .query_service
            .get_trace_spans(
                Some(trace1.as_bytes()),
                None,
                Some(&start),
                Some(&end),
                None,
            )
            .await?;
        assert_eq!(
            result.len(),
            2,
            "After write #1: expected 2 spans, got {}",
            result.len()
        );

        // Write batch #2 (2 more spans, different trace)
        let trace2 = TraceId::from_bytes([0xB2; 16]);
        let spans2 = vec![
            make_span(
                &trace2,
                SpanId::from_bytes([0xB2; 8]),
                None,
                "svc_vis",
                "op3",
                vec![],
            ),
            make_span(
                &trace2,
                SpanId::from_bytes([0xB3; 8]),
                Some(SpanId::from_bytes([0xB2; 8])),
                "svc_vis",
                "op4",
                vec![],
            ),
        ];
        service.write_spans_direct(spans2).await?;

        let result = service
            .query_service
            .get_trace_spans(
                Some(trace2.as_bytes()),
                None,
                Some(&start),
                Some(&end),
                None,
            )
            .await?;
        assert_eq!(
            result.len(),
            2,
            "After write #2: expected 2 spans for trace2, got {} (stale snapshot?)",
            result.len()
        );

        // Write batch #3 (2 more spans, third trace)
        let trace3 = TraceId::from_bytes([0xB4; 16]);
        let spans3 = vec![
            make_span(
                &trace3,
                SpanId::from_bytes([0xB4; 8]),
                None,
                "svc_vis",
                "op5",
                vec![],
            ),
            make_span(
                &trace3,
                SpanId::from_bytes([0xB5; 8]),
                Some(SpanId::from_bytes([0xB4; 8])),
                "svc_vis",
                "op6",
                vec![],
            ),
        ];
        service.write_spans_direct(spans3).await?;

        let result = service
            .query_service
            .get_trace_spans(
                Some(trace3.as_bytes()),
                None,
                Some(&start),
                Some(&end),
                None,
            )
            .await?;
        assert_eq!(
            result.len(),
            2,
            "After write #3: expected 2 spans for trace3, got {} (stale snapshot?)",
            result.len()
        );

        service.shutdown().await?;
        cleanup();
        Ok(())
    }
}