mirror of
				https://github.com/meilisearch/meilisearch.git
				synced 2025-11-04 01:46:28 +00:00 
			
		
		
		
	Merge branch 'main' into word-pair-proximity-docids-refactor
This commit is contained in:
		@@ -40,7 +40,7 @@ pub fn extract_fid_word_count_docids<R: io::Read + io::Seek>(
 | 
			
		||||
    let mut cursor = docid_word_positions.into_cursor()?;
 | 
			
		||||
    while let Some((key, value)) = cursor.move_on_next()? {
 | 
			
		||||
        let (document_id_bytes, _word_bytes) = try_split_array_at(key)
 | 
			
		||||
            .ok_or_else(|| SerializationError::Decoding { db_name: Some(DOCID_WORD_POSITIONS) })?;
 | 
			
		||||
            .ok_or(SerializationError::Decoding { db_name: Some(DOCID_WORD_POSITIONS) })?;
 | 
			
		||||
        let document_id = u32::from_be_bytes(document_id_bytes);
 | 
			
		||||
 | 
			
		||||
        let curr_document_id = *current_document_id.get_or_insert(document_id);
 | 
			
		||||
 
 | 
			
		||||
@@ -60,5 +60,5 @@ pub fn extract_geo_points<R: io::Read + io::Seek>(
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    Ok(writer_into_reader(writer)?)
 | 
			
		||||
    writer_into_reader(writer)
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
@@ -51,7 +51,7 @@ pub fn extract_word_docids<R: io::Read + io::Seek>(
 | 
			
		||||
    let mut cursor = docid_word_positions.into_cursor()?;
 | 
			
		||||
    while let Some((key, positions)) = cursor.move_on_next()? {
 | 
			
		||||
        let (document_id_bytes, word_bytes) = try_split_array_at(key)
 | 
			
		||||
            .ok_or_else(|| SerializationError::Decoding { db_name: Some(DOCID_WORD_POSITIONS) })?;
 | 
			
		||||
            .ok_or(SerializationError::Decoding { db_name: Some(DOCID_WORD_POSITIONS) })?;
 | 
			
		||||
        let document_id = u32::from_be_bytes(document_id_bytes);
 | 
			
		||||
 | 
			
		||||
        let bitmap = RoaringBitmap::from_iter(Some(document_id));
 | 
			
		||||
 
 | 
			
		||||
@@ -39,7 +39,7 @@ pub fn extract_word_pair_proximity_docids<R: io::Read + io::Seek>(
 | 
			
		||||
    let mut cursor = docid_word_positions.into_cursor()?;
 | 
			
		||||
    while let Some((key, value)) = cursor.move_on_next()? {
 | 
			
		||||
        let (document_id_bytes, word_bytes) = try_split_array_at(key)
 | 
			
		||||
            .ok_or_else(|| SerializationError::Decoding { db_name: Some(DOCID_WORD_POSITIONS) })?;
 | 
			
		||||
            .ok_or(SerializationError::Decoding { db_name: Some(DOCID_WORD_POSITIONS) })?;
 | 
			
		||||
        let document_id = u32::from_be_bytes(document_id_bytes);
 | 
			
		||||
        let word = str::from_utf8(word_bytes)?;
 | 
			
		||||
 | 
			
		||||
@@ -81,7 +81,7 @@ pub fn extract_word_pair_proximity_docids<R: io::Read + io::Seek>(
 | 
			
		||||
///
 | 
			
		||||
/// This list is used by the engine to calculate the documents containing words that are
 | 
			
		||||
/// close to each other.
 | 
			
		||||
fn document_word_positions_into_sorter<'b>(
 | 
			
		||||
fn document_word_positions_into_sorter(
 | 
			
		||||
    document_id: DocumentId,
 | 
			
		||||
    mut word_positions_heap: BinaryHeap<PeekedWordPosition<vec::IntoIter<u32>>>,
 | 
			
		||||
    word_pair_proximity_docids_sorter: &mut grenad::Sorter<MergeFn>,
 | 
			
		||||
 
 | 
			
		||||
@@ -33,7 +33,7 @@ pub fn extract_word_position_docids<R: io::Read + io::Seek>(
 | 
			
		||||
    let mut cursor = docid_word_positions.into_cursor()?;
 | 
			
		||||
    while let Some((key, value)) = cursor.move_on_next()? {
 | 
			
		||||
        let (document_id_bytes, word_bytes) = try_split_array_at(key)
 | 
			
		||||
            .ok_or_else(|| SerializationError::Decoding { db_name: Some(DOCID_WORD_POSITIONS) })?;
 | 
			
		||||
            .ok_or(SerializationError::Decoding { db_name: Some(DOCID_WORD_POSITIONS) })?;
 | 
			
		||||
        let document_id = DocumentId::from_be_bytes(document_id_bytes);
 | 
			
		||||
 | 
			
		||||
        for position in read_u32_ne_bytes(value) {
 | 
			
		||||
 
 | 
			
		||||
@@ -96,7 +96,7 @@ pub(crate) fn data_from_obkv_documents(
 | 
			
		||||
 | 
			
		||||
    spawn_extraction_task::<_, _, Vec<grenad::Reader<File>>>(
 | 
			
		||||
        docid_word_positions_chunks.clone(),
 | 
			
		||||
        indexer.clone(),
 | 
			
		||||
        indexer,
 | 
			
		||||
        lmdb_writer_sx.clone(),
 | 
			
		||||
        extract_word_pair_proximity_docids,
 | 
			
		||||
        merge_cbo_roaring_bitmaps,
 | 
			
		||||
@@ -106,7 +106,7 @@ pub(crate) fn data_from_obkv_documents(
 | 
			
		||||
 | 
			
		||||
    spawn_extraction_task::<_, _, Vec<grenad::Reader<File>>>(
 | 
			
		||||
        docid_word_positions_chunks.clone(),
 | 
			
		||||
        indexer.clone(),
 | 
			
		||||
        indexer,
 | 
			
		||||
        lmdb_writer_sx.clone(),
 | 
			
		||||
        extract_fid_word_count_docids,
 | 
			
		||||
        merge_cbo_roaring_bitmaps,
 | 
			
		||||
@@ -116,7 +116,7 @@ pub(crate) fn data_from_obkv_documents(
 | 
			
		||||
 | 
			
		||||
    spawn_extraction_task::<_, _, Vec<(grenad::Reader<File>, grenad::Reader<File>)>>(
 | 
			
		||||
        docid_word_positions_chunks.clone(),
 | 
			
		||||
        indexer.clone(),
 | 
			
		||||
        indexer,
 | 
			
		||||
        lmdb_writer_sx.clone(),
 | 
			
		||||
        move |doc_word_pos, indexer| extract_word_docids(doc_word_pos, indexer, &exact_attributes),
 | 
			
		||||
        merge_roaring_bitmaps,
 | 
			
		||||
@@ -128,8 +128,8 @@ pub(crate) fn data_from_obkv_documents(
 | 
			
		||||
    );
 | 
			
		||||
 | 
			
		||||
    spawn_extraction_task::<_, _, Vec<grenad::Reader<File>>>(
 | 
			
		||||
        docid_word_positions_chunks.clone(),
 | 
			
		||||
        indexer.clone(),
 | 
			
		||||
        docid_word_positions_chunks,
 | 
			
		||||
        indexer,
 | 
			
		||||
        lmdb_writer_sx.clone(),
 | 
			
		||||
        extract_word_position_docids,
 | 
			
		||||
        merge_cbo_roaring_bitmaps,
 | 
			
		||||
@@ -138,8 +138,8 @@ pub(crate) fn data_from_obkv_documents(
 | 
			
		||||
    );
 | 
			
		||||
 | 
			
		||||
    spawn_extraction_task::<_, _, Vec<grenad::Reader<File>>>(
 | 
			
		||||
        docid_fid_facet_strings_chunks.clone(),
 | 
			
		||||
        indexer.clone(),
 | 
			
		||||
        docid_fid_facet_strings_chunks,
 | 
			
		||||
        indexer,
 | 
			
		||||
        lmdb_writer_sx.clone(),
 | 
			
		||||
        extract_facet_string_docids,
 | 
			
		||||
        keep_first_prefix_value_merge_roaring_bitmaps,
 | 
			
		||||
@@ -148,8 +148,8 @@ pub(crate) fn data_from_obkv_documents(
 | 
			
		||||
    );
 | 
			
		||||
 | 
			
		||||
    spawn_extraction_task::<_, _, Vec<grenad::Reader<File>>>(
 | 
			
		||||
        docid_fid_facet_numbers_chunks.clone(),
 | 
			
		||||
        indexer.clone(),
 | 
			
		||||
        docid_fid_facet_numbers_chunks,
 | 
			
		||||
        indexer,
 | 
			
		||||
        lmdb_writer_sx.clone(),
 | 
			
		||||
        extract_facet_number_docids,
 | 
			
		||||
        merge_cbo_roaring_bitmaps,
 | 
			
		||||
@@ -183,12 +183,12 @@ fn spawn_extraction_task<FE, FS, M>(
 | 
			
		||||
{
 | 
			
		||||
    rayon::spawn(move || {
 | 
			
		||||
        let chunks: Result<M> =
 | 
			
		||||
            chunks.into_par_iter().map(|chunk| extract_fn(chunk, indexer.clone())).collect();
 | 
			
		||||
            chunks.into_par_iter().map(|chunk| extract_fn(chunk, indexer)).collect();
 | 
			
		||||
        rayon::spawn(move || match chunks {
 | 
			
		||||
            Ok(chunks) => {
 | 
			
		||||
                debug!("merge {} database", name);
 | 
			
		||||
                let reader = chunks.merge(merge_fn, &indexer);
 | 
			
		||||
                let _ = lmdb_writer_sx.send(reader.map(|r| serialize_fn(r)));
 | 
			
		||||
                let _ = lmdb_writer_sx.send(reader.map(serialize_fn));
 | 
			
		||||
            }
 | 
			
		||||
            Err(e) => {
 | 
			
		||||
                let _ = lmdb_writer_sx.send(Err(e));
 | 
			
		||||
@@ -255,7 +255,7 @@ fn send_and_extract_flattened_documents_data(
 | 
			
		||||
            || {
 | 
			
		||||
                let (documents_ids, docid_word_positions_chunk) = extract_docid_word_positions(
 | 
			
		||||
                    flattened_documents_chunk.clone(),
 | 
			
		||||
                    indexer.clone(),
 | 
			
		||||
                    indexer,
 | 
			
		||||
                    searchable_fields,
 | 
			
		||||
                    stop_words.as_ref(),
 | 
			
		||||
                    max_positions_per_attributes,
 | 
			
		||||
@@ -279,7 +279,7 @@ fn send_and_extract_flattened_documents_data(
 | 
			
		||||
                    fid_facet_exists_docids_chunk,
 | 
			
		||||
                ) = extract_fid_docid_facet_values(
 | 
			
		||||
                    flattened_documents_chunk.clone(),
 | 
			
		||||
                    indexer.clone(),
 | 
			
		||||
                    indexer,
 | 
			
		||||
                    faceted_fields,
 | 
			
		||||
                )?;
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -61,7 +61,7 @@ pub fn sorter_into_reader(
 | 
			
		||||
    );
 | 
			
		||||
    sorter.write_into_stream_writer(&mut writer)?;
 | 
			
		||||
 | 
			
		||||
    Ok(writer_into_reader(writer)?)
 | 
			
		||||
    writer_into_reader(writer)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
pub fn writer_into_reader(writer: grenad::Writer<File>) -> Result<grenad::Reader<File>> {
 | 
			
		||||
@@ -134,7 +134,7 @@ impl<R: io::Read + io::Seek> MergerBuilder<R> {
 | 
			
		||||
        );
 | 
			
		||||
        merger.write_into_stream_writer(&mut writer)?;
 | 
			
		||||
 | 
			
		||||
        Ok(writer_into_reader(writer)?)
 | 
			
		||||
        writer_into_reader(writer)
 | 
			
		||||
    }
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@@ -180,7 +180,6 @@ pub fn grenad_obkv_into_chunks<R: io::Read + io::Seek>(
 | 
			
		||||
    let mut continue_reading = true;
 | 
			
		||||
    let mut cursor = reader.into_cursor()?;
 | 
			
		||||
 | 
			
		||||
    let indexer_clone = indexer.clone();
 | 
			
		||||
    let mut transposer = move || {
 | 
			
		||||
        if !continue_reading {
 | 
			
		||||
            return Ok(None);
 | 
			
		||||
@@ -188,8 +187,8 @@ pub fn grenad_obkv_into_chunks<R: io::Read + io::Seek>(
 | 
			
		||||
 | 
			
		||||
        let mut current_chunk_size = 0u64;
 | 
			
		||||
        let mut obkv_documents = create_writer(
 | 
			
		||||
            indexer_clone.chunk_compression_type,
 | 
			
		||||
            indexer_clone.chunk_compression_level,
 | 
			
		||||
            indexer.chunk_compression_type,
 | 
			
		||||
            indexer.chunk_compression_level,
 | 
			
		||||
            tempfile::tempfile()?,
 | 
			
		||||
        );
 | 
			
		||||
 | 
			
		||||
@@ -224,7 +223,7 @@ pub fn write_into_lmdb_database(
 | 
			
		||||
        match iter.next().transpose()? {
 | 
			
		||||
            Some((key, old_val)) if key == k => {
 | 
			
		||||
                let vals = &[Cow::Borrowed(old_val), Cow::Borrowed(v)][..];
 | 
			
		||||
                let val = merge(k, &vals)?;
 | 
			
		||||
                let val = merge(k, vals)?;
 | 
			
		||||
                // safety: we don't keep references from inside the LMDB database.
 | 
			
		||||
                unsafe { iter.put_current(k, &val)? };
 | 
			
		||||
            }
 | 
			
		||||
 
 | 
			
		||||
@@ -88,7 +88,7 @@ pub fn keep_latest_obkv<'a>(_key: &[u8], obkvs: &[Cow<'a, [u8]>]) -> Result<Cow<
 | 
			
		||||
/// Merge all the obks in the order we see them.
 | 
			
		||||
pub fn merge_obkvs<'a>(_key: &[u8], obkvs: &[Cow<'a, [u8]>]) -> Result<Cow<'a, [u8]>> {
 | 
			
		||||
    Ok(obkvs
 | 
			
		||||
        .into_iter()
 | 
			
		||||
        .iter()
 | 
			
		||||
        .cloned()
 | 
			
		||||
        .reduce(|acc, current| {
 | 
			
		||||
            let first = obkv::KvReader::new(&acc);
 | 
			
		||||
 
 | 
			
		||||
@@ -106,7 +106,7 @@ where
 | 
			
		||||
    ) -> Result<IndexDocuments<'t, 'u, 'i, 'a, F>> {
 | 
			
		||||
        let transform = Some(Transform::new(
 | 
			
		||||
            wtxn,
 | 
			
		||||
            &index,
 | 
			
		||||
            index,
 | 
			
		||||
            indexer_config,
 | 
			
		||||
            config.update_method,
 | 
			
		||||
            config.autogenerate_docids,
 | 
			
		||||
@@ -291,18 +291,12 @@ where
 | 
			
		||||
        // Run extraction pipeline in parallel.
 | 
			
		||||
        pool.install(|| {
 | 
			
		||||
            // split obkv file into several chunks
 | 
			
		||||
            let original_chunk_iter = grenad_obkv_into_chunks(
 | 
			
		||||
                original_documents,
 | 
			
		||||
                pool_params.clone(),
 | 
			
		||||
                documents_chunk_size,
 | 
			
		||||
            );
 | 
			
		||||
            let original_chunk_iter =
 | 
			
		||||
                grenad_obkv_into_chunks(original_documents, pool_params, documents_chunk_size);
 | 
			
		||||
 | 
			
		||||
            // split obkv file into several chunks
 | 
			
		||||
            let flattened_chunk_iter = grenad_obkv_into_chunks(
 | 
			
		||||
                flattened_documents,
 | 
			
		||||
                pool_params.clone(),
 | 
			
		||||
                documents_chunk_size,
 | 
			
		||||
            );
 | 
			
		||||
            let flattened_chunk_iter =
 | 
			
		||||
                grenad_obkv_into_chunks(flattened_documents, pool_params, documents_chunk_size);
 | 
			
		||||
 | 
			
		||||
            let result = original_chunk_iter.and_then(|original_chunk| {
 | 
			
		||||
                let flattened_chunk = flattened_chunk_iter?;
 | 
			
		||||
@@ -341,7 +335,7 @@ where
 | 
			
		||||
        }
 | 
			
		||||
 | 
			
		||||
        let index_documents_ids = self.index.documents_ids(self.wtxn)?;
 | 
			
		||||
        let index_is_empty = index_documents_ids.len() == 0;
 | 
			
		||||
        let index_is_empty = index_documents_ids.is_empty();
 | 
			
		||||
        let mut final_documents_ids = RoaringBitmap::new();
 | 
			
		||||
        let mut word_pair_proximity_docids = None;
 | 
			
		||||
        let mut word_position_docids = None;
 | 
			
		||||
@@ -378,7 +372,7 @@ where
 | 
			
		||||
            };
 | 
			
		||||
 | 
			
		||||
            let (docids, is_merged_database) =
 | 
			
		||||
                write_typed_chunk_into_index(typed_chunk, &self.index, self.wtxn, index_is_empty)?;
 | 
			
		||||
                write_typed_chunk_into_index(typed_chunk, self.index, self.wtxn, index_is_empty)?;
 | 
			
		||||
            if !docids.is_empty() {
 | 
			
		||||
                final_documents_ids |= docids;
 | 
			
		||||
                let documents_seen_count = final_documents_ids.len();
 | 
			
		||||
@@ -475,7 +469,7 @@ where
 | 
			
		||||
        );
 | 
			
		||||
        let common_prefix_fst_words: Vec<_> = common_prefix_fst_words
 | 
			
		||||
            .as_slice()
 | 
			
		||||
            .linear_group_by_key(|x| x.chars().nth(0).unwrap())
 | 
			
		||||
            .linear_group_by_key(|x| x.chars().next().unwrap())
 | 
			
		||||
            .collect();
 | 
			
		||||
 | 
			
		||||
        // We retrieve the newly added words between the previous and new prefix word fst.
 | 
			
		||||
@@ -498,9 +492,9 @@ where
 | 
			
		||||
            execute_word_prefix_docids(
 | 
			
		||||
                self.wtxn,
 | 
			
		||||
                word_docids,
 | 
			
		||||
                self.index.word_docids.clone(),
 | 
			
		||||
                self.index.word_prefix_docids.clone(),
 | 
			
		||||
                &self.indexer_config,
 | 
			
		||||
                self.index.word_docids,
 | 
			
		||||
                self.index.word_prefix_docids,
 | 
			
		||||
                self.indexer_config,
 | 
			
		||||
                &new_prefix_fst_words,
 | 
			
		||||
                &common_prefix_fst_words,
 | 
			
		||||
                &del_prefix_fst_words,
 | 
			
		||||
@@ -511,9 +505,9 @@ where
 | 
			
		||||
            execute_word_prefix_docids(
 | 
			
		||||
                self.wtxn,
 | 
			
		||||
                exact_word_docids,
 | 
			
		||||
                self.index.exact_word_docids.clone(),
 | 
			
		||||
                self.index.exact_word_prefix_docids.clone(),
 | 
			
		||||
                &self.indexer_config,
 | 
			
		||||
                self.index.exact_word_docids,
 | 
			
		||||
                self.index.exact_word_prefix_docids,
 | 
			
		||||
                self.indexer_config,
 | 
			
		||||
                &new_prefix_fst_words,
 | 
			
		||||
                &common_prefix_fst_words,
 | 
			
		||||
                &del_prefix_fst_words,
 | 
			
		||||
@@ -596,12 +590,7 @@ fn execute_word_prefix_docids(
 | 
			
		||||
    builder.chunk_compression_level = indexer_config.chunk_compression_level;
 | 
			
		||||
    builder.max_nb_chunks = indexer_config.max_nb_chunks;
 | 
			
		||||
    builder.max_memory = indexer_config.max_memory;
 | 
			
		||||
    builder.execute(
 | 
			
		||||
        cursor,
 | 
			
		||||
        &new_prefix_fst_words,
 | 
			
		||||
        &common_prefix_fst_words,
 | 
			
		||||
        &del_prefix_fst_words,
 | 
			
		||||
    )?;
 | 
			
		||||
    builder.execute(cursor, new_prefix_fst_words, common_prefix_fst_words, del_prefix_fst_words)?;
 | 
			
		||||
    Ok(())
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -72,10 +72,10 @@ fn create_fields_mapping(
 | 
			
		||||
        // we sort by id here to ensure a deterministic mapping of the fields, that preserves
 | 
			
		||||
        // the original ordering.
 | 
			
		||||
        .sorted_by_key(|(&id, _)| id)
 | 
			
		||||
        .map(|(field, name)| match index_field_map.id(&name) {
 | 
			
		||||
        .map(|(field, name)| match index_field_map.id(name) {
 | 
			
		||||
            Some(id) => Ok((*field, id)),
 | 
			
		||||
            None => index_field_map
 | 
			
		||||
                .insert(&name)
 | 
			
		||||
                .insert(name)
 | 
			
		||||
                .ok_or(Error::UserError(UserError::AttributeLimitReached))
 | 
			
		||||
                .map(|id| (*field, id)),
 | 
			
		||||
        })
 | 
			
		||||
@@ -192,7 +192,7 @@ impl<'a, 'i> Transform<'a, 'i> {
 | 
			
		||||
            // 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));
 | 
			
		||||
            field_buffer_cache.sort_unstable_by(|(f1, _), (f2, _)| f1.cmp(f2));
 | 
			
		||||
 | 
			
		||||
            // Build the new obkv document.
 | 
			
		||||
            let mut writer = obkv::KvWriter::new(&mut obkv_buffer);
 | 
			
		||||
@@ -202,24 +202,23 @@ impl<'a, 'i> Transform<'a, 'i> {
 | 
			
		||||
 | 
			
		||||
            let mut original_docid = None;
 | 
			
		||||
 | 
			
		||||
            let docid =
 | 
			
		||||
                match self.new_external_documents_ids_builder.entry(external_id.clone().into()) {
 | 
			
		||||
                    Entry::Occupied(entry) => *entry.get() as u32,
 | 
			
		||||
                    Entry::Vacant(entry) => {
 | 
			
		||||
                        // If the document was already in the db we mark it as a replaced document.
 | 
			
		||||
                        // It'll be deleted later. We keep its original docid to insert it in the grenad.
 | 
			
		||||
                        if let Some(docid) = external_documents_ids.get(entry.key()) {
 | 
			
		||||
                            self.replaced_documents_ids.insert(docid);
 | 
			
		||||
                            original_docid = Some(docid);
 | 
			
		||||
                        }
 | 
			
		||||
                        let docid = self
 | 
			
		||||
                            .available_documents_ids
 | 
			
		||||
                            .next()
 | 
			
		||||
                            .ok_or(UserError::DocumentLimitReached)?;
 | 
			
		||||
                        entry.insert(docid as u64);
 | 
			
		||||
                        docid
 | 
			
		||||
            let docid = match self.new_external_documents_ids_builder.entry((*external_id).into()) {
 | 
			
		||||
                Entry::Occupied(entry) => *entry.get() as u32,
 | 
			
		||||
                Entry::Vacant(entry) => {
 | 
			
		||||
                    // If the document was already in the db we mark it as a replaced document.
 | 
			
		||||
                    // It'll be deleted later. We keep its original docid to insert it in the grenad.
 | 
			
		||||
                    if let Some(docid) = external_documents_ids.get(entry.key()) {
 | 
			
		||||
                        self.replaced_documents_ids.insert(docid);
 | 
			
		||||
                        original_docid = Some(docid);
 | 
			
		||||
                    }
 | 
			
		||||
                };
 | 
			
		||||
                    let docid = 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 {
 | 
			
		||||
@@ -239,12 +238,12 @@ impl<'a, 'i> Transform<'a, 'i> {
 | 
			
		||||
                    // 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);
 | 
			
		||||
                    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.
 | 
			
		||||
                    self.original_sorter.insert(&docid.to_be_bytes(), base_obkv)?;
 | 
			
		||||
                    match self.flatten_from_fields_ids_map(KvReader::new(&base_obkv))? {
 | 
			
		||||
                    match self.flatten_from_fields_ids_map(KvReader::new(base_obkv))? {
 | 
			
		||||
                        Some(buffer) => {
 | 
			
		||||
                            self.flattened_sorter.insert(docid.to_be_bytes(), &buffer)?
 | 
			
		||||
                        }
 | 
			
		||||
@@ -453,7 +452,7 @@ impl<'a, 'i> Transform<'a, 'i> {
 | 
			
		||||
    {
 | 
			
		||||
        let primary_key = self
 | 
			
		||||
            .index
 | 
			
		||||
            .primary_key(&wtxn)?
 | 
			
		||||
            .primary_key(wtxn)?
 | 
			
		||||
            .ok_or(Error::UserError(UserError::MissingPrimaryKey))?
 | 
			
		||||
            .to_string();
 | 
			
		||||
 | 
			
		||||
@@ -520,7 +519,7 @@ impl<'a, 'i> Transform<'a, 'i> {
 | 
			
		||||
            self.new_external_documents_ids_builder.into_iter().collect();
 | 
			
		||||
 | 
			
		||||
        new_external_documents_ids_builder
 | 
			
		||||
            .sort_unstable_by(|(left, _), (right, _)| left.cmp(&right));
 | 
			
		||||
            .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)
 | 
			
		||||
@@ -614,7 +613,7 @@ impl<'a, 'i> Transform<'a, 'i> {
 | 
			
		||||
            let mut flattened: Vec<_> = flattened.into_iter().collect();
 | 
			
		||||
            // we reorder the field to get all the known field first
 | 
			
		||||
            flattened.sort_unstable_by_key(|(key, _)| {
 | 
			
		||||
                new_fields_ids_map.id(&key).unwrap_or(FieldId::MAX)
 | 
			
		||||
                new_fields_ids_map.id(key).unwrap_or(FieldId::MAX)
 | 
			
		||||
            });
 | 
			
		||||
 | 
			
		||||
            for (key, value) in flattened {
 | 
			
		||||
 
 | 
			
		||||
@@ -175,7 +175,7 @@ pub(crate) fn write_typed_chunk_into_index(
 | 
			
		||||
            let mut cursor = fid_docid_facet_number.into_cursor()?;
 | 
			
		||||
            while let Some((key, value)) = cursor.move_on_next()? {
 | 
			
		||||
                if valid_lmdb_key(key) {
 | 
			
		||||
                    index_fid_docid_facet_numbers.put(wtxn, key, &value)?;
 | 
			
		||||
                    index_fid_docid_facet_numbers.put(wtxn, key, value)?;
 | 
			
		||||
                }
 | 
			
		||||
            }
 | 
			
		||||
        }
 | 
			
		||||
@@ -185,7 +185,7 @@ pub(crate) fn write_typed_chunk_into_index(
 | 
			
		||||
            let mut cursor = fid_docid_facet_string.into_cursor()?;
 | 
			
		||||
            while let Some((key, value)) = cursor.move_on_next()? {
 | 
			
		||||
                if valid_lmdb_key(key) {
 | 
			
		||||
                    index_fid_docid_facet_strings.put(wtxn, key, &value)?;
 | 
			
		||||
                    index_fid_docid_facet_strings.put(wtxn, key, value)?;
 | 
			
		||||
                }
 | 
			
		||||
            }
 | 
			
		||||
        }
 | 
			
		||||
 
 | 
			
		||||
		Reference in New Issue
	
	Block a user