2020-10-23 14:11:00 +02:00
|
|
|
use std::borrow::Cow;
|
2023-10-30 14:47:51 +01:00
|
|
|
use std::collections::btree_map::Entry as BEntry;
|
|
|
|
use std::collections::hash_map::Entry as HEntry;
|
2024-04-03 11:19:45 +02:00
|
|
|
use std::collections::HashMap;
|
2020-10-23 14:11:00 +02:00
|
|
|
use std::fs::File;
|
2023-01-30 17:17:35 +01:00
|
|
|
use std::io::{Read, Seek};
|
2020-10-23 14:11:00 +02:00
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
use fxhash::FxHashMap;
|
2021-08-31 11:44:15 +02:00
|
|
|
use itertools::Itertools;
|
2023-10-12 11:46:56 +02:00
|
|
|
use obkv::{KvReader, KvReaderU16, KvWriter};
|
2020-10-23 14:11:00 +02:00
|
|
|
use roaring::RoaringBitmap;
|
2022-06-21 11:12:51 +02:00
|
|
|
use serde_json::Value;
|
2022-04-11 15:43:18 +02:00
|
|
|
use smartstring::SmartString;
|
2020-10-23 14:11:00 +02:00
|
|
|
|
2023-02-16 18:42:47 +01:00
|
|
|
use super::helpers::{
|
2023-11-02 13:37:54 +01:00
|
|
|
create_sorter, create_writer, keep_first, obkvs_keep_last_addition_merge_deletions,
|
|
|
|
obkvs_merge_additions_and_deletions, sorter_into_reader, MergeFn,
|
2023-02-16 18:42:47 +01:00
|
|
|
};
|
2022-06-15 15:36:27 +02:00
|
|
|
use super::{IndexDocumentsMethod, IndexerConfig};
|
2022-06-20 13:48:02 +02:00
|
|
|
use crate::documents::{DocumentsBatchIndex, EnrichedDocument, EnrichedDocumentsBatchReader};
|
2021-08-31 11:44:15 +02:00
|
|
|
use crate::error::{Error, InternalError, UserError};
|
2022-12-19 15:59:22 +01:00
|
|
|
use crate::index::{db_name, main_key};
|
2024-03-26 13:27:43 +01:00
|
|
|
use crate::update::del_add::{
|
|
|
|
del_add_from_two_obkvs, into_del_add_obkv, DelAdd, DelAddOperation, KvReaderDelAdd,
|
|
|
|
};
|
2023-11-02 13:37:54 +01:00
|
|
|
use crate::update::index_documents::GrenadParameters;
|
2024-03-26 13:27:43 +01:00
|
|
|
use crate::update::settings::{InnerIndexSettings, InnerIndexSettingsDiff};
|
2024-03-19 14:33:32 +01:00
|
|
|
use crate::update::{AvailableDocumentsIds, UpdateIndexingStep};
|
2023-11-22 18:21:19 +01:00
|
|
|
use crate::{FieldDistribution, FieldId, FieldIdMapMissingEntry, FieldsIdsMap, Index, Result};
|
2020-10-24 16:23:08 +02:00
|
|
|
|
2020-10-23 14:11:00 +02:00
|
|
|
pub struct TransformOutput {
|
2021-01-20 17:27:43 +01:00
|
|
|
pub primary_key: String,
|
2024-03-26 13:27:43 +01:00
|
|
|
pub settings_diff: InnerIndexSettingsDiff,
|
2021-06-21 15:57:41 +02:00
|
|
|
pub field_distribution: FieldDistribution,
|
2020-10-23 14:11:00 +02:00
|
|
|
pub documents_count: usize,
|
2024-05-21 14:53:26 +02:00
|
|
|
pub original_documents: Option<File>,
|
2022-03-23 17:28:41 +01:00
|
|
|
pub flattened_documents: File,
|
2020-10-23 14:11:00 +02:00
|
|
|
}
|
|
|
|
|
2020-11-22 11:54:04 +01:00
|
|
|
/// Extract the external ids, deduplicate and compute the new internal documents ids
|
2020-11-01 11:50:10 +01:00
|
|
|
/// and fields ids, writing all the documents under their internal ids into a final file.
|
|
|
|
///
|
|
|
|
/// Outputs the new `FieldsIdsMap`, the new `UsersIdsDocumentsIds` map, the new documents ids,
|
|
|
|
/// the replaced documents ids, the number of documents in this update and the file
|
|
|
|
/// containing all those documents.
|
2021-12-08 14:12:07 +01:00
|
|
|
pub struct Transform<'a, 'i> {
|
2020-10-26 20:18:10 +01:00
|
|
|
pub index: &'i Index,
|
2022-03-23 17:28:41 +01:00
|
|
|
fields_ids_map: FieldsIdsMap,
|
|
|
|
|
2021-12-08 14:12:07 +01:00
|
|
|
indexer_settings: &'a IndexerConfig,
|
2020-10-31 21:46:55 +01:00
|
|
|
pub autogenerate_docids: bool,
|
2021-12-08 14:12:07 +01:00
|
|
|
pub index_documents_method: IndexDocumentsMethod,
|
2022-06-07 15:44:55 +02:00
|
|
|
available_documents_ids: AvailableDocumentsIds,
|
2021-12-08 14:12:07 +01:00
|
|
|
|
2023-02-14 17:55:26 +01:00
|
|
|
// Both grenad follows the same format:
|
|
|
|
// key | value
|
|
|
|
// u32 | 1 byte for the Operation byte, the rest is the obkv of the document stored
|
2022-03-23 17:28:41 +01:00
|
|
|
original_sorter: grenad::Sorter<MergeFn>,
|
|
|
|
flattened_sorter: grenad::Sorter<MergeFn>,
|
2023-02-14 17:55:26 +01:00
|
|
|
|
2023-05-22 11:15:14 +02:00
|
|
|
replaced_documents_ids: RoaringBitmap,
|
|
|
|
new_documents_ids: RoaringBitmap,
|
2022-06-13 17:59:34 +02:00
|
|
|
// To increase the cache locality and decrease the heap usage we use compact smartstring.
|
2022-04-11 15:43:18 +02:00
|
|
|
new_external_documents_ids_builder: FxHashMap<SmartString<smartstring::Compact>, u64>,
|
2021-12-08 14:12:07 +01:00
|
|
|
documents_count: usize,
|
2020-10-23 14:11:00 +02:00
|
|
|
}
|
|
|
|
|
2023-02-14 17:55:26 +01:00
|
|
|
/// This enum is specific to the grenad sorter stored in the transform.
|
|
|
|
/// It's used as the first byte of the grenads and tells you if the document id was an addition or a deletion.
|
2023-02-08 12:53:38 +01:00
|
|
|
#[repr(u8)]
|
2023-02-16 18:42:47 +01:00
|
|
|
pub enum Operation {
|
2023-02-08 12:53:38 +01:00
|
|
|
Addition,
|
|
|
|
Deletion,
|
|
|
|
}
|
|
|
|
|
2021-08-31 11:44:15 +02:00
|
|
|
/// Create a mapping between the field ids found in the document batch and the one that were
|
|
|
|
/// already present in the index.
|
|
|
|
///
|
|
|
|
/// If new fields are present in the addition, they are added to the index field ids map.
|
|
|
|
fn create_fields_mapping(
|
|
|
|
index_field_map: &mut FieldsIdsMap,
|
|
|
|
batch_field_map: &DocumentsBatchIndex,
|
|
|
|
) -> Result<HashMap<FieldId, FieldId>> {
|
|
|
|
batch_field_map
|
|
|
|
.iter()
|
|
|
|
// we sort by id here to ensure a deterministic mapping of the fields, that preserves
|
|
|
|
// the original ordering.
|
|
|
|
.sorted_by_key(|(&id, _)| id)
|
2022-10-13 22:02:54 +02:00
|
|
|
.map(|(field, name)| match index_field_map.id(name) {
|
2021-08-31 11:44:15 +02:00
|
|
|
Some(id) => Ok((*field, id)),
|
|
|
|
None => index_field_map
|
2022-10-13 22:02:54 +02:00
|
|
|
.insert(name)
|
2021-08-31 11:44:15 +02:00
|
|
|
.ok_or(Error::UserError(UserError::AttributeLimitReached))
|
|
|
|
.map(|id| (*field, id)),
|
|
|
|
})
|
|
|
|
.collect()
|
2021-05-06 21:16:40 +02:00
|
|
|
}
|
|
|
|
|
2021-12-08 14:12:07 +01:00
|
|
|
impl<'a, 'i> Transform<'a, 'i> {
|
|
|
|
pub fn new(
|
2022-03-23 17:28:41 +01:00
|
|
|
wtxn: &mut heed::RwTxn,
|
2021-12-08 14:12:07 +01:00
|
|
|
index: &'i Index,
|
|
|
|
indexer_settings: &'a IndexerConfig,
|
|
|
|
index_documents_method: IndexDocumentsMethod,
|
|
|
|
autogenerate_docids: bool,
|
2022-03-23 17:28:41 +01:00
|
|
|
) -> Result<Self> {
|
2021-12-08 14:12:07 +01:00
|
|
|
// We must choose the appropriate merge function for when two or more documents
|
|
|
|
// with the same user id must be merged or fully replaced in the same batch.
|
|
|
|
let merge_function = match index_documents_method {
|
2023-10-12 11:46:56 +02:00
|
|
|
IndexDocumentsMethod::ReplaceDocuments => obkvs_keep_last_addition_merge_deletions,
|
|
|
|
IndexDocumentsMethod::UpdateDocuments => obkvs_merge_additions_and_deletions,
|
2021-12-08 14:12:07 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
// We initialize the sorter with the user indexing settings.
|
2023-11-02 14:47:43 +01:00
|
|
|
let original_sorter = create_sorter(
|
|
|
|
grenad::SortAlgorithm::Stable,
|
|
|
|
merge_function,
|
|
|
|
indexer_settings.chunk_compression_type,
|
|
|
|
indexer_settings.chunk_compression_level,
|
|
|
|
indexer_settings.max_nb_chunks,
|
|
|
|
indexer_settings.max_memory.map(|mem| mem / 2),
|
|
|
|
);
|
2021-12-08 14:12:07 +01:00
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
// We initialize the sorter with the user indexing settings.
|
2023-11-02 14:47:43 +01:00
|
|
|
let flattened_sorter = create_sorter(
|
|
|
|
grenad::SortAlgorithm::Stable,
|
|
|
|
merge_function,
|
|
|
|
indexer_settings.chunk_compression_type,
|
|
|
|
indexer_settings.chunk_compression_level,
|
|
|
|
indexer_settings.max_nb_chunks,
|
|
|
|
indexer_settings.max_memory.map(|mem| mem / 2),
|
|
|
|
);
|
2022-06-07 15:44:55 +02:00
|
|
|
let documents_ids = index.documents_ids(wtxn)?;
|
2022-03-23 17:28:41 +01:00
|
|
|
|
|
|
|
Ok(Transform {
|
2021-12-08 14:12:07 +01:00
|
|
|
index,
|
2022-03-23 17:28:41 +01:00
|
|
|
fields_ids_map: index.fields_ids_map(wtxn)?,
|
2021-12-08 14:12:07 +01:00
|
|
|
indexer_settings,
|
|
|
|
autogenerate_docids,
|
2023-10-25 14:14:15 +02:00
|
|
|
available_documents_ids: AvailableDocumentsIds::from_documents_ids(&documents_ids),
|
2022-03-23 17:28:41 +01:00
|
|
|
original_sorter,
|
|
|
|
flattened_sorter,
|
2021-12-08 14:12:07 +01:00
|
|
|
index_documents_method,
|
2022-03-23 17:28:41 +01:00
|
|
|
replaced_documents_ids: RoaringBitmap::new(),
|
|
|
|
new_documents_ids: RoaringBitmap::new(),
|
|
|
|
new_external_documents_ids_builder: FxHashMap::default(),
|
|
|
|
documents_count: 0,
|
|
|
|
})
|
2021-12-08 14:12:07 +01:00
|
|
|
}
|
|
|
|
|
2024-01-23 09:42:48 +01:00
|
|
|
#[tracing::instrument(level = "trace", skip_all, target = "indexing::documents")]
|
2022-10-05 17:41:07 +02:00
|
|
|
pub fn read_documents<R, FP, FA>(
|
2021-12-08 14:12:07 +01:00
|
|
|
&mut self,
|
2022-06-20 13:48:02 +02:00
|
|
|
reader: EnrichedDocumentsBatchReader<R>,
|
2021-12-08 14:12:07 +01:00
|
|
|
wtxn: &mut heed::RwTxn,
|
2022-10-05 17:41:07 +02:00
|
|
|
progress_callback: FP,
|
|
|
|
should_abort: FA,
|
2021-12-08 14:12:07 +01:00
|
|
|
) -> Result<usize>
|
2020-11-11 12:39:09 +01:00
|
|
|
where
|
2021-08-31 11:44:15 +02:00
|
|
|
R: Read + Seek,
|
2022-10-05 17:41:07 +02:00
|
|
|
FP: Fn(UpdateIndexingStep) + Sync,
|
|
|
|
FA: Fn() -> bool + Sync,
|
2020-11-11 12:39:09 +01:00
|
|
|
{
|
2023-11-01 10:07:03 +01:00
|
|
|
puffin::profile_function!();
|
|
|
|
|
2022-07-18 16:08:01 +02:00
|
|
|
let (mut cursor, fields_index) = reader.into_cursor_and_fields_index();
|
2023-10-28 12:56:46 +02:00
|
|
|
let external_documents_ids = self.index.external_documents_ids();
|
2022-07-18 16:08:01 +02:00
|
|
|
let mapping = create_fields_mapping(&mut self.fields_ids_map, &fields_index)?;
|
2020-10-31 16:10:15 +01:00
|
|
|
|
2022-06-21 11:12:51 +02:00
|
|
|
let primary_key = cursor.primary_key().to_string();
|
2022-06-21 14:41:19 +02:00
|
|
|
let primary_key_id =
|
|
|
|
self.fields_ids_map.insert(&primary_key).ok_or(UserError::AttributeLimitReached)?;
|
2022-03-23 17:28:41 +01:00
|
|
|
|
2020-10-31 16:10:15 +01:00
|
|
|
let mut obkv_buffer = Vec::new();
|
2023-10-31 16:46:16 +01:00
|
|
|
let mut document_sorter_value_buffer = Vec::new();
|
|
|
|
let mut document_sorter_key_buffer = Vec::new();
|
2020-11-11 12:39:09 +01:00
|
|
|
let mut documents_count = 0;
|
2022-07-18 12:41:58 +02:00
|
|
|
let mut docid_buffer: Vec<u8> = Vec::new();
|
2022-03-23 17:28:41 +01:00
|
|
|
let mut field_buffer: Vec<(u16, Cow<[u8]>)> = Vec::new();
|
2022-06-20 13:48:02 +02:00
|
|
|
while let Some(enriched_document) = cursor.next_enriched_document()? {
|
2022-06-21 14:41:19 +02:00
|
|
|
let EnrichedDocument { document, document_id } = enriched_document;
|
2022-06-20 13:48:02 +02:00
|
|
|
|
2022-10-05 17:41:07 +02:00
|
|
|
if should_abort() {
|
|
|
|
return Err(Error::InternalError(InternalError::AbortedIndexation));
|
|
|
|
}
|
|
|
|
|
2022-07-18 12:41:58 +02:00
|
|
|
// drop_and_reuse is called instead of .clear() to communicate to the compiler that field_buffer
|
|
|
|
// does not keep references from the cursor between loop iterations
|
2021-08-31 11:44:15 +02:00
|
|
|
let mut field_buffer_cache = drop_and_reuse(field_buffer);
|
2021-12-08 14:12:07 +01:00
|
|
|
if self.indexer_settings.log_every_n.map_or(false, |len| documents_count % len == 0) {
|
2021-08-31 11:44:15 +02:00
|
|
|
progress_callback(UpdateIndexingStep::RemapDocumentAddition {
|
2020-11-11 12:39:09 +01:00
|
|
|
documents_seen: documents_count,
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2022-06-21 14:41:19 +02:00
|
|
|
// When the document id has been auto-generated by the `enrich_documents_batch`
|
|
|
|
// we must insert this document id into the remaped document.
|
|
|
|
let external_id = document_id.value();
|
|
|
|
if document_id.is_generated() {
|
2022-07-18 12:41:58 +02:00
|
|
|
serde_json::to_writer(&mut docid_buffer, external_id)
|
|
|
|
.map_err(InternalError::SerdeJson)?;
|
|
|
|
field_buffer_cache.push((primary_key_id, Cow::from(&docid_buffer)));
|
2022-06-21 14:41:19 +02:00
|
|
|
}
|
|
|
|
|
2021-08-31 11:44:15 +02:00
|
|
|
for (k, v) in document.iter() {
|
2022-03-23 17:28:41 +01:00
|
|
|
let mapped_id =
|
|
|
|
*mapping.get(&k).ok_or(InternalError::FieldIdMappingMissingEntry { key: k })?;
|
|
|
|
field_buffer_cache.push((mapped_id, Cow::from(v)));
|
2020-10-31 16:10:15 +01:00
|
|
|
}
|
|
|
|
|
2021-08-31 11:44:15 +02:00
|
|
|
// Insertion in a obkv need to be done with keys ordered. For now they are ordered
|
|
|
|
// according to the document addition key order, so we sort it according to the
|
|
|
|
// fieldids map keys order.
|
2022-10-13 22:02:54 +02:00
|
|
|
field_buffer_cache.sort_unstable_by(|(f1, _), (f2, _)| f1.cmp(f2));
|
2020-10-31 12:54:43 +01:00
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
// Build the new obkv document.
|
2023-11-01 10:07:03 +01:00
|
|
|
let mut writer = KvWriter::new(&mut obkv_buffer);
|
2021-08-31 11:44:15 +02:00
|
|
|
for (k, v) in field_buffer_cache.iter() {
|
|
|
|
writer.insert(*k, v)?;
|
2020-10-23 14:11:00 +02:00
|
|
|
}
|
|
|
|
|
2022-06-13 17:59:34 +02:00
|
|
|
let mut original_docid = None;
|
2022-10-13 22:02:54 +02:00
|
|
|
let docid = match self.new_external_documents_ids_builder.entry((*external_id).into()) {
|
2023-10-30 14:47:51 +01:00
|
|
|
HEntry::Occupied(entry) => *entry.get() as u32,
|
|
|
|
HEntry::Vacant(entry) => {
|
2023-10-28 12:56:46 +02:00
|
|
|
let docid = match external_documents_ids.get(wtxn, entry.key())? {
|
2023-10-12 11:46:56 +02:00
|
|
|
Some(docid) => {
|
|
|
|
// If it was already in the list of replaced documents it means it was deleted
|
|
|
|
// by the remove_document method. We should starts as if it never existed.
|
|
|
|
if self.replaced_documents_ids.insert(docid) {
|
|
|
|
original_docid = Some(docid);
|
|
|
|
}
|
|
|
|
|
|
|
|
docid
|
2023-02-14 19:09:40 +01:00
|
|
|
}
|
2023-10-12 11:46:56 +02:00
|
|
|
None => self
|
|
|
|
.available_documents_ids
|
|
|
|
.next()
|
|
|
|
.ok_or(UserError::DocumentLimitReached)?,
|
|
|
|
};
|
2022-10-13 22:02:54 +02:00
|
|
|
entry.insert(docid as u64);
|
|
|
|
docid
|
|
|
|
}
|
|
|
|
};
|
2022-03-23 17:28:41 +01:00
|
|
|
|
2022-07-05 19:15:16 +02:00
|
|
|
let mut skip_insertion = false;
|
2022-06-13 17:59:34 +02:00
|
|
|
if let Some(original_docid) = original_docid {
|
2023-11-22 18:21:19 +01:00
|
|
|
let original_key = original_docid;
|
2022-03-23 17:28:41 +01:00
|
|
|
let base_obkv = self
|
|
|
|
.index
|
|
|
|
.documents
|
2023-11-27 11:52:22 +01:00
|
|
|
.remap_data_type::<heed::types::Bytes>()
|
2022-06-13 17:59:34 +02:00
|
|
|
.get(wtxn, &original_key)?
|
2022-03-23 17:28:41 +01:00
|
|
|
.ok_or(InternalError::DatabaseMissingEntry {
|
|
|
|
db_name: db_name::DOCUMENTS,
|
|
|
|
key: None,
|
|
|
|
})?;
|
|
|
|
|
2022-07-05 19:15:16 +02:00
|
|
|
// we check if the two documents are exactly equal. If it's the case we can skip this document entirely
|
|
|
|
if base_obkv == obkv_buffer {
|
|
|
|
// we're not replacing anything
|
|
|
|
self.replaced_documents_ids.remove(original_docid);
|
|
|
|
// and we need to put back the original id as it was before
|
2022-10-13 22:02:54 +02:00
|
|
|
self.new_external_documents_ids_builder.remove(external_id);
|
2022-07-05 19:15:16 +02:00
|
|
|
skip_insertion = true;
|
|
|
|
} else {
|
|
|
|
// we associate the base document with the new key, everything will get merged later.
|
2023-11-20 10:53:40 +01:00
|
|
|
let deladd_operation = match self.index_documents_method {
|
|
|
|
IndexDocumentsMethod::UpdateDocuments => {
|
|
|
|
DelAddOperation::DeletionAndAddition
|
|
|
|
}
|
|
|
|
IndexDocumentsMethod::ReplaceDocuments => DelAddOperation::Deletion,
|
|
|
|
};
|
2023-10-31 16:46:16 +01:00
|
|
|
document_sorter_key_buffer.clear();
|
|
|
|
document_sorter_key_buffer.extend_from_slice(&docid.to_be_bytes());
|
|
|
|
document_sorter_key_buffer.extend_from_slice(external_id.as_bytes());
|
|
|
|
document_sorter_value_buffer.clear();
|
|
|
|
document_sorter_value_buffer.push(Operation::Addition as u8);
|
2023-10-12 11:46:56 +02:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(base_obkv),
|
2023-11-20 10:53:40 +01:00
|
|
|
deladd_operation,
|
2023-10-31 16:46:16 +01:00
|
|
|
&mut document_sorter_value_buffer,
|
2023-10-12 11:46:56 +02:00
|
|
|
)?;
|
2023-11-02 15:31:37 +01:00
|
|
|
self.original_sorter
|
|
|
|
.insert(&document_sorter_key_buffer, &document_sorter_value_buffer)?;
|
2023-11-01 10:07:03 +01:00
|
|
|
let base_obkv = KvReader::new(base_obkv);
|
2024-03-26 13:27:43 +01:00
|
|
|
if let Some(flattened_obkv) =
|
|
|
|
Self::flatten_from_fields_ids_map(&base_obkv, &mut self.fields_ids_map)?
|
|
|
|
{
|
2023-11-01 10:07:03 +01:00
|
|
|
// we recreate our buffer with the flattened documents
|
2023-10-31 16:46:16 +01:00
|
|
|
document_sorter_value_buffer.clear();
|
|
|
|
document_sorter_value_buffer.push(Operation::Addition as u8);
|
2023-11-01 10:07:03 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&flattened_obkv),
|
2023-11-20 10:53:40 +01:00
|
|
|
deladd_operation,
|
2023-10-31 16:46:16 +01:00
|
|
|
&mut document_sorter_value_buffer,
|
2023-11-01 10:07:03 +01:00
|
|
|
)?;
|
2022-07-05 19:15:16 +02:00
|
|
|
}
|
2023-11-02 15:31:37 +01:00
|
|
|
self.flattened_sorter
|
|
|
|
.insert(docid.to_be_bytes(), &document_sorter_value_buffer)?;
|
2022-04-12 11:22:36 +02:00
|
|
|
}
|
2022-03-23 17:28:41 +01:00
|
|
|
}
|
|
|
|
|
2022-07-05 19:15:16 +02:00
|
|
|
if !skip_insertion {
|
2022-08-17 15:03:37 +02:00
|
|
|
self.new_documents_ids.insert(docid);
|
2023-02-08 12:53:38 +01:00
|
|
|
|
2023-10-31 16:46:16 +01:00
|
|
|
document_sorter_key_buffer.clear();
|
|
|
|
document_sorter_key_buffer.extend_from_slice(&docid.to_be_bytes());
|
|
|
|
document_sorter_key_buffer.extend_from_slice(external_id.as_bytes());
|
|
|
|
document_sorter_value_buffer.clear();
|
|
|
|
document_sorter_value_buffer.push(Operation::Addition as u8);
|
2023-10-12 11:46:56 +02:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&obkv_buffer),
|
2023-11-20 10:53:40 +01:00
|
|
|
DelAddOperation::Addition,
|
2023-10-31 16:46:16 +01:00
|
|
|
&mut document_sorter_value_buffer,
|
2023-10-12 11:46:56 +02:00
|
|
|
)?;
|
2022-07-05 19:15:16 +02:00
|
|
|
// We use the extracted/generated user id as the key for this document.
|
2023-10-31 16:46:16 +01:00
|
|
|
self.original_sorter
|
|
|
|
.insert(&document_sorter_key_buffer, &document_sorter_value_buffer)?;
|
2021-08-31 11:44:15 +02:00
|
|
|
|
2023-11-01 10:07:03 +01:00
|
|
|
let flattened_obkv = KvReader::new(&obkv_buffer);
|
2024-03-26 13:27:43 +01:00
|
|
|
if let Some(obkv) =
|
|
|
|
Self::flatten_from_fields_ids_map(&flattened_obkv, &mut self.fields_ids_map)?
|
|
|
|
{
|
2023-10-31 16:46:16 +01:00
|
|
|
document_sorter_value_buffer.clear();
|
|
|
|
document_sorter_value_buffer.push(Operation::Addition as u8);
|
2023-11-01 10:07:03 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&obkv),
|
2023-11-20 10:53:40 +01:00
|
|
|
DelAddOperation::Addition,
|
2023-10-31 16:46:16 +01:00
|
|
|
&mut document_sorter_value_buffer,
|
2023-11-01 10:07:03 +01:00
|
|
|
)?
|
2022-07-05 19:15:16 +02:00
|
|
|
}
|
2023-10-31 16:46:16 +01:00
|
|
|
self.flattened_sorter.insert(docid.to_be_bytes(), &document_sorter_value_buffer)?;
|
2022-03-23 17:28:41 +01:00
|
|
|
}
|
2022-07-05 19:15:16 +02:00
|
|
|
documents_count += 1;
|
2022-03-23 17:28:41 +01:00
|
|
|
|
2021-08-31 11:44:15 +02:00
|
|
|
progress_callback(UpdateIndexingStep::RemapDocumentAddition {
|
|
|
|
documents_seen: documents_count,
|
|
|
|
});
|
|
|
|
|
|
|
|
field_buffer = drop_and_reuse(field_buffer_cache);
|
2022-07-18 12:41:58 +02:00
|
|
|
docid_buffer.clear();
|
2022-03-23 17:28:41 +01:00
|
|
|
obkv_buffer.clear();
|
2020-10-23 14:11:00 +02:00
|
|
|
}
|
|
|
|
|
2021-08-31 11:44:15 +02:00
|
|
|
progress_callback(UpdateIndexingStep::RemapDocumentAddition {
|
2020-11-11 12:39:09 +01:00
|
|
|
documents_seen: documents_count,
|
|
|
|
});
|
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
self.index.put_fields_ids_map(wtxn, &self.fields_ids_map)?;
|
2022-06-21 11:12:51 +02:00
|
|
|
self.index.put_primary_key(wtxn, &primary_key)?;
|
2021-12-08 14:12:07 +01:00
|
|
|
self.documents_count += documents_count;
|
2020-10-31 16:10:15 +01:00
|
|
|
// Now that we have a valid sorter that contains the user id and the obkv we
|
|
|
|
// give it to the last transforming function which returns the TransformOutput.
|
2021-12-08 14:12:07 +01:00
|
|
|
Ok(documents_count)
|
2020-10-31 16:10:15 +01:00
|
|
|
}
|
|
|
|
|
2023-02-16 18:42:47 +01:00
|
|
|
/// The counter part of `read_documents` that removes documents either from the transform or the database.
|
|
|
|
/// It can be called before, after or in between two calls of the `read_documents`.
|
|
|
|
///
|
|
|
|
/// It needs to update all the internal datastructure in the transform.
|
|
|
|
/// - If the document is coming from the database -> it's marked as a to_delete document
|
|
|
|
/// - If the document to remove was inserted by the `read_documents` method before AND was present in the db,
|
|
|
|
/// it's marked as `to_delete` + added into the grenad to ensure we don't reinsert it.
|
|
|
|
/// - If the document to remove was inserted by the `read_documents` method before but was NOT present in the db,
|
|
|
|
/// it's added into the grenad to ensure we don't insert it + removed from the list of new documents ids.
|
|
|
|
/// - If the document to remove was not present in either the db or the transform we do nothing.
|
2024-01-23 09:42:48 +01:00
|
|
|
#[tracing::instrument(level = "trace", skip_all, target = "indexing::documents")]
|
2023-02-08 16:06:09 +01:00
|
|
|
pub fn remove_documents<FA>(
|
|
|
|
&mut self,
|
|
|
|
mut to_remove: Vec<String>,
|
|
|
|
wtxn: &mut heed::RwTxn,
|
|
|
|
should_abort: FA,
|
|
|
|
) -> Result<usize>
|
|
|
|
where
|
|
|
|
FA: Fn() -> bool + Sync,
|
|
|
|
{
|
2023-11-01 10:07:03 +01:00
|
|
|
puffin::profile_function!();
|
|
|
|
|
2023-02-08 16:06:09 +01:00
|
|
|
// there may be duplicates in the documents to remove.
|
|
|
|
to_remove.sort_unstable();
|
|
|
|
to_remove.dedup();
|
|
|
|
|
2023-10-28 12:56:46 +02:00
|
|
|
let external_documents_ids = self.index.external_documents_ids();
|
2023-02-08 16:06:09 +01:00
|
|
|
|
|
|
|
let mut documents_deleted = 0;
|
2023-10-31 16:46:16 +01:00
|
|
|
let mut document_sorter_value_buffer = Vec::new();
|
|
|
|
let mut document_sorter_key_buffer = Vec::new();
|
2023-02-08 16:06:09 +01:00
|
|
|
for to_remove in to_remove {
|
|
|
|
if should_abort() {
|
|
|
|
return Err(Error::InternalError(InternalError::AbortedIndexation));
|
|
|
|
}
|
|
|
|
|
2023-10-12 11:46:56 +02:00
|
|
|
// Check if the document has been added in the current indexing process.
|
2023-10-31 16:46:16 +01:00
|
|
|
let deleted_from_current =
|
|
|
|
match self.new_external_documents_ids_builder.entry((*to_remove).into()) {
|
|
|
|
// if the document was added in a previous iteration of the transform we make it as deleted in the sorters.
|
|
|
|
HEntry::Occupied(entry) => {
|
|
|
|
let docid = *entry.get() as u32;
|
|
|
|
// Key is the concatenation of the internal docid and the external one.
|
|
|
|
document_sorter_key_buffer.clear();
|
|
|
|
document_sorter_key_buffer.extend_from_slice(&docid.to_be_bytes());
|
|
|
|
document_sorter_key_buffer.extend_from_slice(to_remove.as_bytes());
|
|
|
|
document_sorter_value_buffer.clear();
|
|
|
|
document_sorter_value_buffer.push(Operation::Deletion as u8);
|
|
|
|
obkv::KvWriterU16::new(&mut document_sorter_value_buffer).finish().unwrap();
|
|
|
|
self.original_sorter
|
|
|
|
.insert(&document_sorter_key_buffer, &document_sorter_value_buffer)?;
|
|
|
|
self.flattened_sorter
|
|
|
|
.insert(docid.to_be_bytes(), &document_sorter_value_buffer)?;
|
|
|
|
|
|
|
|
// we must NOT update the list of replaced_documents_ids
|
|
|
|
// Either:
|
|
|
|
// 1. It's already in it and there is nothing to do
|
|
|
|
// 2. It wasn't in it because the document was created by a previous batch and since
|
|
|
|
// we're removing it there is nothing to do.
|
|
|
|
self.new_documents_ids.remove(docid);
|
|
|
|
entry.remove_entry();
|
|
|
|
true
|
|
|
|
}
|
|
|
|
HEntry::Vacant(_) => false,
|
|
|
|
};
|
2023-10-12 11:46:56 +02:00
|
|
|
|
|
|
|
// If the document was already in the db we mark it as a `to_delete` document.
|
|
|
|
// Then we push the document in sorters in deletion mode.
|
2023-10-28 12:56:46 +02:00
|
|
|
let deleted_from_db = match external_documents_ids.get(wtxn, &to_remove)? {
|
2023-10-12 11:46:56 +02:00
|
|
|
Some(docid) => {
|
2023-11-13 13:51:22 +01:00
|
|
|
self.remove_document_from_db(
|
|
|
|
docid,
|
|
|
|
to_remove,
|
|
|
|
wtxn,
|
|
|
|
&mut document_sorter_key_buffer,
|
2023-10-31 16:46:16 +01:00
|
|
|
&mut document_sorter_value_buffer,
|
2023-10-12 11:46:56 +02:00
|
|
|
)?;
|
|
|
|
true
|
2023-02-08 16:06:09 +01:00
|
|
|
}
|
2023-10-12 11:46:56 +02:00
|
|
|
None => false,
|
2023-02-08 16:06:09 +01:00
|
|
|
};
|
|
|
|
|
2023-10-12 11:46:56 +02:00
|
|
|
// increase counter only if the document existed somewhere before.
|
|
|
|
if deleted_from_current || deleted_from_db {
|
|
|
|
documents_deleted += 1;
|
|
|
|
}
|
2023-02-08 16:06:09 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
Ok(documents_deleted)
|
|
|
|
}
|
|
|
|
|
2023-11-09 16:23:20 +01:00
|
|
|
/// Removes documents from db using their internal document ids.
|
2023-11-09 14:23:02 +01:00
|
|
|
///
|
2023-11-09 16:23:20 +01:00
|
|
|
/// # Warning
|
|
|
|
///
|
|
|
|
/// This function is dangerous and will only work correctly if:
|
|
|
|
///
|
|
|
|
/// - All the passed ids currently exist in the database
|
|
|
|
/// - No batching using the standards `remove_documents` and `add_documents` took place
|
|
|
|
///
|
|
|
|
/// TODO: make it impossible to call `remove_documents` or `add_documents` on an instance that calls this function.
|
2024-01-23 09:42:48 +01:00
|
|
|
#[tracing::instrument(level = "trace", skip_all, target = "indexing::details")]
|
2023-11-09 14:23:02 +01:00
|
|
|
pub fn remove_documents_from_db_no_batch<FA>(
|
|
|
|
&mut self,
|
|
|
|
to_remove: &RoaringBitmap,
|
|
|
|
wtxn: &mut heed::RwTxn,
|
|
|
|
should_abort: FA,
|
|
|
|
) -> Result<usize>
|
|
|
|
where
|
|
|
|
FA: Fn() -> bool + Sync,
|
|
|
|
{
|
|
|
|
puffin::profile_function!();
|
|
|
|
|
|
|
|
let mut documents_deleted = 0;
|
|
|
|
let mut document_sorter_value_buffer = Vec::new();
|
|
|
|
let mut document_sorter_key_buffer = Vec::new();
|
|
|
|
let external_ids = self.index.external_id_of(wtxn, to_remove.iter())?;
|
|
|
|
|
2023-11-13 13:51:22 +01:00
|
|
|
for (internal_docid, external_docid) in to_remove.iter().zip(external_ids) {
|
2023-11-09 14:23:02 +01:00
|
|
|
let external_docid = external_docid?;
|
|
|
|
if should_abort() {
|
|
|
|
return Err(Error::InternalError(InternalError::AbortedIndexation));
|
|
|
|
}
|
2023-11-13 13:51:22 +01:00
|
|
|
self.remove_document_from_db(
|
|
|
|
internal_docid,
|
|
|
|
external_docid,
|
|
|
|
wtxn,
|
|
|
|
&mut document_sorter_key_buffer,
|
2023-11-09 14:23:02 +01:00
|
|
|
&mut document_sorter_value_buffer,
|
|
|
|
)?;
|
|
|
|
|
|
|
|
documents_deleted += 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
Ok(documents_deleted)
|
|
|
|
}
|
|
|
|
|
2023-11-13 13:51:22 +01:00
|
|
|
fn remove_document_from_db(
|
|
|
|
&mut self,
|
|
|
|
internal_docid: u32,
|
|
|
|
external_docid: String,
|
|
|
|
txn: &heed::RoTxn,
|
|
|
|
document_sorter_key_buffer: &mut Vec<u8>,
|
|
|
|
document_sorter_value_buffer: &mut Vec<u8>,
|
|
|
|
) -> Result<()> {
|
|
|
|
self.replaced_documents_ids.insert(internal_docid);
|
|
|
|
|
|
|
|
// fetch the obkv document
|
2023-11-22 18:21:19 +01:00
|
|
|
let original_key = internal_docid;
|
2023-11-13 13:51:22 +01:00
|
|
|
let base_obkv = self
|
|
|
|
.index
|
|
|
|
.documents
|
2023-11-27 11:52:22 +01:00
|
|
|
.remap_data_type::<heed::types::Bytes>()
|
2023-11-13 13:51:22 +01:00
|
|
|
.get(txn, &original_key)?
|
|
|
|
.ok_or(InternalError::DatabaseMissingEntry {
|
|
|
|
db_name: db_name::DOCUMENTS,
|
|
|
|
key: None,
|
|
|
|
})?;
|
|
|
|
|
|
|
|
// Key is the concatenation of the internal docid and the external one.
|
|
|
|
document_sorter_key_buffer.clear();
|
|
|
|
document_sorter_key_buffer.extend_from_slice(&internal_docid.to_be_bytes());
|
|
|
|
document_sorter_key_buffer.extend_from_slice(external_docid.as_bytes());
|
|
|
|
// push it as to delete in the original_sorter
|
|
|
|
document_sorter_value_buffer.clear();
|
|
|
|
document_sorter_value_buffer.push(Operation::Deletion as u8);
|
2023-11-20 10:53:40 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(base_obkv),
|
|
|
|
DelAddOperation::Deletion,
|
|
|
|
document_sorter_value_buffer,
|
|
|
|
)?;
|
2023-11-13 13:51:22 +01:00
|
|
|
self.original_sorter.insert(&document_sorter_key_buffer, &document_sorter_value_buffer)?;
|
|
|
|
|
|
|
|
// flatten it and push it as to delete in the flattened_sorter
|
|
|
|
let flattened_obkv = KvReader::new(base_obkv);
|
2024-03-26 13:27:43 +01:00
|
|
|
if let Some(obkv) =
|
|
|
|
Self::flatten_from_fields_ids_map(&flattened_obkv, &mut self.fields_ids_map)?
|
|
|
|
{
|
2023-11-13 13:51:22 +01:00
|
|
|
// we recreate our buffer with the flattened documents
|
|
|
|
document_sorter_value_buffer.clear();
|
|
|
|
document_sorter_value_buffer.push(Operation::Deletion as u8);
|
2023-11-20 10:53:40 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&obkv),
|
|
|
|
DelAddOperation::Deletion,
|
|
|
|
document_sorter_value_buffer,
|
|
|
|
)?;
|
2023-11-13 13:51:22 +01:00
|
|
|
}
|
|
|
|
self.flattened_sorter
|
|
|
|
.insert(internal_docid.to_be_bytes(), &document_sorter_value_buffer)?;
|
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
// Flatten a document from the fields ids map contained in self and insert the new
|
2022-04-12 11:22:36 +02:00
|
|
|
// created fields. Returns `None` if the document doesn't need to be flattened.
|
2024-03-26 13:27:43 +01:00
|
|
|
#[tracing::instrument(
|
|
|
|
level = "trace",
|
|
|
|
skip(obkv, fields_ids_map),
|
|
|
|
target = "indexing::transform"
|
|
|
|
)]
|
|
|
|
fn flatten_from_fields_ids_map(
|
|
|
|
obkv: &KvReader<FieldId>,
|
|
|
|
fields_ids_map: &mut FieldsIdsMap,
|
|
|
|
) -> Result<Option<Vec<u8>>> {
|
2022-04-12 11:22:36 +02:00
|
|
|
if obkv
|
|
|
|
.iter()
|
|
|
|
.all(|(_, value)| !json_depth_checker::should_flatten_from_unchecked_slice(value))
|
|
|
|
{
|
|
|
|
return Ok(None);
|
|
|
|
}
|
|
|
|
|
2022-04-25 14:09:52 +02:00
|
|
|
// store the keys and values the original obkv + the flattened json
|
|
|
|
// We first extract all the key+value out of the obkv. If a value is not nested
|
|
|
|
// we keep a reference on its value. If the value is nested we'll get its value
|
|
|
|
// as an owned `Vec<u8>` after flattening it.
|
|
|
|
let mut key_value: Vec<(FieldId, Cow<[u8]>)> = Vec::new();
|
|
|
|
|
|
|
|
// the object we're going to use to store the fields that need to be flattened.
|
2022-03-23 17:28:41 +01:00
|
|
|
let mut doc = serde_json::Map::new();
|
|
|
|
|
2022-04-25 14:09:52 +02:00
|
|
|
// we recreate a json containing only the fields that needs to be flattened.
|
|
|
|
// all the raw values get inserted directly in the `key_value` vec.
|
|
|
|
for (key, value) in obkv.iter() {
|
|
|
|
if json_depth_checker::should_flatten_from_unchecked_slice(value) {
|
2024-03-26 13:27:43 +01:00
|
|
|
let key = fields_ids_map.name(key).ok_or(FieldIdMapMissingEntry::FieldId {
|
2022-04-25 14:09:52 +02:00
|
|
|
field_id: key,
|
|
|
|
process: "Flatten from fields ids map.",
|
|
|
|
})?;
|
|
|
|
|
|
|
|
let value = serde_json::from_slice::<Value>(value)
|
|
|
|
.map_err(crate::error::InternalError::SerdeJson)?;
|
|
|
|
doc.insert(key.to_string(), value);
|
|
|
|
} else {
|
|
|
|
key_value.push((key, value.into()));
|
|
|
|
}
|
2022-03-23 17:28:41 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
let flattened = flatten_serde_json::flatten(&doc);
|
|
|
|
|
2022-04-25 14:09:52 +02:00
|
|
|
// Once we have the flattened version we insert all the new generated fields_ids
|
|
|
|
// (if any) in the fields ids map and serialize the value.
|
|
|
|
for (key, value) in flattened.into_iter() {
|
2024-03-26 13:27:43 +01:00
|
|
|
let fid = fields_ids_map.insert(&key).ok_or(UserError::AttributeLimitReached)?;
|
2022-03-23 17:28:41 +01:00
|
|
|
let value = serde_json::to_vec(&value).map_err(InternalError::SerdeJson)?;
|
2022-04-25 14:09:52 +02:00
|
|
|
key_value.push((fid, value.into()));
|
2022-03-23 17:28:41 +01:00
|
|
|
}
|
|
|
|
|
2022-04-25 14:09:52 +02:00
|
|
|
// we sort the key. If there was a conflict between the obkv and the new generated value the
|
|
|
|
// keys will be consecutive.
|
|
|
|
key_value.sort_unstable_by_key(|(key, _)| *key);
|
|
|
|
|
|
|
|
let mut buffer = Vec::new();
|
|
|
|
Self::create_obkv_from_key_value(&mut key_value, &mut buffer)?;
|
2022-04-12 11:22:36 +02:00
|
|
|
Ok(Some(buffer))
|
2022-03-23 17:28:41 +01:00
|
|
|
}
|
|
|
|
|
2022-04-25 14:09:52 +02:00
|
|
|
/// Generate an obkv from a slice of key / value sorted by key.
|
|
|
|
fn create_obkv_from_key_value(
|
|
|
|
key_value: &mut [(FieldId, Cow<[u8]>)],
|
|
|
|
output_buffer: &mut Vec<u8>,
|
|
|
|
) -> Result<()> {
|
|
|
|
debug_assert!(
|
|
|
|
key_value.windows(2).all(|vec| vec[0].0 <= vec[1].0),
|
|
|
|
"The slice of key / value pair must be sorted."
|
|
|
|
);
|
|
|
|
|
|
|
|
output_buffer.clear();
|
|
|
|
let mut writer = KvWriter::new(output_buffer);
|
|
|
|
|
|
|
|
let mut skip_next_value = false;
|
|
|
|
for things in key_value.windows(2) {
|
|
|
|
if skip_next_value {
|
|
|
|
skip_next_value = false;
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
let (key1, value1) = &things[0];
|
|
|
|
let (key2, value2) = &things[1];
|
|
|
|
|
|
|
|
// now we're going to look for conflicts between the keys. For example the following documents would cause a conflict:
|
|
|
|
// { "doggo.name": "jean", "doggo": { "name": "paul" } }
|
|
|
|
// we should find a first "doggo.name" from the obkv and a second one from the flattening.
|
|
|
|
// but we must generate the following document:
|
|
|
|
// { "doggo.name": ["jean", "paul"] }
|
|
|
|
// thus we're going to merge the value from the obkv and the flattened document in a single array and skip the next
|
|
|
|
// iteration.
|
|
|
|
if key1 == key2 {
|
|
|
|
skip_next_value = true;
|
|
|
|
|
|
|
|
let value1 = serde_json::from_slice(value1)
|
|
|
|
.map_err(crate::error::InternalError::SerdeJson)?;
|
|
|
|
let value2 = serde_json::from_slice(value2)
|
|
|
|
.map_err(crate::error::InternalError::SerdeJson)?;
|
|
|
|
let value = match (value1, value2) {
|
|
|
|
(Value::Array(mut left), Value::Array(mut right)) => {
|
|
|
|
left.append(&mut right);
|
|
|
|
Value::Array(left)
|
|
|
|
}
|
|
|
|
(Value::Array(mut array), value) | (value, Value::Array(mut array)) => {
|
|
|
|
array.push(value);
|
|
|
|
Value::Array(array)
|
|
|
|
}
|
|
|
|
(left, right) => Value::Array(vec![left, right]),
|
|
|
|
};
|
|
|
|
|
|
|
|
let value = serde_json::to_vec(&value).map_err(InternalError::SerdeJson)?;
|
|
|
|
writer.insert(*key1, value)?;
|
|
|
|
} else {
|
|
|
|
writer.insert(*key1, value1)?;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if !skip_next_value {
|
|
|
|
// the unwrap is safe here, we know there was at least one value in the document
|
|
|
|
let (key, value) = key_value.last().unwrap();
|
|
|
|
writer.insert(*key, value)?;
|
|
|
|
}
|
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
|
2020-11-01 11:50:10 +01:00
|
|
|
/// Generate the `TransformOutput` based on the given sorter that can be generated from any
|
|
|
|
/// format like CSV, JSON or JSON stream. This sorter must contain a key that is the document
|
2020-10-31 16:10:15 +01:00
|
|
|
/// id for the user side and the value must be an obkv where keys are valid fields ids.
|
2024-01-23 09:42:48 +01:00
|
|
|
#[tracing::instrument(level = "trace", skip_all, target = "indexing::transform")]
|
2021-12-08 14:12:07 +01:00
|
|
|
pub(crate) fn output_from_sorter<F>(
|
2020-10-31 16:10:15 +01:00
|
|
|
self,
|
2021-12-08 14:12:07 +01:00
|
|
|
wtxn: &mut heed::RwTxn,
|
2020-11-11 12:39:09 +01:00
|
|
|
progress_callback: F,
|
2021-06-14 16:46:19 +02:00
|
|
|
) -> Result<TransformOutput>
|
2020-11-11 12:39:09 +01:00
|
|
|
where
|
|
|
|
F: Fn(UpdateIndexingStep) + Sync,
|
2020-10-31 16:10:15 +01:00
|
|
|
{
|
2023-07-10 18:41:54 +02:00
|
|
|
puffin::profile_function!();
|
|
|
|
|
2021-12-08 14:12:07 +01:00
|
|
|
let primary_key = self
|
|
|
|
.index
|
2022-10-13 22:02:54 +02:00
|
|
|
.primary_key(wtxn)?
|
2022-12-19 15:59:22 +01:00
|
|
|
.ok_or(Error::InternalError(InternalError::DatabaseMissingEntry {
|
|
|
|
db_name: db_name::MAIN,
|
|
|
|
key: Some(main_key::PRIMARY_KEY_KEY),
|
|
|
|
}))?
|
2021-12-08 14:12:07 +01:00
|
|
|
.to_string();
|
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
// We create a final writer to write the new documents in order from the sorter.
|
|
|
|
let mut writer = create_writer(
|
2021-12-08 14:12:07 +01:00
|
|
|
self.indexer_settings.chunk_compression_type,
|
|
|
|
self.indexer_settings.chunk_compression_level,
|
2022-03-23 17:28:41 +01:00
|
|
|
tempfile::tempfile()?,
|
2020-10-29 14:20:03 +01:00
|
|
|
);
|
2020-10-23 14:11:00 +02:00
|
|
|
|
2022-06-13 17:59:34 +02:00
|
|
|
// To compute the field distribution we need to;
|
|
|
|
// 1. Remove all the deleted documents from the field distribution
|
|
|
|
// 2. Add all the new documents to the field distribution
|
2022-03-23 17:28:41 +01:00
|
|
|
let mut field_distribution = self.index.field_distribution(wtxn)?;
|
2022-06-13 17:59:34 +02:00
|
|
|
|
|
|
|
// Here we are going to do the document count + field distribution + `write_into_stream_writer`
|
2022-03-23 17:28:41 +01:00
|
|
|
let mut iter = self.original_sorter.into_stream_merger_iter()?;
|
|
|
|
// used only for the callback
|
2020-10-23 14:11:00 +02:00
|
|
|
let mut documents_count = 0;
|
2020-11-11 12:39:09 +01:00
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
while let Some((key, val)) = iter.next()? {
|
2023-10-12 11:46:56 +02:00
|
|
|
// skip first byte corresponding to the operation type (Deletion or Addition).
|
2023-02-08 12:53:38 +01:00
|
|
|
let val = &val[1..];
|
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
// send a callback to show at which step we are
|
|
|
|
documents_count += 1;
|
|
|
|
progress_callback(UpdateIndexingStep::ComputeIdsAndMergeDocuments {
|
|
|
|
documents_seen: documents_count,
|
|
|
|
total_documents: self.documents_count,
|
|
|
|
});
|
|
|
|
|
2023-10-30 16:57:08 +01:00
|
|
|
for (key, value) in KvReader::new(val) {
|
2023-10-30 14:47:51 +01:00
|
|
|
let reader = KvReaderDelAdd::new(value);
|
|
|
|
match (reader.get(DelAdd::Deletion), reader.get(DelAdd::Addition)) {
|
|
|
|
(None, None) => {}
|
|
|
|
(None, Some(_)) => {
|
|
|
|
// New field
|
|
|
|
let name = self.fields_ids_map.name(key).ok_or(
|
|
|
|
FieldIdMapMissingEntry::FieldId {
|
|
|
|
field_id: key,
|
|
|
|
process: "Computing field distribution in transform.",
|
|
|
|
},
|
|
|
|
)?;
|
|
|
|
*field_distribution.entry(name.to_string()).or_insert(0) += 1;
|
|
|
|
}
|
|
|
|
(Some(_), None) => {
|
|
|
|
// Field removed
|
|
|
|
let name = self.fields_ids_map.name(key).ok_or(
|
|
|
|
FieldIdMapMissingEntry::FieldId {
|
|
|
|
field_id: key,
|
|
|
|
process: "Computing field distribution in transform.",
|
|
|
|
},
|
|
|
|
)?;
|
|
|
|
match field_distribution.entry(name.to_string()) {
|
|
|
|
BEntry::Vacant(_) => { /* Bug? trying to remove a non-existing field */
|
|
|
|
}
|
|
|
|
BEntry::Occupied(mut entry) => {
|
|
|
|
// attempt to remove one
|
|
|
|
match entry.get_mut().checked_sub(1) {
|
2023-10-30 16:57:08 +01:00
|
|
|
Some(0) => {
|
|
|
|
entry.remove();
|
|
|
|
}
|
2023-10-30 14:47:51 +01:00
|
|
|
Some(new_val) => {
|
|
|
|
*entry.get_mut() = new_val;
|
|
|
|
}
|
|
|
|
None => {
|
2023-10-30 16:57:08 +01:00
|
|
|
unreachable!("Attempting to remove a field that wasn't in the field distribution")
|
2023-10-30 14:47:51 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
(Some(_), Some(_)) => {
|
|
|
|
// Value change, no field distribution change
|
|
|
|
}
|
|
|
|
}
|
2021-05-04 21:01:11 +02:00
|
|
|
}
|
2022-03-23 17:28:41 +01:00
|
|
|
writer.insert(key, val)?;
|
2020-10-23 14:11:00 +02:00
|
|
|
}
|
|
|
|
|
2022-03-23 17:28:41 +01:00
|
|
|
let mut original_documents = writer.into_inner()?;
|
|
|
|
// We then extract the file and reset the seek to be able to read it again.
|
2023-01-30 17:17:35 +01:00
|
|
|
original_documents.rewind()?;
|
2020-11-11 12:39:09 +01:00
|
|
|
|
2020-10-29 14:20:03 +01:00
|
|
|
// We create a final writer to write the new documents in order from the sorter.
|
2021-12-08 14:12:07 +01:00
|
|
|
let mut writer = create_writer(
|
|
|
|
self.indexer_settings.chunk_compression_type,
|
|
|
|
self.indexer_settings.chunk_compression_level,
|
2022-02-16 15:28:48 +01:00
|
|
|
tempfile::tempfile()?,
|
|
|
|
);
|
2023-02-08 12:53:38 +01:00
|
|
|
|
|
|
|
// Once we have written all the documents into the final sorter, we write the nested documents
|
|
|
|
// into this writer.
|
|
|
|
// We get rids of the `Operation` byte and skip the deleted documents as well.
|
|
|
|
let mut iter = self.flattened_sorter.into_stream_merger_iter()?;
|
|
|
|
while let Some((key, val)) = iter.next()? {
|
2023-10-12 11:46:56 +02:00
|
|
|
// skip first byte corresponding to the operation type (Deletion or Addition).
|
2023-02-08 12:53:38 +01:00
|
|
|
let val = &val[1..];
|
|
|
|
writer.insert(key, val)?;
|
|
|
|
}
|
2022-03-23 17:28:41 +01:00
|
|
|
let mut flattened_documents = writer.into_inner()?;
|
2023-01-30 17:17:35 +01:00
|
|
|
flattened_documents.rewind()?;
|
2022-03-23 17:28:41 +01:00
|
|
|
|
|
|
|
let mut new_external_documents_ids_builder: Vec<_> =
|
|
|
|
self.new_external_documents_ids_builder.into_iter().collect();
|
|
|
|
|
|
|
|
new_external_documents_ids_builder
|
2022-10-13 22:02:54 +02:00
|
|
|
.sort_unstable_by(|(left, _), (right, _)| left.cmp(right));
|
2022-03-23 17:28:41 +01:00
|
|
|
let mut fst_new_external_documents_ids_builder = fst::MapBuilder::memory();
|
|
|
|
new_external_documents_ids_builder.into_iter().try_for_each(|(key, value)| {
|
|
|
|
fst_new_external_documents_ids_builder.insert(key, value)
|
|
|
|
})?;
|
2020-10-23 14:11:00 +02:00
|
|
|
|
2024-04-15 13:30:41 +02:00
|
|
|
let old_inner_settings = InnerIndexSettings::from_index(self.index, wtxn)?;
|
2024-03-26 13:27:43 +01:00
|
|
|
let mut new_inner_settings = old_inner_settings.clone();
|
|
|
|
new_inner_settings.fields_ids_map = self.fields_ids_map;
|
|
|
|
let settings_diff = InnerIndexSettingsDiff {
|
|
|
|
old: old_inner_settings,
|
|
|
|
new: new_inner_settings,
|
2024-04-03 11:19:45 +02:00
|
|
|
embedding_configs_updated: false,
|
|
|
|
settings_update_only: false,
|
2024-03-26 13:27:43 +01:00
|
|
|
};
|
|
|
|
|
2020-10-23 14:11:00 +02:00
|
|
|
Ok(TransformOutput {
|
2020-10-31 16:10:15 +01:00
|
|
|
primary_key,
|
2024-03-26 13:27:43 +01:00
|
|
|
settings_diff,
|
2021-06-17 15:16:20 +02:00
|
|
|
field_distribution,
|
2022-03-23 17:28:41 +01:00
|
|
|
documents_count: self.documents_count,
|
2024-05-21 14:53:26 +02:00
|
|
|
original_documents: Some(
|
|
|
|
original_documents.into_inner().map_err(|err| err.into_error())?,
|
|
|
|
),
|
2023-09-28 16:26:01 +02:00
|
|
|
flattened_documents: flattened_documents
|
|
|
|
.into_inner()
|
2023-10-10 15:12:23 +02:00
|
|
|
.map_err(|err| err.into_error())?,
|
2020-11-03 13:20:11 +01:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2024-04-17 10:54:48 +02:00
|
|
|
/// Rebind the field_ids of the provided document to their values
|
|
|
|
/// based on the field_ids_maps difference between the old and the new settings,
|
|
|
|
/// then fill the provided buffers with delta documents using KvWritterDelAdd.
|
2024-03-26 13:27:43 +01:00
|
|
|
fn rebind_existing_document(
|
|
|
|
old_obkv: KvReader<FieldId>,
|
|
|
|
settings_diff: &InnerIndexSettingsDiff,
|
|
|
|
original_obkv_buffer: &mut Vec<u8>,
|
|
|
|
flattened_obkv_buffer: &mut Vec<u8>,
|
|
|
|
) -> Result<()> {
|
|
|
|
let mut old_fields_ids_map = settings_diff.old.fields_ids_map.clone();
|
|
|
|
let mut new_fields_ids_map = settings_diff.new.fields_ids_map.clone();
|
|
|
|
let mut obkv_writer = KvWriter::<_, FieldId>::memory();
|
|
|
|
// We iterate over the new `FieldsIdsMap` ids in order and construct the new obkv.
|
|
|
|
for (id, name) in new_fields_ids_map.iter() {
|
|
|
|
if let Some(val) = old_fields_ids_map.id(name).and_then(|id| old_obkv.get(id)) {
|
|
|
|
obkv_writer.insert(id, val)?;
|
|
|
|
}
|
|
|
|
}
|
2024-04-03 11:19:45 +02:00
|
|
|
let data = obkv_writer.into_inner()?;
|
|
|
|
let new_obkv = KvReader::<FieldId>::new(&data);
|
2024-03-26 13:27:43 +01:00
|
|
|
|
|
|
|
// take the non-flattened version if flatten_from_fields_ids_map returns None.
|
2024-04-03 11:19:45 +02:00
|
|
|
let old_flattened = Self::flatten_from_fields_ids_map(&old_obkv, &mut old_fields_ids_map)?;
|
|
|
|
let old_flattened =
|
|
|
|
old_flattened.as_deref().map_or_else(|| old_obkv, KvReader::<FieldId>::new);
|
|
|
|
let new_flattened = Self::flatten_from_fields_ids_map(&new_obkv, &mut new_fields_ids_map)?;
|
|
|
|
let new_flattened =
|
|
|
|
new_flattened.as_deref().map_or_else(|| new_obkv, KvReader::<FieldId>::new);
|
2024-03-26 13:27:43 +01:00
|
|
|
|
|
|
|
original_obkv_buffer.clear();
|
|
|
|
flattened_obkv_buffer.clear();
|
|
|
|
|
|
|
|
del_add_from_two_obkvs(&old_obkv, &new_obkv, original_obkv_buffer)?;
|
|
|
|
del_add_from_two_obkvs(&old_flattened, &new_flattened, flattened_obkv_buffer)?;
|
|
|
|
|
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
|
2022-12-06 11:38:15 +01:00
|
|
|
/// Clear all databases. Returns a `TransformOutput` with a file that contains the documents
|
|
|
|
/// of the index with the attributes reordered accordingly to the `FieldsIdsMap` given as argument.
|
|
|
|
///
|
2020-11-03 13:20:11 +01:00
|
|
|
// TODO this can be done in parallel by using the rayon `ThreadPool`.
|
2022-12-06 11:38:15 +01:00
|
|
|
pub fn prepare_for_documents_reindexing(
|
2020-11-03 13:20:11 +01:00
|
|
|
self,
|
2023-11-22 18:21:19 +01:00
|
|
|
wtxn: &mut heed::RwTxn<'i>,
|
2024-03-26 13:27:43 +01:00
|
|
|
settings_diff: InnerIndexSettingsDiff,
|
2021-06-16 18:33:33 +02:00
|
|
|
) -> Result<TransformOutput> {
|
2021-12-08 14:12:07 +01:00
|
|
|
// There already has been a document addition, the primary key should be set by now.
|
2022-12-19 15:59:22 +01:00
|
|
|
let primary_key = self
|
|
|
|
.index
|
|
|
|
.primary_key(wtxn)?
|
|
|
|
.ok_or(InternalError::DatabaseMissingEntry {
|
|
|
|
db_name: db_name::MAIN,
|
|
|
|
key: Some(main_key::PRIMARY_KEY_KEY),
|
|
|
|
})?
|
|
|
|
.to_string();
|
2021-12-08 14:12:07 +01:00
|
|
|
let field_distribution = self.index.field_distribution(wtxn)?;
|
2022-12-06 11:38:15 +01:00
|
|
|
|
2021-12-08 14:12:07 +01:00
|
|
|
let documents_ids = self.index.documents_ids(wtxn)?;
|
2020-11-03 13:20:11 +01:00
|
|
|
let documents_count = documents_ids.len() as usize;
|
|
|
|
|
2023-11-02 13:37:54 +01:00
|
|
|
// We initialize the sorter with the user indexing settings.
|
2024-05-21 14:53:26 +02:00
|
|
|
let mut original_sorter = if settings_diff.reindex_vectors() {
|
|
|
|
Some(create_sorter(
|
|
|
|
grenad::SortAlgorithm::Stable,
|
|
|
|
keep_first,
|
|
|
|
self.indexer_settings.chunk_compression_type,
|
|
|
|
self.indexer_settings.chunk_compression_level,
|
|
|
|
self.indexer_settings.max_nb_chunks,
|
|
|
|
self.indexer_settings.max_memory.map(|mem| mem / 2),
|
|
|
|
))
|
|
|
|
} else {
|
|
|
|
None
|
|
|
|
};
|
2022-03-23 17:28:41 +01:00
|
|
|
|
2023-11-02 13:37:54 +01:00
|
|
|
// We initialize the sorter with the user indexing settings.
|
|
|
|
let mut flattened_sorter = create_sorter(
|
|
|
|
grenad::SortAlgorithm::Stable,
|
|
|
|
keep_first,
|
2021-12-08 14:12:07 +01:00
|
|
|
self.indexer_settings.chunk_compression_type,
|
|
|
|
self.indexer_settings.chunk_compression_level,
|
2023-11-02 13:37:54 +01:00
|
|
|
self.indexer_settings.max_nb_chunks,
|
|
|
|
self.indexer_settings.max_memory.map(|mem| mem / 2),
|
2022-02-16 15:28:48 +01:00
|
|
|
);
|
2020-11-03 13:20:11 +01:00
|
|
|
|
2024-03-26 13:27:43 +01:00
|
|
|
let mut original_obkv_buffer = Vec::new();
|
|
|
|
let mut flattened_obkv_buffer = Vec::new();
|
2023-11-02 13:37:54 +01:00
|
|
|
let mut document_sorter_key_buffer = Vec::new();
|
2023-11-02 15:37:05 +01:00
|
|
|
for result in self.index.external_documents_ids().iter(wtxn)? {
|
2023-11-02 13:37:54 +01:00
|
|
|
let (external_id, docid) = result?;
|
2024-03-26 13:27:43 +01:00
|
|
|
let old_obkv = self.index.documents.get(wtxn, &docid)?.ok_or(
|
2023-11-02 13:37:54 +01:00
|
|
|
InternalError::DatabaseMissingEntry { db_name: db_name::DOCUMENTS, key: None },
|
|
|
|
)?;
|
2020-11-03 13:20:11 +01:00
|
|
|
|
2024-03-26 13:27:43 +01:00
|
|
|
Self::rebind_existing_document(
|
|
|
|
old_obkv,
|
|
|
|
&settings_diff,
|
|
|
|
&mut original_obkv_buffer,
|
|
|
|
&mut flattened_obkv_buffer,
|
|
|
|
)?;
|
2023-11-02 13:37:54 +01:00
|
|
|
|
|
|
|
document_sorter_key_buffer.clear();
|
|
|
|
document_sorter_key_buffer.extend_from_slice(&docid.to_be_bytes());
|
|
|
|
document_sorter_key_buffer.extend_from_slice(external_id.as_bytes());
|
2024-05-21 14:53:26 +02:00
|
|
|
if let Some(original_sorter) = original_sorter.as_mut() {
|
|
|
|
original_sorter.insert(&document_sorter_key_buffer, &original_obkv_buffer)?;
|
|
|
|
}
|
2024-03-26 13:27:43 +01:00
|
|
|
flattened_sorter.insert(docid.to_be_bytes(), &flattened_obkv_buffer)?;
|
2020-11-03 13:20:11 +01:00
|
|
|
}
|
|
|
|
|
2023-11-02 13:37:54 +01:00
|
|
|
let grenad_params = GrenadParameters {
|
|
|
|
chunk_compression_type: self.indexer_settings.chunk_compression_type,
|
|
|
|
chunk_compression_level: self.indexer_settings.chunk_compression_level,
|
|
|
|
max_memory: self.indexer_settings.max_memory,
|
|
|
|
max_nb_chunks: self.indexer_settings.max_nb_chunks, // default value, may be chosen.
|
|
|
|
};
|
2022-03-23 17:28:41 +01:00
|
|
|
|
2023-11-02 13:37:54 +01:00
|
|
|
// Once we have written all the documents, we merge everything into a Reader.
|
|
|
|
let flattened_documents = sorter_into_reader(flattened_sorter, grenad_params)?;
|
2024-05-21 14:53:26 +02:00
|
|
|
let original_documents = match original_sorter {
|
|
|
|
Some(original_sorter) => Some(sorter_into_reader(original_sorter, grenad_params)?),
|
|
|
|
None => None,
|
|
|
|
};
|
2020-11-03 13:20:11 +01:00
|
|
|
|
2024-03-26 13:27:43 +01:00
|
|
|
Ok(TransformOutput {
|
2020-11-03 13:20:11 +01:00
|
|
|
primary_key,
|
2021-06-17 15:16:20 +02:00
|
|
|
field_distribution,
|
2024-03-26 13:27:43 +01:00
|
|
|
settings_diff,
|
2020-11-03 13:20:11 +01:00
|
|
|
documents_count,
|
2024-05-21 14:53:26 +02:00
|
|
|
original_documents: original_documents.map(|od| od.into_inner().into_inner()),
|
2023-11-02 13:37:54 +01:00
|
|
|
flattened_documents: flattened_documents.into_inner().into_inner(),
|
2024-03-26 13:27:43 +01:00
|
|
|
})
|
2020-10-23 14:11:00 +02:00
|
|
|
}
|
|
|
|
}
|
2020-10-31 16:10:15 +01:00
|
|
|
|
2021-08-31 11:44:15 +02:00
|
|
|
/// Drops all the value of type `U` in vec, and reuses the allocation to create a `Vec<T>`.
|
|
|
|
///
|
|
|
|
/// The size and alignment of T and U must match.
|
|
|
|
fn drop_and_reuse<U, T>(mut vec: Vec<U>) -> Vec<T> {
|
|
|
|
debug_assert_eq!(std::mem::align_of::<U>(), std::mem::align_of::<T>());
|
|
|
|
debug_assert_eq!(std::mem::size_of::<U>(), std::mem::size_of::<T>());
|
|
|
|
vec.clear();
|
|
|
|
debug_assert!(vec.is_empty());
|
|
|
|
vec.into_iter().map(|_| unreachable!()).collect()
|
|
|
|
}
|
|
|
|
|
2023-02-14 18:23:57 +01:00
|
|
|
#[cfg(test)]
|
|
|
|
mod test {
|
|
|
|
use super::*;
|
|
|
|
|
|
|
|
#[test]
|
|
|
|
fn merge_obkvs() {
|
2023-10-12 11:46:56 +02:00
|
|
|
let mut additive_doc_0 = Vec::new();
|
|
|
|
let mut deletive_doc_0 = Vec::new();
|
|
|
|
let mut del_add_doc_0 = Vec::new();
|
|
|
|
let mut kv_writer = KvWriter::memory();
|
2023-02-14 18:23:57 +01:00
|
|
|
kv_writer.insert(0_u8, [0]).unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
let buffer = kv_writer.into_inner().unwrap();
|
2023-11-20 10:53:40 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&buffer),
|
|
|
|
DelAddOperation::Addition,
|
|
|
|
&mut additive_doc_0,
|
|
|
|
)
|
|
|
|
.unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
additive_doc_0.insert(0, Operation::Addition as u8);
|
2023-11-20 10:53:40 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&buffer),
|
|
|
|
DelAddOperation::Deletion,
|
|
|
|
&mut deletive_doc_0,
|
|
|
|
)
|
|
|
|
.unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
deletive_doc_0.insert(0, Operation::Deletion as u8);
|
2023-11-20 10:53:40 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&buffer),
|
|
|
|
DelAddOperation::DeletionAndAddition,
|
|
|
|
&mut del_add_doc_0,
|
|
|
|
)
|
|
|
|
.unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
del_add_doc_0.insert(0, Operation::Addition as u8);
|
|
|
|
|
|
|
|
let mut additive_doc_1 = Vec::new();
|
|
|
|
let mut kv_writer = KvWriter::memory();
|
|
|
|
kv_writer.insert(1_u8, [1]).unwrap();
|
|
|
|
let buffer = kv_writer.into_inner().unwrap();
|
2023-11-20 10:53:40 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&buffer),
|
|
|
|
DelAddOperation::Addition,
|
|
|
|
&mut additive_doc_1,
|
|
|
|
)
|
|
|
|
.unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
additive_doc_1.insert(0, Operation::Addition as u8);
|
|
|
|
|
|
|
|
let mut additive_doc_0_1 = Vec::new();
|
|
|
|
let mut kv_writer = KvWriter::memory();
|
|
|
|
kv_writer.insert(0_u8, [0]).unwrap();
|
|
|
|
kv_writer.insert(1_u8, [1]).unwrap();
|
|
|
|
let buffer = kv_writer.into_inner().unwrap();
|
2023-11-20 10:53:40 +01:00
|
|
|
into_del_add_obkv(
|
|
|
|
KvReaderU16::new(&buffer),
|
|
|
|
DelAddOperation::Addition,
|
|
|
|
&mut additive_doc_0_1,
|
|
|
|
)
|
|
|
|
.unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
additive_doc_0_1.insert(0, Operation::Addition as u8);
|
|
|
|
|
|
|
|
let ret = obkvs_merge_additions_and_deletions(&[], &[Cow::from(additive_doc_0.as_slice())])
|
|
|
|
.unwrap();
|
|
|
|
assert_eq!(*ret, additive_doc_0);
|
2023-02-14 18:23:57 +01:00
|
|
|
|
2023-10-12 11:46:56 +02:00
|
|
|
let ret = obkvs_merge_additions_and_deletions(
|
|
|
|
&[],
|
|
|
|
&[Cow::from(deletive_doc_0.as_slice()), Cow::from(additive_doc_0.as_slice())],
|
|
|
|
)
|
|
|
|
.unwrap();
|
|
|
|
assert_eq!(*ret, del_add_doc_0);
|
|
|
|
|
|
|
|
let ret = obkvs_merge_additions_and_deletions(
|
|
|
|
&[],
|
|
|
|
&[Cow::from(additive_doc_0.as_slice()), Cow::from(deletive_doc_0.as_slice())],
|
|
|
|
)
|
|
|
|
.unwrap();
|
|
|
|
assert_eq!(*ret, deletive_doc_0);
|
|
|
|
|
|
|
|
let ret = obkvs_merge_additions_and_deletions(
|
|
|
|
&[],
|
|
|
|
&[
|
|
|
|
Cow::from(additive_doc_1.as_slice()),
|
|
|
|
Cow::from(deletive_doc_0.as_slice()),
|
|
|
|
Cow::from(additive_doc_0.as_slice()),
|
|
|
|
],
|
|
|
|
)
|
|
|
|
.unwrap();
|
|
|
|
assert_eq!(*ret, del_add_doc_0);
|
2023-02-14 18:23:57 +01:00
|
|
|
|
2023-10-12 11:46:56 +02:00
|
|
|
let ret = obkvs_merge_additions_and_deletions(
|
2023-02-14 18:23:57 +01:00
|
|
|
&[],
|
2023-10-12 11:46:56 +02:00
|
|
|
&[Cow::from(additive_doc_1.as_slice()), Cow::from(additive_doc_0.as_slice())],
|
2023-02-14 18:23:57 +01:00
|
|
|
)
|
|
|
|
.unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
assert_eq!(*ret, additive_doc_0_1);
|
2023-02-14 18:23:57 +01:00
|
|
|
|
2023-10-12 11:46:56 +02:00
|
|
|
let ret = obkvs_keep_last_addition_merge_deletions(
|
2023-02-14 18:23:57 +01:00
|
|
|
&[],
|
2023-10-12 11:46:56 +02:00
|
|
|
&[Cow::from(additive_doc_1.as_slice()), Cow::from(additive_doc_0.as_slice())],
|
2023-02-14 18:23:57 +01:00
|
|
|
)
|
|
|
|
.unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
assert_eq!(*ret, additive_doc_0);
|
2023-02-14 18:23:57 +01:00
|
|
|
|
2023-10-12 11:46:56 +02:00
|
|
|
let ret = obkvs_keep_last_addition_merge_deletions(
|
2023-02-14 18:23:57 +01:00
|
|
|
&[],
|
|
|
|
&[
|
2023-10-12 11:46:56 +02:00
|
|
|
Cow::from(deletive_doc_0.as_slice()),
|
|
|
|
Cow::from(additive_doc_1.as_slice()),
|
|
|
|
Cow::from(additive_doc_0.as_slice()),
|
2023-02-14 18:23:57 +01:00
|
|
|
],
|
|
|
|
)
|
|
|
|
.unwrap();
|
2023-10-12 11:46:56 +02:00
|
|
|
assert_eq!(*ret, del_add_doc_0);
|
2023-02-14 18:23:57 +01:00
|
|
|
}
|
|
|
|
}
|