mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-04 20:37:15 +02:00
Merge branch 'indexer-edition-2024' into indexer-edition-2024-doc-chunks
This commit is contained in:
commit
ee03743355
1130 changed files with 8255 additions and 6799 deletions
729
crates/milli/src/update/new/extract/cache.rs
Normal file
729
crates/milli/src/update/new/extract/cache.rs
Normal file
|
@ -0,0 +1,729 @@
|
|||
//! # How the Merge Algorithm works
|
||||
//!
|
||||
//! Each extractor create #Threads caches and balances the entries
|
||||
//! based on the hash of the keys. To do that we can use the
|
||||
//! hashbrown::hash_map::RawEntryBuilderMut::from_key_hashed_nocheck.
|
||||
//! This way we can compute the hash on our own, decide on the cache to
|
||||
//! target, and insert it into the right HashMap.
|
||||
//!
|
||||
//! #Thread -> caches
|
||||
//! t1 -> [t1c1, t1c2, t1c3]
|
||||
//! t2 -> [t2c1, t2c2, t2c3]
|
||||
//! t3 -> [t3c1, t3c2, t3c3]
|
||||
//!
|
||||
//! When the extractors are done filling the caches, we want to merge
|
||||
//! the content of all the caches. We do a transpose and each thread is
|
||||
//! assigned the associated cache. By doing that we know that every key
|
||||
//! is put in a known cache and will collide with keys in the other
|
||||
//! caches of the other threads.
|
||||
//!
|
||||
//! #Thread -> caches
|
||||
//! t1 -> [t1c1, t2c1, t3c1]
|
||||
//! t2 -> [t1c2, t2c2, t3c2]
|
||||
//! t3 -> [t1c3, t2c3, t3c3]
|
||||
//!
|
||||
//! When we encountered a miss in the other caches we must still try
|
||||
//! to find it in the spilled entries. This is the reason why we use
|
||||
//! a grenad sorter/reader so that we can seek "efficiently" for a key.
|
||||
//!
|
||||
//! ## More Detailled Algorithm
|
||||
//!
|
||||
//! Each sub-cache has an in-memory HashMap and some spilled
|
||||
//! lexicographically ordered entries on disk (grenad). We first iterate
|
||||
//! over the spilled entries of all the caches at once by using a merge
|
||||
//! join algorithm. This algorithm will merge the entries by using its
|
||||
//! merge function.
|
||||
//!
|
||||
//! Everytime a merged entry is emited by the merge join algorithm we also
|
||||
//! fetch the value from the other in-memory caches (HashMaps) to finish
|
||||
//! the merge. Everytime we retrieve an entry from the in-memory caches
|
||||
//! we mark them with a tombstone for later.
|
||||
//!
|
||||
//! Once we are done with the spilled entries we iterate over the in-memory
|
||||
//! HashMaps. We iterate over the first one, retrieve the content from the
|
||||
//! other onces and mark them with a tombstone again. We also make sure
|
||||
//! to ignore the dead (tombstoned) ones.
|
||||
//!
|
||||
//! ## Memory Control
|
||||
//!
|
||||
//! We can detect that there are no more memory available when the
|
||||
//! bump allocator reaches a threshold. When this is the case we
|
||||
//! freeze the cache. There is one bump allocator by thread and the
|
||||
//! memory must be well balanced as we manage one type of extraction
|
||||
//! at a time with well-balanced documents.
|
||||
//!
|
||||
//! It means that the unknown new keys added to the
|
||||
//! cache are directly spilled to disk: basically a key followed by a
|
||||
//! del/add bitmap. For the known keys we can keep modifying them in
|
||||
//! the materialized version in the cache: update the del/add bitmaps.
|
||||
//!
|
||||
//! For now we can use a grenad sorter for spilling even thought I think
|
||||
//! it's not the most efficient way (too many files open, sorting entries).
|
||||
|
||||
use std::cmp::Ordering;
|
||||
use std::collections::binary_heap::PeekMut;
|
||||
use std::collections::BinaryHeap;
|
||||
use std::fs::File;
|
||||
use std::hash::BuildHasher;
|
||||
use std::io::BufReader;
|
||||
use std::{io, iter, mem};
|
||||
|
||||
use bumpalo::Bump;
|
||||
use grenad::ReaderCursor;
|
||||
use hashbrown::hash_map::RawEntryMut;
|
||||
use hashbrown::HashMap;
|
||||
use raw_collections::bbbul::{BitPacker, BitPacker4x};
|
||||
use raw_collections::map::FrozenMap;
|
||||
use raw_collections::{Bbbul, FrozenBbbul};
|
||||
use roaring::RoaringBitmap;
|
||||
use rustc_hash::FxBuildHasher;
|
||||
|
||||
use crate::update::del_add::{DelAdd, KvWriterDelAdd};
|
||||
use crate::update::new::indexer::document_changes::MostlySend;
|
||||
use crate::update::new::KvReaderDelAdd;
|
||||
use crate::update::MergeDeladdCboRoaringBitmaps;
|
||||
use crate::{CboRoaringBitmapCodec, Result};
|
||||
|
||||
/// A cache that stores bytes keys associated to CboDelAddRoaringBitmaps.
|
||||
///
|
||||
/// Internally balances the content over `N` buckets for future merging.
|
||||
pub struct BalancedCaches<'extractor> {
|
||||
hasher: FxBuildHasher,
|
||||
alloc: &'extractor Bump,
|
||||
max_memory: Option<usize>,
|
||||
caches: InnerCaches<'extractor>,
|
||||
}
|
||||
|
||||
enum InnerCaches<'extractor> {
|
||||
Normal(NormalCaches<'extractor>),
|
||||
Spilling(SpillingCaches<'extractor>),
|
||||
}
|
||||
|
||||
impl<'extractor> BalancedCaches<'extractor> {
|
||||
pub fn new_in(buckets: usize, max_memory: Option<usize>, alloc: &'extractor Bump) -> Self {
|
||||
Self {
|
||||
hasher: FxBuildHasher,
|
||||
max_memory,
|
||||
caches: InnerCaches::Normal(NormalCaches {
|
||||
caches: iter::repeat_with(|| HashMap::with_hasher_in(FxBuildHasher, alloc))
|
||||
.take(buckets)
|
||||
.collect(),
|
||||
}),
|
||||
alloc,
|
||||
}
|
||||
}
|
||||
|
||||
fn buckets(&self) -> usize {
|
||||
match &self.caches {
|
||||
InnerCaches::Normal(caches) => caches.caches.len(),
|
||||
InnerCaches::Spilling(caches) => caches.caches.len(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn insert_del_u32(&mut self, key: &[u8], n: u32) -> Result<()> {
|
||||
if self.max_memory.map_or(false, |mm| self.alloc.allocated_bytes() >= mm) {
|
||||
self.start_spilling()?;
|
||||
}
|
||||
|
||||
let buckets = self.buckets();
|
||||
match &mut self.caches {
|
||||
InnerCaches::Normal(normal) => {
|
||||
normal.insert_del_u32(&self.hasher, self.alloc, buckets, key, n);
|
||||
Ok(())
|
||||
}
|
||||
InnerCaches::Spilling(spilling) => {
|
||||
spilling.insert_del_u32(&self.hasher, self.alloc, buckets, key, n)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn insert_add_u32(&mut self, key: &[u8], n: u32) -> Result<()> {
|
||||
if self.max_memory.map_or(false, |mm| self.alloc.allocated_bytes() >= mm) {
|
||||
self.start_spilling()?;
|
||||
}
|
||||
|
||||
let buckets = self.buckets();
|
||||
match &mut self.caches {
|
||||
InnerCaches::Normal(normal) => {
|
||||
normal.insert_add_u32(&self.hasher, self.alloc, buckets, key, n);
|
||||
Ok(())
|
||||
}
|
||||
InnerCaches::Spilling(spilling) => {
|
||||
spilling.insert_add_u32(&self.hasher, self.alloc, buckets, key, n)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Make sure the cache is no longer allocating data
|
||||
/// and writes every new and unknow entry to disk.
|
||||
fn start_spilling(&mut self) -> Result<()> {
|
||||
let BalancedCaches { hasher: _, alloc, max_memory: _, caches } = self;
|
||||
|
||||
if let InnerCaches::Normal(normal_caches) = caches {
|
||||
eprintln!(
|
||||
"We are spilling after we allocated {} bytes on thread #{}",
|
||||
alloc.allocated_bytes(),
|
||||
rayon::current_thread_index().unwrap_or(0)
|
||||
);
|
||||
|
||||
let allocated: usize = normal_caches.caches.iter().map(|m| m.allocation_size()).sum();
|
||||
eprintln!("The last allocated HashMap took {allocated} bytes");
|
||||
|
||||
let dummy = NormalCaches { caches: Vec::new() };
|
||||
let NormalCaches { caches: cache_maps } = mem::replace(normal_caches, dummy);
|
||||
*caches = InnerCaches::Spilling(SpillingCaches::from_cache_maps(cache_maps));
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn freeze(&mut self) -> Result<Vec<FrozenCache<'_, 'extractor>>> {
|
||||
match &mut self.caches {
|
||||
InnerCaches::Normal(NormalCaches { caches }) => caches
|
||||
.iter_mut()
|
||||
.enumerate()
|
||||
.map(|(bucket, map)| {
|
||||
// safety: we are transmuting the Bbbul into a FrozenBbbul
|
||||
// that are the same size.
|
||||
let map = unsafe {
|
||||
std::mem::transmute::<
|
||||
&mut HashMap<
|
||||
&[u8],
|
||||
DelAddBbbul<BitPacker4x>, // from this
|
||||
FxBuildHasher,
|
||||
&Bump,
|
||||
>,
|
||||
&mut HashMap<
|
||||
&[u8],
|
||||
FrozenDelAddBbbul<BitPacker4x>, // to that
|
||||
FxBuildHasher,
|
||||
&Bump,
|
||||
>,
|
||||
>(map)
|
||||
};
|
||||
Ok(FrozenCache { bucket, cache: FrozenMap::new(map), spilled: Vec::new() })
|
||||
})
|
||||
.collect(),
|
||||
InnerCaches::Spilling(SpillingCaches { caches, spilled_entries, .. }) => caches
|
||||
.iter_mut()
|
||||
.zip(mem::take(spilled_entries))
|
||||
.enumerate()
|
||||
.map(|(bucket, (map, sorter))| {
|
||||
let spilled = sorter
|
||||
.into_reader_cursors()?
|
||||
.into_iter()
|
||||
.map(ReaderCursor::into_inner)
|
||||
.map(BufReader::new)
|
||||
.map(|bufreader| grenad::Reader::new(bufreader).map_err(Into::into))
|
||||
.collect::<Result<_>>()?;
|
||||
// safety: we are transmuting the Bbbul into a FrozenBbbul
|
||||
// that are the same size.
|
||||
let map = unsafe {
|
||||
std::mem::transmute::<
|
||||
&mut HashMap<
|
||||
&[u8],
|
||||
DelAddBbbul<BitPacker4x>, // from this
|
||||
FxBuildHasher,
|
||||
&Bump,
|
||||
>,
|
||||
&mut HashMap<
|
||||
&[u8],
|
||||
FrozenDelAddBbbul<BitPacker4x>, // to that
|
||||
FxBuildHasher,
|
||||
&Bump,
|
||||
>,
|
||||
>(map)
|
||||
};
|
||||
Ok(FrozenCache { bucket, cache: FrozenMap::new(map), spilled })
|
||||
})
|
||||
.collect(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
unsafe impl MostlySend for BalancedCaches<'_> {}
|
||||
|
||||
struct NormalCaches<'extractor> {
|
||||
caches: Vec<
|
||||
HashMap<
|
||||
&'extractor [u8],
|
||||
DelAddBbbul<'extractor, BitPacker4x>,
|
||||
FxBuildHasher,
|
||||
&'extractor Bump,
|
||||
>,
|
||||
>,
|
||||
}
|
||||
|
||||
impl<'extractor> NormalCaches<'extractor> {
|
||||
pub fn insert_del_u32(
|
||||
&mut self,
|
||||
hasher: &FxBuildHasher,
|
||||
alloc: &'extractor Bump,
|
||||
buckets: usize,
|
||||
key: &[u8],
|
||||
n: u32,
|
||||
) {
|
||||
let hash = hasher.hash_one(key);
|
||||
let bucket = compute_bucket_from_hash(buckets, hash);
|
||||
|
||||
match self.caches[bucket].raw_entry_mut().from_hash(hash, |&k| k == key) {
|
||||
RawEntryMut::Occupied(mut entry) => {
|
||||
entry.get_mut().del.get_or_insert_with(|| Bbbul::new_in(alloc)).insert(n);
|
||||
}
|
||||
RawEntryMut::Vacant(entry) => {
|
||||
entry.insert_hashed_nocheck(
|
||||
hash,
|
||||
alloc.alloc_slice_copy(key),
|
||||
DelAddBbbul::new_del_u32_in(n, alloc),
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn insert_add_u32(
|
||||
&mut self,
|
||||
hasher: &FxBuildHasher,
|
||||
alloc: &'extractor Bump,
|
||||
buckets: usize,
|
||||
key: &[u8],
|
||||
n: u32,
|
||||
) {
|
||||
let hash = hasher.hash_one(key);
|
||||
let bucket = compute_bucket_from_hash(buckets, hash);
|
||||
match self.caches[bucket].raw_entry_mut().from_hash(hash, |&k| k == key) {
|
||||
RawEntryMut::Occupied(mut entry) => {
|
||||
entry.get_mut().add.get_or_insert_with(|| Bbbul::new_in(alloc)).insert(n);
|
||||
}
|
||||
RawEntryMut::Vacant(entry) => {
|
||||
entry.insert_hashed_nocheck(
|
||||
hash,
|
||||
alloc.alloc_slice_copy(key),
|
||||
DelAddBbbul::new_add_u32_in(n, alloc),
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct SpillingCaches<'extractor> {
|
||||
caches: Vec<
|
||||
HashMap<
|
||||
&'extractor [u8],
|
||||
DelAddBbbul<'extractor, BitPacker4x>,
|
||||
FxBuildHasher,
|
||||
&'extractor Bump,
|
||||
>,
|
||||
>,
|
||||
spilled_entries: Vec<grenad::Sorter<MergeDeladdCboRoaringBitmaps>>,
|
||||
deladd_buffer: Vec<u8>,
|
||||
cbo_buffer: Vec<u8>,
|
||||
}
|
||||
|
||||
impl<'extractor> SpillingCaches<'extractor> {
|
||||
fn from_cache_maps(
|
||||
caches: Vec<
|
||||
HashMap<
|
||||
&'extractor [u8],
|
||||
DelAddBbbul<'extractor, BitPacker4x>,
|
||||
FxBuildHasher,
|
||||
&'extractor Bump,
|
||||
>,
|
||||
>,
|
||||
) -> SpillingCaches<'extractor> {
|
||||
SpillingCaches {
|
||||
spilled_entries: iter::repeat_with(|| {
|
||||
let mut builder = grenad::SorterBuilder::new(MergeDeladdCboRoaringBitmaps);
|
||||
builder.dump_threshold(0);
|
||||
builder.allow_realloc(false);
|
||||
builder.build()
|
||||
})
|
||||
.take(caches.len())
|
||||
.collect(),
|
||||
caches,
|
||||
deladd_buffer: Vec::new(),
|
||||
cbo_buffer: Vec::new(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn insert_del_u32(
|
||||
&mut self,
|
||||
hasher: &FxBuildHasher,
|
||||
alloc: &'extractor Bump,
|
||||
buckets: usize,
|
||||
key: &[u8],
|
||||
n: u32,
|
||||
) -> Result<()> {
|
||||
let hash = hasher.hash_one(key);
|
||||
let bucket = compute_bucket_from_hash(buckets, hash);
|
||||
match self.caches[bucket].raw_entry_mut().from_hash(hash, |&k| k == key) {
|
||||
RawEntryMut::Occupied(mut entry) => {
|
||||
entry.get_mut().del.get_or_insert_with(|| Bbbul::new_in(alloc)).insert(n);
|
||||
Ok(())
|
||||
}
|
||||
RawEntryMut::Vacant(_entry) => spill_entry_to_sorter(
|
||||
&mut self.spilled_entries[bucket],
|
||||
&mut self.deladd_buffer,
|
||||
&mut self.cbo_buffer,
|
||||
key,
|
||||
DelAddRoaringBitmap::new_del_u32(n),
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn insert_add_u32(
|
||||
&mut self,
|
||||
hasher: &FxBuildHasher,
|
||||
alloc: &'extractor Bump,
|
||||
buckets: usize,
|
||||
key: &[u8],
|
||||
n: u32,
|
||||
) -> Result<()> {
|
||||
let hash = hasher.hash_one(key);
|
||||
let bucket = compute_bucket_from_hash(buckets, hash);
|
||||
match self.caches[bucket].raw_entry_mut().from_hash(hash, |&k| k == key) {
|
||||
RawEntryMut::Occupied(mut entry) => {
|
||||
entry.get_mut().add.get_or_insert_with(|| Bbbul::new_in(alloc)).insert(n);
|
||||
Ok(())
|
||||
}
|
||||
RawEntryMut::Vacant(_entry) => spill_entry_to_sorter(
|
||||
&mut self.spilled_entries[bucket],
|
||||
&mut self.deladd_buffer,
|
||||
&mut self.cbo_buffer,
|
||||
key,
|
||||
DelAddRoaringBitmap::new_add_u32(n),
|
||||
),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn compute_bucket_from_hash(buckets: usize, hash: u64) -> usize {
|
||||
hash as usize % buckets
|
||||
}
|
||||
|
||||
fn spill_entry_to_sorter(
|
||||
spilled_entries: &mut grenad::Sorter<MergeDeladdCboRoaringBitmaps>,
|
||||
deladd_buffer: &mut Vec<u8>,
|
||||
cbo_buffer: &mut Vec<u8>,
|
||||
key: &[u8],
|
||||
deladd: DelAddRoaringBitmap,
|
||||
) -> Result<()> {
|
||||
deladd_buffer.clear();
|
||||
let mut value_writer = KvWriterDelAdd::new(deladd_buffer);
|
||||
|
||||
match deladd {
|
||||
DelAddRoaringBitmap { del: Some(del), add: None } => {
|
||||
cbo_buffer.clear();
|
||||
CboRoaringBitmapCodec::serialize_into(&del, cbo_buffer);
|
||||
value_writer.insert(DelAdd::Deletion, &cbo_buffer)?;
|
||||
}
|
||||
DelAddRoaringBitmap { del: None, add: Some(add) } => {
|
||||
cbo_buffer.clear();
|
||||
CboRoaringBitmapCodec::serialize_into(&add, cbo_buffer);
|
||||
value_writer.insert(DelAdd::Addition, &cbo_buffer)?;
|
||||
}
|
||||
DelAddRoaringBitmap { del: Some(del), add: Some(add) } => {
|
||||
cbo_buffer.clear();
|
||||
CboRoaringBitmapCodec::serialize_into(&del, cbo_buffer);
|
||||
value_writer.insert(DelAdd::Deletion, &cbo_buffer)?;
|
||||
|
||||
cbo_buffer.clear();
|
||||
CboRoaringBitmapCodec::serialize_into(&add, cbo_buffer);
|
||||
value_writer.insert(DelAdd::Addition, &cbo_buffer)?;
|
||||
}
|
||||
DelAddRoaringBitmap { del: None, add: None } => return Ok(()),
|
||||
}
|
||||
|
||||
let bytes = value_writer.into_inner().unwrap();
|
||||
spilled_entries.insert(key, bytes).map_err(Into::into)
|
||||
}
|
||||
|
||||
pub struct FrozenCache<'a, 'extractor> {
|
||||
bucket: usize,
|
||||
cache: FrozenMap<
|
||||
'a,
|
||||
'extractor,
|
||||
&'extractor [u8],
|
||||
FrozenDelAddBbbul<'extractor, BitPacker4x>,
|
||||
FxBuildHasher,
|
||||
>,
|
||||
spilled: Vec<grenad::Reader<BufReader<File>>>,
|
||||
}
|
||||
|
||||
pub fn transpose_and_freeze_caches<'a, 'extractor>(
|
||||
caches: &'a mut [BalancedCaches<'extractor>],
|
||||
) -> Result<Vec<Vec<FrozenCache<'a, 'extractor>>>> {
|
||||
let width = caches.first().map(BalancedCaches::buckets).unwrap_or(0);
|
||||
let mut bucket_caches: Vec<_> = iter::repeat_with(Vec::new).take(width).collect();
|
||||
|
||||
for thread_cache in caches {
|
||||
for frozen in thread_cache.freeze()? {
|
||||
bucket_caches[frozen.bucket].push(frozen);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(bucket_caches)
|
||||
}
|
||||
|
||||
/// Merges the caches that must be all associated to the same bucket.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// - If the bucket IDs in these frozen caches are not exactly the same.
|
||||
pub fn merge_caches<F>(frozen: Vec<FrozenCache>, mut f: F) -> Result<()>
|
||||
where
|
||||
F: for<'a> FnMut(&'a [u8], DelAddRoaringBitmap) -> Result<()>,
|
||||
{
|
||||
let mut maps = Vec::new();
|
||||
let mut readers = Vec::new();
|
||||
let mut current_bucket = None;
|
||||
for FrozenCache { bucket, cache, ref mut spilled } in frozen {
|
||||
assert_eq!(*current_bucket.get_or_insert(bucket), bucket);
|
||||
maps.push(cache);
|
||||
readers.append(spilled);
|
||||
}
|
||||
|
||||
// First manage the spilled entries by looking into the HashMaps,
|
||||
// merge them and mark them as dummy.
|
||||
let mut heap = BinaryHeap::new();
|
||||
for (source_index, source) in readers.into_iter().enumerate() {
|
||||
let mut cursor = source.into_cursor()?;
|
||||
if cursor.move_on_next()?.is_some() {
|
||||
heap.push(Entry { cursor, source_index });
|
||||
}
|
||||
}
|
||||
|
||||
loop {
|
||||
let mut first_entry = match heap.pop() {
|
||||
Some(entry) => entry,
|
||||
None => break,
|
||||
};
|
||||
|
||||
let (first_key, first_value) = match first_entry.cursor.current() {
|
||||
Some((key, value)) => (key, value),
|
||||
None => break,
|
||||
};
|
||||
|
||||
let mut output = DelAddRoaringBitmap::from_bytes(first_value)?;
|
||||
while let Some(mut entry) = heap.peek_mut() {
|
||||
if let Some((key, _value)) = entry.cursor.current() {
|
||||
if first_key == key {
|
||||
let new = DelAddRoaringBitmap::from_bytes(first_value)?;
|
||||
output = output.merge(new);
|
||||
// When we are done we the current value of this entry move make
|
||||
// it move forward and let the heap reorganize itself (on drop)
|
||||
if entry.cursor.move_on_next()?.is_none() {
|
||||
PeekMut::pop(entry);
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Once we merged all of the spilled bitmaps we must also
|
||||
// fetch the entries from the non-spilled entries (the HashMaps).
|
||||
for (map_index, map) in maps.iter_mut().enumerate() {
|
||||
if first_entry.source_index != map_index {
|
||||
if let Some(new) = map.get_mut(first_key) {
|
||||
output.append_and_clear_bbbul(new);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// We send the merged entry outside.
|
||||
(f)(first_key, output)?;
|
||||
|
||||
// Don't forget to put the first entry back into the heap.
|
||||
if first_entry.cursor.move_on_next()?.is_some() {
|
||||
heap.push(first_entry)
|
||||
}
|
||||
}
|
||||
|
||||
// Then manage the content on the HashMap entries that weren't taken (mem::take).
|
||||
while let Some(mut map) = maps.pop() {
|
||||
for (key, bbbul) in map.iter_mut() {
|
||||
let mut output = DelAddRoaringBitmap::empty();
|
||||
output.append_and_clear_bbbul(bbbul);
|
||||
|
||||
// Make sure we don't try to work with entries already managed by the spilled
|
||||
if !bbbul.is_empty() {
|
||||
for rhs in maps.iter_mut() {
|
||||
if let Some(new) = rhs.get_mut(key) {
|
||||
output.append_and_clear_bbbul(new);
|
||||
}
|
||||
}
|
||||
|
||||
// We send the merged entry outside.
|
||||
(f)(key, output)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
struct Entry<R> {
|
||||
cursor: ReaderCursor<R>,
|
||||
source_index: usize,
|
||||
}
|
||||
|
||||
impl<R> Ord for Entry<R> {
|
||||
fn cmp(&self, other: &Entry<R>) -> Ordering {
|
||||
let skey = self.cursor.current().map(|(k, _)| k);
|
||||
let okey = other.cursor.current().map(|(k, _)| k);
|
||||
skey.cmp(&okey).then(self.source_index.cmp(&other.source_index)).reverse()
|
||||
}
|
||||
}
|
||||
|
||||
impl<R> Eq for Entry<R> {}
|
||||
|
||||
impl<R> PartialEq for Entry<R> {
|
||||
fn eq(&self, other: &Entry<R>) -> bool {
|
||||
self.cmp(other) == Ordering::Equal
|
||||
}
|
||||
}
|
||||
|
||||
impl<R> PartialOrd for Entry<R> {
|
||||
fn partial_cmp(&self, other: &Entry<R>) -> Option<Ordering> {
|
||||
Some(self.cmp(other))
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DelAddBbbul<'bump, B> {
|
||||
pub del: Option<Bbbul<'bump, B>>,
|
||||
pub add: Option<Bbbul<'bump, B>>,
|
||||
}
|
||||
|
||||
impl<'bump, B: BitPacker> DelAddBbbul<'bump, B> {
|
||||
pub fn insert_del_u32_in(&mut self, n: u32, bump: &'bump Bump) {
|
||||
self.del.get_or_insert_with(|| Bbbul::new_in(bump)).insert(n);
|
||||
}
|
||||
|
||||
pub fn insert_add_u32_in(&mut self, n: u32, bump: &'bump Bump) {
|
||||
self.add.get_or_insert_with(|| Bbbul::new_in(bump)).insert(n);
|
||||
}
|
||||
|
||||
pub fn new_del_u32_in(n: u32, bump: &'bump Bump) -> Self {
|
||||
let mut bbbul = Bbbul::new_in(bump);
|
||||
bbbul.insert(n);
|
||||
DelAddBbbul { del: Some(bbbul), add: None }
|
||||
}
|
||||
|
||||
pub fn new_add_u32_in(n: u32, bump: &'bump Bump) -> Self {
|
||||
let mut bbbul = Bbbul::new_in(bump);
|
||||
bbbul.insert(n);
|
||||
DelAddBbbul { del: None, add: Some(bbbul) }
|
||||
}
|
||||
}
|
||||
|
||||
pub struct FrozenDelAddBbbul<'bump, B> {
|
||||
pub del: Option<FrozenBbbul<'bump, B>>,
|
||||
pub add: Option<FrozenBbbul<'bump, B>>,
|
||||
}
|
||||
|
||||
impl<'bump, B> FrozenDelAddBbbul<'bump, B> {
|
||||
fn is_empty(&self) -> bool {
|
||||
self.del.is_none() && self.add.is_none()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Default, Clone)]
|
||||
pub struct DelAddRoaringBitmap {
|
||||
pub del: Option<RoaringBitmap>,
|
||||
pub add: Option<RoaringBitmap>,
|
||||
}
|
||||
|
||||
impl DelAddRoaringBitmap {
|
||||
fn from_bytes(bytes: &[u8]) -> io::Result<DelAddRoaringBitmap> {
|
||||
let reader = KvReaderDelAdd::from_slice(bytes);
|
||||
|
||||
let del = match reader.get(DelAdd::Deletion) {
|
||||
Some(bytes) => CboRoaringBitmapCodec::deserialize_from(bytes).map(Some)?,
|
||||
None => None,
|
||||
};
|
||||
|
||||
let add = match reader.get(DelAdd::Addition) {
|
||||
Some(bytes) => CboRoaringBitmapCodec::deserialize_from(bytes).map(Some)?,
|
||||
None => None,
|
||||
};
|
||||
|
||||
Ok(DelAddRoaringBitmap { del, add })
|
||||
}
|
||||
|
||||
pub fn empty() -> DelAddRoaringBitmap {
|
||||
DelAddRoaringBitmap { del: None, add: None }
|
||||
}
|
||||
|
||||
pub fn is_empty(&self) -> bool {
|
||||
let DelAddRoaringBitmap { del, add } = self;
|
||||
del.is_none() && add.is_none()
|
||||
}
|
||||
|
||||
pub fn insert_del_u32(&mut self, n: u32) {
|
||||
self.del.get_or_insert_with(RoaringBitmap::new).insert(n);
|
||||
}
|
||||
|
||||
pub fn insert_add_u32(&mut self, n: u32) {
|
||||
self.add.get_or_insert_with(RoaringBitmap::new).insert(n);
|
||||
}
|
||||
|
||||
pub fn new_del_u32(n: u32) -> Self {
|
||||
DelAddRoaringBitmap { del: Some(RoaringBitmap::from([n])), add: None }
|
||||
}
|
||||
|
||||
pub fn new_add_u32(n: u32) -> Self {
|
||||
DelAddRoaringBitmap { del: None, add: Some(RoaringBitmap::from([n])) }
|
||||
}
|
||||
|
||||
pub fn append_and_clear_bbbul<B: BitPacker>(&mut self, bbbul: &mut FrozenDelAddBbbul<'_, B>) {
|
||||
let FrozenDelAddBbbul { del, add } = bbbul;
|
||||
|
||||
if let Some(ref mut bbbul) = del.take() {
|
||||
let del = self.del.get_or_insert_with(RoaringBitmap::new);
|
||||
let mut iter = bbbul.iter_and_clear();
|
||||
while let Some(block) = iter.next_block() {
|
||||
del.append(block.iter().copied());
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(ref mut bbbul) = add.take() {
|
||||
let add = self.add.get_or_insert_with(RoaringBitmap::new);
|
||||
let mut iter = bbbul.iter_and_clear();
|
||||
while let Some(block) = iter.next_block() {
|
||||
add.append(block.iter().copied());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn merge(self, rhs: DelAddRoaringBitmap) -> DelAddRoaringBitmap {
|
||||
let DelAddRoaringBitmap { del, add } = self;
|
||||
let DelAddRoaringBitmap { del: ndel, add: nadd } = rhs;
|
||||
|
||||
let del = match (del, ndel) {
|
||||
(None, None) => None,
|
||||
(None, Some(del)) | (Some(del), None) => Some(del),
|
||||
(Some(del), Some(ndel)) => Some(del | ndel),
|
||||
};
|
||||
|
||||
let add = match (add, nadd) {
|
||||
(None, None) => None,
|
||||
(None, Some(add)) | (Some(add), None) => Some(add),
|
||||
(Some(add), Some(nadd)) => Some(add | nadd),
|
||||
};
|
||||
|
||||
DelAddRoaringBitmap { del, add }
|
||||
}
|
||||
|
||||
pub fn apply_to(&self, documents_ids: &mut RoaringBitmap) {
|
||||
let DelAddRoaringBitmap { del, add } = self;
|
||||
|
||||
if let Some(del) = del {
|
||||
*documents_ids -= del;
|
||||
}
|
||||
|
||||
if let Some(add) = add {
|
||||
*documents_ids |= add;
|
||||
}
|
||||
}
|
||||
}
|
73
crates/milli/src/update/new/extract/documents.rs
Normal file
73
crates/milli/src/update/new/extract/documents.rs
Normal file
|
@ -0,0 +1,73 @@
|
|||
use std::cell::RefCell;
|
||||
|
||||
use bumpalo::Bump;
|
||||
|
||||
use super::DelAddRoaringBitmap;
|
||||
use crate::update::new::channel::DocumentsSender;
|
||||
use crate::update::new::document::write_to_obkv;
|
||||
use crate::update::new::indexer::document_changes::{
|
||||
DocumentChangeContext, Extractor, FullySend, RefCellExt as _,
|
||||
};
|
||||
use crate::update::new::DocumentChange;
|
||||
use crate::Result;
|
||||
|
||||
pub struct DocumentsExtractor<'a> {
|
||||
documents_sender: &'a DocumentsSender<'a>,
|
||||
}
|
||||
|
||||
impl<'a> DocumentsExtractor<'a> {
|
||||
pub fn new(documents_sender: &'a DocumentsSender<'a>) -> Self {
|
||||
Self { documents_sender }
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'extractor> Extractor<'extractor> for DocumentsExtractor<'a> {
|
||||
type Data = FullySend<RefCell<DelAddRoaringBitmap>>;
|
||||
|
||||
fn init_data(&self, _extractor_alloc: &'extractor Bump) -> Result<Self::Data> {
|
||||
Ok(FullySend(RefCell::new(DelAddRoaringBitmap::empty())))
|
||||
}
|
||||
|
||||
fn process(
|
||||
&self,
|
||||
change: DocumentChange,
|
||||
context: &DocumentChangeContext<Self::Data>,
|
||||
) -> Result<()> {
|
||||
let mut document_buffer = Vec::new();
|
||||
let mut delta_documents_ids = context.data.0.borrow_mut_or_yield();
|
||||
|
||||
let new_fields_ids_map = context.new_fields_ids_map.borrow_or_yield();
|
||||
let new_fields_ids_map = &*new_fields_ids_map;
|
||||
let new_fields_ids_map = new_fields_ids_map.local_map();
|
||||
|
||||
let external_docid = change.external_docid().to_owned();
|
||||
|
||||
// document but we need to create a function that collects and compresses documents.
|
||||
match change {
|
||||
DocumentChange::Deletion(deletion) => {
|
||||
let docid = deletion.docid();
|
||||
self.documents_sender.delete(docid, external_docid).unwrap();
|
||||
delta_documents_ids.insert_del_u32(docid);
|
||||
}
|
||||
/// TODO: change NONE by SOME(vector) when implemented
|
||||
DocumentChange::Update(update) => {
|
||||
let docid = update.docid();
|
||||
let content =
|
||||
update.new(&context.txn, context.index, &context.db_fields_ids_map)?;
|
||||
let content =
|
||||
write_to_obkv(&content, None, new_fields_ids_map, &mut document_buffer)?;
|
||||
self.documents_sender.uncompressed(docid, external_docid, content).unwrap();
|
||||
}
|
||||
DocumentChange::Insertion(insertion) => {
|
||||
let docid = insertion.docid();
|
||||
let content = insertion.new();
|
||||
let content =
|
||||
write_to_obkv(&content, None, new_fields_ids_map, &mut document_buffer)?;
|
||||
self.documents_sender.uncompressed(docid, external_docid, content).unwrap();
|
||||
delta_documents_ids.insert_add_u32(docid);
|
||||
// extracted_dictionary_sender.send(self, dictionary: &[u8]);
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
271
crates/milli/src/update/new/extract/faceted/extract_facets.rs
Normal file
271
crates/milli/src/update/new/extract/faceted/extract_facets.rs
Normal file
|
@ -0,0 +1,271 @@
|
|||
use std::cell::RefCell;
|
||||
use std::collections::HashSet;
|
||||
use std::ops::DerefMut as _;
|
||||
|
||||
use bumpalo::Bump;
|
||||
use heed::RoTxn;
|
||||
use serde_json::Value;
|
||||
|
||||
use super::super::cache::BalancedCaches;
|
||||
use super::facet_document::extract_document_facets;
|
||||
use super::FacetKind;
|
||||
use crate::facet::value_encoding::f64_into_bytes;
|
||||
use crate::update::new::extract::DocidsExtractor;
|
||||
use crate::update::new::indexer::document_changes::{
|
||||
extract, DocumentChangeContext, DocumentChanges, Extractor, FullySend, IndexingContext,
|
||||
Progress, RefCellExt, ThreadLocal,
|
||||
};
|
||||
use crate::update::new::DocumentChange;
|
||||
use crate::update::GrenadParameters;
|
||||
use crate::{DocumentId, FieldId, Index, Result, MAX_FACET_VALUE_LENGTH};
|
||||
|
||||
pub struct FacetedExtractorData<'a> {
|
||||
attributes_to_extract: &'a [&'a str],
|
||||
grenad_parameters: GrenadParameters,
|
||||
buckets: usize,
|
||||
}
|
||||
|
||||
impl<'a, 'extractor> Extractor<'extractor> for FacetedExtractorData<'a> {
|
||||
type Data = RefCell<BalancedCaches<'extractor>>;
|
||||
|
||||
fn init_data(&self, extractor_alloc: &'extractor Bump) -> Result<Self::Data> {
|
||||
Ok(RefCell::new(BalancedCaches::new_in(
|
||||
self.buckets,
|
||||
self.grenad_parameters.max_memory,
|
||||
extractor_alloc,
|
||||
)))
|
||||
}
|
||||
|
||||
fn process<'doc>(
|
||||
&self,
|
||||
changes: impl Iterator<Item = Result<DocumentChange<'doc>>>,
|
||||
context: &DocumentChangeContext<Self::Data>,
|
||||
) -> Result<()> {
|
||||
for change in changes {
|
||||
let change = change?;
|
||||
FacetedDocidsExtractor::extract_document_change(
|
||||
context,
|
||||
self.attributes_to_extract,
|
||||
change,
|
||||
)?
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
pub struct FacetedDocidsExtractor;
|
||||
|
||||
impl FacetedDocidsExtractor {
|
||||
fn extract_document_change(
|
||||
context: &DocumentChangeContext<RefCell<BalancedCaches>>,
|
||||
attributes_to_extract: &[&str],
|
||||
document_change: DocumentChange,
|
||||
) -> Result<()> {
|
||||
let index = &context.index;
|
||||
let rtxn = &context.txn;
|
||||
let mut new_fields_ids_map = context.new_fields_ids_map.borrow_mut_or_yield();
|
||||
let mut cached_sorter = context.data.borrow_mut_or_yield();
|
||||
match document_change {
|
||||
DocumentChange::Deletion(inner) => extract_document_facets(
|
||||
attributes_to_extract,
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map.deref_mut(),
|
||||
&mut |fid, value| {
|
||||
Self::facet_fn_with_options(
|
||||
&context.doc_alloc,
|
||||
cached_sorter.deref_mut(),
|
||||
BalancedCaches::insert_del_u32,
|
||||
inner.docid(),
|
||||
fid,
|
||||
value,
|
||||
)
|
||||
},
|
||||
),
|
||||
DocumentChange::Update(inner) => {
|
||||
extract_document_facets(
|
||||
attributes_to_extract,
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map.deref_mut(),
|
||||
&mut |fid, value| {
|
||||
Self::facet_fn_with_options(
|
||||
&context.doc_alloc,
|
||||
cached_sorter.deref_mut(),
|
||||
BalancedCaches::insert_del_u32,
|
||||
inner.docid(),
|
||||
fid,
|
||||
value,
|
||||
)
|
||||
},
|
||||
)?;
|
||||
|
||||
extract_document_facets(
|
||||
attributes_to_extract,
|
||||
inner.merged(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map.deref_mut(),
|
||||
&mut |fid, value| {
|
||||
Self::facet_fn_with_options(
|
||||
&context.doc_alloc,
|
||||
cached_sorter.deref_mut(),
|
||||
BalancedCaches::insert_add_u32,
|
||||
inner.docid(),
|
||||
fid,
|
||||
value,
|
||||
)
|
||||
},
|
||||
)
|
||||
}
|
||||
DocumentChange::Insertion(inner) => extract_document_facets(
|
||||
attributes_to_extract,
|
||||
inner.inserted(),
|
||||
new_fields_ids_map.deref_mut(),
|
||||
&mut |fid, value| {
|
||||
Self::facet_fn_with_options(
|
||||
&context.doc_alloc,
|
||||
cached_sorter.deref_mut(),
|
||||
BalancedCaches::insert_add_u32,
|
||||
inner.docid(),
|
||||
fid,
|
||||
value,
|
||||
)
|
||||
},
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
fn facet_fn_with_options<'extractor>(
|
||||
doc_alloc: &Bump,
|
||||
cached_sorter: &mut BalancedCaches<'extractor>,
|
||||
cache_fn: impl Fn(&mut BalancedCaches<'extractor>, &[u8], u32) -> Result<()>,
|
||||
docid: DocumentId,
|
||||
fid: FieldId,
|
||||
value: &Value,
|
||||
) -> Result<()> {
|
||||
let mut buffer = bumpalo::collections::Vec::new_in(doc_alloc);
|
||||
// Exists
|
||||
// key: fid
|
||||
buffer.push(FacetKind::Exists as u8);
|
||||
buffer.extend_from_slice(&fid.to_be_bytes());
|
||||
cache_fn(cached_sorter, &buffer, docid)?;
|
||||
|
||||
match value {
|
||||
// Number
|
||||
// key: fid - level - orderedf64 - orignalf64
|
||||
Value::Number(number) => {
|
||||
if let Some((n, ordered)) =
|
||||
number.as_f64().and_then(|n| f64_into_bytes(n).map(|ordered| (n, ordered)))
|
||||
{
|
||||
buffer.clear();
|
||||
buffer.push(FacetKind::Number as u8);
|
||||
buffer.extend_from_slice(&fid.to_be_bytes());
|
||||
buffer.push(0); // level 0
|
||||
buffer.extend_from_slice(&ordered);
|
||||
buffer.extend_from_slice(&n.to_be_bytes());
|
||||
cache_fn(cached_sorter, &buffer, docid)
|
||||
} else {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
// String
|
||||
// key: fid - level - truncated_string
|
||||
Value::String(s) => {
|
||||
let normalized = crate::normalize_facet(s);
|
||||
let truncated = truncate_str(&normalized);
|
||||
buffer.clear();
|
||||
buffer.push(FacetKind::String as u8);
|
||||
buffer.extend_from_slice(&fid.to_be_bytes());
|
||||
buffer.push(0); // level 0
|
||||
buffer.extend_from_slice(truncated.as_bytes());
|
||||
cache_fn(cached_sorter, &buffer, docid)
|
||||
}
|
||||
// Null
|
||||
// key: fid
|
||||
Value::Null => {
|
||||
buffer.clear();
|
||||
buffer.push(FacetKind::Null as u8);
|
||||
buffer.extend_from_slice(&fid.to_be_bytes());
|
||||
cache_fn(cached_sorter, &buffer, docid)
|
||||
}
|
||||
// Empty
|
||||
// key: fid
|
||||
Value::Array(a) if a.is_empty() => {
|
||||
buffer.clear();
|
||||
buffer.push(FacetKind::Empty as u8);
|
||||
buffer.extend_from_slice(&fid.to_be_bytes());
|
||||
cache_fn(cached_sorter, &buffer, docid)
|
||||
}
|
||||
Value::Object(o) if o.is_empty() => {
|
||||
buffer.clear();
|
||||
buffer.push(FacetKind::Empty as u8);
|
||||
buffer.extend_from_slice(&fid.to_be_bytes());
|
||||
cache_fn(cached_sorter, &buffer, docid)
|
||||
}
|
||||
// Otherwise, do nothing
|
||||
/// TODO: What about Value::Bool?
|
||||
_ => Ok(()),
|
||||
}
|
||||
}
|
||||
|
||||
fn attributes_to_extract<'a>(rtxn: &'a RoTxn, index: &'a Index) -> Result<HashSet<String>> {
|
||||
index.user_defined_faceted_fields(rtxn)
|
||||
}
|
||||
}
|
||||
|
||||
/// Truncates a string to the biggest valid LMDB key size.
|
||||
fn truncate_str(s: &str) -> &str {
|
||||
let index = s
|
||||
.char_indices()
|
||||
.map(|(idx, _)| idx)
|
||||
.chain(std::iter::once(s.len()))
|
||||
.take_while(|idx| idx <= &MAX_FACET_VALUE_LENGTH)
|
||||
.last();
|
||||
|
||||
&s[..index.unwrap_or(0)]
|
||||
}
|
||||
|
||||
impl DocidsExtractor for FacetedDocidsExtractor {
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::extract::faceted")]
|
||||
fn run_extraction<'pl, 'fid, 'indexer, 'index, 'extractor, DC: DocumentChanges<'pl>, MSP, SP>(
|
||||
grenad_parameters: GrenadParameters,
|
||||
document_changes: &DC,
|
||||
indexing_context: IndexingContext<'fid, 'indexer, 'index, MSP, SP>,
|
||||
extractor_allocs: &mut ThreadLocal<FullySend<Bump>>,
|
||||
finished_steps: u16,
|
||||
total_steps: u16,
|
||||
step_name: &'static str,
|
||||
) -> Result<Vec<BalancedCaches<'extractor>>>
|
||||
where
|
||||
MSP: Fn() -> bool + Sync,
|
||||
SP: Fn(Progress) + Sync,
|
||||
{
|
||||
let index = indexing_context.index;
|
||||
let rtxn = index.read_txn()?;
|
||||
let attributes_to_extract = Self::attributes_to_extract(&rtxn, index)?;
|
||||
let attributes_to_extract: Vec<_> =
|
||||
attributes_to_extract.iter().map(|s| s.as_ref()).collect();
|
||||
let datastore = ThreadLocal::new();
|
||||
|
||||
{
|
||||
let span =
|
||||
tracing::trace_span!(target: "indexing::documents::extract", "docids_extraction");
|
||||
let _entered = span.enter();
|
||||
|
||||
let extractor = FacetedExtractorData {
|
||||
attributes_to_extract: &attributes_to_extract,
|
||||
grenad_parameters,
|
||||
buckets: rayon::current_num_threads(),
|
||||
};
|
||||
extract(
|
||||
document_changes,
|
||||
&extractor,
|
||||
indexing_context,
|
||||
extractor_allocs,
|
||||
&datastore,
|
||||
finished_steps,
|
||||
total_steps,
|
||||
step_name,
|
||||
)?;
|
||||
}
|
||||
|
||||
Ok(datastore.into_iter().map(RefCell::into_inner).collect())
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
use serde_json::Value;
|
||||
|
||||
use crate::update::new::document::Document;
|
||||
use crate::update::new::extract::perm_json_p;
|
||||
use crate::{FieldId, GlobalFieldsIdsMap, InternalError, Result, UserError};
|
||||
|
||||
pub fn extract_document_facets<'doc>(
|
||||
attributes_to_extract: &[&str],
|
||||
document: impl Document<'doc>,
|
||||
field_id_map: &mut GlobalFieldsIdsMap,
|
||||
facet_fn: &mut impl FnMut(FieldId, &Value) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
for res in document.iter_top_level_fields() {
|
||||
let (field_name, value) = res?;
|
||||
|
||||
let mut tokenize_field = |name: &str, value: &Value| match field_id_map.id_or_insert(name) {
|
||||
Some(field_id) => facet_fn(field_id, value),
|
||||
None => Err(UserError::AttributeLimitReached.into()),
|
||||
};
|
||||
|
||||
// if the current field is searchable or contains a searchable attribute
|
||||
if perm_json_p::select_field(field_name, Some(attributes_to_extract), &[]) {
|
||||
// parse json.
|
||||
match serde_json::value::to_value(value).map_err(InternalError::SerdeJson)? {
|
||||
Value::Object(object) => perm_json_p::seek_leaf_values_in_object(
|
||||
&object,
|
||||
Some(attributes_to_extract),
|
||||
&[], // skip no attributes
|
||||
field_name,
|
||||
&mut tokenize_field,
|
||||
)?,
|
||||
Value::Array(array) => perm_json_p::seek_leaf_values_in_array(
|
||||
&array,
|
||||
Some(attributes_to_extract),
|
||||
&[], // skip no attributes
|
||||
field_name,
|
||||
&mut tokenize_field,
|
||||
)?,
|
||||
value => tokenize_field(field_name, &value)?,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
34
crates/milli/src/update/new/extract/faceted/mod.rs
Normal file
34
crates/milli/src/update/new/extract/faceted/mod.rs
Normal file
|
@ -0,0 +1,34 @@
|
|||
mod extract_facets;
|
||||
mod facet_document;
|
||||
|
||||
pub use extract_facets::FacetedDocidsExtractor;
|
||||
|
||||
#[repr(u8)]
|
||||
#[derive(Debug, Clone, Copy)]
|
||||
pub enum FacetKind {
|
||||
Number = 0,
|
||||
String = 1,
|
||||
Null = 2,
|
||||
Empty = 3,
|
||||
Exists,
|
||||
}
|
||||
|
||||
impl From<u8> for FacetKind {
|
||||
fn from(value: u8) -> Self {
|
||||
match value {
|
||||
0 => Self::Number,
|
||||
1 => Self::String,
|
||||
2 => Self::Null,
|
||||
3 => Self::Empty,
|
||||
4 => Self::Exists,
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl FacetKind {
|
||||
pub fn extract_from_key(key: &[u8]) -> (FacetKind, &[u8]) {
|
||||
debug_assert!(key.len() > 3);
|
||||
(FacetKind::from(key[0]), &key[1..])
|
||||
}
|
||||
}
|
146
crates/milli/src/update/new/extract/mod.rs
Normal file
146
crates/milli/src/update/new/extract/mod.rs
Normal file
|
@ -0,0 +1,146 @@
|
|||
mod cache;
|
||||
mod documents;
|
||||
mod faceted;
|
||||
mod searchable;
|
||||
mod vectors;
|
||||
|
||||
use bumpalo::Bump;
|
||||
pub use cache::{merge_caches, transpose_and_freeze_caches, BalancedCaches, DelAddRoaringBitmap};
|
||||
pub use documents::*;
|
||||
pub use faceted::*;
|
||||
pub use searchable::*;
|
||||
pub use vectors::EmbeddingExtractor;
|
||||
|
||||
use super::indexer::document_changes::{
|
||||
DocumentChanges, FullySend, IndexingContext, Progress, ThreadLocal,
|
||||
};
|
||||
use crate::update::{GrenadParameters, MergeDeladdCboRoaringBitmaps};
|
||||
use crate::Result;
|
||||
|
||||
pub trait DocidsExtractor {
|
||||
fn run_extraction<'pl, 'fid, 'indexer, 'index, 'extractor, DC: DocumentChanges<'pl>, MSP, SP>(
|
||||
grenad_parameters: GrenadParameters,
|
||||
document_changes: &DC,
|
||||
indexing_context: IndexingContext<'fid, 'indexer, 'index, MSP, SP>,
|
||||
extractor_allocs: &'extractor mut ThreadLocal<FullySend<Bump>>,
|
||||
finished_steps: u16,
|
||||
total_steps: u16,
|
||||
step_name: &'static str,
|
||||
) -> Result<Merger<File, MergeDeladdCboRoaringBitmaps>>
|
||||
where
|
||||
MSP: Fn() -> bool + Sync,
|
||||
SP: Fn(Progress) + Sync;
|
||||
}
|
||||
|
||||
/// TODO move in permissive json pointer
|
||||
pub mod perm_json_p {
|
||||
use serde_json::{Map, Value};
|
||||
|
||||
use crate::Result;
|
||||
const SPLIT_SYMBOL: char = '.';
|
||||
|
||||
/// Returns `true` if the `selector` match the `key`.
|
||||
///
|
||||
/// ```text
|
||||
/// Example:
|
||||
/// `animaux` match `animaux`
|
||||
/// `animaux.chien` match `animaux`
|
||||
/// `animaux.chien` match `animaux`
|
||||
/// `animaux.chien.nom` match `animaux`
|
||||
/// `animaux.chien.nom` match `animaux.chien`
|
||||
/// -----------------------------------------
|
||||
/// `animaux` doesn't match `animaux.chien`
|
||||
/// `animaux.` doesn't match `animaux`
|
||||
/// `animaux.ch` doesn't match `animaux.chien`
|
||||
/// `animau` doesn't match `animaux`
|
||||
/// ```
|
||||
pub fn contained_in(selector: &str, key: &str) -> bool {
|
||||
selector.starts_with(key)
|
||||
&& selector[key.len()..].chars().next().map(|c| c == SPLIT_SYMBOL).unwrap_or(true)
|
||||
}
|
||||
|
||||
pub fn seek_leaf_values_in_object(
|
||||
value: &Map<String, Value>,
|
||||
selectors: Option<&[&str]>,
|
||||
skip_selectors: &[&str],
|
||||
base_key: &str,
|
||||
seeker: &mut impl FnMut(&str, &Value) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
if value.is_empty() {
|
||||
seeker(base_key, &Value::Object(Map::with_capacity(0)))?;
|
||||
}
|
||||
|
||||
for (key, value) in value.iter() {
|
||||
let base_key = if base_key.is_empty() {
|
||||
key.to_string()
|
||||
} else {
|
||||
format!("{}{}{}", base_key, SPLIT_SYMBOL, key)
|
||||
};
|
||||
|
||||
// here if the user only specified `doggo` we need to iterate in all the fields of `doggo`
|
||||
// so we check the contained_in on both side
|
||||
let should_continue = select_field(&base_key, selectors, skip_selectors);
|
||||
if should_continue {
|
||||
match value {
|
||||
Value::Object(object) => seek_leaf_values_in_object(
|
||||
object,
|
||||
selectors,
|
||||
skip_selectors,
|
||||
&base_key,
|
||||
seeker,
|
||||
),
|
||||
Value::Array(array) => seek_leaf_values_in_array(
|
||||
array,
|
||||
selectors,
|
||||
skip_selectors,
|
||||
&base_key,
|
||||
seeker,
|
||||
),
|
||||
value => seeker(&base_key, value),
|
||||
}?;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn seek_leaf_values_in_array(
|
||||
values: &[Value],
|
||||
selectors: Option<&[&str]>,
|
||||
skip_selectors: &[&str],
|
||||
base_key: &str,
|
||||
seeker: &mut impl FnMut(&str, &Value) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
if values.is_empty() {
|
||||
seeker(base_key, &Value::Array(vec![]))?;
|
||||
}
|
||||
|
||||
for value in values {
|
||||
match value {
|
||||
Value::Object(object) => {
|
||||
seek_leaf_values_in_object(object, selectors, skip_selectors, base_key, seeker)
|
||||
}
|
||||
Value::Array(array) => {
|
||||
seek_leaf_values_in_array(array, selectors, skip_selectors, base_key, seeker)
|
||||
}
|
||||
value => seeker(base_key, value),
|
||||
}?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn select_field(
|
||||
field_name: &str,
|
||||
selectors: Option<&[&str]>,
|
||||
skip_selectors: &[&str],
|
||||
) -> bool {
|
||||
selectors.map_or(true, |selectors| {
|
||||
selectors.iter().any(|selector| {
|
||||
contained_in(selector, field_name) || contained_in(field_name, selector)
|
||||
})
|
||||
}) && !skip_selectors.iter().any(|skip_selector| {
|
||||
contained_in(skip_selector, field_name) || contained_in(field_name, skip_selector)
|
||||
})
|
||||
}
|
||||
}
|
|
@ -0,0 +1,400 @@
|
|||
use std::cell::RefCell;
|
||||
use std::collections::HashMap;
|
||||
use std::mem::size_of;
|
||||
use std::ops::DerefMut as _;
|
||||
|
||||
use bumpalo::collections::vec::Vec as BumpVec;
|
||||
use bumpalo::Bump;
|
||||
use heed::RoTxn;
|
||||
|
||||
use super::tokenize_document::{tokenizer_builder, DocumentTokenizer};
|
||||
use crate::update::new::extract::cache::BalancedCaches;
|
||||
use crate::update::new::extract::perm_json_p::contained_in;
|
||||
use crate::update::new::indexer::document_changes::{
|
||||
for_each_document_change, DocumentChangeContext, DocumentChanges, Extractor, FullySend,
|
||||
IndexingContext, MostlySend, RefCellExt, ThreadLocal,
|
||||
};
|
||||
use crate::update::new::DocumentChange;
|
||||
use crate::update::GrenadParameters;
|
||||
use crate::{bucketed_position, DocumentId, FieldId, Index, Result, MAX_POSITION_PER_ATTRIBUTE};
|
||||
|
||||
const MAX_COUNTED_WORDS: usize = 30;
|
||||
|
||||
pub struct WordDocidsBalancedCaches<'extractor> {
|
||||
word_fid_docids: BalancedCaches<'extractor>,
|
||||
word_docids: BalancedCaches<'extractor>,
|
||||
exact_word_docids: BalancedCaches<'extractor>,
|
||||
word_position_docids: BalancedCaches<'extractor>,
|
||||
fid_word_count_docids: BalancedCaches<'extractor>,
|
||||
fid_word_count: HashMap<FieldId, (usize, usize)>,
|
||||
current_docid: Option<DocumentId>,
|
||||
}
|
||||
|
||||
unsafe impl<'extractor> MostlySend for WordDocidsBalancedCaches<'extractor> {}
|
||||
|
||||
impl<'extractor> WordDocidsBalancedCaches<'extractor> {
|
||||
/// TODO Make sure to give the same max_memory to all of them, without splitting it
|
||||
pub fn new_in(buckets: usize, max_memory: Option<usize>, alloc: &'extractor Bump) -> Self {
|
||||
Self {
|
||||
word_fid_docids: BalancedCaches::new_in(buckets, max_memory, alloc),
|
||||
word_docids: BalancedCaches::new_in(buckets, max_memory, alloc),
|
||||
exact_word_docids: BalancedCaches::new_in(buckets, max_memory, alloc),
|
||||
word_position_docids: BalancedCaches::new_in(buckets, max_memory, alloc),
|
||||
fid_word_count_docids: BalancedCaches::new_in(buckets, max_memory, alloc),
|
||||
fid_word_count: HashMap::new(),
|
||||
current_docid: None,
|
||||
}
|
||||
}
|
||||
|
||||
fn insert_add_u32(
|
||||
&mut self,
|
||||
field_id: FieldId,
|
||||
position: u16,
|
||||
word: &str,
|
||||
exact: bool,
|
||||
docid: u32,
|
||||
bump: &Bump,
|
||||
) -> Result<()> {
|
||||
let word_bytes = word.as_bytes();
|
||||
if exact {
|
||||
self.exact_word_docids.insert_add_u32(word_bytes, docid)?;
|
||||
} else {
|
||||
self.word_docids.insert_add_u32(word_bytes, docid)?;
|
||||
}
|
||||
|
||||
let buffer_size = word_bytes.len() + 1 + size_of::<FieldId>();
|
||||
let mut buffer = BumpVec::with_capacity_in(buffer_size, bump);
|
||||
|
||||
buffer.clear();
|
||||
buffer.extend_from_slice(word_bytes);
|
||||
buffer.push(0);
|
||||
buffer.extend_from_slice(&field_id.to_be_bytes());
|
||||
self.word_fid_docids.insert_add_u32(&buffer, docid)?;
|
||||
|
||||
let position = bucketed_position(position);
|
||||
buffer.clear();
|
||||
buffer.extend_from_slice(word_bytes);
|
||||
buffer.push(0);
|
||||
buffer.extend_from_slice(&position.to_be_bytes());
|
||||
self.word_position_docids.insert_add_u32(&buffer, docid)?;
|
||||
|
||||
if self.current_docid.map_or(false, |id| docid != id) {
|
||||
self.flush_fid_word_count(&mut buffer)?;
|
||||
}
|
||||
|
||||
self.fid_word_count
|
||||
.entry(field_id)
|
||||
.and_modify(|(_current_count, new_count)| *new_count += 1)
|
||||
.or_insert((0, 1));
|
||||
self.current_docid = Some(docid);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn insert_del_u32(
|
||||
&mut self,
|
||||
field_id: FieldId,
|
||||
position: u16,
|
||||
word: &str,
|
||||
exact: bool,
|
||||
docid: u32,
|
||||
bump: &Bump,
|
||||
) -> Result<()> {
|
||||
let word_bytes = word.as_bytes();
|
||||
if exact {
|
||||
self.exact_word_docids.insert_del_u32(word_bytes, docid)?;
|
||||
} else {
|
||||
self.word_docids.insert_del_u32(word_bytes, docid)?;
|
||||
}
|
||||
|
||||
let buffer_size = word_bytes.len() + 1 + size_of::<FieldId>();
|
||||
let mut buffer = BumpVec::with_capacity_in(buffer_size, bump);
|
||||
|
||||
buffer.clear();
|
||||
buffer.extend_from_slice(word_bytes);
|
||||
buffer.push(0);
|
||||
buffer.extend_from_slice(&field_id.to_be_bytes());
|
||||
self.word_fid_docids.insert_del_u32(&buffer, docid)?;
|
||||
|
||||
let position = bucketed_position(position);
|
||||
buffer.clear();
|
||||
buffer.extend_from_slice(word_bytes);
|
||||
buffer.push(0);
|
||||
buffer.extend_from_slice(&position.to_be_bytes());
|
||||
self.word_position_docids.insert_del_u32(&buffer, docid)?;
|
||||
|
||||
if self.current_docid.map_or(false, |id| docid != id) {
|
||||
self.flush_fid_word_count(&mut buffer)?;
|
||||
}
|
||||
|
||||
self.fid_word_count
|
||||
.entry(field_id)
|
||||
.and_modify(|(current_count, _new_count)| *current_count += 1)
|
||||
.or_insert((1, 0));
|
||||
|
||||
self.current_docid = Some(docid);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flush_fid_word_count(&mut self, buffer: &mut BumpVec<u8>) -> Result<()> {
|
||||
for (fid, (current_count, new_count)) in self.fid_word_count.drain() {
|
||||
if current_count != new_count {
|
||||
if current_count <= MAX_COUNTED_WORDS {
|
||||
buffer.clear();
|
||||
buffer.extend_from_slice(&fid.to_be_bytes());
|
||||
buffer.push(current_count as u8);
|
||||
self.fid_word_count_docids
|
||||
.insert_del_u32(buffer, self.current_docid.unwrap())?;
|
||||
}
|
||||
if new_count <= MAX_COUNTED_WORDS {
|
||||
buffer.clear();
|
||||
buffer.extend_from_slice(&fid.to_be_bytes());
|
||||
buffer.push(new_count as u8);
|
||||
self.fid_word_count_docids
|
||||
.insert_add_u32(buffer, self.current_docid.unwrap())?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
pub struct WordDocidsCaches<'extractor> {
|
||||
pub word_docids: Vec<BalancedCaches<'extractor>>,
|
||||
pub word_fid_docids: Vec<BalancedCaches<'extractor>>,
|
||||
pub exact_word_docids: Vec<BalancedCaches<'extractor>>,
|
||||
pub word_position_docids: Vec<BalancedCaches<'extractor>>,
|
||||
pub fid_word_count_docids: Vec<BalancedCaches<'extractor>>,
|
||||
}
|
||||
|
||||
impl<'extractor> WordDocidsCaches<'extractor> {
|
||||
fn new() -> Self {
|
||||
Self {
|
||||
word_docids: Vec::new(),
|
||||
word_fid_docids: Vec::new(),
|
||||
exact_word_docids: Vec::new(),
|
||||
word_position_docids: Vec::new(),
|
||||
fid_word_count_docids: Vec::new(),
|
||||
}
|
||||
}
|
||||
|
||||
fn push(&mut self, other: WordDocidsBalancedCaches<'extractor>) -> Result<()> {
|
||||
let WordDocidsBalancedCaches {
|
||||
word_docids,
|
||||
word_fid_docids,
|
||||
exact_word_docids,
|
||||
word_position_docids,
|
||||
fid_word_count_docids,
|
||||
fid_word_count: _,
|
||||
current_docid: _,
|
||||
} = other;
|
||||
|
||||
self.word_docids.push(word_docids);
|
||||
self.word_fid_docids.push(word_fid_docids);
|
||||
self.exact_word_docids.push(exact_word_docids);
|
||||
self.word_position_docids.push(word_position_docids);
|
||||
self.fid_word_count_docids.push(fid_word_count_docids);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
pub struct WordDocidsExtractorData<'a> {
|
||||
tokenizer: &'a DocumentTokenizer<'a>,
|
||||
grenad_parameters: GrenadParameters,
|
||||
buckets: usize,
|
||||
}
|
||||
|
||||
impl<'a, 'extractor> Extractor<'extractor> for WordDocidsExtractorData<'a> {
|
||||
type Data = RefCell<Option<WordDocidsBalancedCaches<'extractor>>>;
|
||||
|
||||
fn init_data(&self, extractor_alloc: &'extractor Bump) -> Result<Self::Data> {
|
||||
Ok(RefCell::new(Some(WordDocidsBalancedCaches::new_in(
|
||||
self.buckets,
|
||||
self.grenad_parameters.max_memory,
|
||||
extractor_alloc,
|
||||
))))
|
||||
}
|
||||
|
||||
fn process(
|
||||
&self,
|
||||
change: DocumentChange,
|
||||
context: &DocumentChangeContext<Self::Data>,
|
||||
) -> Result<()> {
|
||||
WordDocidsExtractors::extract_document_change(context, self.tokenizer, change)
|
||||
}
|
||||
}
|
||||
|
||||
pub struct WordDocidsExtractors;
|
||||
|
||||
impl WordDocidsExtractors {
|
||||
pub fn run_extraction<'pl, 'fid, 'indexer, 'index, 'extractor, DC: DocumentChanges<'pl>>(
|
||||
grenad_parameters: GrenadParameters,
|
||||
document_changes: &DC,
|
||||
indexing_context: IndexingContext<'fid, 'indexer, 'index>,
|
||||
extractor_allocs: &'extractor mut ThreadLocal<FullySend<Bump>>,
|
||||
) -> Result<WordDocidsCaches<'extractor>> {
|
||||
let index = indexing_context.index;
|
||||
let rtxn = index.read_txn()?;
|
||||
|
||||
let stop_words = index.stop_words(&rtxn)?;
|
||||
let allowed_separators = index.allowed_separators(&rtxn)?;
|
||||
let allowed_separators: Option<Vec<_>> =
|
||||
allowed_separators.as_ref().map(|s| s.iter().map(String::as_str).collect());
|
||||
let dictionary = index.dictionary(&rtxn)?;
|
||||
let dictionary: Option<Vec<_>> =
|
||||
dictionary.as_ref().map(|s| s.iter().map(String::as_str).collect());
|
||||
let builder = tokenizer_builder(
|
||||
stop_words.as_ref(),
|
||||
allowed_separators.as_deref(),
|
||||
dictionary.as_deref(),
|
||||
);
|
||||
let tokenizer = builder.into_tokenizer();
|
||||
|
||||
let attributes_to_extract = Self::attributes_to_extract(&rtxn, index)?;
|
||||
let attributes_to_skip = Self::attributes_to_skip(&rtxn, index)?;
|
||||
let localized_attributes_rules =
|
||||
index.localized_attributes_rules(&rtxn)?.unwrap_or_default();
|
||||
|
||||
let document_tokenizer = DocumentTokenizer {
|
||||
tokenizer: &tokenizer,
|
||||
attribute_to_extract: attributes_to_extract.as_deref(),
|
||||
attribute_to_skip: attributes_to_skip.as_slice(),
|
||||
localized_attributes_rules: &localized_attributes_rules,
|
||||
max_positions_per_attributes: MAX_POSITION_PER_ATTRIBUTE,
|
||||
};
|
||||
|
||||
let datastore = ThreadLocal::new();
|
||||
|
||||
{
|
||||
let span =
|
||||
tracing::trace_span!(target: "indexing::documents::extract", "docids_extraction");
|
||||
let _entered = span.enter();
|
||||
|
||||
let extractor = WordDocidsExtractorData {
|
||||
tokenizer: &document_tokenizer,
|
||||
grenad_parameters,
|
||||
buckets: rayon::current_num_threads(),
|
||||
};
|
||||
|
||||
for_each_document_change(
|
||||
document_changes,
|
||||
&extractor,
|
||||
indexing_context,
|
||||
extractor_allocs,
|
||||
&datastore,
|
||||
)?;
|
||||
}
|
||||
|
||||
let mut merger = WordDocidsCaches::new();
|
||||
for cache in datastore.into_iter().flat_map(RefCell::into_inner) {
|
||||
merger.push(cache)?;
|
||||
}
|
||||
|
||||
Ok(merger)
|
||||
}
|
||||
|
||||
fn extract_document_change(
|
||||
context: &DocumentChangeContext<RefCell<Option<WordDocidsBalancedCaches>>>,
|
||||
document_tokenizer: &DocumentTokenizer,
|
||||
document_change: DocumentChange,
|
||||
) -> Result<()> {
|
||||
let index = &context.index;
|
||||
let rtxn = &context.txn;
|
||||
let mut cached_sorter_ref = context.data.borrow_mut_or_yield();
|
||||
let cached_sorter = cached_sorter_ref.as_mut().unwrap();
|
||||
let mut new_fields_ids_map = context.new_fields_ids_map.borrow_mut_or_yield();
|
||||
let new_fields_ids_map = new_fields_ids_map.deref_mut();
|
||||
let doc_alloc = &context.doc_alloc;
|
||||
|
||||
let exact_attributes = index.exact_attributes(rtxn)?;
|
||||
let is_exact_attribute =
|
||||
|fname: &str| exact_attributes.iter().any(|attr| contained_in(fname, attr));
|
||||
match document_change {
|
||||
DocumentChange::Deletion(inner) => {
|
||||
let mut token_fn = |fname: &str, fid, pos, word: &str| {
|
||||
cached_sorter.insert_del_u32(
|
||||
fid,
|
||||
pos,
|
||||
word,
|
||||
is_exact_attribute(fname),
|
||||
inner.docid(),
|
||||
doc_alloc,
|
||||
)
|
||||
};
|
||||
document_tokenizer.tokenize_document(
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map,
|
||||
&mut token_fn,
|
||||
)?;
|
||||
}
|
||||
DocumentChange::Update(inner) => {
|
||||
let mut token_fn = |fname: &str, fid, pos, word: &str| {
|
||||
cached_sorter.insert_del_u32(
|
||||
fid,
|
||||
pos,
|
||||
word,
|
||||
is_exact_attribute(fname),
|
||||
inner.docid(),
|
||||
doc_alloc,
|
||||
)
|
||||
};
|
||||
document_tokenizer.tokenize_document(
|
||||
inner.current(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map,
|
||||
&mut token_fn,
|
||||
)?;
|
||||
|
||||
let mut token_fn = |fname: &str, fid, pos, word: &str| {
|
||||
cached_sorter.insert_add_u32(
|
||||
fid,
|
||||
pos,
|
||||
word,
|
||||
is_exact_attribute(fname),
|
||||
inner.docid(),
|
||||
doc_alloc,
|
||||
)
|
||||
};
|
||||
document_tokenizer.tokenize_document(
|
||||
inner.new(rtxn, index, context.db_fields_ids_map)?,
|
||||
new_fields_ids_map,
|
||||
&mut token_fn,
|
||||
)?;
|
||||
}
|
||||
DocumentChange::Insertion(inner) => {
|
||||
let mut token_fn = |fname: &str, fid, pos, word: &str| {
|
||||
cached_sorter.insert_add_u32(
|
||||
fid,
|
||||
pos,
|
||||
word,
|
||||
is_exact_attribute(fname),
|
||||
inner.docid(),
|
||||
doc_alloc,
|
||||
)
|
||||
};
|
||||
document_tokenizer.tokenize_document(
|
||||
inner.new(),
|
||||
new_fields_ids_map,
|
||||
&mut token_fn,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
|
||||
let buffer_size = size_of::<FieldId>();
|
||||
let mut buffer = BumpVec::with_capacity_in(buffer_size, &context.doc_alloc);
|
||||
cached_sorter.flush_fid_word_count(&mut buffer)
|
||||
}
|
||||
|
||||
fn attributes_to_extract<'a>(
|
||||
rtxn: &'a RoTxn,
|
||||
index: &'a Index,
|
||||
) -> Result<Option<Vec<&'a str>>> {
|
||||
index.user_defined_searchable_fields(rtxn).map_err(Into::into)
|
||||
}
|
||||
|
||||
fn attributes_to_skip<'a>(_rtxn: &'a RoTxn, _index: &'a Index) -> Result<Vec<&'a str>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
}
|
|
@ -0,0 +1,178 @@
|
|||
use std::cell::RefCell;
|
||||
use std::collections::VecDeque;
|
||||
use std::rc::Rc;
|
||||
|
||||
use heed::RoTxn;
|
||||
|
||||
use super::tokenize_document::DocumentTokenizer;
|
||||
use super::SearchableExtractor;
|
||||
use crate::proximity::{index_proximity, MAX_DISTANCE};
|
||||
use crate::update::new::document::Document;
|
||||
use crate::update::new::extract::cache::BalancedCaches;
|
||||
use crate::update::new::indexer::document_changes::{DocumentChangeContext, RefCellExt};
|
||||
use crate::update::new::DocumentChange;
|
||||
use crate::{FieldId, GlobalFieldsIdsMap, Index, Result};
|
||||
|
||||
pub struct WordPairProximityDocidsExtractor;
|
||||
|
||||
impl SearchableExtractor for WordPairProximityDocidsExtractor {
|
||||
fn attributes_to_extract<'a>(
|
||||
rtxn: &'a RoTxn,
|
||||
index: &'a Index,
|
||||
) -> Result<Option<Vec<&'a str>>> {
|
||||
index.user_defined_searchable_fields(rtxn).map_err(Into::into)
|
||||
}
|
||||
|
||||
fn attributes_to_skip<'a>(_rtxn: &'a RoTxn, _index: &'a Index) -> Result<Vec<&'a str>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
// This method is reimplemented to count the number of words in the document in each field
|
||||
// and to store the docids of the documents that have a number of words in a given field
|
||||
// equal to or under than MAX_COUNTED_WORDS.
|
||||
fn extract_document_change(
|
||||
context: &DocumentChangeContext<RefCell<BalancedCaches>>,
|
||||
document_tokenizer: &DocumentTokenizer,
|
||||
document_change: DocumentChange,
|
||||
) -> Result<()> {
|
||||
let doc_alloc = &context.doc_alloc;
|
||||
|
||||
let index = context.index;
|
||||
let rtxn = &context.txn;
|
||||
|
||||
let mut key_buffer = bumpalo::collections::Vec::new_in(doc_alloc);
|
||||
let mut del_word_pair_proximity = bumpalo::collections::Vec::new_in(doc_alloc);
|
||||
let mut add_word_pair_proximity = bumpalo::collections::Vec::new_in(doc_alloc);
|
||||
|
||||
let mut new_fields_ids_map = context.new_fields_ids_map.borrow_mut_or_yield();
|
||||
let new_fields_ids_map = &mut *new_fields_ids_map;
|
||||
|
||||
let mut cached_sorter = context.data.borrow_mut_or_yield();
|
||||
let cached_sorter = &mut *cached_sorter;
|
||||
|
||||
// is a vecdequeue, and will be smol, so can stay on the heap for now
|
||||
let mut word_positions: VecDeque<(Rc<str>, u16)> =
|
||||
VecDeque::with_capacity(MAX_DISTANCE as usize);
|
||||
|
||||
let docid = document_change.docid();
|
||||
match document_change {
|
||||
DocumentChange::Deletion(inner) => {
|
||||
let document = inner.current(rtxn, index, context.db_fields_ids_map)?;
|
||||
process_document_tokens(
|
||||
document,
|
||||
document_tokenizer,
|
||||
new_fields_ids_map,
|
||||
&mut word_positions,
|
||||
&mut |(w1, w2), prox| {
|
||||
del_word_pair_proximity.push(((w1, w2), prox));
|
||||
},
|
||||
)?;
|
||||
}
|
||||
DocumentChange::Update(inner) => {
|
||||
let document = inner.current(rtxn, index, context.db_fields_ids_map)?;
|
||||
process_document_tokens(
|
||||
document,
|
||||
document_tokenizer,
|
||||
new_fields_ids_map,
|
||||
&mut word_positions,
|
||||
&mut |(w1, w2), prox| {
|
||||
del_word_pair_proximity.push(((w1, w2), prox));
|
||||
},
|
||||
)?;
|
||||
let document = inner.merged(rtxn, index, context.db_fields_ids_map)?;
|
||||
process_document_tokens(
|
||||
document,
|
||||
document_tokenizer,
|
||||
new_fields_ids_map,
|
||||
&mut word_positions,
|
||||
&mut |(w1, w2), prox| {
|
||||
add_word_pair_proximity.push(((w1, w2), prox));
|
||||
},
|
||||
)?;
|
||||
}
|
||||
DocumentChange::Insertion(inner) => {
|
||||
let document = inner.inserted();
|
||||
process_document_tokens(
|
||||
document,
|
||||
document_tokenizer,
|
||||
new_fields_ids_map,
|
||||
&mut word_positions,
|
||||
&mut |(w1, w2), prox| {
|
||||
add_word_pair_proximity.push(((w1, w2), prox));
|
||||
},
|
||||
)?;
|
||||
}
|
||||
}
|
||||
|
||||
del_word_pair_proximity.sort_unstable();
|
||||
del_word_pair_proximity.dedup_by(|(k1, _), (k2, _)| k1 == k2);
|
||||
for ((w1, w2), prox) in del_word_pair_proximity.iter() {
|
||||
let key = build_key(*prox, w1, w2, &mut key_buffer);
|
||||
cached_sorter.insert_del_u32(key, docid)?;
|
||||
}
|
||||
|
||||
add_word_pair_proximity.sort_unstable();
|
||||
add_word_pair_proximity.dedup_by(|(k1, _), (k2, _)| k1 == k2);
|
||||
for ((w1, w2), prox) in add_word_pair_proximity.iter() {
|
||||
let key = build_key(*prox, w1, w2, &mut key_buffer);
|
||||
cached_sorter.insert_add_u32(key, docid)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn build_key<'a>(
|
||||
prox: u8,
|
||||
w1: &str,
|
||||
w2: &str,
|
||||
key_buffer: &'a mut bumpalo::collections::Vec<u8>,
|
||||
) -> &'a [u8] {
|
||||
key_buffer.clear();
|
||||
key_buffer.push(prox);
|
||||
key_buffer.extend_from_slice(w1.as_bytes());
|
||||
key_buffer.push(0);
|
||||
key_buffer.extend_from_slice(w2.as_bytes());
|
||||
key_buffer.as_slice()
|
||||
}
|
||||
|
||||
fn word_positions_into_word_pair_proximity(
|
||||
word_positions: &mut VecDeque<(Rc<str>, u16)>,
|
||||
word_pair_proximity: &mut impl FnMut((Rc<str>, Rc<str>), u8),
|
||||
) {
|
||||
let (head_word, head_position) = word_positions.pop_front().unwrap();
|
||||
for (word, position) in word_positions.iter() {
|
||||
let prox = index_proximity(head_position as u32, *position as u32) as u8;
|
||||
if prox > 0 && prox < MAX_DISTANCE as u8 {
|
||||
word_pair_proximity((head_word.clone(), word.clone()), prox);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn process_document_tokens<'doc>(
|
||||
document: impl Document<'doc>,
|
||||
document_tokenizer: &DocumentTokenizer,
|
||||
fields_ids_map: &mut GlobalFieldsIdsMap,
|
||||
word_positions: &mut VecDeque<(Rc<str>, u16)>,
|
||||
word_pair_proximity: &mut impl FnMut((Rc<str>, Rc<str>), u8),
|
||||
) -> Result<()> {
|
||||
let mut token_fn = |_fname: &str, _fid: FieldId, pos: u16, word: &str| {
|
||||
// drain the proximity window until the head word is considered close to the word we are inserting.
|
||||
while word_positions
|
||||
.front()
|
||||
.map_or(false, |(_w, p)| index_proximity(*p as u32, pos as u32) >= MAX_DISTANCE)
|
||||
{
|
||||
word_positions_into_word_pair_proximity(word_positions, word_pair_proximity);
|
||||
}
|
||||
|
||||
// insert the new word.
|
||||
word_positions.push_back((Rc::from(word), pos));
|
||||
Ok(())
|
||||
};
|
||||
document_tokenizer.tokenize_document(document, fields_ids_map, &mut token_fn)?;
|
||||
|
||||
while !word_positions.is_empty() {
|
||||
word_positions_into_word_pair_proximity(word_positions, word_pair_proximity);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
163
crates/milli/src/update/new/extract/searchable/mod.rs
Normal file
163
crates/milli/src/update/new/extract/searchable/mod.rs
Normal file
|
@ -0,0 +1,163 @@
|
|||
mod extract_word_docids;
|
||||
mod extract_word_pair_proximity_docids;
|
||||
mod tokenize_document;
|
||||
|
||||
use std::cell::RefCell;
|
||||
use std::marker::PhantomData;
|
||||
|
||||
use bumpalo::Bump;
|
||||
pub use extract_word_docids::{WordDocidsCaches, WordDocidsExtractors};
|
||||
pub use extract_word_pair_proximity_docids::WordPairProximityDocidsExtractor;
|
||||
use heed::RoTxn;
|
||||
use tokenize_document::{tokenizer_builder, DocumentTokenizer};
|
||||
|
||||
use super::cache::BalancedCaches;
|
||||
use super::DocidsExtractor;
|
||||
use crate::update::new::indexer::document_changes::{
|
||||
extract, DocumentChangeContext, DocumentChanges, Extractor, FullySend, IndexingContext,
|
||||
Progress, ThreadLocal,
|
||||
};
|
||||
use crate::update::new::DocumentChange;
|
||||
use crate::update::GrenadParameters;
|
||||
use crate::{Index, Result, MAX_POSITION_PER_ATTRIBUTE};
|
||||
|
||||
pub struct SearchableExtractorData<'a, EX: SearchableExtractor> {
|
||||
tokenizer: &'a DocumentTokenizer<'a>,
|
||||
grenad_parameters: GrenadParameters,
|
||||
buckets: usize,
|
||||
_ex: PhantomData<EX>,
|
||||
}
|
||||
|
||||
impl<'a, 'extractor, EX: SearchableExtractor + Sync> Extractor<'extractor>
|
||||
for SearchableExtractorData<'a, EX>
|
||||
{
|
||||
type Data = RefCell<BalancedCaches<'extractor>>;
|
||||
|
||||
fn init_data(&self, extractor_alloc: &'extractor Bump) -> Result<Self::Data> {
|
||||
Ok(RefCell::new(BalancedCaches::new_in(
|
||||
self.buckets,
|
||||
self.grenad_parameters.max_memory,
|
||||
extractor_alloc,
|
||||
)))
|
||||
}
|
||||
|
||||
fn process<'doc>(
|
||||
&self,
|
||||
changes: impl Iterator<Item = Result<DocumentChange<'doc>>>,
|
||||
context: &DocumentChangeContext<Self::Data>,
|
||||
) -> Result<()> {
|
||||
for change in changes {
|
||||
let change = change?;
|
||||
EX::extract_document_change(context, self.tokenizer, change)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
pub trait SearchableExtractor: Sized + Sync {
|
||||
fn run_extraction<'pl, 'fid, 'indexer, 'index, 'extractor, DC: DocumentChanges<'pl>, MSP, SP>(
|
||||
grenad_parameters: GrenadParameters,
|
||||
document_changes: &DC,
|
||||
indexing_context: IndexingContext<'fid, 'indexer, 'index, MSP, SP>,
|
||||
extractor_allocs: &'extractor mut ThreadLocal<FullySend<Bump>>,
|
||||
finished_steps: u16,
|
||||
total_steps: u16,
|
||||
step_name: &'static str,
|
||||
) -> Result<Vec<BalancedCaches<'extractor>>>
|
||||
where
|
||||
MSP: Fn() -> bool + Sync,
|
||||
SP: Fn(Progress) + Sync,
|
||||
{
|
||||
let rtxn = indexing_context.index.read_txn()?;
|
||||
let stop_words = indexing_context.index.stop_words(&rtxn)?;
|
||||
let allowed_separators = indexing_context.index.allowed_separators(&rtxn)?;
|
||||
let allowed_separators: Option<Vec<_>> =
|
||||
allowed_separators.as_ref().map(|s| s.iter().map(String::as_str).collect());
|
||||
let dictionary = indexing_context.index.dictionary(&rtxn)?;
|
||||
let dictionary: Option<Vec<_>> =
|
||||
dictionary.as_ref().map(|s| s.iter().map(String::as_str).collect());
|
||||
let builder = tokenizer_builder(
|
||||
stop_words.as_ref(),
|
||||
allowed_separators.as_deref(),
|
||||
dictionary.as_deref(),
|
||||
);
|
||||
let tokenizer = builder.into_tokenizer();
|
||||
|
||||
let attributes_to_extract = Self::attributes_to_extract(&rtxn, indexing_context.index)?;
|
||||
let attributes_to_skip = Self::attributes_to_skip(&rtxn, indexing_context.index)?;
|
||||
let localized_attributes_rules =
|
||||
indexing_context.index.localized_attributes_rules(&rtxn)?.unwrap_or_default();
|
||||
|
||||
let document_tokenizer = DocumentTokenizer {
|
||||
tokenizer: &tokenizer,
|
||||
attribute_to_extract: attributes_to_extract.as_deref(),
|
||||
attribute_to_skip: attributes_to_skip.as_slice(),
|
||||
localized_attributes_rules: &localized_attributes_rules,
|
||||
max_positions_per_attributes: MAX_POSITION_PER_ATTRIBUTE,
|
||||
};
|
||||
|
||||
let extractor_data: SearchableExtractorData<Self> = SearchableExtractorData {
|
||||
tokenizer: &document_tokenizer,
|
||||
grenad_parameters,
|
||||
buckets: rayon::current_num_threads(),
|
||||
_ex: PhantomData,
|
||||
};
|
||||
|
||||
let datastore = ThreadLocal::new();
|
||||
|
||||
{
|
||||
let span =
|
||||
tracing::trace_span!(target: "indexing::documents::extract", "docids_extraction");
|
||||
let _entered = span.enter();
|
||||
extract(
|
||||
document_changes,
|
||||
&extractor_data,
|
||||
indexing_context,
|
||||
extractor_allocs,
|
||||
&datastore,
|
||||
finished_steps,
|
||||
total_steps,
|
||||
step_name,
|
||||
)?;
|
||||
}
|
||||
|
||||
Ok(datastore.into_iter().map(RefCell::into_inner).collect())
|
||||
}
|
||||
|
||||
fn extract_document_change(
|
||||
context: &DocumentChangeContext<RefCell<BalancedCaches>>,
|
||||
document_tokenizer: &DocumentTokenizer,
|
||||
document_change: DocumentChange,
|
||||
) -> Result<()>;
|
||||
|
||||
fn attributes_to_extract<'a>(rtxn: &'a RoTxn, index: &'a Index)
|
||||
-> Result<Option<Vec<&'a str>>>;
|
||||
|
||||
fn attributes_to_skip<'a>(rtxn: &'a RoTxn, index: &'a Index) -> Result<Vec<&'a str>>;
|
||||
}
|
||||
|
||||
impl<T: SearchableExtractor> DocidsExtractor for T {
|
||||
fn run_extraction<'pl, 'fid, 'indexer, 'index, 'extractor, DC: DocumentChanges<'pl>, MSP, SP>(
|
||||
grenad_parameters: GrenadParameters,
|
||||
document_changes: &DC,
|
||||
indexing_context: IndexingContext<'fid, 'indexer, 'index, MSP, SP>,
|
||||
extractor_allocs: &'extractor mut ThreadLocal<FullySend<Bump>>,
|
||||
finished_steps: u16,
|
||||
total_steps: u16,
|
||||
step_name: &'static str,
|
||||
) -> Result<Vec<BalancedCaches<'extractor>>>
|
||||
where
|
||||
MSP: Fn() -> bool + Sync,
|
||||
SP: Fn(Progress) + Sync,
|
||||
{
|
||||
Self::run_extraction(
|
||||
grenad_parameters,
|
||||
document_changes,
|
||||
indexing_context,
|
||||
extractor_allocs,
|
||||
finished_steps,
|
||||
total_steps,
|
||||
step_name,
|
||||
)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,275 @@
|
|||
use std::collections::HashMap;
|
||||
|
||||
use charabia::{SeparatorKind, Token, TokenKind, Tokenizer, TokenizerBuilder};
|
||||
use serde_json::Value;
|
||||
|
||||
use crate::proximity::MAX_DISTANCE;
|
||||
use crate::update::new::document::Document;
|
||||
use crate::update::new::extract::perm_json_p::{
|
||||
seek_leaf_values_in_array, seek_leaf_values_in_object, select_field,
|
||||
};
|
||||
use crate::{
|
||||
FieldId, GlobalFieldsIdsMap, InternalError, LocalizedAttributesRule, Result, UserError,
|
||||
MAX_WORD_LENGTH,
|
||||
};
|
||||
|
||||
pub struct DocumentTokenizer<'a> {
|
||||
pub tokenizer: &'a Tokenizer<'a>,
|
||||
pub attribute_to_extract: Option<&'a [&'a str]>,
|
||||
pub attribute_to_skip: &'a [&'a str],
|
||||
pub localized_attributes_rules: &'a [LocalizedAttributesRule],
|
||||
pub max_positions_per_attributes: u32,
|
||||
}
|
||||
|
||||
impl<'a> DocumentTokenizer<'a> {
|
||||
pub fn tokenize_document<'doc>(
|
||||
&self,
|
||||
document: impl Document<'doc>,
|
||||
field_id_map: &mut GlobalFieldsIdsMap,
|
||||
token_fn: &mut impl FnMut(&str, FieldId, u16, &str) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
let mut field_position = HashMap::new();
|
||||
|
||||
for entry in document.iter_top_level_fields() {
|
||||
let (field_name, value) = entry?;
|
||||
|
||||
let mut tokenize_field = |name: &str, value: &Value| {
|
||||
let Some(field_id) = field_id_map.id_or_insert(name) else {
|
||||
return Err(UserError::AttributeLimitReached.into());
|
||||
};
|
||||
|
||||
let position = field_position
|
||||
.entry(field_id)
|
||||
.and_modify(|counter| *counter += MAX_DISTANCE)
|
||||
.or_insert(0);
|
||||
if *position >= self.max_positions_per_attributes {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
match value {
|
||||
Value::Number(n) => {
|
||||
let token = n.to_string();
|
||||
if let Ok(position) = (*position).try_into() {
|
||||
token_fn(name, field_id, position, token.as_str())?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
Value::String(text) => {
|
||||
// create an iterator of token with their positions.
|
||||
let locales = self
|
||||
.localized_attributes_rules
|
||||
.iter()
|
||||
.find(|rule| rule.match_str(field_name))
|
||||
.map(|rule| rule.locales());
|
||||
let tokens = process_tokens(
|
||||
*position,
|
||||
self.tokenizer.tokenize_with_allow_list(text.as_str(), locales),
|
||||
)
|
||||
.take_while(|(p, _)| *p < self.max_positions_per_attributes);
|
||||
|
||||
for (index, token) in tokens {
|
||||
// keep a word only if it is not empty and fit in a LMDB key.
|
||||
let token = token.lemma().trim();
|
||||
if !token.is_empty() && token.len() <= MAX_WORD_LENGTH {
|
||||
*position = index;
|
||||
if let Ok(position) = (*position).try_into() {
|
||||
token_fn(name, field_id, position, token)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
_ => Ok(()),
|
||||
}
|
||||
};
|
||||
|
||||
// if the current field is searchable or contains a searchable attribute
|
||||
if select_field(field_name, self.attribute_to_extract, self.attribute_to_skip) {
|
||||
// parse json.
|
||||
match serde_json::to_value(value).map_err(InternalError::SerdeJson)? {
|
||||
Value::Object(object) => seek_leaf_values_in_object(
|
||||
&object,
|
||||
self.attribute_to_extract,
|
||||
self.attribute_to_skip,
|
||||
field_name,
|
||||
&mut tokenize_field,
|
||||
)?,
|
||||
Value::Array(array) => seek_leaf_values_in_array(
|
||||
&array,
|
||||
self.attribute_to_extract,
|
||||
self.attribute_to_skip,
|
||||
field_name,
|
||||
&mut tokenize_field,
|
||||
)?,
|
||||
value => tokenize_field(field_name, &value)?,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
/// take an iterator on tokens and compute their relative position depending on separator kinds
|
||||
/// if it's an `Hard` separator we add an additional relative proximity of MAX_DISTANCE between words,
|
||||
/// else we keep the standard proximity of 1 between words.
|
||||
fn process_tokens<'a>(
|
||||
start_offset: u32,
|
||||
tokens: impl Iterator<Item = Token<'a>>,
|
||||
) -> impl Iterator<Item = (u32, Token<'a>)> {
|
||||
tokens
|
||||
.skip_while(|token| token.is_separator())
|
||||
.scan((start_offset, None), |(offset, prev_kind), mut token| {
|
||||
match token.kind {
|
||||
TokenKind::Word | TokenKind::StopWord if !token.lemma().is_empty() => {
|
||||
*offset += match *prev_kind {
|
||||
Some(TokenKind::Separator(SeparatorKind::Hard)) => MAX_DISTANCE,
|
||||
Some(_) => 1,
|
||||
None => 0,
|
||||
};
|
||||
*prev_kind = Some(token.kind)
|
||||
}
|
||||
TokenKind::Separator(SeparatorKind::Hard) => {
|
||||
*prev_kind = Some(token.kind);
|
||||
}
|
||||
TokenKind::Separator(SeparatorKind::Soft)
|
||||
if *prev_kind != Some(TokenKind::Separator(SeparatorKind::Hard)) =>
|
||||
{
|
||||
*prev_kind = Some(token.kind);
|
||||
}
|
||||
_ => token.kind = TokenKind::Unknown,
|
||||
}
|
||||
Some((*offset, token))
|
||||
})
|
||||
.filter(|(_, t)| t.is_word())
|
||||
}
|
||||
|
||||
/// Factorize tokenizer building.
|
||||
pub fn tokenizer_builder<'a>(
|
||||
stop_words: Option<&'a fst::Set<&'a [u8]>>,
|
||||
allowed_separators: Option<&'a [&str]>,
|
||||
dictionary: Option<&'a [&str]>,
|
||||
) -> TokenizerBuilder<'a, &'a [u8]> {
|
||||
let mut tokenizer_builder = TokenizerBuilder::new();
|
||||
if let Some(stop_words) = stop_words {
|
||||
tokenizer_builder.stop_words(stop_words);
|
||||
}
|
||||
if let Some(dictionary) = dictionary {
|
||||
tokenizer_builder.words_dict(dictionary);
|
||||
}
|
||||
if let Some(separators) = allowed_separators {
|
||||
tokenizer_builder.separators(separators);
|
||||
}
|
||||
|
||||
tokenizer_builder
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use bumpalo::Bump;
|
||||
use charabia::TokenizerBuilder;
|
||||
use meili_snap::snapshot;
|
||||
use raw_collections::RawMap;
|
||||
use serde_json::json;
|
||||
use serde_json::value::RawValue;
|
||||
|
||||
use super::*;
|
||||
use crate::fields_ids_map::metadata::{FieldIdMapWithMetadata, MetadataBuilder};
|
||||
use crate::update::new::document::{DocumentFromVersions, Versions};
|
||||
use crate::FieldsIdsMap;
|
||||
|
||||
#[test]
|
||||
fn test_tokenize_document() {
|
||||
let mut fields_ids_map = FieldsIdsMap::new();
|
||||
|
||||
let document = json!({
|
||||
"doggo": { "name": "doggo",
|
||||
"age": 10,},
|
||||
"catto": {
|
||||
"catto": {
|
||||
"name": "pesti",
|
||||
"age": 23,
|
||||
}
|
||||
},
|
||||
"doggo.name": ["doggo", "catto"],
|
||||
"not-me": "UNSEARCHABLE",
|
||||
"me-nether": {"nope": "unsearchable"}
|
||||
});
|
||||
|
||||
let _field_1_id = fields_ids_map.insert("doggo").unwrap();
|
||||
let _field_2_id = fields_ids_map.insert("catto").unwrap();
|
||||
let _field_3_id = fields_ids_map.insert("doggo.name").unwrap();
|
||||
let _field_4_id = fields_ids_map.insert("not-me").unwrap();
|
||||
let _field_5_id = fields_ids_map.insert("me-nether").unwrap();
|
||||
|
||||
let mut tb = TokenizerBuilder::default();
|
||||
let document_tokenizer = DocumentTokenizer {
|
||||
tokenizer: &tb.build(),
|
||||
attribute_to_extract: None,
|
||||
attribute_to_skip: &["not-me", "me-nether.nope"],
|
||||
localized_attributes_rules: &[],
|
||||
max_positions_per_attributes: 1000,
|
||||
};
|
||||
|
||||
let fields_ids_map = FieldIdMapWithMetadata::new(
|
||||
fields_ids_map,
|
||||
MetadataBuilder::new(Default::default(), Default::default(), Default::default(), None),
|
||||
);
|
||||
|
||||
let fields_ids_map_lock = std::sync::RwLock::new(fields_ids_map);
|
||||
let mut global_fields_ids_map = GlobalFieldsIdsMap::new(&fields_ids_map_lock);
|
||||
|
||||
let mut words = std::collections::BTreeMap::new();
|
||||
|
||||
let document = document.to_string();
|
||||
|
||||
let bump = Bump::new();
|
||||
let document: &RawValue = serde_json::from_str(&document).unwrap();
|
||||
let document = RawMap::from_raw_value(document, &bump).unwrap();
|
||||
|
||||
let document = Versions::single(document);
|
||||
let document = DocumentFromVersions::new(&document);
|
||||
|
||||
document_tokenizer
|
||||
.tokenize_document(
|
||||
document,
|
||||
&mut global_fields_ids_map,
|
||||
&mut |_fname, fid, pos, word| {
|
||||
words.insert([fid, pos], word.to_string());
|
||||
Ok(())
|
||||
},
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
snapshot!(format!("{:#?}", words), @r###"
|
||||
{
|
||||
[
|
||||
2,
|
||||
0,
|
||||
]: "doggo",
|
||||
[
|
||||
2,
|
||||
MAX_DISTANCE,
|
||||
]: "doggo",
|
||||
[
|
||||
2,
|
||||
16,
|
||||
]: "catto",
|
||||
[
|
||||
3,
|
||||
0,
|
||||
]: "10",
|
||||
[
|
||||
4,
|
||||
0,
|
||||
]: "pesti",
|
||||
[
|
||||
5,
|
||||
0,
|
||||
]: "23",
|
||||
}
|
||||
"###);
|
||||
}
|
||||
}
|
432
crates/milli/src/update/new/extract/vectors/mod.rs
Normal file
432
crates/milli/src/update/new/extract/vectors/mod.rs
Normal file
|
@ -0,0 +1,432 @@
|
|||
use std::cell::RefCell;
|
||||
|
||||
use bumpalo::collections::Vec as BVec;
|
||||
use bumpalo::Bump;
|
||||
use hashbrown::HashMap;
|
||||
|
||||
use super::cache::DelAddRoaringBitmap;
|
||||
use crate::error::FaultSource;
|
||||
use crate::prompt::Prompt;
|
||||
use crate::update::new::channel::EmbeddingSender;
|
||||
use crate::update::new::indexer::document_changes::{Extractor, FullySend};
|
||||
use crate::update::new::vector_document::VectorDocument;
|
||||
use crate::update::new::DocumentChange;
|
||||
use crate::vector::error::{
|
||||
EmbedErrorKind, PossibleEmbeddingMistakes, UnusedVectorsDistributionBump,
|
||||
};
|
||||
use crate::vector::{Embedder, Embedding, EmbeddingConfigs};
|
||||
use crate::{DocumentId, FieldDistribution, InternalError, Result, ThreadPoolNoAbort, UserError};
|
||||
|
||||
pub struct EmbeddingExtractor<'a> {
|
||||
embedders: &'a EmbeddingConfigs,
|
||||
sender: &'a EmbeddingSender<'a>,
|
||||
possible_embedding_mistakes: PossibleEmbeddingMistakes,
|
||||
threads: &'a ThreadPoolNoAbort,
|
||||
}
|
||||
|
||||
impl<'a> EmbeddingExtractor<'a> {
|
||||
pub fn new(
|
||||
embedders: &'a EmbeddingConfigs,
|
||||
sender: &'a EmbeddingSender<'a>,
|
||||
field_distribution: &'a FieldDistribution,
|
||||
threads: &'a ThreadPoolNoAbort,
|
||||
) -> Self {
|
||||
let possible_embedding_mistakes = PossibleEmbeddingMistakes::new(field_distribution);
|
||||
Self { embedders, sender, threads, possible_embedding_mistakes }
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'extractor> Extractor<'extractor> for EmbeddingExtractor<'a> {
|
||||
type Data = FullySend<RefCell<HashMap<String, DelAddRoaringBitmap>>>;
|
||||
|
||||
fn init_data<'doc>(
|
||||
&'doc self,
|
||||
_extractor_alloc: raw_collections::alloc::RefBump<'extractor>,
|
||||
) -> crate::Result<Self::Data> {
|
||||
/// TODO: use the extractor_alloc in the hashbrown once you merge the branch where it is no longer a RefBump
|
||||
Ok(FullySend(Default::default()))
|
||||
}
|
||||
|
||||
fn process<'doc>(
|
||||
&'doc self,
|
||||
changes: impl Iterator<Item = crate::Result<DocumentChange<'doc>>>,
|
||||
context: &'doc crate::update::new::indexer::document_changes::DocumentChangeContext<
|
||||
Self::Data,
|
||||
>,
|
||||
) -> crate::Result<()> {
|
||||
let embedders = self.embedders.inner_as_ref();
|
||||
let mut unused_vectors_distribution =
|
||||
UnusedVectorsDistributionBump::new_in(&context.doc_alloc);
|
||||
|
||||
let mut all_chunks = BVec::with_capacity_in(embedders.len(), &context.doc_alloc);
|
||||
for (embedder_name, (embedder, prompt, _is_quantized)) in embedders {
|
||||
let embedder_id =
|
||||
context.index.embedder_category_id.get(&context.txn, embedder_name)?.ok_or_else(
|
||||
|| InternalError::DatabaseMissingEntry {
|
||||
db_name: "embedder_category_id",
|
||||
key: None,
|
||||
},
|
||||
)?;
|
||||
all_chunks.push(Chunks::new(
|
||||
embedder,
|
||||
embedder_id,
|
||||
embedder_name,
|
||||
prompt,
|
||||
&context.data.0,
|
||||
&self.possible_embedding_mistakes,
|
||||
self.threads,
|
||||
self.sender,
|
||||
&context.doc_alloc,
|
||||
))
|
||||
}
|
||||
|
||||
for change in changes {
|
||||
let change = change?;
|
||||
match change {
|
||||
DocumentChange::Deletion(_deletion) => {
|
||||
// handled by document sender
|
||||
}
|
||||
DocumentChange::Update(update) => {
|
||||
let old_vectors = update.current_vectors(
|
||||
&context.txn,
|
||||
context.index,
|
||||
context.db_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
let new_vectors = update.updated_vectors(&context.doc_alloc, self.embedders)?;
|
||||
|
||||
if let Some(new_vectors) = &new_vectors {
|
||||
unused_vectors_distribution.append(new_vectors);
|
||||
}
|
||||
|
||||
for chunks in &mut all_chunks {
|
||||
let embedder_name = chunks.embedder_name();
|
||||
let prompt = chunks.prompt();
|
||||
|
||||
let old_vectors = old_vectors.vectors_for_key(embedder_name)?.unwrap();
|
||||
if let Some(new_vectors) = new_vectors.as_ref().and_then(|new_vectors| {
|
||||
new_vectors.vectors_for_key(embedder_name).transpose()
|
||||
}) {
|
||||
let new_vectors = new_vectors?;
|
||||
match (old_vectors.regenerate, new_vectors.regenerate) {
|
||||
(true, true) | (false, false) => todo!(),
|
||||
_ => {
|
||||
chunks.set_regenerate(update.docid(), new_vectors.regenerate);
|
||||
}
|
||||
}
|
||||
// do we have set embeddings?
|
||||
if let Some(embeddings) = new_vectors.embeddings {
|
||||
chunks.set_vectors(
|
||||
update.docid(),
|
||||
embeddings
|
||||
.into_vec(&context.doc_alloc, embedder_name)
|
||||
.map_err(|error| UserError::InvalidVectorsEmbedderConf {
|
||||
document_id: update.external_document_id().to_string(),
|
||||
error,
|
||||
})?,
|
||||
);
|
||||
} else if new_vectors.regenerate {
|
||||
let new_rendered = prompt.render_document(
|
||||
update.current(
|
||||
&context.txn,
|
||||
context.index,
|
||||
context.db_fields_ids_map,
|
||||
)?,
|
||||
context.new_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
let old_rendered = prompt.render_document(
|
||||
update.merged(
|
||||
&context.txn,
|
||||
context.index,
|
||||
context.db_fields_ids_map,
|
||||
)?,
|
||||
context.new_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
if new_rendered != old_rendered {
|
||||
chunks.set_autogenerated(
|
||||
update.docid(),
|
||||
new_rendered,
|
||||
&unused_vectors_distribution,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
} else if old_vectors.regenerate {
|
||||
let old_rendered = prompt.render_document(
|
||||
update.current(
|
||||
&context.txn,
|
||||
context.index,
|
||||
context.db_fields_ids_map,
|
||||
)?,
|
||||
context.new_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
let new_rendered = prompt.render_document(
|
||||
update.merged(
|
||||
&context.txn,
|
||||
context.index,
|
||||
context.db_fields_ids_map,
|
||||
)?,
|
||||
context.new_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
if new_rendered != old_rendered {
|
||||
chunks.set_autogenerated(
|
||||
update.docid(),
|
||||
new_rendered,
|
||||
&unused_vectors_distribution,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
DocumentChange::Insertion(insertion) => {
|
||||
let new_vectors =
|
||||
insertion.inserted_vectors(&context.doc_alloc, self.embedders)?;
|
||||
if let Some(new_vectors) = &new_vectors {
|
||||
unused_vectors_distribution.append(new_vectors);
|
||||
}
|
||||
|
||||
for chunks in &mut all_chunks {
|
||||
let embedder_name = chunks.embedder_name();
|
||||
let prompt = chunks.prompt();
|
||||
// if no inserted vectors, then regenerate: true + no embeddings => autogenerate
|
||||
if let Some(new_vectors) = new_vectors.as_ref().and_then(|new_vectors| {
|
||||
new_vectors.vectors_for_key(embedder_name).transpose()
|
||||
}) {
|
||||
let new_vectors = new_vectors?;
|
||||
chunks.set_regenerate(insertion.docid(), new_vectors.regenerate);
|
||||
if let Some(embeddings) = new_vectors.embeddings {
|
||||
chunks.set_vectors(
|
||||
insertion.docid(),
|
||||
embeddings
|
||||
.into_vec(&context.doc_alloc, embedder_name)
|
||||
.map_err(|error| UserError::InvalidVectorsEmbedderConf {
|
||||
document_id: insertion
|
||||
.external_document_id()
|
||||
.to_string(),
|
||||
error,
|
||||
})?,
|
||||
);
|
||||
} else if new_vectors.regenerate {
|
||||
let rendered = prompt.render_document(
|
||||
insertion.inserted(),
|
||||
context.new_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
chunks.set_autogenerated(
|
||||
insertion.docid(),
|
||||
rendered,
|
||||
&unused_vectors_distribution,
|
||||
)?;
|
||||
}
|
||||
} else {
|
||||
let rendered = prompt.render_document(
|
||||
insertion.inserted(),
|
||||
context.new_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
chunks.set_autogenerated(
|
||||
insertion.docid(),
|
||||
rendered,
|
||||
&unused_vectors_distribution,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for chunk in all_chunks {
|
||||
chunk.drain(&unused_vectors_distribution)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
// **Warning**: the destructor of this struct is not normally run, make sure that all its fields:
|
||||
// 1. don't have side effects tied to they destructors
|
||||
// 2. if allocated, are allocated inside of the bumpalo
|
||||
//
|
||||
// Currently this is the case as:
|
||||
// 1. BVec are inside of the bumaplo
|
||||
// 2. All other fields are either trivial (u8) or references.
|
||||
struct Chunks<'a> {
|
||||
texts: BVec<'a, &'a str>,
|
||||
ids: BVec<'a, DocumentId>,
|
||||
|
||||
embedder: &'a Embedder,
|
||||
embedder_id: u8,
|
||||
embedder_name: &'a str,
|
||||
prompt: &'a Prompt,
|
||||
possible_embedding_mistakes: &'a PossibleEmbeddingMistakes,
|
||||
user_provided: &'a RefCell<HashMap<String, DelAddRoaringBitmap>>,
|
||||
threads: &'a ThreadPoolNoAbort,
|
||||
sender: &'a EmbeddingSender<'a>,
|
||||
}
|
||||
|
||||
impl<'a> Chunks<'a> {
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn new(
|
||||
embedder: &'a Embedder,
|
||||
embedder_id: u8,
|
||||
embedder_name: &'a str,
|
||||
prompt: &'a Prompt,
|
||||
user_provided: &'a RefCell<HashMap<String, DelAddRoaringBitmap>>,
|
||||
possible_embedding_mistakes: &'a PossibleEmbeddingMistakes,
|
||||
threads: &'a ThreadPoolNoAbort,
|
||||
sender: &'a EmbeddingSender<'a>,
|
||||
doc_alloc: &'a Bump,
|
||||
) -> Self {
|
||||
let capacity = embedder.prompt_count_in_chunk_hint() * embedder.chunk_count_hint();
|
||||
let texts = BVec::with_capacity_in(capacity, doc_alloc);
|
||||
let ids = BVec::with_capacity_in(capacity, doc_alloc);
|
||||
Self {
|
||||
texts,
|
||||
ids,
|
||||
embedder,
|
||||
prompt,
|
||||
possible_embedding_mistakes,
|
||||
threads,
|
||||
sender,
|
||||
embedder_id,
|
||||
embedder_name,
|
||||
user_provided,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn set_autogenerated(
|
||||
&mut self,
|
||||
docid: DocumentId,
|
||||
rendered: &'a str,
|
||||
unused_vectors_distribution: &UnusedVectorsDistributionBump,
|
||||
) -> Result<()> {
|
||||
if self.texts.len() < self.texts.capacity() {
|
||||
self.texts.push(rendered);
|
||||
self.ids.push(docid);
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
Self::embed_chunks(
|
||||
&mut self.texts,
|
||||
&mut self.ids,
|
||||
self.embedder,
|
||||
self.embedder_id,
|
||||
self.embedder_name,
|
||||
self.possible_embedding_mistakes,
|
||||
unused_vectors_distribution,
|
||||
self.threads,
|
||||
self.sender,
|
||||
)
|
||||
}
|
||||
|
||||
pub fn drain(
|
||||
mut self,
|
||||
unused_vectors_distribution: &UnusedVectorsDistributionBump,
|
||||
) -> Result<()> {
|
||||
let res = Self::embed_chunks(
|
||||
&mut self.texts,
|
||||
&mut self.ids,
|
||||
self.embedder,
|
||||
self.embedder_id,
|
||||
self.embedder_name,
|
||||
self.possible_embedding_mistakes,
|
||||
unused_vectors_distribution,
|
||||
self.threads,
|
||||
self.sender,
|
||||
);
|
||||
// optimization: don't run bvec dtors as they only contain bumpalo allocated stuff
|
||||
std::mem::forget(self);
|
||||
res
|
||||
}
|
||||
|
||||
pub fn embed_chunks(
|
||||
texts: &mut BVec<'a, &'a str>,
|
||||
ids: &mut BVec<'a, DocumentId>,
|
||||
embedder: &Embedder,
|
||||
embedder_id: u8,
|
||||
embedder_name: &str,
|
||||
possible_embedding_mistakes: &PossibleEmbeddingMistakes,
|
||||
unused_vectors_distribution: &UnusedVectorsDistributionBump,
|
||||
threads: &ThreadPoolNoAbort,
|
||||
sender: &EmbeddingSender<'a>,
|
||||
) -> Result<()> {
|
||||
let res = match embedder.embed_chunks_ref(texts.as_slice(), threads) {
|
||||
Ok(embeddings) => {
|
||||
for (docid, embedding) in ids.into_iter().zip(embeddings) {
|
||||
sender.set_vector(*docid, embedder_id, embedding).unwrap();
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
Err(error) => {
|
||||
if let FaultSource::Bug = error.fault {
|
||||
Err(crate::Error::InternalError(crate::InternalError::VectorEmbeddingError(
|
||||
error.into(),
|
||||
)))
|
||||
} else {
|
||||
let mut msg = format!(
|
||||
r"While embedding documents for embedder `{embedder_name}`: {error}"
|
||||
);
|
||||
|
||||
if let EmbedErrorKind::ManualEmbed(_) = &error.kind {
|
||||
msg += &format!("\n- Note: `{embedder_name}` has `source: userProvided`, so documents must provide embeddings as an array in `_vectors.{embedder_name}`.");
|
||||
}
|
||||
|
||||
let mut hint_count = 0;
|
||||
|
||||
for (vector_misspelling, count) in
|
||||
possible_embedding_mistakes.vector_mistakes().take(2)
|
||||
{
|
||||
msg += &format!("\n- Hint: try replacing `{vector_misspelling}` by `_vectors` in {count} document(s).");
|
||||
hint_count += 1;
|
||||
}
|
||||
|
||||
for (embedder_misspelling, count) in possible_embedding_mistakes
|
||||
.embedder_mistakes_bump(embedder_name, unused_vectors_distribution)
|
||||
.take(2)
|
||||
{
|
||||
msg += &format!("\n- Hint: try replacing `_vectors.{embedder_misspelling}` by `_vectors.{embedder_name}` in {count} document(s).");
|
||||
hint_count += 1;
|
||||
}
|
||||
|
||||
if hint_count == 0 {
|
||||
if let EmbedErrorKind::ManualEmbed(_) = &error.kind {
|
||||
msg += &format!(
|
||||
"\n- Hint: opt-out for a document with `_vectors.{embedder_name}: null`"
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
Err(crate::Error::UserError(crate::UserError::DocumentEmbeddingError(msg)))
|
||||
}
|
||||
}
|
||||
};
|
||||
texts.clear();
|
||||
ids.clear();
|
||||
res
|
||||
}
|
||||
|
||||
pub fn prompt(&self) -> &'a Prompt {
|
||||
self.prompt
|
||||
}
|
||||
|
||||
pub fn embedder_name(&self) -> &'a str {
|
||||
self.embedder_name
|
||||
}
|
||||
|
||||
fn set_regenerate(&self, docid: DocumentId, regenerate: bool) {
|
||||
let mut user_provided = self.user_provided.borrow_mut();
|
||||
let user_provided = user_provided.entry_ref(self.embedder_name).or_default();
|
||||
if regenerate {
|
||||
// regenerate == !user_provided
|
||||
user_provided.del.get_or_insert(Default::default()).insert(docid);
|
||||
} else {
|
||||
user_provided.add.get_or_insert(Default::default()).insert(docid);
|
||||
}
|
||||
}
|
||||
|
||||
fn set_vectors(&self, docid: DocumentId, embeddings: Vec<Embedding>) {
|
||||
self.sender.set_vectors(docid, self.embedder_id, embeddings).unwrap();
|
||||
}
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue