lance 7.0.0

A columnar data format that is 100x faster than Parquet for random access.
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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

use std::cmp::{Ordering as CmpOrdering, Reverse};
use std::collections::{BinaryHeap, HashMap};
use std::ops::Range;
use std::sync::atomic::{AtomicPtr, AtomicU16, AtomicU32, AtomicU64, AtomicUsize, Ordering};
use std::sync::{Arc, Mutex, MutexGuard};

use arrow_array::builder::{Float32Builder, ListBuilder, UInt32Builder};
use arrow_array::{ArrayRef, RecordBatch};
use arrow_schema::{DataType, Field, Schema, SchemaRef};
use crossbeam_queue::ArrayQueue;
use lance_core::{Error, Result};
use rand::rngs::SmallRng;
use rand::{Rng, SeedableRng};
use rayon::prelude::*;
use serde::{Deserialize, Serialize};

use super::storage::VectorSource;

const HNSW_METADATA_KEY: &str = "lance:hnsw";
const VECTOR_ID_COL: &str = "__vector_id";
const NEIGHBORS_COL: &str = "__neighbors";
const DIST_COL: &str = "_distance";
const DEFAULT_SEED: u64 = 100;
const WORD_BITS: usize = usize::BITS as usize;

/// Parameters for HNSW graph construction.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub struct BuildParams {
    /// Maximum number of graph levels.
    pub max_level: u16,
    /// Number of neighbors retained on upper levels. Level 0 retains `2 * m`.
    pub m: usize,
    /// Beam width used while inserting nodes.
    pub ef_construction: usize,
    /// Lance-compatible prefetch hint retained in exported metadata.
    pub prefetch_distance: Option<usize>,
    /// Random seed used for deterministic level assignment.
    #[serde(skip, default = "default_seed")]
    pub seed: u64,
}

impl Default for BuildParams {
    fn default() -> Self {
        Self {
            max_level: 7,
            m: 20,
            ef_construction: 150,
            prefetch_distance: Some(2),
            seed: DEFAULT_SEED,
        }
    }
}

impl BuildParams {
    /// Defaults intended for active MemTable indexing.
    pub fn mem_wal_default() -> Self {
        Self {
            m: 12,
            ef_construction: 64,
            ..Self::default()
        }
    }

    /// Set the maximum number of graph levels.
    pub fn max_level(mut self, max_level: u16) -> Self {
        self.max_level = max_level;
        self
    }

    /// Set the HNSW `M` parameter.
    pub fn num_edges(mut self, m: usize) -> Self {
        self.m = m;
        self
    }

    /// Set the construction beam width.
    pub fn ef_construction(mut self, ef_construction: usize) -> Self {
        self.ef_construction = ef_construction;
        self
    }

    /// Set the deterministic level-assignment seed.
    pub fn seed(mut self, seed: u64) -> Self {
        self.seed = seed;
        self
    }

    fn validate(&self) -> Result<()> {
        if self.max_level == 0 {
            return Err(Error::invalid_input("max_level must be greater than 0"));
        }
        if self.max_level as u32 > u64::BITS {
            return Err(Error::invalid_input(format!(
                "max_level must be <= {}, got {}",
                u64::BITS,
                self.max_level
            )));
        }
        if self.m == 0 {
            return Err(Error::invalid_input("m must be greater than 0"));
        }
        if self.ef_construction < self.m {
            return Err(Error::invalid_input(format!(
                "ef_construction must be >= m, got ef_construction={} and m={}",
                self.ef_construction, self.m
            )));
        }
        Ok(())
    }
}

fn default_seed() -> u64 {
    DEFAULT_SEED
}

/// Query parameters for graph search.
#[derive(Debug, Clone, Copy)]
pub struct SearchParams {
    /// Number of nearest neighbors to return.
    pub k: usize,
    /// Beam width used by the bottom-level search.
    pub ef: usize,
}

impl SearchParams {
    /// Create search params, using `max(k, ef)` internally.
    pub fn new(k: usize, ef: usize) -> Self {
        Self { k, ef }
    }
}

/// Candidate with an internal vector id and distance.
#[derive(Debug, Clone, Copy, PartialEq)]
pub struct ScoredPoint {
    pub id: u32,
    pub distance: f32,
}

impl ScoredPoint {
    fn new(id: u32, distance: f32) -> Self {
        Self { id, distance }
    }
}

impl Eq for ScoredPoint {}

impl PartialOrd for ScoredPoint {
    fn partial_cmp(&self, other: &Self) -> Option<CmpOrdering> {
        Some(self.cmp(other))
    }
}

impl Ord for ScoredPoint {
    fn cmp(&self, other: &Self) -> CmpOrdering {
        self.distance
            .partial_cmp(&other.distance)
            .unwrap_or(CmpOrdering::Equal)
    }
}

/// Search result with Lance row id attached.
#[derive(Debug, Clone, Copy, PartialEq)]
pub struct SearchResult {
    pub id: u32,
    pub row_id: u64,
    pub distance: f32,
}

