datapress-datafusion 0.3.2

Apache Arrow + DataFusion-backed implementation of the datapress Parquet/Delta dataset HTTP server.
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
use std::cmp::Ordering;
use std::collections::HashMap;
use std::sync::{Arc, Mutex};

use arc_swap::ArcSwap;
use arrow::array::{
    Array, ArrayRef, BooleanArray, Decimal128Array, Decimal256Array, Float32Array, Float64Array,
    Int8Array, Int16Array, Int32Array, Int64Array, LargeStringArray, RecordBatch, Scalar,
    StringArray, StringViewArray, UInt8Array, UInt16Array, UInt32Array, UInt64Array,
};
use arrow::compute;
use arrow::compute::kernels::cmp::{eq, gt, gt_eq, lt, lt_eq, neq};
use arrow::datatypes::{DataType, Field, Schema};
use async_trait::async_trait;
use parquet::arrow::ProjectionMask;
use parquet::arrow::arrow_reader::{ArrowReaderOptions, ParquetRecordBatchReaderBuilder};
use serde_json::Value as JsonValue;

use datafusion::datasource::file_format::parquet::ParquetFormat;
use datafusion::datasource::listing::{
    ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl,
};
use datafusion::datasource::{MemTable, TableProvider};
use datafusion::prelude::SessionContext;
use datafusion::scalar::ScalarValue;

use object_store::aws::AmazonS3Builder;
use url::Url;

use datapress_core::backend::{
    ArrowIpcStream, Backend, DatasetSummary, ReloadStats, arrow_ipc_stream_channel,
};
use datapress_core::config::{
    AddressingStyle, AppConfig, DatasetConfig, IndexConfig, IndexMode, Partitioning, ResolvedCreds,
    S3Config, SourceKind,
};
use datapress_core::errors::AppError;
use datapress_core::models::{CountRequest, Predicate, QueryRequest};
use datapress_core::schema::{ColumnInfo, DatasetSchema, LogicalType};

// ---------------------------------------------------------------------------
// Public types
// ---------------------------------------------------------------------------

/// Pre-built equality index: lowercase col name → string-encoded value → sorted row ids.
type EqIndex = HashMap<String, HashMap<String, Vec<u32>>>;

/// Per-dataset state: schema metadata, the resident chunks, and the
/// equality index built per the dataset's `[dataset.index]` policy.
///
/// `data` is the dataset as a `Vec<RecordBatch>` — exactly the chunks
/// produced by the underlying reader, after temporal columns are cast to
/// `Utf8`. We deliberately do **not** call `concat_batches` to fuse them
/// into one batch: on wide schemas (hundreds of columns) that transiently
/// allocates a second full copy of the decoded Arrow data, pushing peak
/// RSS to ~2× the resident size and OOM-killing the process at startup.
///
/// When `lazy` is true the dataset is *not* materialised: `data` is empty,
/// `index` is empty, and every query is dispatched to DataFusion SQL
/// against a registered `ListingTable`. `arrow_schema` still carries the
/// inferred schema so discovery endpoints work.
pub struct DatasetState {
    pub schema: DatasetSchema,
    pub data: Vec<RecordBatch>,
    pub arrow_schema: Arc<Schema>,
    pub index: EqIndex,
    pub lazy: bool,
}

impl DatasetState {
    /// Sum of `num_rows()` across all resident chunks. `0` for lazy datasets.
    pub fn num_rows(&self) -> usize {
        self.data.iter().map(|b| b.num_rows()).sum()
    }
}

/// Multi-dataset registry. Each dataset is registered in the shared
/// `SessionContext` under its configured name. The per-dataset state is
/// held behind `ArcSwap` so a reload can atomically replace it without
/// blocking concurrent queries.
pub struct Store {
    ctx: SessionContext,
    max_page_size: u64,
    /// Original dataset configs, indexed by name. Reload reads the source
    /// path from here — clients can't redirect a reload at an arbitrary file.
    configs: HashMap<String, DatasetConfig>,
    /// Hot-swappable snapshot of all currently loaded datasets.
    datasets: ArcSwap<HashMap<String, Arc<DatasetState>>>,
    /// Per-name reload mutex. Serialises concurrent reloads of the same
    /// dataset; reloads of different datasets proceed in parallel.
    reload_locks: Mutex<HashMap<String, Arc<tokio::sync::Mutex<()>>>>,
}

impl Store {
    /// Load every dataset declared in `cfg`.
    pub async fn load(cfg: &AppConfig) -> Result<Self, AppError> {
        // One-shot init for the deltalake S3 backend. Safe to call more
        // than once — the handlers are idempotent.
        if cfg
            .datasets
            .iter()
            .any(|d| d.source.kind == SourceKind::Delta && d.source.is_s3())
        {
            deltalake::aws::register_handlers(None);
        }

        let ctx = SessionContext::new();
        let mut datasets = HashMap::with_capacity(cfg.datasets.len());
        let mut configs = HashMap::with_capacity(cfg.datasets.len());

        for d in &cfg.datasets {
            let (state, provider) = build_dataset(d, &ctx).await?;
            ctx.register_table(d.name.as_str(), provider)?;
            datasets.insert(d.name.clone(), Arc::new(state));
            configs.insert(d.name.clone(), d.clone());
        }
        Ok(Self {
            ctx,
            max_page_size: cfg.server.max_page_size.max(1),
            configs,
            datasets: ArcSwap::from_pointee(datasets),
            reload_locks: Mutex::new(HashMap::new()),
        })
    }

    /// Sorted list of dataset names.
    pub fn names(&self) -> Vec<String> {
        let snap = self.datasets.load();
        let mut v: Vec<String> = snap.keys().cloned().collect();
        v.sort();
        v
    }

    pub fn dataset(&self, name: &str) -> Result<Arc<DatasetState>, AppError> {
        self.datasets
            .load()
            .get(name)
            .cloned()
            .ok_or_else(|| AppError::NotFound(format!("dataset: {name}")))
    }

    /// JSON for the first row of the dataset, or `null` if empty. Used by
    /// `GET /api/datasets/{name}/schema` for discoverability.
    pub async fn sample(&self, name: &str) -> Result<String, AppError> {
        let st = self.dataset(name)?;

        // Lazy datasets have no resident batch — pull one row via SQL.
        if st.lazy {
            let table = DatasetSchema::quote_ident(&st.schema.name);
            let sql = format!("SELECT * FROM {table} LIMIT 1");
            let df = self.ctx.sql(&sql).await?;
            let batches = df.collect().await?;
            if batches.is_empty() || batches.iter().all(|b| b.num_rows() == 0) {
                return Ok("null".into());
            }
            let arr = serialize(&batches[0].slice(0, 1))?;
            let trimmed = arr.trim();
            let inner = trimmed
                .strip_prefix('[')
                .and_then(|s| s.strip_suffix(']'))
                .unwrap_or(trimmed);
            return Ok(inner.to_string());
        }

        let first = match st.data.iter().find(|b| b.num_rows() > 0) {
            Some(b) => b,
            None => return Ok("null".into()),
        };
        let arr = serialize(&first.slice(0, 1))?;
        // strip the outer [] to return a single object
        let trimmed = arr.trim();
        let inner = trimmed
            .strip_prefix('[')
            .and_then(|s| s.strip_suffix(']'))
            .unwrap_or(trimmed);
        Ok(inner.to_string())
    }

    /// Rebuild `name` from disk and atomically swap it in. Concurrent queries
    /// against the same name continue to see the *old* `Arc<DatasetState>`
    /// until they finish; the old data is dropped once the last reference
    /// goes away.
    pub async fn reload(&self, name: &str) -> Result<ReloadStats, AppError> {
        // 1. Look up the dataset config. Not finding it = 404.
        let cfg = self
            .configs
            .get(name)
            .ok_or_else(|| AppError::NotFound(format!("dataset: {name}")))?
            .clone();

        // 2. Per-name lock: only one reload of this dataset at a time.
        let lock = {
            let mut locks = self.reload_locks.lock().unwrap();
            locks
                .entry(name.to_string())
                .or_insert_with(|| Arc::new(tokio::sync::Mutex::new(())))
                .clone()
        };
        let _guard = lock.lock().await;

        let started = std::time::Instant::now();

        // 3. Heavy lifting (source read + index build). Parquet/delta
        // readers are themselves async, so we don't wrap in `web::block`.
        let (state, provider) = build_dataset(&cfg, &self.ctx).await?;
        let rows = state.num_rows();

        // 4. Atomic swap.
        //   a) Replace the MemTable inside the SessionContext.
        //   b) ArcSwap a new snapshot map with the updated Arc<DatasetState>.
        // In-flight queries already hold the old provider + old Arc; they
        // run to completion. New queries see the new data.
        let _ = self.ctx.deregister_table(name)?;
        self.ctx.register_table(name, provider)?;

        let mut new_map = (**self.datasets.load()).clone();
        new_map.insert(name.to_string(), Arc::new(state));
        self.datasets.store(Arc::new(new_map));

        let elapsed_ms = started.elapsed().as_millis();
        log::info!("reloaded dataset '{name}': {rows} rows in {elapsed_ms} ms");
        Ok(ReloadStats { rows, elapsed_ms })
    }

    /// Run a `QueryRequest` against `name`. Empty predicates → O(1) Arrow
    /// slice. Otherwise → DataFusion SQL on the single registered table.
    /// Lazy datasets skip the in-memory hot paths and always dispatch to SQL.
    pub async fn query(&self, name: &str, req: &QueryRequest) -> Result<String, AppError> {
        let batch = self.query_batch(name, req).await?;
        if batch.num_rows() == 0 {
            return Ok("[]".to_string());
        }
        serialize(&batch)
    }

    /// Same plan as [`Self::query`], but encode the result page as an
    /// Arrow IPC stream (one schema message + one batch + EOS). Empty
    /// results still produce a valid, self-describing zero-batch stream.
    pub async fn query_arrow(&self, name: &str, req: &QueryRequest) -> Result<Vec<u8>, AppError> {
        let batch = self.query_batch(name, req).await?;
        let schema = batch.schema();
        let mut buf = Vec::with_capacity(8 * 1024);
        {
            let mut w = arrow::ipc::writer::StreamWriter::try_new(&mut buf, schema.as_ref())?;
            if batch.num_rows() > 0 {
                w.write(&batch)?;
            }
            w.finish()?;
        }
        Ok(buf)
    }

    pub async fn query_arrow_stream(
        &self,
        name: &str,
        req: &QueryRequest,
    ) -> Result<ArrowIpcStream, AppError> {
        let batches = self.query_batches(name, req).await?;
        Ok(stream_arrow_batches(batches))
    }    pub async fn query_arrow_stream_all(
        &self,
        name: &str,
        req: &QueryRequest,
    ) -> Result<ArrowIpcStream, AppError> {
        let batches = self.query_batches_all(name, req).await?;
        Ok(stream_arrow_batches(batches))
    }

    /// Encode the entire dataset as a single self-contained Parquet file.
    ///
    /// Collects every row (all columns, no predicates, no paging) and runs
    /// it through a single [`parquet::arrow::ArrowWriter`], so the result
    /// carries the row-group + footer metadata a Parquet reader needs to
    /// answer `count(*)` straight from the footer. Powers the cached
    /// `GET /datasets/{name}/parquet` HTTP endpoint.
    pub async fn parquet(&self, name: &str) -> Result<bytes::Bytes, AppError> {
        // All rows, all columns, no predicates / ordering / limit.
        let req = QueryRequest {
            columns: Vec::new(),
            predicates: Vec::new(),
            group_by: Vec::new(),
            aggregations: Vec::new(),
            distinct: false,
            order_by: Vec::new(),
            limit: None,
            page: 1,
            page_size: 1,
        };
        let st = self.dataset(name)?;
        let batches = self.query_batches_all(name, &req).await?;
        // Use the actual batch schema when we have rows so the writer schema
        // matches exactly (projection/nullability); fall back to the
        // dataset schema for an empty dataset.
        let schema = batches
            .first()
            .map(|b| b.schema())
            .unwrap_or_else(|| st.arrow_schema.clone());

        let mut buf: Vec<u8> = Vec::with_capacity(64 * 1024);
        {
            let props = parquet::file::properties::WriterProperties::builder()
                .set_compression(parquet::basic::Compression::SNAPPY)
                .build();
            let mut writer =
                parquet::arrow::ArrowWriter::try_new(&mut buf, schema, Some(props))
                    .map_err(|e| AppError::Internal(format!("parquet writer init: {e}")))?;
            for batch in &batches {
                if batch.num_rows() > 0 {
                    writer
                        .write(batch)
                        .map_err(|e| AppError::Internal(format!("parquet write: {e}")))?;
                }
            }
            writer
                .close()
                .map_err(|e| AppError::Internal(format!("parquet finish: {e}")))?;
        }
        Ok(bytes::Bytes::from(buf))
    }

    /// Compute the result page as a single `RecordBatch`. Shared between
    /// the JSON and Arrow IPC encoders.
    async fn query_batch(&self, name: &str, req: &QueryRequest) -> Result<RecordBatch, AppError> {
        let batches = self.query_batches(name, req).await?;
        if batches.is_empty() {
            return Ok(RecordBatch::new_empty(Arc::new(
                arrow::datatypes::Schema::empty(),
            )));
        }
        if batches.len() == 1 {
            return Ok(batches.into_iter().next().expect("checked len"));
        }
        if batches.iter().all(|b| b.num_rows() == 0) {
            return Ok(RecordBatch::new_empty(batches[0].schema()));
        }
        let batch = compute::concat_batches(&batches[0].schema(), batches.iter())?;
        Ok(batch)
    }

    /// Compute the result page as Arrow batches. Arrow IPC responses can
    /// write these directly, while JSON callers concatenate via
    /// [`Self::query_batch`] for the existing row conversion path.
    async fn query_batches(
        &self,
        name: &str,
        req: &QueryRequest,
    ) -> Result<Vec<RecordBatch>, AppError> {
        let st = self.dataset(name)?;

        let page = req.page.max(1);
        let page_size = req.page_size.clamp(1, self.max_page_size);
        let offset = ((page - 1) * page_size) as usize;
        let limit = page_size as usize;

        self.query_batches_inner(st, req, Some((offset, limit)))
            .await
    }

    /// Compute all matching rows as Arrow batches for the one-request
    /// streaming endpoint. `page` and `page_size` are intentionally ignored;
    /// optional `limit` still caps the total result size.
    async fn query_batches_all(
        &self,
        name: &str,
        req: &QueryRequest,
    ) -> Result<Vec<RecordBatch>, AppError> {
        let st = self.dataset(name)?;
        self.query_batches_inner(st, req, None).await
    }

    async fn query_batches_inner(
        &self,
        st: Arc<DatasetState>,
        req: &QueryRequest,
        page_window: Option<(usize, usize)>,
    ) -> Result<Vec<RecordBatch>, AppError> {
        let (offset, limit) = page_window.unwrap_or((0, req.limit.unwrap_or(u64::MAX) as usize));

        // In-memory hot paths only fire when:
        //   - the dataset is materialised,
        //   - the caller did not ask for ordering,
        //   - and did not ask for a hard `limit` cap on a paged request.
        // Both of the latter two require sorting / capping that the SQL
        // engine handles uniformly across all data types.
        let can_fast_path = !st.lazy
            && req.order_by.is_empty()
            && (page_window.is_none() || req.limit.is_none())
            && req.group_by.is_empty()
            && !req.distinct;

        if can_fast_path {
            let total = st.num_rows();

            // No predicates -> O(1) raw Arrow slices over resident batches,
            // no engine overhead.
            if req.predicates.is_empty() {
                if page_window.is_none() && req.limit.is_none() {
                    return st
                        .data
                        .iter()
                        .cloned()
                        .map(|batch| project(&st.schema, batch, &req.columns))
                        .collect();
                }
                let start = offset.min(total);
                let len = limit.min(total - start);
                let batch = slice_global(&st.data, &st.arrow_schema, start, len)?;
                return Ok(vec![project(&st.schema, batch, &req.columns)?]);
            }

            // Index fast path: if every predicate is eq/in on an indexed column,
            // resolve via the pre-built equality index.
            if let Some(rows) = try_index(&st.index, &req.predicates) {
                let batch = take_page(&st.data, &st.arrow_schema, &rows, offset, limit)?;
                return Ok(vec![project(&st.schema, batch, &req.columns)?]);
            }
        }

        // Fallback (and only path for lazy datasets): DataFusion SQL.
        let (sql, params) = match page_window {
            Some(_) => build_query_sql(&st.schema, req, self.max_page_size)?,
            None => build_query_stream_sql(&st.schema, req)?,
        };
        let mut df = self.ctx.sql(&sql).await?;
        if !params.is_empty() {
            df = df.with_param_values(params)?;
        }
        let batches = df.collect().await?;
        if batches.is_empty() || batches.iter().all(|b| b.num_rows() == 0) {
            let schema = batches
                .first()
                .map(|b| b.schema())
                .unwrap_or_else(|| Arc::new(arrow::datatypes::Schema::empty()));
            return Ok(vec![RecordBatch::new_empty(schema)]);
        }
        Ok(batches)
    }
}

fn stream_arrow_batches(batches: Vec<RecordBatch>) -> ArrowIpcStream {
    let schema = batches
        .first()
        .map(|batch| batch.schema())
        .unwrap_or_else(|| Arc::new(arrow::datatypes::Schema::empty()));
    let (mut writer, stream) = arrow_ipc_stream_channel(8);

    tokio::task::spawn_blocking(move || {
        let result = (|| -> Result<(), AppError> {
            let mut w = arrow::ipc::writer::StreamWriter::try_new(&mut writer, schema.as_ref())?;
            for batch in batches {
                if batch.num_rows() > 0 {
                    w.write(&batch)?;
                }
            }
            w.finish()?;
            Ok(())
        })();
        if let Err(err) = result {
            log::error!("datafusion arrow stream failed: {err}");
            writer.send_error(err);
        }
    });

    stream
}

impl Store {
    /// Return the number of rows matching `req.predicates`. With no
    /// predicates this is a cheap metadata lookup on materialised datasets
    /// and a `SELECT COUNT(*)` on lazy ones.
    pub async fn count(&self, name: &str, req: &CountRequest) -> Result<i64, AppError> {
        let st = self.dataset(name)?;

        if !st.lazy {
            // No predicates → resident row count, no scan.
            if req.predicates.is_empty() {
                return Ok(st.num_rows() as i64);
            }
            // Index fast path: same eligibility rules as `query`.
            if let Some(rows) = try_index(&st.index, &req.predicates) {
                return Ok(rows.len() as i64);
            }
        }

        // Fallback: DataFusion SQL — same predicate translation as `query`,
        // with predicate values bound as typed parameters.
        let (sql, params) = build_count_sql(&st.schema, &req.predicates)?;
        let mut df = self.ctx.sql(&sql).await?;
        if !params.is_empty() {
            df = df.with_param_values(params)?;
        }
        let batches = df.collect().await?;
        let n = batches
            .first()
            .and_then(|b| {
                b.column(0)
                    .as_any()
                    .downcast_ref::<arrow::array::Int64Array>()
            })
            .filter(|a| !a.is_empty())
            .map(|a| a.value(0))
            .unwrap_or(0);
        Ok(n)
    }
}

// ---------------------------------------------------------------------------
// Dataset loading
// ---------------------------------------------------------------------------

async fn build_dataset(
    d: &DatasetConfig,
    ctx: &SessionContext,
) -> Result<(DatasetState, Arc<dyn TableProvider>), AppError> {
    // Lazy datasets: register a ListingTable straight against the source
    // and skip the materialise / index / partition pipeline below. Delta
    // is rejected — deltalake reads the transaction log eagerly to know
    // which parquet files are current, so "lazy delta" doesn't map onto
    // ListingTable cleanly.
    if d.lazy {
        match (d.source.kind, d.source.is_s3()) {
            (SourceKind::Parquet, false) => return build_lazy_local_parquet(d, ctx).await,
            (SourceKind::Parquet, true) => return build_lazy_s3_parquet(d, ctx).await,
            (SourceKind::Delta, _) => {
                return Err(AppError::Internal(format!(
                    "dataset '{}': lazy mode is not supported for delta sources",
                    d.name
                )));
            }
        }
    }

    // Fetch raw RecordBatches from whichever backing store the dataset
    // is configured to use. All four (parquet, delta) x (local, s3)
    // combinations converge into one Vec<RecordBatch>; the materialisation
    // / indexing / partitioning logic below is shared.
    let raw_batches: Vec<RecordBatch> = match (d.source.kind, d.source.is_s3()) {
        (SourceKind::Parquet, false) => read_local_parquet(d)?,
        (SourceKind::Parquet, true) => read_s3_parquet(d, ctx).await?,
        (SourceKind::Delta, false) => read_delta(d, HashMap::new()).await?,
        (SourceKind::Delta, true) => read_delta(d, delta_s3_options(d)?).await?,
    };
    if raw_batches.is_empty() {
        return Err(AppError::Internal(format!(
            "dataset '{}': source produced no batches",
            d.name
        )));
    }

    let chunks = raw_batches;
    let arrow_sch = chunks[0].schema();

    // Build DatasetSchema from the Arrow schema.
    let columns: Vec<ColumnInfo> = arrow_sch
        .fields()
        .iter()
        .map(|f| {
            let dt = f.data_type();
            ColumnInfo {
                name: f.name().clone(),
                logical: arrow_to_logical(dt),
                sql_type: format!("{dt:?}"),
                nullable: f.is_nullable(),
            }
        })
        .collect();
    let schema = DatasetSchema::new(&d.name, columns);

    // Build the equality index per the per-dataset policy. Operates on the
    // chunked representation directly so we never have to materialise a
    // single concatenated batch (which would double peak RSS on wide
    // schemas — see `DatasetState` docs).
    let index = build_eq_index_with_policy(&chunks, &d.index);

    // Partition for parallel scans by the SQL fallback path. We distribute
    // the existing batches round-robin across `n_parts` partitions instead
    // of re-slicing a concatenated batch — `clone()` on a RecordBatch is
    // an Arc-clone of the column buffers, not a copy.
    let n_parts = std::thread::available_parallelism()
        .map(|n| n.get())
        .unwrap_or(4);
    let mut parts: Vec<Vec<RecordBatch>> = (0..n_parts).map(|_| Vec::new()).collect();
    for (i, b) in chunks.iter().enumerate() {
        if b.num_rows() == 0 {
            continue;
        }
        parts[i % n_parts].push(b.clone());
    }
    parts.retain(|p| !p.is_empty());
    let provider: Arc<dyn TableProvider> = Arc::new(MemTable::try_new(arrow_sch.clone(), parts)?);

    let total_rows: usize = chunks.iter().map(|b| b.num_rows()).sum();
    let mem_mb: usize = chunks
        .iter()
        .flat_map(|b| b.columns().iter())
        .map(|c| c.get_buffer_memory_size())
        .sum::<usize>()
        / 1_048_576;
    log::info!(
        "dataset '{}' [{}]: {} rows, {} cols, {} MB, {} chunks, {} indexed cols",
        d.name,
        d.source.kind.as_str(),
        total_rows,
        schema.columns.len(),
        mem_mb,
        chunks.len(),
        index.len()
    );

    Ok((
        DatasetState {
            schema,
            data: chunks,
            arrow_schema: arrow_sch,
            index,
            lazy: false,
        },
        provider,
    ))
}

/// Build a lazy state + `ListingTable` provider for a local parquet dataset.
/// The dataset is never read into RAM; DataFusion streams row groups on
/// each query. The returned `DatasetState.data` is an empty `Vec` —
/// `arrow_schema` still carries the inferred Arrow schema for discovery.
async fn build_lazy_local_parquet(
    d: &DatasetConfig,
    ctx: &SessionContext,
) -> Result<(DatasetState, Arc<dyn TableProvider>), AppError> {
    let (url, part_keys) = lazy_local_listing(d)?;

    let mut opts =
        ListingOptions::new(Arc::new(ParquetFormat::default())).with_file_extension(".parquet");
    if !part_keys.is_empty() {
        opts = opts.with_table_partition_cols(
            part_keys
                .iter()
                .map(|k| (k.clone(), DataType::Utf8))
                .collect(),
        );
    }

    let session_state = ctx.state();
    // `infer_schema` returns the *file* schema (without partition columns);
    // `ListingTable` appends the declared partition columns on top.
    let file_schema = opts.infer_schema(&session_state, &url).await.map_err(|e| {
        AppError::Internal(format!("dataset '{}': infer parquet schema: {e}", d.name))
    })?;

    let cfg = ListingTableConfig::new(url)
        .with_listing_options(opts)
        .with_schema(file_schema.clone());
    let table = ListingTable::try_new(cfg).map_err(|e| {
        AppError::Internal(format!("dataset '{}': ListingTable::try_new: {e}", d.name))
    })?;
    let provider: Arc<dyn TableProvider> = Arc::new(table);

    // Discovery schema = file columns + partition columns (Utf8).
    let mut fields: Vec<Field> = file_schema
        .fields()
        .iter()
        .map(|f| f.as_ref().clone())
        .collect();
    for k in &part_keys {
        if !fields.iter().any(|f| f.name() == k) {
            fields.push(Field::new(k, DataType::Utf8, false));
        }
    }
    let arrow_sch = Arc::new(Schema::new(fields));

    let columns: Vec<ColumnInfo> = arrow_sch
        .fields()
        .iter()
        .map(|f| {
            let dt = f.data_type();
            ColumnInfo {
                name: f.name().clone(),
                logical: arrow_to_logical(dt),
                sql_type: format!("{dt:?}"),
                nullable: f.is_nullable(),
            }
        })
        .collect();
    let schema = DatasetSchema::new(&d.name, columns);

    log::info!(
        "dataset '{}' [{}, lazy]: {} cols ({} partition), no materialise, no index",
        d.name,
        d.source.kind.as_str(),
        schema.columns.len(),
        part_keys.len()
    );

    Ok((
        DatasetState {
            schema,
            data: Vec::new(),
            arrow_schema: arrow_sch,
            index: EqIndex::new(),
            lazy: true,
        },
        provider,
    ))
}

/// Resolve a local lazy-parquet location into a single `ListingTableUrl`
/// rooted at the dataset base plus the ordered hive partition keys (if any).
/// Handles three shapes: a glob (`root/city=*/*.parquet`), a directory
/// (hive root or flat folder of parquets), and a single `*.parquet` file.
fn lazy_local_listing(d: &DatasetConfig) -> Result<(ListingTableUrl, Vec<String>), AppError> {
    let loc = &d.source.location;

    if loc.contains('*') || loc.contains('?') || loc.contains('[') {
        let parts: Vec<&str> = loc.split('/').collect();
        let first_wild = parts
            .iter()
            .position(|c| c.contains('*') || c.contains('?') || c.contains('['))
            .unwrap_or(parts.len());
        let base = parts[..first_wild].join("/");
        let base = if base.is_empty() {
            "/".to_string()
        } else {
            base
        };
        // Partition keys: `key=…` components between the base and the file
        // pattern (the final component).
        let upper = parts.len().saturating_sub(1);
        let keys: Vec<String> = parts[first_wild.min(upper)..upper]
            .iter()
            .filter_map(|c| c.split_once('=').map(|(k, _)| k.to_string()))
            .filter(|k| !k.is_empty())
            .collect();
        return Ok((dir_url(std::path::Path::new(&base), d)?, keys));
    }

    let path = std::path::Path::new(loc);
    if path.is_dir() {
        let keys = discover_hive_keys(path);
        return Ok((dir_url(path, d)?, keys));
    }

    let url = ListingTableUrl::parse(loc)
        .map_err(|e| AppError::Internal(format!("dataset '{}': bad url '{loc}': {e}", d.name)))?;
    Ok((url, Vec::new()))
}

/// Parse a directory path into a `ListingTableUrl` (trailing slash so
/// DataFusion treats it as a directory root, not a single object).
fn dir_url(path: &std::path::Path, d: &DatasetConfig) -> Result<ListingTableUrl, AppError> {
    let s = path.to_str().ok_or_else(|| {
        AppError::Internal(format!(
            "dataset '{}': non-utf8 path {}",
            d.name,
            path.display()
        ))
    })?;
    let s = if s.ends_with('/') {
        s.to_string()
    } else {
        format!("{s}/")
    };
    ListingTableUrl::parse(&s)
        .map_err(|e| AppError::Internal(format!("dataset '{}': bad url '{s}': {e}", d.name)))
}

/// Walk down a directory following the first `key=value` subdirectory at
/// each level to discover the ordered hive partition keys. Returns an empty
/// vec for a flat (non-partitioned) folder.
fn discover_hive_keys(base: &std::path::Path) -> Vec<String> {
    let mut keys = Vec::new();
    let mut cur = base.to_path_buf();
    loop {
        let Ok(rd) = std::fs::read_dir(&cur) else {
            break;
        };
        let mut next: Option<(String, std::path::PathBuf)> = None;
        for entry in rd.flatten() {
            let p = entry.path();
            if !p.is_dir() {
                continue;
            }
            let Some(name) = p.file_name().and_then(|n| n.to_str()) else {
                continue;
            };
            if let Some((k, v)) = name.split_once('=')
                && !k.is_empty()
                && !v.is_empty()
            {
                next = Some((k.to_string(), p));
                break;
            }
        }
        match next {
            Some((k, p)) => {
                keys.push(k);
                cur = p;
            }
            None => break,
        }
    }
    keys
}

/// Lazy S3 parquet: register the dataset's S3 object store on `ctx`, then
/// build a `ListingTable` rooted at the `s3://bucket/prefix/` location with
/// any discovered hive partition columns. DataFusion does the directory
/// listing through the registered store and streams row groups on each
/// query — no local enumeration needed.
async fn build_lazy_s3_parquet(
    d: &DatasetConfig,
    ctx: &SessionContext,
) -> Result<(DatasetState, Arc<dyn TableProvider>), AppError> {
    register_s3_object_store(d, ctx)?;

    let (provider, file_schema, part_keys) = build_s3_listing_table(d, ctx).await?;

    // Discovery schema = file columns + partition columns (Utf8).
    let mut fields: Vec<Field> = file_schema
        .fields()
        .iter()
        .map(|f| f.as_ref().clone())
        .collect();
    for k in &part_keys {
        if !fields.iter().any(|f| f.name() == k) {
            fields.push(Field::new(k, DataType::Utf8, false));
        }
    }
    let arrow_sch = Arc::new(Schema::new(fields));

    let columns: Vec<ColumnInfo> = arrow_sch
        .fields()
        .iter()
        .map(|f| {
            let dt = f.data_type();
            ColumnInfo {
                name: f.name().clone(),
                logical: arrow_to_logical(dt),
                sql_type: format!("{dt:?}"),
                nullable: f.is_nullable(),
            }
        })
        .collect();
    let schema = DatasetSchema::new(&d.name, columns);

    log::info!(
        "dataset '{}' [{}, lazy, s3]: {} cols ({} partition, no materialise, no index)",
        d.name,
        d.source.kind.as_str(),
        schema.columns.len(),
        part_keys.len()
    );

    Ok((
        DatasetState {
            schema,
            data: Vec::new(),
            arrow_schema: arrow_sch,
            index: EqIndex::new(),
            lazy: true,
        },
        provider,
    ))
}

/// Build a `ListingTable` provider for an S3 parquet source, resolving the
/// base prefix and hive partition keys via [`s3_listing`]. The registered
/// S3 object store must already be present on `ctx`. Returns the provider,
/// the inferred *file* schema (without partition columns), and the ordered
/// partition keys.
async fn build_s3_listing_table(
    d: &DatasetConfig,
    ctx: &SessionContext,
) -> Result<(Arc<dyn TableProvider>, Arc<Schema>, Vec<String>), AppError> {
    let (url, part_keys) = s3_listing(d, ctx).await?;

    let mut opts =
        ListingOptions::new(Arc::new(ParquetFormat::default())).with_file_extension(".parquet");
    if !part_keys.is_empty() {
        opts = opts.with_table_partition_cols(
            part_keys
                .iter()
                .map(|k| (k.clone(), DataType::Utf8))
                .collect(),
        );
    }

    let session_state = ctx.state();
    let file_schema = opts.infer_schema(&session_state, &url).await.map_err(|e| {
        AppError::Internal(format!(
            "dataset '{}': infer parquet schema on s3: {e}",
            d.name
        ))
    })?;

    let cfg = ListingTableConfig::new(url)
        .with_listing_options(opts)
        .with_schema(file_schema.clone());
    let table = ListingTable::try_new(cfg).map_err(|e| {
        AppError::Internal(format!(
            "dataset '{}': ListingTable::try_new (s3): {e}",
            d.name
        ))
    })?;
    Ok((Arc::new(table), file_schema, part_keys))
}

/// Resolve an S3 parquet source into a `(base ListingTableUrl, partition
/// keys)` pair. Hive keys come from the location glob when present
/// (`s3://bucket/events/year=*/...`), otherwise — for a plain prefix — they
/// are discovered by listing the registered object store. Honours the
/// dataset's `partitioning` mode (`auto` / `hive` / `none`).
async fn s3_listing(
    d: &DatasetConfig,
    ctx: &SessionContext,
) -> Result<(ListingTableUrl, Vec<String>), AppError> {
    let s3 = d.s3.clone().unwrap_or_default();
    let want_partitions = !matches!(s3.partitioning, Partitioning::None);
    let loc = &d.source.location;

    if d.source.has_glob() {
        let (base, keys) = split_glob_base_keys(loc);
        let base = format!("{}/", base.trim_end_matches('/'));
        let url = ListingTableUrl::parse(&base).map_err(|e| {
            AppError::Internal(format!("dataset '{}': bad s3 url '{base}': {e}", d.name))
        })?;
        let keys = if want_partitions { keys } else { Vec::new() };
        return Ok((url, keys));
    }

    let base = if loc.ends_with('/') {
        loc.clone()
    } else {
        format!("{loc}/")
    };
    let url = ListingTableUrl::parse(&base).map_err(|e| {
        AppError::Internal(format!("dataset '{}': bad s3 url '{base}': {e}", d.name))
    })?;
    let keys = if want_partitions {
        discover_s3_hive_keys(ctx, &url).await
    } else {
        Vec::new()
    };
    Ok((url, keys))
}

/// Split a glob location into its non-wildcard base path and the ordered
/// hive partition keys (`key=…` directory components between the base and
/// the final file pattern). Works for both local and `s3://` paths.
fn split_glob_base_keys(loc: &str) -> (String, Vec<String>) {
    let parts: Vec<&str> = loc.split('/').collect();
    let first_wild = parts
        .iter()
        .position(|c| c.contains('*') || c.contains('?') || c.contains('['))
        .unwrap_or(parts.len());
    let base = parts[..first_wild].join("/");
    let base = if base.is_empty() {
        "/".to_string()
    } else {
        base
    };
    let upper = parts.len().saturating_sub(1);
    let keys: Vec<String> = parts[first_wild.min(upper)..upper]
        .iter()
        .filter_map(|c| c.split_once('=').map(|(k, _)| k.to_string()))
        .filter(|k| !k.is_empty())
        .collect();
    (base, keys)
}

/// Discover ordered hive partition keys for an S3 prefix by walking the
/// object store one `key=value/` level at a time via delimiter listings.
/// Best-effort: any listing error stops discovery and returns what was found
/// so far (empty = treat as a flat folder).
async fn discover_s3_hive_keys(ctx: &SessionContext, url: &ListingTableUrl) -> Vec<String> {
    let store = match ctx.runtime_env().object_store(url.object_store()) {
        Ok(s) => s,
        Err(_) => return Vec::new(),
    };
    let mut keys = Vec::new();
    let mut prefix = url.prefix().clone();
    loop {
        let listing = match store.list_with_delimiter(Some(&prefix)).await {
            Ok(l) => l,
            Err(_) => break,
        };
        let mut next: Option<object_store::path::Path> = None;
        for cp in &listing.common_prefixes {
            if let Some(seg) = cp.parts().next_back() {
                let seg = seg.as_ref().to_string();
                if let Some((k, v)) = seg.split_once('=')
                    && !k.is_empty()
                    && !v.is_empty()
                {
                    keys.push(k.to_string());
                    next = Some(cp.clone());
                    break;
                }
            }
        }
        match next {
            Some(p) => prefix = p,
            None => break,
        }
    }
    keys
}

/// Original local-parquet code path — sync file I/O. We set a large reader
/// batch size so wide schemas (hundreds of columns) don't pay per-array
/// metadata overhead on thousands of small (default 1024-row) batches.
///
/// Two memory-saving knobs are applied here:
///
/// * **Column projection** — if `d.columns` is non-empty, only those
///   columns are decoded; everything else is skipped at the parquet reader
///   level (no Arrow array is ever allocated for the dropped columns).
/// * **Dictionary preservation** — Utf8 columns whose parquet column chunks
///   carry a dictionary page are materialised as Arrow
///   `Dictionary(Int32, Utf8)` instead of plain `Utf8`. Low-cardinality
///   string columns (state, country, severity, …) stay represented as
///   `n_unique` string slots plus an Int32 index per row instead of
///   `n_rows` independent strings — typically 10×–50× smaller for
///   real-world data.
fn read_local_parquet(d: &DatasetConfig) -> Result<Vec<RecordBatch>, AppError> {
    let files = d.resolve_local_parquet_files()?;
    let mut all = Vec::new();
    let wanted: Option<std::collections::HashSet<String>> = if d.columns.is_empty() {
        None
    } else {
        Some(d.columns.iter().map(|c| c.to_lowercase()).collect())
    };

    for f in &files {
        let file = std::fs::File::open(f)
            .map_err(|e| AppError::Internal(format!("open {}: {e}", f.display())))?;

        // First pass: peek the parquet metadata + default Arrow schema so we
        // can (a) decide a column projection and (b) override Utf8 columns
        // that are dictionary-encoded in the file so the reader materialises
        // them as Arrow Dictionary arrays instead of expanding to plain Utf8.
        let probe = ParquetRecordBatchReaderBuilder::try_new(
            file.try_clone()
                .map_err(|e| AppError::Internal(format!("dup fd {}: {e}", f.display())))?,
        )?;
        let parquet_schema = probe.parquet_schema().clone();
        let arrow_schema = probe.schema().clone();
        let metadata = probe.metadata().clone();
        drop(probe);

        // Column projection (top-level / leaf indices for flat schemas).
        let projection = if let Some(w) = &wanted {
            let indices: Vec<usize> = arrow_schema
                .fields()
                .iter()
                .enumerate()
                .filter(|(_, fld)| w.contains(&fld.name().to_lowercase()))
                .map(|(i, _)| i)
                .collect();
            if indices.is_empty() {
                return Err(AppError::Internal(format!(
                    "dataset '{}': no columns from `columns = {:?}` match parquet schema for {}",
                    d.name,
                    d.columns,
                    f.display()
                )));
            }
            ProjectionMask::roots(&parquet_schema, indices)
        } else {
            ProjectionMask::all()
        };

        // Dictionary override: any Utf8 column whose first row group carries
        // a dictionary page is re-typed to Dictionary(Int32, Utf8). The
        // override schema must still describe every column in the parquet
        // file (projection is applied separately). Skipped entirely when
        // the dataset has `dict_encode = false` — escape hatch for cases
        // where the override interacts badly with null propagation in the
        // downstream engine.
        let mut new_fields: Vec<Field> = arrow_schema
            .fields()
            .iter()
            .map(|f| f.as_ref().clone())
            .collect();
        if d.dict_encode
            && let Some(rg0) = metadata.row_groups().first()
        {
            for (i, fld) in arrow_schema.fields().iter().enumerate() {
                if !matches!(
                    fld.data_type(),
                    DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View
                ) {
                    continue;
                }
                if let Some(col) = rg0.columns().get(i)
                    && col.dictionary_page_offset().is_some()
                {
                    new_fields[i] = Field::new(
                        fld.name(),
                        DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
                        fld.is_nullable(),
                    );
                }
            }
        }
        let forced_schema = Arc::new(Schema::new(new_fields));

        let opts = ArrowReaderOptions::new().with_schema(forced_schema);
        let reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, opts)?
            .with_batch_size(65_536)
            .with_projection(projection)
            .build()?;
        // Hive-style partition columns (`city=NYC/…`) live in the path, not
        // the file. Fold them in as constant Utf8 columns so they show up in
        // the schema and are queryable — matching the DuckDB backend.
        let pairs = hive_pairs(f);
        for batch in reader {
            let batch = batch.map_err(|e| AppError::Internal(e.to_string()))?;
            all.push(if pairs.is_empty() {
                batch
            } else {
                append_partition_cols(&batch, &pairs)?
            });
        }
    }
    if all.is_empty() {
        return Err(AppError::Internal(format!(
            "dataset '{}': parquet source is empty",
            d.name
        )));
    }
    Ok(all)
}

/// Ordered hive-style partition `(key, value)` pairs encoded in a path, i.e.
/// directory components shaped like `key=value` (e.g. `year=2024/city=NYC`).
fn hive_pairs(path: &std::path::Path) -> Vec<(String, String)> {
    path.components()
        .filter_map(|c| c.as_os_str().to_str())
        .filter_map(|seg| {
            let (k, v) = seg.split_once('=')?;
            if k.is_empty() || v.is_empty() || v.contains('=') {
                return None;
            }
            Some((k.to_string(), v.to_string()))
        })
        .collect()
}

/// Append constant Utf8 columns for each hive partition pair. A partition
/// key that collides with a real file column is skipped (the file wins).
fn append_partition_cols(
    batch: &RecordBatch,
    pairs: &[(String, String)],
) -> Result<RecordBatch, AppError> {
    let n = batch.num_rows();
    let mut fields: Vec<Field> = batch
        .schema()
        .fields()
        .iter()
        .map(|f| f.as_ref().clone())
        .collect();
    let mut cols: Vec<ArrayRef> = batch.columns().to_vec();
    for (k, v) in pairs {
        if fields.iter().any(|f| f.name() == k) {
            continue;
        }
        fields.push(Field::new(k, DataType::Utf8, false));
        cols.push(Arc::new(StringArray::from(vec![v.as_str(); n])));
    }
    RecordBatch::try_new(Arc::new(Schema::new(fields)), cols)
        .map_err(|e| AppError::Internal(e.to_string()))
}

/// Register an `AmazonS3` object store on the SessionContext, build a
/// `ListingTable` (with any hive partition columns) over the dataset prefix,
/// and stream the whole dataset back through `DataFrame::collect`. Using a
/// ListingTable here — rather than a bare `read_parquet` — means S3 sources
/// get the same hive-partition handling as local parquet.
async fn read_s3_parquet(
    d: &DatasetConfig,
    ctx: &SessionContext,
) -> Result<Vec<RecordBatch>, AppError> {
    register_s3_object_store(d, ctx)?;
    let (provider, _file_schema, _keys) = build_s3_listing_table(d, ctx).await?;
    let df = ctx
        .read_table(provider)
        .map_err(|e| AppError::Internal(format!("dataset '{}': s3 read_table: {e}", d.name)))?;
    Ok(df.collect().await?)
}

/// Open a Delta table (local or S3) and stream every row back as a Vec of
/// `RecordBatch`. We materialise eagerly so the rest of the backend can
/// treat all datasets uniformly (single in-memory batch + eq-index).
async fn read_delta(
    d: &DatasetConfig,
    opts: HashMap<String, String>,
) -> Result<Vec<RecordBatch>, AppError> {
    let url = deltalake::ensure_table_uri(&d.source.location).map_err(|e| {
        AppError::Internal(format!(
            "dataset '{}': bad delta location '{}': {e}",
            d.name, d.source.location
        ))
    })?;
    let table = deltalake::open_table_with_storage_options(url, opts)
        .await
        .map_err(|e| {
            AppError::Internal(format!(
                "dataset '{}': delta open '{}': {e}",
                d.name, d.source.location
            ))
        })?;
    let provider = table.table_provider().await.map_err(|e| {
        AppError::Internal(format!("dataset '{}': delta table_provider: {e}", d.name))
    })?;
    // Drive a full scan via a throwaway SessionContext so we end up with
    // an in-memory Vec<RecordBatch> the shared materialise path can use.
    let scan_ctx = SessionContext::new();
    let df = scan_ctx
        .read_table(provider)
        .map_err(|e| AppError::Internal(format!("dataset '{}': delta read_table: {e}", d.name)))?;
    Ok(df.collect().await?)
}

/// Build the storage-options HashMap that `deltalake::open_table_with_storage_options`
/// expects for S3 access. Keys mirror the AWS env-var names; deltalake
/// passes them through to object_store internally.
fn delta_s3_options(d: &DatasetConfig) -> Result<HashMap<String, String>, AppError> {
    let creds = d.resolved_creds();
    let region = d.resolved_region();
    let s3 = d.s3.clone().unwrap_or_default();
    let (bucket, _) = d.source.s3_bucket()?;

    let mut opts = HashMap::new();
    opts.insert("AWS_REGION".into(), region);
    if let Some(ep) = s3.effective_endpoint(bucket) {
        opts.insert("AWS_ENDPOINT_URL".into(), ep);
    }
    if s3.allow_http {
        opts.insert("AWS_ALLOW_HTTP".into(), "true".into());
    }
    opts.insert(
        "AWS_VIRTUAL_HOSTED_STYLE_REQUEST".into(),
        (s3.addressing_style == AddressingStyle::Virtual).to_string(),
    );
    if let Some(k) = creds.access_key_id {
        opts.insert("AWS_ACCESS_KEY_ID".into(), k);
    }
    if let Some(s) = creds.secret_access_key {
        opts.insert("AWS_SECRET_ACCESS_KEY".into(), s);
    }
    if let Some(t) = creds.session_token {
        opts.insert("AWS_SESSION_TOKEN".into(), t);
    }
    // Read-only paths don't need the S3 lock-provider plumbing.
    opts.insert("AWS_S3_ALLOW_UNSAFE_RENAME".into(), "true".into());
    Ok(opts)
}

/// Construct an `AmazonS3` object_store from the dataset's `[dataset.s3]`
/// block + resolved credentials and register it on `ctx` under
/// `s3://bucket/`.
fn register_s3_object_store(d: &DatasetConfig, ctx: &SessionContext) -> Result<(), AppError> {
    let (bucket, _key) = d.source.s3_bucket()?;
    let creds = d.resolved_creds();
    let region = d.resolved_region();
    let s3 = d.s3.clone().unwrap_or_default();

    let store = build_s3(bucket, &region, &s3, &creds).map_err(|e| {
        AppError::Internal(format!(
            "dataset '{}': build S3 store for '{bucket}': {e}",
            d.name
        ))
    })?;

    let url = Url::parse(&format!("s3://{bucket}"))
        .map_err(|e| AppError::Internal(format!("invalid s3 URL for bucket {bucket}: {e}")))?;
    ctx.register_object_store(&url, Arc::new(store));
    Ok(())
}

fn build_s3(
    bucket: &str,
    region: &str,
    s3: &S3Config,
    creds: &ResolvedCreds,
) -> Result<object_store::aws::AmazonS3, object_store::Error> {
    let mut b = AmazonS3Builder::new()
        .with_bucket_name(bucket)
        .with_region(region)
        .with_allow_http(s3.allow_http)
        .with_virtual_hosted_style_request(s3.addressing_style == AddressingStyle::Virtual);
    if let Some(ep) = s3.effective_endpoint(bucket) {
        b = b.with_endpoint(ep);
    }
    if let Some(k) = creds.access_key_id.as_deref() {
        b = b.with_access_key_id(k);
    }
    if let Some(s) = creds.secret_access_key.as_deref() {
        b = b.with_secret_access_key(s);
    }
    if let Some(t) = creds.session_token.as_deref() {
        b = b.with_token(t);
    }
    b.build()
}

fn arrow_to_logical(dt: &DataType) -> LogicalType {
    match dt {
        DataType::Boolean => LogicalType::Bool,
        DataType::Int8
        | DataType::Int16
        | DataType::Int32
        | DataType::Int64
        | DataType::UInt8
        | DataType::UInt16
        | DataType::UInt32
        | DataType::UInt64 => LogicalType::Int,
        DataType::Float16 | DataType::Float32 | DataType::Float64 => LogicalType::Float,
        DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => LogicalType::Utf8,
        // Dictionary-encoded strings are reported as plain strings — clients
        // (and the rest of the backend) shouldn't have to care that we keep
        // a compressed representation in memory.
        DataType::Dictionary(_, v)
            if matches!(
                v.as_ref(),
                DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View
            ) =>
        {
            LogicalType::Utf8
        }
        DataType::Date32
        | DataType::Date64
        | DataType::Time32(_)
        | DataType::Time64(_)
        | DataType::Timestamp(_, _)
        | DataType::Duration(_)
        | DataType::Interval(_) => LogicalType::Temporal,
        _ => LogicalType::Other,
    }
}

// ---------------------------------------------------------------------------
// Per-batch projection
// ---------------------------------------------------------------------------

fn project(
    schema: &DatasetSchema,
    batch: RecordBatch,
    columns: &[String],
) -> Result<RecordBatch, AppError> {
    if columns.is_empty() {
        return Ok(batch);
    }
    let indices: Vec<usize> = columns
        .iter()
        .map(|c| {
            schema
                .find(c)
                .map(|info| schema.by_name[&info.name.to_lowercase()])
        })
        .collect::<Result<_, _>>()?;
    let fields: Vec<Field> = indices
        .iter()
        .map(|&i| batch.schema().field(i).clone())
        .collect();
    let cols: Vec<ArrayRef> = indices.iter().map(|&i| batch.column(i).clone()).collect();
    Ok(RecordBatch::try_new(Arc::new(Schema::new(fields)), cols)?)
}

// ---------------------------------------------------------------------------
// SQL builder
// ---------------------------------------------------------------------------

/// Accumulates the typed literal values for a parameterised query.
///
/// Predicate values are never interpolated into the SQL text. Instead each
/// value is pushed here and the builder emits a positional placeholder
/// (`$1`, `$2`, …) referencing it. The collected [`ScalarValue`]s are bound
/// to the logical plan via [`DataFrame::with_param_values`], so user input
/// reaches the engine as typed scalars and can never alter the query
/// structure (no SQL injection surface, no escaping to get wrong).
#[derive(Default)]
struct Params {
    values: Vec<ScalarValue>,
}

impl Params {
    fn new() -> Self {
        Self::default()
    }

    /// Bind `v` and return its `$N` placeholder token.
    fn bind(&mut self, v: ScalarValue) -> String {
        self.values.push(v);
        format!("${}", self.values.len())
    }

    fn into_values(self) -> Vec<ScalarValue> {
        self.values
    }
}

fn build_query_sql(
    schema: &DatasetSchema,
    req: &QueryRequest,
    max_page_size: u64,
) -> Result<(String, Vec<ScalarValue>), AppError> {
    let (limit, offset) = req.effective_limit_offset(max_page_size);
    build_query_sql_with_suffix(schema, req, &format!(" LIMIT {limit} OFFSET {offset}"))
}

fn build_query_stream_sql(
    schema: &DatasetSchema,
    req: &QueryRequest,
) -> Result<(String, Vec<ScalarValue>), AppError> {
    let suffix = req
        .limit
        .map(|limit| format!(" LIMIT {limit}"))
        .unwrap_or_default();
    build_query_sql_with_suffix(schema, req, &suffix)
}

fn build_query_sql_with_suffix(
    schema: &DatasetSchema,
    req: &QueryRequest,
    suffix: &str,
) -> Result<(String, Vec<ScalarValue>), AppError> {
    let agg_plan = req.agg_plan(schema)?;

    let cols = if let Some(plan) = &agg_plan {
        // Group cols, then aggregations, each aliased to the JSON output key.
        let mut parts: Vec<String> = plan
            .group_cols
            .iter()
            .map(|c| DatasetSchema::quote_ident(c))
            .collect();
        for a in &plan.aggs {
            let expr = a.sql_expr()?;
            parts.push(format!(
                "{expr} AS {}",
                DatasetSchema::quote_ident(&a.alias)
            ));
        }
        parts.join(", ")
    } else if req.columns.is_empty() {
        if req.distinct {
            "DISTINCT *".to_string()
        } else {
            "*".to_string()
        }
    } else {
        let list = req
            .columns
            .iter()
            .map(|c| {
                schema
                    .find(c)
                    .map(|info| DatasetSchema::quote_ident(&info.name))
            })
            .collect::<Result<Vec<_>, _>>()?
            .join(", ");
        if req.distinct {
            format!("DISTINCT {list}")
        } else {
            list
        }
    };

    let mut params = Params::new();
    let clauses: Vec<String> = req
        .predicates
        .iter()
        .map(|p| pred_to_sql(schema, p, &mut params))
        .collect::<Result<_, _>>()?;

    let table = DatasetSchema::quote_ident(&schema.name);
    let where_clause = if clauses.is_empty() {
        String::new()
    } else {
        format!(" WHERE {}", clauses.join(" AND "))
    };
    let group_clause = match &agg_plan {
        Some(p) => format!(
            " GROUP BY {}",
            p.group_cols
                .iter()
                .map(|c| DatasetSchema::quote_ident(c))
                .collect::<Vec<_>>()
                .join(", "),
        ),
        None => String::new(),
    };
    let order_clause = match req.order_by_sql(schema, agg_plan.as_ref())? {
        Some(s) => format!(" ORDER BY {s}"),
        None => String::new(),
    };
    let sql =
        format!("SELECT {cols} FROM {table}{where_clause}{group_clause}{order_clause}{suffix}");
    Ok((sql, params.into_values()))
}

fn build_count_sql(
    schema: &DatasetSchema,
    predicates: &[Predicate],
) -> Result<(String, Vec<ScalarValue>), AppError> {
    let mut params = Params::new();
    let clauses: Vec<String> = predicates
        .iter()
        .map(|p| pred_to_sql(schema, p, &mut params))
        .collect::<Result<_, _>>()?;
    let table = DatasetSchema::quote_ident(&schema.name);
    let where_clause = if clauses.is_empty() {
        String::new()
    } else {
        format!(" WHERE {}", clauses.join(" AND "))
    };
    let sql = format!("SELECT COUNT(*) FROM {table}{where_clause}");
    Ok((sql, params.into_values()))
}

fn pred_to_sql(
    schema: &DatasetSchema,
    pred: &Predicate,
    params: &mut Params,
) -> Result<String, AppError> {
    let info = schema.find(&pred.col)?;
    let col = DatasetSchema::quote_ident(&info.name);

    match pred.op.as_str() {
        "is_null" => return Ok(format!("{col} IS NULL")),
        "is_not_null" => return Ok(format!("{col} IS NOT NULL")),
        _ => {}
    }

    let val = pred
        .val
        .as_ref()
        .ok_or_else(|| AppError::InvalidValue(format!("'{}' requires a value", pred.op)))?;

    if pred.op == "in" {
        let items = val
            .as_array()
            .filter(|a| !a.is_empty())
            .ok_or_else(|| AppError::InvalidValue("'in' needs a non-empty array".into()))?;
        let placeholders: Vec<String> = items
            .iter()
            .map(|item| Ok(params.bind(json_to_scalar(item)?)))
            .collect::<Result<_, AppError>>()?;
        return Ok(format!("{col} IN ({})", placeholders.join(", ")));
    }

    let sql_op = match pred.op.as_str() {
        "eq" => "=",
        "neq" => "!=",
        "gt" => ">",
        "gte" => ">=",
        "lt" => "<",
        "lte" => "<=",
        "like" => "LIKE",
        "ilike" => "ILIKE",
        other => return Err(AppError::UnknownOperator(other.into())),
    };
    let placeholder = params.bind(json_to_scalar(val)?);
    Ok(format!("{col} {sql_op} {placeholder}"))
}

/// Convert a JSON predicate value into a typed Arrow [`ScalarValue`] for
/// binding as a query parameter. The engine applies the usual numeric
/// widening / comparison coercion against the target column type.
fn json_to_scalar(val: &JsonValue) -> Result<ScalarValue, AppError> {
    match val {
        JsonValue::String(s) => Ok(ScalarValue::Utf8(Some(s.clone()))),
        JsonValue::Bool(b) => Ok(ScalarValue::Boolean(Some(*b))),
        JsonValue::Null => Ok(ScalarValue::Null),
        JsonValue::Number(n) => {
            if let Some(i) = n.as_i64() {
                Ok(ScalarValue::Int64(Some(i)))
            } else if let Some(u) = n.as_u64() {
                Ok(ScalarValue::UInt64(Some(u)))
            } else if let Some(f) = n.as_f64() {
                Ok(ScalarValue::Float64(Some(f)))
            } else {
                Err(AppError::InvalidValue(
                    "unsupported numeric literal in predicate".into(),
                ))
            }
        }
        _ => Err(AppError::InvalidValue(
            "unsupported literal type in predicate".into(),
        )),
    }
}

// ---------------------------------------------------------------------------
// Equality index — built once at startup, queried on every predicate request
// ---------------------------------------------------------------------------

fn json_index_key(val: &JsonValue) -> Option<String> {
    match val {
        JsonValue::String(s) => Some(s.clone()),
        JsonValue::Number(n) => Some(n.to_string()),
        JsonValue::Bool(b) => Some(b.to_string()),
        _ => None,
    }
}

fn intersect_sorted(a: &[u32], b: &[u32]) -> Vec<u32> {
    let mut out = Vec::new();
    let (mut i, mut j) = (0, 0);
    while i < a.len() && j < b.len() {
        match a[i].cmp(&b[j]) {
            Ordering::Equal => {
                out.push(a[i]);
                i += 1;
                j += 1;
            }
            Ordering::Less => i += 1,
            Ordering::Greater => j += 1,
        }
    }
    out
}

fn union_sorted(a: &[u32], b: &[u32]) -> Vec<u32> {
    let mut out = Vec::with_capacity(a.len() + b.len());
    let (mut i, mut j) = (0, 0);
    while i < a.len() && j < b.len() {
        match a[i].cmp(&b[j]) {
            Ordering::Less => {
                out.push(a[i]);
                i += 1;
            }
            Ordering::Greater => {
                out.push(b[j]);
                j += 1;
            }
            Ordering::Equal => {
                out.push(a[i]);
                i += 1;
                j += 1;
            }
        }
    }
    out.extend_from_slice(&a[i..]);
    out.extend_from_slice(&b[j..]);
    out
}

fn try_index(index: &EqIndex, predicates: &[Predicate]) -> Option<Vec<u32>> {
    if predicates.is_empty() || index.is_empty() {
        return None;
    }

    let mut result: Option<Vec<u32>> = None;
    for pred in predicates {
        let col_lower = pred.col.to_lowercase();
        let col_map = index.get(&col_lower)?;

        let rows: Vec<u32> = match pred.op.as_str() {
            "eq" => {
                let key = json_index_key(pred.val.as_ref()?)?;
                col_map.get(&key).cloned().unwrap_or_default()
            }
            "in" => {
                let items = pred.val.as_ref()?.as_array()?;
                let mut merged: Vec<u32> = Vec::new();
                for item in items {
                    if let Some(r) = col_map.get(&json_index_key(item)?) {
                        merged = union_sorted(&merged, r);
                    }
                }
                merged
            }
            _ => return None,
        };

        result = Some(match result {
            None => rows,
            Some(r) => intersect_sorted(&r, &rows),
        });
    }
    result
}

/// Return rows `[offset, offset+limit)` from a chunked dataset by slicing
/// the underlying batches (zero-copy) and concatenating the (small) page.
fn slice_global(
    chunks: &[RecordBatch],
    schema: &Arc<Schema>,
    offset: usize,
    limit: usize,
) -> Result<RecordBatch, AppError> {
    if limit == 0 || chunks.is_empty() {
        return Ok(RecordBatch::new_empty(schema.clone()));
    }
    let mut out = Vec::new();
    let mut to_skip = offset;
    let mut remaining = limit;
    for b in chunks {
        if remaining == 0 {
            break;
        }
        let n = b.num_rows();
        if to_skip >= n {
            to_skip -= n;
            continue;
        }
        let take = remaining.min(n - to_skip);
        out.push(b.slice(to_skip, take));
        to_skip = 0;
        remaining -= take;
    }
    if out.is_empty() {
        return Ok(RecordBatch::new_empty(schema.clone()));
    }
    compute::concat_batches(schema, out.iter()).map_err(AppError::from)
}

/// Materialise the page `rows[offset..offset+limit]` from a chunked dataset.
/// Row ids are global (across the concatenation of all chunks). We map each
/// requested row to its (chunk, local-index), `take` per chunk, then stitch
/// the per-chunk results back together preserving the original row order.
fn take_page(
    chunks: &[RecordBatch],
    schema: &Arc<Schema>,
    rows: &[u32],
    offset: usize,
    limit: usize,
) -> Result<RecordBatch, AppError> {
    let start = offset.min(rows.len());
    let len = limit.min(rows.len() - start);
    if len == 0 || chunks.is_empty() {
        return Ok(RecordBatch::new_empty(schema.clone()));
    }

    // Prefix-sum table: `offsets[i]` is the first global row id of chunk `i`,
    // and `offsets.last()` is the total row count.
    let mut offsets: Vec<u32> = Vec::with_capacity(chunks.len() + 1);
    let mut acc: u32 = 0;
    offsets.push(0);
    for b in chunks {
        acc = acc
            .checked_add(b.num_rows() as u32)
            .expect("row count exceeds u32::MAX");
        offsets.push(acc);
    }

    // Bucket each global row id into the chunk that contains it, remembering
    // the original output position so we can restore page order at the end.
    let mut buckets: Vec<Vec<(u32, u32)>> = (0..chunks.len()).map(|_| Vec::new()).collect();
    for (out_pos, &gid) in rows[start..start + len].iter().enumerate() {
        let bi = offsets.partition_point(|&x| x <= gid).saturating_sub(1);
        let local = gid - offsets[bi];
        buckets[bi].push((out_pos as u32, local));
    }

    // Per-chunk take, recording the destination index for each emitted row.
    let mut takens: Vec<RecordBatch> = Vec::new();
    let mut dest: Vec<u32> = Vec::with_capacity(len);
    for (bi, bucket) in buckets.iter().enumerate() {
        if bucket.is_empty() {
            continue;
        }
        let idx = UInt32Array::from(bucket.iter().map(|(_, l)| *l).collect::<Vec<u32>>());
        let cols: Vec<ArrayRef> = chunks[bi]
            .columns()
            .iter()
            .map(|c| {
                arrow::compute::take(c.as_ref(), &idx, None::<arrow::compute::TakeOptions>)
                    .map_err(AppError::from)
            })
            .collect::<Result<_, _>>()?;
        takens.push(RecordBatch::try_new(chunks[bi].schema(), cols)?);
        dest.extend(bucket.iter().map(|(out_pos, _)| *out_pos));
    }

    // Stitch per-chunk results then permute to restore the requested order.
    let stitched = compute::concat_batches(schema, takens.iter())?;
    let mut inv = vec![0u32; len];
    for (i, &d) in dest.iter().enumerate() {
        inv[d as usize] = i as u32;
    }
    let perm = UInt32Array::from(inv);
    let cols: Vec<ArrayRef> = stitched
        .columns()
        .iter()
        .map(|c| {
            arrow::compute::take(c.as_ref(), &perm, None::<arrow::compute::TakeOptions>)
                .map_err(AppError::from)
        })
        .collect::<Result<_, _>>()?;
    RecordBatch::try_new(stitched.schema(), cols).map_err(AppError::from)
}

