mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-03 20:07:09 +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
437
crates/milli/src/update/new/channel.rs
Normal file
437
crates/milli/src/update/new/channel.rs
Normal file
|
@ -0,0 +1,437 @@
|
|||
use std::marker::PhantomData;
|
||||
use std::sync::atomic::{AtomicUsize, Ordering};
|
||||
|
||||
use crossbeam_channel::{IntoIter, Receiver, SendError, Sender};
|
||||
use grenad::Merger;
|
||||
use hashbrown::HashMap;
|
||||
use heed::types::Bytes;
|
||||
use roaring::RoaringBitmap;
|
||||
|
||||
use super::extract::FacetKind;
|
||||
use super::StdResult;
|
||||
use crate::index::main_key::DOCUMENTS_IDS_KEY;
|
||||
use crate::update::new::KvReaderFieldId;
|
||||
use crate::update::MergeDeladdCboRoaringBitmaps;
|
||||
use crate::vector::Embedding;
|
||||
use crate::{DocumentId, Index};
|
||||
|
||||
/// The capacity of the channel is currently in number of messages.
|
||||
pub fn extractor_writer_channel(cap: usize) -> (ExtractorSender, WriterReceiver) {
|
||||
let (sender, receiver) = crossbeam_channel::bounded(cap);
|
||||
(
|
||||
ExtractorSender {
|
||||
sender,
|
||||
send_count: Default::default(),
|
||||
writer_contentious_count: Default::default(),
|
||||
extractor_contentious_count: Default::default(),
|
||||
},
|
||||
WriterReceiver(receiver),
|
||||
)
|
||||
}
|
||||
|
||||
pub struct KeyValueEntry {
|
||||
pub key_length: usize,
|
||||
pub data: Box<[u8]>,
|
||||
}
|
||||
|
||||
impl KeyValueEntry {
|
||||
pub fn from_small_key_value(key: &[u8], value: &[u8]) -> Self {
|
||||
let mut data = Vec::with_capacity(key.len() + value.len());
|
||||
data.extend_from_slice(key);
|
||||
data.extend_from_slice(value);
|
||||
KeyValueEntry { key_length: key.len(), data: data.into_boxed_slice() }
|
||||
}
|
||||
|
||||
pub fn from_small_key_bitmap(key: &[u8], bitmap: RoaringBitmap) -> Self {
|
||||
let mut data = Vec::with_capacity(key.len() + bitmap.serialized_size());
|
||||
data.extend_from_slice(key);
|
||||
bitmap.serialize_into(&mut data).unwrap();
|
||||
KeyValueEntry { key_length: key.len(), data: data.into_boxed_slice() }
|
||||
}
|
||||
|
||||
pub fn key(&self) -> &[u8] {
|
||||
&self.data[..self.key_length]
|
||||
}
|
||||
|
||||
pub fn value(&self) -> &[u8] {
|
||||
&self.data[self.key_length..]
|
||||
}
|
||||
}
|
||||
|
||||
pub struct KeyEntry {
|
||||
data: Box<[u8]>,
|
||||
}
|
||||
|
||||
impl KeyEntry {
|
||||
pub fn from_key(key: &[u8]) -> Self {
|
||||
KeyEntry { data: key.to_vec().into_boxed_slice() }
|
||||
}
|
||||
|
||||
pub fn entry(&self) -> &[u8] {
|
||||
self.data.as_ref()
|
||||
}
|
||||
}
|
||||
|
||||
pub enum EntryOperation {
|
||||
Delete(KeyEntry),
|
||||
Write(KeyValueEntry),
|
||||
}
|
||||
|
||||
pub enum WriterOperation {
|
||||
DbOperation(DbOperation),
|
||||
ArroyOperation(ArroyOperation),
|
||||
}
|
||||
|
||||
pub enum ArroyOperation {
|
||||
/// TODO: call when deleting regular documents
|
||||
DeleteVectors {
|
||||
docid: DocumentId,
|
||||
},
|
||||
SetVectors {
|
||||
docid: DocumentId,
|
||||
embedder_id: u8,
|
||||
embeddings: Vec<Embedding>,
|
||||
},
|
||||
SetVector {
|
||||
docid: DocumentId,
|
||||
embedder_id: u8,
|
||||
embedding: Embedding,
|
||||
},
|
||||
Finish {
|
||||
user_provided: HashMap<String, RoaringBitmap>,
|
||||
},
|
||||
}
|
||||
|
||||
pub struct DbOperation {
|
||||
database: Database,
|
||||
entry: EntryOperation,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub enum Database {
|
||||
Documents,
|
||||
ExternalDocumentsIds,
|
||||
ExactWordDocids,
|
||||
FidWordCountDocids,
|
||||
Main,
|
||||
WordDocids,
|
||||
WordFidDocids,
|
||||
WordPairProximityDocids,
|
||||
WordPositionDocids,
|
||||
FacetIdIsNullDocids,
|
||||
FacetIdIsEmptyDocids,
|
||||
FacetIdExistsDocids,
|
||||
FacetIdF64NumberDocids,
|
||||
FacetIdStringDocids,
|
||||
}
|
||||
|
||||
impl Database {
|
||||
pub fn database(&self, index: &Index) -> heed::Database<Bytes, Bytes> {
|
||||
match self {
|
||||
Database::Documents => index.documents.remap_types(),
|
||||
Database::ExternalDocumentsIds => index.external_documents_ids.remap_types(),
|
||||
Database::ExactWordDocids => index.exact_word_docids.remap_types(),
|
||||
Database::Main => index.main.remap_types(),
|
||||
Database::WordDocids => index.word_docids.remap_types(),
|
||||
Database::WordFidDocids => index.word_fid_docids.remap_types(),
|
||||
Database::WordPositionDocids => index.word_position_docids.remap_types(),
|
||||
Database::FidWordCountDocids => index.field_id_word_count_docids.remap_types(),
|
||||
Database::WordPairProximityDocids => index.word_pair_proximity_docids.remap_types(),
|
||||
Database::FacetIdIsNullDocids => index.facet_id_is_null_docids.remap_types(),
|
||||
Database::FacetIdIsEmptyDocids => index.facet_id_is_empty_docids.remap_types(),
|
||||
Database::FacetIdExistsDocids => index.facet_id_exists_docids.remap_types(),
|
||||
Database::FacetIdF64NumberDocids => index.facet_id_f64_docids.remap_types(),
|
||||
Database::FacetIdStringDocids => index.facet_id_string_docids.remap_types(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<FacetKind> for Database {
|
||||
fn from(value: FacetKind) -> Self {
|
||||
match value {
|
||||
FacetKind::Number => Database::FacetIdF64NumberDocids,
|
||||
FacetKind::String => Database::FacetIdStringDocids,
|
||||
FacetKind::Null => Database::FacetIdIsNullDocids,
|
||||
FacetKind::Empty => Database::FacetIdIsEmptyDocids,
|
||||
FacetKind::Exists => Database::FacetIdExistsDocids,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl DbOperation {
|
||||
pub fn database(&self, index: &Index) -> heed::Database<Bytes, Bytes> {
|
||||
self.database.database(index)
|
||||
}
|
||||
|
||||
pub fn entry(self) -> EntryOperation {
|
||||
self.entry
|
||||
}
|
||||
}
|
||||
|
||||
pub struct WriterReceiver(Receiver<WriterOperation>);
|
||||
|
||||
impl IntoIterator for WriterReceiver {
|
||||
type Item = WriterOperation;
|
||||
type IntoIter = IntoIter<Self::Item>;
|
||||
|
||||
fn into_iter(self) -> Self::IntoIter {
|
||||
self.0.into_iter()
|
||||
}
|
||||
}
|
||||
|
||||
pub struct ExtractorSender {
|
||||
sender: Sender<WriterOperation>,
|
||||
/// The number of message we sent in total in the channel.
|
||||
send_count: AtomicUsize,
|
||||
/// The number of times we sent something in a channel that was full.
|
||||
writer_contentious_count: AtomicUsize,
|
||||
/// The number of times we sent something in a channel that was empty.
|
||||
extractor_contentious_count: AtomicUsize,
|
||||
}
|
||||
|
||||
impl Drop for ExtractorSender {
|
||||
fn drop(&mut self) {
|
||||
let send_count = *self.send_count.get_mut();
|
||||
let writer_contentious_count = *self.writer_contentious_count.get_mut();
|
||||
let extractor_contentious_count = *self.extractor_contentious_count.get_mut();
|
||||
eprintln!(
|
||||
"Extractor channel stats: {send_count} sends, \
|
||||
{writer_contentious_count} writer contentions ({}%), \
|
||||
{extractor_contentious_count} extractor contentions ({}%)",
|
||||
(writer_contentious_count as f32 / send_count as f32) * 100.0,
|
||||
(extractor_contentious_count as f32 / send_count as f32) * 100.0
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
impl ExtractorSender {
|
||||
pub fn docids<D: DatabaseType>(&self) -> WordDocidsSender<'_, D> {
|
||||
WordDocidsSender { sender: self, _marker: PhantomData }
|
||||
}
|
||||
|
||||
pub fn facet_docids(&self) -> FacetDocidsSender<'_> {
|
||||
FacetDocidsSender { sender: self }
|
||||
}
|
||||
|
||||
pub fn documents(&self) -> DocumentsSender<'_> {
|
||||
DocumentsSender(self)
|
||||
}
|
||||
|
||||
pub fn send_documents_ids(&self, documents_ids: RoaringBitmap) -> StdResult<(), SendError<()>> {
|
||||
let entry = EntryOperation::Write(KeyValueEntry::from_small_key_bitmap(
|
||||
DOCUMENTS_IDS_KEY.as_bytes(),
|
||||
documents_ids,
|
||||
));
|
||||
match self.send_db_operation(DbOperation { database: Database::Main, entry }) {
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}
|
||||
}
|
||||
|
||||
fn send_db_operation(&self, op: DbOperation) -> StdResult<(), SendError<()>> {
|
||||
if self.sender.is_full() {
|
||||
self.writer_contentious_count.fetch_add(1, Ordering::SeqCst);
|
||||
}
|
||||
if self.sender.is_empty() {
|
||||
self.extractor_contentious_count.fetch_add(1, Ordering::SeqCst);
|
||||
}
|
||||
|
||||
self.send_count.fetch_add(1, Ordering::SeqCst);
|
||||
match self.sender.send(WriterOperation::DbOperation(op)) {
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub enum ExactWordDocids {}
|
||||
pub enum FidWordCountDocids {}
|
||||
pub enum WordDocids {}
|
||||
pub enum WordFidDocids {}
|
||||
pub enum WordPairProximityDocids {}
|
||||
pub enum WordPositionDocids {}
|
||||
|
||||
pub trait DatabaseType {
|
||||
const DATABASE: Database;
|
||||
}
|
||||
|
||||
impl DatabaseType for ExactWordDocids {
|
||||
const DATABASE: Database = Database::ExactWordDocids;
|
||||
}
|
||||
|
||||
impl DatabaseType for FidWordCountDocids {
|
||||
const DATABASE: Database = Database::FidWordCountDocids;
|
||||
}
|
||||
|
||||
impl DatabaseType for WordDocids {
|
||||
const DATABASE: Database = Database::WordDocids;
|
||||
}
|
||||
|
||||
impl DatabaseType for WordFidDocids {
|
||||
const DATABASE: Database = Database::WordFidDocids;
|
||||
}
|
||||
|
||||
impl DatabaseType for WordPairProximityDocids {
|
||||
const DATABASE: Database = Database::WordPairProximityDocids;
|
||||
}
|
||||
|
||||
impl DatabaseType for WordPositionDocids {
|
||||
const DATABASE: Database = Database::WordPositionDocids;
|
||||
}
|
||||
|
||||
pub trait DocidsSender {
|
||||
fn write(&self, key: &[u8], value: &[u8]) -> StdResult<(), SendError<()>>;
|
||||
fn delete(&self, key: &[u8]) -> StdResult<(), SendError<()>>;
|
||||
}
|
||||
|
||||
pub struct WordDocidsSender<'a, D> {
|
||||
sender: &'a ExtractorSender,
|
||||
_marker: PhantomData<D>,
|
||||
}
|
||||
|
||||
impl<D: DatabaseType> DocidsSender for WordDocidsSender<'_, D> {
|
||||
fn write(&self, key: &[u8], value: &[u8]) -> StdResult<(), SendError<()>> {
|
||||
let entry = EntryOperation::Write(KeyValueEntry::from_small_key_value(key, value));
|
||||
match self.sender.send_db_operation(DbOperation { database: D::DATABASE, entry }) {
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}
|
||||
}
|
||||
|
||||
fn delete(&self, key: &[u8]) -> StdResult<(), SendError<()>> {
|
||||
let entry = EntryOperation::Delete(KeyEntry::from_key(key));
|
||||
match self.sender.send_db_operation(DbOperation { database: D::DATABASE, entry }) {
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct FacetDocidsSender<'a> {
|
||||
sender: &'a ExtractorSender,
|
||||
}
|
||||
|
||||
impl DocidsSender for FacetDocidsSender<'_> {
|
||||
fn write(&self, key: &[u8], value: &[u8]) -> StdResult<(), SendError<()>> {
|
||||
let (facet_kind, key) = FacetKind::extract_from_key(key);
|
||||
let database = Database::from(facet_kind);
|
||||
// let entry = EntryOperation::Write(KeyValueEntry::from_small_key_value(key, value));
|
||||
let entry = match facet_kind {
|
||||
// skip level group size
|
||||
FacetKind::String | FacetKind::Number => {
|
||||
// add facet group size
|
||||
let value = [&[1], value].concat();
|
||||
EntryOperation::Write(KeyValueEntry::from_small_key_value(key, &value))
|
||||
}
|
||||
_ => EntryOperation::Write(KeyValueEntry::from_small_key_value(key, value)),
|
||||
};
|
||||
match self.sender.send_db_operation(DbOperation { database, entry }) {
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}
|
||||
}
|
||||
|
||||
fn delete(&self, key: &[u8]) -> StdResult<(), SendError<()>> {
|
||||
let (facet_kind, key) = FacetKind::extract_from_key(key);
|
||||
let database = Database::from(facet_kind);
|
||||
let entry = EntryOperation::Delete(KeyEntry::from_key(key));
|
||||
match self.sender.send_db_operation(DbOperation { database, entry }) {
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DocumentsSender<'a>(&'a ExtractorSender);
|
||||
|
||||
impl DocumentsSender<'_> {
|
||||
/// TODO do that efficiently
|
||||
pub fn uncompressed(
|
||||
&self,
|
||||
docid: DocumentId,
|
||||
external_id: String,
|
||||
document: &KvReaderFieldId,
|
||||
) -> StdResult<(), SendError<()>> {
|
||||
let entry = EntryOperation::Write(KeyValueEntry::from_small_key_value(
|
||||
&docid.to_be_bytes(),
|
||||
document.as_bytes(),
|
||||
));
|
||||
match self.0.send_db_operation(DbOperation { database: Database::Documents, entry }) {
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}?;
|
||||
|
||||
let entry = EntryOperation::Write(KeyValueEntry::from_small_key_value(
|
||||
external_id.as_bytes(),
|
||||
&docid.to_be_bytes(),
|
||||
));
|
||||
match self
|
||||
.0
|
||||
.send_db_operation(DbOperation { database: Database::ExternalDocumentsIds, entry })
|
||||
{
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn delete(&self, docid: DocumentId, external_id: String) -> StdResult<(), SendError<()>> {
|
||||
let entry = EntryOperation::Delete(KeyEntry::from_key(&docid.to_be_bytes()));
|
||||
match self.0.send_db_operation(DbOperation { database: Database::Documents, entry }) {
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}?;
|
||||
|
||||
let entry = EntryOperation::Delete(KeyEntry::from_key(external_id.as_bytes()));
|
||||
match self
|
||||
.0
|
||||
.send_db_operation(DbOperation { database: Database::ExternalDocumentsIds, entry })
|
||||
{
|
||||
Ok(()) => Ok(()),
|
||||
Err(SendError(_)) => Err(SendError(())),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct EmbeddingSender<'a>(&'a Sender<WriterOperation>);
|
||||
|
||||
impl EmbeddingSender<'_> {
|
||||
pub fn set_vectors(
|
||||
&self,
|
||||
docid: DocumentId,
|
||||
embedder_id: u8,
|
||||
embeddings: Vec<Embedding>,
|
||||
) -> StdResult<(), SendError<()>> {
|
||||
self.0
|
||||
.send(WriterOperation::ArroyOperation(ArroyOperation::SetVectors {
|
||||
docid,
|
||||
embedder_id,
|
||||
embeddings,
|
||||
}))
|
||||
.map_err(|_| SendError(()))
|
||||
}
|
||||
|
||||
pub fn set_vector(
|
||||
&self,
|
||||
docid: DocumentId,
|
||||
embedder_id: u8,
|
||||
embedding: Embedding,
|
||||
) -> StdResult<(), SendError<()>> {
|
||||
self.0
|
||||
.send(WriterOperation::ArroyOperation(ArroyOperation::SetVector {
|
||||
docid,
|
||||
embedder_id,
|
||||
embedding,
|
||||
}))
|
||||
.map_err(|_| SendError(()))
|
||||
}
|
||||
|
||||
/// Marks all embedders as "to be built"
|
||||
pub fn finish(
|
||||
self,
|
||||
user_provided: HashMap<String, RoaringBitmap>,
|
||||
) -> StdResult<(), SendError<()>> {
|
||||
self.0
|
||||
.send(WriterOperation::ArroyOperation(ArroyOperation::Finish { user_provided }))
|
||||
.map_err(|_| SendError(()))
|
||||
}
|
||||
}
|
398
crates/milli/src/update/new/document.rs
Normal file
398
crates/milli/src/update/new/document.rs
Normal file
|
@ -0,0 +1,398 @@
|
|||
use std::collections::{BTreeMap, BTreeSet};
|
||||
|
||||
use heed::RoTxn;
|
||||
use raw_collections::RawMap;
|
||||
use serde_json::value::RawValue;
|
||||
|
||||
use super::vector_document::VectorDocument;
|
||||
use super::{KvReaderFieldId, KvWriterFieldId};
|
||||
use crate::documents::FieldIdMapper;
|
||||
use crate::vector::parsed_vectors::RESERVED_VECTORS_FIELD_NAME;
|
||||
use crate::{DocumentId, GlobalFieldsIdsMap, Index, InternalError, Result, UserError};
|
||||
|
||||
/// A view into a document that can represent either the current version from the DB,
|
||||
/// the update data from payload or other means, or the merged updated version.
|
||||
///
|
||||
/// The 'doc lifetime is meant to live sufficiently for the document to be handled by the extractors.
|
||||
pub trait Document<'doc> {
|
||||
/// Iterate over all **top-level** fields of the document, returning their name and raw JSON value.
|
||||
///
|
||||
/// - The returned values *may* contain nested fields.
|
||||
/// - The `_vectors` and `_geo` fields are **ignored** by this method, meaning they are **not returned** by this method.
|
||||
fn iter_top_level_fields(&self) -> impl Iterator<Item = Result<(&'doc str, &'doc RawValue)>>;
|
||||
|
||||
fn len(&self) -> usize;
|
||||
|
||||
fn is_empty(&self) -> bool {
|
||||
self.len() == 0
|
||||
}
|
||||
|
||||
fn top_level_field(&self, k: &str) -> Result<Option<&'doc RawValue>>;
|
||||
|
||||
/// Returns the unparsed value of the `_vectors` field from the document data.
|
||||
///
|
||||
/// This field alone is insufficient to retrieve vectors, as they may be stored in a dedicated location in the database.
|
||||
/// Use a [`super::vector_document::VectorDocument`] to access the vector.
|
||||
///
|
||||
/// This method is meant as a convenience for implementors of [`super::vector_document::VectorDocument`].
|
||||
fn vectors_field(&self) -> Result<Option<&'doc RawValue>>;
|
||||
|
||||
/// Returns the unparsed value of the `_geo` field from the document data.
|
||||
///
|
||||
/// This field alone is insufficient to retrieve geo data, as they may be stored in a dedicated location in the database.
|
||||
/// Use a [`super::geo_document::GeoDocument`] to access the vector.
|
||||
///
|
||||
/// This method is meant as a convenience for implementors of [`super::geo_document::GeoDocument`].
|
||||
fn geo_field(&self) -> Result<Option<&'doc RawValue>>;
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct DocumentFromDb<'t, Mapper: FieldIdMapper>
|
||||
where
|
||||
Mapper: FieldIdMapper,
|
||||
{
|
||||
fields_ids_map: &'t Mapper,
|
||||
content: &'t KvReaderFieldId,
|
||||
}
|
||||
|
||||
impl<'t, Mapper: FieldIdMapper> Clone for DocumentFromDb<'t, Mapper> {
|
||||
#[inline]
|
||||
fn clone(&self) -> Self {
|
||||
*self
|
||||
}
|
||||
}
|
||||
impl<'t, Mapper: FieldIdMapper> Copy for DocumentFromDb<'t, Mapper> {}
|
||||
|
||||
impl<'t, Mapper: FieldIdMapper> Document<'t> for DocumentFromDb<'t, Mapper> {
|
||||
fn iter_top_level_fields(&self) -> impl Iterator<Item = Result<(&'t str, &'t RawValue)>> {
|
||||
let mut it = self.content.iter();
|
||||
|
||||
std::iter::from_fn(move || {
|
||||
let (fid, value) = it.next()?;
|
||||
|
||||
let res = (|| loop {
|
||||
let name = self.fields_ids_map.name(fid).ok_or(
|
||||
InternalError::FieldIdMapMissingEntry(crate::FieldIdMapMissingEntry::FieldId {
|
||||
field_id: fid,
|
||||
process: "getting current document",
|
||||
}),
|
||||
)?;
|
||||
|
||||
if name == RESERVED_VECTORS_FIELD_NAME || name == "_geo" {
|
||||
continue;
|
||||
}
|
||||
|
||||
let value =
|
||||
serde_json::from_slice(value).map_err(crate::InternalError::SerdeJson)?;
|
||||
|
||||
return Ok((name, value));
|
||||
})();
|
||||
|
||||
Some(res)
|
||||
})
|
||||
}
|
||||
|
||||
fn vectors_field(&self) -> Result<Option<&'t RawValue>> {
|
||||
self.field(RESERVED_VECTORS_FIELD_NAME)
|
||||
}
|
||||
|
||||
fn geo_field(&self) -> Result<Option<&'t RawValue>> {
|
||||
self.field("_geo")
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.content.iter().count()
|
||||
}
|
||||
|
||||
fn top_level_field(&self, k: &str) -> Result<Option<&'t RawValue>> {
|
||||
self.field(k)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'t, Mapper: FieldIdMapper> DocumentFromDb<'t, Mapper> {
|
||||
pub fn new(
|
||||
docid: DocumentId,
|
||||
rtxn: &'t RoTxn,
|
||||
index: &'t Index,
|
||||
db_fields_ids_map: &'t Mapper,
|
||||
) -> Result<Option<Self>> {
|
||||
index.documents.get(rtxn, &docid).map_err(crate::Error::from).map(|reader| {
|
||||
reader.map(|reader| Self { fields_ids_map: db_fields_ids_map, content: reader })
|
||||
})
|
||||
}
|
||||
|
||||
pub fn field(&self, name: &str) -> Result<Option<&'t RawValue>> {
|
||||
let Some(fid) = self.fields_ids_map.id(name) else {
|
||||
return Ok(None);
|
||||
};
|
||||
let Some(value) = self.content.get(fid) else { return Ok(None) };
|
||||
Ok(Some(serde_json::from_slice(value).map_err(InternalError::SerdeJson)?))
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct DocumentFromVersions<'a, 'doc> {
|
||||
versions: &'a Versions<'doc>,
|
||||
}
|
||||
|
||||
impl<'a, 'doc> DocumentFromVersions<'a, 'doc> {
|
||||
pub fn new(versions: &'a Versions<'doc>) -> Self {
|
||||
Self { versions }
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'doc> Document<'doc> for DocumentFromVersions<'a, 'doc> {
|
||||
fn iter_top_level_fields(&self) -> impl Iterator<Item = Result<(&'doc str, &'doc RawValue)>> {
|
||||
self.versions.iter_top_level_fields().map(Ok)
|
||||
}
|
||||
|
||||
fn vectors_field(&self) -> Result<Option<&'doc RawValue>> {
|
||||
Ok(self.versions.vectors_field())
|
||||
}
|
||||
|
||||
fn geo_field(&self) -> Result<Option<&'doc RawValue>> {
|
||||
Ok(self.versions.geo_field())
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.versions.len()
|
||||
}
|
||||
|
||||
fn top_level_field(&self, k: &str) -> Result<Option<&'doc RawValue>> {
|
||||
Ok(self.versions.top_level_field(k))
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct MergedDocument<'a, 'doc, 't, Mapper: FieldIdMapper> {
|
||||
new_doc: DocumentFromVersions<'a, 'doc>,
|
||||
db: Option<DocumentFromDb<'t, Mapper>>,
|
||||
}
|
||||
|
||||
impl<'a, 'doc, 't, Mapper: FieldIdMapper> MergedDocument<'a, 'doc, 't, Mapper> {
|
||||
pub fn with_db(
|
||||
docid: DocumentId,
|
||||
rtxn: &'t RoTxn,
|
||||
index: &'t Index,
|
||||
db_fields_ids_map: &'t Mapper,
|
||||
new_doc: DocumentFromVersions<'a, 'doc>,
|
||||
) -> Result<Self> {
|
||||
let db = DocumentFromDb::new(docid, rtxn, index, db_fields_ids_map)?;
|
||||
Ok(Self { new_doc, db })
|
||||
}
|
||||
|
||||
pub fn without_db(new_doc: DocumentFromVersions<'a, 'doc>) -> Self {
|
||||
Self { new_doc, db: None }
|
||||
}
|
||||
}
|
||||
|
||||
impl<'d, 'doc: 'd, 't: 'd, Mapper: FieldIdMapper> Document<'d>
|
||||
for MergedDocument<'d, 'doc, 't, Mapper>
|
||||
{
|
||||
fn iter_top_level_fields(&self) -> impl Iterator<Item = Result<(&'d str, &'d RawValue)>> {
|
||||
let mut new_doc_it = self.new_doc.iter_top_level_fields();
|
||||
let mut db_it = self.db.iter().flat_map(|db| db.iter_top_level_fields());
|
||||
let mut seen_fields = BTreeSet::new();
|
||||
|
||||
std::iter::from_fn(move || {
|
||||
if let Some(next) = new_doc_it.next() {
|
||||
if let Ok((name, _)) = next {
|
||||
seen_fields.insert(name);
|
||||
}
|
||||
return Some(next);
|
||||
}
|
||||
loop {
|
||||
match db_it.next()? {
|
||||
Ok((name, value)) => {
|
||||
if seen_fields.contains(name) {
|
||||
continue;
|
||||
}
|
||||
return Some(Ok((name, value)));
|
||||
}
|
||||
Err(err) => return Some(Err(err)),
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
fn vectors_field(&self) -> Result<Option<&'d RawValue>> {
|
||||
if let Some(vectors) = self.new_doc.vectors_field()? {
|
||||
return Ok(Some(vectors));
|
||||
}
|
||||
|
||||
let Some(db) = self.db else { return Ok(None) };
|
||||
|
||||
db.vectors_field()
|
||||
}
|
||||
|
||||
fn geo_field(&self) -> Result<Option<&'d RawValue>> {
|
||||
if let Some(geo) = self.new_doc.geo_field()? {
|
||||
return Ok(Some(geo));
|
||||
}
|
||||
|
||||
let Some(db) = self.db else { return Ok(None) };
|
||||
|
||||
db.geo_field()
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.iter_top_level_fields().count()
|
||||
}
|
||||
|
||||
fn top_level_field(&self, k: &str) -> Result<Option<&'d RawValue>> {
|
||||
if let Some(f) = self.new_doc.top_level_field(k)? {
|
||||
return Ok(Some(f));
|
||||
}
|
||||
if let Some(db) = self.db {
|
||||
return db.field(k);
|
||||
}
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'doc, D> Document<'doc> for &D
|
||||
where
|
||||
D: Document<'doc>,
|
||||
{
|
||||
fn iter_top_level_fields(&self) -> impl Iterator<Item = Result<(&'doc str, &'doc RawValue)>> {
|
||||
D::iter_top_level_fields(self)
|
||||
}
|
||||
|
||||
fn vectors_field(&self) -> Result<Option<&'doc RawValue>> {
|
||||
D::vectors_field(self)
|
||||
}
|
||||
|
||||
fn geo_field(&self) -> Result<Option<&'doc RawValue>> {
|
||||
D::geo_field(self)
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
D::len(self)
|
||||
}
|
||||
|
||||
fn top_level_field(&self, k: &str) -> Result<Option<&'doc RawValue>> {
|
||||
D::top_level_field(self, k)
|
||||
}
|
||||
}
|
||||
|
||||
/// Turn this document into an obkv, whose fields are indexed by the provided `FieldIdMapper`.
|
||||
///
|
||||
/// The produced obkv is suitable for storing into the documents DB, meaning:
|
||||
///
|
||||
/// - It contains the contains of `_vectors` that are not configured as an embedder
|
||||
/// - It contains all the top-level fields of the document, with their raw JSON value as value.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// - If the document contains a top-level field that is not present in `fields_ids_map`.
|
||||
///
|
||||
pub fn write_to_obkv<'s, 'a, 'map>(
|
||||
document: &'s impl Document<'s>,
|
||||
vector_document: Option<&'s impl VectorDocument<'s>>,
|
||||
fields_ids_map: &'a mut GlobalFieldsIdsMap<'map>,
|
||||
mut document_buffer: &'a mut Vec<u8>,
|
||||
) -> Result<&'a KvReaderFieldId>
|
||||
where
|
||||
's: 'a,
|
||||
{
|
||||
// will be used in 'inject_vectors
|
||||
let vectors_value: Box<RawValue>;
|
||||
|
||||
document_buffer.clear();
|
||||
let mut unordered_field_buffer = Vec::new();
|
||||
unordered_field_buffer.clear();
|
||||
|
||||
let mut writer = KvWriterFieldId::new(&mut document_buffer);
|
||||
|
||||
for res in document.iter_top_level_fields() {
|
||||
let (field_name, value) = res?;
|
||||
let field_id =
|
||||
fields_ids_map.id_or_insert(field_name).ok_or(UserError::AttributeLimitReached)?;
|
||||
unordered_field_buffer.push((field_id, value));
|
||||
}
|
||||
|
||||
'inject_vectors: {
|
||||
let Some(vector_document) = vector_document else { break 'inject_vectors };
|
||||
|
||||
let vectors_fid = fields_ids_map
|
||||
.id_or_insert(RESERVED_VECTORS_FIELD_NAME)
|
||||
.ok_or(UserError::AttributeLimitReached)?;
|
||||
|
||||
let mut vectors = BTreeMap::new();
|
||||
for res in vector_document.iter_vectors() {
|
||||
let (name, entry) = res?;
|
||||
if entry.has_configured_embedder {
|
||||
continue; // we don't write vectors with configured embedder in documents
|
||||
}
|
||||
vectors.insert(
|
||||
name,
|
||||
serde_json::json!({
|
||||
"regenerate": entry.regenerate,
|
||||
// TODO: consider optimizing the shape of embedders here to store an array of f32 rather than a JSON object
|
||||
"embeddings": entry.embeddings,
|
||||
}),
|
||||
);
|
||||
}
|
||||
|
||||
vectors_value = serde_json::value::to_raw_value(&vectors).unwrap();
|
||||
unordered_field_buffer.push((vectors_fid, &vectors_value));
|
||||
}
|
||||
|
||||
unordered_field_buffer.sort_by_key(|(fid, _)| *fid);
|
||||
for (fid, value) in unordered_field_buffer.iter() {
|
||||
writer.insert(*fid, value.get().as_bytes()).unwrap();
|
||||
}
|
||||
|
||||
writer.finish().unwrap();
|
||||
Ok(KvReaderFieldId::from_slice(document_buffer))
|
||||
}
|
||||
|
||||
pub type Entry<'doc> = (&'doc str, &'doc RawValue);
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct Versions<'doc> {
|
||||
data: RawMap<'doc>,
|
||||
}
|
||||
|
||||
impl<'doc> Versions<'doc> {
|
||||
pub fn multiple(
|
||||
mut versions: impl Iterator<Item = Result<RawMap<'doc>>>,
|
||||
) -> Result<Option<Self>> {
|
||||
let Some(data) = versions.next() else { return Ok(None) };
|
||||
let mut data = data?;
|
||||
for future_version in versions {
|
||||
let future_version = future_version?;
|
||||
for (field, value) in future_version {
|
||||
data.insert(field, value);
|
||||
}
|
||||
}
|
||||
Ok(Some(Self::single(data)))
|
||||
}
|
||||
|
||||
pub fn single(version: RawMap<'doc>) -> Self {
|
||||
Self { data: version }
|
||||
}
|
||||
|
||||
pub fn iter_top_level_fields(&self) -> impl Iterator<Item = (&'doc str, &'doc RawValue)> + '_ {
|
||||
self.data.iter().filter(|(k, _)| *k != RESERVED_VECTORS_FIELD_NAME && *k != "_geo")
|
||||
}
|
||||
|
||||
pub fn vectors_field(&self) -> Option<&'doc RawValue> {
|
||||
self.data.get(RESERVED_VECTORS_FIELD_NAME)
|
||||
}
|
||||
|
||||
pub fn geo_field(&self) -> Option<&'doc RawValue> {
|
||||
self.data.get("_geo")
|
||||
}
|
||||
|
||||
pub fn len(&self) -> usize {
|
||||
self.data.len()
|
||||
}
|
||||
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.data.is_empty()
|
||||
}
|
||||
pub fn top_level_field(&self, k: &str) -> Option<&'doc RawValue> {
|
||||
self.data.get(k)
|
||||
}
|
||||
}
|
191
crates/milli/src/update/new/document_change.rs
Normal file
191
crates/milli/src/update/new/document_change.rs
Normal file
|
@ -0,0 +1,191 @@
|
|||
use bumpalo::Bump;
|
||||
use heed::RoTxn;
|
||||
|
||||
use super::document::{DocumentFromDb, DocumentFromVersions, MergedDocument, Versions};
|
||||
use super::vector_document::{
|
||||
MergedVectorDocument, VectorDocumentFromDb, VectorDocumentFromVersions,
|
||||
};
|
||||
use crate::documents::FieldIdMapper;
|
||||
use crate::vector::EmbeddingConfigs;
|
||||
use crate::{DocumentId, Index, Result};
|
||||
|
||||
pub enum DocumentChange<'doc> {
|
||||
Deletion(Deletion<'doc>),
|
||||
Update(Update<'doc>),
|
||||
Insertion(Insertion<'doc>),
|
||||
}
|
||||
|
||||
pub struct Deletion<'doc> {
|
||||
docid: DocumentId,
|
||||
external_document_id: &'doc str,
|
||||
}
|
||||
|
||||
pub struct Update<'doc> {
|
||||
docid: DocumentId,
|
||||
external_document_id: &'doc str,
|
||||
new: Versions<'doc>,
|
||||
has_deletion: bool,
|
||||
}
|
||||
|
||||
pub struct Insertion<'doc> {
|
||||
docid: DocumentId,
|
||||
external_document_id: &'doc str,
|
||||
new: Versions<'doc>,
|
||||
}
|
||||
|
||||
impl<'doc> DocumentChange<'doc> {
|
||||
pub fn docid(&self) -> DocumentId {
|
||||
match &self {
|
||||
Self::Deletion(inner) => inner.docid(),
|
||||
Self::Update(inner) => inner.docid(),
|
||||
Self::Insertion(inner) => inner.docid(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn external_docid(&self) -> &'doc str {
|
||||
match self {
|
||||
DocumentChange::Deletion(deletion) => deletion.external_document_id(),
|
||||
DocumentChange::Update(update) => update.external_document_id(),
|
||||
DocumentChange::Insertion(insertion) => insertion.external_document_id(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<'doc> Deletion<'doc> {
|
||||
pub fn create(docid: DocumentId, external_document_id: &'doc str) -> Self {
|
||||
Self { docid, external_document_id }
|
||||
}
|
||||
|
||||
pub fn docid(&self) -> DocumentId {
|
||||
self.docid
|
||||
}
|
||||
|
||||
pub fn external_document_id(&self) -> &'doc str {
|
||||
self.external_document_id
|
||||
}
|
||||
|
||||
pub fn current<'a, Mapper: FieldIdMapper>(
|
||||
&self,
|
||||
rtxn: &'a RoTxn,
|
||||
index: &'a Index,
|
||||
mapper: &'a Mapper,
|
||||
) -> Result<DocumentFromDb<'a, Mapper>> {
|
||||
Ok(DocumentFromDb::new(self.docid, rtxn, index, mapper)?.ok_or(
|
||||
crate::error::UserError::UnknownInternalDocumentId { document_id: self.docid },
|
||||
)?)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'doc> Insertion<'doc> {
|
||||
pub fn create(docid: DocumentId, external_document_id: &'doc str, new: Versions<'doc>) -> Self {
|
||||
Insertion { docid, external_document_id, new }
|
||||
}
|
||||
|
||||
pub fn docid(&self) -> DocumentId {
|
||||
self.docid
|
||||
}
|
||||
|
||||
pub fn external_document_id(&self) -> &'doc str {
|
||||
self.external_document_id
|
||||
}
|
||||
pub fn inserted(&self) -> DocumentFromVersions<'_, 'doc> {
|
||||
DocumentFromVersions::new(&self.new)
|
||||
}
|
||||
|
||||
pub fn inserted_vectors(
|
||||
&self,
|
||||
doc_alloc: &'doc Bump,
|
||||
embedders: &'doc EmbeddingConfigs,
|
||||
) -> Result<Option<VectorDocumentFromVersions<'doc>>> {
|
||||
VectorDocumentFromVersions::new(&self.new, doc_alloc, embedders)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'doc> Update<'doc> {
|
||||
pub fn create(
|
||||
docid: DocumentId,
|
||||
external_document_id: &'doc str,
|
||||
new: Versions<'doc>,
|
||||
has_deletion: bool,
|
||||
) -> Self {
|
||||
Update { docid, new, external_document_id, has_deletion }
|
||||
}
|
||||
|
||||
pub fn docid(&self) -> DocumentId {
|
||||
self.docid
|
||||
}
|
||||
|
||||
pub fn external_document_id(&self) -> &'doc str {
|
||||
self.external_document_id
|
||||
}
|
||||
pub fn current<'a, Mapper: FieldIdMapper>(
|
||||
&self,
|
||||
rtxn: &'a RoTxn,
|
||||
index: &'a Index,
|
||||
mapper: &'a Mapper,
|
||||
) -> Result<DocumentFromDb<'a, Mapper>> {
|
||||
Ok(DocumentFromDb::new(self.docid, rtxn, index, mapper)?.ok_or(
|
||||
crate::error::UserError::UnknownInternalDocumentId { document_id: self.docid },
|
||||
)?)
|
||||
}
|
||||
|
||||
pub fn current_vectors<'a, Mapper: FieldIdMapper>(
|
||||
&self,
|
||||
rtxn: &'a RoTxn,
|
||||
index: &'a Index,
|
||||
mapper: &'a Mapper,
|
||||
doc_alloc: &'a Bump,
|
||||
) -> Result<VectorDocumentFromDb<'a>> {
|
||||
Ok(VectorDocumentFromDb::new(self.docid, index, rtxn, mapper, doc_alloc)?.ok_or(
|
||||
crate::error::UserError::UnknownInternalDocumentId { document_id: self.docid },
|
||||
)?)
|
||||
}
|
||||
|
||||
pub fn updated(&self) -> DocumentFromVersions<'_, 'doc> {
|
||||
DocumentFromVersions::new(&self.new)
|
||||
}
|
||||
|
||||
pub fn merged<'t, Mapper: FieldIdMapper>(
|
||||
&self,
|
||||
rtxn: &'t RoTxn,
|
||||
index: &'t Index,
|
||||
mapper: &'t Mapper,
|
||||
) -> Result<MergedDocument<'_, 'doc, 't, Mapper>> {
|
||||
if self.has_deletion {
|
||||
Ok(MergedDocument::without_db(DocumentFromVersions::new(&self.new)))
|
||||
} else {
|
||||
MergedDocument::with_db(
|
||||
self.docid,
|
||||
rtxn,
|
||||
index,
|
||||
mapper,
|
||||
DocumentFromVersions::new(&self.new),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
pub fn updated_vectors(
|
||||
&self,
|
||||
doc_alloc: &'doc Bump,
|
||||
embedders: &'doc EmbeddingConfigs,
|
||||
) -> Result<Option<VectorDocumentFromVersions<'doc>>> {
|
||||
VectorDocumentFromVersions::new(&self.new, doc_alloc, embedders)
|
||||
}
|
||||
|
||||
pub fn merged_vectors<Mapper: FieldIdMapper>(
|
||||
&self,
|
||||
rtxn: &'doc RoTxn,
|
||||
index: &'doc Index,
|
||||
mapper: &'doc Mapper,
|
||||
doc_alloc: &'doc Bump,
|
||||
embedders: &'doc EmbeddingConfigs,
|
||||
) -> Result<Option<MergedVectorDocument<'doc>>> {
|
||||
if self.has_deletion {
|
||||
MergedVectorDocument::without_db(&self.new, doc_alloc, embedders)
|
||||
} else {
|
||||
MergedVectorDocument::with_db(
|
||||
self.docid, index, rtxn, mapper, &self.new, doc_alloc, embedders,
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
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();
|
||||
}
|
||||
}
|
264
crates/milli/src/update/new/facet_search_builder.rs
Normal file
264
crates/milli/src/update/new/facet_search_builder.rs
Normal file
|
@ -0,0 +1,264 @@
|
|||
use std::collections::{BTreeSet, HashMap};
|
||||
|
||||
use charabia::normalizer::NormalizerOption;
|
||||
use charabia::{Language, Normalize, StrDetection, Token};
|
||||
use grenad::Sorter;
|
||||
use heed::types::{Bytes, SerdeJson};
|
||||
use heed::{BytesDecode, BytesEncode, RoTxn, RwTxn};
|
||||
|
||||
use super::extract::FacetKind;
|
||||
use super::fst_merger_builder::FstMergerBuilder;
|
||||
use super::KvReaderDelAdd;
|
||||
use crate::heed_codec::facet::{FacetGroupKey, FacetGroupKeyCodec};
|
||||
use crate::heed_codec::StrRefCodec;
|
||||
use crate::update::del_add::{DelAdd, KvWriterDelAdd};
|
||||
use crate::update::{create_sorter, MergeDeladdBtreesetString};
|
||||
use crate::{
|
||||
BEU16StrCodec, FieldId, GlobalFieldsIdsMap, Index, LocalizedAttributesRule, Result,
|
||||
MAX_FACET_VALUE_LENGTH,
|
||||
};
|
||||
|
||||
pub struct FacetSearchBuilder<'indexer> {
|
||||
registered_facets: HashMap<FieldId, usize>,
|
||||
normalized_facet_string_docids_sorter: Sorter<MergeDeladdBtreesetString>,
|
||||
global_fields_ids_map: GlobalFieldsIdsMap<'indexer>,
|
||||
localized_attributes_rules: Vec<LocalizedAttributesRule>,
|
||||
// Buffered data below
|
||||
buffer: Vec<u8>,
|
||||
localized_field_ids: HashMap<FieldId, Option<Vec<Language>>>,
|
||||
}
|
||||
|
||||
impl<'indexer> FacetSearchBuilder<'indexer> {
|
||||
pub fn new(
|
||||
global_fields_ids_map: GlobalFieldsIdsMap<'indexer>,
|
||||
localized_attributes_rules: Vec<LocalizedAttributesRule>,
|
||||
) -> Self {
|
||||
let registered_facets = HashMap::new();
|
||||
let normalized_facet_string_docids_sorter = create_sorter(
|
||||
grenad::SortAlgorithm::Stable,
|
||||
MergeDeladdBtreesetString,
|
||||
grenad::CompressionType::None,
|
||||
None,
|
||||
None,
|
||||
Some(0),
|
||||
true,
|
||||
);
|
||||
|
||||
Self {
|
||||
registered_facets,
|
||||
normalized_facet_string_docids_sorter,
|
||||
buffer: Vec::new(),
|
||||
global_fields_ids_map,
|
||||
localized_attributes_rules,
|
||||
localized_field_ids: HashMap::new(),
|
||||
}
|
||||
}
|
||||
|
||||
fn extract_key_data<'k>(&self, key: &'k [u8]) -> Result<Option<FacetGroupKey<&'k str>>> {
|
||||
match FacetKind::from(key[0]) {
|
||||
// Only strings are searchable
|
||||
FacetKind::String => Ok(Some(
|
||||
FacetGroupKeyCodec::<StrRefCodec>::bytes_decode(&key[1..])
|
||||
.map_err(heed::Error::Encoding)?,
|
||||
)),
|
||||
_ => Ok(None),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn register_from_key(&mut self, deladd: DelAdd, facet_key: &[u8]) -> Result<()> {
|
||||
let Some(FacetGroupKey { field_id, level: _level, left_bound }) =
|
||||
self.extract_key_data(facet_key)?
|
||||
else {
|
||||
return Ok(());
|
||||
};
|
||||
|
||||
if deladd == DelAdd::Addition {
|
||||
self.registered_facets.entry(field_id).and_modify(|count| *count += 1).or_insert(1);
|
||||
}
|
||||
|
||||
let locales = self.locales(field_id);
|
||||
let hyper_normalized_value = normalize_facet_string(left_bound, locales);
|
||||
|
||||
let set = BTreeSet::from_iter(std::iter::once(left_bound));
|
||||
|
||||
// as the facet string is the same, we can put the deletion and addition in the same obkv.
|
||||
self.buffer.clear();
|
||||
let mut obkv = KvWriterDelAdd::new(&mut self.buffer);
|
||||
let val = SerdeJson::bytes_encode(&set).map_err(heed::Error::Encoding)?;
|
||||
obkv.insert(deladd, val)?;
|
||||
obkv.finish()?;
|
||||
|
||||
let key: (u16, &str) = (field_id, hyper_normalized_value.as_ref());
|
||||
let key_bytes = BEU16StrCodec::bytes_encode(&key).map_err(heed::Error::Encoding)?;
|
||||
self.normalized_facet_string_docids_sorter.insert(key_bytes, &self.buffer)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn locales(&mut self, field_id: FieldId) -> Option<&[Language]> {
|
||||
if !self.localized_field_ids.contains_key(&field_id) {
|
||||
let Some(field_name) = self.global_fields_ids_map.name(field_id) else {
|
||||
unreachable!("Field id {} not found in the global fields ids map", field_id);
|
||||
};
|
||||
|
||||
let locales = self
|
||||
.localized_attributes_rules
|
||||
.iter()
|
||||
.find(|rule| rule.match_str(field_name))
|
||||
.map(|rule| rule.locales.clone());
|
||||
|
||||
self.localized_field_ids.insert(field_id, locales);
|
||||
}
|
||||
|
||||
self.localized_field_ids.get(&field_id).unwrap().as_deref()
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::facet_fst")]
|
||||
pub fn merge_and_write(self, index: &Index, wtxn: &mut RwTxn, rtxn: &RoTxn) -> Result<()> {
|
||||
let reader = self.normalized_facet_string_docids_sorter.into_reader_cursors()?;
|
||||
let mut builder = grenad::MergerBuilder::new(MergeDeladdBtreesetString);
|
||||
builder.extend(reader);
|
||||
|
||||
let database = index.facet_id_normalized_string_strings.remap_types::<Bytes, Bytes>();
|
||||
|
||||
let mut merger_iter = builder.build().into_stream_merger_iter()?;
|
||||
let mut current_field_id = None;
|
||||
let mut fst;
|
||||
let mut fst_merger_builder: Option<FstMergerBuilder> = None;
|
||||
while let Some((key, deladd)) = merger_iter.next()? {
|
||||
let (field_id, normalized_facet_string) =
|
||||
BEU16StrCodec::bytes_decode(key).map_err(heed::Error::Encoding)?;
|
||||
|
||||
if current_field_id != Some(field_id) {
|
||||
if let Some(fst_merger_builder) = fst_merger_builder {
|
||||
let mmap = fst_merger_builder.build(&mut callback)?;
|
||||
index
|
||||
.facet_id_string_fst
|
||||
.remap_data_type::<Bytes>()
|
||||
.put(wtxn, &field_id, &mmap)?;
|
||||
}
|
||||
|
||||
fst = index.facet_id_string_fst.get(rtxn, &field_id)?;
|
||||
fst_merger_builder = Some(FstMergerBuilder::new(fst.as_ref())?);
|
||||
current_field_id = Some(field_id);
|
||||
}
|
||||
|
||||
let previous = database.get(rtxn, key)?;
|
||||
let deladd: &KvReaderDelAdd = deladd.into();
|
||||
let del = deladd.get(DelAdd::Deletion);
|
||||
let add = deladd.get(DelAdd::Addition);
|
||||
|
||||
match merge_btreesets(previous, del, add)? {
|
||||
Operation::Write(value) => {
|
||||
match fst_merger_builder.as_mut() {
|
||||
Some(fst_merger_builder) => {
|
||||
fst_merger_builder.register(
|
||||
DelAdd::Addition,
|
||||
normalized_facet_string.as_bytes(),
|
||||
&mut callback,
|
||||
)?;
|
||||
}
|
||||
None => unreachable!(),
|
||||
}
|
||||
let key = (field_id, normalized_facet_string);
|
||||
let key_bytes =
|
||||
BEU16StrCodec::bytes_encode(&key).map_err(heed::Error::Encoding)?;
|
||||
database.put(wtxn, &key_bytes, &value)?;
|
||||
}
|
||||
Operation::Delete => {
|
||||
match fst_merger_builder.as_mut() {
|
||||
Some(fst_merger_builder) => {
|
||||
fst_merger_builder.register(
|
||||
DelAdd::Deletion,
|
||||
normalized_facet_string.as_bytes(),
|
||||
&mut callback,
|
||||
)?;
|
||||
}
|
||||
None => unreachable!(),
|
||||
}
|
||||
let key = (field_id, normalized_facet_string);
|
||||
let key_bytes =
|
||||
BEU16StrCodec::bytes_encode(&key).map_err(heed::Error::Encoding)?;
|
||||
database.delete(wtxn, &key_bytes)?;
|
||||
}
|
||||
Operation::Ignore => (),
|
||||
}
|
||||
}
|
||||
|
||||
if let (Some(field_id), Some(fst_merger_builder)) = (current_field_id, fst_merger_builder) {
|
||||
let mmap = fst_merger_builder.build(&mut callback)?;
|
||||
index.facet_id_string_fst.remap_data_type::<Bytes>().put(wtxn, &field_id, &mmap)?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn callback(_bytes: &[u8], _deladd: DelAdd, _is_modified: bool) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn merge_btreesets(
|
||||
current: Option<&[u8]>,
|
||||
del: Option<&[u8]>,
|
||||
add: Option<&[u8]>,
|
||||
) -> Result<Operation> {
|
||||
let mut result: BTreeSet<String> = match current {
|
||||
Some(current) => SerdeJson::bytes_decode(current).map_err(heed::Error::Encoding)?,
|
||||
None => BTreeSet::new(),
|
||||
};
|
||||
if let Some(del) = del {
|
||||
let del: BTreeSet<String> = SerdeJson::bytes_decode(del).map_err(heed::Error::Encoding)?;
|
||||
result = result.difference(&del).cloned().collect();
|
||||
}
|
||||
if let Some(add) = add {
|
||||
let add: BTreeSet<String> = SerdeJson::bytes_decode(add).map_err(heed::Error::Encoding)?;
|
||||
result.extend(add);
|
||||
}
|
||||
|
||||
/// TODO remove allocation
|
||||
let result = SerdeJson::bytes_encode(&result).map_err(heed::Error::Encoding)?.into_owned();
|
||||
if Some(result.as_ref()) == current {
|
||||
Ok(Operation::Ignore)
|
||||
} else if result.is_empty() {
|
||||
Ok(Operation::Delete)
|
||||
} else {
|
||||
Ok(Operation::Write(result))
|
||||
}
|
||||
}
|
||||
|
||||
/// Normalizes the facet string and truncates it to the max length.
|
||||
fn normalize_facet_string(facet_string: &str, locales: Option<&[Language]>) -> String {
|
||||
let options: NormalizerOption = NormalizerOption { lossy: true, ..Default::default() };
|
||||
let mut detection = StrDetection::new(facet_string, locales);
|
||||
|
||||
let script = detection.script();
|
||||
// Detect the language of the facet string only if several locales are explicitly provided.
|
||||
let language = match locales {
|
||||
Some(&[language]) => Some(language),
|
||||
Some(multiple_locales) if multiple_locales.len() > 1 => detection.language(),
|
||||
_ => None,
|
||||
};
|
||||
|
||||
let token = Token {
|
||||
lemma: std::borrow::Cow::Borrowed(facet_string),
|
||||
script,
|
||||
language,
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
// truncate the facet string to the max length
|
||||
token
|
||||
.normalize(&options)
|
||||
.lemma
|
||||
.char_indices()
|
||||
.take_while(|(idx, _)| *idx < MAX_FACET_VALUE_LENGTH)
|
||||
.map(|(_, c)| c)
|
||||
.collect()
|
||||
}
|
||||
|
||||
enum Operation {
|
||||
Write(Vec<u8>),
|
||||
Delete,
|
||||
Ignore,
|
||||
}
|
155
crates/milli/src/update/new/fst_merger_builder.rs
Normal file
155
crates/milli/src/update/new/fst_merger_builder.rs
Normal file
|
@ -0,0 +1,155 @@
|
|||
use std::{fs::File, io::BufWriter};
|
||||
|
||||
use fst::{Set, SetBuilder, Streamer};
|
||||
use memmap2::Mmap;
|
||||
use tempfile::tempfile;
|
||||
|
||||
use crate::{update::del_add::DelAdd, InternalError, Result};
|
||||
|
||||
pub struct FstMergerBuilder<'a> {
|
||||
stream: Option<fst::set::Stream<'a>>,
|
||||
fst_builder: SetBuilder<BufWriter<File>>,
|
||||
last: Option<Vec<u8>>,
|
||||
inserted_words: usize,
|
||||
}
|
||||
|
||||
impl<'a> FstMergerBuilder<'a> {
|
||||
pub fn new<D: AsRef<[u8]>>(fst: Option<&'a Set<D>>) -> Result<Self> {
|
||||
Ok(Self {
|
||||
stream: fst.map(|fst| fst.stream()),
|
||||
fst_builder: SetBuilder::new(BufWriter::new(tempfile()?))?,
|
||||
last: None,
|
||||
inserted_words: 0,
|
||||
})
|
||||
}
|
||||
|
||||
pub fn register(
|
||||
&mut self,
|
||||
deladd: DelAdd,
|
||||
right: &[u8],
|
||||
insertion_callback: &mut impl FnMut(&[u8], DelAdd, bool) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
if let Some(left) = self.last.take() {
|
||||
let (left_inserted, right_inserted) =
|
||||
self.compare_and_insert(deladd, left.as_slice(), right, insertion_callback)?;
|
||||
|
||||
// left was not inserted, so we keep it for the next iteration
|
||||
if !left_inserted {
|
||||
self.last = Some(left);
|
||||
}
|
||||
|
||||
// right was inserted, so we can stop
|
||||
if right_inserted {
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(mut stream) = self.stream.take() {
|
||||
while let Some(left) = stream.next() {
|
||||
let (left_inserted, right_inserted) =
|
||||
self.compare_and_insert(deladd, left, right, insertion_callback)?;
|
||||
|
||||
// left was not inserted, so we keep it for the next iteration
|
||||
if !left_inserted {
|
||||
self.last = Some(left.to_vec());
|
||||
}
|
||||
|
||||
// right was inserted, so we can stop
|
||||
if right_inserted {
|
||||
self.stream = Some(stream);
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If we reach this point, it means that the stream is empty
|
||||
// and we need to insert the incoming word
|
||||
self.insert(right, deladd, true, insertion_callback)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn compare_and_insert(
|
||||
&mut self,
|
||||
deladd: DelAdd,
|
||||
left: &[u8],
|
||||
right: &[u8],
|
||||
insertion_callback: &mut impl FnMut(&[u8], DelAdd, bool) -> Result<()>,
|
||||
) -> Result<(bool, bool)> {
|
||||
let mut left_inserted = false;
|
||||
let mut right_inserted = false;
|
||||
match left.cmp(right) {
|
||||
std::cmp::Ordering::Less => {
|
||||
// We need to insert the last word from the current fst
|
||||
self.insert(left, DelAdd::Addition, false, insertion_callback)?;
|
||||
|
||||
left_inserted = true;
|
||||
}
|
||||
std::cmp::Ordering::Equal => {
|
||||
self.insert(right, deladd, true, insertion_callback)?;
|
||||
|
||||
left_inserted = true;
|
||||
right_inserted = true;
|
||||
}
|
||||
std::cmp::Ordering::Greater => {
|
||||
self.insert(right, deladd, true, insertion_callback)?;
|
||||
|
||||
right_inserted = true;
|
||||
}
|
||||
}
|
||||
|
||||
Ok((left_inserted, right_inserted))
|
||||
}
|
||||
|
||||
fn insert(
|
||||
&mut self,
|
||||
bytes: &[u8],
|
||||
deladd: DelAdd,
|
||||
is_modified: bool,
|
||||
insertion_callback: &mut impl FnMut(&[u8], DelAdd, bool) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
// Addition: We insert the word
|
||||
// Deletion: We delete the word by not inserting it
|
||||
if deladd == DelAdd::Addition {
|
||||
self.inserted_words += 1;
|
||||
self.fst_builder.insert(bytes)?;
|
||||
}
|
||||
|
||||
insertion_callback(bytes, deladd, is_modified)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn drain_stream(
|
||||
&mut self,
|
||||
insertion_callback: &mut impl FnMut(&[u8], DelAdd, bool) -> Result<()>,
|
||||
) -> Result<()> {
|
||||
if let Some(last) = self.last.take() {
|
||||
self.insert(last.as_slice(), DelAdd::Addition, false, insertion_callback)?;
|
||||
}
|
||||
|
||||
if let Some(mut stream) = self.stream.take() {
|
||||
while let Some(current) = stream.next() {
|
||||
self.insert(current, DelAdd::Addition, false, insertion_callback)?;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn build(
|
||||
mut self,
|
||||
insertion_callback: &mut impl FnMut(&[u8], DelAdd, bool) -> Result<()>,
|
||||
) -> Result<Mmap> {
|
||||
self.drain_stream(insertion_callback)?;
|
||||
|
||||
let fst_file = self
|
||||
.fst_builder
|
||||
.into_inner()?
|
||||
.into_inner()
|
||||
.map_err(|_| InternalError::IndexingMergingKeys { process: "building-fst" })?;
|
||||
let fst_mmap = unsafe { Mmap::map(&fst_file)? };
|
||||
|
||||
Ok(fst_mmap)
|
||||
}
|
||||
}
|
619
crates/milli/src/update/new/indexer/de.rs
Normal file
619
crates/milli/src/update/new/indexer/de.rs
Normal file
|
@ -0,0 +1,619 @@
|
|||
use std::ops::ControlFlow;
|
||||
|
||||
use bumpalo::Bump;
|
||||
use serde::de::{DeserializeSeed, Deserializer as _, Visitor};
|
||||
use serde_json::value::RawValue;
|
||||
|
||||
use crate::documents::{
|
||||
validate_document_id_str, DocumentIdExtractionError, FieldIdMapper, PrimaryKey,
|
||||
};
|
||||
use crate::fields_ids_map::MutFieldIdMapper;
|
||||
use crate::{FieldId, UserError};
|
||||
|
||||
// visits a document to fill the top level fields of the field id map and retrieve the external document id.
|
||||
pub struct FieldAndDocidExtractor<'p, 'indexer, Mapper: MutFieldIdMapper> {
|
||||
fields_ids_map: &'p mut Mapper,
|
||||
primary_key: &'p PrimaryKey<'p>,
|
||||
indexer: &'indexer Bump,
|
||||
}
|
||||
|
||||
impl<'p, 'indexer, Mapper: MutFieldIdMapper> FieldAndDocidExtractor<'p, 'indexer, Mapper> {
|
||||
pub fn new(
|
||||
fields_ids_map: &'p mut Mapper,
|
||||
primary_key: &'p PrimaryKey<'p>,
|
||||
indexer: &'indexer Bump,
|
||||
) -> Self {
|
||||
Self { fields_ids_map, primary_key, indexer }
|
||||
}
|
||||
}
|
||||
|
||||
impl<'de, 'p, 'indexer: 'de, Mapper: MutFieldIdMapper> Visitor<'de>
|
||||
for FieldAndDocidExtractor<'p, 'indexer, Mapper>
|
||||
{
|
||||
type Value =
|
||||
Result<Result<DeOrBumpStr<'de, 'indexer>, DocumentIdExtractionError>, crate::UserError>;
|
||||
|
||||
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
write!(formatter, "a map")
|
||||
}
|
||||
|
||||
fn visit_map<A>(mut self, mut map: A) -> Result<Self::Value, A::Error>
|
||||
where
|
||||
A: serde::de::MapAccess<'de>,
|
||||
{
|
||||
let mut docid = None;
|
||||
|
||||
while let Some(((level_name, right), (fid, fields_ids_map))) =
|
||||
map.next_key_seed(ComponentsSeed {
|
||||
name: self.primary_key.name(),
|
||||
visitor: MutFieldIdMapVisitor(self.fields_ids_map),
|
||||
})?
|
||||
{
|
||||
let Some(_fid) = fid else {
|
||||
return Ok(Err(crate::UserError::AttributeLimitReached));
|
||||
};
|
||||
self.fields_ids_map = fields_ids_map;
|
||||
|
||||
let value: &'de RawValue = map.next_value()?;
|
||||
|
||||
match match_component(level_name, right, value, self.indexer, &mut docid) {
|
||||
ControlFlow::Continue(()) => continue,
|
||||
ControlFlow::Break(Err(err)) => return Err(serde::de::Error::custom(err)),
|
||||
ControlFlow::Break(Ok(err)) => return Ok(Ok(Err(err))),
|
||||
}
|
||||
}
|
||||
|
||||
Ok(Ok(match docid {
|
||||
Some(docid) => Ok(docid),
|
||||
None => Err(DocumentIdExtractionError::MissingDocumentId),
|
||||
}))
|
||||
}
|
||||
}
|
||||
|
||||
struct NestedPrimaryKeyVisitor<'a, 'bump> {
|
||||
components: &'a str,
|
||||
bump: &'bump Bump,
|
||||
}
|
||||
|
||||
impl<'de, 'a, 'bump: 'de> Visitor<'de> for NestedPrimaryKeyVisitor<'a, 'bump> {
|
||||
type Value = std::result::Result<Option<DeOrBumpStr<'de, 'bump>>, DocumentIdExtractionError>;
|
||||
|
||||
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
write!(formatter, "a map")
|
||||
}
|
||||
|
||||
fn visit_map<A>(self, mut map: A) -> std::result::Result<Self::Value, A::Error>
|
||||
where
|
||||
A: serde::de::MapAccess<'de>,
|
||||
{
|
||||
let mut docid = None;
|
||||
while let Some(((matched_component, right), _)) = map.next_key_seed(ComponentsSeed {
|
||||
name: self.components,
|
||||
visitor: serde::de::IgnoredAny,
|
||||
})? {
|
||||
let value: &'de RawValue = map.next_value()?;
|
||||
|
||||
match match_component(matched_component, right, value, self.bump, &mut docid) {
|
||||
ControlFlow::Continue(()) => continue,
|
||||
ControlFlow::Break(Err(err)) => return Err(serde::de::Error::custom(err)),
|
||||
ControlFlow::Break(Ok(err)) => return Ok(Err(err)),
|
||||
}
|
||||
}
|
||||
Ok(Ok(docid))
|
||||
}
|
||||
}
|
||||
|
||||
/// Either a `&'de str` or a `&'bump str`.
|
||||
pub enum DeOrBumpStr<'de, 'bump: 'de> {
|
||||
/// Lifetime of the deserializer
|
||||
De(&'de str),
|
||||
/// Lifetime of the allocator
|
||||
Bump(&'bump str),
|
||||
}
|
||||
|
||||
impl<'de, 'bump: 'de> DeOrBumpStr<'de, 'bump> {
|
||||
/// Returns a `&'bump str`, possibly allocating to extend its lifetime.
|
||||
pub fn to_bump(&self, bump: &'bump Bump) -> &'bump str {
|
||||
match self {
|
||||
DeOrBumpStr::De(de) => bump.alloc_str(de),
|
||||
DeOrBumpStr::Bump(bump) => bump,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a `&'de str`.
|
||||
///
|
||||
/// This function never allocates because `'bump: 'de`.
|
||||
pub fn to_de(&self) -> &'de str {
|
||||
match self {
|
||||
DeOrBumpStr::De(de) => de,
|
||||
DeOrBumpStr::Bump(bump) => bump,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct ComponentsSeed<'a, V> {
|
||||
name: &'a str,
|
||||
visitor: V,
|
||||
}
|
||||
|
||||
impl<'de, 'a, V: Visitor<'de>> DeserializeSeed<'de> for ComponentsSeed<'a, V> {
|
||||
type Value = ((&'a str, &'a str), V::Value);
|
||||
|
||||
fn deserialize<D>(self, deserializer: D) -> Result<Self::Value, D::Error>
|
||||
where
|
||||
D: serde::Deserializer<'de>,
|
||||
{
|
||||
struct ComponentsSeedVisitor<'a, V> {
|
||||
name: &'a str,
|
||||
visitor: V,
|
||||
}
|
||||
|
||||
impl<'a, V> ComponentsSeedVisitor<'a, V> {
|
||||
fn match_str(&self, v: &str) -> (&'a str, &'a str) {
|
||||
let p = PrimaryKey::Nested { name: self.name };
|
||||
for (name, right) in p.possible_level_names() {
|
||||
if name == v {
|
||||
return (name, right);
|
||||
}
|
||||
}
|
||||
("", self.name)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'de, 'a, V: Visitor<'de>> Visitor<'de> for ComponentsSeedVisitor<'a, V> {
|
||||
type Value = ((&'a str, &'a str), V::Value);
|
||||
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
write!(formatter, "expecting a string")
|
||||
}
|
||||
fn visit_borrowed_str<E>(self, v: &'de str) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
let matched = self.match_str(v);
|
||||
let inner = self.visitor.visit_borrowed_str(v)?;
|
||||
Ok((matched, inner))
|
||||
}
|
||||
|
||||
fn visit_str<E>(self, v: &str) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
let matched = self.match_str(v);
|
||||
let inner = self.visitor.visit_str(v)?;
|
||||
|
||||
Ok((matched, inner))
|
||||
}
|
||||
}
|
||||
deserializer
|
||||
.deserialize_str(ComponentsSeedVisitor { name: self.name, visitor: self.visitor })
|
||||
}
|
||||
}
|
||||
|
||||
struct MutFieldIdMapVisitor<'a, Mapper: MutFieldIdMapper>(&'a mut Mapper);
|
||||
|
||||
impl<'de, 'a, Mapper: MutFieldIdMapper> Visitor<'de> for MutFieldIdMapVisitor<'a, Mapper> {
|
||||
type Value = (Option<FieldId>, &'a mut Mapper);
|
||||
|
||||
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
write!(formatter, "expecting a string")
|
||||
}
|
||||
fn visit_borrowed_str<E>(self, v: &'de str) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok((self.0.insert(v), self.0))
|
||||
}
|
||||
|
||||
fn visit_str<E>(self, v: &str) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok((self.0.insert(v), self.0))
|
||||
}
|
||||
}
|
||||
|
||||
pub struct FieldIdMapVisitor<'a, Mapper: FieldIdMapper>(pub &'a Mapper);
|
||||
|
||||
impl<'de, 'a, Mapper: FieldIdMapper> Visitor<'de> for FieldIdMapVisitor<'a, Mapper> {
|
||||
type Value = Option<FieldId>;
|
||||
|
||||
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
write!(formatter, "expecting a string")
|
||||
}
|
||||
fn visit_borrowed_str<E>(self, v: &'de str) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(self.0.id(v))
|
||||
}
|
||||
|
||||
fn visit_str<E>(self, v: &str) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(self.0.id(v))
|
||||
}
|
||||
}
|
||||
pub struct DocumentIdVisitor<'indexer>(pub &'indexer Bump);
|
||||
|
||||
impl<'de, 'indexer: 'de> Visitor<'de> for DocumentIdVisitor<'indexer> {
|
||||
type Value = std::result::Result<DeOrBumpStr<'de, 'indexer>, DocumentIdExtractionError>;
|
||||
|
||||
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
write!(formatter, "an integer or a string")
|
||||
}
|
||||
|
||||
fn visit_borrowed_str<E>(self, v: &'de str) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(validate_document_id_str(v)
|
||||
.ok_or_else(|| {
|
||||
DocumentIdExtractionError::InvalidDocumentId(UserError::InvalidDocumentId {
|
||||
document_id: serde_json::Value::String(v.to_owned()),
|
||||
})
|
||||
})
|
||||
.map(DeOrBumpStr::De))
|
||||
}
|
||||
|
||||
fn visit_str<E>(self, v: &str) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
let v = self.0.alloc_str(v);
|
||||
Ok(match self.visit_borrowed_str(v)? {
|
||||
Ok(_) => Ok(DeOrBumpStr::Bump(v)),
|
||||
Err(err) => Err(err),
|
||||
})
|
||||
}
|
||||
|
||||
fn visit_u64<E>(self, v: u64) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
use std::fmt::Write as _;
|
||||
|
||||
let mut out = bumpalo::collections::String::new_in(self.0);
|
||||
write!(&mut out, "{v}").unwrap();
|
||||
Ok(Ok(DeOrBumpStr::Bump(out.into_bump_str())))
|
||||
}
|
||||
|
||||
fn visit_i64<E>(self, v: i64) -> std::result::Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
use std::fmt::Write as _;
|
||||
|
||||
let mut out = bumpalo::collections::String::new_in(self.0);
|
||||
write!(&mut out, "{v}").unwrap();
|
||||
Ok(Ok(DeOrBumpStr::Bump(out.into_bump_str())))
|
||||
}
|
||||
}
|
||||
|
||||
pub fn match_component<'de, 'indexer: 'de>(
|
||||
first_level_name: &str,
|
||||
right: &str,
|
||||
value: &'de RawValue,
|
||||
bump: &'indexer Bump,
|
||||
docid: &mut Option<DeOrBumpStr<'de, 'indexer>>,
|
||||
) -> ControlFlow<Result<DocumentIdExtractionError, serde_json::Error>, ()> {
|
||||
if first_level_name.is_empty() {
|
||||
return ControlFlow::Continue(());
|
||||
}
|
||||
|
||||
let value = if right.is_empty() {
|
||||
match value.deserialize_any(DocumentIdVisitor(bump)).map_err(|_err| {
|
||||
DocumentIdExtractionError::InvalidDocumentId(UserError::InvalidDocumentId {
|
||||
document_id: serde_json::to_value(value).unwrap(),
|
||||
})
|
||||
}) {
|
||||
Ok(Ok(value)) => value,
|
||||
Ok(Err(err)) | Err(err) => return ControlFlow::Break(Ok(err)),
|
||||
}
|
||||
} else {
|
||||
// if right is not empty, recursively extract right components from value
|
||||
let res = value.deserialize_map(NestedPrimaryKeyVisitor { components: right, bump });
|
||||
match res {
|
||||
Ok(Ok(Some(value))) => value,
|
||||
Ok(Ok(None)) => return ControlFlow::Continue(()),
|
||||
Ok(Err(err)) => return ControlFlow::Break(Ok(err)),
|
||||
Err(err) if err.is_data() => return ControlFlow::Continue(()), // we expected the field to be a map, but it was not and that's OK.
|
||||
Err(err) => return ControlFlow::Break(Err(err)),
|
||||
}
|
||||
};
|
||||
if let Some(_previous_value) = docid.replace(value) {
|
||||
return ControlFlow::Break(Ok(DocumentIdExtractionError::TooManyDocumentIds(2)));
|
||||
}
|
||||
ControlFlow::Continue(())
|
||||
}
|
||||
|
||||
pub struct DeserrRawValue<'a> {
|
||||
value: &'a RawValue,
|
||||
alloc: &'a Bump,
|
||||
}
|
||||
|
||||
impl<'a> DeserrRawValue<'a> {
|
||||
pub fn new_in(value: &'a RawValue, alloc: &'a Bump) -> Self {
|
||||
Self { value, alloc }
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DeserrRawVec<'a> {
|
||||
vec: raw_collections::RawVec<'a>,
|
||||
alloc: &'a Bump,
|
||||
}
|
||||
|
||||
impl<'a> deserr::Sequence for DeserrRawVec<'a> {
|
||||
type Value = DeserrRawValue<'a>;
|
||||
|
||||
type Iter = DeserrRawVecIter<'a>;
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.vec.len()
|
||||
}
|
||||
|
||||
fn into_iter(self) -> Self::Iter {
|
||||
DeserrRawVecIter { it: self.vec.into_iter(), alloc: self.alloc }
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DeserrRawVecIter<'a> {
|
||||
it: raw_collections::vec::iter::IntoIter<'a>,
|
||||
alloc: &'a Bump,
|
||||
}
|
||||
|
||||
impl<'a> Iterator for DeserrRawVecIter<'a> {
|
||||
type Item = DeserrRawValue<'a>;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
let next = self.it.next()?;
|
||||
Some(DeserrRawValue { value: next, alloc: self.alloc })
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DeserrRawMap<'a> {
|
||||
map: raw_collections::RawMap<'a>,
|
||||
alloc: &'a Bump,
|
||||
}
|
||||
|
||||
impl<'a> deserr::Map for DeserrRawMap<'a> {
|
||||
type Value = DeserrRawValue<'a>;
|
||||
|
||||
type Iter = DeserrRawMapIter<'a>;
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.map.len()
|
||||
}
|
||||
|
||||
fn remove(&mut self, _key: &str) -> Option<Self::Value> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
fn into_iter(self) -> Self::Iter {
|
||||
DeserrRawMapIter { it: self.map.into_iter(), alloc: self.alloc }
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DeserrRawMapIter<'a> {
|
||||
it: raw_collections::map::iter::IntoIter<'a>,
|
||||
alloc: &'a Bump,
|
||||
}
|
||||
|
||||
impl<'a> Iterator for DeserrRawMapIter<'a> {
|
||||
type Item = (String, DeserrRawValue<'a>);
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
let (name, value) = self.it.next()?;
|
||||
Some((name.to_string(), DeserrRawValue { value, alloc: self.alloc }))
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> deserr::IntoValue for DeserrRawValue<'a> {
|
||||
type Sequence = DeserrRawVec<'a>;
|
||||
|
||||
type Map = DeserrRawMap<'a>;
|
||||
|
||||
fn kind(&self) -> deserr::ValueKind {
|
||||
self.value.deserialize_any(DeserrKindVisitor).unwrap()
|
||||
}
|
||||
|
||||
fn into_value(self) -> deserr::Value<Self> {
|
||||
self.value.deserialize_any(DeserrRawValueVisitor { alloc: self.alloc }).unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DeserrKindVisitor;
|
||||
|
||||
impl<'de> Visitor<'de> for DeserrKindVisitor {
|
||||
type Value = deserr::ValueKind;
|
||||
|
||||
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
write!(formatter, "any value")
|
||||
}
|
||||
|
||||
fn visit_bool<E>(self, _v: bool) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::ValueKind::Boolean)
|
||||
}
|
||||
|
||||
fn visit_i64<E>(self, _v: i64) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::ValueKind::NegativeInteger)
|
||||
}
|
||||
|
||||
fn visit_u64<E>(self, _v: u64) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::ValueKind::Integer)
|
||||
}
|
||||
|
||||
fn visit_f64<E>(self, _v: f64) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::ValueKind::Float)
|
||||
}
|
||||
|
||||
fn visit_str<E>(self, _v: &str) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::ValueKind::String)
|
||||
}
|
||||
|
||||
fn visit_none<E>(self) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::ValueKind::Null)
|
||||
}
|
||||
|
||||
fn visit_some<D>(self, deserializer: D) -> Result<Self::Value, D::Error>
|
||||
where
|
||||
D: serde::Deserializer<'de>,
|
||||
{
|
||||
deserializer.deserialize_any(self)
|
||||
}
|
||||
|
||||
fn visit_unit<E>(self) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::ValueKind::Null)
|
||||
}
|
||||
|
||||
fn visit_newtype_struct<D>(self, deserializer: D) -> Result<Self::Value, D::Error>
|
||||
where
|
||||
D: serde::Deserializer<'de>,
|
||||
{
|
||||
deserializer.deserialize_any(self)
|
||||
}
|
||||
|
||||
fn visit_seq<A>(self, _seq: A) -> Result<Self::Value, A::Error>
|
||||
where
|
||||
A: serde::de::SeqAccess<'de>,
|
||||
{
|
||||
Ok(deserr::ValueKind::Sequence)
|
||||
}
|
||||
|
||||
fn visit_map<A>(self, _map: A) -> Result<Self::Value, A::Error>
|
||||
where
|
||||
A: serde::de::MapAccess<'de>,
|
||||
{
|
||||
Ok(deserr::ValueKind::Map)
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DeserrRawValueVisitor<'a> {
|
||||
alloc: &'a Bump,
|
||||
}
|
||||
|
||||
impl<'de> Visitor<'de> for DeserrRawValueVisitor<'de> {
|
||||
type Value = deserr::Value<DeserrRawValue<'de>>;
|
||||
|
||||
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
write!(formatter, "any value")
|
||||
}
|
||||
|
||||
fn visit_bool<E>(self, v: bool) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::Value::Boolean(v))
|
||||
}
|
||||
|
||||
fn visit_i64<E>(self, v: i64) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::Value::NegativeInteger(v))
|
||||
}
|
||||
|
||||
fn visit_u64<E>(self, v: u64) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::Value::Integer(v))
|
||||
}
|
||||
|
||||
fn visit_f64<E>(self, v: f64) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::Value::Float(v))
|
||||
}
|
||||
|
||||
fn visit_str<E>(self, v: &str) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::Value::String(v.to_string()))
|
||||
}
|
||||
|
||||
fn visit_string<E>(self, v: String) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::Value::String(v))
|
||||
}
|
||||
|
||||
fn visit_none<E>(self) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::Value::Null)
|
||||
}
|
||||
|
||||
fn visit_some<D>(self, deserializer: D) -> Result<Self::Value, D::Error>
|
||||
where
|
||||
D: serde::Deserializer<'de>,
|
||||
{
|
||||
deserializer.deserialize_any(self)
|
||||
}
|
||||
|
||||
fn visit_unit<E>(self) -> Result<Self::Value, E>
|
||||
where
|
||||
E: serde::de::Error,
|
||||
{
|
||||
Ok(deserr::Value::Null)
|
||||
}
|
||||
|
||||
fn visit_newtype_struct<D>(self, deserializer: D) -> Result<Self::Value, D::Error>
|
||||
where
|
||||
D: serde::Deserializer<'de>,
|
||||
{
|
||||
deserializer.deserialize_any(self)
|
||||
}
|
||||
|
||||
fn visit_seq<A>(self, mut seq: A) -> Result<Self::Value, A::Error>
|
||||
where
|
||||
A: serde::de::SeqAccess<'de>,
|
||||
{
|
||||
let mut raw_vec = raw_collections::RawVec::new_in(&self.alloc);
|
||||
while let Some(next) = seq.next_element()? {
|
||||
raw_vec.push(next);
|
||||
}
|
||||
Ok(deserr::Value::Sequence(DeserrRawVec { vec: raw_vec, alloc: self.alloc }))
|
||||
}
|
||||
|
||||
fn visit_map<A>(self, map: A) -> Result<Self::Value, A::Error>
|
||||
where
|
||||
A: serde::de::MapAccess<'de>,
|
||||
{
|
||||
let _ = map;
|
||||
Err(serde::de::Error::invalid_type(serde::de::Unexpected::Map, &self))
|
||||
}
|
||||
|
||||
fn visit_enum<A>(self, data: A) -> Result<Self::Value, A::Error>
|
||||
where
|
||||
A: serde::de::EnumAccess<'de>,
|
||||
{
|
||||
let _ = data;
|
||||
Err(serde::de::Error::invalid_type(serde::de::Unexpected::Enum, &self))
|
||||
}
|
||||
}
|
521
crates/milli/src/update/new/indexer/document_changes.rs
Normal file
521
crates/milli/src/update/new/indexer/document_changes.rs
Normal file
|
@ -0,0 +1,521 @@
|
|||
use std::cell::{Cell, Ref, RefCell, RefMut};
|
||||
use std::sync::{Arc, RwLock};
|
||||
|
||||
use bumpalo::Bump;
|
||||
use heed::RoTxn;
|
||||
use rayon::iter::IndexedParallelIterator;
|
||||
|
||||
use super::super::document_change::DocumentChange;
|
||||
use crate::fields_ids_map::metadata::FieldIdMapWithMetadata;
|
||||
use crate::update::new::parallel_iterator_ext::ParallelIteratorExt as _;
|
||||
use crate::{FieldsIdsMap, GlobalFieldsIdsMap, Index, InternalError, Result};
|
||||
|
||||
pub trait RefCellExt<T: ?Sized> {
|
||||
fn try_borrow_or_yield(&self) -> std::result::Result<Ref<'_, T>, std::cell::BorrowError>;
|
||||
fn try_borrow_mut_or_yield(
|
||||
&self,
|
||||
) -> std::result::Result<RefMut<'_, T>, std::cell::BorrowMutError>;
|
||||
|
||||
fn borrow_or_yield(&self) -> Ref<'_, T> {
|
||||
self.try_borrow_or_yield().unwrap()
|
||||
}
|
||||
|
||||
fn borrow_mut_or_yield(&self) -> RefMut<'_, T> {
|
||||
self.try_borrow_mut_or_yield().unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: ?Sized> RefCellExt<T> for RefCell<T> {
|
||||
fn try_borrow_or_yield(&self) -> std::result::Result<Ref<'_, T>, std::cell::BorrowError> {
|
||||
/// TODO: move this trait and impl elsewhere
|
||||
loop {
|
||||
match self.try_borrow() {
|
||||
Ok(borrow) => break Ok(borrow),
|
||||
Err(error) => {
|
||||
tracing::warn!("dynamic borrow failed, yielding to local tasks");
|
||||
match rayon::yield_local() {
|
||||
Some(rayon::Yield::Executed) => continue,
|
||||
_ => return Err(error),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn try_borrow_mut_or_yield(
|
||||
&self,
|
||||
) -> std::result::Result<RefMut<'_, T>, std::cell::BorrowMutError> {
|
||||
loop {
|
||||
match self.try_borrow_mut() {
|
||||
Ok(borrow) => break Ok(borrow),
|
||||
Err(error) => {
|
||||
tracing::warn!("dynamic borrow failed, yielding to local tasks");
|
||||
|
||||
match rayon::yield_local() {
|
||||
Some(rayon::Yield::Executed) => continue,
|
||||
_ => return Err(error),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A trait for types that are **not** [`Send`] only because they would then allow concurrent access to a type that is not [`Sync`].
|
||||
///
|
||||
/// The primary example of such a type is `&T`, with `T: !Sync`.
|
||||
///
|
||||
/// In the authors' understanding, a type can be `!Send` for two distinct reasons:
|
||||
///
|
||||
/// 1. Because it contains data that *genuinely* cannot be moved between threads, such as thread-local data.
|
||||
/// 2. Because sending the type would allow concurrent access to a `!Sync` type, which is undefined behavior.
|
||||
///
|
||||
/// `MostlySend` exists to be used in bounds where you need a type whose data is **not** *attached* to a thread
|
||||
/// because you might access it from a different thread, but where you will never access the type **concurrently** from
|
||||
/// multiple threads.
|
||||
///
|
||||
/// Like [`Send`], `MostlySend` assumes properties on types that cannot be verified by the compiler, which is why implementing
|
||||
/// this trait is unsafe.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// Implementers of this trait promises that the following properties hold on the implementing type:
|
||||
///
|
||||
/// 1. Its data can be accessed from any thread and will be the same regardless of the thread accessing it.
|
||||
/// 2. Any operation that can be performed on the type does not depend on the thread that executes it.
|
||||
///
|
||||
/// As these properties are subtle and are not generally tracked by the Rust type system, great care should be taken before
|
||||
/// implementing `MostlySend` on a type, especially a foreign type.
|
||||
///
|
||||
/// - An example of a type that verifies (1) and (2) is [`std::rc::Rc`] (when `T` is `Send` and `Sync`).
|
||||
/// - An example of a type that doesn't verify (1) is thread-local data.
|
||||
/// - An example of a type that doesn't verify (2) is [`std::sync::MutexGuard`]: a lot of mutex implementations require that
|
||||
/// a lock is returned to the operating system on the same thread that initially locked the mutex, failing to uphold this
|
||||
/// invariant will cause Undefined Behavior
|
||||
/// (see last § in [the nomicon](https://doc.rust-lang.org/nomicon/send-and-sync.html)).
|
||||
///
|
||||
/// It is **always safe** to implement this trait on a type that is `Send`, but no placeholder impl is provided due to limitations in
|
||||
/// coherency. Use the [`FullySend`] wrapper in this situation.
|
||||
pub unsafe trait MostlySend {}
|
||||
|
||||
#[derive(Debug, Clone, Copy, Default, Eq, PartialEq, Ord, PartialOrd, Hash)]
|
||||
pub struct FullySend<T>(pub T);
|
||||
|
||||
// SAFETY: a type **fully** send is always mostly send as well.
|
||||
unsafe impl<T> MostlySend for FullySend<T> where T: Send {}
|
||||
|
||||
unsafe impl<T> MostlySend for RefCell<T> where T: MostlySend {}
|
||||
|
||||
unsafe impl<T> MostlySend for Option<T> where T: MostlySend {}
|
||||
|
||||
impl<T> FullySend<T> {
|
||||
pub fn into(self) -> T {
|
||||
self.0
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> From<T> for FullySend<T> {
|
||||
fn from(value: T) -> Self {
|
||||
Self(value)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, Copy, Default, PartialEq, Eq, PartialOrd, Ord, Hash)]
|
||||
struct MostlySendWrapper<T>(T);
|
||||
|
||||
impl<T: MostlySend> MostlySendWrapper<T> {
|
||||
/// # Safety
|
||||
///
|
||||
/// - (P1) Users of this type will never access the type concurrently from multiple threads without synchronization
|
||||
unsafe fn new(t: T) -> Self {
|
||||
Self(t)
|
||||
}
|
||||
|
||||
fn as_ref(&self) -> &T {
|
||||
&self.0
|
||||
}
|
||||
|
||||
fn as_mut(&mut self) -> &mut T {
|
||||
&mut self.0
|
||||
}
|
||||
|
||||
fn into_inner(self) -> T {
|
||||
self.0
|
||||
}
|
||||
}
|
||||
|
||||
/// # Safety
|
||||
///
|
||||
/// 1. `T` is [`MostlySend`], so by its safety contract it can be accessed by any thread and all of its operations are available
|
||||
/// from any thread.
|
||||
/// 2. (P1) of `MostlySendWrapper::new` forces the user to never access the value from multiple threads concurrently.
|
||||
unsafe impl<T: MostlySend> Send for MostlySendWrapper<T> {}
|
||||
|
||||
/// A wrapper around [`thread_local::ThreadLocal`] that accepts [`MostlySend`] `T`s.
|
||||
#[derive(Default)]
|
||||
pub struct ThreadLocal<T: MostlySend> {
|
||||
inner: thread_local::ThreadLocal<MostlySendWrapper<T>>,
|
||||
// FIXME: this should be necessary
|
||||
//_no_send: PhantomData<*mut ()>,
|
||||
}
|
||||
|
||||
impl<T: MostlySend> ThreadLocal<T> {
|
||||
pub fn new() -> Self {
|
||||
Self { inner: thread_local::ThreadLocal::new() }
|
||||
}
|
||||
|
||||
pub fn with_capacity(capacity: usize) -> Self {
|
||||
Self { inner: thread_local::ThreadLocal::with_capacity(capacity) }
|
||||
}
|
||||
|
||||
pub fn clear(&mut self) {
|
||||
self.inner.clear()
|
||||
}
|
||||
|
||||
pub fn get(&self) -> Option<&T> {
|
||||
self.inner.get().map(|t| t.as_ref())
|
||||
}
|
||||
|
||||
pub fn get_or<F>(&self, create: F) -> &T
|
||||
where
|
||||
F: FnOnce() -> T,
|
||||
{
|
||||
/// TODO: move ThreadLocal, MostlySend, FullySend to a dedicated file
|
||||
self.inner.get_or(|| unsafe { MostlySendWrapper::new(create()) }).as_ref()
|
||||
}
|
||||
|
||||
pub fn get_or_try<F, E>(&self, create: F) -> std::result::Result<&T, E>
|
||||
where
|
||||
F: FnOnce() -> std::result::Result<T, E>,
|
||||
{
|
||||
self.inner
|
||||
.get_or_try(|| unsafe { Ok(MostlySendWrapper::new(create()?)) })
|
||||
.map(MostlySendWrapper::as_ref)
|
||||
}
|
||||
|
||||
pub fn get_or_default(&self) -> &T
|
||||
where
|
||||
T: Default,
|
||||
{
|
||||
self.inner.get_or_default().as_ref()
|
||||
}
|
||||
|
||||
pub fn iter_mut(&mut self) -> IterMut<T> {
|
||||
IterMut(self.inner.iter_mut())
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: MostlySend> IntoIterator for ThreadLocal<T> {
|
||||
type Item = T;
|
||||
|
||||
type IntoIter = IntoIter<T>;
|
||||
|
||||
fn into_iter(self) -> Self::IntoIter {
|
||||
IntoIter(self.inner.into_iter())
|
||||
}
|
||||
}
|
||||
|
||||
pub struct IterMut<'a, T: MostlySend>(thread_local::IterMut<'a, MostlySendWrapper<T>>);
|
||||
|
||||
impl<'a, T: MostlySend> Iterator for IterMut<'a, T> {
|
||||
type Item = &'a mut T;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
self.0.next().map(|t| t.as_mut())
|
||||
}
|
||||
}
|
||||
|
||||
pub struct IntoIter<T: MostlySend>(thread_local::IntoIter<MostlySendWrapper<T>>);
|
||||
|
||||
impl<T: MostlySend> Iterator for IntoIter<T> {
|
||||
type Item = T;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
self.0.next().map(|t| t.into_inner())
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DocumentChangeContext<
|
||||
'doc, // covariant lifetime of a single `process` call
|
||||
'extractor: 'doc, // invariant lifetime of the extractor_allocs
|
||||
'fid: 'doc, // invariant lifetime of the new_fields_ids_map
|
||||
'indexer: 'doc, // covariant lifetime of objects that outlive a single `process` call
|
||||
T: MostlySend,
|
||||
> {
|
||||
/// The index we're indexing in
|
||||
pub index: &'indexer Index,
|
||||
/// The fields ids map as it was at the start of this indexing process. Contains at least all top-level fields from documents
|
||||
/// inside of the DB.
|
||||
pub db_fields_ids_map: &'indexer FieldsIdsMap,
|
||||
/// A transaction providing data from the DB before all indexing operations
|
||||
pub txn: RoTxn<'indexer>,
|
||||
|
||||
/// Global field id map that is up to date with the current state of the indexing process.
|
||||
///
|
||||
/// - Inserting a field will take a lock
|
||||
/// - Retrieving a field may take a lock as well
|
||||
pub new_fields_ids_map: &'doc std::cell::RefCell<GlobalFieldsIdsMap<'fid>>,
|
||||
|
||||
/// Data allocated in this allocator is cleared between each call to `process`.
|
||||
pub doc_alloc: Bump,
|
||||
|
||||
/// Data allocated in this allocator is not cleared between each call to `process`, unless the data spills.
|
||||
pub extractor_alloc: &'extractor Bump,
|
||||
|
||||
/// Pool of doc allocators, used to retrieve the doc allocator we provided for the documents
|
||||
doc_allocs: &'doc ThreadLocal<FullySend<Cell<Bump>>>,
|
||||
|
||||
/// Extractor-specific data
|
||||
pub data: &'doc T,
|
||||
}
|
||||
|
||||
impl<
|
||||
'doc, // covariant lifetime of a single `process` call
|
||||
'data: 'doc, // invariant on T lifetime of the datastore
|
||||
'extractor: 'doc, // invariant lifetime of extractor_allocs
|
||||
'fid: 'doc, // invariant lifetime of fields ids map
|
||||
'indexer: 'doc, // covariant lifetime of objects that survive a `process` call
|
||||
T: MostlySend,
|
||||
> DocumentChangeContext<'doc, 'extractor, 'fid, 'indexer, T>
|
||||
{
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn new<F>(
|
||||
index: &'indexer Index,
|
||||
db_fields_ids_map: &'indexer FieldsIdsMap,
|
||||
new_fields_ids_map: &'fid RwLock<FieldIdMapWithMetadata>,
|
||||
extractor_allocs: &'extractor ThreadLocal<FullySend<Bump>>,
|
||||
doc_allocs: &'doc ThreadLocal<FullySend<Cell<Bump>>>,
|
||||
datastore: &'data ThreadLocal<T>,
|
||||
fields_ids_map_store: &'doc ThreadLocal<FullySend<RefCell<GlobalFieldsIdsMap<'fid>>>>,
|
||||
init_data: F,
|
||||
) -> Result<Self>
|
||||
where
|
||||
F: FnOnce(&'extractor Bump) -> Result<T>,
|
||||
{
|
||||
let doc_alloc =
|
||||
doc_allocs.get_or(|| FullySend(Cell::new(Bump::with_capacity(1024 * 1024 * 1024))));
|
||||
let doc_alloc = doc_alloc.0.take();
|
||||
let fields_ids_map = fields_ids_map_store
|
||||
.get_or(|| RefCell::new(GlobalFieldsIdsMap::new(new_fields_ids_map)).into());
|
||||
|
||||
let fields_ids_map = &fields_ids_map.0;
|
||||
let extractor_alloc = extractor_allocs.get_or_default();
|
||||
|
||||
let data = datastore.get_or_try(move || init_data(&extractor_alloc.0))?;
|
||||
|
||||
let txn = index.read_txn()?;
|
||||
Ok(DocumentChangeContext {
|
||||
index,
|
||||
txn,
|
||||
db_fields_ids_map,
|
||||
new_fields_ids_map: fields_ids_map,
|
||||
doc_alloc,
|
||||
extractor_alloc: &extractor_alloc.0,
|
||||
data,
|
||||
doc_allocs,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/// An internal iterator (i.e. using `foreach`) of `DocumentChange`s
|
||||
pub trait Extractor<'extractor>: Sync {
|
||||
type Data: MostlySend;
|
||||
|
||||
fn init_data<'doc>(&'doc self, extractor_alloc: &'extractor Bump) -> Result<Self::Data>;
|
||||
|
||||
fn process<'doc>(
|
||||
&'doc self,
|
||||
changes: impl Iterator<Item = Result<DocumentChange<'doc>>>,
|
||||
context: &'doc DocumentChangeContext<Self::Data>,
|
||||
) -> Result<()>;
|
||||
}
|
||||
|
||||
pub trait DocumentChanges<'pl // lifetime of the underlying payload
|
||||
>: Sync {
|
||||
type Item: Send;
|
||||
|
||||
fn iter(&self, chunk_size: usize) -> impl IndexedParallelIterator<Item = impl AsRef<[Self::Item]>>;
|
||||
|
||||
fn len(&self) -> usize;
|
||||
|
||||
fn is_empty(&self) -> bool {
|
||||
self.len() == 0
|
||||
}
|
||||
|
||||
fn item_to_document_change<'doc, // lifetime of a single `process` call
|
||||
T: MostlySend>(
|
||||
&'doc self,
|
||||
context: &'doc DocumentChangeContext<T>,
|
||||
item: &'doc Self::Item,
|
||||
) -> Result<Option<DocumentChange<'doc>>> where 'pl: 'doc // the payload must survive the process calls
|
||||
;
|
||||
}
|
||||
|
||||
pub struct IndexingContext<
|
||||
'fid, // invariant lifetime of fields ids map
|
||||
'indexer, // covariant lifetime of objects that are borrowed during the entire indexing operation
|
||||
'index, // covariant lifetime of the index
|
||||
MSP,
|
||||
SP,
|
||||
> where
|
||||
MSP: Fn() -> bool + Sync,
|
||||
SP: Fn(Progress) + Sync,
|
||||
{
|
||||
pub index: &'index Index,
|
||||
pub db_fields_ids_map: &'indexer FieldsIdsMap,
|
||||
pub new_fields_ids_map: &'fid RwLock<FieldIdMapWithMetadata>,
|
||||
pub doc_allocs: &'indexer ThreadLocal<FullySend<Cell<Bump>>>,
|
||||
pub fields_ids_map_store: &'indexer ThreadLocal<FullySend<RefCell<GlobalFieldsIdsMap<'fid>>>>,
|
||||
pub must_stop_processing: &'indexer MSP,
|
||||
pub send_progress: &'indexer SP,
|
||||
}
|
||||
|
||||
impl<
|
||||
'fid, // invariant lifetime of fields ids map
|
||||
'indexer, // covariant lifetime of objects that are borrowed during the entire indexing operation
|
||||
'index, // covariant lifetime of the index
|
||||
MSP,
|
||||
SP,
|
||||
> Copy
|
||||
for IndexingContext<
|
||||
'fid, // invariant lifetime of fields ids map
|
||||
'indexer, // covariant lifetime of objects that are borrowed during the entire indexing operation
|
||||
'index, // covariant lifetime of the index
|
||||
MSP,
|
||||
SP,
|
||||
>
|
||||
where
|
||||
MSP: Fn() -> bool + Sync,
|
||||
SP: Fn(Progress) + Sync,
|
||||
{
|
||||
}
|
||||
|
||||
impl<
|
||||
'fid, // invariant lifetime of fields ids map
|
||||
'indexer, // covariant lifetime of objects that are borrowed during the entire indexing operation
|
||||
'index, // covariant lifetime of the index
|
||||
MSP,
|
||||
SP,
|
||||
> Clone
|
||||
for IndexingContext<
|
||||
'fid, // invariant lifetime of fields ids map
|
||||
'indexer, // covariant lifetime of objects that are borrowed during the entire indexing operation
|
||||
'index, // covariant lifetime of the index
|
||||
MSP,
|
||||
SP,
|
||||
>
|
||||
where
|
||||
MSP: Fn() -> bool + Sync,
|
||||
SP: Fn(Progress) + Sync,
|
||||
{
|
||||
fn clone(&self) -> Self {
|
||||
*self
|
||||
}
|
||||
}
|
||||
|
||||
const CHUNK_SIZE: usize = 100;
|
||||
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn extract<
|
||||
'pl, // covariant lifetime of the underlying payload
|
||||
'extractor, // invariant lifetime of extractor_alloc
|
||||
'fid, // invariant lifetime of fields ids map
|
||||
'indexer, // covariant lifetime of objects that are borrowed during the entire indexing
|
||||
'data, // invariant on EX::Data lifetime of datastore
|
||||
'index, // covariant lifetime of the index
|
||||
EX,
|
||||
DC: DocumentChanges<'pl>,
|
||||
MSP,
|
||||
SP,
|
||||
>(
|
||||
document_changes: &DC,
|
||||
extractor: &EX,
|
||||
IndexingContext {
|
||||
index,
|
||||
db_fields_ids_map,
|
||||
new_fields_ids_map,
|
||||
doc_allocs,
|
||||
fields_ids_map_store,
|
||||
must_stop_processing,
|
||||
send_progress,
|
||||
}: IndexingContext<'fid, 'indexer, 'index, MSP, SP>,
|
||||
extractor_allocs: &'extractor mut ThreadLocal<FullySend<Bump>>,
|
||||
datastore: &'data ThreadLocal<EX::Data>,
|
||||
finished_steps: u16,
|
||||
total_steps: u16,
|
||||
step_name: &'static str,
|
||||
) -> Result<()>
|
||||
where
|
||||
EX: Extractor<'extractor>,
|
||||
MSP: Fn() -> bool + Sync,
|
||||
SP: Fn(Progress) + Sync,
|
||||
{
|
||||
eprintln!("We are resetting the extractor allocators");
|
||||
// Clean up and reuse the extractor allocs
|
||||
for extractor_alloc in extractor_allocs.iter_mut() {
|
||||
eprintln!("\tWith {} bytes resetted", extractor_alloc.0.allocated_bytes());
|
||||
extractor_alloc.0.reset();
|
||||
}
|
||||
|
||||
let total_documents = document_changes.len();
|
||||
|
||||
let pi = document_changes.iter(CHUNK_SIZE);
|
||||
pi.enumerate().try_arc_for_each_try_init(
|
||||
|| {
|
||||
DocumentChangeContext::new(
|
||||
index,
|
||||
db_fields_ids_map,
|
||||
new_fields_ids_map,
|
||||
extractor_allocs,
|
||||
doc_allocs,
|
||||
datastore,
|
||||
fields_ids_map_store,
|
||||
move |index_alloc| extractor.init_data(index_alloc),
|
||||
)
|
||||
},
|
||||
|context, (finished_documents, items)| {
|
||||
if (must_stop_processing)() {
|
||||
return Err(Arc::new(InternalError::AbortedIndexation.into()));
|
||||
}
|
||||
let finished_documents = finished_documents * CHUNK_SIZE;
|
||||
|
||||
(send_progress)(Progress {
|
||||
finished_steps,
|
||||
total_steps,
|
||||
step_name,
|
||||
finished_total_documents: Some((finished_documents as u32, total_documents as u32)),
|
||||
});
|
||||
|
||||
// Clean up and reuse the document-specific allocator
|
||||
context.doc_alloc.reset();
|
||||
|
||||
let items = items.as_ref();
|
||||
let changes = items.iter().filter_map(|item| {
|
||||
document_changes.item_to_document_change(context, item).transpose()
|
||||
});
|
||||
|
||||
let res = extractor.process(changes, context).map_err(Arc::new);
|
||||
|
||||
// send back the doc_alloc in the pool
|
||||
context.doc_allocs.get_or_default().0.set(std::mem::take(&mut context.doc_alloc));
|
||||
|
||||
res
|
||||
},
|
||||
)?;
|
||||
|
||||
(send_progress)(Progress {
|
||||
finished_steps,
|
||||
total_steps,
|
||||
step_name,
|
||||
finished_total_documents: Some((total_documents as u32, total_documents as u32)),
|
||||
});
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub struct Progress {
|
||||
pub finished_steps: u16,
|
||||
pub total_steps: u16,
|
||||
pub step_name: &'static str,
|
||||
pub finished_total_documents: Option<(u32, u32)>,
|
||||
}
|
190
crates/milli/src/update/new/indexer/document_deletion.rs
Normal file
190
crates/milli/src/update/new/indexer/document_deletion.rs
Normal file
|
@ -0,0 +1,190 @@
|
|||
use bumpalo::collections::CollectIn;
|
||||
use bumpalo::Bump;
|
||||
use rayon::iter::IndexedParallelIterator;
|
||||
use rayon::slice::ParallelSlice as _;
|
||||
use roaring::RoaringBitmap;
|
||||
|
||||
use super::document_changes::{DocumentChangeContext, DocumentChanges, MostlySend};
|
||||
use crate::documents::PrimaryKey;
|
||||
use crate::update::new::{Deletion, DocumentChange};
|
||||
use crate::{DocumentId, Result};
|
||||
|
||||
#[derive(Default)]
|
||||
pub struct DocumentDeletion {
|
||||
pub to_delete: RoaringBitmap,
|
||||
}
|
||||
|
||||
impl DocumentDeletion {
|
||||
pub fn new() -> Self {
|
||||
Self { to_delete: Default::default() }
|
||||
}
|
||||
|
||||
pub fn delete_documents_by_docids(&mut self, docids: RoaringBitmap) {
|
||||
self.to_delete |= docids;
|
||||
}
|
||||
|
||||
pub fn into_changes<'indexer>(
|
||||
self,
|
||||
indexer: &'indexer Bump,
|
||||
primary_key: PrimaryKey<'indexer>,
|
||||
) -> DocumentDeletionChanges<'indexer> {
|
||||
let to_delete: bumpalo::collections::Vec<_> =
|
||||
self.to_delete.into_iter().collect_in(indexer);
|
||||
|
||||
let to_delete = to_delete.into_bump_slice();
|
||||
|
||||
DocumentDeletionChanges { to_delete, primary_key }
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DocumentDeletionChanges<'indexer> {
|
||||
to_delete: &'indexer [DocumentId],
|
||||
primary_key: PrimaryKey<'indexer>,
|
||||
}
|
||||
|
||||
impl<'pl> DocumentChanges<'pl> for DocumentDeletionChanges<'pl> {
|
||||
type Item = DocumentId;
|
||||
|
||||
fn iter(
|
||||
&self,
|
||||
chunk_size: usize,
|
||||
) -> impl IndexedParallelIterator<Item = impl AsRef<[Self::Item]>> {
|
||||
self.to_delete.par_chunks(chunk_size)
|
||||
}
|
||||
|
||||
fn item_to_document_change<
|
||||
'doc, // lifetime of a single `process` call
|
||||
T: MostlySend,
|
||||
>(
|
||||
&'doc self,
|
||||
context: &'doc DocumentChangeContext<T>,
|
||||
docid: &'doc Self::Item,
|
||||
) -> Result<Option<DocumentChange<'doc>>>
|
||||
where
|
||||
'pl: 'doc, // the payload must survive the process calls
|
||||
{
|
||||
let current = context.index.document(&context.txn, *docid)?;
|
||||
|
||||
let external_document_id = self.primary_key.extract_docid_from_db(
|
||||
current,
|
||||
&context.db_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
|
||||
let external_document_id = external_document_id.to_bump(&context.doc_alloc);
|
||||
|
||||
Ok(Some(DocumentChange::Deletion(Deletion::create(*docid, external_document_id))))
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.to_delete.len()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use std::cell::RefCell;
|
||||
use std::marker::PhantomData;
|
||||
use std::sync::RwLock;
|
||||
|
||||
use bumpalo::Bump;
|
||||
|
||||
use crate::fields_ids_map::metadata::{FieldIdMapWithMetadata, MetadataBuilder};
|
||||
use crate::index::tests::TempIndex;
|
||||
use crate::update::new::indexer::document_changes::{
|
||||
extract, DocumentChangeContext, Extractor, IndexingContext, MostlySend, ThreadLocal,
|
||||
};
|
||||
use crate::update::new::indexer::DocumentDeletion;
|
||||
use crate::update::new::DocumentChange;
|
||||
use crate::DocumentId;
|
||||
|
||||
#[test]
|
||||
fn test_deletions() {
|
||||
struct DeletionWithData<'extractor> {
|
||||
deleted: RefCell<
|
||||
hashbrown::HashSet<DocumentId, hashbrown::DefaultHashBuilder, &'extractor Bump>,
|
||||
>,
|
||||
}
|
||||
|
||||
unsafe impl<'extractor> MostlySend for DeletionWithData<'extractor> {}
|
||||
|
||||
struct TrackDeletion<'extractor>(PhantomData<&'extractor ()>);
|
||||
|
||||
impl<'extractor> Extractor<'extractor> for TrackDeletion<'extractor> {
|
||||
type Data = DeletionWithData<'extractor>;
|
||||
|
||||
fn init_data(&self, extractor_alloc: &'extractor Bump) -> crate::Result<Self::Data> {
|
||||
let deleted = RefCell::new(hashbrown::HashSet::new_in(extractor_alloc));
|
||||
Ok(DeletionWithData { deleted })
|
||||
}
|
||||
|
||||
fn process<'doc>(
|
||||
&self,
|
||||
changes: impl Iterator<Item = crate::Result<DocumentChange<'doc>>>,
|
||||
context: &DocumentChangeContext<Self::Data>,
|
||||
) -> crate::Result<()> {
|
||||
for change in changes {
|
||||
let change = change?;
|
||||
context.data.deleted.borrow_mut().insert(change.docid());
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
let mut deletions = DocumentDeletion::new();
|
||||
deletions.delete_documents_by_docids(vec![0, 2, 42].into_iter().collect());
|
||||
let indexer = Bump::new();
|
||||
|
||||
let index = TempIndex::new();
|
||||
|
||||
let rtxn = index.read_txn().unwrap();
|
||||
|
||||
let db_fields_ids_map = index.fields_ids_map(&rtxn).unwrap();
|
||||
let metadata_builder = MetadataBuilder::from_index(&index, &rtxn).unwrap();
|
||||
let fields_ids_map =
|
||||
RwLock::new(FieldIdMapWithMetadata::new(db_fields_ids_map.clone(), metadata_builder));
|
||||
|
||||
let fields_ids_map_store = ThreadLocal::new();
|
||||
|
||||
let mut extractor_allocs = ThreadLocal::new();
|
||||
let doc_allocs = ThreadLocal::new();
|
||||
|
||||
let deletion_tracker = TrackDeletion(PhantomData);
|
||||
|
||||
let changes = deletions
|
||||
.into_changes(&indexer, crate::documents::PrimaryKey::Flat { name: "id", field_id: 0 });
|
||||
|
||||
let context = IndexingContext {
|
||||
index: &index,
|
||||
db_fields_ids_map: &db_fields_ids_map,
|
||||
new_fields_ids_map: &fields_ids_map,
|
||||
doc_allocs: &doc_allocs,
|
||||
fields_ids_map_store: &fields_ids_map_store,
|
||||
must_stop_processing: &(|| false),
|
||||
send_progress: &(|_progress| {}),
|
||||
};
|
||||
|
||||
for _ in 0..3 {
|
||||
let datastore = ThreadLocal::new();
|
||||
|
||||
extract(
|
||||
&changes,
|
||||
&deletion_tracker,
|
||||
context,
|
||||
&mut extractor_allocs,
|
||||
&datastore,
|
||||
0,
|
||||
1,
|
||||
"test",
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
for (index, data) in datastore.into_iter().enumerate() {
|
||||
println!("deleted by {index}: {:?}", data.deleted.borrow());
|
||||
}
|
||||
for alloc in extractor_allocs.iter_mut() {
|
||||
alloc.0.reset();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
412
crates/milli/src/update/new/indexer/document_operation.rs
Normal file
412
crates/milli/src/update/new/indexer/document_operation.rs
Normal file
|
@ -0,0 +1,412 @@
|
|||
use bumpalo::collections::CollectIn;
|
||||
use bumpalo::Bump;
|
||||
use heed::RoTxn;
|
||||
use memmap2::Mmap;
|
||||
use rayon::slice::ParallelSlice;
|
||||
use serde_json::value::RawValue;
|
||||
use IndexDocumentsMethod as Idm;
|
||||
|
||||
use super::super::document_change::DocumentChange;
|
||||
use super::document_changes::{DocumentChangeContext, DocumentChanges, MostlySend};
|
||||
use crate::documents::PrimaryKey;
|
||||
use crate::update::new::document::Versions;
|
||||
use crate::update::new::{Deletion, Insertion, Update};
|
||||
use crate::update::{AvailableIds, IndexDocumentsMethod};
|
||||
use crate::{DocumentId, Error, FieldsIdsMap, Index, Result, UserError};
|
||||
|
||||
pub struct DocumentOperation<'pl> {
|
||||
operations: Vec<Payload<'pl>>,
|
||||
index_documents_method: IndexDocumentsMethod,
|
||||
}
|
||||
|
||||
pub struct DocumentOperationChanges<'pl> {
|
||||
docids_version_offsets: &'pl [(&'pl str, ((u32, bool), &'pl [InnerDocOp<'pl>]))],
|
||||
index_documents_method: IndexDocumentsMethod,
|
||||
}
|
||||
|
||||
pub enum Payload<'pl> {
|
||||
Addition(&'pl [u8]),
|
||||
Deletion(&'pl [&'pl str]),
|
||||
}
|
||||
|
||||
pub struct PayloadStats {
|
||||
pub document_count: usize,
|
||||
pub bytes: u64,
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub enum InnerDocOp<'pl> {
|
||||
Addition(DocumentOffset<'pl>),
|
||||
Deletion,
|
||||
}
|
||||
|
||||
/// Represents an offset where a document lives
|
||||
/// in an mmapped grenad reader file.
|
||||
#[derive(Clone)]
|
||||
pub struct DocumentOffset<'pl> {
|
||||
/// The mmapped payload files.
|
||||
pub content: &'pl [u8],
|
||||
}
|
||||
|
||||
impl<'pl> DocumentOperation<'pl> {
|
||||
pub fn new(method: IndexDocumentsMethod) -> Self {
|
||||
Self { operations: Default::default(), index_documents_method: method }
|
||||
}
|
||||
|
||||
/// TODO please give me a type
|
||||
/// The payload is expected to be in the grenad format
|
||||
pub fn add_documents(&mut self, payload: &'pl Mmap) -> Result<PayloadStats> {
|
||||
payload.advise(memmap2::Advice::Sequential)?;
|
||||
let document_count =
|
||||
memchr::memmem::find_iter(&payload[..], "}{").count().saturating_add(1);
|
||||
self.operations.push(Payload::Addition(&payload[..]));
|
||||
Ok(PayloadStats { bytes: payload.len() as u64, document_count })
|
||||
}
|
||||
|
||||
pub fn delete_documents(&mut self, to_delete: &'pl [&'pl str]) {
|
||||
self.operations.push(Payload::Deletion(to_delete))
|
||||
}
|
||||
|
||||
pub fn into_changes(
|
||||
self,
|
||||
indexer: &'pl Bump,
|
||||
index: &Index,
|
||||
rtxn: &RoTxn,
|
||||
primary_key: &PrimaryKey,
|
||||
new_fields_ids_map: &mut FieldsIdsMap,
|
||||
) -> Result<DocumentOperationChanges<'pl>> {
|
||||
// will contain nodes from the intermediate hashmap
|
||||
let document_changes_alloc = Bump::with_capacity(1024 * 1024 * 1024); // 1 MiB
|
||||
|
||||
let documents_ids = index.documents_ids(rtxn)?;
|
||||
let mut available_docids = AvailableIds::new(&documents_ids);
|
||||
let mut docids_version_offsets =
|
||||
hashbrown::HashMap::<&'pl str, _, _, _>::new_in(&document_changes_alloc);
|
||||
|
||||
for operation in self.operations {
|
||||
match operation {
|
||||
Payload::Addition(payload) => {
|
||||
let mut iter =
|
||||
serde_json::Deserializer::from_slice(payload).into_iter::<&RawValue>();
|
||||
|
||||
/// TODO manage the error
|
||||
let mut previous_offset = 0;
|
||||
while let Some(document) =
|
||||
iter.next().transpose().map_err(UserError::SerdeJson)?
|
||||
{
|
||||
let external_document_id = primary_key.extract_fields_and_docid(
|
||||
document,
|
||||
new_fields_ids_map,
|
||||
indexer,
|
||||
)?;
|
||||
|
||||
let external_document_id = external_document_id.to_de();
|
||||
|
||||
let current_offset = iter.byte_offset();
|
||||
let document_operation = InnerDocOp::Addition(DocumentOffset {
|
||||
content: &payload[previous_offset..current_offset],
|
||||
});
|
||||
|
||||
match docids_version_offsets.get_mut(external_document_id) {
|
||||
None => {
|
||||
let (docid, is_new) = match index
|
||||
.external_documents_ids()
|
||||
.get(rtxn, external_document_id)?
|
||||
{
|
||||
Some(docid) => (docid, false),
|
||||
None => (
|
||||
available_docids.next().ok_or(Error::UserError(
|
||||
UserError::DocumentLimitReached,
|
||||
))?,
|
||||
true,
|
||||
),
|
||||
};
|
||||
|
||||
docids_version_offsets.insert(
|
||||
external_document_id,
|
||||
(
|
||||
(docid, is_new),
|
||||
bumpalo::vec![in indexer; document_operation],
|
||||
),
|
||||
);
|
||||
}
|
||||
Some((_, offsets)) => {
|
||||
let useless_previous_addition = match self.index_documents_method {
|
||||
IndexDocumentsMethod::ReplaceDocuments => {
|
||||
MergeDocumentForReplacement::USELESS_PREVIOUS_CHANGES
|
||||
}
|
||||
IndexDocumentsMethod::UpdateDocuments => {
|
||||
MergeDocumentForUpdates::USELESS_PREVIOUS_CHANGES
|
||||
}
|
||||
};
|
||||
|
||||
if useless_previous_addition {
|
||||
offsets.clear();
|
||||
}
|
||||
|
||||
offsets.push(document_operation);
|
||||
}
|
||||
}
|
||||
|
||||
previous_offset = iter.byte_offset();
|
||||
}
|
||||
}
|
||||
Payload::Deletion(to_delete) => {
|
||||
for external_document_id in to_delete {
|
||||
match docids_version_offsets.get_mut(external_document_id) {
|
||||
None => {
|
||||
let (docid, is_new) = match index
|
||||
.external_documents_ids()
|
||||
.get(rtxn, external_document_id)?
|
||||
{
|
||||
Some(docid) => (docid, false),
|
||||
None => (
|
||||
available_docids.next().ok_or(Error::UserError(
|
||||
UserError::DocumentLimitReached,
|
||||
))?,
|
||||
true,
|
||||
),
|
||||
};
|
||||
|
||||
docids_version_offsets.insert(
|
||||
external_document_id,
|
||||
(
|
||||
(docid, is_new),
|
||||
bumpalo::vec![in indexer; InnerDocOp::Deletion],
|
||||
),
|
||||
);
|
||||
}
|
||||
Some((_, offsets)) => {
|
||||
offsets.clear();
|
||||
offsets.push(InnerDocOp::Deletion);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TODO We must drain the HashMap into a Vec because rayon::hash_map::IntoIter: !Clone
|
||||
let mut docids_version_offsets: bumpalo::collections::vec::Vec<_> = docids_version_offsets
|
||||
.drain()
|
||||
.map(|(item, (docid, v))| (item, (docid, v.into_bump_slice())))
|
||||
.collect_in(indexer);
|
||||
// Reorder the offsets to make sure we iterate on the file sequentially
|
||||
let sort_function_key = match self.index_documents_method {
|
||||
Idm::ReplaceDocuments => MergeDocumentForReplacement::sort_key,
|
||||
Idm::UpdateDocuments => MergeDocumentForUpdates::sort_key,
|
||||
};
|
||||
|
||||
// And finally sort them
|
||||
docids_version_offsets.sort_unstable_by_key(|(_, (_, docops))| sort_function_key(docops));
|
||||
let docids_version_offsets = docids_version_offsets.into_bump_slice();
|
||||
Ok(DocumentOperationChanges {
|
||||
docids_version_offsets,
|
||||
index_documents_method: self.index_documents_method,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl<'pl> DocumentChanges<'pl> for DocumentOperationChanges<'pl> {
|
||||
type Item = (&'pl str, ((u32, bool), &'pl [InnerDocOp<'pl>]));
|
||||
|
||||
fn iter(
|
||||
&self,
|
||||
chunk_size: usize,
|
||||
) -> impl rayon::prelude::IndexedParallelIterator<Item = impl AsRef<[Self::Item]>> {
|
||||
self.docids_version_offsets.par_chunks(chunk_size)
|
||||
}
|
||||
|
||||
fn item_to_document_change<'doc, T: MostlySend + 'doc>(
|
||||
&'doc self,
|
||||
context: &'doc DocumentChangeContext<T>,
|
||||
item: &'doc Self::Item,
|
||||
) -> Result<Option<DocumentChange<'doc>>>
|
||||
where
|
||||
'pl: 'doc,
|
||||
{
|
||||
let document_merge_function = match self.index_documents_method {
|
||||
Idm::ReplaceDocuments => MergeDocumentForReplacement::merge,
|
||||
Idm::UpdateDocuments => MergeDocumentForUpdates::merge,
|
||||
};
|
||||
|
||||
let (external_doc, ((internal_docid, is_new), operations)) = *item;
|
||||
|
||||
let change = document_merge_function(
|
||||
internal_docid,
|
||||
external_doc,
|
||||
is_new,
|
||||
&context.doc_alloc,
|
||||
operations,
|
||||
)?;
|
||||
Ok(change)
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.docids_version_offsets.len()
|
||||
}
|
||||
}
|
||||
|
||||
trait MergeChanges {
|
||||
/// Whether the payloads in the list of operations are useless or not.
|
||||
const USELESS_PREVIOUS_CHANGES: bool;
|
||||
|
||||
/// Returns a key that is used to order the payloads the right way.
|
||||
fn sort_key(docops: &[InnerDocOp]) -> usize;
|
||||
|
||||
fn merge<'doc>(
|
||||
docid: DocumentId,
|
||||
external_docid: &'doc str,
|
||||
is_new: bool,
|
||||
doc_alloc: &'doc Bump,
|
||||
operations: &'doc [InnerDocOp],
|
||||
) -> Result<Option<DocumentChange<'doc>>>;
|
||||
}
|
||||
|
||||
struct MergeDocumentForReplacement;
|
||||
|
||||
impl MergeChanges for MergeDocumentForReplacement {
|
||||
const USELESS_PREVIOUS_CHANGES: bool = true;
|
||||
|
||||
/// Reorders to read only the last change.
|
||||
fn sort_key(docops: &[InnerDocOp]) -> usize {
|
||||
let f = |ido: &_| match ido {
|
||||
InnerDocOp::Addition(add) => Some(add.content.as_ptr() as usize),
|
||||
InnerDocOp::Deletion => None,
|
||||
};
|
||||
docops.iter().rev().find_map(f).unwrap_or(0)
|
||||
}
|
||||
|
||||
/// Returns only the most recent version of a document based on the updates from the payloads.
|
||||
///
|
||||
/// This function is only meant to be used when doing a replacement and not an update.
|
||||
fn merge<'doc>(
|
||||
docid: DocumentId,
|
||||
external_doc: &'doc str,
|
||||
is_new: bool,
|
||||
doc_alloc: &'doc Bump,
|
||||
operations: &'doc [InnerDocOp],
|
||||
) -> Result<Option<DocumentChange<'doc>>> {
|
||||
match operations.last() {
|
||||
Some(InnerDocOp::Addition(DocumentOffset { content })) => {
|
||||
let document = serde_json::from_slice(content).unwrap();
|
||||
let document = raw_collections::RawMap::from_raw_value(document, doc_alloc)
|
||||
.map_err(UserError::SerdeJson)?;
|
||||
|
||||
if is_new {
|
||||
Ok(Some(DocumentChange::Insertion(Insertion::create(
|
||||
docid,
|
||||
external_doc,
|
||||
Versions::single(document),
|
||||
))))
|
||||
} else {
|
||||
Ok(Some(DocumentChange::Update(Update::create(
|
||||
docid,
|
||||
external_doc,
|
||||
Versions::single(document),
|
||||
true,
|
||||
))))
|
||||
}
|
||||
}
|
||||
Some(InnerDocOp::Deletion) => {
|
||||
return if is_new {
|
||||
let deletion = Deletion::create(docid, external_doc);
|
||||
Ok(Some(DocumentChange::Deletion(deletion)))
|
||||
} else {
|
||||
Ok(None)
|
||||
};
|
||||
}
|
||||
None => unreachable!("We must not have empty set of operations on a document"),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct MergeDocumentForUpdates;
|
||||
|
||||
impl MergeChanges for MergeDocumentForUpdates {
|
||||
const USELESS_PREVIOUS_CHANGES: bool = false;
|
||||
|
||||
/// Reorders to read the first changes first so that it's faster to read the first one and then the rest.
|
||||
fn sort_key(docops: &[InnerDocOp]) -> usize {
|
||||
let f = |ido: &_| match ido {
|
||||
InnerDocOp::Addition(add) => Some(add.content.as_ptr() as usize),
|
||||
InnerDocOp::Deletion => None,
|
||||
};
|
||||
docops.iter().find_map(f).unwrap_or(0)
|
||||
}
|
||||
|
||||
/// Reads the previous version of a document from the database, the new versions
|
||||
/// in the grenad update files and merges them to generate a new boxed obkv.
|
||||
///
|
||||
/// This function is only meant to be used when doing an update and not a replacement.
|
||||
fn merge<'doc>(
|
||||
docid: DocumentId,
|
||||
external_docid: &'doc str,
|
||||
is_new: bool,
|
||||
doc_alloc: &'doc Bump,
|
||||
operations: &'doc [InnerDocOp],
|
||||
) -> Result<Option<DocumentChange<'doc>>> {
|
||||
if operations.is_empty() {
|
||||
unreachable!("We must not have empty set of operations on a document");
|
||||
}
|
||||
|
||||
let last_deletion = operations.iter().rposition(|op| matches!(op, InnerDocOp::Deletion));
|
||||
let operations = &operations[last_deletion.map_or(0, |i| i + 1)..];
|
||||
|
||||
let has_deletion = last_deletion.is_some();
|
||||
|
||||
if operations.is_empty() {
|
||||
return if !is_new {
|
||||
let deletion = Deletion::create(docid, external_docid);
|
||||
Ok(Some(DocumentChange::Deletion(deletion)))
|
||||
} else {
|
||||
Ok(None)
|
||||
};
|
||||
}
|
||||
|
||||
let versions = match operations {
|
||||
[single] => {
|
||||
let DocumentOffset { content } = match single {
|
||||
InnerDocOp::Addition(offset) => offset,
|
||||
InnerDocOp::Deletion => {
|
||||
unreachable!("Deletion in document operations")
|
||||
}
|
||||
};
|
||||
let document = serde_json::from_slice(content).unwrap();
|
||||
let document = raw_collections::RawMap::from_raw_value(document, doc_alloc)
|
||||
.map_err(UserError::SerdeJson)?;
|
||||
|
||||
Some(Versions::single(document))
|
||||
}
|
||||
operations => {
|
||||
let versions = operations.iter().map(|operation| {
|
||||
let DocumentOffset { content } = match operation {
|
||||
InnerDocOp::Addition(offset) => offset,
|
||||
InnerDocOp::Deletion => {
|
||||
unreachable!("Deletion in document operations")
|
||||
}
|
||||
};
|
||||
|
||||
let document = serde_json::from_slice(content).unwrap();
|
||||
let document = raw_collections::RawMap::from_raw_value(document, doc_alloc)
|
||||
.map_err(UserError::SerdeJson)?;
|
||||
Ok(document)
|
||||
});
|
||||
Versions::multiple(versions)?
|
||||
}
|
||||
};
|
||||
|
||||
let Some(versions) = versions else { return Ok(None) };
|
||||
|
||||
if is_new {
|
||||
Ok(Some(DocumentChange::Insertion(Insertion::create(docid, external_docid, versions))))
|
||||
} else {
|
||||
Ok(Some(DocumentChange::Update(Update::create(
|
||||
docid,
|
||||
external_docid,
|
||||
versions,
|
||||
has_deletion,
|
||||
))))
|
||||
}
|
||||
}
|
||||
}
|
519
crates/milli/src/update/new/indexer/mod.rs
Normal file
519
crates/milli/src/update/new/indexer/mod.rs
Normal file
|
@ -0,0 +1,519 @@
|
|||
use std::cmp::Ordering;
|
||||
use std::sync::RwLock;
|
||||
use std::thread::{self, Builder};
|
||||
|
||||
use big_s::S;
|
||||
use document_changes::{
|
||||
for_each_document_change, DocumentChanges, FullySend, IndexingContext, ThreadLocal,
|
||||
};
|
||||
pub use document_deletion::DocumentDeletion;
|
||||
pub use document_operation::DocumentOperation;
|
||||
use heed::types::{Bytes, DecodeIgnore, Str};
|
||||
use heed::{RoTxn, RwTxn};
|
||||
use itertools::{merge_join_by, EitherOrBoth};
|
||||
pub use partial_dump::PartialDump;
|
||||
use rayon::ThreadPool;
|
||||
use time::OffsetDateTime;
|
||||
pub use update_by_function::UpdateByFunction;
|
||||
|
||||
use super::channel::*;
|
||||
use super::extract::*;
|
||||
use super::facet_search_builder::FacetSearchBuilder;
|
||||
use super::merger::{FacetDatabases, FacetFieldIdsDelta};
|
||||
use super::word_fst_builder::PrefixDelta;
|
||||
use super::words_prefix_docids::{
|
||||
compute_word_prefix_docids, compute_word_prefix_fid_docids, compute_word_prefix_position_docids,
|
||||
};
|
||||
use super::{StdResult, TopLevelMap};
|
||||
use crate::documents::{PrimaryKey, DEFAULT_PRIMARY_KEY};
|
||||
use crate::facet::FacetType;
|
||||
use crate::index::main_key::{WORDS_FST_KEY, WORDS_PREFIXES_FST_KEY};
|
||||
use crate::proximity::ProximityPrecision;
|
||||
use crate::update::del_add::DelAdd;
|
||||
use crate::update::new::word_fst_builder::{PrefixData, WordFstBuilder};
|
||||
use crate::update::new::words_prefix_docids::compute_exact_word_prefix_docids;
|
||||
use crate::update::new::{merge_and_send_docids, merge_and_send_facet_docids};
|
||||
use crate::update::settings::InnerIndexSettings;
|
||||
use crate::update::{FacetsUpdateBulk, GrenadParameters};
|
||||
use crate::{FieldsIdsMap, GlobalFieldsIdsMap, Index, Result, UserError};
|
||||
|
||||
pub mod de;
|
||||
pub mod document_changes;
|
||||
mod document_deletion;
|
||||
mod document_operation;
|
||||
mod partial_dump;
|
||||
mod update_by_function;
|
||||
|
||||
/// This is the main function of this crate.
|
||||
///
|
||||
/// Give it the output of the [`Indexer::document_changes`] method and it will execute it in the [`rayon::ThreadPool`].
|
||||
///
|
||||
/// TODO return stats
|
||||
pub fn index<'pl, 'indexer, 'index, DC>(
|
||||
wtxn: &mut RwTxn,
|
||||
index: &'index Index,
|
||||
db_fields_ids_map: &'indexer FieldsIdsMap,
|
||||
new_fields_ids_map: FieldsIdsMap,
|
||||
new_primary_key: Option<PrimaryKey<'pl>>,
|
||||
pool: &ThreadPool,
|
||||
document_changes: &DC,
|
||||
) -> Result<()>
|
||||
where
|
||||
DC: DocumentChanges<'pl>,
|
||||
{
|
||||
// TODO find a better channel limit
|
||||
let (extractor_sender, writer_receiver) = extractor_writer_channel(10_000);
|
||||
let new_fields_ids_map = RwLock::new(new_fields_ids_map);
|
||||
|
||||
let global_fields_ids_map = GlobalFieldsIdsMap::new(&new_fields_ids_map);
|
||||
let fields_ids_map_store = ThreadLocal::with_capacity(pool.current_num_threads());
|
||||
let mut extractor_allocs = ThreadLocal::with_capacity(pool.current_num_threads());
|
||||
let doc_allocs = ThreadLocal::with_capacity(pool.current_num_threads());
|
||||
|
||||
let indexing_context = IndexingContext {
|
||||
index,
|
||||
db_fields_ids_map,
|
||||
new_fields_ids_map: &new_fields_ids_map,
|
||||
doc_allocs: &doc_allocs,
|
||||
fields_ids_map_store: &fields_ids_map_store,
|
||||
};
|
||||
|
||||
thread::scope(|s| -> crate::Result<_> {
|
||||
let indexer_span = tracing::Span::current();
|
||||
// TODO manage the errors correctly
|
||||
let extractor_handle = Builder::new().name(S("indexer-extractors")).spawn_scoped(s, move || {
|
||||
pool.in_place_scope(|_s| {
|
||||
let span = tracing::trace_span!(target: "indexing::documents", parent: &indexer_span, "extract");
|
||||
let _entered = span.enter();
|
||||
|
||||
// document but we need to create a function that collects and compresses documents.
|
||||
let rtxn = index.read_txn().unwrap();
|
||||
let document_sender = extractor_sender.documents();
|
||||
let document_extractor = DocumentsExtractor::new(&document_sender);
|
||||
let datastore = ThreadLocal::with_capacity(pool.current_num_threads());
|
||||
for_each_document_change(document_changes, &document_extractor, indexing_context, &mut extractor_allocs, &datastore)?;
|
||||
|
||||
let mut documents_ids = index.documents_ids(&rtxn)?;
|
||||
let delta_documents_ids = datastore.into_iter().map(|FullySend(d)| d.into_inner()).reduce(DelAddRoaringBitmap::merge).unwrap_or_default();
|
||||
delta_documents_ids.apply_to(&mut documents_ids);
|
||||
extractor_sender.send_documents_ids(documents_ids).unwrap();
|
||||
|
||||
// document_sender.finish().unwrap();
|
||||
|
||||
const TEN_GIB: usize = 10 * 1024 * 1024 * 1024;
|
||||
let current_num_threads = rayon::current_num_threads();
|
||||
let max_memory = TEN_GIB / current_num_threads;
|
||||
eprintln!("A maximum of {max_memory} bytes will be used for each of the {current_num_threads} threads");
|
||||
let grenad_parameters = GrenadParameters {
|
||||
max_memory: Some(max_memory),
|
||||
..GrenadParameters::default()
|
||||
};
|
||||
|
||||
let facet_field_ids_delta;
|
||||
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::extract", "faceted");
|
||||
let _entered = span.enter();
|
||||
facet_field_ids_delta = merge_and_send_facet_docids(
|
||||
FacetedDocidsExtractor::run_extraction(grenad_parameters, document_changes, indexing_context, &mut extractor_allocs)?,
|
||||
FacetDatabases::new(index),
|
||||
index,
|
||||
extractor_sender.facet_docids(),
|
||||
)?;
|
||||
}
|
||||
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::extract", "word_docids");
|
||||
let _entered = span.enter();
|
||||
|
||||
let WordDocidsCaches {
|
||||
word_docids,
|
||||
word_fid_docids,
|
||||
exact_word_docids,
|
||||
word_position_docids,
|
||||
fid_word_count_docids,
|
||||
} = WordDocidsExtractors::run_extraction(grenad_parameters, document_changes, indexing_context, &mut extractor_allocs)?;
|
||||
|
||||
// TODO Word Docids Merger
|
||||
// extractor_sender.send_searchable::<WordDocids>(word_docids).unwrap();
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::merge", "word_docids");
|
||||
let _entered = span.enter();
|
||||
merge_and_send_docids(
|
||||
word_docids,
|
||||
index.word_docids.remap_types(),
|
||||
index,
|
||||
extractor_sender.docids::<WordDocids>(),
|
||||
)?;
|
||||
}
|
||||
|
||||
// Word Fid Docids Merging
|
||||
// extractor_sender.send_searchable::<WordFidDocids>(word_fid_docids).unwrap();
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::merge", "word_fid_docids");
|
||||
let _entered = span.enter();
|
||||
merge_and_send_docids(
|
||||
word_fid_docids,
|
||||
index.word_fid_docids.remap_types(),
|
||||
index,
|
||||
extractor_sender.docids::<WordFidDocids>()
|
||||
)?;
|
||||
}
|
||||
|
||||
// Exact Word Docids Merging
|
||||
// extractor_sender.send_searchable::<ExactWordDocids>(exact_word_docids).unwrap();
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::merge", "exact_word_docids");
|
||||
let _entered = span.enter();
|
||||
merge_and_send_docids(
|
||||
exact_word_docids,
|
||||
index.exact_word_docids.remap_types(),
|
||||
index,
|
||||
extractor_sender.docids::<ExactWordDocids>(),
|
||||
)?;
|
||||
}
|
||||
|
||||
// Word Position Docids Merging
|
||||
// extractor_sender.send_searchable::<WordPositionDocids>(word_position_docids).unwrap();
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::merge", "word_position_docids");
|
||||
let _entered = span.enter();
|
||||
merge_and_send_docids(
|
||||
word_position_docids,
|
||||
index.word_position_docids.remap_types(),
|
||||
index,
|
||||
extractor_sender.docids::<WordPositionDocids>(),
|
||||
)?;
|
||||
}
|
||||
|
||||
// Fid Word Count Docids Merging
|
||||
// extractor_sender.send_searchable::<FidWordCountDocids>(fid_word_count_docids).unwrap();
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::merge", "fid_word_count_docids");
|
||||
let _entered = span.enter();
|
||||
merge_and_send_docids(
|
||||
fid_word_count_docids,
|
||||
index.field_id_word_count_docids.remap_types(),
|
||||
index,
|
||||
extractor_sender.docids::<FidWordCountDocids>(),
|
||||
)?;
|
||||
}
|
||||
}
|
||||
|
||||
// run the proximity extraction only if the precision is by word
|
||||
// this works only if the settings didn't change during this transaction.
|
||||
let proximity_precision = index.proximity_precision(&rtxn)?.unwrap_or_default();
|
||||
if proximity_precision == ProximityPrecision::ByWord {
|
||||
let span = tracing::trace_span!(target: "indexing::documents::extract", "word_pair_proximity_docids");
|
||||
let _entered = span.enter();
|
||||
let caches = <WordPairProximityDocidsExtractor as DocidsExtractor>::run_extraction(grenad_parameters, document_changes, indexing_context, &mut extractor_allocs)?;
|
||||
merge_and_send_docids(
|
||||
caches,
|
||||
index.word_pair_proximity_docids.remap_types(),
|
||||
index,
|
||||
extractor_sender.docids::<WordPairProximityDocids>(),
|
||||
)?;
|
||||
}
|
||||
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::extract", "FINISH");
|
||||
let _entered = span.enter();
|
||||
}
|
||||
|
||||
// TODO THIS IS TOO MUCH
|
||||
// - [ ] Extract fieldid docid facet number
|
||||
// - [ ] Extract fieldid docid facet string
|
||||
// - [ ] Extract facetid string fst
|
||||
// - [ ] Extract facetid normalized string strings
|
||||
|
||||
// TODO Inverted Indexes again
|
||||
// - [x] Extract fieldid facet isempty docids
|
||||
// - [x] Extract fieldid facet isnull docids
|
||||
// - [x] Extract fieldid facet exists docids
|
||||
|
||||
// TODO This is the normal system
|
||||
// - [x] Extract fieldid facet number docids
|
||||
// - [x] Extract fieldid facet string docids
|
||||
|
||||
// TODO use None when needed
|
||||
Result::Ok(facet_field_ids_delta)
|
||||
})
|
||||
})?;
|
||||
|
||||
for operation in writer_receiver {
|
||||
let database = operation.database(index);
|
||||
match operation.entry() {
|
||||
EntryOperation::Delete(e) => {
|
||||
if !database.delete(wtxn, e.entry())? {
|
||||
unreachable!("We tried to delete an unknown key")
|
||||
}
|
||||
}
|
||||
EntryOperation::Write(e) => database.put(wtxn, e.key(), e.value())?,
|
||||
}
|
||||
}
|
||||
|
||||
/// TODO handle the panicking threads
|
||||
let facet_field_ids_delta = extractor_handle.join().unwrap()?;
|
||||
|
||||
if let Some(prefix_delta) = compute_word_fst(index, wtxn)? {
|
||||
compute_prefix_database(index, wtxn, prefix_delta)?;
|
||||
}
|
||||
|
||||
compute_facet_search_database(index, wtxn, global_fields_ids_map)?;
|
||||
|
||||
compute_facet_level_database(index, wtxn, facet_field_ids_delta)?;
|
||||
|
||||
Result::Ok(())
|
||||
})?;
|
||||
|
||||
// required to into_inner the new_fields_ids_map
|
||||
drop(fields_ids_map_store);
|
||||
|
||||
let fields_ids_map = new_fields_ids_map.into_inner().unwrap();
|
||||
index.put_fields_ids_map(wtxn, &fields_ids_map)?;
|
||||
|
||||
if let Some(new_primary_key) = new_primary_key {
|
||||
index.put_primary_key(wtxn, new_primary_key.name())?;
|
||||
}
|
||||
|
||||
// used to update the localized and weighted maps while sharing the update code with the settings pipeline.
|
||||
let mut inner_index_settings = InnerIndexSettings::from_index(index, wtxn)?;
|
||||
inner_index_settings.recompute_facets(wtxn, index)?;
|
||||
inner_index_settings.recompute_searchables(wtxn, index)?;
|
||||
|
||||
index.set_updated_at(wtxn, &OffsetDateTime::now_utc())?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
fn compute_prefix_database(
|
||||
index: &Index,
|
||||
wtxn: &mut RwTxn,
|
||||
prefix_delta: PrefixDelta,
|
||||
) -> Result<()> {
|
||||
eprintln!("prefix_delta: {:?}", &prefix_delta);
|
||||
let PrefixDelta { modified, deleted } = prefix_delta;
|
||||
// Compute word prefix docids
|
||||
compute_word_prefix_docids(wtxn, index, &modified, &deleted)?;
|
||||
// Compute exact word prefix docids
|
||||
compute_exact_word_prefix_docids(wtxn, index, &modified, &deleted)?;
|
||||
// Compute word prefix fid docids
|
||||
compute_word_prefix_fid_docids(wtxn, index, &modified, &deleted)?;
|
||||
// Compute word prefix position docids
|
||||
compute_word_prefix_position_docids(wtxn, index, &modified, &deleted)
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing")]
|
||||
fn compute_word_fst(index: &Index, wtxn: &mut RwTxn) -> Result<Option<PrefixDelta>> {
|
||||
let rtxn = index.read_txn()?;
|
||||
let words_fst = index.words_fst(&rtxn)?;
|
||||
let mut word_fst_builder = WordFstBuilder::new(&words_fst)?;
|
||||
let prefix_settings = index.prefix_settings(&rtxn)?;
|
||||
word_fst_builder.with_prefix_settings(prefix_settings);
|
||||
|
||||
let previous_words = index.word_docids.iter(&rtxn)?.remap_data_type::<Bytes>();
|
||||
let current_words = index.word_docids.iter(wtxn)?.remap_data_type::<Bytes>();
|
||||
for eob in merge_join_by(previous_words, current_words, |lhs, rhs| match (lhs, rhs) {
|
||||
(Ok((l, _)), Ok((r, _))) => l.cmp(r),
|
||||
(Err(_), _) | (_, Err(_)) => Ordering::Equal,
|
||||
}) {
|
||||
match eob {
|
||||
EitherOrBoth::Both(lhs, rhs) => {
|
||||
let (word, lhs_bytes) = lhs?;
|
||||
let (_, rhs_bytes) = rhs?;
|
||||
if lhs_bytes != rhs_bytes {
|
||||
word_fst_builder.register_word(DelAdd::Addition, word.as_ref())?;
|
||||
}
|
||||
}
|
||||
EitherOrBoth::Left(result) => {
|
||||
let (word, _) = result?;
|
||||
word_fst_builder.register_word(DelAdd::Deletion, word.as_ref())?;
|
||||
}
|
||||
EitherOrBoth::Right(result) => {
|
||||
let (word, _) = result?;
|
||||
word_fst_builder.register_word(DelAdd::Addition, word.as_ref())?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let span = tracing::trace_span!(target: "indexing::documents::merge", "words_fst");
|
||||
let _entered = span.enter();
|
||||
|
||||
let (word_fst_mmap, prefix_data) = word_fst_builder.build(index, &rtxn)?;
|
||||
// extractor_sender.main().write_words_fst(word_fst_mmap).unwrap();
|
||||
index.main.remap_types::<Str, Bytes>().put(wtxn, WORDS_FST_KEY, &word_fst_mmap)?;
|
||||
if let Some(PrefixData { prefixes_fst_mmap, prefix_delta }) = prefix_data {
|
||||
// extractor_sender.main().write_words_prefixes_fst(prefixes_fst_mmap).unwrap();
|
||||
index.main.remap_types::<Str, Bytes>().put(
|
||||
wtxn,
|
||||
WORDS_PREFIXES_FST_KEY,
|
||||
&prefixes_fst_mmap,
|
||||
)?;
|
||||
Ok(Some(prefix_delta))
|
||||
} else {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::facet_search")]
|
||||
fn compute_facet_search_database(
|
||||
index: &Index,
|
||||
wtxn: &mut RwTxn,
|
||||
global_fields_ids_map: GlobalFieldsIdsMap,
|
||||
) -> Result<()> {
|
||||
let rtxn = index.read_txn()?;
|
||||
let localized_attributes_rules = index.localized_attributes_rules(&rtxn)?;
|
||||
let mut facet_search_builder = FacetSearchBuilder::new(
|
||||
global_fields_ids_map,
|
||||
localized_attributes_rules.unwrap_or_default(),
|
||||
);
|
||||
|
||||
let previous_facet_id_string_docids = index
|
||||
.facet_id_string_docids
|
||||
.iter(&rtxn)?
|
||||
.remap_data_type::<DecodeIgnore>()
|
||||
.filter(|r| r.as_ref().map_or(true, |(k, _)| k.level == 0));
|
||||
let current_facet_id_string_docids = index
|
||||
.facet_id_string_docids
|
||||
.iter(wtxn)?
|
||||
.remap_data_type::<DecodeIgnore>()
|
||||
.filter(|r| r.as_ref().map_or(true, |(k, _)| k.level == 0));
|
||||
for eob in merge_join_by(
|
||||
previous_facet_id_string_docids,
|
||||
current_facet_id_string_docids,
|
||||
|lhs, rhs| match (lhs, rhs) {
|
||||
(Ok((l, _)), Ok((r, _))) => l.cmp(r),
|
||||
(Err(_), _) | (_, Err(_)) => Ordering::Equal,
|
||||
},
|
||||
) {
|
||||
match eob {
|
||||
EitherOrBoth::Both(lhs, rhs) => {
|
||||
let (_, _) = lhs?;
|
||||
let (_, _) = rhs?;
|
||||
}
|
||||
EitherOrBoth::Left(result) => {
|
||||
let (key, _) = result?;
|
||||
facet_search_builder
|
||||
.register_from_key(DelAdd::Deletion, key.left_bound.as_ref())?;
|
||||
}
|
||||
EitherOrBoth::Right(result) => {
|
||||
let (key, _) = result?;
|
||||
facet_search_builder
|
||||
.register_from_key(DelAdd::Addition, key.left_bound.as_ref())?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
facet_search_builder.merge_and_write(index, wtxn, &rtxn)
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::facet_field_ids")]
|
||||
fn compute_facet_level_database(
|
||||
index: &Index,
|
||||
wtxn: &mut RwTxn,
|
||||
facet_field_ids_delta: FacetFieldIdsDelta,
|
||||
) -> Result<()> {
|
||||
eprintln!("facet_field_ids_delta: {:?}", &facet_field_ids_delta);
|
||||
if let Some(modified_facet_string_ids) = facet_field_ids_delta.modified_facet_string_ids() {
|
||||
let span = tracing::trace_span!(target: "indexing::facet_field_ids", "string");
|
||||
let _entered = span.enter();
|
||||
FacetsUpdateBulk::new_not_updating_level_0(
|
||||
index,
|
||||
modified_facet_string_ids,
|
||||
FacetType::String,
|
||||
)
|
||||
.execute(wtxn)?;
|
||||
}
|
||||
if let Some(modified_facet_number_ids) = facet_field_ids_delta.modified_facet_number_ids() {
|
||||
let span = tracing::trace_span!(target: "indexing::facet_field_ids", "number");
|
||||
let _entered = span.enter();
|
||||
FacetsUpdateBulk::new_not_updating_level_0(
|
||||
index,
|
||||
modified_facet_number_ids,
|
||||
FacetType::Number,
|
||||
)
|
||||
.execute(wtxn)?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Returns the primary key that has already been set for this index or the
|
||||
/// one we will guess by searching for the first key that contains "id" as a substring,
|
||||
/// and whether the primary key changed
|
||||
/// TODO move this elsewhere
|
||||
pub fn retrieve_or_guess_primary_key<'a>(
|
||||
rtxn: &'a RoTxn<'a>,
|
||||
index: &Index,
|
||||
new_fields_ids_map: &mut FieldsIdsMap,
|
||||
primary_key_from_op: Option<&'a str>,
|
||||
first_document: Option<&'a TopLevelMap<'a>>,
|
||||
) -> Result<StdResult<(PrimaryKey<'a>, bool), UserError>> {
|
||||
// make sure that we have a declared primary key, either fetching it from the index or attempting to guess it.
|
||||
|
||||
// do we have an existing declared primary key?
|
||||
let (primary_key, has_changed) = if let Some(primary_key_from_db) = index.primary_key(rtxn)? {
|
||||
// did we request a primary key in the operation?
|
||||
match primary_key_from_op {
|
||||
// we did, and it is different from the DB one
|
||||
Some(primary_key_from_op) if primary_key_from_op != primary_key_from_db => {
|
||||
// is the index empty?
|
||||
if index.number_of_documents(rtxn)? == 0 {
|
||||
// change primary key
|
||||
(primary_key_from_op, true)
|
||||
} else {
|
||||
return Ok(Err(UserError::PrimaryKeyCannotBeChanged(
|
||||
primary_key_from_db.to_string(),
|
||||
)));
|
||||
}
|
||||
}
|
||||
_ => (primary_key_from_db, false),
|
||||
}
|
||||
} else {
|
||||
// no primary key in the DB => let's set one
|
||||
// did we request a primary key in the operation?
|
||||
let primary_key = if let Some(primary_key_from_op) = primary_key_from_op {
|
||||
// set primary key from operation
|
||||
primary_key_from_op
|
||||
} else {
|
||||
// guess primary key
|
||||
let first_document = match first_document {
|
||||
Some(document) => document,
|
||||
// previous indexer when no pk is set + we send an empty payload => index_primary_key_no_candidate_found
|
||||
None => return Ok(Err(UserError::NoPrimaryKeyCandidateFound)),
|
||||
};
|
||||
|
||||
let mut guesses: Vec<&str> = first_document
|
||||
.keys()
|
||||
.map(AsRef::as_ref)
|
||||
.filter(|name| name.to_lowercase().ends_with(DEFAULT_PRIMARY_KEY))
|
||||
.collect();
|
||||
|
||||
// sort the keys in lexicographical order, so that fields are always in the same order.
|
||||
guesses.sort_unstable();
|
||||
|
||||
match guesses.as_slice() {
|
||||
[] => return Ok(Err(UserError::NoPrimaryKeyCandidateFound)),
|
||||
[name] => {
|
||||
tracing::info!("Primary key was not specified in index. Inferred to '{name}'");
|
||||
*name
|
||||
}
|
||||
multiple => {
|
||||
return Ok(Err(UserError::MultiplePrimaryKeyCandidatesFound {
|
||||
candidates: multiple
|
||||
.iter()
|
||||
.map(|candidate| candidate.to_string())
|
||||
.collect(),
|
||||
}))
|
||||
}
|
||||
}
|
||||
};
|
||||
(primary_key, true)
|
||||
};
|
||||
|
||||
match PrimaryKey::new_or_insert(primary_key, new_fields_ids_map) {
|
||||
Ok(primary_key) => Ok(Ok((primary_key, has_changed))),
|
||||
Err(err) => Ok(Err(err)),
|
||||
}
|
||||
}
|
86
crates/milli/src/update/new/indexer/partial_dump.rs
Normal file
86
crates/milli/src/update/new/indexer/partial_dump.rs
Normal file
|
@ -0,0 +1,86 @@
|
|||
use std::ops::DerefMut;
|
||||
|
||||
use rayon::iter::IndexedParallelIterator;
|
||||
use serde_json::value::RawValue;
|
||||
|
||||
use super::document_changes::{DocumentChangeContext, DocumentChanges, MostlySend, RefCellExt};
|
||||
use crate::documents::PrimaryKey;
|
||||
use crate::update::concurrent_available_ids::ConcurrentAvailableIds;
|
||||
use crate::update::new::document::{DocumentFromVersions, Versions};
|
||||
use crate::update::new::{DocumentChange, Insertion};
|
||||
use crate::{Error, InternalError, Result, UserError};
|
||||
|
||||
pub struct PartialDump<I> {
|
||||
iter: I,
|
||||
}
|
||||
|
||||
impl<I> PartialDump<I> {
|
||||
pub fn new_from_jsonlines(iter: I) -> Self {
|
||||
PartialDump { iter }
|
||||
}
|
||||
|
||||
pub fn into_changes<'index>(
|
||||
self,
|
||||
concurrent_available_ids: &'index ConcurrentAvailableIds,
|
||||
primary_key: &'index PrimaryKey,
|
||||
) -> PartialDumpChanges<'index, I> {
|
||||
/// Note for future self:
|
||||
/// - We recommend sending chunks of documents in this `PartialDumpIndexer` we therefore need to create a custom take_while_size method (that doesn't drop items).
|
||||
PartialDumpChanges { iter: self.iter, concurrent_available_ids, primary_key }
|
||||
}
|
||||
}
|
||||
|
||||
pub struct PartialDumpChanges<'doc, I> {
|
||||
iter: I,
|
||||
concurrent_available_ids: &'doc ConcurrentAvailableIds,
|
||||
primary_key: &'doc PrimaryKey<'doc>,
|
||||
}
|
||||
|
||||
impl<'index, Iter> DocumentChanges<'index> for PartialDumpChanges<'index, Iter>
|
||||
where
|
||||
Iter: IndexedParallelIterator<Item = Box<RawValue>> + Clone + Sync + 'index,
|
||||
{
|
||||
type Item = Box<RawValue>;
|
||||
|
||||
fn iter(
|
||||
&self,
|
||||
chunk_size: usize,
|
||||
) -> impl IndexedParallelIterator<Item = impl AsRef<[Self::Item]>> {
|
||||
self.iter.clone().chunks(chunk_size)
|
||||
}
|
||||
|
||||
fn item_to_document_change<'doc, T: MostlySend + 'doc>(
|
||||
&'doc self,
|
||||
context: &'doc DocumentChangeContext<T>,
|
||||
document: &'doc Self::Item,
|
||||
) -> Result<Option<DocumentChange<'doc>>>
|
||||
where
|
||||
'index: 'doc,
|
||||
{
|
||||
let doc_alloc = &context.doc_alloc;
|
||||
let docid = match self.concurrent_available_ids.next() {
|
||||
Some(id) => id,
|
||||
None => return Err(Error::UserError(UserError::DocumentLimitReached)),
|
||||
};
|
||||
|
||||
let mut fields_ids_map = context.new_fields_ids_map.borrow_mut_or_yield();
|
||||
let fields_ids_map = fields_ids_map.deref_mut();
|
||||
|
||||
let document = doc_alloc.alloc_str(document.get());
|
||||
let document: &RawValue = unsafe { std::mem::transmute(document) };
|
||||
|
||||
let external_document_id =
|
||||
self.primary_key.extract_fields_and_docid(document, fields_ids_map, doc_alloc)?;
|
||||
let external_document_id = external_document_id.to_de();
|
||||
|
||||
let document = raw_collections::RawMap::from_raw_value(document, doc_alloc)
|
||||
.map_err(InternalError::SerdeJson)?;
|
||||
|
||||
let insertion = Insertion::create(docid, external_document_id, Versions::single(document));
|
||||
Ok(Some(DocumentChange::Insertion(insertion)))
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.iter.len()
|
||||
}
|
||||
}
|
211
crates/milli/src/update/new/indexer/update_by_function.rs
Normal file
211
crates/milli/src/update/new/indexer/update_by_function.rs
Normal file
|
@ -0,0 +1,211 @@
|
|||
use raw_collections::RawMap;
|
||||
use rayon::iter::IndexedParallelIterator;
|
||||
use rayon::slice::ParallelSlice as _;
|
||||
use rhai::{Dynamic, Engine, OptimizationLevel, Scope, AST};
|
||||
use roaring::RoaringBitmap;
|
||||
|
||||
use super::document_changes::{DocumentChangeContext, MostlySend, RefCellExt};
|
||||
use super::DocumentChanges;
|
||||
use crate::documents::Error::InvalidDocumentFormat;
|
||||
use crate::documents::PrimaryKey;
|
||||
use crate::error::{FieldIdMapMissingEntry, InternalError};
|
||||
use crate::update::new::document::Versions;
|
||||
use crate::update::new::{Deletion, DocumentChange, KvReaderFieldId, Update};
|
||||
use crate::{all_obkv_to_json, Error, FieldsIdsMap, Object, Result, UserError};
|
||||
|
||||
pub struct UpdateByFunction {
|
||||
documents: RoaringBitmap,
|
||||
context: Option<Object>,
|
||||
code: String,
|
||||
}
|
||||
|
||||
pub struct UpdateByFunctionChanges<'doc> {
|
||||
primary_key: &'doc PrimaryKey<'doc>,
|
||||
engine: Engine,
|
||||
ast: AST,
|
||||
context: Option<Dynamic>,
|
||||
// It is sad that the RoaringBitmap doesn't
|
||||
// implement IndexedParallelIterator
|
||||
documents: Vec<u32>,
|
||||
}
|
||||
|
||||
impl UpdateByFunction {
|
||||
pub fn new(documents: RoaringBitmap, context: Option<Object>, code: String) -> Self {
|
||||
UpdateByFunction { documents, context, code }
|
||||
}
|
||||
|
||||
pub fn into_changes<'index>(
|
||||
self,
|
||||
primary_key: &'index PrimaryKey,
|
||||
) -> Result<UpdateByFunctionChanges<'index>> {
|
||||
let Self { documents, context, code } = self;
|
||||
|
||||
// Setup the security and limits of the Engine
|
||||
let mut engine = Engine::new();
|
||||
engine.set_optimization_level(OptimizationLevel::Full);
|
||||
engine.set_max_call_levels(1000);
|
||||
// It is an arbitrary value. We need to let users define this in the settings.
|
||||
engine.set_max_operations(1_000_000);
|
||||
engine.set_max_variables(1000);
|
||||
engine.set_max_functions(30);
|
||||
engine.set_max_expr_depths(100, 1000);
|
||||
engine.set_max_string_size(1024 * 1024 * 1024); // 1 GiB
|
||||
engine.set_max_array_size(10_000);
|
||||
engine.set_max_map_size(10_000);
|
||||
|
||||
let ast = engine.compile(code).map_err(UserError::DocumentEditionCompilationError)?;
|
||||
let context = match context {
|
||||
Some(context) => {
|
||||
Some(serde_json::from_value(context.into()).map_err(InternalError::SerdeJson)?)
|
||||
}
|
||||
None => None,
|
||||
};
|
||||
|
||||
Ok(UpdateByFunctionChanges {
|
||||
primary_key,
|
||||
engine,
|
||||
ast,
|
||||
context,
|
||||
documents: documents.into_iter().collect(),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl<'index> DocumentChanges<'index> for UpdateByFunctionChanges<'index> {
|
||||
type Item = u32;
|
||||
|
||||
fn iter(
|
||||
&self,
|
||||
chunk_size: usize,
|
||||
) -> impl IndexedParallelIterator<Item = impl AsRef<[Self::Item]>> {
|
||||
self.documents.as_slice().par_chunks(chunk_size)
|
||||
}
|
||||
|
||||
fn item_to_document_change<'doc, T: MostlySend + 'doc>(
|
||||
&self,
|
||||
context: &'doc DocumentChangeContext<T>,
|
||||
docid: &'doc Self::Item,
|
||||
) -> Result<Option<DocumentChange<'doc>>>
|
||||
where
|
||||
'index: 'doc,
|
||||
{
|
||||
let DocumentChangeContext {
|
||||
index,
|
||||
db_fields_ids_map,
|
||||
txn,
|
||||
new_fields_ids_map,
|
||||
doc_alloc,
|
||||
..
|
||||
} = context;
|
||||
|
||||
let docid = *docid;
|
||||
|
||||
// safety: Both documents *must* exists in the database as
|
||||
// their IDs comes from the list of documents ids.
|
||||
let document = index.document(txn, docid)?;
|
||||
let rhai_document = obkv_to_rhaimap(document, db_fields_ids_map)?;
|
||||
let json_document = all_obkv_to_json(document, db_fields_ids_map)?;
|
||||
|
||||
let document_id = self
|
||||
.primary_key
|
||||
.document_id(document, db_fields_ids_map)?
|
||||
.map_err(|_| InvalidDocumentFormat)?;
|
||||
|
||||
let mut scope = Scope::new();
|
||||
if let Some(context) = self.context.as_ref().cloned() {
|
||||
scope.push_constant_dynamic("context", context.clone());
|
||||
}
|
||||
scope.push("doc", rhai_document);
|
||||
// We run the user script which edits "doc" scope variable reprensenting
|
||||
// the document and ignore the output and even the type of it, i.e., Dynamic.
|
||||
let _ = self
|
||||
.engine
|
||||
.eval_ast_with_scope::<Dynamic>(&mut scope, &self.ast)
|
||||
.map_err(UserError::DocumentEditionRuntimeError)?;
|
||||
|
||||
match scope.remove::<Dynamic>("doc") {
|
||||
// If the "doc" variable has been set to (), we effectively delete the document.
|
||||
Some(doc) if doc.is_unit() => Ok(Some(DocumentChange::Deletion(Deletion::create(
|
||||
docid,
|
||||
doc_alloc.alloc_str(&document_id),
|
||||
)))),
|
||||
None => unreachable!("missing doc variable from the Rhai scope"),
|
||||
Some(new_document) => match new_document.try_cast() {
|
||||
Some(new_rhai_document) => {
|
||||
let mut buffer = bumpalo::collections::Vec::new_in(doc_alloc);
|
||||
serde_json::to_writer(&mut buffer, &new_rhai_document)
|
||||
.map_err(InternalError::SerdeJson)?;
|
||||
let raw_new_doc = serde_json::from_slice(buffer.into_bump_slice())
|
||||
.map_err(InternalError::SerdeJson)?;
|
||||
|
||||
// Note: This condition is not perfect. Sometimes it detect changes
|
||||
// like with floating points numbers and consider updating
|
||||
// the document even if nothing actually changed.
|
||||
//
|
||||
// Future: Use a custom function rhai function to track changes.
|
||||
// <https://docs.rs/rhai/latest/rhai/struct.Engine.html#method.register_indexer_set>
|
||||
if json_document != rhaimap_to_object(new_rhai_document) {
|
||||
let mut global_fields_ids_map = new_fields_ids_map.borrow_mut_or_yield();
|
||||
let new_document_id = self
|
||||
.primary_key
|
||||
.extract_fields_and_docid(
|
||||
raw_new_doc,
|
||||
&mut *global_fields_ids_map,
|
||||
doc_alloc,
|
||||
)?
|
||||
.to_de();
|
||||
|
||||
if document_id != new_document_id {
|
||||
Err(Error::UserError(UserError::DocumentEditionCannotModifyPrimaryKey))
|
||||
} else {
|
||||
let raw_new_doc = RawMap::from_raw_value(raw_new_doc, doc_alloc)
|
||||
.map_err(InternalError::SerdeJson)?;
|
||||
|
||||
Ok(Some(DocumentChange::Update(Update::create(
|
||||
docid,
|
||||
new_document_id,
|
||||
Versions::single(raw_new_doc),
|
||||
true, // It is like document replacement
|
||||
))))
|
||||
}
|
||||
} else {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
None => Err(Error::UserError(UserError::DocumentEditionDocumentMustBeObject)),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.documents.len()
|
||||
}
|
||||
}
|
||||
|
||||
fn obkv_to_rhaimap(obkv: &KvReaderFieldId, fields_ids_map: &FieldsIdsMap) -> Result<rhai::Map> {
|
||||
let all_keys = obkv.iter().map(|(k, _v)| k).collect::<Vec<_>>();
|
||||
let map: Result<rhai::Map> = all_keys
|
||||
.iter()
|
||||
.copied()
|
||||
.flat_map(|id| obkv.get(id).map(|value| (id, value)))
|
||||
.map(|(id, value)| {
|
||||
let name = fields_ids_map.name(id).ok_or(FieldIdMapMissingEntry::FieldId {
|
||||
field_id: id,
|
||||
process: "all_obkv_to_rhaimap",
|
||||
})?;
|
||||
let value = serde_json::from_slice(value).map_err(InternalError::SerdeJson)?;
|
||||
Ok((name.into(), value))
|
||||
})
|
||||
.collect();
|
||||
|
||||
map
|
||||
}
|
||||
|
||||
fn rhaimap_to_object(map: rhai::Map) -> Object {
|
||||
let mut output = Object::new();
|
||||
for (key, value) in map {
|
||||
let value = serde_json::to_value(&value).unwrap();
|
||||
output.insert(key.into(), value);
|
||||
}
|
||||
output
|
||||
}
|
255
crates/milli/src/update/new/merger.rs
Normal file
255
crates/milli/src/update/new/merger.rs
Normal file
|
@ -0,0 +1,255 @@
|
|||
use std::io::{self};
|
||||
|
||||
use bincode::ErrorKind;
|
||||
use hashbrown::HashSet;
|
||||
use heed::types::Bytes;
|
||||
use heed::{Database, RoTxn};
|
||||
use rayon::iter::{IntoParallelIterator, ParallelIterator};
|
||||
use roaring::RoaringBitmap;
|
||||
|
||||
use super::channel::*;
|
||||
use super::extract::{
|
||||
merge_caches, transpose_and_freeze_caches, BalancedCaches, DelAddRoaringBitmap, FacetKind,
|
||||
};
|
||||
use super::DocumentChange;
|
||||
use crate::{CboRoaringBitmapCodec, Error, FieldId, GeoPoint, GlobalFieldsIdsMap, Index, Result};
|
||||
|
||||
pub struct GeoExtractor {
|
||||
rtree: Option<rstar::RTree<GeoPoint>>,
|
||||
}
|
||||
|
||||
impl GeoExtractor {
|
||||
pub fn new(rtxn: &RoTxn, index: &Index) -> Result<Option<Self>> {
|
||||
let is_sortable = index.sortable_fields(rtxn)?.contains("_geo");
|
||||
let is_filterable = index.filterable_fields(rtxn)?.contains("_geo");
|
||||
if is_sortable || is_filterable {
|
||||
Ok(Some(GeoExtractor { rtree: index.geo_rtree(rtxn)? }))
|
||||
} else {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
pub fn manage_change(
|
||||
&mut self,
|
||||
fidmap: &mut GlobalFieldsIdsMap,
|
||||
change: &DocumentChange,
|
||||
) -> Result<()> {
|
||||
match change {
|
||||
DocumentChange::Deletion(_) => todo!(),
|
||||
DocumentChange::Update(_) => todo!(),
|
||||
DocumentChange::Insertion(_) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn serialize_rtree<W: io::Write>(self, writer: &mut W) -> Result<bool> {
|
||||
match self.rtree {
|
||||
Some(rtree) => {
|
||||
// TODO What should I do?
|
||||
bincode::serialize_into(writer, &rtree).map(|_| true).map_err(|e| match *e {
|
||||
ErrorKind::Io(e) => Error::IoError(e),
|
||||
ErrorKind::InvalidUtf8Encoding(_) => todo!(),
|
||||
ErrorKind::InvalidBoolEncoding(_) => todo!(),
|
||||
ErrorKind::InvalidCharEncoding => todo!(),
|
||||
ErrorKind::InvalidTagEncoding(_) => todo!(),
|
||||
ErrorKind::DeserializeAnyNotSupported => todo!(),
|
||||
ErrorKind::SizeLimit => todo!(),
|
||||
ErrorKind::SequenceMustHaveLength => todo!(),
|
||||
ErrorKind::Custom(_) => todo!(),
|
||||
})
|
||||
}
|
||||
None => Ok(false),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::merge")]
|
||||
pub fn merge_and_send_docids<'extractor>(
|
||||
mut caches: Vec<BalancedCaches<'extractor>>,
|
||||
database: Database<Bytes, Bytes>,
|
||||
index: &Index,
|
||||
docids_sender: impl DocidsSender + Sync,
|
||||
) -> Result<()> {
|
||||
transpose_and_freeze_caches(&mut caches)?.into_par_iter().try_for_each(|frozen| {
|
||||
let rtxn = index.read_txn()?;
|
||||
let mut buffer = Vec::new();
|
||||
merge_caches(frozen, |key, DelAddRoaringBitmap { del, add }| {
|
||||
let current = database.get(&rtxn, key)?;
|
||||
match merge_cbo_bitmaps(current, del, add)? {
|
||||
Operation::Write(bitmap) => {
|
||||
let value = cbo_bitmap_serialize_into_vec(&bitmap, &mut buffer);
|
||||
docids_sender.write(key, value).unwrap();
|
||||
Ok(())
|
||||
}
|
||||
Operation::Delete => {
|
||||
docids_sender.delete(key).unwrap();
|
||||
Ok(())
|
||||
}
|
||||
Operation::Ignore => Ok(()),
|
||||
}
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::merge")]
|
||||
pub fn merge_and_send_facet_docids<'extractor>(
|
||||
mut caches: Vec<BalancedCaches<'extractor>>,
|
||||
database: FacetDatabases,
|
||||
index: &Index,
|
||||
docids_sender: impl DocidsSender + Sync,
|
||||
) -> Result<FacetFieldIdsDelta> {
|
||||
transpose_and_freeze_caches(&mut caches)?
|
||||
.into_par_iter()
|
||||
.map(|frozen| {
|
||||
let mut facet_field_ids_delta = FacetFieldIdsDelta::default();
|
||||
let rtxn = index.read_txn()?;
|
||||
let mut buffer = Vec::new();
|
||||
merge_caches(frozen, |key, DelAddRoaringBitmap { del, add }| {
|
||||
let current = database.get_cbo_roaring_bytes_value(&rtxn, key)?;
|
||||
match merge_cbo_bitmaps(current, del, add)? {
|
||||
Operation::Write(bitmap) => {
|
||||
facet_field_ids_delta.register_from_key(key);
|
||||
let value = cbo_bitmap_serialize_into_vec(&bitmap, &mut buffer);
|
||||
docids_sender.write(key, value).unwrap();
|
||||
Ok(())
|
||||
}
|
||||
Operation::Delete => {
|
||||
facet_field_ids_delta.register_from_key(key);
|
||||
docids_sender.delete(key).unwrap();
|
||||
Ok(())
|
||||
}
|
||||
Operation::Ignore => Ok(()),
|
||||
}
|
||||
})?;
|
||||
|
||||
Ok(facet_field_ids_delta)
|
||||
})
|
||||
.reduce(|| Ok(FacetFieldIdsDelta::default()), |lhs, rhs| Ok(lhs?.merge(rhs?)))
|
||||
}
|
||||
|
||||
pub struct FacetDatabases<'a> {
|
||||
index: &'a Index,
|
||||
}
|
||||
|
||||
impl<'a> FacetDatabases<'a> {
|
||||
pub fn new(index: &'a Index) -> Self {
|
||||
Self { index }
|
||||
}
|
||||
|
||||
fn get_cbo_roaring_bytes_value<'t>(
|
||||
&self,
|
||||
rtxn: &'t RoTxn<'_>,
|
||||
key: &[u8],
|
||||
) -> heed::Result<Option<&'t [u8]>> {
|
||||
let (facet_kind, key) = FacetKind::extract_from_key(key);
|
||||
|
||||
let value =
|
||||
super::channel::Database::from(facet_kind).database(self.index).get(rtxn, key)?;
|
||||
match facet_kind {
|
||||
// skip level group size
|
||||
FacetKind::String | FacetKind::Number => Ok(value.map(|v| &v[1..])),
|
||||
_ => Ok(value),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Default)]
|
||||
pub struct FacetFieldIdsDelta {
|
||||
/// The field ids that have been modified
|
||||
modified_facet_string_ids: HashSet<FieldId>,
|
||||
modified_facet_number_ids: HashSet<FieldId>,
|
||||
}
|
||||
|
||||
impl FacetFieldIdsDelta {
|
||||
fn register_facet_string_id(&mut self, field_id: FieldId) {
|
||||
self.modified_facet_string_ids.insert(field_id);
|
||||
}
|
||||
|
||||
fn register_facet_number_id(&mut self, field_id: FieldId) {
|
||||
self.modified_facet_number_ids.insert(field_id);
|
||||
}
|
||||
|
||||
fn register_from_key(&mut self, key: &[u8]) {
|
||||
let (facet_kind, field_id) = self.extract_key_data(key);
|
||||
match facet_kind {
|
||||
FacetKind::Number => self.register_facet_number_id(field_id),
|
||||
FacetKind::String => self.register_facet_string_id(field_id),
|
||||
_ => (),
|
||||
}
|
||||
}
|
||||
|
||||
fn extract_key_data(&self, key: &[u8]) -> (FacetKind, FieldId) {
|
||||
let facet_kind = FacetKind::from(key[0]);
|
||||
let field_id = FieldId::from_be_bytes([key[1], key[2]]);
|
||||
(facet_kind, field_id)
|
||||
}
|
||||
|
||||
pub fn modified_facet_string_ids(&self) -> Option<Vec<FieldId>> {
|
||||
if self.modified_facet_string_ids.is_empty() {
|
||||
None
|
||||
} else {
|
||||
Some(self.modified_facet_string_ids.iter().copied().collect())
|
||||
}
|
||||
}
|
||||
|
||||
pub fn modified_facet_number_ids(&self) -> Option<Vec<FieldId>> {
|
||||
if self.modified_facet_number_ids.is_empty() {
|
||||
None
|
||||
} else {
|
||||
Some(self.modified_facet_number_ids.iter().copied().collect())
|
||||
}
|
||||
}
|
||||
|
||||
pub fn merge(mut self, rhs: Self) -> Self {
|
||||
let Self { modified_facet_number_ids, modified_facet_string_ids } = rhs;
|
||||
modified_facet_number_ids.into_iter().for_each(|fid| {
|
||||
self.modified_facet_number_ids.insert(fid);
|
||||
});
|
||||
modified_facet_string_ids.into_iter().for_each(|fid| {
|
||||
self.modified_facet_string_ids.insert(fid);
|
||||
});
|
||||
self
|
||||
}
|
||||
}
|
||||
|
||||
enum Operation {
|
||||
Write(RoaringBitmap),
|
||||
Delete,
|
||||
Ignore,
|
||||
}
|
||||
|
||||
/// A function that merges the DelAdd CboRoaringBitmaps with the current bitmap.
|
||||
fn merge_cbo_bitmaps(
|
||||
current: Option<&[u8]>,
|
||||
del: Option<RoaringBitmap>,
|
||||
add: Option<RoaringBitmap>,
|
||||
) -> Result<Operation> {
|
||||
let current = current.map(CboRoaringBitmapCodec::deserialize_from).transpose()?;
|
||||
match (current, del, add) {
|
||||
(None, None, None) => Ok(Operation::Ignore), // but it's strange
|
||||
(None, None, Some(add)) => Ok(Operation::Write(add)),
|
||||
(None, Some(_del), None) => Ok(Operation::Ignore), // but it's strange
|
||||
(None, Some(_del), Some(add)) => Ok(Operation::Write(add)),
|
||||
(Some(_current), None, None) => Ok(Operation::Ignore), // but it's strange
|
||||
(Some(current), None, Some(add)) => Ok(Operation::Write(current | add)),
|
||||
(Some(current), Some(del), add) => {
|
||||
let output = match add {
|
||||
Some(add) => (¤t - del) | add,
|
||||
None => ¤t - del,
|
||||
};
|
||||
if output.is_empty() {
|
||||
Ok(Operation::Delete)
|
||||
} else if current == output {
|
||||
Ok(Operation::Ignore)
|
||||
} else {
|
||||
Ok(Operation::Write(output))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// TODO Return the slice directly from the serialize_into method
|
||||
fn cbo_bitmap_serialize_into_vec<'b>(bitmap: &RoaringBitmap, buffer: &'b mut Vec<u8>) -> &'b [u8] {
|
||||
buffer.clear();
|
||||
CboRoaringBitmapCodec::serialize_into(bitmap, buffer);
|
||||
buffer.as_slice()
|
||||
}
|
29
crates/milli/src/update/new/mod.rs
Normal file
29
crates/milli/src/update/new/mod.rs
Normal file
|
@ -0,0 +1,29 @@
|
|||
pub use document_change::{Deletion, DocumentChange, Insertion, Update};
|
||||
pub use merger::{
|
||||
merge_and_send_docids, merge_and_send_facet_docids, FacetDatabases, FacetFieldIdsDelta,
|
||||
};
|
||||
pub use top_level_map::{CowStr, TopLevelMap};
|
||||
|
||||
use super::del_add::DelAdd;
|
||||
use crate::FieldId;
|
||||
|
||||
mod channel;
|
||||
pub mod document;
|
||||
mod document_change;
|
||||
mod extract;
|
||||
mod facet_search_builder;
|
||||
mod fst_merger_builder;
|
||||
pub mod indexer;
|
||||
mod merger;
|
||||
mod parallel_iterator_ext;
|
||||
mod top_level_map;
|
||||
pub mod vector_document;
|
||||
mod word_fst_builder;
|
||||
mod words_prefix_docids;
|
||||
|
||||
/// TODO move them elsewhere
|
||||
pub type StdResult<T, E> = std::result::Result<T, E>;
|
||||
pub type KvReaderDelAdd = obkv::KvReader<DelAdd>;
|
||||
pub type KvReaderFieldId = obkv::KvReader<FieldId>;
|
||||
pub type KvWriterDelAdd<W> = obkv::KvWriter<W, DelAdd>;
|
||||
pub type KvWriterFieldId<W> = obkv::KvWriter<W, FieldId>;
|
33
crates/milli/src/update/new/parallel_iterator_ext.rs
Normal file
33
crates/milli/src/update/new/parallel_iterator_ext.rs
Normal file
|
@ -0,0 +1,33 @@
|
|||
use std::sync::Arc;
|
||||
|
||||
use rayon::iter::ParallelIterator;
|
||||
|
||||
pub trait ParallelIteratorExt: ParallelIterator {
|
||||
/// A method to run a closure of all the items and return an owned error.
|
||||
///
|
||||
/// The init function is ran only as necessary which is basically once by thread.
|
||||
fn try_arc_for_each_try_init<F, INIT, T, E>(self, init: INIT, op: F) -> Result<(), E>
|
||||
where
|
||||
E: Send + Sync,
|
||||
F: Fn(&mut T, Self::Item) -> Result<(), Arc<E>> + Sync + Send + Clone,
|
||||
INIT: Fn() -> Result<T, E> + Sync + Send + Clone,
|
||||
{
|
||||
let result = self.try_for_each_init(
|
||||
move || match init() {
|
||||
Ok(t) => Ok(t),
|
||||
Err(err) => Err(Arc::new(err)),
|
||||
},
|
||||
move |result, item| match result {
|
||||
Ok(t) => op(t, item),
|
||||
Err(err) => Err(err.clone()),
|
||||
},
|
||||
);
|
||||
|
||||
match result {
|
||||
Ok(()) => Ok(()),
|
||||
Err(err) => Err(Arc::into_inner(err).expect("the error must be only owned by us")),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: ParallelIterator> ParallelIteratorExt for T {}
|
66
crates/milli/src/update/new/top_level_map.rs
Normal file
66
crates/milli/src/update/new/top_level_map.rs
Normal file
|
@ -0,0 +1,66 @@
|
|||
use std::borrow::{Borrow, Cow};
|
||||
use std::collections::BTreeMap;
|
||||
use std::{fmt, ops};
|
||||
|
||||
use serde::{Deserialize, Serialize};
|
||||
use serde_json::value::RawValue;
|
||||
use serde_json::{Map, Value};
|
||||
|
||||
#[derive(Deserialize, Serialize)]
|
||||
pub struct TopLevelMap<'p>(#[serde(borrow)] pub BTreeMap<CowStr<'p>, &'p RawValue>);
|
||||
|
||||
impl TryFrom<&'_ TopLevelMap<'_>> for Map<String, Value> {
|
||||
type Error = serde_json::Error;
|
||||
|
||||
fn try_from(tlmap: &TopLevelMap<'_>) -> Result<Self, Self::Error> {
|
||||
let mut object = Map::new();
|
||||
for (k, v) in &tlmap.0 {
|
||||
let value = serde_json::from_str(v.get())?;
|
||||
object.insert(k.to_string(), value);
|
||||
}
|
||||
Ok(object)
|
||||
}
|
||||
}
|
||||
|
||||
impl TryFrom<TopLevelMap<'_>> for Map<String, Value> {
|
||||
type Error = serde_json::Error;
|
||||
|
||||
fn try_from(tlmap: TopLevelMap<'_>) -> Result<Self, Self::Error> {
|
||||
TryFrom::try_from(&tlmap)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'p> ops::Deref for TopLevelMap<'p> {
|
||||
type Target = BTreeMap<CowStr<'p>, &'p RawValue>;
|
||||
|
||||
fn deref(&self) -> &Self::Target {
|
||||
&self.0
|
||||
}
|
||||
}
|
||||
|
||||
impl ops::DerefMut for TopLevelMap<'_> {
|
||||
fn deref_mut(&mut self) -> &mut Self::Target {
|
||||
&mut self.0
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Deserialize, Serialize, PartialEq, Eq, PartialOrd, Ord, Hash, Clone)]
|
||||
pub struct CowStr<'p>(#[serde(borrow)] pub Cow<'p, str>);
|
||||
|
||||
impl fmt::Display for CowStr<'_> {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
fmt::Display::fmt(&self.0, f)
|
||||
}
|
||||
}
|
||||
|
||||
impl AsRef<str> for CowStr<'_> {
|
||||
fn as_ref(&self) -> &str {
|
||||
self.0.as_ref()
|
||||
}
|
||||
}
|
||||
|
||||
impl<'doc> Borrow<str> for CowStr<'doc> {
|
||||
fn borrow(&self) -> &str {
|
||||
self.0.borrow()
|
||||
}
|
||||
}
|
300
crates/milli/src/update/new/vector_document.rs
Normal file
300
crates/milli/src/update/new/vector_document.rs
Normal file
|
@ -0,0 +1,300 @@
|
|||
use std::collections::BTreeSet;
|
||||
|
||||
use bumpalo::Bump;
|
||||
use deserr::{Deserr, IntoValue};
|
||||
use heed::RoTxn;
|
||||
use raw_collections::RawMap;
|
||||
use serde::Serialize;
|
||||
use serde_json::value::RawValue;
|
||||
|
||||
use super::document::{Document, DocumentFromDb, DocumentFromVersions, Versions};
|
||||
use super::indexer::de::DeserrRawValue;
|
||||
use crate::documents::FieldIdMapper;
|
||||
use crate::index::IndexEmbeddingConfig;
|
||||
use crate::vector::parsed_vectors::{
|
||||
RawVectors, VectorOrArrayOfVectors, RESERVED_VECTORS_FIELD_NAME,
|
||||
};
|
||||
use crate::vector::{Embedding, EmbeddingConfigs};
|
||||
use crate::{DocumentId, Index, InternalError, Result, UserError};
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(untagged)]
|
||||
pub enum Embeddings<'doc> {
|
||||
FromJsonExplicit(&'doc RawValue),
|
||||
FromJsonImplicityUserProvided(&'doc RawValue),
|
||||
FromDb(Vec<Embedding>),
|
||||
}
|
||||
impl<'doc> Embeddings<'doc> {
|
||||
pub fn into_vec(
|
||||
self,
|
||||
doc_alloc: &'doc Bump,
|
||||
embedder_name: &str,
|
||||
) -> std::result::Result<Vec<Embedding>, deserr::errors::JsonError> {
|
||||
match self {
|
||||
Embeddings::FromJsonExplicit(value) => {
|
||||
let vectors_ref = deserr::ValuePointerRef::Key {
|
||||
key: RESERVED_VECTORS_FIELD_NAME,
|
||||
prev: &deserr::ValuePointerRef::Origin,
|
||||
};
|
||||
let embedders_ref =
|
||||
deserr::ValuePointerRef::Key { key: embedder_name, prev: &vectors_ref };
|
||||
|
||||
let embeddings_ref =
|
||||
deserr::ValuePointerRef::Key { key: "embeddings", prev: &embedders_ref };
|
||||
|
||||
let v: VectorOrArrayOfVectors = VectorOrArrayOfVectors::deserialize_from_value(
|
||||
DeserrRawValue::new_in(value, doc_alloc).into_value(),
|
||||
embeddings_ref,
|
||||
)?;
|
||||
Ok(v.into_array_of_vectors().unwrap_or_default())
|
||||
}
|
||||
Embeddings::FromJsonImplicityUserProvided(value) => {
|
||||
let vectors_ref = deserr::ValuePointerRef::Key {
|
||||
key: RESERVED_VECTORS_FIELD_NAME,
|
||||
prev: &deserr::ValuePointerRef::Origin,
|
||||
};
|
||||
let embedders_ref =
|
||||
deserr::ValuePointerRef::Key { key: embedder_name, prev: &vectors_ref };
|
||||
|
||||
let v: VectorOrArrayOfVectors = VectorOrArrayOfVectors::deserialize_from_value(
|
||||
DeserrRawValue::new_in(value, doc_alloc).into_value(),
|
||||
embedders_ref,
|
||||
)?;
|
||||
Ok(v.into_array_of_vectors().unwrap_or_default())
|
||||
}
|
||||
Embeddings::FromDb(vec) => Ok(vec),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct VectorEntry<'doc> {
|
||||
pub has_configured_embedder: bool,
|
||||
pub embeddings: Option<Embeddings<'doc>>,
|
||||
pub regenerate: bool,
|
||||
}
|
||||
|
||||
pub trait VectorDocument<'doc> {
|
||||
fn iter_vectors(&self) -> impl Iterator<Item = Result<(&'doc str, VectorEntry<'doc>)>>;
|
||||
|
||||
fn vectors_for_key(&self, key: &str) -> Result<Option<VectorEntry<'doc>>>;
|
||||
}
|
||||
|
||||
pub struct VectorDocumentFromDb<'t> {
|
||||
docid: DocumentId,
|
||||
embedding_config: Vec<IndexEmbeddingConfig>,
|
||||
index: &'t Index,
|
||||
vectors_field: Option<RawMap<'t>>,
|
||||
rtxn: &'t RoTxn<'t>,
|
||||
doc_alloc: &'t Bump,
|
||||
}
|
||||
|
||||
impl<'t> VectorDocumentFromDb<'t> {
|
||||
pub fn new<Mapper: FieldIdMapper>(
|
||||
docid: DocumentId,
|
||||
index: &'t Index,
|
||||
rtxn: &'t RoTxn,
|
||||
db_fields_ids_map: &'t Mapper,
|
||||
doc_alloc: &'t Bump,
|
||||
) -> Result<Option<Self>> {
|
||||
let Some(document) = DocumentFromDb::new(docid, rtxn, index, db_fields_ids_map)? else {
|
||||
return Ok(None);
|
||||
};
|
||||
let vectors = document.vectors_field()?;
|
||||
let vectors_field = match vectors {
|
||||
Some(vectors) => {
|
||||
Some(RawMap::from_raw_value(vectors, doc_alloc).map_err(InternalError::SerdeJson)?)
|
||||
}
|
||||
None => None,
|
||||
};
|
||||
|
||||
let embedding_config = index.embedding_configs(rtxn)?;
|
||||
|
||||
Ok(Some(Self { docid, embedding_config, index, vectors_field, rtxn, doc_alloc }))
|
||||
}
|
||||
|
||||
fn entry_from_db(
|
||||
&self,
|
||||
embedder_id: u8,
|
||||
config: &IndexEmbeddingConfig,
|
||||
) -> Result<VectorEntry<'t>> {
|
||||
let readers = self.index.arroy_readers(self.rtxn, embedder_id, config.config.quantized());
|
||||
let mut vectors = Vec::new();
|
||||
for reader in readers {
|
||||
let reader = reader?;
|
||||
let Some(vector) = reader.item_vector(self.rtxn, self.docid)? else {
|
||||
break;
|
||||
};
|
||||
|
||||
vectors.push(vector);
|
||||
}
|
||||
Ok(VectorEntry {
|
||||
has_configured_embedder: true,
|
||||
embeddings: Some(Embeddings::FromDb(vectors)),
|
||||
regenerate: !config.user_provided.contains(self.docid),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl<'t> VectorDocument<'t> for VectorDocumentFromDb<'t> {
|
||||
fn iter_vectors(&self) -> impl Iterator<Item = Result<(&'t str, VectorEntry<'t>)>> {
|
||||
self.embedding_config
|
||||
.iter()
|
||||
.map(|config| {
|
||||
let embedder_id =
|
||||
self.index.embedder_category_id.get(self.rtxn, &config.name)?.unwrap();
|
||||
let entry = self.entry_from_db(embedder_id, config)?;
|
||||
let config_name = self.doc_alloc.alloc_str(config.name.as_str());
|
||||
Ok((&*config_name, entry))
|
||||
})
|
||||
.chain(self.vectors_field.iter().flat_map(|map| map.iter()).map(|(name, value)| {
|
||||
Ok((name, entry_from_raw_value(value, false).map_err(InternalError::SerdeJson)?))
|
||||
}))
|
||||
}
|
||||
|
||||
fn vectors_for_key(&self, key: &str) -> Result<Option<VectorEntry<'t>>> {
|
||||
Ok(match self.index.embedder_category_id.get(self.rtxn, key)? {
|
||||
Some(embedder_id) => {
|
||||
let config =
|
||||
self.embedding_config.iter().find(|config| config.name == key).unwrap();
|
||||
Some(self.entry_from_db(embedder_id, config)?)
|
||||
}
|
||||
None => match self.vectors_field.as_ref().and_then(|obkv| obkv.get(key)) {
|
||||
Some(embedding_from_doc) => Some(
|
||||
entry_from_raw_value(embedding_from_doc, false)
|
||||
.map_err(InternalError::SerdeJson)?,
|
||||
),
|
||||
None => None,
|
||||
},
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
fn entry_from_raw_value(
|
||||
value: &RawValue,
|
||||
has_configured_embedder: bool,
|
||||
) -> std::result::Result<VectorEntry<'_>, serde_json::Error> {
|
||||
let value: RawVectors = serde_json::from_str(value.get())?;
|
||||
|
||||
Ok(match value {
|
||||
RawVectors::Explicit(raw_explicit_vectors) => VectorEntry {
|
||||
has_configured_embedder,
|
||||
embeddings: raw_explicit_vectors.embeddings.map(Embeddings::FromJsonExplicit),
|
||||
regenerate: raw_explicit_vectors.regenerate,
|
||||
},
|
||||
RawVectors::ImplicitlyUserProvided(value) => VectorEntry {
|
||||
has_configured_embedder,
|
||||
embeddings: Some(Embeddings::FromJsonImplicityUserProvided(value)),
|
||||
regenerate: false,
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
pub struct VectorDocumentFromVersions<'doc> {
|
||||
vectors: RawMap<'doc>,
|
||||
embedders: &'doc EmbeddingConfigs,
|
||||
}
|
||||
|
||||
impl<'doc> VectorDocumentFromVersions<'doc> {
|
||||
pub fn new(
|
||||
versions: &Versions<'doc>,
|
||||
bump: &'doc Bump,
|
||||
embedders: &'doc EmbeddingConfigs,
|
||||
) -> Result<Option<Self>> {
|
||||
let document = DocumentFromVersions::new(versions);
|
||||
if let Some(vectors_field) = document.vectors_field()? {
|
||||
let vectors =
|
||||
RawMap::from_raw_value(vectors_field, bump).map_err(UserError::SerdeJson)?;
|
||||
Ok(Some(Self { vectors, embedders }))
|
||||
} else {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<'doc> VectorDocument<'doc> for VectorDocumentFromVersions<'doc> {
|
||||
fn iter_vectors(&self) -> impl Iterator<Item = Result<(&'doc str, VectorEntry<'doc>)>> {
|
||||
self.vectors.iter().map(|(embedder, vectors)| {
|
||||
let vectors = entry_from_raw_value(vectors, self.embedders.contains(embedder))
|
||||
.map_err(UserError::SerdeJson)?;
|
||||
Ok((embedder, vectors))
|
||||
})
|
||||
}
|
||||
|
||||
fn vectors_for_key(&self, key: &str) -> Result<Option<VectorEntry<'doc>>> {
|
||||
let Some(vectors) = self.vectors.get(key) else { return Ok(None) };
|
||||
let vectors = entry_from_raw_value(vectors, self.embedders.contains(key))
|
||||
.map_err(UserError::SerdeJson)?;
|
||||
Ok(Some(vectors))
|
||||
}
|
||||
}
|
||||
|
||||
pub struct MergedVectorDocument<'doc> {
|
||||
new_doc: Option<VectorDocumentFromVersions<'doc>>,
|
||||
db: Option<VectorDocumentFromDb<'doc>>,
|
||||
}
|
||||
|
||||
impl<'doc> MergedVectorDocument<'doc> {
|
||||
pub fn with_db<Mapper: FieldIdMapper>(
|
||||
docid: DocumentId,
|
||||
index: &'doc Index,
|
||||
rtxn: &'doc RoTxn,
|
||||
db_fields_ids_map: &'doc Mapper,
|
||||
versions: &Versions<'doc>,
|
||||
doc_alloc: &'doc Bump,
|
||||
embedders: &'doc EmbeddingConfigs,
|
||||
) -> Result<Option<Self>> {
|
||||
let db = VectorDocumentFromDb::new(docid, index, rtxn, db_fields_ids_map, doc_alloc)?;
|
||||
let new_doc = VectorDocumentFromVersions::new(versions, doc_alloc, embedders)?;
|
||||
Ok(if db.is_none() && new_doc.is_none() { None } else { Some(Self { new_doc, db }) })
|
||||
}
|
||||
|
||||
pub fn without_db(
|
||||
versions: &Versions<'doc>,
|
||||
doc_alloc: &'doc Bump,
|
||||
embedders: &'doc EmbeddingConfigs,
|
||||
) -> Result<Option<Self>> {
|
||||
let Some(new_doc) = VectorDocumentFromVersions::new(versions, doc_alloc, embedders)? else {
|
||||
return Ok(None);
|
||||
};
|
||||
Ok(Some(Self { new_doc: Some(new_doc), db: None }))
|
||||
}
|
||||
}
|
||||
|
||||
impl<'doc> VectorDocument<'doc> for MergedVectorDocument<'doc> {
|
||||
fn iter_vectors(&self) -> impl Iterator<Item = Result<(&'doc str, VectorEntry<'doc>)>> {
|
||||
let mut new_doc_it = self.new_doc.iter().flat_map(|new_doc| new_doc.iter_vectors());
|
||||
let mut db_it = self.db.iter().flat_map(|db| db.iter_vectors());
|
||||
let mut seen_fields = BTreeSet::new();
|
||||
|
||||
std::iter::from_fn(move || {
|
||||
if let Some(next) = new_doc_it.next() {
|
||||
if let Ok((name, _)) = next {
|
||||
seen_fields.insert(name);
|
||||
}
|
||||
return Some(next);
|
||||
}
|
||||
loop {
|
||||
match db_it.next()? {
|
||||
Ok((name, value)) => {
|
||||
if seen_fields.contains(name) {
|
||||
continue;
|
||||
}
|
||||
return Some(Ok((name, value)));
|
||||
}
|
||||
Err(err) => return Some(Err(err)),
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
fn vectors_for_key(&self, key: &str) -> Result<Option<VectorEntry<'doc>>> {
|
||||
if let Some(new_doc) = &self.new_doc {
|
||||
if let Some(entry) = new_doc.vectors_for_key(key)? {
|
||||
return Ok(Some(entry));
|
||||
}
|
||||
}
|
||||
|
||||
let Some(db) = self.db.as_ref() else { return Ok(None) };
|
||||
db.vectors_for_key(key)
|
||||
}
|
||||
}
|
198
crates/milli/src/update/new/word_fst_builder.rs
Normal file
198
crates/milli/src/update/new/word_fst_builder.rs
Normal file
|
@ -0,0 +1,198 @@
|
|||
use std::io::BufWriter;
|
||||
|
||||
use fst::{Set, SetBuilder, Streamer};
|
||||
use memmap2::Mmap;
|
||||
use std::collections::HashSet;
|
||||
use tempfile::tempfile;
|
||||
|
||||
use crate::{index::PrefixSettings, update::del_add::DelAdd, InternalError, Prefix, Result};
|
||||
|
||||
use super::fst_merger_builder::FstMergerBuilder;
|
||||
|
||||
pub struct WordFstBuilder<'a> {
|
||||
word_fst_builder: FstMergerBuilder<'a>,
|
||||
prefix_fst_builder: Option<PrefixFstBuilder>,
|
||||
registered_words: usize,
|
||||
}
|
||||
|
||||
impl<'a> WordFstBuilder<'a> {
|
||||
pub fn new(words_fst: &'a Set<std::borrow::Cow<'a, [u8]>>) -> Result<Self> {
|
||||
Ok(Self {
|
||||
word_fst_builder: FstMergerBuilder::new(Some(words_fst))?,
|
||||
prefix_fst_builder: None,
|
||||
registered_words: 0,
|
||||
})
|
||||
}
|
||||
|
||||
pub fn with_prefix_settings(&mut self, prefix_settings: PrefixSettings) -> &Self {
|
||||
self.prefix_fst_builder = PrefixFstBuilder::new(prefix_settings);
|
||||
self
|
||||
}
|
||||
|
||||
pub fn register_word(&mut self, deladd: DelAdd, right: &[u8]) -> Result<()> {
|
||||
if deladd == DelAdd::Addition {
|
||||
self.registered_words += 1;
|
||||
}
|
||||
|
||||
self.word_fst_builder.register(deladd, right, &mut |bytes, deladd, is_modified| {
|
||||
if let Some(prefix_fst_builder) = &mut self.prefix_fst_builder {
|
||||
prefix_fst_builder.insert_word(bytes, deladd, is_modified)
|
||||
} else {
|
||||
Ok(())
|
||||
}
|
||||
})?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn build(
|
||||
mut self,
|
||||
index: &crate::Index,
|
||||
rtxn: &heed::RoTxn,
|
||||
) -> Result<(Mmap, Option<PrefixData>)> {
|
||||
let words_fst_mmap = self.word_fst_builder.build(&mut |bytes, deladd, is_modified| {
|
||||
if let Some(prefix_fst_builder) = &mut self.prefix_fst_builder {
|
||||
prefix_fst_builder.insert_word(bytes, deladd, is_modified)
|
||||
} else {
|
||||
Ok(())
|
||||
}
|
||||
})?;
|
||||
|
||||
let prefix_data = self
|
||||
.prefix_fst_builder
|
||||
.map(|prefix_fst_builder| prefix_fst_builder.build(index, rtxn))
|
||||
.transpose()?;
|
||||
|
||||
Ok((words_fst_mmap, prefix_data))
|
||||
}
|
||||
}
|
||||
|
||||
pub struct PrefixData {
|
||||
pub prefixes_fst_mmap: Mmap,
|
||||
pub prefix_delta: PrefixDelta,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct PrefixDelta {
|
||||
pub modified: HashSet<Prefix>,
|
||||
pub deleted: HashSet<Prefix>,
|
||||
}
|
||||
|
||||
struct PrefixFstBuilder {
|
||||
prefix_count_threshold: u64,
|
||||
max_prefix_length: usize,
|
||||
/// TODO: Replace the full memory allocation
|
||||
prefix_fst_builders: Vec<SetBuilder<Vec<u8>>>,
|
||||
current_prefix: Vec<Prefix>,
|
||||
current_prefix_count: Vec<u64>,
|
||||
modified_prefixes: HashSet<Prefix>,
|
||||
current_prefix_is_modified: Vec<bool>,
|
||||
}
|
||||
|
||||
impl PrefixFstBuilder {
|
||||
pub fn new(prefix_settings: PrefixSettings) -> Option<Self> {
|
||||
let PrefixSettings { prefix_count_threshold, max_prefix_length, compute_prefixes } =
|
||||
prefix_settings;
|
||||
|
||||
if !compute_prefixes {
|
||||
return None;
|
||||
}
|
||||
|
||||
let mut prefix_fst_builders = Vec::new();
|
||||
for _ in 0..max_prefix_length {
|
||||
prefix_fst_builders.push(SetBuilder::memory());
|
||||
}
|
||||
|
||||
Some(Self {
|
||||
prefix_count_threshold,
|
||||
max_prefix_length,
|
||||
prefix_fst_builders,
|
||||
current_prefix: vec![Prefix::new(); max_prefix_length],
|
||||
current_prefix_count: vec![0; max_prefix_length],
|
||||
modified_prefixes: HashSet::new(),
|
||||
current_prefix_is_modified: vec![false; max_prefix_length],
|
||||
})
|
||||
}
|
||||
|
||||
fn insert_word(&mut self, bytes: &[u8], deladd: DelAdd, is_modified: bool) -> Result<()> {
|
||||
for n in 0..self.max_prefix_length {
|
||||
let current_prefix = &mut self.current_prefix[n];
|
||||
let current_prefix_count = &mut self.current_prefix_count[n];
|
||||
let builder = &mut self.prefix_fst_builders[n];
|
||||
let current_prefix_is_modified = &mut self.current_prefix_is_modified[n];
|
||||
|
||||
// We try to get the first n bytes out of this string but we only want
|
||||
// to split at valid characters bounds. If we try to split in the middle of
|
||||
// a character we ignore this word and go to the next one.
|
||||
let word = std::str::from_utf8(bytes)?;
|
||||
let prefix = match word.get(..=n) {
|
||||
Some(prefix) => prefix,
|
||||
None => continue,
|
||||
};
|
||||
|
||||
// This is the first iteration of the loop,
|
||||
// or the current word doesn't starts with the current prefix.
|
||||
if *current_prefix_count == 0 || prefix != current_prefix.as_str() {
|
||||
*current_prefix = Prefix::from(prefix);
|
||||
*current_prefix_count = 0;
|
||||
*current_prefix_is_modified = false;
|
||||
}
|
||||
|
||||
if deladd == DelAdd::Addition {
|
||||
*current_prefix_count += 1;
|
||||
}
|
||||
|
||||
if is_modified && !*current_prefix_is_modified {
|
||||
if *current_prefix_count > self.prefix_count_threshold {
|
||||
self.modified_prefixes.insert(current_prefix.clone());
|
||||
}
|
||||
|
||||
*current_prefix_is_modified = true;
|
||||
}
|
||||
|
||||
// There is enough words corresponding to this prefix to add it to the cache.
|
||||
if *current_prefix_count == self.prefix_count_threshold {
|
||||
builder.insert(prefix)?;
|
||||
|
||||
if *current_prefix_is_modified {
|
||||
self.modified_prefixes.insert(current_prefix.clone());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn build(self, index: &crate::Index, rtxn: &heed::RoTxn) -> Result<PrefixData> {
|
||||
// We merge all of the previously computed prefixes into on final set.
|
||||
let mut prefix_fsts = Vec::new();
|
||||
for builder in self.prefix_fst_builders.into_iter() {
|
||||
let prefix_fst = builder.into_set();
|
||||
prefix_fsts.push(prefix_fst);
|
||||
}
|
||||
let op = fst::set::OpBuilder::from_iter(prefix_fsts.iter());
|
||||
let mut builder = SetBuilder::new(BufWriter::new(tempfile()?))?;
|
||||
builder.extend_stream(op.r#union())?;
|
||||
let prefix_fst_file = builder.into_inner()?.into_inner().map_err(|_| {
|
||||
InternalError::IndexingMergingKeys { process: "building-words-prefixes-fst" }
|
||||
})?;
|
||||
let prefix_fst_mmap = unsafe { Mmap::map(&prefix_fst_file)? };
|
||||
let new_prefix_fst = Set::new(&prefix_fst_mmap)?;
|
||||
let old_prefix_fst = index.words_prefixes_fst(rtxn)?;
|
||||
let mut deleted_prefixes = HashSet::new();
|
||||
{
|
||||
let mut deleted_prefixes_stream = old_prefix_fst.op().add(&new_prefix_fst).difference();
|
||||
while let Some(prefix) = deleted_prefixes_stream.next() {
|
||||
deleted_prefixes.insert(Prefix::from(std::str::from_utf8(prefix)?));
|
||||
}
|
||||
}
|
||||
|
||||
Ok(PrefixData {
|
||||
prefixes_fst_mmap: prefix_fst_mmap,
|
||||
prefix_delta: PrefixDelta {
|
||||
modified: self.modified_prefixes,
|
||||
deleted: deleted_prefixes,
|
||||
},
|
||||
})
|
||||
}
|
||||
}
|
282
crates/milli/src/update/new/words_prefix_docids.rs
Normal file
282
crates/milli/src/update/new/words_prefix_docids.rs
Normal file
|
@ -0,0 +1,282 @@
|
|||
use std::cell::RefCell;
|
||||
use std::collections::HashSet;
|
||||
use std::io::{BufReader, BufWriter, Read, Seek, Write};
|
||||
|
||||
use hashbrown::HashMap;
|
||||
use heed::types::Bytes;
|
||||
use heed::{BytesDecode, Database, RoTxn, RwTxn};
|
||||
use rayon::iter::{IntoParallelIterator, ParallelIterator as _};
|
||||
use roaring::MultiOps;
|
||||
use tempfile::tempfile;
|
||||
use thread_local::ThreadLocal;
|
||||
|
||||
use super::indexer::document_changes::RefCellExt;
|
||||
use crate::heed_codec::StrBEU16Codec;
|
||||
use crate::{CboRoaringBitmapCodec, Index, Prefix, Result};
|
||||
|
||||
struct WordPrefixDocids {
|
||||
database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
prefix_database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
}
|
||||
|
||||
impl WordPrefixDocids {
|
||||
fn new(
|
||||
database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
prefix_database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
) -> WordPrefixDocids {
|
||||
WordPrefixDocids { database, prefix_database }
|
||||
}
|
||||
|
||||
fn execute(
|
||||
self,
|
||||
wtxn: &mut heed::RwTxn,
|
||||
prefix_to_compute: &HashSet<Prefix>,
|
||||
prefix_to_delete: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
delete_prefixes(wtxn, &self.prefix_database, prefix_to_delete)?;
|
||||
self.recompute_modified_prefixes(wtxn, prefix_to_compute)
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
fn recompute_modified_prefixes(
|
||||
&self,
|
||||
wtxn: &mut RwTxn,
|
||||
prefixes: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
// We fetch the docids associated to the newly added word prefix fst only.
|
||||
// And collect the CboRoaringBitmaps pointers in an HashMap.
|
||||
let frozen = FrozenPrefixBitmaps::from_prefixes(self.database, wtxn, prefixes)?;
|
||||
|
||||
// We access this HashMap in parallel to compute the *union* of all
|
||||
// of them and *serialize* them into files. There is one file by CPU.
|
||||
let local_entries = ThreadLocal::with_capacity(rayon::current_num_threads());
|
||||
prefixes.into_par_iter().map(AsRef::as_ref).try_for_each(|prefix| {
|
||||
let refcell = local_entries.get_or_try(|| {
|
||||
tempfile().map(BufWriter::new).map(|f| RefCell::new((Vec::new(), f, Vec::new())))
|
||||
})?;
|
||||
|
||||
let mut refmut = refcell.borrow_mut_or_yield();
|
||||
let (ref mut index, ref mut file, ref mut buffer) = *refmut;
|
||||
|
||||
let output = frozen
|
||||
.bitmaps(prefix)
|
||||
.unwrap()
|
||||
.iter()
|
||||
.map(|bytes| CboRoaringBitmapCodec::deserialize_from(bytes))
|
||||
.union()?;
|
||||
|
||||
buffer.clear();
|
||||
CboRoaringBitmapCodec::serialize_into(&output, buffer);
|
||||
index.push(PrefixEntry { prefix, serialized_length: buffer.len() });
|
||||
file.write_all(buffer)
|
||||
})?;
|
||||
|
||||
drop(frozen);
|
||||
|
||||
// We iterate over all the collected and serialized bitmaps through
|
||||
// the files and entries to eventually put them in the final database.
|
||||
for refcell in local_entries {
|
||||
let (index, file, mut buffer) = refcell.into_inner();
|
||||
let mut file = file.into_inner().map_err(|e| e.into_error())?;
|
||||
file.rewind()?;
|
||||
let mut file = BufReader::new(file);
|
||||
for PrefixEntry { prefix, serialized_length } in index {
|
||||
buffer.resize(serialized_length, 0);
|
||||
file.read_exact(&mut buffer)?;
|
||||
self.prefix_database.remap_data_type::<Bytes>().put(
|
||||
wtxn,
|
||||
prefix.as_bytes(),
|
||||
&buffer,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
/// Represents a prefix and the lenght the bitmap takes on disk.
|
||||
struct PrefixEntry<'a> {
|
||||
prefix: &'a str,
|
||||
serialized_length: usize,
|
||||
}
|
||||
|
||||
/// Stores prefixes along with all the pointers to the associated
|
||||
/// CBoRoaringBitmaps.
|
||||
///
|
||||
/// They are collected synchronously and stored into an HashMap. The
|
||||
/// Synchronous process is doing a small amount of work by just storing
|
||||
/// pointers. It can then be accessed in parallel to get the associated
|
||||
/// bitmaps pointers.
|
||||
struct FrozenPrefixBitmaps<'a, 'rtxn> {
|
||||
prefixes_bitmaps: HashMap<&'a str, Vec<&'rtxn [u8]>>,
|
||||
}
|
||||
|
||||
impl<'a, 'rtxn> FrozenPrefixBitmaps<'a, 'rtxn> {
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
pub fn from_prefixes(
|
||||
database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
rtxn: &'rtxn RoTxn,
|
||||
prefixes: &'a HashSet<Prefix>,
|
||||
) -> heed::Result<Self> {
|
||||
let database = database.remap_data_type::<Bytes>();
|
||||
|
||||
let mut prefixes_bitmaps = HashMap::new();
|
||||
for prefix in prefixes {
|
||||
let mut bitmap_bytes = Vec::new();
|
||||
for result in database.prefix_iter(rtxn, prefix.as_bytes())? {
|
||||
let (_word, bytes) = result?;
|
||||
bitmap_bytes.push(bytes);
|
||||
}
|
||||
assert!(prefixes_bitmaps.insert(prefix.as_str(), bitmap_bytes).is_none());
|
||||
}
|
||||
|
||||
Ok(Self { prefixes_bitmaps })
|
||||
}
|
||||
|
||||
pub fn bitmaps(&self, key: &str) -> Option<&[&'rtxn [u8]]> {
|
||||
self.prefixes_bitmaps.get(key).map(AsRef::as_ref)
|
||||
}
|
||||
}
|
||||
|
||||
unsafe impl<'a, 'rtxn> Sync for FrozenPrefixBitmaps<'a, 'rtxn> {}
|
||||
|
||||
struct WordPrefixIntegerDocids {
|
||||
database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
prefix_database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
}
|
||||
|
||||
impl WordPrefixIntegerDocids {
|
||||
fn new(
|
||||
database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
prefix_database: Database<Bytes, CboRoaringBitmapCodec>,
|
||||
) -> WordPrefixIntegerDocids {
|
||||
WordPrefixIntegerDocids { database, prefix_database }
|
||||
}
|
||||
|
||||
fn execute(
|
||||
self,
|
||||
wtxn: &mut heed::RwTxn,
|
||||
prefix_to_compute: &HashSet<Prefix>,
|
||||
prefix_to_delete: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
delete_prefixes(wtxn, &self.prefix_database, prefix_to_delete)?;
|
||||
self.recompute_modified_prefixes(wtxn, prefix_to_compute)
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
fn recompute_modified_prefixes(
|
||||
&self,
|
||||
wtxn: &mut RwTxn,
|
||||
prefixes: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
// We fetch the docids associated to the newly added word prefix fst only.
|
||||
// We use a HashMap to store the docids associated to each position, may be RAM consuming.
|
||||
let mut integer_docids = HashMap::new();
|
||||
let mut key_buffer = Vec::new();
|
||||
for prefix in prefixes {
|
||||
let prefix = prefix.as_bytes();
|
||||
for result in self.database.prefix_iter(wtxn, prefix)? {
|
||||
let (key, data) = result?;
|
||||
let (_word, pos) =
|
||||
StrBEU16Codec::bytes_decode(key).map_err(heed::Error::Decoding)?;
|
||||
|
||||
match integer_docids.get_mut(&pos) {
|
||||
Some(docids) => {
|
||||
*docids |= &data;
|
||||
}
|
||||
None => {
|
||||
integer_docids.insert(pos, data);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (pos, docids) in integer_docids.iter_mut() {
|
||||
if !docids.is_empty() {
|
||||
key_buffer.clear();
|
||||
key_buffer.extend_from_slice(prefix);
|
||||
key_buffer.push(0);
|
||||
key_buffer.extend_from_slice(&pos.to_be_bytes());
|
||||
self.prefix_database.put(wtxn, &key_buffer, docids)?;
|
||||
}
|
||||
docids.clear();
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
fn delete_prefixes(
|
||||
wtxn: &mut RwTxn,
|
||||
prefix_database: &Database<Bytes, CboRoaringBitmapCodec>,
|
||||
prefixes: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
// We remove all the entries that are no more required in this word prefix docids database.
|
||||
for prefix in prefixes {
|
||||
let mut iter = prefix_database.prefix_iter_mut(wtxn, prefix.as_bytes())?;
|
||||
while iter.next().transpose()?.is_some() {
|
||||
// safety: we do not keep a reference on database entries.
|
||||
unsafe { iter.del_current()? };
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
pub fn compute_word_prefix_docids(
|
||||
wtxn: &mut RwTxn,
|
||||
index: &Index,
|
||||
prefix_to_compute: &HashSet<Prefix>,
|
||||
prefix_to_delete: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
WordPrefixDocids::new(
|
||||
index.word_docids.remap_key_type(),
|
||||
index.word_prefix_docids.remap_key_type(),
|
||||
)
|
||||
.execute(wtxn, prefix_to_compute, prefix_to_delete)
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
pub fn compute_exact_word_prefix_docids(
|
||||
wtxn: &mut RwTxn,
|
||||
index: &Index,
|
||||
prefix_to_compute: &HashSet<Prefix>,
|
||||
prefix_to_delete: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
WordPrefixDocids::new(
|
||||
index.exact_word_docids.remap_key_type(),
|
||||
index.exact_word_prefix_docids.remap_key_type(),
|
||||
)
|
||||
.execute(wtxn, prefix_to_compute, prefix_to_delete)
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
pub fn compute_word_prefix_fid_docids(
|
||||
wtxn: &mut RwTxn,
|
||||
index: &Index,
|
||||
prefix_to_compute: &HashSet<Prefix>,
|
||||
prefix_to_delete: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
WordPrefixIntegerDocids::new(
|
||||
index.word_fid_docids.remap_key_type(),
|
||||
index.word_prefix_fid_docids.remap_key_type(),
|
||||
)
|
||||
.execute(wtxn, prefix_to_compute, prefix_to_delete)
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::prefix")]
|
||||
pub fn compute_word_prefix_position_docids(
|
||||
wtxn: &mut RwTxn,
|
||||
index: &Index,
|
||||
prefix_to_compute: &HashSet<Prefix>,
|
||||
prefix_to_delete: &HashSet<Prefix>,
|
||||
) -> Result<()> {
|
||||
WordPrefixIntegerDocids::new(
|
||||
index.word_position_docids.remap_key_type(),
|
||||
index.word_prefix_position_docids.remap_key_type(),
|
||||
)
|
||||
.execute(wtxn, prefix_to_compute, prefix_to_delete)
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue