lance-index 4.0.1

Lance indices implementation
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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

use std::any::Any;
use std::collections::BTreeMap;
use std::iter::once;
use std::time::Instant;
use std::{collections::HashMap, sync::Arc};

use super::lance_format::LanceIndexStore;
use super::{
    AnyQuery, BuiltinIndexType, IndexReader, IndexStore, IndexWriter, MetricsCollector,
    ScalarIndex, ScalarIndexParams, SearchResult, TextQuery,
};
use crate::frag_reuse::FragReuseIndex;
use crate::metrics::NoOpMetricsCollector;
use crate::pbold;
use crate::scalar::expression::{ScalarQueryParser, TextQueryParser};
use crate::scalar::registry::{
    DefaultTrainingRequest, ScalarIndexPlugin, TrainingCriteria, TrainingOrdering, TrainingRequest,
    VALUE_COLUMN_NAME,
};
use crate::scalar::{CreatedIndex, UpdateCriteria};
use crate::vector::VectorIndex;
use crate::{Index, IndexType};
use arrow::array::{AsArray, UInt32Builder};
use arrow::datatypes::{UInt32Type, UInt64Type};
use arrow_array::{BinaryArray, RecordBatch, UInt32Array};
use arrow_schema::{DataType, Field, Schema, SchemaRef};
use async_trait::async_trait;
use datafusion::execution::SendableRecordBatchStream;
use deepsize::DeepSizeOf;
use futures::{FutureExt, Stream, StreamExt, TryStreamExt, stream};
use lance_arrow::iter_str_array;
use lance_core::cache::{CacheKey, LanceCache, WeakLanceCache};
use lance_core::error::LanceOptionExt;
use lance_core::utils::address::RowAddress;
use lance_core::utils::tempfile::TempDir;
use lance_core::utils::tokio::get_num_compute_intensive_cpus;
use lance_core::utils::tracing::{IO_TYPE_LOAD_SCALAR_PART, TRACE_IO_EVENTS};
use lance_core::{Error, utils::mask::RowAddrTreeMap};
use lance_core::{ROW_ID, Result};
use lance_io::object_store::ObjectStore;
use log::info;
use roaring::{RoaringBitmap, RoaringTreemap};
use serde::Serialize;
use tantivy::tokenizer::TextAnalyzer;
use tracing::instrument;

const TOKENS_COL: &str = "tokens";
const POSTING_LIST_COL: &str = "posting_list";
const POSTINGS_FILENAME: &str = "ngram_postings.lance";
const NGRAM_INDEX_VERSION: u32 = 0;

use std::sync::LazyLock;

pub static TOKENS_FIELD: LazyLock<Field> =
    LazyLock::new(|| Field::new(TOKENS_COL, DataType::UInt32, true));
pub static POSTINGS_FIELD: LazyLock<Field> =
    LazyLock::new(|| Field::new(POSTING_LIST_COL, DataType::Binary, false));
pub static POSTINGS_SCHEMA: LazyLock<SchemaRef> = LazyLock::new(|| {
    Arc::new(Schema::new(vec![
        TOKENS_FIELD.clone(),
        POSTINGS_FIELD.clone(),
    ]))
});
pub static TEXT_PREPPER: LazyLock<TextAnalyzer> = LazyLock::new(|| {
    TextAnalyzer::builder(tantivy::tokenizer::RawTokenizer::default())
        .filter(tantivy::tokenizer::LowerCaser)
        .filter(tantivy::tokenizer::AsciiFoldingFilter)
        .build()
});
/// Currently we ALWAYS use trigrams with ascii folding and lower casing.  We may want to make this configurable in the future.
pub static NGRAM_TOKENIZER: LazyLock<TextAnalyzer> = LazyLock::new(|| {
    TextAnalyzer::builder(tantivy::tokenizer::NgramTokenizer::all_ngrams(3, 3).unwrap())
        .filter(tantivy::tokenizer::AlphaNumOnlyFilter)
        .build()
});

// Helper function to apply a function to each token in a text
fn tokenize_visitor(tokenizer: &TextAnalyzer, text: &str, mut visitor: impl FnMut(&String)) {
    // The token_stream method is mutable.  As far as I can tell this is to enforce exclusivity and not
    // true mutability.  For example, the object returned by `token_stream` has thread-local state but
    // it is reset each time `token_stream` is called.
    //
    // However, I don't see this documented anywhere and I'm not sure about relying on it.  For now, we
    // make a clone as that seems to be the safer option.  All the tokenizers we use here should be trivially
    // cloneable (although it requires a heap allocation so may be worth investigating in the future)
    let mut prepper = TEXT_PREPPER.clone();
    let mut tokenizer = tokenizer.clone();
    let mut raw_stream = prepper.token_stream(text);
    while raw_stream.advance() {
        let mut token_stream = tokenizer.token_stream(&raw_stream.token().text);
        while token_stream.advance() {
            visitor(&token_stream.token().text);
        }
    }
}

const ALPHA_SPAN: usize = 37;
const MAX_TOKEN: usize = ALPHA_SPAN.pow(2) + ALPHA_SPAN;
const MIN_TOKEN: usize = 0;
const NGRAM_N: usize = 3;

// Convert an ngram (string) to a token (u32).  This helps avoid heap allocations
// and it makes it easier to partition the tokens for shuffling
//
// There are 36 alphanumeric values and we add 1 for the NULL token giving us 37^3
// potential tokens.
//
// "" => 0
// "?" => 37^2 * ?
// "?$" => 37^2 * ? + 37 * $
// "?$#" => 37^2 * ? + 37 * $ + #
// ...
//
// The ?,$,# represent the position in the alphabet (+1 to distinguish from NULL)
//
// Small strings get the larger multipliers because those ngrams are
// less likely to be unique and will have larger bitmaps.  We want to
// spread those out.
//
// NOTE: Today we hard-code trigrams and we do not include 1-grams or 2-grams so this
// function is more general than it needs to be...just in case.
fn ngram_to_token(ngram: &str, ngram_length: usize) -> u32 {
    let mut token = 0;
    // Empty string will get 0
    for (idx, byte) in ngram.bytes().enumerate() {
        let pos = if byte <= b'9' {
            byte - b'0'
        } else if byte <= b'z' {
            byte - b'a' + 10
        } else {
            unreachable!()
        } + 1;
        debug_assert!(pos < ALPHA_SPAN as u8);
        let mult = ALPHA_SPAN.pow(ngram_length as u32 - idx as u32 - 1) as u32;
        token += pos as u32 * mult;
    }
    token
}

/// Basic stats about an ngram index
#[derive(Serialize)]
struct NGramStatistics {
    num_ngrams: usize,
}

/// The row ids that contain a given ngram
#[derive(Debug)]
pub struct NGramPostingList {
    bitmap: RoaringTreemap,
}

impl DeepSizeOf for NGramPostingList {
    fn deep_size_of_children(&self, _: &mut deepsize::Context) -> usize {
        self.bitmap.serialized_size()
    }
}

// Cache key implementation for type-safe cache access
#[derive(Debug, Clone)]
pub struct NGramPostingListKey {
    pub row_offset: u32,
}

impl CacheKey for NGramPostingListKey {
    type ValueType = NGramPostingList;

    fn key(&self) -> std::borrow::Cow<'_, str> {
        format!("posting-list-{}", self.row_offset).into()
    }
}

