Remove useless Transform methods

This commit is contained in:
Clément Renault
2024-11-18 11:59:18 +01:00
parent 41dbdd2d18
commit 670aff5553
2 changed files with 11 additions and 1170 deletions

View File

@ -1,33 +1,29 @@
use std::borrow::Cow;
use std::collections::btree_map::Entry as BEntry;
use std::collections::hash_map::Entry as HEntry;
use std::collections::{BTreeMap, HashMap, HashSet};
use std::fs::File;
use std::io::{Read, Seek};
use either::Either;
use fxhash::FxHashMap;
use itertools::Itertools;
use obkv::{KvReader, KvReaderU16, KvWriter};
use obkv::{KvReader, KvWriter};
use roaring::RoaringBitmap;
use serde_json::Value;
use smartstring::SmartString;
use super::helpers::{
create_sorter, create_writer, sorter_into_reader, EitherObkvMerge,
ObkvsKeepLastAdditionMergeDeletions, ObkvsMergeAdditionsAndDeletions,
create_sorter, sorter_into_reader, EitherObkvMerge, ObkvsKeepLastAdditionMergeDeletions,
ObkvsMergeAdditionsAndDeletions,
};
use super::{IndexDocumentsMethod, IndexerConfig, KeepFirst};
use crate::documents::{DocumentsBatchIndex, EnrichedDocument, EnrichedDocumentsBatchReader};
use crate::documents::DocumentsBatchIndex;
use crate::error::{Error, InternalError, UserError};
use crate::index::{db_name, main_key};
use crate::update::del_add::{
into_del_add_obkv, into_del_add_obkv_conditional_operation, DelAdd, DelAddOperation,
KvReaderDelAdd,
into_del_add_obkv, into_del_add_obkv_conditional_operation, DelAddOperation,
};
use crate::update::index_documents::GrenadParameters;
use crate::update::settings::{InnerIndexSettings, InnerIndexSettingsDiff};
use crate::update::{AvailableIds, UpdateIndexingStep};
use crate::update::settings::InnerIndexSettingsDiff;
use crate::update::AvailableIds;
use crate::vector::parsed_vectors::{ExplicitVectors, VectorOrArrayOfVectors};
use crate::vector::settings::WriteBackToDocuments;
use crate::vector::ArroyWrapper;
@ -157,399 +153,6 @@ impl<'a, 'i> Transform<'a, 'i> {
})
}
#[tracing::instrument(level = "trace", skip_all, target = "indexing::documents")]
pub fn read_documents<R, FP, FA>(
&mut self,
reader: EnrichedDocumentsBatchReader<R>,
wtxn: &mut heed::RwTxn<'_>,
progress_callback: FP,
should_abort: FA,
) -> Result<usize>
where
R: Read + Seek,
FP: Fn(UpdateIndexingStep) + Sync,
FA: Fn() -> bool + Sync,
{
let (mut cursor, fields_index) = reader.into_cursor_and_fields_index();
let external_documents_ids = self.index.external_documents_ids();
let mapping = create_fields_mapping(&mut self.fields_ids_map, &fields_index)?;
let primary_key = cursor.primary_key().to_string();
let primary_key_id =
self.fields_ids_map.insert(&primary_key).ok_or(UserError::AttributeLimitReached)?;
let mut obkv_buffer = Vec::new();
let mut document_sorter_value_buffer = Vec::new();
let mut document_sorter_key_buffer = Vec::new();
let mut documents_count = 0;
let mut docid_buffer: Vec<u8> = Vec::new();
let mut field_buffer: Vec<(u16, Cow<'_, [u8]>)> = Vec::new();
while let Some(enriched_document) = cursor.next_enriched_document()? {
let EnrichedDocument { document, document_id } = enriched_document;
if should_abort() {
return Err(Error::InternalError(InternalError::AbortedIndexation));
}
// 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
let mut field_buffer_cache = drop_and_reuse(field_buffer);
if self.indexer_settings.log_every_n.map_or(false, |len| documents_count % len == 0) {
progress_callback(UpdateIndexingStep::RemapDocumentAddition {
documents_seen: documents_count,
});
}
// 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() {
serde_json::to_writer(&mut docid_buffer, external_id)
.map_err(InternalError::SerdeJson)?;
field_buffer_cache.push((primary_key_id, Cow::from(&docid_buffer)));
}
for (k, v) in document.iter() {
let mapped_id =
*mapping.get(&k).ok_or(InternalError::FieldIdMappingMissingEntry { key: k })?;
field_buffer_cache.push((mapped_id, Cow::from(v)));
}
// 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.
field_buffer_cache.sort_unstable_by(|(f1, _), (f2, _)| f1.cmp(f2));
// Build the new obkv document.
let mut writer = KvWriter::new(&mut obkv_buffer);
for (k, v) in field_buffer_cache.iter() {
writer.insert(*k, v)?;
}
let mut original_docid = None;
let docid = match self.new_external_documents_ids_builder.entry((*external_id).into()) {
HEntry::Occupied(entry) => *entry.get() as u32,
HEntry::Vacant(entry) => {
let docid = match external_documents_ids.get(wtxn, entry.key())? {
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
}
None => self
.available_documents_ids
.next()
.ok_or(UserError::DocumentLimitReached)?,
};
entry.insert(docid as u64);
docid
}
};
let mut skip_insertion = false;
if let Some(original_docid) = original_docid {
let original_key = original_docid;
let base_obkv = self
.index
.documents
.remap_data_type::<heed::types::Bytes>()
.get(wtxn, &original_key)?
.ok_or(InternalError::DatabaseMissingEntry {
db_name: db_name::DOCUMENTS,
key: None,
})?;
// 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
self.new_external_documents_ids_builder.remove(external_id);
skip_insertion = true;
} else {
// we associate the base document with the new key, everything will get merged later.
let deladd_operation = match self.index_documents_method {
IndexDocumentsMethod::UpdateDocuments => {
DelAddOperation::DeletionAndAddition
}
IndexDocumentsMethod::ReplaceDocuments => DelAddOperation::Deletion,
};
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);
into_del_add_obkv(
KvReaderU16::from_slice(base_obkv),
deladd_operation,
&mut document_sorter_value_buffer,
)?;
self.original_sorter
.insert(&document_sorter_key_buffer, &document_sorter_value_buffer)?;
let base_obkv = KvReader::from_slice(base_obkv);
if let Some(flattened_obkv) =
Self::flatten_from_fields_ids_map(base_obkv, &mut self.fields_ids_map)?
{
// we recreate our buffer with the flattened documents
document_sorter_value_buffer.clear();
document_sorter_value_buffer.push(Operation::Addition as u8);
into_del_add_obkv(
KvReaderU16::from_slice(&flattened_obkv),
deladd_operation,
&mut document_sorter_value_buffer,
)?;
}
self.flattened_sorter
.insert(docid.to_be_bytes(), &document_sorter_value_buffer)?;
}
}
if !skip_insertion {
self.new_documents_ids.insert(docid);
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);
into_del_add_obkv(
KvReaderU16::from_slice(&obkv_buffer),
DelAddOperation::Addition,
&mut document_sorter_value_buffer,
)?;
// We use the extracted/generated user id as the key for this document.
self.original_sorter
.insert(&document_sorter_key_buffer, &document_sorter_value_buffer)?;
let flattened_obkv = KvReader::from_slice(&obkv_buffer);
if let Some(obkv) =
Self::flatten_from_fields_ids_map(flattened_obkv, &mut self.fields_ids_map)?
{
document_sorter_value_buffer.clear();
document_sorter_value_buffer.push(Operation::Addition as u8);
into_del_add_obkv(
KvReaderU16::from_slice(&obkv),
DelAddOperation::Addition,
&mut document_sorter_value_buffer,
)?
}
self.flattened_sorter.insert(docid.to_be_bytes(), &document_sorter_value_buffer)?;
}
documents_count += 1;
progress_callback(UpdateIndexingStep::RemapDocumentAddition {
documents_seen: documents_count,
});
field_buffer = drop_and_reuse(field_buffer_cache);
docid_buffer.clear();
obkv_buffer.clear();
}
progress_callback(UpdateIndexingStep::RemapDocumentAddition {
documents_seen: documents_count,
});
self.index.put_fields_ids_map(wtxn, &self.fields_ids_map)?;
self.index.put_primary_key(wtxn, &primary_key)?;
self.documents_count += documents_count;
// 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.
Ok(documents_count)
}
/// 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.
#[tracing::instrument(level = "trace", skip_all, target = "indexing::documents")]
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,
{
// there may be duplicates in the documents to remove.
to_remove.sort_unstable();
to_remove.dedup();
let external_documents_ids = self.index.external_documents_ids();
let mut documents_deleted = 0;
let mut document_sorter_value_buffer = Vec::new();
let mut document_sorter_key_buffer = Vec::new();
for to_remove in to_remove {
if should_abort() {
return Err(Error::InternalError(InternalError::AbortedIndexation));
}
// Check if the document has been added in the current indexing process.
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,
};
// 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.
let deleted_from_db = match external_documents_ids.get(wtxn, &to_remove)? {
Some(docid) => {
self.remove_document_from_db(
docid,
to_remove,
wtxn,
&mut document_sorter_key_buffer,
&mut document_sorter_value_buffer,
)?;
true
}
None => false,
};
// increase counter only if the document existed somewhere before.
if deleted_from_current || deleted_from_db {
documents_deleted += 1;
}
}
Ok(documents_deleted)
}
/// Removes documents from db using their internal document ids.
///
/// # 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.
#[tracing::instrument(level = "trace", skip_all, target = "indexing::details")]
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,
{
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())?;
for (internal_docid, external_docid) in to_remove.iter().zip(external_ids) {
let external_docid = external_docid?;
if should_abort() {
return Err(Error::InternalError(InternalError::AbortedIndexation));
}
self.remove_document_from_db(
internal_docid,
external_docid,
wtxn,
&mut document_sorter_key_buffer,
&mut document_sorter_value_buffer,
)?;
documents_deleted += 1;
}
Ok(documents_deleted)
}
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
let original_key = internal_docid;
let base_obkv = self
.index
.documents
.remap_data_type::<heed::types::Bytes>()
.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);
into_del_add_obkv(
KvReaderU16::from_slice(base_obkv),
DelAddOperation::Deletion,
document_sorter_value_buffer,
)?;
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::from_slice(base_obkv);
if let Some(obkv) =
Self::flatten_from_fields_ids_map(flattened_obkv, &mut self.fields_ids_map)?
{
// we recreate our buffer with the flattened documents
document_sorter_value_buffer.clear();
document_sorter_value_buffer.push(Operation::Deletion as u8);
into_del_add_obkv(
KvReaderU16::from_slice(&obkv),
DelAddOperation::Deletion,
document_sorter_value_buffer,
)?;
}
self.flattened_sorter
.insert(internal_docid.to_be_bytes(), &document_sorter_value_buffer)?;
Ok(())
}
// Flatten a document from the fields ids map contained in self and insert the new
// created fields. Returns `None` if the document doesn't need to be flattened.
#[tracing::instrument(
@ -677,167 +280,6 @@ impl<'a, 'i> Transform<'a, 'i> {
Ok(())
}
/// 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
/// id for the user side and the value must be an obkv where keys are valid fields ids.
#[tracing::instrument(level = "trace", skip_all, target = "indexing::transform")]
pub(crate) fn output_from_sorter<F>(
self,
wtxn: &mut heed::RwTxn<'_>,
progress_callback: F,
) -> Result<TransformOutput>
where
F: Fn(UpdateIndexingStep) + Sync,
{
let primary_key = self
.index
.primary_key(wtxn)?
.ok_or(Error::InternalError(InternalError::DatabaseMissingEntry {
db_name: db_name::MAIN,
key: Some(main_key::PRIMARY_KEY_KEY),
}))?
.to_string();
// We create a final writer to write the new documents in order from the sorter.
let mut writer = create_writer(
self.indexer_settings.chunk_compression_type,
self.indexer_settings.chunk_compression_level,
tempfile::tempfile()?,
);
// 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
let mut field_distribution = self.index.field_distribution(wtxn)?;
// Here we are going to do the document count + field distribution + `write_into_stream_writer`
let mut iter = self.original_sorter.into_stream_merger_iter()?;
// used only for the callback
let mut documents_count = 0;
while let Some((key, val)) = iter.next()? {
// skip first byte corresponding to the operation type (Deletion or Addition).
let val = &val[1..];
// 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,
});
for (key, value) in KvReader::from_slice(val) {
let reader = KvReaderDelAdd::from_slice(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) {
Some(0) => {
entry.remove();
}
Some(new_val) => {
*entry.get_mut() = new_val;
}
None => {
unreachable!("Attempting to remove a field that wasn't in the field distribution")
}
}
}
}
}
(Some(_), Some(_)) => {
// Value change, no field distribution change
}
}
}
writer.insert(key, val)?;
}
let mut original_documents = writer.into_inner()?;
// We then extract the file and reset the seek to be able to read it again.
original_documents.rewind()?;
// We create a final writer to write the new documents in order from the sorter.
let mut writer = create_writer(
self.indexer_settings.chunk_compression_type,
self.indexer_settings.chunk_compression_level,
tempfile::tempfile()?,
);
// 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()? {
// skip first byte corresponding to the operation type (Deletion or Addition).
let val = &val[1..];
writer.insert(key, val)?;
}
let mut flattened_documents = writer.into_inner()?;
flattened_documents.rewind()?;
let mut new_external_documents_ids_builder: Vec<_> =
self.new_external_documents_ids_builder.into_iter().collect();
new_external_documents_ids_builder
.sort_unstable_by(|(left, _), (right, _)| left.cmp(right));
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)
})?;
let old_inner_settings = InnerIndexSettings::from_index(self.index, wtxn)?;
let fields_ids_map = self.fields_ids_map;
let primary_key_id = self.index.primary_key(wtxn)?.and_then(|name| fields_ids_map.id(name));
let mut new_inner_settings = old_inner_settings.clone();
new_inner_settings.fields_ids_map = fields_ids_map;
let embedding_config_updates = Default::default();
let settings_update_only = false;
let settings_diff = InnerIndexSettingsDiff::new(
old_inner_settings,
new_inner_settings,
primary_key_id,
embedding_config_updates,
settings_update_only,
);
Ok(TransformOutput {
primary_key,
settings_diff,
field_distribution,
documents_count: self.documents_count,
original_documents: Some(
original_documents.into_inner().map_err(|err| err.into_error())?,
),
flattened_documents: Some(
flattened_documents.into_inner().map_err(|err| err.into_error())?,
),
})
}
/// 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.
@ -1145,6 +587,7 @@ fn drop_and_reuse<U, T>(mut vec: Vec<U>) -> Vec<T> {
#[cfg(test)]
mod test {
use grenad::MergeFunction;
use obkv::KvReaderU16;
use super::*;