tonbo 0.4.0-a1

Embedded database for serverless and edge runtimes, storing data as Parquet on S3
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
use std::{
    collections::{BTreeMap, BTreeSet},
    pin::Pin,
    sync::Arc,
    time::Instant,
};

use aisle::PruneRequest;
use arrow_array::RecordBatch;
use arrow_schema::{DataType, Schema, SchemaRef};
use fusio::{
    DynFs,
    executor::{Executor, Mutex, Timer},
};
use fusio_parquet::reader::AsyncReader;
use futures::{Stream, StreamExt, TryStreamExt, stream};
use parquet::{
    arrow::{
        ProjectionMask,
        arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions},
    },
    errors::ParquetError,
    file::metadata::{PageIndexPolicy, ParquetMetaDataReader},
};
use typed_arrow_dyn::DynRow;

use crate::{
    db::DbInner,
    extractor::{KeyExtractError, KeyProjection, projection_for_columns},
    inmem::{
        immutable::{self, ImmutableSegment, memtable::ImmutableVisibleEntry},
        mutable::memtable::DynRowScanEntry,
    },
    key::{KeyOwned, KeyRow},
    mode::fingerprint_schema,
    mutation::DynMutation,
    mvcc::{MVCC_COMMIT_COL, Timestamp},
    ondisk::{
        bloom::{BatchedAsyncReader, BloomFilterCache, SstBloomFilterProvider},
        metadata::ParquetMetadataCache,
        scan::{DeleteStreamWithExtractor, SstableScan, UnpinExec},
        sstable::{
            ParquetStreamOptions, SsTableError, open_parquet_stream_with_metadata,
            split_predicate_for_row_filter, storage_path_from_manifest, validate_page_indexes,
        },
    },
    query::{
        Expr, ScalarValue,
        scan::{
            DeleteSelection, RowSet, ScanPlan, ScanSelection, SstScanSelection, SstSelection,
            key_bounds_for_predicate, key_range_for_predicate, next_prefix_string,
            projection_with_predicate,
        },
        stream::{
            Order, OwnedImmutableScan, OwnedMutableScan, ScanStream, merge::MergeStream,
            package::PackageStream,
        },
    },
    transaction::{Snapshot as TxSnapshot, TransactionScan},
};

async fn parquet_metadata_cached<E>(
    fs: Arc<dyn DynFs>,
    path: &fusio::path::Path,
    cache: Arc<E::Mutex<ParquetMetadataCache>>,
    executor: E,
) -> Result<Arc<parquet::file::metadata::ParquetMetaData>, SsTableError>
where
    E: Executor + Clone + 'static,
{
    if let Some(cached) = {
        let guard = cache.lock().await;
        guard.get(path)
    } {
        return Ok(cached);
    }

    let file = fs.open(path).await.map_err(SsTableError::Fs)?;
    let size = file.size().await.map_err(SsTableError::Fs)?;
    let mut reader = AsyncReader::new(file, size, UnpinExec(executor))
        .await
        .map_err(SsTableError::Fs)?;
    let metadata = ParquetMetaDataReader::new()
        .with_page_index_policy(PageIndexPolicy::Optional)
        .load_and_finish(&mut reader, size)
        .await
        .map_err(SsTableError::Parquet)?;
    validate_page_indexes(path, &metadata)?;
    let metadata = Arc::new(metadata);

    let mut guard = cache.lock().await;
    guard.insert(path, Arc::clone(&metadata));
    Ok(metadata)
}

async fn pruner_for_schema<E>(
    cache: Arc<E::Mutex<crate::db::PrunerCache>>,
    schema: &SchemaRef,
) -> Option<Arc<aisle::Pruner>>
where
    E: Executor + Clone + 'static,
{
    let fingerprint = fingerprint_schema(schema);
    let mut guard = cache.lock().await;
    if let Some(pruner) = guard.get(&fingerprint) {
        return Some(Arc::clone(pruner));
    }
    let pruner = Arc::new(aisle::Pruner::try_new(Arc::clone(schema)).ok()?);
    guard.insert(fingerprint, Arc::clone(&pruner));
    Some(pruner)
}

struct PruneContext<E>
where
    E: Executor + Clone + 'static,
{
    fs: Arc<dyn DynFs>,
    metadata_cache: Arc<E::Mutex<ParquetMetadataCache>>,
    bloom_cache: Arc<E::Mutex<BloomFilterCache>>,
    pruner_cache: Arc<E::Mutex<crate::db::PrunerCache>>,
    executor: E,
}

/// Default package size for a scan
pub const DEFAULT_SCAN_BATCH_ROWS: usize = 1024;

/// Setup-phase timing breakdown for a single scan stream construction.
///
/// Values are measured in nanoseconds and cover setup work before consuming
/// the first output batch.
#[derive(Debug, Clone, Copy, Default)]
pub struct ScanSetupProfile {
    snapshot_ns: u64,
    plan_scan_ns: u64,
    build_scan_streams_ns: u64,
    merge_init_ns: u64,
    package_init_ns: u64,
}

impl ScanSetupProfile {
    /// Time spent resolving snapshot/read timestamp.
    pub fn snapshot_ns(&self) -> u64 {
        self.snapshot_ns
    }

    /// Time spent building the scan plan (predicate/projection pruning).
    pub fn plan_scan_ns(&self) -> u64 {
        self.plan_scan_ns
    }

    /// Time spent constructing scan streams from the plan.
    pub fn build_scan_streams_ns(&self) -> u64 {
        self.build_scan_streams_ns
    }

    /// Time spent initializing the merge stream.
    pub fn merge_init_ns(&self) -> u64 {
        self.merge_init_ns
    }

    /// Time spent initializing the package stream.
    pub fn package_init_ns(&self) -> u64 {
        self.package_init_ns
    }
}

impl TxSnapshot {
    /// Plan a scan using this snapshot for MVCC visibility and manifest pinning.
    pub(crate) async fn plan_scan<FS, E>(
        &self,
        db: &DbInner<FS, E>,
        predicate: &Expr,
        projected_schema: Option<&SchemaRef>,
        limit: Option<usize>,
    ) -> Result<ScanPlan, crate::db::DBError>
    where
        FS: crate::manifest::ManifestFs<E>,
        E: Executor + Timer + Clone,
        <FS as fusio::fs::Fs>::File: fusio::durability::FileCommit,
    {
        self.plan_scan_with_context(db, predicate, projected_schema, limit, false)
            .await
    }

    async fn plan_scan_with_context<FS, E>(
        &self,
        db: &DbInner<FS, E>,
        predicate: &Expr,
        projected_schema: Option<&SchemaRef>,
        limit: Option<usize>,
        force_predicate_columns: bool,
    ) -> Result<ScanPlan, crate::db::DBError>
    where
        FS: crate::manifest::ManifestFs<E>,
        E: Executor + Timer + Clone,
        <FS as fusio::fs::Fs>::File: fusio::durability::FileCommit,
    {
        if let Some(column) = find_bloom_filter_column(predicate) {
            return Err(crate::db::DBError::UnsupportedPredicate {
                reason: format!(
                    "bloom filter predicates are not supported yet (column '{column}')"
                ),
            });
        }
        let projected_schema = projected_schema.cloned();
        let projection_schema = projected_schema.as_ref().unwrap_or(&db.schema);
        let predicate_scan_schema =
            projection_with_predicate(&db.schema, projection_schema, Some(predicate))?;
        let split = split_predicate_for_row_filter(predicate, &predicate_scan_schema);
        let mut pushdown_predicate = split.pushdown;
        let mut residual_predicate = split.residual;
        let read_ts = self.read_view().read_ts();
        let key_schema = db.extractor().key_schema();
        let key_bounds = key_bounds_for_predicate(predicate, &key_schema);
        let key_range = key_range_for_predicate(predicate, &key_schema);
        let (immutable_indexes, immutable_row_sets) = {
            let seal = db.seal_state_lock();
            let prune_input: Vec<&ImmutableSegment> =
                seal.immutables.iter().map(|arc| arc.as_ref()).collect();
            let indexes = immutable::prune_segments(&prune_input, key_bounds.as_ref(), read_ts);
            let row_sets = indexes
                .iter()
                .filter_map(|idx| seal.immutables.get(*idx))
                .map(|segment| RowSet::all(segment.entry_count()))
                .collect();
            (indexes, row_sets)
        };
        let mutable_row_set = {
            let row_count = db.mem.row_count();
            if row_count == 0 {
                RowSet::all(0)
            } else {
                let (min_key, max_key) = db.mem.key_bounds();
                let mut keep = true;
                if let (Some(min_key), Some(max_key)) = (min_key.as_ref(), max_key.as_ref())
                    && let Some(bounds) = key_bounds.as_ref()
                    && !bounds.overlaps(min_key, max_key)
                {
                    keep = false;
                }
                if keep
                    && let Some((min_commit_ts, _)) = db.mem.commit_ts_bounds()
                    && min_commit_ts > read_ts
                {
                    keep = false;
                }
                if keep {
                    RowSet::all(row_count)
                } else {
                    RowSet::none(row_count)
                }
            }
        };
        let fs = Arc::clone(&db.fs);
        let executor: E = (**db.executor()).clone();
        let prune_ctx = PruneContext {
            fs: Arc::clone(&fs),
            metadata_cache: db.metadata_cache(),
            bloom_cache: db.bloom_cache(),
            pruner_cache: db.pruner_cache(),
            executor: executor.clone(),
        };
        let has_non_sst = !mutable_row_set.is_empty() || !immutable_indexes.is_empty();
        let non_sst_residual = if force_predicate_columns && !matches!(predicate, Expr::True) {
            Some(predicate.clone())
        } else if has_non_sst {
            split_predicate_for_non_sst(predicate, &key_schema).residual
        } else {
            None
        };
        residual_predicate = combine_predicates_with_and(residual_predicate, non_sst_residual);
        let mut sst_selections = Vec::new();
        let mut sst_requires_full_residual = false;
        for entry in self
            .table_snapshot()
            .latest_version
            .as_ref()
            .map(|v| v.ssts())
            .unwrap_or(&[])
            .iter()
            .flatten()
        {
            if let Some(bounds) = key_bounds.as_ref() {
                if bounds.is_empty() {
                    continue;
                }
                if let Some(stats) = entry.stats()
                    && let (Some(min_key), Some(max_key)) =
                        (stats.min_key.as_ref(), stats.max_key.as_ref())
                    && !bounds.overlaps(min_key, max_key)
                {
                    continue;
                }
            }
            if let Some(min_commit_ts) = entry.stats().and_then(|stats| stats.min_commit_ts)
                && min_commit_ts > read_ts
            {
                continue;
            }
            let (mut selection, requires_residual) = prune_sst_selection(
                &prune_ctx,
                &storage_path_from_manifest(&db.sst_root, entry.data_path()),
                predicate,
                read_ts,
                &predicate_scan_schema,
                &key_schema,
            )
            .await?;
            let has_delete_sidecar = entry.delete_path().is_some();
            if selection.row_set.is_empty() && !has_delete_sidecar {
                continue;
            }
            if requires_residual {
                sst_requires_full_residual = true;
            }
            if let Some(delete_path) = entry.delete_path() {
                let delete_selection = plan_delete_sidecar_selection(
                    Arc::clone(&prune_ctx.fs),
                    &storage_path_from_manifest(&db.sst_root, delete_path),
                    &key_schema,
                    Arc::clone(&prune_ctx.metadata_cache),
                    prune_ctx.executor.clone(),
                )
                .await?;
                selection.delete_selection = Some(delete_selection);
            }
            sst_selections.push(SstScanSelection {
                entry: entry.clone(),
                selection: ScanSelection::Sst(selection),
            });
        }
        if sst_requires_full_residual && !matches!(predicate, Expr::True) {
            // Keep the full predicate for post-merge filtering when any SST
            // schema cannot accept the pushdown portion.
            residual_predicate = Some(predicate.clone());
        }
        if sst_selections.is_empty() {
            pushdown_predicate = None;
        }
        let needs_predicate_columns = force_predicate_columns || residual_predicate.is_some();
        let scan_schema = if needs_predicate_columns {
            Arc::clone(&predicate_scan_schema)
        } else {
            Arc::clone(projection_schema)
        };
        if !needs_predicate_columns && scan_schema.as_ref() != predicate_scan_schema.as_ref() {
            for sst in &mut sst_selections {
                if let ScanSelection::Sst(selection) = &mut sst.selection {
                    reproject_sst_selection(selection, &scan_schema, &key_schema)?;
                }
            }
        }
        Ok(ScanPlan {
            pushdown_predicate,
            immutable_indexes,
            mutable_row_set,
            immutable_row_sets,
            mutable_selection: key_range
                .clone()
                .map(ScanSelection::KeyRange)
                .unwrap_or(ScanSelection::AllRows),
            immutable_selection: key_range
                .map(ScanSelection::KeyRange)
                .unwrap_or(ScanSelection::AllRows),
            sst_selections,
            residual_predicate,
            projected_schema,
            scan_schema,
            limit,
            read_ts,
            _snapshot: self.table_snapshot().clone(),
        })
    }
}

async fn prune_sst_selection<E>(
    ctx: &PruneContext<E>,
    data_path: &fusio::path::Path,
    predicate: &Expr,
    read_ts: Timestamp,
    scan_schema: &SchemaRef,
    key_schema: &SchemaRef,
) -> Result<(SstSelection, bool), SsTableError>
where
    E: Executor + Clone + 'static,
{
    let metadata = parquet_metadata_cached(
        Arc::clone(&ctx.fs),
        data_path,
        Arc::clone(&ctx.metadata_cache),
        ctx.executor.clone(),
    )
    .await?;
    let options = ArrowReaderOptions::new().with_page_index(true);
    let arrow_metadata = ArrowReaderMetadata::try_new(Arc::clone(&metadata), options)
        .map_err(SsTableError::Parquet)?;
    let schema = arrow_metadata.schema();
    let requires_residual = split_predicate_for_row_filter(predicate, schema)
        .residual
        .is_some();
    let commit_predicate = Expr::lt_eq(MVCC_COMMIT_COL, ScalarValue::UInt64(Some(read_ts.get())));
    let prune_predicate = if matches!(predicate, Expr::True) {
        commit_predicate
    } else {
        Expr::and(vec![predicate.clone(), commit_predicate])
    };
    let bloom_file = ctx.fs.open(data_path).await.map_err(SsTableError::Fs)?;
    let size = bloom_file.size().await.map_err(SsTableError::Fs)?;
    let reader = BatchedAsyncReader::new(bloom_file, size, ctx.executor.clone())
        .await
        .map_err(SsTableError::Fs)?;
    let mut provider = SstBloomFilterProvider::new(
        data_path.clone(),
        Arc::clone(&metadata),
        reader,
        Arc::clone(&ctx.bloom_cache),
    );
    let prune_result =
        if let Some(pruner) = pruner_for_schema::<E>(Arc::clone(&ctx.pruner_cache), schema).await {
            pruner
                .prune_ir_with_bloom_provider(
                    metadata.as_ref(),
                    std::slice::from_ref(&prune_predicate),
                    &mut provider,
                )
                .await
        } else {
            PruneRequest::new(metadata.as_ref(), schema.as_ref())
                .with_predicate(&prune_predicate)
                .enable_page_index(true)
                .enable_bloom_filter(true)
                .prune_async(&mut provider)
                .await
        };
    let mut row_groups = prune_result.row_groups().to_vec();
    // Preserve PK-ascending scan order by keeping row groups in file order.
    row_groups.sort_unstable();
    row_groups.dedup();
    let selected_row_groups_rows = if row_groups.is_empty() {
        0usize
    } else {
        row_groups
            .iter()
            .map(|idx| {
                usize::try_from(metadata.row_group(*idx).num_rows()).map_err(|_| {
                    SsTableError::Parquet(ParquetError::General(
                        "parquet row group count exceeds usize::MAX".to_string(),
                    ))
                })
            })
            .collect::<Result<Vec<_>, _>>()?
            .into_iter()
            .sum::<usize>()
    };
    let total_rows = selected_row_groups_rows;
    let row_set = match prune_result.row_selection().cloned() {
        Some(selection) => RowSet::from_row_selection(total_rows, selection).map_err(|err| {
            SsTableError::RowSelection {
                reason: err.to_string(),
            }
        })?,
        None => RowSet::all(total_rows),
    };
    let total_row_groups = metadata.num_row_groups();
    let row_groups = if row_groups.len() == total_row_groups {
        None
    } else {
        Some(row_groups)
    };

    let (projection, projected_schema) =
        sst_projection_from_metadata(&arrow_metadata, scan_schema, key_schema)?;

    Ok((
        SstSelection {
            row_groups,
            row_set,
            metadata,
            projection,
            projected_schema,
            delete_selection: None,
        },
        requires_residual,
    ))
}

fn sst_projection_from_metadata(
    arrow_metadata: &ArrowReaderMetadata,
    scan_schema: &SchemaRef,
    key_schema: &SchemaRef,
) -> Result<(ProjectionMask, SchemaRef), SsTableError> {
    let mut required = BTreeSet::new();
    for field in scan_schema.fields() {
        required.insert(field.name().to_string());
    }
    for field in key_schema.fields() {
        required.insert(field.name().to_string());
    }
    required.insert(MVCC_COMMIT_COL.to_string());

    let mut remaining = required;
    let mut projected_fields = Vec::new();
    let mut root_indices = Vec::new();
    for (idx, field) in arrow_metadata.schema().fields().iter().enumerate() {
        if remaining.remove(field.name()) {
            projected_fields.push(field.clone());
            root_indices.push(idx);
        }
    }

    if let Some(missing) = remaining.iter().next() {
        return Err(KeyExtractError::NoSuchField {
            name: missing.to_string(),
        }
        .into());
    }

    let projected_schema = Arc::new(Schema::new(projected_fields));
    let projection = ProjectionMask::roots(arrow_metadata.parquet_schema(), root_indices);
    Ok((projection, projected_schema))
}

fn reproject_sst_selection(
    selection: &mut SstSelection,
    scan_schema: &SchemaRef,
    key_schema: &SchemaRef,
) -> Result<(), SsTableError> {
    let options = ArrowReaderOptions::new().with_page_index(true);
    let arrow_metadata = ArrowReaderMetadata::try_new(Arc::clone(&selection.metadata), options)
        .map_err(SsTableError::Parquet)?;
    let (projection, projected_schema) =
        sst_projection_from_metadata(&arrow_metadata, scan_schema, key_schema)?;
    selection.projection = projection;
    selection.projected_schema = projected_schema;
    Ok(())
}

fn schema_projection_indices(
    base_schema: &SchemaRef,
    target_schema: &SchemaRef,
) -> Result<Vec<usize>, KeyExtractError> {
    let mut indices = Vec::with_capacity(target_schema.fields().len());
    for field in target_schema.fields() {
        let Some((idx, _)) = base_schema
            .fields()
            .iter()
            .enumerate()
            .find(|(_, candidate)| candidate.name() == field.name())
        else {
            return Err(KeyExtractError::NoSuchField {
                name: field.name().to_string(),
            });
        };
        indices.push(idx);
    }
    Ok(indices)
}

async fn plan_delete_sidecar_selection<E>(
    fs: Arc<dyn DynFs>,
    delete_path: &fusio::path::Path,
    key_schema: &SchemaRef,
    metadata_cache: Arc<E::Mutex<ParquetMetadataCache>>,
    executor: E,
) -> Result<DeleteSelection, SsTableError>
where
    E: Executor + Clone + 'static,
{
    let metadata = parquet_metadata_cached(fs, delete_path, metadata_cache, executor).await?;
    let options = ArrowReaderOptions::new().with_page_index(true);
    let arrow_metadata = ArrowReaderMetadata::try_new(Arc::clone(&metadata), options)
        .map_err(SsTableError::Parquet)?;
    let file_schema = arrow_metadata.schema();
    let parquet_schema = arrow_metadata.parquet_schema();

    let mut required = BTreeSet::new();
    for field in key_schema.fields() {
        required.insert(field.name().to_string());
    }
    required.insert(MVCC_COMMIT_COL.to_string());

    let mut remaining = required;
    let mut root_indices = Vec::new();
    for (idx, field) in file_schema.fields().iter().enumerate() {
        if remaining.remove(field.name()) {
            root_indices.push(idx);
        }
    }

    if let Some(missing) = remaining.iter().next() {
        return Err(KeyExtractError::NoSuchField {
            name: missing.to_string(),
        }
        .into());
    }

    let projection = ProjectionMask::roots(parquet_schema, root_indices);
    Ok(DeleteSelection {
        metadata,
        projection,
    })
}

impl<FS, E> DbInner<FS, E>
where
    FS: crate::manifest::ManifestFs<E>,
    E: Executor + Timer + Clone,
    <FS as fusio::fs::Fs>::File: fusio::durability::FileCommit,
{
    /// Execute the scan plan with MVCC visibility
    #[cfg(test)]
    pub(crate) async fn execute_scan<'a>(
        &'a self,
        plan: ScanPlan,
    ) -> Result<impl Stream<Item = Result<RecordBatch, crate::db::DBError>> + 'a, crate::db::DBError>
    {
        let result_projection = plan
            .projected_schema
            .clone()
            .unwrap_or_else(|| Arc::clone(&self.schema));
        let scan_schema = Arc::clone(&plan.scan_schema);
        let streams = self.build_scan_streams(&plan, None).await?;

        if streams.is_empty() {
            let stream = stream::empty::<Result<RecordBatch, crate::db::DBError>>();
            return Ok(Box::pin(stream)
                as Pin<
                    Box<dyn Stream<Item = Result<RecordBatch, crate::db::DBError>> + 'a>,
                >);
        }

        let ScanPlan {
            residual_predicate,
            limit,
            ..
        } = plan;
        let needs_post_filter = residual_predicate.is_some();
        let limit_for_merge = if needs_post_filter { None } else { limit };
        let limit_for_package = if needs_post_filter { limit } else { None };
        let merge = MergeStream::from_vec(streams, limit_for_merge, Some(Order::Asc))
            .await
            .map_err(crate::db::DBError::from)?;
        let package = PackageStream::with_limit(
            DEFAULT_SCAN_BATCH_ROWS,
            merge,
            Arc::clone(&scan_schema),
            Arc::clone(&result_projection),
            residual_predicate,
            limit_for_package,
        )
        .map_err(crate::db::DBError::from)?;

        let mapped = package.map(|result| result.map_err(crate::db::DBError::from));
        Ok(Box::pin(mapped)
            as Pin<
                Box<dyn Stream<Item = Result<RecordBatch, crate::db::DBError>> + 'a>,
            >)
    }

    pub(crate) async fn build_scan_streams<'a>(
        &'a self,
        plan: &ScanPlan,
        txn_scan: Option<TransactionScan<'a>>,
    ) -> Result<Vec<ScanStream<'a, E>>, crate::db::DBError> {
        let mut streams = Vec::new();
        if let Some(txn_scan) = txn_scan {
            streams.push(ScanStream::from(txn_scan));
        }
        let scan_schema = Arc::clone(&plan.scan_schema);
        let key_schema = self.extractor().key_schema();
        if !plan.mutable_row_set.is_empty() {
            let mutable_scan = match &plan.mutable_selection {
                ScanSelection::AllRows | ScanSelection::Sst(_) => OwnedMutableScan::from_guard(
                    self.mem.read(),
                    Some(Arc::clone(&scan_schema)),
                    plan.read_ts,
                )?,
                ScanSelection::KeyRange(range) => OwnedMutableScan::from_guard_range(
                    self.mem.read(),
                    Some(Arc::clone(&scan_schema)),
                    plan.read_ts,
                    range.start.clone(),
                    range.end.clone(),
                )?,
            };
            streams.push(ScanStream::from(mutable_scan));
        }

        let immutables: Vec<Arc<ImmutableSegment>> = {
            let seal = self.seal_state_lock();
            plan.immutable_indexes
                .iter()
                .filter_map(|idx| seal.immutables.get(*idx).cloned())
                .collect()
        };
        for segment in immutables {
            let owned = match &plan.immutable_selection {
                ScanSelection::AllRows | ScanSelection::Sst(_) => OwnedImmutableScan::from_arc(
                    Arc::clone(&segment),
                    Some(Arc::clone(&scan_schema)),
                    plan.read_ts,
                )?,
                ScanSelection::KeyRange(range) => OwnedImmutableScan::from_arc_range(
                    Arc::clone(&segment),
                    Some(Arc::clone(&scan_schema)),
                    plan.read_ts,
                    range.start.clone(),
                    range.end.clone(),
                )?,
            };
            streams.push(ScanStream::from(owned));
        }

        // Add SSTable scans for each SST entry in the plan
        for sst in plan.sst_selections() {
            let selection = match &sst.selection {
                ScanSelection::Sst(selection) => selection,
                ScanSelection::AllRows => {
                    return Err(crate::db::DBError::SsTable(
                        SsTableError::InvalidScanSelection {
                            selection: "AllRows",
                        },
                    ));
                }
                ScanSelection::KeyRange(_) => {
                    return Err(crate::db::DBError::SsTable(
                        SsTableError::InvalidScanSelection {
                            selection: "KeyRange",
                        },
                    ));
                }
            };
            let data_path = storage_path_from_manifest(&self.sst_root, sst.entry.data_path());
            let executor: E = (**self.executor()).clone();

            let projected_schema = Arc::clone(&selection.projected_schema);
            let projection_indices = schema_projection_indices(&projected_schema, &scan_schema)?;
            let key_indices = schema_projection_indices(&projected_schema, &key_schema)?;
            let data_extractor: Arc<dyn KeyProjection> =
                projection_for_columns(projected_schema, key_indices)?.into();

            let options = ParquetStreamOptions {
                projection: Some(selection.projection.clone()),
                row_groups: selection.row_groups.clone(),
                row_selection: selection.row_set.to_row_selection(),
                row_filter_predicate: plan.pushdown_predicate.as_ref(),
            };
            let data_stream = open_parquet_stream_with_metadata(
                Arc::clone(&self.fs),
                data_path,
                Arc::clone(&selection.metadata),
                options,
                executor.clone(),
            )
            .await
            .map_err(crate::db::DBError::SsTable)?;

            // Open delete sidecar stream if present (streaming merge, no eager loading)
            let delete_stream_with_extractor = if let Some(delete_path) = sst.entry.delete_path() {
                let delete_selection = selection.delete_selection.as_ref().ok_or_else(|| {
                    crate::db::DBError::SsTable(SsTableError::InvalidScanSelection {
                        selection: "missing delete sidecar selection",
                    })
                })?;
                let delete_path = storage_path_from_manifest(&self.sst_root, delete_path);
                let options = ParquetStreamOptions {
                    projection: Some(delete_selection.projection.clone()),
                    row_groups: None,
                    row_selection: None,
                    row_filter_predicate: None,
                };
                let stream = open_parquet_stream_with_metadata(
                    Arc::clone(&self.fs),
                    delete_path,
                    Arc::clone(&delete_selection.metadata),
                    options,
                    executor.clone(),
                )
                .await
                .map_err(crate::db::DBError::SsTable)?;
                Some(DeleteStreamWithExtractor {
                    stream,
                    extractor: Arc::clone(self.delete_extractor()),
                })
            } else {
                if selection.delete_selection.is_some() {
                    return Err(crate::db::DBError::SsTable(
                        SsTableError::InvalidScanSelection {
                            selection: "unexpected delete sidecar selection",
                        },
                    ));
                }
                None
            };

            let sstable_scan = SstableScan::new(
                data_stream,
                delete_stream_with_extractor,
                data_extractor,
                projection_indices,
                Some(Order::Asc),
                plan.read_ts,
            );

            streams.push(ScanStream::from(sstable_scan));
        }

        Ok(streams)
    }

    pub(crate) fn scan_immutable_rows_at(
        &self,
        read_ts: Timestamp,
    ) -> Result<Vec<(KeyRow, DynRow)>, KeyExtractError> {
        let mut rows = Vec::new();
        let segments = self.seal_state_lock().immutables.clone();
        for segment in segments {
            let scan = segment.scan_visible(None, read_ts)?;
            for result in scan {
                match result {
                    Ok(ImmutableVisibleEntry::Row(key_view, row_raw)) => {
                        let row = row_raw.into_owned().map_err(|err| {
                            KeyExtractError::Arrow(arrow_schema::ArrowError::ComputeError(
                                err.to_string(),
                            ))
                        })?;
                        let (key_row, _) = key_view.into_parts();
                        rows.push((key_row, row));
                    }
                    Ok(ImmutableVisibleEntry::Tombstone(_)) => {}
                    Err(err) => return Err(KeyExtractError::from(err)),
                }
            }
        }
        Ok(rows)
    }

    pub(crate) fn scan_mutable_rows_at(
        &self,
        read_ts: Timestamp,
    ) -> Result<Vec<(KeyRow, DynRow)>, KeyExtractError> {
        let mut rows = Vec::new();
        let mem = self.mem.read();
        let scan = mem.scan_visible(None, read_ts)?;
        for entry in scan {
            match entry {
                Ok(DynRowScanEntry::Row(key_view, row_raw)) => {
                    let row = row_raw.into_owned().map_err(|err| {
                        KeyExtractError::Arrow(arrow_schema::ArrowError::ComputeError(
                            err.to_string(),
                        ))
                    })?;
                    let (key_row, _) = key_view.into_parts();
                    rows.push((key_row, row));
                }
                Ok(DynRowScanEntry::Tombstone(_)) => {}
                Err(err) => return Err(KeyExtractError::from(err)),
            }
        }
        Ok(rows)
    }

    /// Begin building a scan query with fluent API.
    #[cfg(test)]
    pub fn scan(&self) -> ScanBuilder<'_, FS, E> {
        ScanBuilder::new(self)
    }
}

/// Snapshot source for scan operations.
///
/// Controls whether a scan uses a pre-existing snapshot or creates one lazily.
pub(crate) enum SnapshotSource<'a> {
    /// Create a new snapshot when the scan executes (DB-level scan).
    Lazy,
    /// Use a pre-existing snapshot (Snapshot::scan or Transaction::scan).
    Preexisting(&'a TxSnapshot),
}

/// Staged mutations to overlay on scan results (for Transaction::scan).
pub(crate) struct StagedOverlay<'a> {
    /// Staged mutations (uncommitted writes).
    pub(crate) staging: &'a BTreeMap<KeyOwned, DynMutation<DynRow, ()>>,
    /// Schema for the staged rows.
    pub(crate) schema: &'a SchemaRef,
}

/// Fluent builder for constructing scan queries.
///
/// Use [`crate::db::DB::scan`] to create a new builder, then chain methods to
/// configure the scan before executing with [`stream`](ScanBuilder::stream) or
/// [`collect`](ScanBuilder::collect). Transaction and snapshot scan variants
/// wrap the same builder internally.
///
/// # Scan Hierarchy
///
/// All scans ultimately use `Snapshot` for MVCC visibility:
/// - `Snapshot::scan()` - Core implementation using a pre-existing snapshot
/// - `Transaction::scan()` - Wraps snapshot scan + overlays uncommitted writes
/// - `DB::scan()` - Creates an ephemeral snapshot internally
///
/// # Example
///
/// ```no_run
/// use std::sync::Arc;
///
/// use arrow_array::{Int32Array, RecordBatch, StringArray};
/// use arrow_schema::{DataType, Field, Schema};
/// use fusio::{executor::tokio::TokioExecutor, mem::fs::InMemoryFs};
/// use tonbo::{
///     db::{DB, DbBuilder, Expr, ScalarValue},
///     schema::SchemaBuilder,
/// };
///
/// #[tokio::main]
/// async fn main() -> Result<(), Box<dyn std::error::Error>> {
///     let schema = Arc::new(Schema::new(vec![
///         Field::new("id", DataType::Utf8, false),
///         Field::new("value", DataType::Int32, false),
///     ]));
///     let config = SchemaBuilder::from_schema(Arc::clone(&schema))
///         .primary_key("id")
///         .build()?;
///
///     // In-memory DB; use DbBuilder::on_disk/object_store for durable backends.
///     let db: DB<InMemoryFs, TokioExecutor> = DbBuilder::from_schema_key_name(schema, "id")?
///         .in_memory("scan-example")?
///         .open_with_executor(Arc::new(TokioExecutor::default()))
///         .await?;
///
///     // Seed data
///     let batch = RecordBatch::try_new(
///         Arc::clone(&config.schema()),
///         vec![
///             Arc::new(StringArray::from(vec!["a", "b"])) as _,
///             Arc::new(Int32Array::from(vec![1, 2])) as _,
///         ],
///     )?;
///     db.ingest(batch).await?;
///
///     // Scan with predicate + limit
///     let pred = Expr::eq("id", ScalarValue::Utf8(Some("a".to_string())));
///     let batches = db.scan().filter(pred).limit(1).collect().await?;
///     assert_eq!(batches.iter().map(|b| b.num_rows()).sum::<usize>(), 1);
///     Ok(())
/// }
/// ```
pub struct ScanBuilder<'a, FS, E>
where
    FS: crate::manifest::ManifestFs<E>,
    E: Executor + Timer + Clone + 'static,
    <FS as fusio::fs::Fs>::File: fusio::durability::FileCommit,
{
    db: &'a DbInner<FS, E>,
    /// Snapshot source: lazy (DB scan) or pre-existing (Snapshot/Transaction scan).
    snapshot_source: SnapshotSource<'a>,
    /// Optional staged mutations overlay (Transaction scan only).
    staged_overlay: Option<StagedOverlay<'a>>,
    predicate: Option<Expr>,
    projection: Option<SchemaRef>,
    limit: Option<usize>,
}

impl<'a, FS, E> ScanBuilder<'a, FS, E>
where
    FS: crate::manifest::ManifestFs<E>,
    E: Executor + Timer + Clone + 'static,
    <FS as fusio::fs::Fs>::File: fusio::durability::FileCommit,
{
    /// Create a new scan builder for DB-level scans (lazy snapshot, no staging).
    pub(crate) fn new(db: &'a DbInner<FS, E>) -> Self {
        Self {
            db,
            snapshot_source: SnapshotSource::Lazy,
            staged_overlay: None,
            predicate: None,
            projection: None,
            limit: None,
        }
    }

    /// Create a scan builder from an existing snapshot (core scan implementation).
    ///
    /// This is used by `Snapshot::scan()` and forms the basis for all other scans.
    pub(crate) fn from_snapshot(db: &'a DbInner<FS, E>, snapshot: &'a TxSnapshot) -> Self {
        Self {
            db,
            snapshot_source: SnapshotSource::Preexisting(snapshot),
            staged_overlay: None,
            predicate: None,
            projection: None,
            limit: None,
        }
    }

    /// Create a scan builder from an existing snapshot using the public DB handle.
    ///
    /// This is the public entry point for `Snapshot::scan(&db)`.
    pub(crate) fn from_snapshot_with_db(
        db: &'a super::DB<FS, E>,
        snapshot: &'a TxSnapshot,
    ) -> Self {
        Self::from_snapshot(&db.inner, snapshot)
    }

    /// Create a new scan builder with transaction context (includes staging buffer).
    ///
    /// Used by `Transaction::scan()` to include uncommitted writes.
    pub(crate) fn with_transaction_overlay(
        db: &'a DbInner<FS, E>,
        snapshot: &'a TxSnapshot,
        staging: &'a BTreeMap<KeyOwned, DynMutation<DynRow, ()>>,
        schema: &'a SchemaRef,
    ) -> Self {
        Self {
            db,
            snapshot_source: SnapshotSource::Preexisting(snapshot),
            staged_overlay: Some(StagedOverlay { staging, schema }),
            predicate: None,
            projection: None,
            limit: None,
        }
    }

    /// Set the filter predicate for this scan.
    ///
    /// Only rows matching the predicate will be returned.
    /// If not called, all rows are returned.
    #[must_use]
    pub fn filter(mut self, predicate: Expr) -> Self {
        self.predicate = Some(predicate);
        self
    }

    /// Set the projection schema for this scan.
    ///
    /// Only the specified columns will be returned in the result batches.
    /// If not called, all columns are returned.
    #[must_use]
    pub fn projection(mut self, schema: SchemaRef) -> Self {
        self.projection = Some(schema);
        self
    }

    /// Limit the number of rows returned by this scan.
    #[must_use]
    pub fn limit(mut self, limit: usize) -> Self {
        self.limit = Some(limit);
        self
    }

    /// Execute the scan and return a stream of record batches.
    pub async fn stream(
        self,
    ) -> Result<impl Stream<Item = Result<RecordBatch, crate::db::DBError>> + 'a, crate::db::DBError>
    {
        let (stream, _) = self.stream_with_profile().await?;
        Ok(stream)
    }

    /// Execute the scan and return both stream and setup-phase timing profile.
    pub async fn stream_with_profile(
        self,
    ) -> Result<
        (
            impl Stream<Item = Result<RecordBatch, crate::db::DBError>> + 'a,
            ScanSetupProfile,
        ),
        crate::db::DBError,
    > {
        // Take ownership of all fields upfront to avoid partial move issues
        let Self {
            db,
            snapshot_source,
            staged_overlay,
            predicate,
            projection,
            limit,
        } = self;

        let predicate = predicate.unwrap_or(Expr::True);
        let mut profile = ScanSetupProfile::default();

        // Resolve snapshot: use pre-existing or create new one
        let snapshot = match snapshot_source {
            SnapshotSource::Preexisting(snap) => snap.clone(),
            SnapshotSource::Lazy => {
                let started = Instant::now();
                let snapshot = db.begin_snapshot().await?;
                profile.snapshot_ns = duration_ns_u64(started.elapsed());
                snapshot
            }
        };

        // Build transaction scan if we have staged mutations
        let txn_scan = match staged_overlay {
            Some(overlay) if !overlay.staging.is_empty() => {
                // We need to plan first to get the scan_schema
                let started = Instant::now();
                let plan = snapshot
                    .plan_scan_with_context(db, &predicate, projection.as_ref(), limit, true)
                    .await?;
                profile.plan_scan_ns = duration_ns_u64(started.elapsed());
                Some((
                    TransactionScan::new(
                        overlay.staging,
                        overlay.schema,
                        plan.read_ts,
                        Some(&plan.scan_schema),
                    )
                    .map_err(crate::db::DBError::from)?,
                    plan,
                ))
            }
            _ => None,
        };

        // Build plan (reuse if already created for transaction scan)
        let plan = match txn_scan {
            Some((scan, plan)) => {
                // Execute with transaction scan overlay
                return execute_with_txn_scan(db, plan, Some(scan), profile).await;
            }
            None => {
                let started = Instant::now();
                let plan = snapshot
                    .plan_scan(db, &predicate, projection.as_ref(), limit)
                    .await?;
                profile.plan_scan_ns = duration_ns_u64(started.elapsed());
                plan
            }
        };

        execute_with_txn_scan(db, plan, None, profile).await
    }

    /// Execute the scan and collect all batches into a vector.
    pub async fn collect(self) -> Result<Vec<RecordBatch>, crate::db::DBError> {
        self.stream().await?.try_collect().await
    }
}

async fn execute_with_txn_scan<'a, FS, E>(
    db: &'a DbInner<FS, E>,
    plan: ScanPlan,
    txn_scan: Option<TransactionScan<'a>>,
    mut profile: ScanSetupProfile,
) -> Result<
    (
        Pin<Box<dyn Stream<Item = Result<RecordBatch, crate::db::DBError>> + 'a>>,
        ScanSetupProfile,
    ),
    crate::db::DBError,
>
where
    FS: crate::manifest::ManifestFs<E>,
    E: Executor + Timer + Clone + 'static,
    <FS as fusio::fs::Fs>::File: fusio::durability::FileCommit,
{
    let result_projection = plan
        .projected_schema
        .clone()
        .unwrap_or_else(|| Arc::clone(&db.schema));
    let scan_schema = Arc::clone(&plan.scan_schema);
    let build_streams_started = Instant::now();
    let streams = db.build_scan_streams(&plan, txn_scan).await?;
    profile.build_scan_streams_ns = duration_ns_u64(build_streams_started.elapsed());

    if streams.is_empty() {
        let stream = stream::empty::<Result<RecordBatch, crate::db::DBError>>();
        return Ok((
            Box::pin(stream)
                as Pin<Box<dyn Stream<Item = Result<RecordBatch, crate::db::DBError>> + 'a>>,
            profile,
        ));
    }

    let ScanPlan {
        residual_predicate,
        limit,
        ..
    } = plan;
    let needs_post_filter = residual_predicate.is_some();
    let limit_for_merge = if needs_post_filter { None } else { limit };
    let limit_for_package = if needs_post_filter { limit } else { None };
    // Only apply limit early when no post-merge filtering is required.
    let merge_started = Instant::now();
    let merge = MergeStream::from_vec(streams, limit_for_merge, Some(Order::Asc))
        .await
        .map_err(crate::db::DBError::from)?;
    profile.merge_init_ns = duration_ns_u64(merge_started.elapsed());
    let package_started = Instant::now();
    let package = PackageStream::with_limit(
        DEFAULT_SCAN_BATCH_ROWS,
        merge,
        Arc::clone(&scan_schema),
        Arc::clone(&result_projection),
        residual_predicate,
        limit_for_package,
    )
    .map_err(crate::db::DBError::from)?;
    profile.package_init_ns = duration_ns_u64(package_started.elapsed());

    let mapped = package.map(|result| result.map_err(crate::db::DBError::from));
    Ok((
        Box::pin(mapped)
            as Pin<Box<dyn Stream<Item = Result<RecordBatch, crate::db::DBError>> + 'a>>,
        profile,
    ))
}

fn duration_ns_u64(duration: std::time::Duration) -> u64 {
    u64::try_from(duration.as_nanos()).unwrap_or(u64::MAX)
}

fn find_bloom_filter_column(predicate: &Expr) -> Option<&str> {
    match predicate {
        Expr::BloomFilterEq { column, .. } | Expr::BloomFilterInList { column, .. } => {
            Some(column.as_str())
        }
        Expr::And(children) | Expr::Or(children) => children
            .iter()
            .find_map(|child| find_bloom_filter_column(child)),
        Expr::Not(child) => find_bloom_filter_column(child.as_ref()),
        _ => None,
    }
}

#[derive(Debug)]
struct NonSstPredicateSplit {
    pushdown: Option<Expr>,
    residual: Option<Expr>,
}

fn split_predicate_for_non_sst(predicate: &Expr, key_schema: &SchemaRef) -> NonSstPredicateSplit {
    if matches!(predicate, Expr::True) {
        return NonSstPredicateSplit {
            pushdown: None,
            residual: None,
        };
    }
    if matches!(predicate, Expr::False) {
        return NonSstPredicateSplit {
            pushdown: Some(Expr::False),
            residual: None,
        };
    }
    if key_schema.fields().len() != 1 {
        return NonSstPredicateSplit {
            pushdown: None,
            residual: Some(predicate.clone()),
        };
    }
    let Some(key_field) = key_schema.fields().first() else {
        return NonSstPredicateSplit {
            pushdown: None,
            residual: Some(predicate.clone()),
        };
    };
    split_predicate_for_non_sst_inner(predicate, key_field.name(), key_field.data_type())
}

fn split_predicate_for_non_sst_inner(
    predicate: &Expr,
    key_column: &str,
    key_type: &DataType,
) -> NonSstPredicateSplit {
    match predicate {
        Expr::True => NonSstPredicateSplit {
            pushdown: None,
            residual: None,
        },
        Expr::False => NonSstPredicateSplit {
            pushdown: Some(Expr::False),
            residual: None,
        },
        Expr::Cmp { column, op, .. } => {
            if column != key_column || matches!(op, aisle::CmpOp::NotEq) {
                return NonSstPredicateSplit {
                    pushdown: None,
                    residual: Some(predicate.clone()),
                };
            }
            NonSstPredicateSplit {
                pushdown: Some(predicate.clone()),
                residual: None,
            }
        }
        Expr::Between { column, .. } => {
            if column != key_column {
                return NonSstPredicateSplit {
                    pushdown: None,
                    residual: Some(predicate.clone()),
                };
            }
            NonSstPredicateSplit {
                pushdown: Some(predicate.clone()),
                residual: None,
            }
        }
        Expr::StartsWith { column, prefix } => {
            if column != key_column || prefix.is_empty() || !is_string_key_type(key_type) {
                return NonSstPredicateSplit {
                    pushdown: None,
                    residual: Some(predicate.clone()),
                };
            }
            // Non-SST scans rely on key-range pruning for StartsWith. If there is
            // no strict upper bound, keep a residual filter to avoid false positives.
            if next_prefix_string(prefix).is_none() {
                return NonSstPredicateSplit {
                    pushdown: None,
                    residual: Some(predicate.clone()),
                };
            }
            NonSstPredicateSplit {
                pushdown: Some(predicate.clone()),
                residual: None,
            }
        }
        Expr::And(children) => {
            let mut pushdown = Vec::new();
            let mut residual = Vec::new();
            for child in children {
                let split = split_predicate_for_non_sst_inner(child, key_column, key_type);
                if let Some(expr) = split.pushdown {
                    pushdown.push(expr);
                }
                if let Some(expr) = split.residual {
                    residual.push(expr);
                }
            }
            NonSstPredicateSplit {
                pushdown: combine_and_parts(pushdown),
                residual: combine_and_parts(residual),
            }
        }
        _ => NonSstPredicateSplit {
            pushdown: None,
            residual: Some(predicate.clone()),
        },
    }
}

fn is_string_key_type(data_type: &DataType) -> bool {
    matches!(
        data_type,
        DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View
    )
}

fn combine_and_parts(mut predicates: Vec<Expr>) -> Option<Expr> {
    match predicates.len() {
        0 => None,
        1 => Some(predicates.remove(0)),
        _ => Some(Expr::and(predicates)),
    }
}

fn combine_predicates_with_and(lhs: Option<Expr>, rhs: Option<Expr>) -> Option<Expr> {
    match (lhs, rhs) {
        (Some(lhs), Some(rhs)) => Some(Expr::and(vec![lhs, rhs])),
        (Some(expr), None) | (None, Some(expr)) => Some(expr),
        (None, None) => None,
    }
}

#[cfg(test)]
mod tests {
    use std::sync::Arc;

    use arrow_schema::{DataType, Field, Schema};

    use super::*;

    #[test]
    fn split_non_sst_true_with_composite_key_keeps_residual_empty() {
        let key_schema = Arc::new(Schema::new(vec![
            Field::new("k1", DataType::Utf8, false),
            Field::new("k2", DataType::Int64, false),
        ]));
        let split = split_predicate_for_non_sst(&Expr::True, &key_schema);
        assert!(split.pushdown.is_none());
        assert!(split.residual.is_none());
    }

    #[test]
    fn split_non_sst_starts_with_without_upper_bound_requires_residual() {
        let key_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Utf8, false)]));
        let predicate = Expr::StartsWith {
            column: "id".to_string(),
            prefix: char::MAX.to_string(),
        };

        let split = split_predicate_for_non_sst(&predicate, &key_schema);
        assert!(split.pushdown.is_none());
        assert!(split.residual.is_some());
    }

    #[test]
    fn split_non_sst_starts_with_unicode_gap_keeps_full_pushdown() {
        let key_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Utf8, false)]));
        let predicate = Expr::StartsWith {
            column: "id".to_string(),
            prefix: "\u{d7ff}".to_string(),
        };

        let split = split_predicate_for_non_sst(&predicate, &key_schema);
        assert!(split.pushdown.is_some());
        assert!(split.residual.is_none());
    }

    #[test]
    fn split_non_sst_starts_with_on_non_string_key_requires_residual() {
        let key_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int64, false)]));
        let predicate = Expr::StartsWith {
            column: "id".to_string(),
            prefix: "12".to_string(),
        };

        let split = split_predicate_for_non_sst(&predicate, &key_schema);
        assert!(split.pushdown.is_none());
        assert!(split.residual.is_some());
    }
}