mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-03 11:57:07 +02:00
Merge remote-tracking branch 'origin/main' into tmp-release-v1.15.1
This commit is contained in:
commit
9bda9a9a64
58 changed files with 2312 additions and 1756 deletions
|
@ -50,7 +50,7 @@ impl AttributePatterns {
|
|||
///
|
||||
/// * `pattern` - The pattern to match against.
|
||||
/// * `str` - The string to match against the pattern.
|
||||
fn match_pattern(pattern: &str, str: &str) -> PatternMatch {
|
||||
pub fn match_pattern(pattern: &str, str: &str) -> PatternMatch {
|
||||
// If the pattern is a wildcard, return Match
|
||||
if pattern == "*" {
|
||||
return PatternMatch::Match;
|
||||
|
|
|
@ -1,11 +1,13 @@
|
|||
use std::cmp::Ordering;
|
||||
|
||||
use heed::RoTxn;
|
||||
use itertools::Itertools;
|
||||
use roaring::RoaringBitmap;
|
||||
|
||||
use crate::score_details::{ScoreDetails, ScoreValue, ScoringStrategy};
|
||||
use crate::search::new::{distinct_fid, distinct_single_docid};
|
||||
use crate::search::SemanticSearch;
|
||||
use crate::{MatchingWords, Result, Search, SearchResult};
|
||||
use crate::{Index, MatchingWords, Result, Search, SearchResult};
|
||||
|
||||
struct ScoreWithRatioResult {
|
||||
matching_words: MatchingWords,
|
||||
|
@ -91,7 +93,10 @@ impl ScoreWithRatioResult {
|
|||
keyword_results: Self,
|
||||
from: usize,
|
||||
length: usize,
|
||||
) -> (SearchResult, u32) {
|
||||
distinct: Option<&str>,
|
||||
index: &Index,
|
||||
rtxn: &RoTxn<'_>,
|
||||
) -> Result<(SearchResult, u32)> {
|
||||
#[derive(Clone, Copy)]
|
||||
enum ResultSource {
|
||||
Semantic,
|
||||
|
@ -106,8 +111,9 @@ impl ScoreWithRatioResult {
|
|||
vector_results.document_scores.len() + keyword_results.document_scores.len(),
|
||||
);
|
||||
|
||||
let mut documents_seen = RoaringBitmap::new();
|
||||
for ((docid, (main_score, _sub_score)), source) in vector_results
|
||||
let distinct_fid = distinct_fid(distinct, index, rtxn)?;
|
||||
let mut excluded_documents = RoaringBitmap::new();
|
||||
for res in vector_results
|
||||
.document_scores
|
||||
.into_iter()
|
||||
.zip(std::iter::repeat(ResultSource::Semantic))
|
||||
|
@ -121,13 +127,33 @@ impl ScoreWithRatioResult {
|
|||
compare_scores(left, right).is_ge()
|
||||
},
|
||||
)
|
||||
// remove documents we already saw
|
||||
.filter(|((docid, _), _)| documents_seen.insert(*docid))
|
||||
// remove documents we already saw and apply distinct rule
|
||||
.filter_map(|item @ ((docid, _), _)| {
|
||||
if !excluded_documents.insert(docid) {
|
||||
// the document was already added, or is indistinct from an already-added document.
|
||||
return None;
|
||||
}
|
||||
|
||||
if let Some(distinct_fid) = distinct_fid {
|
||||
if let Err(error) = distinct_single_docid(
|
||||
index,
|
||||
rtxn,
|
||||
distinct_fid,
|
||||
docid,
|
||||
&mut excluded_documents,
|
||||
) {
|
||||
return Some(Err(error));
|
||||
}
|
||||
}
|
||||
|
||||
Some(Ok(item))
|
||||
})
|
||||
// start skipping **after** the filter
|
||||
.skip(from)
|
||||
// take **after** skipping
|
||||
.take(length)
|
||||
{
|
||||
let ((docid, (main_score, _sub_score)), source) = res?;
|
||||
if let ResultSource::Semantic = source {
|
||||
semantic_hit_count += 1;
|
||||
}
|
||||
|
@ -136,10 +162,24 @@ impl ScoreWithRatioResult {
|
|||
document_scores.push(main_score);
|
||||
}
|
||||
|
||||
(
|
||||
// compute the set of candidates from both sets
|
||||
let candidates = vector_results.candidates | keyword_results.candidates;
|
||||
let must_remove_redundant_candidates = distinct_fid.is_some();
|
||||
let candidates = if must_remove_redundant_candidates {
|
||||
// patch-up the candidates to remove the indistinct documents, then add back the actual hits
|
||||
let mut candidates = candidates - excluded_documents;
|
||||
for docid in &documents_ids {
|
||||
candidates.insert(*docid);
|
||||
}
|
||||
candidates
|
||||
} else {
|
||||
candidates
|
||||
};
|
||||
|
||||
Ok((
|
||||
SearchResult {
|
||||
matching_words: keyword_results.matching_words,
|
||||
candidates: vector_results.candidates | keyword_results.candidates,
|
||||
candidates,
|
||||
documents_ids,
|
||||
document_scores,
|
||||
degraded: vector_results.degraded | keyword_results.degraded,
|
||||
|
@ -147,7 +187,7 @@ impl ScoreWithRatioResult {
|
|||
| keyword_results.used_negative_operator,
|
||||
},
|
||||
semantic_hit_count,
|
||||
)
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -226,8 +266,15 @@ impl Search<'_> {
|
|||
let keyword_results = ScoreWithRatioResult::new(keyword_results, 1.0 - semantic_ratio);
|
||||
let vector_results = ScoreWithRatioResult::new(vector_results, semantic_ratio);
|
||||
|
||||
let (merge_results, semantic_hit_count) =
|
||||
ScoreWithRatioResult::merge(vector_results, keyword_results, self.offset, self.limit);
|
||||
let (merge_results, semantic_hit_count) = ScoreWithRatioResult::merge(
|
||||
vector_results,
|
||||
keyword_results,
|
||||
self.offset,
|
||||
self.limit,
|
||||
search.distinct.as_deref(),
|
||||
search.index,
|
||||
search.rtxn,
|
||||
)?;
|
||||
assert!(merge_results.documents_ids.len() <= self.limit);
|
||||
Ok((merge_results, Some(semantic_hit_count)))
|
||||
}
|
||||
|
|
|
@ -4,7 +4,9 @@ use super::logger::SearchLogger;
|
|||
use super::ranking_rules::{BoxRankingRule, RankingRuleQueryTrait};
|
||||
use super::SearchContext;
|
||||
use crate::score_details::{ScoreDetails, ScoringStrategy};
|
||||
use crate::search::new::distinct::{apply_distinct_rule, distinct_single_docid, DistinctOutput};
|
||||
use crate::search::new::distinct::{
|
||||
apply_distinct_rule, distinct_fid, distinct_single_docid, DistinctOutput,
|
||||
};
|
||||
use crate::{Result, TimeBudget};
|
||||
|
||||
pub struct BucketSortOutput {
|
||||
|
@ -35,16 +37,7 @@ pub fn bucket_sort<'ctx, Q: RankingRuleQueryTrait>(
|
|||
logger.ranking_rules(&ranking_rules);
|
||||
logger.initial_universe(universe);
|
||||
|
||||
let distinct_field = match distinct {
|
||||
Some(distinct) => Some(distinct),
|
||||
None => ctx.index.distinct_field(ctx.txn)?,
|
||||
};
|
||||
|
||||
let distinct_fid = if let Some(field) = distinct_field {
|
||||
ctx.index.fields_ids_map(ctx.txn)?.id(field)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
let distinct_fid = distinct_fid(distinct, ctx.index, ctx.txn)?;
|
||||
|
||||
if universe.len() < from as u64 {
|
||||
return Ok(BucketSortOutput {
|
||||
|
|
|
@ -9,7 +9,7 @@ use crate::heed_codec::facet::{
|
|||
FacetGroupKey, FacetGroupKeyCodec, FacetGroupValueCodec, FieldDocIdFacetCodec,
|
||||
};
|
||||
use crate::heed_codec::BytesRefCodec;
|
||||
use crate::{Index, Result, SearchContext};
|
||||
use crate::{FieldId, Index, Result, SearchContext};
|
||||
|
||||
pub struct DistinctOutput {
|
||||
pub remaining: RoaringBitmap,
|
||||
|
@ -121,3 +121,18 @@ pub fn facet_string_values<'a>(
|
|||
fn facet_values_prefix_key(distinct: u16, id: u32) -> [u8; FID_SIZE + DOCID_SIZE] {
|
||||
concat_arrays::concat_arrays!(distinct.to_be_bytes(), id.to_be_bytes())
|
||||
}
|
||||
|
||||
pub fn distinct_fid(
|
||||
query_distinct_field: Option<&str>,
|
||||
index: &Index,
|
||||
rtxn: &RoTxn<'_>,
|
||||
) -> Result<Option<FieldId>> {
|
||||
let distinct_field = match query_distinct_field {
|
||||
Some(distinct) => Some(distinct),
|
||||
None => index.distinct_field(rtxn)?,
|
||||
};
|
||||
|
||||
let distinct_fid =
|
||||
if let Some(field) = distinct_field { index.fields_ids_map(rtxn)?.id(field) } else { None };
|
||||
Ok(distinct_fid)
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ use std::time::Duration;
|
|||
use bucket_sort::{bucket_sort, BucketSortOutput};
|
||||
use charabia::{Language, TokenizerBuilder};
|
||||
use db_cache::DatabaseCache;
|
||||
pub use distinct::{distinct_fid, distinct_single_docid};
|
||||
use exact_attribute::ExactAttribute;
|
||||
use graph_based_ranking_rule::{Exactness, Fid, Position, Proximity, Typo};
|
||||
use heed::RoTxn;
|
||||
|
@ -51,6 +52,7 @@ pub use self::geo_sort::{Parameter as GeoSortParameter, Strategy as GeoSortStrat
|
|||
use self::graph_based_ranking_rule::Words;
|
||||
use self::interner::Interned;
|
||||
use self::vector_sort::VectorSort;
|
||||
use crate::attribute_patterns::{match_pattern, PatternMatch};
|
||||
use crate::constants::RESERVED_GEO_FIELD_NAME;
|
||||
use crate::index::PrefixSearch;
|
||||
use crate::localized_attributes_rules::LocalizedFieldIds;
|
||||
|
@ -119,17 +121,37 @@ impl<'ctx> SearchContext<'ctx> {
|
|||
let searchable_fields_weights = self.index.searchable_fields_and_weights(self.txn)?;
|
||||
let exact_attributes_ids = self.index.exact_attributes_ids(self.txn)?;
|
||||
|
||||
let mut wildcard = false;
|
||||
let mut universal_wildcard = false;
|
||||
|
||||
let mut restricted_fids = RestrictedFids::default();
|
||||
for field_name in attributes_to_search_on {
|
||||
if field_name == "*" {
|
||||
wildcard = true;
|
||||
universal_wildcard = true;
|
||||
// we cannot early exit as we want to returns error in case of unknown fields
|
||||
continue;
|
||||
}
|
||||
let searchable_weight =
|
||||
searchable_fields_weights.iter().find(|(name, _, _)| name == field_name);
|
||||
|
||||
// The field is not searchable but may contain a wildcard pattern
|
||||
if searchable_weight.is_none() && field_name.contains("*") {
|
||||
let matching_searchable_weights: Vec<_> = searchable_fields_weights
|
||||
.iter()
|
||||
.filter(|(name, _, _)| match_pattern(field_name, name) == PatternMatch::Match)
|
||||
.collect();
|
||||
|
||||
if !matching_searchable_weights.is_empty() {
|
||||
for (_name, fid, weight) in matching_searchable_weights {
|
||||
if exact_attributes_ids.contains(fid) {
|
||||
restricted_fids.exact.push((*fid, *weight));
|
||||
} else {
|
||||
restricted_fids.tolerant.push((*fid, *weight));
|
||||
}
|
||||
}
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
let (fid, weight) = match searchable_weight {
|
||||
// The Field id exist and the field is searchable
|
||||
Some((_name, fid, weight)) => (*fid, *weight),
|
||||
|
@ -159,7 +181,7 @@ impl<'ctx> SearchContext<'ctx> {
|
|||
};
|
||||
}
|
||||
|
||||
if wildcard {
|
||||
if universal_wildcard {
|
||||
self.restricted_fids = None;
|
||||
} else {
|
||||
self.restricted_fids = Some(restricted_fids);
|
||||
|
|
|
@ -72,7 +72,7 @@ fn test_2gram_simple() {
|
|||
let index = create_index();
|
||||
index
|
||||
.update_settings(|s| {
|
||||
s.set_autorize_typos(false);
|
||||
s.set_authorize_typos(false);
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
|
@ -103,7 +103,7 @@ fn test_3gram_simple() {
|
|||
let index = create_index();
|
||||
index
|
||||
.update_settings(|s| {
|
||||
s.set_autorize_typos(false);
|
||||
s.set_authorize_typos(false);
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
|
@ -153,7 +153,7 @@ fn test_no_disable_ngrams() {
|
|||
let index = create_index();
|
||||
index
|
||||
.update_settings(|s| {
|
||||
s.set_autorize_typos(false);
|
||||
s.set_authorize_typos(false);
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
|
@ -179,7 +179,7 @@ fn test_2gram_prefix() {
|
|||
let index = create_index();
|
||||
index
|
||||
.update_settings(|s| {
|
||||
s.set_autorize_typos(false);
|
||||
s.set_authorize_typos(false);
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
|
@ -208,7 +208,7 @@ fn test_3gram_prefix() {
|
|||
let index = create_index();
|
||||
index
|
||||
.update_settings(|s| {
|
||||
s.set_autorize_typos(false);
|
||||
s.set_authorize_typos(false);
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
|
@ -260,7 +260,7 @@ fn test_disable_split_words() {
|
|||
let index = create_index();
|
||||
index
|
||||
.update_settings(|s| {
|
||||
s.set_autorize_typos(false);
|
||||
s.set_authorize_typos(false);
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
|
|
|
@ -151,7 +151,7 @@ fn test_no_typo() {
|
|||
let index = create_index();
|
||||
index
|
||||
.update_settings(|s| {
|
||||
s.set_autorize_typos(false);
|
||||
s.set_authorize_typos(false);
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
|
|
|
@ -19,10 +19,7 @@ use crate::update::{
|
|||
};
|
||||
use crate::vector::settings::{EmbedderSource, EmbeddingSettings};
|
||||
use crate::vector::EmbeddingConfigs;
|
||||
use crate::{
|
||||
db_snap, obkv_to_json, Filter, FilterableAttributesRule, Index, Search, SearchResult,
|
||||
ThreadPoolNoAbortBuilder,
|
||||
};
|
||||
use crate::{db_snap, obkv_to_json, Filter, FilterableAttributesRule, Index, Search, SearchResult};
|
||||
|
||||
pub(crate) struct TempIndex {
|
||||
pub inner: Index,
|
||||
|
@ -62,15 +59,8 @@ impl TempIndex {
|
|||
wtxn: &mut RwTxn<'t>,
|
||||
documents: Mmap,
|
||||
) -> Result<(), crate::error::Error> {
|
||||
let local_pool;
|
||||
let indexer_config = &self.indexer_config;
|
||||
let pool = match &indexer_config.thread_pool {
|
||||
Some(pool) => pool,
|
||||
None => {
|
||||
local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap();
|
||||
&local_pool
|
||||
}
|
||||
};
|
||||
let pool = &indexer_config.thread_pool;
|
||||
|
||||
let rtxn = self.inner.read_txn()?;
|
||||
let db_fields_ids_map = self.inner.fields_ids_map(&rtxn)?;
|
||||
|
@ -153,15 +143,8 @@ impl TempIndex {
|
|||
wtxn: &mut RwTxn<'t>,
|
||||
external_document_ids: Vec<String>,
|
||||
) -> Result<(), crate::error::Error> {
|
||||
let local_pool;
|
||||
let indexer_config = &self.indexer_config;
|
||||
let pool = match &indexer_config.thread_pool {
|
||||
Some(pool) => pool,
|
||||
None => {
|
||||
local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap();
|
||||
&local_pool
|
||||
}
|
||||
};
|
||||
let pool = &indexer_config.thread_pool;
|
||||
|
||||
let rtxn = self.inner.read_txn()?;
|
||||
let db_fields_ids_map = self.inner.fields_ids_map(&rtxn)?;
|
||||
|
@ -231,15 +214,8 @@ fn aborting_indexation() {
|
|||
let mut wtxn = index.inner.write_txn().unwrap();
|
||||
let should_abort = AtomicBool::new(false);
|
||||
|
||||
let local_pool;
|
||||
let indexer_config = &index.indexer_config;
|
||||
let pool = match &indexer_config.thread_pool {
|
||||
Some(pool) => pool,
|
||||
None => {
|
||||
local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap();
|
||||
&local_pool
|
||||
}
|
||||
};
|
||||
let pool = &indexer_config.thread_pool;
|
||||
|
||||
let rtxn = index.inner.read_txn().unwrap();
|
||||
let db_fields_ids_map = index.inner.fields_ids_map(&rtxn).unwrap();
|
||||
|
|
|
@ -54,6 +54,10 @@ impl ThreadPoolNoAbortBuilder {
|
|||
ThreadPoolNoAbortBuilder::default()
|
||||
}
|
||||
|
||||
pub fn new_for_indexing() -> ThreadPoolNoAbortBuilder {
|
||||
ThreadPoolNoAbortBuilder::default().thread_name(|index| format!("indexing-thread:{index}"))
|
||||
}
|
||||
|
||||
pub fn thread_name<F>(mut self, closure: F) -> Self
|
||||
where
|
||||
F: FnMut(usize) -> String + 'static,
|
||||
|
|
|
@ -33,7 +33,6 @@ use crate::documents::{obkv_to_object, DocumentsBatchReader};
|
|||
use crate::error::{Error, InternalError};
|
||||
use crate::index::{PrefixSearch, PrefixSettings};
|
||||
use crate::progress::Progress;
|
||||
use crate::thread_pool_no_abort::ThreadPoolNoAbortBuilder;
|
||||
pub use crate::update::index_documents::helpers::CursorClonableMmap;
|
||||
use crate::update::{
|
||||
IndexerConfig, UpdateIndexingStep, WordPrefixDocids, WordPrefixIntegerDocids, WordsPrefixesFst,
|
||||
|
@ -228,24 +227,7 @@ where
|
|||
let possible_embedding_mistakes =
|
||||
crate::vector::error::PossibleEmbeddingMistakes::new(&field_distribution);
|
||||
|
||||
let backup_pool;
|
||||
let pool = match self.indexer_config.thread_pool {
|
||||
Some(ref pool) => pool,
|
||||
None => {
|
||||
// We initialize a backup pool with the default
|
||||
// settings if none have already been set.
|
||||
#[allow(unused_mut)]
|
||||
let mut pool_builder = ThreadPoolNoAbortBuilder::new();
|
||||
|
||||
#[cfg(test)]
|
||||
{
|
||||
pool_builder = pool_builder.num_threads(1);
|
||||
}
|
||||
|
||||
backup_pool = pool_builder.build()?;
|
||||
&backup_pool
|
||||
}
|
||||
};
|
||||
let pool = &self.indexer_config.thread_pool;
|
||||
|
||||
// create LMDB writer channel
|
||||
let (lmdb_writer_sx, lmdb_writer_rx): (
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
use grenad::CompressionType;
|
||||
|
||||
use super::GrenadParameters;
|
||||
use crate::thread_pool_no_abort::ThreadPoolNoAbort;
|
||||
use crate::{thread_pool_no_abort::ThreadPoolNoAbort, ThreadPoolNoAbortBuilder};
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct IndexerConfig {
|
||||
|
@ -9,9 +9,10 @@ pub struct IndexerConfig {
|
|||
pub max_nb_chunks: Option<usize>,
|
||||
pub documents_chunk_size: Option<usize>,
|
||||
pub max_memory: Option<usize>,
|
||||
pub max_threads: Option<usize>,
|
||||
pub chunk_compression_type: CompressionType,
|
||||
pub chunk_compression_level: Option<u32>,
|
||||
pub thread_pool: Option<ThreadPoolNoAbort>,
|
||||
pub thread_pool: ThreadPoolNoAbort,
|
||||
pub max_positions_per_attributes: Option<u32>,
|
||||
pub skip_index_budget: bool,
|
||||
}
|
||||
|
@ -27,16 +28,39 @@ impl IndexerConfig {
|
|||
}
|
||||
}
|
||||
|
||||
/// By default use only 1 thread for indexing in tests
|
||||
#[cfg(test)]
|
||||
pub fn default_thread_pool_and_threads() -> (ThreadPoolNoAbort, Option<usize>) {
|
||||
let pool = ThreadPoolNoAbortBuilder::new_for_indexing()
|
||||
.num_threads(1)
|
||||
.build()
|
||||
.expect("failed to build default rayon thread pool");
|
||||
|
||||
(pool, Some(1))
|
||||
}
|
||||
|
||||
#[cfg(not(test))]
|
||||
pub fn default_thread_pool_and_threads() -> (ThreadPoolNoAbort, Option<usize>) {
|
||||
let pool = ThreadPoolNoAbortBuilder::new_for_indexing()
|
||||
.build()
|
||||
.expect("failed to build default rayon thread pool");
|
||||
|
||||
(pool, None)
|
||||
}
|
||||
|
||||
impl Default for IndexerConfig {
|
||||
fn default() -> Self {
|
||||
let (thread_pool, max_threads) = default_thread_pool_and_threads();
|
||||
|
||||
Self {
|
||||
max_threads,
|
||||
thread_pool,
|
||||
log_every_n: None,
|
||||
max_nb_chunks: None,
|
||||
documents_chunk_size: None,
|
||||
max_memory: None,
|
||||
chunk_compression_type: CompressionType::None,
|
||||
chunk_compression_level: None,
|
||||
thread_pool: None,
|
||||
max_positions_per_attributes: None,
|
||||
skip_index_budget: false,
|
||||
}
|
||||
|
|
|
@ -5,7 +5,7 @@ pub use self::concurrent_available_ids::ConcurrentAvailableIds;
|
|||
pub use self::facet::bulk::FacetsUpdateBulk;
|
||||
pub use self::facet::incremental::FacetsUpdateIncrementalInner;
|
||||
pub use self::index_documents::*;
|
||||
pub use self::indexer_config::IndexerConfig;
|
||||
pub use self::indexer_config::{default_thread_pool_and_threads, IndexerConfig};
|
||||
pub use self::new::ChannelCongestion;
|
||||
pub use self::settings::{validate_embedding_settings, Setting, Settings};
|
||||
pub use self::update_step::UpdateIndexingStep;
|
||||
|
|
|
@ -8,7 +8,7 @@ use hashbrown::HashMap;
|
|||
use serde_json::Value;
|
||||
|
||||
use super::super::cache::BalancedCaches;
|
||||
use super::facet_document::extract_document_facets;
|
||||
use super::facet_document::{extract_document_facets, extract_geo_document};
|
||||
use super::FacetKind;
|
||||
use crate::fields_ids_map::metadata::Metadata;
|
||||
use crate::filterable_attributes_rules::match_faceted_field;
|
||||
|
@ -90,17 +90,12 @@ impl FacetedDocidsExtractor {
|
|||
let mut cached_sorter = context.data.borrow_mut_or_yield();
|
||||
let mut del_add_facet_value = DelAddFacetValue::new(&context.doc_alloc);
|
||||
let docid = document_change.docid();
|
||||
let res = match document_change {
|
||||
DocumentChange::Deletion(inner) => extract_document_facets(
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
inner.external_document_id(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
is_geo_enabled,
|
||||
&mut |fid, meta, depth, value| {
|
||||
|
||||
// Using a macro avoid borrowing the parameters as mutable in both closures at
|
||||
// the same time by postponing their creation
|
||||
macro_rules! facet_fn {
|
||||
(del) => {
|
||||
|fid: FieldId, meta: Metadata, depth: perm_json_p::Depth, value: &Value| {
|
||||
Self::facet_fn_with_options(
|
||||
&context.doc_alloc,
|
||||
cached_sorter.deref_mut(),
|
||||
|
@ -114,91 +109,10 @@ impl FacetedDocidsExtractor {
|
|||
depth,
|
||||
value,
|
||||
)
|
||||
},
|
||||
),
|
||||
DocumentChange::Update(inner) => {
|
||||
let has_changed = inner.has_changed_for_fields(
|
||||
&mut |field_name| {
|
||||
match_faceted_field(
|
||||
field_name,
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
)
|
||||
},
|
||||
rtxn,
|
||||
index,
|
||||
context.db_fields_ids_map,
|
||||
)?;
|
||||
let has_changed_for_geo_fields =
|
||||
inner.has_changed_for_geo_fields(rtxn, index, context.db_fields_ids_map)?;
|
||||
if !has_changed && !has_changed_for_geo_fields {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
extract_document_facets(
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
inner.external_document_id(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
is_geo_enabled,
|
||||
&mut |fid, meta, depth, value| {
|
||||
Self::facet_fn_with_options(
|
||||
&context.doc_alloc,
|
||||
cached_sorter.deref_mut(),
|
||||
BalancedCaches::insert_del_u32,
|
||||
&mut del_add_facet_value,
|
||||
DelAddFacetValue::insert_del,
|
||||
docid,
|
||||
fid,
|
||||
meta,
|
||||
filterable_attributes,
|
||||
depth,
|
||||
value,
|
||||
)
|
||||
},
|
||||
)?;
|
||||
|
||||
extract_document_facets(
|
||||
inner.merged(rtxn, index, context.db_fields_ids_map)?,
|
||||
inner.external_document_id(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
is_geo_enabled,
|
||||
&mut |fid, meta, depth, value| {
|
||||
Self::facet_fn_with_options(
|
||||
&context.doc_alloc,
|
||||
cached_sorter.deref_mut(),
|
||||
BalancedCaches::insert_add_u32,
|
||||
&mut del_add_facet_value,
|
||||
DelAddFacetValue::insert_add,
|
||||
docid,
|
||||
fid,
|
||||
meta,
|
||||
filterable_attributes,
|
||||
depth,
|
||||
value,
|
||||
)
|
||||
},
|
||||
)
|
||||
}
|
||||
DocumentChange::Insertion(inner) => extract_document_facets(
|
||||
inner.inserted(),
|
||||
inner.external_document_id(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
is_geo_enabled,
|
||||
&mut |fid, meta, depth, value| {
|
||||
};
|
||||
(add) => {
|
||||
|fid: FieldId, meta: Metadata, depth: perm_json_p::Depth, value: &Value| {
|
||||
Self::facet_fn_with_options(
|
||||
&context.doc_alloc,
|
||||
cached_sorter.deref_mut(),
|
||||
|
@ -212,12 +126,116 @@ impl FacetedDocidsExtractor {
|
|||
depth,
|
||||
value,
|
||||
)
|
||||
},
|
||||
),
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
match document_change {
|
||||
DocumentChange::Deletion(inner) => {
|
||||
let mut del = facet_fn!(del);
|
||||
|
||||
extract_document_facets(
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map.deref_mut(),
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
&mut del,
|
||||
)?;
|
||||
|
||||
if is_geo_enabled {
|
||||
extract_geo_document(
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
inner.external_document_id(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
&mut del,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
DocumentChange::Update(inner) => {
|
||||
let has_changed_for_facets = inner.has_changed_for_fields(
|
||||
&mut |field_name| {
|
||||
match_faceted_field(
|
||||
field_name,
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
)
|
||||
},
|
||||
rtxn,
|
||||
index,
|
||||
context.db_fields_ids_map,
|
||||
)?;
|
||||
|
||||
// 1. Maybe update doc
|
||||
if has_changed_for_facets {
|
||||
extract_document_facets(
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map.deref_mut(),
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
&mut facet_fn!(del),
|
||||
)?;
|
||||
|
||||
extract_document_facets(
|
||||
inner.merged(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map.deref_mut(),
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
&mut facet_fn!(add),
|
||||
)?;
|
||||
}
|
||||
|
||||
// 2. Maybe update geo
|
||||
if is_geo_enabled
|
||||
&& inner.has_changed_for_geo_fields(rtxn, index, context.db_fields_ids_map)?
|
||||
{
|
||||
extract_geo_document(
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
inner.external_document_id(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
&mut facet_fn!(del),
|
||||
)?;
|
||||
extract_geo_document(
|
||||
inner.merged(rtxn, index, context.db_fields_ids_map)?,
|
||||
inner.external_document_id(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
&mut facet_fn!(add),
|
||||
)?;
|
||||
}
|
||||
}
|
||||
DocumentChange::Insertion(inner) => {
|
||||
let mut add = facet_fn!(add);
|
||||
|
||||
extract_document_facets(
|
||||
inner.inserted(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
filterable_attributes,
|
||||
sortable_fields,
|
||||
asc_desc_fields,
|
||||
distinct_field,
|
||||
&mut add,
|
||||
)?;
|
||||
|
||||
if is_geo_enabled {
|
||||
extract_geo_document(
|
||||
inner.inserted(),
|
||||
inner.external_document_id(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
&mut add,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
del_add_facet_value.send_data(docid, sender, &context.doc_alloc).unwrap();
|
||||
res
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
|
|
|
@ -15,13 +15,11 @@ use crate::{
|
|||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn extract_document_facets<'doc>(
|
||||
document: impl Document<'doc>,
|
||||
external_document_id: &str,
|
||||
field_id_map: &mut GlobalFieldsIdsMap,
|
||||
filterable_attributes: &[FilterableAttributesRule],
|
||||
sortable_fields: &HashSet<String>,
|
||||
asc_desc_fields: &HashSet<String>,
|
||||
distinct_field: &Option<String>,
|
||||
is_geo_enabled: bool,
|
||||
facet_fn: &mut impl FnMut(FieldId, Metadata, perm_json_p::Depth, &Value) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
// return the match result for the given field name.
|
||||
|
@ -101,17 +99,24 @@ pub fn extract_document_facets<'doc>(
|
|||
}
|
||||
}
|
||||
|
||||
if is_geo_enabled {
|
||||
if let Some(geo_value) = document.geo_field()? {
|
||||
if let Some([lat, lng]) = extract_geo_coordinates(external_document_id, geo_value)? {
|
||||
let ((lat_fid, lat_meta), (lng_fid, lng_meta)) = field_id_map
|
||||
.id_with_metadata_or_insert("_geo.lat")
|
||||
.zip(field_id_map.id_with_metadata_or_insert("_geo.lng"))
|
||||
.ok_or(UserError::AttributeLimitReached)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
facet_fn(lat_fid, lat_meta, perm_json_p::Depth::OnBaseKey, &lat.into())?;
|
||||
facet_fn(lng_fid, lng_meta, perm_json_p::Depth::OnBaseKey, &lng.into())?;
|
||||
}
|
||||
pub fn extract_geo_document<'doc>(
|
||||
document: impl Document<'doc>,
|
||||
external_document_id: &str,
|
||||
field_id_map: &mut GlobalFieldsIdsMap,
|
||||
facet_fn: &mut impl FnMut(FieldId, Metadata, perm_json_p::Depth, &Value) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
if let Some(geo_value) = document.geo_field()? {
|
||||
if let Some([lat, lng]) = extract_geo_coordinates(external_document_id, geo_value)? {
|
||||
let ((lat_fid, lat_meta), (lng_fid, lng_meta)) = field_id_map
|
||||
.id_with_metadata_or_insert("_geo.lat")
|
||||
.zip(field_id_map.id_with_metadata_or_insert("_geo.lng"))
|
||||
.ok_or(UserError::AttributeLimitReached)?;
|
||||
|
||||
facet_fn(lat_fid, lat_meta, perm_json_p::Depth::OnBaseKey, &lat.into())?;
|
||||
facet_fn(lng_fid, lng_meta, perm_json_p::Depth::OnBaseKey, &lng.into())?;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -336,7 +336,7 @@ impl<'a, 't, 'i> Settings<'a, 't, 'i> {
|
|||
self.primary_key = Setting::Set(primary_key);
|
||||
}
|
||||
|
||||
pub fn set_autorize_typos(&mut self, val: bool) {
|
||||
pub fn set_authorize_typos(&mut self, val: bool) {
|
||||
self.authorize_typos = Setting::Set(val);
|
||||
}
|
||||
|
||||
|
|
|
@ -792,7 +792,7 @@ fn test_disable_typo() {
|
|||
|
||||
index
|
||||
.update_settings_using_wtxn(&mut txn, |settings| {
|
||||
settings.set_autorize_typos(false);
|
||||
settings.set_authorize_typos(false);
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue