Merge remote-tracking branch 'origin/search-refactor-tests-doc' into search-refactor

This commit is contained in:
Loïc Lecrenier 2023-04-07 09:53:24 +02:00
commit f7d90ad19f
27 changed files with 4364 additions and 1427 deletions

View File

@ -0,0 +1,195 @@
use roaring::RoaringBitmap;
use super::logger::SearchLogger;
use super::ranking_rules::{BoxRankingRule, RankingRuleQueryTrait};
use super::SearchContext;
use crate::search::new::distinct::{apply_distinct_rule, distinct_single_docid, DistinctOutput};
use crate::Result;
pub fn bucket_sort<'ctx, Q: RankingRuleQueryTrait>(
ctx: &mut SearchContext<'ctx>,
mut ranking_rules: Vec<BoxRankingRule<'ctx, Q>>,
query: &Q,
universe: &RoaringBitmap,
from: usize,
length: usize,
logger: &mut dyn SearchLogger<Q>,
) -> Result<Vec<u32>> {
logger.initial_query(query);
logger.ranking_rules(&ranking_rules);
logger.initial_universe(universe);
let distinct_fid = if let Some(field) = ctx.index.distinct_field(ctx.txn)? {
ctx.index.fields_ids_map(ctx.txn)?.id(field)
} else {
None
};
if universe.len() < from as u64 {
return Ok(vec![]);
}
if ranking_rules.is_empty() {
if let Some(distinct_fid) = distinct_fid {
let mut excluded = RoaringBitmap::new();
let mut results = vec![];
for docid in universe.iter() {
if results.len() >= from + length {
break;
}
if excluded.contains(docid) {
continue;
}
distinct_single_docid(ctx.index, ctx.txn, distinct_fid, docid, &mut excluded)?;
results.push(docid);
}
return Ok(results);
} else {
return Ok(universe.iter().skip(from).take(length).collect());
};
}
let ranking_rules_len = ranking_rules.len();
logger.start_iteration_ranking_rule(0, ranking_rules[0].as_ref(), query, universe);
ranking_rules[0].start_iteration(ctx, logger, universe, query)?;
let mut ranking_rule_universes: Vec<RoaringBitmap> =
vec![RoaringBitmap::default(); ranking_rules_len];
ranking_rule_universes[0] = universe.clone();
let mut cur_ranking_rule_index = 0;
/// Finish iterating over the current ranking rule, yielding
/// control to the parent (or finishing the search if not possible).
/// Update the candidates accordingly and inform the logger.
macro_rules! back {
() => {
assert!(ranking_rule_universes[cur_ranking_rule_index].is_empty());
logger.end_iteration_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&ranking_rule_universes[cur_ranking_rule_index],
);
ranking_rule_universes[cur_ranking_rule_index].clear();
ranking_rules[cur_ranking_rule_index].end_iteration(ctx, logger);
if cur_ranking_rule_index == 0 {
break;
} else {
cur_ranking_rule_index -= 1;
}
};
}
let mut results = vec![];
let mut cur_offset = 0usize;
/// Add the candidates to the results. Take `distinct`, `from`, `length`, and `cur_offset`
/// into account and inform the logger.
macro_rules! maybe_add_to_results {
($candidates:expr) => {
// First apply the distinct rule on the candidates, reducing the universes if necessary
let candidates = if let Some(distinct_fid) = distinct_fid {
let DistinctOutput { remaining, excluded } = apply_distinct_rule(ctx, distinct_fid, $candidates)?;
for universe in ranking_rule_universes.iter_mut() {
*universe -= &excluded;
}
remaining
} else {
$candidates.clone()
};
let len = candidates.len();
// if the candidates are empty, there is nothing to do;
if !candidates.is_empty() {
// if we still haven't reached the first document to return
if cur_offset < from {
// and if no document from this bucket can be returned
if cur_offset + (candidates.len() as usize) < from {
// then just skip the bucket
logger.skip_bucket_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&candidates,
);
} else {
// otherwise, skip some of the documents and add some of the rest, in order of ids
let all_candidates = candidates.iter().collect::<Vec<_>>();
let (skipped_candidates, candidates) =
all_candidates.split_at(from - cur_offset);
logger.skip_bucket_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&skipped_candidates.into_iter().collect(),
);
let candidates = candidates
.iter()
.take(length - results.len())
.copied()
.collect::<Vec<_>>();
logger.add_to_results(&candidates);
results.extend(&candidates);
}
} else {
// if we have passed the offset already, add some of the documents (up to the limit)
let candidates =
candidates.iter().take(length - results.len()).collect::<Vec<u32>>();
logger.add_to_results(&candidates);
results.extend(&candidates);
}
}
cur_offset += len as usize;
};
}
while results.len() < length {
// The universe for this bucket is zero or one element, so we don't need to sort
// anything, just extend the results and go back to the parent ranking rule.
if ranking_rule_universes[cur_ranking_rule_index].len() <= 1 {
maybe_add_to_results!(&ranking_rule_universes[cur_ranking_rule_index]);
ranking_rule_universes[cur_ranking_rule_index].clear();
back!();
continue;
}
let Some(next_bucket) = ranking_rules[cur_ranking_rule_index].next_bucket(ctx, logger, &ranking_rule_universes[cur_ranking_rule_index])? else {
back!();
continue;
};
logger.next_bucket_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&ranking_rule_universes[cur_ranking_rule_index],
&next_bucket.candidates,
);
debug_assert!(
ranking_rule_universes[cur_ranking_rule_index].is_superset(&next_bucket.candidates)
);
ranking_rule_universes[cur_ranking_rule_index] -= &next_bucket.candidates;
if cur_ranking_rule_index == ranking_rules_len - 1
|| next_bucket.candidates.len() <= 1
|| cur_offset + (next_bucket.candidates.len() as usize) < from
{
maybe_add_to_results!(&next_bucket.candidates);
continue;
}
cur_ranking_rule_index += 1;
ranking_rule_universes[cur_ranking_rule_index] = next_bucket.candidates.clone();
logger.start_iteration_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&next_bucket.query,
&ranking_rule_universes[cur_ranking_rule_index],
);
ranking_rules[cur_ranking_rule_index].start_iteration(
ctx,
logger,
&next_bucket.candidates,
&next_bucket.query,
)?;
}
Ok(results)
}

View File

@ -1,3 +1,4 @@
use std::borrow::Cow;
use std::collections::hash_map::Entry;
use std::hash::Hash;
@ -24,6 +25,8 @@ pub struct DatabaseCache<'ctx> {
pub word_docids: FxHashMap<Interned<String>, Option<&'ctx [u8]>>,
pub exact_word_docids: FxHashMap<Interned<String>, Option<&'ctx [u8]>>,
pub word_prefix_docids: FxHashMap<Interned<String>, Option<&'ctx [u8]>>,
pub words_fst: Option<fst::Set<Cow<'ctx, [u8]>>>,
pub word_position_docids: FxHashMap<(Interned<String>, u16), Option<&'ctx [u8]>>,
pub word_fid_docids: FxHashMap<(Interned<String>, u16), Option<&'ctx [u8]>>,
}
@ -51,6 +54,16 @@ impl<'ctx> DatabaseCache<'ctx> {
}
}
impl<'ctx> SearchContext<'ctx> {
pub fn get_words_fst(&mut self) -> Result<fst::Set<Cow<'ctx, [u8]>>> {
if let Some(fst) = self.db_cache.words_fst.clone() {
Ok(fst)
} else {
let fst = self.index.words_fst(self.txn)?;
self.db_cache.words_fst = Some(fst.clone());
Ok(fst)
}
}
/// Retrieve or insert the given value in the `word_docids` database.
pub fn get_db_word_docids(&mut self, word: Interned<String>) -> Result<Option<&'ctx [u8]>> {
DatabaseCache::get_value(

View File

@ -41,7 +41,7 @@ pub fn apply_distinct_rule(
}
/// Apply the distinct rule defined by [`apply_distinct_rule`] for a single document id.
fn distinct_single_docid(
pub fn distinct_single_docid(
index: &Index,
txn: &RoTxn,
field_id: u16,

View File

@ -176,6 +176,9 @@ impl<T> Interner<T> {
pub fn iter_mut(&mut self) -> impl Iterator<Item = (Interned<T>, &mut T)> {
self.stable_store.iter_mut().enumerate().map(|(i, x)| (Interned::from_raw(i as u16), x))
}
pub fn freeze(self) -> FixedSizeInterner<T> {
FixedSizeInterner { stable_store: self.stable_store }
}
}
/// A store of values of type `T`, each linked to a value of type `From`

View File

@ -8,9 +8,7 @@ use roaring::RoaringBitmap;
use crate::search::new::interner::{Interned, MappedInterner};
use crate::search::new::query_graph::QueryNodeData;
use crate::search::new::query_term::{
Lazy, LocatedQueryTermSubset, OneTypoTerm, QueryTerm, TwoTypoTerm, ZeroTypoTerm,
};
use crate::search::new::query_term::LocatedQueryTermSubset;
use crate::search::new::ranking_rule_graph::{
DeadEndsCache, Edge, ProximityCondition, ProximityGraph, RankingRuleGraph,
RankingRuleGraphTrait, TypoCondition, TypoGraph,
@ -439,87 +437,26 @@ results.{cur_ranking_rule}{cur_activated_id} {{
positions: _,
term_ids: _,
}) => {
let QueryTerm {
original,
is_ngram: _,
is_prefix: _,
max_nbr_typos,
zero_typo,
one_typo,
two_typo,
} = ctx.term_interner.get(term_subset.original);
let original = ctx.word_interner.get(*original);
writeln!(
file,
"{node_idx} : \"{original}\" {{
"{node_idx} : \"{}\" {{
shape: class
max_nbr_typo: {max_nbr_typos}"
max_nbr_typo: {}",
term_subset.description(ctx),
term_subset.max_nbr_typos(ctx)
)
.unwrap();
let ZeroTypoTerm { phrase, exact: zero_typo, prefix_of, synonyms, use_prefix_db } =
zero_typo;
for w in zero_typo.iter().copied() {
if term_subset.zero_typo_subset.contains_word(w) {
let w = ctx.word_interner.get(w);
writeln!(file, "\"{w}\" : 0").unwrap();
}
for w in term_subset.all_single_words_except_prefix_db(ctx).unwrap() {
let w = ctx.word_interner.get(w);
writeln!(file, "{w}: word").unwrap();
}
for w in prefix_of.iter().copied() {
if term_subset.zero_typo_subset.contains_word(w) {
let w = ctx.word_interner.get(w);
writeln!(file, "\"{w}\" : 0P").unwrap();
}
for p in term_subset.all_phrases(ctx).unwrap() {
writeln!(file, "{}: phrase", p.description(ctx)).unwrap();
}
if let Some(phrase) = phrase {
if term_subset.zero_typo_subset.contains_phrase(*phrase) {
let phrase = ctx.phrase_interner.get(*phrase);
let phrase_str = phrase.description(&ctx.word_interner);
writeln!(file, "\"{phrase_str}\" : phrase").unwrap();
}
}
for synonym in synonyms.iter().copied() {
if term_subset.zero_typo_subset.contains_phrase(synonym) {
let phrase = ctx.phrase_interner.get(synonym);
let phrase_str = phrase.description(&ctx.word_interner);
writeln!(file, "\"{phrase_str}\" : synonym").unwrap();
}
}
if let Some(use_prefix_db) = use_prefix_db {
if term_subset.zero_typo_subset.contains_word(*use_prefix_db) {
let p = ctx.word_interner.get(*use_prefix_db);
writeln!(file, "use prefix DB : {p}").unwrap();
}
}
if let Lazy::Init(one_typo) = one_typo {
let OneTypoTerm { split_words, one_typo } = one_typo;
for w in one_typo.iter().copied() {
if term_subset.one_typo_subset.contains_word(w) {
let w = ctx.word_interner.get(w);
writeln!(file, "\"{w}\" : 1").unwrap();
}
}
if let Some(split_words) = split_words {
if term_subset.one_typo_subset.contains_phrase(*split_words) {
let phrase = ctx.phrase_interner.get(*split_words);
let phrase_str = phrase.description(&ctx.word_interner);
writeln!(file, "\"{phrase_str}\" : split_words").unwrap();
}
}
}
if let Lazy::Init(two_typo) = two_typo {
let TwoTypoTerm { two_typos } = two_typo;
for w in two_typos.iter().copied() {
if term_subset.two_typo_subset.contains_word(w) {
let w = ctx.word_interner.get(w);
writeln!(file, "\"{w}\" : 2").unwrap();
}
}
if let Some(w) = term_subset.use_prefix_db(ctx) {
let w = ctx.word_interner.get(w);
writeln!(file, "{w}: prefix db").unwrap();
}
writeln!(file, "}}").unwrap();

View File

@ -1,6 +1,8 @@
// #[cfg(test)]
pub mod detailed;
pub mod test_logger;
use roaring::RoaringBitmap;
use super::interner::{Interned, MappedInterner};

View File

@ -1,3 +1,4 @@
mod bucket_sort;
mod db_cache;
mod distinct;
mod graph_based_ranking_rule;
@ -18,6 +19,9 @@ mod sort;
// TODO: documentation + comments
mod words;
#[cfg(test)]
mod tests;
use std::collections::HashSet;
use charabia::TokenizerBuilder;
@ -29,7 +33,8 @@ pub use logger::detailed::DetailedSearchLogger;
pub use logger::{DefaultSearchLogger, SearchLogger};
use query_graph::{QueryGraph, QueryNode};
use query_term::{located_query_terms_from_string, Phrase, QueryTerm};
use ranking_rules::{bucket_sort, PlaceholderQuery, RankingRuleOutput, RankingRuleQueryTrait};
use ranking_rules::{PlaceholderQuery, RankingRuleOutput, RankingRuleQueryTrait};
use bucket_sort::bucket_sort;
use resolve_query_graph::PhraseDocIdsCache;
use roaring::RoaringBitmap;
use words::Words;

View File

@ -1,11 +1,10 @@
use super::interner::{FixedSizeInterner, Interned};
use super::query_term::{
self, number_of_typos_allowed, LocatedQueryTerm, LocatedQueryTermSubset, NTypoTermSubset,
QueryTermSubset,
self, number_of_typos_allowed, LocatedQueryTerm, LocatedQueryTermSubset, QueryTermSubset,
};
use super::small_bitmap::SmallBitmap;
use super::SearchContext;
use crate::search::new::interner::DedupInterner;
use crate::search::new::interner::Interner;
use crate::Result;
use std::cmp::Ordering;
use std::collections::BTreeMap;
@ -107,12 +106,7 @@ impl QueryGraph {
let new_node_idx = add_node(
&mut nodes_data,
QueryNodeData::Term(LocatedQueryTermSubset {
term_subset: QueryTermSubset {
original: Interned::from_raw(term_idx as u16),
zero_typo_subset: NTypoTermSubset::All,
one_typo_subset: NTypoTermSubset::All,
two_typo_subset: NTypoTermSubset::All,
},
term_subset: QueryTermSubset::full(Interned::from_raw(term_idx as u16)),
positions: terms[term_idx].positions.clone(),
term_ids: term_idx as u8..=term_idx as u8,
}),
@ -126,12 +120,7 @@ impl QueryGraph {
let ngram_idx = add_node(
&mut nodes_data,
QueryNodeData::Term(LocatedQueryTermSubset {
term_subset: QueryTermSubset {
original: ngram.value,
zero_typo_subset: NTypoTermSubset::All,
one_typo_subset: NTypoTermSubset::All,
two_typo_subset: NTypoTermSubset::All,
},
term_subset: QueryTermSubset::full(ngram.value),
positions: ngram.positions,
term_ids: term_idx as u8 - 1..=term_idx as u8,
}),
@ -146,12 +135,7 @@ impl QueryGraph {
let ngram_idx = add_node(
&mut nodes_data,
QueryNodeData::Term(LocatedQueryTermSubset {
term_subset: QueryTermSubset {
original: ngram.value,
zero_typo_subset: NTypoTermSubset::All,
one_typo_subset: NTypoTermSubset::All,
two_typo_subset: NTypoTermSubset::All,
},
term_subset: QueryTermSubset::full(ngram.value),
positions: ngram.positions,
term_ids: term_idx as u8 - 2..=term_idx as u8,
}),
@ -329,7 +313,7 @@ impl QueryGraph {
let mut at_least_one_phrase = false;
for (node_id, node) in self.nodes.iter() {
let QueryNodeData::Term(t) = &node.data else { continue };
if ctx.term_interner.get(t.term_subset.original).zero_typo.phrase.is_some() {
if t.term_subset.original_phrase(ctx).is_some() {
at_least_one_phrase = true;
continue;
}
@ -361,29 +345,13 @@ impl QueryGraph {
Build a query graph from a list of paths
The paths are composed of source and dest terms.
If the source term is `None`, then the last dest term is used
as the predecessor of the dest term. If the source is Some(_),
then an edge is built between the last dest term and the source,
and between the source and new dest term.
Note that the resulting graph will not correspond to a perfect
representation of the set of paths.
For example, consider the following paths:
```txt
PATH 1 : a -> b1 -> c1 -> d -> e1
PATH 2 : a -> b2 -> c2 -> d -> e2
```
Then the resulting graph will be:
```txt
b1 c1 e1
a d
b2 c2 e2
```
which is different from the fully correct representation:
```txt
b1 c1 d e1
@ -399,21 +367,51 @@ impl QueryGraph {
pub fn build_from_paths(
paths: Vec<Vec<(Option<LocatedQueryTermSubset>, LocatedQueryTermSubset)>>,
) -> Self {
let mut node_data = DedupInterner::default();
let root_node = node_data.insert(QueryNodeData::Start);
let end_node = node_data.insert(QueryNodeData::End);
let mut node_data = Interner::default();
let root_node = node_data.push(QueryNodeData::Start);
let end_node = node_data.push(QueryNodeData::End);
let mut paths_with_single_terms = vec![];
for path in paths {
let mut processed_path = vec![];
let mut prev_dest_term: Option<LocatedQueryTermSubset> = None;
for (start_term, dest_term) in path {
if let Some(prev_dest_term) = prev_dest_term.take() {
if let Some(mut start_term) = start_term {
if start_term.term_ids == prev_dest_term.term_ids {
start_term.term_subset.intersect(&prev_dest_term.term_subset);
processed_path.push(start_term);
} else {
processed_path.push(prev_dest_term);
processed_path.push(start_term);
}
} else {
processed_path.push(prev_dest_term);
}
} else if let Some(start_term) = start_term {
processed_path.push(start_term);
}
prev_dest_term = Some(dest_term);
}
if let Some(prev_dest_term) = prev_dest_term {
processed_path.push(prev_dest_term);
}
paths_with_single_terms.push(processed_path);
}
// TODO: make a prefix tree of the processed paths to avoid uselessly duplicating nodes
let mut paths_with_ids = vec![];
for path in paths {
for path in paths_with_single_terms {
let mut path_with_ids = vec![];
for node in path {
let (start_term, end_term) = node;
let src_node_id = start_term.map(|x| node_data.insert(QueryNodeData::Term(x)));
let dest_node_id = node_data.insert(QueryNodeData::Term(end_term));
path_with_ids.push((src_node_id, dest_node_id));
for term in path {
let id = node_data.push(QueryNodeData::Term(term));
path_with_ids.push(Interned::from_raw(id.into_raw()));
}
paths_with_ids.push(path_with_ids);
}
let nodes_data = node_data.freeze();
let nodes_data_len = nodes_data.len();
let mut nodes = nodes_data.map_move(|n| QueryNode {
@ -422,28 +420,22 @@ impl QueryGraph {
successors: SmallBitmap::new(nodes_data_len),
});
let root_node = Interned::from_raw(root_node.into_raw());
let end_node = Interned::from_raw(end_node.into_raw());
let root_node = Interned::<QueryNode>::from_raw(root_node.into_raw());
let end_node = Interned::<QueryNode>::from_raw(end_node.into_raw());
for path in paths_with_ids {
let mut prev_node = root_node;
for node in path {
let (start_term, dest_term) = node;
let end_term = Interned::from_raw(dest_term.into_raw());
let src = if let Some(start_term) = start_term {
let start_term = Interned::from_raw(start_term.into_raw());
nodes.get_mut(prev_node).successors.insert(start_term);
nodes.get_mut(start_term).predecessors.insert(prev_node);
start_term
} else {
prev_node
};
nodes.get_mut(src).successors.insert(end_term);
nodes.get_mut(end_term).predecessors.insert(src);
prev_node = end_term;
let mut prev_node_id = root_node;
for node_id in path {
let prev_node = nodes.get_mut(prev_node_id);
prev_node.successors.insert(node_id);
let node = nodes.get_mut(node_id);
node.predecessors.insert(prev_node_id);
prev_node_id = node_id;
}
nodes.get_mut(prev_node).successors.insert(end_node);
nodes.get_mut(end_node).predecessors.insert(prev_node);
let prev_node = nodes.get_mut(prev_node_id);
prev_node.successors.insert(end_node);
let node = nodes.get_mut(end_node);
node.predecessors.insert(prev_node_id);
}
QueryGraph { root_node, end_node, nodes }

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,381 @@
use fst::automaton::Str;
use fst::{Automaton, IntoStreamer, Streamer};
use heed::types::DecodeIgnore;
use heed::BytesDecode;
use std::borrow::Cow;
use std::collections::BTreeSet;
use std::ops::ControlFlow;
use super::*;
use crate::search::fst_utils::{Complement, Intersection, StartsWith, Union};
use crate::search::new::query_term::TwoTypoTerm;
use crate::search::new::{limits, SearchContext};
use crate::search::{build_dfa, get_first};
use crate::{CboRoaringBitmapLenCodec, Result, MAX_WORD_LENGTH};
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
pub enum NumberOfTypos {
Zero,
One,
Two,
}
pub enum ZeroOrOneTypo {
Zero,
One,
}
impl Interned<QueryTerm> {
pub fn compute_fully_if_needed(self, ctx: &mut SearchContext) -> Result<()> {
let s = ctx.term_interner.get_mut(self);
if s.max_nbr_typos == 0 {
s.one_typo = Lazy::Init(OneTypoTerm::default());
s.two_typo = Lazy::Init(TwoTypoTerm::default());
} else if s.max_nbr_typos == 1 && s.one_typo.is_uninit() {
assert!(s.two_typo.is_uninit());
self.initialize_one_typo_subterm(ctx)?;
let s = ctx.term_interner.get_mut(self);
assert!(s.one_typo.is_init());
s.two_typo = Lazy::Init(TwoTypoTerm::default());
} else if s.max_nbr_typos > 1 && s.two_typo.is_uninit() {
assert!(s.two_typo.is_uninit());
self.initialize_one_and_two_typo_subterm(ctx)?;
let s = ctx.term_interner.get_mut(self);
assert!(s.one_typo.is_init() && s.two_typo.is_init());
}
Ok(())
}
}
fn find_zero_typo_prefix_derivations(
word_interned: Interned<String>,
fst: fst::Set<Cow<[u8]>>,
word_interner: &mut DedupInterner<String>,
mut visit: impl FnMut(Interned<String>) -> Result<ControlFlow<()>>,
) -> Result<()> {
let word = word_interner.get(word_interned).to_owned();
let word = word.as_str();
let prefix = Str::new(word).starts_with();
let mut stream = fst.search(prefix).into_stream();
while let Some(derived_word) = stream.next() {
let derived_word = std::str::from_utf8(derived_word)?.to_owned();
let derived_word_interned = word_interner.insert(derived_word);
if derived_word_interned != word_interned {
let cf = visit(derived_word_interned)?;
if cf.is_break() {
break;
}
}
}
Ok(())
}
fn find_zero_one_typo_derivations(
ctx: &mut SearchContext,
word_interned: Interned<String>,
is_prefix: bool,
mut visit: impl FnMut(Interned<String>, ZeroOrOneTypo) -> Result<ControlFlow<()>>,
) -> Result<()> {
let fst = ctx.get_words_fst()?;
let word = ctx.word_interner.get(word_interned).to_owned();
let word = word.as_str();
let dfa = build_dfa(word, 1, is_prefix);
let starts = StartsWith(Str::new(get_first(word)));
let mut stream = fst.search_with_state(Intersection(starts, &dfa)).into_stream();
while let Some((derived_word, state)) = stream.next() {
let derived_word = std::str::from_utf8(derived_word)?;
let derived_word = ctx.word_interner.insert(derived_word.to_owned());
let d = dfa.distance(state.1);
match d.to_u8() {
0 => {
if derived_word != word_interned {
let cf = visit(derived_word, ZeroOrOneTypo::Zero)?;
if cf.is_break() {
break;
}
}
}
1 => {
let cf = visit(derived_word, ZeroOrOneTypo::One)?;
if cf.is_break() {
break;
}
}
_ => {
unreachable!("One typo dfa produced multiple typos")
}
}
}
Ok(())
}
fn find_zero_one_two_typo_derivations(
word_interned: Interned<String>,
is_prefix: bool,
fst: fst::Set<Cow<[u8]>>,
word_interner: &mut DedupInterner<String>,
mut visit: impl FnMut(Interned<String>, NumberOfTypos) -> Result<ControlFlow<()>>,
) -> Result<()> {
let word = word_interner.get(word_interned).to_owned();
let word = word.as_str();
let starts = StartsWith(Str::new(get_first(word)));
let first = Intersection(build_dfa(word, 1, is_prefix), Complement(&starts));
let second_dfa = build_dfa(word, 2, is_prefix);
let second = Intersection(&second_dfa, &starts);
let automaton = Union(first, &second);
let mut stream = fst.search_with_state(automaton).into_stream();
while let Some((derived_word, state)) = stream.next() {
let derived_word = std::str::from_utf8(derived_word)?;
let derived_word_interned = word_interner.insert(derived_word.to_owned());
// in the case the typo is on the first letter, we know the number of typo
// is two
if get_first(derived_word) != get_first(word) {
let cf = visit(derived_word_interned, NumberOfTypos::Two)?;
if cf.is_break() {
break;
}
} else {
// Else, we know that it is the second dfa that matched and compute the
// correct distance
let d = second_dfa.distance((state.1).0);
match d.to_u8() {
0 => {
if derived_word_interned != word_interned {
let cf = visit(derived_word_interned, NumberOfTypos::Zero)?;
if cf.is_break() {
break;
}
}
}
1 => {
let cf = visit(derived_word_interned, NumberOfTypos::One)?;
if cf.is_break() {
break;
}
}
2 => {
let cf = visit(derived_word_interned, NumberOfTypos::Two)?;
if cf.is_break() {
break;
}
}
_ => unreachable!("2 typos DFA produced a distance greater than 2"),
}
}
}
Ok(())
}
pub fn partially_initialized_term_from_word(
ctx: &mut SearchContext,
word: &str,
max_typo: u8,
is_prefix: bool,
) -> Result<QueryTerm> {
let word_interned = ctx.word_interner.insert(word.to_owned());
if word.len() > MAX_WORD_LENGTH {
return Ok({
QueryTerm {
original: ctx.word_interner.insert(word.to_owned()),
ngram_words: None,
is_prefix: false,
max_nbr_typos: 0,
zero_typo: <_>::default(),
one_typo: Lazy::Init(<_>::default()),
two_typo: Lazy::Init(<_>::default()),
}
});
}
let fst = ctx.index.words_fst(ctx.txn)?;
let use_prefix_db = is_prefix
&& ctx
.index
.word_prefix_docids
.remap_data_type::<DecodeIgnore>()
.get(ctx.txn, word)?
.is_some();
let use_prefix_db = if use_prefix_db { Some(word_interned) } else { None };
let mut zero_typo = None;
let mut prefix_of = BTreeSet::new();
if fst.contains(word) {
zero_typo = Some(word_interned);
}
if is_prefix && use_prefix_db.is_none() {
find_zero_typo_prefix_derivations(
word_interned,
fst,
&mut ctx.word_interner,
|derived_word| {
if prefix_of.len() < limits::MAX_PREFIX_COUNT {
prefix_of.insert(derived_word);
Ok(ControlFlow::Continue(()))
} else {
Ok(ControlFlow::Break(()))
}
},
)?;
}
let synonyms = ctx.index.synonyms(ctx.txn)?;
let mut synonym_word_count = 0;
let synonyms = synonyms
.get(&vec![word.to_owned()])
.cloned()
.unwrap_or_default()
.into_iter()
.take(limits::MAX_SYNONYM_PHRASE_COUNT)
.filter_map(|words| {
if synonym_word_count + words.len() > limits::MAX_SYNONYM_WORD_COUNT {
return None;
}
synonym_word_count += words.len();
let words = words.into_iter().map(|w| Some(ctx.word_interner.insert(w))).collect();
Some(ctx.phrase_interner.insert(Phrase { words }))
})
.collect();
let zero_typo =
ZeroTypoTerm { phrase: None, exact: zero_typo, prefix_of, synonyms, use_prefix_db };
Ok(QueryTerm {
original: word_interned,
ngram_words: None,
max_nbr_typos: max_typo,
is_prefix,
zero_typo,
one_typo: Lazy::Uninit,
two_typo: Lazy::Uninit,
})
}
fn find_split_words(ctx: &mut SearchContext, word: &str) -> Result<Option<Interned<Phrase>>> {
if let Some((l, r)) = split_best_frequency(ctx, word)? {
Ok(Some(ctx.phrase_interner.insert(Phrase { words: vec![Some(l), Some(r)] })))
} else {
Ok(None)
}
}
impl Interned<QueryTerm> {
fn initialize_one_typo_subterm(self, ctx: &mut SearchContext) -> Result<()> {
let self_mut = ctx.term_interner.get_mut(self);
let QueryTerm { original, is_prefix, one_typo, .. } = self_mut;
let original = *original;
let is_prefix = *is_prefix;
// let original_str = ctx.word_interner.get(*original).to_owned();
if one_typo.is_init() {
return Ok(());
}
let mut one_typo_words = BTreeSet::new();
find_zero_one_typo_derivations(ctx, original, is_prefix, |derived_word, nbr_typos| {
match nbr_typos {
ZeroOrOneTypo::Zero => {}
ZeroOrOneTypo::One => {
if one_typo_words.len() < limits::MAX_ONE_TYPO_COUNT {
one_typo_words.insert(derived_word);
} else {
return Ok(ControlFlow::Break(()));
}
}
}
Ok(ControlFlow::Continue(()))
})?;
let original_str = ctx.word_interner.get(original).to_owned();
let split_words = find_split_words(ctx, original_str.as_str())?;
let one_typo = OneTypoTerm { split_words, one_typo: one_typo_words };
let self_mut = ctx.term_interner.get_mut(self);
self_mut.one_typo = Lazy::Init(one_typo);
Ok(())
}
fn initialize_one_and_two_typo_subterm(self, ctx: &mut SearchContext) -> Result<()> {
let self_mut = ctx.term_interner.get_mut(self);
let QueryTerm { original, is_prefix, two_typo, .. } = self_mut;
let original_str = ctx.word_interner.get(*original).to_owned();
if two_typo.is_init() {
return Ok(());
}
let mut one_typo_words = BTreeSet::new();
let mut two_typo_words = BTreeSet::new();
find_zero_one_two_typo_derivations(
*original,
*is_prefix,
ctx.index.words_fst(ctx.txn)?,
&mut ctx.word_interner,
|derived_word, nbr_typos| {
if one_typo_words.len() >= limits::MAX_ONE_TYPO_COUNT
&& two_typo_words.len() >= limits::MAX_TWO_TYPOS_COUNT
{
// No chance we will add either one- or two-typo derivations anymore, stop iterating.
return Ok(ControlFlow::Break(()));
}
match nbr_typos {
NumberOfTypos::Zero => {}
NumberOfTypos::One => {
if one_typo_words.len() < limits::MAX_ONE_TYPO_COUNT {
one_typo_words.insert(derived_word);
}
}
NumberOfTypos::Two => {
if two_typo_words.len() < limits::MAX_TWO_TYPOS_COUNT {
two_typo_words.insert(derived_word);
}
}
}
Ok(ControlFlow::Continue(()))
},
)?;
let split_words = find_split_words(ctx, original_str.as_str())?;
let self_mut = ctx.term_interner.get_mut(self);
let one_typo = OneTypoTerm { one_typo: one_typo_words, split_words };
let two_typo = TwoTypoTerm { two_typos: two_typo_words };
self_mut.one_typo = Lazy::Init(one_typo);
self_mut.two_typo = Lazy::Init(two_typo);
Ok(())
}
}
/// Split the original word into the two words that appear the
/// most next to each other in the index.
///
/// Return `None` if the original word cannot be split.
fn split_best_frequency(
ctx: &mut SearchContext,
original: &str,
) -> Result<Option<(Interned<String>, Interned<String>)>> {
let chars = original.char_indices().skip(1);
let mut best = None;
for (i, _) in chars {
let (left, right) = original.split_at(i);
let left = ctx.word_interner.insert(left.to_owned());
let right = ctx.word_interner.insert(right.to_owned());
if let Some(docid_bytes) = ctx.get_db_word_pair_proximity_docids(left, right, 1)? {
let frequency =
CboRoaringBitmapLenCodec::bytes_decode(docid_bytes).ok_or(heed::Error::Decoding)?;
if best.map_or(true, |(old, _, _)| frequency > old) {
best = Some((frequency, left, right));
}
}
}
Ok(best.map(|(_, left, right)| (left, right)))
}

View File

@ -0,0 +1,370 @@
mod compute_derivations;
mod ntypo_subset;
mod parse_query;
mod phrase;
use super::interner::{DedupInterner, Interned};
use super::{limits, SearchContext};
use crate::Result;
use std::collections::BTreeSet;
use std::ops::RangeInclusive;
use either::Either;
pub use ntypo_subset::NTypoTermSubset;
pub use parse_query::{located_query_terms_from_string, make_ngram, number_of_typos_allowed};
pub use phrase::Phrase;
use compute_derivations::partially_initialized_term_from_word;
/// A set of word derivations attached to a location in the search query.
#[derive(Clone, PartialEq, Eq, Hash)]
pub struct LocatedQueryTermSubset {
pub term_subset: QueryTermSubset,
pub positions: RangeInclusive<u16>,
pub term_ids: RangeInclusive<u8>,
}
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct QueryTermSubset {
original: Interned<QueryTerm>,
zero_typo_subset: NTypoTermSubset,
one_typo_subset: NTypoTermSubset,
two_typo_subset: NTypoTermSubset,
}
#[derive(Clone, PartialEq, Eq, Hash)]
pub struct QueryTerm {
original: Interned<String>,
ngram_words: Option<Vec<Interned<String>>>,
max_nbr_typos: u8,
is_prefix: bool,
zero_typo: ZeroTypoTerm,
// May not be computed yet
one_typo: Lazy<OneTypoTerm>,
// May not be computed yet
two_typo: Lazy<TwoTypoTerm>,
}
// SubTerms will be in a dedup interner
#[derive(Default, Clone, PartialEq, Eq, Hash)]
struct ZeroTypoTerm {
/// The original phrase, if any
phrase: Option<Interned<Phrase>>,
/// A single word equivalent to the original term, with zero typos
exact: Option<Interned<String>>,
/// All the words that contain the original word as prefix
prefix_of: BTreeSet<Interned<String>>,
/// All the synonyms of the original word or phrase
synonyms: BTreeSet<Interned<Phrase>>,
/// A prefix in the prefix databases matching the original word
use_prefix_db: Option<Interned<String>>,
}
#[derive(Default, Clone, PartialEq, Eq, Hash)]
struct OneTypoTerm {
/// The original word split into multiple consecutive words
split_words: Option<Interned<Phrase>>,
/// Words that are 1 typo away from the original word
one_typo: BTreeSet<Interned<String>>,
}
#[derive(Default, Clone, PartialEq, Eq, Hash)]
struct TwoTypoTerm {
/// Words that are 2 typos away from the original word
two_typos: BTreeSet<Interned<String>>,
}
#[derive(Clone, PartialEq, Eq, Hash)]
pub enum Lazy<T> {
Uninit,
Init(T),
}
impl<T> Lazy<T> {
pub fn is_init(&self) -> bool {
match self {
Lazy::Uninit => false,
Lazy::Init(_) => true,
}
}
pub fn is_uninit(&self) -> bool {
match self {
Lazy::Uninit => true,
Lazy::Init(_) => false,
}
}
}
#[derive(Clone, Copy)]
pub enum ExactTerm {
Phrase(Interned<Phrase>),
Word(Interned<String>),
}
impl ExactTerm {
pub fn interned_words<'ctx>(
&self,
ctx: &'ctx SearchContext<'ctx>,
) -> impl Iterator<Item = Option<Interned<String>>> + 'ctx {
match *self {
ExactTerm::Phrase(phrase) => {
let phrase = ctx.phrase_interner.get(phrase);
Either::Left(phrase.words.iter().copied())
}
ExactTerm::Word(word) => Either::Right(std::iter::once(Some(word))),
}
}
}
impl QueryTermSubset {
pub fn exact_term(&self, ctx: &SearchContext) -> Option<ExactTerm> {
let full_query_term = ctx.term_interner.get(self.original);
if full_query_term.ngram_words.is_some() {
return None;
}
// TODO: included in subset
if let Some(phrase) = full_query_term.zero_typo.phrase {
self.zero_typo_subset.contains_phrase(phrase).then_some(ExactTerm::Phrase(phrase))
} else if let Some(word) = full_query_term.zero_typo.exact {
self.zero_typo_subset.contains_word(word).then_some(ExactTerm::Word(word))
} else {
None
}
}
pub fn empty(for_term: Interned<QueryTerm>) -> Self {
Self {
original: for_term,
zero_typo_subset: NTypoTermSubset::Nothing,
one_typo_subset: NTypoTermSubset::Nothing,
two_typo_subset: NTypoTermSubset::Nothing,
}
}
pub fn full(for_term: Interned<QueryTerm>) -> Self {
Self {
original: for_term,
zero_typo_subset: NTypoTermSubset::All,
one_typo_subset: NTypoTermSubset::All,
two_typo_subset: NTypoTermSubset::All,
}
}
pub fn union(&mut self, other: &Self) {
assert!(self.original == other.original);
self.zero_typo_subset.union(&other.zero_typo_subset);
self.one_typo_subset.union(&other.one_typo_subset);
self.two_typo_subset.union(&other.two_typo_subset);
}
pub fn intersect(&mut self, other: &Self) {
assert!(self.original == other.original);
self.zero_typo_subset.intersect(&other.zero_typo_subset);
self.one_typo_subset.intersect(&other.one_typo_subset);
self.two_typo_subset.intersect(&other.two_typo_subset);
}
pub fn use_prefix_db(&self, ctx: &SearchContext) -> Option<Interned<String>> {
let original = ctx.term_interner.get(self.original);
let Some(use_prefix_db) = original.zero_typo.use_prefix_db else {
return None
};
match &self.zero_typo_subset {
NTypoTermSubset::All => Some(use_prefix_db),
NTypoTermSubset::Subset { words, phrases: _ } => {
// TODO: use a subset of prefix words instead
if words.contains(&use_prefix_db) {
Some(use_prefix_db)
} else {
None
}
}
NTypoTermSubset::Nothing => None,
}
}
pub fn all_single_words_except_prefix_db(
&self,
ctx: &mut SearchContext,
) -> Result<BTreeSet<Interned<String>>> {
let mut result = BTreeSet::default();
// TODO: a compute_partially funtion
if !self.one_typo_subset.is_empty() || !self.two_typo_subset.is_empty() {
self.original.compute_fully_if_needed(ctx)?;
}
let original = ctx.term_interner.get_mut(self.original);
if !self.zero_typo_subset.is_empty() {
let ZeroTypoTerm {
phrase: _,
exact: zero_typo,
prefix_of,
synonyms: _,
use_prefix_db: _,
} = &original.zero_typo;
result.extend(zero_typo.iter().copied());
result.extend(prefix_of.iter().copied());
};
match &self.one_typo_subset {
NTypoTermSubset::All => {
let Lazy::Init(OneTypoTerm { split_words: _, one_typo }) = &original.one_typo else {
panic!()
};
result.extend(one_typo.iter().copied())
}
NTypoTermSubset::Subset { words, phrases: _ } => {
let Lazy::Init(OneTypoTerm { split_words: _, one_typo }) = &original.one_typo else {
panic!()
};
result.extend(one_typo.intersection(words));
}
NTypoTermSubset::Nothing => {}
};
match &self.two_typo_subset {
NTypoTermSubset::All => {
let Lazy::Init(TwoTypoTerm { two_typos }) = &original.two_typo else {
panic!()
};
result.extend(two_typos.iter().copied());
}
NTypoTermSubset::Subset { words, phrases: _ } => {
let Lazy::Init(TwoTypoTerm { two_typos }) = &original.two_typo else {
panic!()
};
result.extend(two_typos.intersection(words));
}
NTypoTermSubset::Nothing => {}
};
Ok(result)
}
pub fn all_phrases(&self, ctx: &mut SearchContext) -> Result<BTreeSet<Interned<Phrase>>> {
let mut result = BTreeSet::default();
if !self.one_typo_subset.is_empty() {
// TODO: compute less than fully if possible
self.original.compute_fully_if_needed(ctx)?;
}
let original = ctx.term_interner.get_mut(self.original);
let ZeroTypoTerm { phrase, exact: _, prefix_of: _, synonyms, use_prefix_db: _ } =
&original.zero_typo;
result.extend(phrase.iter().copied());
result.extend(synonyms.iter().copied());
if !self.one_typo_subset.is_empty() {
let Lazy::Init(OneTypoTerm { split_words, one_typo: _ }) = &original.one_typo else {
panic!();
};
result.extend(split_words.iter().copied());
}
Ok(result)
}
pub fn original_phrase(&self, ctx: &SearchContext) -> Option<Interned<Phrase>> {
let t = ctx.term_interner.get(self.original);
if let Some(p) = t.zero_typo.phrase {
if self.zero_typo_subset.contains_phrase(p) {
return Some(p);
}
}
None
}
pub fn max_nbr_typos(&self, ctx: &SearchContext) -> u8 {
let t = ctx.term_interner.get(self.original);
match t.max_nbr_typos {
0 => 0,
1 => {
if self.one_typo_subset.is_empty() {
0
} else {
1
}
}
2 => {
if self.two_typo_subset.is_empty() {
if self.one_typo_subset.is_empty() {
0
} else {
1
}
} else {
2
}
}
_ => panic!(),
}
}
pub fn clear_zero_typo_subset(&mut self) {
self.zero_typo_subset = NTypoTermSubset::Nothing;
}
pub fn clear_one_typo_subset(&mut self) {
self.one_typo_subset = NTypoTermSubset::Nothing;
}
pub fn clear_two_typo_subset(&mut self) {
self.two_typo_subset = NTypoTermSubset::Nothing;
}
pub fn description(&self, ctx: &SearchContext) -> String {
let t = ctx.term_interner.get(self.original);
ctx.word_interner.get(t.original).to_owned()
}
}
impl ZeroTypoTerm {
fn is_empty(&self) -> bool {
let ZeroTypoTerm { phrase, exact: zero_typo, prefix_of, synonyms, use_prefix_db } = self;
phrase.is_none()
&& zero_typo.is_none()
&& prefix_of.is_empty()
&& synonyms.is_empty()
&& use_prefix_db.is_none()
}
}
impl OneTypoTerm {
fn is_empty(&self) -> bool {
let OneTypoTerm { split_words, one_typo } = self;
one_typo.is_empty() && split_words.is_none()
}
}
impl TwoTypoTerm {
fn is_empty(&self) -> bool {
let TwoTypoTerm { two_typos } = self;
two_typos.is_empty()
}
}
impl QueryTerm {
fn is_empty(&self) -> bool {
let Lazy::Init(one_typo) = &self.one_typo else {
return false;
};
let Lazy::Init(two_typo) = &self.two_typo else {
return false;
};
self.zero_typo.is_empty() && one_typo.is_empty() && two_typo.is_empty()
}
}
impl Interned<QueryTerm> {
/// Return the original word from the given query term
fn original_single_word(self, ctx: &SearchContext) -> Option<Interned<String>> {
let self_ = ctx.term_interner.get(self);
if self_.ngram_words.is_some() {
None
} else {
Some(self_.original)
}
}
}
/// A query term coupled with its position in the user's search query.
#[derive(Clone)]
pub struct LocatedQueryTerm {
pub value: Interned<QueryTerm>,
pub positions: RangeInclusive<u16>,
}
impl LocatedQueryTerm {
/// Return `true` iff the term is empty
pub fn is_empty(&self, interner: &DedupInterner<QueryTerm>) -> bool {
interner.get(self.value).is_empty()
}
}

View File

@ -0,0 +1,80 @@
use std::collections::BTreeSet;
use crate::search::new::interner::Interned;
use super::Phrase;
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub enum NTypoTermSubset {
All,
Subset {
words: BTreeSet<Interned<String>>,
phrases: BTreeSet<Interned<Phrase>>,
// TODO: prefixes: BTreeSet<Interned<String>>,
},
Nothing,
}
impl NTypoTermSubset {
pub fn contains_word(&self, word: Interned<String>) -> bool {
match self {
NTypoTermSubset::All => true,
NTypoTermSubset::Subset { words, phrases: _ } => words.contains(&word),
NTypoTermSubset::Nothing => false,
}
}
pub fn contains_phrase(&self, phrase: Interned<Phrase>) -> bool {
match self {
NTypoTermSubset::All => true,
NTypoTermSubset::Subset { words: _, phrases } => phrases.contains(&phrase),
NTypoTermSubset::Nothing => false,
}
}
pub fn is_empty(&self) -> bool {
match self {
NTypoTermSubset::All => false,
NTypoTermSubset::Subset { words, phrases } => words.is_empty() && phrases.is_empty(),
NTypoTermSubset::Nothing => true,
}
}
pub fn union(&mut self, other: &Self) {
match self {
Self::All => {}
Self::Subset { words, phrases } => match other {
Self::All => {
*self = Self::All;
}
Self::Subset { words: w2, phrases: p2 } => {
words.extend(w2);
phrases.extend(p2);
}
Self::Nothing => {}
},
Self::Nothing => {
*self = other.clone();
}
}
}
pub fn intersect(&mut self, other: &Self) {
match self {
Self::All => *self = other.clone(),
Self::Subset { words, phrases } => match other {
Self::All => {}
Self::Subset { words: w2, phrases: p2 } => {
let mut ws = BTreeSet::new();
for w in words.intersection(w2) {
ws.insert(*w);
}
let mut ps = BTreeSet::new();
for p in phrases.intersection(p2) {
ps.insert(*p);
}
*words = ws;
*phrases = ps;
}
Self::Nothing => *self = Self::Nothing,
},
Self::Nothing => {}
}
}
}

View File

@ -0,0 +1,281 @@
use charabia::{normalizer::NormalizedTokenIter, SeparatorKind, TokenKind};
use crate::{Result, SearchContext, MAX_WORD_LENGTH};
use super::*;
/// Convert the tokenised search query into a list of located query terms.
// TODO: checking if the positions are correct for phrases, separators, ngrams
pub fn located_query_terms_from_string(
ctx: &mut SearchContext,
query: NormalizedTokenIter<&[u8]>,
words_limit: Option<usize>,
) -> Result<Vec<LocatedQueryTerm>> {
let nbr_typos = number_of_typos_allowed(ctx)?;
let mut located_terms = Vec::new();
let mut phrase: Option<PhraseBuilder> = None;
let parts_limit = words_limit.unwrap_or(usize::MAX);
// start with the last position as we will wrap around to position 0 at the beginning of the loop below.
let mut position = u16::MAX;
let mut peekable = query.take(super::limits::MAX_TOKEN_COUNT).peekable();
while let Some(token) = peekable.next() {
// early return if word limit is exceeded
if located_terms.len() >= parts_limit {
return Ok(located_terms);
}
match token.kind {
TokenKind::Word | TokenKind::StopWord => {
// On first loop, goes from u16::MAX to 0, then normal increment.
position = position.wrapping_add(1);
// 1. if the word is quoted we push it in a phrase-buffer waiting for the ending quote,
// 2. if the word is not the last token of the query and is not a stop_word we push it as a non-prefix word,
// 3. if the word is the last token of the query we push it as a prefix word.
if let Some(phrase) = &mut phrase {
phrase.push_word(ctx, &token, position)
} else if peekable.peek().is_some() {
match token.kind {
TokenKind::Word => {
let word = token.lemma();
let term = partially_initialized_term_from_word(
ctx,
word,
nbr_typos(word),
false,
)?;
let located_term = LocatedQueryTerm {
value: ctx.term_interner.push(term),
positions: position..=position,
};
located_terms.push(located_term);
}
TokenKind::StopWord | TokenKind::Separator(_) | TokenKind::Unknown => {}
}
} else {
let word = token.lemma();
let term =
partially_initialized_term_from_word(ctx, word, nbr_typos(word), true)?;
let located_term = LocatedQueryTerm {
value: ctx.term_interner.push(term),
positions: position..=position,
};
located_terms.push(located_term);
}
}
TokenKind::Separator(separator_kind) => {
match separator_kind {
SeparatorKind::Hard => {
position += 1;
}
SeparatorKind::Soft => {
position += 0;
}
}
phrase = 'phrase: {
let phrase = phrase.take();
// If we have a hard separator inside a phrase, we immediately start a new phrase
let phrase = if separator_kind == SeparatorKind::Hard {
if let Some(phrase) = phrase {
if let Some(located_query_term) = phrase.build(ctx) {
located_terms.push(located_query_term)
}
Some(PhraseBuilder::empty())
} else {
None
}
} else {
phrase
};
// We close and start a new phrase depending on the number of double quotes
let mut quote_count = token.lemma().chars().filter(|&s| s == '"').count();
if quote_count == 0 {
break 'phrase phrase;
}
// Consume the closing quote and the phrase
if let Some(phrase) = phrase {
// Per the check above, quote_count > 0
quote_count -= 1;
if let Some(located_query_term) = phrase.build(ctx) {
located_terms.push(located_query_term)
}
}
// Start new phrase if the token ends with an opening quote
(quote_count % 2 == 1).then_some(PhraseBuilder::empty())
};
}
_ => (),
}
}
// If a quote is never closed, we consider all of the end of the query as a phrase.
if let Some(phrase) = phrase.take() {
if let Some(located_query_term) = phrase.build(ctx) {
located_terms.push(located_query_term);
}
}
Ok(located_terms)
}
pub fn number_of_typos_allowed<'ctx>(
ctx: &SearchContext<'ctx>,
) -> Result<impl Fn(&str) -> u8 + 'ctx> {
let authorize_typos = ctx.index.authorize_typos(ctx.txn)?;
let min_len_one_typo = ctx.index.min_word_len_one_typo(ctx.txn)?;
let min_len_two_typos = ctx.index.min_word_len_two_typos(ctx.txn)?;
// TODO: should `exact_words` also disable prefix search, ngrams, split words, or synonyms?
let exact_words = ctx.index.exact_words(ctx.txn)?;
Ok(Box::new(move |word: &str| {
if !authorize_typos
|| word.len() < min_len_one_typo as usize
|| exact_words.as_ref().map_or(false, |fst| fst.contains(word))
{
0
} else if word.len() < min_len_two_typos as usize {
1
} else {
2
}
}))
}
pub fn make_ngram(
ctx: &mut SearchContext,
terms: &[LocatedQueryTerm],
number_of_typos_allowed: &impl Fn(&str) -> u8,
) -> Result<Option<LocatedQueryTerm>> {
assert!(!terms.is_empty());
for t in terms {
if ctx.term_interner.get(t.value).zero_typo.phrase.is_some() {
return Ok(None);
}
}
for ts in terms.windows(2) {
let [t1, t2] = ts else { panic!() };
if *t1.positions.end() != t2.positions.start() - 1 {
return Ok(None);
}
}
let mut words_interned = vec![];
for term in terms {
if let Some(original_term_word) = term.value.original_single_word(ctx) {
words_interned.push(original_term_word);
} else {
return Ok(None);
}
}
let words =
words_interned.iter().map(|&i| ctx.word_interner.get(i).to_owned()).collect::<Vec<_>>();
let start = *terms.first().as_ref().unwrap().positions.start();
let end = *terms.last().as_ref().unwrap().positions.end();
let is_prefix = ctx.term_interner.get(terms.last().as_ref().unwrap().value).is_prefix;
let ngram_str = words.join("");
if ngram_str.len() > MAX_WORD_LENGTH {
return Ok(None);
}
let ngram_str_interned = ctx.word_interner.insert(ngram_str.clone());
let max_nbr_typos =
number_of_typos_allowed(ngram_str.as_str()).saturating_sub(terms.len() as u8 - 1);
let mut term = partially_initialized_term_from_word(ctx, &ngram_str, max_nbr_typos, is_prefix)?;
// Now add the synonyms
let index_synonyms = ctx.index.synonyms(ctx.txn)?;
term.zero_typo.synonyms.extend(
index_synonyms.get(&words).cloned().unwrap_or_default().into_iter().map(|words| {
let words = words.into_iter().map(|w| Some(ctx.word_interner.insert(w))).collect();
ctx.phrase_interner.insert(Phrase { words })
}),
);
let term = QueryTerm {
original: ngram_str_interned,
ngram_words: Some(words_interned),
is_prefix,
max_nbr_typos,
zero_typo: term.zero_typo,
one_typo: Lazy::Uninit,
two_typo: Lazy::Uninit,
};
let term = LocatedQueryTerm { value: ctx.term_interner.push(term), positions: start..=end };
Ok(Some(term))
}
struct PhraseBuilder {
words: Vec<Option<Interned<String>>>,
start: u16,
end: u16,
}
impl PhraseBuilder {
fn empty() -> Self {
Self { words: Default::default(), start: u16::MAX, end: u16::MAX }
}
fn is_empty(&self) -> bool {
self.words.is_empty()
}
// precondition: token has kind Word or StopWord
fn push_word(&mut self, ctx: &mut SearchContext, token: &charabia::Token, position: u16) {
if self.is_empty() {
self.start = position;
}
self.end = position;
if let TokenKind::StopWord = token.kind {
self.words.push(None);
} else {
// token has kind Word
let word = ctx.word_interner.insert(token.lemma().to_string());
// TODO: in a phrase, check that every word exists
// otherwise return an empty term
self.words.push(Some(word));
}
}
fn build(self, ctx: &mut SearchContext) -> Option<LocatedQueryTerm> {
if self.is_empty() {
return None;
}
Some(LocatedQueryTerm {
value: ctx.term_interner.push({
let phrase = ctx.phrase_interner.insert(Phrase { words: self.words });
let phrase_desc = phrase.description(ctx);
QueryTerm {
original: ctx.word_interner.insert(phrase_desc),
ngram_words: None,
max_nbr_typos: 0,
is_prefix: false,
zero_typo: ZeroTypoTerm {
phrase: Some(phrase),
exact: None,
prefix_of: BTreeSet::default(),
synonyms: BTreeSet::default(),
use_prefix_db: None,
},
one_typo: Lazy::Uninit,
two_typo: Lazy::Uninit,
}
}),
positions: self.start..=self.end,
})
}
}

View File

@ -0,0 +1,16 @@
use itertools::Itertools;
use crate::{search::new::interner::Interned, SearchContext};
/// A phrase in the user's search query, consisting of several words
/// that must appear side-by-side in the search results.
#[derive(Default, Clone, PartialEq, Eq, Hash)]
pub struct Phrase {
pub words: Vec<Option<Interned<String>>>,
}
impl Interned<Phrase> {
pub fn description(self, ctx: &SearchContext) -> String {
let p = ctx.phrase_interner.get(self);
p.words.iter().flatten().map(|w| ctx.word_interner.get(*w)).join(" ")
}
}

View File

@ -57,9 +57,7 @@ impl RankingRuleGraphTrait for ProximityGraph {
Ok(format!("{cost}: cost"))
}
ProximityCondition::Term { term } => {
let original_term = ctx.term_interner.get(term.term_subset.original);
let original_word = ctx.word_interner.get(original_term.original);
Ok(format!("{original_word} : exists"))
Ok(format!("{} : exists", term.term_subset.description(ctx)))
}
}
}

View File

@ -3,7 +3,7 @@ use roaring::RoaringBitmap;
use super::{ComputedCondition, DeadEndsCache, RankingRuleGraph, RankingRuleGraphTrait};
use crate::search::new::interner::{DedupInterner, Interned, MappedInterner};
use crate::search::new::logger::SearchLogger;
use crate::search::new::query_term::{LocatedQueryTermSubset, NTypoTermSubset};
use crate::search::new::query_term::LocatedQueryTermSubset;
use crate::search::new::resolve_query_graph::compute_query_term_subset_docids;
use crate::search::new::{QueryGraph, QueryNode, SearchContext};
use crate::Result;
@ -43,8 +43,7 @@ impl RankingRuleGraphTrait for TypoGraph {
_from: Option<&LocatedQueryTermSubset>,
to_term: &LocatedQueryTermSubset,
) -> Result<Vec<(u32, Interned<Self::Condition>)>> {
let term = to_term; // LocatedQueryTermSubset { term_subset, positions: _, term_ids } = to_term;
let original_full_term = ctx.term_interner.get(term.term_subset.original);
let term = to_term;
let mut edges = vec![];
// Ngrams have a base typo cost
@ -52,20 +51,20 @@ impl RankingRuleGraphTrait for TypoGraph {
// 3-gram -> equivalent to 2 typos
let base_cost = if term.term_ids.len() == 1 { 0 } else { term.term_ids.len() as u32 };
for nbr_typos in 0..=original_full_term.max_nbr_typos {
for nbr_typos in 0..=term.term_subset.max_nbr_typos(ctx) {
let mut term = term.clone();
match nbr_typos {
0 => {
term.term_subset.one_typo_subset = NTypoTermSubset::Nothing;
term.term_subset.two_typo_subset = NTypoTermSubset::Nothing;
term.term_subset.clear_one_typo_subset();
term.term_subset.clear_two_typo_subset();
}
1 => {
term.term_subset.zero_typo_subset = NTypoTermSubset::Nothing;
term.term_subset.two_typo_subset = NTypoTermSubset::Nothing;
term.term_subset.clear_zero_typo_subset();
term.term_subset.clear_two_typo_subset();
}
2 => {
term.term_subset.zero_typo_subset = NTypoTermSubset::Nothing;
term.term_subset.one_typo_subset = NTypoTermSubset::Nothing;
term.term_subset.clear_zero_typo_subset();
term.term_subset.clear_one_typo_subset();
}
_ => panic!(),
};
@ -92,9 +91,6 @@ impl RankingRuleGraphTrait for TypoGraph {
fn label_for_condition(ctx: &mut SearchContext, condition: &Self::Condition) -> Result<String> {
let TypoCondition { term, nbr_typos } = condition;
let original_term = ctx.term_interner.get(term.term_subset.original);
let original = ctx.word_interner.get(original_term.original);
Ok(format!("{original}: {nbr_typos}"))
Ok(format!("{}: {nbr_typos}", term.term_subset.description(ctx)))
}
}

View File

@ -2,8 +2,6 @@ use roaring::RoaringBitmap;
use super::logger::SearchLogger;
use super::{QueryGraph, SearchContext};
// use crate::search::new::sort::Sort;
use crate::search::new::distinct::{apply_distinct_rule, DistinctOutput};
use crate::Result;
/// An internal trait implemented by only [`PlaceholderQuery`] and [`QueryGraph`]
@ -69,171 +67,3 @@ pub struct RankingRuleOutput<Q> {
/// The allowed candidates for the child ranking rule
pub candidates: RoaringBitmap,
}
pub fn bucket_sort<'ctx, Q: RankingRuleQueryTrait>(
ctx: &mut SearchContext<'ctx>,
mut ranking_rules: Vec<BoxRankingRule<'ctx, Q>>,
query: &Q,
universe: &RoaringBitmap,
from: usize,
length: usize,
logger: &mut dyn SearchLogger<Q>,
) -> Result<Vec<u32>> {
logger.initial_query(query);
logger.ranking_rules(&ranking_rules);
logger.initial_universe(universe);
let distinct_fid = if let Some(field) = ctx.index.distinct_field(ctx.txn)? {
ctx.index.fields_ids_map(ctx.txn)?.id(field)
} else {
None
};
if universe.len() < from as u64 {
return Ok(vec![]);
}
let ranking_rules_len = ranking_rules.len();
logger.start_iteration_ranking_rule(0, ranking_rules[0].as_ref(), query, universe);
ranking_rules[0].start_iteration(ctx, logger, universe, query)?;
let mut ranking_rule_universes: Vec<RoaringBitmap> =
vec![RoaringBitmap::default(); ranking_rules_len];
ranking_rule_universes[0] = universe.clone();
let mut cur_ranking_rule_index = 0;
/// Finish iterating over the current ranking rule, yielding
/// control to the parent (or finishing the search if not possible).
/// Update the candidates accordingly and inform the logger.
macro_rules! back {
() => {
assert!(ranking_rule_universes[cur_ranking_rule_index].is_empty());
logger.end_iteration_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&ranking_rule_universes[cur_ranking_rule_index],
);
ranking_rule_universes[cur_ranking_rule_index].clear();
ranking_rules[cur_ranking_rule_index].end_iteration(ctx, logger);
if cur_ranking_rule_index == 0 {
break;
} else {
cur_ranking_rule_index -= 1;
}
};
}
let mut results = vec![];
let mut cur_offset = 0usize;
/// Add the candidates to the results. Take `distinct`, `from`, `length`, and `cur_offset`
/// into account and inform the logger.
macro_rules! maybe_add_to_results {
($candidates:expr) => {
// First apply the distinct rule on the candidates, reducing the universes if necessary
let candidates = if let Some(distinct_fid) = distinct_fid {
let DistinctOutput { remaining, excluded } = apply_distinct_rule(ctx, distinct_fid, $candidates)?;
for universe in ranking_rule_universes.iter_mut() {
*universe -= &excluded;
}
remaining
} else {
$candidates.clone()
};
let len = candidates.len();
// if the candidates are empty, there is nothing to do;
if !candidates.is_empty() {
// if we still haven't reached the first document to return
if cur_offset < from {
// and if no document from this bucket can be returned
if cur_offset + (candidates.len() as usize) < from {
// then just skip the bucket
logger.skip_bucket_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&candidates,
);
} else {
// otherwise, skip some of the documents and add some of the rest, in order of ids
let all_candidates = candidates.iter().collect::<Vec<_>>();
let (skipped_candidates, candidates) =
all_candidates.split_at(from - cur_offset);
logger.skip_bucket_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&skipped_candidates.into_iter().collect(),
);
let candidates = candidates
.iter()
.take(length - results.len())
.copied()
.collect::<Vec<_>>();
logger.add_to_results(&candidates);
results.extend(&candidates);
}
} else {
// if we have passed the offset already, add some of the documents (up to the limit)
let candidates =
candidates.iter().take(length - results.len()).collect::<Vec<u32>>();
logger.add_to_results(&candidates);
results.extend(&candidates);
}
}
cur_offset += len as usize;
};
}
while results.len() < length {
// The universe for this bucket is zero or one element, so we don't need to sort
// anything, just extend the results and go back to the parent ranking rule.
if ranking_rule_universes[cur_ranking_rule_index].len() <= 1 {
maybe_add_to_results!(&ranking_rule_universes[cur_ranking_rule_index]);
ranking_rule_universes[cur_ranking_rule_index].clear();
back!();
continue;
}
let Some(next_bucket) = ranking_rules[cur_ranking_rule_index].next_bucket(ctx, logger, &ranking_rule_universes[cur_ranking_rule_index])? else {
back!();
continue;
};
logger.next_bucket_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&ranking_rule_universes[cur_ranking_rule_index],
&next_bucket.candidates,
);
debug_assert!(
ranking_rule_universes[cur_ranking_rule_index].is_superset(&next_bucket.candidates)
);
ranking_rule_universes[cur_ranking_rule_index] -= &next_bucket.candidates;
if cur_ranking_rule_index == ranking_rules_len - 1
|| next_bucket.candidates.len() <= 1
|| cur_offset + (next_bucket.candidates.len() as usize) < from
{
maybe_add_to_results!(&next_bucket.candidates);
continue;
}
cur_ranking_rule_index += 1;
ranking_rule_universes[cur_ranking_rule_index] = next_bucket.candidates.clone();
logger.start_iteration_ranking_rule(
cur_ranking_rule_index,
ranking_rules[cur_ranking_rule_index].as_ref(),
&next_bucket.query,
&ranking_rule_universes[cur_ranking_rule_index],
);
ranking_rules[cur_ranking_rule_index].start_iteration(
ctx,
logger,
&next_bucket.candidates,
&next_bucket.query,
)?;
}
Ok(results)
}

View File

@ -0,0 +1,590 @@
/*!
This module tests the "distinct attribute" feature, and its
interaction with other ranking rules.
1. no duplicate distinct attributes are ever returned
2. only the best document (according to the search rules) for each distinct value appears in the result
3. if a document does not have a distinct attribute, then the distinct rule does not apply to it
It doesn't test properly:
- combination of distinct + exhaustive_nbr_hits (because we know it's incorrect)
- distinct attributes with arrays (because we know it's incorrect as well)
*/
use std::collections::HashSet;
use big_s::S;
use heed::RoTxn;
use maplit::hashset;
use crate::{
index::tests::TempIndex, AscDesc, Criterion, Index, Member, Search, SearchResult,
TermsMatchingStrategy,
};
use super::collect_field_values;
fn create_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_sortable_fields(hashset! { S("rank1"), S("letter") });
s.set_distinct_field("letter".to_owned());
s.set_criteria(vec![Criterion::Words]);
})
.unwrap();
index
.add_documents(documents!([
{
"id": 0,
"letter": "A",
"rank1": 0,
"text": "the quick brown fox jamps over the lazy dog",
},
{
"id": 1,
"letter": "A",
"rank1": 1,
"text": "the quick brown fox jumpes over the lazy dog",
},
{
"id": 2,
"letter": "B",
"rank1": 0,
"text": "the quick brown foxjumps over the lazy dog",
},
{
"id": 3,
"letter": "B",
"rank1": 1,
"text": "the quick brown fox jumps over the lazy dog",
},
{
"id": 4,
"letter": "B",
"rank1": 2,
"text": "the quick brown fox jumps over the lazy",
},
{
"id": 5,
"letter": "C",
"rank1": 0,
"text": "the quickbrownfox jumps over the lazy",
},
{
"id": 6,
"letter": "C",
"rank1": 1,
"text": "the quick brown fox jumpss over the lazy",
},
{
"id": 7,
"letter": "C",
"rank1": 2,
"text": "the quick brown fox jumps over the lazy",
},
{
"id": 8,
"letter": "D",
"rank1": 0,
"text": "the quick brown fox jumps over the lazy",
},
{
"id": 9,
"letter": "E",
"rank1": 0,
"text": "the quick brown fox jumps over the lazy",
},
{
"id": 10,
"letter": "E",
"rank1": 1,
"text": "the quackbrown foxjunps over",
},
{
"id": 11,
"letter": "E",
"rank1": 2,
"text": "the quicko browno fox junps over",
},
{
"id": 12,
"letter": "E",
"rank1": 3,
"text": "the quicko browno fox jumps over",
},
{
"id": 13,
"letter": "E",
"rank1": 4,
"text": "the quick brewn fox jumps over",
},
{
"id": 14,
"letter": "E",
"rank1": 5,
"text": "the quick brown fox jumps over",
},
{
"id": 15,
"letter": "F",
"rank1": 0,
"text": "the quick brownf fox jumps over",
},
{
"id": 16,
"letter": "F",
"rank1": 1,
"text": "the quic brown fox jamps over",
},
{
"id": 17,
"letter": "F",
"rank1": 2,
"text": "thequick browns fox jimps",
},
{
"id": 18,
"letter": "G",
"rank1": 0,
"text": "the qick brown fox jumps",
},
{
"id": 19,
"letter": "G",
"rank1": 1,
"text": "the quick brownfoxjumps",
},
{
"id": 20,
"letter": "H",
"rank1": 0,
"text": "the quick brow fox jumps",
},
{
"id": 21,
"letter": "I",
"rank1": 0,
"text": "the quick brown fox jpmps",
},
{
"id": 22,
"letter": "I",
"rank1": 1,
"text": "the quick brown fox jumps",
},
{
"id": 23,
"letter": "I",
"rank1": 2,
"text": "the quick",
},
{
"id": 24,
"rank1": 0,
"text": "the quick",
},
{
"id": 25,
"rank1": 1,
"text": "the quick brown",
},
{
"id": 26,
"rank1": 2,
"text": "the quick brown fox",
},
{
"id": 26,
"rank1": 3,
"text": "the quick brown fox jumps over the lazy dog",
},
]))
.unwrap();
index
}
fn verify_distinct(index: &Index, txn: &RoTxn, docids: &[u32]) -> Vec<String> {
let vs = collect_field_values(index, txn, index.distinct_field(txn).unwrap().unwrap(), docids);
let mut unique = HashSet::new();
for v in vs.iter() {
if v == "__does_not_exist__" {
continue;
}
assert!(unique.insert(v.clone()));
}
vs
}
#[test]
fn test_distinct_placeholder_no_ranking_rules() {
let index = create_index();
let txn = index.read_txn().unwrap();
let s = Search::new(&txn, &index);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 2, 5, 8, 9, 15, 18, 20, 21, 24, 25, 26]");
let distinct_values = verify_distinct(&index, &txn, &documents_ids);
insta::assert_debug_snapshot!(distinct_values, @r###"
[
"\"A\"",
"\"B\"",
"\"C\"",
"\"D\"",
"\"E\"",
"\"F\"",
"\"G\"",
"\"H\"",
"\"I\"",
"__does_not_exist__",
"__does_not_exist__",
"__does_not_exist__",
]
"###);
}
#[test]
fn test_distinct_placeholder_sort() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Sort]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.sort_criteria(vec![AscDesc::Desc(Member::Field(S("rank1")))]);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[14, 26, 4, 7, 17, 23, 1, 19, 25, 8, 20, 24]");
let distinct_values = verify_distinct(&index, &txn, &documents_ids);
insta::assert_debug_snapshot!(distinct_values, @r###"
[
"\"E\"",
"__does_not_exist__",
"\"B\"",
"\"C\"",
"\"F\"",
"\"I\"",
"\"A\"",
"\"G\"",
"__does_not_exist__",
"\"D\"",
"\"H\"",
"__does_not_exist__",
]
"###);
let rank_values = collect_field_values(&index, &txn, "rank1", &documents_ids);
insta::assert_debug_snapshot!(rank_values, @r###"
[
"5",
"3",
"2",
"2",
"2",
"2",
"1",
"1",
"1",
"0",
"0",
"0",
]
"###);
let mut s = Search::new(&txn, &index);
s.sort_criteria(vec![AscDesc::Desc(Member::Field(S("letter")))]);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[21, 20, 18, 15, 9, 8, 5, 2, 0, 24, 25, 26]");
let distinct_values = verify_distinct(&index, &txn, &documents_ids);
insta::assert_debug_snapshot!(distinct_values, @r###"
[
"\"I\"",
"\"H\"",
"\"G\"",
"\"F\"",
"\"E\"",
"\"D\"",
"\"C\"",
"\"B\"",
"\"A\"",
"__does_not_exist__",
"__does_not_exist__",
"__does_not_exist__",
]
"###);
let rank_values = collect_field_values(&index, &txn, "rank1", &documents_ids);
insta::assert_debug_snapshot!(rank_values, @r###"
[
"0",
"0",
"0",
"0",
"0",
"0",
"0",
"0",
"0",
"0",
"1",
"3",
]
"###);
let mut s = Search::new(&txn, &index);
s.sort_criteria(vec![
AscDesc::Desc(Member::Field(S("letter"))),
AscDesc::Desc(Member::Field(S("rank1"))),
]);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[23, 20, 19, 17, 14, 8, 7, 4, 1, 26, 25, 24]");
let distinct_values = verify_distinct(&index, &txn, &documents_ids);
insta::assert_debug_snapshot!(distinct_values, @r###"
[
"\"I\"",
"\"H\"",
"\"G\"",
"\"F\"",
"\"E\"",
"\"D\"",
"\"C\"",
"\"B\"",
"\"A\"",
"__does_not_exist__",
"__does_not_exist__",
"__does_not_exist__",
]
"###);
let rank_values = collect_field_values(&index, &txn, "rank1", &documents_ids);
insta::assert_debug_snapshot!(rank_values, @r###"
[
"2",
"0",
"1",
"2",
"5",
"0",
"2",
"2",
"1",
"3",
"1",
"0",
]
"###);
}
#[test]
fn test_distinct_words() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Words]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.query("the quick brown fox jumps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 2, 26, 5, 8, 9, 15, 18, 20, 21, 25, 24]");
let distinct_values = verify_distinct(&index, &txn, &documents_ids);
insta::assert_debug_snapshot!(distinct_values, @r###"
[
"\"A\"",
"\"B\"",
"__does_not_exist__",
"\"C\"",
"\"D\"",
"\"E\"",
"\"F\"",
"\"G\"",
"\"H\"",
"\"I\"",
"__does_not_exist__",
"__does_not_exist__",
]
"###);
let text_values = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(text_values, @r###"
[
"\"the quick brown fox jamps over the lazy dog\"",
"\"the quick brown foxjumps over the lazy dog\"",
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quickbrownfox jumps over the lazy\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brownf fox jumps over\"",
"\"the qick brown fox jumps\"",
"\"the quick brow fox jumps\"",
"\"the quick brown fox jpmps\"",
"\"the quick brown\"",
"\"the quick\"",
]
"###);
}
#[test]
fn test_distinct_sort_words() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Sort, Criterion::Words, Criterion::Desc(S("rank1"))]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.query("the quick brown fox jumps over the lazy dog");
s.sort_criteria(vec![AscDesc::Desc(Member::Field(S("letter")))]);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[22, 20, 19, 16, 9, 8, 7, 3, 1, 26, 25, 24]");
let distinct_values = verify_distinct(&index, &txn, &documents_ids);
insta::assert_debug_snapshot!(distinct_values, @r###"
[
"\"I\"",
"\"H\"",
"\"G\"",
"\"F\"",
"\"E\"",
"\"D\"",
"\"C\"",
"\"B\"",
"\"A\"",
"__does_not_exist__",
"__does_not_exist__",
"__does_not_exist__",
]
"###);
let rank_values = collect_field_values(&index, &txn, "rank1", &documents_ids);
insta::assert_debug_snapshot!(rank_values, @r###"
[
"1",
"0",
"1",
"1",
"0",
"0",
"2",
"1",
"1",
"3",
"1",
"0",
]
"###);
let text_values = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(text_values, @r###"
[
"\"the quick brown fox jumps\"",
"\"the quick brow fox jumps\"",
"\"the quick brownfoxjumps\"",
"\"the quic brown fox jamps over\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumpes over the lazy dog\"",
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown\"",
"\"the quick\"",
]
"###);
}
#[test]
fn test_distinct_all_candidates() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Sort]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.sort_criteria(vec![AscDesc::Desc(Member::Field(S("rank1")))]);
s.exhaustive_number_hits(true);
let SearchResult { documents_ids, candidates, .. } = s.execute().unwrap();
let candidates = candidates.iter().collect::<Vec<_>>();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[14, 26, 4, 7, 17, 23, 1, 19, 25, 8, 20, 24]");
// TODO: this is incorrect!
insta::assert_snapshot!(format!("{candidates:?}"), @"[0, 2, 5, 8, 9, 15, 18, 20, 21, 24, 25, 26]");
}
#[test]
fn test_distinct_typo() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Words, Criterion::Typo]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.query("the quick brown fox jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[3, 26, 0, 7, 8, 9, 15, 22, 18, 20, 25, 24]");
let distinct_values = verify_distinct(&index, &txn, &documents_ids);
insta::assert_debug_snapshot!(distinct_values, @r###"
[
"\"B\"",
"__does_not_exist__",
"\"A\"",
"\"C\"",
"\"D\"",
"\"E\"",
"\"F\"",
"\"I\"",
"\"G\"",
"\"H\"",
"__does_not_exist__",
"__does_not_exist__",
]
"###);
let text_values = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(text_values, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jamps over the lazy dog\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brownf fox jumps over\"",
"\"the quick brown fox jumps\"",
"\"the qick brown fox jumps\"",
"\"the quick brow fox jumps\"",
"\"the quick brown\"",
"\"the quick\"",
]
"###);
}

View File

@ -0,0 +1,22 @@
use crate::{index::tests::TempIndex, Search, SearchResult};
#[test]
fn test_kanji_language_detection() {
let index = TempIndex::new();
index
.add_documents(documents!([
{ "id": 0, "title": "The quick (\"brown\") fox can't jump 32.3 feet, right? Brr, it's 29.3°F!" },
{ "id": 1, "title": "東京のお寿司。" },
{ "id": 2, "title": "הַשּׁוּעָל הַמָּהִיר (״הַחוּם״) לֹא יָכוֹל לִקְפֹּץ 9.94 מֶטְרִים, נָכוֹן? ברר, 1.5°C- בַּחוּץ!" }
]))
.unwrap();
let txn = index.write_txn().unwrap();
let mut search = Search::new(&txn, &index);
search.query("東京");
let SearchResult { documents_ids, .. } = search.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[1]");
}

View File

@ -0,0 +1,30 @@
pub mod distinct;
#[cfg(feature = "default")]
pub mod language;
pub mod ngram_split_words;
pub mod proximity;
pub mod proximity_typo;
pub mod sort;
pub mod typo;
pub mod typo_proximity;
pub mod words_tms;
fn collect_field_values(
index: &crate::Index,
txn: &heed::RoTxn,
fid: &str,
docids: &[u32],
) -> Vec<String> {
let mut values = vec![];
let fid = index.fields_ids_map(txn).unwrap().id(fid).unwrap();
for doc in index.documents(txn, docids.iter().copied()).unwrap() {
if let Some(v) = doc.1.get(fid) {
let v: serde_json::Value = serde_json::from_slice(v).unwrap();
let v = v.to_string();
values.push(v);
} else {
values.push("__does_not_exist__".to_owned());
}
}
values
}

View File

@ -0,0 +1,372 @@
/*!
This module tests the following properties:
1. Two consecutive words from a query can be combined into a "2gram"
2. Three consecutive words from a query can be combined into a "3gram"
3. A word from the query can be split into two consecutive words (split words)
4. A 2gram can be split into two words
5. A 3gram cannot be split into two words
6. 2grams can contain up to 1 typo
7. 3grams cannot have typos
8. 2grams and 3grams can be prefix tolerant
9. Disabling typo tolerance also disable the split words feature
10. Disabling typo tolerance does not disable prefix tolerance
11. Disabling typo tolerance does not disable ngram tolerance
12. Prefix tolerance is disabled for the last word if a space follows it
13. Ngrams cannot be formed by combining a phrase and a word or two phrases
*/
use crate::{
index::tests::TempIndex, search::new::tests::collect_field_values, Criterion, Search,
SearchResult, TermsMatchingStrategy,
};
fn create_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_criteria(vec![Criterion::Words]);
})
.unwrap();
index
.add_documents(documents!([
{
"id": 0,
"text": "the sun flowers are pretty"
},
{
"id": 1,
"text": "the sun flower is tall"
},
{
"id": 2,
"text": "the sunflowers are pretty"
},
{
"id": 3,
"text": "the sunflower is tall"
},
{
"id": 4,
"text": "the sunflawer is tall"
},
{
"id": 5,
"text": "sunflowering is not a verb"
}
]))
.unwrap();
index
}
#[test]
fn test_2gram_simple() {
let index = create_index();
index
.update_settings(|s| {
s.set_autorize_typos(false);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sun flower");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// will also match documents with "sunflower" + prefix tolerance
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 1, 2, 3, 5]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flowers are pretty\"",
"\"the sun flower is tall\"",
"\"the sunflowers are pretty\"",
"\"the sunflower is tall\"",
"\"sunflowering is not a verb\"",
]
"###);
}
#[test]
fn test_3gram_simple() {
let index = create_index();
index
.update_settings(|s| {
s.set_autorize_typos(false);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sun flower s are");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 2]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flowers are pretty\"",
"\"the sunflowers are pretty\"",
]
"###);
}
#[test]
fn test_2gram_typo() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sun flawer");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 1, 2, 3, 4, 5]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flowers are pretty\"",
"\"the sun flower is tall\"",
"\"the sunflowers are pretty\"",
"\"the sunflower is tall\"",
"\"the sunflawer is tall\"",
"\"sunflowering is not a verb\"",
]
"###);
}
#[test]
fn test_no_disable_ngrams() {
let index = create_index();
index
.update_settings(|s| {
s.set_autorize_typos(false);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sun flower ");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// documents containing `sunflower`
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[1, 3]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flower is tall\"",
"\"the sunflower is tall\"",
]
"###);
}
#[test]
fn test_2gram_prefix() {
let index = create_index();
index
.update_settings(|s| {
s.set_autorize_typos(false);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sun flow");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// documents containing words beginning with `sunflow`
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 1, 2, 3, 5]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flowers are pretty\"",
"\"the sun flower is tall\"",
"\"the sunflowers are pretty\"",
"\"the sunflower is tall\"",
"\"sunflowering is not a verb\"",
]
"###);
}
#[test]
fn test_3gram_prefix() {
let index = create_index();
index
.update_settings(|s| {
s.set_autorize_typos(false);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("su nf l");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// documents containing a word beginning with sunfl
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[2, 3, 4, 5]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sunflowers are pretty\"",
"\"the sunflower is tall\"",
"\"the sunflawer is tall\"",
"\"sunflowering is not a verb\"",
]
"###);
}
#[test]
fn test_split_words() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sunflower ");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// all the documents with either `sunflower` or `sun flower` + eventual typo
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[1, 2, 3, 4]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flower is tall\"",
"\"the sunflowers are pretty\"",
"\"the sunflower is tall\"",
"\"the sunflawer is tall\"",
]
"###);
}
#[test]
fn test_disable_split_words() {
let index = create_index();
index
.update_settings(|s| {
s.set_autorize_typos(false);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sunflower ");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// no document containing `sun flower`
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[3]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sunflower is tall\"",
]
"###);
}
#[test]
fn test_2gram_split_words() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sunf lower");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// all the documents with "sunflower", "sun flower", (sunflower + 1 typo), or (sunflower as prefix)
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[1, 2, 3, 4, 5]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flower is tall\"",
"\"the sunflowers are pretty\"",
"\"the sunflower is tall\"",
"\"the sunflawer is tall\"",
"\"sunflowering is not a verb\"",
]
"###);
}
#[test]
fn test_3gram_no_split_words() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sunf lo wer");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// no document with `sun flower`
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[2, 3, 5]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sunflowers are pretty\"",
"\"the sunflower is tall\"",
"\"sunflowering is not a verb\"",
]
"###);
}
#[test]
fn test_3gram_no_typos() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sunf la wer");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[4]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sunflawer is tall\"",
]
"###);
}
#[test]
fn test_no_ngram_phrases() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("\"sun\" flower");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 1]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flowers are pretty\"",
"\"the sun flower is tall\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("\"sun\" \"flower\"");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[1]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the sun flower is tall\"",
]
"###);
}

View File

@ -0,0 +1,471 @@
/*!
This module tests the Proximity ranking rule:
1. A proximity of >7 always has the same cost.
2. Phrase terms can be in sprximity to other terms via their start and end words,
but we need to make sure that the phrase exists in the document that meets this
proximity condition. This is especially relevant with split words and synonyms.
3. An ngram has the same sprximity cost as its component words being consecutive.
e.g. `sunflower` equivalent to `sun flower`.
4. The prefix databases can be used to find the sprximity between two words, but
they store fewer sprximities than the regular word sprximity DB.
*/
use std::collections::HashMap;
use crate::{
index::tests::TempIndex, search::new::tests::collect_field_values, Criterion, Search,
SearchResult, TermsMatchingStrategy,
};
fn create_simple_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_criteria(vec![Criterion::Words, Criterion::Proximity]);
})
.unwrap();
index
.add_documents(documents!([
{
"id": 0,
"text": "the very quick dark brown and smart fox did jump over the terribly lazy and small dog"
},
{
"id": 1,
"text": "the. quick brown fox jumps over the lazy. dog"
},
{
"id": 2,
"text": "the quick brown fox jumps over the lazy. dog"
},
{
"id": 3,
"text": "dog the quick brown fox jumps over the lazy"
},
{
"id": 4,
"text": "the quickbrown fox jumps over the lazy dog"
},
{
"id": 5,
"text": "brown quick fox jumps over the lazy dog"
},
{
"id": 6,
"text": "the really quick brown fox jumps over the very lazy dog"
},
{
"id": 7,
"text": "the really quick brown fox jumps over the lazy dog"
},
{
"id": 8,
"text": "the quick brown fox jumps over the lazy"
},
{
"id": 9,
"text": "the quack brown fox jumps over the lazy"
},
{
"id": 9,
"text": "the quack brown fox jumps over the lazy dog"
},
{
"id": 10,
"text": "the quick brown fox jumps over the lazy dog"
}
]))
.unwrap();
index
}
fn create_edge_cases_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_criteria(vec![Criterion::Words, Criterion::Proximity]);
})
.unwrap();
index.add_documents(documents!([
{
// This document will insert "s" in the prefix database
"id": 0,
"text": "
saa sab sac sae saf sag sah sai saj sak sal sam san sao sap saq sar sasa sat sau sav saw sax say saz
sba sbb sbc sbe sbf sbg sbh sbi sbj sbk sbl sbm sbn sbo sbp sbq sbr sbsb sbt sbu sbv sbw sbx sby sbz
sca scb scc sce scf scg sch sci scj sck scl scm scn sco scp scq scr scsc sct scu scv scw scx scy scz
sda sdb sdc sde sdf sdg sdh sdi sdj sdk sdl sdm sdn sdo sdp sdq sdr sdsd sdt sdu sdv sdw sdx sdy sdz
sea seb sec see sef seg seh sei sej sek sel sem sen seo sep seq ser sese set seu sev sew sex sey sez
sfa sfb sfc sfe sff sfg sfh sfi sfj sfk sfl sfm sfn sfo sfp sfq sfr sfsf sft sfu sfv sfw sfx sfy sfz
sga sgb sgc sge sgf sgg sgh sgi sgj sgk sgl sgm sgn sgo sgp sgq sgr sgsg sgt sgu sgv sgw sgx sgy sgz
ska skb skc ske skf skg skh ski skj skk skl skm skn sko skp skq skr sksk skt sku skv skw skx sky skz
sla slb slc sle slf slg slh sli slj slk sll slm sln slo slp slq slr slsl slt slu slv slw slx sly slz
sma smb smc sme smf smg smh smi smj smk sml smm smn smo smp smq smr smsm smt smu smv smw smx smy smz
sna snb snc sne snf sng snh sni snj snk snl snm snn sno snp snq snr snsn snt snu snv snw snx sny snz
soa sob soc soe sof sog soh soi soj sok sol som son soo sop soq sor soso sot sou sov sow sox soy soz
spa spb spc spe spf spg sph spi spj spk spl spm spn spo spp spq spr spsp spt spu spv spw spx spy spz
sqa sqb sqc sqe sqf sqg sqh sqi sqj sqk sql sqm sqn sqo sqp sqq sqr sqsq sqt squ sqv sqw sqx sqy sqz
sra srb src sre srf srg srh sri srj srk srl srm srn sro srp srq srr srsr srt sru srv srw srx sry srz
ssa ssb ssc sse ssf ssg ssh ssi ssj ssk ssl ssm ssn sso ssp ssq ssr ssss sst ssu ssv ssw ssx ssy ssz
sta stb stc ste stf stg sth sti stj stk stl stm stn sto stp stq str stst stt stu stv stw stx sty stz
"
},
// The next 5 documents lay out a trap with the split word, phrase search, or synonym `sun flower`.
// If the search query is "sunflower", the split word "Sun Flower" will match some documents.
// If the query is `sunflower wilting`, then we should make sure that
// the sprximity condition `flower wilting: sprx N` also comes with the condition
// `sun wilting: sprx N+1`. TODO: this is not the exact condition we use for now.
// We only check that the phrase `sun flower` exists and `flower wilting: sprx N`, which
// is better than nothing but not the best.
{
"id": 1,
"text": "Sun Flower sounds like the title of a painting, maybe about a plant wilting under the heat."
},
{
"id": 2,
"text": "Sun Flower sounds like the title of a painting, maybe about a flower wilting under the heat."
},
{
"id": 3,
// This document matches the query `sunflower wilting`, but the sprximity condition
// between `sunflower` and `wilting` cannot be through the split-word `Sun Flower`
// which would reduce to only `flower` and `wilting` being in sprximity.
"text": "A flower wilting under the sun, unlike a sunflower"
},
{
// This should be the best document for `sunflower wilting`
"id": 4,
"text": "sun flower wilting under the heat"
},
{
// This is also the best document for `sunflower wilting`
"id": 5,
"text": "sunflower wilting under the heat"
},
{
// Prox MAX between `best` and `s` prefix
"id": 6,
"text": "this is the best meal I have ever had in such a beautiful summer day"
},
{
// Prox 5 between `best` and `s` prefix
"id": 7,
"text": "this is the best cooked meal of the summer"
},
{
// Prox 4 between `best` and `s` prefix
"id": 8,
"text": "this is the best meal of the summer"
},
{
// Prox 3 between `best` and `s` prefix
"id": 9,
"text": "this is the best meal of summer"
},
{
// Prox 1 between `best` and `s` prefix
"id": 10,
"text": "this is the best summer meal"
},
{
// Reverse Prox 3 between `best` and `s` prefix
"id": 11,
"text": "summer x y best"
},
{
// Reverse Prox 2 between `best` and `s` prefix
"id": 12,
"text": "summer x best"
},
{
// Reverse Prox 1 between `best` and `s` prefix
"id": 13,
"text": "summer best"
},
{
// This document will insert "win" in the prefix database
"id": 14,
"text": "
winaa winab winac winae winaf winag winah winai winaj winak winal winam winan winao winap winaq winar winasa winat winau winav winaw winax winay winaz
winba winbb winbc winbe winbf winbg winbh winbi winbj winbk winbl winbm winbn winbo winbp winbq winbr winbsb winbt winbu winbv winbw winbx winby winbz
winca wincb wincc wince wincf wincg winch winci wincj winck wincl wincm wincn winco wincp wincq wincr wincsc winct wincu wincv wincw wincx wincy wincz
winda windb windc winde windf windg windh windi windj windk windl windm windn windo windp windq windr windsd windt windu windv windw windx windy windz
winea wineb winec winee winef wineg wineh winei winej winek winel winem winen wineo winep wineq winer winese winet wineu winev winew winex winey winez
winfa winfb winfc winfe winff winfg winfh winfi winfj winfk winfl winfm winfn winfo winfp winfq winfr winfsf winft winfu winfv winfw winfx winfy winfz
winga wingb wingc winge wingf wingg wingh wingi wingj wingk wingl wingm wingn wingo wingp wingq wingr wingsg wingt wingu wingv wingw wingx wingy wingz
winka winkb winkc winke winkf winkg winkh winki winkj winkk winkl winkm winkn winko winkp winkq winkr winksk winkt winku winkv winkw winkx winky winkz
winla winlb winlc winle winlf winlg winlh winli winlj winlk winll winlm winln winlo winlp winlq winlr winlsl winlt winlu winlv winlw winlx winly winlz
winma winmb winmc winme winmf winmg winmh winmi winmj winmk winml winmm winmn winmo winmp winmq winmr winmsm winmt winmu winmv winmw winmx winmy winmz
winna winnb winnc winne winnf winng winnh winni winnj winnk winnl winnm winnn winno winnp winnq winnr winnsn winnt winnu winnv winnw winnx winny winnz
winoa winob winoc winoe winof winog winoh winoi winoj winok winol winom winon winoo winop winoq winor winoso winot winou winov winow winox winoy winoz
winpa winpb winpc winpe winpf winpg winph winpi winpj winpk winpl winpm winpn winpo winpp winpq winpr winpsp winpt winpu winpv winpw winpx winpy winpz
winqa winqb winqc winqe winqf winqg winqh winqi winqj winqk winql winqm winqn winqo winqp winqq winqr winqsq winqt winqu winqv winqw winqx winqy winqz
winra winrb winrc winre winrf winrg winrh winri winrj winrk winrl winrm winrn winro winrp winrq winrr winrsr winrt winru winrv winrw winrx winry winrz
winsa winsb winsc winse winsf winsg winsh winsi winsj winsk winsl winsm winsn winso winsp winsq winsr winsss winst winsu winsv winsw winsx winsy winsz
winta wintb wintc winte wintf wintg winth winti wintj wintk wintl wintm wintn winto wintp wintq wintr wintst wintt wintu wintv wintw wintx winty wintz
"
},
{
// Prox MAX between `best` and `win` prefix
"id": 15,
"text": "this is the best meal I have ever had in such a beautiful winter day"
},
{
// Prox 5 between `best` and `win` prefix
"id": 16,
"text": "this is the best cooked meal of the winter"
},
{
// Prox 4 between `best` and `win` prefix
"id": 17,
"text": "this is the best meal of the winter"
},
{
// Prox 3 between `best` and `win` prefix
"id": 18,
"text": "this is the best meal of winter"
},
{
// Prox 1 between `best` and `win` prefix
"id": 19,
"text": "this is the best winter meal"
},
{
// Reverse Prox 3 between `best` and `win` prefix
"id": 20,
"text": "winter x y best"
},
{
// Reverse Prox 2 between `best` and `win` prefix
"id": 21,
"text": "winter x best"
},
{
// Reverse Prox 1 between `best` and `win` prefix
"id": 22,
"text": "winter best"
},
])).unwrap();
index
}
#[test]
fn test_proximity_simple() {
let index = create_simple_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quick brown fox jumps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[4, 9, 10, 7, 6, 5, 2, 3, 0, 1]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quickbrown fox jumps over the lazy dog\"",
"\"the quack brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the lazy dog\"",
"\"the really quick brown fox jumps over the lazy dog\"",
"\"the really quick brown fox jumps over the very lazy dog\"",
"\"brown quick fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the lazy. dog\"",
"\"dog the quick brown fox jumps over the lazy\"",
"\"the very quick dark brown and smart fox did jump over the terribly lazy and small dog\"",
"\"the. quick brown fox jumps over the lazy. dog\"",
]
"###);
}
#[test]
fn test_proximity_split_word() {
let index = create_edge_cases_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("sunflower wilting");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[2, 4, 5, 1, 3]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
// TODO: "2" and "4" should be swapped ideally
insta::assert_debug_snapshot!(texts, @r###"
[
"\"Sun Flower sounds like the title of a painting, maybe about a flower wilting under the heat.\"",
"\"sun flower wilting under the heat\"",
"\"sunflower wilting under the heat\"",
"\"Sun Flower sounds like the title of a painting, maybe about a plant wilting under the heat.\"",
"\"A flower wilting under the sun, unlike a sunflower\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("\"sun flower\" wilting");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[2, 4, 1]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
// TODO: "2" and "4" should be swapped ideally
insta::assert_debug_snapshot!(texts, @r###"
[
"\"Sun Flower sounds like the title of a painting, maybe about a flower wilting under the heat.\"",
"\"sun flower wilting under the heat\"",
"\"Sun Flower sounds like the title of a painting, maybe about a plant wilting under the heat.\"",
]
"###);
drop(txn);
index
.update_settings(|s| {
let mut syns = HashMap::new();
syns.insert("xyz".to_owned(), vec!["sun flower".to_owned()]);
s.set_synonyms(syns);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("xyz wilting");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[2, 4, 1]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
// TODO: "2" and "4" should be swapped ideally
insta::assert_debug_snapshot!(texts, @r###"
[
"\"Sun Flower sounds like the title of a painting, maybe about a flower wilting under the heat.\"",
"\"sun flower wilting under the heat\"",
"\"Sun Flower sounds like the title of a painting, maybe about a plant wilting under the heat.\"",
]
"###);
}
#[test]
fn test_proximity_prefix_db() {
let index = create_edge_cases_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("best s");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[10, 13, 9, 12, 8, 6, 7, 11, 15]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
// This test illustrates the loss of precision from using the prefix DB
insta::assert_debug_snapshot!(texts, @r###"
[
"\"this is the best summer meal\"",
"\"summer best\"",
"\"this is the best meal of summer\"",
"\"summer x best\"",
"\"this is the best meal of the summer\"",
"\"this is the best meal I have ever had in such a beautiful summer day\"",
"\"this is the best cooked meal of the summer\"",
"\"summer x y best\"",
"\"this is the best meal I have ever had in such a beautiful winter day\"",
]
"###);
// Difference when using the `su` prefix, which is not in the prefix DB
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("best su");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[10, 13, 9, 12, 8, 11, 7, 6, 15]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"this is the best summer meal\"",
"\"summer best\"",
"\"this is the best meal of summer\"",
"\"summer x best\"",
"\"this is the best meal of the summer\"",
"\"summer x y best\"",
"\"this is the best cooked meal of the summer\"",
"\"this is the best meal I have ever had in such a beautiful summer day\"",
"\"this is the best meal I have ever had in such a beautiful winter day\"",
]
"###);
// Note that there is a case where a prefix is in the prefix DB but not in the
// **proximity** prefix DB. In that case, its sprximity score will always be
// the maximum. This happens for prefixes that are larger than 2 bytes.
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("best win");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[15, 16, 17, 18, 19, 20, 21, 22]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"this is the best meal I have ever had in such a beautiful winter day\"",
"\"this is the best cooked meal of the winter\"",
"\"this is the best meal of the winter\"",
"\"this is the best meal of winter\"",
"\"this is the best winter meal\"",
"\"winter x y best\"",
"\"winter x best\"",
"\"winter best\"",
]
"###);
// Now using `wint`, which is not in the prefix DB:
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("best wint");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[19, 22, 18, 21, 17, 20, 16, 15]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"this is the best winter meal\"",
"\"winter best\"",
"\"this is the best meal of winter\"",
"\"winter x best\"",
"\"this is the best meal of the winter\"",
"\"winter x y best\"",
"\"this is the best cooked meal of the winter\"",
"\"this is the best meal I have ever had in such a beautiful winter day\"",
]
"###);
// and using `wi` which is in the prefix DB and proximity prefix DB
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("best wi");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[19, 22, 18, 21, 17, 15, 16, 20]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"this is the best winter meal\"",
"\"winter best\"",
"\"this is the best meal of winter\"",
"\"winter x best\"",
"\"this is the best meal of the winter\"",
"\"this is the best meal I have ever had in such a beautiful winter day\"",
"\"this is the best cooked meal of the winter\"",
"\"winter x y best\"",
]
"###);
}

View File

@ -0,0 +1,75 @@
/*!
This module tests the interactions between the proximity and typo ranking rules.
The proximity ranking rule should transform the query graph such that it
only contains the word pairs that it used to compute its bucket.
TODO: This is not currently implemented.
*/
use crate::{
index::tests::TempIndex, search::new::tests::collect_field_values, Criterion, Search,
SearchResult, TermsMatchingStrategy,
};
fn create_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_criteria(vec![Criterion::Words, Criterion::Proximity, Criterion::Typo]);
})
.unwrap();
index
.add_documents(documents!([
// Basic trap.
//
// We have one document with the perfect word pair: `sommer - holiday`
// and another with the perfect word pair: `sommer holidty`.
//
// The proximity ranking rule will put them both in the same bucket, and it
// should minify the query graph to make it represent:
// EITHER:
// sommer + holiday
// OR:
// sommer + holidty
//
// Such that the child typo ranking rule does not find any match
// for its zero-typo bucket `summer + holiday`, even though both documents
// contain these two exact words.
{
"id": 0,
"text": "summer. holiday. sommer holidty"
},
{
"id": 1,
"text": "summer. holiday. sommer holiday"
},
]))
.unwrap();
index
}
#[test]
fn test_trap_basic() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("summer holiday");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 1]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
// TODO: this is incorrect, 1 should come before 0
insta::assert_debug_snapshot!(texts, @r###"
[
"\"summer. holiday. sommer holidty\"",
"\"summer. holiday. sommer holiday\"",
]
"###);
}

View File

@ -0,0 +1,316 @@
/*!
This module tests the `sort` ranking rule:
1. an error is returned if the sort ranking rule exists but no fields-to-sort were given at search time
2. an error is returned if the fields-to-sort are not sortable
3. it is possible to add multiple fields-to-sort at search time
4. custom sort ranking rules can be added to the settings, they interact with the generic `sort` ranking rule as expected
5. numbers appear before strings
6. documents with either: (1) no value, (2) null, or (3) an object for the field-to-sort appear at the end of the bucket
7. boolean values are translated to strings
8. if a field contains an array, it is sorted by the best value in the array according to the sort rule
*/
use big_s::S;
use maplit::hashset;
use crate::{
index::tests::TempIndex, search::new::tests::collect_field_values, AscDesc, Criterion, Member,
Search, SearchResult, TermsMatchingStrategy,
};
fn create_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_sortable_fields(hashset! { S("rank"), S("vague"), S("letter") });
s.set_criteria(vec![Criterion::Sort]);
})
.unwrap();
index
.add_documents(documents!([
{
"id": 0,
"letter": "A",
"rank": 0,
"vague": 0,
},
{
"id": 1,
"letter": "A",
"rank": 1,
"vague": "0",
},
{
"id": 2,
"letter": "B",
"rank": 0,
"vague": 1,
},
{
"id": 3,
"letter": "B",
"rank": 1,
"vague": "1",
},
{
"id": 4,
"letter": "B",
"rank": 2,
"vague": [1, 2],
},
{
"id": 5,
"letter": "C",
"rank": 0,
"vague": [1, "2"],
},
{
"id": 6,
"letter": "C",
"rank": 1,
},
{
"id": 7,
"letter": "C",
"rank": 2,
"vague": null,
},
{
"id": 8,
"letter": "D",
"rank": 0,
"vague": [null, null, ""]
},
{
"id": 9,
"letter": "E",
"rank": 0,
"vague": ""
},
{
"id": 10,
"letter": "E",
"rank": 1,
"vague": {
"sub": 0,
}
},
{
"id": 11,
"letter": "E",
"rank": 2,
"vague": true,
},
{
"id": 12,
"letter": "E",
"rank": 3,
"vague": false,
},
{
"id": 13,
"letter": "E",
"rank": 4,
"vague": 1.5673,
},
{
"id": 14,
"letter": "E",
"rank": 5,
},
{
"id": 15,
"letter": "F",
"rank": 0,
},
{
"id": 16,
"letter": "F",
"rank": 1,
},
{
"id": 17,
"letter": "F",
"rank": 2,
},
{
"id": 18,
"letter": "G",
"rank": 0,
},
{
"id": 19,
"letter": "G",
"rank": 1,
},
{
"id": 20,
"letter": "H",
"rank": 0,
"vague": true,
},
{
"id": 21,
"letter": "I",
"rank": 0,
"vague": false,
},
{
"id": 22,
"letter": "I",
"rank": 1,
"vague": [1.1367, "help", null]
},
{
"id": 23,
"letter": "I",
"rank": 2,
"vague": [1.2367, "hello"]
},
]))
.unwrap();
index
}
#[test]
fn test_sort() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.sort_criteria(vec![AscDesc::Desc(Member::Field(S("letter")))]);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[21, 22, 23, 20, 18, 19, 15, 16, 17, 9, 10, 11, 12, 13, 14, 8, 5, 6, 7, 2]");
let letter_values = collect_field_values(&index, &txn, "letter", &documents_ids);
insta::assert_debug_snapshot!(letter_values, @r###"
[
"\"I\"",
"\"I\"",
"\"I\"",
"\"H\"",
"\"G\"",
"\"G\"",
"\"F\"",
"\"F\"",
"\"F\"",
"\"E\"",
"\"E\"",
"\"E\"",
"\"E\"",
"\"E\"",
"\"E\"",
"\"D\"",
"\"C\"",
"\"C\"",
"\"C\"",
"\"B\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.sort_criteria(vec![AscDesc::Desc(Member::Field(S("rank")))]);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[14, 13, 12, 4, 7, 11, 17, 23, 1, 3, 6, 10, 16, 19, 22, 0, 2, 5, 8, 9]");
let rank_values = collect_field_values(&index, &txn, "rank", &documents_ids);
insta::assert_debug_snapshot!(rank_values, @r###"
[
"5",
"4",
"3",
"2",
"2",
"2",
"2",
"2",
"1",
"1",
"1",
"1",
"1",
"1",
"1",
"0",
"0",
"0",
"0",
"0",
]
"###);
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.sort_criteria(vec![AscDesc::Asc(Member::Field(S("vague")))]);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 2, 4, 5, 22, 23, 13, 1, 3, 12, 21, 11, 20, 6, 7, 8, 9, 10, 14, 15]");
let vague_values = collect_field_values(&index, &txn, "vague", &documents_ids);
insta::assert_debug_snapshot!(vague_values, @r###"
[
"0",
"1",
"[1,2]",
"[1,\"2\"]",
"[1.1367,\"help\",null]",
"[1.2367,\"hello\"]",
"1.5673",
"\"0\"",
"\"1\"",
"false",
"false",
"true",
"true",
"__does_not_exist__",
"null",
"[null,null,\"\"]",
"\"\"",
"{\"sub\":0}",
"__does_not_exist__",
"__does_not_exist__",
]
"###);
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.sort_criteria(vec![AscDesc::Desc(Member::Field(S("vague")))]);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[4, 13, 23, 22, 2, 5, 0, 11, 20, 12, 21, 3, 1, 6, 7, 8, 9, 10, 14, 15]");
let vague_values = collect_field_values(&index, &txn, "vague", &documents_ids);
insta::assert_debug_snapshot!(vague_values, @r###"
[
"[1,2]",
"1.5673",
"[1.2367,\"hello\"]",
"[1.1367,\"help\",null]",
"1",
"[1,\"2\"]",
"0",
"true",
"true",
"false",
"false",
"\"1\"",
"\"0\"",
"__does_not_exist__",
"null",
"[null,null,\"\"]",
"\"\"",
"{\"sub\":0}",
"__does_not_exist__",
"__does_not_exist__",
]
"###);
}

View File

@ -0,0 +1,497 @@
/*!
This module tests the following properties:
1. The `words` ranking rule is typo-tolerant
2. Typo-tolerance handles missing letters, extra letters, replaced letters, and swapped letters (at least)
3. Words which are < `min_word_len_one_typo` are not typo tolerant
4. Words which are >= `min_word_len_one_typo` but < `min_word_len_two_typos` can have one typo
5. Words which are >= `min_word_len_two_typos` can have two typos
6. A typo on the first letter of a word counts as two typos
7. Phrases are not typo tolerant
8. 2grams can have 1 typo if they are larger than `min_word_len_two_typos`
9. 3grams are not typo tolerant
10. The `typo` ranking rule assumes the role of the `words` ranking rule implicitly
if `words` doesn't exist before it.
11. The `typo` ranking rule places documents with the same number of typos in the same bucket
12. Prefix tolerance costs nothing according to the typo ranking rule
13. Split words cost 1 typo according to the typo ranking rule
14. Synonyms cost nothing according to the typo ranking rule
*/
use std::collections::HashMap;
use crate::{
index::tests::TempIndex, search::new::tests::collect_field_values, Criterion, Search,
SearchResult, TermsMatchingStrategy,
};
fn create_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_criteria(vec![Criterion::Words]);
})
.unwrap();
index
.add_documents(documents!([
{
"id": 0,
"text": "the quick brown fox jumps over the lazy dog"
},
{
"id": 1,
"text": "the quick brown foxes jump over the lazy dog"
},
{
"id": 2,
"text": "the quick brown fax sends a letter to the dog"
},
{
"id": 3,
"text": "the quickest brownest fox jumps over the laziest dog"
},
{
"id": 4,
"text": "a fox doesn't quack, that crown goes to the duck."
},
{
"id": 5,
"text": "the quicker browner fox jumped over the lazier dog"
},
{
"id": 6,
"text": "the extravagant fox skyrocketed over the languorous dog" // thanks thesaurus
},
{
"id": 7,
"text": "the quick brown fox jumps over the lazy"
},
{
"id": 8,
"text": "the quick brown fox jumps over the"
},
{
"id": 9,
"text": "the quick brown fox jumps over"
},
{
"id": 10,
"text": "the quick brown fox jumps"
},
{
"id": 11,
"text": "the quick brown fox"
},
{
"id": 12,
"text": "the quick brown"
},
{
"id": 13,
"text": "the quick"
},
{
"id": 14,
"text": "netwolk interconections sunflawar"
},
{
"id": 15,
"text": "network interconnections sunflawer"
},
{
"id": 16,
"text": "network interconnection sunflower"
},
{
"id": 17,
"text": "network interconnection sun flower"
},
{
"id": 18,
"text": "network interconnection sunflowering"
},
{
"id": 19,
"text": "network interconnection sun flowering"
},
{
"id": 20,
"text": "network interconnection sunflowar"
},
{
"id": 21,
"text": "the fast brownish fox jumps over the lackadaisical dog"
},
{
"id": 22,
"text": "the quick brown fox jumps over the lackadaisical dog"
},
{
"id": 23,
"text": "the quivk brown fox jumps over the lazy dog"
},
]))
.unwrap();
index
}
#[test]
fn test_no_typo() {
let index = create_index();
index
.update_settings(|s| {
s.set_autorize_typos(false);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quick brown fox jumps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
]
"###);
}
#[test]
fn test_default_typo() {
let index = create_index();
let txn = index.read_txn().unwrap();
let ot = index.min_word_len_one_typo(&txn).unwrap();
let tt = index.min_word_len_two_typos(&txn).unwrap();
insta::assert_debug_snapshot!(ot, @"5");
insta::assert_debug_snapshot!(tt, @"9");
// 0 typo
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quick brown fox jumps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 23]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quivk brown fox jumps over the lazy dog\"",
]
"###);
// 1 typo on one word, replaced letter
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quack brown fox jumps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
]
"###);
// 1 typo on one word, missing letter, extra letter
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quicest brownest fox jummps over the laziest dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[3]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quickest brownest fox jumps over the laziest dog\"",
]
"###);
// 1 typo on one word, swapped letters
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quikc borwn fox jupms over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
]
"###);
// 1 first letter typo on a word <5 bytes, replaced letter
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the nuick brown fox jumps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[]");
// 1 first letter typo on a word <5 bytes, missing letter
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the uick brown fox jumps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[]");
// 1 typo on all words >=5 bytes, replaced letters
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quack brawn fox junps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
]
"###);
// 2 typos on words < 9 bytes
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quckest brawnert fox jumps over the aziest dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[]");
// 2 typos on words >= 9 bytes: missing letters, missing first letter, replaced letters
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the extravant fox kyrocketed over the lamguorout dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[6]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the extravagant fox skyrocketed over the languorous dog\"",
]
"###);
// 2 typos on words >= 9 bytes: 2 extra letters in a single word, swapped letters + extra letter, replaced letters
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the extravaganttt fox sktyrocnketed over the lagnuorrous dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[6]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the extravagant fox skyrocketed over the languorous dog\"",
]
"###);
}
#[test]
fn test_phrase_no_typo_allowed() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the \"quick brewn\" fox jumps over the lazy dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @"[]");
}
#[test]
fn test_ngram_typos() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the extra lagant fox skyrocketed over the languorous dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[6]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the extravagant fox skyrocketed over the languorous dog\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the ex tra lagant fox skyrocketed over the languorous dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @"[]");
}
#[test]
fn test_typo_ranking_rule_not_preceded_by_words_ranking_rule() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Typo]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.query("the quick brown fox jumps over the lazy dog");
let SearchResult { documents_ids: ids_1, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{ids_1:?}"), @"[0, 23, 7, 8, 9, 22, 10, 11, 1, 2, 12, 13, 4, 3, 5, 6, 21]");
let texts = collect_field_values(&index, &txn, "text", &ids_1);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quivk brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the\"",
"\"the quick brown fox jumps over\"",
"\"the quick brown fox jumps over the lackadaisical dog\"",
"\"the quick brown fox jumps\"",
"\"the quick brown fox\"",
"\"the quick brown foxes jump over the lazy dog\"",
"\"the quick brown fax sends a letter to the dog\"",
"\"the quick brown\"",
"\"the quick\"",
"\"a fox doesn't quack, that crown goes to the duck.\"",
"\"the quickest brownest fox jumps over the laziest dog\"",
"\"the quicker browner fox jumped over the lazier dog\"",
"\"the extravagant fox skyrocketed over the languorous dog\"",
"\"the fast brownish fox jumps over the lackadaisical dog\"",
]
"###);
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Words, Criterion::Typo]);
})
.unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::Last);
s.query("the quick brown fox jumps over the lazy dog");
let SearchResult { documents_ids: ids_2, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{ids_2:?}"), @"[0, 23, 7, 8, 9, 22, 10, 11, 1, 2, 12, 13, 4, 3, 5, 6, 21]");
assert_eq!(ids_1, ids_2);
}
#[test]
fn test_typo_bucketing() {
let index = create_index();
let txn = index.read_txn().unwrap();
// First do the search with just the Words ranking rule
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("network interconnection sunflower");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[14, 15, 16, 17, 18, 20]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"netwolk interconections sunflawar\"",
"\"network interconnections sunflawer\"",
"\"network interconnection sunflower\"",
"\"network interconnection sun flower\"",
"\"network interconnection sunflowering\"",
"\"network interconnection sunflowar\"",
]
"###);
// Then with the typo ranking rule
drop(txn);
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Typo]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("network interconnection sunflower");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[16, 18, 17, 20, 15, 14]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"network interconnection sunflower\"",
"\"network interconnection sunflowering\"",
"\"network interconnection sun flower\"",
"\"network interconnection sunflowar\"",
"\"network interconnections sunflawer\"",
"\"netwolk interconections sunflawar\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("network interconnection sun flower");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[17, 19, 16, 18, 20, 15]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"network interconnection sun flower\"",
"\"network interconnection sun flowering\"",
"\"network interconnection sunflower\"",
"\"network interconnection sunflowering\"",
"\"network interconnection sunflowar\"",
"\"network interconnections sunflawer\"",
]
"###);
}
#[test]
fn test_typo_synonyms() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Typo]);
let mut synonyms = HashMap::new();
synonyms.insert("lackadaisical".to_owned(), vec!["lazy".to_owned()]);
synonyms.insert("fast brownish".to_owned(), vec!["quick brown".to_owned()]);
s.set_synonyms(synonyms);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the quick brown fox jumps over the lackadaisical dog");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[0, 22, 23]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the lackadaisical dog\"",
"\"the quivk brown fox jumps over the lazy dog\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("the fast brownish fox jumps over the lackadaisical dog");
// TODO: is this correct? interaction of ngrams + synonyms means that the
// multi-word synonyms end up having a typo cost. This is probably not what we want.
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[21, 0, 22]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the fast brownish fox jumps over the lackadaisical dog\"",
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the lackadaisical dog\"",
]
"###);
}

View File

@ -0,0 +1,124 @@
/*!
This module tests the interactions between the typo and proximity ranking rules.
The typo ranking rule should transform the query graph such that it only contains
the combinations of word derivations that it used to compute its bucket.
The proximity ranking rule should then look for proximities only between those specific derivations.
For example, given the the search query `beautiful summer` and the dataset:
```text
{ "id": 0, "text": "beautigul summer...... beautiful day in the summer" }
{ "id": 1, "text": "beautiful summer" }
```
Then the document with id `1` should be returned before `0`.
The proximity ranking rule is not allowed to look for the proximity between `beautigul` and `summer`
because the typo ranking rule before it only used the derivation `beautiful`.
*/
use crate::{
index::tests::TempIndex, search::new::tests::collect_field_values, Criterion, Search,
SearchResult, TermsMatchingStrategy,
};
fn create_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_criteria(vec![Criterion::Words, Criterion::Typo, Criterion::Proximity]);
})
.unwrap();
index
.add_documents(documents!([
// trap explained in the module documentation
{
"id": 0,
"text": "beautigul summer. beautiful x y z summer"
},
{
"id": 1,
"text": "beautiful summer"
},
// the next 2 documents set up a more complicated trap
// with the query `beautiful summer`, we will have:
// 1. documents with no typos, id 0 and 1
// 2. documents with 1 typos: id 2 and 3, those are interpreted as EITHER
// - id 2: "beautigul + summer" ; OR
// - id 3: "beautiful + sommer"
// To sort these two documents, the proximity ranking rule must use only the
// word pairs: `beautigul -- summer` and `beautiful -- sommer` even though
// all variations of `beautiful` and `sommer` were used by the typo ranking rule.
{
"id": 2,
"text": "beautigul sommer. beautigul x summer"
},
{
"id": 3,
"text": "beautiful sommer"
},
// The next two documents lay out an even more complex trap.
// With the user query `delicious sweet dessert`, the typo ranking rule will return one bucket of:
// - id 4: delicitous + sweet + dessert
// - id 5: beautiful + sweet + desgert
// The word pairs that the proximity ranking rules is allowed to use are
// EITHER:
// delicitous -- sweet AND sweet -- dessert
// OR
// delicious -- sweet AND sweet -- desgert
// So the word pair to use for the terms `summer` and `dessert` depend on the
// word pairs explored before them.
{
"id": 4,
"text": "delicitous. sweet. dessert. delicitous sweet desgert",
},
{
"id": 5,
"text": "delicious. sweet desgert. delicious sweet desgert",
},
]))
.unwrap();
index
}
#[test]
fn test_trap_basic_and_complex1() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("beautiful summer");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[1, 0, 3, 2]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"beautiful summer\"",
"\"beautigul summer. beautiful x y z summer\"",
"\"beautiful sommer\"",
"\"beautigul sommer. beautigul x summer\"",
]
"###);
}
#[test]
fn test_trap_complex2() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.terms_matching_strategy(TermsMatchingStrategy::All);
s.query("delicious sweet dessert");
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[5, 4]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"delicious. sweet desgert. delicious sweet desgert\"",
"\"delicitous. sweet. dessert. delicitous sweet desgert\"",
]
"###);
}

View File

@ -0,0 +1,437 @@
/*!
This module tests the following properties:
1. The `last` term matching strategy starts removing terms from the query
starting from the end if no more results match it.
2. Phrases are never deleted by the `last` term matching strategy
3. Duplicate words don't affect the ranking of a document according to the `words` ranking rule
4. The proximity of the first and last word of a phrase to its adjacent terms is taken into
account by the proximity ranking rule.
5. Unclosed double quotes still make a phrase
6. The `all` term matching strategy does not remove any term from the query
7. The search is capable of returning no results if no documents match the query
*/
use crate::{
index::tests::TempIndex, search::new::tests::collect_field_values, Criterion, Search,
SearchResult, TermsMatchingStrategy,
};
fn create_index() -> TempIndex {
let index = TempIndex::new();
index
.update_settings(|s| {
s.set_primary_key("id".to_owned());
s.set_searchable_fields(vec!["text".to_owned()]);
s.set_criteria(vec![Criterion::Words]);
})
.unwrap();
index
.add_documents(documents!([
{
"id": 0,
"text": "",
},
{
"id": 1,
"text": "the",
},
{
"id": 2,
"text": "the quick",
},
{
"id": 3,
"text": "the quick brown",
},
{
"id": 4,
"text": "the quick brown fox",
},
{
"id": 5,
"text": "the quick brown fox jumps",
},
{
"id": 6,
"text": "the quick brown fox jumps over",
},
{
"id": 7,
"text": "the quick brown fox jumps over the",
},
{
"id": 8,
"text": "the quick brown fox jumps over the lazy",
},
{
"id": 9,
"text": "the quick brown fox jumps over the lazy dog",
},
{
"id": 10,
"text": "the brown quick fox jumps over the lazy dog",
},
{
"id": 11,
"text": "the quick brown fox talks to the lazy and slow dog",
},
{
"id": 12,
"text": "the quick brown fox talks to the lazy dog",
},
{
"id": 13,
"text": "the mighty and quick brown fox jumps over the lazy dog",
},
{
"id": 14,
"text": "the great quick brown fox jumps over the lazy dog",
},
{
"id": 15,
"text": "this quick brown and very scary fox jumps over the lazy dog",
},
{
"id": 16,
"text": "this quick brown and scary fox jumps over the lazy dog",
},
{
"id": 17,
"text": "the quick brown fox jumps over the really lazy dog",
},
{
"id": 18,
"text": "the brown quick fox jumps over the really lazy dog",
},
{
"id": 19,
"text": "the brown quick fox immediately jumps over the really lazy dog",
},
{
"id": 20,
"text": "the brown quick fox immediately jumps over the really lazy blue dog",
},
{
"id": 21,
"text": "the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.",
},
{
"id": 22,
"text": "the, quick, brown, fox, jumps, over, the, lazy, dog",
}
]))
.unwrap();
index
}
#[test]
fn test_words_tms_last_simple() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.query("the quick brown fox jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// 6 and 7 have the same score because "the" appears twice
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9, 10, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 8, 6, 7, 5, 4, 11, 12, 3]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the brown quick fox jumps over the lazy dog\"",
"\"the mighty and quick brown fox jumps over the lazy dog\"",
"\"the great quick brown fox jumps over the lazy dog\"",
"\"this quick brown and very scary fox jumps over the lazy dog\"",
"\"this quick brown and scary fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the really lazy dog\"",
"\"the brown quick fox jumps over the really lazy dog\"",
"\"the brown quick fox immediately jumps over the really lazy dog\"",
"\"the brown quick fox immediately jumps over the really lazy blue dog\"",
"\"the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.\"",
"\"the, quick, brown, fox, jumps, over, the, lazy, dog\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over\"",
"\"the quick brown fox jumps over the\"",
"\"the quick brown fox jumps\"",
"\"the quick brown fox\"",
"\"the quick brown fox talks to the lazy and slow dog\"",
"\"the quick brown fox talks to the lazy dog\"",
"\"the quick brown\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.query("extravagant the quick brown fox jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[]");
}
#[test]
fn test_words_tms_last_phrase() {
let index = create_index();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.query("\"the quick brown fox\" jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// "The quick brown fox" is a phrase, not deleted by this term matching strategy
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9, 17, 21, 8, 6, 7, 5, 4, 11, 12]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the really lazy dog\"",
"\"the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over\"",
"\"the quick brown fox jumps over the\"",
"\"the quick brown fox jumps\"",
"\"the quick brown fox\"",
"\"the quick brown fox talks to the lazy and slow dog\"",
"\"the quick brown fox talks to the lazy dog\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.query("\"the quick brown fox\" jumps over the \"lazy\" dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// "lazy" is a phrase, not deleted by this term matching strategy
// but words before it can be deleted
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9, 17, 21, 8, 11, 12]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the really lazy dog\"",
"\"the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox talks to the lazy and slow dog\"",
"\"the quick brown fox talks to the lazy dog\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.query("\"the quick brown fox jumps over the lazy dog\"");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// The whole query is a phrase, no terms are removed
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.query("\"the quick brown fox jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// The whole query is still a phrase, even without closing quotes, so no terms are removed
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
]
"###);
}
#[test]
fn test_words_proximity_tms_last_simple() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Words, Criterion::Proximity]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.query("the quick brown fox jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// 7 is better than 6 because of the proximity between "the" and its surrounding terms
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9, 21, 14, 17, 13, 10, 18, 19, 20, 16, 15, 22, 8, 7, 6, 5, 4, 11, 12, 3]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.\"",
"\"the great quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the really lazy dog\"",
"\"the mighty and quick brown fox jumps over the lazy dog\"",
"\"the brown quick fox jumps over the lazy dog\"",
"\"the brown quick fox jumps over the really lazy dog\"",
"\"the brown quick fox immediately jumps over the really lazy dog\"",
"\"the brown quick fox immediately jumps over the really lazy blue dog\"",
"\"this quick brown and scary fox jumps over the lazy dog\"",
"\"this quick brown and very scary fox jumps over the lazy dog\"",
"\"the, quick, brown, fox, jumps, over, the, lazy, dog\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the\"",
"\"the quick brown fox jumps over\"",
"\"the quick brown fox jumps\"",
"\"the quick brown fox\"",
"\"the quick brown fox talks to the lazy and slow dog\"",
"\"the quick brown fox talks to the lazy dog\"",
"\"the quick brown\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.query("the brown quick fox jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// 10 is better than 9 because of the proximity between "quick" and "brown"
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[10, 18, 19, 9, 20, 21, 14, 17, 13, 16, 15, 22, 8, 7, 6, 5, 4, 11, 12, 3]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the brown quick fox jumps over the lazy dog\"",
"\"the brown quick fox jumps over the really lazy dog\"",
"\"the brown quick fox immediately jumps over the really lazy dog\"",
"\"the quick brown fox jumps over the lazy dog\"",
"\"the brown quick fox immediately jumps over the really lazy blue dog\"",
"\"the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.\"",
"\"the great quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the really lazy dog\"",
"\"the mighty and quick brown fox jumps over the lazy dog\"",
"\"this quick brown and scary fox jumps over the lazy dog\"",
"\"this quick brown and very scary fox jumps over the lazy dog\"",
"\"the, quick, brown, fox, jumps, over, the, lazy, dog\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the\"",
"\"the quick brown fox jumps over\"",
"\"the quick brown fox jumps\"",
"\"the quick brown fox\"",
"\"the quick brown fox talks to the lazy and slow dog\"",
"\"the quick brown fox talks to the lazy dog\"",
"\"the quick brown\"",
]
"###);
}
#[test]
fn test_words_proximity_tms_last_phrase() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Words, Criterion::Proximity]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.query("the \"quick brown\" fox jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// "quick brown" is a phrase. The proximity of its first and last words
// to their adjacent query words should be taken into account
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9, 21, 14, 17, 13, 16, 15, 8, 7, 6, 5, 4, 11, 12, 3]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.\"",
"\"the great quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the really lazy dog\"",
"\"the mighty and quick brown fox jumps over the lazy dog\"",
"\"this quick brown and scary fox jumps over the lazy dog\"",
"\"this quick brown and very scary fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the\"",
"\"the quick brown fox jumps over\"",
"\"the quick brown fox jumps\"",
"\"the quick brown fox\"",
"\"the quick brown fox talks to the lazy and slow dog\"",
"\"the quick brown fox talks to the lazy dog\"",
"\"the quick brown\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.query("the \"quick brown\" \"fox jumps\" over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::Last);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
// "quick brown" is a phrase. The proximity of its first and last words
// to their adjacent query words should be taken into account.
// The same applies to `fox jumps`.
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9, 21, 14, 17, 13, 16, 15, 8, 7, 6, 5]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.\"",
"\"the great quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the really lazy dog\"",
"\"the mighty and quick brown fox jumps over the lazy dog\"",
"\"this quick brown and scary fox jumps over the lazy dog\"",
"\"this quick brown and very scary fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the lazy\"",
"\"the quick brown fox jumps over the\"",
"\"the quick brown fox jumps over\"",
"\"the quick brown fox jumps\"",
]
"###);
}
#[test]
fn test_words_tms_all() {
let index = create_index();
index
.update_settings(|s| {
s.set_criteria(vec![Criterion::Words, Criterion::Proximity]);
})
.unwrap();
let txn = index.read_txn().unwrap();
let mut s = Search::new(&txn, &index);
s.query("the quick brown fox jumps over the lazy dog");
s.terms_matching_strategy(TermsMatchingStrategy::All);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[9, 21, 14, 17, 13, 10, 18, 19, 20, 16, 15, 22]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @r###"
[
"\"the quick brown fox jumps over the lazy dog\"",
"\"the quick brown. quick brown fox. brown fox jumps. fox jumps over. over the lazy. the lazy dog.\"",
"\"the great quick brown fox jumps over the lazy dog\"",
"\"the quick brown fox jumps over the really lazy dog\"",
"\"the mighty and quick brown fox jumps over the lazy dog\"",
"\"the brown quick fox jumps over the lazy dog\"",
"\"the brown quick fox jumps over the really lazy dog\"",
"\"the brown quick fox immediately jumps over the really lazy dog\"",
"\"the brown quick fox immediately jumps over the really lazy blue dog\"",
"\"this quick brown and scary fox jumps over the lazy dog\"",
"\"this quick brown and very scary fox jumps over the lazy dog\"",
"\"the, quick, brown, fox, jumps, over, the, lazy, dog\"",
]
"###);
let mut s = Search::new(&txn, &index);
s.query("extravagant");
s.terms_matching_strategy(TermsMatchingStrategy::All);
let SearchResult { documents_ids, .. } = s.execute().unwrap();
insta::assert_snapshot!(format!("{documents_ids:?}"), @"[]");
let texts = collect_field_values(&index, &txn, "text", &documents_ids);
insta::assert_debug_snapshot!(texts, @"[]");
}