drasi-lib 0.8.3

Embedded Drasi for in-process data change processing using continuous queries
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
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
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
// Copyright 2025 The Drasi Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use anyhow::{Context, Result};
use async_trait::async_trait;
use log::{debug, error, info, warn};
use std::collections::HashMap;
use std::collections::HashSet;
use std::sync::Arc;
use tokio::sync::{Notify, RwLock};

// Import drasi-core components
use drasi_core::{
    evaluation::context::{QueryPartEvaluationContext, QueryVariables},
    evaluation::functions::FunctionRegistry,
    evaluation::variable_value::VariableValue,
    in_memory_index::in_memory_checkpoint_store::InMemoryCheckpointStore,
    interface::CheckpointStore,
    middleware::MiddlewareTypeRegistry,
    query::{ContinuousQuery, QueryBuilder},
};
use drasi_functions_cypher::CypherFunctionSet;
use drasi_functions_gql::GQLFunctionSet;
use drasi_query_ast::api::{QueryConfiguration, QueryParser};
use drasi_query_cypher::CypherParser;
use drasi_query_gql::GQLParser;

use crate::channels::*;
use crate::component_graph::{ComponentGraph, ComponentKind, ComponentUpdateSender};
use crate::config::SourceSubscriptionSettings;
use crate::config::{QueryConfig, QueryLanguage, QueryRuntime};
use crate::managers::{
    log_component_error, log_component_start, log_component_stop, ComponentLogKey,
    ComponentLogRegistry,
};
use crate::queries::label_extractor::{LabelExtractor, QueryLabels};
use crate::queries::output_state::{
    FetchError, OutboxGap, OutboxResponse, QueryOutputState, SnapshotResponse,
};
use crate::queries::PriorityQueue;
use crate::queries::QueryBase;
use crate::sources::FutureQueueSource;
use crate::sources::Source;
use crate::sources::SourceManager;
use tracing::Instrument;

/// Default query configuration
struct DefaultQueryConfig;

impl QueryConfiguration for DefaultQueryConfig {
    fn get_aggregating_function_names(&self) -> HashSet<String> {
        let mut set = HashSet::new();
        set.insert("count".into());
        set.insert("sum".into());
        set.insert("min".into());
        set.insert("max".into());
        set.insert("avg".into());
        set.insert("collect".into());
        set.insert("stdev".into());
        set.insert("stdevp".into());
        set
    }
}

/// Convert QueryVariables (`BTreeMap<Box<str>, VariableValue>`) to JSON
fn convert_query_variables_to_json(vars: &QueryVariables) -> serde_json::Value {
    let mut result = serde_json::Map::new();
    for (key, value) in vars.iter() {
        result.insert(key.to_string(), convert_variable_value_to_json(value));
    }
    serde_json::Value::Object(result)
}

/// Convert a single VariableValue to JSON
fn convert_variable_value_to_json(value: &VariableValue) -> serde_json::Value {
    match value {
        VariableValue::Null => serde_json::Value::Null,
        VariableValue::Bool(b) => serde_json::Value::Bool(*b),
        VariableValue::Float(f) => {
            if f.is_f64() {
                // from_f64 returns None for NaN/Infinity, but is_f64() already checks finiteness
                let s = f.to_string();
                s.parse::<f64>()
                    .ok()
                    .and_then(serde_json::Number::from_f64)
                    .map(serde_json::Value::Number)
                    .unwrap_or_else(|| serde_json::Value::String(s))
            } else {
                serde_json::Value::String(f.to_string())
            }
        }
        VariableValue::Integer(i) => {
            if let Some(val) = i.as_i64() {
                serde_json::Value::Number(serde_json::Number::from(val))
            } else if let Some(val) = i.as_u64() {
                serde_json::Value::Number(serde_json::Number::from(val))
            } else {
                serde_json::Value::String(i.to_string())
            }
        }
        VariableValue::String(s) => serde_json::Value::String(s.clone()),
        VariableValue::List(list) => {
            serde_json::Value::Array(list.iter().map(convert_variable_value_to_json).collect())
        }
        VariableValue::Object(map) => {
            let mut result = serde_json::Map::new();
            for (k, v) in map.iter() {
                result.insert(k.clone(), convert_variable_value_to_json(v));
            }
            serde_json::Value::Object(result)
        }
        // For complex types, convert to string representation
        _ => serde_json::Value::String(format!("{value:?}")),
    }
}

#[async_trait]
pub trait Query: Send + Sync {
    /// Start the query - subscribes to sources and begins processing events
    async fn start(&self) -> Result<()>;
    async fn stop(&self) -> Result<()>;
    async fn status(&self) -> ComponentStatus;
    fn get_config(&self) -> &QueryConfig;
    fn as_any(&self) -> &dyn std::any::Any;

    /// Return the number of active subscription forwarder tasks (diagnostic/testing).
    async fn subscription_count(&self) -> usize {
        0
    }

    /// Subscribe to query results for reactions
    /// Returns a broadcast receiver for Arc-wrapped QueryResults
    async fn subscribe(&self, reaction_id: String) -> Result<QuerySubscriptionResponse>;

    /// Fetch a snapshot of the live result set.
    ///
    /// Returns the current results (as an `im::HashMap` clone — O(1) via structural sharing)
    /// and the `as_of_sequence` reflecting the latest emission.
    ///
    /// Blocks until bootstrap completes. Returns `FetchError::TimedOut` if bootstrap
    /// does not complete within 5 minutes, or `FetchError::NotRunning` if the query
    /// terminates in a non-Running state.
    async fn fetch_snapshot(&self) -> Result<SnapshotResponse, FetchError>;

    /// Fetch outbox entries after the given sequence number.
    ///
    /// Returns `Ok(OutboxResponse)` if the requested position is still in the ring buffer,
    /// or `Err(FetchError::OutboxGap)` if it has been evicted.
    ///
    /// Blocks until bootstrap completes, with the same timeout/error semantics as
    /// `fetch_snapshot`.
    async fn fetch_outbox(&self, after_sequence: u64) -> Result<OutboxResponse, FetchError>;
}

/// Bootstrap phase tracking for each source
#[derive(Debug, Clone, PartialEq)]
enum BootstrapPhase {
    NotStarted,
    InProgress,
    Completed,
}

/// Dispatch query evaluation results to the current result set and all subscribed reactions.
///
/// Shared between the regular event processing path and the future queue drain path.
/// Uses `QueryOutputState` for O(1) result-set updates keyed by `row_signature`,
/// increments the sequence counter, and pushes to the outbox ring buffer.
async fn dispatch_query_results(
    results: &[QueryPartEvaluationContext],
    source_id: &str,
    query_id: &str,
    output_state: &RwLock<QueryOutputState>,
    dispatchers: &RwLock<Vec<Box<dyn ChangeDispatcher<QueryResult> + Send + Sync>>>,
    profiling: crate::profiling::ProfilingMetadata,
) {
    // Convert Drasi results to our QueryResult format, filtering out Noops
    let converted_results: Vec<ResultDiff> = results
        .iter()
        .filter_map(|ctx| match ctx {
            QueryPartEvaluationContext::Adding {
                after,
                row_signature,
            } => Some(ResultDiff::Add {
                data: convert_query_variables_to_json(after),
                row_signature: *row_signature,
            }),
            QueryPartEvaluationContext::Removing {
                before,
                row_signature,
            } => Some(ResultDiff::Delete {
                data: convert_query_variables_to_json(before),
                row_signature: *row_signature,
            }),
            QueryPartEvaluationContext::Updating {
                before,
                after,
                row_signature,
            } => Some(ResultDiff::Update {
                data: convert_query_variables_to_json(after),
                before: convert_query_variables_to_json(before),
                after: convert_query_variables_to_json(after),
                grouping_keys: None,
                row_signature: *row_signature,
            }),
            // NOTE: When a group empties (last contributor removed), core emits
            // Aggregation { default_after: true, .. } with identity values (count:0,
            // sum:0, etc.) rather than Removing. Proper empty-group → Delete detection
            // requires core-level `is_at_identity()` on each accumulator (see PR #409).
            // Without that infrastructure, this conversion preserves current behavior:
            // the row stays in the result set with zeroed-out values.
            QueryPartEvaluationContext::Aggregation {
                before,
                after,
                row_signature,
                ..
            } => Some(ResultDiff::Aggregation {
                before: before.as_ref().map(convert_query_variables_to_json),
                after: convert_query_variables_to_json(after),
                row_signature: *row_signature,
            }),
            QueryPartEvaluationContext::Noop => None,
        })
        .collect();

    // If all results were Noops, skip outbox/sequence advancement and dispatch
    if converted_results.is_empty() {
        return;
    }

    // Apply diffs to the output state, build QueryResult, increment sequence,
    // push to outbox, and get back the Arc for zero-copy dispatch — all in one
    // write-lock acquisition.
    let arc_result = {
        let mut state = output_state.write().await;
        state.apply_diffs(&converted_results);

        let result_count = converted_results.len();
        let query_result = QueryResult::with_profiling(
            query_id.to_string(),
            0, // sequence assigned by advance_sequence_and_push
            chrono::Utc::now(),
            converted_results,
            {
                let mut meta = HashMap::new();
                meta.insert(
                    "source_id".to_string(),
                    serde_json::Value::String(source_id.to_string()),
                );
                meta.insert(
                    "processed_by".to_string(),
                    serde_json::Value::String("drasi-core".to_string()),
                );
                meta.insert(
                    "result_count".to_string(),
                    serde_json::Value::Number(result_count.into()),
                );
                meta
            },
            profiling,
        );

        state.advance_sequence_and_push(query_result)
    };

    debug!(
        "Query '{query_id}' sending {} results to reactions (seq={})",
        arc_result.results.len(),
        arc_result.sequence
    );

    // Dispatch query result to all subscribed reactions
    let dispatchers = dispatchers.read().await;
    for dispatcher in dispatchers.iter() {
        if let Err(e) = dispatcher.dispatch_change(arc_result.clone()).await {
            debug!("Failed to dispatch result for query '{query_id}': {e}");
        }
    }
}

pub struct DrasiQuery {
    // DrasiLib instance ID for log routing isolation
    instance_id: String,
    // Use QueryBase for common functionality
    base: QueryBase,
    output_state: Arc<RwLock<QueryOutputState>>,
    // Pre-computed config hash for bootstrap APIs
    config_hash: u64,
    // Priority queue for ordered event processing
    priority_queue: PriorityQueue,
    // Reference to SourceManager for direct subscription
    source_manager: Arc<SourceManager>,
    // Track subscription tasks for cleanup
    subscription_tasks: Arc<RwLock<Vec<tokio::task::JoinHandle<()>>>>,
    // Abort handles for bootstrap + supervisor tasks (for cleanup on stop)
    bootstrap_abort_handles: Arc<RwLock<Vec<tokio::task::AbortHandle>>>,
    // Track bootstrap state per source
    bootstrap_state: Arc<RwLock<HashMap<String, BootstrapPhase>>>,
    // IndexFactory for creating storage backend indexes
    index_factory: Arc<crate::indexes::IndexFactory>,
    // Middleware registry for query middleware
    middleware_registry: Arc<MiddlewareTypeRegistry>,
    // FutureQueueSource for temporal query support
    future_queue_source: Arc<RwLock<Option<Arc<FutureQueueSource>>>>,
    // Persisted checkpoint_store across stop/start cycles for checkpoint recovery
    checkpoint_store: Arc<RwLock<Option<Arc<dyn CheckpointStore>>>>,
    // Configurable bootstrap timeout for fetch APIs
    bootstrap_timeout: std::time::Duration,
    // Resolved recovery policy: per-query → global default → Strict
    resolved_recovery_policy: crate::recovery::RecoveryPolicy,
    // Track which source IDs we subscribed to, for cleanup in stop()
    subscribed_source_ids: Arc<RwLock<Vec<String>>>,
}

impl DrasiQuery {
    pub fn new(
        instance_id: impl Into<String>,
        config: QueryConfig,
        source_manager: Arc<SourceManager>,
        index_factory: Arc<crate::indexes::IndexFactory>,
        middleware_registry: Arc<MiddlewareTypeRegistry>,
        default_recovery_policy: Option<crate::recovery::RecoveryPolicy>,
    ) -> Result<Self> {
        // Create priority queue with configured capacity (fallback to 10000 if not set)
        let priority_capacity = config.priority_queue_capacity.unwrap_or(10000);
        let priority_queue = PriorityQueue::new(priority_capacity);
        let outbox_capacity = config.outbox_capacity;
        let bootstrap_timeout = std::time::Duration::from_secs(config.bootstrap_timeout_secs);
        let config_hash = crate::queries::compute_config_hash(&config);

        // Resolve recovery policy: per-query → global default → Strict
        let resolved_recovery_policy = config
            .recovery_policy
            .or(default_recovery_policy)
            .unwrap_or_default();

        // Create QueryBase for common functionality
        let base = QueryBase::new(config).context("Failed to create QueryBase")?;

        Ok(Self {
            instance_id: instance_id.into(),
            base,
            output_state: Arc::new(RwLock::new(QueryOutputState::new(outbox_capacity))),
            config_hash,
            priority_queue,
            source_manager,
            subscription_tasks: Arc::new(RwLock::new(Vec::new())),
            bootstrap_abort_handles: Arc::new(RwLock::new(Vec::new())),
            bootstrap_state: Arc::new(RwLock::new(HashMap::new())),
            index_factory,
            middleware_registry,
            future_queue_source: Arc::new(RwLock::new(None)),
            checkpoint_store: Arc::new(RwLock::new(None)),
            bootstrap_timeout,
            resolved_recovery_policy,
            subscribed_source_ids: Arc::new(RwLock::new(Vec::new())),
        })
    }

    /// Initialize the query with runtime context.
    ///
    /// Wires the status handle to the component graph, following the same
    /// pattern as Source and Reaction initialization.
    pub async fn initialize(&self, context: crate::context::QueryRuntimeContext) {
        self.base.initialize(context).await;
    }

    pub async fn get_current_results(&self) -> Vec<serde_json::Value> {
        self.output_state.read().await.get_results_as_vec()
    }

    /// Wait until the query has finished bootstrapping (status is no longer `Starting`).
    ///
    /// Returns `Ok(())` if the query reaches `Running` status.
    /// Returns `Err(FetchError::NotRunning)` if it reaches a terminal non-Running state.
    /// Returns `Err(FetchError::TimedOut)` if bootstrap doesn't complete within the
    /// configured `bootstrap_timeout_secs`.
    async fn wait_until_running(&self) -> Result<(), FetchError> {
        let mut status_rx = self.base.status_handle().subscribe_status();

        // Check current value first (avoids waiting if already transitioned)
        let current = *status_rx.borrow_and_update();
        match current {
            ComponentStatus::Running => return Ok(()),
            ComponentStatus::Starting => {} // need to wait
            other => return Err(FetchError::NotRunning { status: other }),
        }

        // Wait for a non-Starting status, with timeout
        let result = tokio::time::timeout(
            self.bootstrap_timeout,
            status_rx.wait_for(|s| *s != ComponentStatus::Starting),
        )
        .await;

        match result {
            Ok(Ok(status_ref)) => {
                let status = *status_ref;
                if status == ComponentStatus::Running {
                    Ok(())
                } else {
                    Err(FetchError::NotRunning { status })
                }
            }
            Ok(Err(_)) => {
                // Watch channel closed — sender dropped, treat as not running
                Err(FetchError::NotRunning {
                    status: ComponentStatus::Stopped,
                })
            }
            Err(_) => Err(FetchError::TimedOut),
        }
    }
}

#[cfg(test)]
impl DrasiQuery {
    /// Count active subscription forwarder tasks (testing helper)
    pub async fn subscription_task_count(&self) -> usize {
        self.subscription_tasks.read().await.len()
    }

    /// Access the checkpoint store (for internal/test use only).
    #[doc(hidden)]
    pub async fn get_checkpoint_store(&self) -> Option<Arc<dyn CheckpointStore>> {
        self.checkpoint_store.read().await.clone()
    }
}

/// Clear all persistent indexes on config hash mismatch or hash read failure.
///
/// Called to remove stale element/archive/result/future data so that
/// the subsequent bootstrap runs against a clean state.
async fn clear_persistent_indexes(
    query_id: &str,
    element_index: &Option<Arc<dyn drasi_core::interface::ElementIndex>>,
    archive_index: &Option<Arc<dyn drasi_core::interface::ElementArchiveIndex>>,
    result_index: &Option<Arc<dyn drasi_core::interface::ResultIndex>>,
    future_queue: &Option<Arc<dyn drasi_core::interface::FutureQueue>>,
) -> anyhow::Result<()> {
    use drasi_core::interface::IndexError;

    if let Some(ei) = element_index {
        match ei.clear().await {
            Ok(()) | Err(IndexError::NotSupported) => {}
            Err(e) => {
                return Err(anyhow::anyhow!(
                    "Query '{query_id}' failed to clear element index"
                ))
                .context(format!("{e:?}"));
            }
        }
    }
    if let Some(ai) = archive_index {
        match ai.clear().await {
            Ok(()) | Err(IndexError::NotSupported) => {}
            Err(e) => {
                return Err(anyhow::anyhow!(
                    "Query '{query_id}' failed to clear archive index"
                ))
                .context(format!("{e:?}"));
            }
        }
    }
    if let Some(ri) = result_index {
        match ri.clear().await {
            Ok(()) | Err(IndexError::NotSupported) => {}
            Err(e) => {
                return Err(anyhow::anyhow!(
                    "Query '{query_id}' failed to clear result index"
                ))
                .context(format!("{e:?}"));
            }
        }
    }
    if let Some(fq) = future_queue {
        match fq.clear().await {
            Ok(()) | Err(IndexError::NotSupported) => {}
            Err(e) => {
                return Err(anyhow::anyhow!(
                    "Query '{query_id}' failed to clear future queue"
                ))
                .context(format!("{e:?}"));
            }
        }
    }
    Ok(())
}

#[async_trait]
impl Query for DrasiQuery {
    async fn start(&self) -> Result<()> {
        log_component_start("Query", &self.base.config.id);

        self.bootstrap_state.write().await.clear();

        // Set Starting on the local status handle. The manager has already validated
        // and applied the Starting transition on the graph via validate_and_transition().
        // This local update is needed because internal query logic (e.g., the bootstrap
        // completion check at line ~983) reads the handle's local status to decide
        // whether to transition to Running.
        //
        // INVARIANT: The graph must already be in Starting state before this point.
        // The idempotency check in update_status_with_message() ensures the duplicate
        // Starting update sent via mpsc is safely ignored.
        debug_assert!(
            matches!(
                self.base.status_handle().get_status().await,
                ComponentStatus::Stopped | ComponentStatus::Error | ComponentStatus::Starting
            ),
            "DrasiQuery::start() called but local handle is not in expected pre-start state"
        );
        self.base
            .set_status(
                ComponentStatus::Starting,
                Some("Starting query".to_string()),
            )
            .await;

        // Build and initialize the actual Drasi ContinuousQuery
        let query_str = self.base.config.query.clone();

        // Create a parser and function registry based on the query language
        let config = Arc::new(DefaultQueryConfig);
        let (parser, function_registry): (Arc<dyn QueryParser>, Arc<FunctionRegistry>) =
            match self.base.config.query_language {
                QueryLanguage::Cypher => {
                    debug!(
                        "Query '{}' using Cypher parser and function set",
                        self.base.config.id
                    );
                    (
                        Arc::new(CypherParser::new(config)),
                        Arc::new(FunctionRegistry::new()).with_cypher_function_set(),
                    )
                }
                QueryLanguage::GQL => {
                    debug!(
                        "Query '{}' using GQL parser and function set",
                        self.base.config.id
                    );
                    (
                        Arc::new(GQLParser::new(config)),
                        Arc::new(FunctionRegistry::new()).with_gql_function_set(),
                    )
                }
            };

        let mut builder =
            QueryBuilder::new(&query_str, parser).with_function_registry(function_registry);

        // Configure middleware registry and middleware
        builder = builder.with_middleware_registry(self.middleware_registry.clone());

        // Add all middleware configurations from config
        for mw in &self.base.config.middleware {
            builder = builder.with_source_middleware(Arc::new(mw.clone()));
        }

        // Configure source pipelines for all subscriptions
        for sub in &self.base.config.sources {
            builder = builder.with_source_pipeline(&sub.source_id, &sub.pipeline);
        }

        // Add joins if configured
        if let Some(joins) = &self.base.config.joins {
            debug!(
                "Query '{}' has {} configured joins",
                self.base.config.id,
                joins.len()
            );
            let drasi_joins: Vec<drasi_core::models::QueryJoin> =
                joins.iter().cloned().map(|j| j.into()).collect();
            builder = builder.with_joins(drasi_joins);
        }

        // Build indexes - either from configured backend or default in-memory.
        // Keep a reference to the checkpoint_store for persistence.
        // Reuse the persisted checkpoint_store across stop/start cycles so that
        // in-memory checkpoints survive restarts within the same process lifetime.
        let checkpoint_store: Arc<dyn CheckpointStore>;
        // Keep index references for potential clearing on config hash mismatch.
        let element_index: Option<Arc<dyn drasi_core::interface::ElementIndex>>;
        let archive_index: Option<Arc<dyn drasi_core::interface::ElementArchiveIndex>>;
        let result_index: Option<Arc<dyn drasi_core::interface::ResultIndex>>;
        let future_queue: Option<Arc<dyn drasi_core::interface::FutureQueue>>;
        let session_control: Option<Arc<dyn drasi_core::interface::SessionControl>>;

        if let Some(backend_ref) = &self.base.config.storage_backend {
            debug!(
                "Query '{}' using storage backend: {:?}",
                self.base.config.id, backend_ref
            );
            let index_factory = self.index_factory.clone();

            // Drop the previous checkpoint store handle before re-opening.
            // For backends like RocksDB that hold an exclusive lock on the
            // data directory, the old handle must be released before we can
            // open a new one.  Checkpoint data is already persisted on disk.
            *self.checkpoint_store.write().await = None;

            let created = index_factory
                .build(backend_ref, &self.base.config.id)
                .await
                .context("Failed to build indexes")?;

            // Use backend-provided checkpoint store, or create in-memory fallback
            checkpoint_store = match created.checkpoint_store {
                Some(store) => store,
                None => {
                    // Backend didn't provide one; reuse persisted or create new
                    let existing = self.checkpoint_store.read().await.clone();
                    existing.unwrap_or_else(|| Arc::new(InMemoryCheckpointStore::new()))
                }
            };

            // Hold references for potential clearing before passing to builder
            element_index = Some(created.set.element_index.clone());
            archive_index = Some(created.set.archive_index.clone());
            result_index = Some(created.set.result_index.clone());
            future_queue = Some(created.set.future_queue.clone());
            session_control = Some(created.set.session_control.clone());

            builder = builder
                .with_element_index(created.set.element_index)
                .with_archive_index(created.set.archive_index)
                .with_result_index(created.set.result_index)
                .with_future_queue(created.set.future_queue)
                .with_session_control(created.set.session_control);
        } else {
            debug!(
                "Query '{}' using default in-memory indexes",
                self.base.config.id
            );
            // Reuse persisted checkpoint_store if available (e.g., after stop/restart)
            let existing = self.checkpoint_store.read().await.clone();
            checkpoint_store = existing.unwrap_or_else(|| Arc::new(InMemoryCheckpointStore::new()));
            element_index = None;
            archive_index = None;
            result_index = None;
            future_queue = None;
            session_control = None;
        };

        // Persist the checkpoint_store for future stop/start cycles
        *self.checkpoint_store.write().await = Some(checkpoint_store.clone());

        let continuous_query = match builder.try_build().await {
            Ok(query) => query,
            Err(e) => {
                error!("Failed to build query '{}': {}", self.base.config.id, e);
                self.base
                    .set_status(
                        ComponentStatus::Error,
                        Some(format!("Failed to build query: {e}")),
                    )
                    .await;

                return Err(anyhow::anyhow!("Failed to build query: {e}"));
            }
        };

        // Extract labels from the query for bootstrap
        let labels = match crate::queries::LabelExtractor::extract_labels(
            &query_str,
            &self.base.config.query_language,
        ) {
            Ok(labels) => labels,
            Err(e) => {
                warn!("Failed to extract labels from query '{}': {}. Bootstrap will request all data.",
                    self.base.config.id, e);
                crate::queries::QueryLabels {
                    node_labels: vec![],
                    relation_labels: vec![],
                }
            }
        };

        // Build subscription settings for each source
        let subscription_settings =
            match crate::queries::SubscriptionSettingsBuilder::build_subscription_settings(
                &self.base.config,
                &labels,
            ) {
                Ok(settings) => settings,
                Err(e) => {
                    error!(
                        "Failed to build subscription settings for query '{}': {}",
                        self.base.config.id, e
                    );
                    self.base
                        .set_status(
                            ComponentStatus::Error,
                            Some(format!("Failed to build subscription settings: {e}")),
                        )
                        .await;

                    return Err(anyhow::anyhow!(
                        "Failed to build subscription settings: {e}"
                    ));
                }
            };

        // Read the last checkpoints and propagate source_position to subscription settings
        // so sources can resume from where they left off.
        //
        // Only propagate checkpoint recovery when the checkpoint store is persistent.
        // Volatile (in-memory) stores don't survive restarts, and their paired element
        // indexes rebuild fresh on each start — bootstrap must run to populate the
        // graph state. Skipping bootstrap against an empty graph would produce
        // incorrect results.
        let mut subscription_settings = subscription_settings;
        let has_persistent_backend = checkpoint_store.is_persistent();
        let mut checkpoint_sequences_per_source: std::collections::HashMap<String, u64> =
            std::collections::HashMap::new();
        if has_persistent_backend {
            // Config hash check: detect query configuration changes that require
            // a full re-bootstrap. If the stored hash doesn't match the current
            // config, all checkpoints are cleared so sources bootstrap from scratch.
            //
            // Checkpoint operations require an active session for transactional
            // backends (e.g., RocksDB). Wrap read/write/clear calls in begin/commit.
            let current_hash = super::compute_config_hash(&self.base.config);

            if let Some(sc) = &session_control {
                sc.begin()
                    .await
                    .context("Failed to begin session for config hash check")?;
            }

            let config_matches = match checkpoint_store.read_config_hash().await {
                Ok(Some(stored_hash)) if stored_hash == current_hash => {
                    debug!(
                        "Query '{}' config hash matches stored hash ({current_hash}), resuming",
                        self.base.config.id
                    );
                    true
                }
                Ok(Some(stored_hash)) => {
                    info!(
                        "Query '{}' config hash changed ({stored_hash} -> {current_hash}), clearing all persistent state for full bootstrap",
                        self.base.config.id
                    );
                    // Clear checkpoints first. Only write the new config hash if
                    // clearing succeeded — otherwise stale checkpoints would be
                    // resumed with the wrong config on the next restart.
                    match checkpoint_store.clear_checkpoints().await {
                        Ok(()) => {
                            if let Err(e) = checkpoint_store.write_config_hash(current_hash).await {
                                warn!(
                                    "Query '{}' failed to write new config hash: {e}",
                                    self.base.config.id
                                );
                            }
                        }
                        Err(e) => {
                            let msg = format!(
                                "Query '{}' failed to clear checkpoints on config change: {e}. \
                                 Cannot start with stale checkpoint data from a different config.",
                                self.base.config.id
                            );
                            error!("{msg}");
                            self.base
                                .set_status(ComponentStatus::Error, Some(msg.clone()))
                                .await;
                            return Err(anyhow::anyhow!(msg));
                        }
                    }
                    // Also clear persistent element/result/archive/future indexes
                    // so stale data from the old config cannot be read during bootstrap.
                    if let Err(e) = clear_persistent_indexes(
                        &self.base.config.id,
                        &element_index,
                        &archive_index,
                        &result_index,
                        &future_queue,
                    )
                    .await
                    {
                        let msg = format!(
                            "Query '{}' failed to clear persistent indexes on config change: {e}",
                            self.base.config.id
                        );
                        error!("{msg}");
                        self.base
                            .set_status(ComponentStatus::Error, Some(msg.clone()))
                            .await;
                        return Err(anyhow::anyhow!(msg));
                    }
                    false
                }
                Ok(None) => {
                    info!(
                        "Query '{}' no stored config hash (first run), writing hash {current_hash}",
                        self.base.config.id
                    );
                    if let Err(e) = checkpoint_store.write_config_hash(current_hash).await {
                        warn!(
                            "Query '{}' failed to write config hash: {e}",
                            self.base.config.id
                        );
                    }
                    false
                }
                Err(e) => {
                    warn!(
                        "Query '{}' failed to read config hash, clearing persistent state and starting fresh: {e}",
                        self.base.config.id
                    );
                    // Cannot trust persistent state if config hash is unreadable —
                    // clear indexes and checkpoints to ensure a clean bootstrap.
                    if let Err(ce) = checkpoint_store.clear_checkpoints().await {
                        let msg = format!(
                            "Query '{}' failed to clear checkpoints on hash read failure: {ce}",
                            self.base.config.id
                        );
                        error!("{msg}");
                        self.base
                            .set_status(ComponentStatus::Error, Some(msg.clone()))
                            .await;
                        return Err(anyhow::anyhow!(msg));
                    }
                    if let Err(ie) = clear_persistent_indexes(
                        &self.base.config.id,
                        &element_index,
                        &archive_index,
                        &result_index,
                        &future_queue,
                    )
                    .await
                    {
                        let msg = format!(
                            "Query '{}' failed to clear persistent indexes on hash read failure: {ie}",
                            self.base.config.id
                        );
                        error!("{msg}");
                        self.base
                            .set_status(ComponentStatus::Error, Some(msg.clone()))
                            .await;
                        return Err(anyhow::anyhow!(msg));
                    }
                    false
                }
            };

            // Only read checkpoints if the config hash matched — otherwise we
            // cleared them above and a full bootstrap will run.
            if config_matches {
                match checkpoint_store.read_all_checkpoints().await {
                    Ok(checkpoints) => {
                        for settings in &mut subscription_settings {
                            if let Some(cp) = checkpoints.get(&settings.source_id) {
                                checkpoint_sequences_per_source
                                    .insert(settings.source_id.clone(), cp.sequence);
                                settings.last_sequence = Some(cp.sequence);
                                settings.request_position_handle = true;
                                if let Some(pos) = &cp.source_position {
                                    settings.resume_from = Some(pos.clone());
                                }
                                debug!(
                                    "Query '{}' resuming source '{}' from checkpoint: seq={}",
                                    self.base.config.id, settings.source_id, cp.sequence
                                );
                            }
                        }
                    }
                    Err(e) => {
                        warn!(
                            "Query '{}' failed to read checkpoints, starting fresh: {e}",
                            self.base.config.id
                        );
                    }
                }
            }

            // For persistent queries, always request position handles so sources
            // can track the query's durable progress for min-watermark advancement,
            // even on first run before any checkpoints exist.
            for settings in &mut subscription_settings {
                settings.request_position_handle = true;
            }

            // Commit the startup session used for config hash + checkpoint reads/writes.
            if let Some(sc) = &session_control {
                if let Err(e) = sc.commit().await {
                    warn!(
                        "Query '{}' failed to commit startup session: {e}",
                        self.base.config.id
                    );
                }
            }
        }

        // Set up FutureQueueSource for temporal query support.
        // This creates a virtual source that polls the future queue and emits
        // FuturesDue control signals, integrating temporal queries into the
        // standard source subscription mechanism.
        debug!(
            "Query '{}' setting up FutureQueueSource for temporal queries",
            self.base.config.id
        );

        let future_queue_source = Arc::new(FutureQueueSource::new(
            continuous_query.future_queue(),
            self.base.config.id.clone(),
        ));

        // Subscribe BEFORE starting so the dispatcher exists when the polling loop runs
        let fq_receiver = future_queue_source
            .subscribe()
            .await
            .map_err(|e| anyhow::anyhow!("Failed to subscribe to FutureQueueSource: {e}"))?;

        // Store for lifecycle cleanup in stop()
        *self.future_queue_source.write().await = Some(Arc::clone(&future_queue_source));

        info!(
            "Query '{}' subscribing to {} sources: {:?}",
            self.base.config.id,
            self.base.config.sources.len(),
            self.base
                .config
                .sources
                .iter()
                .map(|s| &s.source_id)
                .collect::<Vec<_>>()
        );

        let mut bootstrap_channels: Vec<(
            String,
            tokio::sync::mpsc::Receiver<crate::channels::BootstrapEvent>,
            Option<
                tokio::sync::oneshot::Receiver<anyhow::Result<crate::bootstrap::BootstrapResult>>,
            >,
        )> = Vec::new();
        let mut subscription_tasks: Vec<tokio::task::JoinHandle<()>> = Vec::new();

        // Build list of sources to subscribe to
        let mut sources_to_subscribe: Vec<(String, Arc<dyn Source>, SourceSubscriptionSettings)> =
            Vec::new();

        // Add regular sources from SourceManager
        for (idx, subscription) in self.base.config.sources.iter().enumerate() {
            let source_id = &subscription.source_id;
            match self.source_manager.get_source_instance(source_id).await {
                Some(src) => {
                    sources_to_subscribe.push((
                        source_id.clone(),
                        src,
                        subscription_settings[idx].clone(),
                    ));
                }
                None => {
                    error!(
                        "Query '{}' failed to find source '{}' in SourceManager",
                        self.base.config.id, source_id
                    );
                    // Cleanup already-spawned tasks before returning error
                    for handle in subscription_tasks.drain(..) {
                        handle.abort();
                        let _ = handle.await;
                    }
                    self.base
                        .set_status(
                            ComponentStatus::Error,
                            Some(format!("Source '{source_id}' not found")),
                        )
                        .await;
                    return Err(crate::managers::ComponentNotFoundError::new(
                        "source",
                        source_id.as_str(),
                    )
                    .into());
                }
            }
        }

        // Compatibility validation: persistent queries must not use volatile sources.
        // A volatile source (supports_replay() == false) cannot guarantee event replay
        // after a restart, so resuming from checkpoints could produce incorrect results
        // (gaps in the data stream).
        if has_persistent_backend {
            let volatile_sources: Vec<&str> = sources_to_subscribe
                .iter()
                .filter(|(_, src, _)| !src.supports_replay())
                .map(|(id, _, _)| id.as_str())
                .collect();
            if !volatile_sources.is_empty() {
                let reason = format!(
                    "source(s) {volatile_sources:?} do not support replay; checkpoint-based recovery requires durable sources"
                );
                let msg = format!(
                    "Query '{}' has a persistent backend but {reason}",
                    self.base.config.id
                );
                error!("{msg}");
                self.base
                    .set_status(ComponentStatus::Error, Some(msg))
                    .await;
                return Err(crate::recovery::RecoveryError::IncompatibleSource {
                    query_id: self.base.config.id.clone(),
                    source_id: volatile_sources.join(", "),
                    reason,
                }
                .into());
            }
        }

        let mut position_handles: std::collections::HashMap<
            String,
            Arc<std::sync::atomic::AtomicU64>,
        > = std::collections::HashMap::new();

        // Subscribe to all sources. If a PositionUnavailable error occurs and
        // the AutoReset policy is active, we clear all persistent state and
        // retry the entire loop with resume_from cleared to trigger full
        // re-bootstrap. The retry runs at most once.
        let mut auto_reset_retry = false;

        'subscribe_loop: loop {
            // On AutoReset retry: clear resume positions so sources bootstrap from scratch.
            if auto_reset_retry {
                info!(
                    "Query '{}' auto-reset: clearing resume positions and re-subscribing all sources",
                    self.base.config.id
                );
                for (_, _, settings) in &mut sources_to_subscribe {
                    settings.resume_from = None;
                    settings.last_sequence = None;
                    settings.request_position_handle = has_persistent_backend;
                }
                // Reset per-loop accumulators
                bootstrap_channels.clear();
                subscription_tasks.clear();
                position_handles.clear();
                self.bootstrap_state.write().await.clear();
                checkpoint_sequences_per_source.clear();
            }

            for (source_id, source, settings) in &sources_to_subscribe {
                let subscription_response = match source.subscribe(settings.clone()).await {
                    Ok(response) => response,
                    Err(e) => {
                        // Check if this is a PositionUnavailable error (gap detection)
                        if let Some(source_err) = e.downcast_ref::<crate::sources::SourceError>() {
                            match source_err {
                                crate::sources::SourceError::PositionUnavailable { .. } => {
                                    match self.resolved_recovery_policy {
                                        crate::recovery::RecoveryPolicy::Strict => {
                                            let msg = format!(
                                                "Query '{}' source '{}' cannot resume from checkpoint position (Strict policy): {e}",
                                                self.base.config.id, source_id
                                            );
                                            error!("{msg}");
                                            // Cleanup already-spawned tasks
                                            for handle in subscription_tasks.drain(..) {
                                                handle.abort();
                                                let _ = handle.await;
                                            }
                                            // Release position handles for already-subscribed sources
                                            for (sid, _, _) in &sources_to_subscribe {
                                                if let Some(src) = self
                                                    .source_manager
                                                    .get_source_instance(sid)
                                                    .await
                                                {
                                                    src.remove_position_handle(
                                                        &self.base.config.id,
                                                    )
                                                    .await;
                                                }
                                            }
                                            self.base
                                                .set_status(ComponentStatus::Error, Some(msg))
                                                .await;
                                            return Err(e.context(format!(
                                                "PositionUnavailable for source '{source_id}' with Strict recovery policy"
                                            )));
                                        }
                                        crate::recovery::RecoveryPolicy::AutoReset => {
                                            if auto_reset_retry {
                                                // Already retried once — don't loop forever
                                                let msg = format!(
                                                    "Query '{}' auto-reset retry failed for source '{}': {e}",
                                                    self.base.config.id, source_id
                                                );
                                                error!("{msg}");
                                                for handle in subscription_tasks.drain(..) {
                                                    handle.abort();
                                                    let _ = handle.await;
                                                }
                                                // Release position handles for already-subscribed sources
                                                for (sid, _, _) in &sources_to_subscribe {
                                                    if let Some(src) = self
                                                        .source_manager
                                                        .get_source_instance(sid)
                                                        .await
                                                    {
                                                        src.remove_position_handle(
                                                            &self.base.config.id,
                                                        )
                                                        .await;
                                                    }
                                                }
                                                self.base
                                                    .set_status(ComponentStatus::Error, Some(msg))
                                                    .await;
                                                return Err(e.context(
                                                    "AutoReset retry failed with PositionUnavailable",
                                                ));
                                            }

                                            warn!(
                                                "Query '{}' source '{}' position unavailable — AutoReset: wiping persistent state and re-bootstrapping all sources",
                                                self.base.config.id, source_id
                                            );

                                            // Abort already-spawned subscription tasks from this loop iteration
                                            for handle in subscription_tasks.drain(..) {
                                                handle.abort();
                                                let _ = handle.await;
                                            }

                                            // Drain queued events so stale pre-reset events don't
                                            // get processed after re-bootstrap.
                                            let drained = self.priority_queue.drain().await;
                                            if !drained.is_empty() {
                                                debug!(
                                                    "Query '{}' auto-reset: drained {} stale events from priority queue",
                                                    self.base.config.id, drained.len()
                                                );
                                            }

                                            // Release position handles for sources that subscribed
                                            // before the failure, so they can advance their watermark.
                                            for (sid, _, _) in &sources_to_subscribe {
                                                if let Some(src) = self
                                                    .source_manager
                                                    .get_source_instance(sid)
                                                    .await
                                                {
                                                    src.remove_position_handle(
                                                        &self.base.config.id,
                                                    )
                                                    .await;
                                                }
                                            }

                                            // Clear all persistent state. If clearing fails,
                                            // abort startup rather than mixing stale state with
                                            // a fresh bootstrap.
                                            if has_persistent_backend {
                                                // Begin a session for the clear operations
                                                if let Some(sc) = &session_control {
                                                    if let Err(e) = sc.begin().await {
                                                        let msg = format!(
                                                            "Query '{}' auto-reset failed: could not begin session: {e}",
                                                            self.base.config.id
                                                        );
                                                        error!("{msg}");
                                                        self.base
                                                            .set_status(
                                                                ComponentStatus::Error,
                                                                Some(msg),
                                                            )
                                                            .await;
                                                        return Err(anyhow::anyhow!(
                                                            "AutoReset aborted: failed to begin session for clearing: {e}",
                                                        ));
                                                    }
                                                }

                                                if let Err(ie) = clear_persistent_indexes(
                                                    &self.base.config.id,
                                                    &element_index,
                                                    &archive_index,
                                                    &result_index,
                                                    &future_queue,
                                                )
                                                .await
                                                {
                                                    if let Some(sc) = &session_control {
                                                        let _ = sc.rollback();
                                                    }
                                                    let msg = format!(
                                                        "Query '{}' auto-reset failed: could not clear persistent indexes: {ie}",
                                                        self.base.config.id
                                                    );
                                                    error!("{msg}");
                                                    self.base
                                                        .set_status(
                                                            ComponentStatus::Error,
                                                            Some(msg),
                                                        )
                                                        .await;
                                                    return Err(anyhow::anyhow!(
                                                        "AutoReset aborted: failed to clear persistent indexes: {ie}",
                                                    ));
                                                }
                                                if let Err(ce) =
                                                    checkpoint_store.clear_checkpoints().await
                                                {
                                                    // Rollback on failure
                                                    if let Some(sc) = &session_control {
                                                        let _ = sc.rollback();
                                                    }
                                                    let msg = format!(
                                                        "Query '{}' auto-reset failed: could not clear checkpoints: {ce}",
                                                        self.base.config.id
                                                    );
                                                    error!("{msg}");
                                                    self.base
                                                        .set_status(
                                                            ComponentStatus::Error,
                                                            Some(msg),
                                                        )
                                                        .await;
                                                    return Err(anyhow::anyhow!(
                                                        "AutoReset aborted: failed to clear checkpoints: {ce}",
                                                    ));
                                                }
                                                // Write current config hash so next normal restart resumes correctly
                                                let current_hash =
                                                    super::compute_config_hash(&self.base.config);
                                                if let Err(he) = checkpoint_store
                                                    .write_config_hash(current_hash)
                                                    .await
                                                {
                                                    warn!(
                                                        "Query '{}' failed to write config hash during auto-reset: {he}",
                                                        self.base.config.id
                                                    );
                                                }

                                                // Commit the clearing session
                                                if let Some(sc) = &session_control {
                                                    if let Err(e) = sc.commit().await {
                                                        warn!(
                                                            "Query '{}' failed to commit auto-reset session: {e}",
                                                            self.base.config.id
                                                        );
                                                    }
                                                }
                                            }

                                            auto_reset_retry = true;
                                            continue 'subscribe_loop;
                                        }
                                    }
                                }
                            }
                        }

                        // Generic (non-PositionUnavailable) subscribe error
                        error!(
                            "Query '{}' failed to subscribe to source '{}': {}",
                            self.base.config.id, source_id, e
                        );
                        // Cleanup already-spawned tasks before returning error
                        for handle in subscription_tasks.drain(..) {
                            handle.abort();
                            let _ = handle.await;
                        }
                        // Release position handles for already-subscribed sources
                        for (sid, _, _) in &sources_to_subscribe {
                            if let Some(src) = self.source_manager.get_source_instance(sid).await {
                                src.remove_position_handle(&self.base.config.id).await;
                            }
                        }
                        self.base
                            .set_status(
                                ComponentStatus::Error,
                                Some(format!("Failed to subscribe to source '{source_id}': {e}")),
                            )
                            .await;
                        return Err(anyhow::anyhow!(
                            "Failed to subscribe to source '{source_id}': {e}"
                        ));
                    }
                };

                info!(
                    "Query '{}' successfully subscribed to source '{}'",
                    self.base.config.id, source_id
                );

                // Store bootstrap channel if provided
                // Also initialize bootstrap state only for sources that support bootstrap
                if let Some(bootstrap_rx) = subscription_response.bootstrap_receiver {
                    bootstrap_channels.push((
                        source_id.clone(),
                        bootstrap_rx,
                        subscription_response.bootstrap_result_receiver,
                    ));
                    self.bootstrap_state
                        .write()
                        .await
                        .insert(source_id.to_string(), BootstrapPhase::NotStarted);
                }

                // Collect position handle if source provides one
                if let Some(handle) = subscription_response.position_handle {
                    position_handles.insert(source_id.clone(), handle);
                }

                // Spawn task to forward events from receiver to priority queue
                let mut receiver = subscription_response.receiver;
                let priority_queue = self.priority_queue.clone();
                let query_id = self.base.config.id.clone();
                let source_id_clone = source_id.clone();
                let instance_id = self.instance_id.clone();

                // Get source dispatch mode to determine enqueue strategy
                let dispatch_mode = source.dispatch_mode();
                let use_blocking_enqueue =
                    matches!(dispatch_mode, crate::channels::DispatchMode::Channel);

                let span = tracing::info_span!(
                    "query_source_forwarder",
                    instance_id = %instance_id,
                    component_id = %query_id,
                    component_type = "query"
                );
                let task = tokio::spawn(
                    async move {
                        debug!(
                            "Query '{query_id}' started event forwarder for source '{source_id_clone}' (dispatch_mode: {dispatch_mode:?}, blocking_enqueue: {use_blocking_enqueue})"
                        );

                        loop {
                            match receiver.recv().await {
                                Ok(arc_event) => {
                                    // Use appropriate enqueue method based on dispatch mode
                                    if use_blocking_enqueue {
                                        // Channel mode: Use blocking enqueue to prevent message loss
                                        // This creates backpressure when the priority queue is full
                                        priority_queue.enqueue_wait(arc_event).await;
                                    } else {
                                        // Broadcast mode: Use non-blocking enqueue to prevent deadlock
                                        // Messages may be dropped when priority queue is full
                                        if !priority_queue.enqueue(arc_event).await {
                                            warn!(
                                                "Query '{query_id}' priority queue at capacity, dropping event from source '{source_id_clone}' (broadcast mode)"
                                            );
                                        }
                                    }
                                }
                                Err(e) => {
                                    error!(
                                        "Query '{query_id}' receiver error for source '{source_id_clone}': {e}"
                                    );
                                    info!(
                                        "Query '{query_id}' channel closed for source '{source_id_clone}'"
                                    );
                                    break;
                                }
                            }
                        }

                        debug!("Query '{query_id}' event forwarder exited for source '{source_id_clone}'");
                    }
                    .instrument(span),
                );

                subscription_tasks.push(task);
            }

            // All sources subscribed successfully — break out of the retry loop
            break;
        }

        // Store subscription tasks and record subscribed source IDs for cleanup in stop()
        *self.subscription_tasks.write().await = subscription_tasks;
        *self.subscribed_source_ids.write().await = sources_to_subscribe
            .iter()
            .map(|(id, _, _)| id.clone())
            .collect();

        // Wrap continuous_query in Arc for sharing across tasks
        let continuous_query = Arc::new(continuous_query);

        // Gate that blocks the streaming event processor until bootstrap completes.
        // Events buffer safely in the priority queue during bootstrap.
        let bootstrap_gate = Arc::new(Notify::new());

        // Channel for the bootstrap supervisor to send handover checkpoints
        // to the processor task after all bootstrap tasks complete.
        let (handover_tx, handover_rx) =
            tokio::sync::oneshot::channel::<std::collections::HashMap<String, u64>>();

        // NEW: Handle bootstrap channels
        if !bootstrap_channels.is_empty() {
            info!(
                "Query '{}' starting bootstrap from {} sources",
                self.base.config.id,
                bootstrap_channels.len()
            );

            // Emit bootstrapStarted control signal
            let mut metadata = HashMap::new();
            metadata.insert(
                "control_signal".to_string(),
                serde_json::json!("bootstrapStarted"),
            );
            metadata.insert(
                "source_count".to_string(),
                serde_json::json!(bootstrap_channels.len()),
            );

            let control_result = QueryResult::new(
                self.base.config.id.clone(),
                0,
                chrono::Utc::now(),
                vec![],
                metadata,
            );

            // Dispatch the control signal to all subscribed reactions
            self.base.dispatch_query_result(control_result).await.ok();
            info!(
                "[BOOTSTRAP] Emitted bootstrapStarted signal for query '{}'",
                self.base.config.id
            );

            // Process bootstrap events from each source
            let continuous_query_clone = continuous_query.clone();
            let base_dispatchers = self.base.dispatchers.clone();
            let query_id = self.base.config.id.clone();
            let bootstrap_state = self.bootstrap_state.clone();
            let instance_id = self.instance_id.clone();
            let bootstrap_output_state = self.output_state.clone();

            let mut bootstrap_handles = Vec::new();
            let mut abort_handles = Vec::new();

            for (source_id, mut bootstrap_rx, bootstrap_result_rx) in bootstrap_channels {
                // Mark source bootstrap as in progress
                bootstrap_state
                    .write()
                    .await
                    .insert(source_id.to_string(), BootstrapPhase::InProgress);

                info!(
                    "[BOOTSTRAP] Query '{query_id}' processing bootstrap from source '{source_id}'"
                );

                let continuous_query_ref = continuous_query_clone.clone();
                let query_id_clone = query_id.clone();
                let source_id_clone = source_id.clone();
                let bootstrap_state_clone = bootstrap_state.clone();
                let instance_id_clone = instance_id.clone();
                let output_state_clone = bootstrap_output_state.clone();

                let span = tracing::info_span!(
                    "query_bootstrap",
                    instance_id = %instance_id_clone,
                    component_id = %query_id,
                    component_type = "query"
                );
                let handle: tokio::task::JoinHandle<(String, Option<crate::bootstrap::BootstrapResult>)> = tokio::spawn(
                    async move {
                        let mut count = 0u64;

                        while let Some(bootstrap_event) = bootstrap_rx.recv().await {
                            count += 1;

                            // Process bootstrap change through ContinuousQuery
                            match continuous_query_ref
                                .process_source_change(bootstrap_event.change)
                                .await
                            {
                                Ok(results) => {
                                    if !results.is_empty() {
                                        debug!(
                                            "[BOOTSTRAP] Query '{}' received {} results from bootstrap event {}",
                                            query_id_clone, results.len(), count
                                        );

                                        // Convert results to ResultDiffs and apply to output state.
                                        // During bootstrap, we only update the result set (no outbox
                                        // push, no sequence increment, no dispatch to reactions).
                                        let diffs: Vec<ResultDiff> = results
                                            .iter()
                                            .map(|ctx| match ctx {
                                                QueryPartEvaluationContext::Adding { after, row_signature } => {
                                                    ResultDiff::Add {
                                                        data: convert_query_variables_to_json(after),
                                                        row_signature: *row_signature,
                                                    }
                                                }
                                                QueryPartEvaluationContext::Removing { before, row_signature } => {
                                                    ResultDiff::Delete {
                                                        data: convert_query_variables_to_json(before),
                                                        row_signature: *row_signature,
                                                    }
                                                }
                                                QueryPartEvaluationContext::Updating { before, after, row_signature } => {
                                                    ResultDiff::Update {
                                                        data: convert_query_variables_to_json(after),
                                                        before: convert_query_variables_to_json(before),
                                                        after: convert_query_variables_to_json(after),
                                                        grouping_keys: None,
                                                        row_signature: *row_signature,
                                                    }
                                                }
                                                QueryPartEvaluationContext::Aggregation { before, after, row_signature, .. } => {
                                                    ResultDiff::Aggregation {
                                                        before: before.as_ref().map(convert_query_variables_to_json),
                                                        after: convert_query_variables_to_json(after),
                                                        row_signature: *row_signature,
                                                    }
                                                }
                                                QueryPartEvaluationContext::Noop => ResultDiff::Noop,
                                            })
                                            .collect();

                                        let mut state = output_state_clone.write().await;
                                        state.apply_diffs(&diffs);
                                    }
                                }
                                Err(e) => {
                                    error!(
                                        "[BOOTSTRAP] Query '{query_id_clone}' failed to process bootstrap event from source '{source_id_clone}': {e}"
                                    );
                                }
                            }
                        }

                        info!(
                            "[BOOTSTRAP] Query '{query_id_clone}' completed bootstrap from source '{source_id_clone}' ({count} events)"
                        );

                        // Mark source bootstrap as completed
                        {
                            let mut state = bootstrap_state_clone.write().await;
                            state.insert(source_id_clone.to_string(), BootstrapPhase::Completed);
                        }

                        // Await the BootstrapResult from the source's bootstrap provider.
                        // This contains handover metadata (last_sequence, sequences_aligned).
                        let bootstrap_result = if let Some(rx) = bootstrap_result_rx {
                            match rx.await {
                                Ok(Ok(result)) => {
                                    debug!(
                                        "[BOOTSTRAP] Query '{}' received handover from source '{}': \
                                         last_sequence={:?}, sequences_aligned={}",
                                        query_id_clone, source_id_clone,
                                        result.last_sequence, result.sequences_aligned
                                    );
                                    Some(result)
                                }
                                Ok(Err(e)) => {
                                    error!(
                                        "[BOOTSTRAP] Query '{query_id_clone}' bootstrap provider failed for source '{source_id_clone}': {e}"
                                    );
                                    None
                                }
                                Err(_) => {
                                    warn!(
                                        "[BOOTSTRAP] Query '{query_id_clone}' bootstrap result channel dropped for source '{source_id_clone}'"
                                    );
                                    None
                                }
                            }
                        } else {
                            None
                        };

                        (source_id_clone, bootstrap_result)
                    }
                    .instrument(span),
                );
                abort_handles.push(handle.abort_handle());
                bootstrap_handles.push(handle);
            }

            // Supervisor task: joins all bootstrap tasks, computes handover
            // checkpoints, emits the bootstrapCompleted signal, and opens the gate.
            // Also handles panics by transitioning to Error.
            {
                let bootstrap_gate_clone = bootstrap_gate.clone();
                let reporter_clone = self.base.status_handle();
                let query_id_clone = self.base.config.id.clone();
                let instance_id_clone = self.instance_id.clone();
                let base_dispatchers_clone = base_dispatchers.clone();
                let checkpoint_store_for_supervisor = checkpoint_store.clone();
                let session_control_for_supervisor = session_control.clone();

                let span = tracing::info_span!(
                    "bootstrap_supervisor",
                    instance_id = %instance_id_clone,
                    component_id = %query_id_clone,
                    component_type = "query"
                );
                let supervisor_handle = tokio::spawn(
                    async move {
                        let join_results = futures::future::join_all(bootstrap_handles).await;
                        let panic_count = join_results.iter().filter(|r| matches!(r, Err(e) if e.is_panic())).count();

                        if panic_count > 0 {
                            error!(
                                "[BOOTSTRAP] Query '{query_id_clone}' {panic_count} bootstrap task(s) panicked, \
                                 transitioning to Error and opening gate"
                            );

                            reporter_clone.set_status(
                                ComponentStatus::Error,
                                Some(format!("Bootstrap failed: {panic_count} task(s) panicked")),
                            ).await;

                            // Send empty handover so processor doesn't block
                            let _ = handover_tx.send(std::collections::HashMap::new());
                            bootstrap_gate_clone.notify_one();
                            return;
                        }

                        // Collect handover checkpoints from BootstrapResults.
                        //
                        // Two purposes:
                        // 1. **Dedup map** (in-memory): For aligned sources with last_sequence,
                        //    buffered streaming events at or below that sequence are filtered.
                        //    Only populated when `sequences_aligned == true`.
                        // 2. **Recovery checkpoint** (persisted): For ANY source that provides
                        //    a last_sequence, we persist a checkpoint (optionally with
                        //    source_position bytes) so crash-after-bootstrap can resume
                        //    from the snapshot boundary without re-bootstrapping.
                        let mut handover_checkpoints: std::collections::HashMap<String, u64> =
                            std::collections::HashMap::new();

                        // Tracks source positions from bootstrap for persistence
                        let mut handover_positions: std::collections::HashMap<String, (u64, Option<bytes::Bytes>)> =
                            std::collections::HashMap::new();

                        for (source_id, bootstrap_result) in join_results.iter().filter_map(|r| r.as_ref().ok()) {
                            if let Some(br) = bootstrap_result {
                                // Dedup map: only for aligned sources
                                if br.sequences_aligned {
                                    if let Some(seq) = br.last_sequence {
                                        debug!(
                                            "[BOOTSTRAP] Query '{query_id_clone}' handover checkpoint for '{source_id}': seq={seq}"
                                        );
                                        handover_checkpoints.insert(source_id.clone(), seq);
                                    }
                                } else {
                                    debug!(
                                        "[BOOTSTRAP] Query '{query_id_clone}' source '{source_id}' sequences not aligned, \
                                         no dedup checkpoint (accept all buffered events)"
                                    );
                                }

                                // Recovery checkpoint: persist whenever we have a sequence
                                // OR a source_position. source_position enables native
                                // stream resumption; sequence enables dedup. Bootstrap
                                // events don't go through dispatch_event(), so sequence
                                // may be None even when source_position is present. Use
                                // 0 as the sentinel sequence in that case.
                                if br.last_sequence.is_some() || br.source_position.is_some() {
                                    let seq = br.last_sequence.unwrap_or(0);
                                    // Validate source_position size (same limit as dispatch_event)
                                    let position = br.source_position.as_ref().and_then(|pos| {
                                        if pos.len() > crate::sources::base::SourceBase::MAX_SOURCE_POSITION_BYTES {
                                            warn!(
                                                "[BOOTSTRAP] Query '{query_id_clone}' source '{source_id}' \
                                                 bootstrap source_position is {} bytes (> {} limit); \
                                                 dropping position, checkpoint will have sequence only",
                                                pos.len(),
                                                crate::sources::base::SourceBase::MAX_SOURCE_POSITION_BYTES
                                            );
                                            None
                                        } else {
                                            Some(pos.clone())
                                        }
                                    });
                                    handover_positions.insert(source_id.clone(), (seq, position));
                                }
                            }
                        }

                        info!(
                            "[BOOTSTRAP] Query '{query_id_clone}' all sources completed bootstrap, \
                             handover checkpoints: {handover_checkpoints:?}"
                        );

                        // Persist recovery checkpoints before opening the gate.
                        // This ensures crash-after-bootstrap-before-first-streaming-event
                        // doesn't lose progress and avoids a redundant re-bootstrap.
                        if !handover_positions.is_empty() {
                            let session_ok = if let Some(sc) = &session_control_for_supervisor {
                                match sc.begin().await {
                                    Ok(()) => true,
                                    Err(e) => {
                                        warn!(
                                            "[BOOTSTRAP] Query '{query_id_clone}' failed to begin session \
                                             for handover persistence: {e}; checkpoints will not be \
                                             persisted until the first streaming event"
                                        );
                                        false
                                    }
                                }
                            } else {
                                true // no session control needed (e.g. in-memory store)
                            };

                            if session_ok {
                                for (source_id, (seq, position)) in &handover_positions {
                                    if let Err(e) = checkpoint_store_for_supervisor
                                        .stage_checkpoint(source_id, *seq, position.as_ref())
                                        .await
                                    {
                                        warn!(
                                            "[BOOTSTRAP] Query '{query_id_clone}' failed to persist \
                                             handover checkpoint for '{source_id}': {e}"
                                        );
                                    }
                                }
                                if let Some(sc) = &session_control_for_supervisor {
                                    if let Err(e) = sc.commit().await {
                                        warn!(
                                            "[BOOTSTRAP] Query '{query_id_clone}' failed to commit \
                                             handover checkpoints: {e}"
                                        );
                                    }
                                }
                            }
                        }

                        // Send handover checkpoints to the processor task
                        let _ = handover_tx.send(handover_checkpoints);

                        // Emit bootstrapCompleted control signal
                        let mut metadata = HashMap::new();
                        metadata.insert(
                            "control_signal".to_string(),
                            serde_json::json!("bootstrapCompleted"),
                        );

                        let control_result = QueryResult::new(
                            query_id_clone.clone(),
                            0,
                            chrono::Utc::now(),
                            vec![],
                            metadata,
                        );

                        let arc_result = Arc::new(control_result);

                        // Dispatch bootstrapCompleted signal to all reactions
                        let dispatchers = base_dispatchers_clone.read().await;
                        let mut dispatched = false;
                        for dispatcher in dispatchers.iter() {
                            if dispatcher.dispatch_change(arc_result.clone()).await.is_ok() {
                                dispatched = true;
                            }
                        }

                        if !dispatched {
                            debug!(
                                "No reactions subscribed to query '{query_id_clone}' for bootstrapCompleted signal"
                            );
                        } else {
                            info!(
                                "[BOOTSTRAP] Emitted bootstrapCompleted signal for query '{query_id_clone}'"
                            );
                        }

                        // Open the bootstrap gate so the event processor can start
                        bootstrap_gate_clone.notify_one();
                        info!("[BOOTSTRAP] Query '{query_id_clone}' bootstrap gate opened");
                    }
                    .instrument(span),
                );
                abort_handles.push(supervisor_handle.abort_handle());
            }

            // Store abort handles for cleanup on stop()
            *self.bootstrap_abort_handles.write().await = abort_handles;
        } else {
            info!(
                "Query '{}' no bootstrap channels, skipping bootstrap",
                self.base.config.id
            );
            // No bootstrap needed — send empty handover and open the gate immediately
            let _ = handover_tx.send(std::collections::HashMap::new());
            bootstrap_gate.notify_one();
        }

        // Spawn FutureQueueSource forwarder task (same pattern as other sources)
        {
            let fq_priority_queue = self.priority_queue.clone();
            let fq_forwarder = tokio::spawn(async move {
                let mut receiver = fq_receiver;
                while let Ok(event) = receiver.recv().await {
                    fq_priority_queue.enqueue_wait(event).await;
                }
            });
            self.subscription_tasks.write().await.push(fq_forwarder);
        }

        // Spawn event processor task that reads from priority queue
        let continuous_query_for_processor = continuous_query.clone();
        let checkpoint_store_for_processor = checkpoint_store.clone();
        let base_dispatchers = self.base.dispatchers.clone();
        let query_id = self.base.config.id.clone();
        let output_state = self.output_state.clone();
        let task_handle_clone = self.base.task_handle.clone();
        let priority_queue = self.priority_queue.clone();
        let instance_id = self.instance_id.clone();
        let reporter_for_processor = self.base.status_handle();
        let fq_source_for_processor = Arc::clone(&future_queue_source);
        let position_handles_for_processor = position_handles;
        let source_ids_for_processor: Vec<String> = self
            .base
            .config
            .sources
            .iter()
            .map(|s| s.source_id.clone())
            .collect();

        // Create shutdown channel for graceful termination
        let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel::<()>();
        self.base.set_shutdown_tx(shutdown_tx).await;

        let span = tracing::info_span!(
            "query_processor",
            instance_id = %instance_id,
            component_id = %query_id,
            component_type = "query"
        );
        let handle = tokio::spawn(
            async move {
                info!("Query '{query_id}' waiting for bootstrap gate before processing events");

                // Wait for bootstrap to complete (or immediate signal if no bootstrap).
                // If shutdown arrives while waiting, exit cleanly.
                tokio::select! {
                    biased;

                    _ = &mut shutdown_rx => {
                        info!(
                            "Query '{query_id}' received shutdown during bootstrap wait, exiting"
                        );
                        return;
                    }

                    _ = bootstrap_gate.notified() => {
                        info!("Query '{query_id}' bootstrap gate opened, starting event processing");
                    }
                }

                // Bootstrap complete — transition to Running only if still Starting.
                // If stop() was called during bootstrap, status may already be
                // Stopping and we must not overwrite it.
                let should_run = matches!(reporter_for_processor.get_status().await, ComponentStatus::Starting);

                if should_run {
                    reporter_for_processor.set_status(
                        ComponentStatus::Running,
                        Some("Query started successfully".to_string()),
                    ).await;
                } else {
                    let current = reporter_for_processor.get_status().await;
                    warn!(
                        "Query '{query_id}' bootstrap completed but status is {current:?}, \
                         skipping transition to Running"
                    );
                }

                // Start FutureQueueSource after bootstrap completes
                if let Err(e) = fq_source_for_processor.start().await {
                    error!("Query '{query_id}' failed to start FutureQueueSource: {e}");
                    reporter_for_processor
                        .set_status(
                            ComponentStatus::Error,
                            Some(format!("Future queue start failed: {e}")),
                        )
                        .await;
                    return;
                }

                info!("Query '{query_id}' starting priority queue event processor");

                // Initialize dedup filter from stored checkpoints (if resuming).
                // Then apply handover checkpoints from the bootstrap supervisor
                // which override the initial state for bootstrapped sources.
                let mut dedup = super::SequenceDedup::new(checkpoint_sequences_per_source.clone());
                if let Ok(handover) = handover_rx.await {
                    for (source_id, seq) in &handover {
                        dedup.advance(source_id, *seq);
                    }
                    if !handover.is_empty() {
                        info!(
                            "Query '{query_id}' applied {} handover checkpoint(s) to dedup filter",
                            handover.len()
                        );
                    }

                    // Update position handles with handover checkpoints so sources
                    // know the query's durable progress from the start.
                    for (source_id, seq) in &handover {
                        if let Some(handle) = position_handles_for_processor.get(source_id) {
                            handle.store(*seq, std::sync::atomic::Ordering::Release);
                        }
                    }
                }

                loop {
                    // Check if query is still running
                    let current_status = reporter_for_processor.get_status().await;
                    if !matches!(current_status, ComponentStatus::Running) {
                        info!(
                            "Query '{query_id}' status changed to non-running ({current_status:?}), exiting processing loop"
                        );
                        break;
                    }

                    tokio::select! {
                        biased;

                        _ = &mut shutdown_rx => {
                            info!(
                                "Query '{query_id}' received shutdown signal, exiting processing loop"
                            );
                            break;
                        }

                        // Dequeue events from priority queue (blocks until available)
                        arc_event = priority_queue.dequeue() => {
                            // Try to extract without cloning if we have sole ownership (zero-copy path).
                            let parts =
                                match SourceEventWrapper::try_unwrap_arc(arc_event) {
                                    Ok(parts) => parts,
                                    Err(arc) => {
                                        crate::channels::events::SourceEventParts {
                                            source_id: arc.source_id.clone(),
                                            event: arc.event.clone(),
                                            timestamp: arc.timestamp,
                                            profiling: arc.profiling.clone(),
                                            sequence: arc.sequence,
                                            source_position: arc.source_position.clone(),
                                        }
                                    }
                                };
                            let source_id = parts.source_id;
                            let event = parts.event;
                            let profiling_opt = parts.profiling;
                            let sequence = parts.sequence;
                            let source_position = parts.source_position;

                            debug!("Query '{query_id}' processing event from source '{source_id}'");

                            // Dedup: skip events already processed for this source
                            if dedup.should_skip(&source_id, sequence) {
                                debug!(
                                    "Query '{query_id}' skipping duplicate event from '{source_id}' (seq={seq}, checkpoint={cp})",
                                    seq = sequence.unwrap_or(0),
                                    cp = dedup.checkpoint_for(&source_id).unwrap_or(0)
                                );
                                continue;
                            }

                            match event {
                                SourceEvent::Control(SourceControl::FuturesDue) => {
                                    // Drain all due futures atomically within sessions
                                    loop {
                                        match continuous_query_for_processor.process_due_futures().await {
                                            Ok(Some(due_result)) => {
                                                if !due_result.results.is_empty() {
                                                    let profiling = crate::profiling::ProfilingMetadata::new();
                                                    dispatch_query_results(
                                                        &due_result.results,
                                                        &due_result.source_id,
                                                        &query_id,
                                                        &output_state,
                                                        &base_dispatchers,
                                                        profiling,
                                                    )
                                                    .await;
                                                }
                                            }
                                            Ok(None) => break,
                                            Err(e) => {
                                                error!("Query '{query_id}' failed to process due futures: {e}");
                                                break;
                                            }
                                        }
                                    }
                                    continue;
                                }
                                SourceEvent::Change(source_change) => {
                                    let mut profiling =
                                        profiling_opt.unwrap_or_else(crate::profiling::ProfilingMetadata::new);
                                    profiling.query_receive_ns = Some(crate::profiling::timestamp_ns());
                                    profiling.query_core_call_ns = Some(crate::profiling::timestamp_ns());

                                    // Stage checkpoint inside the session via pre-commit hook.
                                    // This ensures checkpoint persistence is atomic with index updates.
                                    let cp_store = checkpoint_store_for_processor.clone();
                                    let cp_source_id = source_id.clone();
                                    let cp_position = source_position.clone();
                                    let hook = move || {
                                        async move {
                                            if let Some(seq) = sequence {
                                                // Enforce position size limit at checkpoint time:
                                                // oversized positions are skipped to preserve the
                                                // last known good position in the store.
                                                let pos_ref = match &cp_position {
                                                    Some(p) if p.len() <= crate::sources::base::SourceBase::MAX_SOURCE_POSITION_BYTES => Some(p),
                                                    _ => None,
                                                };
                                                cp_store
                                                    .stage_checkpoint(&cp_source_id, seq, pos_ref)
                                                    .await?;
                                            }
                                            Ok(())
                                        }
                                    };

                                    match continuous_query_for_processor
                                        .process_source_change_with_hook(source_change, hook)
                                        .await
                                    {
                                        Ok(results) => {
                                            profiling.query_core_return_ns = Some(crate::profiling::timestamp_ns());

                                            // Advance dedup and notify source on successful commit
                                            if let Some(seq) = sequence {
                                                dedup.advance(&source_id, seq);

                                                if let Some(handle) = position_handles_for_processor.get(&source_id) {
                                                    handle.store(seq, std::sync::atomic::Ordering::Release);
                                                }
                                            }

                                            if !results.is_empty() {
                                                profiling.query_send_ns = Some(crate::profiling::timestamp_ns());
                                                dispatch_query_results(
                                                    &results,
                                                    &source_id,
                                                    &query_id,
                                                    &output_state,
                                                    &base_dispatchers,
                                                    profiling,
                                                )
                                                .await;
                                            }
                                        }
                                        Err(e) => {
                                            error!("Query '{query_id}' failed to process source change: {e}");
                                        }
                                    }
                                }
                                SourceEvent::Control(_) => {
                                    debug!("Query '{query_id}' ignoring control event from source '{source_id}'");
                                    continue;
                                }
                            }
                        }
                    }
                }

                fq_source_for_processor.stop().await;

            info!("Query '{query_id}' processing task exited");
        }
        .instrument(span),
    );

        // Store the task handle
        *task_handle_clone.write().await = Some(handle);

        Ok(())
    }

    async fn stop(&self) -> Result<()> {
        log_component_stop("Query", &self.base.config.id);

        // Set Stopping on the local status handle. The manager has already validated
        // and applied the Stopping transition on the graph via validate_and_transition().
        // This local update is needed because the event processing loop checks the
        // handle's local status to decide when to exit.
        //
        // INVARIANT: The graph must already be in Stopping state before this point.
        debug_assert!(
            matches!(
                self.base.status_handle().get_status().await,
                ComponentStatus::Running | ComponentStatus::Starting | ComponentStatus::Stopping
            ),
            "DrasiQuery::stop() called but local handle is not in expected pre-stop state"
        );
        self.base
            .set_status(
                ComponentStatus::Stopping,
                Some("Stopping query".to_string()),
            )
            .await;

        // Abort bootstrap tasks and supervisor
        let bootstrap_aborts: Vec<_> = {
            let mut handles = self.bootstrap_abort_handles.write().await;
            handles.drain(..).collect()
        };
        for handle in bootstrap_aborts {
            handle.abort();
        }

        // Drain and abort source subscription forwarders so they don't leak across restarts
        let subscription_handles: Vec<_> = {
            let mut tasks = self.subscription_tasks.write().await;
            tasks.drain(..).collect()
        };

        for handle in subscription_handles {
            handle.abort();
            let _ = handle.await;
        }

        // Stop the FutureQueueSource polling task
        if let Some(fq) = self.future_queue_source.write().await.take() {
            fq.stop().await;
        }

        // Release position handles so sources can advance their min-watermark.
        // Each subscribed source may hold a position handle for this query.
        {
            let source_ids = self.subscribed_source_ids.read().await;
            for source_id in source_ids.iter() {
                if let Some(source) = self.source_manager.get_source_instance(source_id).await {
                    source.remove_position_handle(&self.base.config.id).await;
                    debug!(
                        "Query '{}' released position handle for source '{}'",
                        self.base.config.id, source_id
                    );
                }
            }
        }
        // Clear tracked source IDs
        self.subscribed_source_ids.write().await.clear();

        // Use QueryBase common stop behavior to finish shutting down the processor task
        self.base.stop_common().await?;

        self.base
            .set_status(
                ComponentStatus::Stopped,
                Some("Query stopped successfully".to_string()),
            )
            .await;

        Ok(())
    }

    async fn status(&self) -> ComponentStatus {
        self.base.get_status().await
    }

    fn get_config(&self) -> &QueryConfig {
        &self.base.config
    }

    fn as_any(&self) -> &dyn std::any::Any {
        self
    }

    async fn subscription_count(&self) -> usize {
        self.subscription_tasks.read().await.len()
    }

    async fn subscribe(&self, reaction_id: String) -> Result<QuerySubscriptionResponse> {
        debug!(
            "Reaction '{}' subscribing to query '{}'",
            reaction_id, self.base.config.id
        );

        self.base
            .subscribe(&reaction_id)
            .await
            .context("Failed to subscribe to query")
    }

    async fn fetch_snapshot(&self) -> Result<SnapshotResponse, FetchError> {
        // Block until bootstrap is complete (status transitions from Starting to Running).
        // This ensures reactions don't observe a partial result set during initialization.
        self.wait_until_running().await?;

        let (results_clone, as_of_sequence) = {
            let state = self.output_state.read().await;
            (state.clone_results(), state.as_of_sequence())
        };
        Ok(SnapshotResponse::new(
            results_clone,
            as_of_sequence,
            self.config_hash,
        ))
    }

    async fn fetch_outbox(&self, after_sequence: u64) -> Result<OutboxResponse, FetchError> {
        // Block until bootstrap is complete — outbox is only populated by live processing.
        self.wait_until_running().await?;

        let state = self.output_state.read().await;
        let results = state
            .fetch_outbox_after(after_sequence)
            .map_err(|mut gap| {
                gap.config_hash = self.config_hash;
                gap
            })?;
        Ok(OutboxResponse {
            latest_sequence: state.as_of_sequence(),
            results,
            config_hash: self.config_hash,
        })
    }
}

pub struct QueryManager {
    instance_id: String,
    source_manager: Arc<SourceManager>,
    index_factory: Arc<crate::indexes::IndexFactory>,
    middleware_registry: Arc<MiddlewareTypeRegistry>,
    log_registry: Arc<ComponentLogRegistry>,
    /// Shared component graph — the single source of truth for component metadata,
    /// state, relationships, runtime instances, AND event history.
    graph: Arc<RwLock<ComponentGraph>>,
    /// Channel sender for routing status updates through the graph update loop.
    /// Managers send transitional states (Starting, Stopping, Reconfiguring) here;
    /// the loop applies them to the graph and records events automatically.
    update_tx: ComponentUpdateSender,
    /// Global default recovery policy. Per-query overrides this; if neither is set,
    /// defaults to `Strict`.
    default_recovery_policy: Option<crate::recovery::RecoveryPolicy>,
    /// Cached query labels extracted at registration time to avoid re-parsing
    /// queries on every `get_graph_schema()` call.
    label_cache: RwLock<HashMap<String, QueryLabels>>,
}

impl QueryManager {
    pub fn new(
        instance_id: impl Into<String>,
        source_manager: Arc<SourceManager>,
        index_factory: Arc<crate::indexes::IndexFactory>,
        middleware_registry: Arc<MiddlewareTypeRegistry>,
        log_registry: Arc<ComponentLogRegistry>,
        graph: Arc<RwLock<ComponentGraph>>,
        update_tx: ComponentUpdateSender,
        default_recovery_policy: Option<crate::recovery::RecoveryPolicy>,
    ) -> Self {
        Self {
            instance_id: instance_id.into(),
            source_manager,
            index_factory,
            middleware_registry,
            log_registry,
            graph,
            update_tx,
            default_recovery_policy,
            label_cache: RwLock::new(HashMap::new()),
        }
    }

    /// Register and provision a new query from the given configuration.
    ///
    /// # Errors
    /// Returns an error if provisioning fails (e.g., invalid config or duplicate ID).
    pub async fn add_query(&self, config: QueryConfig) -> Result<()> {
        self.provision_query(config).await
    }

    pub async fn add_query_without_save(&self, config: QueryConfig) -> Result<()> {
        self.provision_query(config).await
    }

    /// Add a pre-created query instance (for testing)
    pub async fn add_query_instance_for_test(&self, query: Arc<dyn Query>) -> Result<()> {
        let query_id = query.get_config().id.clone();

        // Cache labels from the query config
        let config = query.get_config();
        match LabelExtractor::extract_labels(&config.query, &config.query_language) {
            Ok(labels) => {
                self.label_cache
                    .write()
                    .await
                    .insert(query_id.clone(), labels);
            }
            Err(e) => {
                warn!("Failed to extract labels for test query '{query_id}': {e}");
            }
        }

        let mut graph = self.graph.write().await;
        if graph.has_runtime(&query_id) {
            return Err(anyhow::anyhow!("Query with id '{query_id}' already exists"));
        }
        graph.set_runtime(&query_id, Box::new(query))?;
        Ok(())
    }

    /// Provision a query for runtime — create the DrasiQuery, initialize, and store it.
    ///
    /// This method handles runtime-only operations: creating the DrasiQuery instance,
    /// initializing it with the runtime context, and storing it in the runtime map.
    /// Graph registration (node creation, dependency edges) must be done by the caller
    /// beforehand via `ComponentGraph::register_query()`.
    pub async fn provision_query(&self, config: QueryConfig) -> Result<()> {
        // Cache labels at registration time to avoid re-parsing on every get_graph_schema() call
        match LabelExtractor::extract_labels(&config.query, &config.query_language) {
            Ok(labels) => {
                self.label_cache
                    .write()
                    .await
                    .insert(config.id.clone(), labels);
            }
            Err(e) => {
                warn!("Failed to extract labels for query '{}': {e}", config.id);
            }
        }

        // Create the query instance
        let query = DrasiQuery::new(
            &self.instance_id,
            config.clone(),
            self.source_manager.clone(),
            self.index_factory.clone(),
            self.middleware_registry.clone(),
            self.default_recovery_policy,
        )?;

        // Wire status handle to graph via context (same pattern as Source/Reaction)
        let context = crate::context::QueryRuntimeContext::new(
            &self.instance_id,
            &config.id,
            self.update_tx.clone(),
        );
        query.initialize(context).await;

        let query: Arc<dyn Query> = Arc::new(query);

        let query_id = config.id.clone();
        let should_auto_start = config.auto_start;

        // Store the runtime instance in the graph
        {
            let mut graph = self.graph.write().await;
            graph.set_runtime(&config.id, Box::new(query))?;
        }

        info!("Provisioned query: {} with bootstrap support", config.id);

        // Note: Auto-start is handled by the caller (server.add_query)
        // which has access to the data router for subscriptions
        if should_auto_start {
            info!("Query '{query_id}' is configured for auto-start (will be started by caller)");
        }

        Ok(())
    }

    /// Start a query by ID, subscribing it to its sources and beginning event processing.
    ///
    /// # Errors
    /// Returns an error if the query is not found or the start transition fails.
    pub async fn start_query(&self, id: String) -> Result<()> {
        let query =
            crate::managers::lifecycle_helpers::get_runtime::<Arc<dyn Query>>(&self.graph, &id)
                .await
                .ok_or_else(|| {
                    anyhow::Error::new(crate::managers::ComponentNotFoundError::new("query", &id))
                })?;

        crate::managers::lifecycle_helpers::start_component(&self.graph, &id, "query", &query).await
    }

    /// Stop a running query by ID, unsubscribing it from sources and halting event processing.
    ///
    /// # Errors
    /// Returns an error if the query is not found or the stop transition fails.
    pub async fn stop_query(&self, id: String) -> Result<()> {
        let query =
            crate::managers::lifecycle_helpers::get_runtime::<Arc<dyn Query>>(&self.graph, &id)
                .await
                .ok_or_else(|| {
                    anyhow::Error::new(crate::managers::ComponentNotFoundError::new("query", &id))
                })?;

        crate::managers::lifecycle_helpers::stop_component(&self.graph, &id, "query", &query).await
    }

    /// Return the current lifecycle status of the query with the given ID.
    ///
    /// # Errors
    /// Returns an error if the query is not found in the component graph.
    pub async fn get_query_status(&self, id: String) -> Result<ComponentStatus> {
        crate::managers::lifecycle_helpers::get_component_status(&self.graph, &id, "Query").await
    }

    /// Get a query instance for subscription by reactions
    /// Returns Arc<dyn Query> which reactions can use to subscribe to query results
    pub async fn get_query_instance(&self, query_id: &str) -> Result<Arc<dyn Query>, String> {
        let graph = self.graph.read().await;
        if let Some(query) = graph.get_runtime::<Arc<dyn Query>>(query_id) {
            Ok(Arc::clone(query))
        } else {
            Err(format!(
                "Query '{query_id}' not found. Available queries can be listed using list_queries()."
            ))
        }
    }

    /// Retrieve the full runtime descriptor for a query, including its status and configuration.
    ///
    /// # Errors
    /// Returns an error if the query is not found.
    pub async fn get_query(&self, id: String) -> Result<QueryRuntime> {
        let graph = self.graph.read().await;
        let query = graph.get_runtime::<Arc<dyn Query>>(&id).cloned();

        if let Some(query) = query {
            let status = graph
                .get_component(&id)
                .map(|n| n.status)
                .unwrap_or(ComponentStatus::Stopped);
            let config = query.get_config();
            let error_message = match &status {
                ComponentStatus::Error => graph.get_last_error(&id),
                _ => None,
            };
            drop(graph);
            let runtime = QueryRuntime {
                id: config.id.clone(),
                query: config.query.clone(),
                status,
                error_message,
                source_subscriptions: config.sources.clone(),
                joins: config.joins.clone(),
            };
            Ok(runtime)
        } else {
            Err(crate::managers::ComponentNotFoundError::new("query", &id).into())
        }
    }

    /// Update a query by replacing it with a new configuration.
    ///
    /// Flow: validate exists → validate status → set Reconfiguring via graph →
    /// stop if running/starting → wait for stopped → provision new →
    /// replace runtime (if still exists) → restart if was running.
    /// Graph node, edges, and event history are preserved.
    pub async fn update_query(&self, id: String, new_config: QueryConfig) -> Result<()> {
        let old_query = {
            let graph = self.graph.read().await;
            graph.get_runtime::<Arc<dyn Query>>(&id).cloned()
        };

        if let Some(old_query) = old_query {
            // Verify the new config has the same ID
            if new_config.id != id {
                return Err(anyhow::anyhow!(
                    "New query ID '{}' does not match existing query ID '{}'",
                    new_config.id,
                    id
                ));
            }

            crate::managers::lifecycle_helpers::reconfigure_component::<Arc<dyn Query>, _, _, _>(
                &self.graph,
                &id,
                "query",
                &old_query,
                || async {},
                || self.provision_query(new_config),
                || self.start_query(id.clone()),
            )
            .await
        } else {
            Err(crate::managers::ComponentNotFoundError::new("query", &id).into())
        }
    }

    /// Teardown a query's runtime state — stop and remove from runtime map.
    ///
    /// This method handles runtime-only operations. Graph deregistration
    /// (node removal, edge cleanup) must be done by the caller afterwards via
    /// `ComponentGraph::deregister()`.
    ///
    /// The caller should validate dependencies via `graph.can_remove()` before calling this.
    pub async fn teardown_query(&self, id: String) -> Result<()> {
        // Before teardown: grab the query config to determine if persistent
        // state cleanup is needed. After teardown_component, the runtime is
        // removed from the graph and we can no longer inspect it.
        let query_config = {
            let graph = self.graph.read().await;
            graph
                .get_runtime::<Arc<dyn Query>>(&id)
                .map(|q| q.get_config().clone())
        };

        self.label_cache.write().await.remove(&id);
        crate::managers::lifecycle_helpers::teardown_component::<Arc<dyn Query>, _, _>(
            &self.graph,
            &id,
            "query",
            ComponentType::Query,
            &self.instance_id,
            &self.log_registry,
            false,
            || async {},
        )
        .await?;

        // After teardown: clear persistent indexes + checkpoints so a future
        // query with the same ID starts fresh. Only needed for persistent backends.
        if let Some(config) = query_config {
            if let Some(backend_ref) = &config.storage_backend {
                if !self.index_factory.is_volatile(backend_ref) {
                    info!("Query '{id}' removed — clearing persistent indexes and checkpoints");
                    match self.index_factory.build(backend_ref, &id).await {
                        Ok(created) => {
                            // Wrap clearing in a session for transactional backends
                            if let Err(e) = created.set.session_control.begin().await {
                                warn!(
                                    "Query '{id}' failed to begin session for removal cleanup: {e}"
                                );
                            } else {
                                if let Err(e) = clear_persistent_indexes(
                                    &id,
                                    &Some(created.set.element_index),
                                    &Some(created.set.archive_index),
                                    &Some(created.set.result_index),
                                    &Some(created.set.future_queue),
                                )
                                .await
                                {
                                    warn!(
                                        "Query '{id}' failed to clear persistent indexes on removal: {e}"
                                    );
                                }

                                if let Some(checkpoint_store) = created.checkpoint_store {
                                    if let Err(e) = checkpoint_store.clear_checkpoints().await {
                                        warn!(
                                            "Query '{id}' failed to clear checkpoints on removal: {e}"
                                        );
                                    }
                                }

                                if let Err(e) = created.set.session_control.commit().await {
                                    warn!(
                                        "Query '{id}' failed to commit removal cleanup session: {e}"
                                    );
                                }
                            }
                        }
                        Err(e) => {
                            warn!(
                                "Query '{id}' failed to build indexes for cleanup on removal: {e}"
                            );
                        }
                    }
                }
            }
        }

        Ok(())
    }

    /// List all registered queries with their current lifecycle status.
    pub async fn list_queries(&self) -> Vec<(String, ComponentStatus)> {
        crate::managers::lifecycle_helpers::list_components(&self.graph, &ComponentKind::Query)
            .await
    }

    pub async fn get_query_config(&self, id: &str) -> Option<QueryConfig> {
        let graph = self.graph.read().await;
        graph
            .get_runtime::<Arc<dyn Query>>(id)
            .map(|q| q.get_config().clone())
    }

    /// Return all cached query labels as (query_id, labels) pairs.
    ///
    /// Labels are extracted at query registration time and cached to avoid
    /// re-parsing the query string on every `get_graph_schema()` call.
    pub async fn get_all_query_labels(&self) -> Vec<(String, QueryLabels)> {
        self.label_cache
            .read()
            .await
            .iter()
            .map(|(id, labels)| (id.clone(), labels.clone()))
            .collect()
    }

    pub async fn get_query_results(&self, id: &str) -> Result<Vec<serde_json::Value>> {
        let query = {
            let graph = self.graph.read().await;
            graph.get_runtime::<Arc<dyn Query>>(id).cloned()
        };

        if let Some(query) = query {
            // Check if the query is running
            let status = query.status().await;
            if status != ComponentStatus::Running {
                return Err(anyhow::anyhow!("Query '{id}' is not running"));
            }

            let snapshot = query
                .fetch_snapshot()
                .await
                .map_err(|e| anyhow::anyhow!("Failed to fetch snapshot: {e}"))?;
            Ok(snapshot.to_vec())
        } else {
            Err(crate::managers::ComponentNotFoundError::new("query", id).into())
        }
    }

    /// Start all queries that are configured for auto-start.
    ///
    /// # Errors
    /// Returns an error if any query fails to start.
    pub async fn start_all(&self) -> Result<()> {
        crate::managers::lifecycle_helpers::start_all_components::<Arc<dyn Query>, _, _>(
            &self.graph,
            &ComponentKind::Query,
            "query",
            |q| q.get_config().auto_start,
            |id, query| async move {
                // Validate and apply Starting transition atomically through the graph
                {
                    let mut graph = self.graph.write().await;
                    graph.validate_and_transition(
                        &id,
                        ComponentStatus::Starting,
                        Some("Starting query".to_string()),
                    )?;
                }

                if let Err(e) = query.start().await {
                    let mut graph = self.graph.write().await;
                    let _ = graph.validate_and_transition(
                        &id,
                        ComponentStatus::Error,
                        Some(format!("Start failed: {e}")),
                    );
                    return Err(e);
                }
                Ok(())
            },
        )
        .await
    }

    /// Stop all currently running or starting queries.
    ///
    /// # Errors
    /// Returns an error listing any queries that failed to stop.
    pub async fn stop_all(&self) -> Result<()> {
        let query_ids: Vec<String> = {
            let graph = self.graph.read().await;
            graph
                .list_by_kind(&ComponentKind::Query)
                .iter()
                .map(|(id, _)| id.clone())
                .collect()
        };

        let mut failed_queries = Vec::new();

        for id in query_ids {
            let is_active = {
                let graph = self.graph.read().await;
                graph
                    .get_component(&id)
                    .map(|n| {
                        matches!(
                            n.status,
                            ComponentStatus::Running | ComponentStatus::Starting
                        )
                    })
                    .unwrap_or(false)
            };

            if is_active {
                if let Err(e) = self.stop_query(id.clone()).await {
                    log_component_error("Query", &id, &e.to_string());
                    failed_queries.push((id, e.to_string()));
                }
            }
        }

        if !failed_queries.is_empty() {
            let error_msg = failed_queries
                .iter()
                .map(|(id, err)| format!("{id}: {err}"))
                .collect::<Vec<_>>()
                .join(", ");
            Err(anyhow::anyhow!("Failed to stop some queries: {error_msg}"))
        } else {
            Ok(())
        }
    }

    /// Record a component event — delegates to the graph's centralized event history.
    pub async fn record_event(&self, event: ComponentEvent) {
        let mut graph = self.graph.write().await;
        graph.record_event(event);
    }

    /// Get events for a specific query.
    ///
    /// Returns events in chronological order (oldest first).
    pub async fn get_query_events(&self, id: &str) -> Vec<ComponentEvent> {
        self.graph.read().await.get_events(id)
    }

    /// Get all events across all queries.
    ///
    /// Returns events sorted by timestamp (oldest first).
    pub async fn get_all_events(&self) -> Vec<ComponentEvent> {
        let graph = self.graph.read().await;
        graph
            .get_all_events()
            .into_iter()
            .filter(|e| e.component_type == ComponentType::Query)
            .collect()
    }

    /// Subscribe to live logs for a query.
    ///
    /// Returns the log history and a broadcast receiver for new logs.
    /// Returns None if the query doesn't exist.
    pub async fn subscribe_logs(
        &self,
        id: &str,
    ) -> Option<(
        Vec<crate::managers::LogMessage>,
        tokio::sync::broadcast::Receiver<crate::managers::LogMessage>,
    )> {
        // Verify the query exists in the graph
        {
            let graph = self.graph.read().await;
            if !graph.has_runtime(id) {
                return None;
            }
        }

        let log_key = ComponentLogKey::new(&self.instance_id, ComponentType::Query, id);
        Some(self.log_registry.subscribe_by_key(&log_key).await)
    }

    /// Subscribe to live events for a query.
    ///
    /// Returns the event history and a broadcast receiver for new events.
    /// Returns None if the query doesn't exist.
    pub async fn subscribe_events(
        &self,
        id: &str,
    ) -> Option<(
        Vec<ComponentEvent>,
        tokio::sync::broadcast::Receiver<ComponentEvent>,
    )> {
        let graph = self.graph.read().await;
        if !graph.has_runtime(id) {
            return None;
        }
        graph.subscribe_events(id)
    }
}

#[async_trait]
impl crate::reactions::QueryProvider for QueryManager {
    async fn get_query_instance(&self, id: &str) -> Result<Arc<dyn Query>> {
        self.get_query_instance(id)
            .await
            .map_err(|e| anyhow::anyhow!(e))
    }
}