1use std::ops::Range;
5use std::pin::Pin;
6use std::sync::{Arc, LazyLock};
7use std::task::{Context, Poll};
8
9use arrow::array::AsArray;
10use arrow_array::{Array, Float32Array, Int64Array, RecordBatch};
11use arrow_schema::{DataType, Field as ArrowField, Schema as ArrowSchema, SchemaRef, SortOptions};
12use arrow_select::concat::concat_batches;
13use async_recursion::async_recursion;
14use chrono::Utc;
15use datafusion::common::{exec_datafusion_err, DFSchema, JoinType, NullEquality, SchemaExt};
16use datafusion::functions_aggregate;
17use datafusion::functions_aggregate::count::count_udaf;
18use datafusion::logical_expr::{col, lit, Expr, ScalarUDF};
19use datafusion::physical_expr::PhysicalSortExpr;
20use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec;
21use datafusion::physical_plan::expressions;
22use datafusion::physical_plan::projection::ProjectionExec as DFProjectionExec;
23use datafusion::physical_plan::sorts::sort::SortExec;
24use datafusion::physical_plan::{
25 aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy},
26 display::DisplayableExecutionPlan,
27 expressions::Literal,
28 limit::GlobalLimitExec,
29 repartition::RepartitionExec,
30 union::UnionExec,
31 ExecutionPlan, SendableRecordBatchStream,
32};
33use datafusion::scalar::ScalarValue;
34use datafusion_expr::execution_props::ExecutionProps;
35use datafusion_expr::ExprSchemable;
36use datafusion_functions::core::getfield::GetFieldFunc;
37use datafusion_physical_expr::{aggregate::AggregateExprBuilder, expressions::Column};
38use datafusion_physical_expr::{create_physical_expr, LexOrdering, Partitioning, PhysicalExpr};
39use datafusion_physical_plan::joins::PartitionMode;
40use datafusion_physical_plan::projection::ProjectionExec;
41use datafusion_physical_plan::stream::RecordBatchStreamAdapter;
42use datafusion_physical_plan::{empty::EmptyExec, joins::HashJoinExec};
43use futures::future::BoxFuture;
44use futures::stream::{Stream, StreamExt};
45use futures::{FutureExt, TryStreamExt};
46use lance_arrow::floats::{coerce_float_vector, FloatType};
47use lance_arrow::{DataTypeExt, SchemaExt as ArrowSchemaExt};
48use lance_core::datatypes::{
49 escape_field_path_for_project, format_field_path, BlobHandling, Field, OnMissing, Projection,
50};
51use lance_core::error::LanceOptionExt;
52use lance_core::utils::address::RowAddress;
53use lance_core::utils::mask::{RowAddrMask, RowAddrTreeMap};
54use lance_core::utils::tokio::get_num_compute_intensive_cpus;
55use lance_core::{ROW_ADDR, ROW_ID, ROW_OFFSET};
56use lance_datafusion::exec::{
57 analyze_plan, execute_plan, LanceExecutionOptions, OneShotExec, StrictBatchSizeExec,
58};
59use lance_datafusion::expr::safe_coerce_scalar;
60use lance_datafusion::projection::ProjectionPlan;
61use lance_file::reader::FileReaderOptions;
62use lance_index::scalar::expression::{IndexExprResult, PlannerIndexExt, INDEX_EXPR_RESULT_SCHEMA};
63use lance_index::scalar::inverted::query::{
64 fill_fts_query_column, FtsQuery, FtsQueryNode, FtsSearchParams, MatchQuery, PhraseQuery,
65};
66use lance_index::scalar::inverted::{SCORE_COL, SCORE_FIELD};
67use lance_index::scalar::FullTextSearchQuery;
68use lance_index::vector::{Query, DIST_COL};
69use lance_index::IndexCriteria;
70use lance_index::{metrics::NoOpMetricsCollector, scalar::inverted::FTS_SCHEMA};
71use lance_index::{scalar::expression::ScalarIndexExpr, DatasetIndexExt};
72use lance_io::stream::RecordBatchStream;
73use lance_linalg::distance::MetricType;
74use lance_table::format::{Fragment, IndexMetadata};
75use roaring::RoaringBitmap;
76use tracing::{info_span, instrument, Span};
77
78use super::Dataset;
79use crate::dataset::row_offsets_to_row_addresses;
80use crate::dataset::utils::SchemaAdapter;
81use crate::index::vector::utils::{
82 default_distance_type_for, get_vector_dim, get_vector_type, validate_distance_type_for,
83};
84use crate::index::DatasetIndexInternalExt;
85use crate::io::exec::filtered_read::{FilteredReadExec, FilteredReadOptions};
86use crate::io::exec::fts::{BoostQueryExec, FlatMatchQueryExec, MatchQueryExec, PhraseQueryExec};
87use crate::io::exec::knn::MultivectorScoringExec;
88use crate::io::exec::scalar_index::{MaterializeIndexExec, ScalarIndexExec};
89use crate::io::exec::{get_physical_optimizer, AddRowOffsetExec, LanceFilterExec, LanceScanConfig};
90use crate::io::exec::{
91 knn::new_knn_exec, project, AddRowAddrExec, FilterPlan as ExprFilterPlan,
92 KNNVectorDistanceExec, LancePushdownScanExec, LanceScanExec, Planner, PreFilterSource,
93 ScanConfig, TakeExec,
94};
95use crate::{datatypes::Schema, io::exec::fts::BooleanQueryExec};
96use crate::{Error, Result};
97
98pub use lance_datafusion::exec::{ExecutionStatsCallback, ExecutionSummaryCounts};
99#[cfg(feature = "substrait")]
100use lance_datafusion::substrait::parse_substrait;
101use snafu::location;
102
103pub(crate) const BATCH_SIZE_FALLBACK: usize = 8192;
104
105fn parse_env_var<T: std::str::FromStr>(env_var_name: &str, default_val: &str) -> Option<T>
107where
108 T::Err: std::fmt::Display,
109{
110 std::env::var(env_var_name)
111 .ok()
112 .and_then(|val| match val.parse() {
113 Ok(value) => Some(value),
114 Err(e) => {
115 log::warn!(
116 "Failed to parse the environment variable {}='{}': {}, the default value is: {}.",
117 env_var_name,
118 val,
119 e,
120 default_val
121 );
122 None
123 }
124 })
125}
126
127pub fn get_default_batch_size() -> Option<usize> {
130 parse_env_var("LANCE_DEFAULT_BATCH_SIZE", &BATCH_SIZE_FALLBACK.to_string())
131}
132
133pub const LEGACY_DEFAULT_FRAGMENT_READAHEAD: usize = 4;
134
135pub static DEFAULT_FRAGMENT_READAHEAD: LazyLock<Option<usize>> = LazyLock::new(|| {
136 parse_env_var(
137 "LANCE_DEFAULT_FRAGMENT_READAHEAD",
138 &LEGACY_DEFAULT_FRAGMENT_READAHEAD.to_string(),
139 )
140});
141
142const DEFAULT_XTR_OVERFETCH_VALUE: u32 = 10;
143
144pub static DEFAULT_XTR_OVERFETCH: LazyLock<u32> = LazyLock::new(|| {
145 parse_env_var(
146 "LANCE_XTR_OVERFETCH",
147 &DEFAULT_XTR_OVERFETCH_VALUE.to_string(),
148 )
149 .unwrap_or(DEFAULT_XTR_OVERFETCH_VALUE)
150});
151
152const DEFAULT_IO_BUFFER_SIZE_VALUE: u64 = 2 * 1024 * 1024 * 1024;
156
157pub static DEFAULT_IO_BUFFER_SIZE: LazyLock<u64> = LazyLock::new(|| {
158 parse_env_var(
159 "LANCE_DEFAULT_IO_BUFFER_SIZE",
160 &DEFAULT_IO_BUFFER_SIZE_VALUE.to_string(),
161 )
162 .unwrap_or(DEFAULT_IO_BUFFER_SIZE_VALUE)
163});
164
165#[derive(Debug, Clone)]
170pub struct ColumnOrdering {
171 pub ascending: bool,
172 pub nulls_first: bool,
173 pub column_name: String,
174}
175
176impl ColumnOrdering {
177 pub fn asc_nulls_first(column_name: String) -> Self {
178 Self {
179 ascending: true,
180 nulls_first: true,
181 column_name,
182 }
183 }
184
185 pub fn asc_nulls_last(column_name: String) -> Self {
186 Self {
187 ascending: true,
188 nulls_first: false,
189 column_name,
190 }
191 }
192
193 pub fn desc_nulls_first(column_name: String) -> Self {
194 Self {
195 ascending: false,
196 nulls_first: true,
197 column_name,
198 }
199 }
200
201 pub fn desc_nulls_last(column_name: String) -> Self {
202 Self {
203 ascending: false,
204 nulls_first: false,
205 column_name,
206 }
207 }
208}
209
210#[derive(Clone)]
224pub enum MaterializationStyle {
225 Heuristic,
237 AllLate,
239 AllEarly,
241 AllEarlyExcept(Vec<u32>),
243}
244
245impl MaterializationStyle {
246 pub fn all_early_except(columns: &[impl AsRef<str>], schema: &Schema) -> Result<Self> {
247 let field_ids = schema
248 .project(columns)?
249 .field_ids()
250 .into_iter()
251 .map(|id| id as u32)
252 .collect();
253 Ok(Self::AllEarlyExcept(field_ids))
254 }
255}
256
257#[derive(Debug)]
258struct PlannedFilteredScan {
259 plan: Arc<dyn ExecutionPlan>,
260 limit_pushed_down: bool,
261 filter_pushed_down: bool,
262}
263
264pub struct FilterPlan {
265 query_filter: Option<QueryFilter>,
267 refine_query_filter: bool,
268 expr_filter_plan: ExprFilterPlan,
270}
271
272impl FilterPlan {
273 pub fn new(query_filter: Option<QueryFilter>, expr_filter_plan: ExprFilterPlan) -> Self {
274 Self {
275 query_filter,
276 refine_query_filter: false,
277 expr_filter_plan,
278 }
279 }
280
281 pub fn disable_refine(&mut self) {
282 self.expr_filter_plan = ExprFilterPlan::default();
283 self.refine_query_filter = false;
284 }
285
286 pub fn make_refine_only(&mut self) {
287 self.expr_filter_plan.make_refine_only();
288 self.refine_query_filter = true;
289 }
290
291 pub fn fts_filter(&self) -> Option<FullTextSearchQuery> {
292 match &self.query_filter {
293 Some(QueryFilter::Fts(query)) => Some(query.clone()),
294 _ => None,
295 }
296 }
297
298 pub fn vector_filter(&self) -> Option<Query> {
299 match &self.query_filter {
300 Some(QueryFilter::Vector(query)) => Some(query.clone()),
301 _ => None,
302 }
303 }
304
305 pub fn has_refine(&self) -> bool {
306 self.expr_filter_plan.has_refine() || self.refine_query_filter
307 }
308
309 pub async fn refine_columns(&self, dataset: &Arc<Dataset>) -> Result<Vec<String>> {
310 let mut columns = vec![];
311
312 if self.expr_filter_plan.has_refine() {
313 columns.extend(self.expr_filter_plan.refine_columns());
314 }
315
316 if self.refine_query_filter {
317 match &self.query_filter {
318 Some(QueryFilter::Fts(fts_query)) => {
319 let cols = if fts_query.columns().is_empty() {
320 let indexed_columns = fts_indexed_columns(dataset.clone()).await?;
321 let q = fill_fts_query_column(&fts_query.query, &indexed_columns, false)?;
322 q.columns()
323 } else {
324 fts_query.columns()
325 };
326
327 if let FtsQuery::Match(_) = &fts_query.query {
330 columns.extend(cols.iter().cloned().collect::<Vec<_>>());
331 }
332 }
333 Some(QueryFilter::Vector(vector_query)) => {
334 columns.push(vector_query.column.clone());
335 }
336 None => {}
337 }
338 }
339
340 Ok(columns)
341 }
342
343 pub async fn refine_filter(
344 &self,
345 input: Arc<dyn ExecutionPlan>,
346 scanner: &Scanner,
347 ) -> Result<Arc<dyn ExecutionPlan>> {
348 let mut plan = input;
349
350 if self.refine_query_filter {
351 match &self.query_filter {
352 Some(QueryFilter::Fts(fts_query)) => {
353 plan = scanner.flat_fts(plan, fts_query).await?;
354 }
355 Some(QueryFilter::Vector(vector_query)) => {
356 plan = scanner.flat_knn(plan, vector_query)?;
357 }
358 None => {}
359 }
360 }
361
362 if let Some(refine_expr) = &self.expr_filter_plan.refine_expr {
363 plan = Arc::new(LanceFilterExec::try_new(refine_expr.clone(), plan)?);
366 }
367
368 Ok(plan)
369 }
370}
371
372#[derive(Debug, Clone, Default)]
373pub struct LanceFilter {
374 query_filter: Option<QueryFilter>,
375 expr_filter: Option<ExprFilter>,
376}
377
378impl LanceFilter {
379 pub fn is_none(&self) -> bool {
380 self.query_filter.is_none() && self.expr_filter.is_none()
381 }
382}
383
384#[derive(Debug, Clone)]
386pub enum QueryFilter {
387 Fts(FullTextSearchQuery),
388 Vector(Query),
389}
390
391#[derive(Debug, Clone)]
393pub enum ExprFilter {
394 Sql(String),
396 Substrait(Vec<u8>),
398 Datafusion(Expr),
400}
401
402impl ExprFilter {
403 #[allow(unused)]
412 #[instrument(level = "trace", name = "filter_to_df", skip_all)]
413 pub fn to_datafusion(&self, dataset_schema: &Schema, full_schema: &Schema) -> Result<Expr> {
414 match self {
415 Self::Sql(sql) => {
416 let schema = Arc::new(ArrowSchema::from(full_schema));
417 let planner = Planner::new(schema.clone());
418 let filter = planner.parse_filter(sql)?;
419
420 let df_schema = DFSchema::try_from(schema)?;
421 let (ret_type, _) = filter.data_type_and_nullable(&df_schema)?;
422 if ret_type != DataType::Boolean {
423 return Err(Error::InvalidInput {
424 source: format!("The filter {} does not return a boolean", filter).into(),
425 location: location!(),
426 });
427 }
428
429 let optimized = planner.optimize_expr(filter).map_err(|e| {
430 Error::invalid_input(
431 format!("Error optimizing sql filter: {sql} ({e})"),
432 location!(),
433 )
434 })?;
435 Ok(optimized)
436 }
437 #[cfg(feature = "substrait")]
438 Self::Substrait(expr) => {
439 use lance_datafusion::exec::{get_session_context, LanceExecutionOptions};
440
441 let ctx = get_session_context(&LanceExecutionOptions::default());
442 let state = ctx.state();
443 let schema = Arc::new(ArrowSchema::from(dataset_schema));
444 let expr = parse_substrait(expr, schema.clone(), &ctx.state())
445 .now_or_never()
446 .expect("could not parse the Substrait filter in a synchronous fashion")?;
447 let planner = Planner::new(schema);
448 planner.optimize_expr(expr.clone()).map_err(|e| {
449 Error::invalid_input(
450 format!("Error optimizing substrait filter: {expr:?} ({e})"),
451 location!(),
452 )
453 })
454 }
455 #[cfg(not(feature = "substrait"))]
456 Self::Substrait(_) => Err(Error::NotSupported {
457 source: "Substrait filter is not supported in this build".into(),
458 location: location!(),
459 }),
460 Self::Datafusion(expr) => Ok(expr.clone()),
461 }
462 }
463}
464
465#[derive(Clone)]
479pub struct Scanner {
480 dataset: Arc<Dataset>,
481
482 projection_plan: ProjectionPlan,
489 blob_handling: BlobHandling,
490
491 prefilter: bool,
493
494 materialization_style: MaterializationStyle,
496
497 filter: LanceFilter,
499
500 full_text_query: Option<FullTextSearchQuery>,
502
503 batch_size: Option<usize>,
505
506 batch_readahead: usize,
508
509 fragment_readahead: Option<usize>,
511
512 io_buffer_size: Option<u64>,
514
515 limit: Option<i64>,
516 offset: Option<i64>,
517
518 ordering: Option<Vec<ColumnOrdering>>,
527
528 nearest: Option<Query>,
529
530 use_scalar_index: bool,
536
537 use_stats: bool,
541
542 ordered: bool,
546
547 fragments: Option<Vec<Fragment>>,
549
550 fast_search: bool,
557
558 include_deleted_rows: bool,
560
561 scan_stats_callback: Option<ExecutionStatsCallback>,
563
564 strict_batch_size: bool,
569
570 file_reader_options: Option<FileReaderOptions>,
572
573 legacy_with_row_id: bool,
594 legacy_with_row_addr: bool,
596 explicit_projection: bool,
599 autoproject_scoring_columns: bool,
601}
602
603#[derive(Debug, Clone)]
605pub enum TakeOperation {
606 RowIds(Vec<u64>),
608 RowAddrs(Vec<u64>),
610 RowOffsets(Vec<u64>),
615}
616
617impl TakeOperation {
618 fn extract_u64_list(list: &[Expr]) -> Option<Vec<u64>> {
619 let mut u64s = Vec::with_capacity(list.len());
620 for expr in list {
621 if let Expr::Literal(lit, _) = expr {
622 if let Some(ScalarValue::UInt64(Some(val))) =
623 safe_coerce_scalar(lit, &DataType::UInt64)
624 {
625 u64s.push(val);
626 } else {
627 return None;
628 }
629 } else {
630 return None;
631 }
632 }
633 Some(u64s)
634 }
635
636 fn merge(self, other: Self) -> Option<Self> {
637 match (self, other) {
638 (Self::RowIds(mut left), Self::RowIds(right)) => {
639 left.extend(right);
640 Some(Self::RowIds(left))
641 }
642 (Self::RowAddrs(mut left), Self::RowAddrs(right)) => {
643 left.extend(right);
644 Some(Self::RowAddrs(left))
645 }
646 (Self::RowOffsets(mut left), Self::RowOffsets(right)) => {
647 left.extend(right);
648 Some(Self::RowOffsets(left))
649 }
650 _ => None,
651 }
652 }
653
654 fn try_from_expr(expr: &Expr) -> Option<(Self, Option<Expr>)> {
672 if let Expr::BinaryExpr(binary) = expr {
673 match binary.op {
674 datafusion_expr::Operator::And => {
675 let left_take = Self::try_from_expr(&binary.left);
676 let right_take = Self::try_from_expr(&binary.right);
677 match (left_take, right_take) {
678 (Some(_), Some(_)) => {
679 return None;
685 }
686 (Some((left_op, left_rem)), None) => {
687 let remainder = match left_rem {
688 Some(expr) => Expr::and(expr, binary.right.as_ref().clone()),
692 None => binary.right.as_ref().clone(),
693 };
694 return Some((left_op, Some(remainder)));
695 }
696 (None, Some((right_op, right_rem))) => {
697 let remainder = match right_rem {
698 Some(expr) => Expr::and(expr, binary.left.as_ref().clone()),
699 None => binary.left.as_ref().clone(),
700 };
701 return Some((right_op, Some(remainder)));
702 }
703 (None, None) => {
704 return None;
705 }
706 }
707 }
708 datafusion_expr::Operator::Eq => {
709 if let (Expr::Column(col), Expr::Literal(lit, _)) =
711 (binary.left.as_ref(), binary.right.as_ref())
712 {
713 if let Some(ScalarValue::UInt64(Some(val))) =
714 safe_coerce_scalar(lit, &DataType::UInt64)
715 {
716 if col.name == ROW_ID {
717 return Some((Self::RowIds(vec![val]), None));
718 } else if col.name == ROW_ADDR {
719 return Some((Self::RowAddrs(vec![val]), None));
720 } else if col.name == ROW_OFFSET {
721 return Some((Self::RowOffsets(vec![val]), None));
722 }
723 }
724 }
725 }
726 datafusion_expr::Operator::Or => {
727 let left_take = Self::try_from_expr(&binary.left);
728 let right_take = Self::try_from_expr(&binary.right);
729 if let (Some(left), Some(right)) = (left_take, right_take) {
730 if left.1.is_some() || right.1.is_some() {
731 return None;
738 }
739 return left.0.merge(right.0).map(|op| (op, None));
740 }
741 }
742 _ => {}
743 }
744 } else if let Expr::InList(in_expr) = expr {
745 if let Expr::Column(col) = in_expr.expr.as_ref() {
746 if let Some(u64s) = Self::extract_u64_list(&in_expr.list) {
747 if col.name == ROW_ID {
748 return Some((Self::RowIds(u64s), None));
749 } else if col.name == ROW_ADDR {
750 return Some((Self::RowAddrs(u64s), None));
751 } else if col.name == ROW_OFFSET {
752 return Some((Self::RowOffsets(u64s), None));
753 }
754 }
755 }
756 }
757 None
758 }
759}
760
761impl Scanner {
762 pub fn new(dataset: Arc<Dataset>) -> Self {
763 let projection_plan =
764 ProjectionPlan::full(dataset.clone(), dataset.blob_version()).unwrap();
765 let file_reader_options = dataset.file_reader_options.clone();
766 let mut scanner = Self {
767 dataset,
768 projection_plan,
769 blob_handling: BlobHandling::default(),
770 prefilter: false,
771 materialization_style: MaterializationStyle::Heuristic,
772 filter: LanceFilter::default(),
773 full_text_query: None,
774 batch_size: None,
775 batch_readahead: get_num_compute_intensive_cpus(),
776 fragment_readahead: None,
777 io_buffer_size: None,
778 limit: None,
779 offset: None,
780 ordering: None,
781 nearest: None,
782 use_stats: true,
783 ordered: true,
784 fragments: None,
785 fast_search: false,
786 use_scalar_index: true,
787 include_deleted_rows: false,
788 scan_stats_callback: None,
789 strict_batch_size: false,
790 file_reader_options,
791 legacy_with_row_addr: false,
792 legacy_with_row_id: false,
793 explicit_projection: false,
794 autoproject_scoring_columns: true,
795 };
796 scanner.apply_blob_handling();
797 scanner
798 }
799
800 fn apply_blob_handling(&mut self) {
801 let projection = self
802 .projection_plan
803 .physical_projection
804 .clone()
805 .with_blob_handling(self.blob_handling.clone());
806 self.projection_plan.physical_projection = projection;
807 }
808
809 pub fn blob_handling(&mut self, blob_handling: BlobHandling) -> &mut Self {
810 self.blob_handling = blob_handling;
811 self.apply_blob_handling();
812 self
813 }
814
815 pub fn from_fragment(dataset: Arc<Dataset>, fragment: Fragment) -> Self {
816 Self {
817 fragments: Some(vec![fragment]),
818 ..Self::new(dataset)
819 }
820 }
821
822 pub fn with_fragments(&mut self, fragments: Vec<Fragment>) -> &mut Self {
826 self.fragments = Some(fragments);
827 self
828 }
829
830 fn get_batch_size(&self) -> usize {
831 get_default_batch_size().unwrap_or_else(|| {
837 self.batch_size.unwrap_or_else(|| {
838 std::cmp::max(
839 self.dataset.object_store().block_size() / 4,
840 BATCH_SIZE_FALLBACK,
841 )
842 })
843 })
844 }
845
846 fn ensure_not_fragment_scan(&self) -> Result<()> {
847 if self.is_fragment_scan() {
848 Err(Error::not_supported(
849 "This operation is not supported for fragment scan".to_string(),
850 location!(),
851 ))
852 } else {
853 Ok(())
854 }
855 }
856
857 fn is_fragment_scan(&self) -> bool {
858 self.fragments.is_some()
859 }
860
861 pub fn empty_project(&mut self) -> Result<&mut Self> {
865 self.project(&[] as &[&str])
866 }
867
868 pub fn project<T: AsRef<str>>(&mut self, columns: &[T]) -> Result<&mut Self> {
872 let transformed_columns: Vec<(&str, String)> = columns
873 .iter()
874 .map(|c| (c.as_ref(), escape_field_path_for_project(c.as_ref())))
875 .collect();
876
877 self.project_with_transform(&transformed_columns)
878 }
879
880 pub fn project_with_transform(
884 &mut self,
885 columns: &[(impl AsRef<str>, impl AsRef<str>)],
886 ) -> Result<&mut Self> {
887 self.explicit_projection = true;
888 self.projection_plan = ProjectionPlan::from_expressions(
889 self.dataset.clone(),
890 columns,
891 self.dataset.blob_version(),
892 )?;
893 if self.legacy_with_row_id {
894 self.projection_plan.include_row_id();
895 }
896 if self.legacy_with_row_addr {
897 self.projection_plan.include_row_addr();
898 }
899 self.apply_blob_handling();
900 Ok(self)
901 }
902
903 pub fn prefilter(&mut self, should_prefilter: bool) -> &mut Self {
912 self.prefilter = should_prefilter;
913 self
914 }
915
916 pub fn scan_stats_callback(&mut self, callback: ExecutionStatsCallback) -> &mut Self {
918 self.scan_stats_callback = Some(callback);
919 self
920 }
921
922 pub fn materialization_style(&mut self, style: MaterializationStyle) -> &mut Self {
934 self.materialization_style = style;
935 self
936 }
937
938 pub fn filter(&mut self, filter: &str) -> Result<&mut Self> {
954 self.filter.expr_filter = Some(ExprFilter::Sql(filter.to_string()));
955 Ok(self)
956 }
957
958 pub fn filter_query(&mut self, filter: QueryFilter) -> Result<&mut Self> {
977 self.filter.query_filter = Some(filter);
978 Ok(self)
979 }
980
981 pub fn full_text_search(&mut self, query: FullTextSearchQuery) -> Result<&mut Self> {
995 let fields = query.columns();
996 if !fields.is_empty() {
997 for field in fields.iter() {
998 if self.dataset.schema().field(field).is_none() {
999 return Err(Error::invalid_input(
1000 format!("Column {} not found", field),
1001 location!(),
1002 ));
1003 }
1004 }
1005 }
1006
1007 self.full_text_query = Some(query);
1008 Ok(self)
1009 }
1010
1011 pub fn filter_substrait(&mut self, filter: &[u8]) -> Result<&mut Self> {
1016 self.filter.expr_filter = Some(ExprFilter::Substrait(filter.to_vec()));
1017 Ok(self)
1018 }
1019
1020 pub fn filter_expr(&mut self, filter: Expr) -> &mut Self {
1021 self.filter.expr_filter = Some(ExprFilter::Datafusion(filter));
1022 self
1023 }
1024
1025 pub fn batch_size(&mut self, batch_size: usize) -> &mut Self {
1027 self.batch_size = Some(batch_size);
1028 self
1029 }
1030
1031 pub fn include_deleted_rows(&mut self) -> &mut Self {
1042 self.include_deleted_rows = true;
1043 self
1044 }
1045
1046 pub fn io_buffer_size(&mut self, size: u64) -> &mut Self {
1065 self.io_buffer_size = Some(size);
1066 self
1067 }
1068
1069 pub fn batch_readahead(&mut self, nbatches: usize) -> &mut Self {
1072 self.batch_readahead = nbatches;
1073 self
1074 }
1075
1076 pub fn fragment_readahead(&mut self, nfragments: usize) -> &mut Self {
1080 self.fragment_readahead = Some(nfragments);
1081 self
1082 }
1083
1084 pub fn scan_in_order(&mut self, ordered: bool) -> &mut Self {
1098 self.ordered = ordered;
1099 self
1100 }
1101
1102 pub fn use_scalar_index(&mut self, use_scalar_index: bool) -> &mut Self {
1108 self.use_scalar_index = use_scalar_index;
1109 self
1110 }
1111
1112 pub fn strict_batch_size(&mut self, strict_batch_size: bool) -> &mut Self {
1119 self.strict_batch_size = strict_batch_size;
1120 self
1121 }
1122
1123 pub fn limit(&mut self, limit: Option<i64>, offset: Option<i64>) -> Result<&mut Self> {
1130 if limit.unwrap_or_default() < 0 {
1131 return Err(Error::invalid_input(
1132 "Limit must be non-negative".to_string(),
1133 location!(),
1134 ));
1135 }
1136 if let Some(off) = offset {
1137 if off < 0 {
1138 return Err(Error::invalid_input(
1139 "Offset must be non-negative".to_string(),
1140 location!(),
1141 ));
1142 }
1143 }
1144 self.limit = limit;
1145 self.offset = offset;
1146 Ok(self)
1147 }
1148
1149 pub fn nearest(&mut self, column: &str, q: &dyn Array, k: usize) -> Result<&mut Self> {
1153 if !self.prefilter {
1154 self.ensure_not_fragment_scan()?;
1157 }
1158
1159 if k == 0 {
1160 return Err(Error::invalid_input(
1161 "k must be positive".to_string(),
1162 location!(),
1163 ));
1164 }
1165 if q.is_empty() {
1166 return Err(Error::invalid_input(
1167 "Query vector must have non-zero length".to_string(),
1168 location!(),
1169 ));
1170 }
1171 let (vector_type, element_type) = get_vector_type(self.dataset.schema(), column)?;
1173 let dim = get_vector_dim(self.dataset.schema(), column)?;
1174
1175 let q = match q.data_type() {
1176 DataType::List(_) | DataType::FixedSizeList(_, _) => {
1177 if !matches!(vector_type, DataType::List(_)) {
1178 return Err(Error::invalid_input(
1179 format!(
1180 "Query is multivector but column {}({})is not multivector",
1181 column, vector_type,
1182 ),
1183 location!(),
1184 ));
1185 }
1186
1187 if let Some(list_array) = q.as_list_opt::<i32>() {
1188 for i in 0..list_array.len() {
1189 let vec = list_array.value(i);
1190 if vec.len() != dim {
1191 return Err(Error::invalid_input(
1192 format!(
1193 "query dim({}) doesn't match the column {} vector dim({})",
1194 vec.len(),
1195 column,
1196 dim,
1197 ),
1198 location!(),
1199 ));
1200 }
1201 }
1202 list_array.values().clone()
1203 } else {
1204 let fsl = q.as_fixed_size_list();
1205 if fsl.value_length() as usize != dim {
1206 return Err(Error::invalid_input(
1207 format!(
1208 "query dim({}) doesn't match the column {} vector dim({})",
1209 fsl.value_length(),
1210 column,
1211 dim,
1212 ),
1213 location!(),
1214 ));
1215 }
1216 fsl.values().clone()
1217 }
1218 }
1219 _ => {
1220 if q.len() != dim {
1221 return Err(Error::invalid_input(
1222 format!(
1223 "query dim({}) doesn't match the column {} vector dim({})",
1224 q.len(),
1225 column,
1226 dim,
1227 ),
1228 location!(),
1229 ));
1230 }
1231 q.slice(0, q.len())
1232 }
1233 };
1234
1235 let key = match &element_type {
1236 dt if dt == q.data_type() => q,
1237 dt if dt.is_floating() => coerce_float_vector(
1238 q.as_any().downcast_ref::<Float32Array>().unwrap(),
1239 FloatType::try_from(dt)?,
1240 )?,
1241 _ => {
1242 return Err(Error::invalid_input(
1243 format!(
1244 "Column {} has element type {} and the query vector is {}",
1245 column,
1246 element_type,
1247 q.data_type(),
1248 ),
1249 location!(),
1250 ));
1251 }
1252 };
1253
1254 self.nearest = Some(Query {
1255 column: column.to_string(),
1256 key,
1257 k,
1258 lower_bound: None,
1259 upper_bound: None,
1260 minimum_nprobes: 1,
1261 maximum_nprobes: None,
1262 ef: None,
1263 refine_factor: None,
1264 metric_type: None,
1265 use_index: true,
1266 dist_q_c: 0.0,
1267 });
1268 Ok(self)
1269 }
1270
1271 #[cfg(test)]
1272 fn nearest_mut(&mut self) -> Option<&mut Query> {
1273 self.nearest.as_mut()
1274 }
1275
1276 pub fn distance_range(
1278 &mut self,
1279 lower_bound: Option<f32>,
1280 upper_bound: Option<f32>,
1281 ) -> &mut Self {
1282 if let Some(q) = self.nearest.as_mut() {
1283 q.lower_bound = lower_bound;
1284 q.upper_bound = upper_bound;
1285 }
1286 self
1287 }
1288
1289 pub fn nprobes(&mut self, n: usize) -> &mut Self {
1294 if let Some(q) = self.nearest.as_mut() {
1295 q.minimum_nprobes = n;
1296 q.maximum_nprobes = Some(n);
1297 } else {
1298 log::warn!("nprobes is not set because nearest has not been called yet");
1299 }
1300 self
1301 }
1302
1303 #[deprecated(note = "Use nprobes instead")]
1308 pub fn nprobs(&mut self, n: usize) -> &mut Self {
1309 if let Some(q) = self.nearest.as_mut() {
1310 q.minimum_nprobes = n;
1311 q.maximum_nprobes = Some(n);
1312 } else {
1313 log::warn!("nprobes is not set because nearest has not been called yet");
1314 }
1315 self
1316 }
1317
1318 pub fn minimum_nprobes(&mut self, n: usize) -> &mut Self {
1326 if let Some(q) = self.nearest.as_mut() {
1327 q.minimum_nprobes = n;
1328 } else {
1329 log::warn!("minimum_nprobes is not set because nearest has not been called yet");
1330 }
1331 self
1332 }
1333
1334 pub fn maximum_nprobes(&mut self, n: usize) -> &mut Self {
1346 if let Some(q) = self.nearest.as_mut() {
1347 q.maximum_nprobes = Some(n);
1348 } else {
1349 log::warn!("maximum_nprobes is not set because nearest has not been called yet");
1350 }
1351 self
1352 }
1353
1354 pub fn ef(&mut self, ef: usize) -> &mut Self {
1355 if let Some(q) = self.nearest.as_mut() {
1356 q.ef = Some(ef);
1357 }
1358 self
1359 }
1360
1361 pub fn fast_search(&mut self) -> &mut Self {
1367 if let Some(q) = self.nearest.as_mut() {
1368 q.use_index = true;
1369 }
1370 self.fast_search = true;
1371 self.projection_plan.include_row_id(); self
1373 }
1374
1375 pub fn refine(&mut self, factor: u32) -> &mut Self {
1385 if let Some(q) = self.nearest.as_mut() {
1386 q.refine_factor = Some(factor)
1387 };
1388 self
1389 }
1390
1391 pub fn distance_metric(&mut self, metric_type: MetricType) -> &mut Self {
1393 if let Some(q) = self.nearest.as_mut() {
1394 q.metric_type = Some(metric_type)
1395 }
1396 self
1397 }
1398
1399 pub fn order_by(&mut self, ordering: Option<Vec<ColumnOrdering>>) -> Result<&mut Self> {
1405 if let Some(ordering) = &ordering {
1406 if ordering.is_empty() {
1407 self.ordering = None;
1408 return Ok(self);
1409 }
1410 for column in ordering {
1412 self.dataset
1413 .schema()
1414 .field(&column.column_name)
1415 .ok_or(Error::invalid_input(
1416 format!("Column {} not found", &column.column_name),
1417 location!(),
1418 ))?;
1419 }
1420 }
1421 self.ordering = ordering;
1422 Ok(self)
1423 }
1424
1425 pub fn use_index(&mut self, use_index: bool) -> &mut Self {
1427 if let Some(q) = self.nearest.as_mut() {
1428 q.use_index = use_index
1429 }
1430 self
1431 }
1432
1433 pub fn with_row_id(&mut self) -> &mut Self {
1435 self.legacy_with_row_id = true;
1436 self.projection_plan.include_row_id();
1437 self
1438 }
1439
1440 pub fn with_row_address(&mut self) -> &mut Self {
1442 self.legacy_with_row_addr = true;
1443 self.projection_plan.include_row_addr();
1444 self
1445 }
1446
1447 pub fn disable_scoring_autoprojection(&mut self) -> &mut Self {
1463 self.autoproject_scoring_columns = false;
1464 self
1465 }
1466
1467 pub fn with_file_reader_options(&mut self, options: FileReaderOptions) -> &mut Self {
1469 self.file_reader_options = Some(options);
1470 self
1471 }
1472
1473 fn create_column_expr(
1475 column_name: &str,
1476 dataset: &Dataset,
1477 arrow_schema: &ArrowSchema,
1478 ) -> Result<Arc<dyn PhysicalExpr>> {
1479 let lance_schema = dataset.schema();
1480 let field_path = lance_schema
1481 .resolve_case_insensitive(column_name)
1482 .ok_or_else(|| {
1483 Error::invalid_input(
1484 format!("Field '{}' not found in schema", column_name),
1485 location!(),
1486 )
1487 })?;
1488
1489 if field_path.len() == 1 {
1490 expressions::col(&field_path[0].name, arrow_schema).map_err(|e| Error::Internal {
1492 message: format!(
1493 "Failed to create column expression for '{}': {}",
1494 column_name, e
1495 ),
1496 location: location!(),
1497 })
1498 } else {
1499 let get_field_func = ScalarUDF::from(GetFieldFunc::default());
1501
1502 let mut expr = Expr::Column(datafusion::common::Column::new_unqualified(
1505 &field_path[0].name,
1506 ));
1507 for nested_field in &field_path[1..] {
1508 expr = get_field_func.call(vec![expr, lit(&nested_field.name)]);
1509 }
1510
1511 let df_schema = Arc::new(DFSchema::try_from(arrow_schema.clone())?);
1513 let execution_props = ExecutionProps::new().with_query_execution_start_time(Utc::now());
1514 create_physical_expr(&expr, &df_schema, &execution_props).map_err(|e| Error::Internal {
1515 message: format!(
1516 "Failed to create physical expression for nested field '{}': {}",
1517 column_name, e
1518 ),
1519 location: location!(),
1520 })
1521 }
1522 }
1523
1524 pub fn use_stats(&mut self, use_stats: bool) -> &mut Self {
1528 self.use_stats = use_stats;
1529 self
1530 }
1531
1532 pub async fn schema(&self) -> Result<SchemaRef> {
1534 let plan = self.create_plan().await?;
1535 Ok(plan.schema())
1536 }
1537
1538 pub fn get_expr_filter(&self) -> Result<Option<Expr>> {
1545 if let Some(filter) = &self.filter.expr_filter {
1546 let filter_schema = self.filterable_schema()?;
1547 Ok(Some(filter.to_datafusion(
1548 self.dataset.schema(),
1549 filter_schema.as_ref(),
1550 )?))
1551 } else {
1552 Ok(None)
1553 }
1554 }
1555
1556 fn add_extra_columns(&self, schema: Schema) -> Result<Schema> {
1557 let mut extra_columns = vec![ArrowField::new(ROW_OFFSET, DataType::UInt64, true)];
1558
1559 if self.nearest.as_ref().is_some() {
1560 extra_columns.push(ArrowField::new(DIST_COL, DataType::Float32, true));
1561 };
1562
1563 if self.full_text_query.is_some() {
1564 extra_columns.push(ArrowField::new(SCORE_COL, DataType::Float32, true));
1565 }
1566
1567 schema.merge(&ArrowSchema::new(extra_columns))
1568 }
1569
1570 fn filterable_schema(&self) -> Result<Arc<Schema>> {
1575 let base_schema = Projection::full(self.dataset.clone())
1576 .with_row_id()
1577 .with_row_addr()
1578 .with_row_last_updated_at_version()
1579 .with_row_created_at_version()
1580 .to_schema();
1581
1582 Ok(Arc::new(self.add_extra_columns(base_schema)?))
1583 }
1584
1585 pub(crate) fn calculate_final_projection(
1590 &self,
1591 current_schema: &ArrowSchema,
1592 ) -> Result<Vec<(Arc<dyn PhysicalExpr>, String)>> {
1593 let mut output_expr = self.projection_plan.to_physical_exprs(current_schema)?;
1596
1597 if self.autoproject_scoring_columns {
1600 if self.nearest.is_some() && output_expr.iter().all(|(_, name)| name != DIST_COL) {
1601 if self.explicit_projection {
1602 log::warn!("Deprecation warning, this behavior will change in the future. This search specified output columns but did not include `_distance`. Currently the `_distance` column will be included. In the future it will not. Call `disable_scoring_autoprojection` to adopt the future behavior and avoid this warning");
1603 }
1604 let vector_expr = expressions::col(DIST_COL, current_schema)?;
1605 output_expr.push((vector_expr, DIST_COL.to_string()));
1606 }
1607 if self.full_text_query.is_some()
1608 && output_expr.iter().all(|(_, name)| name != SCORE_COL)
1609 {
1610 if self.explicit_projection {
1611 log::warn!("Deprecation warning, this behavior will change in the future. This search specified output columns but did not include `_score`. Currently the `_score` column will be included. In the future it will not. Call `disable_scoring_autoprojection` to adopt the future behavior and avoid this warning");
1612 }
1613 let score_expr = expressions::col(SCORE_COL, current_schema)?;
1614 output_expr.push((score_expr, SCORE_COL.to_string()));
1615 }
1616 }
1617
1618 if self.legacy_with_row_id {
1619 let row_id_pos = output_expr
1620 .iter()
1621 .position(|(_, name)| name == ROW_ID)
1622 .ok_or_else(|| Error::Internal {
1623 message:
1624 "user specified with_row_id but the _rowid column was not in the output"
1625 .to_string(),
1626 location: location!(),
1627 })?;
1628 if row_id_pos != output_expr.len() - 1 {
1629 let row_id_expr = output_expr.remove(row_id_pos);
1631 output_expr.push(row_id_expr);
1632 }
1633 }
1634
1635 if self.legacy_with_row_addr {
1636 let row_addr_pos = output_expr.iter().position(|(_, name)| name == ROW_ADDR).ok_or_else(|| {
1637 Error::Internal {
1638 message: "user specified with_row_address but the _rowaddr column was not in the output".to_string(),
1639 location: location!(),
1640 }
1641 })?;
1642 if row_addr_pos != output_expr.len() - 1 {
1643 let row_addr_expr = output_expr.remove(row_addr_pos);
1645 output_expr.push(row_addr_expr);
1646 }
1647 }
1648
1649 Ok(output_expr)
1650 }
1651
1652 #[instrument(skip_all)]
1654 pub fn try_into_stream(&self) -> BoxFuture<'_, Result<DatasetRecordBatchStream>> {
1655 async move {
1657 let plan = self.create_plan().await?;
1658
1659 Ok(DatasetRecordBatchStream::new(execute_plan(
1660 plan,
1661 LanceExecutionOptions {
1662 batch_size: self.batch_size,
1663 execution_stats_callback: self.scan_stats_callback.clone(),
1664 ..Default::default()
1665 },
1666 )?))
1667 }
1668 .boxed()
1669 }
1670
1671 pub(crate) async fn try_into_dfstream(
1672 &self,
1673 mut options: LanceExecutionOptions,
1674 ) -> Result<SendableRecordBatchStream> {
1675 let plan = self.create_plan().await?;
1676
1677 if options.execution_stats_callback.is_none() {
1679 options.execution_stats_callback = self.scan_stats_callback.clone();
1680 }
1681
1682 execute_plan(plan, options)
1683 }
1684
1685 pub async fn try_into_batch(&self) -> Result<RecordBatch> {
1686 let stream = self.try_into_stream().await?;
1687 let schema = stream.schema();
1688 let batches = stream.try_collect::<Vec<_>>().await?;
1689 Ok(concat_batches(&schema, &batches)?)
1690 }
1691
1692 pub fn create_count_plan(&self) -> BoxFuture<'_, Result<Arc<dyn ExecutionPlan>>> {
1693 async move {
1695 if self.projection_plan.physical_projection.is_empty() {
1696 return Err(Error::invalid_input("count_rows called but with_row_id is false".to_string(), location!()));
1697 }
1698 if !self.projection_plan.physical_projection.is_metadata_only() {
1699 let physical_schema = self.projection_plan.physical_projection.to_schema();
1700 let columns: Vec<&str> = physical_schema.fields
1701 .iter()
1702 .map(|field| field.name.as_str())
1703 .collect();
1704
1705 let msg = format!(
1706 "count_rows should not be called on a plan selecting columns. selected columns: [{}]",
1707 columns.join(", ")
1708 );
1709
1710 return Err(Error::invalid_input(msg, location!()));
1711 }
1712
1713 if self.limit.is_some() || self.offset.is_some() {
1714 log::warn!(
1715 "count_rows called with limit or offset which could have surprising results"
1716 );
1717 }
1718
1719 let plan = self.create_plan().await?;
1720 let one = Arc::new(Literal::new(ScalarValue::UInt8(Some(1))));
1722
1723 let input_phy_exprs: &[Arc<dyn PhysicalExpr>] = &[one];
1724 let schema = plan.schema();
1725
1726 let mut builder = AggregateExprBuilder::new(count_udaf(), input_phy_exprs.to_vec());
1727 builder = builder.schema(schema);
1728 builder = builder.alias("count_rows".to_string());
1729
1730 let count_expr = builder.build()?;
1731
1732 let plan_schema = plan.schema();
1733 Ok(Arc::new(AggregateExec::try_new(
1734 AggregateMode::Single,
1735 PhysicalGroupBy::new_single(Vec::new()),
1736 vec![Arc::new(count_expr)],
1737 vec![None],
1738 plan,
1739 plan_schema,
1740 )?) as Arc<dyn ExecutionPlan>)
1741 }
1742 .boxed()
1743 }
1744
1745 #[instrument(skip_all)]
1750 pub fn count_rows(&self) -> BoxFuture<'_, Result<u64>> {
1751 async move {
1753 let count_plan = self.create_count_plan().await?;
1754 let mut stream = execute_plan(count_plan, LanceExecutionOptions::default())?;
1755
1756 if let Some(first_batch) = stream.next().await {
1758 let batch = first_batch?;
1759 let array = batch
1760 .column(0)
1761 .as_any()
1762 .downcast_ref::<Int64Array>()
1763 .ok_or(Error::invalid_input(
1764 "Count plan did not return a UInt64Array".to_string(),
1765 location!(),
1766 ))?;
1767 Ok(array.value(0) as u64)
1768 } else {
1769 Ok(0)
1770 }
1771 }
1772 .boxed()
1773 }
1774
1775 fn is_early_field(&self, field: &Field) -> bool {
1796 match self.materialization_style {
1797 MaterializationStyle::AllEarly => true,
1798 MaterializationStyle::AllLate => false,
1799 MaterializationStyle::AllEarlyExcept(ref cols) => !cols.contains(&(field.id as u32)),
1800 MaterializationStyle::Heuristic => {
1801 if field.is_blob() {
1802 return true;
1807 }
1808
1809 let byte_width = field.data_type().byte_width_opt();
1810 let is_cloud = self.dataset.object_store().is_cloud();
1811 if is_cloud {
1812 byte_width.is_some_and(|bw| bw < 1000)
1813 } else {
1814 byte_width.is_some_and(|bw| bw < 10)
1815 }
1816 }
1817 }
1818 }
1819
1820 fn calc_eager_projection(
1825 &self,
1826 filter_plan: &ExprFilterPlan,
1827 desired_projection: &Projection,
1828 ) -> Result<Projection> {
1829 let filter_columns = filter_plan.all_columns();
1836
1837 let filter_schema = self
1838 .dataset
1839 .empty_projection()
1840 .union_columns(filter_columns, OnMissing::Error)?
1841 .into_schema();
1842
1843 Ok(desired_projection
1845 .clone()
1846 .subtract_predicate(|f| !self.is_early_field(f))
1848 .union_schema(&filter_schema))
1850 }
1851
1852 fn validate_options(&self) -> Result<()> {
1853 if self.include_deleted_rows && !self.projection_plan.physical_projection.with_row_id {
1854 return Err(Error::InvalidInput {
1855 source: "include_deleted_rows is set but with_row_id is false".into(),
1856 location: location!(),
1857 });
1858 }
1859
1860 Ok(())
1861 }
1862
1863 async fn create_filter_plan(&self, use_scalar_index: bool) -> Result<FilterPlan> {
1864 let filter_schema = self.filterable_schema()?;
1865 let planner = Planner::new(Arc::new(filter_schema.as_ref().into()));
1866
1867 let filter_plan = if let Some(filter) = self.filter.expr_filter.as_ref() {
1869 let expr = filter.to_datafusion(self.dataset.schema(), filter_schema.as_ref())?;
1870 let index_info = self.dataset.scalar_index_info().await?;
1871 let filter_plan =
1872 planner.create_filter_plan(expr.clone(), &index_info, use_scalar_index)?;
1873
1874 if filter_plan.index_query.is_some() {
1877 let fragments = if let Some(fragments) = self.fragments.as_ref() {
1878 fragments
1879 } else {
1880 self.dataset.fragments()
1881 };
1882 let mut has_missing_row_count = false;
1883 for frag in fragments {
1884 if frag.physical_rows.is_none() {
1885 has_missing_row_count = true;
1886 break;
1887 }
1888 }
1889 if has_missing_row_count {
1890 let filter_plan =
1893 planner.create_filter_plan(expr.clone(), &index_info, false)?;
1894 FilterPlan::new(self.filter.query_filter.clone(), filter_plan)
1895 } else {
1896 FilterPlan::new(self.filter.query_filter.clone(), filter_plan)
1897 }
1898 } else {
1899 FilterPlan::new(self.filter.query_filter.clone(), filter_plan)
1900 }
1901 } else {
1902 FilterPlan::new(self.filter.query_filter.clone(), ExprFilterPlan::default())
1903 };
1904
1905 if filter_plan.query_filter.is_some()
1907 && self.nearest.is_none()
1908 && self.full_text_query.is_none()
1909 {
1910 return Err(Error::InvalidInput {
1911 source: "Query filter can only be used with full text search or vector search"
1912 .into(),
1913 location: location!(),
1914 });
1915 }
1916 if self.nearest.is_some() && filter_plan.vector_filter().is_some() {
1917 return Err(Error::InvalidInput {
1918 source: "Query filter can't be used with vector search".into(),
1919 location: location!(),
1920 });
1921 }
1922 if self.full_text_query.is_some() && filter_plan.fts_filter().is_some() {
1923 return Err(Error::InvalidInput {
1924 source: "Fts filter can't be used with fts search".into(),
1925 location: location!(),
1926 });
1927 }
1928
1929 Ok(filter_plan)
1930 }
1931
1932 async fn get_scan_range(&self, filter_plan: &ExprFilterPlan) -> Result<Option<Range<u64>>> {
1933 if filter_plan.has_any_filter() {
1934 Ok(None)
1936 } else if self.ordering.is_some() {
1937 Ok(None)
1940 } else {
1941 match (self.limit, self.offset) {
1942 (None, None) => Ok(None),
1943 (Some(limit), None) => {
1944 let num_rows = self.dataset.count_all_rows().await? as i64;
1945 Ok(Some(0..limit.min(num_rows) as u64))
1946 }
1947 (None, Some(offset)) => {
1948 let num_rows = self.dataset.count_all_rows().await? as i64;
1949 Ok(Some(offset.min(num_rows) as u64..num_rows as u64))
1950 }
1951 (Some(limit), Some(offset)) => {
1952 let num_rows = self.dataset.count_all_rows().await? as i64;
1953 Ok(Some(
1954 offset.min(num_rows) as u64..(offset + limit).min(num_rows) as u64,
1955 ))
1956 }
1957 }
1958 }
1959 }
1960
1961 #[instrument(level = "debug", skip_all)]
2007 pub async fn create_plan(&self) -> Result<Arc<dyn ExecutionPlan>> {
2008 log::trace!("creating scanner plan");
2009 self.validate_options()?;
2010
2011 let use_scalar_index = self.use_scalar_index && (self.prefilter || self.nearest.is_none());
2013 let mut filter_plan = self.create_filter_plan(use_scalar_index).await?;
2014
2015 let mut use_limit_node = true;
2016 let mut plan: Arc<dyn ExecutionPlan> = match (&self.nearest, &self.full_text_query) {
2018 (Some(_), None) => self.vector_search_source(&mut filter_plan).await?,
2019 (None, Some(query)) => self.fts_search_source(&mut filter_plan, query).await?,
2020 (None, None) => {
2021 if self.projection_plan.has_output_cols()
2022 && self.projection_plan.physical_projection.is_empty()
2023 {
2024 let output_expr = self.calculate_final_projection(&ArrowSchema::empty())?;
2035 return Err(Error::NotSupported {
2036 source: format!("Scans must request at least one column. Received only dynamic expressions: {:?}", output_expr).into(),
2037 location: location!(),
2038 });
2039 }
2040
2041 let take_op = filter_plan
2042 .expr_filter_plan
2043 .full_expr
2044 .as_ref()
2045 .and_then(TakeOperation::try_from_expr);
2046 if let Some((take_op, remainder)) = take_op {
2047 filter_plan.expr_filter_plan = remainder
2050 .map(ExprFilterPlan::new_refine_only)
2051 .unwrap_or(ExprFilterPlan::default());
2052 self.take_source(take_op).await?
2053 } else {
2054 let planned_read = self
2055 .filtered_read_source(&mut filter_plan.expr_filter_plan)
2056 .await?;
2057 if planned_read.limit_pushed_down {
2058 use_limit_node = false;
2059 }
2060 if planned_read.filter_pushed_down {
2061 filter_plan.disable_refine();
2062 }
2063 planned_read.plan
2064 }
2065 }
2066 _ => {
2067 return Err(Error::InvalidInput {
2068 source: "Cannot have both nearest and full text search".into(),
2069 location: location!(),
2070 })
2071 }
2072 };
2073
2074 let mut pre_filter_projection = self.dataset.empty_projection();
2077
2078 if filter_plan.has_refine() {
2081 pre_filter_projection = pre_filter_projection.union_columns(
2083 filter_plan.refine_columns(&self.dataset).await?,
2084 OnMissing::Ignore,
2085 )?;
2086 }
2087
2088 if let Some(ordering) = &self.ordering {
2093 pre_filter_projection = pre_filter_projection.union_columns(
2094 ordering.iter().map(|col| &col.column_name),
2095 OnMissing::Error,
2096 )?;
2097 }
2098
2099 plan = self.take(plan, pre_filter_projection)?;
2100
2101 plan = filter_plan.refine_filter(plan, self).await?;
2103
2104 if let Some(ordering) = &self.ordering {
2106 let ordering_columns = ordering.iter().map(|col| &col.column_name);
2107 let projection_with_ordering = self
2108 .dataset
2109 .empty_projection()
2110 .union_columns(ordering_columns, OnMissing::Error)?;
2111 plan = self.take(plan, projection_with_ordering)?;
2113 let col_exprs = ordering
2114 .iter()
2115 .map(|col| {
2116 Ok(PhysicalSortExpr {
2117 expr: Self::create_column_expr(
2118 &col.column_name,
2119 &self.dataset,
2120 plan.schema().as_ref(),
2121 )?,
2122 options: SortOptions {
2123 descending: !col.ascending,
2124 nulls_first: col.nulls_first,
2125 },
2126 })
2127 })
2128 .collect::<Result<Vec<_>>>()?;
2129 plan = Arc::new(SortExec::new(
2130 LexOrdering::new(col_exprs)
2131 .ok_or(exec_datafusion_err!("Unexpected empty sort expressions"))?,
2132 plan,
2133 ));
2134 }
2135
2136 if use_limit_node && (self.limit.unwrap_or(0) > 0 || self.offset.is_some()) {
2138 plan = self.limit_node(plan);
2139 }
2140
2141 plan = self.take(plan, self.projection_plan.physical_projection.clone())?;
2143
2144 if self.projection_plan.must_add_row_offset {
2146 plan = Arc::new(AddRowOffsetExec::try_new(plan, self.dataset.clone()).await?);
2147 }
2148
2149 let final_projection = self.calculate_final_projection(plan.schema().as_ref())?;
2151
2152 plan = Arc::new(DFProjectionExec::try_new(final_projection, plan)?);
2153
2154 if self.strict_batch_size {
2156 plan = Arc::new(StrictBatchSizeExec::new(plan, self.get_batch_size()));
2157 }
2158
2159 let optimizer = get_physical_optimizer();
2160 let options = Default::default();
2161 for rule in optimizer.rules {
2162 plan = rule.optimize(plan, &options)?;
2163 }
2164
2165 Ok(plan)
2166 }
2167
2168 fn filter_references_version_columns(&self, filter_plan: &ExprFilterPlan) -> bool {
2170 use lance_core::{ROW_CREATED_AT_VERSION, ROW_LAST_UPDATED_AT_VERSION};
2171
2172 if let Some(refine_expr) = &filter_plan.refine_expr {
2173 let column_names = Planner::column_names_in_expr(refine_expr);
2174 for col_name in column_names {
2175 if col_name == ROW_CREATED_AT_VERSION || col_name == ROW_LAST_UPDATED_AT_VERSION {
2176 return true;
2177 }
2178 }
2179 }
2180 false
2181 }
2182
2183 async fn legacy_filtered_read(
2189 &self,
2190 filter_plan: &ExprFilterPlan,
2191 projection: Projection,
2192 make_deletions_null: bool,
2193 fragments: Option<Arc<Vec<Fragment>>>,
2194 scan_range: Option<Range<u64>>,
2195 is_prefilter: bool,
2196 ) -> Result<PlannedFilteredScan> {
2197 let fragments = fragments.unwrap_or(self.dataset.fragments().clone());
2198 let mut filter_pushed_down = false;
2199
2200 let plan: Arc<dyn ExecutionPlan> = if filter_plan.has_index_query() {
2201 if self.include_deleted_rows {
2202 return Err(Error::InvalidInput {
2203 source: "Cannot include deleted rows in a scalar indexed scan".into(),
2204 location: location!(),
2205 });
2206 }
2207 self.scalar_indexed_scan(projection, filter_plan, fragments)
2208 .await
2209 } else if !is_prefilter
2210 && filter_plan.has_refine()
2211 && self.batch_size.is_none()
2212 && self.use_stats
2213 && !self.filter_references_version_columns(filter_plan)
2214 {
2215 filter_pushed_down = true;
2216 self.pushdown_scan(false, filter_plan)
2217 } else {
2218 let ordered = if self.ordering.is_some() || self.nearest.is_some() {
2219 false
2221 } else if projection.with_row_last_updated_at_version
2222 || projection.with_row_created_at_version
2223 {
2224 true
2227 } else {
2228 self.ordered
2229 };
2230
2231 let projection = if let Some(refine_expr) = filter_plan.refine_expr.as_ref() {
2232 if is_prefilter {
2233 let refine_cols = Planner::column_names_in_expr(refine_expr);
2234 projection.union_columns(refine_cols, OnMissing::Error)?
2235 } else {
2236 projection
2237 }
2238 } else {
2239 projection
2240 };
2241
2242 let scan_range = if filter_plan.has_refine() {
2244 None
2245 } else {
2246 scan_range
2247 };
2248
2249 let scan = self.scan_fragments(
2250 projection.with_row_id,
2251 self.projection_plan.physical_projection.with_row_addr,
2252 self.projection_plan
2253 .physical_projection
2254 .with_row_last_updated_at_version,
2255 self.projection_plan
2256 .physical_projection
2257 .with_row_created_at_version,
2258 make_deletions_null,
2259 Arc::new(projection.to_bare_schema()),
2260 fragments,
2261 scan_range,
2262 ordered,
2263 );
2264
2265 if filter_plan.has_refine() && is_prefilter {
2266 Ok(Arc::new(LanceFilterExec::try_new(
2267 filter_plan.refine_expr.clone().unwrap(),
2268 scan,
2269 )?) as Arc<dyn ExecutionPlan>)
2270 } else {
2271 Ok(scan)
2272 }
2273 }?;
2274 Ok(PlannedFilteredScan {
2275 plan,
2276 limit_pushed_down: false,
2277 filter_pushed_down,
2278 })
2279 }
2280
2281 async fn new_filtered_read(
2285 &self,
2286 filter_plan: &ExprFilterPlan,
2287 projection: Projection,
2288 make_deletions_null: bool,
2289 fragments: Option<Arc<Vec<Fragment>>>,
2290 scan_range: Option<Range<u64>>,
2291 ) -> Result<Arc<dyn ExecutionPlan>> {
2292 let mut read_options = FilteredReadOptions::basic_full_read(&self.dataset)
2293 .with_filter_plan(filter_plan.clone())
2294 .with_projection(projection);
2295
2296 if let Some(fragments) = fragments {
2297 read_options = read_options.with_fragments(fragments);
2298 }
2299
2300 if let Some(scan_range) = scan_range {
2301 read_options = read_options.with_scan_range_before_filter(scan_range)?;
2302 }
2303
2304 if let Some(batch_size) = self.batch_size {
2305 read_options = read_options.with_batch_size(batch_size as u32);
2306 }
2307
2308 if let Some(fragment_readahead) = self.fragment_readahead {
2309 read_options = read_options.with_fragment_readahead(fragment_readahead);
2310 }
2311
2312 if make_deletions_null {
2313 read_options = read_options.with_deleted_rows()?;
2314 }
2315
2316 if let Some(io_buffer_size_bytes) = self.io_buffer_size {
2317 read_options = read_options.with_io_buffer_size(io_buffer_size_bytes);
2318 }
2319
2320 let index_input = filter_plan.index_query.clone().map(|index_query| {
2321 Arc::new(ScalarIndexExec::new(self.dataset.clone(), index_query))
2322 as Arc<dyn ExecutionPlan>
2323 });
2324
2325 Ok(Arc::new(FilteredReadExec::try_new(
2326 self.dataset.clone(),
2327 read_options,
2328 index_input,
2329 )?))
2330 }
2331
2332 async fn filtered_read(
2336 &self,
2337 filter_plan: &ExprFilterPlan,
2338 projection: Projection,
2339 make_deletions_null: bool,
2340 fragments: Option<Arc<Vec<Fragment>>>,
2341 scan_range: Option<Range<u64>>,
2342 is_prefilter: bool,
2343 ) -> Result<PlannedFilteredScan> {
2344 if self.dataset.is_legacy_storage() {
2346 self.legacy_filtered_read(
2347 filter_plan,
2348 projection,
2349 make_deletions_null,
2350 fragments,
2351 scan_range,
2352 is_prefilter,
2353 )
2354 .await
2355 } else {
2356 let limit_pushed_down = scan_range.is_some();
2357 let plan = self
2358 .new_filtered_read(
2359 filter_plan,
2360 projection,
2361 make_deletions_null,
2362 fragments,
2363 scan_range,
2364 )
2365 .await?;
2366 Ok(PlannedFilteredScan {
2367 filter_pushed_down: true,
2368 limit_pushed_down,
2369 plan,
2370 })
2371 }
2372 }
2373
2374 fn u64s_as_take_input(&self, u64s: Vec<u64>) -> Result<Arc<dyn ExecutionPlan>> {
2375 let row_addrs = RowAddrTreeMap::from_iter(u64s);
2376 let row_addr_mask = RowAddrMask::from_allowed(row_addrs);
2377 let index_result = IndexExprResult::Exact(row_addr_mask);
2378 let fragments_covered =
2379 RoaringBitmap::from_iter(self.dataset.fragments().iter().map(|f| f.id as u32));
2380 let batch = index_result.serialize_to_arrow(&fragments_covered)?;
2381 let stream = futures::stream::once(async move { Ok(batch) });
2382 let stream = Box::pin(RecordBatchStreamAdapter::new(
2383 INDEX_EXPR_RESULT_SCHEMA.clone(),
2384 stream,
2385 ));
2386 Ok(Arc::new(OneShotExec::new(stream)))
2387 }
2388
2389 async fn take_source(&self, take_op: TakeOperation) -> Result<Arc<dyn ExecutionPlan>> {
2390 let projection = self.projection_plan.physical_projection.clone();
2393
2394 let input = match take_op {
2395 TakeOperation::RowIds(ids) => self.u64s_as_take_input(ids),
2396 TakeOperation::RowAddrs(addrs) => self.u64s_as_take_input(addrs),
2397 TakeOperation::RowOffsets(offsets) => {
2398 let mut addrs =
2399 row_offsets_to_row_addresses(self.dataset.as_ref(), &offsets).await?;
2400 addrs.retain(|addr| *addr != RowAddress::TOMBSTONE_ROW);
2401 self.u64s_as_take_input(addrs)
2402 }
2403 }?;
2404
2405 Ok(Arc::new(FilteredReadExec::try_new(
2406 self.dataset.clone(),
2407 FilteredReadOptions::new(projection),
2408 Some(input),
2409 )?))
2410 }
2411
2412 async fn filtered_read_source(
2413 &self,
2414 filter_plan: &mut ExprFilterPlan,
2415 ) -> Result<PlannedFilteredScan> {
2416 log::trace!("source is a filtered read");
2417 let mut projection = if filter_plan.has_refine() {
2418 self.calc_eager_projection(filter_plan, &self.projection_plan.physical_projection)?
2422 .with_row_id()
2423 } else {
2424 self.projection_plan.physical_projection.clone()
2427 };
2428
2429 if projection.is_empty() {
2430 projection.with_row_addr = true;
2433 }
2434
2435 let scan_range = if filter_plan.is_empty() {
2436 log::trace!("pushing scan_range into filtered_read");
2437 self.get_scan_range(filter_plan).await?
2438 } else {
2439 None
2440 };
2441
2442 self.filtered_read(
2443 filter_plan,
2444 projection,
2445 self.include_deleted_rows,
2446 self.fragments.clone().map(Arc::new),
2447 scan_range,
2448 false,
2449 )
2450 .await
2451 }
2452
2453 async fn fts_search_source(
2454 &self,
2455 filter_plan: &mut FilterPlan,
2456 query: &FullTextSearchQuery,
2457 ) -> Result<Arc<dyn ExecutionPlan>> {
2458 log::trace!("source is an fts search");
2459 if self.include_deleted_rows {
2460 return Err(Error::InvalidInput {
2461 source: "Cannot include deleted rows in an FTS search".into(),
2462 location: location!(),
2463 });
2464 }
2465
2466 if self.prefilter {
2468 let source: Arc<dyn ExecutionPlan> = match &filter_plan.vector_filter() {
2469 Some(vector_query) => {
2470 let vector_plan = self
2471 .vector_search(&filter_plan.expr_filter_plan, vector_query)
2472 .await?;
2473 self.flat_fts(vector_plan, query).await?
2474 }
2475 None => self.fts(&filter_plan.expr_filter_plan, query).await?,
2476 };
2477 filter_plan.disable_refine();
2479 Ok(source)
2480 } else {
2481 filter_plan.make_refine_only();
2484 self.fts(&ExprFilterPlan::default(), query).await
2485 }
2486 }
2487
2488 async fn vector_search_source(
2489 &self,
2490 filter_plan: &mut FilterPlan,
2491 ) -> Result<Arc<dyn ExecutionPlan>> {
2492 if self.include_deleted_rows {
2493 return Err(Error::InvalidInput {
2494 source: "Cannot include deleted rows in a nearest neighbor search".into(),
2495 location: location!(),
2496 });
2497 }
2498 let Some(query) = self.nearest.as_ref() else {
2499 return Err(Error::invalid_input(
2500 "No nearest query".to_string(),
2501 location!(),
2502 ));
2503 };
2504
2505 if self.prefilter {
2506 log::trace!("source is a vector search (prefilter)");
2507 let source: Arc<dyn ExecutionPlan> = match &filter_plan.fts_filter() {
2509 Some(fts_query) => {
2510 let fts_plan = self.fts(&filter_plan.expr_filter_plan, fts_query).await?;
2511 let projection = self
2512 .dataset
2513 .empty_projection()
2514 .union_column(&query.column, OnMissing::Error)?;
2515 let plan = self.take(fts_plan, projection)?;
2516
2517 self.flat_knn(plan, query)?
2518 }
2519 None => {
2520 self.vector_search(&filter_plan.expr_filter_plan, query)
2521 .await?
2522 }
2523 };
2524
2525 filter_plan.disable_refine();
2526 Ok(source)
2527 } else {
2528 log::trace!("source is a vector search (postfilter)");
2529 filter_plan.make_refine_only();
2532 self.vector_search(&ExprFilterPlan::default(), query).await
2533 }
2534 }
2535
2536 async fn fragments_covered_by_fts_leaf(
2537 &self,
2538 column: &str,
2539 accum: &mut RoaringBitmap,
2540 ) -> Result<bool> {
2541 let index = self
2542 .dataset
2543 .load_scalar_index(IndexCriteria::default().for_column(column).supports_fts())
2544 .await?;
2545 match index {
2546 Some(index) => match &index.fragment_bitmap {
2547 Some(fragmap) => {
2548 *accum |= fragmap;
2549 Ok(true)
2550 }
2551 None => Ok(false),
2552 },
2553 None => Ok(false),
2554 }
2555 }
2556
2557 #[async_recursion]
2558 async fn fragments_covered_by_fts_query_helper(
2559 &self,
2560 query: &FtsQuery,
2561 accum: &mut RoaringBitmap,
2562 ) -> Result<bool> {
2563 match query {
2564 FtsQuery::Match(match_query) => {
2565 self.fragments_covered_by_fts_leaf(
2566 match_query.column.as_ref().ok_or(Error::invalid_input(
2567 "the column must be specified in the query".to_string(),
2568 location!(),
2569 ))?,
2570 accum,
2571 )
2572 .await
2573 }
2574 FtsQuery::Boost(boost) => Ok(self
2575 .fragments_covered_by_fts_query_helper(&boost.negative, accum)
2576 .await?
2577 & self
2578 .fragments_covered_by_fts_query_helper(&boost.positive, accum)
2579 .await?),
2580 FtsQuery::MultiMatch(multi_match) => {
2581 for mq in &multi_match.match_queries {
2582 if !self
2583 .fragments_covered_by_fts_leaf(
2584 mq.column.as_ref().ok_or(Error::invalid_input(
2585 "the column must be specified in the query".to_string(),
2586 location!(),
2587 ))?,
2588 accum,
2589 )
2590 .await?
2591 {
2592 return Ok(false);
2593 }
2594 }
2595 Ok(true)
2596 }
2597 FtsQuery::Phrase(phrase_query) => {
2598 self.fragments_covered_by_fts_leaf(
2599 phrase_query.column.as_ref().ok_or(Error::invalid_input(
2600 "the column must be specified in the query".to_string(),
2601 location!(),
2602 ))?,
2603 accum,
2604 )
2605 .await
2606 }
2607 FtsQuery::Boolean(bool_query) => {
2608 for query in bool_query.must.iter() {
2609 if !self
2610 .fragments_covered_by_fts_query_helper(query, accum)
2611 .await?
2612 {
2613 return Ok(false);
2614 }
2615 }
2616 for query in &bool_query.should {
2617 if !self
2618 .fragments_covered_by_fts_query_helper(query, accum)
2619 .await?
2620 {
2621 return Ok(false);
2622 }
2623 }
2624 Ok(true)
2625 }
2626 }
2627 }
2628
2629 async fn fragments_covered_by_fts_query(&self, query: &FtsQuery) -> Result<RoaringBitmap> {
2630 let all_fragments = self.get_fragments_as_bitmap();
2631
2632 let mut referenced_fragments = RoaringBitmap::new();
2633 if !self
2634 .fragments_covered_by_fts_query_helper(query, &mut referenced_fragments)
2635 .await?
2636 {
2637 Ok(all_fragments)
2639 } else {
2640 Ok(all_fragments & referenced_fragments)
2642 }
2643 }
2644
2645 async fn fts(
2647 &self,
2648 filter_plan: &ExprFilterPlan,
2649 query: &FullTextSearchQuery,
2650 ) -> Result<Arc<dyn ExecutionPlan>> {
2651 let columns = query.columns();
2652 let mut params = query.params();
2653 if params.limit.is_none() {
2654 let search_limit = match (self.limit, self.offset) {
2655 (Some(limit), Some(offset)) => Some((limit + offset) as usize),
2656 (Some(limit), None) => Some(limit as usize),
2657 (None, Some(_)) => None, (None, None) => None,
2659 };
2660 params = params.with_limit(search_limit);
2661 }
2662 let query = if columns.is_empty() {
2663 let indexed_columns = fts_indexed_columns(self.dataset.clone()).await?;
2666 fill_fts_query_column(&query.query, &indexed_columns, false)?
2667 } else {
2668 query.query.clone()
2669 };
2670
2671 let prefilter_source = self
2675 .prefilter_source(
2676 filter_plan,
2677 self.fragments_covered_by_fts_query(&query).await?,
2678 )
2679 .await?;
2680 let fts_exec = self
2681 .plan_fts(&query, ¶ms, filter_plan, &prefilter_source)
2682 .await?;
2683 Ok(fts_exec)
2684 }
2685
2686 async fn plan_fts(
2687 &self,
2688 query: &FtsQuery,
2689 params: &FtsSearchParams,
2690 filter_plan: &ExprFilterPlan,
2691 prefilter_source: &PreFilterSource,
2692 ) -> Result<Arc<dyn ExecutionPlan>> {
2693 let plan: Arc<dyn ExecutionPlan> = match query {
2694 FtsQuery::Match(query) => {
2695 self.plan_match_query(query, params, filter_plan, prefilter_source)
2696 .await?
2697 }
2698 FtsQuery::Phrase(query) => {
2699 self.plan_phrase_query(query, params, prefilter_source)
2700 .await?
2701 }
2702
2703 FtsQuery::Boost(query) => {
2704 let unlimited_params = params.clone().with_limit(None);
2708 let positive_exec = Box::pin(self.plan_fts(
2709 &query.positive,
2710 &unlimited_params,
2711 filter_plan,
2712 prefilter_source,
2713 ));
2714 let negative_exec = Box::pin(self.plan_fts(
2715 &query.negative,
2716 &unlimited_params,
2717 filter_plan,
2718 prefilter_source,
2719 ));
2720 let (positive_exec, negative_exec) =
2721 futures::future::try_join(positive_exec, negative_exec).await?;
2722 Arc::new(BoostQueryExec::new(
2723 query.clone(),
2724 params.clone(),
2725 positive_exec,
2726 negative_exec,
2727 ))
2728 }
2729
2730 FtsQuery::MultiMatch(query) => {
2731 let mut children = Vec::with_capacity(query.match_queries.len());
2732 for match_query in &query.match_queries {
2733 let child =
2734 self.plan_match_query(match_query, params, filter_plan, prefilter_source);
2735 children.push(child);
2736 }
2737 let children = futures::future::try_join_all(children).await?;
2738
2739 let schema = children[0].schema();
2740 let group_expr = vec![(
2741 expressions::col(ROW_ID, schema.as_ref())?,
2742 ROW_ID.to_string(),
2743 )];
2744
2745 let fts_node = UnionExec::try_new(children)?;
2746 let fts_node = Arc::new(RepartitionExec::try_new(
2747 fts_node,
2748 Partitioning::RoundRobinBatch(1),
2749 )?);
2750 let fts_node = Arc::new(AggregateExec::try_new(
2752 AggregateMode::Single,
2753 PhysicalGroupBy::new_single(group_expr),
2754 vec![Arc::new(
2755 AggregateExprBuilder::new(
2756 functions_aggregate::min_max::max_udaf(),
2757 vec![expressions::col(SCORE_COL, &schema)?],
2758 )
2759 .schema(schema.clone())
2760 .alias(SCORE_COL)
2761 .build()?,
2762 )],
2763 vec![None],
2764 fts_node,
2765 schema,
2766 )?);
2767 let sort_expr = PhysicalSortExpr {
2768 expr: expressions::col(SCORE_COL, fts_node.schema().as_ref())?,
2769 options: SortOptions {
2770 descending: true,
2771 nulls_first: false,
2772 },
2773 };
2774
2775 Arc::new(
2776 SortExec::new([sort_expr].into(), fts_node)
2777 .with_fetch(self.limit.map(|l| l as usize)),
2778 )
2779 }
2780 FtsQuery::Boolean(query) => {
2781 let unlimited_params = params.clone().with_limit(None);
2786
2787 let mut should = Vec::with_capacity(query.should.len());
2789 for subquery in &query.should {
2790 let plan = Box::pin(self.plan_fts(
2791 subquery,
2792 &unlimited_params,
2793 filter_plan,
2794 prefilter_source,
2795 ))
2796 .await?;
2797 should.push(plan);
2798 }
2799 let should = if should.is_empty() {
2800 Arc::new(EmptyExec::new(FTS_SCHEMA.clone()))
2801 } else if should.len() == 1 {
2802 should.pop().unwrap()
2803 } else {
2804 let unioned = UnionExec::try_new(should)?;
2805 Arc::new(RepartitionExec::try_new(
2806 unioned,
2807 Partitioning::RoundRobinBatch(1),
2808 )?)
2809 };
2810
2811 let mut must = None;
2813 for query in &query.must {
2814 let plan = Box::pin(self.plan_fts(
2815 query,
2816 &unlimited_params,
2817 filter_plan,
2818 prefilter_source,
2819 ))
2820 .await?;
2821 if let Some(joined_plan) = must {
2822 must = Some(Arc::new(HashJoinExec::try_new(
2823 joined_plan,
2824 plan,
2825 vec![(
2826 Arc::new(Column::new_with_schema(ROW_ID, &FTS_SCHEMA)?),
2827 Arc::new(Column::new_with_schema(ROW_ID, &FTS_SCHEMA)?),
2828 )],
2829 None,
2830 &datafusion_expr::JoinType::Inner,
2831 None,
2832 datafusion_physical_plan::joins::PartitionMode::CollectLeft,
2833 NullEquality::NullEqualsNothing,
2834 )?) as _);
2835 } else {
2836 must = Some(plan);
2837 }
2838 }
2839
2840 let mut must_not = Vec::with_capacity(query.must_not.len());
2842 for query in &query.must_not {
2843 let plan = Box::pin(self.plan_fts(
2844 query,
2845 &unlimited_params,
2846 filter_plan,
2847 prefilter_source,
2848 ))
2849 .await?;
2850 must_not.push(plan);
2851 }
2852 let must_not = if must_not.is_empty() {
2853 Arc::new(EmptyExec::new(FTS_SCHEMA.clone()))
2854 } else if must_not.len() == 1 {
2855 must_not.pop().unwrap()
2856 } else {
2857 let unioned = UnionExec::try_new(must_not)?;
2858 Arc::new(RepartitionExec::try_new(
2859 unioned,
2860 Partitioning::RoundRobinBatch(1),
2861 )?)
2862 };
2863
2864 if query.should.is_empty() && must.is_none() {
2865 return Err(Error::invalid_input(
2866 "boolean query must have at least one should/must query".to_string(),
2867 location!(),
2868 ));
2869 }
2870
2871 Arc::new(BooleanQueryExec::new(
2872 query.clone(),
2873 params.clone(),
2874 should,
2875 must,
2876 must_not,
2877 ))
2878 }
2879 };
2880
2881 Ok(plan)
2882 }
2883
2884 async fn plan_phrase_query(
2885 &self,
2886 query: &PhraseQuery,
2887 params: &FtsSearchParams,
2888 prefilter_source: &PreFilterSource,
2889 ) -> Result<Arc<dyn ExecutionPlan>> {
2890 let column = query.column.clone().ok_or(Error::invalid_input(
2891 "the column must be specified in the query".to_string(),
2892 location!(),
2893 ))?;
2894
2895 let index_meta = self
2896 .dataset
2897 .load_scalar_index(IndexCriteria::default().for_column(&column).supports_fts())
2898 .await?
2899 .ok_or(Error::invalid_input(
2900 format!("No Inverted index found for column {}", column),
2901 location!(),
2902 ))?;
2903
2904 let details_any =
2905 crate::index::scalar::fetch_index_details(&self.dataset, &column, &index_meta).await?;
2906 let details = details_any
2907 .as_ref()
2908 .to_msg::<lance_index::pbold::InvertedIndexDetails>()?;
2909 if !details.with_position {
2910 return Err(Error::invalid_input(
2911 "position is not found but required for phrase queries, try recreating the index with position"
2912 .to_string(),
2913 location!(),
2914 ));
2915 }
2916
2917 Ok(Arc::new(PhraseQueryExec::new(
2918 self.dataset.clone(),
2919 query.clone(),
2920 params.clone(),
2921 prefilter_source.clone(),
2922 )))
2923 }
2924
2925 async fn plan_match_query(
2926 &self,
2927 query: &MatchQuery,
2928 params: &FtsSearchParams,
2929 filter_plan: &ExprFilterPlan,
2930 prefilter_source: &PreFilterSource,
2931 ) -> Result<Arc<dyn ExecutionPlan>> {
2932 let column = query
2933 .column
2934 .as_ref()
2935 .ok_or(Error::invalid_input(
2936 "the column must be specified in the query".to_string(),
2937 location!(),
2938 ))?
2939 .clone();
2940
2941 let index = self
2942 .dataset
2943 .load_scalar_index(IndexCriteria::default().for_column(&column).supports_fts())
2944 .await?;
2945
2946 let (match_plan, flat_match_plan) = match &index {
2947 Some(index) => {
2948 let match_plan: Arc<dyn ExecutionPlan> = Arc::new(MatchQueryExec::new(
2949 self.dataset.clone(),
2950 query.clone(),
2951 params.clone(),
2952 prefilter_source.clone(),
2953 ));
2954
2955 let unindexed_fragments = self.dataset.unindexed_fragments(&index.name).await?;
2956 if unindexed_fragments.is_empty() {
2957 (Some(match_plan), None)
2958 } else {
2959 let flat_match_plan = self
2960 .plan_flat_match_query(unindexed_fragments, query, params, filter_plan)
2961 .await?;
2962 (Some(match_plan), Some(flat_match_plan))
2963 }
2964 }
2965 None => {
2966 let unindexed_fragments = self.dataset.fragments().iter().cloned().collect();
2967 let flat_match_plan = self
2968 .plan_flat_match_query(unindexed_fragments, query, params, filter_plan)
2969 .await?;
2970 (None, Some(flat_match_plan))
2971 }
2972 };
2973
2974 let plan = match (match_plan, flat_match_plan) {
2976 (Some(match_plan), Some(flat_match_plan)) => {
2977 let match_plan = UnionExec::try_new(vec![match_plan, flat_match_plan])?;
2978 let match_plan = Arc::new(RepartitionExec::try_new(
2979 match_plan,
2980 Partitioning::RoundRobinBatch(1),
2981 )?);
2982 let sort_expr = PhysicalSortExpr {
2983 expr: expressions::col(SCORE_COL, match_plan.schema().as_ref())?,
2984 options: SortOptions {
2985 descending: true,
2986 nulls_first: false,
2987 },
2988 };
2989 Arc::new(SortExec::new([sort_expr].into(), match_plan).with_fetch(params.limit))
2990 }
2991 (Some(match_plan), None) => match_plan,
2992 (None, Some(flat_match_plan)) => flat_match_plan,
2993 (None, None) => unreachable!(),
2994 };
2995
2996 Ok(plan)
2997 }
2998
2999 async fn plan_flat_match_query(
3001 &self,
3002 fragments: Vec<Fragment>,
3003 query: &MatchQuery,
3004 params: &FtsSearchParams,
3005 filter_plan: &ExprFilterPlan,
3006 ) -> Result<Arc<dyn ExecutionPlan>> {
3007 let column = query
3008 .column
3009 .as_ref()
3010 .ok_or(Error::invalid_input(
3011 "the column must be specified in the query".to_string(),
3012 location!(),
3013 ))?
3014 .clone();
3015
3016 let mut columns = vec![column];
3017 if let Some(expr) = filter_plan.full_expr.as_ref() {
3018 let filter_columns = Planner::column_names_in_expr(expr);
3019 columns.extend(filter_columns);
3020 }
3021 let flat_fts_scan_schema = Arc::new(self.dataset.schema().project(&columns).unwrap());
3022 let mut scan_node = self.scan_fragments(
3023 true,
3024 false,
3025 false,
3026 false,
3027 false,
3028 flat_fts_scan_schema,
3029 Arc::new(fragments),
3030 None,
3031 false,
3032 );
3033
3034 if let Some(expr) = filter_plan.full_expr.as_ref() {
3035 scan_node = Arc::new(LanceFilterExec::try_new(expr.clone(), scan_node)?);
3037 }
3038
3039 let flat_match_plan = Arc::new(FlatMatchQueryExec::new(
3040 self.dataset.clone(),
3041 query.clone(),
3042 params.clone(),
3043 scan_node,
3044 FTS_SCHEMA.clone(),
3045 ));
3046 Ok(flat_match_plan)
3047 }
3048
3049 async fn vector_search(
3051 &self,
3052 filter_plan: &ExprFilterPlan,
3053 q: &Query,
3054 ) -> Result<Arc<dyn ExecutionPlan>> {
3055 let mut q = q.clone();
3056
3057 let (vector_type, element_type) = get_vector_type(self.dataset.schema(), &q.column)?;
3059
3060 let column_id = self.dataset.schema().field_id(q.column.as_str())?;
3061 let use_index = q.use_index;
3062 let indices = if use_index {
3063 self.dataset.load_indices().await?
3064 } else {
3065 Arc::new(vec![])
3066 };
3067 let matching_index = if let Some(index) =
3069 indices.iter().find(|i| i.fields.contains(&column_id))
3070 {
3071 let idx = self
3075 .dataset
3076 .open_vector_index(
3077 q.column.as_str(),
3078 &index.uuid.to_string(),
3079 &NoOpMetricsCollector,
3080 )
3081 .await?;
3082 let index_metric = idx.metric_type();
3083
3084 let use_this_index = match q.metric_type {
3086 Some(user_metric) => {
3087 if user_metric == index_metric {
3088 true
3089 } else {
3090 log::warn!(
3091 "Requested metric {:?} is incompatible with index metric {:?}, falling back to brute-force search",
3092 user_metric,
3093 index_metric
3094 );
3095 false
3096 }
3097 }
3098 None => true, };
3100
3101 if use_this_index {
3102 Some((index, idx, index_metric))
3103 } else {
3104 None
3105 }
3106 } else {
3107 None
3108 };
3109
3110 if let Some((index, _idx, index_metric)) = matching_index {
3111 log::trace!("index found for vector search");
3112 q.metric_type = Some(index_metric);
3114 validate_distance_type_for(index_metric, &element_type)?;
3115
3116 if matches!(q.refine_factor, Some(0)) {
3117 return Err(Error::invalid_input(
3118 "Refine factor cannot be zero".to_string(),
3119 location!(),
3120 ));
3121 }
3122
3123 let deltas = self.dataset.load_indices_by_name(&index.name).await?;
3125 let ann_node = match vector_type {
3126 DataType::FixedSizeList(_, _) => self.ann(&q, &deltas, filter_plan).await?,
3127 DataType::List(_) => self.multivec_ann(&q, &deltas, filter_plan).await?,
3128 _ => unreachable!(),
3129 };
3130
3131 let mut knn_node = if q.refine_factor.is_some() {
3132 let vector_projection = self
3133 .dataset
3134 .empty_projection()
3135 .union_column(&q.column, OnMissing::Error)
3136 .unwrap();
3137 let knn_node_with_vector = self.take(ann_node, vector_projection)?;
3138 self.flat_knn(knn_node_with_vector, &q)?
3139 } else {
3140 ann_node
3141 }; if !self.fast_search {
3144 knn_node = self.knn_combined(&q, index, knn_node, filter_plan).await?;
3145 }
3146
3147 Ok(knn_node)
3148 } else {
3149 let metric = q
3151 .metric_type
3152 .unwrap_or_else(|| default_distance_type_for(&element_type));
3153 q.metric_type = Some(metric);
3154 validate_distance_type_for(metric, &element_type)?;
3155 let mut columns = vec![q.column.clone()];
3157 if let Some(refine_expr) = filter_plan.refine_expr.as_ref() {
3158 columns.extend(Planner::column_names_in_expr(refine_expr));
3159 }
3160 let mut vector_scan_projection = self
3161 .dataset
3162 .empty_projection()
3163 .with_row_id()
3164 .union_columns(&columns, OnMissing::Error)?;
3165
3166 vector_scan_projection.with_row_addr =
3167 self.projection_plan.physical_projection.with_row_addr;
3168
3169 let PlannedFilteredScan { mut plan, .. } = self
3170 .filtered_read(
3171 filter_plan,
3172 vector_scan_projection,
3173 true,
3174 None,
3175 None,
3176 true,
3177 )
3178 .await?;
3179
3180 if let Some(refine_expr) = &filter_plan.refine_expr {
3181 plan = Arc::new(LanceFilterExec::try_new(refine_expr.clone(), plan)?);
3182 }
3183 Ok(self.flat_knn(plan, &q)?)
3184 }
3185 }
3186
3187 async fn knn_combined(
3189 &self,
3190 q: &Query,
3191 index: &IndexMetadata,
3192 mut knn_node: Arc<dyn ExecutionPlan>,
3193 filter_plan: &ExprFilterPlan,
3194 ) -> Result<Arc<dyn ExecutionPlan>> {
3195 let unindexed_fragments = self.dataset.unindexed_fragments(&index.name).await?;
3197 if !unindexed_fragments.is_empty() {
3198 let idx = self
3201 .dataset
3202 .open_vector_index(
3203 q.column.as_str(),
3204 &index.uuid.to_string(),
3205 &NoOpMetricsCollector,
3206 )
3207 .await?;
3208 let mut q = q.clone();
3209 q.metric_type = Some(idx.metric_type());
3210
3211 if knn_node.schema().column_with_name(&q.column).is_none() {
3214 let vector_projection = self
3215 .dataset
3216 .empty_projection()
3217 .union_column(&q.column, OnMissing::Error)
3218 .unwrap();
3219 knn_node = self.take(knn_node, vector_projection)?;
3220 }
3221
3222 let mut columns = vec![q.column.clone()];
3223 if let Some(expr) = filter_plan.full_expr.as_ref() {
3224 let filter_columns = Planner::column_names_in_expr(expr);
3225 columns.extend(filter_columns);
3226 }
3227 let vector_scan_projection = Arc::new(self.dataset.schema().project(&columns).unwrap());
3228 let mut scan_node = self.scan_fragments(
3232 true,
3233 false,
3234 false,
3235 false,
3236 false,
3237 vector_scan_projection,
3238 Arc::new(unindexed_fragments),
3239 None,
3241 false,
3244 );
3245
3246 if let Some(expr) = filter_plan.full_expr.as_ref() {
3247 scan_node = Arc::new(LanceFilterExec::try_new(expr.clone(), scan_node)?);
3249 }
3250 let topk_appended = self.flat_knn(scan_node, &q)?;
3252
3253 let topk_appended = project(topk_appended, knn_node.schema().as_ref())?;
3257 assert!(topk_appended
3258 .schema()
3259 .equivalent_names_and_types(&knn_node.schema()));
3260 let unioned = UnionExec::try_new(vec![Arc::new(topk_appended), knn_node])?;
3262 let unioned = RepartitionExec::try_new(
3264 unioned,
3265 datafusion::physical_plan::Partitioning::RoundRobinBatch(1),
3266 )?;
3267 return self.flat_knn(Arc::new(unioned), &q);
3269 }
3270
3271 Ok(knn_node)
3272 }
3273
3274 #[async_recursion]
3275 async fn fragments_covered_by_index_query(
3276 &self,
3277 index_expr: &ScalarIndexExpr,
3278 ) -> Result<RoaringBitmap> {
3279 match index_expr {
3280 ScalarIndexExpr::And(lhs, rhs) => {
3281 Ok(self.fragments_covered_by_index_query(lhs).await?
3282 & self.fragments_covered_by_index_query(rhs).await?)
3283 }
3284 ScalarIndexExpr::Or(lhs, rhs) => Ok(self.fragments_covered_by_index_query(lhs).await?
3285 & self.fragments_covered_by_index_query(rhs).await?),
3286 ScalarIndexExpr::Not(expr) => self.fragments_covered_by_index_query(expr).await,
3287 ScalarIndexExpr::Query(search) => {
3288 let idx = self
3289 .dataset
3290 .load_scalar_index(IndexCriteria::default().with_name(&search.index_name))
3291 .await?
3292 .expect("Index not found even though it must have been found earlier");
3293 Ok(idx
3294 .fragment_bitmap
3295 .expect("scalar indices should always have a fragment bitmap"))
3296 }
3297 }
3298 }
3299
3300 async fn partition_frags_by_coverage(
3308 &self,
3309 index_expr: &ScalarIndexExpr,
3310 fragments: Arc<Vec<Fragment>>,
3311 ) -> Result<(Vec<Fragment>, Vec<Fragment>)> {
3312 let covered_frags = self.fragments_covered_by_index_query(index_expr).await?;
3313 let mut relevant_frags = Vec::with_capacity(fragments.len());
3314 let mut missing_frags = Vec::with_capacity(fragments.len());
3315 for fragment in fragments.iter() {
3316 if covered_frags.contains(fragment.id as u32) {
3317 relevant_frags.push(fragment.clone());
3318 } else {
3319 missing_frags.push(fragment.clone());
3320 }
3321 }
3322 Ok((relevant_frags, missing_frags))
3323 }
3324
3325 async fn scalar_indexed_scan(
3328 &self,
3329 projection: Projection,
3330 filter_plan: &ExprFilterPlan,
3331 fragments: Arc<Vec<Fragment>>,
3332 ) -> Result<Arc<dyn ExecutionPlan>> {
3333 log::trace!("scalar indexed scan");
3334 let index_expr = filter_plan.index_query.as_ref().unwrap();
3341
3342 let needs_recheck = index_expr.needs_recheck();
3343
3344 let (relevant_frags, missing_frags) = self
3346 .partition_frags_by_coverage(index_expr, fragments)
3347 .await?;
3348
3349 let mut plan: Arc<dyn ExecutionPlan> = Arc::new(MaterializeIndexExec::new(
3350 self.dataset.clone(),
3351 index_expr.clone(),
3352 Arc::new(relevant_frags),
3353 ));
3354
3355 let refine_expr = filter_plan.refine_expr.as_ref();
3356
3357 let needs_take =
3360 needs_recheck || projection.has_data_fields() || filter_plan.refine_expr.is_some();
3361 if needs_take {
3362 let mut take_projection = projection.clone();
3363 if needs_recheck {
3364 let filter_expr = index_expr.to_expr();
3366 let filter_cols = Planner::column_names_in_expr(&filter_expr);
3367 take_projection = take_projection.union_columns(filter_cols, OnMissing::Error)?;
3368 }
3369 if let Some(refine_expr) = refine_expr {
3370 let refine_cols = Planner::column_names_in_expr(refine_expr);
3371 take_projection = take_projection.union_columns(refine_cols, OnMissing::Error)?;
3372 }
3373 log::trace!("need to take additional columns for scalar_indexed_scan");
3374 plan = self.take(plan, take_projection)?;
3375 }
3376
3377 let post_take_filter = match (needs_recheck, refine_expr) {
3378 (false, None) => None,
3379 (true, None) => {
3380 Some(index_expr.to_expr())
3382 }
3383 (true, Some(_)) => Some(filter_plan.full_expr.as_ref().unwrap().clone()),
3384 (false, Some(refine_expr)) => Some(refine_expr.clone()),
3385 };
3386
3387 if let Some(post_take_filter) = post_take_filter {
3388 let planner = Planner::new(plan.schema());
3389 let optimized_filter = planner.optimize_expr(post_take_filter)?;
3390
3391 log::trace!("applying post-take filter to indexed scan");
3392 plan = Arc::new(LanceFilterExec::try_new(optimized_filter, plan)?);
3393 }
3394
3395 if self.projection_plan.physical_projection.with_row_addr {
3396 plan = Arc::new(AddRowAddrExec::try_new(plan, self.dataset.clone(), 0)?);
3397 }
3398
3399 let new_data_path: Option<Arc<dyn ExecutionPlan>> = if !missing_frags.is_empty() {
3400 log::trace!(
3401 "scalar_indexed_scan will need full scan of {} missing fragments",
3402 missing_frags.len()
3403 );
3404
3405 let filter = filter_plan.full_expr.as_ref().unwrap();
3418 let filter_cols = Planner::column_names_in_expr(filter);
3419 let scan_projection = projection.union_columns(filter_cols, OnMissing::Error)?;
3420
3421 let scan_schema = Arc::new(scan_projection.to_bare_schema());
3422 let scan_arrow_schema = Arc::new(scan_schema.as_ref().into());
3423 let planner = Planner::new(scan_arrow_schema);
3424 let optimized_filter = planner.optimize_expr(filter.clone())?;
3425
3426 let new_data_scan = self.scan_fragments(
3427 true,
3428 self.projection_plan.physical_projection.with_row_addr,
3429 self.projection_plan
3430 .physical_projection
3431 .with_row_last_updated_at_version,
3432 self.projection_plan
3433 .physical_projection
3434 .with_row_created_at_version,
3435 false,
3436 scan_schema,
3437 missing_frags.into(),
3438 None,
3440 false,
3441 );
3442 let filtered = Arc::new(LanceFilterExec::try_new(optimized_filter, new_data_scan)?);
3443 Some(Arc::new(project(filtered, plan.schema().as_ref())?))
3444 } else {
3445 log::trace!("scalar_indexed_scan will not need full scan of any missing fragments");
3446 None
3447 };
3448
3449 if let Some(new_data_path) = new_data_path {
3450 let unioned = UnionExec::try_new(vec![plan, new_data_path])?;
3451 let unioned = Arc::new(RepartitionExec::try_new(
3453 unioned,
3454 datafusion::physical_plan::Partitioning::RoundRobinBatch(1),
3455 )?);
3456 Ok(unioned)
3457 } else {
3458 Ok(plan)
3459 }
3460 }
3461
3462 fn get_io_buffer_size(&self) -> u64 {
3463 self.io_buffer_size.unwrap_or(*DEFAULT_IO_BUFFER_SIZE)
3464 }
3465
3466 #[allow(clippy::too_many_arguments)]
3471 pub(crate) fn scan(
3472 &self,
3473 with_row_id: bool,
3474 with_row_address: bool,
3475 with_row_last_updated_at_version: bool,
3476 with_row_created_at_version: bool,
3477 with_make_deletions_null: bool,
3478 range: Option<Range<u64>>,
3479 projection: Arc<Schema>,
3480 ) -> Arc<dyn ExecutionPlan> {
3481 let fragments = if let Some(fragment) = self.fragments.as_ref() {
3482 Arc::new(fragment.clone())
3483 } else {
3484 self.dataset.fragments().clone()
3485 };
3486 let ordered = if self.ordering.is_some() || self.nearest.is_some() {
3487 false
3489 } else {
3490 self.ordered
3491 };
3492 self.scan_fragments(
3493 with_row_id,
3494 with_row_address,
3495 with_row_last_updated_at_version,
3496 with_row_created_at_version,
3497 with_make_deletions_null,
3498 projection,
3499 fragments,
3500 range,
3501 ordered,
3502 )
3503 }
3504
3505 #[allow(clippy::too_many_arguments)]
3506 fn scan_fragments(
3507 &self,
3508 with_row_id: bool,
3509 with_row_address: bool,
3510 with_row_last_updated_at_version: bool,
3511 with_row_created_at_version: bool,
3512 with_make_deletions_null: bool,
3513 projection: Arc<Schema>,
3514 fragments: Arc<Vec<Fragment>>,
3515 range: Option<Range<u64>>,
3516 ordered: bool,
3517 ) -> Arc<dyn ExecutionPlan> {
3518 log::trace!("scan_fragments covered {} fragments", fragments.len());
3519 let config = LanceScanConfig {
3520 batch_size: self.get_batch_size(),
3521 batch_readahead: self.batch_readahead,
3522 fragment_readahead: self.fragment_readahead,
3523 io_buffer_size: self.get_io_buffer_size(),
3524 with_row_id,
3525 with_row_address,
3526 with_row_last_updated_at_version,
3527 with_row_created_at_version,
3528 with_make_deletions_null,
3529 ordered_output: ordered,
3530 };
3531 Arc::new(LanceScanExec::new(
3532 self.dataset.clone(),
3533 fragments,
3534 range,
3535 projection,
3536 config,
3537 ))
3538 }
3539
3540 fn pushdown_scan(
3541 &self,
3542 make_deletions_null: bool,
3543 filter_plan: &ExprFilterPlan,
3544 ) -> Result<Arc<dyn ExecutionPlan>> {
3545 log::trace!("pushdown_scan");
3546
3547 let config = ScanConfig {
3548 batch_readahead: self.batch_readahead,
3549 fragment_readahead: self
3550 .fragment_readahead
3551 .unwrap_or(LEGACY_DEFAULT_FRAGMENT_READAHEAD),
3552 with_row_id: self.projection_plan.physical_projection.with_row_id,
3553 with_row_address: self.projection_plan.physical_projection.with_row_addr,
3554 make_deletions_null,
3555 ordered_output: self.ordered,
3556 file_reader_options: self
3557 .file_reader_options
3558 .clone()
3559 .or_else(|| self.dataset.file_reader_options.clone()),
3560 };
3561
3562 let fragments = if let Some(fragment) = self.fragments.as_ref() {
3563 Arc::new(fragment.clone())
3564 } else {
3565 self.dataset.fragments().clone()
3566 };
3567
3568 Ok(Arc::new(LancePushdownScanExec::try_new(
3569 self.dataset.clone(),
3570 fragments,
3571 Arc::new(self.projection_plan.physical_projection.to_bare_schema()),
3572 filter_plan.refine_expr.clone().unwrap(),
3573 config,
3574 )?))
3575 }
3576
3577 async fn flat_fts(
3578 &self,
3579 input: Arc<dyn ExecutionPlan>,
3580 q: &FullTextSearchQuery,
3581 ) -> Result<Arc<dyn ExecutionPlan>> {
3582 let fts_query = if q.columns().is_empty() {
3583 let indexed_columns = fts_indexed_columns(self.dataset.clone()).await?;
3584 fill_fts_query_column(&q.query, &indexed_columns, false)?
3585 } else {
3586 q.query.clone()
3587 };
3588
3589 match &fts_query {
3590 FtsQuery::Match(match_query) => {
3591 let schema = Arc::new((input.schema()).try_with_column(SCORE_FIELD.clone())?);
3592
3593 let column = match_query
3594 .column
3595 .as_ref()
3596 .ok_or(Error::invalid_input(
3597 "the column must be specified in the query".to_string(),
3598 location!(),
3599 ))?
3600 .clone();
3601 let input = if schema.column_with_name(&column).is_none() {
3602 let projection = self
3603 .dataset
3604 .empty_projection()
3605 .union_column(&column, OnMissing::Error)?;
3606 self.take(input, projection)?
3607 } else {
3608 input
3609 };
3610
3611 Ok(Arc::new(FlatMatchQueryExec::new(
3612 self.dataset.clone(),
3613 match_query.clone(),
3614 q.params(),
3615 input,
3616 schema,
3617 )))
3618 }
3619 _ => {
3620 let default_filter = ExprFilterPlan::default();
3621 let fts_plan = self.fts(&default_filter, q).await?;
3622
3623 let vector_row_id = Column::new_with_schema(ROW_ID, input.schema().as_ref())?;
3624 let fts_row_id = Column::new_with_schema(ROW_ID, fts_plan.schema().as_ref())?;
3625 let join = HashJoinExec::try_new(
3626 input,
3627 fts_plan,
3628 vec![(Arc::new(vector_row_id), Arc::new(fts_row_id))],
3629 None,
3630 &JoinType::Inner,
3631 None,
3632 PartitionMode::CollectLeft,
3633 NullEquality::NullEqualsNull,
3634 )?;
3635
3636 let schema = join.schema();
3637 let mut projection_exprs = Vec::new();
3638 let mut contain_rowid = false;
3639 for field in schema.fields() {
3640 if field.name() == ROW_ID {
3641 if contain_rowid {
3642 continue;
3643 }
3644 contain_rowid = true;
3645 }
3646 projection_exprs.push((
3647 Arc::new(Column::new_with_schema(field.name(), schema.as_ref())?)
3648 as Arc<dyn PhysicalExpr>,
3649 field.name().clone(),
3650 ));
3651 }
3652
3653 let projection_exec = ProjectionExec::try_new(projection_exprs, Arc::new(join))?;
3654 Ok(Arc::new(projection_exec))
3655 }
3656 }
3657 }
3658
3659 fn flat_knn(&self, input: Arc<dyn ExecutionPlan>, q: &Query) -> Result<Arc<dyn ExecutionPlan>> {
3661 let metric_type = match q.metric_type {
3663 Some(m) => m,
3664 None => {
3665 let (_, element_type) = get_vector_type(self.dataset.schema(), &q.column)?;
3666 default_distance_type_for(&element_type)
3667 }
3668 };
3669 let flat_dist = Arc::new(KNNVectorDistanceExec::try_new(
3670 input,
3671 &q.column,
3672 q.key.clone(),
3673 metric_type,
3674 )?);
3675
3676 let lower: Option<(Expr, Arc<dyn PhysicalExpr>)> = q
3677 .lower_bound
3678 .map(|v| -> Result<(Expr, Arc<dyn PhysicalExpr>)> {
3679 let logical = col(DIST_COL).gt_eq(lit(v));
3680 let schema = flat_dist.schema();
3681 let df_schema = DFSchema::try_from(schema)?;
3682 let physical = create_physical_expr(&logical, &df_schema, &ExecutionProps::new())?;
3683 Ok::<(Expr, Arc<dyn PhysicalExpr>), _>((logical, physical))
3684 })
3685 .transpose()?;
3686
3687 let upper = q
3688 .upper_bound
3689 .map(|v| -> Result<(Expr, Arc<dyn PhysicalExpr>)> {
3690 let logical = col(DIST_COL).lt(lit(v));
3691 let schema = flat_dist.schema();
3692 let df_schema = DFSchema::try_from(schema)?;
3693 let physical = create_physical_expr(&logical, &df_schema, &ExecutionProps::new())?;
3694 Ok::<(Expr, Arc<dyn PhysicalExpr>), _>((logical, physical))
3695 })
3696 .transpose()?;
3697
3698 let filter_expr = match (lower, upper) {
3699 (Some((llog, _)), Some((ulog, _))) => {
3700 let logical = llog.and(ulog);
3701 let schema = flat_dist.schema();
3702 let df_schema = DFSchema::try_from(schema)?;
3703 let physical = create_physical_expr(&logical, &df_schema, &ExecutionProps::new())?;
3704 Some((logical, physical))
3705 }
3706 (Some((llog, lphys)), None) => Some((llog, lphys)),
3707 (None, Some((ulog, uphys))) => Some((ulog, uphys)),
3708 (None, None) => None,
3709 };
3710
3711 let knn_plan: Arc<dyn ExecutionPlan> = if let Some(filter_expr) = filter_expr {
3712 Arc::new(LanceFilterExec::try_new(filter_expr.0, flat_dist)?)
3713 } else {
3714 flat_dist
3715 };
3716
3717 let sort = SortExec::new(
3719 [
3720 PhysicalSortExpr {
3721 expr: expressions::col(DIST_COL, knn_plan.schema().as_ref())?,
3722 options: SortOptions {
3723 descending: false,
3724 nulls_first: false,
3725 },
3726 },
3727 PhysicalSortExpr {
3728 expr: expressions::col(ROW_ID, knn_plan.schema().as_ref())?,
3729 options: SortOptions {
3730 descending: false,
3731 nulls_first: false,
3732 },
3733 },
3734 ]
3735 .into(),
3736 knn_plan,
3737 )
3738 .with_fetch(Some(q.k));
3739
3740 let logical_not_null = col(DIST_COL).is_not_null();
3741 let not_nulls = Arc::new(LanceFilterExec::try_new(logical_not_null, Arc::new(sort))?);
3742
3743 Ok(not_nulls)
3744 }
3745
3746 fn get_fragments_as_bitmap(&self) -> RoaringBitmap {
3747 if let Some(fragments) = &self.fragments {
3748 RoaringBitmap::from_iter(fragments.iter().map(|f| f.id as u32))
3749 } else {
3750 RoaringBitmap::from_iter(self.dataset.fragments().iter().map(|f| f.id as u32))
3751 }
3752 }
3753
3754 fn get_indexed_frags(&self, index: &[IndexMetadata]) -> RoaringBitmap {
3755 let all_fragments = self.get_fragments_as_bitmap();
3756
3757 let mut all_indexed_frags = RoaringBitmap::new();
3758 for idx in index {
3759 if let Some(fragmap) = idx.fragment_bitmap.as_ref() {
3760 all_indexed_frags |= fragmap;
3761 } else {
3762 return all_fragments;
3765 }
3766 }
3767
3768 all_indexed_frags & all_fragments
3769 }
3770
3771 async fn ann(
3773 &self,
3774 q: &Query,
3775 index: &[IndexMetadata],
3776 filter_plan: &ExprFilterPlan,
3777 ) -> Result<Arc<dyn ExecutionPlan>> {
3778 let prefilter_source = self
3779 .prefilter_source(filter_plan, self.get_indexed_frags(index))
3780 .await?;
3781 let inner_fanout_search = new_knn_exec(self.dataset.clone(), index, q, prefilter_source)?;
3782 let sort_expr = PhysicalSortExpr {
3783 expr: expressions::col(DIST_COL, inner_fanout_search.schema().as_ref())?,
3784 options: SortOptions {
3785 descending: false,
3786 nulls_first: false,
3787 },
3788 };
3789 let sort_expr_row_id = PhysicalSortExpr {
3790 expr: expressions::col(ROW_ID, inner_fanout_search.schema().as_ref())?,
3791 options: SortOptions {
3792 descending: false,
3793 nulls_first: false,
3794 },
3795 };
3796 Ok(Arc::new(
3797 SortExec::new([sort_expr, sort_expr_row_id].into(), inner_fanout_search)
3798 .with_fetch(Some(q.k * q.refine_factor.unwrap_or(1) as usize)),
3799 ))
3800 }
3801
3802 async fn multivec_ann(
3804 &self,
3805 q: &Query,
3806 index: &[IndexMetadata],
3807 filter_plan: &ExprFilterPlan,
3808 ) -> Result<Arc<dyn ExecutionPlan>> {
3809 let over_fetch_factor = *DEFAULT_XTR_OVERFETCH;
3814
3815 let prefilter_source = self
3816 .prefilter_source(filter_plan, self.get_indexed_frags(index))
3817 .await?;
3818 let dim = get_vector_dim(self.dataset.schema(), &q.column)?;
3819
3820 let num_queries = q.key.len() / dim;
3821 let new_queries = (0..num_queries)
3822 .map(|i| q.key.slice(i * dim, dim))
3823 .map(|query_vec| {
3824 let mut new_query = q.clone();
3825 new_query.key = query_vec;
3826 new_query.refine_factor = Some(over_fetch_factor);
3830 new_query
3831 });
3832 let mut ann_nodes = Vec::with_capacity(new_queries.len());
3833 for query in new_queries {
3834 let ann_node = new_knn_exec(
3836 self.dataset.clone(),
3837 index,
3838 &query,
3839 prefilter_source.clone(),
3840 )?;
3841 let sort_expr = PhysicalSortExpr {
3842 expr: expressions::col(DIST_COL, ann_node.schema().as_ref())?,
3843 options: SortOptions {
3844 descending: false,
3845 nulls_first: false,
3846 },
3847 };
3848 let sort_expr_row_id = PhysicalSortExpr {
3849 expr: expressions::col(ROW_ID, ann_node.schema().as_ref())?,
3850 options: SortOptions {
3851 descending: false,
3852 nulls_first: false,
3853 },
3854 };
3855 let ann_node = Arc::new(
3856 SortExec::new([sort_expr, sort_expr_row_id].into(), ann_node)
3857 .with_fetch(Some(q.k * over_fetch_factor as usize)),
3858 );
3859 ann_nodes.push(ann_node as Arc<dyn ExecutionPlan>);
3860 }
3861
3862 let ann_node = Arc::new(MultivectorScoringExec::try_new(ann_nodes, q.clone())?);
3863
3864 let sort_expr = PhysicalSortExpr {
3865 expr: expressions::col(DIST_COL, ann_node.schema().as_ref())?,
3866 options: SortOptions {
3867 descending: false,
3868 nulls_first: false,
3869 },
3870 };
3871 let sort_expr_row_id = PhysicalSortExpr {
3872 expr: expressions::col(ROW_ID, ann_node.schema().as_ref())?,
3873 options: SortOptions {
3874 descending: false,
3875 nulls_first: false,
3876 },
3877 };
3878 let ann_node = Arc::new(
3879 SortExec::new([sort_expr, sort_expr_row_id].into(), ann_node)
3880 .with_fetch(Some(q.k * q.refine_factor.unwrap_or(1) as usize)),
3881 );
3882
3883 Ok(ann_node)
3884 }
3885
3886 async fn prefilter_source(
3891 &self,
3892 filter_plan: &ExprFilterPlan,
3893 required_frags: RoaringBitmap,
3894 ) -> Result<PreFilterSource> {
3895 if filter_plan.is_empty() {
3896 log::trace!("no filter plan, no prefilter");
3897 return Ok(PreFilterSource::None);
3898 }
3899
3900 let fragments = Arc::new(
3901 self.dataset
3902 .manifest
3903 .fragments
3904 .iter()
3905 .filter(|f| required_frags.contains(f.id as u32))
3906 .cloned()
3907 .collect::<Vec<_>>(),
3908 );
3909
3910 if filter_plan.is_exact_index_search() && self.fragments.is_none() {
3916 let index_query = filter_plan.index_query.as_ref().expect_ok()?;
3917 let (_, missing_frags) = self
3918 .partition_frags_by_coverage(index_query, fragments.clone())
3919 .await?;
3920
3921 if missing_frags.is_empty() {
3922 log::trace!("prefilter entirely satisfied by exact index search");
3923 return Ok(PreFilterSource::ScalarIndexQuery(Arc::new(
3928 ScalarIndexExec::new(self.dataset.clone(), index_query.clone()),
3929 )));
3930 } else {
3931 log::trace!("exact index search did not cover all fragments");
3932 }
3933 }
3934
3935 log::trace!(
3937 "prefilter is a filtered read of {} fragments",
3938 fragments.len()
3939 );
3940 let PlannedFilteredScan { plan, .. } = self
3941 .filtered_read(
3942 filter_plan,
3943 self.dataset.empty_projection().with_row_id(),
3944 false,
3945 Some(fragments),
3946 None,
3947 true,
3948 )
3949 .await?;
3950 Ok(PreFilterSource::FilteredRowIds(plan))
3951 }
3952
3953 fn take(
3955 &self,
3956 input: Arc<dyn ExecutionPlan>,
3957 output_projection: Projection,
3958 ) -> Result<Arc<dyn ExecutionPlan>> {
3959 let coalesced = Arc::new(CoalesceBatchesExec::new(
3960 input.clone(),
3961 self.get_batch_size(),
3962 ));
3963 if let Some(take_plan) =
3964 TakeExec::try_new(self.dataset.clone(), coalesced, output_projection)?
3965 {
3966 Ok(Arc::new(take_plan))
3967 } else {
3968 Ok(input)
3970 }
3971 }
3972
3973 fn limit_node(&self, plan: Arc<dyn ExecutionPlan>) -> Arc<dyn ExecutionPlan> {
3975 Arc::new(GlobalLimitExec::new(
3976 plan,
3977 *self.offset.as_ref().unwrap_or(&0) as usize,
3978 self.limit.map(|l| l as usize),
3979 ))
3980 }
3981
3982 #[instrument(level = "info", skip(self))]
3983 pub async fn analyze_plan(&self) -> Result<String> {
3984 let plan = self.create_plan().await?;
3985 let res = analyze_plan(
3986 plan,
3987 LanceExecutionOptions {
3988 batch_size: self.batch_size,
3989 ..Default::default()
3990 },
3991 )
3992 .await;
3993 res
3994 }
3995
3996 #[instrument(level = "info", skip(self))]
3997 pub async fn explain_plan(&self, verbose: bool) -> Result<String> {
3998 let plan = self.create_plan().await?;
3999 let display = DisplayableExecutionPlan::new(plan.as_ref());
4000
4001 Ok(format!("{}", display.indent(verbose)))
4002 }
4003}
4004
4005async fn fts_indexed_columns(dataset: Arc<Dataset>) -> Result<Vec<String>> {
4008 let mut indexed_columns = Vec::new();
4009 for field in dataset.schema().fields_pre_order() {
4010 let is_string_field = match field.data_type() {
4012 DataType::Utf8 | DataType::LargeUtf8 => true,
4013 DataType::List(inner_field) | DataType::LargeList(inner_field) => {
4014 matches!(
4015 inner_field.data_type(),
4016 DataType::Utf8 | DataType::LargeUtf8
4017 )
4018 }
4019 _ => false,
4020 };
4021
4022 if is_string_field {
4023 let column_path =
4025 if let Some(ancestors) = dataset.schema().field_ancestry_by_id(field.id) {
4026 let field_refs: Vec<&str> = ancestors.iter().map(|f| f.name.as_str()).collect();
4027 format_field_path(&field_refs)
4028 } else {
4029 continue; };
4031
4032 let has_fts_index = dataset
4034 .load_scalar_index(
4035 IndexCriteria::default()
4036 .for_column(&column_path)
4037 .supports_fts(),
4038 )
4039 .await?
4040 .is_some();
4041
4042 if has_fts_index {
4043 indexed_columns.push(column_path);
4044 }
4045 }
4046 }
4047 Ok(indexed_columns)
4048}
4049
4050#[pin_project::pin_project]
4054pub struct DatasetRecordBatchStream {
4055 #[pin]
4056 exec_node: SendableRecordBatchStream,
4057 span: Span,
4058}
4059
4060impl DatasetRecordBatchStream {
4061 pub fn new(exec_node: SendableRecordBatchStream) -> Self {
4062 let schema = exec_node.schema();
4063 let adapter = SchemaAdapter::new(schema.clone());
4064 let exec_node = if SchemaAdapter::requires_logical_conversion(&schema) {
4065 adapter.to_logical_stream(exec_node)
4066 } else {
4067 exec_node
4068 };
4069
4070 let span = info_span!("DatasetRecordBatchStream");
4071 Self { exec_node, span }
4072 }
4073}
4074
4075impl RecordBatchStream for DatasetRecordBatchStream {
4076 fn schema(&self) -> SchemaRef {
4077 self.exec_node.schema()
4078 }
4079}
4080
4081impl Stream for DatasetRecordBatchStream {
4082 type Item = Result<RecordBatch>;
4083
4084 fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
4085 let mut this = self.project();
4086 let _guard = this.span.enter();
4087 match this.exec_node.poll_next_unpin(cx) {
4088 Poll::Ready(result) => Poll::Ready(result.map(|r| Ok(r?))),
4089 Poll::Pending => Poll::Pending,
4090 }
4091 }
4092}
4093
4094impl From<DatasetRecordBatchStream> for SendableRecordBatchStream {
4095 fn from(stream: DatasetRecordBatchStream) -> Self {
4096 stream.exec_node
4097 }
4098}
4099
4100#[cfg(test)]
4101pub mod test_dataset {
4102
4103 use super::*;
4104
4105 use std::{collections::HashMap, vec};
4106
4107 use arrow_array::{
4108 ArrayRef, FixedSizeListArray, Int32Array, RecordBatch, RecordBatchIterator, StringArray,
4109 };
4110 use arrow_schema::{ArrowError, DataType};
4111 use lance_arrow::FixedSizeListArrayExt;
4112 use lance_core::utils::tempfile::TempStrDir;
4113 use lance_file::version::LanceFileVersion;
4114 use lance_index::{
4115 scalar::{inverted::tokenizer::InvertedIndexParams, ScalarIndexParams},
4116 IndexType,
4117 };
4118
4119 use crate::dataset::WriteParams;
4120 use crate::index::vector::VectorIndexParams;
4121
4122 pub struct TestVectorDataset {
4132 pub tmp_dir: TempStrDir,
4133 pub schema: Arc<ArrowSchema>,
4134 pub dataset: Dataset,
4135 dimension: u32,
4136 }
4137
4138 impl TestVectorDataset {
4139 pub async fn new(
4140 data_storage_version: LanceFileVersion,
4141 stable_row_ids: bool,
4142 ) -> Result<Self> {
4143 Self::new_with_dimension(data_storage_version, stable_row_ids, 32).await
4144 }
4145
4146 pub async fn new_with_dimension(
4147 data_storage_version: LanceFileVersion,
4148 stable_row_ids: bool,
4149 dimension: u32,
4150 ) -> Result<Self> {
4151 let path = TempStrDir::default();
4152
4153 let metadata: HashMap<String, String> =
4155 vec![("dataset".to_string(), "vector".to_string())]
4156 .into_iter()
4157 .collect();
4158
4159 let schema = Arc::new(ArrowSchema::new_with_metadata(
4160 vec![
4161 ArrowField::new("i", DataType::Int32, true),
4162 ArrowField::new("s", DataType::Utf8, true),
4163 ArrowField::new(
4164 "vec",
4165 DataType::FixedSizeList(
4166 Arc::new(ArrowField::new("item", DataType::Float32, true)),
4167 dimension as i32,
4168 ),
4169 true,
4170 ),
4171 ],
4172 metadata,
4173 ));
4174
4175 let batches: Vec<RecordBatch> = (0..5)
4176 .map(|i| {
4177 let vector_values: Float32Array =
4178 (0..dimension * 80).map(|v| v as f32).collect();
4179 let vectors =
4180 FixedSizeListArray::try_new_from_values(vector_values, dimension as i32)
4181 .unwrap();
4182 RecordBatch::try_new(
4183 schema.clone(),
4184 vec![
4185 Arc::new(Int32Array::from_iter_values(i * 80..(i + 1) * 80)),
4186 Arc::new(StringArray::from_iter_values(
4187 (i * 80..(i + 1) * 80).map(|v| format!("s-{}", v)),
4188 )),
4189 Arc::new(vectors),
4190 ],
4191 )
4192 })
4193 .collect::<std::result::Result<Vec<_>, ArrowError>>()?;
4194
4195 let params = WriteParams {
4196 max_rows_per_group: 10,
4197 max_rows_per_file: 200,
4198 data_storage_version: Some(data_storage_version),
4199 enable_stable_row_ids: stable_row_ids,
4200 ..Default::default()
4201 };
4202 let reader = RecordBatchIterator::new(batches.into_iter().map(Ok), schema.clone());
4203
4204 let dataset = Dataset::write(reader, &path, Some(params)).await?;
4205
4206 Ok(Self {
4207 tmp_dir: path,
4208 schema,
4209 dataset,
4210 dimension,
4211 })
4212 }
4213
4214 pub async fn make_vector_index(&mut self) -> Result<()> {
4215 let params = VectorIndexParams::ivf_pq(2, 8, 2, MetricType::L2, 2);
4216 self.dataset
4217 .create_index(
4218 &["vec"],
4219 IndexType::Vector,
4220 Some("idx".to_string()),
4221 ¶ms,
4222 true,
4223 )
4224 .await?;
4225 Ok(())
4226 }
4227
4228 pub async fn make_scalar_index(&mut self) -> Result<()> {
4229 self.dataset
4230 .create_index(
4231 &["i"],
4232 IndexType::Scalar,
4233 None,
4234 &ScalarIndexParams::default(),
4235 true,
4236 )
4237 .await?;
4238 Ok(())
4239 }
4240
4241 pub async fn make_fts_index(&mut self) -> Result<()> {
4242 let params = InvertedIndexParams::default().with_position(true);
4243 self.dataset
4244 .create_index(&["s"], IndexType::Inverted, None, ¶ms, true)
4245 .await?;
4246 Ok(())
4247 }
4248
4249 pub async fn append_new_data(&mut self) -> Result<()> {
4250 let vector_values: Float32Array = (0..10)
4251 .flat_map(|i| vec![i as f32; self.dimension as usize].into_iter())
4252 .collect();
4253 let new_vectors =
4254 FixedSizeListArray::try_new_from_values(vector_values, self.dimension as i32)
4255 .unwrap();
4256 let new_data: Vec<ArrayRef> = vec![
4257 Arc::new(Int32Array::from_iter_values(400..410)), Arc::new(StringArray::from_iter_values(
4259 (400..410).map(|v| format!("s-{}", v)),
4260 )),
4261 Arc::new(new_vectors),
4262 ];
4263 let reader = RecordBatchIterator::new(
4264 vec![RecordBatch::try_new(self.schema.clone(), new_data).unwrap()]
4265 .into_iter()
4266 .map(Ok),
4267 self.schema.clone(),
4268 );
4269 self.dataset.append(reader, None).await?;
4270 Ok(())
4271 }
4272 }
4273}
4274
4275#[cfg(test)]
4276mod test {
4277
4278 use std::collections::BTreeSet;
4279 use std::time::{Duration, Instant};
4280 use std::vec;
4281
4282 use arrow::array::as_primitive_array;
4283 use arrow::datatypes::{Float64Type, Int32Type, Int64Type};
4284 use arrow_array::cast::AsArray;
4285 use arrow_array::types::{Float32Type, UInt64Type};
4286 use arrow_array::{
4287 ArrayRef, FixedSizeListArray, Float16Array, Int32Array, LargeStringArray, PrimitiveArray,
4288 RecordBatchIterator, StringArray, StructArray, UInt8Array,
4289 };
4290
4291 use arrow_ord::sort::sort_to_indices;
4292 use arrow_schema::Fields;
4293 use arrow_select::take;
4294 use datafusion::logical_expr::{col, lit};
4295 use half::f16;
4296 use lance_arrow::{FixedSizeListArrayExt, SchemaExt};
4297 use lance_core::utils::tempfile::TempStrDir;
4298 use lance_core::{ROW_CREATED_AT_VERSION, ROW_LAST_UPDATED_AT_VERSION};
4299 use lance_datagen::{
4300 array, gen_batch, ArrayGeneratorExt, BatchCount, ByteCount, Dimension, RowCount,
4301 };
4302 use lance_file::version::LanceFileVersion;
4303 use lance_index::optimize::OptimizeOptions;
4304 use lance_index::scalar::inverted::query::{MatchQuery, PhraseQuery};
4305 use lance_index::vector::hnsw::builder::HnswBuildParams;
4306 use lance_index::vector::ivf::IvfBuildParams;
4307 use lance_index::vector::pq::PQBuildParams;
4308 use lance_index::vector::sq::builder::SQBuildParams;
4309 use lance_index::{scalar::ScalarIndexParams, IndexType};
4310 use lance_io::assert_io_gt;
4311 use lance_io::object_store::ObjectStoreParams;
4312
4313 use lance_linalg::distance::DistanceType;
4314 use lance_testing::datagen::{BatchGenerator, IncrementingInt32, RandomVector};
4315 use object_store::throttle::ThrottleConfig;
4316 use rstest::rstest;
4317
4318 use super::*;
4319 use crate::dataset::optimize::{compact_files, CompactionOptions};
4320 use crate::dataset::scanner::test_dataset::TestVectorDataset;
4321 use crate::dataset::WriteMode;
4322 use crate::dataset::WriteParams;
4323 use crate::index::vector::{StageParams, VectorIndexParams};
4324 use crate::utils::test::{
4325 assert_plan_node_equals, DatagenExt, FragmentCount, FragmentRowCount, ThrottledStoreWrapper,
4326 };
4327
4328 #[test]
4329 fn test_env_var_parsing() {
4330 std::env::set_var("LANCE_DEFAULT_BATCH_SIZE", "not_a_number");
4334 let result = get_default_batch_size();
4335 assert_eq!(result, None, "Should return None for invalid batch size");
4336
4337 std::env::set_var("LANCE_DEFAULT_BATCH_SIZE", "2048");
4339 let result = get_default_batch_size();
4340 assert_eq!(result, Some(2048), "Should parse valid batch size");
4341
4342 std::env::remove_var("LANCE_DEFAULT_BATCH_SIZE");
4344 let result = get_default_batch_size();
4345 assert_eq!(result, None, "Should return None when env var is not set");
4346 }
4347
4348 #[test]
4349 fn test_parse_env_var() {
4350 let test_var = "LANCE_TEST_PARSE_ENV_VAR_USIZE";
4354
4355 std::env::set_var(test_var, "12345");
4357 let result: Option<usize> = parse_env_var(test_var, "Using default.");
4358 assert_eq!(result, Some(12345));
4359
4360 std::env::set_var(test_var, "not_a_number");
4362 let result: Option<usize> = parse_env_var(test_var, "Using default.");
4363 assert_eq!(result, None);
4364
4365 std::env::remove_var(test_var);
4367 let result: Option<usize> = parse_env_var(test_var, "Using default.");
4368 assert_eq!(result, None);
4369
4370 let test_var_u32 = "LANCE_TEST_PARSE_ENV_VAR_U32";
4372 std::env::set_var(test_var_u32, "42");
4373 let result: Option<u32> = parse_env_var(test_var_u32, "Using default value.");
4374 assert_eq!(result, Some(42));
4375
4376 std::env::set_var(test_var_u32, "invalid");
4377 let result: Option<u32> = parse_env_var(test_var_u32, "Using default value.");
4378 assert_eq!(result, None);
4379
4380 std::env::remove_var(test_var_u32);
4381
4382 let test_var_u64 = "LANCE_TEST_PARSE_ENV_VAR_U64";
4384 std::env::set_var(test_var_u64, "9999999999");
4385 let result: Option<u64> = parse_env_var(test_var_u64, "Using default value.");
4386 assert_eq!(result, Some(9999999999));
4387
4388 std::env::set_var(test_var_u64, "-1");
4389 let result: Option<u64> = parse_env_var(test_var_u64, "Using default value.");
4390 assert_eq!(result, None);
4391
4392 std::env::remove_var(test_var_u64);
4393 }
4394
4395 async fn make_binary_vector_dataset() -> Result<(TempStrDir, Dataset)> {
4396 let tmp_dir = TempStrDir::default();
4397 let dim = 4;
4398 let schema = Arc::new(ArrowSchema::new(vec![
4399 ArrowField::new("id", DataType::Int32, false),
4400 ArrowField::new(
4401 "bin",
4402 DataType::FixedSizeList(
4403 Arc::new(ArrowField::new("item", DataType::UInt8, true)),
4404 dim,
4405 ),
4406 false,
4407 ),
4408 ]));
4409
4410 let vectors = FixedSizeListArray::try_new_from_values(
4411 UInt8Array::from(vec![
4412 0b0000_1111u8,
4413 0,
4414 0,
4415 0, 0b0000_0011u8,
4417 0,
4418 0,
4419 0, 0u8,
4421 0,
4422 0,
4423 0,
4424 ]),
4425 dim,
4426 )?;
4427 let ids = Int32Array::from(vec![0, 1, 2]);
4428
4429 let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(ids), Arc::new(vectors)])?;
4430 let reader = RecordBatchIterator::new(vec![Ok(batch)], schema.clone());
4431 Dataset::write(reader, &tmp_dir, None).await?;
4432 let dataset = Dataset::open(&tmp_dir).await?;
4433 Ok((tmp_dir, dataset))
4434 }
4435
4436 #[tokio::test]
4437 async fn test_batch_size() {
4438 let schema = Arc::new(ArrowSchema::new(vec![
4439 ArrowField::new("i", DataType::Int32, true),
4440 ArrowField::new("s", DataType::Utf8, true),
4441 ]));
4442
4443 let batches: Vec<RecordBatch> = (0..5)
4444 .map(|i| {
4445 RecordBatch::try_new(
4446 schema.clone(),
4447 vec![
4448 Arc::new(Int32Array::from_iter_values(i * 20..(i + 1) * 20)),
4449 Arc::new(StringArray::from_iter_values(
4450 (i * 20..(i + 1) * 20).map(|v| format!("s-{}", v)),
4451 )),
4452 ],
4453 )
4454 .unwrap()
4455 })
4456 .collect();
4457
4458 for use_filter in [false, true] {
4459 let test_dir = TempStrDir::default();
4460 let test_uri = &test_dir;
4461 let write_params = WriteParams {
4462 max_rows_per_file: 40,
4463 max_rows_per_group: 10,
4464 ..Default::default()
4465 };
4466 let batches =
4467 RecordBatchIterator::new(batches.clone().into_iter().map(Ok), schema.clone());
4468 Dataset::write(batches, test_uri, Some(write_params))
4469 .await
4470 .unwrap();
4471
4472 let dataset = Dataset::open(test_uri).await.unwrap();
4473 let mut builder = dataset.scan();
4474 builder.batch_size(8);
4475 if use_filter {
4476 builder.filter("i IS NOT NULL").unwrap();
4477 }
4478 let mut stream = builder.try_into_stream().await.unwrap();
4479 let mut rows_read = 0;
4480 while let Some(next) = stream.next().await {
4481 let next = next.unwrap();
4482 let expected = 8.min(100 - rows_read);
4483 assert_eq!(next.num_rows(), expected);
4484 rows_read += next.num_rows();
4485 }
4486 }
4487 }
4488
4489 #[tokio::test]
4490 async fn test_strict_batch_size() {
4491 let dataset = lance_datagen::gen_batch()
4492 .col("x", array::step::<Int32Type>())
4493 .anon_col(array::step::<Int64Type>())
4494 .into_ram_dataset(FragmentCount::from(7), FragmentRowCount::from(6))
4495 .await
4496 .unwrap();
4497
4498 let mut scan = dataset.scan();
4499 scan.batch_size(10)
4500 .strict_batch_size(true)
4501 .filter("x % 2 == 0")
4502 .unwrap();
4503
4504 let batches = scan
4505 .try_into_stream()
4506 .await
4507 .unwrap()
4508 .try_collect::<Vec<_>>()
4509 .await
4510 .unwrap();
4511
4512 let batch_sizes = batches.iter().map(|b| b.num_rows()).collect::<Vec<_>>();
4513 assert_eq!(batch_sizes, vec![10, 10, 1]);
4514 }
4515
4516 #[tokio::test]
4517 async fn test_column_not_exist() {
4518 let dataset = lance_datagen::gen_batch()
4519 .col("x", array::step::<Int32Type>())
4520 .into_ram_dataset(FragmentCount::from(7), FragmentRowCount::from(6))
4521 .await
4522 .unwrap();
4523
4524 let check_err_msg = |r: Result<DatasetRecordBatchStream>| {
4525 let Err(err) = r else {
4526 panic!(
4527 "Expected an error to be raised saying column y is not found but got no error"
4528 )
4529 };
4530
4531 assert!(
4532 err.to_string().contains("No field named y"),
4533 "Expected error to contain 'No field named y' but got {}",
4534 err
4535 );
4536 };
4537
4538 let mut scan = dataset.scan();
4539 scan.project(&["x", "y"]).unwrap();
4540 check_err_msg(scan.try_into_stream().await);
4541
4542 let mut scan = dataset.scan();
4543 scan.project(&["y"]).unwrap();
4544 check_err_msg(scan.try_into_stream().await);
4545
4546 let mut scan = dataset.scan();
4549 scan.project_with_transform(&[("foo", "1")]).unwrap();
4550 match scan.try_into_stream().await {
4551 Ok(_) => panic!("Expected an error to be raised saying not supported"),
4552 Err(e) => {
4553 assert!(
4554 e.to_string().contains("Received only dynamic expressions"),
4555 "Expected error to contain 'Received only dynamic expressions' but got {}",
4556 e
4557 );
4558 }
4559 }
4560 }
4561
4562 #[cfg(not(windows))]
4563 #[tokio::test]
4564 async fn test_local_object_store() {
4565 let schema = Arc::new(ArrowSchema::new(vec![
4566 ArrowField::new("i", DataType::Int32, true),
4567 ArrowField::new("s", DataType::Utf8, true),
4568 ]));
4569
4570 let batches: Vec<RecordBatch> = (0..5)
4571 .map(|i| {
4572 RecordBatch::try_new(
4573 schema.clone(),
4574 vec![
4575 Arc::new(Int32Array::from_iter_values(i * 20..(i + 1) * 20)),
4576 Arc::new(StringArray::from_iter_values(
4577 (i * 20..(i + 1) * 20).map(|v| format!("s-{}", v)),
4578 )),
4579 ],
4580 )
4581 .unwrap()
4582 })
4583 .collect();
4584
4585 let test_dir = TempStrDir::default();
4586 let test_uri = &test_dir;
4587 let write_params = WriteParams {
4588 max_rows_per_file: 40,
4589 max_rows_per_group: 10,
4590 ..Default::default()
4591 };
4592 let batches = RecordBatchIterator::new(batches.clone().into_iter().map(Ok), schema.clone());
4593 Dataset::write(batches, test_uri, Some(write_params))
4594 .await
4595 .unwrap();
4596
4597 let dataset = Dataset::open(&format!("file-object-store://{}", test_uri))
4598 .await
4599 .unwrap();
4600 let mut builder = dataset.scan();
4601 builder.batch_size(8);
4602 let mut stream = builder.try_into_stream().await.unwrap();
4603 let mut rows_read = 0;
4604 while let Some(next) = stream.next().await {
4605 let next = next.unwrap();
4606 let expected = 8.min(100 - rows_read);
4607 assert_eq!(next.num_rows(), expected);
4608 rows_read += next.num_rows();
4609 }
4610 }
4611
4612 #[tokio::test]
4613 async fn test_filter_parsing() -> Result<()> {
4614 let test_ds = TestVectorDataset::new(LanceFileVersion::Stable, false).await?;
4615 let dataset = &test_ds.dataset;
4616
4617 let mut scan = dataset.scan();
4618 assert!(scan.filter.is_none());
4619
4620 scan.filter("i > 50")?;
4621 assert_eq!(scan.get_expr_filter().unwrap(), Some(col("i").gt(lit(50))));
4622
4623 for use_stats in [false, true] {
4624 let batches = scan
4625 .project(&["s"])?
4626 .use_stats(use_stats)
4627 .try_into_stream()
4628 .await?
4629 .try_collect::<Vec<_>>()
4630 .await?;
4631 let batch = concat_batches(&batches[0].schema(), &batches)?;
4632
4633 let expected_batch = RecordBatch::try_new(
4634 Arc::new(test_ds.schema.project(&[1])?),
4636 vec![Arc::new(StringArray::from_iter_values(
4637 (51..400).map(|v| format!("s-{}", v)),
4638 ))],
4639 )?;
4640 assert_eq!(batch, expected_batch);
4641 }
4642 Ok(())
4643 }
4644
4645 #[tokio::test]
4646 async fn test_scan_regexp_match_and_non_empty_captions() {
4647 let schema = Arc::new(ArrowSchema::new(vec![
4650 ArrowField::new("keywords", DataType::Utf8, true),
4651 ArrowField::new("natural_caption", DataType::Utf8, true),
4652 ArrowField::new("poetic_caption", DataType::Utf8, true),
4653 ]));
4654
4655 let batch = RecordBatch::try_new(
4656 schema.clone(),
4657 vec![
4658 Arc::new(StringArray::from(vec![
4659 Some("Liberty for all"),
4660 Some("peace"),
4661 Some("revolution now"),
4662 Some("Liberty"),
4663 Some("revolutionary"),
4664 Some("none"),
4665 ])) as ArrayRef,
4666 Arc::new(StringArray::from(vec![
4667 Some("a"),
4668 Some("b"),
4669 None,
4670 Some(""),
4671 Some("c"),
4672 Some("d"),
4673 ])) as ArrayRef,
4674 Arc::new(StringArray::from(vec![
4675 Some("x"),
4676 Some(""),
4677 Some("y"),
4678 Some("z"),
4679 None,
4680 Some("w"),
4681 ])) as ArrayRef,
4682 ],
4683 )
4684 .unwrap();
4685
4686 let reader = RecordBatchIterator::new(vec![Ok(batch.clone())], schema.clone());
4687 let dataset = Dataset::write(reader, "memory://", None).await.unwrap();
4688
4689 let mut scan = dataset.scan();
4690 scan.filter(
4691 "regexp_match(keywords, 'Liberty|revolution') AND \
4692 (natural_caption IS NOT NULL AND natural_caption <> '' AND \
4693 poetic_caption IS NOT NULL AND poetic_caption <> '')",
4694 )
4695 .unwrap();
4696
4697 let out = scan.try_into_batch().await.unwrap();
4698 assert_eq!(out.num_rows(), 1);
4699
4700 let out_keywords = out
4701 .column_by_name("keywords")
4702 .unwrap()
4703 .as_string::<i32>()
4704 .value(0);
4705 let out_nat = out
4706 .column_by_name("natural_caption")
4707 .unwrap()
4708 .as_string::<i32>()
4709 .value(0);
4710 let out_poetic = out
4711 .column_by_name("poetic_caption")
4712 .unwrap()
4713 .as_string::<i32>()
4714 .value(0);
4715
4716 assert_eq!(out_keywords, "Liberty for all");
4717 assert_eq!(out_nat, "a");
4718 assert_eq!(out_poetic, "x");
4719 }
4720
4721 #[tokio::test]
4722 async fn test_nested_projection() {
4723 let point_fields: Fields = vec![
4724 ArrowField::new("x", DataType::Float32, true),
4725 ArrowField::new("y", DataType::Float32, true),
4726 ]
4727 .into();
4728 let metadata_fields: Fields = vec![
4729 ArrowField::new("location", DataType::Struct(point_fields), true),
4730 ArrowField::new("age", DataType::Int32, true),
4731 ]
4732 .into();
4733 let metadata_field = ArrowField::new("metadata", DataType::Struct(metadata_fields), true);
4734 let schema = Arc::new(ArrowSchema::new(vec![
4735 metadata_field,
4736 ArrowField::new("idx", DataType::Int32, true),
4737 ]));
4738 let data = lance_datagen::rand(&schema)
4739 .into_ram_dataset(FragmentCount::from(7), FragmentRowCount::from(6))
4740 .await
4741 .unwrap();
4742
4743 let mut scan = data.scan();
4744 scan.project(&["metadata.location.x", "metadata.age"])
4745 .unwrap();
4746 let batch = scan.try_into_batch().await.unwrap();
4747
4748 assert_eq!(
4749 batch.schema().as_ref(),
4750 &ArrowSchema::new(vec![
4751 ArrowField::new("metadata.location.x", DataType::Float32, true),
4752 ArrowField::new("metadata.age", DataType::Int32, true),
4753 ])
4754 );
4755
4756 let take_schema = data.schema().project_by_ids(&[0, 2, 4], false);
4760
4761 let taken = data.take_rows(&[0, 5], take_schema).await.unwrap();
4762
4763 let part_point_fields = Fields::from(vec![ArrowField::new("x", DataType::Float32, true)]);
4765 let part_metadata_fields = Fields::from(vec![
4766 ArrowField::new("location", DataType::Struct(part_point_fields), true),
4767 ArrowField::new("age", DataType::Int32, true),
4768 ]);
4769 let part_schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
4770 "metadata",
4771 DataType::Struct(part_metadata_fields),
4772 true,
4773 )]));
4774
4775 assert_eq!(taken.schema(), part_schema);
4776 }
4777
4778 #[rstest]
4779 #[tokio::test]
4780 async fn test_limit(
4781 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
4782 data_storage_version: LanceFileVersion,
4783 ) -> Result<()> {
4784 let test_ds = TestVectorDataset::new(data_storage_version, false).await?;
4785 let dataset = &test_ds.dataset;
4786
4787 let full_data = dataset.scan().try_into_batch().await?.slice(19, 2);
4788
4789 let actual = dataset
4790 .scan()
4791 .limit(Some(2), Some(19))?
4792 .try_into_batch()
4793 .await?;
4794
4795 assert_eq!(actual.num_rows(), 2);
4796 assert_eq!(actual, full_data);
4797 Ok(())
4798 }
4799
4800 #[test_log::test(tokio::test)]
4801 async fn test_limit_cancel() {
4802 let throttled = Arc::new(ThrottledStoreWrapper {
4811 config: ThrottleConfig {
4812 wait_get_per_call: Duration::from_secs(1),
4813 ..Default::default()
4814 },
4815 });
4816 let write_params = WriteParams {
4817 store_params: Some(ObjectStoreParams {
4818 object_store_wrapper: Some(throttled.clone()),
4819 ..Default::default()
4820 }),
4821 max_rows_per_file: 1,
4822 ..Default::default()
4823 };
4824
4825 let dataset = gen_batch()
4827 .col("i", array::step::<Int32Type>().with_random_nulls(0.1))
4828 .into_ram_dataset_with_params(
4829 FragmentCount::from(2000),
4830 FragmentRowCount::from(1),
4831 Some(write_params),
4832 )
4833 .await
4834 .unwrap();
4835
4836 let mut scan = dataset.scan();
4837 scan.filter("i IS NOT NULL").unwrap();
4838 scan.limit(Some(10), None).unwrap();
4839
4840 let start = Instant::now();
4841 scan.try_into_stream()
4842 .await
4843 .unwrap()
4844 .try_collect::<Vec<_>>()
4845 .await
4846 .unwrap();
4847 let duration = start.elapsed();
4848
4849 assert!(duration < Duration::from_secs(10));
4853 }
4854
4855 #[rstest]
4856 #[tokio::test]
4857 async fn test_knn_nodes(
4858 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
4859 data_storage_version: LanceFileVersion,
4860 #[values(false, true)] stable_row_ids: bool,
4861 #[values(false, true)] build_index: bool,
4862 ) {
4863 let mut test_ds = TestVectorDataset::new(data_storage_version, stable_row_ids)
4864 .await
4865 .unwrap();
4866 if build_index {
4867 test_ds.make_vector_index().await.unwrap();
4868 }
4869 let dataset = &test_ds.dataset;
4870
4871 let mut scan = dataset.scan();
4872 let key: Float32Array = (32..64).map(|v| v as f32).collect();
4873 scan.nearest("vec", &key, 5).unwrap();
4874 scan.refine(5);
4875
4876 let batch = scan.try_into_batch().await.unwrap();
4877
4878 assert_eq!(batch.num_rows(), 5);
4879 assert_eq!(
4880 batch.schema().as_ref(),
4881 &ArrowSchema::new(vec![
4882 ArrowField::new("i", DataType::Int32, true),
4883 ArrowField::new("s", DataType::Utf8, true),
4884 ArrowField::new(
4885 "vec",
4886 DataType::FixedSizeList(
4887 Arc::new(ArrowField::new("item", DataType::Float32, true)),
4888 32,
4889 ),
4890 true,
4891 ),
4892 ArrowField::new(DIST_COL, DataType::Float32, true),
4893 ])
4894 .with_metadata([("dataset".into(), "vector".into())].into())
4895 );
4896
4897 let expected_i = BTreeSet::from_iter(vec![1, 81, 161, 241, 321]);
4898 let column_i = batch.column_by_name("i").unwrap();
4899 let actual_i: BTreeSet<i32> = as_primitive_array::<Int32Type>(column_i.as_ref())
4900 .values()
4901 .iter()
4902 .copied()
4903 .collect();
4904 assert_eq!(expected_i, actual_i);
4905 }
4906
4907 #[rstest]
4908 #[tokio::test]
4909 async fn test_can_project_distance() {
4910 let test_ds = TestVectorDataset::new(LanceFileVersion::Stable, true)
4911 .await
4912 .unwrap();
4913 let dataset = &test_ds.dataset;
4914
4915 let mut scan = dataset.scan();
4916 let key: Float32Array = (32..64).map(|v| v as f32).collect();
4917 scan.nearest("vec", &key, 5).unwrap();
4918 scan.refine(5);
4919 scan.project(&["_distance"]).unwrap();
4920
4921 let batch = scan.try_into_batch().await.unwrap();
4922
4923 assert_eq!(batch.num_rows(), 5);
4924 assert_eq!(batch.num_columns(), 1);
4925 }
4926
4927 #[rstest]
4928 #[tokio::test]
4929 async fn test_knn_with_new_data(
4930 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
4931 data_storage_version: LanceFileVersion,
4932 #[values(false, true)] stable_row_ids: bool,
4933 ) {
4934 let mut test_ds = TestVectorDataset::new(data_storage_version, stable_row_ids)
4935 .await
4936 .unwrap();
4937 test_ds.make_vector_index().await.unwrap();
4938 test_ds.append_new_data().await.unwrap();
4939 let dataset = &test_ds.dataset;
4940
4941 let key: Float32Array = [0f32; 32].into_iter().collect();
4943 let k = 20;
4946
4947 #[derive(Debug)]
4948 struct TestCase {
4949 filter: Option<&'static str>,
4950 limit: Option<i64>,
4951 use_index: bool,
4952 }
4953
4954 let mut cases = vec![];
4955 for filter in [Some("i > 100"), None] {
4956 for limit in [None, Some(10)] {
4957 for use_index in [true, false] {
4958 cases.push(TestCase {
4959 filter,
4960 limit,
4961 use_index,
4962 });
4963 }
4964 }
4965 }
4966
4967 for case in cases {
4969 let mut scanner = dataset.scan();
4970 scanner
4971 .nearest("vec", &key, k)
4972 .unwrap()
4973 .limit(case.limit, None)
4974 .unwrap()
4975 .refine(3)
4976 .use_index(case.use_index);
4977 if let Some(filter) = case.filter {
4978 scanner.filter(filter).unwrap();
4979 }
4980
4981 let result = scanner
4982 .try_into_stream()
4983 .await
4984 .unwrap()
4985 .try_collect::<Vec<_>>()
4986 .await
4987 .unwrap();
4988 assert!(!result.is_empty());
4989 let result = concat_batches(&result[0].schema(), result.iter()).unwrap();
4990
4991 if case.filter.is_some() {
4992 let result_rows = result.num_rows();
4993 let expected_rows = case.limit.unwrap_or(k as i64) as usize;
4994 assert!(
4995 result_rows <= expected_rows,
4996 "Expected less than {} rows, got {}",
4997 expected_rows,
4998 result_rows
4999 );
5000 } else {
5001 assert_eq!(result.num_rows(), case.limit.unwrap_or(k as i64) as usize);
5003 }
5004
5005 assert_eq!(
5007 as_primitive_array::<Int32Type>(result.column(0).as_ref()).value(0),
5008 400
5009 );
5010 }
5011 }
5012
5013 #[rstest]
5014 #[tokio::test]
5015 async fn test_knn_with_prefilter(
5016 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5017 data_storage_version: LanceFileVersion,
5018 #[values(false, true)] stable_row_ids: bool,
5019 ) {
5020 let mut test_ds = TestVectorDataset::new(data_storage_version, stable_row_ids)
5021 .await
5022 .unwrap();
5023 test_ds.make_vector_index().await.unwrap();
5024 let dataset = &test_ds.dataset;
5025
5026 let mut scan = dataset.scan();
5027 let key: Float32Array = (32..64).map(|v| v as f32).collect();
5028 scan.filter("i > 100").unwrap();
5029 scan.prefilter(true);
5030 scan.project(&["i", "vec"]).unwrap();
5031 scan.nearest("vec", &key, 5).unwrap();
5032 scan.use_index(false);
5033
5034 let results = scan
5035 .try_into_stream()
5036 .await
5037 .unwrap()
5038 .try_collect::<Vec<_>>()
5039 .await
5040 .unwrap();
5041
5042 assert_eq!(results.len(), 1);
5043 let batch = &results[0];
5044
5045 assert_eq!(batch.num_rows(), 5);
5046 assert_eq!(
5047 batch.schema().as_ref(),
5048 &ArrowSchema::new(vec![
5049 ArrowField::new("i", DataType::Int32, true),
5050 ArrowField::new(
5051 "vec",
5052 DataType::FixedSizeList(
5053 Arc::new(ArrowField::new("item", DataType::Float32, true)),
5054 32,
5055 ),
5056 true,
5057 ),
5058 ArrowField::new(DIST_COL, DataType::Float32, true),
5059 ])
5060 .with_metadata([("dataset".into(), "vector".into())].into())
5061 );
5062
5063 let exact_i = BTreeSet::from_iter(vec![161, 241, 321]);
5065 let close_i = BTreeSet::from_iter(vec![161, 241, 321, 160, 162, 240, 242, 320, 322]);
5067 let column_i = batch.column_by_name("i").unwrap();
5068 let actual_i: BTreeSet<i32> = as_primitive_array::<Int32Type>(column_i.as_ref())
5069 .values()
5070 .iter()
5071 .copied()
5072 .collect();
5073 assert!(exact_i.is_subset(&actual_i));
5074 assert!(actual_i.is_subset(&close_i));
5075 }
5076
5077 #[rstest]
5078 #[tokio::test]
5079 async fn test_knn_filter_new_data(
5080 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5081 data_storage_version: LanceFileVersion,
5082 #[values(false, true)] stable_row_ids: bool,
5083 ) {
5084 let mut test_ds = TestVectorDataset::new(data_storage_version, stable_row_ids)
5088 .await
5089 .unwrap();
5090 test_ds.make_vector_index().await.unwrap();
5091 test_ds.append_new_data().await.unwrap();
5092 let dataset = &test_ds.dataset;
5093
5094 let key: Float32Array = [0f32; 32].into_iter().collect();
5096
5097 let mut query = dataset.scan();
5098 query.nearest("vec", &key, 20).unwrap();
5099
5100 let results = query
5102 .try_into_stream()
5103 .await
5104 .unwrap()
5105 .try_collect::<Vec<_>>()
5106 .await
5107 .unwrap();
5108
5109 let results_i = results[0]["i"]
5110 .as_primitive::<Int32Type>()
5111 .values()
5112 .iter()
5113 .copied()
5114 .collect::<BTreeSet<_>>();
5115
5116 assert!(results_i.contains(&400));
5117
5118 for prefilter in [false, true] {
5120 let mut query = dataset.scan();
5121 query
5122 .filter("i != 400")
5123 .unwrap()
5124 .prefilter(prefilter)
5125 .nearest("vec", &key, 20)
5126 .unwrap();
5127
5128 let results = query
5129 .try_into_stream()
5130 .await
5131 .unwrap()
5132 .try_collect::<Vec<_>>()
5133 .await
5134 .unwrap();
5135
5136 let results_i = results[0]["i"]
5137 .as_primitive::<Int32Type>()
5138 .values()
5139 .iter()
5140 .copied()
5141 .collect::<BTreeSet<_>>();
5142
5143 assert!(!results_i.contains(&400));
5144 }
5145 }
5146
5147 #[rstest]
5148 #[tokio::test]
5149 async fn test_knn_with_filter(
5150 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5151 data_storage_version: LanceFileVersion,
5152 #[values(false, true)] stable_row_ids: bool,
5153 ) {
5154 let test_ds = TestVectorDataset::new(data_storage_version, stable_row_ids)
5155 .await
5156 .unwrap();
5157 let dataset = &test_ds.dataset;
5158
5159 let mut scan = dataset.scan();
5160 let key: Float32Array = (32..64).map(|v| v as f32).collect();
5161 scan.nearest("vec", &key, 5).unwrap();
5162 scan.filter("i > 100").unwrap();
5163 scan.project(&["i", "vec"]).unwrap();
5164 scan.refine(5);
5165
5166 let results = scan
5167 .try_into_stream()
5168 .await
5169 .unwrap()
5170 .try_collect::<Vec<_>>()
5171 .await
5172 .unwrap();
5173
5174 assert_eq!(results.len(), 1);
5175 let batch = &results[0];
5176
5177 assert_eq!(batch.num_rows(), 3);
5178 assert_eq!(
5179 batch.schema().as_ref(),
5180 &ArrowSchema::new(vec![
5181 ArrowField::new("i", DataType::Int32, true),
5182 ArrowField::new(
5183 "vec",
5184 DataType::FixedSizeList(
5185 Arc::new(ArrowField::new("item", DataType::Float32, true)),
5186 32,
5187 ),
5188 true,
5189 ),
5190 ArrowField::new(DIST_COL, DataType::Float32, true),
5191 ])
5192 .with_metadata([("dataset".into(), "vector".into())].into())
5193 );
5194
5195 let expected_i = BTreeSet::from_iter(vec![161, 241, 321]);
5196 let column_i = batch.column_by_name("i").unwrap();
5197 let actual_i: BTreeSet<i32> = as_primitive_array::<Int32Type>(column_i.as_ref())
5198 .values()
5199 .iter()
5200 .copied()
5201 .collect();
5202 assert_eq!(expected_i, actual_i);
5203 }
5204
5205 #[rstest]
5206 #[tokio::test]
5207 async fn test_refine_factor(
5208 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5209 data_storage_version: LanceFileVersion,
5210 #[values(false, true)] stable_row_ids: bool,
5211 ) {
5212 let test_ds = TestVectorDataset::new(data_storage_version, stable_row_ids)
5213 .await
5214 .unwrap();
5215 let dataset = &test_ds.dataset;
5216
5217 let mut scan = dataset.scan();
5218 let key: Float32Array = (32..64).map(|v| v as f32).collect();
5219 scan.nearest("vec", &key, 5).unwrap();
5220 scan.refine(5);
5221
5222 let results = scan
5223 .try_into_stream()
5224 .await
5225 .unwrap()
5226 .try_collect::<Vec<_>>()
5227 .await
5228 .unwrap();
5229
5230 assert_eq!(results.len(), 1);
5231 let batch = &results[0];
5232
5233 assert_eq!(batch.num_rows(), 5);
5234 assert_eq!(
5235 batch.schema().as_ref(),
5236 &ArrowSchema::new(vec![
5237 ArrowField::new("i", DataType::Int32, true),
5238 ArrowField::new("s", DataType::Utf8, true),
5239 ArrowField::new(
5240 "vec",
5241 DataType::FixedSizeList(
5242 Arc::new(ArrowField::new("item", DataType::Float32, true)),
5243 32,
5244 ),
5245 true,
5246 ),
5247 ArrowField::new(DIST_COL, DataType::Float32, true),
5248 ])
5249 .with_metadata([("dataset".into(), "vector".into())].into())
5250 );
5251
5252 let expected_i = BTreeSet::from_iter(vec![1, 81, 161, 241, 321]);
5253 let column_i = batch.column_by_name("i").unwrap();
5254 let actual_i: BTreeSet<i32> = as_primitive_array::<Int32Type>(column_i.as_ref())
5255 .values()
5256 .iter()
5257 .copied()
5258 .collect();
5259 assert_eq!(expected_i, actual_i);
5260 }
5261
5262 #[tokio::test]
5263 async fn test_binary_vectors_default_to_hamming() {
5264 let (_tmp_dir, dataset) = make_binary_vector_dataset().await.unwrap();
5265 let query = UInt8Array::from(vec![0b0000_1111u8, 0, 0, 0]);
5266
5267 let mut scan = dataset.scan();
5268 scan.nearest("bin", &query, 3).unwrap();
5269
5270 assert_eq!(scan.nearest.as_ref().unwrap().metric_type, None);
5272
5273 let batch = scan.try_into_batch().await.unwrap();
5274 let ids = batch
5275 .column_by_name("id")
5276 .unwrap()
5277 .as_primitive::<Int32Type>()
5278 .values();
5279 assert_eq!(ids, &[0, 1, 2]);
5280 let distances = batch
5281 .column_by_name(DIST_COL)
5282 .unwrap()
5283 .as_primitive::<Float32Type>()
5284 .values();
5285 assert_eq!(distances, &[0.0, 2.0, 4.0]);
5286 }
5287
5288 #[tokio::test]
5289 async fn test_binary_vectors_invalid_distance_error() {
5290 let (_tmp_dir, dataset) = make_binary_vector_dataset().await.unwrap();
5291 let query = UInt8Array::from(vec![0b0000_1111u8, 0, 0, 0]);
5292
5293 let mut scan = dataset.scan();
5294 scan.nearest("bin", &query, 1).unwrap();
5295 scan.distance_metric(DistanceType::L2);
5296
5297 let err = scan.try_into_batch().await.unwrap_err();
5298 assert!(matches!(err, Error::InvalidInput { .. }));
5299 let message = err.to_string();
5300 assert!(
5301 message.contains("l2") && message.contains("UInt8"),
5302 "unexpected message: {message}"
5303 );
5304 }
5305
5306 #[tokio::test]
5310 async fn test_knn_metric_mismatch_falls_back_to_flat_search() {
5311 let mut test_ds = TestVectorDataset::new(LanceFileVersion::Stable, true)
5312 .await
5313 .unwrap();
5314 test_ds.make_vector_index().await.unwrap();
5316
5317 let dataset = &test_ds.dataset;
5318 let key: Float32Array = (32..64).map(|v| v as f32).collect();
5319
5320 let mut scan = dataset.scan();
5322 scan.nearest("vec", &key, 5).unwrap();
5323 scan.distance_metric(DistanceType::Dot);
5324
5325 let plan = scan.explain_plan(false).await.unwrap();
5327 assert!(
5328 !plan.contains("ANNSubIndex"),
5329 "Expected flat search, but got ANN index in plan:\n{}",
5330 plan
5331 );
5332 assert!(
5334 plan.contains("KNNVectorDistance") && plan.to_lowercase().contains("dot"),
5335 "Expected flat KNN with Dot metric in plan:\n{}",
5336 plan
5337 );
5338
5339 let dot_batch = dataset
5341 .scan()
5342 .nearest("vec", &key, 5)
5343 .unwrap()
5344 .distance_metric(DistanceType::Dot)
5345 .try_into_batch()
5346 .await
5347 .unwrap();
5348
5349 let l2_batch = dataset
5350 .scan()
5351 .nearest("vec", &key, 5)
5352 .unwrap()
5353 .distance_metric(DistanceType::L2)
5354 .try_into_batch()
5355 .await
5356 .unwrap();
5357
5358 let dot_distances: Vec<f32> = dot_batch
5359 .column_by_name(DIST_COL)
5360 .unwrap()
5361 .as_primitive::<Float32Type>()
5362 .values()
5363 .to_vec();
5364 let l2_distances: Vec<f32> = l2_batch
5365 .column_by_name(DIST_COL)
5366 .unwrap()
5367 .as_primitive::<Float32Type>()
5368 .values()
5369 .to_vec();
5370
5371 assert_ne!(dot_distances, l2_distances);
5373 }
5374
5375 #[tokio::test]
5378 async fn test_knn_no_metric_uses_index_metric() {
5379 let mut test_ds = TestVectorDataset::new(LanceFileVersion::Stable, true)
5380 .await
5381 .unwrap();
5382 test_ds.make_vector_index().await.unwrap();
5384
5385 let dataset = &test_ds.dataset;
5386 let key: Float32Array = (32..64).map(|v| v as f32).collect();
5387
5388 let mut scan = dataset.scan();
5390 scan.nearest("vec", &key, 5).unwrap();
5391 let plan = scan.explain_plan(false).await.unwrap();
5395 assert!(
5396 plan.contains("ANNSubIndex") && plan.to_lowercase().contains("l2"),
5397 "Expected ANN index with L2 metric in plan:\n{}",
5398 plan
5399 );
5400 }
5401
5402 #[rstest]
5403 #[tokio::test]
5404 async fn test_only_row_id(
5405 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5406 data_storage_version: LanceFileVersion,
5407 ) {
5408 let test_ds = TestVectorDataset::new(data_storage_version, false)
5409 .await
5410 .unwrap();
5411 let dataset = &test_ds.dataset;
5412
5413 let mut scan = dataset.scan();
5414 scan.project::<&str>(&[]).unwrap().with_row_id();
5415
5416 let batch = scan.try_into_batch().await.unwrap();
5417
5418 assert_eq!(batch.num_columns(), 1);
5419 assert_eq!(batch.num_rows(), 400);
5420 let expected_schema =
5421 ArrowSchema::new(vec![ArrowField::new(ROW_ID, DataType::UInt64, true)])
5422 .with_metadata(dataset.schema().metadata.clone());
5423 assert_eq!(batch.schema().as_ref(), &expected_schema,);
5424
5425 let expected_row_ids: Vec<u64> = (0..200_u64).chain((1 << 32)..((1 << 32) + 200)).collect();
5426 let actual_row_ids: Vec<u64> = as_primitive_array::<UInt64Type>(batch.column(0).as_ref())
5427 .values()
5428 .iter()
5429 .copied()
5430 .collect();
5431 assert_eq!(expected_row_ids, actual_row_ids);
5432 }
5433
5434 #[tokio::test]
5435 async fn test_scan_unordered_with_row_id() {
5436 let test_ds = TestVectorDataset::new(LanceFileVersion::Legacy, false)
5438 .await
5439 .unwrap();
5440 let dataset = &test_ds.dataset;
5441
5442 let mut scan = dataset.scan();
5443 scan.with_row_id();
5444
5445 let ordered_batches = scan
5446 .try_into_stream()
5447 .await
5448 .unwrap()
5449 .try_collect::<Vec<RecordBatch>>()
5450 .await
5451 .unwrap();
5452 assert!(ordered_batches.len() > 2);
5453 let ordered_batch =
5454 concat_batches(&ordered_batches[0].schema(), ordered_batches.iter()).unwrap();
5455
5456 scan.scan_in_order(false);
5458 for _ in 0..10 {
5459 let unordered_batches = scan
5460 .try_into_stream()
5461 .await
5462 .unwrap()
5463 .try_collect::<Vec<RecordBatch>>()
5464 .await
5465 .unwrap();
5466 let unordered_batch =
5467 concat_batches(&unordered_batches[0].schema(), unordered_batches.iter()).unwrap();
5468
5469 assert_eq!(ordered_batch.num_rows(), unordered_batch.num_rows());
5470
5471 if ordered_batch != unordered_batch {
5473 let sort_indices = sort_to_indices(&unordered_batch[ROW_ID], None, None).unwrap();
5474
5475 let ordered_i = ordered_batch["i"].clone();
5476 let sorted_i = take::take(&unordered_batch["i"], &sort_indices, None).unwrap();
5477
5478 assert_eq!(&ordered_i, &sorted_i);
5479
5480 break;
5481 }
5482 }
5483 }
5484
5485 #[tokio::test]
5486 async fn test_scan_with_wildcard() {
5487 let data = gen_batch()
5488 .col("x", array::step::<Float64Type>())
5489 .col("y", array::step::<Float64Type>())
5490 .into_ram_dataset(FragmentCount::from(1), FragmentRowCount::from(100))
5491 .await
5492 .unwrap();
5493
5494 let check_cols = async |projection: &[&str], expected_cols: &[&str]| {
5495 let mut scan = data.scan();
5496 scan.project(projection).unwrap();
5497 let stream = scan.try_into_stream().await.unwrap();
5498 let schema = stream.schema();
5499 let field_names = schema.field_names();
5500 assert_eq!(field_names, expected_cols);
5501 };
5502
5503 check_cols(&["*"], &["x", "y"]).await;
5504 check_cols(&["x", "y"], &["x", "y"]).await;
5505 check_cols(&["x"], &["x"]).await;
5506 check_cols(&["_rowid", "*"], &["_rowid", "x", "y"]).await;
5507 check_cols(&["*", "_rowid"], &["x", "y", "_rowid"]).await;
5508 check_cols(
5509 &["_rowid", "*", "_rowoffset"],
5510 &["_rowid", "x", "y", "_rowoffset"],
5511 )
5512 .await;
5513
5514 let check_exprs = async |exprs: &[&str], expected_cols: &[&str]| {
5515 let mut scan = data.scan();
5516 let projection = exprs
5517 .iter()
5518 .map(|e| (e.to_string(), e.to_string()))
5519 .collect::<Vec<_>>();
5520 scan.project_with_transform(&projection).unwrap();
5521 let stream = scan.try_into_stream().await.unwrap();
5522 let schema = stream.schema();
5523 let field_names = schema.field_names();
5524 assert_eq!(field_names, expected_cols);
5525 };
5526
5527 check_exprs(&["_rowid", "*", "x * 2"], &["_rowid", "x", "y", "x * 2"]).await;
5529
5530 let check_fails = |projection: &[&str]| {
5531 let mut scan = data.scan();
5532 assert!(scan.project(projection).is_err());
5533 };
5534
5535 check_fails(&["x", "*"]);
5537 check_fails(&["_rowid", "_rowid"]);
5538 }
5539
5540 #[rstest]
5541 #[tokio::test]
5542 async fn test_scan_order(
5543 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5544 data_storage_version: LanceFileVersion,
5545 ) {
5546 let test_dir = TempStrDir::default();
5547 let test_uri = &test_dir;
5548
5549 let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
5550 "i",
5551 DataType::Int32,
5552 true,
5553 )]));
5554
5555 let batch1 = RecordBatch::try_new(
5556 schema.clone(),
5557 vec![Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5]))],
5558 )
5559 .unwrap();
5560
5561 let batch2 = RecordBatch::try_new(
5562 schema.clone(),
5563 vec![Arc::new(Int32Array::from(vec![6, 7, 8]))],
5564 )
5565 .unwrap();
5566
5567 let params = WriteParams {
5568 mode: WriteMode::Append,
5569 data_storage_version: Some(data_storage_version),
5570 ..Default::default()
5571 };
5572
5573 let write_batch = |batch: RecordBatch| async {
5574 let reader = RecordBatchIterator::new(vec![Ok(batch)], schema.clone());
5575 Dataset::write(reader, test_uri, Some(params)).await
5576 };
5577
5578 write_batch.clone()(batch1.clone()).await.unwrap();
5579 write_batch(batch2.clone()).await.unwrap();
5580
5581 let dataset = Arc::new(Dataset::open(test_uri).await.unwrap());
5582 let fragment1 = dataset.get_fragment(0).unwrap().metadata().clone();
5583 let fragment2 = dataset.get_fragment(1).unwrap().metadata().clone();
5584
5585 let mut scanner = dataset.scan();
5587 scanner.with_fragments(vec![fragment1.clone(), fragment2.clone()]);
5588 let output = scanner
5589 .try_into_stream()
5590 .await
5591 .unwrap()
5592 .try_collect::<Vec<_>>()
5593 .await
5594 .unwrap();
5595 assert_eq!(output.len(), 2);
5596 assert_eq!(output[0], batch1);
5597 assert_eq!(output[1], batch2);
5598
5599 let mut scanner = dataset.scan();
5601 scanner.with_fragments(vec![fragment2, fragment1]);
5602 let output = scanner
5603 .try_into_stream()
5604 .await
5605 .unwrap()
5606 .try_collect::<Vec<_>>()
5607 .await
5608 .unwrap();
5609 assert_eq!(output.len(), 2);
5610 assert_eq!(output[0], batch2);
5611 assert_eq!(output[1], batch1);
5612 }
5613
5614 #[rstest]
5615 #[tokio::test]
5616 async fn test_scan_sort(
5617 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5618 data_storage_version: LanceFileVersion,
5619 ) {
5620 let test_dir = TempStrDir::default();
5621 let test_uri = &test_dir;
5622
5623 let data = gen_batch()
5624 .col("int", array::cycle::<Int32Type>(vec![5, 4, 1, 2, 3]))
5625 .col(
5626 "str",
5627 array::cycle_utf8_literals(&["a", "b", "c", "e", "d"]),
5628 );
5629
5630 let sorted_by_int = gen_batch()
5631 .col("int", array::cycle::<Int32Type>(vec![1, 2, 3, 4, 5]))
5632 .col(
5633 "str",
5634 array::cycle_utf8_literals(&["c", "e", "d", "b", "a"]),
5635 )
5636 .into_batch_rows(RowCount::from(5))
5637 .unwrap();
5638
5639 let sorted_by_str = gen_batch()
5640 .col("int", array::cycle::<Int32Type>(vec![5, 4, 1, 3, 2]))
5641 .col(
5642 "str",
5643 array::cycle_utf8_literals(&["a", "b", "c", "d", "e"]),
5644 )
5645 .into_batch_rows(RowCount::from(5))
5646 .unwrap();
5647
5648 Dataset::write(
5649 data.into_reader_rows(RowCount::from(5), BatchCount::from(1)),
5650 test_uri,
5651 Some(WriteParams {
5652 data_storage_version: Some(data_storage_version),
5653 ..Default::default()
5654 }),
5655 )
5656 .await
5657 .unwrap();
5658
5659 let dataset = Arc::new(Dataset::open(test_uri).await.unwrap());
5660
5661 let batches_by_int = dataset
5662 .scan()
5663 .order_by(Some(vec![ColumnOrdering::asc_nulls_first(
5664 "int".to_string(),
5665 )]))
5666 .unwrap()
5667 .try_into_stream()
5668 .await
5669 .unwrap()
5670 .try_collect::<Vec<_>>()
5671 .await
5672 .unwrap();
5673
5674 assert_eq!(batches_by_int[0], sorted_by_int);
5675
5676 let batches_by_str = dataset
5677 .scan()
5678 .order_by(Some(vec![ColumnOrdering::asc_nulls_first(
5679 "str".to_string(),
5680 )]))
5681 .unwrap()
5682 .try_into_stream()
5683 .await
5684 .unwrap()
5685 .try_collect::<Vec<_>>()
5686 .await
5687 .unwrap();
5688
5689 assert_eq!(batches_by_str[0], sorted_by_str);
5690
5691 dataset
5693 .scan()
5694 .order_by(Some(vec![]))
5695 .unwrap()
5696 .try_into_stream()
5697 .await
5698 .unwrap()
5699 .try_collect::<Vec<_>>()
5700 .await
5701 .unwrap();
5702 }
5703
5704 #[rstest]
5705 #[tokio::test]
5706 async fn test_sort_multi_columns(
5707 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5708 data_storage_version: LanceFileVersion,
5709 ) {
5710 let test_dir = TempStrDir::default();
5711 let test_uri = &test_dir;
5712
5713 let data = gen_batch()
5714 .col("int", array::cycle::<Int32Type>(vec![5, 5, 1, 1, 3]))
5715 .col(
5716 "float",
5717 array::cycle::<Float32Type>(vec![7.3, -f32::NAN, f32::NAN, 4.3, f32::INFINITY]),
5718 );
5719
5720 let sorted_by_int_then_float = gen_batch()
5721 .col("int", array::cycle::<Int32Type>(vec![1, 1, 3, 5, 5]))
5722 .col(
5723 "float",
5724 array::cycle::<Float32Type>(vec![4.3, f32::NAN, f32::INFINITY, -f32::NAN, 7.3]),
5726 )
5727 .into_batch_rows(RowCount::from(5))
5728 .unwrap();
5729
5730 Dataset::write(
5731 data.into_reader_rows(RowCount::from(5), BatchCount::from(1)),
5732 test_uri,
5733 Some(WriteParams {
5734 data_storage_version: Some(data_storage_version),
5735 ..Default::default()
5736 }),
5737 )
5738 .await
5739 .unwrap();
5740
5741 let dataset = Arc::new(Dataset::open(test_uri).await.unwrap());
5742
5743 let batches_by_int_then_float = dataset
5744 .scan()
5745 .order_by(Some(vec![
5746 ColumnOrdering::asc_nulls_first("int".to_string()),
5747 ColumnOrdering::asc_nulls_first("float".to_string()),
5748 ]))
5749 .unwrap()
5750 .try_into_stream()
5751 .await
5752 .unwrap()
5753 .try_collect::<Vec<_>>()
5754 .await
5755 .unwrap();
5756
5757 assert_eq!(batches_by_int_then_float[0], sorted_by_int_then_float);
5758 }
5759
5760 #[rstest]
5761 #[tokio::test]
5762 async fn test_ann_prefilter(
5763 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5764 data_storage_version: LanceFileVersion,
5765 #[values(false, true)] stable_row_ids: bool,
5766 #[values(
5767 VectorIndexParams::ivf_pq(2, 8, 2, MetricType::L2, 2),
5768 VectorIndexParams::with_ivf_hnsw_sq_params(
5769 MetricType::L2,
5770 IvfBuildParams::new(2),
5771 HnswBuildParams::default(),
5772 SQBuildParams::default()
5773 )
5774 )]
5775 index_params: VectorIndexParams,
5776 ) {
5777 use lance_arrow::{fixed_size_list_type, FixedSizeListArrayExt};
5778
5779 let test_dir = TempStrDir::default();
5780 let test_uri = &test_dir;
5781
5782 let schema = Arc::new(ArrowSchema::new(vec![
5783 ArrowField::new("filterable", DataType::Int32, true),
5784 ArrowField::new("vector", fixed_size_list_type(2, DataType::Float32), true),
5785 ]));
5786
5787 let vector_values = Float32Array::from_iter_values((0..600).map(|x| x as f32));
5788
5789 let batches = vec![RecordBatch::try_new(
5790 schema.clone(),
5791 vec![
5792 Arc::new(Int32Array::from_iter_values(0..300)),
5793 Arc::new(FixedSizeListArray::try_new_from_values(vector_values, 2).unwrap()),
5794 ],
5795 )
5796 .unwrap()];
5797
5798 let write_params = WriteParams {
5799 data_storage_version: Some(data_storage_version),
5800 max_rows_per_file: 300, enable_stable_row_ids: stable_row_ids,
5802 ..Default::default()
5803 };
5804 let batches = RecordBatchIterator::new(batches.into_iter().map(Ok), schema.clone());
5805 let mut dataset = Dataset::write(batches, test_uri, Some(write_params))
5806 .await
5807 .unwrap();
5808
5809 dataset
5810 .create_index(&["vector"], IndexType::Vector, None, &index_params, false)
5811 .await
5812 .unwrap();
5813
5814 let query_key = Arc::new(Float32Array::from_iter_values((0..2).map(|x| x as f32)));
5815 let mut scan = dataset.scan();
5816 scan.filter("filterable > 5").unwrap();
5817 scan.nearest("vector", query_key.as_ref(), 1).unwrap();
5818 scan.minimum_nprobes(100);
5819 scan.with_row_id();
5820
5821 let batches = scan
5822 .try_into_stream()
5823 .await
5824 .unwrap()
5825 .try_collect::<Vec<_>>()
5826 .await
5827 .unwrap();
5828
5829 assert_eq!(batches.len(), 0);
5830
5831 scan.prefilter(true);
5832
5833 let batches = scan
5834 .try_into_stream()
5835 .await
5836 .unwrap()
5837 .try_collect::<Vec<_>>()
5838 .await
5839 .unwrap();
5840 assert_eq!(batches.len(), 1);
5841
5842 let first_match = batches[0][ROW_ID].as_primitive::<UInt64Type>().values()[0];
5843
5844 assert_eq!(6, first_match);
5845 }
5846
5847 #[rstest]
5848 #[tokio::test]
5849 async fn test_filter_on_large_utf8(
5850 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5851 data_storage_version: LanceFileVersion,
5852 ) {
5853 let test_dir = TempStrDir::default();
5854 let test_uri = &test_dir;
5855
5856 let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
5857 "ls",
5858 DataType::LargeUtf8,
5859 true,
5860 )]));
5861
5862 let batches = vec![RecordBatch::try_new(
5863 schema.clone(),
5864 vec![Arc::new(LargeStringArray::from_iter_values(
5865 (0..10).map(|v| format!("s-{}", v)),
5866 ))],
5867 )
5868 .unwrap()];
5869
5870 let write_params = WriteParams {
5871 data_storage_version: Some(data_storage_version),
5872 ..Default::default()
5873 };
5874 let batches = RecordBatchIterator::new(batches.into_iter().map(Ok), schema.clone());
5875 Dataset::write(batches, test_uri, Some(write_params))
5876 .await
5877 .unwrap();
5878
5879 let dataset = Dataset::open(test_uri).await.unwrap();
5880 let mut scan = dataset.scan();
5881 scan.filter("ls = 's-8'").unwrap();
5882
5883 let batches = scan
5884 .try_into_stream()
5885 .await
5886 .unwrap()
5887 .try_collect::<Vec<_>>()
5888 .await
5889 .unwrap();
5890 let batch = &batches[0];
5891
5892 let expected = RecordBatch::try_new(
5893 schema.clone(),
5894 vec![Arc::new(LargeStringArray::from_iter_values(
5895 (8..9).map(|v| format!("s-{}", v)),
5896 ))],
5897 )
5898 .unwrap();
5899
5900 assert_eq!(batch, &expected);
5901 }
5902
5903 #[rstest]
5904 #[tokio::test]
5905 async fn test_filter_with_regex(
5906 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
5907 data_storage_version: LanceFileVersion,
5908 ) {
5909 let test_dir = TempStrDir::default();
5910 let test_uri = &test_dir;
5911
5912 let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
5913 "ls",
5914 DataType::Utf8,
5915 true,
5916 )]));
5917
5918 let batches = vec![RecordBatch::try_new(
5919 schema.clone(),
5920 vec![Arc::new(StringArray::from_iter_values(
5921 (0..20).map(|v| format!("s-{}", v)),
5922 ))],
5923 )
5924 .unwrap()];
5925
5926 let write_params = WriteParams {
5927 data_storage_version: Some(data_storage_version),
5928 ..Default::default()
5929 };
5930 let batches = RecordBatchIterator::new(batches.into_iter().map(Ok), schema.clone());
5931 Dataset::write(batches, test_uri, Some(write_params))
5932 .await
5933 .unwrap();
5934
5935 let dataset = Dataset::open(test_uri).await.unwrap();
5936 let mut scan = dataset.scan();
5937 scan.filter("regexp_match(ls, 's-1.')").unwrap();
5938
5939 let stream = scan.try_into_stream().await.unwrap();
5940 let batches = stream.try_collect::<Vec<_>>().await.unwrap();
5941 let batch = &batches[0];
5942
5943 let expected = RecordBatch::try_new(
5944 schema.clone(),
5945 vec![Arc::new(StringArray::from_iter_values(
5946 (10..=19).map(|v| format!("s-{}", v)),
5947 ))],
5948 )
5949 .unwrap();
5950
5951 assert_eq!(batch, &expected);
5952 }
5953
5954 #[tokio::test]
5955 async fn test_filter_proj_bug() {
5956 let struct_i_field = ArrowField::new("i", DataType::Int32, true);
5957 let struct_o_field = ArrowField::new("o", DataType::Utf8, true);
5958 let schema = Arc::new(ArrowSchema::new(vec![
5959 ArrowField::new(
5960 "struct",
5961 DataType::Struct(vec![struct_i_field.clone(), struct_o_field.clone()].into()),
5962 true,
5963 ),
5964 ArrowField::new("s", DataType::Utf8, true),
5965 ]));
5966
5967 let input_batches: Vec<RecordBatch> = (0..5)
5968 .map(|i| {
5969 let struct_i_arr: Arc<Int32Array> =
5970 Arc::new(Int32Array::from_iter_values(i * 20..(i + 1) * 20));
5971 let struct_o_arr: Arc<StringArray> = Arc::new(StringArray::from_iter_values(
5972 (i * 20..(i + 1) * 20).map(|v| format!("o-{:02}", v)),
5973 ));
5974 RecordBatch::try_new(
5975 schema.clone(),
5976 vec![
5977 Arc::new(StructArray::from(vec![
5978 (Arc::new(struct_i_field.clone()), struct_i_arr as ArrayRef),
5979 (Arc::new(struct_o_field.clone()), struct_o_arr as ArrayRef),
5980 ])),
5981 Arc::new(StringArray::from_iter_values(
5982 (i * 20..(i + 1) * 20).map(|v| format!("s-{}", v)),
5983 )),
5984 ],
5985 )
5986 .unwrap()
5987 })
5988 .collect();
5989 let batches =
5990 RecordBatchIterator::new(input_batches.clone().into_iter().map(Ok), schema.clone());
5991 let test_dir = TempStrDir::default();
5992 let test_uri = &test_dir;
5993 let write_params = WriteParams {
5994 max_rows_per_file: 40,
5995 max_rows_per_group: 10,
5996 data_storage_version: Some(LanceFileVersion::Legacy),
5997 ..Default::default()
5998 };
5999 Dataset::write(batches, test_uri, Some(write_params))
6000 .await
6001 .unwrap();
6002
6003 let dataset = Dataset::open(test_uri).await.unwrap();
6004 let batches = dataset
6005 .scan()
6006 .filter("struct.i >= 20")
6007 .unwrap()
6008 .try_into_stream()
6009 .await
6010 .unwrap()
6011 .try_collect::<Vec<_>>()
6012 .await
6013 .unwrap();
6014 let batch = concat_batches(&batches[0].schema(), &batches).unwrap();
6015
6016 let expected_batch = concat_batches(&schema, &input_batches.as_slice()[1..]).unwrap();
6017 assert_eq!(batch, expected_batch);
6018
6019 let batches = dataset
6021 .scan()
6022 .filter("struct.o >= 'o-20'")
6023 .unwrap()
6024 .try_into_stream()
6025 .await
6026 .unwrap()
6027 .try_collect::<Vec<_>>()
6028 .await
6029 .unwrap();
6030 let batch = concat_batches(&batches[0].schema(), &batches).unwrap();
6031 assert_eq!(batch, expected_batch);
6032
6033 let batches = dataset
6035 .scan()
6036 .project(vec!["struct"].as_slice())
6037 .unwrap()
6038 .try_into_stream()
6039 .await
6040 .unwrap()
6041 .try_collect::<Vec<_>>()
6042 .await
6043 .unwrap();
6044 concat_batches(&batches[0].schema(), &batches).unwrap();
6045 }
6046
6047 #[rstest]
6048 #[tokio::test]
6049 async fn test_ann_with_deletion(
6050 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
6051 data_storage_version: LanceFileVersion,
6052 #[values(false, true)] stable_row_ids: bool,
6053 ) {
6054 let vec_params = vec![
6055 VectorIndexParams::ivf_pq(4, 8, 2, MetricType::L2, 2),
6058 ];
6059 for params in vec_params {
6060 use lance_arrow::FixedSizeListArrayExt;
6061
6062 let test_dir = TempStrDir::default();
6063 let test_uri = &test_dir;
6064
6065 let schema = Arc::new(ArrowSchema::new(vec![
6067 ArrowField::new("i", DataType::Int32, true),
6068 ArrowField::new(
6069 "vec",
6070 DataType::FixedSizeList(
6071 Arc::new(ArrowField::new("item", DataType::Float32, true)),
6072 32,
6073 ),
6074 true,
6075 ),
6076 ]));
6077
6078 let vector_values: Float32Array =
6080 (0..32 * 512).map(|v| (v / 32) as f32 + 1.0).collect();
6081 let vectors = FixedSizeListArray::try_new_from_values(vector_values, 32).unwrap();
6082
6083 let batches = vec![RecordBatch::try_new(
6084 schema.clone(),
6085 vec![
6086 Arc::new(Int32Array::from_iter_values(0..512)),
6087 Arc::new(vectors.clone()),
6088 ],
6089 )
6090 .unwrap()];
6091
6092 let reader = RecordBatchIterator::new(batches.into_iter().map(Ok), schema.clone());
6093 let mut dataset = Dataset::write(
6094 reader,
6095 test_uri,
6096 Some(WriteParams {
6097 data_storage_version: Some(data_storage_version),
6098 enable_stable_row_ids: stable_row_ids,
6099 ..Default::default()
6100 }),
6101 )
6102 .await
6103 .unwrap();
6104
6105 assert_eq!(dataset.index_cache_entry_count().await, 0);
6106 dataset
6107 .create_index(
6108 &["vec"],
6109 IndexType::Vector,
6110 Some("idx".to_string()),
6111 ¶ms,
6112 true,
6113 )
6114 .await
6115 .unwrap();
6116
6117 let mut scan = dataset.scan();
6118 let key: Float32Array = (0..32).map(|_v| 1.0_f32).collect();
6120 scan.nearest("vec", &key, 5).unwrap();
6121 scan.refine(100);
6122 scan.minimum_nprobes(100);
6123
6124 assert_eq!(
6125 dataset.index_cache_entry_count().await,
6126 2, );
6128 let results = scan
6129 .try_into_stream()
6130 .await
6131 .unwrap()
6132 .try_collect::<Vec<_>>()
6133 .await
6134 .unwrap();
6135
6136 assert_eq!(
6137 dataset.index_cache_entry_count().await,
6138 5 + dataset.versions().await.unwrap().len()
6139 );
6140 assert_eq!(results.len(), 1);
6141 let batch = &results[0];
6142
6143 let expected_i = BTreeSet::from_iter(vec![0, 1, 2, 3, 4]);
6144 let column_i = batch.column_by_name("i").unwrap();
6145 let actual_i: BTreeSet<i32> = as_primitive_array::<Int32Type>(column_i.as_ref())
6146 .values()
6147 .iter()
6148 .copied()
6149 .collect();
6150 assert_eq!(expected_i, actual_i);
6151
6152 dataset.delete("i = 1").await.unwrap();
6155 let mut scan = dataset.scan();
6156 scan.nearest("vec", &key, 5).unwrap();
6157 scan.refine(100);
6158 scan.minimum_nprobes(100);
6159
6160 let results = scan
6161 .try_into_stream()
6162 .await
6163 .unwrap()
6164 .try_collect::<Vec<_>>()
6165 .await
6166 .unwrap();
6167
6168 assert_eq!(results.len(), 1);
6169 let batch = &results[0];
6170
6171 let expected_i = BTreeSet::from_iter(vec![0, 2, 3, 4, 5]);
6173 let column_i = batch.column_by_name("i").unwrap();
6174 let actual_i: BTreeSet<i32> = as_primitive_array::<Int32Type>(column_i.as_ref())
6175 .values()
6176 .iter()
6177 .copied()
6178 .collect();
6179 assert_eq!(expected_i, actual_i);
6180
6181 let batches = vec![RecordBatch::try_new(
6184 schema.clone(),
6185 vec![
6186 Arc::new(Int32Array::from_iter_values(512..1024)),
6187 Arc::new(vectors),
6188 ],
6189 )
6190 .unwrap()];
6191
6192 let reader = RecordBatchIterator::new(batches.into_iter().map(Ok), schema.clone());
6193 let mut dataset = Dataset::write(
6194 reader,
6195 test_uri,
6196 Some(WriteParams {
6197 mode: WriteMode::Append,
6198 data_storage_version: Some(data_storage_version),
6199 ..Default::default()
6200 }),
6201 )
6202 .await
6203 .unwrap();
6204 dataset
6205 .create_index(
6206 &["vec"],
6207 IndexType::Vector,
6208 Some("idx".to_string()),
6209 ¶ms,
6210 true,
6211 )
6212 .await
6213 .unwrap();
6214
6215 dataset.delete("i < 512").await.unwrap();
6216
6217 let mut scan = dataset.scan();
6218 scan.nearest("vec", &key, 5).unwrap();
6219 scan.refine(100);
6220 scan.minimum_nprobes(100);
6221
6222 let results = scan
6223 .try_into_stream()
6224 .await
6225 .unwrap()
6226 .try_collect::<Vec<_>>()
6227 .await
6228 .unwrap();
6229
6230 assert_eq!(results.len(), 1);
6231 let batch = &results[0];
6232
6233 let expected_i = BTreeSet::from_iter(vec![512, 513, 514, 515, 516]);
6235 let column_i = batch.column_by_name("i").unwrap();
6236 let actual_i: BTreeSet<i32> = as_primitive_array::<Int32Type>(column_i.as_ref())
6237 .values()
6238 .iter()
6239 .copied()
6240 .collect();
6241 assert_eq!(expected_i, actual_i);
6242 }
6243 }
6244
6245 #[tokio::test]
6246 async fn test_projection_order() {
6247 let vec_params = VectorIndexParams::ivf_pq(4, 8, 2, MetricType::L2, 2);
6248 let mut data = gen_batch()
6249 .col("vec", array::rand_vec::<Float32Type>(Dimension::from(4)))
6250 .col("text", array::rand_utf8(ByteCount::from(10), false))
6251 .into_ram_dataset(FragmentCount::from(3), FragmentRowCount::from(100))
6252 .await
6253 .unwrap();
6254 data.create_index(&["vec"], IndexType::Vector, None, &vec_params, true)
6255 .await
6256 .unwrap();
6257
6258 let mut scan = data.scan();
6259 scan.nearest("vec", &Float32Array::from(vec![1.0, 1.0, 1.0, 1.0]), 5)
6260 .unwrap();
6261 scan.with_row_id().project(&["text"]).unwrap();
6262
6263 let results = scan
6264 .try_into_stream()
6265 .await
6266 .unwrap()
6267 .try_collect::<Vec<_>>()
6268 .await
6269 .unwrap();
6270
6271 assert_eq!(
6272 results[0].schema().field_names(),
6273 vec!["text", "_distance", "_rowid"]
6274 );
6275 }
6276
6277 #[rstest]
6278 #[tokio::test]
6279 async fn test_count_rows_with_filter(
6280 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
6281 data_storage_version: LanceFileVersion,
6282 ) {
6283 let test_dir = TempStrDir::default();
6284 let test_uri = &test_dir;
6285 let mut data_gen = BatchGenerator::new().col(Box::new(
6286 IncrementingInt32::new().named("Filter_me".to_owned()),
6287 ));
6288 Dataset::write(
6289 data_gen.batch(32),
6290 test_uri,
6291 Some(WriteParams {
6292 data_storage_version: Some(data_storage_version),
6293 ..Default::default()
6294 }),
6295 )
6296 .await
6297 .unwrap();
6298
6299 let dataset = Dataset::open(test_uri).await.unwrap();
6300 assert_eq!(32, dataset.count_rows(None).await.unwrap());
6301 assert_eq!(
6302 16,
6303 dataset
6304 .count_rows(Some("`Filter_me` > 15".to_string()))
6305 .await
6306 .unwrap()
6307 );
6308 }
6309
6310 #[rstest]
6311 #[tokio::test]
6312 async fn test_dynamic_projection(
6313 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
6314 data_storage_version: LanceFileVersion,
6315 ) {
6316 let test_dir = TempStrDir::default();
6317 let test_uri = &test_dir;
6318 let mut data_gen =
6319 BatchGenerator::new().col(Box::new(IncrementingInt32::new().named("i".to_owned())));
6320 Dataset::write(
6321 data_gen.batch(32),
6322 test_uri,
6323 Some(WriteParams {
6324 data_storage_version: Some(data_storage_version),
6325 ..Default::default()
6326 }),
6327 )
6328 .await
6329 .unwrap();
6330
6331 let dataset = Dataset::open(test_uri).await.unwrap();
6332 assert_eq!(dataset.count_rows(None).await.unwrap(), 32);
6333
6334 let mut scanner = dataset.scan();
6335
6336 let scan_res = scanner
6337 .project_with_transform(&[("bool", "i > 15")])
6338 .unwrap()
6339 .try_into_batch()
6340 .await
6341 .unwrap();
6342
6343 assert_eq!(1, scan_res.num_columns());
6344
6345 let bool_col = scan_res
6346 .column_by_name("bool")
6347 .expect("bool column should exist");
6348 let bool_arr = bool_col.as_boolean();
6349 for i in 0..32 {
6350 if i > 15 {
6351 assert!(bool_arr.value(i));
6352 } else {
6353 assert!(!bool_arr.value(i));
6354 }
6355 }
6356 }
6357
6358 #[rstest]
6359 #[tokio::test]
6360 async fn test_column_casting_function(
6361 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
6362 data_storage_version: LanceFileVersion,
6363 ) {
6364 let test_dir = TempStrDir::default();
6365 let test_uri = &test_dir;
6366 let mut data_gen =
6367 BatchGenerator::new().col(Box::new(RandomVector::new().named("vec".to_owned())));
6368 Dataset::write(
6369 data_gen.batch(32),
6370 test_uri,
6371 Some(WriteParams {
6372 data_storage_version: Some(data_storage_version),
6373 ..Default::default()
6374 }),
6375 )
6376 .await
6377 .unwrap();
6378
6379 let dataset = Dataset::open(test_uri).await.unwrap();
6380 assert_eq!(dataset.count_rows(None).await.unwrap(), 32);
6381
6382 let mut scanner = dataset.scan();
6383
6384 let scan_res = scanner
6385 .project_with_transform(&[("f16", "_cast_list_f16(vec)")])
6386 .unwrap()
6387 .try_into_batch()
6388 .await
6389 .unwrap();
6390
6391 assert_eq!(1, scan_res.num_columns());
6392 assert_eq!(32, scan_res.num_rows());
6393 assert_eq!("f16", scan_res.schema().field(0).name());
6394
6395 let mut scanner = dataset.scan();
6396 let scan_res_original = scanner
6397 .project(&["vec"])
6398 .unwrap()
6399 .try_into_batch()
6400 .await
6401 .unwrap();
6402
6403 let f32_col: &Float32Array = scan_res_original
6404 .column_by_name("vec")
6405 .unwrap()
6406 .as_fixed_size_list()
6407 .values()
6408 .as_primitive();
6409 let f16_col: &Float16Array = scan_res
6410 .column_by_name("f16")
6411 .unwrap()
6412 .as_fixed_size_list()
6413 .values()
6414 .as_primitive();
6415
6416 for (f32_val, f16_val) in f32_col.iter().zip(f16_col.iter()) {
6417 let f32_val = f32_val.unwrap();
6418 let f16_val = f16_val.unwrap();
6419 assert_eq!(f16::from_f32(f32_val), f16_val);
6420 }
6421 }
6422
6423 struct ScalarIndexTestFixture {
6424 _test_dir: TempStrDir,
6425 dataset: Dataset,
6426 sample_query: Arc<dyn Array>,
6427 delete_query: Arc<dyn Array>,
6428 original_version: u64,
6430 compact_version: u64,
6432 append_version: u64,
6434 updated_version: u64,
6436 delete_version: u64,
6438 append_then_delete_version: u64,
6440 }
6441
6442 #[derive(Debug, PartialEq)]
6443 struct ScalarTestParams {
6444 use_index: bool,
6445 use_projection: bool,
6446 use_deleted_data: bool,
6447 use_new_data: bool,
6448 with_row_id: bool,
6449 use_compaction: bool,
6450 use_updated: bool,
6451 }
6452
6453 impl ScalarIndexTestFixture {
6454 async fn new(data_storage_version: LanceFileVersion, use_stable_row_ids: bool) -> Self {
6455 let test_dir = TempStrDir::default();
6456 let test_uri = &test_dir;
6457
6458 let data = gen_batch()
6464 .col(
6465 "vector",
6466 array::rand_vec::<Float32Type>(Dimension::from(32)),
6467 )
6468 .col("indexed", array::step::<Int32Type>())
6469 .col("not_indexed", array::step::<Int32Type>())
6470 .into_batch_rows(RowCount::from(1000))
6471 .unwrap();
6472
6473 let mut dataset = Dataset::write(
6475 RecordBatchIterator::new(vec![Ok(data.clone())], data.schema().clone()),
6476 test_uri,
6477 Some(WriteParams {
6478 max_rows_per_file: 500,
6479 data_storage_version: Some(data_storage_version),
6480 enable_stable_row_ids: use_stable_row_ids,
6481 ..Default::default()
6482 }),
6483 )
6484 .await
6485 .unwrap();
6486
6487 dataset
6488 .create_index(
6489 &["vector"],
6490 IndexType::Vector,
6491 None,
6492 &VectorIndexParams::ivf_pq(2, 8, 2, MetricType::L2, 2),
6493 false,
6494 )
6495 .await
6496 .unwrap();
6497
6498 dataset
6499 .create_index(
6500 &["indexed"],
6501 IndexType::Scalar,
6502 None,
6503 &ScalarIndexParams::default(),
6504 false,
6505 )
6506 .await
6507 .unwrap();
6508
6509 let original_version = dataset.version().version;
6510 let sample_query = data["vector"].as_fixed_size_list().value(50);
6511 let delete_query = data["vector"].as_fixed_size_list().value(75);
6512
6513 let new_indexed =
6518 arrow_arith::numeric::add(&data["indexed"], &Int32Array::new_scalar(1000)).unwrap();
6519 let new_not_indexed =
6520 arrow_arith::numeric::add(&data["indexed"], &Int32Array::new_scalar(1000)).unwrap();
6521 let append_data = RecordBatch::try_new(
6522 data.schema(),
6523 vec![data["vector"].clone(), new_indexed, new_not_indexed],
6524 )
6525 .unwrap();
6526
6527 dataset
6528 .append(
6529 RecordBatchIterator::new(vec![Ok(append_data)], data.schema()),
6530 Some(WriteParams {
6531 data_storage_version: Some(data_storage_version),
6532 ..Default::default()
6533 }),
6534 )
6535 .await
6536 .unwrap();
6537
6538 let append_version = dataset.version().version;
6539
6540 dataset
6543 .optimize_indices(&OptimizeOptions::merge(1))
6544 .await
6545 .unwrap();
6546 let updated_version = dataset.version().version;
6547
6548 dataset.checkout_version(append_version).await.unwrap();
6551 dataset.restore().await.unwrap();
6552
6553 dataset.delete("not_indexed = 75").await.unwrap();
6554
6555 let append_then_delete_version = dataset.version().version;
6556
6557 let mut dataset = dataset.checkout_version(original_version).await.unwrap();
6560 dataset.restore().await.unwrap();
6561
6562 dataset.delete("not_indexed = 75").await.unwrap();
6563
6564 let delete_version = dataset.version().version;
6565
6566 compact_files(&mut dataset, CompactionOptions::default(), None)
6568 .await
6569 .unwrap();
6570 let compact_version = dataset.version().version;
6571 dataset.checkout_version(original_version).await.unwrap();
6572 dataset.restore().await.unwrap();
6573
6574 Self {
6575 _test_dir: test_dir,
6576 dataset,
6577 sample_query,
6578 delete_query,
6579 original_version,
6580 compact_version,
6581 append_version,
6582 updated_version,
6583 delete_version,
6584 append_then_delete_version,
6585 }
6586 }
6587
6588 fn sample_query(&self) -> &PrimitiveArray<Float32Type> {
6589 self.sample_query.as_primitive::<Float32Type>()
6590 }
6591
6592 fn delete_query(&self) -> &PrimitiveArray<Float32Type> {
6593 self.delete_query.as_primitive::<Float32Type>()
6594 }
6595
6596 async fn get_dataset(&self, params: &ScalarTestParams) -> Dataset {
6597 let version = if params.use_compaction {
6598 if params.use_deleted_data || params.use_new_data || params.use_updated {
6600 panic!(
6601 "There is no test data combining new/deleted/updated data with compaction"
6602 );
6603 } else {
6604 self.compact_version
6605 }
6606 } else if params.use_updated {
6607 if params.use_deleted_data || params.use_new_data || params.use_compaction {
6609 panic!(
6610 "There is no test data combining updated data with new/deleted/compaction"
6611 );
6612 } else {
6613 self.updated_version
6614 }
6615 } else {
6616 match (params.use_new_data, params.use_deleted_data) {
6617 (false, false) => self.original_version,
6618 (false, true) => self.delete_version,
6619 (true, false) => self.append_version,
6620 (true, true) => self.append_then_delete_version,
6621 }
6622 };
6623 self.dataset.checkout_version(version).await.unwrap()
6624 }
6625
6626 async fn run_query(
6627 &self,
6628 query: &str,
6629 vector: Option<&PrimitiveArray<Float32Type>>,
6630 params: &ScalarTestParams,
6631 ) -> (String, RecordBatch) {
6632 let dataset = self.get_dataset(params).await;
6633 let mut scan = dataset.scan();
6634 if let Some(vector) = vector {
6635 scan.nearest("vector", vector, 10).unwrap();
6636 }
6637 if params.use_projection {
6638 scan.project(&["indexed"]).unwrap();
6639 }
6640 if params.with_row_id {
6641 scan.with_row_id();
6642 }
6643 scan.scan_in_order(true);
6644 scan.use_index(params.use_index);
6645 scan.filter(query).unwrap();
6646 scan.prefilter(true);
6647
6648 let plan = scan.explain_plan(true).await.unwrap();
6649 let batch = scan.try_into_batch().await.unwrap();
6650
6651 if params.use_projection {
6652 let mut expected_columns = 1;
6654 if vector.is_some() {
6655 expected_columns += 1;
6657 }
6658 if params.with_row_id {
6659 expected_columns += 1;
6660 }
6661 assert_eq!(batch.num_columns(), expected_columns);
6662 } else {
6663 let mut expected_columns = 3;
6664 if vector.is_some() {
6665 expected_columns += 1;
6667 }
6668 if params.with_row_id {
6669 expected_columns += 1;
6670 }
6671 assert_eq!(batch.num_columns(), expected_columns);
6673 }
6674
6675 (plan, batch)
6676 }
6677
6678 fn assert_none<F: Fn(i32) -> bool>(
6679 &self,
6680 batch: &RecordBatch,
6681 predicate: F,
6682 message: &str,
6683 ) {
6684 let indexed = batch["indexed"].as_primitive::<Int32Type>();
6685 if indexed.iter().map(|val| val.unwrap()).any(predicate) {
6686 panic!("{}", message);
6687 }
6688 }
6689
6690 fn assert_one<F: Fn(i32) -> bool>(&self, batch: &RecordBatch, predicate: F, message: &str) {
6691 let indexed = batch["indexed"].as_primitive::<Int32Type>();
6692 if !indexed.iter().map(|val| val.unwrap()).any(predicate) {
6693 panic!("{}", message);
6694 }
6695 }
6696
6697 async fn check_vector_scalar_indexed_and_refine(&self, params: &ScalarTestParams) {
6698 let (query_plan, batch) = self
6699 .run_query(
6700 "indexed != 50 AND ((not_indexed < 100) OR (not_indexed >= 1000 AND not_indexed < 1100))",
6701 Some(self.sample_query()),
6702 params,
6703 )
6704 .await;
6705 if self.dataset.is_legacy_storage() {
6707 assert!(query_plan.contains("MaterializeIndex"));
6708 }
6709 self.assert_none(
6711 &batch,
6712 |val| val == 50,
6713 "The query contained 50 even though it was filtered",
6714 );
6715 if !params.use_new_data {
6716 self.assert_none(
6718 &batch,
6719 |val| (100..1000).contains(&val) || (val >= 1100),
6720 "The non-indexed refine filter was not applied",
6721 );
6722 }
6723
6724 if params.use_new_data || params.use_updated {
6726 self.assert_one(
6727 &batch,
6728 |val| val == 1050,
6729 "The query did not contain 1050 from the new data",
6730 );
6731 }
6732 }
6733
6734 async fn check_vector_scalar_indexed_only(&self, params: &ScalarTestParams) {
6735 let (query_plan, batch) = self
6736 .run_query("indexed != 50", Some(self.sample_query()), params)
6737 .await;
6738 if self.dataset.is_legacy_storage() {
6739 if params.use_index {
6740 assert!(query_plan.contains("ScalarIndexQuery"));
6743 } else {
6744 assert!(query_plan.contains("MaterializeIndex"));
6746 }
6747 }
6748 self.assert_none(
6750 &batch,
6751 |val| val == 50,
6752 "The query contained 50 even though it was filtered",
6753 );
6754 if params.use_new_data {
6756 self.assert_one(
6757 &batch,
6758 |val| val == 1050,
6759 "The query did not contain 1050 from the new data",
6760 );
6761 if !params.use_new_data {
6762 let (_, batch) = self
6764 .run_query("indexed == 1050", Some(self.sample_query()), params)
6765 .await;
6766 assert_eq!(batch.num_rows(), 1);
6767 }
6768 }
6769 if params.use_deleted_data {
6770 let (_, batch) = self
6771 .run_query("indexed == 75", Some(self.delete_query()), params)
6772 .await;
6773 if !params.use_new_data {
6774 assert_eq!(batch.num_rows(), 0);
6775 }
6776 }
6777 }
6778
6779 async fn check_vector_queries(&self, params: &ScalarTestParams) {
6780 self.check_vector_scalar_indexed_only(params).await;
6781 self.check_vector_scalar_indexed_and_refine(params).await;
6782 }
6783
6784 async fn check_simple_indexed_only(&self, params: &ScalarTestParams) {
6785 let (query_plan, batch) = self.run_query("indexed != 50", None, params).await;
6786 if self.dataset.is_legacy_storage() {
6788 assert!(query_plan.contains("MaterializeIndex"));
6789 } else {
6790 assert!(query_plan.contains("LanceRead"));
6791 }
6792 self.assert_none(
6794 &batch,
6795 |val| val == 50,
6796 "The query contained 50 even though it was filtered",
6797 );
6798 let mut expected_num_rows = if params.use_new_data || params.use_updated {
6799 1999
6800 } else {
6801 999
6802 };
6803 if params.use_deleted_data || params.use_compaction {
6804 expected_num_rows -= 1;
6805 }
6806 assert_eq!(batch.num_rows(), expected_num_rows);
6807
6808 if params.use_new_data || params.use_updated {
6810 let (_, batch) = self.run_query("indexed == 1050", None, params).await;
6811 assert_eq!(batch.num_rows(), 1);
6812 }
6813
6814 if params.use_deleted_data || params.use_compaction {
6816 let (_, batch) = self.run_query("indexed == 75", None, params).await;
6817 assert_eq!(batch.num_rows(), 0);
6818 }
6819 }
6820
6821 async fn check_simple_indexed_and_refine(&self, params: &ScalarTestParams) {
6822 let (query_plan, batch) = self.run_query(
6823 "indexed != 50 AND ((not_indexed < 100) OR (not_indexed >= 1000 AND not_indexed < 1100))",
6824 None,
6825 params
6826 ).await;
6827 if self.dataset.is_legacy_storage() {
6829 assert!(query_plan.contains("MaterializeIndex"));
6830 } else {
6831 assert!(query_plan.contains("LanceRead"));
6832 }
6833 self.assert_none(
6835 &batch,
6836 |val| val == 50,
6837 "The query contained 50 even though it was filtered",
6838 );
6839 self.assert_none(
6841 &batch,
6842 |val| (100..1000).contains(&val) || (val >= 1100),
6843 "The non-indexed refine filter was not applied",
6844 );
6845
6846 let mut expected_num_rows = if params.use_new_data || params.use_updated {
6847 199
6848 } else {
6849 99
6850 };
6851 if params.use_deleted_data || params.use_compaction {
6852 expected_num_rows -= 1;
6853 }
6854 assert_eq!(batch.num_rows(), expected_num_rows);
6855 }
6856
6857 async fn check_simple_queries(&self, params: &ScalarTestParams) {
6858 self.check_simple_indexed_only(params).await;
6859 self.check_simple_indexed_and_refine(params).await;
6860 }
6861 }
6862
6863 #[rstest]
6867 #[tokio::test]
6868 async fn test_secondary_index_scans(
6869 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
6870 data_storage_version: LanceFileVersion,
6871 #[values(false, true)] use_stable_row_ids: bool,
6872 ) {
6873 let fixture = Box::pin(ScalarIndexTestFixture::new(
6874 data_storage_version,
6875 use_stable_row_ids,
6876 ))
6877 .await;
6878
6879 for use_index in [false, true] {
6880 for use_projection in [false, true] {
6881 for use_deleted_data in [false, true] {
6882 for use_new_data in [false, true] {
6883 let compaction_choices =
6888 if use_deleted_data || use_new_data || use_stable_row_ids {
6889 vec![false]
6890 } else {
6891 vec![false, true]
6892 };
6893 for use_compaction in compaction_choices {
6894 let updated_choices =
6895 if use_deleted_data || use_new_data || use_compaction {
6896 vec![false]
6897 } else {
6898 vec![false, true]
6899 };
6900 for use_updated in updated_choices {
6901 for with_row_id in [false, true] {
6902 let params = ScalarTestParams {
6903 use_index,
6904 use_projection,
6905 use_deleted_data,
6906 use_new_data,
6907 with_row_id,
6908 use_compaction,
6909 use_updated,
6910 };
6911 fixture.check_vector_queries(¶ms).await;
6912 fixture.check_simple_queries(¶ms).await;
6913 }
6914 }
6915 }
6916 }
6917 }
6918 }
6919 }
6920 }
6921
6922 #[tokio::test]
6923 async fn can_filter_row_id() {
6924 let dataset = lance_datagen::gen_batch()
6925 .col("x", array::step::<Int32Type>())
6926 .into_ram_dataset(FragmentCount::from(1), FragmentRowCount::from(1000))
6927 .await
6928 .unwrap();
6929
6930 let mut scan = dataset.scan();
6931 scan.with_row_id();
6932 scan.project::<&str>(&[]).unwrap();
6933 scan.filter("_rowid == 50").unwrap();
6934 let batch = scan.try_into_batch().await.unwrap();
6935 assert_eq!(batch.num_rows(), 1);
6936 assert_eq!(batch.column(0).as_primitive::<UInt64Type>().values()[0], 50);
6937 }
6938
6939 #[rstest]
6940 #[tokio::test]
6941 async fn test_index_take_batch_size() {
6942 let fixture = Box::pin(ScalarIndexTestFixture::new(LanceFileVersion::Stable, false)).await;
6943 let stream = fixture
6944 .dataset
6945 .scan()
6946 .filter("indexed > 0")
6947 .unwrap()
6948 .batch_size(16)
6949 .try_into_stream()
6950 .await
6951 .unwrap();
6952 let batches = stream.collect::<Vec<_>>().await;
6953 assert_eq!(batches.len(), 1000_usize.div_ceil(16));
6954 }
6955
6956 async fn assert_plan_equals(
6960 dataset: &Dataset,
6961 plan: impl Fn(&mut Scanner) -> Result<&mut Scanner>,
6962 expected: &str,
6963 ) -> Result<()> {
6964 let mut scan = dataset.scan();
6965 plan(&mut scan)?;
6966 let exec_plan = scan.create_plan().await?;
6967 assert_plan_node_equals(exec_plan, expected).await
6968 }
6969
6970 #[tokio::test]
6971 async fn test_count_plan() {
6972 let dim = 256;
6974 let fixture = TestVectorDataset::new_with_dimension(LanceFileVersion::Stable, true, dim)
6975 .await
6976 .unwrap();
6977
6978 let err = fixture
6980 .dataset
6981 .scan()
6982 .create_count_plan()
6983 .await
6984 .unwrap_err();
6985 assert!(matches!(err, Error::InvalidInput { .. }));
6986
6987 let mut scan = fixture.dataset.scan();
6988 scan.project(&Vec::<String>::default()).unwrap();
6989
6990 let err = scan.create_count_plan().await.unwrap_err();
6992 assert!(matches!(err, Error::InvalidInput { .. }));
6993
6994 scan.with_row_id();
6995
6996 let plan = scan.create_count_plan().await.unwrap();
6997
6998 assert_plan_node_equals(
6999 plan,
7000 "AggregateExec: mode=Single, gby=[], aggr=[count_rows]
7001 LanceRead: uri=..., projection=[], num_fragments=2, range_before=None, range_after=None, row_id=true, row_addr=false, full_filter=--, refine_filter=--",
7002 )
7003 .await
7004 .unwrap();
7005
7006 scan.filter("s == ''").unwrap();
7007
7008 let plan = scan.create_count_plan().await.unwrap();
7009
7010 assert_plan_node_equals(
7011 plan,
7012 "AggregateExec: mode=Single, gby=[], aggr=[count_rows]
7013 ProjectionExec: expr=[_rowid@1 as _rowid]
7014 LanceRead: uri=..., projection=[s], num_fragments=2, range_before=None, range_after=None, row_id=true, row_addr=false, full_filter=s = Utf8(\"\"), refine_filter=s = Utf8(\"\")",
7015 )
7016 .await
7017 .unwrap();
7018 }
7019
7020 #[tokio::test]
7021 async fn test_inexact_scalar_index_plans() {
7022 let data = gen_batch()
7023 .col("ngram", array::rand_utf8(ByteCount::from(5), false))
7024 .col("exact", array::rand_type(&DataType::UInt32))
7025 .col("no_index", array::rand_type(&DataType::UInt32))
7026 .into_reader_rows(RowCount::from(1000), BatchCount::from(5));
7027
7028 let mut dataset = Dataset::write(data, "memory://test", None).await.unwrap();
7029 dataset
7030 .create_index(
7031 &["ngram"],
7032 IndexType::NGram,
7033 None,
7034 &ScalarIndexParams::default(),
7035 true,
7036 )
7037 .await
7038 .unwrap();
7039 dataset
7040 .create_index(
7041 &["exact"],
7042 IndexType::BTree,
7043 None,
7044 &ScalarIndexParams::default(),
7045 true,
7046 )
7047 .await
7048 .unwrap();
7049
7050 assert_plan_equals(
7052 &dataset,
7053 |scanner| scanner.filter("contains(ngram, 'test string')"),
7054 "LanceRead: uri=..., projection=[ngram, exact, no_index], num_fragments=1, \
7055 range_before=None, range_after=None, row_id=false, row_addr=false, \
7056 full_filter=contains(ngram, Utf8(\"test string\")), refine_filter=--
7057 ScalarIndexQuery: query=[contains(ngram, Utf8(\"test string\"))]@ngram_idx",
7058 )
7059 .await
7060 .unwrap();
7061
7062 assert_plan_equals(
7064 &dataset,
7065 |scanner| scanner.filter("contains(ngram, 'test string') and exact < 50"),
7066 "LanceRead: uri=..., projection=[ngram, exact, no_index], num_fragments=1, \
7067 range_before=None, range_after=None, row_id=false, row_addr=false, \
7068 full_filter=contains(ngram, Utf8(\"test string\")) AND exact < UInt32(50), \
7069 refine_filter=--
7070 ScalarIndexQuery: query=AND([contains(ngram, Utf8(\"test string\"))]@ngram_idx,[exact < 50]@exact_idx)",
7071 )
7072 .await
7073 .unwrap();
7074
7075 assert_plan_equals(
7077 &dataset,
7078 |scanner| {
7079 scanner.filter("contains(ngram, 'test string') and exact < 50 AND no_index > 100")
7080 },
7081 "ProjectionExec: expr=[ngram@0 as ngram, exact@1 as exact, no_index@2 as no_index]
7082 LanceRead: uri=..., projection=[ngram, exact, no_index], num_fragments=1, range_before=None, \
7083 range_after=None, row_id=true, row_addr=false, full_filter=contains(ngram, Utf8(\"test string\")) AND exact < UInt32(50) AND no_index > UInt32(100), \
7084 refine_filter=no_index > UInt32(100)
7085 ScalarIndexQuery: query=AND([contains(ngram, Utf8(\"test string\"))]@ngram_idx,[exact < 50]@exact_idx)",
7086 )
7087 .await
7088 .unwrap();
7089 }
7090
7091 #[rstest]
7092 #[tokio::test]
7093 async fn test_late_materialization(
7094 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
7095 data_storage_version: LanceFileVersion,
7096 ) {
7097 use lance_io::assert_io_lt;
7098 use lance_table::io::commit::RenameCommitHandler;
7101 let data = gen_batch()
7102 .col(
7103 "vector",
7104 array::rand_vec::<Float32Type>(Dimension::from(32)),
7105 )
7106 .col("indexed", array::step::<Int32Type>())
7107 .col("not_indexed", array::step::<Int32Type>())
7108 .into_reader_rows(RowCount::from(1000), BatchCount::from(20));
7109
7110 let mut dataset = Dataset::write(
7111 data,
7112 "memory://test",
7113 Some(WriteParams {
7114 commit_handler: Some(Arc::new(RenameCommitHandler)),
7115 data_storage_version: Some(data_storage_version),
7116 ..Default::default()
7117 }),
7118 )
7119 .await
7120 .unwrap();
7121 dataset
7122 .create_index(
7123 &["indexed"],
7124 IndexType::Scalar,
7125 None,
7126 &ScalarIndexParams::default(),
7127 false,
7128 )
7129 .await
7130 .unwrap();
7131
7132 let _ = dataset.object_store().io_stats_incremental(); dataset.scan().try_into_batch().await.unwrap();
7135 let io_stats = dataset.object_store().io_stats_incremental();
7136 let full_scan_bytes = io_stats.read_bytes;
7137
7138 dataset
7140 .scan()
7141 .use_stats(false)
7142 .filter("not_indexed = 50")
7143 .unwrap()
7144 .try_into_batch()
7145 .await
7146 .unwrap();
7147 let io_stats = dataset.object_store().io_stats_incremental();
7148 assert_io_lt!(io_stats, read_bytes, full_scan_bytes);
7149 let filtered_scan_bytes = io_stats.read_bytes;
7150
7151 if data_storage_version == LanceFileVersion::Legacy {
7154 dataset
7155 .scan()
7156 .filter("not_indexed = 50")
7157 .unwrap()
7158 .try_into_batch()
7159 .await
7160 .unwrap();
7161 let io_stats = dataset.object_store().io_stats_incremental();
7162 assert_io_lt!(io_stats, read_bytes, filtered_scan_bytes);
7163 }
7164
7165 dataset
7169 .scan()
7170 .filter("indexed = 50")
7171 .unwrap()
7172 .try_into_batch()
7173 .await
7174 .unwrap();
7175 let io_stats = dataset.object_store().io_stats_incremental();
7176 assert_io_lt!(io_stats, read_bytes, full_scan_bytes);
7177 let index_scan_bytes = io_stats.read_bytes;
7178
7179 dataset
7182 .scan()
7183 .filter("indexed = 50")
7184 .unwrap()
7185 .try_into_batch()
7186 .await
7187 .unwrap();
7188 let io_stats = dataset.object_store().io_stats_incremental();
7189 assert_io_lt!(io_stats, read_bytes, index_scan_bytes);
7190 }
7191
7192 #[rstest]
7193 #[tokio::test]
7194 async fn test_project_nested(
7195 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
7196 data_storage_version: LanceFileVersion,
7197 ) -> Result<()> {
7198 let struct_i_field = ArrowField::new("i", DataType::Int32, true);
7199 let struct_o_field = ArrowField::new("o", DataType::Utf8, true);
7200 let schema = Arc::new(ArrowSchema::new(vec![
7201 ArrowField::new(
7202 "struct",
7203 DataType::Struct(vec![struct_i_field.clone(), struct_o_field.clone()].into()),
7204 true,
7205 ),
7206 ArrowField::new("s", DataType::Utf8, true),
7207 ]));
7208
7209 let input_batches: Vec<RecordBatch> = (0..5)
7210 .map(|i| {
7211 let struct_i_arr: Arc<Int32Array> =
7212 Arc::new(Int32Array::from_iter_values(i * 20..(i + 1) * 20));
7213 let struct_o_arr: Arc<StringArray> = Arc::new(StringArray::from_iter_values(
7214 (i * 20..(i + 1) * 20).map(|v| format!("o-{:02}", v)),
7215 ));
7216 RecordBatch::try_new(
7217 schema.clone(),
7218 vec![
7219 Arc::new(StructArray::from(vec![
7220 (Arc::new(struct_i_field.clone()), struct_i_arr as ArrayRef),
7221 (Arc::new(struct_o_field.clone()), struct_o_arr as ArrayRef),
7222 ])),
7223 Arc::new(StringArray::from_iter_values(
7224 (i * 20..(i + 1) * 20).map(|v| format!("s-{}", v)),
7225 )),
7226 ],
7227 )
7228 .unwrap()
7229 })
7230 .collect();
7231 let batches =
7232 RecordBatchIterator::new(input_batches.clone().into_iter().map(Ok), schema.clone());
7233 let test_dir = TempStrDir::default();
7234 let test_uri = &test_dir;
7235 let write_params = WriteParams {
7236 max_rows_per_file: 40,
7237 max_rows_per_group: 10,
7238 data_storage_version: Some(data_storage_version),
7239 ..Default::default()
7240 };
7241 Dataset::write(batches, test_uri, Some(write_params))
7242 .await
7243 .unwrap();
7244
7245 let dataset = Dataset::open(test_uri).await.unwrap();
7246
7247 let batches = dataset
7248 .scan()
7249 .project(&["struct.i"])
7250 .unwrap()
7251 .try_into_stream()
7252 .await
7253 .unwrap()
7254 .try_collect::<Vec<_>>()
7255 .await
7256 .unwrap();
7257 let batch = concat_batches(&batches[0].schema(), &batches).unwrap();
7258 assert!(batch.column_by_name("struct.i").is_some());
7259 Ok(())
7260 }
7261
7262 #[rstest]
7263 #[tokio::test]
7264 async fn test_plans(
7265 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
7266 data_storage_version: LanceFileVersion,
7267 #[values(false, true)] stable_row_id: bool,
7268 ) -> Result<()> {
7269 use lance_index::scalar::inverted::query::BoostQuery;
7272 let dim = 256;
7273 let mut dataset =
7274 TestVectorDataset::new_with_dimension(data_storage_version, stable_row_id, dim).await?;
7275 let lance_schema = dataset.dataset.schema();
7276
7277 if data_storage_version == LanceFileVersion::Legacy {
7281 log::info!("Test case: Pushdown scan");
7282 assert_plan_equals(
7283 &dataset.dataset,
7284 |scan| scan.project(&["s"])?.filter("i > 10 and i < 20"),
7285 "LancePushdownScan: uri=..., projection=[s], predicate=i > Int32(10) AND i < Int32(20), row_id=false, row_addr=false, ordered=true"
7286 ).await?;
7287 }
7288
7289 log::info!("Test case: Project and filter");
7290 let expected = if data_storage_version == LanceFileVersion::Legacy {
7291 "ProjectionExec: expr=[s@2 as s]
7292 Take: columns=\"i, _rowid, (s)\"
7293 CoalesceBatchesExec: target_batch_size=8192
7294 FilterExec: i@0 > 10 AND i@0 < 20
7295 LanceScan: uri..., projection=[i], row_id=true, row_addr=false, ordered=true, range=None"
7296 } else {
7297 "ProjectionExec: expr=[s@2 as s]
7298 Take: columns=\"i, _rowid, (s)\"
7299 CoalesceBatchesExec: target_batch_size=8192
7300 LanceRead: ..., projection=[i], num_fragments=2, range_before=None, range_after=None, row_id=true, row_addr=false, full_filter=i > Int32(10) AND i < Int32(20), refine_filter=i > Int32(10) AND i < Int32(20)"
7301 };
7302 assert_plan_equals(
7303 &dataset.dataset,
7304 |scan| {
7305 scan.use_stats(false)
7306 .project(&["s"])?
7307 .filter("i > 10 and i < 20")
7308 },
7309 expected,
7310 )
7311 .await?;
7312
7313 log::info!("Test case: Late materialization");
7316 let expected = if data_storage_version == LanceFileVersion::Legacy {
7317 "ProjectionExec: expr=[i@0 as i, s@1 as s, vec@3 as vec]
7318 Take: columns=\"i, s, _rowid, (vec)\"
7319 CoalesceBatchesExec: target_batch_size=8192
7320 FilterExec: s@1 IS NOT NULL
7321 LanceScan: uri..., projection=[i, s], row_id=true, row_addr=false, ordered=true, range=None"
7322 } else {
7323 "ProjectionExec: expr=[i@0 as i, s@1 as s, vec@3 as vec]
7324 Take: columns=\"i, s, _rowid, (vec)\"
7325 CoalesceBatchesExec: target_batch_size=8192
7326 LanceRead: uri=..., projection=[i, s], num_fragments=2, range_before=None, range_after=None, \
7327 row_id=true, row_addr=false, full_filter=s IS NOT NULL, refine_filter=s IS NOT NULL"
7328 };
7329 assert_plan_equals(
7330 &dataset.dataset,
7331 |scan| scan.use_stats(false).filter("s IS NOT NULL"),
7332 expected,
7333 )
7334 .await?;
7335
7336 log::info!("Test case: Custom materialization (all early)");
7338 let expected = if data_storage_version == LanceFileVersion::Legacy {
7339 "ProjectionExec: expr=[i@0 as i, s@1 as s, vec@2 as vec]
7340 FilterExec: s@1 IS NOT NULL
7341 LanceScan: uri..., projection=[i, s, vec], row_id=true, row_addr=false, ordered=true, range=None"
7342 } else {
7343 "ProjectionExec: expr=[i@0 as i, s@1 as s, vec@2 as vec]
7344 LanceRead: uri=..., projection=[i, s, vec], num_fragments=2, range_before=None, \
7345 range_after=None, row_id=true, row_addr=false, full_filter=s IS NOT NULL, refine_filter=s IS NOT NULL"
7346 };
7347 assert_plan_equals(
7348 &dataset.dataset,
7349 |scan| {
7350 scan.use_stats(false)
7351 .materialization_style(MaterializationStyle::AllEarly)
7352 .filter("s IS NOT NULL")
7353 },
7354 expected,
7355 )
7356 .await?;
7357
7358 log::info!("Test case: Custom materialization 2 (all late)");
7359 let expected = if data_storage_version == LanceFileVersion::Legacy {
7360 "ProjectionExec: expr=[i@2 as i, s@0 as s, vec@3 as vec]
7361 Take: columns=\"s, _rowid, (i), (vec)\"
7362 CoalesceBatchesExec: target_batch_size=8192
7363 FilterExec: s@0 IS NOT NULL
7364 LanceScan: uri..., projection=[s], row_id=true, row_addr=false, ordered=true, range=None"
7365 } else {
7366 "ProjectionExec: expr=[i@2 as i, s@0 as s, vec@3 as vec]
7367 Take: columns=\"s, _rowid, (i), (vec)\"
7368 CoalesceBatchesExec: target_batch_size=8192
7369 LanceRead: uri=..., projection=[s], num_fragments=2, range_before=None, \
7370 range_after=None, row_id=true, row_addr=false, full_filter=s IS NOT NULL, refine_filter=s IS NOT NULL"
7371 };
7372 assert_plan_equals(
7373 &dataset.dataset,
7374 |scan| {
7375 scan.use_stats(false)
7376 .materialization_style(MaterializationStyle::AllLate)
7377 .filter("s IS NOT NULL")
7378 },
7379 expected,
7380 )
7381 .await?;
7382
7383 log::info!("Test case: Custom materialization 3 (mixed)");
7384 let expected = if data_storage_version == LanceFileVersion::Legacy {
7385 "ProjectionExec: expr=[i@3 as i, s@0 as s, vec@1 as vec]
7386 Take: columns=\"s, vec, _rowid, (i)\"
7387 CoalesceBatchesExec: target_batch_size=8192
7388 FilterExec: s@0 IS NOT NULL
7389 LanceScan: uri..., projection=[s, vec], row_id=true, row_addr=false, ordered=true, range=None"
7390 } else {
7391 "ProjectionExec: expr=[i@3 as i, s@0 as s, vec@1 as vec]
7392 Take: columns=\"s, vec, _rowid, (i)\"
7393 CoalesceBatchesExec: target_batch_size=8192
7394 LanceRead: uri=..., projection=[s, vec], num_fragments=2, range_before=None, range_after=None, \
7395 row_id=true, row_addr=false, full_filter=s IS NOT NULL, refine_filter=s IS NOT NULL"
7396 };
7397 assert_plan_equals(
7398 &dataset.dataset,
7399 |scan| {
7400 scan.use_stats(false)
7401 .materialization_style(
7402 MaterializationStyle::all_early_except(&["i"], lance_schema).unwrap(),
7403 )
7404 .filter("s IS NOT NULL")
7405 },
7406 expected,
7407 )
7408 .await?;
7409
7410 log::info!("Test case: Scan out of order");
7411 let expected = if data_storage_version == LanceFileVersion::Legacy {
7412 "LanceScan: uri=..., projection=[s], row_id=true, row_addr=false, ordered=false, range=None"
7413 } else {
7414 "LanceRead: uri=..., projection=[s], num_fragments=2, range_before=None, range_after=None, row_id=true, \
7415 row_addr=false, full_filter=--, refine_filter=--"
7416 };
7417 assert_plan_equals(
7418 &dataset.dataset,
7419 |scan| Ok(scan.project(&["s"])?.with_row_id().scan_in_order(false)),
7420 expected,
7421 )
7422 .await?;
7423
7424 let q: Float32Array = (32..32 + dim).map(|v| v as f32).collect();
7427 log::info!("Test case: Basic KNN");
7428 let expected = if data_storage_version == LanceFileVersion::Legacy {
7429 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@0 as vec, _distance@2 as _distance]
7430 Take: columns=\"vec, _rowid, _distance, (i), (s)\"
7431 CoalesceBatchesExec: target_batch_size=8192
7432 FilterExec: _distance@2 IS NOT NULL
7433 SortExec: TopK(fetch=5), expr=...
7434 KNNVectorDistance: metric=l2
7435 LanceScan: uri=..., projection=[vec], row_id=true, row_addr=false, ordered=false, range=None"
7436 } else {
7437 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@0 as vec, _distance@2 as _distance]
7438 Take: columns=\"vec, _rowid, _distance, (i), (s)\"
7439 CoalesceBatchesExec: target_batch_size=8192
7440 FilterExec: _distance@2 IS NOT NULL
7441 SortExec: TopK(fetch=5), expr=...
7442 KNNVectorDistance: metric=l2
7443 LanceRead: uri=..., projection=[vec], num_fragments=2, range_before=None, range_after=None, \
7444 row_id=true, row_addr=false, full_filter=--, refine_filter=--"
7445 };
7446 assert_plan_equals(
7447 &dataset.dataset,
7448 |scan| scan.nearest("vec", &q, 5),
7449 expected,
7450 )
7451 .await?;
7452
7453 let q: Float32Array = (32..32 + dim).map(|v| v as f32).collect();
7456 log::info!("Test case: KNN with extraneous limit");
7457 let expected = if data_storage_version == LanceFileVersion::Legacy {
7458 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@0 as vec, _distance@2 as _distance]
7459 Take: columns=\"vec, _rowid, _distance, (i), (s)\"
7460 CoalesceBatchesExec: target_batch_size=8192
7461 GlobalLimitExec: skip=0, fetch=1
7462 FilterExec: _distance@2 IS NOT NULL
7463 SortExec: TopK(fetch=5), expr=...
7464 KNNVectorDistance: metric=l2
7465 LanceScan: uri=..., projection=[vec], row_id=true, row_addr=false, ordered=false, range=None"
7466 } else {
7467 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@0 as vec, _distance@2 as _distance]
7468 Take: columns=\"vec, _rowid, _distance, (i), (s)\"
7469 CoalesceBatchesExec: target_batch_size=8192
7470 GlobalLimitExec: skip=0, fetch=1
7471 FilterExec: _distance@2 IS NOT NULL
7472 SortExec: TopK(fetch=5), expr=...
7473 KNNVectorDistance: metric=l2
7474 LanceRead: uri=..., projection=[vec], num_fragments=2, range_before=None, range_after=None, \
7475 row_id=true, row_addr=false, full_filter=--, refine_filter=--"
7476 };
7477 assert_plan_equals(
7478 &dataset.dataset,
7479 |scan| scan.nearest("vec", &q, 5)?.limit(Some(1), None),
7480 expected,
7481 )
7482 .await?;
7483
7484 dataset.make_vector_index().await?;
7487 log::info!("Test case: Basic ANN");
7488 let expected =
7489 "ProjectionExec: expr=[i@2 as i, s@3 as s, vec@4 as vec, _distance@0 as _distance]
7490 Take: columns=\"_distance, _rowid, (i), (s), (vec)\"
7491 CoalesceBatchesExec: target_batch_size=8192
7492 SortExec: TopK(fetch=42), expr=...
7493 ANNSubIndex: name=..., k=42, deltas=1, metric=L2
7494 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1";
7495 assert_plan_equals(
7496 &dataset.dataset,
7497 |scan| scan.nearest("vec", &q, 42),
7498 expected,
7499 )
7500 .await?;
7501
7502 log::info!("Test case: ANN with refine");
7503 let expected =
7504 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@1 as vec, _distance@2 as _distance]
7505 Take: columns=\"_rowid, vec, _distance, (i), (s)\"
7506 CoalesceBatchesExec: target_batch_size=8192
7507 FilterExec: _distance@... IS NOT NULL
7508 SortExec: TopK(fetch=10), expr=...
7509 KNNVectorDistance: metric=l2
7510 Take: columns=\"_distance, _rowid, (vec)\"
7511 CoalesceBatchesExec: target_batch_size=8192
7512 SortExec: TopK(fetch=40), expr=...
7513 ANNSubIndex: name=..., k=40, deltas=1, metric=L2
7514 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1";
7515 assert_plan_equals(
7516 &dataset.dataset,
7517 |scan| Ok(scan.nearest("vec", &q, 10)?.refine(4)),
7518 expected,
7519 )
7520 .await?;
7521
7522 log::info!("Test case: ANN with index disabled");
7524 let expected = if data_storage_version == LanceFileVersion::Legacy {
7525 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@0 as vec, _distance@2 as _distance]
7526 Take: columns=\"vec, _rowid, _distance, (i), (s)\"
7527 CoalesceBatchesExec: target_batch_size=8192
7528 FilterExec: _distance@... IS NOT NULL
7529 SortExec: TopK(fetch=13), expr=...
7530 KNNVectorDistance: metric=l2
7531 LanceScan: uri=..., projection=[vec], row_id=true, row_addr=false, ordered=false, range=None"
7532 } else {
7533 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@0 as vec, _distance@2 as _distance]
7534 Take: columns=\"vec, _rowid, _distance, (i), (s)\"
7535 CoalesceBatchesExec: target_batch_size=8192
7536 FilterExec: _distance@... IS NOT NULL
7537 SortExec: TopK(fetch=13), expr=...
7538 KNNVectorDistance: metric=l2
7539 LanceRead: uri=..., projection=[vec], num_fragments=2, range_before=None, range_after=None, \
7540 row_id=true, row_addr=false, full_filter=--, refine_filter=--"
7541 };
7542 assert_plan_equals(
7543 &dataset.dataset,
7544 |scan| Ok(scan.nearest("vec", &q, 13)?.use_index(false)),
7545 expected,
7546 )
7547 .await?;
7548
7549 log::info!("Test case: ANN with postfilter");
7550 let expected = "ProjectionExec: expr=[s@3 as s, vec@4 as vec, _distance@0 as _distance, _rowid@1 as _rowid]
7551 Take: columns=\"_distance, _rowid, i, (s), (vec)\"
7552 CoalesceBatchesExec: target_batch_size=8192
7553 FilterExec: i@2 > 10
7554 Take: columns=\"_distance, _rowid, (i)\"
7555 CoalesceBatchesExec: target_batch_size=8192
7556 SortExec: TopK(fetch=17), expr=...
7557 ANNSubIndex: name=..., k=17, deltas=1, metric=L2
7558 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1";
7559 assert_plan_equals(
7560 &dataset.dataset,
7561 |scan| {
7562 Ok(scan
7563 .nearest("vec", &q, 17)?
7564 .filter("i > 10")?
7565 .project(&["s", "vec"])?
7566 .with_row_id())
7567 },
7568 expected,
7569 )
7570 .await?;
7571
7572 log::info!("Test case: ANN with prefilter");
7573 let expected = if data_storage_version == LanceFileVersion::Legacy {
7574 "ProjectionExec: expr=[i@2 as i, s@3 as s, vec@4 as vec, _distance@0 as _distance]
7575 Take: columns=\"_distance, _rowid, (i), (s), (vec)\"
7576 CoalesceBatchesExec: target_batch_size=8192
7577 SortExec: TopK(fetch=17), expr=...
7578 ANNSubIndex: name=..., k=17, deltas=1, metric=L2
7579 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7580 FilterExec: i@0 > 10
7581 LanceScan: uri=..., projection=[i], row_id=true, row_addr=false, ordered=false, range=None"
7582 } else {
7583 "ProjectionExec: expr=[i@2 as i, s@3 as s, vec@4 as vec, _distance@0 as _distance]
7584 Take: columns=\"_distance, _rowid, (i), (s), (vec)\"
7585 CoalesceBatchesExec: target_batch_size=8192
7586 SortExec: TopK(fetch=17), expr=...
7587 ANNSubIndex: name=..., k=17, deltas=1, metric=L2
7588 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7589 LanceRead: uri=..., projection=[], num_fragments=2, range_before=None, range_after=None, \
7590 row_id=true, row_addr=false, full_filter=i > Int32(10), refine_filter=i > Int32(10)
7591"
7592 };
7593 assert_plan_equals(
7594 &dataset.dataset,
7595 |scan| {
7596 Ok(scan
7597 .nearest("vec", &q, 17)?
7598 .filter("i > 10")?
7599 .prefilter(true))
7600 },
7601 expected,
7602 )
7603 .await?;
7604
7605 dataset.append_new_data().await?;
7606 log::info!("Test case: Combined KNN/ANN");
7607 let expected = "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@1 as vec, _distance@2 as _distance]
7608 Take: columns=\"_rowid, vec, _distance, (i), (s)\"
7609 CoalesceBatchesExec: target_batch_size=8192
7610 FilterExec: _distance@... IS NOT NULL
7611 SortExec: TopK(fetch=6), expr=...
7612 KNNVectorDistance: metric=l2
7613 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7614 UnionExec
7615 ProjectionExec: expr=[_distance@2 as _distance, _rowid@1 as _rowid, vec@0 as vec]
7616 FilterExec: _distance@... IS NOT NULL
7617 SortExec: TopK(fetch=6), expr=...
7618 KNNVectorDistance: metric=l2
7619 LanceScan: uri=..., projection=[vec], row_id=true, row_addr=false, ordered=false, range=None
7620 Take: columns=\"_distance, _rowid, (vec)\"
7621 CoalesceBatchesExec: target_batch_size=8192
7622 SortExec: TopK(fetch=6), expr=...
7623 ANNSubIndex: name=..., k=6, deltas=1, metric=L2
7624 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1";
7625 assert_plan_equals(
7626 &dataset.dataset,
7627 |scan| scan.nearest("vec", &q, 6),
7628 expected,
7631 )
7632 .await?;
7633
7634 log::info!("Test case: Combined KNN/ANN with postfilter");
7636 let expected = "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@1 as vec, _distance@2 as _distance]
7637 Take: columns=\"_rowid, vec, _distance, i, (s)\"
7638 CoalesceBatchesExec: target_batch_size=8192
7639 FilterExec: i@3 > 10
7640 Take: columns=\"_rowid, vec, _distance, (i)\"
7641 CoalesceBatchesExec: target_batch_size=8192
7642 FilterExec: _distance@... IS NOT NULL
7643 SortExec: TopK(fetch=15), expr=...
7644 KNNVectorDistance: metric=l2
7645 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7646 UnionExec
7647 ProjectionExec: expr=[_distance@2 as _distance, _rowid@1 as _rowid, vec@0 as vec]
7648 FilterExec: _distance@... IS NOT NULL
7649 SortExec: TopK(fetch=15), expr=...
7650 KNNVectorDistance: metric=l2
7651 LanceScan: uri=..., projection=[vec], row_id=true, row_addr=false, ordered=false, range=None
7652 Take: columns=\"_distance, _rowid, (vec)\"
7653 CoalesceBatchesExec: target_batch_size=8192
7654 SortExec: TopK(fetch=15), expr=...
7655 ANNSubIndex: name=..., k=15, deltas=1, metric=L2
7656 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1";
7657 assert_plan_equals(
7658 &dataset.dataset,
7659 |scan| scan.nearest("vec", &q, 15)?.filter("i > 10"),
7660 expected,
7661 )
7662 .await?;
7663
7664 log::info!("Test case: Combined KNN/ANN with prefilter");
7666 let expected = if data_storage_version == LanceFileVersion::Legacy {
7667 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@1 as vec, _distance@2 as _distance]
7668 Take: columns=\"_rowid, vec, _distance, (i), (s)\"
7669 CoalesceBatchesExec: target_batch_size=8192
7670 FilterExec: _distance@... IS NOT NULL
7671 SortExec: TopK(fetch=5), expr=...
7672 KNNVectorDistance: metric=l2
7673 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7674 UnionExec
7675 ProjectionExec: expr=[_distance@3 as _distance, _rowid@2 as _rowid, vec@0 as vec]
7676 FilterExec: _distance@... IS NOT NULL
7677 SortExec: TopK(fetch=5), expr=...
7678 KNNVectorDistance: metric=l2
7679 FilterExec: i@1 > 10
7680 LanceScan: uri=..., projection=[vec, i], row_id=true, row_addr=false, ordered=false, range=None
7681 Take: columns=\"_distance, _rowid, (vec)\"
7682 CoalesceBatchesExec: target_batch_size=8192
7683 SortExec: TopK(fetch=5), expr=...
7684 ANNSubIndex: name=..., k=5, deltas=1, metric=L2
7685 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7686 FilterExec: i@0 > 10
7687 LanceScan: uri=..., projection=[i], row_id=true, row_addr=false, ordered=false, range=None"
7688 } else {
7689 "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@1 as vec, _distance@2 as _distance]
7690 Take: columns=\"_rowid, vec, _distance, (i), (s)\"
7691 CoalesceBatchesExec: target_batch_size=8192
7692 FilterExec: _distance@... IS NOT NULL
7693 SortExec: TopK(fetch=5), expr=...
7694 KNNVectorDistance: metric=l2
7695 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7696 UnionExec
7697 ProjectionExec: expr=[_distance@3 as _distance, _rowid@2 as _rowid, vec@0 as vec]
7698 FilterExec: _distance@... IS NOT NULL
7699 SortExec: TopK(fetch=5), expr=...
7700 KNNVectorDistance: metric=l2
7701 FilterExec: i@1 > 10
7702 LanceScan: uri=..., projection=[vec, i], row_id=true, row_addr=false, ordered=false, range=None
7703 Take: columns=\"_distance, _rowid, (vec)\"
7704 CoalesceBatchesExec: target_batch_size=8192
7705 SortExec: TopK(fetch=5), expr=...
7706 ANNSubIndex: name=..., k=5, deltas=1, metric=L2
7707 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7708 LanceRead: uri=..., projection=[], num_fragments=2, range_before=None, range_after=None, \
7709 row_id=true, row_addr=false, full_filter=i > Int32(10), refine_filter=i > Int32(10)"
7710 };
7711 assert_plan_equals(
7712 &dataset.dataset,
7713 |scan| {
7714 Ok(scan
7715 .nearest("vec", &q, 5)?
7716 .filter("i > 10")?
7717 .prefilter(true))
7718 },
7719 expected,
7722 )
7723 .await?;
7724
7725 dataset.make_vector_index().await?;
7729 dataset.make_scalar_index().await?;
7730
7731 log::info!("Test case: ANN with scalar index");
7732 let expected =
7733 "ProjectionExec: expr=[i@2 as i, s@3 as s, vec@4 as vec, _distance@0 as _distance]
7734 Take: columns=\"_distance, _rowid, (i), (s), (vec)\"
7735 CoalesceBatchesExec: target_batch_size=8192
7736 SortExec: TopK(fetch=5), expr=...
7737 ANNSubIndex: name=..., k=5, deltas=1, metric=L2
7738 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7739 ScalarIndexQuery: query=[i > 10]@i_idx";
7740 assert_plan_equals(
7741 &dataset.dataset,
7742 |scan| {
7743 Ok(scan
7744 .nearest("vec", &q, 5)?
7745 .filter("i > 10")?
7746 .prefilter(true))
7747 },
7748 expected,
7749 )
7750 .await?;
7751
7752 log::info!("Test case: ANN with scalar index disabled");
7753 let expected = if data_storage_version == LanceFileVersion::Legacy {
7754 "ProjectionExec: expr=[i@2 as i, s@3 as s, vec@4 as vec, _distance@0 as _distance]
7755 Take: columns=\"_distance, _rowid, (i), (s), (vec)\"
7756 CoalesceBatchesExec: target_batch_size=8192
7757 SortExec: TopK(fetch=5), expr=...
7758 ANNSubIndex: name=..., k=5, deltas=1, metric=L2
7759 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7760 FilterExec: i@0 > 10
7761 LanceScan: uri=..., projection=[i], row_id=true, row_addr=false, ordered=false, range=None"
7762 } else {
7763 "ProjectionExec: expr=[i@2 as i, s@3 as s, vec@4 as vec, _distance@0 as _distance]
7764 Take: columns=\"_distance, _rowid, (i), (s), (vec)\"
7765 CoalesceBatchesExec: target_batch_size=8192
7766 SortExec: TopK(fetch=5), expr=...
7767 ANNSubIndex: name=..., k=5, deltas=1, metric=L2
7768 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7769 LanceRead: uri=..., projection=[], num_fragments=3, range_before=None, \
7770 range_after=None, row_id=true, row_addr=false, full_filter=i > Int32(10), refine_filter=i > Int32(10)"
7771 };
7772 assert_plan_equals(
7773 &dataset.dataset,
7774 |scan| {
7775 Ok(scan
7776 .nearest("vec", &q, 5)?
7777 .use_scalar_index(false)
7778 .filter("i > 10")?
7779 .prefilter(true))
7780 },
7781 expected,
7782 )
7783 .await?;
7784
7785 dataset.append_new_data().await?;
7786
7787 log::info!("Test case: Combined KNN/ANN with scalar index");
7788 let expected = "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@1 as vec, _distance@2 as _distance]
7789 Take: columns=\"_rowid, vec, _distance, (i), (s)\"
7790 CoalesceBatchesExec: target_batch_size=8192
7791 FilterExec: _distance@... IS NOT NULL
7792 SortExec: TopK(fetch=8), expr=...
7793 KNNVectorDistance: metric=l2
7794 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7795 UnionExec
7796 ProjectionExec: expr=[_distance@3 as _distance, _rowid@2 as _rowid, vec@0 as vec]
7797 FilterExec: _distance@... IS NOT NULL
7798 SortExec: TopK(fetch=8), expr=...
7799 KNNVectorDistance: metric=l2
7800 FilterExec: i@1 > 10
7801 LanceScan: uri=..., projection=[vec, i], row_id=true, row_addr=false, ordered=false, range=None
7802 Take: columns=\"_distance, _rowid, (vec)\"
7803 CoalesceBatchesExec: target_batch_size=8192
7804 SortExec: TopK(fetch=8), expr=...
7805 ANNSubIndex: name=..., k=8, deltas=1, metric=L2
7806 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7807 ScalarIndexQuery: query=[i > 10]@i_idx";
7808 assert_plan_equals(
7809 &dataset.dataset,
7810 |scan| {
7811 Ok(scan
7812 .nearest("vec", &q, 8)?
7813 .filter("i > 10")?
7814 .prefilter(true))
7815 },
7816 expected,
7817 )
7818 .await?;
7819
7820 log::info!(
7822 "Test case: Combined KNN/ANN with updated scalar index and outdated vector index"
7823 );
7824 let expected = "ProjectionExec: expr=[i@3 as i, s@4 as s, vec@1 as vec, _distance@2 as _distance]
7825 Take: columns=\"_rowid, vec, _distance, (i), (s)\"
7826 CoalesceBatchesExec: target_batch_size=8192
7827 FilterExec: _distance@... IS NOT NULL
7828 SortExec: TopK(fetch=11), expr=...
7829 KNNVectorDistance: metric=l2
7830 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7831 UnionExec
7832 ProjectionExec: expr=[_distance@3 as _distance, _rowid@2 as _rowid, vec@0 as vec]
7833 FilterExec: _distance@... IS NOT NULL
7834 SortExec: TopK(fetch=11), expr=...
7835 KNNVectorDistance: metric=l2
7836 FilterExec: i@1 > 10
7837 LanceScan: uri=..., projection=[vec, i], row_id=true, row_addr=false, ordered=false, range=None
7838 Take: columns=\"_distance, _rowid, (vec)\"
7839 CoalesceBatchesExec: target_batch_size=8192
7840 SortExec: TopK(fetch=11), expr=...
7841 ANNSubIndex: name=..., k=11, deltas=1, metric=L2
7842 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1
7843 ScalarIndexQuery: query=[i > 10]@i_idx";
7844 dataset.make_scalar_index().await?;
7845 assert_plan_equals(
7846 &dataset.dataset,
7847 |scan| {
7848 Ok(scan
7849 .nearest("vec", &q, 11)?
7850 .filter("i > 10")?
7851 .prefilter(true))
7852 },
7853 expected,
7854 )
7855 .await?;
7856
7857 log::info!("Test case: Filtered read with scalar index");
7860 let expected = if data_storage_version == LanceFileVersion::Legacy {
7861 "ProjectionExec: expr=[s@1 as s]
7862 Take: columns=\"_rowid, (s)\"
7863 CoalesceBatchesExec: target_batch_size=8192
7864 MaterializeIndex: query=[i > 10]@i_idx"
7865 } else {
7866 "LanceRead: uri=..., projection=[s], num_fragments=4, range_before=None, \
7867 range_after=None, row_id=false, row_addr=false, full_filter=i > Int32(10), refine_filter=--
7868 ScalarIndexQuery: query=[i > 10]@i_idx"
7869 };
7870 assert_plan_equals(
7871 &dataset.dataset,
7872 |scan| scan.project(&["s"])?.filter("i > 10"),
7873 expected,
7874 )
7875 .await?;
7876
7877 if data_storage_version != LanceFileVersion::Legacy {
7878 log::info!(
7879 "Test case: Filtered read with scalar index disabled (late materialization)"
7880 );
7881 assert_plan_equals(
7882 &dataset.dataset,
7883 |scan| {
7884 scan.project(&["s"])?
7885 .use_scalar_index(false)
7886 .filter("i > 10")
7887 },
7888 "ProjectionExec: expr=[s@2 as s]
7889 Take: columns=\"i, _rowid, (s)\"
7890 CoalesceBatchesExec: target_batch_size=8192
7891 LanceRead: uri=..., projection=[i], num_fragments=4, range_before=None, \
7892 range_after=None, row_id=true, row_addr=false, full_filter=i > Int32(10), refine_filter=i > Int32(10)",
7893 )
7894 .await?;
7895 }
7896
7897 log::info!("Test case: Empty projection");
7898 let expected = if data_storage_version == LanceFileVersion::Legacy {
7899 "ProjectionExec: expr=[_rowaddr@0 as _rowaddr]
7900 AddRowAddrExec
7901 MaterializeIndex: query=[i > 10]@i_idx"
7902 } else {
7903 "LanceRead: uri=..., projection=[], num_fragments=4, range_before=None, \
7904 range_after=None, row_id=false, row_addr=true, full_filter=i > Int32(10), refine_filter=--
7905 ScalarIndexQuery: query=[i > 10]@i_idx"
7906 };
7907 assert_plan_equals(
7908 &dataset.dataset,
7909 |scan| {
7910 scan.filter("i > 10")
7911 .unwrap()
7912 .with_row_address()
7913 .project::<&str>(&[])
7914 },
7915 expected,
7916 )
7917 .await?;
7918
7919 dataset.append_new_data().await?;
7920 log::info!("Test case: Combined Scalar/non-scalar filtered read");
7921 let expected = if data_storage_version == LanceFileVersion::Legacy {
7922 "ProjectionExec: expr=[s@1 as s]
7923 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7924 UnionExec
7925 Take: columns=\"_rowid, (s)\"
7926 CoalesceBatchesExec: target_batch_size=8192
7927 MaterializeIndex: query=[i > 10]@i_idx
7928 ProjectionExec: expr=[_rowid@2 as _rowid, s@1 as s]
7929 FilterExec: i@0 > 10
7930 LanceScan: uri=..., projection=[i, s], row_id=true, row_addr=false, ordered=false, range=None"
7931 } else {
7932 "LanceRead: uri=..., projection=[s], num_fragments=5, range_before=None, \
7933 range_after=None, row_id=false, row_addr=false, full_filter=i > Int32(10), refine_filter=--
7934 ScalarIndexQuery: query=[i > 10]@i_idx"
7935 };
7936 assert_plan_equals(
7937 &dataset.dataset,
7938 |scan| scan.project(&["s"])?.filter("i > 10"),
7939 expected,
7940 )
7941 .await?;
7942
7943 log::info!("Test case: Combined Scalar/non-scalar filtered read with empty projection");
7944 let expected = if data_storage_version == LanceFileVersion::Legacy {
7945 "ProjectionExec: expr=[_rowaddr@0 as _rowaddr]
7946 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7947 UnionExec
7948 AddRowAddrExec
7949 MaterializeIndex: query=[i > 10]@i_idx
7950 ProjectionExec: expr=[_rowaddr@2 as _rowaddr, _rowid@1 as _rowid]
7951 FilterExec: i@0 > 10
7952 LanceScan: uri=..., projection=[i], row_id=true, row_addr=true, ordered=false, range=None"
7953 } else {
7954 "LanceRead: uri=..., projection=[], num_fragments=5, range_before=None, \
7955 range_after=None, row_id=false, row_addr=true, full_filter=i > Int32(10), refine_filter=--
7956 ScalarIndexQuery: query=[i > 10]@i_idx"
7957 };
7958 assert_plan_equals(
7959 &dataset.dataset,
7960 |scan| {
7961 scan.filter("i > 10")
7962 .unwrap()
7963 .with_row_address()
7964 .project::<&str>(&[])
7965 },
7966 expected,
7967 )
7968 .await?;
7969
7970 log::info!("Test case: Dynamic projection");
7973 let expected = if data_storage_version == LanceFileVersion::Legacy {
7974 "ProjectionExec: expr=[regexp_match(s@1, .*) as matches]
7975 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
7976 UnionExec
7977 Take: columns=\"_rowid, (s)\"
7978 CoalesceBatchesExec: target_batch_size=8192
7979 MaterializeIndex: query=[i > 10]@i_idx
7980 ProjectionExec: expr=[_rowid@2 as _rowid, s@1 as s]
7981 FilterExec: i@0 > 10
7982 LanceScan: uri=..., row_id=true, row_addr=false, ordered=false, range=None"
7983 } else {
7984 "ProjectionExec: expr=[regexp_match(s@0, .*) as matches]
7985 LanceRead: uri=..., projection=[s], num_fragments=5, range_before=None, \
7986 range_after=None, row_id=false, row_addr=false, full_filter=i > Int32(10), refine_filter=--
7987 ScalarIndexQuery: query=[i > 10]@i_idx"
7988 };
7989 assert_plan_equals(
7990 &dataset.dataset,
7991 |scan| {
7992 scan.project_with_transform(&[("matches", "regexp_match(s, \".*\")")])?
7993 .filter("i > 10")
7994 },
7995 expected,
7996 )
7997 .await?;
7998
7999 dataset.make_fts_index().await?;
8003 log::info!("Test case: Full text search (match query)");
8004 let expected = r#"ProjectionExec: expr=[s@2 as s, _score@1 as _score, _rowid@0 as _rowid]
8005 Take: columns="_rowid, _score, (s)"
8006 CoalesceBatchesExec: target_batch_size=8192
8007 MatchQuery: column=s, query=hello"#;
8008 assert_plan_equals(
8009 &dataset.dataset,
8010 |scan| {
8011 scan.project(&["s"])?
8012 .with_row_id()
8013 .full_text_search(FullTextSearchQuery::new("hello".to_owned()))
8014 },
8015 expected,
8016 )
8017 .await?;
8018
8019 log::info!("Test case: Full text search (phrase query)");
8020 let expected = r#"ProjectionExec: expr=[s@2 as s, _score@1 as _score, _rowid@0 as _rowid]
8021 Take: columns="_rowid, _score, (s)"
8022 CoalesceBatchesExec: target_batch_size=8192
8023 PhraseQuery: column=s, query=hello world"#;
8024 assert_plan_equals(
8025 &dataset.dataset,
8026 |scan| {
8027 let query = PhraseQuery::new("hello world".to_owned());
8028 scan.project(&["s"])?
8029 .with_row_id()
8030 .full_text_search(FullTextSearchQuery::new_query(query.into()))
8031 },
8032 expected,
8033 )
8034 .await?;
8035
8036 log::info!("Test case: Full text search (boost query)");
8037 let expected = r#"ProjectionExec: expr=[s@2 as s, _score@1 as _score, _rowid@0 as _rowid]
8038 Take: columns="_rowid, _score, (s)"
8039 CoalesceBatchesExec: target_batch_size=8192
8040 BoostQuery: negative_boost=1
8041 MatchQuery: column=s, query=hello
8042 MatchQuery: column=s, query=world"#;
8043 assert_plan_equals(
8044 &dataset.dataset,
8045 |scan| {
8046 let positive =
8047 MatchQuery::new("hello".to_owned()).with_column(Some("s".to_owned()));
8048 let negative =
8049 MatchQuery::new("world".to_owned()).with_column(Some("s".to_owned()));
8050 let query = BoostQuery::new(positive.into(), negative.into(), Some(1.0));
8051 scan.project(&["s"])?
8052 .with_row_id()
8053 .full_text_search(FullTextSearchQuery::new_query(query.into()))
8054 },
8055 expected,
8056 )
8057 .await?;
8058
8059 log::info!("Test case: Full text search with prefilter");
8060 let expected = if data_storage_version == LanceFileVersion::Legacy {
8061 r#"ProjectionExec: expr=[s@2 as s, _score@1 as _score, _rowid@0 as _rowid]
8062 Take: columns="_rowid, _score, (s)"
8063 CoalesceBatchesExec: target_batch_size=8192
8064 MatchQuery: column=s, query=hello
8065 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
8066 UnionExec
8067 MaterializeIndex: query=[i > 10]@i_idx
8068 ProjectionExec: expr=[_rowid@1 as _rowid]
8069 FilterExec: i@0 > 10
8070 LanceScan: uri=..., projection=[i], row_id=true, row_addr=false, ordered=false, range=None"#
8071 } else {
8072 r#"ProjectionExec: expr=[s@2 as s, _score@1 as _score, _rowid@0 as _rowid]
8073 Take: columns="_rowid, _score, (s)"
8074 CoalesceBatchesExec: target_batch_size=8192
8075 MatchQuery: column=s, query=hello
8076 LanceRead: uri=..., projection=[], num_fragments=5, range_before=None, range_after=None, row_id=true, row_addr=false, full_filter=i > Int32(10), refine_filter=--
8077 ScalarIndexQuery: query=[i > 10]@i_idx"#
8078 };
8079 assert_plan_equals(
8080 &dataset.dataset,
8081 |scan| {
8082 scan.project(&["s"])?
8083 .with_row_id()
8084 .filter("i > 10")?
8085 .prefilter(true)
8086 .full_text_search(FullTextSearchQuery::new("hello".to_owned()))
8087 },
8088 expected,
8089 )
8090 .await?;
8091
8092 log::info!("Test case: Full text search with unindexed rows");
8093 let expected = r#"ProjectionExec: expr=[s@2 as s, _score@1 as _score, _rowid@0 as _rowid]
8094 Take: columns="_rowid, _score, (s)"
8095 CoalesceBatchesExec: target_batch_size=8192
8096 SortExec: expr=[_score@1 DESC NULLS LAST], preserve_partitioning=[false]
8097 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
8098 UnionExec
8099 MatchQuery: column=s, query=hello
8100 FlatMatchQuery: column=s, query=hello
8101 LanceScan: uri=..., projection=[s], row_id=true, row_addr=false, ordered=false, range=None"#;
8102 dataset.append_new_data().await?;
8103 assert_plan_equals(
8104 &dataset.dataset,
8105 |scan| {
8106 scan.project(&["s"])?
8107 .with_row_id()
8108 .full_text_search(FullTextSearchQuery::new("hello".to_owned()))
8109 },
8110 expected,
8111 )
8112 .await?;
8113
8114 log::info!("Test case: Full text search with unindexed rows and prefilter");
8115 let expected = if data_storage_version == LanceFileVersion::Legacy {
8116 r#"ProjectionExec: expr=[s@2 as s, _score@1 as _score, _rowid@0 as _rowid]
8117 Take: columns="_rowid, _score, (s)"
8118 CoalesceBatchesExec: target_batch_size=8192
8119 SortExec: expr=[_score@1 DESC NULLS LAST], preserve_partitioning=[false]
8120 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
8121 UnionExec
8122 MatchQuery: column=s, query=hello
8123 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
8124 UnionExec
8125 MaterializeIndex: query=[i > 10]@i_idx
8126 ProjectionExec: expr=[_rowid@1 as _rowid]
8127 FilterExec: i@0 > 10
8128 LanceScan: uri=..., projection=[i], row_id=true, row_addr=false, ordered=false, range=None
8129 FlatMatchQuery: column=s, query=hello
8130 FilterExec: i@1 > 10
8131 LanceScan: uri=..., projection=[s, i], row_id=true, row_addr=false, ordered=false, range=None"#
8132 } else {
8133 r#"ProjectionExec: expr=[s@2 as s, _score@1 as _score, _rowid@0 as _rowid]
8134 Take: columns="_rowid, _score, (s)"
8135 CoalesceBatchesExec: target_batch_size=8192
8136 SortExec: expr=[_score@1 DESC NULLS LAST], preserve_partitioning=[false]
8137 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
8138 UnionExec
8139 MatchQuery: column=s, query=hello
8140 LanceRead: uri=..., projection=[], num_fragments=5, range_before=None, range_after=None, row_id=true, row_addr=false, full_filter=i > Int32(10), refine_filter=--
8141 ScalarIndexQuery: query=[i > 10]@i_idx
8142 FlatMatchQuery: column=s, query=hello
8143 FilterExec: i@1 > 10
8144 LanceScan: uri=..., projection=[s, i], row_id=true, row_addr=false, ordered=false, range=None"#
8145 };
8146 assert_plan_equals(
8147 &dataset.dataset,
8148 |scan| {
8149 scan.project(&["s"])?
8150 .with_row_id()
8151 .filter("i > 10")?
8152 .prefilter(true)
8153 .full_text_search(FullTextSearchQuery::new("hello".to_owned()))
8154 },
8155 expected,
8156 )
8157 .await?;
8158
8159 Ok(())
8160 }
8161
8162 #[tokio::test]
8163 async fn test_fast_search_plan() {
8164 let mut dataset = TestVectorDataset::new(LanceFileVersion::Stable, true)
8166 .await
8167 .unwrap();
8168 dataset.make_vector_index().await.unwrap();
8169 dataset.append_new_data().await.unwrap();
8170
8171 let q: Float32Array = (32..64).map(|v| v as f32).collect();
8172
8173 assert_plan_equals(
8174 &dataset.dataset,
8175 |scan| {
8176 scan.nearest("vec", &q, 32)?
8177 .fast_search()
8178 .project(&["_distance", "_rowid"])
8179 },
8180 "SortExec: TopK(fetch=32), expr=[_distance@0 ASC NULLS LAST, _rowid@1 ASC NULLS LAST]...
8181 ANNSubIndex: name=idx, k=32, deltas=1, metric=L2
8182 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1",
8183 )
8184 .await
8185 .unwrap();
8186
8187 assert_plan_equals(
8188 &dataset.dataset,
8189 |scan| {
8190 scan.nearest("vec", &q, 33)?
8191 .fast_search()
8192 .with_row_id()
8193 .project(&["_distance", "_rowid"])
8194 },
8195 "SortExec: TopK(fetch=33), expr=[_distance@0 ASC NULLS LAST, _rowid@1 ASC NULLS LAST]...
8196 ANNSubIndex: name=idx, k=33, deltas=1, metric=L2
8197 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1",
8198 )
8199 .await
8200 .unwrap();
8201
8202 assert_plan_equals(
8204 &dataset.dataset,
8205 |scan| {
8206 scan.nearest("vec", &q, 34)?
8207 .with_row_id()
8208 .project(&["_distance", "_rowid"])
8209 },
8210 "ProjectionExec: expr=[_distance@2 as _distance, _rowid@0 as _rowid]
8211 FilterExec: _distance@2 IS NOT NULL
8212 SortExec: TopK(fetch=34), expr=[_distance@2 ASC NULLS LAST, _rowid@0 ASC NULLS LAST]...
8213 KNNVectorDistance: metric=l2
8214 RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=2
8215 UnionExec
8216 ProjectionExec: expr=[_distance@2 as _distance, _rowid@1 as _rowid, vec@0 as vec]
8217 FilterExec: _distance@2 IS NOT NULL
8218 SortExec: TopK(fetch=34), expr=[_distance@2 ASC NULLS LAST, _rowid@1 ASC NULLS LAST]...
8219 KNNVectorDistance: metric=l2
8220 LanceScan: uri=..., projection=[vec], row_id=true, row_addr=false, ordered=false, range=None
8221 Take: columns=\"_distance, _rowid, (vec)\"
8222 CoalesceBatchesExec: target_batch_size=8192
8223 SortExec: TopK(fetch=34), expr=[_distance@0 ASC NULLS LAST, _rowid@1 ASC NULLS LAST]...
8224 ANNSubIndex: name=idx, k=34, deltas=1, metric=L2
8225 ANNIvfPartition: uuid=..., minimum_nprobes=1, maximum_nprobes=None, deltas=1",
8226 )
8227 .await
8228 .unwrap();
8229 }
8230
8231 #[rstest]
8232 #[tokio::test]
8233 pub async fn test_scan_planning_io(
8234 #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
8235 data_storage_version: LanceFileVersion,
8236 ) {
8237 use lance_index::scalar::inverted::tokenizer::InvertedIndexParams;
8241 use lance_io::assert_io_eq;
8242 let data = gen_batch()
8243 .col(
8244 "vector",
8245 array::rand_vec::<Float32Type>(Dimension::from(32)),
8246 )
8247 .col("text", array::rand_utf8(ByteCount::from(4), false))
8248 .col("indexed", array::step::<Int32Type>())
8249 .col("not_indexed", array::step::<Int32Type>())
8250 .into_reader_rows(RowCount::from(100), BatchCount::from(5));
8251
8252 let mut dataset = Dataset::write(
8253 data,
8254 "memory://test",
8255 Some(WriteParams {
8256 data_storage_version: Some(data_storage_version),
8257 ..Default::default()
8258 }),
8259 )
8260 .await
8261 .unwrap();
8262 dataset
8263 .create_index(
8264 &["indexed"],
8265 IndexType::Scalar,
8266 None,
8267 &ScalarIndexParams::default(),
8268 false,
8269 )
8270 .await
8271 .unwrap();
8272 dataset
8273 .create_index(
8274 &["text"],
8275 IndexType::Inverted,
8276 None,
8277 &InvertedIndexParams::default(),
8278 false,
8279 )
8280 .await
8281 .unwrap();
8282 dataset
8283 .create_index(
8284 &["vector"],
8285 IndexType::Vector,
8286 None,
8287 &VectorIndexParams {
8288 metric_type: DistanceType::L2,
8289 stages: vec![
8290 StageParams::Ivf(IvfBuildParams {
8291 max_iters: 2,
8292 num_partitions: Some(2),
8293 sample_rate: 2,
8294 ..Default::default()
8295 }),
8296 StageParams::PQ(PQBuildParams {
8297 max_iters: 2,
8298 num_sub_vectors: 2,
8299 ..Default::default()
8300 }),
8301 ],
8302 version: crate::index::vector::IndexFileVersion::Legacy,
8303 },
8304 false,
8305 )
8306 .await
8307 .unwrap();
8308
8309 dataset
8311 .scan()
8312 .prefilter(true)
8313 .filter("indexed > 10")
8314 .unwrap()
8315 .explain_plan(true)
8316 .await
8317 .unwrap();
8318
8319 let io_stats = dataset.object_store().io_stats_incremental();
8321 assert_io_gt!(io_stats, read_iops, 0);
8322
8323 dataset
8325 .scan()
8326 .prefilter(true)
8327 .filter("indexed > 10")
8328 .unwrap()
8329 .explain_plan(true)
8330 .await
8331 .unwrap();
8332
8333 let io_stats = dataset.object_store().io_stats_incremental();
8334 assert_io_eq!(io_stats, read_iops, 0);
8335
8336 dataset
8337 .scan()
8338 .prefilter(true)
8339 .filter("true")
8340 .unwrap()
8341 .explain_plan(true)
8342 .await
8343 .unwrap();
8344
8345 let io_stats = dataset.object_store().io_stats_incremental();
8346 assert_io_eq!(io_stats, read_iops, 0);
8347
8348 dataset
8349 .scan()
8350 .prefilter(true)
8351 .materialization_style(MaterializationStyle::AllEarly)
8352 .filter("true")
8353 .unwrap()
8354 .explain_plan(true)
8355 .await
8356 .unwrap();
8357
8358 let io_stats = dataset.object_store().io_stats_incremental();
8359 assert_io_eq!(io_stats, read_iops, 0);
8360
8361 dataset
8362 .scan()
8363 .prefilter(true)
8364 .materialization_style(MaterializationStyle::AllLate)
8365 .filter("true")
8366 .unwrap()
8367 .explain_plan(true)
8368 .await
8369 .unwrap();
8370
8371 let io_stats = dataset.object_store().io_stats_incremental();
8372 assert_io_eq!(io_stats, read_iops, 0);
8373 }
8374
8375 #[rstest]
8376 #[tokio::test]
8377 pub async fn test_row_meta_columns(
8378 #[values(
8379 (true, false), (false, true), (true, true) )]
8383 columns: (bool, bool),
8384 ) {
8385 let (with_row_id, with_row_address) = columns;
8386 let test_dir = TempStrDir::default();
8387 let uri = &test_dir;
8388
8389 let schema = Arc::new(arrow_schema::Schema::new(vec![
8390 arrow_schema::Field::new("data_item_id", arrow_schema::DataType::Int32, false),
8391 arrow_schema::Field::new("a", arrow_schema::DataType::Int32, false),
8392 ]));
8393
8394 let data = RecordBatch::try_new(
8395 schema.clone(),
8396 vec![
8397 Arc::new(Int32Array::from(vec![1001, 1002, 1003])),
8398 Arc::new(Int32Array::from(vec![1, 2, 3])),
8399 ],
8400 )
8401 .unwrap();
8402
8403 let dataset = Dataset::write(
8404 RecordBatchIterator::new(vec![Ok(data)], schema.clone()),
8405 uri,
8406 None,
8407 )
8408 .await
8409 .unwrap();
8410
8411 let mut scanner = dataset.scan();
8413
8414 let mut projection = vec!["data_item_id".to_string()];
8415 if with_row_id {
8416 scanner.with_row_id();
8417 projection.push(ROW_ID.to_string());
8418 }
8419 if with_row_address {
8420 scanner.with_row_address();
8421 projection.push(ROW_ADDR.to_string());
8422 }
8423
8424 scanner.project(&projection).unwrap();
8425 let stream = scanner.try_into_stream().await.unwrap();
8426 let batch = stream.try_collect::<Vec<_>>().await.unwrap().pop().unwrap();
8427
8428 if with_row_id {
8430 let column = batch.column_by_name(ROW_ID).unwrap();
8431 assert_eq!(column.data_type(), &DataType::UInt64);
8432 }
8433 if with_row_address {
8434 let column = batch.column_by_name(ROW_ADDR).unwrap();
8435 assert_eq!(column.data_type(), &DataType::UInt64);
8436 }
8437
8438 let mut scanner = dataset.scan();
8440 if with_row_id {
8441 scanner.with_row_id();
8442 }
8443 if with_row_address {
8444 scanner.with_row_address();
8445 }
8446 scanner.project(&["data_item_id"]).unwrap();
8447 let stream = scanner.try_into_stream().await.unwrap();
8448 let batch = stream.try_collect::<Vec<_>>().await.unwrap().pop().unwrap();
8449 let meta_column = batch.column_by_name(if with_row_id { ROW_ID } else { ROW_ADDR });
8450 assert!(meta_column.is_some());
8451
8452 let mut scanner = dataset.scan();
8454 if with_row_id {
8455 scanner.project(&[ROW_ID]).unwrap();
8456 } else {
8457 scanner.project(&[ROW_ADDR]).unwrap();
8458 };
8459 let stream = scanner.try_into_stream().await.unwrap();
8460 assert_eq!(stream.schema().fields().len(), 1);
8461 if with_row_id {
8462 assert!(stream.schema().field_with_name(ROW_ID).is_ok());
8463 } else {
8464 assert!(stream.schema().field_with_name(ROW_ADDR).is_ok());
8465 }
8466 }
8467
8468 async fn limit_offset_equivalency_test(scanner: &Scanner) {
8469 async fn test_one(
8470 scanner: &Scanner,
8471 full_result: &RecordBatch,
8472 limit: Option<i64>,
8473 offset: Option<i64>,
8474 ) {
8475 let mut new_scanner = scanner.clone();
8476 new_scanner.limit(limit, offset).unwrap();
8477 if let Some(nearest) = new_scanner.nearest_mut() {
8478 nearest.k = offset.unwrap_or(0).saturating_add(limit.unwrap_or(10_000)) as usize;
8479 }
8480 let result = new_scanner.try_into_batch().await.unwrap();
8481
8482 let resolved_offset = offset.unwrap_or(0).min(full_result.num_rows() as i64);
8483 let resolved_length = limit
8484 .unwrap_or(i64::MAX)
8485 .min(full_result.num_rows() as i64 - resolved_offset);
8486
8487 let expected = full_result.slice(resolved_offset as usize, resolved_length as usize);
8488
8489 if expected != result {
8490 let plan = new_scanner.analyze_plan().await.unwrap();
8491 assert_eq!(
8492 &expected, &result,
8493 "Limit: {:?}, Offset: {:?}, Plan: \n{}",
8494 limit, offset, plan
8495 );
8496 }
8497 }
8498
8499 let mut scanner_full = scanner.clone();
8500 if let Some(nearest) = scanner_full.nearest_mut() {
8501 nearest.k = 500;
8502 }
8503 let full_results = scanner_full.try_into_batch().await.unwrap();
8504
8505 test_one(scanner, &full_results, Some(1), None).await;
8506 test_one(scanner, &full_results, Some(1), Some(1)).await;
8507 test_one(scanner, &full_results, Some(1), Some(2)).await;
8508 test_one(scanner, &full_results, Some(1), Some(10)).await;
8509
8510 test_one(scanner, &full_results, Some(3), None).await;
8511 test_one(scanner, &full_results, Some(3), Some(2)).await;
8512 test_one(scanner, &full_results, Some(3), Some(4)).await;
8513
8514 test_one(scanner, &full_results, None, Some(3)).await;
8515 test_one(scanner, &full_results, None, Some(10)).await;
8516 }
8517
8518 #[tokio::test]
8519 async fn test_scan_limit_offset() {
8520 let test_ds = TestVectorDataset::new(LanceFileVersion::Stable, false)
8521 .await
8522 .unwrap();
8523 let scanner = test_ds.dataset.scan();
8524 limit_offset_equivalency_test(&scanner).await;
8525 }
8526
8527 #[tokio::test]
8528 async fn test_knn_limit_offset() {
8529 let test_ds = TestVectorDataset::new(LanceFileVersion::Stable, false)
8530 .await
8531 .unwrap();
8532 let query_vector = Float32Array::from(vec![0.0; 32]);
8533 let mut scanner = test_ds.dataset.scan();
8534 scanner
8535 .nearest("vec", &query_vector, 5)
8536 .unwrap()
8537 .project(&["i"])
8538 .unwrap();
8539 limit_offset_equivalency_test(&scanner).await;
8540 }
8541
8542 #[tokio::test]
8543 async fn test_ivf_pq_limit_offset() {
8544 let mut test_ds = TestVectorDataset::new(LanceFileVersion::Stable, false)
8545 .await
8546 .unwrap();
8547 test_ds.make_vector_index().await.unwrap();
8548 test_ds.append_new_data().await.unwrap();
8549 let query_vector = Float32Array::from(vec![0.0; 32]);
8550 let mut scanner = test_ds.dataset.scan();
8551 scanner.nearest("vec", &query_vector, 500).unwrap();
8552 limit_offset_equivalency_test(&scanner).await;
8553 }
8554
8555 #[tokio::test]
8556 async fn test_fts_limit_offset() {
8557 let mut test_ds = TestVectorDataset::new(LanceFileVersion::Stable, false)
8558 .await
8559 .unwrap();
8560 test_ds.make_fts_index().await.unwrap();
8561 test_ds.append_new_data().await.unwrap();
8562 let mut scanner = test_ds.dataset.scan();
8563 scanner
8564 .full_text_search(FullTextSearchQuery::new("4".into()))
8565 .unwrap();
8566 limit_offset_equivalency_test(&scanner).await;
8567 }
8568
8569 async fn test_row_offset_read_helper(
8570 ds: &Dataset,
8571 scan_builder: impl FnOnce(&mut Scanner) -> &mut Scanner,
8572 expected_cols: &[&str],
8573 expected_row_offsets: &[u64],
8574 ) {
8575 let mut scanner = ds.scan();
8576 let scanner = scan_builder(&mut scanner);
8577 let stream = scanner.try_into_stream().await.unwrap();
8578
8579 let schema = stream.schema();
8580 let actual_cols = schema
8581 .fields()
8582 .iter()
8583 .map(|f| f.name().as_str())
8584 .collect::<Vec<_>>();
8585 assert_eq!(&actual_cols, expected_cols);
8586
8587 let batches = stream.try_collect::<Vec<_>>().await.unwrap();
8588 let batch = arrow_select::concat::concat_batches(&schema, &batches).unwrap();
8589
8590 let row_offsets = batch
8591 .column_by_name(ROW_OFFSET)
8592 .unwrap()
8593 .as_primitive::<UInt64Type>()
8594 .values();
8595 assert_eq!(row_offsets.as_ref(), expected_row_offsets);
8596 }
8597
8598 #[tokio::test]
8599 async fn test_row_offset_read() {
8600 let mut ds = lance_datagen::gen_batch()
8601 .col("idx", array::step::<Int32Type>())
8602 .into_ram_dataset(FragmentCount::from(3), FragmentRowCount::from(3))
8603 .await
8604 .unwrap();
8605 ds.delete("idx >= 2 AND idx <= 6").await.unwrap();
8609
8610 test_row_offset_read_helper(
8612 &ds,
8613 |scanner| scanner.project(&["idx", ROW_OFFSET]).unwrap(),
8614 &["idx", ROW_OFFSET],
8615 &[0, 1, 2, 3],
8616 )
8617 .await;
8618
8619 test_row_offset_read_helper(
8621 &ds,
8622 |scanner| scanner.project(&[ROW_OFFSET]).unwrap(),
8623 &[ROW_OFFSET],
8624 &[0, 1, 2, 3],
8625 )
8626 .await;
8627
8628 test_row_offset_read_helper(
8630 &ds,
8631 |scanner| {
8632 scanner
8633 .filter("idx > 1")
8634 .unwrap()
8635 .project(&[ROW_OFFSET])
8636 .unwrap()
8637 },
8638 &[ROW_OFFSET],
8639 &[2, 3],
8640 )
8641 .await;
8642 }
8643
8644 #[tokio::test]
8645 async fn test_filter_to_take() {
8646 let mut ds = lance_datagen::gen_batch()
8647 .col("idx", array::step::<Int32Type>())
8648 .into_ram_dataset(FragmentCount::from(3), FragmentRowCount::from(100))
8649 .await
8650 .unwrap();
8651
8652 let row_ids = ds
8653 .scan()
8654 .project(&Vec::<&str>::default())
8655 .unwrap()
8656 .with_row_id()
8657 .try_into_stream()
8658 .await
8659 .unwrap()
8660 .try_collect::<Vec<_>>()
8661 .await
8662 .unwrap();
8663 let schema = row_ids[0].schema();
8664 let row_ids = concat_batches(&schema, row_ids.iter()).unwrap();
8665 let row_ids = row_ids.column(0).as_primitive::<UInt64Type>().clone();
8666
8667 let row_addrs = ds
8668 .scan()
8669 .project(&Vec::<&str>::default())
8670 .unwrap()
8671 .with_row_address()
8672 .try_into_stream()
8673 .await
8674 .unwrap()
8675 .try_collect::<Vec<_>>()
8676 .await
8677 .unwrap();
8678 let schema = row_addrs[0].schema();
8679 let row_addrs = concat_batches(&schema, row_addrs.iter()).unwrap();
8680 let row_addrs = row_addrs.column(0).as_primitive::<UInt64Type>().clone();
8681
8682 ds.delete("idx >= 190 AND idx < 210").await.unwrap();
8683
8684 let ds_copy = ds.clone();
8685 let do_check = async move |filt: &str, expected_idx: &[i32], applies_optimization: bool| {
8686 let mut scanner = ds_copy.scan();
8687 scanner.filter(filt).unwrap();
8688 let plan = scanner.explain_plan(true).await.unwrap();
8690 if applies_optimization {
8691 assert!(
8692 plan.contains("OneShotStream"),
8693 "expected take optimization to be applied. Filter: '{}'. Plan:\n{}",
8694 filt,
8695 plan
8696 );
8697 } else {
8698 assert!(
8699 !plan.contains("OneShotStream"),
8700 "expected take optimization to not be applied. Filter: '{}'. Plan:\n{}",
8701 filt,
8702 plan
8703 );
8704 }
8705
8706 let stream = scanner.try_into_stream().await.unwrap();
8708 let batches = stream.try_collect::<Vec<_>>().await.unwrap();
8709 let idx = batches
8710 .iter()
8711 .map(|b| b.column_by_name("idx").unwrap().as_ref())
8712 .collect::<Vec<_>>();
8713
8714 if idx.is_empty() {
8715 assert!(expected_idx.is_empty());
8716 return;
8717 }
8718
8719 let idx = arrow::compute::concat(&idx).unwrap();
8720 assert_eq!(idx.as_primitive::<Int32Type>().values(), expected_idx);
8721 };
8722 let check =
8723 async |filt: &str, expected_idx: &[i32]| do_check(filt, expected_idx, true).await;
8724 let check_no_opt = async |filt: &str, expected_idx: &[i32]| {
8725 do_check(filt, expected_idx, false).await;
8726 };
8727
8728 check("_rowid = 50", &[50]).await;
8730 check("_rowaddr = 50", &[50]).await;
8731 check("_rowoffset = 50", &[50]).await;
8732
8733 check(
8734 "_rowid = 50 OR _rowid = 51 OR _rowid = 52 OR _rowid = 49",
8735 &[49, 50, 51, 52],
8736 )
8737 .await;
8738 check(
8739 "_rowaddr = 50 OR _rowaddr = 51 OR _rowaddr = 52 OR _rowaddr = 49",
8740 &[49, 50, 51, 52],
8741 )
8742 .await;
8743 check(
8744 "_rowoffset = 50 OR _rowoffset = 51 OR _rowoffset = 52 OR _rowoffset = 49",
8745 &[49, 50, 51, 52],
8746 )
8747 .await;
8748
8749 check("_rowid IN (52, 51, 50, 17)", &[17, 50, 51, 52]).await;
8750 check("_rowaddr IN (52, 51, 50, 17)", &[17, 50, 51, 52]).await;
8751 check("_rowoffset IN (52, 51, 50, 17)", &[17, 50, 51, 52]).await;
8752
8753 check(&format!("_rowid = {}", row_ids.value(190)), &[]).await;
8757 check(&format!("_rowaddr = {}", row_addrs.value(190)), &[]).await;
8758 check("_rowoffset = 190", &[210]).await;
8761
8762 check(&format!("_rowid = {}", row_ids.value(250)), &[250]).await;
8764 check(&format!("_rowaddr = {}", row_addrs.value(250)), &[250]).await;
8765 check("_rowoffset = 250", &[270]).await;
8766
8767 check("_rowoffset = 1000", &[]).await;
8769
8770 check("_rowid IN (5, 10, 15) AND idx > 10", &[15]).await;
8772 check("_rowaddr IN (5, 10, 15) AND idx > 10", &[15]).await;
8773 check("_rowoffset IN (5, 10, 15) AND idx > 10", &[15]).await;
8774 check("idx > 10 AND _rowid IN (5, 10, 15)", &[15]).await;
8775 check("idx > 10 AND _rowaddr IN (5, 10, 15)", &[15]).await;
8776 check("idx > 10 AND _rowoffset IN (5, 10, 15)", &[15]).await;
8777 check("_rowid = 50 AND _rowid = 50", &[50]).await;
8779
8780 check_no_opt("_rowid = 50 AND _rowid = 51", &[]).await;
8782 check_no_opt("(_rowid = 50 AND idx < 100) OR _rowid = 51", &[50, 51]).await;
8783
8784 let mut scanner = ds.scan();
8786 scanner.filter("_rowoffset = 77").unwrap();
8787 scanner
8788 .project_with_transform(&[("foo", "idx * 2")])
8789 .unwrap();
8790 let stream = scanner.try_into_stream().await.unwrap();
8791 let batches = stream.try_collect::<Vec<_>>().await.unwrap();
8792 assert_eq!(batches[0].schema().field(0).name(), "foo");
8793 let val = batches[0].column(0).as_primitive::<Int32Type>().values()[0];
8794 assert_eq!(val, 154);
8795 }
8796
8797 #[tokio::test]
8798 async fn test_nested_field_ordering() {
8799 use arrow_array::StructArray;
8800
8801 let id_array = Int32Array::from(vec![3, 1, 2]);
8803 let nested_values = Int32Array::from(vec![30, 10, 20]);
8804 let nested_struct = StructArray::from(vec![(
8805 Arc::new(ArrowField::new("value", DataType::Int32, false)),
8806 Arc::new(nested_values) as ArrayRef,
8807 )]);
8808
8809 let schema = Arc::new(ArrowSchema::new(vec![
8810 ArrowField::new("id", DataType::Int32, false),
8811 ArrowField::new(
8812 "nested",
8813 DataType::Struct(vec![ArrowField::new("value", DataType::Int32, false)].into()),
8814 false,
8815 ),
8816 ]));
8817
8818 let batch = RecordBatch::try_new(
8819 schema.clone(),
8820 vec![Arc::new(id_array), Arc::new(nested_struct)],
8821 )
8822 .unwrap();
8823
8824 let test_dir = TempStrDir::default();
8825 let test_uri = &test_dir;
8826 let reader = RecordBatchIterator::new(vec![Ok(batch)].into_iter(), schema.clone());
8827
8828 let dataset = Dataset::write(reader, test_uri, None).await.unwrap();
8829
8830 let mut scanner = dataset.scan();
8832 scanner
8833 .order_by(Some(vec![ColumnOrdering {
8834 column_name: "nested.value".to_string(),
8835 ascending: true,
8836 nulls_first: true,
8837 }]))
8838 .unwrap(); let stream = scanner.try_into_stream().await.unwrap();
8841 let batches = stream.try_collect::<Vec<_>>().await.unwrap();
8842
8843 let sorted_ids = batches[0].column(0).as_primitive::<Int32Type>().values();
8845 assert_eq!(sorted_ids[0], 1); assert_eq!(sorted_ids[1], 2); assert_eq!(sorted_ids[2], 3); }
8849
8850 #[tokio::test]
8851 async fn test_limit_with_ordering_not_pushed_down() {
8852 let id_array = Int32Array::from(vec![5, 2, 8, 1, 3, 7, 4, 6]);
8858 let value_array = Int32Array::from(vec![50, 20, 80, 10, 30, 70, 40, 60]);
8859
8860 let schema = Arc::new(ArrowSchema::new(vec![
8861 ArrowField::new("id", DataType::Int32, false),
8862 ArrowField::new("value", DataType::Int32, false),
8863 ]));
8864
8865 let batch = RecordBatch::try_new(
8866 schema.clone(),
8867 vec![Arc::new(id_array), Arc::new(value_array)],
8868 )
8869 .unwrap();
8870
8871 let test_dir = TempStrDir::default();
8872 let test_uri = &test_dir;
8873 let reader = RecordBatchIterator::new(vec![Ok(batch)].into_iter(), schema.clone());
8874
8875 let dataset = Dataset::write(reader, test_uri, None).await.unwrap();
8876
8877 let mut scanner = dataset.scan();
8879 scanner
8880 .order_by(Some(vec![ColumnOrdering {
8881 column_name: "value".to_string(),
8882 ascending: true,
8883 nulls_first: true,
8884 }]))
8885 .unwrap();
8886 scanner.limit(Some(3), None).unwrap();
8887
8888 let stream = scanner.try_into_stream().await.unwrap();
8889 let batches = stream.try_collect::<Vec<_>>().await.unwrap();
8890
8891 let sorted_ids = batches[0].column(0).as_primitive::<Int32Type>().values();
8893 let sorted_values = batches[0].column(1).as_primitive::<Int32Type>().values();
8894 assert_eq!(batches[0].num_rows(), 3);
8895 assert_eq!(sorted_ids[0], 1); assert_eq!(sorted_ids[1], 2); assert_eq!(sorted_ids[2], 3); assert_eq!(sorted_values[0], 10);
8899 assert_eq!(sorted_values[1], 20);
8900 assert_eq!(sorted_values[2], 30);
8901
8902 let mut scanner = dataset.scan();
8904 scanner
8905 .order_by(Some(vec![ColumnOrdering {
8906 column_name: "value".to_string(),
8907 ascending: true,
8908 nulls_first: true,
8909 }]))
8910 .unwrap();
8911 scanner.limit(Some(3), Some(2)).unwrap(); let stream = scanner.try_into_stream().await.unwrap();
8914 let batches = stream.try_collect::<Vec<_>>().await.unwrap();
8915
8916 let sorted_ids = batches[0].column(0).as_primitive::<Int32Type>().values();
8917 let sorted_values = batches[0].column(1).as_primitive::<Int32Type>().values();
8918 assert_eq!(batches[0].num_rows(), 3);
8919 assert_eq!(sorted_ids[0], 3); assert_eq!(sorted_ids[1], 4); assert_eq!(sorted_ids[2], 5); assert_eq!(sorted_values[0], 30);
8923 assert_eq!(sorted_values[1], 40);
8924 assert_eq!(sorted_values[2], 50);
8925
8926 let mut scanner = dataset.scan();
8928 scanner.limit(Some(3), None).unwrap();
8929
8930 let stream = scanner.try_into_stream().await.unwrap();
8931 let batches = stream.try_collect::<Vec<_>>().await.unwrap();
8932
8933 assert_eq!(batches[0].num_rows(), 3);
8935 let unsorted_values = batches[0].column(1).as_primitive::<Int32Type>().values();
8936 assert_eq!(unsorted_values[0], 50);
8938 assert_eq!(unsorted_values[1], 20);
8939 assert_eq!(unsorted_values[2], 80);
8940 }
8941
8942 #[tokio::test]
8943 async fn test_scan_with_version_columns() {
8944 use arrow_array::{Int32Array, RecordBatch, RecordBatchIterator};
8945 use arrow_schema::{DataType, Field as ArrowField, Schema as ArrowSchema};
8946
8947 let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
8949 "id",
8950 DataType::Int32,
8951 false,
8952 )]));
8953
8954 let batch = RecordBatch::try_new(
8955 schema.clone(),
8956 vec![Arc::new(Int32Array::from(vec![1, 2, 3]))],
8957 )
8958 .unwrap();
8959
8960 let test_dir = lance_core::utils::tempfile::TempStrDir::default();
8961 let test_uri = test_dir.as_str();
8962
8963 let reader = RecordBatchIterator::new(vec![Ok(batch)], schema);
8964 let write_params = WriteParams {
8965 enable_stable_row_ids: true,
8966 ..Default::default()
8967 };
8968 Dataset::write(reader, test_uri, Some(write_params))
8969 .await
8970 .unwrap();
8971
8972 let dataset = Dataset::open(test_uri).await.unwrap();
8973 let mut scanner = dataset.scan();
8974
8975 scanner
8976 .project(&[ROW_CREATED_AT_VERSION, ROW_LAST_UPDATED_AT_VERSION])
8977 .unwrap();
8978
8979 let output_schema = scanner.schema().await.unwrap();
8981 assert!(
8982 output_schema
8983 .column_with_name("_row_last_updated_at_version")
8984 .is_some(),
8985 "Schema should include _row_last_updated_at_version"
8986 );
8987 assert!(
8988 output_schema
8989 .column_with_name("_row_created_at_version")
8990 .is_some(),
8991 "Schema should include _row_created_at_version"
8992 );
8993
8994 let batches = scanner
8996 .try_into_stream()
8997 .await
8998 .unwrap()
8999 .try_collect::<Vec<_>>()
9000 .await
9001 .unwrap();
9002
9003 assert_eq!(batches.len(), 1);
9004 let batch = &batches[0];
9005
9006 let last_updated = batch
9008 .column_by_name("_row_last_updated_at_version")
9009 .expect("Should have _row_last_updated_at_version column");
9010 let created_at = batch
9011 .column_by_name("_row_created_at_version")
9012 .expect("Should have _row_created_at_version column");
9013
9014 let last_updated_array = last_updated
9016 .as_any()
9017 .downcast_ref::<arrow_array::UInt64Array>()
9018 .unwrap();
9019 let created_at_array = created_at
9020 .as_any()
9021 .downcast_ref::<arrow_array::UInt64Array>()
9022 .unwrap();
9023
9024 for i in 0..batch.num_rows() {
9025 assert_eq!(
9026 last_updated_array.value(i),
9027 1,
9028 "All rows last updated at version 1"
9029 );
9030 assert_eq!(
9031 created_at_array.value(i),
9032 1,
9033 "All rows created at version 1"
9034 );
9035 }
9036 }
9037
9038 #[test_log::test(test)]
9039 fn test_scan_finishes_all_tasks() {
9040 let runtime = tokio::runtime::Builder::new_multi_thread()
9042 .enable_time()
9043 .build()
9044 .unwrap();
9045
9046 runtime.block_on(async move {
9047 let ds = lance_datagen::gen_batch()
9048 .col("id", lance_datagen::array::step::<Int32Type>())
9049 .into_ram_dataset(FragmentCount::from(1000), FragmentRowCount::from(10))
9050 .await
9051 .unwrap();
9052
9053 let mut stream = ds
9061 .scan()
9062 .fragment_readahead(1000)
9063 .batch_size(1)
9064 .io_buffer_size(1)
9065 .batch_readahead(1)
9066 .try_into_stream()
9067 .await
9068 .unwrap();
9069 stream.next().await.unwrap().unwrap();
9070 });
9071
9072 let start = Instant::now();
9073 while start.elapsed() < Duration::from_secs(10) {
9074 if runtime.handle().metrics().num_alive_tasks() == 0 {
9075 break;
9076 }
9077 std::thread::sleep(Duration::from_millis(100));
9078 }
9079
9080 assert!(
9081 runtime.handle().metrics().num_alive_tasks() == 0,
9082 "Tasks should have finished within 10 seconds but there are still {} tasks running",
9083 runtime.handle().metrics().num_alive_tasks()
9084 );
9085 }
9086}