/// Build the equality index per the dataset's policy, against the chunked
/// representation. Row ids are global across the concatenation of all
/// chunks (so they remain compatible with `take_page` / `slice_global`).
fn build_eq_index_with_policy(chunks: &[RecordBatch], cfg: &IndexConfig) -> EqIndex {
    use rayon::prelude::*;

    if cfg.mode == IndexMode::None || chunks.is_empty() {
        return EqIndex::new();
    }

    let allow: Option<HashMap<String, ()>> = if cfg.mode == IndexMode::List {
        Some(cfg.columns.iter().map(|c| (c.to_lowercase(), ())).collect())
    } else {
        None
    };

    let max_card = if cfg.mode == IndexMode::Auto {
        Some(cfg.max_cardinality)
    } else {
        None
    };

    // Per-chunk starting global row id.
    let mut batch_offsets: Vec<u32> = Vec::with_capacity(chunks.len());
    let mut acc: u32 = 0;
    for b in chunks {
        batch_offsets.push(acc);
        acc = acc
            .checked_add(b.num_rows() as u32)
            .expect("row count exceeds u32::MAX");
    }

    let schema = chunks[0].schema();

    schema
        .fields()
        .par_iter()
        .enumerate()
        .filter_map(|(ci, field)| {
            let col_lower = field.name().to_lowercase();
            if let Some(a) = &allow
                && !a.contains_key(&col_lower)
            {
                return None;
            }

            // Only build for index-friendly types; skip everything else
            // up-front so we don't pay the per-chunk dispatch cost.
            let dtype = field.data_type();
            let dict_utf8 = matches!(dtype,
                DataType::Dictionary(k, v)
                    if matches!(k.as_ref(), DataType::Int32)
                    && matches!(v.as_ref(), DataType::Utf8));
            match dtype {
                DataType::Utf8
                | DataType::Utf8View
                | DataType::Boolean
                | DataType::Int8
                | DataType::Int16
                | DataType::Int32
                | DataType::Int64 => {}
                _ if dict_utf8 => {}
                _ => return None,
            }

            let mut map: HashMap<String, Vec<u32>> = HashMap::new();

            for (bi, batch) in chunks.iter().enumerate() {
                let base = batch_offsets[bi];
                let col = batch.column(ci);

                macro_rules! index_col {
                    ($arr_ty:ty) => {{
                        let arr = col.as_any().downcast_ref::<$arr_ty>()?;
                        for row in 0..arr.len() {
                            if arr.is_null(row) {
                                continue;
                            }
                            let key = arr.value(row).to_string();
                            let gid = base + row as u32;
                            if let Some(v) = map.get_mut(&key) {
                                v.push(gid);
                            } else {
                                if let Some(mc) = max_card {
                                    if map.len() >= mc {
                                        return None;
                                    }
                                }
                                map.insert(key, vec![gid]);
                            }
                        }
                    }};
                }

                if dict_utf8 {
                    // Dictionary(Int32, Utf8): iterate keys + look up the
                    // string value from the (small) dictionary. We allocate
                    // the key string only when the value is new — repeated
                    // values reuse the existing HashMap entry by hash, but
                    // `HashMap::get_mut` still needs the key, so we use a
                    // borrowed lookup via `get` first to avoid the alloc.
                    let arr = col
                        .as_any()
                        .downcast_ref::<arrow::array::DictionaryArray<arrow::datatypes::Int32Type>>(
                        )?;
                    let keys = arr.keys();
                    let values = arr.values().as_any().downcast_ref::<StringArray>()?;
                    for row in 0..arr.len() {
                        if arr.is_null(row) {
                            continue;
                        }
                        let k = keys.value(row) as usize;
                        let s = values.value(k);
                        let gid = base + row as u32;
                        if let Some(v) = map.get_mut(s) {
                            v.push(gid);
                        } else {
                            if let Some(mc) = max_card
                                && map.len() >= mc
                            {
                                return None;
                            }
                            map.insert(s.to_string(), vec![gid]);
                        }
                    }
                } else {
                    match dtype {
                        DataType::Utf8 => index_col!(StringArray),
                        DataType::Utf8View => index_col!(StringViewArray),
                        DataType::Boolean => index_col!(BooleanArray),
                        DataType::Int8 => index_col!(Int8Array),
                        DataType::Int16 => index_col!(Int16Array),
                        DataType::Int32 => index_col!(Int32Array),
                        DataType::Int64 => index_col!(Int64Array),
                        _ => unreachable!(),
                    }
                }
            }

            Some((col_lower, map))
        })
        .collect()
}

// ---------------------------------------------------------------------------
// Serialise-time temporal cast: convert Timestamp/Date/Time columns to Utf8
// on the page batch right before JSON encoding. We deliberately do **not**
// pay this cost at load time — a `Date32` is 4 bytes per row, its ISO-8601
// rendering is ~10–24 bytes per row, and a wide dataset full of temporal
// columns would balloon resident RAM. The cast is applied per returned page
// after pagination, so the cost is paid only for rows the caller requested.
// ---------------------------------------------------------------------------

/// Returns true for Arrow types that `write_value` can render directly. Any
/// type returning false is pre-cast to Utf8 in [`cast_for_serialize`] so the
/// JSON output is faithful rather than silently `null`.
fn writable_inline(dt: &DataType) -> bool {
    match dt {
        DataType::Utf8
        | DataType::LargeUtf8
        | DataType::Utf8View
        | DataType::Boolean
        | DataType::Int8
        | DataType::Int16
        | DataType::Int32
        | DataType::Int64
        | DataType::UInt8
        | DataType::UInt16
        | DataType::UInt32
        | DataType::UInt64
        | DataType::Float32
        | DataType::Float64
        | DataType::Decimal128(_, _)
        | DataType::Decimal256(_, _) => true,
        DataType::Dictionary(k, v)
            if matches!(k.as_ref(), DataType::Int32) && matches!(v.as_ref(), DataType::Utf8) =>
        {
            true
        }
        _ => false,
    }
}

/// Cast any column whose dtype isn't directly writable by `write_value` to
/// `Utf8`, on the bounded page batch. Covers temporals (Timestamp/Date/Time)
/// — kept native in resident memory to save RAM — and also any exotic dtype
/// (Float16, Binary, List, Struct, Decimal-with-unsupported-precision, …)
/// so the JSON serializer never falls back to writing literal `null`.
fn cast_for_serialize(batch: &RecordBatch) -> Result<RecordBatch, AppError> {
    let schema = batch.schema();
    let to_cast: Vec<usize> = schema
        .fields()
        .iter()
        .enumerate()
        .filter_map(|(i, f)| {
            if writable_inline(f.data_type()) {
                None
            } else {
                Some(i)
            }
        })
        .collect();
    if to_cast.is_empty() {
        return Ok(batch.clone());
    }
    let new_fields: Vec<Field> = schema
        .fields()
        .iter()
        .enumerate()
        .map(|(i, f)| {
            if to_cast.contains(&i) {
                Field::new(f.name(), DataType::Utf8, f.is_nullable())
            } else {
                f.as_ref().clone()
            }
        })
        .collect();
    let new_schema = Arc::new(Schema::new(new_fields));
    let cols: Vec<ArrayRef> = batch
        .columns()
        .iter()
        .enumerate()
        .map(|(i, c)| {
            if to_cast.contains(&i) {
                compute::cast(c.as_ref(), &DataType::Utf8).map_err(AppError::from)
            } else {
                Ok(c.clone())
            }
        })
        .collect::<Result<_, _>>()?;
    RecordBatch::try_new(new_schema, cols).map_err(AppError::from)
}

// ---------------------------------------------------------------------------
// Compute helpers — retained for symmetry; reserved for future inline scan
// path. Currently the engine fallback handles all non-index queries.
// ---------------------------------------------------------------------------

#[allow(dead_code)]
#[derive(Clone, Copy)]
enum CmpOp {
    Eq,
    Neq,
    Gt,
    Gte,
    Lt,
    Lte,
    Like,
    ILike,
}

#[allow(dead_code)]
fn eq_str(col: &ArrayRef, val: &str) -> Result<BooleanArray, AppError> {
    let arr = col
        .as_any()
        .downcast_ref::<StringArray>()
        .ok_or_else(|| AppError::InvalidValue("equality: column is not a string".into()))?;
    let s = Scalar::new(StringArray::from(vec![val]));
    Ok(eq(arr, &s)?)
}

#[allow(dead_code)]
fn cmp_scalar(col: &ArrayRef, op: CmpOp, val: &JsonValue) -> Result<BooleanArray, AppError> {
    macro_rules! num_cmp {
        ($arr_type:ty, $cast:ty) => {{
            let n = val
                .as_f64()
                .ok_or_else(|| AppError::InvalidValue("expected number".into()))?
                as $cast;
            let arr = col.as_any().downcast_ref::<$arr_type>().unwrap();
            let s = Scalar::new(<$arr_type>::from(vec![n]));
            Ok(match op {
                CmpOp::Eq => eq(arr, &s)?,
                CmpOp::Neq => neq(arr, &s)?,
                CmpOp::Gt => gt(arr, &s)?,
                CmpOp::Gte => gt_eq(arr, &s)?,
                CmpOp::Lt => lt(arr, &s)?,
                CmpOp::Lte => lt_eq(arr, &s)?,
                CmpOp::Like | CmpOp::ILike => {
                    return Err(AppError::InvalidValue(
                        "LIKE requires a string column".into(),
                    ));
                }
            })
        }};
    }
    match col.data_type() {
        DataType::Utf8 => {
            let s = val
                .as_str()
                .ok_or_else(|| AppError::InvalidValue("expected string".into()))?;
            let arr = col.as_any().downcast_ref::<StringArray>().unwrap();
            let sc = Scalar::new(StringArray::from(vec![s]));
            Ok(match op {
                CmpOp::Eq => eq(arr, &sc)?,
                CmpOp::Neq => neq(arr, &sc)?,
                CmpOp::Gt => gt(arr, &sc)?,
                CmpOp::Gte => gt_eq(arr, &sc)?,
                CmpOp::Lt => lt(arr, &sc)?,
                CmpOp::Lte => lt_eq(arr, &sc)?,
                CmpOp::Like => compute::like(arr, &sc)?,
                CmpOp::ILike => compute::ilike(arr, &sc)?,
            })
        }
        DataType::Int8 => num_cmp!(Int8Array, i8),
        DataType::Int16 => num_cmp!(Int16Array, i16),
        DataType::Int32 => num_cmp!(Int32Array, i32),
        DataType::Int64 => num_cmp!(Int64Array, i64),
        DataType::Float32 => num_cmp!(Float32Array, f32),
        DataType::Float64 => num_cmp!(Float64Array, f64),
        dt => Err(AppError::InvalidValue(format!(
            "unsupported type for comparison: {dt:?}"
        ))),
    }
}

// ---------------------------------------------------------------------------
// Serialisation
// ---------------------------------------------------------------------------

pub fn serialize(batch: &RecordBatch) -> Result<String, AppError> {
    // Temporal columns are kept native in resident memory (compact). Cast
    // them — plus any other dtype `write_value` can't render directly — to
    // Utf8 here, on the bounded page batch, so the JSON output is faithful
    // without paying the load-time RAM cost.
    let batch = cast_for_serialize(batch)?;
    let schema = batch.schema();
    let n_rows = batch.num_rows();

    let keys: Vec<Vec<u8>> = schema
        .fields()
        .iter()
        .map(|f| {
            let mut k = Vec::with_capacity(f.name().len() + 3);
            k.push(b'"');
            k.extend_from_slice(f.name().as_bytes());
            k.extend_from_slice(b"\":");
            k
        })
        .collect();

    let mut buf: Vec<u8> = Vec::with_capacity(n_rows.max(1) * 300);
    buf.push(b'[');

    for row in 0..n_rows {
        if row > 0 {
            buf.push(b',');
        }
        buf.push(b'{');
        for (i, key) in keys.iter().enumerate() {
            if i > 0 {
                buf.push(b',');
            }
            buf.extend_from_slice(key);
            let col = batch.column(i);
            if col.is_null(row) {
                buf.extend_from_slice(b"null");
            } else {
                write_value(&mut buf, col.as_ref(), row);
            }
        }
        buf.push(b'}');
    }

    buf.push(b']');
    Ok(unsafe { String::from_utf8_unchecked(buf) })
}

#[inline]
fn write_value(buf: &mut Vec<u8>, col: &dyn Array, row: usize) {
    match col.data_type() {
        DataType::Utf8 => write_str(
            buf,
            col.as_any()
                .downcast_ref::<StringArray>()
                .unwrap()
                .value(row),
        ),
        DataType::LargeUtf8 => write_str(
            buf,
            col.as_any()
                .downcast_ref::<LargeStringArray>()
                .unwrap()
                .value(row),
        ),
        DataType::Utf8View => write_str(
            buf,
            col.as_any()
                .downcast_ref::<StringViewArray>()
                .unwrap()
                .value(row),
        ),
        DataType::Dictionary(key, value)
            if matches!(key.as_ref(), DataType::Int32)
                && matches!(value.as_ref(), DataType::Utf8) =>
        {
            let dict = col
                .as_any()
                .downcast_ref::<arrow::array::DictionaryArray<arrow::datatypes::Int32Type>>()
                .unwrap();
            let keys = dict.keys();
            let values = dict
                .values()
                .as_any()
                .downcast_ref::<StringArray>()
                .unwrap();
            let k = keys.value(row) as usize;
            write_str(buf, values.value(k));
        }
        DataType::Boolean => {
            let v = col
                .as_any()
                .downcast_ref::<BooleanArray>()
                .unwrap()
                .value(row);
            buf.extend_from_slice(if v { b"true" } else { b"false" });
        }
        DataType::Int8 => {
            let mut b = itoa::Buffer::new();
            buf.extend_from_slice(
                b.format(col.as_any().downcast_ref::<Int8Array>().unwrap().value(row))
                    .as_bytes(),
            );
        }
        DataType::Int16 => {
            let mut b = itoa::Buffer::new();
            buf.extend_from_slice(
                b.format(
                    col.as_any()
                        .downcast_ref::<Int16Array>()
                        .unwrap()
                        .value(row),
                )
                .as_bytes(),
            );
        }
        DataType::Int32 => {
            let mut b = itoa::Buffer::new();
            buf.extend_from_slice(
                b.format(
                    col.as_any()
                        .downcast_ref::<Int32Array>()
                        .unwrap()
                        .value(row),
                )
                .as_bytes(),
            );
        }
        DataType::Int64 => {
            let mut b = itoa::Buffer::new();
            buf.extend_from_slice(
                b.format(
                    col.as_any()
                        .downcast_ref::<Int64Array>()
                        .unwrap()
                        .value(row),
                )
                .as_bytes(),
            );
        }
        DataType::UInt8 => {
            let mut b = itoa::Buffer::new();
            buf.extend_from_slice(
                b.format(
                    col.as_any()
                        .downcast_ref::<UInt8Array>()
                        .unwrap()
                        .value(row),
                )
                .as_bytes(),
            );
        }
        DataType::UInt16 => {
            let mut b = itoa::Buffer::new();
            buf.extend_from_slice(
                b.format(
                    col.as_any()
                        .downcast_ref::<UInt16Array>()
                        .unwrap()
                        .value(row),
                )
                .as_bytes(),
            );
        }
        DataType::UInt32 => {
            let mut b = itoa::Buffer::new();
            buf.extend_from_slice(
                b.format(
                    col.as_any()
                        .downcast_ref::<UInt32Array>()
                        .unwrap()
                        .value(row),
                )
                .as_bytes(),
            );
        }
        DataType::UInt64 => {
            let mut b = itoa::Buffer::new();
            buf.extend_from_slice(
                b.format(
                    col.as_any()
                        .downcast_ref::<UInt64Array>()
                        .unwrap()
                        .value(row),
                )
                .as_bytes(),
            );
        }
        DataType::Decimal128(_, _) => {
            let arr = col.as_any().downcast_ref::<Decimal128Array>().unwrap();
            write_str(buf, &arr.value_as_string(row));
        }
        DataType::Decimal256(_, _) => {
            let arr = col.as_any().downcast_ref::<Decimal256Array>().unwrap();
            write_str(buf, &arr.value_as_string(row));
        }
        DataType::Float32 => {
            let v = col
                .as_any()
                .downcast_ref::<Float32Array>()
                .unwrap()
                .value(row);
            if v.is_finite() {
                let mut b = ryu::Buffer::new();
                buf.extend_from_slice(b.format_finite(v).as_bytes());
            } else {
                buf.extend_from_slice(b"null");
            }
        }
        DataType::Float64 => {
            let v = col
                .as_any()
                .downcast_ref::<Float64Array>()
                .unwrap()
                .value(row);
            if v.is_finite() {
                let mut b = ryu::Buffer::new();
                buf.extend_from_slice(b.format_finite(v).as_bytes());
            } else {
                buf.extend_from_slice(b"null");
            }
        }
        // Any dtype not handled above must have been pre-cast to Utf8 by
        // `cast_for_serialize`. Hitting this arm is a bug — surface it as a
        // visible JSON string rather than a silent null so it can't be
        // mistaken for a real NULL value.
        other => write_str(buf, &format!("<unsupported dtype: {other:?}>")),
    }
}

#[inline]
fn write_str(buf: &mut Vec<u8>, s: &str) {
    buf.push(b'"');
    for &byte in s.as_bytes() {
        match byte {
            b'"' => buf.extend_from_slice(b"\\\""),
            b'\\' => buf.extend_from_slice(b"\\\\"),
            b'\n' => buf.extend_from_slice(b"\\n"),
            b'\r' => buf.extend_from_slice(b"\\r"),
            b'\t' => buf.extend_from_slice(b"\\t"),
            0x00..=0x1f => {
                buf.extend_from_slice(b"\\u00");
                const HEX: &[u8] = b"0123456789abcdef";
                buf.push(HEX[(byte >> 4) as usize]);
                buf.push(HEX[(byte & 0xf) as usize]);
            }
            b => buf.push(b),
        }
    }
    buf.push(b'"');
}

// ---------------------------------------------------------------------------
// Backend trait impl — wires the store into the generic core handlers.
// ---------------------------------------------------------------------------

#[async_trait]
impl Backend for Store {
    fn names(&self) -> Vec<String> {
        Store::names(self)
    }

    fn summary(&self, name: &str) -> Result<DatasetSummary, AppError> {
        let st = self.dataset(name)?;
        Ok(DatasetSummary {
            name: st.schema.name.clone(),
            columns: st.schema.columns.len(),
            rows: st.num_rows(),
        })
    }

    fn schema(&self, name: &str) -> Result<Arc<DatasetSchema>, AppError> {
        let st = self.dataset(name)?;
        Ok(Arc::new(st.schema.clone()))
    }

    fn indexed_columns(&self, name: &str) -> Result<Vec<String>, AppError> {
        let st = self.dataset(name)?;
        // Report indexed columns in the dataset's declared schema order
        // so the `/schema` response is deterministic.
        let mut cols: Vec<String> = st
            .schema
            .columns
            .iter()
            .map(|c| c.name.clone())
            .filter(|n| st.index.contains_key(n))
            .collect();
        // Any indexed columns not in `schema.columns` (shouldn't happen,
        // but be defensive) get appended sorted.
        let mut extras: Vec<String> = st
            .index
            .keys()
            .filter(|n| !cols.iter().any(|c| c == *n))
            .cloned()
            .collect();
        extras.sort();
        cols.extend(extras);
        Ok(cols)
    }

    async fn sample(&self, name: &str) -> Result<String, AppError> {
        Store::sample(self, name).await
    }

    async fn query(&self, name: &str, req: &QueryRequest) -> Result<String, AppError> {
        Store::query(self, name, req).await
    }

    async fn query_arrow(&self, name: &str, req: &QueryRequest) -> Result<Vec<u8>, AppError> {
        Store::query_arrow(self, name, req).await
    }

    async fn query_arrow_stream(
        &self,
        name: &str,
        req: &QueryRequest,
    ) -> Result<ArrowIpcStream, AppError> {
        Store::query_arrow_stream(self, name, req).await
    }

    async fn query_arrow_stream_all(
        &self,
        name: &str,
        req: &QueryRequest,
    ) -> Result<ArrowIpcStream, AppError> {
        Store::query_arrow_stream_all(self, name, req).await
    }

    async fn count(&self, name: &str, req: &CountRequest) -> Result<i64, AppError> {
        Store::count(self, name, req).await
    }

    async fn parquet(&self, name: &str) -> Result<bytes::Bytes, AppError> {
        Store::parquet(self, name).await
    }

    async fn reload(&self, name: &str) -> Result<ReloadStats, AppError> {
        Store::reload(self, name).await
    }
}