#[derive(Debug, Clone, Copy)]
struct BeamLimits {
    ef: usize,
    output_limit: usize,
}

#[derive(Debug, Clone, Copy)]
struct BuildBeamLimits {
    ef: usize,
    visible_len: usize,
    visited_capacity: usize,
}

/// Metadata stored under `lance:hnsw` in Lance HNSW batches.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub struct LanceHnswMetadata {
    pub entry_point: u32,
    pub params: BuildParams,
    pub level_offsets: Vec<usize>,
}

struct PublishedNeighbors {
    ptr: *const u32,
    len: usize,
    _neighbors: Arc<Vec<u32>>,
}

// SAFETY: published neighbor descriptors point into immutable `Arc<Vec<u32>>`
// allocations owned by the descriptor. The pointer and length never change
// after publication.
unsafe impl Send for PublishedNeighbors {}
unsafe impl Sync for PublishedNeighbors {}

struct PackedLevel {
    offsets: Vec<usize>,
    neighbors: Vec<u32>,
}

impl PackedLevel {
    fn empty() -> Self {
        Self {
            offsets: vec![0],
            neighbors: Vec::new(),
        }
    }

    fn neighbors(&self, id: u32) -> Option<&[u32]> {
        let idx = id as usize;
        let start = *self.offsets.get(idx)?;
        let end = *self.offsets.get(idx + 1)?;
        Some(&self.neighbors[start..end])
    }
}

// SAFETY: packed levels are immutable after publication and old snapshots are
// retained until graph drop.
unsafe impl Send for PackedLevel {}
unsafe impl Sync for PackedLevel {}

struct LevelLinks {
    published: AtomicPtr<PublishedNeighbors>,
    #[allow(clippy::vec_box)]
    retired_published: Mutex<Vec<Box<PublishedNeighbors>>>,
    ranked: Mutex<Vec<ScoredPoint>>,
}

impl LevelLinks {
    fn new(capacity: usize) -> Self {
        let neighbors = Arc::new(Vec::new());
        let published = Box::into_raw(Box::new(PublishedNeighbors {
            ptr: neighbors.as_ptr(),
            len: neighbors.len(),
            _neighbors: neighbors,
        }));
        Self {
            published: AtomicPtr::new(published),
            retired_published: Mutex::new(Vec::new()),
            ranked: Mutex::new(Vec::with_capacity(capacity)),
        }
    }

    fn publish_from_ranked(&self, ranked: &[ScoredPoint]) -> Result<()> {
        let neighbors = Arc::new(ranked.iter().map(|point| point.id).collect::<Vec<_>>());
        let published = Box::into_raw(Box::new(PublishedNeighbors {
            ptr: neighbors.as_ptr(),
            len: neighbors.len(),
            _neighbors: neighbors,
        }));
        let old = self.published.swap(published, Ordering::AcqRel);
        self.retired_published
            .lock()
            .map_err(|_| Error::internal("HNSW published-neighbor mutex poisoned"))?
            // SAFETY: `old` was produced by `Box::into_raw` either in
            // `new` or a previous `publish_from_ranked` call. Keeping the box
            // in `retired_published` lets readers that already loaded it
            // finish safely.
            .push(unsafe { Box::from_raw(old) });
        Ok(())
    }
}

impl Drop for LevelLinks {
    fn drop(&mut self) {
        let published = *self.published.get_mut();
        if !published.is_null() {
            // SAFETY: `drop` has exclusive access to the graph. Retired
            // descriptors are owned by `retired_published`; this pointer is
            // the one currently installed in `published`.
            unsafe {
                drop(Box::from_raw(published));
            }
        }
    }
}

struct Node {
    target_level: u16,
    levels: Vec<LevelLinks>,
    dirty_levels: AtomicU64,
}

impl Node {
    fn new(target_level: u16, m: usize) -> Self {
        let levels = (0..=target_level)
            .map(|level| {
                let max_neighbors = if level == 0 { m * 2 } else { m };
                LevelLinks::new(max_neighbors)
            })
            .collect();
        Self {
            target_level,
            levels,
            dirty_levels: AtomicU64::new(0),
        }
    }

    fn has_level(&self, level: u16) -> bool {
        (level as usize) < self.levels.len()
    }

    fn ranked(&self, level: u16) -> Result<MutexGuard<'_, Vec<ScoredPoint>>> {
        self.levels[level as usize]
            .ranked
            .lock()
            .map_err(|_| Error::internal("HNSW neighbor mutex poisoned"))
    }

    fn mark_dirty(&self, level: u16) {
        self.dirty_levels
            .fetch_or(1_u64 << level, Ordering::Release);
    }
}

/// Multi-reader / single-writer HNSW graph.
///
/// Public readers search only the published visible prefix. A writer may build
/// a new contiguous id range with worker threads and then publish that range
/// atomically when the batch completes.
pub struct HnswGraph {
    params: BuildParams,
    nodes: Vec<Node>,
    build_entry_point: AtomicU32,
    build_max_level: AtomicU16,
    visible_entry_point: AtomicU32,
    visible_max_level: AtomicU16,
    indexed_len: AtomicUsize,
    visible_len: AtomicUsize,
    visited_pool: ArrayQueue<VisitedList>,
    packed_level0: AtomicPtr<PackedLevel>,
    #[allow(clippy::vec_box)]
    retired_packed_level0: Mutex<Vec<Box<PackedLevel>>>,
}

impl HnswGraph {
    /// Pre-allocate graph nodes and their random levels.
    pub fn try_new(capacity: usize, params: BuildParams) -> Result<Self> {
        params.validate()?;
        if capacity == 0 {
            return Err(Error::invalid_input("capacity must be greater than 0"));
        }
        if capacity > u32::MAX as usize {
            return Err(Error::invalid_input(format!(
                "capacity must fit in u32, got {capacity}"
            )));
        }

        let mut rng = SmallRng::seed_from_u64(params.seed);
        let mut nodes = Vec::with_capacity(capacity);
        for id in 0..capacity {
            let target_level = if id == 0 {
                0
            } else {
                random_level(&params, &mut rng)
            };
            nodes.push(Node::new(target_level, params.m));
        }

        let pool_size = rayon::current_num_threads().max(1) * 2;
        let visited_pool = ArrayQueue::new(pool_size);
        for _ in 0..pool_size {
            let _ = visited_pool.push(VisitedList::new(0));
        }

        let packed_level0 = Box::into_raw(Box::new(PackedLevel::empty()));

        Ok(Self {
            params,
            nodes,
            build_entry_point: AtomicU32::new(0),
            build_max_level: AtomicU16::new(0),
            visible_entry_point: AtomicU32::new(0),
            visible_max_level: AtomicU16::new(0),
            indexed_len: AtomicUsize::new(0),
            visible_len: AtomicUsize::new(0),
            visited_pool,
            packed_level0: AtomicPtr::new(packed_level0),
            retired_packed_level0: Mutex::new(Vec::new()),
        })
    }

    /// Number of nodes visible to readers.
    pub fn len(&self) -> usize {
        self.visible_len.load(Ordering::Acquire)
    }

    /// Returns true if no nodes are visible to readers.
    pub fn is_empty(&self) -> bool {
        self.len() == 0
    }

    /// Number of nodes already inserted into the graph, including unpublished
    /// in-flight writer batches after insertion completes.
    pub fn indexed_len(&self) -> usize {
        self.indexed_len.load(Ordering::Acquire)
    }

    /// Graph build parameters.
    pub fn params(&self) -> &BuildParams {
        &self.params
    }

    /// Insert and publish a single vector id.
    pub fn insert(&self, id: u32, vectors: &impl VectorSource) -> Result<()> {
        let expected = self.indexed_len.load(Ordering::Acquire);
        if id as usize != expected {
            return Err(Error::invalid_input(format!(
                "insert id must match indexed_len: id={}, indexed_len={expected}",
                id
            )));
        }
        self.validate_source(vectors, id as usize + 1)?;
        self.insert_inner(id, vectors)?;
        self.indexed_len.store(id as usize + 1, Ordering::Release);
        self.publish_dirty(id as usize + 1)?;
        self.publish_visible(id as usize + 1);
        Ok(())
    }

    /// Insert a contiguous id range using Rayon worker threads.
    ///
    /// Readers continue to search the previous visible prefix while this runs.
    /// The full range becomes visible only after every insert in the batch has
    /// completed.
    pub fn insert_batch(&self, ids: Range<u32>, vectors: &impl VectorSource) -> Result<()> {
        if ids.is_empty() {
            return Ok(());
        }
        let expected = self.indexed_len.load(Ordering::Acquire);
        if ids.start as usize != expected {
            return Err(Error::invalid_input(format!(
                "insert range must start at indexed_len: range_start={}, indexed_len={expected}",
                ids.start
            )));
        }
        self.validate_source(vectors, ids.end as usize)?;

        let parallel_start = if ids.start == 0 {
            self.insert_inner(0, vectors)?;
            1
        } else {
            ids.start
        };

        (parallel_start..ids.end)
            .into_par_iter()
            .try_for_each_init(
                || VisitedList::new(0),
                |visited, id| self.insert_inner_with_visited(id, vectors, visited),
            )?;

        self.indexed_len.store(ids.end as usize, Ordering::Release);
        self.publish_dirty(ids.end as usize)?;
        self.publish_visible(ids.end as usize);
        Ok(())
    }

