lance 4.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
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

use std::any::Any;
use std::collections::{HashMap, HashSet};
use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering};
use std::sync::{Arc, LazyLock, Mutex};
use std::time::Instant;

use arrow::array::Float32Builder;
use arrow::datatypes::{Float32Type, UInt32Type, UInt64Type};
use arrow_array::{Array, Float32Array, UInt32Array, UInt64Array};
use arrow_array::{
    ArrayRef, BooleanArray, RecordBatch, StringArray,
    builder::{ListBuilder, UInt32Builder},
    cast::AsArray,
};
use arrow_schema::{DataType, Field, Schema, SchemaRef};
use datafusion::physical_plan::PlanProperties;
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
use datafusion::physical_plan::{
    DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream,
    Statistics,
};
use datafusion::{common::ColumnStatistics, physical_plan::metrics::ExecutionPlanMetricsSet};
use datafusion::{
    common::stats::Precision,
    physical_plan::execution_plan::{Boundedness, EmissionType},
};
use datafusion::{
    error::{DataFusionError, Result as DataFusionResult},
    physical_plan::metrics::MetricsSet,
};
use datafusion_physical_expr::{Distribution, EquivalenceProperties};
use datafusion_physical_plan::metrics::{BaselineMetrics, Count};
use futures::{Stream, StreamExt, TryFutureExt, TryStreamExt, future, stream};
use itertools::Itertools;
use lance_core::ROW_ID;
use lance_core::utils::futures::FinallyStreamExt;
use lance_core::{ROW_ID_FIELD, utils::tokio::get_num_compute_intensive_cpus};
use lance_datafusion::utils::{
    DELTAS_SEARCHED_METRIC, ExecutionPlanMetricsSetExt, PARTITIONS_RANKED_METRIC,
    PARTITIONS_SEARCHED_METRIC,
};
use lance_index::prefilter::PreFilter;
use lance_index::vector::{
    DIST_COL, INDEX_UUID_COLUMN, PART_ID_COLUMN, Query, flat::compute_distance,
};
use lance_index::vector::{DIST_Q_C_COLUMN, VectorIndex};
use lance_linalg::distance::DistanceType;
use lance_linalg::kernels::normalize_arrow;
use lance_table::format::IndexMetadata;
use tokio::sync::Notify;

use crate::dataset::Dataset;
use crate::index::DatasetIndexInternalExt;
use crate::index::prefilter::{DatasetPreFilter, FilterLoader};
use crate::index::vector::utils::{get_vector_type, validate_distance_type_for};
use crate::{Error, Result};
use lance_arrow::*;

use super::utils::{
    FilteredRowIdsToPrefilter, IndexMetrics, InstrumentedRecordBatchStreamAdapter, PreFilterSource,
    SelectionVectorToPrefilter,
};

pub struct AnnPartitionMetrics {
    index_metrics: IndexMetrics,
    partitions_ranked: Count,
    deltas_searched: Count,
    baseline_metrics: BaselineMetrics,
}

impl AnnPartitionMetrics {
    pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self {
        Self {
            index_metrics: IndexMetrics::new(metrics, partition),
            partitions_ranked: metrics.new_count(PARTITIONS_RANKED_METRIC, partition),
            deltas_searched: metrics.new_count(DELTAS_SEARCHED_METRIC, partition),
            baseline_metrics: BaselineMetrics::new(metrics, partition),
        }
    }
}

pub struct AnnIndexMetrics {
    index_metrics: IndexMetrics,
    partitions_searched: Count,
    baseline_metrics: BaselineMetrics,
}

impl AnnIndexMetrics {
    pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self {
        Self {
            index_metrics: IndexMetrics::new(metrics, partition),
            partitions_searched: metrics.new_count(PARTITIONS_SEARCHED_METRIC, partition),
            baseline_metrics: BaselineMetrics::new(metrics, partition),
        }
    }
}

/// [ExecutionPlan] compute vector distance from a query vector.
///
/// Preconditions:
/// - `input` schema must contains `query.column`,
/// - The column must be a vector column.
///
/// WARNING: Internal API with no stability guarantees.
#[derive(Debug)]
pub struct KNNVectorDistanceExec {
    /// Inner input node.
    pub input: Arc<dyn ExecutionPlan>,

    /// The vector query to execute.
    pub query: ArrayRef,
    pub column: String,
    pub distance_type: DistanceType,

    output_schema: SchemaRef,
    properties: PlanProperties,

    metrics: ExecutionPlanMetricsSet,
}

impl DisplayAs for KNNVectorDistanceExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(f, "KNNVectorDistance: metric={}", self.distance_type,)
            }
            DisplayFormatType::TreeRender => {
                write!(f, "KNNVectorDistance\nmetric={}", self.distance_type,)
            }
        }
    }
}

impl KNNVectorDistanceExec {
    /// Create a new [`KNNVectorDistanceExec`] node.
    ///
    /// Returns an error if the preconditions are not met.
    pub fn try_new(
        input: Arc<dyn ExecutionPlan>,
        column: &str,
        query: ArrayRef,
        distance_type: DistanceType,
    ) -> Result<Self> {
        let mut output_schema = input.schema().as_ref().clone();
        let (_, element_type) = get_vector_type(&(&output_schema).try_into()?, column)?;
        validate_distance_type_for(distance_type, &element_type)?;

        // FlatExec appends a distance column to the input schema. The input
        // may already have a distance column (possibly in the wrong position), so
        // we need to remove it before adding a new one.
        if output_schema.column_with_name(DIST_COL).is_some() {
            output_schema = output_schema.without_column(DIST_COL);
        }
        let output_schema = Arc::new(output_schema.try_with_column(Field::new(
            DIST_COL,
            DataType::Float32,
            true,
        ))?);

        // This node has the same partitioning & boundedness as the input node
        // but it destroys any ordering.
        let properties = input
            .properties()
            .clone()
            .with_eq_properties(EquivalenceProperties::new(output_schema.clone()));

        Ok(Self {
            input,
            query,
            column: column.to_string(),
            distance_type,
            output_schema,
            properties,
            metrics: ExecutionPlanMetricsSet::new(),
        })
    }
}

impl ExecutionPlan for KNNVectorDistanceExec {
    fn name(&self) -> &str {
        "KNNVectorDistanceExec"
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    /// Flat KNN inherits the schema from input node, and add one distance column.
    fn schema(&self) -> arrow_schema::SchemaRef {
        self.output_schema.clone()
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        vec![&self.input]
    }

    fn with_new_children(
        self: Arc<Self>,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
        if children.len() != 1 {
            return Err(DataFusionError::Internal(
                "KNNVectorDistanceExec node must have exactly one child".to_string(),
            ));
        }

        Ok(Arc::new(Self::try_new(
            children.pop().expect("length checked"),
            &self.column,
            self.query.clone(),
            self.distance_type,
        )?))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<datafusion::execution::context::TaskContext>,
    ) -> DataFusionResult<SendableRecordBatchStream> {
        let input_stream = self.input.execute(partition, context)?;
        let key = self.query.clone();
        let column = self.column.clone();
        let dt = self.distance_type;
        let stream = input_stream
            .try_filter(|batch| future::ready(batch.num_rows() > 0))
            .map(move |batch| {
                let key = key.clone();
                let column = column.clone();
                async move {
                    let batch = compute_distance(key, dt, &column, batch?)
                        .await
                        .map_err(|e| DataFusionError::External(Box::new(e)))?;

                    let distances = batch[DIST_COL].as_primitive::<Float32Type>();
                    let mask = BooleanArray::from_iter(
                        distances
                            .iter()
                            .map(|v| Some(v.map(|v| !v.is_nan()).unwrap_or(false))),
                    );
                    arrow::compute::filter_record_batch(&batch, &mask)
                        .map_err(|e| DataFusionError::ArrowError(Box::new(e), None))
                }
            })
            .buffer_unordered(get_num_compute_intensive_cpus());
        let schema = self.schema();
        Ok(Box::pin(InstrumentedRecordBatchStreamAdapter::new(
            schema,
            stream.boxed(),
            partition,
            &self.metrics,
        )) as SendableRecordBatchStream)
    }

    fn partition_statistics(&self, partition: Option<usize>) -> DataFusionResult<Statistics> {
        let inner_stats = self.input.partition_statistics(partition)?;
        let schema = self.input.schema();
        let dist_stats = inner_stats
            .column_statistics
            .iter()
            .zip(schema.fields())
            .find(|(_, field)| field.name() == &self.column)
            .map(|(stats, _)| ColumnStatistics {
                null_count: stats.null_count,
                ..Default::default()
            })
            .unwrap_or_default();
        let column_statistics = inner_stats
            .column_statistics
            .into_iter()
            .zip(schema.fields())
            .filter(|(_, field)| field.name() != DIST_COL)
            .map(|(stats, _)| stats)
            .chain(std::iter::once(dist_stats))
            .collect::<Vec<_>>();
        Ok(Statistics {
            num_rows: inner_stats.num_rows,
            column_statistics,
            ..Statistics::new_unknown(self.schema().as_ref())
        })
    }

    fn metrics(&self) -> Option<MetricsSet> {
        Some(self.metrics.clone_inner())
    }

    fn properties(&self) -> &PlanProperties {
        &self.properties
    }

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

pub static KNN_INDEX_SCHEMA: LazyLock<SchemaRef> = LazyLock::new(|| {
    Arc::new(Schema::new(vec![
        Field::new(DIST_COL, DataType::Float32, true),
        ROW_ID_FIELD.clone(),
    ]))
});

pub static KNN_PARTITION_SCHEMA: LazyLock<SchemaRef> = LazyLock::new(|| {
    Arc::new(Schema::new(vec![
        Field::new(
            PART_ID_COLUMN,
            DataType::List(Field::new("item", DataType::UInt32, false).into()),
            false,
        ),
        Field::new(
            DIST_Q_C_COLUMN,
            DataType::List(Field::new("item", DataType::Float32, false).into()),
            false,
        ),
        Field::new(INDEX_UUID_COLUMN, DataType::Utf8, false),
    ]))
});

pub fn new_knn_exec(
    dataset: Arc<Dataset>,
    indices: &[IndexMetadata],
    query: &Query,
    prefilter_source: PreFilterSource,
) -> Result<Arc<dyn ExecutionPlan>> {
    let ivf_node = ANNIvfPartitionExec::try_new(
        dataset.clone(),
        indices.iter().map(|idx| idx.uuid.to_string()).collect_vec(),
        query.clone(),
    )?;

    let sub_index = ANNIvfSubIndexExec::try_new(
        Arc::new(ivf_node),
        dataset,
        indices.to_vec(),
        query.clone(),
        prefilter_source,
    )?;

    Ok(Arc::new(sub_index))
}

/// [ExecutionPlan] to execute the find the closest IVF partitions.
///
/// It searches the partition IDs using the input query.
///
/// It searches all index deltas in parallel.  For each delta it returns a
/// single batch with the partition IDs and the delta index `uuid`:
///
/// The number of partitions returned is at most `maximum_nprobes`.  If
/// `maximum_nprobes` is not set, it will return all partitions.  The partitions
/// are returned in sorted order from closest to farthest.
///
/// Typically, all partition ids will be identical for each delta index (since delta
/// indices have identical partitions) but the downstream nodes do not rely on this.
///
/// TODO: We may want to search the partitions once instead of once per delta index
/// since the centroids are the same.
///
/// ```text
/// {
///    "__ivf_part_id": List<UInt32>,
///    "__index_uuid": String,
/// }
/// ```
#[derive(Debug)]
pub struct ANNIvfPartitionExec {
    pub dataset: Arc<Dataset>,

    /// The vector query to execute.
    pub query: Query,

    /// The UUIDs of the indices to search.
    pub index_uuids: Vec<String>,

    pub properties: PlanProperties,

    pub metrics: ExecutionPlanMetricsSet,
}

impl ANNIvfPartitionExec {
    pub fn try_new(dataset: Arc<Dataset>, index_uuids: Vec<String>, query: Query) -> Result<Self> {
        let dataset_schema = dataset.schema();
        get_vector_type(dataset_schema, &query.column)?;
        if index_uuids.is_empty() {
            return Err(Error::execution(
                "ANNIVFPartitionExec node: no index found for query".to_string(),
            ));
        }

        let schema = KNN_PARTITION_SCHEMA.clone();
        let properties = PlanProperties::new(
            EquivalenceProperties::new(schema),
            Partitioning::RoundRobinBatch(1),
            EmissionType::Incremental,
            Boundedness::Bounded,
        );

        Ok(Self {
            dataset,
            query,
            index_uuids,
            properties,
            metrics: ExecutionPlanMetricsSet::new(),
        })
    }
}

impl DisplayAs for ANNIvfPartitionExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(
                    f,
                    "ANNIvfPartition: uuid={}, minimum_nprobes={}, maximum_nprobes={:?}, deltas={}",
                    self.index_uuids[0],
                    self.query.minimum_nprobes,
                    self.query.maximum_nprobes,
                    self.index_uuids.len()
                )
            }
            DisplayFormatType::TreeRender => {
                write!(
                    f,
                    "ANNIvfPartition\nuuid={}\nminimum_nprobes={}\nmaximum_nprobes={:?}\ndeltas={}",
                    self.index_uuids[0],
                    self.query.minimum_nprobes,
                    self.query.maximum_nprobes,
                    self.index_uuids.len()
                )
            }
        }
    }
}

impl ExecutionPlan for ANNIvfPartitionExec {
    fn name(&self) -> &str {
        "ANNIVFPartitionExec"
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> SchemaRef {
        KNN_PARTITION_SCHEMA.clone()
    }

    fn statistics(&self) -> DataFusionResult<Statistics> {
        Ok(Statistics {
            num_rows: Precision::Exact(self.query.minimum_nprobes),
            ..Statistics::new_unknown(self.schema().as_ref())
        })
    }

    fn properties(&self) -> &PlanProperties {
        &self.properties
    }

    fn metrics(&self) -> Option<MetricsSet> {
        Some(self.metrics.clone_inner())
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        vec![]
    }

    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
        if !children.is_empty() {
            Err(DataFusionError::Internal(
                "ANNIVFPartitionExec node does not accept children".to_string(),
            ))
        } else {
            Ok(self)
        }
    }

    fn execute(
        &self,
        partition: usize,
        _context: Arc<datafusion::execution::TaskContext>,
    ) -> DataFusionResult<SendableRecordBatchStream> {
        let timer = Instant::now();

        let query = self.query.clone();
        let ds = self.dataset.clone();
        let metrics = Arc::new(AnnPartitionMetrics::new(&self.metrics, partition));
        metrics.deltas_searched.add(self.index_uuids.len());
        let metrics_clone = metrics.clone();

        let stream = stream::iter(self.index_uuids.clone())
            .map(move |uuid| {
                let query = query.clone();
                let ds = ds.clone();
                let metrics = metrics.clone();
                async move {
                    let index = ds
                        .open_vector_index(&query.column, &uuid, &metrics.index_metrics)
                        .await?;
                    let mut query = query.clone();
                    if index.metric_type() == DistanceType::Cosine {
                        let key = normalize_arrow(&query.key)?.0;
                        query.key = key;
                    };

                    metrics.partitions_ranked.add(index.total_partitions());

                    let (partitions, dist_q_c) = index.find_partitions(&query).map_err(|e| {
                        DataFusionError::Execution(format!("Failed to find partitions: {}", e))
                    })?;

                    let mut part_list_builder = ListBuilder::new(UInt32Builder::new())
                        .with_field(Field::new("item", DataType::UInt32, false));
                    part_list_builder.append_value(partitions.iter());
                    let partition_col = part_list_builder.finish();

                    let mut dist_q_c_list_builder = ListBuilder::new(Float32Builder::new())
                        .with_field(Field::new("item", DataType::Float32, false));
                    dist_q_c_list_builder.append_value(dist_q_c.iter());
                    let dist_q_c_col = dist_q_c_list_builder.finish();

                    let uuid_col = StringArray::from(vec![uuid.as_str()]);
                    let batch = RecordBatch::try_new(
                        KNN_PARTITION_SCHEMA.clone(),
                        vec![
                            Arc::new(partition_col),
                            Arc::new(dist_q_c_col),
                            Arc::new(uuid_col),
                        ],
                    )?;
                    metrics.baseline_metrics.record_output(batch.num_rows());
                    Ok::<_, DataFusionError>(batch)
                }
            })
            .buffered(self.index_uuids.len())
            .finally(move || {
                metrics_clone.baseline_metrics.done();
                metrics_clone
                    .baseline_metrics
                    .elapsed_compute()
                    .add_duration(timer.elapsed());
            });
        let schema = self.schema();
        Ok(
            Box::pin(RecordBatchStreamAdapter::new(schema, stream.boxed()))
                as SendableRecordBatchStream,
        )
    }

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

/// Datafusion [ExecutionPlan] to run search on vector index partitions.
///
/// A IVF-{PQ/SQ/HNSW} query plan is:
///
/// ```text
/// AnnSubIndexExec: k=10
///   AnnPartitionExec: nprobes=20
/// ```
///
/// The partition index returns one batch per delta with `maximum_nprobes` partitions in sorted order.
///
/// The sub-index then runs a KNN search on each partition in parallel.
///
/// First, the index will search `minimum_probes` partitions on each delta.  If there are enough results
/// at that point to satisfy K then the results will be sorted and returned.
///
/// If there are not enough results then the prefilter will be consulted to determine how many potential
/// results exist.  If the number is smaller than K then those additional results will be fetched directly
/// and given maximum distance.
///
/// If the number of results is larger then additional partitions will be searched in batches of
/// `cpu_parallelism` until min(K, num_filtered_results) are found or `maximum_nprobes` partitions
/// have been searched.
///
/// TODO: In the future, if we can know that a filter will be highly selective (through cost estimation or
/// user-provided hints) we wait for the prefilter results before we load any partitions.  If there are less
/// than K (or some threshold) results then we can return without search.
#[derive(Debug)]
pub struct ANNIvfSubIndexExec {
    /// Inner input source node.
    input: Arc<dyn ExecutionPlan>,

    dataset: Arc<Dataset>,

    indices: Vec<IndexMetadata>,

    /// Vector Query.
    query: Query,

    /// Prefiltering input
    prefilter_source: PreFilterSource,

    /// Datafusion Plan Properties
    properties: PlanProperties,

    metrics: ExecutionPlanMetricsSet,
}

impl ANNIvfSubIndexExec {
    pub fn try_new(
        input: Arc<dyn ExecutionPlan>,
        dataset: Arc<Dataset>,
        indices: Vec<IndexMetadata>,
        query: Query,
        prefilter_source: PreFilterSource,
    ) -> Result<Self> {
        if input.schema().field_with_name(PART_ID_COLUMN).is_err() {
            return Err(Error::index(format!(
                "ANNSubIndexExec node: input schema does not have \"{}\" column",
                PART_ID_COLUMN
            )));
        }
        let properties = PlanProperties::new(
            EquivalenceProperties::new(KNN_INDEX_SCHEMA.clone()),
            Partitioning::RoundRobinBatch(1),
            EmissionType::Final,
            Boundedness::Bounded,
        );
        Ok(Self {
            input,
            dataset,
            indices,
            query,
            prefilter_source,
            properties,
            metrics: ExecutionPlanMetricsSet::new(),
        })
    }
}

impl DisplayAs for ANNIvfSubIndexExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
        let metric_str = self
            .query
            .metric_type
            .map(|m| format!("{:?}", m))
            .unwrap_or_else(|| "default".to_string());
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(
                    f,
                    "ANNSubIndex: name={}, k={}, deltas={}, metric={}",
                    self.indices[0].name,
                    self.query.k * self.query.refine_factor.unwrap_or(1) as usize,
                    self.indices.len(),
                    metric_str
                )
            }
            DisplayFormatType::TreeRender => {
                write!(
                    f,
                    "ANNSubIndex\nname={}\nk={}\ndeltas={}\nmetric={}",
                    self.indices[0].name,
                    self.query.k * self.query.refine_factor.unwrap_or(1) as usize,
                    self.indices.len(),
                    metric_str
                )
            }
        }
    }
}

struct ANNIvfEarlySearchResults {
    k: usize,
    initial_ids: Mutex<Vec<u64>>,
    num_results_found: AtomicUsize,
    deltas_remaining: AtomicUsize,
    all_deltas_done: Notify,
    took_no_rows_shortcut: AtomicBool,
}

impl ANNIvfEarlySearchResults {
    fn new(deltas_remaining: usize, k: usize) -> Self {
        Self {
            k,
            initial_ids: Mutex::new(Vec::with_capacity(k)),
            num_results_found: AtomicUsize::new(0),
            deltas_remaining: AtomicUsize::new(deltas_remaining),
            all_deltas_done: Notify::new(),
            took_no_rows_shortcut: AtomicBool::new(false),
        }
    }

    fn record_batch(&self, batch: &RecordBatch) {
        let mut initial_ids = self.initial_ids.lock().unwrap();
        let ids_to_record = (self.k - initial_ids.len()).min(batch.num_rows());
        initial_ids.extend(
            batch
                .column(1)
                .as_primitive::<UInt64Type>()
                .values()
                .iter()
                .take(ids_to_record),
        );
    }

    fn record_late_batch(&self, num_rows: usize) {
        self.num_results_found
            .fetch_add(num_rows, Ordering::Relaxed);
    }

    async fn wait_for_minimum_to_finish(&self) -> usize {
        if self.deltas_remaining.fetch_sub(1, Ordering::Relaxed) == 1 {
            {
                let new_num_results_found = self.initial_ids.lock().unwrap().len();
                self.num_results_found
                    .store(new_num_results_found, Ordering::Relaxed);
            }
            self.all_deltas_done.notify_waiters();
        } else {
            self.all_deltas_done.notified().await;
        }
        self.num_results_found.load(Ordering::Relaxed)
    }
}

impl ANNIvfSubIndexExec {
    fn late_search(
        index: Arc<dyn VectorIndex>,
        query: Query,
        partitions: Arc<UInt32Array>,
        q_c_dists: Arc<Float32Array>,
        prefilter: Arc<DatasetPreFilter>,
        metrics: Arc<AnnIndexMetrics>,
        state: Arc<ANNIvfEarlySearchResults>,
    ) -> impl Stream<Item = DataFusionResult<RecordBatch>> {
        let stream = futures::stream::once(async move {
            let max_nprobes = query
                .maximum_nprobes
                .unwrap_or(partitions.len())
                .min(partitions.len());
            let min_nprobes = query.minimum_nprobes.min(max_nprobes);
            if max_nprobes <= min_nprobes {
                // We've already searched all partitions, no late search needed
                return futures::stream::empty().boxed();
            }

            let found_so_far = state.wait_for_minimum_to_finish().await;
            if found_so_far >= query.k {
                // We found enough results, no need for late search
                return futures::stream::empty().boxed();
            }

            // We know the prefilter should be ready at this point so we shouldn't
            // need to call wait_for_ready
            let prefilter_mask = prefilter.mask();

            let max_results = prefilter_mask.max_len().map(|x| x as usize);

            if let Some(max_results) = max_results
                && found_so_far < max_results
                && max_results <= query.k
            {
                // In this case there are fewer than k results matching the prefilter so
                // just return the prefilter ids and don't bother searching any further

                // This next if check should be true, because we wouldn't get max_results otherwise
                if let Some(iter_addrs) = prefilter_mask.iter_addrs() {
                    // We only run this on the first delta because the prefilter mask is shared
                    // by all deltas and we don't want to duplicate the rows.
                    if state
                        .took_no_rows_shortcut
                        .compare_exchange(false, true, Ordering::Relaxed, Ordering::Relaxed)
                        .is_ok()
                    {
                        let initial_addrs = state.initial_ids.lock().unwrap();
                        let found_addrs = HashSet::<_>::from_iter(initial_addrs.iter().copied());
                        drop(initial_addrs);
                        let mask_addrs = HashSet::from_iter(iter_addrs.map(u64::from));
                        let not_found_addrs = mask_addrs.difference(&found_addrs);
                        let not_found_addrs =
                            UInt64Array::from_iter_values(not_found_addrs.copied());
                        let not_found_distance =
                            Float32Array::from_value(f32::INFINITY, not_found_addrs.len());
                        let not_found_batch = RecordBatch::try_new(
                            KNN_INDEX_SCHEMA.clone(),
                            vec![Arc::new(not_found_distance), Arc::new(not_found_addrs)],
                        )
                        .unwrap();
                        return futures::stream::once(async move { Ok(not_found_batch) }).boxed();
                    } else {
                        // We meet all the criteria for an early exit, but we aren't first
                        // delta so we just return an empty stream and skip the late search
                        return futures::stream::empty().boxed();
                    }
                }
            }

            // Stop searching if we have k results or we've found all the results
            // that could possible match the prefilter
            let max_results = max_results.unwrap_or(usize::MAX).min(query.k);

            let state_clone = state.clone();

            futures::stream::iter(min_nprobes..max_nprobes)
                .map(move |idx| {
                    let part_id = partitions.value(idx);
                    let mut query = query.clone();
                    query.dist_q_c = q_c_dists.value(idx);
                    let metrics = metrics.clone();
                    let pre_filter = prefilter.clone();
                    let state = state.clone();
                    let index = index.clone();
                    async move {
                        let mut query = query.clone();
                        if index.metric_type() == DistanceType::Cosine {
                            let key = normalize_arrow(&query.key)?.0;
                            query.key = key;
                        };

                        metrics.partitions_searched.add(1);
                        let batch = index
                            .search_in_partition(
                                part_id as usize,
                                &query,
                                pre_filter,
                                &metrics.index_metrics,
                            )
                            .map_err(|e| {
                                DataFusionError::Execution(format!(
                                    "Failed to calculate KNN: {}",
                                    e
                                ))
                            })
                            .await?;
                        metrics.baseline_metrics.record_output(batch.num_rows());
                        state.record_late_batch(batch.num_rows());
                        Ok(batch)
                    }
                })
                .take_while(move |_| {
                    let found_so_far = state_clone.num_results_found.load(Ordering::Relaxed);
                    std::future::ready(found_so_far < max_results)
                })
                .buffered(get_num_compute_intensive_cpus())
                .boxed()
        });
        stream.flatten()
    }

    fn initial_search(
        index: Arc<dyn VectorIndex>,
        query: Query,
        partitions: Arc<UInt32Array>,
        q_c_dists: Arc<Float32Array>,
        prefilter: Arc<DatasetPreFilter>,
        metrics: Arc<AnnIndexMetrics>,
        state: Arc<ANNIvfEarlySearchResults>,
    ) -> impl Stream<Item = DataFusionResult<RecordBatch>> {
        let minimum_nprobes = query.minimum_nprobes.min(partitions.len());
        metrics.partitions_searched.add(minimum_nprobes);

        futures::stream::iter(0..minimum_nprobes)
            .map(move |idx| {
                let part_id = partitions.value(idx);
                let mut query = query.clone();
                query.dist_q_c = q_c_dists.value(idx);
                let metrics = metrics.clone();
                let index = index.clone();
                let pre_filter = prefilter.clone();
                let state = state.clone();
                async move {
                    let mut query = query.clone();
                    if index.metric_type() == DistanceType::Cosine {
                        let key = normalize_arrow(&query.key)?.0;
                        query.key = key;
                    };

                    let batch = index
                        .search_in_partition(
                            part_id as usize,
                            &query,
                            pre_filter,
                            &metrics.index_metrics,
                        )
                        .map_err(|e| {
                            DataFusionError::Execution(format!("Failed to calculate KNN: {}", e))
                        })
                        .await?;
                    metrics.baseline_metrics.record_output(batch.num_rows());
                    state.record_batch(&batch);
                    Ok(batch)
                }
            })
            .buffered(get_num_compute_intensive_cpus())
    }
}

impl ExecutionPlan for ANNIvfSubIndexExec {
    fn name(&self) -> &str {
        "ANNSubIndexExec"
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> arrow_schema::SchemaRef {
        KNN_INDEX_SCHEMA.clone()
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        match &self.prefilter_source {
            PreFilterSource::None => vec![&self.input],
            PreFilterSource::FilteredRowIds(src) => vec![&self.input, &src],
            PreFilterSource::ScalarIndexQuery(src) => vec![&self.input, &src],
        }
    }

    fn required_input_distribution(&self) -> Vec<Distribution> {
        // Prefilter inputs must be a single partition
        self.children()
            .iter()
            .map(|_| Distribution::SinglePartition)
            .collect()
    }

    fn with_new_children(
        self: Arc<Self>,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
        let plan = if children.len() == 1 || children.len() == 2 {
            let prefilter_source = if children.len() == 2 {
                let prefilter = children.pop().expect("length checked");
                match &self.prefilter_source {
                    PreFilterSource::None => PreFilterSource::None,
                    PreFilterSource::FilteredRowIds(_) => {
                        PreFilterSource::FilteredRowIds(prefilter)
                    }
                    PreFilterSource::ScalarIndexQuery(_) => {
                        PreFilterSource::ScalarIndexQuery(prefilter)
                    }
                }
            } else {
                self.prefilter_source.clone()
            };

            Self {
                input: children.pop().expect("length checked"),
                dataset: self.dataset.clone(),
                indices: self.indices.clone(),
                query: self.query.clone(),
                prefilter_source,
                properties: self.properties.clone(),
                metrics: ExecutionPlanMetricsSet::new(),
            }
        } else {
            return Err(DataFusionError::Internal(
                "ANNSubIndexExec node must have exactly one or two (prefilter) child".to_string(),
            ));
        };
        Ok(Arc::new(plan))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<datafusion::execution::context::TaskContext>,
    ) -> DataFusionResult<datafusion::physical_plan::SendableRecordBatchStream> {
        let input_stream = self.input.execute(partition, context.clone())?;
        let schema = self.schema();
        let query = self.query.clone();
        let ds = self.dataset.clone();
        let column = self.query.column.clone();
        let indices = self.indices.clone();
        let prefilter_source = self.prefilter_source.clone();
        let metrics = Arc::new(AnnIndexMetrics::new(&self.metrics, partition));
        let metrics_clone = metrics.clone();
        let timer = Instant::now();

        // Per-delta-index stream:
        //   Stream<(parttitions, index uuid)>
        let per_index_stream = input_stream
            .and_then(move |batch| {
                let part_id_col = batch.column_by_name(PART_ID_COLUMN).unwrap_or_else(|| {
                    panic!("ANNSubIndexExec: input missing {} column", PART_ID_COLUMN)
                });
                let part_id_arr = part_id_col.as_list::<i32>().clone();
                let dist_q_c_col = batch.column_by_name(DIST_Q_C_COLUMN).unwrap_or_else(|| {
                    panic!("ANNSubIndexExec: input missing {} column", DIST_Q_C_COLUMN)
                });
                let dist_q_c_arr = dist_q_c_col.as_list::<i32>().clone();
                let index_uuid_col = batch.column_by_name(INDEX_UUID_COLUMN).unwrap_or_else(|| {
                    panic!(
                        "ANNSubIndexExec: input missing {} column",
                        INDEX_UUID_COLUMN
                    )
                });
                let index_uuid = index_uuid_col.as_string::<i32>().clone();

                let plan: Vec<DataFusionResult<(_, _, _)>> = part_id_arr
                    .iter()
                    .zip(dist_q_c_arr.iter())
                    .zip(index_uuid.iter())
                    .map(|((part_id, dist_q_c), uuid)| {
                        let partitions =
                            Arc::new(part_id.unwrap().as_primitive::<UInt32Type>().clone());
                        let dist_q_c =
                            Arc::new(dist_q_c.unwrap().as_primitive::<Float32Type>().clone());
                        let uuid = uuid.unwrap().to_string();
                        Ok((partitions, dist_q_c, uuid))
                    })
                    .collect_vec();
                async move { DataFusionResult::Ok(stream::iter(plan)) }
            })
            .try_flatten();
        let prefilter_loader = match &prefilter_source {
            PreFilterSource::FilteredRowIds(src_node) => {
                let stream = src_node.execute(partition, context)?;
                Some(Box::new(FilteredRowIdsToPrefilter(stream)) as Box<dyn FilterLoader>)
            }
            PreFilterSource::ScalarIndexQuery(src_node) => {
                let stream = src_node.execute(partition, context)?;
                Some(Box::new(SelectionVectorToPrefilter(stream)) as Box<dyn FilterLoader>)
            }
            PreFilterSource::None => None,
        };

        let pre_filter = Arc::new(DatasetPreFilter::new(
            ds.clone(),
            &indices,
            prefilter_loader,
        ));

        let state = Arc::new(ANNIvfEarlySearchResults::new(indices.len(), query.k));

        Ok(Box::pin(RecordBatchStreamAdapter::new(
            schema,
            per_index_stream
                .and_then(move |(part_ids, q_c_dists, index_uuid)| {
                    let ds = ds.clone();
                    let column = column.clone();
                    let metrics = metrics.clone();
                    let pre_filter = pre_filter.clone();
                    let state = state.clone();
                    let mut query = query.clone();
                    let pruned_nprobes = early_pruning(q_c_dists.values(), query.k);
                    adjust_probes(&mut query, pruned_nprobes);
                    async move {
                        let raw_index = ds
                            .open_vector_index(&column, &index_uuid, &metrics.index_metrics)
                            .await?;

                        let early_search = Self::initial_search(
                            raw_index.clone(),
                            query.clone(),
                            part_ids.clone(),
                            q_c_dists.clone(),
                            pre_filter.clone(),
                            metrics.clone(),
                            state.clone(),
                        );
                        let late_search = Self::late_search(
                            raw_index.clone(),
                            query,
                            part_ids,
                            q_c_dists,
                            pre_filter,
                            metrics,
                            state,
                        );
                        DataFusionResult::Ok(early_search.chain(late_search).boxed())
                    }
                })
                // Must use flatten_unordered to avoid deadlock.
                // Each delta stream is split into an early and late search.  The late search
                // will not start until the early search is complete across all deltas.
                .try_flatten_unordered(None)
                .finally(move || {
                    metrics_clone
                        .baseline_metrics
                        .elapsed_compute()
                        .add_duration(timer.elapsed());
                    metrics_clone.baseline_metrics.done();
                })
                .boxed(),
        )))
    }

    fn partition_statistics(
        &self,
        partition: Option<usize>,
    ) -> DataFusionResult<datafusion::physical_plan::Statistics> {
        Ok(Statistics {
            num_rows: Precision::Exact(
                self.query.k
                    * self.query.refine_factor.unwrap_or(1) as usize
                    * self
                        .input
                        .partition_statistics(partition)?
                        .num_rows
                        .get_value()
                        .unwrap_or(&1),
            ),
            ..Statistics::new_unknown(self.schema().as_ref())
        })
    }

    fn metrics(&self) -> Option<MetricsSet> {
        Some(self.metrics.clone_inner())
    }

    fn properties(&self) -> &PlanProperties {
        &self.properties
    }

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

fn adjust_probes(query: &mut Query, pruned_nprobes: usize) {
    query.minimum_nprobes = query.minimum_nprobes.max(pruned_nprobes);
    if let Some(maximum) = query.maximum_nprobes
        && query.minimum_nprobes > maximum
    {
        query.minimum_nprobes = maximum;
    }
}

fn early_pruning(dists: &[f32], k: usize) -> usize {
    if dists.is_empty() {
        return 0;
    }

    const PRUNING_FACTORS: [f32; 3] = [0.6, 7.0, 81.0];
    let factor = match k {
        ..=1 => PRUNING_FACTORS[0],
        2..=10 => PRUNING_FACTORS[1],
        11.. => PRUNING_FACTORS[2],
    };
    let dist_threshold = dists[0] * factor;
    dists.partition_point(|dist| *dist <= dist_threshold)
}

#[derive(Debug)]
pub struct MultivectorScoringExec {
    // the inputs are sorted ANN search results
    inputs: Vec<Arc<dyn ExecutionPlan>>,
    query: Query,
    properties: PlanProperties,
}

impl MultivectorScoringExec {
    pub fn try_new(inputs: Vec<Arc<dyn ExecutionPlan>>, query: Query) -> Result<Self> {
        let properties = PlanProperties::new(
            EquivalenceProperties::new(KNN_INDEX_SCHEMA.clone()),
            Partitioning::RoundRobinBatch(1),
            EmissionType::Final,
            Boundedness::Bounded,
        );

        Ok(Self {
            inputs,
            query,
            properties,
        })
    }
}

impl DisplayAs for MultivectorScoringExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(f, "MultivectorScoring: k={}", self.query.k)
            }
            DisplayFormatType::TreeRender => {
                write!(f, "MultivectorScoring\nk={}", self.query.k)
            }
        }
    }
}

impl ExecutionPlan for MultivectorScoringExec {
    fn name(&self) -> &str {
        "MultivectorScoringExec"
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> arrow_schema::SchemaRef {
        KNN_INDEX_SCHEMA.clone()
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        self.inputs.iter().collect()
    }

    fn required_input_distribution(&self) -> Vec<Distribution> {
        // This node fully consumes and re-orders the input rows.  It must be
        // run on a single partition.
        self.children()
            .iter()
            .map(|_| Distribution::SinglePartition)
            .collect()
    }

    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
        let plan = Self::try_new(children, self.query.clone())?;
        Ok(Arc::new(plan))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<datafusion::execution::context::TaskContext>,
    ) -> DataFusionResult<SendableRecordBatchStream> {
        let inputs = self
            .inputs
            .iter()
            .map(|input| input.execute(partition, context.clone()))
            .collect::<DataFusionResult<Vec<_>>>()?;

        // collect the top k results from each stream,
        // and max-reduce for each query,
        // records the minimum distance for each query as estimation.
        let mut reduced_inputs = stream::select_all(inputs.into_iter().map(|stream| {
            stream.map(|batch| {
                let batch = batch?;
                let row_ids = batch[ROW_ID].as_primitive::<UInt64Type>();
                let dists = batch[DIST_COL].as_primitive::<Float32Type>();
                debug_assert_eq!(dists.null_count(), 0);

                // max-reduce for the same row id
                let min_sim = dists
                    .values()
                    .last()
                    .map(|dist| 1.0 - *dist)
                    .unwrap_or_default();
                let mut new_row_ids = Vec::with_capacity(row_ids.len());
                let mut new_sims = Vec::with_capacity(row_ids.len());
                let mut visited_row_ids = HashSet::with_capacity(row_ids.len());

                for (row_id, dist) in row_ids.values().iter().zip(dists.values().iter()) {
                    // the results are sorted by distance, so we can skip if we have seen this row id before
                    if visited_row_ids.contains(row_id) {
                        continue;
                    }
                    visited_row_ids.insert(row_id);
                    new_row_ids.push(*row_id);
                    // it's cosine distance, so we need to convert it to similarity
                    new_sims.push(1.0 - *dist);
                }
                let new_row_ids = UInt64Array::from(new_row_ids);
                let new_dists = Float32Array::from(new_sims);

                let batch = RecordBatch::try_new(
                    KNN_INDEX_SCHEMA.clone(),
                    vec![Arc::new(new_dists), Arc::new(new_row_ids)],
                )?;

                Ok::<_, DataFusionError>((min_sim, batch))
            })
        }));

        let k = self.query.k;
        let refactor = self.query.refine_factor.unwrap_or(1) as usize;
        let num_queries = self.inputs.len() as f32;
        let stream = stream::once(async move {
            // at most, we will have k * refine_factor results for each query
            let mut results = HashMap::with_capacity(k * refactor);
            let mut missed_sim_sum = 0.0;
            while let Some((min_sim, batch)) = reduced_inputs.try_next().await? {
                let row_ids = batch[ROW_ID].as_primitive::<UInt64Type>();
                let sims = batch[DIST_COL].as_primitive::<Float32Type>();

                let query_results = row_ids
                    .values()
                    .iter()
                    .copied()
                    .zip(sims.values().iter().copied())
                    .collect::<HashMap<_, _>>();

                // for a row `r`:
                // if `r` is in only `results``, then `results[r] += min_sim`
                // if `r` is in only `query_results`, then `results[r] = query_results[r] + missed_similarities`,
                // here `missed_similarities` is the sum of `min_sim` from previous iterations
                // if `r` is in both, then `results[r] += query_results[r]`
                results.iter_mut().for_each(|(row_id, sim)| {
                    if let Some(new_dist) = query_results.get(row_id) {
                        *sim += new_dist;
                    } else {
                        *sim += min_sim;
                    }
                });
                query_results.into_iter().for_each(|(row_id, sim)| {
                    results.entry(row_id).or_insert(sim + missed_sim_sum);
                });
                missed_sim_sum += min_sim;
            }

            let (row_ids, sims): (Vec<_>, Vec<_>) = results.into_iter().unzip();
            let dists = sims
                .into_iter()
                // it's similarity, so we need to convert it back to distance
                .map(|sim| num_queries - sim)
                .collect::<Vec<_>>();
            let row_ids = UInt64Array::from(row_ids);
            let dists = Float32Array::from(dists);
            let batch = RecordBatch::try_new(
                KNN_INDEX_SCHEMA.clone(),
                vec![Arc::new(dists), Arc::new(row_ids)],
            )?;
            Ok::<_, DataFusionError>(batch)
        });
        Ok(Box::pin(RecordBatchStreamAdapter::new(
            self.schema(),
            stream.boxed(),
        )))
    }

    fn statistics(&self) -> DataFusionResult<Statistics> {
        Ok(Statistics {
            num_rows: Precision::Inexact(
                self.query.k * self.query.refine_factor.unwrap_or(1) as usize,
            ),
            ..Statistics::new_unknown(self.schema().as_ref())
        })
    }

    fn properties(&self) -> &PlanProperties {
        &self.properties
    }

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

#[cfg(test)]
mod tests {
    use super::*;

    use arrow::compute::{concat_batches, sort_to_indices, take_record_batch};
    use arrow::datatypes::Float32Type;
    use arrow_array::{
        ArrayRef, FixedSizeListArray, Float32Array, Int32Array, RecordBatchIterator, StringArray,
    };
    use arrow_schema::{Field as ArrowField, Schema as ArrowSchema};
    use lance_core::utils::tempfile::TempStrDir;
    use lance_datafusion::exec::{ExecutionStatsCallback, ExecutionSummaryCounts};
    use lance_datagen::{BatchCount, RowCount, array};
    use lance_index::optimize::OptimizeOptions;
    use lance_index::vector::ivf::IvfBuildParams;
    use lance_index::vector::pq::PQBuildParams;
    use lance_index::{DatasetIndexExt, IndexType};
    use lance_linalg::distance::MetricType;
    use lance_testing::datagen::generate_random_array;
    use rstest::rstest;

    use crate::dataset::{WriteMode, WriteParams};
    use crate::index::vector::VectorIndexParams;
    use crate::io::exec::testing::TestingExec;

    fn base_query() -> Query {
        Query {
            column: "vec".to_string(),
            key: Arc::new(Float32Array::from(vec![0.0f32])) as ArrayRef,
            k: 10,
            lower_bound: None,
            upper_bound: None,
            minimum_nprobes: 1,
            maximum_nprobes: None,
            ef: None,
            refine_factor: None,
            metric_type: Some(DistanceType::L2),
            use_index: true,
            dist_q_c: 0.0,
        }
    }

    #[test]
    fn test_adjust_probes_rules() {
        let mut query = base_query();
        adjust_probes(&mut query, 10);
        assert_eq!(query.minimum_nprobes, 10);
        assert_eq!(query.maximum_nprobes, None);

        let mut query = base_query();
        query.minimum_nprobes = 20;
        adjust_probes(&mut query, 10);
        assert_eq!(query.minimum_nprobes, 20);
        assert_eq!(query.maximum_nprobes, None);

        let mut query = base_query();
        query.maximum_nprobes = Some(25);
        adjust_probes(&mut query, 10);
        assert_eq!(query.minimum_nprobes, 10);
        assert_eq!(query.maximum_nprobes, Some(25));

        let mut query = base_query();
        query.maximum_nprobes = Some(5);
        adjust_probes(&mut query, 10);
        assert_eq!(query.minimum_nprobes, 5);
        assert_eq!(query.maximum_nprobes, Some(5));

        let mut query = base_query();
        query.minimum_nprobes = 30;
        query.maximum_nprobes = Some(50);
        adjust_probes(&mut query, 10);
        assert_eq!(query.minimum_nprobes, 30);
        assert_eq!(query.maximum_nprobes, Some(50));
    }

    #[tokio::test]
    async fn knn_flat_search() {
        let schema = Arc::new(ArrowSchema::new(vec![
            ArrowField::new("key", DataType::Int32, false),
            ArrowField::new(
                "vector",
                DataType::FixedSizeList(
                    Arc::new(ArrowField::new("item", DataType::Float32, true)),
                    128,
                ),
                true,
            ),
            ArrowField::new("uri", DataType::Utf8, true),
        ]));

        let batches: Vec<RecordBatch> = (0..20)
            .map(|i| {
                RecordBatch::try_new(
                    schema.clone(),
                    vec![
                        Arc::new(Int32Array::from_iter_values(i * 20..(i + 1) * 20)),
                        Arc::new(
                            FixedSizeListArray::try_new_from_values(
                                generate_random_array(128 * 20),
                                128,
                            )
                            .unwrap(),
                        ),
                        Arc::new(StringArray::from_iter_values(
                            (i * 20..(i + 1) * 20).map(|i| format!("s3://bucket/file-{}", i)),
                        )),
                    ],
                )
                .unwrap()
            })
            .collect();

        let test_dir = TempStrDir::default();
        let test_uri = test_dir.as_str();

        let write_params = WriteParams {
            max_rows_per_file: 40,
            max_rows_per_group: 10,
            ..Default::default()
        };
        let vector_arr = batches[0].column_by_name("vector").unwrap();
        let q = as_fixed_size_list_array(&vector_arr).value(5);

        let reader = RecordBatchIterator::new(batches.into_iter().map(Ok), schema.clone());
        Dataset::write(reader, test_uri, Some(write_params))
            .await
            .unwrap();

        let dataset = Dataset::open(test_uri).await.unwrap();
        let stream = dataset
            .scan()
            .nearest("vector", q.as_primitive::<Float32Type>(), 10)
            .unwrap()
            .try_into_stream()
            .await
            .unwrap();
        let results = stream.try_collect::<Vec<_>>().await.unwrap();

        assert!(results[0].schema().column_with_name(DIST_COL).is_some());

        assert_eq!(results.len(), 1);

        let stream = dataset.scan().try_into_stream().await.unwrap();
        let all_with_distances = stream
            .and_then(|batch| compute_distance(q.clone(), DistanceType::L2, "vector", batch))
            .try_collect::<Vec<_>>()
            .await
            .unwrap();
        let all_with_distances =
            concat_batches(&results[0].schema(), all_with_distances.iter()).unwrap();
        let dist_arr = all_with_distances.column_by_name(DIST_COL).unwrap();
        let distances = dist_arr.as_primitive::<Float32Type>();
        let indices = sort_to_indices(distances, None, Some(10)).unwrap();
        let expected = take_record_batch(&all_with_distances, &indices).unwrap();
        assert_eq!(expected, results[0]);
    }

    #[test]
    fn test_create_knn_flat() {
        let dim: usize = 128;
        let schema = Arc::new(ArrowSchema::new(vec![
            ArrowField::new("key", DataType::Int32, false),
            ArrowField::new(
                "vector",
                DataType::FixedSizeList(
                    Arc::new(ArrowField::new("item", DataType::Float32, true)),
                    dim as i32,
                ),
                true,
            ),
            ArrowField::new("uri", DataType::Utf8, true),
        ]));
        let batch = RecordBatch::new_empty(schema);

        let input: Arc<dyn ExecutionPlan> = Arc::new(TestingExec::new(vec![batch]));

        let idx = KNNVectorDistanceExec::try_new(
            input,
            "vector",
            Arc::new(generate_random_array(dim)),
            DistanceType::L2,
        )
        .unwrap();
        assert_eq!(
            idx.schema().as_ref(),
            &ArrowSchema::new(vec![
                ArrowField::new("key", DataType::Int32, false),
                ArrowField::new(
                    "vector",
                    DataType::FixedSizeList(
                        Arc::new(ArrowField::new("item", DataType::Float32, true)),
                        dim as i32,
                    ),
                    true,
                ),
                ArrowField::new("uri", DataType::Utf8, true),
                ArrowField::new(DIST_COL, DataType::Float32, true),
            ])
        );
    }

    #[tokio::test]
    async fn test_multivector_score() {
        let query = Query {
            column: "vector".to_string(),
            key: Arc::new(generate_random_array(1)),
            k: 10,
            lower_bound: None,
            upper_bound: None,
            minimum_nprobes: 1,
            maximum_nprobes: None,
            ef: None,
            refine_factor: None,
            metric_type: Some(DistanceType::Cosine),
            use_index: true,
            dist_q_c: 0.0,
        };

        async fn multivector_scoring(
            inputs: Vec<Arc<dyn ExecutionPlan>>,
            query: Query,
        ) -> Result<HashMap<u64, f32>> {
            let ctx = Arc::new(datafusion::execution::context::TaskContext::default());
            let plan = MultivectorScoringExec::try_new(inputs, query.clone())?;
            let batches = plan
                .execute(0, ctx.clone())
                .unwrap()
                .try_collect::<Vec<_>>()
                .await?;
            let mut results = HashMap::new();
            for batch in batches {
                let row_ids = batch[ROW_ID].as_primitive::<UInt64Type>();
                let dists = batch[DIST_COL].as_primitive::<Float32Type>();
                for (row_id, dist) in row_ids.values().iter().zip(dists.values().iter()) {
                    results.insert(*row_id, *dist);
                }
            }
            Ok(results)
        }

        let batches = (0..3)
            .map(|i| {
                RecordBatch::try_new(
                    KNN_INDEX_SCHEMA.clone(),
                    vec![
                        Arc::new(Float32Array::from(vec![i as f32 + 1.0, i as f32 + 2.0])),
                        Arc::new(UInt64Array::from(vec![i + 1, i + 2])),
                    ],
                )
                .unwrap()
            })
            .collect::<Vec<_>>();

        let mut res: Option<HashMap<_, _>> = None;
        for perm in batches.into_iter().permutations(3) {
            let inputs = perm
                .into_iter()
                .map(|batch| {
                    let input: Arc<dyn ExecutionPlan> = Arc::new(TestingExec::new(vec![batch]));
                    input
                })
                .collect::<Vec<_>>();
            let new_res = multivector_scoring(inputs, query.clone()).await.unwrap();
            assert_eq!(new_res.len(), 4);
            if let Some(res) = &res {
                for (row_id, dist) in new_res.iter() {
                    assert_eq!(res.get(row_id).unwrap(), dist)
                }
            } else {
                res = Some(new_res);
            }
        }
    }

    /// A test dataset for testing the nprobes parameter.
    ///
    /// The dataset has 100 partitions and filterable columns setup to easily create
    /// filters whose results are spread across the partitions evenly.
    struct NprobesTestFixture {
        dataset: Dataset,
        centroids: Arc<dyn Array>,
        _tmp_dir: TempStrDir,
    }

    impl NprobesTestFixture {
        pub async fn new(num_centroids: usize, num_deltas: usize) -> Self {
            let tempdir = TempStrDir::default();
            let tmppath = tempdir.as_str();

            // We create 100 centroids
            // We generate 10,000 vectors evenly divided (100 vectors per centroid)
            // We assign labels 0..60 to the vectors so each label has ~164 vectors
            //   spread out through all of the centroids
            let centroids = array::cycle_unit_circle(num_centroids as u32)
                .generate_default(RowCount::from(num_centroids as u64))
                .unwrap();

            // Let's not deal with fractions
            assert!(100 % num_deltas == 0, "num_deltas must divide 100");
            let rows_per_frag = 100;
            let num_frags = 100;
            let frags_per_delta = num_frags / num_deltas;

            let batches = lance_datagen::gen_batch()
                .col("vector", array::jitter_centroids(centroids.clone(), 0.0001))
                .col("label", array::cycle::<UInt32Type>(Vec::from_iter(0..61)))
                .col("userid", array::step::<UInt64Type>())
                .into_reader_rows(
                    RowCount::from(rows_per_frag),
                    BatchCount::from(num_frags as u32),
                )
                .collect::<Vec<_>>();
            let schema = batches[0].as_ref().unwrap().schema();

            let mut first = true;
            for batches in batches.chunks(frags_per_delta) {
                let delta_batches = batches
                    .iter()
                    .map(|maybe_batch| Ok(maybe_batch.as_ref().unwrap().clone()))
                    .collect::<Vec<_>>();
                let reader = RecordBatchIterator::new(delta_batches, schema.clone());
                let mut dataset = Dataset::write(
                    reader,
                    tmppath,
                    Some(WriteParams {
                        mode: WriteMode::Append,
                        ..Default::default()
                    }),
                )
                .await
                .unwrap();

                let ivf_params = IvfBuildParams::try_with_centroids(
                    num_centroids,
                    Arc::new(centroids.as_fixed_size_list().clone()),
                )
                .unwrap();

                let codebook = array::rand::<Float32Type>()
                    .generate_default(RowCount::from(256 * 2))
                    .unwrap();
                let pq_params = PQBuildParams::with_codebook(2, 8, codebook);
                let index_params =
                    VectorIndexParams::with_ivf_pq_params(MetricType::L2, ivf_params, pq_params);

                if first {
                    first = false;
                    dataset
                        .create_index(&["vector"], IndexType::Vector, None, &index_params, false)
                        .await
                        .unwrap();
                } else {
                    dataset
                        .optimize_indices(&OptimizeOptions::append())
                        .await
                        .unwrap();
                }
            }

            let dataset = Dataset::open(tmppath).await.unwrap();
            Self {
                dataset,
                centroids,
                _tmp_dir: tempdir,
            }
        }

        pub fn get_centroid(&self, idx: usize) -> Arc<dyn Array> {
            let centroids = self.centroids.as_fixed_size_list();
            centroids.value(idx).clone()
        }
    }

    #[derive(Default)]
    struct StatsHolder {
        pub collected_stats: Arc<Mutex<Option<ExecutionSummaryCounts>>>,
    }

    impl StatsHolder {
        fn get_setter(&self) -> ExecutionStatsCallback {
            let collected_stats = self.collected_stats.clone();
            Arc::new(move |stats| {
                *collected_stats.lock().unwrap() = Some(stats.clone());
            })
        }

        fn consume(self) -> ExecutionSummaryCounts {
            self.collected_stats.lock().unwrap().take().unwrap()
        }
    }

    #[rstest]
    #[tokio::test]
    async fn test_no_max_nprobes(#[values(1, 20)] num_deltas: usize) {
        let fixture = NprobesTestFixture::new(100, num_deltas).await;

        let q = fixture.get_centroid(0);
        let stats_holder = StatsHolder::default();

        let results = fixture
            .dataset
            .scan()
            .nearest("vector", q.as_ref(), 50)
            .unwrap()
            .minimum_nprobes(10)
            .prefilter(true)
            .scan_stats_callback(stats_holder.get_setter())
            .filter("label = 17")
            .unwrap()
            .project(&Vec::<String>::new())
            .unwrap()
            .with_row_id()
            .try_into_batch()
            .await
            .unwrap();

        assert_eq!(results.num_rows(), 50);

        let stats = stats_holder.consume();

        // We should not search _all_ partitions because we should hit 50 results partway
        // through the late search.
        // The exact number here is deterministic but depends on the number of CPUs
        if get_num_compute_intensive_cpus() <= 32 {
            assert!(*stats.all_counts.get(PARTITIONS_SEARCHED_METRIC).unwrap() < 100 * num_deltas);
        }
    }

    #[rstest]
    #[tokio::test]
    async fn test_no_prefilter_results(#[values(1, 20)] num_deltas: usize) {
        let fixture = NprobesTestFixture::new(100, num_deltas).await;

        let q = fixture.get_centroid(0);
        let stats_holder = StatsHolder::default();

        let results = fixture
            .dataset
            .scan()
            .nearest("vector", q.as_ref(), 50)
            .unwrap()
            .minimum_nprobes(10)
            .prefilter(true)
            .scan_stats_callback(stats_holder.get_setter())
            .filter("label = 17 AND label = 18")
            .unwrap()
            .project(&Vec::<String>::new())
            .unwrap()
            .with_row_id()
            .try_into_batch()
            .await
            .unwrap();

        assert_eq!(results.num_rows(), 0);

        let stats = stats_holder.consume();
        // We still do the early search because we don't wait for the prefilter to execute
        // We skip the late search because by then we know there are no results
        assert_eq!(
            stats.all_counts.get(PARTITIONS_SEARCHED_METRIC).unwrap(),
            &(10 * num_deltas)
        );
    }

    #[rstest]
    #[tokio::test]
    async fn test_some_max_nprobes(#[values(1, 20)] num_deltas: usize) {
        let fixture = NprobesTestFixture::new(100, num_deltas).await;

        for (max_nprobes, expected_results) in [(10, 16), (20, 33), (30, 48)] {
            let q = fixture.get_centroid(0);
            let stats_holder = StatsHolder::default();
            let results = fixture
                .dataset
                .scan()
                .nearest("vector", q.as_ref(), 50)
                .unwrap()
                .minimum_nprobes(max_nprobes)
                .maximum_nprobes(max_nprobes)
                .prefilter(true)
                .filter("label = 17")
                .unwrap()
                .scan_stats_callback(stats_holder.get_setter())
                .project(&Vec::<String>::new())
                .unwrap()
                .with_row_id()
                .try_into_batch()
                .await
                .unwrap();

            let stats = stats_holder.consume();

            assert_eq!(results.num_rows(), expected_results);
            assert_eq!(
                stats.all_counts.get(PARTITIONS_SEARCHED_METRIC).unwrap(),
                &(max_nprobes * num_deltas)
            );
            assert_eq!(
                stats.all_counts.get(PARTITIONS_RANKED_METRIC).unwrap(),
                &(100 * num_deltas)
            );
        }
    }

    #[rstest]
    #[tokio::test]
    async fn test_fewer_than_k_results(#[values(1, 20)] num_deltas: usize) {
        let fixture = NprobesTestFixture::new(100, num_deltas).await;

        let q = fixture.get_centroid(0);
        let stats_holder = StatsHolder::default();
        let results = fixture
            .dataset
            .scan()
            .nearest("vector", q.as_ref(), 50)
            .unwrap()
            .minimum_nprobes(10)
            .prefilter(true)
            .filter("userid < 20")
            .unwrap()
            .scan_stats_callback(stats_holder.get_setter())
            .project(&Vec::<String>::new())
            .unwrap()
            .with_row_id()
            .try_into_batch()
            .await
            .unwrap();

        let stats = stats_holder.consume();

        // We should only search minimum_nprobes before we look at the prefilter and realize
        // we can cheaply stop early.
        assert_eq!(
            stats.all_counts.get(PARTITIONS_SEARCHED_METRIC).unwrap(),
            &(10 * num_deltas)
        );
        assert_eq!(results.num_rows(), 20);

        // 15 of the results come from beyond the closest 10 partitions and these will have infinite
        // distance.
        let num_infinite_results = results
            .column(0)
            .as_primitive::<Float32Type>()
            .values()
            .iter()
            .filter(|val| val.is_infinite())
            .count();
        assert_eq!(num_infinite_results, 15);

        // If we set a refine factor then the distance should not be infinite.
        let results = fixture
            .dataset
            .scan()
            .nearest("vector", q.as_ref(), 50)
            .unwrap()
            .minimum_nprobes(10)
            .prefilter(true)
            .refine(1)
            .filter("userid < 20")
            .unwrap()
            .project(&Vec::<String>::new())
            .unwrap()
            .with_row_id()
            .try_into_batch()
            .await
            .unwrap();

        assert_eq!(results.num_rows(), 20);
        let num_infinite_results = results
            .column(0)
            .as_primitive::<Float32Type>()
            .values()
            .iter()
            .filter(|val| val.is_infinite())
            .count();
        assert_eq!(num_infinite_results, 0);
    }

    #[rstest]
    #[tokio::test]
    async fn test_dataset_too_small(#[values(1, 20)] num_deltas: usize) {
        let fixture = NprobesTestFixture::new(100, num_deltas).await;

        let q = fixture.get_centroid(0);
        let stats_holder = StatsHolder::default();
        // There is no filter but we only have 10K rows.  Since maximum_nprobes is not set
        // we will search all partitions.
        let results = fixture
            .dataset
            .scan()
            .nearest("vector", q.as_ref(), 40000)
            .unwrap()
            .minimum_nprobes(10)
            .scan_stats_callback(stats_holder.get_setter())
            .project(&Vec::<String>::new())
            .unwrap()
            .with_row_id()
            .try_into_batch()
            .await
            .unwrap();

        let stats = stats_holder.consume();

        assert_eq!(
            stats.all_counts.get(PARTITIONS_SEARCHED_METRIC).unwrap(),
            &(100 * num_deltas)
        );
        assert_eq!(results.num_rows(), 10000);
    }
}