impl NGramPostingList {
    fn try_from_batch(
        batch: RecordBatch,
        frag_reuse_index: Option<Arc<FragReuseIndex>>,
    ) -> Result<Self> {
        let bitmap_bytes = batch.column(0).as_binary::<i32>().value(0);
        let mut bitmap = RoaringTreemap::deserialize_from(bitmap_bytes)
            .map_err(|e| Error::internal(format!("Error deserializing ngram list: {}", e)))?;
        if let Some(frag_reuse_index_ref) = frag_reuse_index.as_ref() {
            bitmap = frag_reuse_index_ref.remap_row_ids_roaring_tree_map(&bitmap);
        }
        Ok(Self { bitmap })
    }

    fn intersect<'a>(lists: impl IntoIterator<Item = &'a Self>) -> RoaringTreemap {
        let mut iter = lists.into_iter();
        let mut result = iter
            .next()
            .map(|list| list.bitmap.clone())
            .unwrap_or_default();
        for list in iter {
            result &= &list.bitmap;
        }
        result
    }
}

/// Reads on-demand ngram posting lists from storage (and stores them in a cache)
struct NGramPostingListReader {
    reader: Arc<dyn IndexReader>,
    frag_reuse_index: Option<Arc<FragReuseIndex>>,
    index_cache: WeakLanceCache,
}

impl DeepSizeOf for NGramPostingListReader {
    fn deep_size_of_children(&self, _: &mut deepsize::Context) -> usize {
        0
    }
}

impl std::fmt::Debug for NGramPostingListReader {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("NGramListReader").finish()
    }
}

impl NGramPostingListReader {
    #[instrument(level = "debug", skip(self, metrics))]
    pub async fn ngram_list(
        &self,
        row_offset: u32,
        metrics: &dyn MetricsCollector,
    ) -> Result<Arc<NGramPostingList>> {
        self.index_cache.get_or_insert_with_key(NGramPostingListKey { row_offset }, || async move {
            metrics.record_part_load();
                tracing::info!(target: TRACE_IO_EVENTS, r#type=IO_TYPE_LOAD_SCALAR_PART, index_type="ngram", part_id=row_offset);
                let batch = self
                    .reader
                    .read_range(
                        row_offset as usize..row_offset as usize + 1,
                        Some(&[POSTING_LIST_COL]),
                    )
                    .await?;
                NGramPostingList::try_from_batch(batch, self.frag_reuse_index.clone())
        }).await
    }
}

/// An ngram index
///
/// At a high level this is an inverted index that maps ngrams (small fixed size substrings) to the
/// row ids that contain them.
///
/// As a simple example consider a 1-gram index.  It would basically be a mapping from
/// each letter to the row ids that contain that letter.  Then, if the user searches for
/// "cat", the index would look up the row ids for "c", "a", and "t", and return the intersection
/// of those row ids because only rows have at least one c, a, and t could possible contain "cat".
///
/// This is an in-exact index, similar to a bloom filter.  It can return false positives and a
/// recheck step is needed to confirm the results.
///
/// Note that it cannot return false negatives.
pub struct NGramIndex {
    /// The mapping from tokens to row offsets
    tokens: HashMap<u32, u32>,
    /// The reader for the posting lists
    list_reader: Arc<NGramPostingListReader>,
    /// The tokenizer used to tokenize text.  Note: not all tokenizers can be used with this index.  For
    /// example, a stemming tokenizer would not work well because "dozing" would stem to "doze" and if the
    /// search term is "zing" it would not match.  As a result, this tokenizer is not as configurable as the
    /// tokenizers used in an inverted index.
    tokenizer: TextAnalyzer,
    io_parallelism: usize,
    /// The store that owns the index
    store: Arc<dyn IndexStore>,
}

impl std::fmt::Debug for NGramIndex {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("NGramIndex")
            .field("tokens", &self.tokens)
            .field("list_reader", &self.list_reader)
            .finish()
    }
}

impl DeepSizeOf for NGramIndex {
    fn deep_size_of_children(&self, context: &mut deepsize::Context) -> usize {
        self.tokens.deep_size_of_children(context)
    }
}

impl NGramIndex {
    async fn from_store(
        store: Arc<dyn IndexStore>,
        frag_reuse_index: Option<Arc<FragReuseIndex>>,
        index_cache: &LanceCache,
    ) -> Result<Self> {
        let tokens = store.open_index_file(POSTINGS_FILENAME).await?;
        let tokens = tokens
            .read_range(0..tokens.num_rows(), Some(&[TOKENS_COL]))
            .await?;

        let tokens_map = HashMap::from_iter(
            tokens
                .column(0)
                .as_primitive::<UInt32Type>()
                .values()
                .iter()
                .copied()
                .enumerate()
                .map(|(idx, token)| (token, idx as u32)),
        );

        let posting_reader = Arc::new(NGramPostingListReader {
            reader: store.open_index_file(POSTINGS_FILENAME).await?,
            frag_reuse_index,
            index_cache: WeakLanceCache::from(index_cache),
        });

        Ok(Self {
            io_parallelism: store.io_parallelism(),
            tokens: tokens_map,
            list_reader: posting_reader,
            tokenizer: NGRAM_TOKENIZER.clone(),
            store,
        })
    }

    fn remap_batch(
        &self,
        batch: RecordBatch,
        mapping: &HashMap<u64, Option<u64>>,
    ) -> Result<RecordBatch> {
        let posting_lists_array = batch
            .column_by_name(POSTING_LIST_COL)
            .expect_ok()?
            .as_binary::<i32>();

        let new_posting_lists = posting_lists_array
            .iter()
            .map(|posting_list| {
                let posting_list = posting_list.unwrap();
                let posting_list = RoaringTreemap::deserialize_from(posting_list)?;
                let new_posting_list =
                    RoaringTreemap::from_iter(posting_list.into_iter().filter_map(|row_id| {
                        match mapping.get(&row_id) {
                            Some(Some(new_row_id)) => Some(*new_row_id),
                            Some(None) => None,
                            None => Some(row_id),
                        }
                    }));
                let mut buf = Vec::with_capacity(new_posting_list.serialized_size());
                new_posting_list.serialize_into(&mut buf)?;
                Ok(buf)
            })
            .collect::<Result<Vec<_>>>()?;

        let new_posting_lists_array = BinaryArray::from_iter_values(new_posting_lists);

        Ok(RecordBatch::try_new(
            POSTINGS_SCHEMA.clone(),
            vec![
                batch.column_by_name(TOKENS_COL).expect_ok()?.clone(),
                Arc::new(new_posting_lists_array),
            ],
        )?)
    }

    async fn load(
        store: Arc<dyn IndexStore>,
        frag_reuse_index: Option<Arc<FragReuseIndex>>,
        index_cache: &LanceCache,
    ) -> Result<Arc<Self>>
    where
        Self: Sized,
    {
        Ok(Arc::new(
            Self::from_store(store, frag_reuse_index, index_cache).await?,
        ))
    }
}

#[async_trait]
impl Index for NGramIndex {
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn as_index(self: Arc<Self>) -> Arc<dyn Index> {
        self
    }

    fn as_vector_index(self: Arc<Self>) -> Result<Arc<dyn VectorIndex>> {
        Err(Error::invalid_input_source(
            "NGramIndex is not a vector index".into(),
        ))
    }

    fn statistics(&self) -> Result<serde_json::Value> {
        let ngram_stats = NGramStatistics {
            num_ngrams: self.tokens.len(),
        };
        serde_json::to_value(ngram_stats)
            .map_err(|e| Error::internal(format!("Error serializing statistics: {}", e)))
    }

    async fn prewarm(&self) -> Result<()> {
        // TODO: NGram index can pre-warm by loading all posting lists into memory
        Ok(())
    }

    fn index_type(&self) -> IndexType {
        IndexType::NGram
    }

    async fn calculate_included_frags(&self) -> Result<RoaringBitmap> {
        let mut frag_ids = RoaringBitmap::new();
        for row_offset in self.tokens.values() {
            let list = self
                .list_reader
                .ngram_list(*row_offset, &NoOpMetricsCollector)
                .await?;
            frag_ids.extend(
                list.bitmap
                    .iter()
                    .map(|row_addr| RowAddress::from(row_addr).fragment_id()),
            );
        }
        Ok(frag_ids)
    }
}

#[async_trait]
impl ScalarIndex for NGramIndex {
    async fn search(
        &self,
        query: &dyn AnyQuery,
        metrics: &dyn MetricsCollector,
    ) -> Result<SearchResult> {
        let query = query
            .as_any()
            .downcast_ref::<TextQuery>()
            .ok_or_else(|| Error::invalid_input_source("Query is not a TextQuery".into()))?;
        match query {
            TextQuery::StringContains(substr) => {
                if substr.len() < NGRAM_N {
                    // We know nothing on short searches, need to recheck all
                    return Ok(SearchResult::at_least(RowAddrTreeMap::new()));
                }

                let mut row_offsets = Vec::with_capacity(substr.len() * 3);
                let mut missing = false;
                tokenize_visitor(&self.tokenizer, substr, |ngram| {
                    let token = ngram_to_token(ngram, NGRAM_N);
                    if let Some(row_offset) = self.tokens.get(&token) {
                        row_offsets.push(*row_offset);
                    } else {
                        missing = true;
                    }
                });
                // At least one token was missing, so we know there are zero results
                if missing {
                    return Ok(SearchResult::exact(RowAddrTreeMap::new()));
                }
                let posting_lists = futures::stream::iter(
                    row_offsets
                        .into_iter()
                        .map(|row_offset| self.list_reader.ngram_list(row_offset, metrics)),
                )
                .buffer_unordered(self.io_parallelism)
                .try_collect::<Vec<_>>()
                .await?;
                metrics.record_comparisons(posting_lists.len());
                let list_refs = posting_lists.iter().map(|list| list.as_ref());
                let row_ids = NGramPostingList::intersect(list_refs);
                Ok(SearchResult::at_most(RowAddrTreeMap::from(row_ids)))
            }
        }
    }

    fn can_remap(&self) -> bool {
        true
    }

    async fn remap(
        &self,
        mapping: &HashMap<u64, Option<u64>>,
        dest_store: &dyn IndexStore,
    ) -> Result<CreatedIndex> {
        let reader = self.store.open_index_file(POSTINGS_FILENAME).await?;
        let mut writer = dest_store
            .new_index_file(POSTINGS_FILENAME, POSTINGS_SCHEMA.clone())
            .await?;

        let mut offset = 0;
        let num_rows = reader.num_rows();
        const BATCH_SIZE: usize = 64;
        while offset < num_rows {
            let batch_size = BATCH_SIZE.min(num_rows - offset);
            let batch = reader.read_range(offset..offset + batch_size, None).await?;
            let batch = self.remap_batch(batch, mapping)?;
            writer.write_record_batch(batch).await?;
            offset += BATCH_SIZE;
        }

        writer.finish().await?;

        Ok(CreatedIndex {
            index_details: prost_types::Any::from_msg(&pbold::NGramIndexDetails::default())
                .unwrap(),
            index_version: NGRAM_INDEX_VERSION,
            files: Some(dest_store.list_files_with_sizes().await?),
        })
    }

    async fn update(
        &self,
        new_data: SendableRecordBatchStream,
        dest_store: &dyn IndexStore,
        _old_data_filter: Option<super::OldIndexDataFilter>,
    ) -> Result<CreatedIndex> {
        let mut builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions::default())?;
        let spill_files = builder.train(new_data).await?;

        builder
            .write_index(dest_store, spill_files, Some(self.store.clone()))
            .await?;

        Ok(CreatedIndex {
            index_details: prost_types::Any::from_msg(&pbold::NGramIndexDetails::default())
                .unwrap(),
            index_version: NGRAM_INDEX_VERSION,
            files: Some(dest_store.list_files_with_sizes().await?),
        })
    }

    fn update_criteria(&self) -> UpdateCriteria {
        UpdateCriteria::only_new_data(TrainingCriteria::new(TrainingOrdering::None).with_row_id())
    }

    fn derive_index_params(&self) -> Result<ScalarIndexParams> {
        Ok(ScalarIndexParams::for_builtin(BuiltinIndexType::NGram))
    }
}

#[derive(Debug, Clone)]
pub struct NGramIndexBuilderOptions {
    tokens_per_spill: usize,
}

// A higher value will use more RAM.  A lower value will have to do more spilling
static DEFAULT_TOKENS_PER_SPILL: LazyLock<usize> = LazyLock::new(|| {
    std::env::var("LANCE_NGRAM_TOKENS_PER_SPILL")
        .unwrap_or_else(|_| "1000000000".to_string())
        .parse()
        .expect("failed to parse LANCE_NGRAM_TOKENS_PER_SPILL")
});
// How many partitions to use for shuffling out the work.  We slightly
// over-allocate this since the amount of work per-partition is not uniform.
//
// Increasing this may increase the performance but it could increase RAM (since we will spill less often)
// and could hurt performance (since there will be more files at the end for the final spill)
static DEFAULT_NUM_PARTITIONS: LazyLock<usize> = LazyLock::new(|| {
    std::env::var("LANCE_NGRAM_NUM_PARTITIONS")
        .map(|s| s.parse().expect("failed to parse LANCE_NGRAM_PARALLELISM"))
        .unwrap_or((get_num_compute_intensive_cpus() * 4).max(128))
});
// Just enough so that tokenizing is faster than I/O
static DEFAULT_TOKENIZE_PARALLELISM: LazyLock<usize> = LazyLock::new(|| {
    std::env::var("LANCE_NGRAM_TOKENIZE_PARALLELISM")
        .map(|s| {
            s.parse()
                .expect("failed to parse LANCE_NGRAM_TOKENIZE_PARALLELISM")
        })
        .unwrap_or(8)
});

impl Default for NGramIndexBuilderOptions {
    fn default() -> Self {
        Self {
            tokens_per_spill: *DEFAULT_TOKENS_PER_SPILL,
        }
    }
}

// An ordered list of tokens and bitmaps
//
// The `tokens` list is ordered by token value.  This makes it easier to merge spill files.
struct NGramIndexSpillState {
    tokens: UInt32Array,
    bitmaps: Vec<RoaringTreemap>,
}

impl NGramIndexSpillState {
    fn try_from_batch(batch: RecordBatch) -> Result<Self> {
        let tokens = batch
            .column_by_name(TOKENS_COL)
            .expect_ok()?
            .as_primitive::<UInt32Type>()
            .clone();
        let postings = batch
            .column_by_name(POSTING_LIST_COL)
            .expect_ok()?
            .as_binary::<i32>();

        let bitmaps = postings
            .into_iter()
            .map(|bytes| {
                RoaringTreemap::deserialize_from(bytes.expect_ok()?)
                    .map_err(|e| Error::internal(format!("Error deserializing ngram list: {}", e)))
            })
            .collect::<Result<Vec<_>>>()?;

        Ok(Self { tokens, bitmaps })
    }

    fn try_into_batch(self) -> Result<RecordBatch> {
        let bitmap_array = BinaryArray::from_iter_values(self.bitmaps.into_iter().map(|bitmap| {
            let mut buf = Vec::with_capacity(bitmap.serialized_size());
            bitmap.serialize_into(&mut buf).unwrap();
            buf
        }));
        Ok(RecordBatch::try_new(
            POSTINGS_SCHEMA.clone(),
            vec![Arc::new(self.tokens), Arc::new(bitmap_array)],
        )?)
    }
}

// As we're building we create a map from ngram to row ids.  When this map gets too large
// we spill it to disk.
struct NGramIndexBuildState {
    tokens_map: BTreeMap<u32, RoaringTreemap>,
}

impl NGramIndexBuildState {
    fn starting() -> Self {
        Self {
            tokens_map: BTreeMap::new(),
        }
    }

    fn take(&mut self) -> Self {
        let mut taken = Self::starting();
        std::mem::swap(&mut self.tokens_map, &mut taken.tokens_map);
        taken
    }

    fn into_spill(self) -> NGramIndexSpillState {
        // We can rely on these being in token order because of BTreeMap
        let tokens = UInt32Array::from_iter_values(self.tokens_map.keys().copied());
        let bitmaps = Vec::from_iter(self.tokens_map.into_values());

        NGramIndexSpillState { bitmaps, tokens }
    }
}

/// A builder for an ngram index
///
/// The builder is a small pipeline.  First, we read in the data and tokenize it.  This
/// stage uses fan-out parallelism to tokenize the data because tokenization may be a little
/// slower than I/O.
///
/// The second stage fans out much wider.  It partitions the tokens into a number of partitions.
/// Each partition has a BTreemap that maps tokens to row ids.  The partitions then build up
/// roaring treemaps.  When a partition gets too full it will spill to disk.
///
/// Once all the data is processed we spill all the parititons to disk and then we merge the
/// spill files into a single index file.
pub struct NGramIndexBuilder {
    tokenizer: TextAnalyzer,
    options: NGramIndexBuilderOptions,
    tmpdir: Arc<TempDir>,
    spill_store: Arc<dyn IndexStore>,

    tokens_seen: usize,
    worker_number: usize,
    has_flushed: bool,

    state: NGramIndexBuildState,
}

impl NGramIndexBuilder {
    pub fn try_new(options: NGramIndexBuilderOptions) -> Result<Self> {
        Self::from_state(NGramIndexBuildState::starting(), options)
    }

    fn clone_worker(&self, worker_number: usize) -> Self {
        let mut bitmaps = Vec::with_capacity(36 * 36 * 36 + 1);
        // Token 0 is always the NULL bitmap
        bitmaps.push(RoaringTreemap::new());
        Self {
            tokenizer: self.tokenizer.clone(),
            state: NGramIndexBuildState::starting(),
            tmpdir: self.tmpdir.clone(),
            spill_store: self.spill_store.clone(),
            options: self.options.clone(),
            tokens_seen: 0,
            worker_number,
            has_flushed: false,
        }
    }

    fn from_state(state: NGramIndexBuildState, options: NGramIndexBuilderOptions) -> Result<Self> {
        let tokenizer = NGRAM_TOKENIZER.clone();

        let tmpdir = Arc::new(TempDir::default());
        let spill_store = Arc::new(LanceIndexStore::new(
            Arc::new(ObjectStore::local()),
            tmpdir.obj_path(),
            Arc::new(LanceCache::no_cache()),
        ));

        Ok(Self {
            tokenizer,
            state,
            tmpdir,
            spill_store,
            options,
            tokens_seen: 0,
            worker_number: 0,
            has_flushed: false,
        })
    }

    fn validate_schema(schema: &Schema) -> Result<()> {
        if schema.fields().len() != 2 {
            return Err(Error::invalid_input_source(
                "Ngram index schema must have exactly two fields".into(),
            ));
        }
        let values_field = schema.field_with_name(VALUE_COLUMN_NAME)?;
        if *values_field.data_type() != DataType::Utf8
            && *values_field.data_type() != DataType::LargeUtf8
        {
            return Err(Error::invalid_input_source(
                "First field in ngram index schema must be of type Utf8/LargeUtf8".into(),
            ));
        }
        let row_id_field = schema.field_with_name(ROW_ID)?;
        if *row_id_field.data_type() != DataType::UInt64 {
            return Err(Error::invalid_input_source(
                "Second field in ngram index schema must be of type UInt64".into(),
            ));
        }
        Ok(())
    }

    async fn process_batch(&mut self, tokens_and_ids: Vec<(u32, u64)>) -> Result<()> {
        let mut tokens_seen = 0;
        for (token, row_id) in tokens_and_ids {
            tokens_seen += 1;
            // This would be a bit simpler with entry API but, at scale, the vast majority
            // of cases will be a hit and we want to avoid cloning the string if we can.  So
            // for now we do the double-hash.  We can simplify in the future with raw_entry
            // when it stabilizes.
            self.state
                .tokens_map
                .entry(token)
                .or_default()
                .insert(row_id);
        }
        self.tokens_seen += tokens_seen;
        if self.tokens_seen >= self.options.tokens_per_spill {
            let state = self.state.take();
            self.flush(state).await?;
        }
        Ok(())
    }

    fn spill_filename(id: usize) -> String {
        format!("spill-{}.lance", id)
    }

    fn tmp_spill_filename(id: usize) -> String {
        format!("spill-{}.lance.tmp", id)
    }

    async fn flush(&mut self, state: NGramIndexBuildState) -> Result<bool> {
        if self.tokens_seen == 0 {
            assert!(state.tokens_map.is_empty());
            return Ok(self.has_flushed);
        }
        self.tokens_seen = 0;
        let spill_state = state.into_spill();
        let flush_start = Instant::now();
        // The primary builder should never flush
        debug_assert_ne!(self.worker_number, 0);
        if self.has_flushed {
            info!("Merging flush for worker {}", self.worker_number);
            // If we have flushed before then we need to merge with the spill file
            let mut writer = self
                .spill_store
                .new_index_file(
                    &Self::tmp_spill_filename(self.worker_number),
                    POSTINGS_SCHEMA.clone(),
                )
                .await?;

            let left_stream = stream::once(std::future::ready(Ok(spill_state)));
            let right_stream =
                Self::stream_spill(self.spill_store.clone(), self.worker_number).await?;
            Self::merge_spill_streams(left_stream, right_stream, writer.as_mut()).await?;
            drop(writer);
            self.spill_store
                .rename_index_file(
                    &Self::tmp_spill_filename(self.worker_number),
                    &Self::spill_filename(self.worker_number),
                )
                .await?;
        } else {
            // If we haven't flushed before we can just write to the spill file
            info!("Initial flush for worker {}", self.worker_number);
            self.has_flushed = true;
            let writer = self
                .spill_store
                .new_index_file(
                    &Self::spill_filename(self.worker_number),
                    POSTINGS_SCHEMA.clone(),
                )
                .await?;
            self.write(writer, spill_state).await?;
        }
        let flush_time = flush_start.elapsed();
        info!(
            "Flushed worker {} in {}ms",
            self.worker_number,
            flush_time.as_millis()
        );
        Ok(true)
    }

    fn tokenize_and_partition(
        tokenizer: &TextAnalyzer,
        batch: RecordBatch,
        num_workers: usize,
    ) -> Result<Vec<Vec<(u32, u64)>>> {
        let text_iter = iter_str_array(batch.column_by_name(VALUE_COLUMN_NAME).expect_ok()?);
        let row_id_col = batch
            .column_by_name(ROW_ID)
            .expect_ok()?
            .as_primitive::<UInt64Type>();
        // Guessing 1000 tokens per row to at least avoid some of the earlier allocations
        let mut partitions = vec![Vec::with_capacity(batch.num_rows() * 1000); num_workers];
        let divisor = (MAX_TOKEN - MIN_TOKEN) / num_workers;
        for (text, row_id) in text_iter.zip(row_id_col.values()) {
            if let Some(text) = text {
                tokenize_visitor(tokenizer, text, |token| {
                    let token = ngram_to_token(token, NGRAM_N);
                    let partition_id = (token as usize).saturating_sub(MIN_TOKEN) / divisor;
                    partitions[partition_id % num_workers].push((token, *row_id));
                });
            } else {
                partitions[0].push((0, *row_id));
            }
        }
        Ok(partitions)
    }

    pub async fn train(&mut self, data: SendableRecordBatchStream) -> Result<Vec<usize>> {
        let schema = data.schema();
        Self::validate_schema(schema.as_ref())?;

        let num_workers = *DEFAULT_NUM_PARTITIONS;
        let mut senders = Vec::with_capacity(num_workers);
        let mut builders = Vec::with_capacity(num_workers);
        for worker_idx in 0..num_workers {
            let (send, mut recv) = tokio::sync::mpsc::channel(2);
            senders.push(send);

            let mut builder = self.clone_worker(worker_idx + 1);
            let future = tokio::spawn(async move {
                while let Some(partition) = recv.recv().await {
                    builder.process_batch(partition).await?;
                }
                Result::Ok(builder)
            });
            builders.push(future);
        }

        let mut partitions_stream = data
            .and_then(|batch| {
                let tokenizer = self.tokenizer.clone();
                std::future::ready(Ok(tokio::task::spawn(async move {
                    Ok(Self::tokenize_and_partition(
                        &tokenizer,
                        batch,
                        num_workers,
                    )?)
                })
                .map(|res| res.unwrap())))
            })
            .try_buffer_unordered(*DEFAULT_TOKENIZE_PARALLELISM);

        while let Some(partitions) = partitions_stream.try_next().await? {
            for (part_idx, partition) in partitions.into_iter().enumerate() {
                senders[part_idx].send(partition).await.unwrap();
            }
        }

        std::mem::drop(senders);
        let builders = futures::future::try_join_all(builders).await?;

        // Final flush is serialized.  If we kick this off in parallel it can
        // use a lot of memory.

        let mut to_spill = Vec::with_capacity(builders.len());

        for builder in builders {
            let mut builder = builder?;
            let state = builder.state.take();
            if builder.flush(state).await? {
                to_spill.push(builder.worker_number);
            }
        }

        Ok(to_spill)
    }

    async fn write(
        &mut self,
        mut writer: Box<dyn IndexWriter>,
        state: NGramIndexSpillState,
    ) -> Result<()> {
        writer.write_record_batch(state.try_into_batch()?).await?;
        writer.finish().await?;

        Ok(())
    }

    async fn stream_spill_reader(
        reader: Arc<dyn IndexReader>,
    ) -> Result<impl Stream<Item = Result<NGramIndexSpillState>>> {
        let num_rows = reader.num_rows();

        Ok(stream::try_unfold(0, move |offset| {
            let reader = reader.clone();
            async move {
                // These are small batches but, in the worst case scenario, each row could
                // be massive (up to 128MB per row at 1B rows) and we end up breaking memory
                let batch_size = std::cmp::min(num_rows - offset, 64);
                if batch_size == 0 {
                    return Ok(None);
                }
                let batch = reader.read_range(offset..offset + batch_size, None).await?;
                let state = NGramIndexSpillState::try_from_batch(batch)?;
                let new_offset = offset + batch_size;
                Ok(Some((state, new_offset)))
            }
            .boxed()
        }))
    }

    async fn stream_spill(
        spill_store: Arc<dyn IndexStore>,
        id: usize,
    ) -> Result<impl Stream<Item = Result<NGramIndexSpillState>>> {
        let reader = spill_store
            .open_index_file(&Self::spill_filename(id))
            .await?;
        Self::stream_spill_reader(reader).await
    }

    fn merge_spill_states(
        left_opt: &mut Option<NGramIndexSpillState>,
        right_opt: &mut Option<NGramIndexSpillState>,
    ) -> NGramIndexSpillState {
        let left = left_opt.take().unwrap();
        let right = right_opt.take().unwrap();

        let item_capacity = left.tokens.len() + right.tokens.len();
        let mut merged_tokens = UInt32Builder::with_capacity(item_capacity);
        let mut merged_bitmaps = Vec::with_capacity(left.bitmaps.len() + right.bitmaps.len());

        let mut left_tokens = left.tokens.values().iter().copied();
        let mut left_bitmaps = left.bitmaps.into_iter();
        let mut right_tokens = right.tokens.values().iter().copied();
        let mut right_bitmaps = right.bitmaps.into_iter();

        let mut left_token = left_tokens.next();
        let mut left_bitmap = left_bitmaps.next();
        let mut right_token = right_tokens.next();
        let mut right_bitmap = right_bitmaps.next();

        while left_token.is_some() && right_token.is_some() {
            let left_token_val = left_token.unwrap();
            let right_token_val = right_token.unwrap();
            match left_token_val.cmp(&right_token_val) {
                std::cmp::Ordering::Less => {
                    merged_tokens.append_value(left_token_val);
                    merged_bitmaps.push(left_bitmap.unwrap());
                    left_token = left_tokens.next();
                    left_bitmap = left_bitmaps.next();
                }
                std::cmp::Ordering::Greater => {
                    merged_tokens.append_value(right_token_val);
                    merged_bitmaps.push(right_bitmap.unwrap());
                    right_token = right_tokens.next();
                    right_bitmap = right_bitmaps.next();
                }
                std::cmp::Ordering::Equal => {
                    merged_tokens.append_value(left_token_val);
                    merged_bitmaps.push(left_bitmap.unwrap() | &right_bitmap.unwrap());
                    left_token = left_tokens.next();
                    left_bitmap = left_bitmaps.next();
                    right_token = right_tokens.next();
                    right_bitmap = right_bitmaps.next();
                }
            }
        }

        let collect_remaining = |cur_token, tokens, cur_bitmap, bitmaps| {
            let tokens = UInt32Array::from_iter_values(once(cur_token).chain(tokens));
            let bitmaps = once(cur_bitmap).chain(bitmaps).collect::<Vec<_>>();
            NGramIndexSpillState { tokens, bitmaps }
        };

        if let Some(left_token) = left_token {
            *left_opt = Some(collect_remaining(
                left_token,
                left_tokens,
                left_bitmap.unwrap(),
                left_bitmaps,
            ));
        } else {
            *left_opt = None;
        }
        if let Some(right_token) = right_token {
            *right_opt = Some(collect_remaining(
                right_token,
                right_tokens,
                right_bitmap.unwrap(),
                right_bitmaps,
            ));
        } else {
            *right_opt = None;
        }

        NGramIndexSpillState {
            tokens: merged_tokens.finish(),
            bitmaps: merged_bitmaps,
        }
    }

    async fn merge_spill_streams(
        mut left_stream: impl Stream<Item = Result<NGramIndexSpillState>> + Unpin,
        mut right_stream: impl Stream<Item = Result<NGramIndexSpillState>> + Unpin,
        writer: &mut dyn IndexWriter,
    ) -> Result<()> {
        let mut left_state = left_stream.try_next().await?;
        let mut right_state = right_stream.try_next().await?;

        while left_state.is_some() || right_state.is_some() {
            if left_state.is_none() {
                // Left is done, full drain right
                let state = right_state.take().expect_ok()?;
                writer.write_record_batch(state.try_into_batch()?).await?;
                while let Some(state) = right_stream.try_next().await? {
                    writer.write_record_batch(state.try_into_batch()?).await?;
                }
            } else if right_state.is_none() {
                // Right is done, full drain left
                let state = left_state.take().expect_ok()?;
                writer.write_record_batch(state.try_into_batch()?).await?;
                while let Some(state) = left_stream.try_next().await? {
                    writer.write_record_batch(state.try_into_batch()?).await?;
                }
            } else {
                // There is a batch from both left and right.  Need to merge them
                let merged = Self::merge_spill_states(&mut left_state, &mut right_state);
                writer.write_record_batch(merged.try_into_batch()?).await?;
                if left_state.is_none() {
                    left_state = left_stream.try_next().await?;
                }
                if right_state.is_none() {
                    right_state = right_stream.try_next().await?;
                }
            }
        }

        writer.finish().await
    }

    async fn merge_spill_files(
        spill_store: Arc<dyn IndexStore>,
        index_of_left: usize,
        index_of_right: usize,
        output_index: usize,
    ) -> Result<()> {
        // We fully load the small file into memory and then stream the large file
        info!(
            "Merge spill files {} and {} into {}",
            index_of_left, index_of_right, output_index
        );

        let mut writer = spill_store
            .new_index_file(&Self::spill_filename(output_index), POSTINGS_SCHEMA.clone())
            .await?;

        let (left_stream, right_stream) = futures::try_join!(
            Self::stream_spill(spill_store.clone(), index_of_left),
            Self::stream_spill(spill_store.clone(), index_of_right)
        )?;

        Self::merge_spill_streams(left_stream, right_stream, writer.as_mut()).await?;

        spill_store
            .delete_index_file(&Self::spill_filename(index_of_left))
            .await?;
        spill_store
            .delete_index_file(&Self::spill_filename(index_of_right))
            .await?;

        Ok(())
    }

    // Can potentially parallelize in the future if this step becomes a bottleneck
    //
    // We can also merge in a more balanced fashion (e.g. binary tree) to reduce the size of
    // intermediate files
    //
    // Note: worker indices start at 1 and not 0 (hence all the +1's)
    async fn merge_spills(&mut self, mut spill_files: Vec<usize>) -> Result<usize> {
        info!(
            "Merging {} index files into one combined index",
            spill_files.len()
        );

        let mut spill_counter = spill_files.iter().max().expect_ok()? + 1;
        while spill_files.len() > 1 {
            let mut new_spills = Vec::with_capacity(spill_files.len() / 2);
            while spill_files.len() >= 2 {
                let left = spill_files.pop().expect_ok()?;
                let right = spill_files.pop().expect_ok()?;
                new_spills.push(tokio::spawn(Self::merge_spill_files(
                    self.spill_store.clone(),
                    left,
                    right,
                    spill_counter + new_spills.len(),
                )));
            }
            for i in 0..new_spills.len() {
                spill_files.push(spill_counter + i);
            }
            spill_counter += new_spills.len();
            futures::future::try_join_all(new_spills).await?;
        }

        spill_files.pop().expect_ok()
    }

    async fn merge_old_index(
        &mut self,
        new_data_num: usize,
        old_index: Arc<dyn IndexStore>,
    ) -> Result<usize> {
        info!("Merging old index into new index");
        let final_num = new_data_num + 1;

        let mut writer = self
            .spill_store
            .new_index_file(&Self::spill_filename(final_num), POSTINGS_SCHEMA.clone())
            .await?;

        let left_stream = Self::stream_spill(self.spill_store.clone(), new_data_num).await?;
        let old_reader = old_index.open_index_file(POSTINGS_FILENAME).await?;
        let right_stream = Self::stream_spill_reader(old_reader).await?;

        Self::merge_spill_streams(left_stream, right_stream, writer.as_mut()).await?;

        self.spill_store
            .delete_index_file(&Self::spill_filename(new_data_num))
            .await?;

        Ok(final_num)
    }

    pub async fn write_index(
        mut self,
        store: &dyn IndexStore,
        spill_files: Vec<usize>,
        old_index: Option<Arc<dyn IndexStore>>,
    ) -> Result<()> {
        let mut writer = store
            .new_index_file(POSTINGS_FILENAME, POSTINGS_SCHEMA.clone())
            .await?;

        if spill_files.is_empty() {
            if let Some(old_index) = old_index {
                // An update with no new data, just copy the old index to the new store
                old_index.copy_index_file(POSTINGS_FILENAME, store).await?;
            } else {
                // Training an index with no data, make an empty index
                let mut writer = store
                    .new_index_file(POSTINGS_FILENAME, POSTINGS_SCHEMA.clone())
                    .await?;
                writer.finish().await?;
            }
            return Ok(());
        }

        let mut index_to_copy = self.merge_spills(spill_files).await?;

        if let Some(old_index) = old_index {
            index_to_copy = self.merge_old_index(index_to_copy, old_index).await?;
        }

        let reader = self
            .spill_store
            .open_index_file(&Self::spill_filename(index_to_copy))
            .await?;

        let num_rows = reader.num_rows();
        let mut offset = 0;

        while offset < num_rows {
            let batch_size = std::cmp::min(num_rows - offset, 64);
            let batch = reader.read_range(offset..offset + batch_size, None).await?;
            writer.write_record_batch(batch).await?;
            offset += batch_size;
        }

        writer.finish().await
    }
}

#[derive(Debug, Default)]
pub struct NGramIndexPlugin;

impl NGramIndexPlugin {
    pub async fn train_ngram_index(
        batches_source: SendableRecordBatchStream,
        index_store: &dyn IndexStore,
    ) -> Result<()> {
        let mut builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions::default())?;

        let spill_files = builder.train(batches_source).await?;

        builder.write_index(index_store, spill_files, None).await
    }
}

#[async_trait]
impl ScalarIndexPlugin for NGramIndexPlugin {
    fn name(&self) -> &str {
        "NGram"
    }

    fn new_training_request(
        &self,
        _params: &str,
        field: &Field,
    ) -> Result<Box<dyn TrainingRequest>> {
        if !matches!(field.data_type(), DataType::Utf8 | DataType::LargeUtf8) {
            return Err(Error::invalid_input_source(format!(
                "A ngram index can only be created on a Utf8 or LargeUtf8 field.  Column has type {:?}",
                field.data_type()
            )
            .into()));
        }
        Ok(Box::new(DefaultTrainingRequest::new(
            TrainingCriteria::new(TrainingOrdering::None).with_row_id(),
        )))
    }

    fn provides_exact_answer(&self) -> bool {
        false
    }

    fn version(&self) -> u32 {
        NGRAM_INDEX_VERSION
    }

    fn new_query_parser(
        &self,
        index_name: String,
        _index_details: &prost_types::Any,
    ) -> Option<Box<dyn ScalarQueryParser>> {
        Some(Box::new(TextQueryParser::new(index_name, true)))
    }

    async fn train_index(
        &self,
        data: SendableRecordBatchStream,
        index_store: &dyn IndexStore,
        _request: Box<dyn TrainingRequest>,
        fragment_ids: Option<Vec<u32>>,
        _progress: Arc<dyn crate::progress::IndexBuildProgress>,
    ) -> Result<CreatedIndex> {
        if fragment_ids.is_some() {
            return Err(Error::invalid_input_source(
                "NGram index does not support fragment training".into(),
            ));
        }

        Self::train_ngram_index(data, index_store).await?;
        Ok(CreatedIndex {
            index_details: prost_types::Any::from_msg(&pbold::NGramIndexDetails::default())
                .unwrap(),
            index_version: NGRAM_INDEX_VERSION,
            files: Some(index_store.list_files_with_sizes().await?),
        })
    }

    async fn load_index(
        &self,
        index_store: Arc<dyn IndexStore>,
        _index_details: &prost_types::Any,
        frag_reuse_index: Option<Arc<FragReuseIndex>>,
        cache: &LanceCache,
    ) -> Result<Arc<dyn ScalarIndex>> {
        Ok(NGramIndex::load(index_store, frag_reuse_index, cache).await? as Arc<dyn ScalarIndex>)
    }
}

#[cfg(test)]
mod tests {
    use std::{
        collections::{HashMap, HashSet},
        sync::Arc,
    };

    use arrow::datatypes::UInt64Type;
    use arrow_array::{Array, RecordBatch, StringArray, UInt64Array};
    use arrow_schema::{DataType, Field, Schema};
    use datafusion::{
        execution::SendableRecordBatchStream, physical_plan::stream::RecordBatchStreamAdapter,
    };
    use datafusion_common::DataFusionError;
    use futures::{TryStreamExt, stream};
    use itertools::Itertools;
    use lance_core::{
        ROW_ID,
        cache::LanceCache,
        utils::{mask::RowAddrTreeMap, tempfile::TempDir},
    };
    use lance_datagen::{BatchCount, ByteCount, RowCount};
    use lance_io::object_store::ObjectStore;
    use tantivy::tokenizer::TextAnalyzer;

    use crate::scalar::{
        ScalarIndex, SearchResult, TextQuery,
        lance_format::LanceIndexStore,
        ngram::{NGramIndex, NGramIndexBuilder, NGramIndexBuilderOptions},
    };
    use crate::{metrics::NoOpMetricsCollector, scalar::registry::VALUE_COLUMN_NAME};

    use super::{NGRAM_TOKENIZER, ngram_to_token, tokenize_visitor};

    fn collect_tokens(analyzer: &TextAnalyzer, text: &str) -> Vec<String> {
        let mut tokens = Vec::with_capacity(text.len() * 3);
        tokenize_visitor(analyzer, text, |token| tokens.push(token.to_owned()));
        tokens
    }

    #[test]
    fn test_tokenizer() {
        let tokenizer = NGRAM_TOKENIZER.clone();

        // ASCII folding
        let tokens = collect_tokens(&tokenizer, "café");
        assert_eq!(
            tokens,
            vec!["caf", "afe"] // spellchecker:disable-line
        );

        // Allow numbers
        let tokens = collect_tokens(&tokenizer, "a1b2");
        assert_eq!(tokens, vec!["a1b", "1b2"]);

        // Remove symbols and UTF-8 that doesn't map to characters
        let tokens = collect_tokens(&tokenizer, "abc👍b!c24");

        assert_eq!(tokens, vec!["abc", "c24"]);

        let tokens = collect_tokens(&tokenizer, "anstoß");

        assert_eq!(tokens, vec!["ans", "nst", "sto", "tos", "oss"]);

        // Lower casing
        let tokens = collect_tokens(&tokenizer, "ABC");
        assert_eq!(tokens, vec!["abc"]);

        // Duplicate tokens
        let tokens = collect_tokens(&tokenizer, "ababab");
        // Confirming that the tokenizer doesn't deduplicate tokens (this can be taken into consideration
        // when training the index)
        assert_eq!(
            tokens,
            vec!["aba", "bab", "aba", "bab"] // spellchecker:disable-line
        );
    }

    async fn do_train(
        mut builder: NGramIndexBuilder,
        data: SendableRecordBatchStream,
    ) -> (NGramIndex, Arc<TempDir>) {
        let spill_files = builder.train(data).await.unwrap();

        let tmpdir = Arc::new(TempDir::default());
        let test_store = LanceIndexStore::new(
            Arc::new(ObjectStore::local()),
            tmpdir.obj_path(),
            Arc::new(LanceCache::no_cache()),
        );

        builder
            .write_index(&test_store, spill_files, None)
            .await
            .unwrap();

        (
            NGramIndex::from_store(Arc::new(test_store), None, &LanceCache::no_cache())
                .await
                .unwrap(),
            tmpdir,
        )
    }

    async fn get_posting_list_for_trigram(index: &NGramIndex, trigram: &str) -> Vec<u64> {
        let token = ngram_to_token(trigram, 3);
        let row_offset = index.tokens[&token];
        let list = index
            .list_reader
            .ngram_list(row_offset, &NoOpMetricsCollector)
            .await
            .unwrap();
        list.bitmap.iter().sorted().collect()
    }

    async fn get_null_posting_list(index: &NGramIndex) -> Vec<u64> {
        let row_offset = index.tokens[&0];
        let list = index
            .list_reader
            .ngram_list(row_offset, &NoOpMetricsCollector)
            .await
            .unwrap();
        list.bitmap.iter().sorted().collect()
    }

    #[test_log::test(tokio::test)]
    async fn test_basic_ngram_index() {
        let data = StringArray::from_iter_values([
            "cat",
            "dog",
            "cat dog",
            "dog cat",
            "elephant",
            "mouse",
            "rhino",
            "giraffe",
            "rhinos nose",
        ]);
        let row_ids = UInt64Array::from_iter_values((0..data.len()).map(|i| i as u64));
        let schema = Arc::new(Schema::new(vec![
            Field::new(VALUE_COLUMN_NAME, DataType::Utf8, false),
            Field::new(ROW_ID, DataType::UInt64, false),
        ]));
        let data =
            RecordBatch::try_new(schema.clone(), vec![Arc::new(data), Arc::new(row_ids)]).unwrap();
        let data = Box::pin(RecordBatchStreamAdapter::new(
            schema,
            stream::once(std::future::ready(Ok(data))),
        ));

        let builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions::default()).unwrap();

        let (index, _tmpdir) = do_train(builder, data).await;
        assert_eq!(index.tokens.len(), 21);

        // Basic search
        let res = index
            .search(
                &TextQuery::StringContains("cat".to_string()),
                &NoOpMetricsCollector,
            )
            .await
            .unwrap();

        let expected = SearchResult::at_most(RowAddrTreeMap::from_iter([0, 2, 3]));

        assert_eq!(expected, res);

        // Whitespace in query
        let res = index
            .search(
                &TextQuery::StringContains("nos nos".to_string()),
                &NoOpMetricsCollector,
            )
            .await
            .unwrap();
        let expected = SearchResult::at_most(RowAddrTreeMap::from_iter([8]));
        assert_eq!(expected, res);

        // No matches
        let res = index
            .search(
                &TextQuery::StringContains("tdo".to_string()),
                &NoOpMetricsCollector,
            )
            .await
            .unwrap();
        let expected = SearchResult::exact(RowAddrTreeMap::new());
        assert_eq!(expected, res);

        // False positive
        let res = index
            .search(
                &TextQuery::StringContains("inose".to_string()),
                &NoOpMetricsCollector,
            )
            .await
            .unwrap();
        let expected = SearchResult::at_most(RowAddrTreeMap::from_iter([8]));
        assert_eq!(expected, res);

        // Too short, don't know anything
        let res = index
            .search(
                &TextQuery::StringContains("ab".to_string()),
                &NoOpMetricsCollector,
            )
            .await
            .unwrap();
        let expected = SearchResult::at_least(RowAddrTreeMap::new());
        assert_eq!(expected, res);

        // One short string but we still get at least one trigram, this is ok
        let res = index
            .search(
                &TextQuery::StringContains("no nos".to_string()),
                &NoOpMetricsCollector,
            )
            .await
            .unwrap();
        let expected = SearchResult::at_most(RowAddrTreeMap::from_iter([8]));
        assert_eq!(expected, res);
    }

    fn test_data_schema() -> Arc<Schema> {
        Arc::new(Schema::new(vec![
            Field::new(VALUE_COLUMN_NAME, DataType::Utf8, true),
            Field::new(ROW_ID, DataType::UInt64, false),
        ]))
    }

    fn simple_data_with_nulls() -> SendableRecordBatchStream {
        let data = StringArray::from_iter(&[Some("cat"), Some("dog"), None, None, Some("cat dog")]);
        let row_ids = UInt64Array::from_iter_values((0..data.len()).map(|i| i as u64));
        let schema = test_data_schema();
        let data =
            RecordBatch::try_new(schema.clone(), vec![Arc::new(data), Arc::new(row_ids)]).unwrap();
        Box::pin(RecordBatchStreamAdapter::new(
            schema,
            stream::once(std::future::ready(Ok(data))),
        ))
    }

    #[test_log::test(tokio::test)]
    async fn test_ngram_nulls() {
        let data = simple_data_with_nulls();

        let builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions::default()).unwrap();

        let (index, _tmpdir) = do_train(builder, data).await;
        assert_eq!(index.tokens.len(), 3);

        let res = index
            .search(
                &TextQuery::StringContains("cat".to_string()),
                &NoOpMetricsCollector,
            )
            .await
            .unwrap();
        let expected = SearchResult::at_most(RowAddrTreeMap::from_iter([0, 4]));
        assert_eq!(expected, res);

        let null_posting_list = get_null_posting_list(&index).await;
        assert_eq!(null_posting_list, vec![2, 3]);

        // TODO: Support IS NULL queries
    }

    fn empty_data() -> SendableRecordBatchStream {
        Box::pin(RecordBatchStreamAdapter::new(
            test_data_schema(),
            stream::empty::<lance_core::error::DataFusionResult<RecordBatch>>(),
        ))
    }

    #[test_log::test(tokio::test)]
    async fn test_train_empty() {
        let builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions::default()).unwrap();

        let (index, _tmpdir) = do_train(builder, empty_data()).await;
        assert_eq!(index.tokens.len(), 0);
    }

    #[test_log::test(tokio::test)]
    async fn test_update_empty() {
        let data = simple_data_with_nulls();

        let builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions::default()).unwrap();
        let (index, _tmpdir) = do_train(builder, empty_data()).await;

        let new_tmpdir = Arc::new(TempDir::default());
        let test_store = Arc::new(LanceIndexStore::new(
            Arc::new(ObjectStore::local()),
            new_tmpdir.obj_path(),
            Arc::new(LanceCache::no_cache()),
        ));

        index.update(data, test_store.as_ref(), None).await.unwrap();

        let index = NGramIndex::from_store(test_store, None, &LanceCache::no_cache())
            .await
            .unwrap();
        assert_eq!(index.tokens.len(), 3);
    }

    async fn row_ids_in_index(index: &NGramIndex) -> Vec<u64> {
        let mut row_ids = HashSet::new();
        for row_offset in index.tokens.values() {
            let list = index
                .list_reader
                .ngram_list(*row_offset, &NoOpMetricsCollector)
                .await
                .unwrap();
            row_ids.extend(list.bitmap.iter());
        }
        row_ids.into_iter().sorted().collect()
    }

    #[test_log::test(tokio::test)]
    async fn test_ngram_index_remap() {
        let data = simple_data_with_nulls();
        let builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions::default()).unwrap();
        let (index, _tmpdir) = do_train(builder, data).await;

        let row_ids = row_ids_in_index(&index).await;
        assert_eq!(row_ids, vec![0, 1, 2, 3, 4]);

        let new_tmpdir = Arc::new(TempDir::default());
        let test_store = Arc::new(LanceIndexStore::new(
            Arc::new(ObjectStore::local()),
            new_tmpdir.obj_path(),
            Arc::new(LanceCache::no_cache()),
        ));

        let remapping = HashMap::from([(2, Some(100)), (3, None), (4, Some(101))]);
        index.remap(&remapping, test_store.as_ref()).await.unwrap();

        let index = NGramIndex::from_store(test_store, None, &LanceCache::no_cache())
            .await
            .unwrap();
        let row_ids = row_ids_in_index(&index).await;
        assert_eq!(row_ids, vec![0, 1, 100, 101]);

        let null_posting_list = get_null_posting_list(&index).await;
        assert_eq!(null_posting_list, vec![100]);
    }

    #[test_log::test(tokio::test)]
    async fn test_ngram_index_merge() {
        let data = simple_data_with_nulls();
        let builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions::default()).unwrap();
        let (index, _tmpdir) = do_train(builder, data).await;

        let data = StringArray::from_iter(&[Some("giraffe"), Some("cat"), None]);
        let row_ids = UInt64Array::from_iter_values((0..data.len()).map(|i| i as u64 + 100));
        let schema = Arc::new(Schema::new(vec![
            Field::new(VALUE_COLUMN_NAME, DataType::Utf8, true),
            Field::new(ROW_ID, DataType::UInt64, false),
        ]));
        let data =
            RecordBatch::try_new(schema.clone(), vec![Arc::new(data), Arc::new(row_ids)]).unwrap();
        let data = Box::pin(RecordBatchStreamAdapter::new(
            schema,
            stream::once(std::future::ready(Ok(data))),
        ));

        let posting_list = get_posting_list_for_trigram(&index, "cat").await;
        assert_eq!(posting_list, vec![0, 4]);

        let new_tmpdir = Arc::new(TempDir::default());
        let test_store = Arc::new(LanceIndexStore::new(
            Arc::new(ObjectStore::local()),
            new_tmpdir.obj_path(),
            Arc::new(LanceCache::no_cache()),
        ));

        index.update(data, test_store.as_ref(), None).await.unwrap();

        let index = NGramIndex::from_store(test_store, None, &LanceCache::no_cache())
            .await
            .unwrap();
        let row_ids = row_ids_in_index(&index).await;
        assert_eq!(row_ids, vec![0, 1, 2, 3, 4, 100, 101, 102]);

        let posting_list = get_posting_list_for_trigram(&index, "cat").await;
        assert_eq!(posting_list, vec![0, 4, 101]);

        let posting_list = get_posting_list_for_trigram(&index, "ffe").await;
        assert_eq!(posting_list, vec![100]);

        let posting_list = get_null_posting_list(&index).await;
        assert_eq!(posting_list, vec![2, 3, 102]);
    }

    #[test_log::test(tokio::test)]
    async fn test_ngram_index_with_spill() {
        let (data, schema) = lance_datagen::gen_batch()
            .col(
                VALUE_COLUMN_NAME,
                lance_datagen::array::rand_utf8(ByteCount::from(50), false),
            )
            .col(ROW_ID, lance_datagen::array::step::<UInt64Type>())
            .into_reader_stream(RowCount::from(128), BatchCount::from(32));

        let data = Box::pin(RecordBatchStreamAdapter::new(
            schema,
            data.map_err(|arrow_err| DataFusionError::ArrowError(Box::new(arrow_err), None)),
        ));

        let builder = NGramIndexBuilder::try_new(NGramIndexBuilderOptions {
            tokens_per_spill: 100,
        })
        .unwrap();

        let (index, _tmpdir) = do_train(builder, data).await;

        assert_eq!(index.tokens.len(), 29012);
    }
}