    /// Insert a contiguous id range serially.
    ///
    /// This is useful when comparing against single-threaded baselines or
    /// debugging graph quality without scheduler effects.
    pub fn insert_batch_serial(&self, ids: Range<u32>, vectors: &impl VectorSource) -> Result<()> {
        if ids.is_empty() {
            return Ok(());
        }
        let expected = self.indexed_len.load(Ordering::Acquire);
        if ids.start as usize != expected {
            return Err(Error::invalid_input(format!(
                "insert range must start at indexed_len: range_start={}, indexed_len={expected}",
                ids.start
            )));
        }
        self.validate_source(vectors, ids.end as usize)?;

        let mut visited = VisitedList::new(0);
        for id in ids.clone() {
            self.insert_inner_with_visited(id, vectors, &mut visited)?;
        }

        self.indexed_len.store(ids.end as usize, Ordering::Release);
        self.publish_dirty(ids.end as usize)?;
        self.publish_visible(ids.end as usize);
        Ok(())
    }

    /// Insert a contiguous id range using the current thread.
    pub fn insert_batch_with_threads(
        &self,
        ids: Range<u32>,
        vectors: &impl VectorSource,
        threads: usize,
    ) -> Result<()> {
        if threads <= 1 {
            return self.insert_batch_serial(ids, vectors);
        }
        self.insert_batch(ids, vectors)
    }

    /// Search the published visible graph.
    pub fn search(
        &self,
        query: &[f32],
        params: SearchParams,
        vectors: &impl VectorSource,
    ) -> Result<Vec<SearchResult>> {
        if params.k == 0 {
            return Ok(Vec::new());
        }
        if query.len() != vectors.dim() {
            return Err(Error::invalid_input(format!(
                "query dimension mismatch: expected {}, got {}",
                vectors.dim(),
                query.len()
            )));
        }

        // The vector snapshot and graph publication are captured separately.
        // Cap search to the common visible prefix so a reader racing with the
        // writer never follows a newly-published graph edge into an older
        // vector snapshot.
        let visible_len = self.visible_len.load(Ordering::Acquire).min(vectors.len());
        if visible_len == 0 {
            return Ok(Vec::new());
        }
        let neighbor_visible_len = visible_len;
        let visible_max_level = self.visible_max_level.load(Ordering::Acquire);
        let entry = self.visible_entry_point.load(Ordering::Acquire);
        if entry as usize >= visible_len {
            return Ok(Vec::new());
        }

        let mut visited = self
            .visited_pool
            .pop()
            .unwrap_or_else(|| VisitedList::new(0));
        let mut ep = ScoredPoint::new(entry, vectors.distance_to(query, entry));
        for level in (1..visible_max_level).rev() {
            ep = self.greedy_search_query(ep, level, neighbor_visible_len, |id| {
                vectors.distance_to(query, id)
            });
        }

        let ef = params.ef.max(params.k);
        let limits = BeamLimits {
            ef,
            output_limit: params.k,
        };
        let candidates =
            self.beam_search_query(ep, 0, limits, neighbor_visible_len, &mut visited, |id| {
                vectors.distance_to(query, id)
            });
        let _ = self.visited_pool.push(visited);

        Ok(candidates
            .into_iter()
            .take(params.k)
            .map(|point| SearchResult {
                id: point.id,
                row_id: vectors.row_id(point.id),
                distance: point.distance,
            })
            .collect())
    }

    /// Emit the Lance HNSW sub-index record batch.
    ///
    /// The resulting batch uses the same schema and `lance:hnsw` metadata
    /// expected by `lance-index`'s `HNSW::load`.
    ///
    /// Call this when no writer batch is in flight. Ordinary search readers
    /// can run concurrently with insertion, but flush export should snapshot a
    /// completed graph prefix.
    pub fn to_lance_hnsw_batch(&self) -> Result<RecordBatch> {
        let visible_len = self.visible_len.load(Ordering::Acquire);
        let max_level = self.params.max_level as usize;
        let mut level_counts = vec![0usize; max_level];
        for id in 0..visible_len {
            let node = &self.nodes[id];
            for count in level_counts
                .iter_mut()
                .take(node.levels.len().min(max_level))
            {
                *count += 1;
            }
        }

        let total_rows: usize = level_counts.iter().sum();
        let mut vector_id_builder = UInt32Builder::with_capacity(total_rows);
        let mut neighbors_builder = ListBuilder::with_capacity(UInt32Builder::new(), total_rows);
        let mut distances_builder = ListBuilder::with_capacity(Float32Builder::new(), total_rows);

        for level in 0..max_level {
            for id in 0..visible_len {
                let node = &self.nodes[id];
                if level >= node.levels.len() {
                    continue;
                }
                let ranked = node.ranked(level as u16)?;
                vector_id_builder.append_value(id as u32);
                neighbors_builder.append_value(ranked.iter().map(|point| Some(point.id)));
                distances_builder.append_value(ranked.iter().map(|point| Some(point.distance)));
            }
        }

        let metadata = LanceHnswMetadata {
            entry_point: self.visible_entry_point.load(Ordering::Acquire),
            params: self.params.clone(),
            level_offsets: level_counts
                .iter()
                .chain(std::iter::once(&0))
                .scan(0, |state, count| {
                    let start = *state;
                    *state += *count;
                    Some(start)
                })
                .collect(),
        };
        let metadata = serde_json::to_string(&metadata)?;
        let schema = lance_hnsw_schema()
            .as_ref()
            .clone()
            .with_metadata(HashMap::from_iter([(
                HNSW_METADATA_KEY.to_string(),
                metadata,
            )]));
        RecordBatch::try_new(
            Arc::new(schema),
            vec![
                Arc::new(vector_id_builder.finish()) as ArrayRef,
                Arc::new(neighbors_builder.finish()) as ArrayRef,
                Arc::new(distances_builder.finish()) as ArrayRef,
            ],
        )
        .map_err(Error::from)
    }

    fn validate_source(&self, vectors: &impl VectorSource, needed_len: usize) -> Result<()> {
        if needed_len > self.nodes.len() {
            return Err(Error::invalid_input(format!(
                "graph capacity {} exhausted: need {needed_len}",
                self.nodes.len()
            )));
        }
        if vectors.len() < needed_len {
            return Err(Error::invalid_input(format!(
                "vector source has {} rows but graph insert needs {needed_len}",
                vectors.len()
            )));
        }
        Ok(())
    }

    fn insert_inner(&self, id: u32, vectors: &impl VectorSource) -> Result<()> {
        let mut visited = self
            .visited_pool
            .pop()
            .unwrap_or_else(|| VisitedList::new(0));
        let result = self.insert_inner_with_visited(id, vectors, &mut visited);
        let _ = self.visited_pool.push(visited);
        result
    }

    fn insert_inner_with_visited(
        &self,
        id: u32,
        vectors: &impl VectorSource,
        visited: &mut VisitedList,
    ) -> Result<()> {
        if id == 0 {
            let target_level = self.nodes[0].target_level;
            self.build_entry_point.store(0, Ordering::Release);
            self.build_max_level
                .store(target_level + 1, Ordering::Release);
            return Ok(());
        }

        let target_level = self.nodes[id as usize].target_level;
        let current_max_level = self.build_max_level.load(Ordering::Acquire).max(1);
        let entry = self.build_entry_point.load(Ordering::Acquire);
        let mut ep = ScoredPoint::new(entry, vectors.distance_between(id, entry));

        for level in (target_level + 1..current_max_level).rev() {
            ep = self.greedy_search_build(ep, level, usize::MAX, |candidate| {
                vectors.distance_between(id, candidate)
            })?;
        }

        let connect_max_level = target_level.min(current_max_level - 1);
        let mut selected_by_level: Vec<Vec<ScoredPoint>> =
            vec![Vec::new(); (target_level + 1) as usize];
        for level in (0..=connect_max_level).rev() {
            let candidates = self.beam_search_build(
                ep,
                level,
                BuildBeamLimits {
                    ef: self.params.ef_construction,
                    visible_len: usize::MAX,
                    visited_capacity: vectors.len(),
                },
                visited,
                |candidate| vectors.distance_between(id, candidate),
            )?;
            let candidates: Vec<_> = candidates
                .into_iter()
                .filter(|point| point.id != id && self.nodes[point.id as usize].has_level(level))
                .collect();

            let selected =
                self.select_neighbors(vectors, &candidates, max_neighbors(self.params.m, level));
            self.set_node_neighbors(id, level, selected.clone())?;
            if let Some(next) = selected.first().copied() {
                ep = next;
            }
            selected_by_level[level as usize] = selected;
        }

        for (level, selected) in selected_by_level.into_iter().enumerate() {
            let level = level as u16;
            for neighbor in selected {
                self.add_reverse_edge(vectors, neighbor.id, id, neighbor.distance, level)?;
            }
        }

        self.promote_build_entry(id, target_level);
        Ok(())
    }

    fn greedy_search_query<F>(
        &self,
        mut current: ScoredPoint,
        level: u16,
        visible_len: usize,
        distance: F,
    ) -> ScoredPoint
    where
        F: Fn(u32) -> f32,
    {
        loop {
            let mut next = None;
            self.visit_published_neighbors(current.id, level, visible_len, |neighbor| {
                let candidate_distance = distance(neighbor);
                if candidate_distance < current.distance
                    && next
                        .map(|point: ScoredPoint| candidate_distance < point.distance)
                        .unwrap_or(true)
                {
                    next = Some(ScoredPoint::new(neighbor, candidate_distance));
                }
            });

            let Some(next_point) = next else {
                break;
            };
            current = next_point;
        }
        current
    }

    fn beam_search_query<F>(
        &self,
        ep: ScoredPoint,
        level: u16,
        limits: BeamLimits,
        visible_len: usize,
        visited: &mut VisitedList,
        distance: F,
    ) -> Vec<ScoredPoint>
    where
        F: Fn(u32) -> f32,
    {
        let mut candidates = BinaryHeap::with_capacity(limits.ef);
        let mut results = BinaryHeap::with_capacity(limits.ef);
        let visited_capacity = if visible_len == usize::MAX {
            self.visible_len.load(Ordering::Acquire)
        } else {
            visible_len
        };
        visited.reset(visited_capacity);
        let _ = visited.insert(ep.id);
        candidates.push(Reverse(ep));
        results.push(ep);

        while let Some(Reverse(current)) = candidates.pop() {
            let furthest = results
                .peek()
                .map(|point| point.distance)
                .unwrap_or(f32::INFINITY);
            if current.distance > furthest && results.len() == limits.ef {
                break;
            }

            self.visit_published_neighbors(current.id, level, visible_len, |neighbor| {
                if !visited.insert(neighbor) {
                    return;
                }
                let candidate = ScoredPoint::new(neighbor, distance(neighbor));
                let furthest = results
                    .peek()
                    .map(|point| point.distance)
                    .unwrap_or(f32::INFINITY);
                if results.len() < limits.ef || candidate.distance < furthest {
                    if results.len() == limits.ef {
                        results.pop();
                    }
                    results.push(candidate);
                    candidates.push(Reverse(candidate));
                }
            });
        }

        let output_limit = limits.output_limit.min(results.len());
        while results.len() > output_limit {
            results.pop();
        }
        results.into_sorted_vec()
    }

    fn greedy_search_build<F>(
        &self,
        mut current: ScoredPoint,
        level: u16,
        visible_len: usize,
        distance: F,
    ) -> Result<ScoredPoint>
    where
        F: Fn(u32) -> f32,
    {
        loop {
            let mut next = None;
            self.visit_build_neighbors(current.id, level, visible_len, |neighbor| {
                let candidate_distance = distance(neighbor);
                if candidate_distance < current.distance
                    && next
                        .map(|point: ScoredPoint| candidate_distance < point.distance)
                        .unwrap_or(true)
                {
                    next = Some(ScoredPoint::new(neighbor, candidate_distance));
                }
            })?;

            let Some(next_point) = next else {
                break;
            };
            current = next_point;
        }
        Ok(current)
    }

    fn beam_search_build<F>(
        &self,
        ep: ScoredPoint,
        level: u16,
        limits: BuildBeamLimits,
        visited: &mut VisitedList,
        distance: F,
    ) -> Result<Vec<ScoredPoint>>
    where
        F: Fn(u32) -> f32,
    {
        let mut candidates = BinaryHeap::with_capacity(limits.ef);
        let mut results = BinaryHeap::with_capacity(limits.ef);
        let visited_capacity = if limits.visible_len == usize::MAX {
            limits.visited_capacity
        } else {
            limits.visible_len
        };
        visited.reset(visited_capacity);
        let _ = visited.insert(ep.id);
        candidates.push(Reverse(ep));
        results.push(ep);

        while let Some(Reverse(current)) = candidates.pop() {
            let furthest = results
                .peek()
                .map(|point| point.distance)
                .unwrap_or(f32::INFINITY);
            if current.distance > furthest && results.len() == limits.ef {
                break;
            }

            self.visit_build_neighbors(current.id, level, limits.visible_len, |neighbor| {
                if !visited.insert(neighbor) {
                    return;
                }
                let candidate = ScoredPoint::new(neighbor, distance(neighbor));
                let furthest = results
                    .peek()
                    .map(|point| point.distance)
                    .unwrap_or(f32::INFINITY);
                if results.len() < limits.ef || candidate.distance < furthest {
                    if results.len() == limits.ef {
                        results.pop();
                    }
                    results.push(candidate);
                    candidates.push(Reverse(candidate));
                }
            })?;
        }

        Ok(results.into_sorted_vec())
    }

    fn visit_published_neighbors<F>(&self, id: u32, level: u16, visible_len: usize, mut visit: F)
    where
        F: FnMut(u32),
    {
        let node = &self.nodes[id as usize];
        if !node.has_level(level) {
            return;
        }
        if level == 0 {
            let packed = self.packed_level0.load(Ordering::Acquire);
            // SAFETY: packed snapshots are immutable and retained until graph
            // drop.
            let packed = unsafe { &*packed };
            if let Some(neighbors) = packed.neighbors(id) {
                if visible_len == usize::MAX {
                    for neighbor in neighbors.iter().copied() {
                        visit(neighbor);
                    }
                    return;
                }
                for neighbor in neighbors.iter().copied() {
                    if neighbor as usize >= visible_len {
                        continue;
                    }
                    visit(neighbor);
                }
                return;
            }
        }
        let published = node.levels[level as usize]
            .published
            .load(Ordering::Acquire);
        // SAFETY: published descriptors are immutable and old descriptors are
        // retained until the graph is dropped.
        let published = unsafe { &*published };
        // SAFETY: the descriptor owns the immutable Arc<Vec<u32>> backing this
        // pointer, and the pointer/length pair was created from that Vec.
        let neighbors = unsafe { std::slice::from_raw_parts(published.ptr, published.len) };
        if visible_len == usize::MAX {
            for neighbor in neighbors.iter().copied() {
                visit(neighbor);
            }
            return;
        }
        for neighbor in neighbors.iter().copied() {
            if neighbor as usize >= visible_len {
                continue;
            }
            visit(neighbor);
        }
    }

    fn visit_build_neighbors<F>(
        &self,
        id: u32,
        level: u16,
        visible_len: usize,
        mut visit: F,
    ) -> Result<()>
    where
        F: FnMut(u32),
    {
        let node = &self.nodes[id as usize];
        if !node.has_level(level) {
            return Ok(());
        }
        let ranked = node.ranked(level)?;
        if visible_len == usize::MAX {
            for neighbor in ranked.iter().map(|point| point.id) {
                visit(neighbor);
            }
            return Ok(());
        }
        for neighbor in ranked.iter().map(|point| point.id) {
            if neighbor as usize >= visible_len {
                continue;
            }
            visit(neighbor);
        }
        Ok(())
    }

    fn select_neighbors(
        &self,
        vectors: &impl VectorSource,
        candidates: &[ScoredPoint],
        limit: usize,
    ) -> Vec<ScoredPoint> {
        if candidates.len() <= limit {
            let mut candidates = candidates.to_vec();
            candidates.sort_unstable();
            return candidates;
        }

        let mut candidates = candidates.to_vec();
        candidates.sort_unstable();
        let mut selected = Vec::with_capacity(limit);
        for candidate in candidates {
            if selected.len() == limit {
                break;
            }
            if selected.is_empty() || vectors.prefers_candidate(candidate, &selected) {
                selected.push(candidate);
            }
        }
        selected
    }

    fn set_node_neighbors(&self, id: u32, level: u16, neighbors: Vec<ScoredPoint>) -> Result<()> {
        let node = &self.nodes[id as usize];
        let mut ranked = node.ranked(level)?;
        *ranked = neighbors;
        node.mark_dirty(level);
        Ok(())
    }

    fn add_reverse_edge(
        &self,
        vectors: &impl VectorSource,
        target: u32,
        neighbor: u32,
        distance: f32,
        level: u16,
    ) -> Result<()> {
        let node = &self.nodes[target as usize];
        if !node.has_level(level) {
            return Ok(());
        }
        let mut ranked = node.ranked(level)?;
        if ranked.iter().any(|point| point.id == neighbor) {
            return Ok(());
        }
        ranked.push(ScoredPoint::new(neighbor, distance));
        let limit = max_neighbors(self.params.m, level);
        if ranked.len() > limit {
            *ranked = self.select_neighbors(vectors, &ranked, limit);
        } else {
            ranked.sort_unstable();
        }
        node.mark_dirty(level);
        Ok(())
    }

    fn promote_build_entry(&self, id: u32, target_level: u16) {
        loop {
            let entry = self.build_entry_point.load(Ordering::Acquire);
            let entry_level = self.nodes[entry as usize].target_level;
            if target_level <= entry_level {
                break;
            }
            if self
                .build_entry_point
                .compare_exchange(entry, id, Ordering::AcqRel, Ordering::Acquire)
                .is_ok()
            {
                self.build_max_level
                    .fetch_max(target_level + 1, Ordering::AcqRel);
                break;
            }
        }
    }

    fn publish_visible(&self, len: usize) {
        let entry = self.build_entry_point.load(Ordering::Acquire);
        self.visible_entry_point.store(entry, Ordering::Release);
        self.visible_max_level.store(
            self.build_max_level.load(Ordering::Acquire),
            Ordering::Release,
        );
        self.visible_len.store(len, Ordering::Release);
    }

    fn publish_dirty(&self, len: usize) -> Result<()> {
        let mut has_level0_update = false;
        for node in self.nodes.iter().take(len) {
            let mut mask = node.dirty_levels.swap(0, Ordering::AcqRel);
            while mask != 0 {
                let level = mask.trailing_zeros() as usize;
                mask &= mask - 1;
                if level >= node.levels.len() {
                    continue;
                }
                let ranked = node.ranked(level as u16)?;
                node.levels[level].publish_from_ranked(&ranked)?;
                has_level0_update |= level == 0;
            }
        }
        if has_level0_update {
            self.rebuild_packed_level0(len)?;
        }
        Ok(())
    }

    fn rebuild_packed_level0(&self, len: usize) -> Result<()> {
        let mut offsets = Vec::with_capacity(len + 1);
        let mut neighbors = Vec::with_capacity(len.saturating_mul(max_neighbors(self.params.m, 0)));
        offsets.push(0);
        for node in self.nodes.iter().take(len) {
            let ranked = node.ranked(0)?;
            neighbors.extend(ranked.iter().map(|point| point.id));
            offsets.push(neighbors.len());
        }

        let packed = Box::into_raw(Box::new(PackedLevel { offsets, neighbors }));
        let old = self.packed_level0.swap(packed, Ordering::AcqRel);
        self.retired_packed_level0
            .lock()
            .map_err(|_| Error::internal("HNSW packed-level mutex poisoned"))?
            // SAFETY: `old` was produced by `Box::into_raw` and is retained so
            // readers that already loaded it can finish safely.
            .push(unsafe { Box::from_raw(old) });
        Ok(())
    }
}

impl Drop for HnswGraph {
    fn drop(&mut self) {
        let packed = *self.packed_level0.get_mut();
        if !packed.is_null() {
            // SAFETY: `drop` has exclusive access to the graph. Retired packed
            // snapshots are owned by `retired_packed_level0`.
            unsafe {
                drop(Box::from_raw(packed));
            }
        }
    }
}

fn lance_hnsw_schema() -> SchemaRef {
    Arc::new(Schema::new(vec![
        Field::new(VECTOR_ID_COL, DataType::UInt32, true),
        Field::new(
            NEIGHBORS_COL,
            DataType::List(Field::new_list_field(DataType::UInt32, true).into()),
            true,
        ),
        Field::new(
            DIST_COL,
            DataType::List(Field::new_list_field(DataType::Float32, true).into()),
            true,
        ),
    ]))
}

fn random_level(params: &BuildParams, rng: &mut SmallRng) -> u16 {
    let ml = 1.0 / (params.m as f32).ln();
    ((-rng.random::<f32>().ln() * ml) as u16).min(params.max_level - 1)
}

fn max_neighbors(m: usize, level: u16) -> usize {
    if level == 0 { m * 2 } else { m }
}

#[derive(Debug)]
struct VisitedList {
    words: Vec<usize>,
    touched: Vec<u32>,
}

impl VisitedList {
    fn new(capacity: usize) -> Self {
        Self {
            words: vec![0; capacity.div_ceil(WORD_BITS)],
            touched: Vec::new(),
        }
    }

    fn reset(&mut self, capacity: usize) {
        for id in self.touched.drain(..) {
            let idx = id as usize;
            self.words[idx / WORD_BITS] &= !(1usize << (idx % WORD_BITS));
        }
        let needed_words = capacity.div_ceil(WORD_BITS);
        if self.words.len() < needed_words {
            self.words.resize(needed_words, 0);
        }
    }

    fn insert(&mut self, id: u32) -> bool {
        let idx = id as usize;
        let word = idx / WORD_BITS;
        let bit = 1usize << (idx % WORD_BITS);
        if self.words[word] & bit == 0 {
            self.words[word] |= bit;
            self.touched.push(id);
            true
        } else {
            false
        }
    }
}

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

    use arrow_array::{ArrayRef, FixedSizeListArray, Float32Array};
    use arrow_schema::{DataType, Field};
    use lance_index::vector::hnsw::HNSW;
    use lance_index::vector::v3::subindex::IvfSubIndex;
    use lance_linalg::distance::DistanceType;

    use super::super::{ArrowFixedSizeListVectorStore, VectorSource};
    use super::*;

    fn fsl(rows: usize, dim: usize) -> Arc<FixedSizeListArray> {
        let mut values = Vec::with_capacity(rows * dim);
        for row in 0..rows {
            for col in 0..dim {
                values.push(row as f32 + col as f32 * 0.001);
            }
        }
        let values = Arc::new(Float32Array::from(values)) as ArrayRef;
        Arc::new(
            FixedSizeListArray::try_new(
                Arc::new(Field::new("item", DataType::Float32, true)),
                dim as i32,
                values,
                None,
            )
            .unwrap(),
        )
    }

    #[test]
    fn test_parallel_insert_searches_visible_graph() {
        let rows = 256;
        let dim = 16;
        let store = Arc::new(
            ArrowFixedSizeListVectorStore::try_new(512, 4, dim, DistanceType::L2).unwrap(),
        );
        let ids = store.append_batch(fsl(rows, dim), 100).unwrap();
        let snapshot = store.snapshot();
        let graph = HnswGraph::try_new(
            512,
            BuildParams::mem_wal_default()
                .num_edges(8)
                .ef_construction(32)
                .seed(7),
        )
        .unwrap();

        graph.insert_batch(ids, &snapshot).unwrap();
        assert_eq!(graph.len(), rows);

        let query = snapshot.vector(42);
        let result = graph
            .search(query, SearchParams::new(5, 32), &snapshot)
            .unwrap();
        assert!(result.iter().any(|point| point.id == 42));
    }

    #[test]
    fn test_lance_hnsw_batch_loads_with_lance_index() {
        let rows = 64;
        let dim = 8;
        let store = Arc::new(
            ArrowFixedSizeListVectorStore::try_new(128, 2, dim, DistanceType::L2).unwrap(),
        );
        let ids = store.append_batch(fsl(rows, dim), 0).unwrap();
        let snapshot = store.snapshot();
        let graph = HnswGraph::try_new(
            128,
            BuildParams::mem_wal_default()
                .num_edges(8)
                .ef_construction(32)
                .seed(11),
        )
        .unwrap();
        graph.insert_batch(ids, &snapshot).unwrap();

        let batch = graph.to_lance_hnsw_batch().unwrap();
        let loaded = HNSW::load(batch).unwrap();
        assert_eq!(loaded.len(), rows);
    }
}