rhei-datafusion 1.5.0

DataFusion OLAP backend for Rhei HTAP engine
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
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
//! DataFusion-backed OLAP engine.
//!
//! Uses Apache DataFusion as the query engine with pluggable storage:
//! - **InMemory**: `Vec<RecordBatch>` in a `HashMap` (default, lost on shutdown)
//! - **ArrowIpc**: persistent `.arrow` files per table
//! - **Parquet**: persistent `.parquet` files per table
//!
//! ## DML Strategy
//!
//! DataFusion's `MemTable` / `ListingTable` don't support INSERT/UPDATE/DELETE DML.
//! Instead, we maintain our own table store (in-memory or file-backed) and
//! re-register tables with DataFusion after mutations. For queries,
//! DataFusion's SQL engine executes against the registered tables.

use std::collections::HashMap;
use std::path::PathBuf;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::Arc;

use arrow::array::{Array, AsArray, BooleanBuilder, RecordBatch};
use arrow::datatypes::{
    DataType, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, SchemaRef,
    UInt16Type, UInt32Type, UInt64Type, UInt8Type,
};
use datafusion::datasource::file_format::arrow::ArrowFormat;
use datafusion::datasource::file_format::parquet::ParquetFormat;
use datafusion::datasource::file_format::FileFormat;
use datafusion::datasource::listing::{
    ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl,
};
use datafusion::datasource::MemTable;
use datafusion::prelude::*;
use sqlparser::ast::{
    AssignmentTarget, BinaryOperator, Expr, FromTable, SetExpr, Statement, TableFactor,
    TableObject, UnaryOperator, Value,
};
use sqlparser::dialect::SQLiteDialect;
use sqlparser::parser::Parser;
use tokio::sync::RwLock;
use tracing::debug;

use crate::error::DfOlapError;
use crate::storage::StorageMode;

// cloud-storage: object_store + url imports
#[cfg(feature = "cloud-storage")]
use object_store::ObjectStore;
#[cfg(feature = "cloud-storage")]
use url::Url;

/// Per-table in-memory data store (used in `InMemory` mode).
struct TableData {
    schema: SchemaRef,
    /// Stored as a flat list of RecordBatches. Periodically compacted.
    batches: Vec<RecordBatch>,
}

/// Per-table metadata for file-backed modes.
struct FileTableMeta {
    schema: SchemaRef,
    /// Table directory: `<base_path>/<table_name>/`
    dir: PathBuf,
}

/// Per-table metadata for cloud-backed modes (S3 / GCS).
#[cfg(feature = "cloud-storage")]
struct CloudTableMeta {
    schema: SchemaRef,
    /// Base URL for this table's objects, e.g. `s3://bucket/prefix/tablename/`.
    table_url: String,
}

/// DataFusion-backed OLAP engine.
///
/// Supports pluggable storage via [`StorageMode`]:
/// - `InMemory`: stores Arrow data in memory, registers as `MemTable`
/// - `ArrowIpc`/`Parquet`: stores data as files on disk, registers as `ListingTable`
/// - `S3Parquet`/`GcsParquet` *(cloud-storage feature)*: Parquet files on object storage,
///   registered as `ListingTable` once the object store is wired into the `SessionContext`
pub struct DataFusionEngine {
    ctx: RwLock<SessionContext>,
    /// In-memory table store (only used in `InMemory` mode).
    tables: RwLock<HashMap<String, TableData>>,
    /// File-backed table metadata (only used in `ArrowIpc`/`Parquet` modes).
    file_tables: RwLock<HashMap<String, FileTableMeta>>,
    /// Cloud-backed table metadata (only used in `S3Parquet`/`GcsParquet` modes).
    #[cfg(feature = "cloud-storage")]
    cloud_tables: RwLock<HashMap<String, CloudTableMeta>>,
    /// Cloud object store handle for direct I/O (only used in cloud modes).
    ///
    /// Stored here to avoid having to look it up from the SessionContext on
    /// every read/write operation.  The same store is also registered with
    /// the SessionContext for DataFusion query routing.
    #[cfg(feature = "cloud-storage")]
    cloud_store: Option<Arc<dyn ObjectStore>>,
    /// Storage mode for this engine instance.
    storage_mode: StorageMode,
    /// Monotonic counter for generating unique file names.
    file_counter: AtomicU64,
}

impl DataFusionEngine {
    /// Create a new DataFusion engine with the given storage mode.
    ///
    /// For file-backed modes, creates the base directory if it doesn't exist and
    /// scans for existing data files to initialize the file counter (avoiding
    /// overwrites on restart).
    ///
    /// For cloud modes (`S3Parquet` / `GcsParquet`, requires the `cloud-storage`
    /// feature), registers the appropriate `object_store` implementation with the
    /// DataFusion `SessionContext`.  Credentials are resolved from the environment
    /// at construction time.
    pub fn with_storage(mode: StorageMode) -> Result<Self, DfOlapError> {
        let start_counter = if let Some(base_path) = mode.base_path() {
            std::fs::create_dir_all(base_path)?;
            Self::scan_max_file_seq(base_path, mode.file_extension())
        } else {
            0
        };

        #[cfg(feature = "cloud-storage")]
        let (ctx, cloud_store) = Self::build_session_context(&mode)?;
        #[cfg(not(feature = "cloud-storage"))]
        let ctx = SessionContext::new();

        Ok(Self {
            ctx: RwLock::new(ctx),
            tables: RwLock::new(HashMap::new()),
            file_tables: RwLock::new(HashMap::new()),
            #[cfg(feature = "cloud-storage")]
            cloud_tables: RwLock::new(HashMap::new()),
            #[cfg(feature = "cloud-storage")]
            cloud_store,
            storage_mode: mode,
            file_counter: AtomicU64::new(start_counter),
        })
    }

    /// Build a `SessionContext`, registering an object store for cloud modes.
    ///
    /// Returns `(ctx, Option<cloud_store>)` — the cloud store is `Some` for
    /// S3/GCS modes and is used for direct object I/O.
    #[cfg(feature = "cloud-storage")]
    fn build_session_context(
        mode: &StorageMode,
    ) -> Result<(SessionContext, Option<Arc<dyn ObjectStore>>), DfOlapError> {
        let ctx = SessionContext::new();
        let mut cloud_store: Option<Arc<dyn ObjectStore>> = None;

        match mode {
            StorageMode::S3Parquet { url } => {
                let bucket = Self::parse_bucket(url, "s3")?;
                let store: Arc<dyn ObjectStore> = Arc::new(
                    object_store::aws::AmazonS3Builder::from_env()
                        .with_bucket_name(&bucket)
                        .build()?,
                );
                // Register under `s3://<bucket>` so DataFusion resolves all
                // `s3://bucket/...` URLs to this store.
                let base_url =
                    Url::parse(&format!("s3://{bucket}")).map_err(DfOlapError::UrlParse)?;
                ctx.runtime_env()
                    .register_object_store(&base_url, store.clone());
                cloud_store = Some(store);
                tracing::info!(bucket, "registered S3 object store");
            }
            StorageMode::GcsParquet { url } => {
                let bucket = Self::parse_bucket(url, "gs")?;
                let store: Arc<dyn ObjectStore> = Arc::new(
                    object_store::gcp::GoogleCloudStorageBuilder::from_env()
                        .with_bucket_name(&bucket)
                        .build()?,
                );
                let base_url =
                    Url::parse(&format!("gs://{bucket}")).map_err(DfOlapError::UrlParse)?;
                ctx.runtime_env()
                    .register_object_store(&base_url, store.clone());
                cloud_store = Some(store);
                tracing::info!(bucket, "registered GCS object store");
            }
            _ => {}
        }

        Ok((ctx, cloud_store))
    }

    /// Parse the bucket name from a cloud URL like `s3://bucket/prefix` or `gs://bucket/prefix`.
    ///
    /// Returns an error if the scheme doesn't match or there is no host component.
    #[cfg(feature = "cloud-storage")]
    fn parse_bucket(url: &str, expected_scheme: &str) -> Result<String, DfOlapError> {
        let parsed = Url::parse(url).map_err(DfOlapError::UrlParse)?;
        if parsed.scheme() != expected_scheme {
            return Err(DfOlapError::Other(format!(
                "expected {expected_scheme}:// URL, got '{url}'"
            )));
        }
        parsed
            .host_str()
            .map(|h| h.to_string())
            .ok_or_else(|| DfOlapError::Other(format!("missing bucket name in URL '{url}'")))
    }

    /// Build the cloud URL for a given table: `<base_url>/<table_name>/`.
    ///
    /// E.g. for `s3://my-bucket/data` and table `events` → `s3://my-bucket/data/events/`.
    #[cfg(feature = "cloud-storage")]
    fn cloud_table_url(base_url: &str, table_name: &str) -> String {
        let base = base_url.trim_end_matches('/');
        format!("{base}/{table_name}/")
    }

    /// Create a new in-memory DataFusion engine (default).
    pub fn new() -> Self {
        // In-memory mode cannot fail (no I/O or object-store construction).
        Self::with_storage(StorageMode::InMemory).expect("in-memory mode cannot fail")
    }

    /// Returns the storage mode of this engine.
    pub fn storage_mode(&self) -> &StorageMode {
        &self.storage_mode
    }

    /// Scan existing files in the base directory to find the maximum sequence number.
    /// Returns the next sequence number to use (max + 1), or 0 if no files exist.
    fn scan_max_file_seq(base_path: &std::path::Path, ext: &str) -> u64 {
        let mut max_seq: u64 = 0;
        let Ok(entries) = std::fs::read_dir(base_path) else {
            return 0;
        };
        // Scan table subdirectories for files like `tablename_000042.parquet`
        for entry in entries.flatten() {
            if !entry.path().is_dir() {
                continue;
            }
            let Ok(files) = std::fs::read_dir(entry.path()) else {
                continue;
            };
            for file in files.flatten() {
                let path = file.path();
                if path.extension().and_then(|x| x.to_str()) != Some(ext) {
                    continue;
                }
                // Extract sequence from `<table>_<seq>.<ext>`
                if let Some(stem) = path.file_stem().and_then(|s| s.to_str()) {
                    if let Some(seq_str) = stem.rsplit('_').next() {
                        if let Ok(seq) = seq_str.parse::<u64>() {
                            max_seq = max_seq.max(seq + 1);
                        }
                    }
                }
            }
        }
        max_seq
    }

    /// Generate a unique file name for a new data file.
    fn next_file_name(&self, table_name: &str) -> String {
        let seq = self.file_counter.fetch_add(1, Ordering::Relaxed);
        let ext = self.storage_mode.file_extension();
        format!("{table_name}_{seq:06}.{ext}")
    }

    /// Get the table directory for a file-backed table.
    fn table_dir(&self, table_name: &str) -> Option<PathBuf> {
        self.storage_mode
            .base_path()
            .map(|base| base.join(table_name))
    }

    // -----------------------------------------------------------------------
    // In-memory table registration
    // -----------------------------------------------------------------------

    /// Re-register a table with DataFusion's SessionContext from in-memory data.
    async fn refresh_table_mem(&self, name: &str) -> Result<(), DfOlapError> {
        let tables = self.tables.read().await;
        let table_data = tables
            .get(name)
            .ok_or_else(|| DfOlapError::TableNotFound(name.to_string()))?;

        // MemTable requires at least one partition.
        // For empty tables, pass a single empty partition (vec of zero batches).
        let partitions = if table_data.batches.is_empty() {
            vec![vec![]]
        } else {
            vec![table_data.batches.clone()]
        };
        let mem_table = MemTable::try_new(table_data.schema.clone(), partitions)?;

        let ctx = self.ctx.write().await;
        let _ = ctx.deregister_table(name);
        ctx.register_table(name, Arc::new(mem_table))?;
        Ok(())
    }

    // -----------------------------------------------------------------------
    // File-backed table registration
    // -----------------------------------------------------------------------

    /// Register a file-backed table as a ListingTable with DataFusion.
    async fn refresh_table_file(&self, name: &str) -> Result<(), DfOlapError> {
        let file_tables = self.file_tables.read().await;
        let meta = file_tables
            .get(name)
            .ok_or_else(|| DfOlapError::TableNotFound(name.to_string()))?;

        let table_path = meta.dir.to_string_lossy().to_string();
        let format: Arc<dyn FileFormat> = match &self.storage_mode {
            StorageMode::ArrowIpc { .. } => Arc::new(ArrowFormat),
            StorageMode::Parquet { .. } => Arc::new(ParquetFormat::default()),
            StorageMode::InMemory => unreachable!(),
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => unreachable!(),
        };

        let ext = self.storage_mode.file_extension();
        let listing_options = ListingOptions::new(format).with_file_extension(ext);

        let config =
            ListingTableConfig::new_with_multi_paths(vec![ListingTableUrl::parse(&table_path)?])
                .with_listing_options(listing_options)
                .with_schema(meta.schema.clone());

        let listing_table = ListingTable::try_new(config)?;

        let ctx = self.ctx.write().await;
        let _ = ctx.deregister_table(name);
        ctx.register_table(name, Arc::new(listing_table))?;
        Ok(())
    }

    /// Register a cloud-backed table as a ListingTable with DataFusion.
    ///
    /// The object store must already be registered with the `SessionContext`
    /// (done in `with_storage`).  DataFusion's `ListingTable` resolves
    /// `s3://` / `gs://` URLs via the registered store.
    #[cfg(feature = "cloud-storage")]
    async fn refresh_table_cloud(&self, name: &str) -> Result<(), DfOlapError> {
        let cloud_tables = self.cloud_tables.read().await;
        let meta = cloud_tables
            .get(name)
            .ok_or_else(|| DfOlapError::TableNotFound(name.to_string()))?;

        let table_url = meta.table_url.clone();
        let schema = meta.schema.clone();
        drop(cloud_tables);

        let format: Arc<dyn FileFormat> = Arc::new(ParquetFormat::default());
        let listing_options = ListingOptions::new(format).with_file_extension("parquet");

        let config =
            ListingTableConfig::new_with_multi_paths(vec![ListingTableUrl::parse(&table_url)?])
                .with_listing_options(listing_options)
                .with_schema(schema);

        let listing_table = ListingTable::try_new(config)?;

        let ctx = self.ctx.write().await;
        let _ = ctx.deregister_table(name);
        ctx.register_table(name, Arc::new(listing_table))?;
        Ok(())
    }

    // -----------------------------------------------------------------------
    // File I/O helpers
    // -----------------------------------------------------------------------

    /// Write RecordBatches to a new file in the table's directory.
    async fn write_batches_to_file(
        &self,
        table_name: &str,
        schema: &SchemaRef,
        batches: &[RecordBatch],
    ) -> Result<PathBuf, DfOlapError> {
        let dir = self
            .table_dir(table_name)
            .ok_or_else(|| DfOlapError::Other("no table dir for in-memory mode".into()))?;

        let file_name = self.next_file_name(table_name);
        let file_path = dir.join(&file_name);

        let schema = schema.clone();
        let batches: Vec<RecordBatch> = batches.to_vec();
        let path = file_path.clone();

        match &self.storage_mode {
            StorageMode::ArrowIpc { .. } => {
                tokio::task::spawn_blocking(move || {
                    let file = std::fs::File::create(&path)?;
                    let mut writer =
                        arrow::ipc::writer::FileWriter::try_new(file, schema.as_ref())?;
                    for batch in &batches {
                        writer.write(batch)?;
                    }
                    writer.finish()?;
                    Ok::<_, DfOlapError>(())
                })
                .await
                .map_err(DfOlapError::from_join)??;
            }
            StorageMode::Parquet { .. } => {
                tokio::task::spawn_blocking(move || {
                    let file = std::fs::File::create(&path)?;
                    let props = parquet::file::properties::WriterProperties::builder()
                        .set_writer_version(parquet::file::properties::WriterVersion::PARQUET_2_0)
                        .build();
                    let mut writer =
                        parquet::arrow::ArrowWriter::try_new(file, schema, Some(props))?;
                    for batch in &batches {
                        writer.write(batch)?;
                    }
                    writer.close()?;
                    Ok::<_, DfOlapError>(())
                })
                .await
                .map_err(DfOlapError::from_join)??;
            }
            StorageMode::InMemory => unreachable!(),
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => unreachable!(),
        }

        Ok(file_path)
    }

    /// List data files in a table's directory, sorted by name for deterministic ordering.
    fn list_data_files(dir: &std::path::Path, ext: &str) -> Result<Vec<PathBuf>, DfOlapError> {
        let mut files: Vec<PathBuf> = std::fs::read_dir(dir)?
            .filter_map(|e| e.ok())
            .map(|e| e.path())
            .filter(|p| p.extension().is_some_and(|x| x.to_str() == Some(ext)))
            .collect();
        files.sort();
        Ok(files)
    }

    /// Read all RecordBatches from all files in a table's directory.
    async fn read_all_batches(
        &self,
        table_name: &str,
    ) -> Result<(SchemaRef, Vec<RecordBatch>), DfOlapError> {
        let file_tables = self.file_tables.read().await;
        let meta = file_tables
            .get(table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.to_string()))?;

        let schema = meta.schema.clone();
        let dir = meta.dir.clone();
        let ext = self.storage_mode.file_extension().to_string();
        let is_arrow_ipc = matches!(self.storage_mode, StorageMode::ArrowIpc { .. });

        drop(file_tables);

