mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-03 20:07:09 +02:00
Move embedder stats out of progress
This commit is contained in:
parent
4cadc8113b
commit
4925b30196
30 changed files with 255 additions and 69 deletions
|
@ -20,7 +20,6 @@ pub trait Step: 'static + Send + Sync {
|
|||
#[derive(Clone, Default)]
|
||||
pub struct Progress {
|
||||
steps: Arc<RwLock<InnerProgress>>,
|
||||
pub embedder_stats: Arc<EmbedderStats>,
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
|
@ -29,6 +28,17 @@ pub struct EmbedderStats {
|
|||
pub total_requests: AtomicUsize
|
||||
}
|
||||
|
||||
impl std::fmt::Debug for EmbedderStats {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
let (error, count) = self.errors.read().unwrap().clone();
|
||||
f.debug_struct("EmbedderStats")
|
||||
.field("errors", &error)
|
||||
.field("total_requests", &self.total_requests.load(Ordering::Relaxed))
|
||||
.field("error_count", &count)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
struct InnerProgress {
|
||||
/// The hierarchy of steps.
|
||||
|
@ -72,19 +82,7 @@ impl Progress {
|
|||
});
|
||||
}
|
||||
|
||||
let embedder_view = {
|
||||
let (last_error, error_count) = match self.embedder_stats.errors.read() {
|
||||
Ok(guard) => (guard.0.clone(), guard.1),
|
||||
Err(_) => (None, 0),
|
||||
};
|
||||
EmbedderStatsView {
|
||||
last_error,
|
||||
request_count: self.embedder_stats.total_requests.load(Ordering::Relaxed) as u32,
|
||||
error_count,
|
||||
}
|
||||
};
|
||||
|
||||
ProgressView { steps: step_view, percentage: percentage * 100.0, embedder: embedder_view }
|
||||
ProgressView { steps: step_view, percentage: percentage * 100.0 }
|
||||
}
|
||||
|
||||
pub fn accumulated_durations(&self) -> IndexMap<String, String> {
|
||||
|
@ -228,7 +226,6 @@ make_enum_progress! {
|
|||
pub struct ProgressView {
|
||||
pub steps: Vec<ProgressStepView>,
|
||||
pub percentage: f32,
|
||||
pub embedder: EmbedderStatsView,
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize, Clone, ToSchema)]
|
||||
|
@ -240,16 +237,6 @@ pub struct ProgressStepView {
|
|||
pub total: u32,
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize, Clone, ToSchema)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
#[schema(rename_all = "camelCase")]
|
||||
pub struct EmbedderStatsView {
|
||||
#[serde(skip_serializing_if = "Option::is_none")]
|
||||
pub last_error: Option<String>,
|
||||
pub request_count: u32,
|
||||
pub error_count: u32,
|
||||
}
|
||||
|
||||
/// Used when the name can change but it's still the same step.
|
||||
/// To avoid conflicts on the `TypeId`, create a unique type every time you use this step:
|
||||
/// ```text
|
||||
|
|
|
@ -44,7 +44,7 @@ pub fn setup_search_index_with_criteria(criteria: &[Criterion]) -> Index {
|
|||
S("america") => vec![S("the united states")],
|
||||
});
|
||||
builder.set_searchable_fields(vec![S("title"), S("description")]);
|
||||
builder.execute(|_| (), || false, None).unwrap();
|
||||
builder.execute(|_| (), || false, Default::default()).unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
||||
// index documents
|
||||
|
@ -95,6 +95,7 @@ pub fn setup_search_index_with_criteria(criteria: &[Criterion]) -> Index {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
|
|
|
@ -103,6 +103,7 @@ impl TempIndex {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
})
|
||||
.unwrap()?;
|
||||
|
@ -134,7 +135,7 @@ impl TempIndex {
|
|||
) -> Result<(), crate::error::Error> {
|
||||
let mut builder = update::Settings::new(wtxn, &self.inner, &self.indexer_config);
|
||||
update(&mut builder);
|
||||
builder.execute(drop, || false, None)?;
|
||||
builder.execute(drop, || false, Default::default())?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
|
@ -185,6 +186,7 @@ impl TempIndex {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
})
|
||||
.unwrap()?;
|
||||
|
@ -259,6 +261,7 @@ fn aborting_indexation() {
|
|||
embedders,
|
||||
&|| should_abort.load(Relaxed),
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
})
|
||||
.unwrap()
|
||||
|
|
|
@ -687,6 +687,8 @@ pub fn extract_embeddings<R: io::Read + io::Seek>(
|
|||
unused_vectors_distribution: &UnusedVectorsDistribution,
|
||||
request_threads: &ThreadPoolNoAbort,
|
||||
) -> Result<grenad::Reader<BufReader<File>>> {
|
||||
println!("Extract embedder stats {}:", embedder_stats.is_some());
|
||||
|
||||
let n_chunks = embedder.chunk_count_hint(); // chunk level parallelism
|
||||
let n_vectors_per_chunk = embedder.prompt_count_in_chunk_hint(); // number of vectors in a single chunk
|
||||
|
||||
|
|
|
@ -50,7 +50,7 @@ pub(crate) fn data_from_obkv_documents(
|
|||
settings_diff: Arc<InnerIndexSettingsDiff>,
|
||||
max_positions_per_attributes: Option<u32>,
|
||||
possible_embedding_mistakes: Arc<PossibleEmbeddingMistakes>,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
embedder_stats: Arc<EmbedderStats>,
|
||||
) -> Result<()> {
|
||||
let (original_pipeline_result, flattened_pipeline_result): (Result<_>, Result<_>) = rayon::join(
|
||||
|| {
|
||||
|
@ -234,7 +234,7 @@ fn send_original_documents_data(
|
|||
embedders_configs: Arc<Vec<IndexEmbeddingConfig>>,
|
||||
settings_diff: Arc<InnerIndexSettingsDiff>,
|
||||
possible_embedding_mistakes: Arc<PossibleEmbeddingMistakes>,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
embedder_stats: Arc<EmbedderStats>,
|
||||
) -> Result<()> {
|
||||
let original_documents_chunk =
|
||||
original_documents_chunk.and_then(|c| unsafe { as_cloneable_grenad(&c) })?;
|
||||
|
@ -274,7 +274,7 @@ fn send_original_documents_data(
|
|||
embedder.clone(),
|
||||
&embedder_name,
|
||||
&possible_embedding_mistakes,
|
||||
embedder_stats.clone(),
|
||||
Some(embedder_stats.clone()),
|
||||
&unused_vectors_distribution,
|
||||
request_threads(),
|
||||
) {
|
||||
|
|
|
@ -81,7 +81,7 @@ pub struct IndexDocuments<'t, 'i, 'a, FP, FA> {
|
|||
added_documents: u64,
|
||||
deleted_documents: u64,
|
||||
embedders: EmbeddingConfigs,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
embedder_stats: Arc<EmbedderStats>,
|
||||
}
|
||||
|
||||
#[derive(Default, Debug, Clone)]
|
||||
|
@ -104,7 +104,7 @@ where
|
|||
config: IndexDocumentsConfig,
|
||||
progress: FP,
|
||||
should_abort: FA,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
embedder_stats: Arc<EmbedderStats>,
|
||||
) -> Result<IndexDocuments<'t, 'i, 'a, FP, FA>> {
|
||||
let transform = Some(Transform::new(
|
||||
wtxn,
|
||||
|
@ -2030,6 +2030,7 @@ mod tests {
|
|||
EmbeddingConfigs::default(),
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2117,6 +2118,7 @@ mod tests {
|
|||
EmbeddingConfigs::default(),
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2302,6 +2304,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2364,6 +2367,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2417,6 +2421,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2469,6 +2474,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2523,6 +2529,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2582,6 +2589,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2634,6 +2642,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2686,6 +2695,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2884,6 +2894,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2943,6 +2954,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
@ -2999,6 +3011,7 @@ mod tests {
|
|||
embedders,
|
||||
&|| false,
|
||||
&Progress::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.unwrap();
|
||||
wtxn.commit().unwrap();
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
use std::cell::RefCell;
|
||||
use std::f32::consts::E;
|
||||
use std::{cell::RefCell, sync::Arc};
|
||||
|
||||
use bumpalo::collections::Vec as BVec;
|
||||
use bumpalo::Bump;
|
||||
|
@ -6,6 +7,7 @@ use hashbrown::{DefaultHashBuilder, HashMap};
|
|||
|
||||
use super::cache::DelAddRoaringBitmap;
|
||||
use crate::error::FaultSource;
|
||||
use crate::progress::EmbedderStats;
|
||||
use crate::prompt::Prompt;
|
||||
use crate::update::new::channel::EmbeddingSender;
|
||||
use crate::update::new::indexer::document_changes::{DocumentChangeContext, Extractor};
|
||||
|
@ -22,6 +24,7 @@ pub struct EmbeddingExtractor<'a, 'b> {
|
|||
embedders: &'a EmbeddingConfigs,
|
||||
sender: EmbeddingSender<'a, 'b>,
|
||||
possible_embedding_mistakes: PossibleEmbeddingMistakes,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
threads: &'a ThreadPoolNoAbort,
|
||||
}
|
||||
|
||||
|
@ -30,10 +33,11 @@ impl<'a, 'b> EmbeddingExtractor<'a, 'b> {
|
|||
embedders: &'a EmbeddingConfigs,
|
||||
sender: EmbeddingSender<'a, 'b>,
|
||||
field_distribution: &'a FieldDistribution,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
threads: &'a ThreadPoolNoAbort,
|
||||
) -> Self {
|
||||
let possible_embedding_mistakes = PossibleEmbeddingMistakes::new(field_distribution);
|
||||
Self { embedders, sender, threads, possible_embedding_mistakes }
|
||||
Self { embedders, sender, threads, possible_embedding_mistakes, embedder_stats }
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -75,6 +79,7 @@ impl<'extractor> Extractor<'extractor> for EmbeddingExtractor<'_, '_> {
|
|||
prompt,
|
||||
context.data,
|
||||
&self.possible_embedding_mistakes,
|
||||
self.embedder_stats.clone(),
|
||||
self.threads,
|
||||
self.sender,
|
||||
&context.doc_alloc,
|
||||
|
@ -307,6 +312,7 @@ struct Chunks<'a, 'b, 'extractor> {
|
|||
dimensions: usize,
|
||||
prompt: &'a Prompt,
|
||||
possible_embedding_mistakes: &'a PossibleEmbeddingMistakes,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
user_provided: &'a RefCell<EmbeddingExtractorData<'extractor>>,
|
||||
threads: &'a ThreadPoolNoAbort,
|
||||
sender: EmbeddingSender<'a, 'b>,
|
||||
|
@ -322,6 +328,7 @@ impl<'a, 'b, 'extractor> Chunks<'a, 'b, 'extractor> {
|
|||
prompt: &'a Prompt,
|
||||
user_provided: &'a RefCell<EmbeddingExtractorData<'extractor>>,
|
||||
possible_embedding_mistakes: &'a PossibleEmbeddingMistakes,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
threads: &'a ThreadPoolNoAbort,
|
||||
sender: EmbeddingSender<'a, 'b>,
|
||||
doc_alloc: &'a Bump,
|
||||
|
@ -336,6 +343,7 @@ impl<'a, 'b, 'extractor> Chunks<'a, 'b, 'extractor> {
|
|||
embedder,
|
||||
prompt,
|
||||
possible_embedding_mistakes,
|
||||
embedder_stats,
|
||||
threads,
|
||||
sender,
|
||||
embedder_id,
|
||||
|
@ -371,6 +379,7 @@ impl<'a, 'b, 'extractor> Chunks<'a, 'b, 'extractor> {
|
|||
self.embedder_id,
|
||||
self.embedder_name,
|
||||
self.possible_embedding_mistakes,
|
||||
self.embedder_stats.clone(),
|
||||
unused_vectors_distribution,
|
||||
self.threads,
|
||||
self.sender,
|
||||
|
@ -389,6 +398,7 @@ impl<'a, 'b, 'extractor> Chunks<'a, 'b, 'extractor> {
|
|||
self.embedder_id,
|
||||
self.embedder_name,
|
||||
self.possible_embedding_mistakes,
|
||||
self.embedder_stats.clone(),
|
||||
unused_vectors_distribution,
|
||||
self.threads,
|
||||
self.sender,
|
||||
|
@ -407,6 +417,7 @@ impl<'a, 'b, 'extractor> Chunks<'a, 'b, 'extractor> {
|
|||
embedder_id: u8,
|
||||
embedder_name: &str,
|
||||
possible_embedding_mistakes: &PossibleEmbeddingMistakes,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
unused_vectors_distribution: &UnusedVectorsDistributionBump,
|
||||
threads: &ThreadPoolNoAbort,
|
||||
sender: EmbeddingSender<'a, 'b>,
|
||||
|
@ -450,7 +461,7 @@ impl<'a, 'b, 'extractor> Chunks<'a, 'b, 'extractor> {
|
|||
return Err(crate::Error::UserError(crate::UserError::DocumentEmbeddingError(msg)));
|
||||
}
|
||||
|
||||
let res = match embedder.embed_index_ref(texts.as_slice(), threads, None) {
|
||||
let res = match embedder.embed_index_ref(texts.as_slice(), threads, embedder_stats) {
|
||||
Ok(embeddings) => {
|
||||
for (docid, embedding) in ids.into_iter().zip(embeddings) {
|
||||
sender.set_vector(*docid, embedder_id, embedding).unwrap();
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
use std::collections::BTreeMap;
|
||||
use std::sync::atomic::AtomicBool;
|
||||
use std::sync::OnceLock;
|
||||
use std::sync::Arc;
|
||||
|
||||
use bumpalo::Bump;
|
||||
use roaring::RoaringBitmap;
|
||||
|
@ -13,6 +14,7 @@ use super::super::thread_local::{FullySend, ThreadLocal};
|
|||
use super::super::FacetFieldIdsDelta;
|
||||
use super::document_changes::{extract, DocumentChanges, IndexingContext};
|
||||
use crate::index::IndexEmbeddingConfig;
|
||||
use crate::progress::EmbedderStats;
|
||||
use crate::progress::MergingWordCache;
|
||||
use crate::proximity::ProximityPrecision;
|
||||
use crate::update::new::extract::EmbeddingExtractor;
|
||||
|
@ -34,6 +36,7 @@ pub(super) fn extract_all<'pl, 'extractor, DC, MSP>(
|
|||
mut index_embeddings: Vec<IndexEmbeddingConfig>,
|
||||
document_ids: &mut RoaringBitmap,
|
||||
modified_docids: &mut RoaringBitmap,
|
||||
embedder_stats: Arc<EmbedderStats>,
|
||||
) -> Result<(FacetFieldIdsDelta, Vec<IndexEmbeddingConfig>)>
|
||||
where
|
||||
DC: DocumentChanges<'pl>,
|
||||
|
@ -245,6 +248,7 @@ where
|
|||
embedders,
|
||||
embedding_sender,
|
||||
field_distribution,
|
||||
Some(embedder_stats),
|
||||
request_threads(),
|
||||
);
|
||||
let mut datastore = ThreadLocal::with_capacity(rayon::current_num_threads());
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
use std::sync::atomic::AtomicBool;
|
||||
use std::sync::{Once, RwLock};
|
||||
use std::thread::{self, Builder};
|
||||
use std::sync::Arc;
|
||||
|
||||
use big_s::S;
|
||||
use document_changes::{DocumentChanges, IndexingContext};
|
||||
|
@ -19,7 +20,7 @@ use super::steps::IndexingStep;
|
|||
use super::thread_local::ThreadLocal;
|
||||
use crate::documents::PrimaryKey;
|
||||
use crate::fields_ids_map::metadata::{FieldIdMapWithMetadata, MetadataBuilder};
|
||||
use crate::progress::Progress;
|
||||
use crate::progress::{EmbedderStats, Progress};
|
||||
use crate::update::GrenadParameters;
|
||||
use crate::vector::{ArroyWrapper, EmbeddingConfigs};
|
||||
use crate::{FieldsIdsMap, GlobalFieldsIdsMap, Index, InternalError, Result, ThreadPoolNoAbort};
|
||||
|
@ -55,6 +56,7 @@ pub fn index<'pl, 'indexer, 'index, DC, MSP>(
|
|||
embedders: EmbeddingConfigs,
|
||||
must_stop_processing: &'indexer MSP,
|
||||
progress: &'indexer Progress,
|
||||
embedder_stats: Arc<EmbedderStats>,
|
||||
) -> Result<ChannelCongestion>
|
||||
where
|
||||
DC: DocumentChanges<'pl>,
|
||||
|
@ -158,6 +160,7 @@ where
|
|||
index_embeddings,
|
||||
document_ids,
|
||||
modified_docids,
|
||||
embedder_stats,
|
||||
)
|
||||
})
|
||||
.unwrap()
|
||||
|
|
|
@ -475,7 +475,7 @@ impl<'a, 't, 'i> Settings<'a, 't, 'i> {
|
|||
progress_callback: &FP,
|
||||
should_abort: &FA,
|
||||
settings_diff: InnerIndexSettingsDiff,
|
||||
embedder_stats: Option<Arc<EmbedderStats>>,
|
||||
embedder_stats: Arc<EmbedderStats>,
|
||||
) -> Result<()>
|
||||
where
|
||||
FP: Fn(UpdateIndexingStep) + Sync,
|
||||
|
@ -1358,7 +1358,7 @@ impl<'a, 't, 'i> Settings<'a, 't, 'i> {
|
|||
}
|
||||
}
|
||||
|
||||
pub fn execute<FP, FA>(mut self, progress_callback: FP, should_abort: FA, embedder_stats: Option<Arc<EmbedderStats>>) -> Result<()>
|
||||
pub fn execute<FP, FA>(mut self, progress_callback: FP, should_abort: FA, embedder_stats: Arc<EmbedderStats>) -> Result<()>
|
||||
where
|
||||
FP: Fn(UpdateIndexingStep) + Sync,
|
||||
FA: Fn() -> bool + Sync,
|
||||
|
|
|
@ -295,6 +295,10 @@ fn embed<S>(
|
|||
where
|
||||
S: Serialize,
|
||||
{
|
||||
use std::backtrace::Backtrace;
|
||||
|
||||
println!("Embedder stats? {}", embedder_stats.is_some());
|
||||
|
||||
let request = data.client.post(&data.url);
|
||||
let request = if let Some(bearer) = &data.bearer {
|
||||
request.set("Authorization", bearer)
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue