mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-03 11:57:07 +02:00
Merge #5351
5351: Bring back v1.13.0 changes into main r=irevoire a=Kerollmops This PR brings back the changes made in v1.13 into the main branch. Co-authored-by: ManyTheFish <many@meilisearch.com> Co-authored-by: Kerollmops <clement@meilisearch.com> Co-authored-by: Louis Dureuil <louis@meilisearch.com> Co-authored-by: Clémentine <clementine@meilisearch.com> Co-authored-by: meili-bors[bot] <89034592+meili-bors[bot]@users.noreply.github.com> Co-authored-by: Tamo <tamo@meilisearch.com> Co-authored-by: Clément Renault <clement@meilisearch.com>
This commit is contained in:
commit
0f1aeb8eaa
101 changed files with 8351 additions and 1518 deletions
|
@ -1,5 +1,5 @@
|
|||
<p align="center">
|
||||
<img alt="the milli logo" src="../assets/milli-logo.svg">
|
||||
<img alt="the milli logo" src="../../assets/milli-logo.svg">
|
||||
</p>
|
||||
|
||||
<p align="center">a concurrent indexer combined with fast and relevant search algorithms</p>
|
||||
|
|
|
@ -22,7 +22,7 @@ use crate::heed_codec::version::VersionCodec;
|
|||
use crate::heed_codec::{BEU16StrCodec, FstSetCodec, StrBEU16Codec, StrRefCodec};
|
||||
use crate::order_by_map::OrderByMap;
|
||||
use crate::proximity::ProximityPrecision;
|
||||
use crate::vector::{ArroyWrapper, Embedding, EmbeddingConfig};
|
||||
use crate::vector::{ArroyStats, ArroyWrapper, Embedding, EmbeddingConfig};
|
||||
use crate::{
|
||||
default_criteria, CboRoaringBitmapCodec, Criterion, DocumentId, ExternalDocumentsIds,
|
||||
FacetDistribution, FieldDistribution, FieldId, FieldIdMapMissingEntry, FieldIdWordCountCodec,
|
||||
|
@ -1731,6 +1731,18 @@ impl Index {
|
|||
let compute_prefixes = self.prefix_search(rtxn)?.unwrap_or_default();
|
||||
Ok(PrefixSettings { compute_prefixes, max_prefix_length: 4, prefix_count_threshold: 100 })
|
||||
}
|
||||
|
||||
pub fn arroy_stats(&self, rtxn: &RoTxn<'_>) -> Result<ArroyStats> {
|
||||
let mut stats = ArroyStats::default();
|
||||
let embedding_configs = self.embedding_configs(rtxn)?;
|
||||
for config in embedding_configs {
|
||||
let embedder_id = self.embedder_category_id.get(rtxn, &config.name)?.unwrap();
|
||||
let reader =
|
||||
ArroyWrapper::new(self.vector_arroy, embedder_id, config.config.quantized());
|
||||
reader.aggregate_stats(rtxn, &mut stats)?;
|
||||
}
|
||||
Ok(stats)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Deserialize, Serialize)]
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
use std::cmp::Ordering;
|
||||
|
||||
use itertools::Itertools;
|
||||
use serde::Serialize;
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use crate::distance_between_two_points;
|
||||
|
||||
|
@ -36,6 +36,15 @@ enum RankOrValue<'a> {
|
|||
Score(f64),
|
||||
}
|
||||
|
||||
#[derive(Clone, Serialize, Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
pub enum WeightedScoreValue {
|
||||
WeightedScore(f64),
|
||||
Sort { asc: bool, value: serde_json::Value },
|
||||
GeoSort { asc: bool, distance: Option<f64> },
|
||||
VectorSort(f64),
|
||||
}
|
||||
|
||||
impl ScoreDetails {
|
||||
pub fn local_score(&self) -> Option<f64> {
|
||||
self.rank().map(Rank::local_score)
|
||||
|
@ -87,6 +96,30 @@ impl ScoreDetails {
|
|||
})
|
||||
}
|
||||
|
||||
pub fn weighted_score_values<'a>(
|
||||
details: impl Iterator<Item = &'a Self> + 'a,
|
||||
weight: f64,
|
||||
) -> impl Iterator<Item = WeightedScoreValue> + 'a {
|
||||
details
|
||||
.map(ScoreDetails::rank_or_value)
|
||||
.coalesce(|left, right| match (left, right) {
|
||||
(RankOrValue::Rank(left), RankOrValue::Rank(right)) => {
|
||||
Ok(RankOrValue::Rank(Rank::merge(left, right)))
|
||||
}
|
||||
(left, right) => Err((left, right)),
|
||||
})
|
||||
.map(move |rank_or_value| match rank_or_value {
|
||||
RankOrValue::Rank(r) => WeightedScoreValue::WeightedScore(r.local_score() * weight),
|
||||
RankOrValue::Sort(s) => {
|
||||
WeightedScoreValue::Sort { asc: s.ascending, value: s.value.clone() }
|
||||
}
|
||||
RankOrValue::GeoSort(g) => {
|
||||
WeightedScoreValue::GeoSort { asc: g.ascending, distance: g.distance() }
|
||||
}
|
||||
RankOrValue::Score(s) => WeightedScoreValue::VectorSort(s * weight),
|
||||
})
|
||||
}
|
||||
|
||||
fn rank_or_value(&self) -> RankOrValue<'_> {
|
||||
match self {
|
||||
ScoreDetails::Words(w) => RankOrValue::Rank(w.rank()),
|
||||
|
@ -423,34 +456,58 @@ pub struct Sort {
|
|||
pub value: serde_json::Value,
|
||||
}
|
||||
|
||||
pub fn compare_sort_values(
|
||||
ascending: bool,
|
||||
left: &serde_json::Value,
|
||||
right: &serde_json::Value,
|
||||
) -> Ordering {
|
||||
use serde_json::Value::*;
|
||||
match (left, right) {
|
||||
(Null, Null) => Ordering::Equal,
|
||||
(Null, _) => Ordering::Less,
|
||||
(_, Null) => Ordering::Greater,
|
||||
// numbers are always before strings
|
||||
(Number(_), String(_)) => Ordering::Greater,
|
||||
(String(_), Number(_)) => Ordering::Less,
|
||||
(Number(left), Number(right)) => {
|
||||
// FIXME: unwrap permitted here?
|
||||
let order = left
|
||||
.as_f64()
|
||||
.unwrap()
|
||||
.partial_cmp(&right.as_f64().unwrap())
|
||||
.unwrap_or(Ordering::Equal);
|
||||
// 12 < 42, and when ascending, we want to see 12 first, so the smallest.
|
||||
// Hence, when ascending, smaller is better
|
||||
if ascending {
|
||||
order.reverse()
|
||||
} else {
|
||||
order
|
||||
}
|
||||
}
|
||||
(String(left), String(right)) => {
|
||||
let order = left.cmp(right);
|
||||
// Taking e.g. "a" and "z"
|
||||
// "a" < "z", and when ascending, we want to see "a" first, so the smallest.
|
||||
// Hence, when ascending, smaller is better
|
||||
if ascending {
|
||||
order.reverse()
|
||||
} else {
|
||||
order
|
||||
}
|
||||
}
|
||||
(left, right) => {
|
||||
tracing::warn!(%left, %right, "sort values that are neither numbers, strings or null, handling as equal");
|
||||
Ordering::Equal
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl PartialOrd for Sort {
|
||||
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
|
||||
if self.ascending != other.ascending {
|
||||
return None;
|
||||
}
|
||||
match (&self.value, &other.value) {
|
||||
(serde_json::Value::Null, serde_json::Value::Null) => Some(Ordering::Equal),
|
||||
(serde_json::Value::Null, _) => Some(Ordering::Less),
|
||||
(_, serde_json::Value::Null) => Some(Ordering::Greater),
|
||||
// numbers are always before strings
|
||||
(serde_json::Value::Number(_), serde_json::Value::String(_)) => Some(Ordering::Greater),
|
||||
(serde_json::Value::String(_), serde_json::Value::Number(_)) => Some(Ordering::Less),
|
||||
(serde_json::Value::Number(left), serde_json::Value::Number(right)) => {
|
||||
// FIXME: unwrap permitted here?
|
||||
let order = left.as_f64().unwrap().partial_cmp(&right.as_f64().unwrap())?;
|
||||
// 12 < 42, and when ascending, we want to see 12 first, so the smallest.
|
||||
// Hence, when ascending, smaller is better
|
||||
Some(if self.ascending { order.reverse() } else { order })
|
||||
}
|
||||
(serde_json::Value::String(left), serde_json::Value::String(right)) => {
|
||||
let order = left.cmp(right);
|
||||
// Taking e.g. "a" and "z"
|
||||
// "a" < "z", and when ascending, we want to see "a" first, so the smallest.
|
||||
// Hence, when ascending, smaller is better
|
||||
Some(if self.ascending { order.reverse() } else { order })
|
||||
}
|
||||
_ => None,
|
||||
}
|
||||
Some(compare_sort_values(self.ascending, &self.value, &other.value))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ use either::Either;
|
|||
pub use matching_words::MatchingWords;
|
||||
use matching_words::{MatchType, PartialMatch};
|
||||
use r#match::{Match, MatchPosition};
|
||||
use serde::Serialize;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use simple_token_kind::SimpleTokenKind;
|
||||
use utoipa::ToSchema;
|
||||
|
||||
|
@ -101,11 +101,11 @@ impl FormatOptions {
|
|||
}
|
||||
}
|
||||
|
||||
#[derive(Serialize, Debug, Clone, PartialEq, Eq, ToSchema)]
|
||||
#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, ToSchema)]
|
||||
pub struct MatchBounds {
|
||||
pub start: usize,
|
||||
pub length: usize,
|
||||
#[serde(skip_serializing_if = "Option::is_none")]
|
||||
#[serde(skip_serializing_if = "Option::is_none", default)]
|
||||
pub indices: Option<Vec<usize>>,
|
||||
}
|
||||
|
||||
|
|
|
@ -563,7 +563,7 @@ fn resolve_sort_criteria<'ctx, Query: RankingRuleQueryTrait>(
|
|||
Ok(())
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "search::universe")]
|
||||
#[tracing::instrument(level = "debug", skip_all, target = "search::universe")]
|
||||
pub fn filtered_universe(
|
||||
index: &Index,
|
||||
txn: &RoTxn<'_>,
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering};
|
||||
use std::sync::Arc;
|
||||
|
||||
use rayon::{ThreadPool, ThreadPoolBuilder};
|
||||
|
@ -9,6 +9,8 @@ use thiserror::Error;
|
|||
#[derive(Debug)]
|
||||
pub struct ThreadPoolNoAbort {
|
||||
thread_pool: ThreadPool,
|
||||
/// The number of active operations.
|
||||
active_operations: AtomicUsize,
|
||||
/// Set to true if the thread pool catched a panic.
|
||||
pool_catched_panic: Arc<AtomicBool>,
|
||||
}
|
||||
|
@ -19,7 +21,9 @@ impl ThreadPoolNoAbort {
|
|||
OP: FnOnce() -> R + Send,
|
||||
R: Send,
|
||||
{
|
||||
self.active_operations.fetch_add(1, Ordering::Relaxed);
|
||||
let output = self.thread_pool.install(op);
|
||||
self.active_operations.fetch_sub(1, Ordering::Relaxed);
|
||||
// While reseting the pool panic catcher we return an error if we catched one.
|
||||
if self.pool_catched_panic.swap(false, Ordering::SeqCst) {
|
||||
Err(PanicCatched)
|
||||
|
@ -31,6 +35,11 @@ impl ThreadPoolNoAbort {
|
|||
pub fn current_num_threads(&self) -> usize {
|
||||
self.thread_pool.current_num_threads()
|
||||
}
|
||||
|
||||
/// The number of active operations.
|
||||
pub fn active_operations(&self) -> usize {
|
||||
self.active_operations.load(Ordering::Relaxed)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Error, Debug)]
|
||||
|
@ -64,6 +73,10 @@ impl ThreadPoolNoAbortBuilder {
|
|||
let catched_panic = pool_catched_panic.clone();
|
||||
move |_result| catched_panic.store(true, Ordering::SeqCst)
|
||||
});
|
||||
Ok(ThreadPoolNoAbort { thread_pool: self.0.build()?, pool_catched_panic })
|
||||
Ok(ThreadPoolNoAbort {
|
||||
thread_pool: self.0.build()?,
|
||||
active_operations: AtomicUsize::new(0),
|
||||
pool_catched_panic,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,6 +5,8 @@ use std::marker::PhantomData;
|
|||
use std::mem;
|
||||
use std::num::NonZeroU16;
|
||||
use std::ops::Range;
|
||||
use std::sync::atomic::{self, AtomicUsize};
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
|
||||
use bbqueue::framed::{FrameGrantR, FrameProducer};
|
||||
|
@ -71,12 +73,23 @@ pub fn extractor_writer_bbqueue(
|
|||
consumer
|
||||
});
|
||||
|
||||
let sent_messages_attempts = Arc::new(AtomicUsize::new(0));
|
||||
let blocking_sent_messages_attempts = Arc::new(AtomicUsize::new(0));
|
||||
|
||||
let (sender, receiver) = flume::bounded(channel_capacity);
|
||||
let sender = ExtractorBbqueueSender { sender, producers, max_grant };
|
||||
let sender = ExtractorBbqueueSender {
|
||||
sender,
|
||||
producers,
|
||||
max_grant,
|
||||
sent_messages_attempts: sent_messages_attempts.clone(),
|
||||
blocking_sent_messages_attempts: blocking_sent_messages_attempts.clone(),
|
||||
};
|
||||
let receiver = WriterBbqueueReceiver {
|
||||
receiver,
|
||||
look_at_consumer: (0..consumers.len()).cycle(),
|
||||
consumers,
|
||||
sent_messages_attempts,
|
||||
blocking_sent_messages_attempts,
|
||||
};
|
||||
(sender, receiver)
|
||||
}
|
||||
|
@ -92,6 +105,12 @@ pub struct ExtractorBbqueueSender<'a> {
|
|||
/// It will never be able to store more than that as the
|
||||
/// buffer cannot split data into two parts.
|
||||
max_grant: usize,
|
||||
/// The total number of attempts to send messages
|
||||
/// over the bbqueue channel.
|
||||
sent_messages_attempts: Arc<AtomicUsize>,
|
||||
/// The number of times an attempt to send a
|
||||
/// messages failed and we had to pause for a bit.
|
||||
blocking_sent_messages_attempts: Arc<AtomicUsize>,
|
||||
}
|
||||
|
||||
pub struct WriterBbqueueReceiver<'a> {
|
||||
|
@ -104,6 +123,12 @@ pub struct WriterBbqueueReceiver<'a> {
|
|||
look_at_consumer: Cycle<Range<usize>>,
|
||||
/// The BBQueue frames to read when waking-up.
|
||||
consumers: Vec<bbqueue::framed::FrameConsumer<'a>>,
|
||||
/// The total number of attempts to send messages
|
||||
/// over the bbqueue channel.
|
||||
sent_messages_attempts: Arc<AtomicUsize>,
|
||||
/// The number of times an attempt to send a
|
||||
/// message failed and we had to pause for a bit.
|
||||
blocking_sent_messages_attempts: Arc<AtomicUsize>,
|
||||
}
|
||||
|
||||
/// The action to perform on the receiver/writer side.
|
||||
|
@ -169,6 +194,16 @@ impl<'a> WriterBbqueueReceiver<'a> {
|
|||
}
|
||||
None
|
||||
}
|
||||
|
||||
/// Returns the total count of attempts to send messages through the BBQueue channel.
|
||||
pub fn sent_messages_attempts(&self) -> usize {
|
||||
self.sent_messages_attempts.load(atomic::Ordering::Relaxed)
|
||||
}
|
||||
|
||||
/// Returns the count of attempts to send messages that had to be paused due to BBQueue being full.
|
||||
pub fn blocking_sent_messages_attempts(&self) -> usize {
|
||||
self.blocking_sent_messages_attempts.load(atomic::Ordering::Relaxed)
|
||||
}
|
||||
}
|
||||
|
||||
pub struct FrameWithHeader<'a> {
|
||||
|
@ -458,10 +493,17 @@ impl<'b> ExtractorBbqueueSender<'b> {
|
|||
}
|
||||
|
||||
// Spin loop to have a frame the size we requested.
|
||||
reserve_and_write_grant(&mut producer, total_length, &self.sender, |grant| {
|
||||
payload_header.serialize_into(grant);
|
||||
Ok(())
|
||||
})?;
|
||||
reserve_and_write_grant(
|
||||
&mut producer,
|
||||
total_length,
|
||||
&self.sender,
|
||||
&self.sent_messages_attempts,
|
||||
&self.blocking_sent_messages_attempts,
|
||||
|grant| {
|
||||
payload_header.serialize_into(grant);
|
||||
Ok(())
|
||||
},
|
||||
)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
@ -500,20 +542,28 @@ impl<'b> ExtractorBbqueueSender<'b> {
|
|||
}
|
||||
|
||||
// Spin loop to have a frame the size we requested.
|
||||
reserve_and_write_grant(&mut producer, total_length, &self.sender, |grant| {
|
||||
let header_size = payload_header.header_size();
|
||||
let (header_bytes, remaining) = grant.split_at_mut(header_size);
|
||||
payload_header.serialize_into(header_bytes);
|
||||
reserve_and_write_grant(
|
||||
&mut producer,
|
||||
total_length,
|
||||
&self.sender,
|
||||
&self.sent_messages_attempts,
|
||||
&self.blocking_sent_messages_attempts,
|
||||
|grant| {
|
||||
let header_size = payload_header.header_size();
|
||||
let (header_bytes, remaining) = grant.split_at_mut(header_size);
|
||||
payload_header.serialize_into(header_bytes);
|
||||
|
||||
if dimensions != 0 {
|
||||
let output_iter = remaining.chunks_exact_mut(dimensions * mem::size_of::<f32>());
|
||||
for (embedding, output) in embeddings.iter().zip(output_iter) {
|
||||
output.copy_from_slice(bytemuck::cast_slice(embedding));
|
||||
if dimensions != 0 {
|
||||
let output_iter =
|
||||
remaining.chunks_exact_mut(dimensions * mem::size_of::<f32>());
|
||||
for (embedding, output) in embeddings.iter().zip(output_iter) {
|
||||
output.copy_from_slice(bytemuck::cast_slice(embedding));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
})?;
|
||||
Ok(())
|
||||
},
|
||||
)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
@ -571,13 +621,20 @@ impl<'b> ExtractorBbqueueSender<'b> {
|
|||
}
|
||||
|
||||
// Spin loop to have a frame the size we requested.
|
||||
reserve_and_write_grant(&mut producer, total_length, &self.sender, |grant| {
|
||||
let header_size = payload_header.header_size();
|
||||
let (header_bytes, remaining) = grant.split_at_mut(header_size);
|
||||
payload_header.serialize_into(header_bytes);
|
||||
let (key_buffer, value_buffer) = remaining.split_at_mut(key_length.get() as usize);
|
||||
key_value_writer(key_buffer, value_buffer)
|
||||
})?;
|
||||
reserve_and_write_grant(
|
||||
&mut producer,
|
||||
total_length,
|
||||
&self.sender,
|
||||
&self.sent_messages_attempts,
|
||||
&self.blocking_sent_messages_attempts,
|
||||
|grant| {
|
||||
let header_size = payload_header.header_size();
|
||||
let (header_bytes, remaining) = grant.split_at_mut(header_size);
|
||||
payload_header.serialize_into(header_bytes);
|
||||
let (key_buffer, value_buffer) = remaining.split_at_mut(key_length.get() as usize);
|
||||
key_value_writer(key_buffer, value_buffer)
|
||||
},
|
||||
)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
@ -619,12 +676,19 @@ impl<'b> ExtractorBbqueueSender<'b> {
|
|||
}
|
||||
|
||||
// Spin loop to have a frame the size we requested.
|
||||
reserve_and_write_grant(&mut producer, total_length, &self.sender, |grant| {
|
||||
let header_size = payload_header.header_size();
|
||||
let (header_bytes, remaining) = grant.split_at_mut(header_size);
|
||||
payload_header.serialize_into(header_bytes);
|
||||
key_writer(remaining)
|
||||
})?;
|
||||
reserve_and_write_grant(
|
||||
&mut producer,
|
||||
total_length,
|
||||
&self.sender,
|
||||
&self.sent_messages_attempts,
|
||||
&self.blocking_sent_messages_attempts,
|
||||
|grant| {
|
||||
let header_size = payload_header.header_size();
|
||||
let (header_bytes, remaining) = grant.split_at_mut(header_size);
|
||||
payload_header.serialize_into(header_bytes);
|
||||
key_writer(remaining)
|
||||
},
|
||||
)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
@ -637,12 +701,18 @@ fn reserve_and_write_grant<F>(
|
|||
producer: &mut FrameProducer,
|
||||
total_length: usize,
|
||||
sender: &flume::Sender<ReceiverAction>,
|
||||
sent_messages_attempts: &AtomicUsize,
|
||||
blocking_sent_messages_attempts: &AtomicUsize,
|
||||
f: F,
|
||||
) -> crate::Result<()>
|
||||
where
|
||||
F: FnOnce(&mut [u8]) -> crate::Result<()>,
|
||||
{
|
||||
loop {
|
||||
// An attempt means trying multiple times
|
||||
// whether is succeeded or not.
|
||||
sent_messages_attempts.fetch_add(1, atomic::Ordering::Relaxed);
|
||||
|
||||
for _ in 0..10_000 {
|
||||
match producer.grant(total_length) {
|
||||
Ok(mut grant) => {
|
||||
|
@ -666,6 +736,10 @@ where
|
|||
return Err(Error::InternalError(InternalError::AbortedIndexation));
|
||||
}
|
||||
|
||||
// We made an attempt to send a message in the
|
||||
// bbqueue channel but it didn't succeed.
|
||||
blocking_sent_messages_attempts.fetch_add(1, atomic::Ordering::Relaxed);
|
||||
|
||||
// We prefer to yield and allow the writing thread
|
||||
// to do its job, especially beneficial when there
|
||||
// is only one CPU core available.
|
||||
|
|
|
@ -144,7 +144,7 @@ impl<'doc> Update<'doc> {
|
|||
)?)
|
||||
}
|
||||
|
||||
pub fn updated(&self) -> DocumentFromVersions<'_, 'doc> {
|
||||
pub fn only_changed_fields(&self) -> DocumentFromVersions<'_, 'doc> {
|
||||
DocumentFromVersions::new(&self.new)
|
||||
}
|
||||
|
||||
|
@ -182,7 +182,7 @@ impl<'doc> Update<'doc> {
|
|||
let mut cached_current = None;
|
||||
let mut updated_selected_field_count = 0;
|
||||
|
||||
for entry in self.updated().iter_top_level_fields() {
|
||||
for entry in self.only_changed_fields().iter_top_level_fields() {
|
||||
let (key, updated_value) = entry?;
|
||||
|
||||
if perm_json_p::select_field(key, fields, &[]) == perm_json_p::Selection::Skip {
|
||||
|
@ -241,7 +241,7 @@ impl<'doc> Update<'doc> {
|
|||
Ok(has_deleted_fields)
|
||||
}
|
||||
|
||||
pub fn updated_vectors(
|
||||
pub fn only_changed_vectors(
|
||||
&self,
|
||||
doc_alloc: &'doc Bump,
|
||||
embedders: &'doc EmbeddingConfigs,
|
||||
|
|
|
@ -199,7 +199,7 @@ impl<'extractor> Extractor<'extractor> for GeoExtractor {
|
|||
.transpose()?;
|
||||
|
||||
let updated_geo = update
|
||||
.updated()
|
||||
.merged(rtxn, index, db_fields_ids_map)?
|
||||
.geo_field()?
|
||||
.map(|geo| extract_geo_coordinates(external_id, geo))
|
||||
.transpose()?;
|
||||
|
|
|
@ -99,7 +99,8 @@ impl<'a, 'b, 'extractor> Extractor<'extractor> for EmbeddingExtractor<'a, 'b> {
|
|||
context.db_fields_ids_map,
|
||||
&context.doc_alloc,
|
||||
)?;
|
||||
let new_vectors = update.updated_vectors(&context.doc_alloc, self.embedders)?;
|
||||
let new_vectors =
|
||||
update.only_changed_vectors(&context.doc_alloc, self.embedders)?;
|
||||
|
||||
if let Some(new_vectors) = &new_vectors {
|
||||
unused_vectors_distribution.append(new_vectors)?;
|
||||
|
|
|
@ -234,7 +234,7 @@ where
|
|||
);
|
||||
let mut datastore = ThreadLocal::with_capacity(rayon::current_num_threads());
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::extract", "vectors");
|
||||
let span = tracing::debug_span!(target: "indexing::documents::extract", "vectors");
|
||||
let _entered = span.enter();
|
||||
|
||||
extract(
|
||||
|
@ -247,7 +247,7 @@ where
|
|||
)?;
|
||||
}
|
||||
{
|
||||
let span = tracing::trace_span!(target: "indexing::documents::merge", "vectors");
|
||||
let span = tracing::debug_span!(target: "indexing::documents::merge", "vectors");
|
||||
let _entered = span.enter();
|
||||
|
||||
for config in &mut index_embeddings {
|
||||
|
|
|
@ -1,5 +1,5 @@
|
|||
use std::sync::atomic::AtomicBool;
|
||||
use std::sync::RwLock;
|
||||
use std::sync::{Once, RwLock};
|
||||
use std::thread::{self, Builder};
|
||||
|
||||
use big_s::S;
|
||||
|
@ -33,6 +33,8 @@ mod post_processing;
|
|||
mod update_by_function;
|
||||
mod write;
|
||||
|
||||
static LOG_MEMORY_METRICS_ONCE: Once = Once::new();
|
||||
|
||||
/// This is the main function of this crate.
|
||||
///
|
||||
/// Give it the output of the [`Indexer::document_changes`] method and it will execute it in the [`rayon::ThreadPool`].
|
||||
|
@ -93,6 +95,15 @@ where
|
|||
},
|
||||
);
|
||||
|
||||
LOG_MEMORY_METRICS_ONCE.call_once(|| {
|
||||
tracing::debug!(
|
||||
"Indexation allocated memory metrics - \
|
||||
Total BBQueue size: {total_bbbuffer_capacity}, \
|
||||
Total extractor memory: {:?}",
|
||||
grenad_parameters.max_memory,
|
||||
);
|
||||
});
|
||||
|
||||
let (extractor_sender, writer_receiver) = pool
|
||||
.install(|| extractor_writer_bbqueue(&mut bbbuffers, total_bbbuffer_capacity, 1000))
|
||||
.unwrap();
|
||||
|
@ -179,13 +190,16 @@ where
|
|||
|
||||
indexing_context.progress.update_progress(IndexingStep::WritingEmbeddingsToDatabase);
|
||||
|
||||
build_vectors(
|
||||
index,
|
||||
wtxn,
|
||||
index_embeddings,
|
||||
&mut arroy_writers,
|
||||
&indexing_context.must_stop_processing,
|
||||
)?;
|
||||
pool.install(|| {
|
||||
build_vectors(
|
||||
index,
|
||||
wtxn,
|
||||
index_embeddings,
|
||||
&mut arroy_writers,
|
||||
&indexing_context.must_stop_processing,
|
||||
)
|
||||
})
|
||||
.unwrap()?;
|
||||
|
||||
post_processing::post_process(
|
||||
indexing_context,
|
||||
|
|
|
@ -72,11 +72,23 @@ pub(super) fn write_to_db(
|
|||
&mut aligned_embedding,
|
||||
)?;
|
||||
}
|
||||
|
||||
write_from_bbqueue(&mut writer_receiver, index, wtxn, arroy_writers, &mut aligned_embedding)?;
|
||||
|
||||
let direct_attempts = writer_receiver.sent_messages_attempts();
|
||||
let blocking_attempts = writer_receiver.blocking_sent_messages_attempts();
|
||||
let congestion_pct = (blocking_attempts as f64 / direct_attempts as f64) * 100.0;
|
||||
tracing::debug!(
|
||||
"Channel congestion metrics - \
|
||||
Attempts: {direct_attempts}, \
|
||||
Blocked attempts: {blocking_attempts} \
|
||||
({congestion_pct:.1}% congestion)"
|
||||
);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "trace", skip_all, target = "indexing::vectors")]
|
||||
#[tracing::instrument(level = "debug", skip_all, target = "indexing::vectors")]
|
||||
pub(super) fn build_vectors<MSP>(
|
||||
index: &Index,
|
||||
wtxn: &mut RwTxn<'_>,
|
||||
|
|
|
@ -1,7 +1,9 @@
|
|||
mod v1_12;
|
||||
mod v1_13;
|
||||
|
||||
use heed::RwTxn;
|
||||
use v1_12::{V1_12_3_To_Current, V1_12_To_V1_12_3};
|
||||
use v1_12::{V1_12_3_To_V1_13_0, V1_12_To_V1_12_3};
|
||||
use v1_13::V1_13_0_To_Current;
|
||||
|
||||
use crate::progress::{Progress, VariableNameStep};
|
||||
use crate::{Index, InternalError, Result};
|
||||
|
@ -26,11 +28,13 @@ pub fn upgrade(
|
|||
progress: Progress,
|
||||
) -> Result<bool> {
|
||||
let from = index.get_version(wtxn)?.unwrap_or(db_version);
|
||||
let upgrade_functions: &[&dyn UpgradeIndex] = &[&V1_12_To_V1_12_3 {}, &V1_12_3_To_Current()];
|
||||
let upgrade_functions: &[&dyn UpgradeIndex] =
|
||||
&[&V1_12_To_V1_12_3 {}, &V1_12_3_To_V1_13_0 {}, &V1_13_0_To_Current()];
|
||||
|
||||
let start = match from {
|
||||
(1, 12, 0..=2) => 0,
|
||||
(1, 12, 3..) => 1,
|
||||
(1, 13, 0) => 2,
|
||||
// We must handle the current version in the match because in case of a failure some index may have been upgraded but not other.
|
||||
(1, 13, _) => return Ok(false),
|
||||
(major, minor, patch) => {
|
||||
|
|
|
@ -1,11 +1,9 @@
|
|||
use heed::RwTxn;
|
||||
|
||||
use crate::constants::{VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH};
|
||||
use super::UpgradeIndex;
|
||||
use crate::progress::Progress;
|
||||
use crate::{make_enum_progress, Index, Result};
|
||||
|
||||
use super::UpgradeIndex;
|
||||
|
||||
#[allow(non_camel_case_types)]
|
||||
pub(super) struct V1_12_To_V1_12_3 {}
|
||||
|
||||
|
@ -33,9 +31,9 @@ impl UpgradeIndex for V1_12_To_V1_12_3 {
|
|||
}
|
||||
|
||||
#[allow(non_camel_case_types)]
|
||||
pub(super) struct V1_12_3_To_Current();
|
||||
pub(super) struct V1_12_3_To_V1_13_0 {}
|
||||
|
||||
impl UpgradeIndex for V1_12_3_To_Current {
|
||||
impl UpgradeIndex for V1_12_3_To_V1_13_0 {
|
||||
fn upgrade(
|
||||
&self,
|
||||
_wtxn: &mut RwTxn,
|
||||
|
@ -43,14 +41,11 @@ impl UpgradeIndex for V1_12_3_To_Current {
|
|||
_original: (u32, u32, u32),
|
||||
_progress: Progress,
|
||||
) -> Result<bool> {
|
||||
Ok(false)
|
||||
// recompute the indexes stats
|
||||
Ok(true)
|
||||
}
|
||||
|
||||
fn target_version(&self) -> (u32, u32, u32) {
|
||||
(
|
||||
VERSION_MAJOR.parse().unwrap(),
|
||||
VERSION_MINOR.parse().unwrap(),
|
||||
VERSION_PATCH.parse().unwrap(),
|
||||
)
|
||||
(1, 13, 0)
|
||||
}
|
||||
}
|
||||
|
|
29
crates/milli/src/update/upgrade/v1_13.rs
Normal file
29
crates/milli/src/update/upgrade/v1_13.rs
Normal file
|
@ -0,0 +1,29 @@
|
|||
use heed::RwTxn;
|
||||
|
||||
use super::UpgradeIndex;
|
||||
use crate::constants::{VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH};
|
||||
use crate::progress::Progress;
|
||||
use crate::{Index, Result};
|
||||
|
||||
#[allow(non_camel_case_types)]
|
||||
pub(super) struct V1_13_0_To_Current();
|
||||
|
||||
impl UpgradeIndex for V1_13_0_To_Current {
|
||||
fn upgrade(
|
||||
&self,
|
||||
_wtxn: &mut RwTxn,
|
||||
_index: &Index,
|
||||
_original: (u32, u32, u32),
|
||||
_progress: Progress,
|
||||
) -> Result<bool> {
|
||||
Ok(false)
|
||||
}
|
||||
|
||||
fn target_version(&self) -> (u32, u32, u32) {
|
||||
(
|
||||
VERSION_MAJOR.parse().unwrap(),
|
||||
VERSION_MINOR.parse().unwrap(),
|
||||
VERSION_PATCH.parse().unwrap(),
|
||||
)
|
||||
}
|
||||
}
|
|
@ -410,8 +410,43 @@ impl ArroyWrapper {
|
|||
fn quantized_db(&self) -> arroy::Database<BinaryQuantizedCosine> {
|
||||
self.database.remap_data_type()
|
||||
}
|
||||
|
||||
pub fn aggregate_stats(
|
||||
&self,
|
||||
rtxn: &RoTxn,
|
||||
stats: &mut ArroyStats,
|
||||
) -> Result<(), arroy::Error> {
|
||||
if self.quantized {
|
||||
for reader in self.readers(rtxn, self.quantized_db()) {
|
||||
let reader = reader?;
|
||||
let documents = reader.item_ids();
|
||||
if documents.is_empty() {
|
||||
break;
|
||||
}
|
||||
stats.documents |= documents;
|
||||
stats.number_of_embeddings += documents.len();
|
||||
}
|
||||
} else {
|
||||
for reader in self.readers(rtxn, self.angular_db()) {
|
||||
let reader = reader?;
|
||||
let documents = reader.item_ids();
|
||||
if documents.is_empty() {
|
||||
break;
|
||||
}
|
||||
stats.documents |= documents;
|
||||
stats.number_of_embeddings += documents.len();
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Default, Clone)]
|
||||
pub struct ArroyStats {
|
||||
pub number_of_embeddings: u64,
|
||||
pub documents: RoaringBitmap,
|
||||
}
|
||||
/// One or multiple embeddings stored consecutively in a flat vector.
|
||||
pub struct Embeddings<F> {
|
||||
data: Vec<F>,
|
||||
|
@ -611,6 +646,7 @@ impl Embedder {
|
|||
}
|
||||
}
|
||||
|
||||
#[tracing::instrument(level = "debug", skip_all, target = "search")]
|
||||
pub fn embed_one(
|
||||
&self,
|
||||
text: String,
|
||||
|
|
|
@ -5,7 +5,7 @@ use rayon::slice::ParallelSlice as _;
|
|||
|
||||
use super::error::{EmbedError, EmbedErrorKind, NewEmbedderError, NewEmbedderErrorKind};
|
||||
use super::rest::{Embedder as RestEmbedder, EmbedderOptions as RestEmbedderOptions};
|
||||
use super::DistributionShift;
|
||||
use super::{DistributionShift, REQUEST_PARALLELISM};
|
||||
use crate::error::FaultSource;
|
||||
use crate::vector::Embedding;
|
||||
use crate::ThreadPoolNoAbort;
|
||||
|
@ -118,14 +118,20 @@ impl Embedder {
|
|||
text_chunks: Vec<Vec<String>>,
|
||||
threads: &ThreadPoolNoAbort,
|
||||
) -> Result<Vec<Vec<Embedding>>, EmbedError> {
|
||||
threads
|
||||
.install(move || {
|
||||
text_chunks.into_par_iter().map(move |chunk| self.embed(&chunk, None)).collect()
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
// This condition helps reduce the number of active rayon jobs
|
||||
// so that we avoid consuming all the LMDB rtxns and avoid stack overflows.
|
||||
if threads.active_operations() >= REQUEST_PARALLELISM {
|
||||
text_chunks.into_iter().map(move |chunk| self.embed(&chunk, None)).collect()
|
||||
} else {
|
||||
threads
|
||||
.install(move || {
|
||||
text_chunks.into_par_iter().map(move |chunk| self.embed(&chunk, None)).collect()
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn embed_chunks_ref(
|
||||
|
@ -133,20 +139,32 @@ impl Embedder {
|
|||
texts: &[&str],
|
||||
threads: &ThreadPoolNoAbort,
|
||||
) -> Result<Vec<Vec<f32>>, EmbedError> {
|
||||
threads
|
||||
.install(move || {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.par_chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed(chunk, None))
|
||||
.collect();
|
||||
// This condition helps reduce the number of active rayon jobs
|
||||
// so that we avoid consuming all the LMDB rtxns and avoid stack overflows.
|
||||
if threads.active_operations() >= REQUEST_PARALLELISM {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed(chunk, None))
|
||||
.collect();
|
||||
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
} else {
|
||||
threads
|
||||
.install(move || {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.par_chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed(chunk, None))
|
||||
.collect();
|
||||
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
}
|
||||
}
|
||||
|
||||
pub fn chunk_count_hint(&self) -> usize {
|
||||
|
|
|
@ -7,7 +7,7 @@ use rayon::slice::ParallelSlice as _;
|
|||
|
||||
use super::error::{EmbedError, NewEmbedderError};
|
||||
use super::rest::{Embedder as RestEmbedder, EmbedderOptions as RestEmbedderOptions};
|
||||
use super::DistributionShift;
|
||||
use super::{DistributionShift, REQUEST_PARALLELISM};
|
||||
use crate::error::FaultSource;
|
||||
use crate::vector::error::EmbedErrorKind;
|
||||
use crate::vector::Embedding;
|
||||
|
@ -255,14 +255,20 @@ impl Embedder {
|
|||
text_chunks: Vec<Vec<String>>,
|
||||
threads: &ThreadPoolNoAbort,
|
||||
) -> Result<Vec<Vec<Embedding>>, EmbedError> {
|
||||
threads
|
||||
.install(move || {
|
||||
text_chunks.into_par_iter().map(move |chunk| self.embed(&chunk, None)).collect()
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
// This condition helps reduce the number of active rayon jobs
|
||||
// so that we avoid consuming all the LMDB rtxns and avoid stack overflows.
|
||||
if threads.active_operations() >= REQUEST_PARALLELISM {
|
||||
text_chunks.into_iter().map(move |chunk| self.embed(&chunk, None)).collect()
|
||||
} else {
|
||||
threads
|
||||
.install(move || {
|
||||
text_chunks.into_par_iter().map(move |chunk| self.embed(&chunk, None)).collect()
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn embed_chunks_ref(
|
||||
|
@ -270,20 +276,31 @@ impl Embedder {
|
|||
texts: &[&str],
|
||||
threads: &ThreadPoolNoAbort,
|
||||
) -> Result<Vec<Vec<f32>>, EmbedError> {
|
||||
threads
|
||||
.install(move || {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.par_chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed(chunk, None))
|
||||
.collect();
|
||||
// This condition helps reduce the number of active rayon jobs
|
||||
// so that we avoid consuming all the LMDB rtxns and avoid stack overflows.
|
||||
if threads.active_operations() >= REQUEST_PARALLELISM {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed(chunk, None))
|
||||
.collect();
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
} else {
|
||||
threads
|
||||
.install(move || {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.par_chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed(chunk, None))
|
||||
.collect();
|
||||
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
}
|
||||
}
|
||||
|
||||
pub fn chunk_count_hint(&self) -> usize {
|
||||
|
|
|
@ -130,6 +130,7 @@ impl Embedder {
|
|||
let client = ureq::AgentBuilder::new()
|
||||
.max_idle_connections(REQUEST_PARALLELISM * 2)
|
||||
.max_idle_connections_per_host(REQUEST_PARALLELISM * 2)
|
||||
.timeout(std::time::Duration::from_secs(30))
|
||||
.build();
|
||||
|
||||
let request = Request::new(options.request)?;
|
||||
|
@ -188,14 +189,20 @@ impl Embedder {
|
|||
text_chunks: Vec<Vec<String>>,
|
||||
threads: &ThreadPoolNoAbort,
|
||||
) -> Result<Vec<Vec<Embedding>>, EmbedError> {
|
||||
threads
|
||||
.install(move || {
|
||||
text_chunks.into_par_iter().map(move |chunk| self.embed(chunk, None)).collect()
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
// This condition helps reduce the number of active rayon jobs
|
||||
// so that we avoid consuming all the LMDB rtxns and avoid stack overflows.
|
||||
if threads.active_operations() >= REQUEST_PARALLELISM {
|
||||
text_chunks.into_iter().map(move |chunk| self.embed(chunk, None)).collect()
|
||||
} else {
|
||||
threads
|
||||
.install(move || {
|
||||
text_chunks.into_par_iter().map(move |chunk| self.embed(chunk, None)).collect()
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn embed_chunks_ref(
|
||||
|
@ -203,20 +210,32 @@ impl Embedder {
|
|||
texts: &[&str],
|
||||
threads: &ThreadPoolNoAbort,
|
||||
) -> Result<Vec<Embedding>, EmbedError> {
|
||||
threads
|
||||
.install(move || {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.par_chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed_ref(chunk, None))
|
||||
.collect();
|
||||
// This condition helps reduce the number of active rayon jobs
|
||||
// so that we avoid consuming all the LMDB rtxns and avoid stack overflows.
|
||||
if threads.active_operations() >= REQUEST_PARALLELISM {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed_ref(chunk, None))
|
||||
.collect();
|
||||
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
} else {
|
||||
threads
|
||||
.install(move || {
|
||||
let embeddings: Result<Vec<Vec<Embedding>>, _> = texts
|
||||
.par_chunks(self.prompt_count_in_chunk_hint())
|
||||
.map(move |chunk| self.embed_ref(chunk, None))
|
||||
.collect();
|
||||
|
||||
let embeddings = embeddings?;
|
||||
Ok(embeddings.into_iter().flatten().collect())
|
||||
})
|
||||
.map_err(|error| EmbedError {
|
||||
kind: EmbedErrorKind::PanicInThreadPool(error),
|
||||
fault: FaultSource::Bug,
|
||||
})?
|
||||
}
|
||||
}
|
||||
|
||||
pub fn chunk_count_hint(&self) -> usize {
|
||||
|
|
|
@ -455,7 +455,7 @@ impl EmbeddingSettings {
|
|||
EmbedderSource::Ollama,
|
||||
EmbedderSource::Rest,
|
||||
],
|
||||
Self::DOCUMENT_TEMPLATE => &[
|
||||
Self::DOCUMENT_TEMPLATE | Self::DOCUMENT_TEMPLATE_MAX_BYTES => &[
|
||||
EmbedderSource::HuggingFace,
|
||||
EmbedderSource::OpenAi,
|
||||
EmbedderSource::Ollama,
|
||||
|
@ -490,6 +490,7 @@ impl EmbeddingSettings {
|
|||
Self::MODEL,
|
||||
Self::API_KEY,
|
||||
Self::DOCUMENT_TEMPLATE,
|
||||
Self::DOCUMENT_TEMPLATE_MAX_BYTES,
|
||||
Self::DIMENSIONS,
|
||||
Self::DISTRIBUTION,
|
||||
Self::URL,
|
||||
|
@ -500,6 +501,7 @@ impl EmbeddingSettings {
|
|||
Self::MODEL,
|
||||
Self::REVISION,
|
||||
Self::DOCUMENT_TEMPLATE,
|
||||
Self::DOCUMENT_TEMPLATE_MAX_BYTES,
|
||||
Self::DISTRIBUTION,
|
||||
Self::BINARY_QUANTIZED,
|
||||
],
|
||||
|
@ -507,6 +509,7 @@ impl EmbeddingSettings {
|
|||
Self::SOURCE,
|
||||
Self::MODEL,
|
||||
Self::DOCUMENT_TEMPLATE,
|
||||
Self::DOCUMENT_TEMPLATE_MAX_BYTES,
|
||||
Self::URL,
|
||||
Self::API_KEY,
|
||||
Self::DIMENSIONS,
|
||||
|
@ -521,6 +524,7 @@ impl EmbeddingSettings {
|
|||
Self::API_KEY,
|
||||
Self::DIMENSIONS,
|
||||
Self::DOCUMENT_TEMPLATE,
|
||||
Self::DOCUMENT_TEMPLATE_MAX_BYTES,
|
||||
Self::URL,
|
||||
Self::REQUEST,
|
||||
Self::RESPONSE,
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue