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
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
use crate::error::TraceEngineError;
use crate::parquet::control::{get_pod_id, ControlTableEngine};
use crate::parquet::tracing::catalog::TraceCatalogProvider;
use crate::parquet::tracing::traits::{arrow_schema_to_delta, resource_attribute_field};
use crate::parquet::utils::match_attr_expr;
use crate::parquet::utils::register_cloud_logstore_factories;
use crate::storage::ObjectStore;
use arrow::array::*;
use arrow::compute;
use arrow::datatypes::*;
use arrow_array::Array;
use arrow_array::RecordBatch;
use chrono::{DateTime, Datelike, Utc};
use datafusion::logical_expr::{cast as df_cast, col, lit, SortExpr};
use datafusion::prelude::*;
use datafusion::scalar::ScalarValue;
use deltalake::operations::optimize::OptimizeType;
use deltalake::{DeltaTable, DeltaTableBuilder, TableProperty};
use scouter_types::sql::{TraceFilters, TraceListItem};
use scouter_types::{Attribute, TraceCursor, TraceId, TracePaginationResponse, TraceSummaryRecord};
use std::sync::Arc;
use tokio::sync::oneshot;
use tokio::sync::{mpsc, RwLock as AsyncRwLock};
use tokio::time::{interval, Duration};
use tracing::{debug, error, info, instrument};
use url::Url;

/// Days from CE epoch to Unix epoch (1970-01-01).
/// Equivalent to `NaiveDate::from_ymd_opt(1970, 1, 1).unwrap().num_days_from_ce()`.
const UNIX_EPOCH_DAYS: i32 = 719_163;

const SUMMARY_TABLE_NAME: &str = "trace_summaries";

/// Control table task name for summary compaction coordination.
const TASK_SUMMARY_OPTIMIZE: &str = "summary_optimize";

// ── Column name constants ────────────────────────────────────────────────────
const TRACE_ID_COL: &str = "trace_id";
const SERVICE_NAME_COL: &str = "service_name";
const SCOPE_NAME_COL: &str = "scope_name";
const SCOPE_VERSION_COL: &str = "scope_version";
const ROOT_OPERATION_COL: &str = "root_operation";
const START_TIME_COL: &str = "start_time";
const END_TIME_COL: &str = "end_time";
const DURATION_MS_COL: &str = "duration_ms";
const STATUS_CODE_COL: &str = "status_code";
const STATUS_MESSAGE_COL: &str = "status_message";
const SPAN_COUNT_COL: &str = "span_count";
const ERROR_COUNT_COL: &str = "error_count";
const SEARCH_BLOB_COL: &str = "search_blob";

const RESOURCE_ATTRIBUTES_COL: &str = "resource_attributes";
const ENTITY_IDS_COL: &str = "entity_ids";
const QUEUE_IDS_COL: &str = "queue_ids";

const PARTITION_DATE_COL: &str = "partition_date";

// ── Schema ───────────────────────────────────────────────────────────────────

fn create_summary_schema() -> Schema {
    Schema::new(vec![
        Field::new(TRACE_ID_COL, DataType::FixedSizeBinary(16), false),
        Field::new(
            SERVICE_NAME_COL,
            DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
            false,
        ),
        Field::new(SCOPE_NAME_COL, DataType::Utf8, false),
        Field::new(SCOPE_VERSION_COL, DataType::Utf8, true),
        Field::new(ROOT_OPERATION_COL, DataType::Utf8, false),
        Field::new(
            START_TIME_COL,
            DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())),
            false,
        ),
        Field::new(
            END_TIME_COL,
            DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())),
            true,
        ),
        Field::new(DURATION_MS_COL, DataType::Int64, true),
        Field::new(STATUS_CODE_COL, DataType::Int32, false),
        Field::new(STATUS_MESSAGE_COL, DataType::Utf8, true),
        Field::new(SPAN_COUNT_COL, DataType::Int64, false),
        Field::new(ERROR_COUNT_COL, DataType::Int64, false),
        resource_attribute_field(),
        Field::new(
            ENTITY_IDS_COL,
            DataType::List(Arc::new(Field::new("item", DataType::Utf8, true))),
            true,
        ),
        Field::new(
            QUEUE_IDS_COL,
            DataType::List(Arc::new(Field::new("item", DataType::Utf8, true))),
            true,
        ),
        Field::new(PARTITION_DATE_COL, DataType::Date32, false),
    ])
}

// ── BatchBuilder ─────────────────────────────────────────────────────────────

struct TraceSummaryBatchBuilder {
    schema: Arc<Schema>,
    trace_id: FixedSizeBinaryBuilder,
    service_name: StringDictionaryBuilder<Int32Type>,
    scope_name: StringBuilder,
    scope_version: StringBuilder,
    root_operation: StringBuilder,
    start_time: TimestampMicrosecondBuilder,
    end_time: TimestampMicrosecondBuilder,
    duration_ms: Int64Builder,
    status_code: Int32Builder,
    status_message: StringBuilder,
    span_count: Int64Builder,
    error_count: Int64Builder,
    resource_attributes: MapBuilder<StringBuilder, StringViewBuilder>,
    entity_ids: ListBuilder<StringBuilder>,
    queue_ids: ListBuilder<StringBuilder>,
    partition_date: Date32Builder,
}

impl TraceSummaryBatchBuilder {
    fn new(schema: Arc<Schema>, capacity: usize) -> Self {
        let map_field_names = MapFieldNames {
            entry: "key_value".to_string(),
            key: "key".to_string(),
            value: "value".to_string(),
        };
        let resource_attributes = MapBuilder::new(
            Some(map_field_names),
            StringBuilder::new(),
            StringViewBuilder::new(),
        );
        Self {
            schema,
            trace_id: FixedSizeBinaryBuilder::with_capacity(capacity, 16),
            service_name: StringDictionaryBuilder::new(),
            scope_name: StringBuilder::with_capacity(capacity, capacity * 16),
            scope_version: StringBuilder::with_capacity(capacity, capacity * 8),
            root_operation: StringBuilder::with_capacity(capacity, capacity * 32),
            start_time: TimestampMicrosecondBuilder::with_capacity(capacity).with_timezone("UTC"),
            end_time: TimestampMicrosecondBuilder::with_capacity(capacity).with_timezone("UTC"),
            duration_ms: Int64Builder::with_capacity(capacity),
            status_code: Int32Builder::with_capacity(capacity),
            status_message: StringBuilder::with_capacity(capacity, capacity * 16),
            span_count: Int64Builder::with_capacity(capacity),
            error_count: Int64Builder::with_capacity(capacity),
            resource_attributes,
            entity_ids: ListBuilder::new(StringBuilder::new()),
            queue_ids: ListBuilder::new(StringBuilder::new()),
            partition_date: Date32Builder::with_capacity(capacity),
        }
    }

    fn append(&mut self, rec: &TraceSummaryRecord) -> Result<(), TraceEngineError> {
        self.trace_id.append_value(rec.trace_id.as_bytes())?;
        self.service_name.append_value(&rec.service_name);
        self.scope_name.append_value(&rec.scope_name);
        if rec.scope_version.is_empty() {
            self.scope_version.append_null();
        } else {
            self.scope_version.append_value(&rec.scope_version);
        }
        self.root_operation.append_value(&rec.root_operation);
        self.start_time
            .append_value(rec.start_time.timestamp_micros());
        match rec.end_time {
            Some(end) => self.end_time.append_value(end.timestamp_micros()),
            None => self.end_time.append_null(),
        }
        let duration = rec
            .end_time
            .map(|end| (end - rec.start_time).num_milliseconds().max(0));
        match duration {
            Some(d) => self.duration_ms.append_value(d),
            None => self.duration_ms.append_null(),
        }
        self.status_code.append_value(rec.status_code);
        if rec.status_message.is_empty() {
            self.status_message.append_null();
        } else {
            self.status_message.append_value(&rec.status_message);
        }
        self.span_count.append_value(rec.span_count);
        self.error_count.append_value(rec.error_count);
        if rec.resource_attributes.is_empty() {
            self.resource_attributes.append(false)?; // null map
        } else {
            for attr in &rec.resource_attributes {
                self.resource_attributes.keys().append_value(&attr.key);
                let value_str =
                    serde_json::to_string(&attr.value).unwrap_or_else(|_| "null".to_string());
                self.resource_attributes.values().append_value(value_str);
            }
            self.resource_attributes.append(true)?;
        }
        if rec.entity_ids.is_empty() {
            self.entity_ids.append_null();
        } else {
            for id in &rec.entity_ids {
                self.entity_ids.values().append_value(id);
            }
            self.entity_ids.append(true);
        }
        if rec.queue_ids.is_empty() {
            self.queue_ids.append_null();
        } else {
            for id in &rec.queue_ids {
                self.queue_ids.values().append_value(id);
            }
            self.queue_ids.append(true);
        }
        // Partition key — days since Unix epoch, derived from start_time
        let days = rec.start_time.date_naive().num_days_from_ce() - UNIX_EPOCH_DAYS;
        self.partition_date.append_value(days);
        Ok(())
    }

    fn finish(mut self) -> Result<RecordBatch, TraceEngineError> {
        let columns: Vec<Arc<dyn Array>> = vec![
            Arc::new(self.trace_id.finish()),
            Arc::new(self.service_name.finish()),
            Arc::new(self.scope_name.finish()),
            Arc::new(self.scope_version.finish()),
            Arc::new(self.root_operation.finish()),
            Arc::new(self.start_time.finish()),
            Arc::new(self.end_time.finish()),
            Arc::new(self.duration_ms.finish()),
            Arc::new(self.status_code.finish()),
            Arc::new(self.status_message.finish()),
            Arc::new(self.span_count.finish()),
            Arc::new(self.error_count.finish()),
            Arc::new(self.resource_attributes.finish()),
            Arc::new(self.entity_ids.finish()),
            Arc::new(self.queue_ids.finish()),
            Arc::new(self.partition_date.finish()),
        ];
        RecordBatch::try_new(self.schema, columns).map_err(Into::into)
    }
}

// ── TableCommand ─────────────────────────────────────────────────────────────

pub enum SummaryTableCommand {
    Write {
        records: Vec<TraceSummaryRecord>,
        respond_to: oneshot::Sender<Result<(), TraceEngineError>>,
    },
    Optimize {
        respond_to: oneshot::Sender<Result<(), TraceEngineError>>,
    },
    Vacuum {
        retention_hours: u64,
        respond_to: oneshot::Sender<Result<(), TraceEngineError>>,
    },
    Shutdown,
}

async fn build_summary_url(object_store: &ObjectStore) -> Result<Url, TraceEngineError> {
    let mut base = object_store.get_base_url()?;
    let mut path = base.path().to_string();
    if !path.ends_with('/') {
        path.push('/');
    }
    path.push_str(SUMMARY_TABLE_NAME);
    base.set_path(&path);
    Ok(base)
}

async fn create_summary_table(
    object_store: &ObjectStore,
    table_url: Url,
    schema: SchemaRef,
) -> Result<DeltaTable, TraceEngineError> {
    info!(
        "Creating trace summary table [{}://.../{} ]",
        table_url.scheme(),
        table_url
            .path_segments()
            .and_then(|mut s| s.next_back())
            .unwrap_or(SUMMARY_TABLE_NAME)
    );
    let store = object_store.as_dyn_object_store();
    let table = DeltaTableBuilder::from_url(table_url.clone())?
        .with_storage_backend(store, table_url)
        .build()?;
    let delta_fields = arrow_schema_to_delta(&schema);
    table
        .create()
        .with_table_name(SUMMARY_TABLE_NAME)
        .with_columns(delta_fields)
        .with_partition_columns(vec![PARTITION_DATE_COL.to_string()])
        // Only collect min/max statistics for non-binary columns.
        // trace_id (FixedSizeBinary) has no meaningful ordering for file-level pruning.
        .with_configuration_property(
            TableProperty::DataSkippingStatsColumns,
            Some("start_time,end_time,service_name,duration_ms,status_code,span_count,error_count,partition_date"),
        )
        .await
        .map_err(Into::into)
}

async fn build_or_create_summary_table(
    object_store: &ObjectStore,
    schema: SchemaRef,
) -> Result<DeltaTable, TraceEngineError> {
    register_cloud_logstore_factories();
    let table_url = build_summary_url(object_store).await?;
    info!(
        "Loading trace summary table [{}://.../{} ]",
        table_url.scheme(),
        table_url
            .path_segments()
            .and_then(|mut s| s.next_back())
            .unwrap_or(SUMMARY_TABLE_NAME)
    );

    // Check whether a Delta log actually exists. For local tables, check the
    // filesystem directly. For remote tables, attempt a full load with the
    // explicit storage backend — required for S3/GCS/Azure where Delta Lake
    // cannot infer the object store from the URL scheme alone.
    let is_delta_table = if table_url.scheme() == "file" {
        if let Ok(path) = table_url.to_file_path() {
            if !path.exists() {
                info!("Creating directory for summary table: {:?}", path);
                std::fs::create_dir_all(&path)?;
            }
            path.join("_delta_log").exists()
        } else {
            false
        }
    } else {
        let store = object_store.as_dyn_object_store();
        match DeltaTableBuilder::from_url(table_url.clone()) {
            Ok(builder) => builder
                .with_storage_backend(store, table_url.clone())
                .load()
                .await
                .is_ok(),
            Err(_) => false,
        }
    };

    if is_delta_table {
        info!(
            "Loaded existing trace summary table [{}://.../{} ]",
            table_url.scheme(),
            table_url
                .path_segments()
                .and_then(|mut s| s.next_back())
                .unwrap_or(SUMMARY_TABLE_NAME)
        );
        let store = object_store.as_dyn_object_store();
        DeltaTableBuilder::from_url(table_url.clone())?
            .with_storage_backend(store, table_url)
            .load()
            .await
            .map_err(Into::into)
    } else {
        info!("Summary table does not exist, creating new table");
        create_summary_table(object_store, table_url, schema).await
    }
}

pub struct TraceSummaryDBEngine {
    schema: Arc<Schema>,
    table: Arc<AsyncRwLock<DeltaTable>>,
    pub ctx: Arc<SessionContext>,
    /// Shared atomic catalog — calls `swap()` to update the summary `TableProvider`
    /// without a deregister/register gap visible to concurrent readers.
    catalog: Arc<TraceCatalogProvider>,
    control: ControlTableEngine,
}

impl TraceSummaryDBEngine {
    /// Create a new `TraceSummaryDBEngine` using the provided shared `SessionContext` and catalog.
    ///
    /// The caller is responsible for passing the `SessionContext` and `TraceCatalogProvider`
    /// created by `TraceSpanDBEngine::new()`. This ensures both `trace_spans` and
    /// `trace_summaries` share the same context and atomic DashMap-backed catalog.
    pub async fn new(
        object_store: &ObjectStore,
        ctx: Arc<SessionContext>,
        catalog: Arc<TraceCatalogProvider>,
    ) -> Result<Self, TraceEngineError> {
        let schema = Arc::new(create_summary_schema());
        let delta_table = build_or_create_summary_table(object_store, schema.clone()).await?;
        // A freshly-created table has no committed Parquet files yet — table_provider()
        // returns an error in that case. Defer registration until the first write.
        if let Ok(provider) = delta_table.table_provider().await {
            catalog.swap(SUMMARY_TABLE_NAME, provider);
        } else {
            info!("Empty summary table at init — deferring catalog registration until first write");
        }

        let control = ControlTableEngine::new(object_store, get_pod_id()).await?;

        Ok(TraceSummaryDBEngine {
            schema,
            table: Arc::new(AsyncRwLock::new(delta_table)),
            ctx,
            catalog,
            control,
        })
    }

    fn build_batch(
        &self,
        records: Vec<TraceSummaryRecord>,
    ) -> Result<RecordBatch, TraceEngineError> {
        let mut builder = TraceSummaryBatchBuilder::new(self.schema.clone(), records.len());
        for rec in &records {
            builder.append(rec)?;
        }
        builder.finish()
    }

    async fn write_records(
        &self,
        records: Vec<TraceSummaryRecord>,
    ) -> Result<(), TraceEngineError> {
        let count = records.len();
        info!("Writing {} trace summaries", count);
        let batch = self.build_batch(records)?;

        let mut table_guard = self.table.write().await;
        // update_incremental is intentionally omitted here.
        //
        // This engine runs as a single-writer actor — no other process commits to this
        // Delta table, so the in-memory state is always current. Calling update_incremental
        // can mutate table_guard into a corrupted intermediate state before the error
        // propagates, producing a DeltaTable whose snapshot may not reflect newly written
        // files — causing stale query results until restart.

        let current_table = table_guard.clone();
        let updated_table = current_table
            .write(vec![batch])
            .with_save_mode(deltalake::protocol::SaveMode::Append)
            .with_partition_columns(vec![PARTITION_DATE_COL.to_string()])
            .await?;

        let new_provider = updated_table.table_provider().await?;
        // Atomic single-step swap — no deregister/register gap where queries see "not found".
        self.catalog.swap(SUMMARY_TABLE_NAME, new_provider);
        updated_table.update_datafusion_session(&self.ctx.state())?;

        *table_guard = updated_table;
        info!("Summary table updated with {} records", count);
        Ok(())
    }

    async fn optimize_table(&self) -> Result<(), TraceEngineError> {
        let mut table_guard = self.table.write().await;
        let (updated_table, _metrics) = table_guard
            .clone()
            .optimize()
            .with_target_size(std::num::NonZero::new(128 * 1024 * 1024).unwrap())
            .with_type(OptimizeType::ZOrder(vec![
                START_TIME_COL.to_string(),
                SERVICE_NAME_COL.to_string(),
            ]))
            .await?;

        self.catalog
            .swap(SUMMARY_TABLE_NAME, updated_table.table_provider().await?);
        updated_table.update_datafusion_session(&self.ctx.state())?;
        *table_guard = updated_table;
        Ok(())
    }

    async fn vacuum_table(&self, retention_hours: u64) -> Result<(), TraceEngineError> {
        let mut table_guard = self.table.write().await;
        let (updated_table, _metrics) = table_guard
            .clone()
            .vacuum()
            .with_retention_period(chrono::Duration::hours(retention_hours as i64))
            .with_enforce_retention_duration(false)
            .await?;

        self.catalog
            .swap(SUMMARY_TABLE_NAME, updated_table.table_provider().await?);
        updated_table.update_datafusion_session(&self.ctx.state())?;
        *table_guard = updated_table;
        Ok(())
    }

    /// Refresh the in-memory Delta table snapshot from shared object storage.
    ///
    /// Runs periodically on every pod so that read pods pick up commits written
    /// by the write pod. Safety: clones the table before `update_incremental` so a
    /// failure leaves the original guard intact.
    async fn refresh_table(&self) -> Result<(), TraceEngineError> {
        let mut table_guard = self.table.write().await;
        let current_version = table_guard.version();

        let mut refreshed = table_guard.clone();
        match refreshed.update_incremental(None).await {
            Ok(_) => {
                if refreshed.version() > current_version {
                    info!(
                        "Summary table refreshed: v{:?} → v{:?}",
                        current_version,
                        refreshed.version()
                    );
                    let new_provider = refreshed.table_provider().await?;
                    // Atomic swap — no gap between deregister and register.
                    self.catalog.swap(SUMMARY_TABLE_NAME, new_provider);
                    refreshed.update_datafusion_session(&self.ctx.state())?;
                    *table_guard = refreshed;
                }
            }
            Err(e) => {
                // Tolerate: empty tables (no log yet), transient network errors.
                // These are expected on freshly-created tables and do not indicate a bug.
                debug!("Summary table refresh skipped: {}", e);
            }
        }
        Ok(())
    }

    /// Try to claim and run the summary optimize task via the control table.
    async fn try_run_optimize(&self, interval_hours: u64) {
        match self.control.try_claim_task(TASK_SUMMARY_OPTIMIZE).await {
            Ok(true) => match self.optimize_table().await {
                Ok(()) => {
                    if let Err(e) = self.vacuum_table(0).await {
                        error!("Post-optimize vacuum failed: {}", e);
                    }

                    let _ = self
                        .control
                        .release_task(
                            TASK_SUMMARY_OPTIMIZE,
                            chrono::Duration::hours(interval_hours as i64),
                        )
                        .await;
                }
                Err(e) => {
                    error!("Summary optimize failed: {}", e);
                    let _ = self
                        .control
                        .release_task_on_failure(TASK_SUMMARY_OPTIMIZE)
                        .await;
                }
            },
            Ok(false) => { /* not due or another pod owns it */ }
            Err(e) => error!("Summary optimize claim check failed: {}", e),
        }
    }

    #[instrument(skip_all, name = "summary_engine_actor")]
    pub fn start_actor(
        self,
        compaction_interval_hours: u64,
        refresh_interval_secs: u64,
    ) -> (
        mpsc::Sender<SummaryTableCommand>,
        tokio::task::JoinHandle<()>,
    ) {
        let (tx, mut rx) = mpsc::channel::<SummaryTableCommand>(100);

        let handle = tokio::spawn(async move {
            info!(refresh_interval_secs, "TraceSummaryDBEngine actor started");

            // Poll every 5 minutes — the actual schedule is in the control table.
            let mut scheduler_ticker = interval(Duration::from_secs(5 * 60));
            scheduler_ticker.tick().await; // skip immediate tick

            // Refresh ticker: picks up commits from the write pod on shared storage.
            // Every pod must refresh its own in-memory snapshot independently.
            // Clamp to 1s minimum — tokio::time::interval panics on Duration::ZERO.
            let mut refresh_ticker = interval(Duration::from_secs(refresh_interval_secs.max(1)));
            refresh_ticker.tick().await; // skip immediate tick

            loop {
                tokio::select! {
                    Some(cmd) = rx.recv() => {
                        match cmd {
                            SummaryTableCommand::Write { records, respond_to } => {
                                let result = self.write_records(records).await;
                                if let Err(ref e) = result {
                                    error!("Summary write failed: {}", e);
                                }
                                let _ = respond_to.send(result);
                            }
                            SummaryTableCommand::Optimize { respond_to } => {
                                // Direct admin request — bypass control table.
                                // Response is sent before vacuum so callers aren't blocked
                                // on the potentially slow file-deletion pass.
                                let _ = respond_to.send(self.optimize_table().await);
                                if let Err(e) = self.vacuum_table(0).await {
                                    error!("Post-optimize vacuum failed: {}", e);
                                }
                            }
                            SummaryTableCommand::Vacuum { retention_hours, respond_to } => {
                                let _ = respond_to.send(self.vacuum_table(retention_hours).await);
                            }
                            SummaryTableCommand::Shutdown => {
                                info!("TraceSummaryDBEngine actor shutting down");
                                break;
                            }
                        }
                    }
                    _ = scheduler_ticker.tick() => {
                        self.try_run_optimize(compaction_interval_hours).await;
                    }
                    _ = refresh_ticker.tick() => {
                        if let Err(e) = self.refresh_table().await {
                            error!("Summary table refresh failed: {}", e);
                        }
                    }
                }
            }
        });

        (tx, handle)
    }
}

// ── Service ──────────────────────────────────────────────────────────────────

pub struct TraceSummaryService {
    engine_tx: mpsc::Sender<SummaryTableCommand>,
    engine_handle: tokio::task::JoinHandle<()>,
    pub query_service: TraceSummaryQueries,
}

impl TraceSummaryService {
    pub async fn new(
        object_store: &ObjectStore,
        compaction_interval_hours: u64,
        ctx: Arc<SessionContext>,
        catalog: Arc<TraceCatalogProvider>,
        refresh_interval_secs: u64,
    ) -> Result<Self, TraceEngineError> {
        let engine = TraceSummaryDBEngine::new(object_store, ctx, catalog).await?;
        let engine_ctx = engine.ctx.clone();
        let (engine_tx, engine_handle) =
            engine.start_actor(compaction_interval_hours, refresh_interval_secs);

        Ok(TraceSummaryService {
            engine_tx,
            engine_handle,
            query_service: TraceSummaryQueries::new(engine_ctx),
        })
    }

    /// Write a batch of `TraceSummaryRecord`s to the Delta Lake summary table.
    pub async fn write_summaries(
        &self,
        records: Vec<TraceSummaryRecord>,
    ) -> Result<(), TraceEngineError> {
        let (tx, rx) = oneshot::channel();
        self.engine_tx
            .send(SummaryTableCommand::Write {
                records,
                respond_to: tx,
            })
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;
        rx.await.map_err(|_| TraceEngineError::ChannelClosed)?
    }

    pub async fn optimize(&self) -> Result<(), TraceEngineError> {
        let (tx, rx) = oneshot::channel();
        self.engine_tx
            .send(SummaryTableCommand::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) = oneshot::channel();
        self.engine_tx
            .send(SummaryTableCommand::Vacuum {
                retention_hours,
                respond_to: tx,
            })
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;
        rx.await.map_err(|_| TraceEngineError::ChannelClosed)?
    }

    /// Signal shutdown without consuming `self` — safe to call from `Arc<TraceSummaryService>`.
    pub async fn signal_shutdown(&self) {
        info!("TraceSummaryService signaling shutdown");
        let _ = self.engine_tx.send(SummaryTableCommand::Shutdown).await;
    }

    pub async fn shutdown(self) -> Result<(), TraceEngineError> {
        info!("TraceSummaryService shutting down");
        self.engine_tx
            .send(SummaryTableCommand::Shutdown)
            .await
            .map_err(|_| TraceEngineError::ChannelClosed)?;
        if let Err(e) = self.engine_handle.await {
            error!("Summary engine handle error: {}", e);
        }
        info!("TraceSummaryService shutdown complete");
        Ok(())
    }
}

// ── Queries ──────────────────────────────────────────────────────────────────

pub struct TraceSummaryQueries {
    ctx: Arc<SessionContext>,
}

impl TraceSummaryQueries {
    pub fn new(ctx: Arc<SessionContext>) -> Self {
        Self { ctx }
    }

    /// Get paginated traces from the Delta Lake summary table.
    ///
    /// The first step is a `GROUP BY trace_id` dedup query that merges any duplicate
    /// rows (from late-arriving spans) using the same rules as `TraceAggregator`:
    ///   - `SUM` for span/error counts, `MIN`/`MAX` for times, `MAX` for status_code
    ///   - `FIRST_VALUE` ordered by `span_count DESC` for string fields
    ///   - `array_distinct(flatten(array_agg(...)))` for entity/queue ID lists (full union)
    ///
    ///   Time filters are pushed into the SQL WHERE clause for partition pruning.
    ///
    ///   Secondary filters (service, errors, cursor) apply to the deduplicated DataFrame.
    pub async fn get_paginated_traces(
        &self,
        filters: &TraceFilters,
    ) -> Result<TracePaginationResponse, TraceEngineError> {
        let limit = filters.limit.unwrap_or(50) as usize;
        let direction = filters.direction.as_deref().unwrap_or("next");

        // ── Dedup: time-filtered GROUP BY trace_id (DataFrame API) ───────────
        use crate::parquet::tracing::queries::{date_lit, ts_lit};
        use datafusion::functions_aggregate::expr_fn::{array_agg, first_value, max, min, sum};
        use datafusion::functions_nested::set_ops::array_distinct;

        let mut df = self.ctx.table(SUMMARY_TABLE_NAME).await?;

        // ① Partition date — directory-level pruning (skips entire partition folders)
        // ② start_time     — row-group-level pruning within matched files
        if let Some(start) = filters.start_time {
            df = df.filter(col(PARTITION_DATE_COL).gt_eq(date_lit(&start)))?;
            df = df.filter(col(START_TIME_COL).gt_eq(ts_lit(&start)))?;
        }
        if let Some(end) = filters.end_time {
            df = df.filter(col(PARTITION_DATE_COL).lt_eq(date_lit(&end)))?;
            df = df.filter(col(START_TIME_COL).lt(ts_lit(&end)))?;
        }

        // ORDER BY specs for FIRST_VALUE aggregates
        // span_count DESC NULLS LAST, end_time DESC NULLS LAST
        let by_span_end: Vec<SortExpr> = vec![
            col(SPAN_COUNT_COL).sort(false, false),
            col(END_TIME_COL).sort(false, false),
        ];
        // status_code DESC, span_count DESC
        let by_status_span: Vec<SortExpr> = vec![
            col(STATUS_CODE_COL).sort(false, false),
            col(SPAN_COUNT_COL).sort(false, false),
        ];

        // Phase 1: aggregate
        // _max_end_us / _min_start_us are hidden Int64 columns used to compute
        // duration_ms post-aggregation (arithmetic across two aggregate exprs
        // cannot be expressed in a single aggregate slot).
        //
        // entity_ids / queue_ids: array_agg without FILTER is intentional.
        // array_flatten treats NULL outer-list elements as empty and skips them,
        // giving identical results to FILTER (WHERE IS NOT NULL) for the GROUP BY
        // case. Unlike the original SQL, this produces [] rather than NULL when
        // ALL rows have null IDs — the safer outcome for downstream deserialization.
        let mut df = df
            .aggregate(
                vec![col(TRACE_ID_COL)],
                vec![
                    min(col(START_TIME_COL)).alias(START_TIME_COL),
                    max(col(END_TIME_COL)).alias(END_TIME_COL),
                    max(df_cast(col(END_TIME_COL), DataType::Int64)).alias("_max_end_us"),
                    min(df_cast(col(START_TIME_COL), DataType::Int64)).alias("_min_start_us"),
                    max(col(STATUS_CODE_COL)).alias(STATUS_CODE_COL),
                    sum(col(SPAN_COUNT_COL)).alias(SPAN_COUNT_COL),
                    sum(col(ERROR_COUNT_COL)).alias(ERROR_COUNT_COL),
                    first_value(col(SERVICE_NAME_COL), by_span_end.clone()).alias(SERVICE_NAME_COL),
                    first_value(col(SCOPE_NAME_COL), by_span_end.clone()).alias(SCOPE_NAME_COL),
                    first_value(col(SCOPE_VERSION_COL), by_span_end.clone())
                        .alias(SCOPE_VERSION_COL),
                    first_value(col(ROOT_OPERATION_COL), by_span_end.clone())
                        .alias(ROOT_OPERATION_COL),
                    first_value(col(STATUS_MESSAGE_COL), by_status_span).alias(STATUS_MESSAGE_COL),
                    first_value(col(RESOURCE_ATTRIBUTES_COL), by_span_end)
                        .alias(RESOURCE_ATTRIBUTES_COL),
                    array_agg(col(ENTITY_IDS_COL)).alias("_entity_ids_raw"),
                    array_agg(col(QUEUE_IDS_COL)).alias("_queue_ids_raw"),
                ],
            )?
            // Phase 2: derive computed columns from hidden aggregates, then drop them
            .with_column(
                DURATION_MS_COL,
                (col("_max_end_us") - col("_min_start_us")) / lit(1000i64),
            )?
            .with_column(
                ENTITY_IDS_COL,
                array_distinct(flatten(col("_entity_ids_raw"))),
            )?
            .with_column(
                QUEUE_IDS_COL,
                array_distinct(flatten(col("_queue_ids_raw"))),
            )?
            .drop_columns(&[
                "_max_end_us",
                "_min_start_us",
                "_entity_ids_raw",
                "_queue_ids_raw",
            ])?;

        // ── Secondary filters ────────────────────────────────────────────────
        if let Some(ref svc) = filters.service_name {
            df = df.filter(col(SERVICE_NAME_COL).eq(lit(svc.as_str())))?;
        }
        match filters.has_errors {
            Some(true) => {
                df = df.filter(col(ERROR_COUNT_COL).gt(lit(0i64)))?;
            }
            Some(false) => {
                df = df.filter(col(ERROR_COUNT_COL).eq(lit(0i64)))?;
            }
            None => {}
        }
        if let Some(sc) = filters.status_code {
            df = df.filter(col(STATUS_CODE_COL).eq(lit(sc)))?;
        }

        // ── entity_uid filter via array_has on List column ────────────────
        if let Some(ref uid) = filters.entity_uid {
            df = df.filter(datafusion::functions_nested::expr_fn::array_has(
                col(ENTITY_IDS_COL),
                lit(uid.as_str()),
            ))?;
        }

        // ── queue_uid filter via array_has on List column ─────────────────
        if let Some(ref uid) = filters.queue_uid {
            df = df.filter(datafusion::functions_nested::expr_fn::array_has(
                col(QUEUE_IDS_COL),
                lit(uid.as_str()),
            ))?;
        }

        // ── trace_ids IN filter ──────────────────────────────────────────────
        if let Some(ref ids) = filters.trace_ids {
            if !ids.is_empty() {
                let binary_ids: Vec<Expr> = ids
                    .iter()
                    .filter_map(|hex| TraceId::hex_to_bytes(hex).ok())
                    .map(|b| lit(ScalarValue::Binary(Some(b))))
                    .collect();
                if !binary_ids.is_empty() {
                    df = df.filter(col(TRACE_ID_COL).in_list(binary_ids, false))?;
                }
            }
        }

        // ── Cursor filter in DataFusion ──────────────────────────────────────
        // Equivalent to Postgres: `(start_time, trace_id) < (cursor_time, cursor_id)`
        // for "next" or `> (cursor_time, cursor_id)` for "previous".
        if let (Some(cursor_time), Some(ref cursor_id)) =
            (filters.cursor_start_time, &filters.cursor_trace_id)
        {
            if let Ok(cursor_bytes) = TraceId::hex_to_bytes(cursor_id) {
                let cursor_ts = lit(ScalarValue::TimestampMicrosecond(
                    Some(cursor_time.timestamp_micros()),
                    Some("UTC".into()),
                ));
                let cursor_tid = lit(ScalarValue::Binary(Some(cursor_bytes)));
                let cursor_expr = if direction == "previous" {
                    col(START_TIME_COL)
                        .gt(cursor_ts.clone())
                        .or(col(START_TIME_COL)
                            .eq(cursor_ts)
                            .and(col(TRACE_ID_COL).gt(cursor_tid)))
                } else {
                    col(START_TIME_COL)
                        .lt(cursor_ts.clone())
                        .or(col(START_TIME_COL)
                            .eq(cursor_ts)
                            .and(col(TRACE_ID_COL).lt(cursor_tid)))
                };
                df = df.filter(cursor_expr)?;
            }
        }

        // ── Attribute filters via span lookup → IN list ──────────────────────
        // Requires shared SessionContext (trace_spans must be registered in self.ctx).
        // We execute the span query eagerly to collect matching trace IDs, then filter
        // the summaries DataFrame with an IN-list predicate. This avoids a cross-table
        // JOIN that causes DataFusion to report ambiguous `trace_id` column references.
        if let Some(ref attr_filters) = filters.attribute_filters {
            if !attr_filters.is_empty() {
                let mut spans_df = self.ctx.table("trace_spans").await?.select_columns(&[
                    TRACE_ID_COL,
                    START_TIME_COL,
                    SEARCH_BLOB_COL,
                ])?;

                // Time predicates on spans for partition pruning
                if let Some(start) = filters.start_time {
                    spans_df = spans_df.filter(col(START_TIME_COL).gt_eq(lit(
                        ScalarValue::TimestampMicrosecond(
                            Some(start.timestamp_micros()),
                            Some("UTC".into()),
                        ),
                    )))?;
                }
                if let Some(end) = filters.end_time {
                    spans_df = spans_df.filter(col(START_TIME_COL).lt(lit(
                        ScalarValue::TimestampMicrosecond(
                            Some(end.timestamp_micros()),
                            Some("UTC".into()),
                        ),
                    )))?;
                }

                // OR-match each filter against search_blob.
                // normalize_attr_filter converts "key:value" → "%key=value%" so the LIKE
                // pattern matches the new pipe-bounded `|key=value|` blob format.
                let mut attr_expr: Option<Expr> = None;
                for f in attr_filters {
                    let pattern = crate::parquet::tracing::queries::normalize_attr_filter(f);
                    let cond = match_attr_expr(col(SEARCH_BLOB_COL), lit(pattern));
                    attr_expr = Some(match attr_expr {
                        None => cond,
                        Some(e) => e.or(cond),
                    });
                }
                if let Some(expr) = attr_expr {
                    spans_df = spans_df.filter(expr)?;
                }

                // Collect matching trace IDs eagerly, then apply as IN-list filter.
                // Use HashSet for O(1) dedup instead of O(n²) Vec::contains().
                let span_batches = spans_df.select_columns(&[TRACE_ID_COL])?.collect().await?;
                let mut seen_ids: std::collections::HashSet<Vec<u8>> =
                    std::collections::HashSet::new();
                let mut binary_ids: Vec<Expr> = Vec::new();
                for batch in &span_batches {
                    // trace_id may be FixedSizeBinary(16) or Binary after Delta round-trip.
                    // Cast to Binary to handle both uniformly.
                    if let Some(col_ref) = batch.column_by_name(TRACE_ID_COL) {
                        let casted = compute::cast(col_ref, &DataType::Binary)?;
                        let col_arr =
                            casted
                                .as_any()
                                .downcast_ref::<BinaryArray>()
                                .ok_or_else(|| {
                                    TraceEngineError::DowncastError("trace_id to BinaryArray")
                                })?;
                        for i in 0..batch.num_rows() {
                            let id_bytes = col_arr.value(i).to_vec();
                            if seen_ids.insert(id_bytes.clone()) {
                                binary_ids.push(lit(ScalarValue::Binary(Some(id_bytes))));
                            }
                        }
                    }
                }

                if !binary_ids.is_empty() {
                    df = df.filter(col(TRACE_ID_COL).in_list(binary_ids, false))?;
                } else {
                    // No matching spans → return empty result
                    df = df.filter(lit(false))?;
                }
            }
        }

        // ── Sort: DESC for "next", ASC for "previous" ────────────────────────
        // "previous" direction fetches the oldest limit+1 items newer than the cursor,
        // which matches the original Rust post-reversal behavior.
        df = if direction == "previous" {
            df.sort(vec![
                col(START_TIME_COL).sort(true, true),
                col(TRACE_ID_COL).sort(true, true),
            ])?
        } else {
            df.sort(vec![
                col(START_TIME_COL).sort(false, false),
                col(TRACE_ID_COL).sort(false, false),
            ])?
        };

        // ── LIMIT pushed into DataFusion (fetch limit+1 to detect next page) ─
        df = df.limit(0, Some(limit + 1))?;

        let batches = df.collect().await?;
        let mut items = batches_to_trace_list_items(batches)?;

        let has_more = items.len() > limit;
        if has_more {
            items.pop(); // remove N+1 sentinel
        }

        // Direction-specific cursor logic — mirrors the original PostgreSQL implementation.
        //
        // "next" (DESC order): items are newest-first. The sentinel tells us if older
        // items exist (has_next). Cursor presence means we navigated forward, so newer
        // items exist behind us (has_previous).
        //
        // "previous" (ASC order): items are oldest-first (closest-to-cursor first).
        // The sentinel tells us if even more newer items exist (has_previous). Cursor
        // presence means we navigated backward, so older items exist ahead (has_next).
        // Items stay in ASC order — no reversal — matching PG behavior exactly.
        let (has_next, next_cursor, has_previous, previous_cursor) = match direction {
            "next" => {
                let next_cursor = if has_more {
                    items.last().map(|item| TraceCursor {
                        start_time: item.start_time,
                        trace_id: item.trace_id.clone(),
                    })
                } else {
                    None
                };

                let previous_cursor = items.first().map(|item| TraceCursor {
                    start_time: item.start_time,
                    trace_id: item.trace_id.clone(),
                });

                (
                    has_more,
                    next_cursor,
                    filters.cursor_start_time.is_some(),
                    previous_cursor,
                )
            }
            "previous" => {
                // ASC order: items.last() is the newest (largest start_time).
                // To continue backward (fetch even newer items), the cursor must
                // point past the current page's newest item so `> cursor` excludes
                // everything already returned.
                let previous_cursor = if has_more {
                    items.last().map(|item| TraceCursor {
                        start_time: item.start_time,
                        trace_id: item.trace_id.clone(),
                    })
                } else {
                    None
                };

                // items.first() is the oldest (smallest start_time).
                // To go forward (back toward newer-first / DESC pages), the cursor
                // must point at the oldest item so `< cursor` fetches older items.
                let next_cursor = items.first().map(|item| TraceCursor {
                    start_time: item.start_time,
                    trace_id: item.trace_id.clone(),
                });

                (
                    filters.cursor_start_time.is_some(),
                    next_cursor,
                    has_more,
                    previous_cursor,
                )
            }
            _ => (false, None, false, None),
        };

        Ok(TracePaginationResponse {
            items,
            has_next,
            next_cursor,
            has_previous,
            previous_cursor,
        })
    }
}

// ── Arrow → TraceListItem conversion ─────────────────────────────────────────

/// Extract attributes from a MapArray at a given row index.
fn extract_map_attributes(map_array: &MapArray, row_idx: usize) -> Vec<Attribute> {
    if map_array.is_null(row_idx) {
        return Vec::new();
    }
    let entry = map_array.value(row_idx);
    let Some(struct_array) = entry.as_any().downcast_ref::<StructArray>() else {
        tracing::warn!("extract_map_attributes: failed to downcast to StructArray");
        return Vec::new();
    };
    let Some(keys_arr) = compute::cast(struct_array.column(0).as_ref(), &DataType::Utf8).ok()
    else {
        tracing::warn!("extract_map_attributes: failed to cast keys to Utf8");
        return Vec::new();
    };
    let Some(keys) = keys_arr.as_any().downcast_ref::<StringArray>() else {
        tracing::warn!("extract_map_attributes: failed to downcast keys to StringArray");
        return Vec::new();
    };
    let Some(values_arr) = compute::cast(struct_array.column(1).as_ref(), &DataType::Utf8).ok()
    else {
        tracing::warn!("extract_map_attributes: failed to cast values to Utf8");
        return Vec::new();
    };
    let Some(values) = values_arr.as_any().downcast_ref::<StringArray>() else {
        tracing::warn!("extract_map_attributes: failed to downcast values to StringArray");
        return Vec::new();
    };

    (0..struct_array.len())
        .map(|i| Attribute {
            key: keys.value(i).to_string(),
            value: serde_json::from_str(values.value(i)).unwrap_or(serde_json::Value::Null),
        })
        .collect()
}

/// Extract a `Vec<String>` from a nullable `ListArray` at a given row index.
fn extract_list_strings(list: Option<&ListArray>, row_idx: usize) -> Vec<String> {
    let Some(list) = list else {
        return Vec::new();
    };
    if list.is_null(row_idx) {
        return Vec::new();
    }
    let inner = list.value(row_idx);
    let str_arr = compute::cast(&inner, &DataType::Utf8)
        .ok()
        .and_then(|a| a.as_any().downcast_ref::<StringArray>().cloned());
    match str_arr {
        Some(arr) => (0..arr.len())
            .filter(|i| !arr.is_null(*i))
            .map(|i| arr.value(i).to_string())
            .collect(),
        None => Vec::new(),
    }
}

fn batches_to_trace_list_items(
    batches: Vec<RecordBatch>,
) -> Result<Vec<TraceListItem>, TraceEngineError> {
    let mut items = Vec::new();

    for batch in &batches {
        // trace_id may come back as FixedSizeBinary(16) or Binary depending on
        // whether DataFusion/Delta round-tripped the schema. Handle both.
        let trace_id_col = batch.column_by_name(TRACE_ID_COL).ok_or_else(|| {
            TraceEngineError::UnsupportedOperation("missing trace_id column".into())
        })?;
        let trace_id_binary = compute::cast(trace_id_col, &DataType::Binary)?;
        let trace_ids = trace_id_binary
            .as_any()
            .downcast_ref::<BinaryArray>()
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("trace_id cast to BinaryArray failed".into())
            })?;

        // Cast all string/dictionary columns to Utf8 uniformly (handles Utf8View,
        // Dictionary(Int32, Utf8), LargeUtf8, etc.).
        let svc_arr = compute::cast(
            batch.column_by_name(SERVICE_NAME_COL).ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing service_name column".into())
            })?,
            &DataType::Utf8,
        )?;
        let service_names = svc_arr
            .as_any()
            .downcast_ref::<StringArray>()
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation(
                    "service_name cast to StringArray failed".into(),
                )
            })?;

        let scope_arr = compute::cast(
            batch.column_by_name(SCOPE_NAME_COL).ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing scope_name column".into())
            })?,
            &DataType::Utf8,
        )?;
        let scope_names = scope_arr
            .as_any()
            .downcast_ref::<StringArray>()
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation(
                    "scope_name cast to StringArray failed".into(),
                )
            })?;

        let scopev_arr = compute::cast(
            batch.column_by_name(SCOPE_VERSION_COL).ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing scope_version column".into())
            })?,
            &DataType::Utf8,
        )?;
        let scope_versions = scopev_arr
            .as_any()
            .downcast_ref::<StringArray>()
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation(
                    "scope_version cast to StringArray failed".into(),
                )
            })?;

        let root_arr = compute::cast(
            batch.column_by_name(ROOT_OPERATION_COL).ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing root_operation column".into())
            })?,
            &DataType::Utf8,
        )?;
        let root_operations = root_arr
            .as_any()
            .downcast_ref::<StringArray>()
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation(
                    "root_operation cast to StringArray failed".into(),
                )
            })?;

        let sm_arr = compute::cast(
            batch.column_by_name(STATUS_MESSAGE_COL).ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing status_message column".into())
            })?,
            &DataType::Utf8,
        )?;
        let status_messages = sm_arr
            .as_any()
            .downcast_ref::<StringArray>()
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation(
                    "status_message cast to StringArray failed".into(),
                )
            })?;

        let resource_attrs_map = batch
            .column_by_name(RESOURCE_ATTRIBUTES_COL)
            .and_then(|c| c.as_any().downcast_ref::<MapArray>())
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing resource_attributes column".into())
            })?;

        let entity_ids_list = batch
            .column_by_name(ENTITY_IDS_COL)
            .and_then(|c| c.as_any().downcast_ref::<ListArray>());

        let queue_ids_list = batch
            .column_by_name(QUEUE_IDS_COL)
            .and_then(|c| c.as_any().downcast_ref::<ListArray>());

        let start_times = batch
            .column_by_name(START_TIME_COL)
            .and_then(|c| c.as_any().downcast_ref::<TimestampMicrosecondArray>())
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing start_time column".into())
            })?;

        let end_times = batch
            .column_by_name(END_TIME_COL)
            .and_then(|c| c.as_any().downcast_ref::<TimestampMicrosecondArray>())
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing end_time column".into())
            })?;

        let durations = batch
            .column_by_name(DURATION_MS_COL)
            .and_then(|c| c.as_any().downcast_ref::<Int64Array>())
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing duration_ms column".into())
            })?;

        let status_codes = batch
            .column_by_name(STATUS_CODE_COL)
            .and_then(|c| c.as_any().downcast_ref::<Int32Array>())
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing status_code column".into())
            })?;

        let span_counts = batch
            .column_by_name(SPAN_COUNT_COL)
            .and_then(|c| c.as_any().downcast_ref::<Int64Array>())
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing span_count column".into())
            })?;

        let error_counts = batch
            .column_by_name(ERROR_COUNT_COL)
            .and_then(|c| c.as_any().downcast_ref::<Int64Array>())
            .ok_or_else(|| {
                TraceEngineError::UnsupportedOperation("missing error_count column".into())
            })?;

        for i in 0..batch.num_rows() {
            let trace_id_hex = hex::encode(trace_ids.value(i));

            let start_time = micros_to_datetime(start_times.value(i))?;
            let end_time = if end_times.is_null(i) {
                None
            } else {
                Some(micros_to_datetime(end_times.value(i))?)
            };
            let duration_ms = if durations.is_null(i) {
                None
            } else {
                Some(durations.value(i))
            };
            let error_count = error_counts.value(i);

            let resource_attributes = extract_map_attributes(resource_attrs_map, i);

            let entity_ids = extract_list_strings(entity_ids_list, i);
            let queue_ids = extract_list_strings(queue_ids_list, i);

            items.push(TraceListItem {
                trace_id: trace_id_hex,
                service_name: service_names.value(i).to_string(),
                scope_name: scope_names.value(i).to_string(),
                scope_version: scope_versions.value(i).to_string(),
                root_operation: root_operations.value(i).to_string(),
                start_time,
                end_time,
                duration_ms,
                status_code: status_codes.value(i),
                status_message: if status_messages.is_null(i) {
                    None
                } else {
                    Some(status_messages.value(i).to_string())
                },
                span_count: span_counts.value(i),
                has_errors: error_count > 0,
                error_count,
                resource_attributes,
                entity_ids,
                queue_ids,
            });
        }
    }

    Ok(items)
}

fn micros_to_datetime(micros: i64) -> Result<DateTime<Utc>, TraceEngineError> {
    DateTime::from_timestamp_micros(micros).ok_or(TraceEngineError::InvalidTimestamp(
        "out-of-range microsecond timestamp",
    ))
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::storage::ObjectStore;
    use scouter_settings::ObjectStorageSettings;
    use scouter_types::sql::TraceFilters;
    use scouter_types::{Attribute, SpanId, TraceId, TraceSpanRecord};
    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);
        }
    }

    fn make_test_object_store(storage_settings: &ObjectStorageSettings) -> ObjectStore {
        ObjectStore::new(storage_settings).unwrap()
    }

    /// Build a standalone `SessionContext` for test use with the scouter_tracing catalog
    /// configured as the default so unqualified table names resolve through our DashMap.
    fn make_test_ctx(object_store: &ObjectStore) -> Arc<SessionContext> {
        Arc::new(
            object_store
                .get_session_with_catalog(
                    crate::parquet::tracing::engine::TRACE_CATALOG_NAME,
                    "default",
                )
                .unwrap(),
        )
    }

    /// Create a `TraceCatalogProvider`, register it on `ctx`, and return it.
    /// Call this after `make_test_ctx` to get the catalog for standalone summary tests.
    fn make_test_catalog(ctx: &Arc<SessionContext>) -> Arc<TraceCatalogProvider> {
        use datafusion::catalog::CatalogProvider;
        let catalog = Arc::new(TraceCatalogProvider::new());
        ctx.register_catalog(
            crate::parquet::tracing::engine::TRACE_CATALOG_NAME,
            Arc::clone(&catalog) as Arc<dyn CatalogProvider>,
        );
        catalog
    }

    fn make_summary(
        trace_id_bytes: [u8; 16],
        service_name: &str,
        error_count: i64,
        resource_attributes: Vec<Attribute>,
    ) -> TraceSummaryRecord {
        let now = Utc::now();
        TraceSummaryRecord {
            trace_id: TraceId::from_bytes(trace_id_bytes),
            service_name: service_name.to_string(),
            scope_name: "test.scope".to_string(),
            scope_version: String::new(),
            root_operation: "root_op".to_string(),
            start_time: now,
            end_time: Some(now + chrono::Duration::milliseconds(200)),
            status_code: if error_count > 0 { 2 } else { 0 },
            status_message: if error_count > 0 {
                "Internal Server Error".to_string()
            } else {
                "OK".to_string()
            },
            span_count: 3,
            error_count,
            resource_attributes,
            entity_ids: vec![],
            queue_ids: vec![],
        }
    }

    /// Basic write + paginate round-trip: writes two summaries and verifies both appear.
    #[tokio::test]
    async fn test_summary_write_and_paginate_basic() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let object_store = make_test_object_store(&storage_settings);
        let ctx = make_test_ctx(&object_store);
        let catalog = make_test_catalog(&ctx);
        let service = TraceSummaryService::new(&object_store, 24, ctx, catalog, 10).await?;

        let s1 = make_summary([1u8; 16], "svc_a", 0, vec![]);
        let s2 = make_summary([2u8; 16], "svc_b", 0, vec![]);
        service.write_summaries(vec![s1, s2]).await?;
        tokio::time::sleep(tokio::time::Duration::from_secs(3)).await;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);
        let filters = TraceFilters {
            service_name: None,
            has_errors: None,
            status_code: None,
            start_time: Some(start),
            end_time: Some(end),
            limit: Some(25),
            cursor_start_time: None,
            cursor_trace_id: None,
            direction: None,
            attribute_filters: None,
            trace_ids: None,
            entity_uid: None,
            queue_uid: None,
        };

        let response = service.query_service.get_paginated_traces(&filters).await?;
        assert!(
            response.items.len() >= 2,
            "Expected at least 2 items, got {}",
            response.items.len()
        );

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

    /// `has_errors = Some(true)` returns only error traces; `Some(false)` returns only non-errors.
    #[tokio::test]
    async fn test_summary_has_errors_filter() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let object_store = make_test_object_store(&storage_settings);
        let ctx = make_test_ctx(&object_store);
        let catalog = make_test_catalog(&ctx);
        let service = TraceSummaryService::new(&object_store, 24, ctx, catalog, 10).await?;

        let ok_summary = make_summary([3u8; 16], "svc", 0, vec![]);
        let err_summary = make_summary([4u8; 16], "svc", 2, vec![]);
        service
            .write_summaries(vec![ok_summary, err_summary])
            .await?;
        tokio::time::sleep(tokio::time::Duration::from_secs(3)).await;

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

        let base_filters = TraceFilters {
            service_name: None,
            has_errors: None,
            status_code: None,
            start_time: Some(start),
            end_time: Some(end),
            limit: Some(25),
            cursor_start_time: None,
            cursor_trace_id: None,
            direction: None,
            attribute_filters: None,
            trace_ids: None,
            entity_uid: None,
            queue_uid: None,
        };

        // has_errors = true → only error trace
        let mut filters_err = base_filters.clone();
        filters_err.has_errors = Some(true);
        let errors_only = service
            .query_service
            .get_paginated_traces(&filters_err)
            .await?;
        for item in &errors_only.items {
            assert!(
                item.error_count > 0,
                "Expected error trace, got: {:?}",
                item
            );
        }
        assert!(
            !errors_only.items.is_empty(),
            "Expected at least one error trace"
        );

        // has_errors = false → only non-error traces
        let mut filters_ok = base_filters.clone();
        filters_ok.has_errors = Some(false);
        let no_errors = service
            .query_service
            .get_paginated_traces(&filters_ok)
            .await?;
        for item in &no_errors.items {
            assert_eq!(
                item.error_count, 0,
                "Expected non-error trace, got error_count={}",
                item.error_count
            );
        }
        assert!(
            !no_errors.items.is_empty(),
            "Expected at least one non-error trace"
        );

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

    /// service_name filter returns only matching service traces.
    #[tokio::test]
    async fn test_summary_service_name_filter() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let object_store = make_test_object_store(&storage_settings);
        let ctx = make_test_ctx(&object_store);
        let catalog = make_test_catalog(&ctx);
        let service = TraceSummaryService::new(&object_store, 24, ctx, catalog, 10).await?;

        let s_alpha = make_summary([5u8; 16], "alpha_service", 0, vec![]);
        let s_beta = make_summary([6u8; 16], "beta_service", 0, vec![]);
        service.write_summaries(vec![s_alpha, s_beta]).await?;
        tokio::time::sleep(tokio::time::Duration::from_secs(3)).await;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);
        let filters = TraceFilters {
            service_name: Some("alpha_service".to_string()),
            has_errors: None,
            status_code: None,
            start_time: Some(start),
            end_time: Some(end),
            limit: Some(25),
            cursor_start_time: None,
            cursor_trace_id: None,
            direction: None,
            attribute_filters: None,
            trace_ids: None,
            entity_uid: None,
            queue_uid: None,
        };

        let response = service.query_service.get_paginated_traces(&filters).await?;
        assert!(
            !response.items.is_empty(),
            "Expected results for alpha_service"
        );
        for item in &response.items {
            assert_eq!(
                item.service_name, "alpha_service",
                "Expected only alpha_service items, got: {}",
                item.service_name
            );
        }

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

    /// trace_ids IN filter returns only the specified traces.
    #[tokio::test]
    async fn test_summary_trace_ids_filter() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let object_store = make_test_object_store(&storage_settings);
        let ctx = make_test_ctx(&object_store);
        let catalog = make_test_catalog(&ctx);
        let service = TraceSummaryService::new(&object_store, 24, ctx, catalog, 10).await?;

        let wanted_id = TraceId::from_bytes([7u8; 16]);
        let unwanted_id = TraceId::from_bytes([8u8; 16]);

        let s1 = make_summary([7u8; 16], "svc", 0, vec![]);
        let s2 = make_summary([8u8; 16], "svc", 0, vec![]);
        service.write_summaries(vec![s1, s2]).await?;
        tokio::time::sleep(tokio::time::Duration::from_secs(3)).await;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);
        let filters = TraceFilters {
            service_name: None,
            has_errors: None,
            status_code: None,
            start_time: Some(start),
            end_time: Some(end),
            limit: Some(25),
            cursor_start_time: None,
            cursor_trace_id: None,
            direction: None,
            attribute_filters: None,
            trace_ids: Some(vec![wanted_id.to_hex()]),
            entity_uid: None,
            queue_uid: None,
        };

        let response = service.query_service.get_paginated_traces(&filters).await?;
        assert_eq!(
            response.items.len(),
            1,
            "Expected exactly 1 item from trace_ids filter"
        );
        assert_eq!(
            response.items[0].trace_id,
            wanted_id.to_hex(),
            "Returned wrong trace_id"
        );
        assert_ne!(
            response.items[0].trace_id,
            unwanted_id.to_hex(),
            "Should not have returned unwanted trace_id"
        );

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

    /// Cursor pagination: first page → next → previous all return correct item counts.
    #[tokio::test]
    async fn test_summary_cursor_pagination() -> Result<(), TraceEngineError> {
        cleanup();
        let storage_settings = ObjectStorageSettings::default();
        let object_store = make_test_object_store(&storage_settings);
        let ctx = make_test_ctx(&object_store);
        let catalog = make_test_catalog(&ctx);
        let service = TraceSummaryService::new(&object_store, 24, ctx, catalog, 10).await?;

        let now = Utc::now();
        let summaries: Vec<TraceSummaryRecord> = (0u8..100)
            .map(|i| {
                let mut s = make_summary([i; 16], "svc", 0, vec![]);
                s.start_time = now - chrono::Duration::minutes(i as i64);
                s
            })
            .collect();
        service.write_summaries(summaries).await?;
        tokio::time::sleep(tokio::time::Duration::from_secs(3)).await;

        let mut filters = TraceFilters {
            start_time: Some(now - chrono::Duration::hours(2)),
            end_time: Some(now + chrono::Duration::hours(1)),
            limit: Some(50),
            ..Default::default()
        };

        // First page
        let first = service.query_service.get_paginated_traces(&filters).await?;
        assert_eq!(first.items.len(), 50, "first page: 50 items");
        assert!(
            first.next_cursor.is_some(),
            "first page: should have next_cursor"
        );

        // Next page
        let next_cur = first.next_cursor.clone().unwrap();
        filters.cursor_start_time = Some(next_cur.start_time);
        filters.cursor_trace_id = Some(next_cur.trace_id.clone());
        filters.direction = Some("next".to_string());
        let second = service.query_service.get_paginated_traces(&filters).await?;
        assert_eq!(second.items.len(), 50, "second page: 50 items");
        assert!(
            second.items[0].start_time <= next_cur.start_time,
            "second page first item must be <= cursor"
        );
        assert!(second.previous_cursor.is_some());

        // Previous page
        let prev_cur = second.previous_cursor.unwrap();
        filters.cursor_start_time = Some(prev_cur.start_time);
        filters.cursor_trace_id = Some(prev_cur.trace_id.clone());
        filters.direction = Some("previous".to_string());
        let prev = service.query_service.get_paginated_traces(&filters).await?;
        assert_eq!(prev.items.len(), 50, "previous page: 50 items");

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

    /// Attribute-filter JOIN path: only traces with matching span attributes are returned.
    #[tokio::test]
    async fn test_summary_attribute_filter_via_join() -> Result<(), TraceEngineError> {
        use crate::parquet::tracing::service::TraceSpanService;

        cleanup();
        let storage_settings = ObjectStorageSettings::default();

        // TraceSpanService owns the SessionContext (trace_spans registered in it)
        let span_service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;
        let shared_ctx = span_service.ctx.clone();

        // TraceSummaryService shares the same ctx + catalog — JOIN to trace_spans will work
        let summary_service = TraceSummaryService::new(
            &span_service.object_store,
            24,
            shared_ctx,
            span_service.catalog.clone(),
            10,
        )
        .await?;

        let now = Utc::now();
        let kafka_trace = TraceId::from_bytes([70u8; 16]);
        let plain_trace = TraceId::from_bytes([80u8; 16]);

        let kafka_span = make_span_record(
            &kafka_trace,
            SpanId::from_bytes([70u8; 8]),
            "svc",
            vec![Attribute {
                key: "component".to_string(),
                value: serde_json::Value::String("kafka".to_string()),
            }],
        );
        let plain_span =
            make_span_record(&plain_trace, SpanId::from_bytes([80u8; 8]), "svc", vec![]);
        span_service
            .write_spans(vec![kafka_span, plain_span])
            .await?;

        let mut kafka_summary = make_summary([70u8; 16], "svc", 0, vec![]);
        kafka_summary.start_time = now;
        let mut plain_summary = make_summary([80u8; 16], "svc", 0, vec![]);
        plain_summary.start_time = now;
        summary_service
            .write_summaries(vec![kafka_summary, plain_summary])
            .await?;

        tokio::time::sleep(tokio::time::Duration::from_secs(4)).await;

        let filters = TraceFilters {
            start_time: Some(now - chrono::Duration::hours(1)),
            end_time: Some(now + chrono::Duration::hours(1)),
            attribute_filters: Some(vec!["component:kafka".to_string()]),
            limit: Some(25),
            ..Default::default()
        };

        let response = summary_service
            .query_service
            .get_paginated_traces(&filters)
            .await?;

        assert!(
            !response.items.is_empty(),
            "attribute filter must return results"
        );
        assert!(
            response
                .items
                .iter()
                .all(|i| i.trace_id == kafka_trace.to_hex()),
            "only kafka trace should appear; got {:?}",
            response
                .items
                .iter()
                .map(|i| &i.trace_id)
                .collect::<Vec<_>>()
        );

        span_service.shutdown().await?;
        summary_service.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// queue_uid filter: only traces whose queue_ids contain the target UID are returned,
    /// and the matching trace's spans can be fetched by trace_id.
    #[tokio::test]
    async fn test_summary_queue_id_filter_and_span_lookup() -> Result<(), TraceEngineError> {
        use crate::parquet::tracing::service::TraceSpanService;

        cleanup();
        let storage_settings = ObjectStorageSettings::default();

        // TraceSpanService owns the SessionContext
        let span_service = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;
        let shared_ctx = span_service.ctx.clone();

        // TraceSummaryService shares the same ctx + catalog so JOIN path works
        let summary_service = TraceSummaryService::new(
            &span_service.object_store,
            24,
            shared_ctx,
            span_service.catalog.clone(),
            10,
        )
        .await?;

        let now = Utc::now();
        let queue_trace = TraceId::from_bytes([90u8; 16]);
        let plain_trace = TraceId::from_bytes([91u8; 16]);
        let target_queue_uid = "queue-record-abc123";

        // Write spans for both traces
        let queue_span = make_span_record(
            &queue_trace,
            SpanId::from_bytes([90u8; 8]),
            "svc_queue",
            vec![],
        );
        let plain_span = make_span_record(
            &plain_trace,
            SpanId::from_bytes([91u8; 8]),
            "svc_queue",
            vec![],
        );
        span_service
            .write_spans_direct(vec![queue_span, plain_span])
            .await?;

        // Write summaries: one with a matching queue_id, one without
        let mut queue_summary = make_summary([90u8; 16], "svc_queue", 0, vec![]);
        queue_summary.start_time = now;
        queue_summary.queue_ids = vec![target_queue_uid.to_string()];

        let mut plain_summary = make_summary([91u8; 16], "svc_queue", 0, vec![]);
        plain_summary.start_time = now;
        // queue_ids left empty — should not appear in results

        summary_service
            .write_summaries(vec![queue_summary, plain_summary])
            .await?;

        tokio::time::sleep(tokio::time::Duration::from_secs(4)).await;

        // ── Step 1: query summaries by queue_uid ─────────────────────────────────
        let filters = TraceFilters {
            start_time: Some(now - chrono::Duration::hours(1)),
            end_time: Some(now + chrono::Duration::hours(1)),
            queue_uid: Some(target_queue_uid.to_string()),
            limit: Some(25),
            ..Default::default()
        };

        let response = summary_service
            .query_service
            .get_paginated_traces(&filters)
            .await?;

        assert!(
            !response.items.is_empty(),
            "queue_uid filter must return at least one result"
        );
        assert!(
            response
                .items
                .iter()
                .all(|i| i.trace_id == queue_trace.to_hex()),
            "only the queue trace should appear; got {:?}",
            response
                .items
                .iter()
                .map(|i| &i.trace_id)
                .collect::<Vec<_>>()
        );

        // ── Step 2: fetch spans for the returned trace_id ─────────────────────────
        let returned_trace_id =
            TraceId::from_hex(&response.items[0].trace_id).expect("trace_id must be valid hex");
        let spans = span_service
            .query_service
            .get_trace_spans(
                Some(returned_trace_id.as_bytes()),
                None,
                Some(&(now - chrono::Duration::hours(1))),
                Some(&(now + chrono::Duration::hours(1))),
                None,
            )
            .await?;

        assert!(
            !spans.is_empty(),
            "should find spans for the returned trace_id"
        );

        span_service.shutdown().await?;
        summary_service.shutdown().await?;
        cleanup();
        Ok(())
    }

    /// Build a deterministic `TraceSpanRecord` for use in summary tests.
    fn make_span_record(
        trace_id: &TraceId,
        span_id: SpanId,
        service_name: &str,
        attributes: Vec<Attribute>,
    ) -> TraceSpanRecord {
        let now = Utc::now();
        TraceSpanRecord {
            created_at: now,
            trace_id: *trace_id,
            span_id,
            parent_span_id: None,
            flags: 1,
            trace_state: String::new(),
            scope_name: "test.scope".to_string(),
            scope_version: None,
            span_name: "op".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: serde_json::Value::Null,
            output: serde_json::Value::Null,
            service_name: service_name.to_string(),
            resource_attributes: vec![],
        }
    }

    /// `resource_attributes` survive a write → read round-trip.
    #[tokio::test]
    async fn test_summary_resource_attributes_roundtrip() -> Result<(), TraceEngineError> {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let object_store = make_test_object_store(&storage_settings);
        let ctx = make_test_ctx(&object_store);
        let catalog = make_test_catalog(&ctx);
        let service = TraceSummaryService::new(&object_store, 24, ctx, catalog, 10).await?;

        let attrs = vec![Attribute {
            key: "cloud.region".to_string(),
            value: serde_json::Value::String("us-east-1".to_string()),
        }];
        let summary = make_summary([9u8; 16], "svc", 0, attrs.clone());
        service.write_summaries(vec![summary]).await?;
        tokio::time::sleep(tokio::time::Duration::from_secs(3)).await;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);
        let filters = TraceFilters {
            service_name: None,
            has_errors: None,
            status_code: None,
            start_time: Some(start),
            end_time: Some(end),
            limit: Some(25),
            cursor_start_time: None,
            cursor_trace_id: None,
            direction: None,
            attribute_filters: None,
            trace_ids: Some(vec![TraceId::from_bytes([9u8; 16]).to_hex()]),
            entity_uid: None,
            queue_uid: None,
        };

        let response = service.query_service.get_paginated_traces(&filters).await?;
        assert_eq!(response.items.len(), 1, "Expected exactly 1 item");
        assert_eq!(
            response.items[0].resource_attributes.len(),
            1,
            "Expected 1 resource attribute"
        );
        assert_eq!(response.items[0].resource_attributes[0].key, "cloud.region");

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

    /// Regression test: multiple sequential writes must be immediately visible to queries.
    /// This catches the stale snapshot bug where re-registration doesn't refresh the
    /// DataFusion session's object store, causing queries after subsequent writes to
    /// return stale results.
    #[tokio::test]
    async fn test_summary_write_visibility_across_multiple_writes() -> Result<(), TraceEngineError>
    {
        cleanup();

        let storage_settings = ObjectStorageSettings::default();
        let object_store = make_test_object_store(&storage_settings);
        let ctx = make_test_ctx(&object_store);
        let catalog = make_test_catalog(&ctx);
        let service = TraceSummaryService::new(&object_store, 24, ctx, catalog, 10).await?;

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);
        let filters = TraceFilters {
            start_time: Some(start),
            end_time: Some(end),
            limit: Some(100),
            ..Default::default()
        };

        // Write batch #1 (2 summaries)
        let s1 = make_summary([0xA0; 16], "svc_vis", 0, vec![]);
        let s2 = make_summary([0xA1; 16], "svc_vis", 0, vec![]);
        service.write_summaries(vec![s1, s2]).await?;

        let response = service.query_service.get_paginated_traces(&filters).await?;
        assert_eq!(
            response.items.len(),
            2,
            "After write #1: expected 2 items, got {}",
            response.items.len()
        );

        // Write batch #2 (2 more summaries)
        let s3 = make_summary([0xA2; 16], "svc_vis", 0, vec![]);
        let s4 = make_summary([0xA3; 16], "svc_vis", 0, vec![]);
        service.write_summaries(vec![s3, s4]).await?;

        let response = service.query_service.get_paginated_traces(&filters).await?;
        assert_eq!(
            response.items.len(),
            4,
            "After write #2: expected 4 items, got {} (stale snapshot?)",
            response.items.len()
        );

        // Write batch #3 (2 more summaries)
        let s5 = make_summary([0xA4; 16], "svc_vis", 0, vec![]);
        let s6 = make_summary([0xA5; 16], "svc_vis", 0, vec![]);
        service.write_summaries(vec![s5, s6]).await?;

        let response = service.query_service.get_paginated_traces(&filters).await?;
        assert_eq!(
            response.items.len(),
            6,
            "After write #3: expected 6 items, got {} (stale snapshot?)",
            response.items.len()
        );

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

    /// Simulate a 2-pod deployment: writer pod commits summaries, reader pod picks them
    /// up via the refresh ticker.
    ///
    /// Both pods share the same local storage directory (equivalent to a shared GCS/S3
    /// bucket in production). Each pod has its own `ObjectStore` + `SessionContext` — there
    /// is no shared memory. The reader's refresh ticker (1s interval) calls
    /// `update_incremental()`, detects the new Delta log entry, and re-registers the
    /// `SessionContext` so subsequent queries return fresh data.
    ///
    /// We build each pod by creating a `TraceSpanService` first (which sets up the
    /// `ObjectStore` + `SessionContext` correctly) and then attaching a `TraceSummaryService`
    /// on top — the same pattern used in server setup. Using `make_test_ctx()` alone is
    /// insufficient because it does not register the object-store URL scheme that the Delta
    /// log uses for Parquet file paths, causing a "Failed to fetch metadata" error at query time.
    #[tokio::test]
    async fn test_distributed_refresh() -> Result<(), TraceEngineError> {
        use crate::parquet::tracing::service::TraceSpanService;

        // Use a unique storage dir to avoid racing with service::tests::test_distributed_refresh,
        // which also uses scouter_storage and runs concurrently in the same binary.
        let storage_settings = ObjectStorageSettings {
            storage_uri: "./scouter_storage_summary_dist".to_string(),
            ..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);
        }

        // "Writer pod" — owns writes; standard refresh interval (not needed on the writer)
        let writer_spans = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;
        let writer = TraceSummaryService::new(
            &writer_spans.object_store,
            24,
            writer_spans.ctx.clone(),
            writer_spans.catalog.clone(),
            10,
        )
        .await?;

        // "Reader pod" — separate ObjectStore + SessionContext; 1s refresh for fast turnaround
        let reader_spans = TraceSpanService::new(&storage_settings, 24, Some(2), None, 10).await?;
        let reader = TraceSummaryService::new(
            &reader_spans.object_store,
            24,
            reader_spans.ctx.clone(),
            reader_spans.catalog.clone(),
            1,
        )
        .await?;

        let summary = make_summary([0xDD_u8; 16], "distributed-svc", 0, vec![]);
        writer.write_summaries(vec![summary]).await?;

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

        let start = Utc::now() - chrono::Duration::hours(1);
        let end = Utc::now() + chrono::Duration::hours(1);
        let filters = TraceFilters {
            service_name: Some("distributed-svc".to_string()),
            has_errors: None,
            status_code: None,
            start_time: Some(start),
            end_time: Some(end),
            limit: Some(25),
            cursor_start_time: None,
            cursor_trace_id: None,
            direction: None,
            attribute_filters: None,
            trace_ids: None,
            entity_uid: None,
            queue_uid: None,
        };

        let response = reader.query_service.get_paginated_traces(&filters).await?;
        assert!(
            !response.items.is_empty(),
            "Reader pod should see summaries written by writer pod after refresh"
        );

        writer.shutdown().await?;
        reader.shutdown().await?;
        writer_spans.shutdown().await?;
        reader_spans.shutdown().await?;
        if storage_path.exists() {
            let _ = std::fs::remove_dir_all(&storage_path);
        }
        Ok(())
    }
}