        tokio::task::spawn_blocking(move || {
            let mut all_batches = Vec::new();
            let files = Self::list_data_files(&dir, &ext)?;

            for path in files {
                if is_arrow_ipc {
                    let file = std::fs::File::open(&path)?;
                    let reader = arrow::ipc::reader::FileReader::try_new(file, None)?;
                    for batch in reader {
                        all_batches.push(batch?);
                    }
                } else {
                    let file = std::fs::File::open(&path)?;
                    let reader = parquet::arrow::arrow_reader::ParquetRecordBatchReader::try_new(
                        file, 8192,
                    )?;
                    for batch in reader {
                        all_batches.push(batch?);
                    }
                }
            }

            Ok::<_, DfOlapError>((schema, all_batches))
        })
        .await
        .map_err(DfOlapError::from_join)?
    }

    /// Remove all data files in a table's directory.
    async fn clear_table_dir(&self, table_name: &str) -> Result<(), DfOlapError> {
        let dir = match self.table_dir(table_name) {
            Some(d) => d,
            None => return Ok(()),
        };
        let ext = self.storage_mode.file_extension().to_string();

        tokio::task::spawn_blocking(move || {
            let files = Self::list_data_files(&dir, &ext)?;
            for path in files {
                std::fs::remove_file(path)?;
            }
            Ok::<_, DfOlapError>(())
        })
        .await
        .map_err(DfOlapError::from_join)?
    }

    // -----------------------------------------------------------------------
    // Cloud (S3 / GCS) I/O helpers
    // -----------------------------------------------------------------------

    /// Return the cloud object store, or error if not in cloud mode.
    #[cfg(feature = "cloud-storage")]
    fn cloud_store(&self) -> Result<Arc<dyn ObjectStore>, DfOlapError> {
        self.cloud_store
            .clone()
            .ok_or_else(|| DfOlapError::Other("cloud store not initialised".into()))
    }

    /// Write RecordBatches as a single Parquet object to the cloud store.
    ///
    /// The object key is `<table_url><table_name>_<seq>.parquet`.
    /// Serialisation happens synchronously in a `spawn_blocking` task; the
    /// resulting bytes are then pushed to the object store via `put`.
    #[cfg(feature = "cloud-storage")]
    async fn write_batches_to_cloud(
        &self,
        table_name: &str,
        schema: &SchemaRef,
        batches: &[RecordBatch],
    ) -> Result<(), DfOlapError> {
        let cloud_tables = self.cloud_tables.read().await;
        let meta = cloud_tables
            .get(table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.to_string()))?;
        let table_url = meta.table_url.clone();
        drop(cloud_tables);

        let store = self.cloud_store()?;

        // Serialise to Parquet in-memory.
        let schema = schema.clone();
        let batches: Vec<RecordBatch> = batches.to_vec();
        let parquet_bytes = tokio::task::spawn_blocking(move || {
            let mut buf = Vec::new();
            let props = parquet::file::properties::WriterProperties::builder()
                .set_writer_version(parquet::file::properties::WriterVersion::PARQUET_2_0)
                .build();
            let mut writer = parquet::arrow::ArrowWriter::try_new(&mut buf, schema, Some(props))?;
            for batch in &batches {
                writer.write(batch)?;
            }
            writer.close()?;
            Ok::<_, DfOlapError>(buf)
        })
        .await
        .map_err(DfOlapError::from_join)??;

        // Build the object path from the table URL path component.
        let seq = self.file_counter.fetch_add(1, Ordering::Relaxed);
        let object_key = format!("{table_name}_{seq:06}.parquet");
        let table_path_prefix = Url::parse(&table_url)
            .map_err(DfOlapError::UrlParse)?
            .path()
            .trim_start_matches('/')
            .trim_end_matches('/')
            .to_string();
        let object_path =
            object_store::path::Path::from(format!("{table_path_prefix}/{object_key}").as_str());

        use object_store::ObjectStoreExt as _;
        store.put(&object_path, parquet_bytes.into()).await?;

        Ok(())
    }

    /// Scan existing Parquet objects under `table_url` and return the next
    /// sequence number to use (max existing index + 1, or 0 if none exist).
    ///
    /// Called from `create_table` **before** the table entry is inserted, so
    /// it cannot use `list_cloud_objects` (which requires the entry to exist).
    #[cfg(feature = "cloud-storage")]
    async fn cloud_seq_for_prefix(
        store: &Arc<dyn ObjectStore>,
        table_url: &str,
        table_name: &str,
    ) -> Result<u64, DfOlapError> {
        let prefix_str = Url::parse(table_url)
            .map_err(DfOlapError::UrlParse)?
            .path()
            .trim_start_matches('/')
            .trim_end_matches('/')
            .to_string();
        let prefix = object_store::path::Path::from(prefix_str.as_str());

        use futures::StreamExt as _;
        use object_store::ObjectStore as _;
        let mut list_stream = store.list(Some(&prefix));

        let file_prefix = format!("{table_name}_");
        let mut max_seq: Option<u64> = None;
        while let Some(item) = list_stream.next().await {
            let m = item?;
            let path_str = m.location.to_string();
            // Only consider files like `<prefix>/<table>_<seq>.parquet`
            let file_name = path_str.rsplit('/').next().unwrap_or("");
            if !file_name.starts_with(&file_prefix) || !file_name.ends_with(".parquet") {
                continue;
            }
            // Strip prefix and `.parquet` suffix to get the seq number string.
            let inner = &file_name[file_prefix.len()..file_name.len() - ".parquet".len()];
            if let Ok(seq) = inner.parse::<u64>() {
                max_seq = Some(max_seq.map_or(seq, |m| m.max(seq)));
            }
        }

        Ok(max_seq.map_or(0, |m| m + 1))
    }

    /// List all Parquet object paths for a cloud-backed table.
    ///
    /// Returns `(store, table_url, Vec<Path>)`.
    #[cfg(feature = "cloud-storage")]
    async fn list_cloud_objects(
        &self,
        table_name: &str,
    ) -> Result<(Arc<dyn ObjectStore>, String, Vec<object_store::path::Path>), DfOlapError> {
        let cloud_tables = self.cloud_tables.read().await;
        let meta = cloud_tables
            .get(table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.to_string()))?;
        let table_url = meta.table_url.clone();
        drop(cloud_tables);

        let store = self.cloud_store()?;

        // The prefix is the path component of the table URL.
        let prefix_str = Url::parse(&table_url)
            .map_err(DfOlapError::UrlParse)?
            .path()
            .trim_start_matches('/')
            .trim_end_matches('/')
            .to_string();
        let prefix = object_store::path::Path::from(prefix_str.as_str());

        use futures::StreamExt as _;
        use object_store::ObjectStore as _;
        let mut list_stream = store.list(Some(&prefix));
        let mut paths = Vec::new();
        while let Some(item) = list_stream.next().await {
            let m = item?;
            if m.location.to_string().ends_with(".parquet") {
                paths.push(m.location);
            }
        }
        paths.sort_by_key(|a| a.to_string());

        Ok((store, table_url, paths))
    }

    /// Read all Parquet objects for a cloud-backed table into memory.
    #[cfg(feature = "cloud-storage")]
    async fn read_all_batches_cloud(
        &self,
        table_name: &str,
    ) -> Result<(SchemaRef, Vec<RecordBatch>), DfOlapError> {
        let cloud_tables = self.cloud_tables.read().await;
        let meta = cloud_tables
            .get(table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.to_string()))?;
        let schema = meta.schema.clone();
        drop(cloud_tables);

        let (store, _, object_paths) = self.list_cloud_objects(table_name).await?;

        let mut all_batches = Vec::new();
        for path in object_paths {
            use object_store::ObjectStoreExt as _;
            let get_result = store.get(&path).await?;
            let bytes = get_result.bytes().await?;
            let mut batch_vec = tokio::task::spawn_blocking(move || {
                // `bytes::Bytes` implements `parquet::file::reader::ChunkReader` directly.
                let reader =
                    parquet::arrow::arrow_reader::ParquetRecordBatchReader::try_new(bytes, 8192)?;
                let mut batches = Vec::new();
                for b in reader {
                    batches.push(b?);
                }
                Ok::<_, DfOlapError>(batches)
            })
            .await
            .map_err(DfOlapError::from_join)??;
            all_batches.append(&mut batch_vec);
        }

        Ok((schema, all_batches))
    }

    /// Delete all Parquet objects for a cloud-backed table.
    #[cfg(feature = "cloud-storage")]
    async fn clear_cloud_table(&self, table_name: &str) -> Result<(), DfOlapError> {
        let (store, _, paths) = self.list_cloud_objects(table_name).await?;
        use object_store::ObjectStoreExt as _;
        for path in paths {
            store.delete(&path).await?;
        }
        Ok(())
    }

    /// Execute a SQL query and collect all results into RecordBatches.
    async fn execute_sql(&self, sql: &str) -> Result<Vec<RecordBatch>, DfOlapError> {
        let ctx = self.ctx.read().await;
        let df = ctx.sql(sql).await?;
        let batches = df.collect().await?;
        Ok(batches)
    }

    /// Align parsed INSERT batches to the table schema (reorder + cast columns).
    fn align_batches_to_schema(
        table_schema: &SchemaRef,
        col_names: &[String],
        batches: &[RecordBatch],
    ) -> Result<(Vec<RecordBatch>, u64), DfOlapError> {
        let mut aligned_batches = Vec::with_capacity(batches.len());
        let mut total_rows = 0u64;
        for batch in batches {
            let mut columns: Vec<Arc<dyn Array>> = Vec::with_capacity(table_schema.fields().len());
            for field in table_schema.fields() {
                let idx = col_names
                    .iter()
                    .position(|c| c == field.name())
                    .ok_or_else(|| {
                        DfOlapError::SchemaMismatch(format!(
                            "column '{}' not in INSERT column list",
                            field.name()
                        ))
                    })?;
                let col = batch.column(idx);
                let col = if col.data_type() != field.data_type() {
                    arrow::compute::cast(col, field.data_type())?
                } else {
                    col.clone()
                };
                columns.push(col);
            }
            let aligned = RecordBatch::try_new(table_schema.clone(), columns)?;
            total_rows += aligned.num_rows() as u64;
            aligned_batches.push(aligned);
        }
        Ok((aligned_batches, total_rows))
    }

    // -----------------------------------------------------------------------
    // In-memory DML
    // -----------------------------------------------------------------------

    async fn execute_insert_mem(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, col_names, batches) = parse_insert_values(sql)?;

        let mut tables = self.tables.write().await;
        let table_data = tables
            .get_mut(&table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.clone()))?;

        let table_schema = table_data.schema.clone();
        let (aligned_batches, total_rows) =
            Self::align_batches_to_schema(&table_schema, &col_names, &batches)?;
        table_data.batches.extend(aligned_batches);
        drop(tables);

        self.refresh_table_mem(&table_name).await?;
        Ok(total_rows)
    }

    async fn execute_update_mem(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, assignments, where_clause) = parse_update(sql)?;

        let mut tables = self.tables.write().await;
        let table_data = tables
            .get_mut(&table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.clone()))?;

        let schema = table_data.schema.clone();
        let mut updated_count = 0u64;

        let all_rows = flatten_batches(&table_data.batches, &schema)?;
        if let Some(all_rows) = all_rows {
            let (updated_batch, count) =
                apply_update(&all_rows, &schema, &assignments, &where_clause)?;
            updated_count = count;
            table_data.batches = vec![updated_batch];
        }

        drop(tables);
        self.refresh_table_mem(&table_name).await?;
        Ok(updated_count)
    }

    async fn execute_delete_mem(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, where_clause) = parse_delete(sql)?;

        let mut tables = self.tables.write().await;
        let table_data = tables
            .get_mut(&table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.clone()))?;

        let schema = table_data.schema.clone();
        let all_rows = flatten_batches(&table_data.batches, &schema)?;

        if let Some(all_rows) = all_rows {
            let (filtered_batch, deleted_count) = apply_delete(&all_rows, &schema, &where_clause)?;
            table_data.batches = if filtered_batch.num_rows() > 0 {
                vec![filtered_batch]
            } else {
                vec![]
            };
            drop(tables);
            self.refresh_table_mem(&table_name).await?;
            Ok(deleted_count)
        } else {
            Ok(0)
        }
    }

    // -----------------------------------------------------------------------
    // File-backed DML
    // -----------------------------------------------------------------------

    async fn execute_insert_file(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, col_names, batches) = parse_insert_values(sql)?;

        let file_tables = self.file_tables.read().await;
        let meta = file_tables
            .get(&table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.clone()))?;
        let table_schema = meta.schema.clone();
        drop(file_tables);

        let (aligned_batches, total_rows) =
            Self::align_batches_to_schema(&table_schema, &col_names, &batches)?;
        self.write_batches_to_file(&table_name, &table_schema, &aligned_batches)
            .await?;
        self.refresh_table_file(&table_name).await?;
        Ok(total_rows)
    }

    async fn execute_update_file(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, assignments, where_clause) = parse_update(sql)?;

        let (schema, existing_batches) = self.read_all_batches(&table_name).await?;
        let all_rows = flatten_batches(&existing_batches, &schema)?;

        if let Some(all_rows) = all_rows {
            let (updated_batch, count) =
                apply_update(&all_rows, &schema, &assignments, &where_clause)?;
            // Rewrite: clear old files, write updated data
            self.clear_table_dir(&table_name).await?;
            if updated_batch.num_rows() > 0 {
                self.write_batches_to_file(&table_name, &schema, &[updated_batch])
                    .await?;
            }
            self.refresh_table_file(&table_name).await?;
            Ok(count)
        } else {
            Ok(0)
        }
    }

    async fn execute_delete_file(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, where_clause) = parse_delete(sql)?;

        let (schema, existing_batches) = self.read_all_batches(&table_name).await?;
        let all_rows = flatten_batches(&existing_batches, &schema)?;

        if let Some(all_rows) = all_rows {
            let (filtered_batch, deleted_count) = apply_delete(&all_rows, &schema, &where_clause)?;
            // Rewrite: clear old files, write remaining data
            self.clear_table_dir(&table_name).await?;
            if filtered_batch.num_rows() > 0 {
                self.write_batches_to_file(&table_name, &schema, &[filtered_batch])
                    .await?;
            }
            self.refresh_table_file(&table_name).await?;
            Ok(deleted_count)
        } else {
            Ok(0)
        }
    }

    // -----------------------------------------------------------------------
    // Cloud-backed DML
    // -----------------------------------------------------------------------

    /// INSERT for cloud-backed tables (S3 / GCS).
    ///
    /// Appends a new Parquet object to the table's cloud prefix.  For
    /// analytical workloads this is efficient — reads accumulate multiple
    /// files which DataFusion's `ListingTable` handles natively.
    #[cfg(feature = "cloud-storage")]
    async fn execute_insert_cloud(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, col_names, batches) = parse_insert_values(sql)?;

        let cloud_tables = self.cloud_tables.read().await;
        let meta = cloud_tables
            .get(&table_name)
            .ok_or_else(|| DfOlapError::TableNotFound(table_name.clone()))?;
        let table_schema = meta.schema.clone();
        drop(cloud_tables);

        let (aligned_batches, total_rows) =
            Self::align_batches_to_schema(&table_schema, &col_names, &batches)?;
        self.write_batches_to_cloud(&table_name, &table_schema, &aligned_batches)
            .await?;
        self.refresh_table_cloud(&table_name).await?;
        Ok(total_rows)
    }

    /// UPDATE for cloud-backed tables (S3 / GCS).
    ///
    /// Downloads all existing Parquet objects, mutates in-memory, then
    /// re-uploads as a single consolidated object.  This is a
    /// read-modify-write cycle and is expensive on large tables — see
    /// module-level comments.
    #[cfg(feature = "cloud-storage")]
    async fn execute_update_cloud(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, assignments, where_clause) = parse_update(sql)?;

        let (schema, existing_batches) = self.read_all_batches_cloud(&table_name).await?;
        let all_rows = flatten_batches(&existing_batches, &schema)?;

        if let Some(all_rows) = all_rows {
            let (updated_batch, count) =
                apply_update(&all_rows, &schema, &assignments, &where_clause)?;
            self.clear_cloud_table(&table_name).await?;
            if updated_batch.num_rows() > 0 {
                self.write_batches_to_cloud(&table_name, &schema, &[updated_batch])
                    .await?;
            }
            self.refresh_table_cloud(&table_name).await?;
            Ok(count)
        } else {
            Ok(0)
        }
    }

    /// DELETE for cloud-backed tables (S3 / GCS).
    ///
    /// Same read-modify-write pattern as [`execute_update_cloud`].
    #[cfg(feature = "cloud-storage")]
    async fn execute_delete_cloud(&self, sql: &str) -> Result<u64, DfOlapError> {
        let (table_name, where_clause) = parse_delete(sql)?;

        let (schema, existing_batches) = self.read_all_batches_cloud(&table_name).await?;
        let all_rows = flatten_batches(&existing_batches, &schema)?;

        if let Some(all_rows) = all_rows {
            let (filtered_batch, deleted_count) = apply_delete(&all_rows, &schema, &where_clause)?;
            self.clear_cloud_table(&table_name).await?;
            if filtered_batch.num_rows() > 0 {
                self.write_batches_to_cloud(&table_name, &schema, &[filtered_batch])
                    .await?;
            }
            self.refresh_table_cloud(&table_name).await?;
            Ok(deleted_count)
        } else {
            Ok(0)
        }
    }

    // -----------------------------------------------------------------------
    // Unified DML dispatch
    // -----------------------------------------------------------------------

    async fn execute_insert(&self, sql: &str) -> Result<u64, DfOlapError> {
        match &self.storage_mode {
            StorageMode::InMemory => self.execute_insert_mem(sql).await,
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => {
                self.execute_insert_cloud(sql).await
            }
            _ => self.execute_insert_file(sql).await,
        }
    }

    async fn execute_update(&self, sql: &str) -> Result<u64, DfOlapError> {
        match &self.storage_mode {
            StorageMode::InMemory => self.execute_update_mem(sql).await,
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => {
                self.execute_update_cloud(sql).await
            }
            _ => self.execute_update_file(sql).await,
        }
    }

    async fn execute_delete(&self, sql: &str) -> Result<u64, DfOlapError> {
        match &self.storage_mode {
            StorageMode::InMemory => self.execute_delete_mem(sql).await,
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => {
                self.execute_delete_cloud(sql).await
            }
            _ => self.execute_delete_file(sql).await,
        }
    }
}

impl Default for DataFusionEngine {
    fn default() -> Self {
        Self::new()
    }
}

impl rhei_core::OlapEngine for DataFusionEngine {
    type Error = DfOlapError;

    async fn query(&self, sql: &str) -> Result<Vec<RecordBatch>, Self::Error> {
        debug!(sql, "DataFusion query");
        self.execute_sql(sql).await
    }

    async fn query_stream(
        &self,
        sql: &str,
    ) -> Result<rhei_core::RecordBatchBoxStream, Self::Error> {
        debug!(sql, "DataFusion query_stream");
        let ctx = self.ctx.read().await;
        let df = ctx.sql(sql).await?;
        let stream = df.execute_stream().await?;
        let mapped = Box::pin(StreamAdapter(stream));
        Ok(mapped)
    }

    async fn execute(&self, sql: &str) -> Result<u64, Self::Error> {
        debug!(sql, "DataFusion execute");
        let trimmed = sql.trim();
        let upper = trimmed.to_ascii_uppercase();

        if upper.starts_with("INSERT") {
            self.execute_insert(trimmed).await
        } else if upper.starts_with("UPDATE") {
            self.execute_update(trimmed).await
        } else if upper.starts_with("DELETE") {
            self.execute_delete(trimmed).await
        } else if upper.starts_with("BEGIN")
            || upper.starts_with("COMMIT")
            || upper.starts_with("ROLLBACK")
        {
            // Transaction markers — no-op
            Ok(0)
        } else {
            // DDL or other — try via DataFusion SQL
            let ctx = self.ctx.read().await;
            let df = ctx.sql(trimmed).await?;
            let _ = df.collect().await?;
            Ok(0)
        }
    }

    async fn load_arrow(&self, table: &str, batches: &[RecordBatch]) -> Result<u64, Self::Error> {
        if batches.is_empty() {
            return Ok(0);
        }

        debug!(table, batch_count = batches.len(), "DataFusion load_arrow");
        rhei_core::validate_identifier(table).map_err(|e| DfOlapError::Other(e.to_string()))?;

        let total_rows: u64 = batches.iter().map(|b| b.num_rows() as u64).sum();

        match &self.storage_mode {
            StorageMode::InMemory => {
                let mut tables = self.tables.write().await;
                let table_data = tables
                    .get_mut(table)
                    .ok_or_else(|| DfOlapError::TableNotFound(table.to_string()))?;

                for batch in batches {
                    table_data.batches.push(batch.clone());
                }
                drop(tables);
                self.refresh_table_mem(table).await?;
            }
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => {
                let cloud_tables = self.cloud_tables.read().await;
                let meta = cloud_tables
                    .get(table)
                    .ok_or_else(|| DfOlapError::TableNotFound(table.to_string()))?;
                let schema = meta.schema.clone();
                drop(cloud_tables);

                self.write_batches_to_cloud(table, &schema, batches).await?;
                self.refresh_table_cloud(table).await?;
            }
            _ => {
                let file_tables = self.file_tables.read().await;
                let meta = file_tables
                    .get(table)
                    .ok_or_else(|| DfOlapError::TableNotFound(table.to_string()))?;
                let schema = meta.schema.clone();
                drop(file_tables);

                self.write_batches_to_file(table, &schema, batches).await?;
                self.refresh_table_file(table).await?;
            }
        }

        Ok(total_rows)
    }

    async fn create_table(
        &self,
        table_name: &str,
        schema: &SchemaRef,
        _primary_key: &[String],
    ) -> Result<(), Self::Error> {
        rhei_core::validate_identifier(table_name)
            .map_err(|e| DfOlapError::Other(e.to_string()))?;
        for field in schema.fields() {
            rhei_core::validate_identifier(field.name())
                .map_err(|e| DfOlapError::Other(e.to_string()))?;
        }
        // Note: DataFusion's in-memory/file-backed tables do not enforce PK constraints.
        // The `primary_key` parameter is accepted for API uniformity but is informational only.

        debug!(
            table = table_name,
            storage = ?self.storage_mode,
            "DataFusion create_table"
        );

        match &self.storage_mode {
            StorageMode::InMemory => {
                let mut tables = self.tables.write().await;
                if tables.contains_key(table_name) {
                    return Ok(());
                }
                tables.insert(
                    table_name.to_string(),
                    TableData {
                        schema: schema.clone(),
                        batches: vec![],
                    },
                );
                drop(tables);
                self.refresh_table_mem(table_name).await?;
            }
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { url } | StorageMode::GcsParquet { url } => {
                let mut cloud_tables = self.cloud_tables.write().await;
                if cloud_tables.contains_key(table_name) {
                    return Ok(());
                }
                let table_url = Self::cloud_table_url(url, table_name);
                cloud_tables.insert(
                    table_name.to_string(),
                    CloudTableMeta {
                        schema: schema.clone(),
                        table_url: table_url.clone(),
                    },
                );
                drop(cloud_tables);

                // On restart the prefix may already contain objects from a
                // previous run.  Scan them now and advance `file_counter` to
                // (max existing index + 1) so the next write doesn't clobber
                // any existing file.
                if let Ok(store) = self.cloud_store() {
                    match Self::cloud_seq_for_prefix(&store, &table_url, table_name).await {
                        Ok(next_seq) => {
                            // fetch_max only bumps the counter if next_seq is
                            // larger than the current value — safe for multiple
                            // concurrent create_table calls.
                            self.file_counter.fetch_max(next_seq, Ordering::Relaxed);
                            if next_seq > 0 {
                                tracing::debug!(
                                    table = table_name,
                                    next_seq,
                                    "cloud restart: advanced file_counter to avoid overwrites"
                                );
                            }
                        }
                        Err(e) => {
                            // A listing failure (e.g. prefix doesn't exist yet, or
                            // transient network error) is non-fatal: we log it and
                            // continue.  The worst-case is an overwrite on restart
                            // which is the pre-existing behaviour.
                            tracing::warn!(
                                table = table_name,
                                error = %e,
                                "cloud_seq_for_prefix failed; file_counter not advanced"
                            );
                        }
                    }
                }

                // Register an empty listing table (no objects yet — OK, DataFusion
                // will return empty results until the first write).
                self.refresh_table_cloud(table_name).await?;
            }
            _ => {
                let mut file_tables = self.file_tables.write().await;
                if file_tables.contains_key(table_name) {
                    return Ok(());
                }
                let dir = self.table_dir(table_name).expect("file mode has base_path");
                tokio::fs::create_dir_all(&dir).await?;
                file_tables.insert(
                    table_name.to_string(),
                    FileTableMeta {
                        schema: schema.clone(),
                        dir,
                    },
                );
                drop(file_tables);
                self.refresh_table_file(table_name).await?;
            }
        }

        Ok(())
    }

    async fn table_exists(&self, table_name: &str) -> Result<bool, Self::Error> {
        match &self.storage_mode {
            StorageMode::InMemory => {
                let tables = self.tables.read().await;
                Ok(tables.contains_key(table_name))
            }
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => {
                let cloud_tables = self.cloud_tables.read().await;
                Ok(cloud_tables.contains_key(table_name))
            }
            _ => {
                let file_tables = self.file_tables.read().await;
                Ok(file_tables.contains_key(table_name))
            }
        }
    }

    async fn add_column(
        &self,
        table_name: &str,
        column_name: &str,
        data_type: &DataType,
    ) -> Result<(), Self::Error> {
        rhei_core::validate_identifier(table_name)
            .map_err(|e| DfOlapError::Other(e.to_string()))?;
        rhei_core::validate_identifier(column_name)
            .map_err(|e| DfOlapError::Other(e.to_string()))?;

        debug!(
            table = table_name,
            column = column_name,
            "DataFusion add_column"
        );

        match &self.storage_mode {
            StorageMode::InMemory => {
                let mut tables = self.tables.write().await;
                let table_data = tables
                    .get_mut(table_name)
                    .ok_or_else(|| DfOlapError::TableNotFound(table_name.to_string()))?;

                let new_schema = append_field(&table_data.schema, column_name, data_type);
                let new_batches =
                    extend_batches_with_null_column(&table_data.batches, &new_schema, data_type)?;
                table_data.schema = new_schema;
                table_data.batches = new_batches;
                drop(tables);
                self.refresh_table_mem(table_name).await?;
            }
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => {
                let (old_schema, existing_batches) =
                    self.read_all_batches_cloud(table_name).await?;
                let new_schema = append_field(&old_schema, column_name, data_type);
                let new_batches =
                    extend_batches_with_null_column(&existing_batches, &new_schema, data_type)?;

                self.clear_cloud_table(table_name).await?;
                if !new_batches.is_empty() {
                    self.write_batches_to_cloud(table_name, &new_schema, &new_batches)
                        .await?;
                }

                let mut cloud_tables = self.cloud_tables.write().await;
                if let Some(meta) = cloud_tables.get_mut(table_name) {
                    meta.schema = new_schema;
                }
                drop(cloud_tables);
                self.refresh_table_cloud(table_name).await?;
            }
            _ => {
                // Read existing data, add column, rewrite
                let (old_schema, existing_batches) = self.read_all_batches(table_name).await?;
                let new_schema = append_field(&old_schema, column_name, data_type);
                let new_batches =
                    extend_batches_with_null_column(&existing_batches, &new_schema, data_type)?;

                self.clear_table_dir(table_name).await?;
                if !new_batches.is_empty() {
                    self.write_batches_to_file(table_name, &new_schema, &new_batches)
                        .await?;
                }

                let mut file_tables = self.file_tables.write().await;
                if let Some(meta) = file_tables.get_mut(table_name) {
                    meta.schema = new_schema;
                }
                drop(file_tables);
                self.refresh_table_file(table_name).await?;
            }
        }

        Ok(())
    }

    async fn drop_column(&self, table_name: &str, column_name: &str) -> Result<(), Self::Error> {
        rhei_core::validate_identifier(table_name)
            .map_err(|e| DfOlapError::Other(e.to_string()))?;
        rhei_core::validate_identifier(column_name)
            .map_err(|e| DfOlapError::Other(e.to_string()))?;

        debug!(
            table = table_name,
            column = column_name,
            "DataFusion drop_column"
        );

        match &self.storage_mode {
            StorageMode::InMemory => {
                let mut tables = self.tables.write().await;
                let table_data = tables
                    .get_mut(table_name)
                    .ok_or_else(|| DfOlapError::TableNotFound(table_name.to_string()))?;

                let col_idx = find_column_index(&table_data.schema, column_name, table_name)?;
                let new_schema = remove_field(&table_data.schema, col_idx);
                let new_batches =
                    remove_column_from_batches(&table_data.batches, &new_schema, col_idx)?;
                table_data.schema = new_schema;
                table_data.batches = new_batches;
                drop(tables);
                self.refresh_table_mem(table_name).await?;
            }
            #[cfg(feature = "cloud-storage")]
            StorageMode::S3Parquet { .. } | StorageMode::GcsParquet { .. } => {
                let (old_schema, existing_batches) =
                    self.read_all_batches_cloud(table_name).await?;
                let col_idx = find_column_index(&old_schema, column_name, table_name)?;
                let new_schema = remove_field(&old_schema, col_idx);
                let new_batches =
                    remove_column_from_batches(&existing_batches, &new_schema, col_idx)?;

                self.clear_cloud_table(table_name).await?;
                if !new_batches.is_empty() {
                    self.write_batches_to_cloud(table_name, &new_schema, &new_batches)
                        .await?;
                }

                let mut cloud_tables = self.cloud_tables.write().await;
                if let Some(meta) = cloud_tables.get_mut(table_name) {
                    meta.schema = new_schema;
                }
                drop(cloud_tables);
                self.refresh_table_cloud(table_name).await?;
            }
            _ => {
                let (old_schema, existing_batches) = self.read_all_batches(table_name).await?;
                let col_idx = find_column_index(&old_schema, column_name, table_name)?;
                let new_schema = remove_field(&old_schema, col_idx);
                let new_batches =
                    remove_column_from_batches(&existing_batches, &new_schema, col_idx)?;

                self.clear_table_dir(table_name).await?;
                if !new_batches.is_empty() {
                    self.write_batches_to_file(table_name, &new_schema, &new_batches)
                        .await?;
                }

                let mut file_tables = self.file_tables.write().await;
                if let Some(meta) = file_tables.get_mut(table_name) {
                    meta.schema = new_schema;
                }
                drop(file_tables);
                self.refresh_table_file(table_name).await?;
            }
        }

        Ok(())
    }
}

// ---------------------------------------------------------------------------
// Schema / batch helpers (shared between in-memory and file modes)
// ---------------------------------------------------------------------------

fn append_field(schema: &SchemaRef, column_name: &str, data_type: &DataType) -> SchemaRef {
    let mut fields: Vec<arrow::datatypes::Field> =
        schema.fields().iter().map(|f| f.as_ref().clone()).collect();
    fields.push(arrow::datatypes::Field::new(
        column_name,
        data_type.clone(),
        true,
    ));
    Arc::new(arrow::datatypes::Schema::new(fields))
}

fn remove_field(schema: &SchemaRef, col_idx: usize) -> SchemaRef {
    let fields: Vec<arrow::datatypes::Field> = schema
        .fields()
        .iter()
        .enumerate()
        .filter(|(i, _)| *i != col_idx)
        .map(|(_, f)| f.as_ref().clone())
        .collect();
    Arc::new(arrow::datatypes::Schema::new(fields))
}

fn find_column_index(
    schema: &SchemaRef,
    column_name: &str,
    table_name: &str,
) -> Result<usize, DfOlapError> {
    schema
        .fields()
        .iter()
        .position(|f| f.name() == column_name)
        .ok_or_else(|| {
            DfOlapError::Other(format!(
                "column '{}' not found in table '{}'",
                column_name, table_name
            ))
        })
}

fn extend_batches_with_null_column(
    batches: &[RecordBatch],
    new_schema: &SchemaRef,
    data_type: &DataType,
) -> Result<Vec<RecordBatch>, DfOlapError> {
    let mut new_batches = Vec::with_capacity(batches.len());
    for batch in batches {
        let null_array = arrow::array::new_null_array(data_type, batch.num_rows());
        let mut columns: Vec<Arc<dyn Array>> = (0..batch.num_columns())
            .map(|i| batch.column(i).clone())
            .collect();
        columns.push(null_array);
        new_batches.push(RecordBatch::try_new(new_schema.clone(), columns)?);
    }
    Ok(new_batches)
}

fn remove_column_from_batches(
    batches: &[RecordBatch],
    new_schema: &SchemaRef,
    col_idx: usize,
) -> Result<Vec<RecordBatch>, DfOlapError> {
    let mut new_batches = Vec::with_capacity(batches.len());
    for batch in batches {
        let columns: Vec<Arc<dyn Array>> = (0..batch.num_columns())
            .filter(|i| *i != col_idx)
            .map(|i| batch.column(i).clone())
            .collect();
        new_batches.push(RecordBatch::try_new(new_schema.clone(), columns)?);
    }
    Ok(new_batches)
}

/// Adapter: maps DataFusion's `SendableRecordBatchStream` to `RecordBatchBoxStream`.
struct StreamAdapter(datafusion::physical_plan::SendableRecordBatchStream);

impl futures_core::Stream for StreamAdapter {
    type Item = Result<RecordBatch, Box<dyn std::error::Error + Send + Sync>>;

    fn poll_next(
        mut self: std::pin::Pin<&mut Self>,
        cx: &mut std::task::Context<'_>,
    ) -> std::task::Poll<Option<Self::Item>> {
        std::pin::Pin::new(&mut self.0).poll_next(cx).map(|opt| {
            opt.map(|r| r.map_err(|e| Box::new(e) as Box<dyn std::error::Error + Send + Sync>))
        })
    }
}

/// A cheaply-cloneable, `Arc`-wrapped [`DataFusionEngine`] that implements
/// [`rhei_core::OlapEngine`].
///
/// All method calls delegate to the inner engine.  Cloning a
/// `SharedDataFusionEngine` only increments the reference count — the
/// underlying engine (and its table store) is shared.
///
/// Use [`SharedDataFusionEngine::new`] to construct from a
/// [`DataFusionEngine`], or access the inner engine through the public `Deref`
/// impl or the `0` field.
#[derive(Clone)]
pub struct SharedDataFusionEngine(pub Arc<DataFusionEngine>);

impl SharedDataFusionEngine {
    /// Wrap a [`DataFusionEngine`] in an `Arc` so it can be shared across tasks.
    pub fn new(engine: DataFusionEngine) -> Self {
        Self(Arc::new(engine))
    }
}

impl std::ops::Deref for SharedDataFusionEngine {
    type Target = DataFusionEngine;
    fn deref(&self) -> &Self::Target {
        &self.0
    }
}

impl rhei_core::OlapEngine for SharedDataFusionEngine {
    type Error = DfOlapError;

    async fn query(&self, sql: &str) -> Result<Vec<RecordBatch>, Self::Error> {
        self.0.query(sql).await
    }

    async fn query_stream(
        &self,
        sql: &str,
    ) -> Result<rhei_core::RecordBatchBoxStream, Self::Error> {
        self.0.query_stream(sql).await
    }

    async fn execute(&self, sql: &str) -> Result<u64, Self::Error> {
        self.0.execute(sql).await
    }

    async fn load_arrow(&self, table: &str, batches: &[RecordBatch]) -> Result<u64, Self::Error> {
        self.0.load_arrow(table, batches).await
    }

    async fn create_table(
        &self,
        table_name: &str,
        schema: &SchemaRef,
        primary_key: &[String],
    ) -> Result<(), Self::Error> {
        self.0.create_table(table_name, schema, primary_key).await
    }

    async fn table_exists(&self, table_name: &str) -> Result<bool, Self::Error> {
        self.0.table_exists(table_name).await
    }

    async fn add_column(
        &self,
        table_name: &str,
        column_name: &str,
        data_type: &DataType,
    ) -> Result<(), Self::Error> {
        self.0.add_column(table_name, column_name, data_type).await
    }

    async fn drop_column(&self, table_name: &str, column_name: &str) -> Result<(), Self::Error> {
        self.0.drop_column(table_name, column_name).await
    }
}

// ---------------------------------------------------------------------------
// SQL parsing helpers — sqlparser-rs AST based
// ---------------------------------------------------------------------------

/// Convert a sqlparser `Expr` from a VALUES list to a SQL literal string
/// compatible with `build_array` / `build_record_batch_from_values`.
///
/// Supported: numbers, strings, booleans, NULL, unary minus on numbers.
/// Returns an error for expressions we can't convert (subqueries, functions,
/// etc. — the sync engine never produces those in VALUES clauses).
fn expr_to_sql_literal(expr: &Expr) -> Result<String, DfOlapError> {
    match expr {
        Expr::Value(v) => match &v.value {
            Value::Number(n, _) => Ok(n.clone()),
            Value::SingleQuotedString(s) => Ok(format!("'{}'", s.replace('\'', "''"))),
            Value::Boolean(b) => Ok(if *b { "TRUE".into() } else { "FALSE".into() }),
            Value::Null => Ok("NULL".into()),
            other => Err(DfOlapError::Other(format!(
                "unsupported value literal: {other:?}"
            ))),
        },
        Expr::UnaryOp {
            op: UnaryOperator::Minus,
            expr: inner,
        } => {
            // Handle negative numbers: -1, -3.14
            if let Expr::Value(v) = inner.as_ref() {
                if let Value::Number(n, _) = &v.value {
                    return Ok(format!("-{n}"));
                }
            }
            Err(DfOlapError::Other(format!(
                "unsupported unary expression: {expr}"
            )))
        }
        other => Err(DfOlapError::Other(format!(
            "unsupported expression in VALUES: {other}"
        ))),
    }
}

/// Extract the unquoted column name from an identifier expression.
fn ident_from_expr(expr: &Expr) -> Result<String, DfOlapError> {
    match expr {
        Expr::Identifier(ident) => Ok(ident.value.clone()),
        Expr::CompoundIdentifier(parts) => parts
            .last()
            .map(|i| i.value.clone())
            .ok_or_else(|| DfOlapError::Other("empty compound identifier".into())),
        other => Err(DfOlapError::Other(format!(
            "expected column name, got: {other}"
        ))),
    }
}

/// Extract `col = val` (and `col IS NULL`) pairs from a WHERE expression tree.
///
/// Supports:
/// - Simple equality: `col = val`
/// - IS NULL: `col IS NULL`  (stored as `(col, "NULL")`)
/// - AND chains combining the above
///
/// Returns an error for anything more complex (OR, inequalities, subqueries).
/// The sync engine produces simple AND-equality / IS NULL WHERE clauses.
fn extract_where_conditions(expr: &Expr) -> Result<Vec<(String, String)>, DfOlapError> {
    match expr {
        Expr::BinaryOp {
            left,
            op: BinaryOperator::And,
            right,
        } => {
            let mut conditions = extract_where_conditions(left)?;
            conditions.extend(extract_where_conditions(right)?);
            Ok(conditions)
        }
        Expr::BinaryOp {
            left,
            op: BinaryOperator::Eq,
            right,
        } => {
            let col = ident_from_expr(left)?;
            let val = expr_to_sql_literal(right)?;
            Ok(vec![(col, val)])
        }
        // col IS NULL — stored as (col, "NULL") so value_matches handles it
        Expr::IsNull(inner) => {
            let col = ident_from_expr(inner)?;
            Ok(vec![(col, "NULL".into())])
        }
        // col IS NOT NULL — stored as a sentinel that value_matches won't treat as NULL
        // (represented as the special token "IS NOT NULL" for value_matches to check)
        Expr::IsNotNull(inner) => {
            let col = ident_from_expr(inner)?;
            Ok(vec![(col, "__IS_NOT_NULL__".into())])
        }
        // Parenthesized expressions (e.g. `(a = 1)`)
        Expr::Nested(inner) => extract_where_conditions(inner),
        other => Err(DfOlapError::Other(format!(
            "unsupported WHERE expression: {other}"
        ))),
    }
}

/// Parse `INSERT INTO <table> (<cols>) VALUES (<vals>), ...`
///
/// Returns `(table_name, column_names, Vec<RecordBatch>)`.
fn parse_insert_values(sql: &str) -> Result<(String, Vec<String>, Vec<RecordBatch>), DfOlapError> {
    let mut stmts = Parser::parse_sql(&SQLiteDialect {}, sql)
        .map_err(|e| DfOlapError::Other(format!("failed to parse INSERT: {e}")))?;

    let stmt = stmts
        .pop()
        .ok_or_else(|| DfOlapError::Other("empty SQL statement".into()))?;

    let insert = match stmt {
        Statement::Insert(ins) => ins,
        other => {
            return Err(DfOlapError::Other(format!(
                "expected INSERT statement, got: {other:?}"
            )));
        }
    };

    // Extract table name (unquoted value from ObjectName)
    let table_name = match &insert.table {
        TableObject::TableName(obj_name) => obj_name
            .0
            .last()
            .and_then(|p| p.as_ident())
            .map(|id| id.value.clone())
            .ok_or_else(|| DfOlapError::Other("empty table name in INSERT".into()))?,
        TableObject::TableFunction(_) => {
            return Err(DfOlapError::Other(
                "INSERT INTO TABLE FUNCTION not supported".into(),
            ));
        }
    };

    rhei_core::validate_identifier(&table_name).map_err(|e| DfOlapError::Other(e.to_string()))?;

    // Column names (unquoted)
    let col_name_strings: Vec<String> = insert.columns.iter().map(|id| id.value.clone()).collect();

    // Extract rows from VALUES clause
    let source = match insert.source {
        Some(q) => q,
        None => return Ok((table_name, col_name_strings, vec![])),
    };

    let values = match *source.body {
        SetExpr::Values(v) => v,
        other => {
            return Err(DfOlapError::Other(format!(
                "INSERT source is not a VALUES clause: {other:?}"
            )));
        }
    };

    if values.rows.is_empty() {
        return Ok((table_name, col_name_strings, vec![]));
    }

    // Convert each Expr to a SQL literal string (for build_record_batch_from_values)
    let rows: Vec<Vec<String>> = values
        .rows
        .iter()
        .map(|row| {
            row.iter()
                .map(expr_to_sql_literal)
                .collect::<Result<_, _>>()
        })
        .collect::<Result<_, _>>()?;

    let col_name_refs: Vec<&str> = col_name_strings.iter().map(|s| s.as_str()).collect();
    let num_cols = col_name_refs.len();

    if num_cols == 0 {
        // `INSERT INTO t VALUES (...)` without an explicit column list would
        // silently produce zero-row RecordBatches, causing a write to appear
        // successful while dropping all data. Reject it explicitly; the
        // CDC sync converter always emits explicit column lists.
        return Err(DfOlapError::Other(format!(
            "INSERT INTO {table_name} requires an explicit column list; `VALUES (...)` without columns is not supported"
        )));
    }

    let batch = build_record_batch_from_values(&col_name_refs, &rows, num_cols)?;
    Ok((table_name, col_name_strings, vec![batch]))
}

/// Build an Arrow RecordBatch from parsed SQL values.
///
/// Values are SQL literals: numbers, 'strings', NULL, TRUE/FALSE.
/// Column types are inferred from the first non-NULL value in each column.
fn build_record_batch_from_values(
    col_names: &[&str],
    rows: &[Vec<String>],
    num_cols: usize,
) -> Result<RecordBatch, DfOlapError> {
    use arrow::array::*;
    use arrow::datatypes::{Field, Schema};

    // Infer types from values
    let mut types = vec![DataType::Utf8; num_cols]; // default
    for col_idx in 0..num_cols {
        for row in rows {
            if col_idx < row.len() {
                let val = &row[col_idx];
                let upper = val.to_ascii_uppercase();
                if upper == "NULL" {
                    continue;
                }
                if upper == "TRUE" || upper == "FALSE" {
                    types[col_idx] = DataType::Boolean;
                    break;
                }
                if val.starts_with('\'') {
                    types[col_idx] = DataType::Utf8;
                    break;
                }
                if val.contains('.') {
                    if val.parse::<f64>().is_ok() {
                        types[col_idx] = DataType::Float64;
                        break;
                    }
                } else if val.parse::<i64>().is_ok() {
                    types[col_idx] = DataType::Int64;
                    break;
                }
                // Default: Utf8
                break;
            }
        }
    }

    let fields: Vec<Field> = col_names
        .iter()
        .zip(types.iter())
        .map(|(name, dt)| Field::new(*name, dt.clone(), true))
        .collect();
    let schema = Arc::new(Schema::new(fields));

    // Build column arrays
    let mut columns: Vec<Arc<dyn Array>> = Vec::with_capacity(num_cols);
    for col_idx in 0..num_cols {
        let col_values: Vec<&str> = rows
            .iter()
            .map(|row| {
                if col_idx < row.len() {
                    row[col_idx].as_str()
                } else {
                    "NULL"
                }
            })
            .collect();

        columns.push(build_array(&types[col_idx], &col_values)?);
    }

    let batch = RecordBatch::try_new(schema, columns)?;
    Ok(batch)
}

/// Build an Arrow array from SQL literal strings.
fn build_array(dt: &DataType, values: &[&str]) -> Result<Arc<dyn Array>, DfOlapError> {
    use arrow::array::*;

    match dt {
        DataType::Int64 => {
            let mut builder = Int64Builder::new();
            for v in values {
                if v.eq_ignore_ascii_case("NULL") {
                    builder.append_null();
                } else {
                    builder.append_value(
                        v.parse::<i64>()
                            .map_err(|e| DfOlapError::Other(format!("parse i64: {e}")))?,
                    );
                }
            }
            Ok(Arc::new(builder.finish()))
        }
        DataType::Float64 => {
            let mut builder = Float64Builder::new();
            for v in values {
                if v.eq_ignore_ascii_case("NULL") {
                    builder.append_null();
                } else {
                    builder.append_value(
                        v.parse::<f64>()
                            .map_err(|e| DfOlapError::Other(format!("parse f64: {e}")))?,
                    );
                }
            }
            Ok(Arc::new(builder.finish()))
        }
        DataType::Boolean => {
            let mut builder = BooleanBuilder::new();
            for v in values {
                let upper = v.to_ascii_uppercase();
                if upper == "NULL" {
                    builder.append_null();
                } else {
                    builder.append_value(upper == "TRUE");
                }
            }
            Ok(Arc::new(builder.finish()))
        }
        _ => {
            let mut builder = StringBuilder::new();
            for v in values {
                if v.eq_ignore_ascii_case("NULL") {
                    builder.append_null();
                } else {
                    // Strip surrounding quotes
                    let stripped = if v.starts_with('\'') && v.ends_with('\'') && v.len() >= 2 {
                        &v[1..v.len() - 1]
                    } else {
                        v
                    };
                    // Unescape ''
                    builder.append_value(stripped.replace("''", "'"));
                }
            }
            Ok(Arc::new(builder.finish()))
        }
    }
}

/// Column assignment or condition: (column_name, value_literal).
type ColVal = (String, String);

/// Parse `UPDATE <table> SET col=val, ... WHERE col=val AND ...`
fn parse_update(sql: &str) -> Result<(String, Vec<ColVal>, Vec<ColVal>), DfOlapError> {
    let mut stmts = Parser::parse_sql(&SQLiteDialect {}, sql)
        .map_err(|e| DfOlapError::Other(format!("failed to parse UPDATE: {e}")))?;

    let stmt = stmts
        .pop()
        .ok_or_else(|| DfOlapError::Other("empty SQL statement".into()))?;

    let update = match stmt {
        Statement::Update(upd) => upd,
        other => {
            return Err(DfOlapError::Other(format!(
                "expected UPDATE statement, got: {other:?}"
            )));
        }
    };

    // Extract table name from the first relation in the FROM clause
    let table_name = match &update.table.relation {
        TableFactor::Table { name, .. } => name
            .0
            .last()
            .and_then(|p| p.as_ident())
            .map(|id| id.value.clone())
            .ok_or_else(|| DfOlapError::Other("empty table name in UPDATE".into()))?,
        other => {
            return Err(DfOlapError::Other(format!(
                "unexpected table factor in UPDATE: {other:?}"
            )));
        }
    };

    // Convert AST assignments → (col, literal_string) pairs
    let assignments: Vec<ColVal> = update
        .assignments
        .iter()
        .map(|a| {
            let col = match &a.target {
                AssignmentTarget::ColumnName(obj) => obj
                    .0
                    .last()
                    .and_then(|p| p.as_ident())
                    .map(|id| id.value.clone())
                    .ok_or_else(|| DfOlapError::Other("empty column name in SET".into()))?,
                AssignmentTarget::Tuple(_) => {
                    return Err(DfOlapError::Other(
                        "tuple assignments in SET not supported".into(),
                    ));
                }
            };
            let val = expr_to_sql_literal(&a.value)?;
            Ok((col, val))
        })
        .collect::<Result<_, DfOlapError>>()?;

    // Extract WHERE conditions
    let where_clause = match &update.selection {
        Some(expr) => extract_where_conditions(expr)?,
        None => vec![],
    };

    Ok((table_name, assignments, where_clause))
}

/// Parse `DELETE FROM <table> WHERE col=val AND ...`
fn parse_delete(sql: &str) -> Result<(String, Vec<(String, String)>), DfOlapError> {
    let mut stmts = Parser::parse_sql(&SQLiteDialect {}, sql)
        .map_err(|e| DfOlapError::Other(format!("failed to parse DELETE: {e}")))?;

    let stmt = stmts
        .pop()
        .ok_or_else(|| DfOlapError::Other("empty SQL statement".into()))?;

    let delete = match stmt {
        Statement::Delete(del) => del,
        other => {
            return Err(DfOlapError::Other(format!(
                "expected DELETE statement, got: {other:?}"
            )));
        }
    };

    // Extract table name from FROM clause
    let tables = match &delete.from {
        FromTable::WithFromKeyword(tables) | FromTable::WithoutKeyword(tables) => tables,
    };

    let table_name = tables
        .first()
        .and_then(|twj| {
            if let TableFactor::Table { name, .. } = &twj.relation {
                name.0
                    .last()
                    .and_then(|p| p.as_ident())
                    .map(|id| id.value.clone())
            } else {
                None
            }
        })
        .ok_or_else(|| DfOlapError::Other("missing table name in DELETE".into()))?;

    // Extract WHERE conditions
    let where_clause = match &delete.selection {
        Some(expr) => extract_where_conditions(expr)?,
        None => vec![],
    };

    Ok((table_name, where_clause))
}

/// Flatten multiple RecordBatches into a single one.
fn flatten_batches(
    batches: &[RecordBatch],
    schema: &SchemaRef,
) -> Result<Option<RecordBatch>, DfOlapError> {
    if batches.is_empty() {
        return Ok(None);
    }

    let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum();
    if total_rows == 0 {
        return Ok(None);
    }

    let batch = arrow::compute::concat_batches(schema, batches)?;
    Ok(Some(batch))
}

/// Apply UPDATE assignments to matching rows, return (new_batch, updated_count).
fn apply_update(
    batch: &RecordBatch,
    schema: &SchemaRef,
    assignments: &[(String, String)],
    where_conditions: &[(String, String)],
) -> Result<(RecordBatch, u64), DfOlapError> {
    let matching = find_matching_rows(batch, schema, where_conditions)?;
    let updated_count = matching.iter().filter(|&&m| m).count() as u64;

    // For each column, either keep original or apply the assignment
    let mut new_columns: Vec<Arc<dyn Array>> = Vec::with_capacity(schema.fields().len());
    for (col_idx, field) in schema.fields().iter().enumerate() {
        // Check if this column has an assignment
        let assignment = assignments.iter().find(|(col, _)| col == field.name());

        if let Some((_, new_val)) = assignment {
            // Build new array: original values for non-matching, new value for matching
            let original = batch.column(col_idx);
            new_columns.push(apply_value_to_matching(
                original,
                &matching,
                new_val,
                field.data_type(),
            )?);
        } else {
            new_columns.push(batch.column(col_idx).clone());
        }
    }

    let new_batch = RecordBatch::try_new(schema.clone(), new_columns)?;
    Ok((new_batch, updated_count))
}

/// Apply DELETE, returning (filtered_batch, deleted_count).
fn apply_delete(
    batch: &RecordBatch,
    schema: &SchemaRef,
    where_conditions: &[(String, String)],
) -> Result<(RecordBatch, u64), DfOlapError> {
    let matching = find_matching_rows(batch, schema, where_conditions)?;
    let deleted_count = matching.iter().filter(|&&m| m).count() as u64;

    // Build a boolean filter (keep = !matching)
    let mut builder = BooleanBuilder::new();
    for &m in &matching {
        builder.append_value(!m);
    }
    let filter_array = builder.finish();

    let new_columns: Vec<Arc<dyn Array>> = (0..batch.num_columns())
        .map(|i| arrow::compute::filter(batch.column(i), &filter_array).map_err(DfOlapError::Arrow))
        .collect::<Result<_, _>>()?;

    let new_batch = RecordBatch::try_new(schema.clone(), new_columns)?;
    Ok((new_batch, deleted_count))
}

/// Find which rows match the WHERE conditions.
fn find_matching_rows(
    batch: &RecordBatch,
    schema: &SchemaRef,
    conditions: &[(String, String)],
) -> Result<Vec<bool>, DfOlapError> {
    let num_rows = batch.num_rows();
    let mut matching = vec![true; num_rows];

    for (col_name, expected_val) in conditions {
        let col_idx = schema
            .fields()
            .iter()
            .position(|f| f.name() == col_name)
            .ok_or_else(|| DfOlapError::Other(format!("column not found: {col_name}")))?;

        let col = batch.column(col_idx);
        for (row_idx, m) in matching.iter_mut().enumerate() {
            if !*m {
                continue;
            }
            *m = value_matches(col, row_idx, expected_val);
        }
    }

    Ok(matching)
}

/// Check if an Arrow value at a given row matches a SQL literal.
///
/// Special sentinels:
/// - `"NULL"` (case-insensitive): matches a null Arrow cell (`IS NULL` semantics)
/// - `"__IS_NOT_NULL__"`: matches any non-null Arrow cell (`IS NOT NULL` semantics)
fn value_matches(array: &dyn Array, row_idx: usize, expected: &str) -> bool {
    // IS NOT NULL sentinel — matches only non-null values
    if expected == "__IS_NOT_NULL__" {
        return !array.is_null(row_idx);
    }
    if array.is_null(row_idx) {
        return expected.eq_ignore_ascii_case("NULL");
    }

    match array.data_type() {
        DataType::Int8 => {
            expected.parse::<i8>().ok() == Some(array.as_primitive::<Int8Type>().value(row_idx))
        }
        DataType::Int16 => {
            expected.parse::<i16>().ok() == Some(array.as_primitive::<Int16Type>().value(row_idx))
        }
        DataType::Int32 => {
            expected.parse::<i32>().ok() == Some(array.as_primitive::<Int32Type>().value(row_idx))
        }
        DataType::Int64 => {
            expected.parse::<i64>().ok() == Some(array.as_primitive::<Int64Type>().value(row_idx))
        }
        DataType::UInt8 => {
            expected.parse::<u8>().ok() == Some(array.as_primitive::<UInt8Type>().value(row_idx))
        }
        DataType::UInt16 => {
            expected.parse::<u16>().ok() == Some(array.as_primitive::<UInt16Type>().value(row_idx))
        }
        DataType::UInt32 => {
            expected.parse::<u32>().ok() == Some(array.as_primitive::<UInt32Type>().value(row_idx))
        }
        DataType::UInt64 => {
            expected.parse::<u64>().ok() == Some(array.as_primitive::<UInt64Type>().value(row_idx))
        }
        DataType::Float32 => {
            expected.parse::<f32>().ok() == Some(array.as_primitive::<Float32Type>().value(row_idx))
        }
        DataType::Float64 => {
            expected.parse::<f64>().ok() == Some(array.as_primitive::<Float64Type>().value(row_idx))
        }
        DataType::Utf8 => {
            let arr = array.as_string::<i32>();
            let stripped =
                if expected.starts_with('\'') && expected.ends_with('\'') && expected.len() >= 2 {
                    &expected[1..expected.len() - 1]
                } else {
                    expected
                };
            arr.value(row_idx) == stripped
        }
        DataType::Boolean => {
            let arr = array.as_boolean();
            match expected.to_ascii_uppercase().as_str() {
                "TRUE" => arr.value(row_idx),
                "FALSE" => !arr.value(row_idx),
                _ => false,
            }
        }
        _ => false,
    }
}

/// Replace values in an array at matching positions with a new SQL literal value.
fn apply_value_to_matching(
    original: &dyn Array,
    matching: &[bool],
    new_val: &str,
    dt: &DataType,
) -> Result<Arc<dyn Array>, DfOlapError> {
    use arrow::array::*;

    match dt {
        DataType::Int64 => {
            let orig = original.as_primitive::<Int64Type>();
            let parsed: i64 = new_val
                .parse()
                .map_err(|e| DfOlapError::Other(format!("parse i64: {e}")))?;
            let mut builder = Int64Builder::new();
            for (i, &m) in matching.iter().enumerate() {
                if m {
                    builder.append_value(parsed);
                } else if orig.is_null(i) {
                    builder.append_null();
                } else {
                    builder.append_value(orig.value(i));
                }
            }
            Ok(Arc::new(builder.finish()))
        }
        DataType::Float64 => {
            let orig = original.as_primitive::<Float64Type>();
            let parsed: f64 = new_val
                .parse()
                .map_err(|e| DfOlapError::Other(format!("parse f64: {e}")))?;
            let mut builder = Float64Builder::new();
            for (i, &m) in matching.iter().enumerate() {
                if m {
                    builder.append_value(parsed);
                } else if orig.is_null(i) {
                    builder.append_null();
                } else {
                    builder.append_value(orig.value(i));
                }
            }
            Ok(Arc::new(builder.finish()))
        }
        DataType::Utf8 => {
            let orig = original.as_string::<i32>();
            let stripped =
                if new_val.starts_with('\'') && new_val.ends_with('\'') && new_val.len() >= 2 {
                    &new_val[1..new_val.len() - 1]
                } else {
                    new_val
                };
            let unescaped = stripped.replace("''", "'");
            let mut builder = StringBuilder::new();
            for (i, &m) in matching.iter().enumerate() {
                if m {
                    builder.append_value(&unescaped);
                } else if orig.is_null(i) {
                    builder.append_null();
                } else {
                    builder.append_value(orig.value(i));
                }
            }
            Ok(Arc::new(builder.finish()))
        }
        DataType::Boolean => {
            let orig = original.as_boolean();
            let parsed = new_val.eq_ignore_ascii_case("TRUE");
            let mut builder = BooleanBuilder::new();
            for (i, &m) in matching.iter().enumerate() {
                if m {
                    builder.append_value(parsed);
                } else if orig.is_null(i) {
                    builder.append_null();
                } else {
                    builder.append_value(orig.value(i));
                }
            }
            Ok(Arc::new(builder.finish()))
        }
        _ => {
            // Fallback: treat as string
            let orig = original.as_string::<i32>();
            let mut builder = StringBuilder::new();
            for (i, &m) in matching.iter().enumerate() {
                if m {
                    builder.append_value(new_val);
                } else if orig.is_null(i) {
                    builder.append_null();
                } else {
                    builder.append_value(orig.value(i));
                }
            }
            Ok(Arc::new(builder.finish()))
        }
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use arrow::datatypes::{Field, Schema};
    use rhei_core::OlapEngine;

    fn users_schema() -> SchemaRef {
        Arc::new(Schema::new(vec![
            Field::new("id", DataType::Int64, false),
            Field::new("name", DataType::Utf8, true),
            Field::new("age", DataType::Int64, true),
        ]))
    }

    fn make_in_memory(_: &std::path::Path) -> DataFusionEngine {
        DataFusionEngine::new()
    }

    fn make_arrow_ipc(tmp: &std::path::Path) -> DataFusionEngine {
        DataFusionEngine::with_storage(StorageMode::ArrowIpc {
            path: tmp.join("arrow_olap"),
        })
        .unwrap()
    }

    fn make_parquet(tmp: &std::path::Path) -> DataFusionEngine {
        DataFusionEngine::with_storage(StorageMode::Parquet {
            path: tmp.join("parquet_olap"),
        })
        .unwrap()
    }

    /// Generate a full test suite for a given storage mode.
    macro_rules! storage_mode_tests {
        ($mod_name:ident, $make_engine:ident) => {
            mod $mod_name {
                use super::*;

                #[tokio::test]
                async fn create_and_query_empty() {
                    let _tmp = tempfile::tempdir().unwrap();
                    let engine = $make_engine(_tmp.path());
                    let schema = users_schema();
                    engine.create_table("users", &schema, &[]).await.unwrap();

                    assert!(engine.table_exists("users").await.unwrap());
                    assert!(!engine.table_exists("nonexistent").await.unwrap());
                }

                #[tokio::test]
                async fn insert_and_query() {
                    let _tmp = tempfile::tempdir().unwrap();
                    let engine = $make_engine(_tmp.path());
                    let schema = users_schema();
                    engine.create_table("users", &schema, &[]).await.unwrap();

                    engine
                        .execute("INSERT INTO users (id, name, age) VALUES (1, 'Alice', 30)")
                        .await
                        .unwrap();
                    engine
                        .execute("INSERT INTO users (id, name, age) VALUES (2, 'Bob', 25)")
                        .await
                        .unwrap();

                    let batches = engine
                        .query("SELECT * FROM users ORDER BY id")
                        .await
                        .unwrap();
                    let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum();
                    assert_eq!(total_rows, 2);
                }

                #[tokio::test]
                async fn update() {
                    let _tmp = tempfile::tempdir().unwrap();
                    let engine = $make_engine(_tmp.path());
                    let schema = users_schema();
                    engine.create_table("users", &schema, &[]).await.unwrap();

                    engine
                        .execute("INSERT INTO users (id, name, age) VALUES (1, 'Alice', 30)")
                        .await
                        .unwrap();

                    let rows = engine
                        .execute("UPDATE users SET age = 31 WHERE id = 1")
                        .await
                        .unwrap();
                    assert_eq!(rows, 1);

                    let batches = engine
                        .query("SELECT age FROM users WHERE id = 1")
                        .await
                        .unwrap();
                    let age = batches[0].column(0).as_primitive::<Int64Type>().value(0);
                    assert_eq!(age, 31);
                }

                #[tokio::test]
                async fn delete() {
                    let _tmp = tempfile::tempdir().unwrap();
                    let engine = $make_engine(_tmp.path());
                    let schema = users_schema();
                    engine.create_table("users", &schema, &[]).await.unwrap();

                    engine
                        .execute(
                            "INSERT INTO users (id, name, age) VALUES (1, 'Alice', 30), (2, 'Bob', 25)",
                        )
                        .await
                        .unwrap();

                    let rows = engine
                        .execute("DELETE FROM users WHERE id = 1")
                        .await
                        .unwrap();
                    assert_eq!(rows, 1);

                    let batches = engine.query("SELECT COUNT(*) FROM users").await.unwrap();
                    let count = batches[0].column(0).as_primitive::<Int64Type>().value(0);
                    assert_eq!(count, 1);
                }

                #[tokio::test]
                async fn load_arrow() {
                    let _tmp = tempfile::tempdir().unwrap();
                    let engine = $make_engine(_tmp.path());
                    let schema = users_schema();
                    engine.create_table("users", &schema, &[]).await.unwrap();

                    let batch = RecordBatch::try_new(
                        schema.clone(),
                        vec![
                            Arc::new(arrow::array::Int64Array::from(vec![1, 2, 3])),
                            Arc::new(arrow::array::StringArray::from(vec![
                                "Alice", "Bob", "Charlie",
                            ])),
                            Arc::new(arrow::array::Int64Array::from(vec![30, 25, 35])),
                        ],
                    )
                    .unwrap();

                    let loaded = engine.load_arrow("users", &[batch]).await.unwrap();
                    assert_eq!(loaded, 3);

                    let batches = engine
                        .query("SELECT COUNT(*) as cnt FROM users")
                        .await
                        .unwrap();
                    let count = batches[0].column(0).as_primitive::<Int64Type>().value(0);
                    assert_eq!(count, 3);
                }

                #[tokio::test]
                async fn aggregate() {
                    let _tmp = tempfile::tempdir().unwrap();
                    let engine = $make_engine(_tmp.path());
                    let schema = users_schema();
                    engine.create_table("users", &schema, &[]).await.unwrap();

                    engine
                        .execute(
                            "INSERT INTO users (id, name, age) VALUES (1, 'Alice', 30), (2, 'Bob', 25), (3, 'Charlie', 35)",
                        )
                        .await
                        .unwrap();

                    let batches = engine
                        .query("SELECT AVG(age) as avg_age FROM users")
                        .await
                        .unwrap();
                    let avg = batches[0].column(0).as_primitive::<Float64Type>().value(0);
                    assert!((avg - 30.0).abs() < 0.01);
                }
            }
        };
    }

    storage_mode_tests!(in_memory, make_in_memory);
    storage_mode_tests!(arrow_ipc, make_arrow_ipc);
    storage_mode_tests!(parquet, make_parquet);

    // -----------------------------------------------------------------------
    // Parser edge-case tests (unit tests, no async / storage needed)
    // -----------------------------------------------------------------------

    /// INSERT with a string value that contains a comma — old regex broke here.
    #[tokio::test]
    async fn insert_string_with_comma() {
        let engine = DataFusionEngine::new();
        let schema = users_schema();
        engine.create_table("users", &schema, &[]).await.unwrap();

        // The name "Alice, B" contains a comma — old parse_value_tuples split on
        // commas inside strings and produced wrong results.
        engine
            .execute("INSERT INTO users (id, name, age) VALUES (1, 'Alice, B', 30)")
            .await
            .unwrap();

        let batches = engine
            .query("SELECT name FROM users WHERE id = 1")
            .await
            .unwrap();
        let name_arr = batches[0].column(0).as_string::<i32>();
        assert_eq!(name_arr.value(0), "Alice, B");
    }

    /// INSERT with a NULL value in a nullable column.
    #[tokio::test]
    async fn insert_null_value() {
        let engine = DataFusionEngine::new();
        let schema = users_schema();
        engine.create_table("users", &schema, &[]).await.unwrap();

        engine
            .execute("INSERT INTO users (id, name, age) VALUES (1, NULL, 30)")
            .await
            .unwrap();

        let batches = engine
            .query("SELECT name FROM users WHERE id = 1")
            .await
            .unwrap();
        assert!(batches[0].column(0).is_null(0));
    }

    /// UPDATE with a compound AND condition in the WHERE clause.
    #[tokio::test]
    async fn update_where_and() {
        let engine = DataFusionEngine::new();
        let schema = Arc::new(arrow::datatypes::Schema::new(vec![
            arrow::datatypes::Field::new("id", DataType::Int64, false),
            arrow::datatypes::Field::new("name", DataType::Utf8, true),
            arrow::datatypes::Field::new("status", DataType::Utf8, true),
        ]));
        engine.create_table("t", &schema, &[]).await.unwrap();

        engine
            .execute("INSERT INTO t (id, name, status) VALUES (1, 'x', 'active')")
            .await
            .unwrap();
        engine
            .execute("INSERT INTO t (id, name, status) VALUES (2, 'y', 'inactive')")
            .await
            .unwrap();

        // Only row 1 matches both conditions
        let updated = engine
            .execute("UPDATE t SET name = 'updated' WHERE id = 1 AND status = 'active'")
            .await
            .unwrap();
        assert_eq!(updated, 1);

        let batches = engine
            .query("SELECT name FROM t WHERE id = 1")
            .await
            .unwrap();
        assert_eq!(batches[0].column(0).as_string::<i32>().value(0), "updated");

        let batches2 = engine
            .query("SELECT name FROM t WHERE id = 2")
            .await
            .unwrap();
        assert_eq!(batches2[0].column(0).as_string::<i32>().value(0), "y");
    }

    /// DELETE with a quoted (double-quoted) table identifier.
    #[tokio::test]
    async fn delete_quoted_identifier() {
        let engine = DataFusionEngine::new();
        let schema = users_schema();
        // Table name is "users" (unquoted in create_table, but DELETE uses "users")
        engine.create_table("users", &schema, &[]).await.unwrap();

        engine
            .execute("INSERT INTO users (id, name, age) VALUES (1, 'Alice', 30)")
            .await
            .unwrap();
        engine
            .execute("INSERT INTO users (id, name, age) VALUES (2, 'Bob', 25)")
            .await
            .unwrap();

        // DELETE with double-quoted table identifier — SQLite dialect supports this
        let deleted = engine
            .execute(r#"DELETE FROM "users" WHERE id = 1"#)
            .await
            .unwrap();
        assert_eq!(deleted, 1);

        let batches = engine.query("SELECT COUNT(*) FROM users").await.unwrap();
        let count = batches[0].column(0).as_primitive::<Int64Type>().value(0);
        assert_eq!(count, 1);
    }

    /// INSERT with a string containing an escaped single quote ('').
    #[tokio::test]
    async fn insert_escaped_single_quote() {
        let engine = DataFusionEngine::new();
        let schema = users_schema();
        engine.create_table("users", &schema, &[]).await.unwrap();

        // "O'Brien" — escaped as O''Brien in SQL
        engine
            .execute("INSERT INTO users (id, name, age) VALUES (1, 'O''Brien', 42)")
            .await
            .unwrap();

        let batches = engine
            .query("SELECT name FROM users WHERE id = 1")
            .await
            .unwrap();
        assert_eq!(batches[0].column(0).as_string::<i32>().value(0), "O'Brien");
    }

    /// Unit test: `parse_insert_values` handles multi-row INSERT correctly.
    #[test]
    fn parse_insert_multi_row() {
        let (table, cols, batches) =
            parse_insert_values("INSERT INTO users (id, name) VALUES (1, 'Alice'), (2, 'Bob')")
                .unwrap();
        assert_eq!(table, "users");
        assert_eq!(cols, vec!["id", "name"]);
        assert_eq!(batches.len(), 1);
        assert_eq!(batches[0].num_rows(), 2);
    }

    /// Unit test: `parse_update` extracts table, assignments, and WHERE correctly.
    #[test]
    fn parse_update_basic() {
        let (table, assignments, where_clause) =
            parse_update("UPDATE users SET name = 'Alice' WHERE id = 1").unwrap();
        assert_eq!(table, "users");
        assert_eq!(
            assignments,
            vec![("name".to_string(), "'Alice'".to_string())]
        );
        assert_eq!(where_clause, vec![("id".to_string(), "1".to_string())]);
    }

    /// Unit test: `parse_delete` with no WHERE clause returns empty conditions.
    #[test]
    fn parse_delete_no_where() {
        let (table, conditions) = parse_delete("DELETE FROM logs").unwrap();
        assert_eq!(table, "logs");
        assert!(conditions.is_empty());
    }

    // -----------------------------------------------------------------------
    // Cloud-storage feature tests
    // -----------------------------------------------------------------------

    /// Verify `StorageMode::S3Parquet` constructs and reports correct attributes.
    #[cfg(feature = "cloud-storage")]
    #[test]
    fn storage_mode_s3_parquet_attributes() {
        let mode = StorageMode::S3Parquet {
            url: "s3://my-bucket/rhei-data".to_string(),
        };
        assert!(mode.is_cloud());
        assert_eq!(mode.file_extension(), "parquet");
        assert!(mode.base_path().is_none());
        assert_eq!(mode.cloud_base_url(), Some("s3://my-bucket/rhei-data"));
    }

    /// Verify `StorageMode::GcsParquet` constructs and reports correct attributes.
    #[cfg(feature = "cloud-storage")]
    #[test]
    fn storage_mode_gcs_parquet_attributes() {
        let mode = StorageMode::GcsParquet {
            url: "gs://gcs-bucket/prefix".to_string(),
        };
        assert!(mode.is_cloud());
        assert_eq!(mode.file_extension(), "parquet");
        assert!(mode.base_path().is_none());
        assert_eq!(mode.cloud_base_url(), Some("gs://gcs-bucket/prefix"));
    }

    /// Verify bucket parsing extracts the host from an S3 URL.
    #[cfg(feature = "cloud-storage")]
    #[test]
    fn parse_bucket_s3() {
        let bucket = DataFusionEngine::parse_bucket("s3://my-bucket/some/prefix", "s3").unwrap();
        assert_eq!(bucket, "my-bucket");
    }

    /// Verify bucket parsing extracts the host from a GCS URL.
    #[cfg(feature = "cloud-storage")]
    #[test]
    fn parse_bucket_gcs() {
        let bucket = DataFusionEngine::parse_bucket("gs://gcs-bucket/data", "gs").unwrap();
        assert_eq!(bucket, "gcs-bucket");
    }

    /// Verify bucket parsing rejects mismatched scheme.
    #[cfg(feature = "cloud-storage")]
    #[test]
    fn parse_bucket_wrong_scheme_returns_error() {
        let result = DataFusionEngine::parse_bucket("gs://bucket/data", "s3");
        assert!(result.is_err());
    }

    /// Verify `cloud_table_url` constructs correct table URLs.
    #[cfg(feature = "cloud-storage")]
    #[test]
    fn cloud_table_url_construction() {
        assert_eq!(
            DataFusionEngine::cloud_table_url("s3://bucket/prefix", "events"),
            "s3://bucket/prefix/events/"
        );
        // Trailing slash on base is tolerated.
        assert_eq!(
            DataFusionEngine::cloud_table_url("s3://bucket/prefix/", "logs"),
            "s3://bucket/prefix/logs/"
        );
    }

    /// Smoke test: `DataFusionEngine::with_storage(S3Parquet)` constructs
    /// without panic.  The object store build (`AmazonS3Builder::from_env`)
    /// may fail if no AWS credentials are available in the environment;
    /// we accept either Ok or an object-store/config error — the important
    /// thing is that the *code path* compiles and runs.
    ///
    /// Set `RHEI_TEST_S3=1` and provide real AWS credentials to exercise
    /// actual reads/writes.
    #[cfg(feature = "cloud-storage")]
    #[test]
    fn s3_parquet_engine_construction_does_not_panic() {
        // AmazonS3Builder::from_env may succeed even without credentials
        // (it will fail later on actual I/O) — we just verify no panic.
        let result = DataFusionEngine::with_storage(StorageMode::S3Parquet {
            url: "s3://test-bucket/test-prefix".to_string(),
        });
        // Construction may fail due to missing credentials or network — that
        // is acceptable.  What matters is no panic / stack overflow.
        let _ = result;
    }

    /// Same smoke test for GCS.
    #[cfg(feature = "cloud-storage")]
    #[test]
    fn gcs_parquet_engine_construction_does_not_panic() {
        let result = DataFusionEngine::with_storage(StorageMode::GcsParquet {
            url: "gs://test-bucket/test-prefix".to_string(),
        });
        let _ = result;
    }

    // -----------------------------------------------------------------------
    // Restart-safety: cloud file_counter initialisation
    // -----------------------------------------------------------------------

    /// Verifies that `cloud_seq_for_prefix` returns 0 when the prefix is empty
    /// and returns (max + 1) when existing objects are present.
    ///
    /// Uses `object_store::memory::InMemory` as a zero-dependency mock.
    #[cfg(feature = "cloud-storage")]
    #[tokio::test]
    async fn cloud_seq_for_prefix_empty_prefix_returns_zero() {
        let store: Arc<dyn ObjectStore> = Arc::new(object_store::memory::InMemory::new());
        // Fake URL — only the path component is used.
        let table_url = "s3://bucket/data/events/";
        let next = DataFusionEngine::cloud_seq_for_prefix(&store, table_url, "events")
            .await
            .unwrap();
        assert_eq!(next, 0, "empty prefix should yield counter = 0");
    }

    /// Verifies that existing objects are parsed and the counter is set to
    /// max_existing_index + 1.
    #[cfg(feature = "cloud-storage")]
    #[tokio::test]
    async fn cloud_seq_for_prefix_advances_past_existing_files() {
        use object_store::ObjectStoreExt as _;

        let store: Arc<dyn ObjectStore> = Arc::new(object_store::memory::InMemory::new());

        // Pre-populate with three objects simulating a previous run:
        //   data/events/events_000000.parquet
        //   data/events/events_000001.parquet
        //   data/events/events_000002.parquet
        for seq in 0u64..3 {
            let path = object_store::path::Path::from(
                format!("data/events/events_{seq:06}.parquet").as_str(),
            );
            store
                .put(&path, bytes::Bytes::from_static(b"dummy").into())
                .await
                .unwrap();
        }

        let table_url = "s3://bucket/data/events/";
        let next = DataFusionEngine::cloud_seq_for_prefix(&store, table_url, "events")
            .await
            .unwrap();
        // Max existing index is 2 → next should be 3.
        assert_eq!(next, 3, "counter should start at max_existing + 1");
    }

    /// End-to-end restart-safety test using `object_store::memory::InMemory`.
    ///
    /// Simulates the scenario where a cloud-backed engine is destroyed and
    /// re-created pointing at the same prefix.  The second engine must not
    /// overwrite files produced by the first engine.
    #[cfg(feature = "cloud-storage")]
    #[tokio::test]
    async fn cloud_engine_restart_does_not_overwrite_existing_files() {
        use object_store::ObjectStoreExt as _;
        use std::sync::atomic::Ordering;

        let store: Arc<dyn ObjectStore> = Arc::new(object_store::memory::InMemory::new());

        // Engine 1: write two files (seq 0 and 1).
        // We bypass with_storage (which would try real AWS/GCS), and instead
        // directly exercise cloud_seq_for_prefix + file_counter via the helper.

        // Pre-populate two objects to simulate files written by engine 1.
        for seq in 0u64..2 {
            let path = object_store::path::Path::from(
                format!("prefix/users/users_{seq:06}.parquet").as_str(),
            );
            store
                .put(&path, bytes::Bytes::from_static(b"parquet-data").into())
                .await
                .unwrap();
        }

        // Engine 2 restarts: call cloud_seq_for_prefix for "users".
        let table_url = "s3://bucket/prefix/users/";
        let next = DataFusionEngine::cloud_seq_for_prefix(&store, table_url, "users")
            .await
            .unwrap();
        assert_eq!(
            next, 2,
            "restarted engine should begin writing at index 2, not 0"
        );

        // Simulate engine 2 advancing its counter and writing a new file.
        let counter = AtomicU64::new(0);
        counter.fetch_max(next, Ordering::Relaxed);
        let seq = counter.fetch_add(1, Ordering::Relaxed);
        assert_eq!(seq, 2, "first write after restart should use index 2");

        // Verify the original two objects are still present (get returns Ok).
        for orig_seq in 0u64..2 {
            let path = object_store::path::Path::from(
                format!("prefix/users/users_{orig_seq:06}.parquet").as_str(),
            );
            let result = store.get(&path).await;
            assert!(
                result.is_ok(),
                "original file users_{orig_seq:06}.parquet must not be overwritten"
            );
        }
    }
}