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:
meili-bors[bot] 2025-02-18 08:05:02 +00:00 committed by GitHub
commit 0f1aeb8eaa
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
101 changed files with 8351 additions and 1518 deletions

View file

@ -31,6 +31,7 @@ use crate::routes::{create_all_stats, Stats};
use crate::Opt;
const ANALYTICS_HEADER: &str = "X-Meilisearch-Client";
const MEILI_SERVER_PROVIDER: &str = "MEILI_SERVER_PROVIDER";
/// Write the instance-uid in the `data.ms` and in `~/.config/MeiliSearch/path-to-db-instance-uid`. Ignore the errors.
fn write_user_id(db_path: &Path, user_id: &InstanceUid) {
@ -195,6 +196,8 @@ struct Infos {
experimental_reduce_indexing_memory_usage: bool,
experimental_max_number_of_batched_tasks: usize,
experimental_limit_batched_tasks_total_size: u64,
experimental_network: bool,
experimental_get_task_documents_route: bool,
gpu_enabled: bool,
db_path: bool,
import_dump: bool,
@ -285,6 +288,8 @@ impl Infos {
logs_route,
edit_documents_by_function,
contains_filter,
network,
get_task_documents_route,
} = features;
// We're going to override every sensible information.
@ -302,6 +307,8 @@ impl Infos {
experimental_replication_parameters,
experimental_enable_logs_route: experimental_enable_logs_route | logs_route,
experimental_reduce_indexing_memory_usage,
experimental_network: network,
experimental_get_task_documents_route: get_task_documents_route,
gpu_enabled: meilisearch_types::milli::vector::is_cuda_enabled(),
db_path: db_path != PathBuf::from("./data.ms"),
import_dump: import_dump.is_some(),
@ -357,7 +364,7 @@ impl Segment {
"cores": sys.cpus().len(),
"ram_size": sys.total_memory(),
"disk_size": disks.iter().map(|disk| disk.total_space()).max(),
"server_provider": std::env::var("MEILI_SERVER_PROVIDER").ok(),
"server_provider": std::env::var(MEILI_SERVER_PROVIDER).ok(),
})
});
let number_of_documents =
@ -380,10 +387,18 @@ impl Segment {
index_scheduler: Arc<IndexScheduler>,
auth_controller: Arc<AuthController>,
) {
const INTERVAL: Duration = Duration::from_secs(60 * 60); // one hour
// The first batch must be sent after one hour.
let interval: Duration = match std::env::var(MEILI_SERVER_PROVIDER) {
Ok(provider) if provider.starts_with("meili_cloud:") => {
Duration::from_secs(60 * 60) // one hour
}
_ => {
// We're an open source instance
Duration::from_secs(60 * 60 * 24) // one day
}
};
let mut interval =
tokio::time::interval_at(tokio::time::Instant::now() + INTERVAL, INTERVAL);
tokio::time::interval_at(tokio::time::Instant::now() + interval, interval);
loop {
select! {

View file

@ -32,6 +32,7 @@ use analytics::Analytics;
use anyhow::bail;
use error::PayloadError;
use extractors::payload::PayloadConfig;
use index_scheduler::versioning::Versioning;
use index_scheduler::{IndexScheduler, IndexSchedulerOptions};
use meilisearch_auth::AuthController;
use meilisearch_types::milli::constants::VERSION_MAJOR;
@ -40,10 +41,9 @@ use meilisearch_types::milli::update::{IndexDocumentsConfig, IndexDocumentsMetho
use meilisearch_types::settings::apply_settings_to_builder;
use meilisearch_types::tasks::KindWithContent;
use meilisearch_types::versioning::{
create_current_version_file, get_version, update_version_file_for_dumpless_upgrade,
VersionFileError, VERSION_MINOR, VERSION_PATCH,
create_current_version_file, get_version, VersionFileError, VERSION_MINOR, VERSION_PATCH,
};
use meilisearch_types::{compression, milli, VERSION_FILE_NAME};
use meilisearch_types::{compression, heed, milli, VERSION_FILE_NAME};
pub use option::Opt;
use option::ScheduleSnapshot;
use search_queue::SearchQueue;
@ -356,14 +356,19 @@ fn open_or_create_database_unchecked(
/// Ensures Meilisearch version is compatible with the database, returns an error in case of version mismatch.
/// Returns the version that was contained in the version file
fn check_version(opt: &Opt, binary_version: (u32, u32, u32)) -> anyhow::Result<(u32, u32, u32)> {
fn check_version(
opt: &Opt,
index_scheduler_opt: &IndexSchedulerOptions,
binary_version: (u32, u32, u32),
) -> anyhow::Result<(u32, u32, u32)> {
let (bin_major, bin_minor, bin_patch) = binary_version;
let (db_major, db_minor, db_patch) = get_version(&opt.db_path)?;
if db_major != bin_major || db_minor != bin_minor || db_patch > bin_patch {
if opt.experimental_dumpless_upgrade {
update_version_file_for_dumpless_upgrade(
&opt.db_path,
opt,
index_scheduler_opt,
(db_major, db_minor, db_patch),
(bin_major, bin_minor, bin_patch),
)?;
@ -380,6 +385,57 @@ fn check_version(opt: &Opt, binary_version: (u32, u32, u32)) -> anyhow::Result<(
Ok((db_major, db_minor, db_patch))
}
/// Persists the version of the current Meilisearch binary to a VERSION file
pub fn update_version_file_for_dumpless_upgrade(
opt: &Opt,
index_scheduler_opt: &IndexSchedulerOptions,
from: (u32, u32, u32),
to: (u32, u32, u32),
) -> Result<(), VersionFileError> {
let (from_major, from_minor, from_patch) = from;
let (to_major, to_minor, to_patch) = to;
// Early exit in case of error
if from_major > to_major
|| (from_major == to_major && from_minor > to_minor)
|| (from_major == to_major && from_minor == to_minor && from_patch > to_patch)
{
return Err(VersionFileError::DowngradeNotSupported {
major: from_major,
minor: from_minor,
patch: from_patch,
});
} else if from_major < 1 || (from_major == to_major && from_minor < 12) {
return Err(VersionFileError::TooOldForAutomaticUpgrade {
major: from_major,
minor: from_minor,
patch: from_patch,
});
}
// In the case of v1.12, the index-scheduler didn't store its internal version at the time.
// => We must write it immediately **in the index-scheduler** otherwise we'll update the version file
// there is a risk of DB corruption if a restart happens after writing the version file but before
// writing the version in the index-scheduler. See <https://github.com/meilisearch/meilisearch/issues/5280>
if from_major == 1 && from_minor == 12 {
let env = unsafe {
heed::EnvOpenOptions::new()
.max_dbs(Versioning::nb_db())
.map_size(index_scheduler_opt.task_db_size)
.open(&index_scheduler_opt.tasks_path)
}?;
let mut wtxn = env.write_txn()?;
let versioning = Versioning::raw_new(&env, &mut wtxn)?;
versioning.set_version(&mut wtxn, (from_major, from_minor, from_patch))?;
wtxn.commit()?;
// Should be instant since we're the only one using the env
env.prepare_for_closing().wait();
}
create_current_version_file(&opt.db_path)?;
Ok(())
}
/// Ensure you're in a valid state and open the IndexScheduler + AuthController for you.
fn open_or_create_database(
opt: &Opt,
@ -387,7 +443,11 @@ fn open_or_create_database(
empty_db: bool,
binary_version: (u32, u32, u32),
) -> anyhow::Result<(IndexScheduler, AuthController)> {
let version = if !empty_db { check_version(opt, binary_version)? } else { binary_version };
let version = if !empty_db {
check_version(opt, &index_scheduler_opt, binary_version)?
} else {
binary_version
};
open_or_create_database_unchecked(opt, index_scheduler_opt, OnFailure::KeepDb, version)
}
@ -431,10 +491,13 @@ fn import_dump(
keys.push(key);
}
// 3. Import the runtime features.
// 3. Import the runtime features and network
let features = dump_reader.features()?.unwrap_or_default();
index_scheduler.put_runtime_features(features)?;
let network = dump_reader.network()?.cloned().unwrap_or_default();
index_scheduler.put_network(network)?;
let indexer_config = index_scheduler.indexer_config();
// /!\ The tasks must be imported AFTER importing the indexes or else the scheduler might
@ -508,9 +571,15 @@ fn import_dump(
index_scheduler.refresh_index_stats(&uid)?;
}
// 5. Import the queue
let mut index_scheduler_dump = index_scheduler.register_dumped_task()?;
// 5.1. Import the batches
for ret in dump_reader.batches()? {
let batch = ret?;
index_scheduler_dump.register_dumped_batch(batch)?;
}
// 5. Import the tasks.
// 5.2. Import the tasks
for ret in dump_reader.tasks()? {
let (task, file) = ret?;
index_scheduler_dump.register_dumped_task(task, file)?;

View file

@ -50,6 +50,8 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
logs_route: Some(false),
edit_documents_by_function: Some(false),
contains_filter: Some(false),
network: Some(false),
get_task_documents_route: Some(false),
})),
(status = 401, description = "The authorization header is missing", body = ResponseError, content_type = "application/json", example = json!(
{
@ -88,6 +90,10 @@ pub struct RuntimeTogglableFeatures {
pub edit_documents_by_function: Option<bool>,
#[deserr(default)]
pub contains_filter: Option<bool>,
#[deserr(default)]
pub network: Option<bool>,
#[deserr(default)]
pub get_task_documents_route: Option<bool>,
}
impl From<meilisearch_types::features::RuntimeTogglableFeatures> for RuntimeTogglableFeatures {
@ -97,6 +103,8 @@ impl From<meilisearch_types::features::RuntimeTogglableFeatures> for RuntimeTogg
logs_route,
edit_documents_by_function,
contains_filter,
network,
get_task_documents_route,
} = value;
Self {
@ -104,6 +112,8 @@ impl From<meilisearch_types::features::RuntimeTogglableFeatures> for RuntimeTogg
logs_route: Some(logs_route),
edit_documents_by_function: Some(edit_documents_by_function),
contains_filter: Some(contains_filter),
network: Some(network),
get_task_documents_route: Some(get_task_documents_route),
}
}
}
@ -114,6 +124,8 @@ pub struct PatchExperimentalFeatureAnalytics {
logs_route: bool,
edit_documents_by_function: bool,
contains_filter: bool,
network: bool,
get_task_documents_route: bool,
}
impl Aggregate for PatchExperimentalFeatureAnalytics {
@ -127,6 +139,8 @@ impl Aggregate for PatchExperimentalFeatureAnalytics {
logs_route: new.logs_route,
edit_documents_by_function: new.edit_documents_by_function,
contains_filter: new.contains_filter,
network: new.network,
get_task_documents_route: new.get_task_documents_route,
})
}
@ -149,6 +163,8 @@ impl Aggregate for PatchExperimentalFeatureAnalytics {
logs_route: Some(false),
edit_documents_by_function: Some(false),
contains_filter: Some(false),
network: Some(false),
get_task_documents_route: Some(false),
})),
(status = 401, description = "The authorization header is missing", body = ResponseError, content_type = "application/json", example = json!(
{
@ -181,16 +197,23 @@ async fn patch_features(
.edit_documents_by_function
.unwrap_or(old_features.edit_documents_by_function),
contains_filter: new_features.0.contains_filter.unwrap_or(old_features.contains_filter),
network: new_features.0.network.unwrap_or(old_features.network),
get_task_documents_route: new_features
.0
.get_task_documents_route
.unwrap_or(old_features.get_task_documents_route),
};
// explicitly destructure for analytics rather than using the `Serialize` implementation, because
// the it renames to camelCase, which we don't want for analytics.
// it renames to camelCase, which we don't want for analytics.
// **Do not** ignore fields with `..` or `_` here, because we want to add them in the future.
let meilisearch_types::features::RuntimeTogglableFeatures {
metrics,
logs_route,
edit_documents_by_function,
contains_filter,
network,
get_task_documents_route,
} = new_features;
analytics.publish(
@ -199,6 +222,8 @@ async fn patch_features(
logs_route,
edit_documents_by_function,
contains_filter,
network,
get_task_documents_route,
},
&req,
);

View file

@ -496,6 +496,12 @@ pub struct IndexStats {
pub number_of_documents: u64,
/// Whether or not the index is currently ingesting document
pub is_indexing: bool,
/// Number of embeddings in the index
#[serde(skip_serializing_if = "Option::is_none")]
pub number_of_embeddings: Option<u64>,
/// Number of embedded documents in the index
#[serde(skip_serializing_if = "Option::is_none")]
pub number_of_embedded_documents: Option<u64>,
/// Association of every field name with the number of times it occurs in the documents.
#[schema(value_type = HashMap<String, u64>)]
pub field_distribution: FieldDistribution,
@ -506,6 +512,8 @@ impl From<index_scheduler::IndexStats> for IndexStats {
IndexStats {
number_of_documents: stats.inner_stats.number_of_documents,
is_indexing: stats.is_indexing,
number_of_embeddings: stats.inner_stats.number_of_embeddings,
number_of_embedded_documents: stats.inner_stats.number_of_embedded_documents,
field_distribution: stats.inner_stats.field_distribution,
}
}
@ -524,6 +532,8 @@ impl From<index_scheduler::IndexStats> for IndexStats {
(status = OK, description = "The stats of the index", body = IndexStats, content_type = "application/json", example = json!(
{
"numberOfDocuments": 10,
"numberOfEmbeddings": 10,
"numberOfEmbeddedDocuments": 10,
"isIndexing": true,
"fieldDistribution": {
"genre": 10,

View file

@ -34,6 +34,7 @@ use crate::routes::features::RuntimeTogglableFeatures;
use crate::routes::indexes::documents::{DocumentDeletionByFilter, DocumentEditionByFunction};
use crate::routes::indexes::IndexView;
use crate::routes::multi_search::SearchResults;
use crate::routes::network::{Network, Remote};
use crate::routes::swap_indexes::SwapIndexesPayload;
use crate::search::{
FederatedSearch, FederatedSearchResult, Federation, FederationOptions, MergeFacets,
@ -54,6 +55,7 @@ mod logs;
mod metrics;
mod multi_search;
mod multi_search_analytics;
pub mod network;
mod open_api_utils;
mod snapshot;
mod swap_indexes;
@ -75,6 +77,7 @@ pub mod tasks;
(path = "/multi-search", api = multi_search::MultiSearchApi),
(path = "/swap-indexes", api = swap_indexes::SwapIndexesApi),
(path = "/experimental-features", api = features::ExperimentalFeaturesApi),
(path = "/network", api = network::NetworkApi),
),
paths(get_health, get_version, get_stats),
tags(
@ -85,7 +88,7 @@ pub mod tasks;
url = "/",
description = "Local server",
)),
components(schemas(PaginationView<KeyView>, PaginationView<IndexView>, IndexView, DocumentDeletionByFilter, AllBatches, BatchStats, ProgressStepView, ProgressView, BatchView, RuntimeTogglableFeatures, SwapIndexesPayload, DocumentEditionByFunction, MergeFacets, FederationOptions, SearchQueryWithIndex, Federation, FederatedSearch, FederatedSearchResult, SearchResults, SearchResultWithIndex, SimilarQuery, SimilarResult, PaginationView<serde_json::Value>, BrowseQuery, UpdateIndexRequest, IndexUid, IndexCreateRequest, KeyView, Action, CreateApiKey, UpdateStderrLogs, LogMode, GetLogs, IndexStats, Stats, HealthStatus, HealthResponse, VersionResponse, Code, ErrorType, AllTasks, TaskView, Status, DetailsView, ResponseError, Settings<Unchecked>, Settings<Checked>, TypoSettings, MinWordSizeTyposSetting, FacetingSettings, PaginationSettings, SummarizedTaskView, Kind))
components(schemas(PaginationView<KeyView>, PaginationView<IndexView>, IndexView, DocumentDeletionByFilter, AllBatches, BatchStats, ProgressStepView, ProgressView, BatchView, RuntimeTogglableFeatures, SwapIndexesPayload, DocumentEditionByFunction, MergeFacets, FederationOptions, SearchQueryWithIndex, Federation, FederatedSearch, FederatedSearchResult, SearchResults, SearchResultWithIndex, SimilarQuery, SimilarResult, PaginationView<serde_json::Value>, BrowseQuery, UpdateIndexRequest, IndexUid, IndexCreateRequest, KeyView, Action, CreateApiKey, UpdateStderrLogs, LogMode, GetLogs, IndexStats, Stats, HealthStatus, HealthResponse, VersionResponse, Code, ErrorType, AllTasks, TaskView, Status, DetailsView, ResponseError, Settings<Unchecked>, Settings<Checked>, TypoSettings, MinWordSizeTyposSetting, FacetingSettings, PaginationSettings, SummarizedTaskView, Kind, Network, Remote))
)]
pub struct MeilisearchApi;
@ -103,7 +106,8 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
.service(web::scope("/multi-search").configure(multi_search::configure))
.service(web::scope("/swap-indexes").configure(swap_indexes::configure))
.service(web::scope("/metrics").configure(metrics::configure))
.service(web::scope("/experimental-features").configure(features::configure));
.service(web::scope("/experimental-features").configure(features::configure))
.service(web::scope("/network").configure(network::configure));
#[cfg(feature = "swagger")]
{
@ -359,9 +363,9 @@ pub async fn running() -> HttpResponse {
#[derive(Serialize, Debug, ToSchema)]
#[serde(rename_all = "camelCase")]
pub struct Stats {
/// The size of the database, in bytes.
/// The disk space used by the database, in bytes.
pub database_size: u64,
#[serde(skip)]
/// The size of the database, in bytes.
pub used_database_size: u64,
/// The date of the last update in the RFC 3339 formats. Can be `null` if no update has ever been processed.
#[serde(serialize_with = "time::serde::rfc3339::option::serialize")]
@ -383,6 +387,7 @@ pub struct Stats {
(status = 200, description = "The stats of the instance", body = Stats, content_type = "application/json", example = json!(
{
"databaseSize": 567,
"usedDatabaseSize": 456,
"lastUpdate": "2019-11-20T09:40:33.711324Z",
"indexes": {
"movies": {

View file

@ -20,6 +20,7 @@ use crate::routes::indexes::search::search_kind;
use crate::search::{
add_search_rules, perform_federated_search, perform_search, FederatedSearch,
FederatedSearchResult, RetrieveVectors, SearchQueryWithIndex, SearchResultWithIndex,
PROXY_SEARCH_HEADER, PROXY_SEARCH_HEADER_VALUE,
};
use crate::search_queue::SearchQueue;
@ -48,6 +49,7 @@ pub struct SearchResults {
/// Bundle multiple search queries in a single API request. Use this endpoint to search through multiple indexes at once.
#[utoipa::path(
post,
request_body = FederatedSearch,
path = "",
tag = "Multi-search",
security(("Bearer" = ["search", "*"])),
@ -186,18 +188,22 @@ pub async fn multi_search_with_post(
let response = match federation {
Some(federation) => {
let search_result = tokio::task::spawn_blocking(move || {
perform_federated_search(&index_scheduler, queries, federation, features)
})
.await;
// check remote header
let is_proxy = req
.headers()
.get(PROXY_SEARCH_HEADER)
.is_some_and(|value| value.as_bytes() == PROXY_SEARCH_HEADER_VALUE.as_bytes());
let search_result =
perform_federated_search(&index_scheduler, queries, federation, features, is_proxy)
.await;
permit.drop().await;
if let Ok(Ok(_)) = search_result {
if search_result.is_ok() {
multi_aggregate.succeed();
}
analytics.publish(multi_aggregate, &req);
HttpResponse::Ok().json(search_result??)
HttpResponse::Ok().json(search_result?)
}
None => {
// Explicitly expect a `(ResponseError, usize)` for the error type rather than `ResponseError` only,

View file

@ -13,6 +13,8 @@ pub struct MultiSearchAggregator {
// sum of the number of distinct indexes in each single request, use with total_received to compute an avg
total_distinct_index_count: usize,
// sum of the number of distinct remotes in each single request, use with total_received to compute an avg
total_distinct_remote_count: usize,
// number of queries with a single index, use with total_received to compute a proportion
total_single_index: usize,
@ -31,46 +33,49 @@ impl MultiSearchAggregator {
pub fn from_federated_search(federated_search: &FederatedSearch) -> Self {
let use_federation = federated_search.federation.is_some();
let distinct_indexes: HashSet<_> = federated_search
.queries
.iter()
.map(|query| {
let query = &query;
// make sure we get a compilation error if a field gets added to / removed from SearchQueryWithIndex
let SearchQueryWithIndex {
index_uid,
federation_options: _,
q: _,
vector: _,
offset: _,
limit: _,
page: _,
hits_per_page: _,
attributes_to_retrieve: _,
retrieve_vectors: _,
attributes_to_crop: _,
crop_length: _,
attributes_to_highlight: _,
show_ranking_score: _,
show_ranking_score_details: _,
show_matches_position: _,
filter: _,
sort: _,
distinct: _,
facets: _,
highlight_pre_tag: _,
highlight_post_tag: _,
crop_marker: _,
matching_strategy: _,
attributes_to_search_on: _,
hybrid: _,
ranking_score_threshold: _,
locales: _,
} = query;
let mut distinct_indexes = HashSet::with_capacity(federated_search.queries.len());
let mut distinct_remotes = HashSet::with_capacity(federated_search.queries.len());
index_uid.as_str()
})
.collect();
// make sure we get a compilation error if a field gets added to / removed from SearchQueryWithIndex
for SearchQueryWithIndex {
index_uid,
federation_options,
q: _,
vector: _,
offset: _,
limit: _,
page: _,
hits_per_page: _,
attributes_to_retrieve: _,
retrieve_vectors: _,
attributes_to_crop: _,
crop_length: _,
attributes_to_highlight: _,
show_ranking_score: _,
show_ranking_score_details: _,
show_matches_position: _,
filter: _,
sort: _,
distinct: _,
facets: _,
highlight_pre_tag: _,
highlight_post_tag: _,
crop_marker: _,
matching_strategy: _,
attributes_to_search_on: _,
hybrid: _,
ranking_score_threshold: _,
locales: _,
} in &federated_search.queries
{
if let Some(federation_options) = federation_options {
if let Some(remote) = &federation_options.remote {
distinct_remotes.insert(remote.as_str());
}
}
distinct_indexes.insert(index_uid.as_str());
}
let show_ranking_score =
federated_search.queries.iter().any(|query| query.show_ranking_score);
@ -81,6 +86,7 @@ impl MultiSearchAggregator {
total_received: 1,
total_succeeded: 0,
total_distinct_index_count: distinct_indexes.len(),
total_distinct_remote_count: distinct_remotes.len(),
total_single_index: if distinct_indexes.len() == 1 { 1 } else { 0 },
total_search_count: federated_search.queries.len(),
show_ranking_score,
@ -110,6 +116,8 @@ impl Aggregate for MultiSearchAggregator {
let total_succeeded = this.total_succeeded.saturating_add(new.total_succeeded);
let total_distinct_index_count =
this.total_distinct_index_count.saturating_add(new.total_distinct_index_count);
let total_distinct_remote_count =
this.total_distinct_remote_count.saturating_add(new.total_distinct_remote_count);
let total_single_index = this.total_single_index.saturating_add(new.total_single_index);
let total_search_count = this.total_search_count.saturating_add(new.total_search_count);
let show_ranking_score = this.show_ranking_score || new.show_ranking_score;
@ -121,6 +129,7 @@ impl Aggregate for MultiSearchAggregator {
total_received,
total_succeeded,
total_distinct_index_count,
total_distinct_remote_count,
total_single_index,
total_search_count,
show_ranking_score,
@ -134,6 +143,7 @@ impl Aggregate for MultiSearchAggregator {
total_received,
total_succeeded,
total_distinct_index_count,
total_distinct_remote_count,
total_single_index,
total_search_count,
show_ranking_score,
@ -152,6 +162,10 @@ impl Aggregate for MultiSearchAggregator {
"total_distinct_index_count": total_distinct_index_count,
"avg_distinct_index_count": (total_distinct_index_count as f64) / (total_received as f64), // not 0 else returned early
},
"remotes": {
"total_distinct_remote_count": total_distinct_remote_count,
"avg_distinct_remote_count": (total_distinct_remote_count as f64) / (total_received as f64), // not 0 else returned early
},
"searches": {
"total_search_count": total_search_count,
"avg_search_count": (total_search_count as f64) / (total_received as f64),

View file

@ -0,0 +1,261 @@
use std::collections::BTreeMap;
use actix_web::web::{self, Data};
use actix_web::{HttpRequest, HttpResponse};
use deserr::actix_web::AwebJson;
use deserr::Deserr;
use index_scheduler::IndexScheduler;
use itertools::{EitherOrBoth, Itertools};
use meilisearch_types::deserr::DeserrJsonError;
use meilisearch_types::error::deserr_codes::{
InvalidNetworkRemotes, InvalidNetworkSearchApiKey, InvalidNetworkSelf, InvalidNetworkUrl,
};
use meilisearch_types::error::ResponseError;
use meilisearch_types::features::{Network as DbNetwork, Remote as DbRemote};
use meilisearch_types::keys::actions;
use meilisearch_types::milli::update::Setting;
use serde::Serialize;
use tracing::debug;
use utoipa::{OpenApi, ToSchema};
use crate::analytics::{Aggregate, Analytics};
use crate::extractors::authentication::policies::ActionPolicy;
use crate::extractors::authentication::GuardedData;
use crate::extractors::sequential_extractor::SeqHandler;
#[derive(OpenApi)]
#[openapi(
paths(get_network, patch_network),
tags((
name = "Network",
description = "The `/network` route allows you to describe the topology of a network of Meilisearch instances.
This route is **synchronous**. This means that no task object will be returned, and any change to the network will be made available immediately.",
external_docs(url = "https://www.meilisearch.com/docs/reference/api/network"),
)),
)]
pub struct NetworkApi;
pub fn configure(cfg: &mut web::ServiceConfig) {
cfg.service(
web::resource("")
.route(web::get().to(get_network))
.route(web::patch().to(SeqHandler(patch_network))),
);
}
/// Get network topology
///
/// Get a list of all Meilisearch instances currently known to this instance.
#[utoipa::path(
get,
path = "",
tag = "Network",
security(("Bearer" = ["network.get", "network.*", "*"])),
responses(
(status = OK, description = "Known nodes are returned", body = Network, content_type = "application/json", example = json!(
{
"self": "ms-0",
"remotes": {
"ms-0": Remote { url: Setting::Set("http://localhost:7700".into()), search_api_key: Setting::Reset },
"ms-1": Remote { url: Setting::Set("http://localhost:7701".into()), search_api_key: Setting::Set("foo".into()) },
"ms-2": Remote { url: Setting::Set("http://localhost:7702".into()), search_api_key: Setting::Set("bar".into()) },
}
})),
(status = 401, description = "The authorization header is missing", body = ResponseError, content_type = "application/json", example = json!(
{
"message": "The Authorization header is missing. It must use the bearer authorization method.",
"code": "missing_authorization_header",
"type": "auth",
"link": "https://docs.meilisearch.com/errors#missing_authorization_header"
}
)),
)
)]
async fn get_network(
index_scheduler: GuardedData<ActionPolicy<{ actions::NETWORK_GET }>, Data<IndexScheduler>>,
) -> Result<HttpResponse, ResponseError> {
index_scheduler.features().check_network("Using the /network route")?;
let network = index_scheduler.network();
debug!(returns = ?network, "Get network");
Ok(HttpResponse::Ok().json(network))
}
#[derive(Debug, Deserr, ToSchema, Serialize)]
#[deserr(error = DeserrJsonError<InvalidNetworkRemotes>, rename_all = camelCase, deny_unknown_fields)]
#[serde(rename_all = "camelCase")]
#[schema(rename_all = "camelCase")]
pub struct Remote {
#[schema(value_type = Option<String>, example = json!({
"ms-0": Remote { url: Setting::Set("http://localhost:7700".into()), search_api_key: Setting::Reset },
"ms-1": Remote { url: Setting::Set("http://localhost:7701".into()), search_api_key: Setting::Set("foo".into()) },
"ms-2": Remote { url: Setting::Set("http://localhost:7702".into()), search_api_key: Setting::Set("bar".into()) },
}))]
#[deserr(default, error = DeserrJsonError<InvalidNetworkUrl>)]
#[serde(default)]
pub url: Setting<String>,
#[schema(value_type = Option<String>, example = json!("XWnBI8QHUc-4IlqbKPLUDuhftNq19mQtjc6JvmivzJU"))]
#[deserr(default, error = DeserrJsonError<InvalidNetworkSearchApiKey>)]
#[serde(default)]
pub search_api_key: Setting<String>,
}
#[derive(Debug, Deserr, ToSchema, Serialize)]
#[deserr(error = DeserrJsonError, rename_all = camelCase, deny_unknown_fields)]
#[serde(rename_all = "camelCase")]
#[schema(rename_all = "camelCase")]
pub struct Network {
#[schema(value_type = Option<BTreeMap<String, Remote>>, example = json!("http://localhost:7700"))]
#[deserr(default, error = DeserrJsonError<InvalidNetworkRemotes>)]
#[serde(default)]
pub remotes: Setting<BTreeMap<String, Option<Remote>>>,
#[schema(value_type = Option<String>, example = json!("ms-00"), rename = "self")]
#[serde(default, rename = "self")]
#[deserr(default, rename = "self", error = DeserrJsonError<InvalidNetworkSelf>)]
pub local: Setting<String>,
}
impl Remote {
pub fn try_into_db_node(self, name: &str) -> Result<DbRemote, ResponseError> {
Ok(DbRemote {
url: self.url.set().ok_or(ResponseError::from_msg(
format!("Missing field `.remotes.{name}.url`"),
meilisearch_types::error::Code::MissingNetworkUrl,
))?,
search_api_key: self.search_api_key.set(),
})
}
}
#[derive(Serialize)]
pub struct PatchNetworkAnalytics {
network_size: usize,
network_has_self: bool,
}
impl Aggregate for PatchNetworkAnalytics {
fn event_name(&self) -> &'static str {
"Network Updated"
}
fn aggregate(self: Box<Self>, new: Box<Self>) -> Box<Self> {
Box::new(Self { network_size: new.network_size, network_has_self: new.network_has_self })
}
fn into_event(self: Box<Self>) -> serde_json::Value {
serde_json::to_value(*self).unwrap_or_default()
}
}
/// Configure Network
///
/// Add or remove nodes from network.
#[utoipa::path(
patch,
path = "",
tag = "Network",
request_body = Network,
security(("Bearer" = ["network.update", "network.*", "*"])),
responses(
(status = OK, description = "New network state is returned", body = Network, content_type = "application/json", example = json!(
{
"self": "ms-0",
"remotes": {
"ms-0": Remote { url: Setting::Set("http://localhost:7700".into()), search_api_key: Setting::Reset },
"ms-1": Remote { url: Setting::Set("http://localhost:7701".into()), search_api_key: Setting::Set("foo".into()) },
"ms-2": Remote { url: Setting::Set("http://localhost:7702".into()), search_api_key: Setting::Set("bar".into()) },
}
})),
(status = 401, description = "The authorization header is missing", body = ResponseError, content_type = "application/json", example = json!(
{
"message": "The Authorization header is missing. It must use the bearer authorization method.",
"code": "missing_authorization_header",
"type": "auth",
"link": "https://docs.meilisearch.com/errors#missing_authorization_header"
}
)),
)
)]
async fn patch_network(
index_scheduler: GuardedData<ActionPolicy<{ actions::NETWORK_UPDATE }>, Data<IndexScheduler>>,
new_network: AwebJson<Network, DeserrJsonError>,
req: HttpRequest,
analytics: Data<Analytics>,
) -> Result<HttpResponse, ResponseError> {
index_scheduler.features().check_network("Using the /network route")?;
let new_network = new_network.0;
let old_network = index_scheduler.network();
debug!(parameters = ?new_network, "Patch network");
let merged_self = match new_network.local {
Setting::Set(new_self) => Some(new_self),
Setting::Reset => None,
Setting::NotSet => old_network.local,
};
let merged_remotes = match new_network.remotes {
Setting::Set(new_remotes) => {
let mut merged_remotes = BTreeMap::new();
for either_or_both in old_network
.remotes
.into_iter()
.merge_join_by(new_remotes.into_iter(), |left, right| left.0.cmp(&right.0))
{
match either_or_both {
EitherOrBoth::Both((key, old), (_, Some(new))) => {
let DbRemote { url: old_url, search_api_key: old_search_api_key } = old;
let Remote { url: new_url, search_api_key: new_search_api_key } = new;
let merged = DbRemote {
url: match new_url {
Setting::Set(new_url) => new_url,
Setting::Reset => {
return Err(ResponseError::from_msg(
format!(
"Field `.remotes.{key}.url` cannot be set to `null`"
),
meilisearch_types::error::Code::InvalidNetworkUrl,
))
}
Setting::NotSet => old_url,
},
search_api_key: match new_search_api_key {
Setting::Set(new_search_api_key) => Some(new_search_api_key),
Setting::Reset => None,
Setting::NotSet => old_search_api_key,
},
};
merged_remotes.insert(key, merged);
}
EitherOrBoth::Both((_, _), (_, None)) | EitherOrBoth::Right((_, None)) => {}
EitherOrBoth::Left((key, node)) => {
merged_remotes.insert(key, node);
}
EitherOrBoth::Right((key, Some(node))) => {
let node = node.try_into_db_node(&key)?;
merged_remotes.insert(key, node);
}
}
}
merged_remotes
}
Setting::Reset => BTreeMap::new(),
Setting::NotSet => old_network.remotes,
};
analytics.publish(
PatchNetworkAnalytics {
network_size: merged_remotes.len(),
network_has_self: merged_self.is_some(),
},
&req,
);
let merged_network = DbNetwork { local: merged_self, remotes: merged_remotes };
index_scheduler.put_network(merged_network.clone())?;
debug!(returns = ?merged_network, "Patch network");
Ok(HttpResponse::Ok().json(merged_network))
}

View file

@ -1,3 +1,5 @@
use std::io::ErrorKind;
use actix_web::web::Data;
use actix_web::{web, HttpRequest, HttpResponse};
use deserr::actix_web::AwebQueryParameter;
@ -16,6 +18,7 @@ use serde::Serialize;
use time::format_description::well_known::Rfc3339;
use time::macros::format_description;
use time::{Date, Duration, OffsetDateTime, Time};
use tokio::io::AsyncReadExt;
use tokio::task;
use utoipa::{IntoParams, OpenApi, ToSchema};
@ -44,7 +47,11 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
.route(web::delete().to(SeqHandler(delete_tasks))),
)
.service(web::resource("/cancel").route(web::post().to(SeqHandler(cancel_tasks))))
.service(web::resource("/{task_id}").route(web::get().to(SeqHandler(get_task))));
.service(web::resource("/{task_id}").route(web::get().to(SeqHandler(get_task))))
.service(
web::resource("/{task_id}/documents")
.route(web::get().to(SeqHandler(get_task_documents_file))),
);
}
#[derive(Debug, Deserr, IntoParams)]
@ -639,6 +646,76 @@ async fn get_task(
}
}
/// Get a task's documents.
///
/// Get a [task's documents file](https://www.meilisearch.com/docs/learn/async/asynchronous_operations).
#[utoipa::path(
get,
path = "/{taskUid}/documents",
tag = "Tasks",
security(("Bearer" = ["tasks.get", "tasks.*", "*"])),
params(("taskUid", format = UInt32, example = 0, description = "The task identifier", nullable = false)),
responses(
(status = 200, description = "The content of the task update", body = serde_json::Value, content_type = "application/x-ndjson"),
(status = 401, description = "The authorization header is missing", body = ResponseError, content_type = "application/json", example = json!(
{
"message": "The Authorization header is missing. It must use the bearer authorization method.",
"code": "missing_authorization_header",
"type": "auth",
"link": "https://docs.meilisearch.com/errors#missing_authorization_header"
}
)),
(status = 404, description = "The task uid does not exists", body = ResponseError, content_type = "application/json", example = json!(
{
"message": "Task :taskUid not found.",
"code": "task_not_found",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors/#task_not_found"
}
))
)
)]
async fn get_task_documents_file(
index_scheduler: GuardedData<ActionPolicy<{ actions::TASKS_GET }>, Data<IndexScheduler>>,
task_uid: web::Path<String>,
) -> Result<HttpResponse, ResponseError> {
index_scheduler.features().check_get_task_documents_route()?;
let task_uid_string = task_uid.into_inner();
let task_uid: TaskId = match task_uid_string.parse() {
Ok(id) => id,
Err(_e) => {
return Err(index_scheduler::Error::InvalidTaskUid { task_uid: task_uid_string }.into())
}
};
let query = index_scheduler::Query { uids: Some(vec![task_uid]), ..Query::default() };
let filters = index_scheduler.filters();
let (tasks, _) = index_scheduler.get_tasks_from_authorized_indexes(&query, filters)?;
if let Some(task) = tasks.first() {
match task.content_uuid() {
Some(uuid) => {
let mut tfile = match index_scheduler.queue.update_file(uuid) {
Ok(file) => tokio::fs::File::from_std(file),
Err(file_store::Error::IoError(e)) if e.kind() == ErrorKind::NotFound => {
return Err(index_scheduler::Error::TaskFileNotFound(task_uid).into())
}
Err(e) => return Err(e.into()),
};
// Yes, that's awful to put everything in memory when we could have streamed it from
// disk but it's really (really) complex to do with the current state of async Rust.
let mut content = String::new();
tfile.read_to_string(&mut content).await?;
Ok(HttpResponse::Ok().content_type("application/x-ndjson").body(content))
}
None => Err(index_scheduler::Error::TaskFileNotFound(task_uid).into()),
}
} else {
Err(index_scheduler::Error::TaskNotFound(task_uid).into())
}
}
pub enum DeserializeDateOption {
Before,
After,

View file

@ -1,923 +0,0 @@
use std::cmp::Ordering;
use std::collections::BTreeMap;
use std::fmt;
use std::iter::Zip;
use std::rc::Rc;
use std::str::FromStr as _;
use std::time::Duration;
use std::vec::{IntoIter, Vec};
use actix_http::StatusCode;
use index_scheduler::{IndexScheduler, RoFeatures};
use indexmap::IndexMap;
use meilisearch_types::deserr::DeserrJsonError;
use meilisearch_types::error::deserr_codes::{
InvalidMultiSearchFacetsByIndex, InvalidMultiSearchMaxValuesPerFacet,
InvalidMultiSearchMergeFacets, InvalidMultiSearchWeight, InvalidSearchLimit,
InvalidSearchOffset,
};
use meilisearch_types::error::ResponseError;
use meilisearch_types::index_uid::IndexUid;
use meilisearch_types::milli::score_details::{ScoreDetails, ScoreValue};
use meilisearch_types::milli::{self, DocumentId, OrderBy, TimeBudget};
use roaring::RoaringBitmap;
use serde::Serialize;
use utoipa::ToSchema;
use super::ranking_rules::{self, RankingRules};
use super::{
compute_facet_distribution_stats, prepare_search, AttributesFormat, ComputedFacets, FacetStats,
HitMaker, HitsInfo, RetrieveVectors, SearchHit, SearchKind, SearchQuery, SearchQueryWithIndex,
};
use crate::error::MeilisearchHttpError;
use crate::routes::indexes::search::search_kind;
pub const DEFAULT_FEDERATED_WEIGHT: f64 = 1.0;
#[derive(Debug, Default, Clone, Copy, PartialEq, deserr::Deserr, ToSchema)]
#[deserr(error = DeserrJsonError, rename_all = camelCase, deny_unknown_fields)]
pub struct FederationOptions {
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchWeight>)]
#[schema(value_type = f64)]
pub weight: Weight,
}
#[derive(Debug, Clone, Copy, PartialEq, deserr::Deserr)]
#[deserr(try_from(f64) = TryFrom::try_from -> InvalidMultiSearchWeight)]
pub struct Weight(f64);
impl Default for Weight {
fn default() -> Self {
Weight(DEFAULT_FEDERATED_WEIGHT)
}
}
impl std::convert::TryFrom<f64> for Weight {
type Error = InvalidMultiSearchWeight;
fn try_from(f: f64) -> Result<Self, Self::Error> {
if f < 0.0 {
Err(InvalidMultiSearchWeight)
} else {
Ok(Weight(f))
}
}
}
impl std::ops::Deref for Weight {
type Target = f64;
fn deref(&self) -> &Self::Target {
&self.0
}
}
#[derive(Debug, deserr::Deserr, ToSchema)]
#[deserr(error = DeserrJsonError, rename_all = camelCase, deny_unknown_fields)]
#[schema(rename_all = "camelCase")]
pub struct Federation {
#[deserr(default = super::DEFAULT_SEARCH_LIMIT(), error = DeserrJsonError<InvalidSearchLimit>)]
pub limit: usize,
#[deserr(default = super::DEFAULT_SEARCH_OFFSET(), error = DeserrJsonError<InvalidSearchOffset>)]
pub offset: usize,
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchFacetsByIndex>)]
pub facets_by_index: BTreeMap<IndexUid, Option<Vec<String>>>,
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchMergeFacets>)]
pub merge_facets: Option<MergeFacets>,
}
#[derive(Copy, Clone, Debug, deserr::Deserr, Default, ToSchema)]
#[deserr(error = DeserrJsonError<InvalidMultiSearchMergeFacets>, rename_all = camelCase, deny_unknown_fields)]
#[schema(rename_all = "camelCase")]
pub struct MergeFacets {
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchMaxValuesPerFacet>)]
pub max_values_per_facet: Option<usize>,
}
#[derive(Debug, deserr::Deserr, ToSchema)]
#[deserr(error = DeserrJsonError, rename_all = camelCase, deny_unknown_fields)]
#[schema(rename_all = "camelCase")]
pub struct FederatedSearch {
pub queries: Vec<SearchQueryWithIndex>,
#[deserr(default)]
pub federation: Option<Federation>,
}
#[derive(Serialize, Clone, ToSchema)]
#[serde(rename_all = "camelCase")]
#[schema(rename_all = "camelCase")]
pub struct FederatedSearchResult {
pub hits: Vec<SearchHit>,
pub processing_time_ms: u128,
#[serde(flatten)]
pub hits_info: HitsInfo,
#[serde(skip_serializing_if = "Option::is_none")]
pub semantic_hit_count: Option<u32>,
#[serde(skip_serializing_if = "Option::is_none")]
#[schema(value_type = Option<BTreeMap<String, BTreeMap<String, u64>>>)]
pub facet_distribution: Option<BTreeMap<String, IndexMap<String, u64>>>,
#[serde(skip_serializing_if = "Option::is_none")]
pub facet_stats: Option<BTreeMap<String, FacetStats>>,
#[serde(skip_serializing_if = "FederatedFacets::is_empty")]
pub facets_by_index: FederatedFacets,
// These fields are only used for analytics purposes
#[serde(skip)]
pub degraded: bool,
#[serde(skip)]
pub used_negative_operator: bool,
}
impl fmt::Debug for FederatedSearchResult {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
let FederatedSearchResult {
hits,
processing_time_ms,
hits_info,
semantic_hit_count,
degraded,
used_negative_operator,
facet_distribution,
facet_stats,
facets_by_index,
} = self;
let mut debug = f.debug_struct("SearchResult");
// The most important thing when looking at a search result is the time it took to process
debug.field("processing_time_ms", &processing_time_ms);
debug.field("hits", &format!("[{} hits returned]", hits.len()));
debug.field("hits_info", &hits_info);
if *used_negative_operator {
debug.field("used_negative_operator", used_negative_operator);
}
if *degraded {
debug.field("degraded", degraded);
}
if let Some(facet_distribution) = facet_distribution {
debug.field("facet_distribution", &facet_distribution);
}
if let Some(facet_stats) = facet_stats {
debug.field("facet_stats", &facet_stats);
}
if let Some(semantic_hit_count) = semantic_hit_count {
debug.field("semantic_hit_count", &semantic_hit_count);
}
if !facets_by_index.is_empty() {
debug.field("facets_by_index", &facets_by_index);
}
debug.finish()
}
}
struct WeightedScore<'a> {
details: &'a [ScoreDetails],
weight: f64,
}
impl<'a> WeightedScore<'a> {
pub fn new(details: &'a [ScoreDetails], weight: f64) -> Self {
Self { details, weight }
}
pub fn weighted_global_score(&self) -> f64 {
ScoreDetails::global_score(self.details.iter()) * self.weight
}
pub fn compare_weighted_global_scores(&self, other: &Self) -> Ordering {
self.weighted_global_score()
.partial_cmp(&other.weighted_global_score())
// both are numbers, possibly infinite
.unwrap()
}
pub fn compare(&self, other: &Self) -> Ordering {
let mut left_it = ScoreDetails::score_values(self.details.iter());
let mut right_it = ScoreDetails::score_values(other.details.iter());
loop {
let left = left_it.next();
let right = right_it.next();
match (left, right) {
(None, None) => return Ordering::Equal,
(None, Some(_)) => return Ordering::Less,
(Some(_), None) => return Ordering::Greater,
(Some(ScoreValue::Score(left)), Some(ScoreValue::Score(right))) => {
let left = left * self.weight;
let right = right * other.weight;
if (left - right).abs() <= f64::EPSILON {
continue;
}
return left.partial_cmp(&right).unwrap();
}
(Some(ScoreValue::Sort(left)), Some(ScoreValue::Sort(right))) => {
match left.partial_cmp(right) {
Some(Ordering::Equal) => continue,
Some(order) => return order,
None => return self.compare_weighted_global_scores(other),
}
}
(Some(ScoreValue::GeoSort(left)), Some(ScoreValue::GeoSort(right))) => {
match left.partial_cmp(right) {
Some(Ordering::Equal) => continue,
Some(order) => return order,
None => {
return self.compare_weighted_global_scores(other);
}
}
}
// not comparable details, use global
(Some(ScoreValue::Score(_)), Some(_))
| (Some(_), Some(ScoreValue::Score(_)))
| (Some(ScoreValue::GeoSort(_)), Some(ScoreValue::Sort(_)))
| (Some(ScoreValue::Sort(_)), Some(ScoreValue::GeoSort(_))) => {
let left_count = left_it.count();
let right_count = right_it.count();
// compare how many remaining groups of rules each side has.
// the group with the most remaining groups wins.
return left_count
.cmp(&right_count)
// breaks ties with the global ranking score
.then_with(|| self.compare_weighted_global_scores(other));
}
}
}
}
}
struct QueryByIndex {
query: SearchQuery,
federation_options: FederationOptions,
query_index: usize,
}
struct SearchResultByQuery<'a> {
documents_ids: Vec<DocumentId>,
document_scores: Vec<Vec<ScoreDetails>>,
federation_options: FederationOptions,
hit_maker: HitMaker<'a>,
query_index: usize,
}
struct SearchResultByQueryIter<'a> {
it: Zip<IntoIter<DocumentId>, IntoIter<Vec<ScoreDetails>>>,
federation_options: FederationOptions,
hit_maker: Rc<HitMaker<'a>>,
query_index: usize,
}
impl<'a> SearchResultByQueryIter<'a> {
fn new(
SearchResultByQuery {
documents_ids,
document_scores,
federation_options,
hit_maker,
query_index,
}: SearchResultByQuery<'a>,
) -> Self {
let it = documents_ids.into_iter().zip(document_scores);
Self { it, federation_options, hit_maker: Rc::new(hit_maker), query_index }
}
}
struct SearchResultByQueryIterItem<'a> {
docid: DocumentId,
score: Vec<ScoreDetails>,
federation_options: FederationOptions,
hit_maker: Rc<HitMaker<'a>>,
query_index: usize,
}
fn merge_index_local_results(
results_by_query: Vec<SearchResultByQuery<'_>>,
) -> impl Iterator<Item = SearchResultByQueryIterItem> + '_ {
itertools::kmerge_by(
results_by_query.into_iter().map(SearchResultByQueryIter::new),
|left: &SearchResultByQueryIterItem, right: &SearchResultByQueryIterItem| {
let left_score = WeightedScore::new(&left.score, *left.federation_options.weight);
let right_score = WeightedScore::new(&right.score, *right.federation_options.weight);
match left_score.compare(&right_score) {
// the biggest score goes first
Ordering::Greater => true,
// break ties using query index
Ordering::Equal => left.query_index < right.query_index,
Ordering::Less => false,
}
},
)
}
fn merge_index_global_results(
results_by_index: Vec<SearchResultByIndex>,
) -> impl Iterator<Item = SearchHitByIndex> {
itertools::kmerge_by(
results_by_index.into_iter().map(|result_by_index| result_by_index.hits.into_iter()),
|left: &SearchHitByIndex, right: &SearchHitByIndex| {
let left_score = WeightedScore::new(&left.score, *left.federation_options.weight);
let right_score = WeightedScore::new(&right.score, *right.federation_options.weight);
match left_score.compare(&right_score) {
// the biggest score goes first
Ordering::Greater => true,
// break ties using query index
Ordering::Equal => left.query_index < right.query_index,
Ordering::Less => false,
}
},
)
}
impl<'a> Iterator for SearchResultByQueryIter<'a> {
type Item = SearchResultByQueryIterItem<'a>;
fn next(&mut self) -> Option<Self::Item> {
let (docid, score) = self.it.next()?;
Some(SearchResultByQueryIterItem {
docid,
score,
federation_options: self.federation_options,
hit_maker: Rc::clone(&self.hit_maker),
query_index: self.query_index,
})
}
}
struct SearchHitByIndex {
hit: SearchHit,
score: Vec<ScoreDetails>,
federation_options: FederationOptions,
query_index: usize,
}
struct SearchResultByIndex {
index: String,
hits: Vec<SearchHitByIndex>,
estimated_total_hits: usize,
degraded: bool,
used_negative_operator: bool,
facets: Option<ComputedFacets>,
}
#[derive(Debug, Clone, Default, Serialize, ToSchema)]
pub struct FederatedFacets(pub BTreeMap<String, ComputedFacets>);
impl FederatedFacets {
pub fn insert(&mut self, index: String, facets: Option<ComputedFacets>) {
if let Some(facets) = facets {
self.0.insert(index, facets);
}
}
pub fn is_empty(&self) -> bool {
self.0.is_empty()
}
pub fn merge(
self,
MergeFacets { max_values_per_facet }: MergeFacets,
facet_order: BTreeMap<String, (String, OrderBy)>,
) -> Option<ComputedFacets> {
if self.is_empty() {
return None;
}
let mut distribution: BTreeMap<String, _> = Default::default();
let mut stats: BTreeMap<String, FacetStats> = Default::default();
for facets_by_index in self.0.into_values() {
for (facet, index_distribution) in facets_by_index.distribution {
match distribution.entry(facet) {
std::collections::btree_map::Entry::Vacant(entry) => {
entry.insert(index_distribution);
}
std::collections::btree_map::Entry::Occupied(mut entry) => {
let distribution = entry.get_mut();
for (value, index_count) in index_distribution {
distribution
.entry(value)
.and_modify(|count| *count += index_count)
.or_insert(index_count);
}
}
}
}
for (facet, index_stats) in facets_by_index.stats {
match stats.entry(facet) {
std::collections::btree_map::Entry::Vacant(entry) => {
entry.insert(index_stats);
}
std::collections::btree_map::Entry::Occupied(mut entry) => {
let stats = entry.get_mut();
stats.min = f64::min(stats.min, index_stats.min);
stats.max = f64::max(stats.max, index_stats.max);
}
}
}
}
// fixup order
for (facet, values) in &mut distribution {
let order_by = facet_order.get(facet).map(|(_, order)| *order).unwrap_or_default();
match order_by {
OrderBy::Lexicographic => {
values.sort_unstable_by(|left, _, right, _| left.cmp(right))
}
OrderBy::Count => {
values.sort_unstable_by(|_, left, _, right| {
left.cmp(right)
// biggest first
.reverse()
})
}
}
if let Some(max_values_per_facet) = max_values_per_facet {
values.truncate(max_values_per_facet)
};
}
Some(ComputedFacets { distribution, stats })
}
}
pub fn perform_federated_search(
index_scheduler: &IndexScheduler,
queries: Vec<SearchQueryWithIndex>,
mut federation: Federation,
features: RoFeatures,
) -> Result<FederatedSearchResult, ResponseError> {
let before_search = std::time::Instant::now();
// this implementation partition the queries by index to guarantee an important property:
// - all the queries to a particular index use the same read transaction.
// This is an important property, otherwise we cannot guarantee the self-consistency of the results.
// 1. partition queries by index
let mut queries_by_index: BTreeMap<String, Vec<QueryByIndex>> = Default::default();
for (query_index, federated_query) in queries.into_iter().enumerate() {
if let Some(pagination_field) = federated_query.has_pagination() {
return Err(MeilisearchHttpError::PaginationInFederatedQuery(
query_index,
pagination_field,
)
.into());
}
if let Some(facets) = federated_query.has_facets() {
let facets = facets.to_owned();
return Err(MeilisearchHttpError::FacetsInFederatedQuery(
query_index,
federated_query.index_uid.into_inner(),
facets,
)
.into());
}
let (index_uid, query, federation_options) = federated_query.into_index_query_federation();
queries_by_index.entry(index_uid.into_inner()).or_default().push(QueryByIndex {
query,
federation_options: federation_options.unwrap_or_default(),
query_index,
})
}
// 2. perform queries, merge and make hits index by index
let required_hit_count = federation.limit + federation.offset;
// In step (2), semantic_hit_count will be set to Some(0) if any search kind uses semantic
// Then in step (3), we'll update its value if there is any semantic search
let mut semantic_hit_count = None;
let mut results_by_index = Vec::with_capacity(queries_by_index.len());
let mut previous_query_data: Option<(RankingRules, usize, String)> = None;
// remember the order and name of first index for each facet when merging with index settings
// to detect if the order is inconsistent for a facet.
let mut facet_order: Option<BTreeMap<String, (String, OrderBy)>> = match federation.merge_facets
{
Some(MergeFacets { .. }) => Some(Default::default()),
_ => None,
};
for (index_uid, queries) in queries_by_index {
let first_query_index = queries.first().map(|query| query.query_index);
let index = match index_scheduler.index(&index_uid) {
Ok(index) => index,
Err(err) => {
let mut err = ResponseError::from(err);
// Patch the HTTP status code to 400 as it defaults to 404 for `index_not_found`, but
// here the resource not found is not part of the URL.
err.code = StatusCode::BAD_REQUEST;
if let Some(query_index) = first_query_index {
err.message = format!("Inside `.queries[{}]`: {}", query_index, err.message);
}
return Err(err);
}
};
// Important: this is the only transaction we'll use for this index during this federated search
let rtxn = index.read_txn()?;
let criteria = index.criteria(&rtxn)?;
let dictionary = index.dictionary(&rtxn)?;
let dictionary: Option<Vec<_>> =
dictionary.as_ref().map(|x| x.iter().map(String::as_str).collect());
let separators = index.allowed_separators(&rtxn)?;
let separators: Option<Vec<_>> =
separators.as_ref().map(|x| x.iter().map(String::as_str).collect());
// each query gets its individual cutoff
let cutoff = index.search_cutoff(&rtxn)?;
let mut degraded = false;
let mut used_negative_operator = false;
let mut candidates = RoaringBitmap::new();
let facets_by_index = federation.facets_by_index.remove(&index_uid).flatten();
// TODO: recover the max size + facets_by_index as return value of this function so as not to ask it for all queries
if let Err(mut error) =
check_facet_order(&mut facet_order, &index_uid, &facets_by_index, &index, &rtxn)
{
error.message = format!(
"Inside `.federation.facetsByIndex.{index_uid}`: {error}{}",
if let Some(query_index) = first_query_index {
format!("\n - Note: index `{index_uid}` used in `.queries[{query_index}]`")
} else {
Default::default()
}
);
return Err(error);
}
// 2.1. Compute all candidates for each query in the index
let mut results_by_query = Vec::with_capacity(queries.len());
for QueryByIndex { query, federation_options, query_index } in queries {
// use an immediately invoked lambda to capture the result without returning from the function
let res: Result<(), ResponseError> = (|| {
let search_kind =
search_kind(&query, index_scheduler, index_uid.to_string(), &index)?;
let canonicalization_kind = match (&search_kind, &query.q) {
(SearchKind::SemanticOnly { .. }, _) => {
ranking_rules::CanonicalizationKind::Vector
}
(_, Some(q)) if !q.is_empty() => ranking_rules::CanonicalizationKind::Keyword,
_ => ranking_rules::CanonicalizationKind::Placeholder,
};
let sort = if let Some(sort) = &query.sort {
let sorts: Vec<_> =
match sort.iter().map(|s| milli::AscDesc::from_str(s)).collect() {
Ok(sorts) => sorts,
Err(asc_desc_error) => {
return Err(milli::Error::from(milli::SortError::from(
asc_desc_error,
))
.into())
}
};
Some(sorts)
} else {
None
};
let ranking_rules = ranking_rules::RankingRules::new(
criteria.clone(),
sort,
query.matching_strategy.into(),
canonicalization_kind,
);
if let Some((previous_ranking_rules, previous_query_index, previous_index_uid)) =
previous_query_data.take()
{
if let Err(error) = ranking_rules.is_compatible_with(&previous_ranking_rules) {
return Err(error.to_response_error(
&ranking_rules,
&previous_ranking_rules,
query_index,
previous_query_index,
&index_uid,
&previous_index_uid,
));
}
previous_query_data = if previous_ranking_rules.constraint_count()
> ranking_rules.constraint_count()
{
Some((previous_ranking_rules, previous_query_index, previous_index_uid))
} else {
Some((ranking_rules, query_index, index_uid.clone()))
};
} else {
previous_query_data = Some((ranking_rules, query_index, index_uid.clone()));
}
match search_kind {
SearchKind::KeywordOnly => {}
_ => semantic_hit_count = Some(0),
}
let retrieve_vectors = RetrieveVectors::new(query.retrieve_vectors);
let time_budget = match cutoff {
Some(cutoff) => TimeBudget::new(Duration::from_millis(cutoff)),
None => TimeBudget::default(),
};
let (mut search, _is_finite_pagination, _max_total_hits, _offset) =
prepare_search(&index, &rtxn, &query, &search_kind, time_budget, features)?;
search.scoring_strategy(milli::score_details::ScoringStrategy::Detailed);
search.offset(0);
search.limit(required_hit_count);
let (result, _semantic_hit_count) =
super::search_from_kind(index_uid.to_string(), search_kind, search)?;
let format = AttributesFormat {
attributes_to_retrieve: query.attributes_to_retrieve,
retrieve_vectors,
attributes_to_highlight: query.attributes_to_highlight,
attributes_to_crop: query.attributes_to_crop,
crop_length: query.crop_length,
crop_marker: query.crop_marker,
highlight_pre_tag: query.highlight_pre_tag,
highlight_post_tag: query.highlight_post_tag,
show_matches_position: query.show_matches_position,
sort: query.sort,
show_ranking_score: query.show_ranking_score,
show_ranking_score_details: query.show_ranking_score_details,
locales: query.locales.map(|l| l.iter().copied().map(Into::into).collect()),
};
let milli::SearchResult {
matching_words,
candidates: query_candidates,
documents_ids,
document_scores,
degraded: query_degraded,
used_negative_operator: query_used_negative_operator,
} = result;
candidates |= query_candidates;
degraded |= query_degraded;
used_negative_operator |= query_used_negative_operator;
let tokenizer = HitMaker::tokenizer(dictionary.as_deref(), separators.as_deref());
let formatter_builder = HitMaker::formatter_builder(matching_words, tokenizer);
let hit_maker =
HitMaker::new(&index, &rtxn, format, formatter_builder).map_err(|e| {
MeilisearchHttpError::from_milli(e, Some(index_uid.to_string()))
})?;
results_by_query.push(SearchResultByQuery {
federation_options,
hit_maker,
query_index,
documents_ids,
document_scores,
});
Ok(())
})();
if let Err(mut error) = res {
error.message = format!("Inside `.queries[{query_index}]`: {}", error.message);
return Err(error);
}
}
// 2.2. merge inside index
let mut documents_seen = RoaringBitmap::new();
let merged_result: Result<Vec<_>, ResponseError> =
merge_index_local_results(results_by_query)
// skip documents we've already seen & mark that we saw the current document
.filter(|SearchResultByQueryIterItem { docid, .. }| documents_seen.insert(*docid))
.take(required_hit_count)
// 2.3 make hits
.map(
|SearchResultByQueryIterItem {
docid,
score,
federation_options,
hit_maker,
query_index,
}| {
let mut hit = hit_maker.make_hit(docid, &score)?;
let weighted_score =
ScoreDetails::global_score(score.iter()) * (*federation_options.weight);
let _federation = serde_json::json!(
{
"indexUid": index_uid,
"queriesPosition": query_index,
"weightedRankingScore": weighted_score,
}
);
hit.document.insert("_federation".to_string(), _federation);
Ok(SearchHitByIndex { hit, score, federation_options, query_index })
},
)
.collect();
let merged_result = merged_result?;
let estimated_total_hits = candidates.len() as usize;
let facets = facets_by_index
.map(|facets_by_index| {
compute_facet_distribution_stats(
&facets_by_index,
&index,
&rtxn,
candidates,
super::Route::MultiSearch,
)
})
.transpose()
.map_err(|mut error| {
error.message = format!(
"Inside `.federation.facetsByIndex.{index_uid}`: {}{}",
error.message,
if let Some(query_index) = first_query_index {
format!("\n - Note: index `{index_uid}` used in `.queries[{query_index}]`")
} else {
Default::default()
}
);
error
})?;
results_by_index.push(SearchResultByIndex {
index: index_uid,
hits: merged_result,
estimated_total_hits,
degraded,
used_negative_operator,
facets,
});
}
// bonus step, make sure to return an error if an index wants a non-faceted field, even if no query actually uses that index.
for (index_uid, facets) in federation.facets_by_index {
let index = match index_scheduler.index(&index_uid) {
Ok(index) => index,
Err(err) => {
let mut err = ResponseError::from(err);
// Patch the HTTP status code to 400 as it defaults to 404 for `index_not_found`, but
// here the resource not found is not part of the URL.
err.code = StatusCode::BAD_REQUEST;
err.message = format!(
"Inside `.federation.facetsByIndex.{index_uid}`: {}\n - Note: index `{index_uid}` is not used in queries",
err.message
);
return Err(err);
}
};
// Important: this is the only transaction we'll use for this index during this federated search
let rtxn = index.read_txn()?;
if let Err(mut error) =
check_facet_order(&mut facet_order, &index_uid, &facets, &index, &rtxn)
{
error.message = format!(
"Inside `.federation.facetsByIndex.{index_uid}`: {error}\n - Note: index `{index_uid}` is not used in queries",
);
return Err(error);
}
if let Some(facets) = facets {
if let Err(mut error) = compute_facet_distribution_stats(
&facets,
&index,
&rtxn,
Default::default(),
super::Route::MultiSearch,
) {
error.message =
format!("Inside `.federation.facetsByIndex.{index_uid}`: {}\n - Note: index `{index_uid}` is not used in queries", error.message);
return Err(error);
}
}
}
// 3. merge hits and metadata across indexes
// 3.1 merge metadata
let (estimated_total_hits, degraded, used_negative_operator, facets) = {
let mut estimated_total_hits = 0;
let mut degraded = false;
let mut used_negative_operator = false;
let mut facets: FederatedFacets = FederatedFacets::default();
for SearchResultByIndex {
index,
hits: _,
estimated_total_hits: estimated_total_hits_by_index,
facets: facets_by_index,
degraded: degraded_by_index,
used_negative_operator: used_negative_operator_by_index,
} in &mut results_by_index
{
estimated_total_hits += *estimated_total_hits_by_index;
degraded |= *degraded_by_index;
used_negative_operator |= *used_negative_operator_by_index;
let facets_by_index = std::mem::take(facets_by_index);
let index = std::mem::take(index);
facets.insert(index, facets_by_index);
}
(estimated_total_hits, degraded, used_negative_operator, facets)
};
// 3.2 merge hits
let merged_hits: Vec<_> = merge_index_global_results(results_by_index)
.skip(federation.offset)
.take(federation.limit)
.inspect(|hit| {
if let Some(semantic_hit_count) = &mut semantic_hit_count {
if hit.score.iter().any(|score| matches!(&score, ScoreDetails::Vector(_))) {
*semantic_hit_count += 1;
}
}
})
.map(|hit| hit.hit)
.collect();
let (facet_distribution, facet_stats, facets_by_index) =
match federation.merge_facets.zip(facet_order) {
Some((merge_facets, facet_order)) => {
let facets = facets.merge(merge_facets, facet_order);
let (facet_distribution, facet_stats) = facets
.map(|ComputedFacets { distribution, stats }| (distribution, stats))
.unzip();
(facet_distribution, facet_stats, FederatedFacets::default())
}
None => (None, None, facets),
};
let search_result = FederatedSearchResult {
hits: merged_hits,
processing_time_ms: before_search.elapsed().as_millis(),
hits_info: HitsInfo::OffsetLimit {
limit: federation.limit,
offset: federation.offset,
estimated_total_hits,
},
semantic_hit_count,
degraded,
used_negative_operator,
facet_distribution,
facet_stats,
facets_by_index,
};
Ok(search_result)
}
fn check_facet_order(
facet_order: &mut Option<BTreeMap<String, (String, OrderBy)>>,
current_index: &str,
facets_by_index: &Option<Vec<String>>,
index: &milli::Index,
rtxn: &milli::heed::RoTxn<'_>,
) -> Result<(), ResponseError> {
if let (Some(facet_order), Some(facets_by_index)) = (facet_order, facets_by_index) {
let index_facet_order = index.sort_facet_values_by(rtxn)?;
for facet in facets_by_index {
let index_facet_order = index_facet_order.get(facet);
let (previous_index, previous_facet_order) = facet_order
.entry(facet.to_owned())
.or_insert_with(|| (current_index.to_owned(), index_facet_order));
if previous_facet_order != &index_facet_order {
return Err(MeilisearchHttpError::InconsistentFacetOrder {
facet: facet.clone(),
previous_facet_order: *previous_facet_order,
previous_uid: previous_index.clone(),
current_uid: current_index.to_owned(),
index_facet_order,
}
.into());
}
}
};
Ok(())
}

View file

@ -0,0 +1,10 @@
mod perform;
mod proxy;
mod types;
mod weighted_scores;
pub use perform::perform_federated_search;
pub use proxy::{PROXY_SEARCH_HEADER, PROXY_SEARCH_HEADER_VALUE};
pub use types::{
FederatedSearch, FederatedSearchResult, Federation, FederationOptions, MergeFacets,
};

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,267 @@
pub use error::ProxySearchError;
use error::ReqwestErrorWithoutUrl;
use meilisearch_types::features::Remote;
use rand::Rng as _;
use reqwest::{Client, Response, StatusCode};
use serde::de::DeserializeOwned;
use serde_json::Value;
use super::types::{FederatedSearch, FederatedSearchResult, Federation};
use crate::search::SearchQueryWithIndex;
pub const PROXY_SEARCH_HEADER: &str = "Meili-Proxy-Search";
pub const PROXY_SEARCH_HEADER_VALUE: &str = "true";
mod error {
use meilisearch_types::error::ResponseError;
use reqwest::StatusCode;
#[derive(Debug, thiserror::Error)]
pub enum ProxySearchError {
#[error("{0}")]
CouldNotSendRequest(ReqwestErrorWithoutUrl),
#[error("could not authenticate against the remote host\n - hint: check that the remote instance was registered with a valid API key having the `search` action")]
AuthenticationError,
#[error(
"could not parse response from the remote host as a federated search response{}\n - hint: check that the remote instance is a Meilisearch instance running the same version",
response_from_remote(response)
)]
CouldNotParseResponse { response: Result<String, ReqwestErrorWithoutUrl> },
#[error("remote host responded with code {}{}\n - hint: check that the remote instance has the correct index configuration for that request\n - hint: check that the `network` experimental feature is enabled on the remote instance", status_code.as_u16(), response_from_remote(response))]
BadRequest { status_code: StatusCode, response: Result<String, ReqwestErrorWithoutUrl> },
#[error("remote host did not answer before the deadline")]
Timeout,
#[error("remote hit does not contain `{0}`\n - hint: check that the remote instance is a Meilisearch instance running the same version")]
MissingPathInResponse(&'static str),
#[error("remote host responded with code {}{}", status_code.as_u16(), response_from_remote(response))]
RemoteError { status_code: StatusCode, response: Result<String, ReqwestErrorWithoutUrl> },
#[error("remote hit contains an unexpected value at path `{path}`: expected {expected_type}, received `{received_value}`\n - hint: check that the remote instance is a Meilisearch instance running the same version")]
UnexpectedValueInPath {
path: &'static str,
expected_type: &'static str,
received_value: String,
},
#[error("could not parse weighted score values in the remote hit: {0}")]
CouldNotParseWeightedScoreValues(serde_json::Error),
}
impl ProxySearchError {
pub fn as_response_error(&self) -> ResponseError {
use meilisearch_types::error::Code;
let message = self.to_string();
let code = match self {
ProxySearchError::CouldNotSendRequest(_) => Code::RemoteCouldNotSendRequest,
ProxySearchError::AuthenticationError => Code::RemoteInvalidApiKey,
ProxySearchError::BadRequest { .. } => Code::RemoteBadRequest,
ProxySearchError::Timeout => Code::RemoteTimeout,
ProxySearchError::RemoteError { .. } => Code::RemoteRemoteError,
ProxySearchError::CouldNotParseResponse { .. }
| ProxySearchError::MissingPathInResponse(_)
| ProxySearchError::UnexpectedValueInPath { .. }
| ProxySearchError::CouldNotParseWeightedScoreValues(_) => Code::RemoteBadResponse,
};
ResponseError::from_msg(message, code)
}
}
#[derive(Debug, thiserror::Error)]
#[error(transparent)]
pub struct ReqwestErrorWithoutUrl(reqwest::Error);
impl ReqwestErrorWithoutUrl {
pub fn new(inner: reqwest::Error) -> Self {
Self(inner.without_url())
}
}
fn response_from_remote(response: &Result<String, ReqwestErrorWithoutUrl>) -> String {
match response {
Ok(response) => {
format!(":\n - response from remote: {}", response)
}
Err(error) => {
format!(":\n - additionally, could not retrieve response from remote: {error}")
}
}
}
}
#[derive(Clone)]
pub struct ProxySearchParams {
pub deadline: Option<std::time::Instant>,
pub try_count: u32,
pub client: reqwest::Client,
}
/// Performs a federated search on a remote host and returns the results
pub async fn proxy_search(
node: &Remote,
queries: Vec<SearchQueryWithIndex>,
federation: Federation,
params: &ProxySearchParams,
) -> Result<FederatedSearchResult, ProxySearchError> {
let url = format!("{}/multi-search", node.url);
let federated = FederatedSearch { queries, federation: Some(federation) };
let search_api_key = node.search_api_key.as_deref();
let max_deadline = std::time::Instant::now() + std::time::Duration::from_secs(5);
let deadline = if let Some(deadline) = params.deadline {
std::time::Instant::min(deadline, max_deadline)
} else {
max_deadline
};
for i in 0..params.try_count {
match try_proxy_search(&url, search_api_key, &federated, &params.client, deadline).await {
Ok(response) => return Ok(response),
Err(retry) => {
let duration = retry.into_duration(i)?;
tokio::time::sleep(duration).await;
}
}
}
try_proxy_search(&url, search_api_key, &federated, &params.client, deadline)
.await
.map_err(Retry::into_error)
}
async fn try_proxy_search(
url: &str,
search_api_key: Option<&str>,
federated: &FederatedSearch,
client: &Client,
deadline: std::time::Instant,
) -> Result<FederatedSearchResult, Retry> {
let timeout = deadline.saturating_duration_since(std::time::Instant::now());
let request = client.post(url).json(&federated).timeout(timeout);
let request = if let Some(search_api_key) = search_api_key {
request.bearer_auth(search_api_key)
} else {
request
};
let request = request.header(PROXY_SEARCH_HEADER, PROXY_SEARCH_HEADER_VALUE);
let response = request.send().await;
let response = match response {
Ok(response) => response,
Err(error) if error.is_timeout() => return Err(Retry::give_up(ProxySearchError::Timeout)),
Err(error) => {
return Err(Retry::retry_later(ProxySearchError::CouldNotSendRequest(
ReqwestErrorWithoutUrl::new(error),
)))
}
};
match response.status() {
status_code if status_code.is_success() => (),
StatusCode::UNAUTHORIZED | StatusCode::FORBIDDEN => {
return Err(Retry::give_up(ProxySearchError::AuthenticationError))
}
status_code if status_code.is_client_error() => {
let response = parse_error(response).await;
return Err(Retry::give_up(ProxySearchError::BadRequest { status_code, response }));
}
status_code if status_code.is_server_error() => {
let response = parse_error(response).await;
return Err(Retry::retry_later(ProxySearchError::RemoteError {
status_code,
response,
}));
}
status_code => {
tracing::warn!(
status_code = status_code.as_u16(),
"remote replied with unexpected status code"
);
}
}
let response = match parse_response(response).await {
Ok(response) => response,
Err(response) => {
return Err(Retry::retry_later(ProxySearchError::CouldNotParseResponse { response }))
}
};
Ok(response)
}
/// Always parse the body of the response of a failed request as JSON.
async fn parse_error(response: Response) -> Result<String, ReqwestErrorWithoutUrl> {
let bytes = match response.bytes().await {
Ok(bytes) => bytes,
Err(error) => return Err(ReqwestErrorWithoutUrl::new(error)),
};
Ok(parse_bytes_as_error(&bytes))
}
fn parse_bytes_as_error(bytes: &[u8]) -> String {
match serde_json::from_slice::<Value>(bytes) {
Ok(value) => value.to_string(),
Err(_) => String::from_utf8_lossy(bytes).into_owned(),
}
}
async fn parse_response<T: DeserializeOwned>(
response: Response,
) -> Result<T, Result<String, ReqwestErrorWithoutUrl>> {
let bytes = match response.bytes().await {
Ok(bytes) => bytes,
Err(error) => return Err(Err(ReqwestErrorWithoutUrl::new(error))),
};
match serde_json::from_slice::<T>(&bytes) {
Ok(value) => Ok(value),
Err(_) => Err(Ok(parse_bytes_as_error(&bytes))),
}
}
pub struct Retry {
error: ProxySearchError,
strategy: RetryStrategy,
}
pub enum RetryStrategy {
GiveUp,
Retry,
}
impl Retry {
pub fn give_up(error: ProxySearchError) -> Self {
Self { error, strategy: RetryStrategy::GiveUp }
}
pub fn retry_later(error: ProxySearchError) -> Self {
Self { error, strategy: RetryStrategy::Retry }
}
pub fn into_duration(self, attempt: u32) -> Result<std::time::Duration, ProxySearchError> {
match self.strategy {
RetryStrategy::GiveUp => Err(self.error),
RetryStrategy::Retry => {
let retry_duration = std::time::Duration::from_nanos((10u64).pow(attempt));
let retry_duration = retry_duration.min(std::time::Duration::from_millis(100)); // don't wait more than 100ms
// randomly up to double the retry duration
let retry_duration = retry_duration
+ rand::thread_rng().gen_range(std::time::Duration::ZERO..retry_duration);
tracing::warn!(
"Attempt #{}, failed with {}, retrying after {}ms.",
attempt,
self.error,
retry_duration.as_millis()
);
Ok(retry_duration)
}
}
}
pub fn into_error(self) -> ProxySearchError {
self.error
}
}

View file

@ -0,0 +1,322 @@
use std::collections::btree_map::Entry;
use std::collections::BTreeMap;
use std::fmt;
use std::vec::Vec;
use indexmap::IndexMap;
use meilisearch_types::deserr::DeserrJsonError;
use meilisearch_types::error::deserr_codes::{
InvalidMultiSearchFacetsByIndex, InvalidMultiSearchMaxValuesPerFacet,
InvalidMultiSearchMergeFacets, InvalidMultiSearchQueryPosition, InvalidMultiSearchRemote,
InvalidMultiSearchWeight, InvalidSearchLimit, InvalidSearchOffset,
};
use meilisearch_types::error::ResponseError;
use meilisearch_types::index_uid::IndexUid;
use meilisearch_types::milli::order_by_map::OrderByMap;
use meilisearch_types::milli::OrderBy;
use serde::{Deserialize, Serialize};
use utoipa::ToSchema;
use super::super::{ComputedFacets, FacetStats, HitsInfo, SearchHit, SearchQueryWithIndex};
pub const DEFAULT_FEDERATED_WEIGHT: f64 = 1.0;
// fields in the response
pub const FEDERATION_HIT: &str = "_federation";
pub const INDEX_UID: &str = "indexUid";
pub const QUERIES_POSITION: &str = "queriesPosition";
pub const WEIGHTED_RANKING_SCORE: &str = "weightedRankingScore";
pub const WEIGHTED_SCORE_VALUES: &str = "weightedScoreValues";
pub const FEDERATION_REMOTE: &str = "remote";
#[derive(Debug, Default, Clone, PartialEq, Serialize, deserr::Deserr, ToSchema)]
#[deserr(error = DeserrJsonError, rename_all = camelCase, deny_unknown_fields)]
#[serde(rename_all = "camelCase")]
pub struct FederationOptions {
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchWeight>)]
#[schema(value_type = f64)]
pub weight: Weight,
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchRemote>)]
pub remote: Option<String>,
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchQueryPosition>)]
pub query_position: Option<usize>,
}
#[derive(Debug, Clone, Copy, PartialEq, Serialize, deserr::Deserr)]
#[deserr(try_from(f64) = TryFrom::try_from -> InvalidMultiSearchWeight)]
pub struct Weight(f64);
impl Default for Weight {
fn default() -> Self {
Weight(DEFAULT_FEDERATED_WEIGHT)
}
}
impl std::convert::TryFrom<f64> for Weight {
type Error = InvalidMultiSearchWeight;
fn try_from(f: f64) -> Result<Self, Self::Error> {
if f < 0.0 {
Err(InvalidMultiSearchWeight)
} else {
Ok(Weight(f))
}
}
}
impl std::ops::Deref for Weight {
type Target = f64;
fn deref(&self) -> &Self::Target {
&self.0
}
}
#[derive(Debug, Clone, deserr::Deserr, Serialize, ToSchema)]
#[deserr(error = DeserrJsonError, rename_all = camelCase, deny_unknown_fields)]
#[schema(rename_all = "camelCase")]
#[serde(rename_all = "camelCase")]
pub struct Federation {
#[deserr(default = super::super::DEFAULT_SEARCH_LIMIT(), error = DeserrJsonError<InvalidSearchLimit>)]
pub limit: usize,
#[deserr(default = super::super::DEFAULT_SEARCH_OFFSET(), error = DeserrJsonError<InvalidSearchOffset>)]
pub offset: usize,
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchFacetsByIndex>)]
pub facets_by_index: BTreeMap<IndexUid, Option<Vec<String>>>,
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchMergeFacets>)]
pub merge_facets: Option<MergeFacets>,
}
#[derive(Copy, Clone, Debug, deserr::Deserr, Serialize, Default, ToSchema)]
#[deserr(error = DeserrJsonError<InvalidMultiSearchMergeFacets>, rename_all = camelCase, deny_unknown_fields)]
#[schema(rename_all = "camelCase")]
#[serde(rename_all = "camelCase")]
pub struct MergeFacets {
#[deserr(default, error = DeserrJsonError<InvalidMultiSearchMaxValuesPerFacet>)]
pub max_values_per_facet: Option<usize>,
}
#[derive(Debug, deserr::Deserr, Serialize, ToSchema)]
#[deserr(error = DeserrJsonError, rename_all = camelCase, deny_unknown_fields)]
#[schema(rename_all = "camelCase")]
#[serde(rename_all = "camelCase")]
pub struct FederatedSearch {
pub queries: Vec<SearchQueryWithIndex>,
#[deserr(default)]
pub federation: Option<Federation>,
}
#[derive(Serialize, Deserialize, Clone, ToSchema)]
#[serde(rename_all = "camelCase")]
#[schema(rename_all = "camelCase")]
pub struct FederatedSearchResult {
pub hits: Vec<SearchHit>,
pub processing_time_ms: u128,
#[serde(flatten)]
pub hits_info: HitsInfo,
#[serde(default, skip_serializing_if = "Option::is_none")]
pub semantic_hit_count: Option<u32>,
#[serde(default, skip_serializing_if = "Option::is_none")]
#[schema(value_type = Option<BTreeMap<String, BTreeMap<String, u64>>>)]
pub facet_distribution: Option<BTreeMap<String, IndexMap<String, u64>>>,
#[serde(default, skip_serializing_if = "Option::is_none")]
pub facet_stats: Option<BTreeMap<String, FacetStats>>,
#[serde(default, skip_serializing_if = "FederatedFacets::is_empty")]
pub facets_by_index: FederatedFacets,
#[serde(default, skip_serializing_if = "Option::is_none")]
pub remote_errors: Option<BTreeMap<String, ResponseError>>,
// These fields are only used for analytics purposes
#[serde(skip)]
pub degraded: bool,
#[serde(skip)]
pub used_negative_operator: bool,
}
impl fmt::Debug for FederatedSearchResult {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
let FederatedSearchResult {
hits,
processing_time_ms,
hits_info,
semantic_hit_count,
degraded,
used_negative_operator,
facet_distribution,
facet_stats,
facets_by_index,
remote_errors,
} = self;
let mut debug = f.debug_struct("SearchResult");
// The most important thing when looking at a search result is the time it took to process
debug.field("processing_time_ms", &processing_time_ms);
debug.field("hits", &format!("[{} hits returned]", hits.len()));
debug.field("hits_info", &hits_info);
if *used_negative_operator {
debug.field("used_negative_operator", used_negative_operator);
}
if *degraded {
debug.field("degraded", degraded);
}
if let Some(facet_distribution) = facet_distribution {
debug.field("facet_distribution", &facet_distribution);
}
if let Some(facet_stats) = facet_stats {
debug.field("facet_stats", &facet_stats);
}
if let Some(semantic_hit_count) = semantic_hit_count {
debug.field("semantic_hit_count", &semantic_hit_count);
}
if !facets_by_index.is_empty() {
debug.field("facets_by_index", &facets_by_index);
}
if let Some(remote_errors) = remote_errors {
debug.field("remote_errors", &remote_errors);
}
debug.finish()
}
}
#[derive(Debug, Clone, Default, Serialize, Deserialize, ToSchema)]
pub struct FederatedFacets(pub BTreeMap<String, ComputedFacets>);
impl FederatedFacets {
pub fn insert(&mut self, index: String, facets: Option<ComputedFacets>) {
if let Some(facets) = facets {
self.0.insert(index, facets);
}
}
pub fn is_empty(&self) -> bool {
self.0.is_empty()
}
pub fn merge(
self,
MergeFacets { max_values_per_facet }: MergeFacets,
facet_order: BTreeMap<String, (String, OrderBy)>,
) -> Option<ComputedFacets> {
if self.is_empty() {
return None;
}
let mut distribution: BTreeMap<String, _> = Default::default();
let mut stats: BTreeMap<String, FacetStats> = Default::default();
for facets_by_index in self.0.into_values() {
for (facet, index_distribution) in facets_by_index.distribution {
match distribution.entry(facet) {
Entry::Vacant(entry) => {
entry.insert(index_distribution);
}
Entry::Occupied(mut entry) => {
let distribution = entry.get_mut();
for (value, index_count) in index_distribution {
distribution
.entry(value)
.and_modify(|count| *count += index_count)
.or_insert(index_count);
}
}
}
}
for (facet, index_stats) in facets_by_index.stats {
match stats.entry(facet) {
Entry::Vacant(entry) => {
entry.insert(index_stats);
}
Entry::Occupied(mut entry) => {
let stats = entry.get_mut();
stats.min = f64::min(stats.min, index_stats.min);
stats.max = f64::max(stats.max, index_stats.max);
}
}
}
}
// fixup order
for (facet, values) in &mut distribution {
let order_by = facet_order.get(facet).map(|(_, order)| *order).unwrap_or_default();
match order_by {
OrderBy::Lexicographic => {
values.sort_unstable_by(|left, _, right, _| left.cmp(right))
}
OrderBy::Count => {
values.sort_unstable_by(|_, left, _, right| {
left.cmp(right)
// biggest first
.reverse()
})
}
}
if let Some(max_values_per_facet) = max_values_per_facet {
values.truncate(max_values_per_facet)
};
}
Some(ComputedFacets { distribution, stats })
}
pub(crate) fn append(&mut self, FederatedFacets(remote_facets_by_index): FederatedFacets) {
for (index, remote_facets) in remote_facets_by_index {
let merged_facets = self.0.entry(index).or_default();
for (remote_facet, remote_stats) in remote_facets.stats {
match merged_facets.stats.entry(remote_facet) {
Entry::Vacant(vacant_entry) => {
vacant_entry.insert(remote_stats);
}
Entry::Occupied(mut occupied_entry) => {
let stats = occupied_entry.get_mut();
stats.min = f64::min(stats.min, remote_stats.min);
stats.max = f64::max(stats.max, remote_stats.max);
}
}
}
for (remote_facet, remote_values) in remote_facets.distribution {
let merged_facet = merged_facets.distribution.entry(remote_facet).or_default();
for (remote_value, remote_count) in remote_values {
let count = merged_facet.entry(remote_value).or_default();
*count += remote_count;
}
}
}
}
pub fn sort_and_truncate(&mut self, facet_order: BTreeMap<String, (OrderByMap, usize)>) {
for (index, facets) in &mut self.0 {
let Some((order_by, max_values_per_facet)) = facet_order.get(index) else {
continue;
};
for (facet, values) in &mut facets.distribution {
match order_by.get(facet) {
OrderBy::Lexicographic => {
values.sort_unstable_by(|left, _, right, _| left.cmp(right))
}
OrderBy::Count => {
values.sort_unstable_by(|_, left, _, right| {
left.cmp(right)
// biggest first
.reverse()
})
}
}
values.truncate(*max_values_per_facet);
}
}
}
}

View file

@ -0,0 +1,88 @@
use std::cmp::Ordering;
use meilisearch_types::milli::score_details::{self, WeightedScoreValue};
pub fn compare(
mut left_it: impl Iterator<Item = WeightedScoreValue>,
left_weighted_global_score: f64,
mut right_it: impl Iterator<Item = WeightedScoreValue>,
right_weighted_global_score: f64,
) -> Ordering {
loop {
let left = left_it.next();
let right = right_it.next();
match (left, right) {
(None, None) => return Ordering::Equal,
(None, Some(_)) => return Ordering::Less,
(Some(_), None) => return Ordering::Greater,
(
Some(
WeightedScoreValue::WeightedScore(left) | WeightedScoreValue::VectorSort(left),
),
Some(
WeightedScoreValue::WeightedScore(right)
| WeightedScoreValue::VectorSort(right),
),
) => {
if (left - right).abs() <= f64::EPSILON {
continue;
}
return left.partial_cmp(&right).unwrap();
}
(
Some(WeightedScoreValue::Sort { asc: left_asc, value: left }),
Some(WeightedScoreValue::Sort { asc: right_asc, value: right }),
) => {
if left_asc != right_asc {
return left_weighted_global_score
.partial_cmp(&right_weighted_global_score)
.unwrap();
}
match score_details::compare_sort_values(left_asc, &left, &right) {
Ordering::Equal => continue,
order => return order,
}
}
(
Some(WeightedScoreValue::GeoSort { asc: left_asc, distance: left }),
Some(WeightedScoreValue::GeoSort { asc: right_asc, distance: right }),
) => {
if left_asc != right_asc {
continue;
}
match (left, right) {
(None, None) => continue,
(None, Some(_)) => return Ordering::Less,
(Some(_), None) => return Ordering::Greater,
(Some(left), Some(right)) => {
if (left - right).abs() <= f64::EPSILON {
continue;
}
return left.partial_cmp(&right).unwrap();
}
}
}
// not comparable details, use global
(Some(WeightedScoreValue::WeightedScore(_)), Some(_))
| (Some(_), Some(WeightedScoreValue::WeightedScore(_)))
| (Some(WeightedScoreValue::VectorSort(_)), Some(_))
| (Some(_), Some(WeightedScoreValue::VectorSort(_)))
| (Some(WeightedScoreValue::GeoSort { .. }), Some(WeightedScoreValue::Sort { .. }))
| (Some(WeightedScoreValue::Sort { .. }), Some(WeightedScoreValue::GeoSort { .. })) => {
let left_count = left_it.count();
let right_count = right_it.count();
// compare how many remaining groups of rules each side has.
// the group with the most remaining groups wins.
return left_count
.cmp(&right_count)
// breaks ties with the global ranking score
.then_with(|| {
left_weighted_global_score
.partial_cmp(&right_weighted_global_score)
.unwrap()
});
}
}
}
}

View file

@ -30,7 +30,7 @@ use milli::{
MatchBounds, MatcherBuilder, SortError, TermsMatchingStrategy, DEFAULT_VALUES_PER_FACET,
};
use regex::Regex;
use serde::Serialize;
use serde::{Deserialize, Serialize};
use serde_json::{json, Value};
#[cfg(test)]
mod mod_test;
@ -41,7 +41,7 @@ use crate::error::MeilisearchHttpError;
mod federated;
pub use federated::{
perform_federated_search, FederatedSearch, FederatedSearchResult, Federation,
FederationOptions, MergeFacets,
FederationOptions, MergeFacets, PROXY_SEARCH_HEADER, PROXY_SEARCH_HEADER_VALUE,
};
mod ranking_rules;
@ -119,7 +119,7 @@ pub struct SearchQuery {
pub locales: Option<Vec<Locale>>,
}
#[derive(Debug, Clone, Copy, PartialEq, Deserr, ToSchema)]
#[derive(Debug, Clone, Copy, PartialEq, Deserr, ToSchema, Serialize)]
#[deserr(try_from(f64) = TryFrom::try_from -> InvalidSearchRankingScoreThreshold)]
pub struct RankingScoreThreshold(f64);
impl std::convert::TryFrom<f64> for RankingScoreThreshold {
@ -275,11 +275,13 @@ impl fmt::Debug for SearchQuery {
}
}
#[derive(Debug, Clone, Default, PartialEq, Deserr, ToSchema)]
#[derive(Debug, Clone, Default, PartialEq, Deserr, ToSchema, Serialize)]
#[deserr(error = DeserrJsonError<InvalidSearchHybridQuery>, rename_all = camelCase, deny_unknown_fields)]
#[serde(rename_all = "camelCase")]
pub struct HybridQuery {
#[deserr(default, error = DeserrJsonError<InvalidSearchSemanticRatio>, default)]
#[schema(value_type = f32, default)]
#[serde(default)]
pub semantic_ratio: SemanticRatio,
#[deserr(error = DeserrJsonError<InvalidSearchEmbedder>)]
pub embedder: String,
@ -369,7 +371,7 @@ impl SearchKind {
}
}
#[derive(Debug, Clone, Copy, PartialEq, Deserr)]
#[derive(Debug, Clone, Copy, PartialEq, Deserr, Serialize)]
#[deserr(try_from(f32) = TryFrom::try_from -> InvalidSearchSemanticRatio)]
pub struct SemanticRatio(f32);
@ -411,8 +413,9 @@ impl SearchQuery {
// This struct contains the fields of `SearchQuery` inline.
// This is because neither deserr nor serde support `flatten` when using `deny_unknown_fields.
// The `From<SearchQueryWithIndex>` implementation ensures both structs remain up to date.
#[derive(Debug, Clone, PartialEq, Deserr, ToSchema)]
#[derive(Debug, Clone, Serialize, PartialEq, Deserr, ToSchema)]
#[deserr(error = DeserrJsonError, rename_all = camelCase, deny_unknown_fields)]
#[serde(rename_all = "camelCase")]
#[schema(rename_all = "camelCase")]
pub struct SearchQueryWithIndex {
#[deserr(error = DeserrJsonError<InvalidIndexUid>, missing_field_error = DeserrJsonError::missing_index_uid)]
@ -493,6 +496,72 @@ impl SearchQueryWithIndex {
self.facets.as_deref().filter(|v| !v.is_empty())
}
pub fn from_index_query_federation(
index_uid: IndexUid,
query: SearchQuery,
federation_options: Option<FederationOptions>,
) -> Self {
let SearchQuery {
q,
vector,
hybrid,
offset,
limit,
page,
hits_per_page,
attributes_to_retrieve,
retrieve_vectors,
attributes_to_crop,
crop_length,
attributes_to_highlight,
show_matches_position,
show_ranking_score,
show_ranking_score_details,
filter,
sort,
distinct,
facets,
highlight_pre_tag,
highlight_post_tag,
crop_marker,
matching_strategy,
attributes_to_search_on,
ranking_score_threshold,
locales,
} = query;
SearchQueryWithIndex {
index_uid,
q,
vector,
hybrid,
offset: if offset == DEFAULT_SEARCH_OFFSET() { None } else { Some(offset) },
limit: if limit == DEFAULT_SEARCH_LIMIT() { None } else { Some(limit) },
page,
hits_per_page,
attributes_to_retrieve,
retrieve_vectors,
attributes_to_crop,
crop_length,
attributes_to_highlight,
show_ranking_score,
show_ranking_score_details,
show_matches_position,
filter,
sort,
distinct,
facets,
highlight_pre_tag,
highlight_post_tag,
crop_marker,
matching_strategy,
attributes_to_search_on,
ranking_score_threshold,
locales,
federation_options,
}
}
pub fn into_index_query_federation(self) -> (IndexUid, SearchQuery, Option<FederationOptions>) {
let SearchQueryWithIndex {
index_uid,
@ -620,8 +689,9 @@ impl TryFrom<Value> for ExternalDocumentId {
}
}
#[derive(Debug, Copy, Clone, PartialEq, Eq, Deserr, ToSchema)]
#[derive(Debug, Copy, Clone, PartialEq, Eq, Deserr, ToSchema, Serialize)]
#[deserr(rename_all = camelCase)]
#[serde(rename_all = "camelCase")]
pub enum MatchingStrategy {
/// Remove query words from last to first
Last,
@ -667,19 +737,19 @@ impl From<FacetValuesSort> for OrderBy {
}
}
#[derive(Debug, Clone, Serialize, PartialEq, ToSchema)]
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, ToSchema)]
pub struct SearchHit {
#[serde(flatten)]
#[schema(additional_properties, inline, value_type = HashMap<String, Value>)]
pub document: Document,
#[serde(rename = "_formatted", skip_serializing_if = "Document::is_empty")]
#[serde(default, rename = "_formatted", skip_serializing_if = "Document::is_empty")]
#[schema(additional_properties, value_type = HashMap<String, Value>)]
pub formatted: Document,
#[serde(rename = "_matchesPosition", skip_serializing_if = "Option::is_none")]
#[serde(default, rename = "_matchesPosition", skip_serializing_if = "Option::is_none")]
pub matches_position: Option<MatchesPosition>,
#[serde(rename = "_rankingScore", skip_serializing_if = "Option::is_none")]
#[serde(default, rename = "_rankingScore", skip_serializing_if = "Option::is_none")]
pub ranking_score: Option<f64>,
#[serde(rename = "_rankingScoreDetails", skip_serializing_if = "Option::is_none")]
#[serde(default, rename = "_rankingScoreDetails", skip_serializing_if = "Option::is_none")]
pub ranking_score_details: Option<serde_json::Map<String, serde_json::Value>>,
}
@ -767,7 +837,7 @@ pub struct SearchResultWithIndex {
pub result: SearchResult,
}
#[derive(Serialize, Debug, Clone, PartialEq, Eq, ToSchema)]
#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, ToSchema)]
#[serde(untagged)]
pub enum HitsInfo {
#[serde(rename_all = "camelCase")]
@ -778,7 +848,7 @@ pub enum HitsInfo {
OffsetLimit { limit: usize, offset: usize, estimated_total_hits: usize },
}
#[derive(Serialize, Debug, Clone, PartialEq, ToSchema)]
#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, ToSchema)]
pub struct FacetStats {
pub min: f64,
pub max: f64,
@ -1061,7 +1131,7 @@ pub fn perform_search(
Ok(result)
}
#[derive(Debug, Clone, Default, Serialize, ToSchema)]
#[derive(Debug, Clone, Default, Serialize, Deserialize, ToSchema)]
pub struct ComputedFacets {
#[schema(value_type = BTreeMap<String, BTreeMap<String, u64>>)]
pub distribution: BTreeMap<String, IndexMap<String, u64>>,

View file

@ -421,7 +421,7 @@ async fn error_add_api_key_invalid_parameters_actions() {
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response, { ".createdAt" => "[ignored]", ".updatedAt" => "[ignored]" }), @r###"
{
"message": "Unknown value `doc.add` at `.actions[0]`: expected one of `*`, `search`, `documents.*`, `documents.add`, `documents.get`, `documents.delete`, `indexes.*`, `indexes.create`, `indexes.get`, `indexes.update`, `indexes.delete`, `indexes.swap`, `tasks.*`, `tasks.cancel`, `tasks.delete`, `tasks.get`, `settings.*`, `settings.get`, `settings.update`, `stats.*`, `stats.get`, `metrics.*`, `metrics.get`, `dumps.*`, `dumps.create`, `snapshots.*`, `snapshots.create`, `version`, `keys.create`, `keys.get`, `keys.update`, `keys.delete`, `experimental.get`, `experimental.update`",
"message": "Unknown value `doc.add` at `.actions[0]`: expected one of `*`, `search`, `documents.*`, `documents.add`, `documents.get`, `documents.delete`, `indexes.*`, `indexes.create`, `indexes.get`, `indexes.update`, `indexes.delete`, `indexes.swap`, `tasks.*`, `tasks.cancel`, `tasks.delete`, `tasks.get`, `settings.*`, `settings.get`, `settings.update`, `stats.*`, `stats.get`, `metrics.*`, `metrics.get`, `dumps.*`, `dumps.create`, `snapshots.*`, `snapshots.create`, `version`, `keys.create`, `keys.get`, `keys.update`, `keys.delete`, `experimental.get`, `experimental.update`, `network.get`, `network.update`",
"code": "invalid_api_key_actions",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_api_key_actions"

View file

@ -68,6 +68,8 @@ pub static AUTHORIZATIONS: Lazy<HashMap<(&'static str, &'static str), HashSet<&'
("GET", "/keys") => hashset!{"keys.get", "*"},
("GET", "/experimental-features") => hashset!{"experimental.get", "*"},
("PATCH", "/experimental-features") => hashset!{"experimental.update", "*"},
("GET", "/network") => hashset!{"network.get", "*"},
("PATCH", "/network") => hashset!{"network.update", "*"},
};
authorizations

View file

@ -93,7 +93,7 @@ async fn create_api_key_bad_actions() {
snapshot!(code, @"400 Bad Request");
snapshot!(json_string!(response), @r###"
{
"message": "Unknown value `doggo` at `.actions[0]`: expected one of `*`, `search`, `documents.*`, `documents.add`, `documents.get`, `documents.delete`, `indexes.*`, `indexes.create`, `indexes.get`, `indexes.update`, `indexes.delete`, `indexes.swap`, `tasks.*`, `tasks.cancel`, `tasks.delete`, `tasks.get`, `settings.*`, `settings.get`, `settings.update`, `stats.*`, `stats.get`, `metrics.*`, `metrics.get`, `dumps.*`, `dumps.create`, `snapshots.*`, `snapshots.create`, `version`, `keys.create`, `keys.get`, `keys.update`, `keys.delete`, `experimental.get`, `experimental.update`",
"message": "Unknown value `doggo` at `.actions[0]`: expected one of `*`, `search`, `documents.*`, `documents.add`, `documents.get`, `documents.delete`, `indexes.*`, `indexes.create`, `indexes.get`, `indexes.update`, `indexes.delete`, `indexes.swap`, `tasks.*`, `tasks.cancel`, `tasks.delete`, `tasks.get`, `settings.*`, `settings.get`, `settings.update`, `stats.*`, `stats.get`, `metrics.*`, `metrics.get`, `dumps.*`, `dumps.create`, `snapshots.*`, `snapshots.create`, `version`, `keys.create`, `keys.get`, `keys.update`, `keys.delete`, `experimental.get`, `experimental.update`, `network.get`, `network.update`",
"code": "invalid_api_key_actions",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_api_key_actions"

View file

@ -41,9 +41,8 @@ async fn list_batches() {
let index = server.index("test");
let (task, _status_code) = index.create(None).await;
index.wait_task(task.uid()).await.succeeded();
index
.add_documents(serde_json::from_str(include_str!("../assets/test_set.json")).unwrap(), None)
.await;
let (task, _status_code) = index.create(None).await;
index.wait_task(task.uid()).await.failed();
let (response, code) = index.list_batches().await;
assert_eq!(code, 200);
assert_eq!(
@ -96,11 +95,12 @@ async fn list_batches_pagination_and_reverse() {
async fn list_batches_with_star_filters() {
let server = Server::new().await;
let index = server.index("test");
let (batch, _code) = index.create(None).await;
index.wait_task(batch.uid()).await.succeeded();
index
.add_documents(serde_json::from_str(include_str!("../assets/test_set.json")).unwrap(), None)
.await;
let (task, _code) = index.create(None).await;
index.wait_task(task.uid()).await.succeeded();
let index = server.index("test");
let (task, _code) = index.create(None).await;
index.wait_task(task.uid()).await.failed();
let (response, code) = index.service.get("/batches?indexUids=test").await;
assert_eq!(code, 200);
assert_eq!(response["results"].as_array().unwrap().len(), 2);
@ -187,9 +187,6 @@ async fn list_batches_invalid_canceled_by_filter() {
let index = server.index("test");
let (task, _status_code) = index.create(None).await;
index.wait_task(task.uid()).await.succeeded();
index
.add_documents(serde_json::from_str(include_str!("../assets/test_set.json")).unwrap(), None)
.await;
let (response, code) = index.filtered_batches(&[], &[], &["0"]).await;
assert_eq!(code, 200, "{}", response);
@ -202,9 +199,8 @@ async fn list_batches_status_and_type_filtered() {
let index = server.index("test");
let (task, _status_code) = index.create(None).await;
index.wait_task(task.uid()).await.succeeded();
index
.add_documents(serde_json::from_str(include_str!("../assets/test_set.json")).unwrap(), None)
.await;
let (task, _status_code) = index.update(Some("id")).await;
index.wait_task(task.uid()).await.succeeded();
let (response, code) = index.filtered_batches(&["indexCreation"], &["failed"], &[]).await;
assert_eq!(code, 200, "{}", response);
@ -212,7 +208,7 @@ async fn list_batches_status_and_type_filtered() {
let (response, code) = index
.filtered_batches(
&["indexCreation", "documentAdditionOrUpdate"],
&["indexCreation", "IndexUpdate"],
&["succeeded", "processing", "enqueued"],
&[],
)

View file

@ -88,6 +88,10 @@ impl Server<Owned> {
self.service.api_key = Some(api_key.as_ref().to_string());
}
pub fn clear_api_key(&mut self) {
self.service.api_key = None;
}
/// Fetch and use the default admin key for nexts http requests.
pub async fn use_admin_key(&mut self, master_key: impl AsRef<str>) {
self.use_api_key(master_key);
@ -159,10 +163,18 @@ impl Server<Owned> {
self.service.get("/tasks").await
}
pub async fn batches(&self) -> (Value, StatusCode) {
self.service.get("/batches").await
}
pub async fn set_features(&self, value: Value) -> (Value, StatusCode) {
self.service.patch("/experimental-features", value).await
}
pub async fn set_network(&self, value: Value) -> (Value, StatusCode) {
self.service.patch("/network", value).await
}
pub async fn get_metrics(&self) -> (Value, StatusCode) {
self.service.get("/metrics").await
}
@ -408,6 +420,10 @@ impl<State> Server<State> {
pub async fn get_features(&self) -> (Value, StatusCode) {
self.service.get("/experimental-features").await
}
pub async fn get_network(&self) -> (Value, StatusCode) {
self.service.get("/network").await
}
}
pub fn default_settings(dir: impl AsRef<Path>) -> Opt {

View file

@ -1803,6 +1803,275 @@ async fn add_documents_with_geo_field() {
"finishedAt": "[date]"
}
"###);
let (response, code) = index.get_all_documents(GetAllDocumentsOptions::default()).await;
snapshot!(code, @"200 OK");
snapshot!(json_string!(response, { ".duration" => "[duration]", ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]" }),
@r###"
{
"results": [
{
"id": "1"
},
{
"id": "2",
"_geo": null
},
{
"id": "3",
"_geo": {
"lat": 1,
"lng": 1
}
},
{
"id": "4",
"_geo": {
"lat": "1",
"lng": "1"
}
}
],
"offset": 0,
"limit": 20,
"total": 4
}
"###);
let (response, code) = index
.search_post(json!({"sort": ["_geoPoint(50.629973371633746,3.0569447399419567):desc"]}))
.await;
snapshot!(code, @"200 OK");
// we are expecting docs 4 and 3 first as they have geo
snapshot!(json_string!(response, { ".processingTimeMs" => "[time]" }),
@r###"
{
"hits": [
{
"id": "4",
"_geo": {
"lat": "1",
"lng": "1"
},
"_geoDistance": 5522018
},
{
"id": "3",
"_geo": {
"lat": 1,
"lng": 1
},
"_geoDistance": 5522018
},
{
"id": "1"
},
{
"id": "2",
"_geo": null
}
],
"query": "",
"processingTimeMs": "[time]",
"limit": 20,
"offset": 0,
"estimatedTotalHits": 4
}
"###);
}
#[actix_rt::test]
async fn update_documents_with_geo_field() {
let server = Server::new().await;
let index = server.index("doggo");
index.update_settings(json!({"sortableAttributes": ["_geo"]})).await;
let documents = json!([
{
"id": "1",
},
{
"id": "2",
"_geo": null,
},
{
"id": "3",
"_geo": { "lat": 1, "lng": 1 },
},
{
"id": "4",
"_geo": { "lat": "1", "lng": "1" },
},
]);
let (task, _status_code) = index.add_documents(documents, None).await;
let response = index.wait_task(task.uid()).await;
snapshot!(json_string!(response, { ".duration" => "[duration]", ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]" }),
@r###"
{
"uid": 1,
"batchUid": 1,
"indexUid": "doggo",
"status": "succeeded",
"type": "documentAdditionOrUpdate",
"canceledBy": null,
"details": {
"receivedDocuments": 4,
"indexedDocuments": 4
},
"error": null,
"duration": "[duration]",
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]"
}
"###);
let (response, code) = index
.search_post(json!({"sort": ["_geoPoint(50.629973371633746,3.0569447399419567):desc"]}))
.await;
snapshot!(code, @"200 OK");
// we are expecting docs 4 and 3 first as they have geo
snapshot!(json_string!(response, { ".processingTimeMs" => "[time]" }),
@r###"
{
"hits": [
{
"id": "4",
"_geo": {
"lat": "1",
"lng": "1"
},
"_geoDistance": 5522018
},
{
"id": "3",
"_geo": {
"lat": 1,
"lng": 1
},
"_geoDistance": 5522018
},
{
"id": "1"
},
{
"id": "2",
"_geo": null
}
],
"query": "",
"processingTimeMs": "[time]",
"limit": 20,
"offset": 0,
"estimatedTotalHits": 4
}
"###);
let updated_documents = json!([{
"id": "3",
"doggo": "kefir",
}]);
let (task, _status_code) = index.update_documents(updated_documents, None).await;
let response = index.wait_task(task.uid()).await;
snapshot!(json_string!(response, { ".duration" => "[duration]", ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]" }),
@r###"
{
"uid": 2,
"batchUid": 2,
"indexUid": "doggo",
"status": "succeeded",
"type": "documentAdditionOrUpdate",
"canceledBy": null,
"details": {
"receivedDocuments": 1,
"indexedDocuments": 1
},
"error": null,
"duration": "[duration]",
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]"
}
"###);
let (response, code) = index.get_all_documents(GetAllDocumentsOptions::default()).await;
snapshot!(code, @"200 OK");
snapshot!(json_string!(response, { ".duration" => "[duration]", ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]" }),
@r###"
{
"results": [
{
"id": "1"
},
{
"id": "2",
"_geo": null
},
{
"id": "3",
"_geo": {
"lat": 1,
"lng": 1
},
"doggo": "kefir"
},
{
"id": "4",
"_geo": {
"lat": "1",
"lng": "1"
}
}
],
"offset": 0,
"limit": 20,
"total": 4
}
"###);
let (response, code) = index
.search_post(json!({"sort": ["_geoPoint(50.629973371633746,3.0569447399419567):desc"]}))
.await;
snapshot!(code, @"200 OK");
// the search response should not have changed: we are expecting docs 4 and 3 first as they have geo
snapshot!(json_string!(response, { ".processingTimeMs" => "[time]" }),
@r###"
{
"hits": [
{
"id": "4",
"_geo": {
"lat": "1",
"lng": "1"
},
"_geoDistance": 5522018
},
{
"id": "3",
"_geo": {
"lat": 1,
"lng": 1
},
"doggo": "kefir",
"_geoDistance": 5522018
},
{
"id": "1"
},
{
"id": "2",
"_geo": null
}
],
"query": "",
"processingTimeMs": "[time]",
"limit": 20,
"offset": 0,
"estimatedTotalHits": 4
}
"###);
}
#[actix_rt::test]

View file

@ -161,6 +161,8 @@ async fn delete_document_by_filter() {
{
"numberOfDocuments": 4,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"color": 3,
"id": 4
@ -208,6 +210,8 @@ async fn delete_document_by_filter() {
{
"numberOfDocuments": 2,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"color": 1,
"id": 2
@ -274,6 +278,8 @@ async fn delete_document_by_filter() {
{
"numberOfDocuments": 1,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"color": 1,
"id": 1

View file

@ -22,6 +22,7 @@ pub enum GetDump {
TestV5,
TestV6WithExperimental,
TestV6WithBatchesAndEnqueuedTasks,
}
impl GetDump {
@ -74,6 +75,10 @@ impl GetDump {
"tests/assets/v6_v1.6.0_use_deactivated_experimental_setting.dump"
)
.into(),
GetDump::TestV6WithBatchesAndEnqueuedTasks => {
exist_relative_path!("tests/assets/v6_v1.13.0_batches_and_enqueued_tasks.dump")
.into()
}
}
}
}

View file

@ -27,9 +27,24 @@ async fn import_dump_v1_movie_raw() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"genres": 53, "id": 53, "overview": 53, "poster": 53, "release_date": 53, "title": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"genres": 53,
"id": 53,
"overview": 53,
"poster": 53,
"release_date": 53,
"title": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -173,6 +188,8 @@ async fn import_dump_v1_movie_with_settings() {
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"genres": 53,
"id": 53,
@ -333,9 +350,24 @@ async fn import_dump_v1_rubygems_with_settings() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"description": 53, "id": 53, "name": 53, "summary": 53, "total_downloads": 53, "version": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"description": 53,
"id": 53,
"name": 53,
"summary": 53,
"total_downloads": 53,
"version": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -483,9 +515,24 @@ async fn import_dump_v2_movie_raw() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"genres": 53, "id": 53, "overview": 53, "poster": 53, "release_date": 53, "title": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"genres": 53,
"id": 53,
"overview": 53,
"poster": 53,
"release_date": 53,
"title": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -623,9 +670,24 @@ async fn import_dump_v2_movie_with_settings() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"genres": 53, "id": 53, "overview": 53, "poster": 53, "release_date": 53, "title": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"genres": 53,
"id": 53,
"overview": 53,
"poster": 53,
"release_date": 53,
"title": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -773,9 +835,24 @@ async fn import_dump_v2_rubygems_with_settings() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"description": 53, "id": 53, "name": 53, "summary": 53, "total_downloads": 53, "version": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"description": 53,
"id": 53,
"name": 53,
"summary": 53,
"total_downloads": 53,
"version": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -920,9 +997,24 @@ async fn import_dump_v3_movie_raw() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"genres": 53, "id": 53, "overview": 53, "poster": 53, "release_date": 53, "title": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"genres": 53,
"id": 53,
"overview": 53,
"poster": 53,
"release_date": 53,
"title": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -1060,9 +1152,24 @@ async fn import_dump_v3_movie_with_settings() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"genres": 53, "id": 53, "overview": 53, "poster": 53, "release_date": 53, "title": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"genres": 53,
"id": 53,
"overview": 53,
"poster": 53,
"release_date": 53,
"title": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -1210,9 +1317,24 @@ async fn import_dump_v3_rubygems_with_settings() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"description": 53, "id": 53, "name": 53, "summary": 53, "total_downloads": 53, "version": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"description": 53,
"id": 53,
"name": 53,
"summary": 53,
"total_downloads": 53,
"version": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -1357,9 +1479,24 @@ async fn import_dump_v4_movie_raw() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"genres": 53, "id": 53, "overview": 53, "poster": 53, "release_date": 53, "title": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"genres": 53,
"id": 53,
"overview": 53,
"poster": 53,
"release_date": 53,
"title": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -1497,9 +1634,24 @@ async fn import_dump_v4_movie_with_settings() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"genres": 53, "id": 53, "overview": 53, "poster": 53, "release_date": 53, "title": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"genres": 53,
"id": 53,
"overview": 53,
"poster": 53,
"release_date": 53,
"title": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -1647,9 +1799,24 @@ async fn import_dump_v4_rubygems_with_settings() {
let (stats, code) = index.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(
stats,
json!({ "numberOfDocuments": 53, "isIndexing": false, "fieldDistribution": {"description": 53, "id": 53, "name": 53, "summary": 53, "total_downloads": 53, "version": 53 }})
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 53,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"description": 53,
"id": 53,
"name": 53,
"summary": 53,
"total_downloads": 53,
"version": 53
}
}
"###
);
let (settings, code) = index.settings().await;
@ -1798,33 +1965,35 @@ async fn import_dump_v5() {
server.wait_task(task["uid"].as_u64().unwrap()).await;
}
let expected_stats = json!({
"numberOfDocuments": 10,
"isIndexing": false,
"fieldDistribution": {
"cast": 10,
"director": 10,
"genres": 10,
"id": 10,
"overview": 10,
"popularity": 10,
"poster_path": 10,
"producer": 10,
"production_companies": 10,
"release_date": 10,
"tagline": 10,
"title": 10,
"vote_average": 10,
"vote_count": 10
}
});
let index1 = server.index("test");
let index2 = server.index("test2");
let (stats, code) = index1.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(stats, expected_stats);
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 10,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"cast": 10,
"director": 10,
"genres": 10,
"id": 10,
"overview": 10,
"popularity": 10,
"poster_path": 10,
"producer": 10,
"production_companies": 10,
"release_date": 10,
"tagline": 10,
"title": 10,
"vote_average": 10,
"vote_count": 10
}
}
"###);
let (docs, code) = index2.get_all_documents(GetAllDocumentsOptions::default()).await;
snapshot!(code, @"200 OK");
@ -1835,7 +2004,32 @@ async fn import_dump_v5() {
let (stats, code) = index2.stats().await;
snapshot!(code, @"200 OK");
assert_eq!(stats, expected_stats);
snapshot!(
json_string!(stats),
@r###"
{
"numberOfDocuments": 10,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"cast": 10,
"director": 10,
"genres": 10,
"id": 10,
"overview": 10,
"popularity": 10,
"poster_path": 10,
"producer": 10,
"production_companies": 10,
"release_date": 10,
"tagline": 10,
"title": 10,
"vote_average": 10,
"vote_count": 10
}
}
"###);
let (keys, code) = server.list_api_keys("").await;
snapshot!(code, @"200 OK");
@ -1908,7 +2102,9 @@ async fn import_dump_v6_containing_experimental_features() {
"metrics": false,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);
@ -1992,6 +2188,63 @@ async fn import_dump_v6_containing_experimental_features() {
.await;
}
#[actix_rt::test]
async fn import_dump_v6_containing_batches_and_enqueued_tasks() {
let temp = tempfile::tempdir().unwrap();
let options = Opt {
import_dump: Some(GetDump::TestV6WithBatchesAndEnqueuedTasks.path()),
..default_settings(temp.path())
};
let mut server = Server::new_auth_with_options(options, temp).await;
server.use_api_key("MASTER_KEY");
server.wait_task(2).await.succeeded();
let (tasks, _) = server.tasks().await;
snapshot!(json_string!(tasks, { ".results[1].startedAt" => "[date]", ".results[1].finishedAt" => "[date]", ".results[1].duration" => "[date]" }), name: "tasks");
let (batches, _) = server.batches().await;
snapshot!(json_string!(batches, { ".results[0].startedAt" => "[date]", ".results[0].finishedAt" => "[date]", ".results[0].duration" => "[date]" }), name: "batches");
let (indexes, code) = server.list_indexes(None, None).await;
assert_eq!(code, 200, "{indexes}");
assert_eq!(indexes["results"].as_array().unwrap().len(), 1);
assert_eq!(indexes["results"][0]["uid"], json!("kefir"));
assert_eq!(indexes["results"][0]["primaryKey"], json!("id"));
let (response, code) = server.get_features().await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"metrics": false,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);
let index = server.index("kefir");
let (documents, _) = index.get_all_documents_raw("").await;
snapshot!(documents, @r#"
{
"results": [
{
"id": 1,
"dog": "kefir"
},
{
"id": 2,
"dog": "intel"
}
],
"offset": 0,
"limit": 20,
"total": 2
}
"#);
}
// In this test we must generate the dump ourselves to ensure the
// `user provided` vectors are well set
#[actix_rt::test]
@ -2069,7 +2322,9 @@ async fn generate_and_import_dump_containing_vectors() {
"metrics": false,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);

View file

@ -0,0 +1,78 @@
---
source: crates/meilisearch/tests/dumps/mod.rs
snapshot_kind: text
---
{
"results": [
{
"uid": 2,
"progress": null,
"details": {
"receivedDocuments": 1,
"indexedDocuments": 1
},
"stats": {
"totalNbTasks": 1,
"status": {
"succeeded": 1
},
"types": {
"documentAdditionOrUpdate": 1
},
"indexUids": {
"kefir": 1
}
},
"duration": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]"
},
{
"uid": 1,
"progress": null,
"details": {
"receivedDocuments": 1,
"indexedDocuments": 1
},
"stats": {
"totalNbTasks": 1,
"status": {
"succeeded": 1
},
"types": {
"documentAdditionOrUpdate": 1
},
"indexUids": {
"kefir": 1
}
},
"duration": "PT0.144827890S",
"startedAt": "2025-02-04T10:15:21.275640274Z",
"finishedAt": "2025-02-04T10:15:21.420468164Z"
},
{
"uid": 0,
"progress": null,
"details": {},
"stats": {
"totalNbTasks": 1,
"status": {
"succeeded": 1
},
"types": {
"indexCreation": 1
},
"indexUids": {
"kefir": 1
}
},
"duration": "PT0.032902186S",
"startedAt": "2025-02-04T10:14:43.559526162Z",
"finishedAt": "2025-02-04T10:14:43.592428348Z"
}
],
"total": 3,
"limit": 20,
"from": 2,
"next": null
}

View file

@ -0,0 +1,78 @@
---
source: crates/meilisearch/tests/dumps/mod.rs
snapshot_kind: text
---
{
"results": [
{
"uid": 3,
"batchUid": null,
"indexUid": null,
"status": "succeeded",
"type": "dumpCreation",
"canceledBy": null,
"details": {
"dumpUid": null
},
"error": null,
"duration": "PT0.000629059S",
"enqueuedAt": "2025-02-04T10:22:31.318175268Z",
"startedAt": "2025-02-04T10:22:31.331701375Z",
"finishedAt": "2025-02-04T10:22:31.332330434Z"
},
{
"uid": 2,
"batchUid": 2,
"indexUid": "kefir",
"status": "succeeded",
"type": "documentAdditionOrUpdate",
"canceledBy": null,
"details": {
"receivedDocuments": 1,
"indexedDocuments": 1
},
"error": null,
"duration": "[date]",
"enqueuedAt": "2025-02-04T10:15:49.212484063Z",
"startedAt": "[date]",
"finishedAt": "[date]"
},
{
"uid": 1,
"batchUid": null,
"indexUid": "kefir",
"status": "succeeded",
"type": "documentAdditionOrUpdate",
"canceledBy": null,
"details": {
"receivedDocuments": 1,
"indexedDocuments": 1
},
"error": null,
"duration": "PT0.144827890S",
"enqueuedAt": "2025-02-04T10:15:21.258630973Z",
"startedAt": "2025-02-04T10:15:21.275640274Z",
"finishedAt": "2025-02-04T10:15:21.420468164Z"
},
{
"uid": 0,
"batchUid": null,
"indexUid": "kefir",
"status": "succeeded",
"type": "indexCreation",
"canceledBy": null,
"details": {
"primaryKey": null
},
"error": null,
"duration": "PT0.032902186S",
"enqueuedAt": "2025-02-04T10:14:43.550379968Z",
"startedAt": "2025-02-04T10:14:43.559526162Z",
"finishedAt": "2025-02-04T10:14:43.592428348Z"
}
],
"total": 4,
"limit": 20,
"from": 3,
"next": null
}

View file

@ -21,7 +21,9 @@ async fn experimental_features() {
"metrics": false,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);
@ -33,7 +35,9 @@ async fn experimental_features() {
"metrics": true,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);
@ -45,7 +49,9 @@ async fn experimental_features() {
"metrics": true,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);
@ -58,7 +64,9 @@ async fn experimental_features() {
"metrics": true,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);
@ -71,7 +79,9 @@ async fn experimental_features() {
"metrics": true,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);
}
@ -91,7 +101,9 @@ async fn experimental_feature_metrics() {
"metrics": true,
"logsRoute": false,
"editDocumentsByFunction": false,
"containsFilter": false
"containsFilter": false,
"network": false,
"getTaskDocumentsRoute": false
}
"###);
@ -146,7 +158,7 @@ async fn errors() {
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Unknown field `NotAFeature`: expected one of `metrics`, `logsRoute`, `editDocumentsByFunction`, `containsFilter`",
"message": "Unknown field `NotAFeature`: expected one of `metrics`, `logsRoute`, `editDocumentsByFunction`, `containsFilter`, `network`, `getTaskDocumentsRoute`",
"code": "bad_request",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#bad_request"

View file

@ -7,6 +7,7 @@ mod dumps;
mod features;
mod index;
mod logs;
mod network;
mod search;
mod settings;
mod similar;

View file

@ -0,0 +1,606 @@
use serde_json::Value::Null;
use crate::common::Server;
use crate::json;
#[actix_rt::test]
async fn error_network_not_enabled() {
let server = Server::new().await;
let (response, code) = server.get_network().await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Using the /network route requires enabling the `network` experimental feature. See https://github.com/orgs/meilisearch/discussions/805",
"code": "feature_not_enabled",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#feature_not_enabled"
}
"###);
let (response, code) = server.set_network(json!({"self": "myself"})).await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Using the /network route requires enabling the `network` experimental feature. See https://github.com/orgs/meilisearch/discussions/805",
"code": "feature_not_enabled",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#feature_not_enabled"
}
"###);
}
#[actix_rt::test]
async fn errors_on_param() {
let server = Server::new().await;
let (response, code) = server.set_features(json!({"network": true})).await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response["network"]), @r#"true"#);
// non-existing param
let (response, code) = server.set_network(json!({"selfie": "myself"})).await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Unknown field `selfie`: expected one of `remotes`, `self`",
"code": "bad_request",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#bad_request"
}
"###);
// self not a string
let (response, code) = server.set_network(json!({"self": 42})).await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Invalid value type at `.self`: expected a string, but found a positive integer: `42`",
"code": "invalid_network_self",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_network_self"
}
"###);
// remotes not an object
let (response, code) = server.set_network(json!({"remotes": 42})).await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Invalid value type at `.remotes`: expected an object, but found a positive integer: `42`",
"code": "invalid_network_remotes",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_network_remotes"
}
"###);
// new remote without url
let (response, code) = server
.set_network(json!({"remotes": {
"new": {
"searchApiKey": "http://localhost:7700"
}
}}))
.await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Missing field `.remotes.new.url`",
"code": "missing_network_url",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#missing_network_url"
}
"###);
// remote with url not a string
let (response, code) = server
.set_network(json!({"remotes": {
"new": {
"url": 7700
}
}}))
.await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Invalid value type at `.remotes.new.url`: expected a string, but found a positive integer: `7700`",
"code": "invalid_network_url",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_network_url"
}
"###);
// remote with non-existing param
let (response, code) = server
.set_network(json!({"remotes": {
"new": {
"url": "http://localhost:7700",
"doggo": "Intel the Beagle"
}
}}))
.await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Unknown field `doggo` inside `.remotes.new`: expected one of `url`, `searchApiKey`",
"code": "invalid_network_remotes",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_network_remotes"
}
"###);
// remote with non-string searchApiKey
let (response, code) = server
.set_network(json!({"remotes": {
"new": {
"url": "http://localhost:7700",
"searchApiKey": 1204664602099962445u64,
}
}}))
.await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Invalid value type at `.remotes.new.searchApiKey`: expected a string, but found a positive integer: `1204664602099962445`",
"code": "invalid_network_search_api_key",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_network_search_api_key"
}
"###);
// setting `null` on URL a posteriori
let (response, code) = server
.set_network(json!({"remotes": {
"kefir": {
"url": "http://localhost:7700",
}
}}))
.await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": null,
"remotes": {
"kefir": {
"url": "http://localhost:7700",
"searchApiKey": null
}
}
}
"###);
let (response, code) = server
.set_network(json!({"remotes": {
"kefir": {
"url": Null,
}
}}))
.await;
meili_snap::snapshot!(code, @"400 Bad Request");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "Field `.remotes.kefir.url` cannot be set to `null`",
"code": "invalid_network_url",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_network_url"
}
"###);
}
#[actix_rt::test]
async fn auth() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let (response, code) = server.set_features(json!({"network": true})).await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response["network"]), @r#"true"#);
let (get_network_key, code) = server
.add_api_key(json!({
"actions": ["network.get"],
"indexes": ["*"],
"expiresAt": serde_json::Value::Null
}))
.await;
meili_snap::snapshot!(code, @"201 Created");
let get_network_key = get_network_key["key"].clone();
let (update_network_key, code) = server
.add_api_key(json!({
"actions": ["network.update"],
"indexes": ["*"],
"expiresAt": serde_json::Value::Null
}))
.await;
meili_snap::snapshot!(code, @"201 Created");
let update_network_key = update_network_key["key"].clone();
let (search_api_key, code) = server
.add_api_key(json!({
"actions": ["search"],
"indexes": ["*"],
"expiresAt": serde_json::Value::Null
}))
.await;
meili_snap::snapshot!(code, @"201 Created");
let search_api_key = search_api_key["key"].clone();
// try with master key
let (response, code) = server
.set_network(json!({
"self": "master"
}))
.await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "master",
"remotes": {}
}
"###);
let (response, code) = server.get_network().await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "master",
"remotes": {}
}
"###);
// try get with get permission
server.use_api_key(get_network_key.as_str().unwrap());
let (response, code) = server.get_network().await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "master",
"remotes": {}
}
"###);
// try update with update permission
server.use_api_key(update_network_key.as_str().unwrap());
let (response, code) = server
.set_network(json!({
"self": "api_key"
}))
.await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "api_key",
"remotes": {}
}
"###);
// try with the other's permission
let (response, code) = server.get_network().await;
meili_snap::snapshot!(code, @"403 Forbidden");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "The provided API key is invalid.",
"code": "invalid_api_key",
"type": "auth",
"link": "https://docs.meilisearch.com/errors#invalid_api_key"
}
"###);
server.use_api_key(get_network_key.as_str().unwrap());
let (response, code) = server
.set_network(json!({
"self": "get_api_key"
}))
.await;
meili_snap::snapshot!(code, @"403 Forbidden");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "The provided API key is invalid.",
"code": "invalid_api_key",
"type": "auth",
"link": "https://docs.meilisearch.com/errors#invalid_api_key"
}
"###);
// try either with bad permission
server.use_api_key(search_api_key.as_str().unwrap());
let (response, code) = server.get_network().await;
meili_snap::snapshot!(code, @"403 Forbidden");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "The provided API key is invalid.",
"code": "invalid_api_key",
"type": "auth",
"link": "https://docs.meilisearch.com/errors#invalid_api_key"
}
"###);
let (response, code) = server
.set_network(json!({
"self": "get_api_key"
}))
.await;
meili_snap::snapshot!(code, @"403 Forbidden");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"message": "The provided API key is invalid.",
"code": "invalid_api_key",
"type": "auth",
"link": "https://docs.meilisearch.com/errors#invalid_api_key"
}
"###);
}
#[actix_rt::test]
async fn get_and_set_network() {
let server = Server::new().await;
let (response, code) = server.set_features(json!({"network": true})).await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response["network"]), @r#"true"#);
let (response, code) = server.get_network().await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": null,
"remotes": {}
}
"###);
// adding self
let (response, code) = server.set_network(json!({"self": "myself"})).await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "myself",
"remotes": {}
}
"###);
// adding remotes
let (response, code) = server
.set_network(json!({"remotes": {
"myself": {
"url": "http://localhost:7700"
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "foo"
}
}}))
.await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "myself",
"remotes": {
"myself": {
"url": "http://localhost:7700",
"searchApiKey": null
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "foo"
}
}
}
"###);
// partially updating one remote
let (response, code) = server
.set_network(json!({"remotes": {
"thy": {
"searchApiKey": "bar"
}
}}))
.await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "myself",
"remotes": {
"myself": {
"url": "http://localhost:7700",
"searchApiKey": null
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "bar"
}
}
}
"###);
// adding one remote
let (response, code) = server
.set_network(json!({"remotes": {
"them": {
"url": "http://localhost:7702",
"searchApiKey": "baz"
}
}}))
.await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "myself",
"remotes": {
"myself": {
"url": "http://localhost:7700",
"searchApiKey": null
},
"them": {
"url": "http://localhost:7702",
"searchApiKey": "baz"
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "bar"
}
}
}
"###);
// deleting one remote
let (response, code) = server
.set_network(json!({"remotes": {
"myself": Null,
}}))
.await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "myself",
"remotes": {
"them": {
"url": "http://localhost:7702",
"searchApiKey": "baz"
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "bar"
}
}
}
"###);
// removing self
let (response, code) = server.set_network(json!({"self": Null})).await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": null,
"remotes": {
"them": {
"url": "http://localhost:7702",
"searchApiKey": "baz"
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "bar"
}
}
}
"###);
// setting self again
let (response, code) = server.set_network(json!({"self": "thy"})).await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "thy",
"remotes": {
"them": {
"url": "http://localhost:7702",
"searchApiKey": "baz"
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "bar"
}
}
}
"###);
// doing nothing
let (response, code) = server.set_network(json!({})).await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "thy",
"remotes": {
"them": {
"url": "http://localhost:7702",
"searchApiKey": "baz"
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "bar"
}
}
}
"###);
// still doing nothing
let (response, code) = server.set_network(json!({"remotes": {}})).await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "thy",
"remotes": {
"them": {
"url": "http://localhost:7702",
"searchApiKey": "baz"
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "bar"
}
}
}
"###);
// good time to check GET
let (response, code) = server.get_network().await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "thy",
"remotes": {
"them": {
"url": "http://localhost:7702",
"searchApiKey": "baz"
},
"thy": {
"url": "http://localhost:7701",
"searchApiKey": "bar"
}
}
}
"###);
// deleting everything
let (response, code) = server
.set_network(json!({
"remotes": Null,
}))
.await;
meili_snap::snapshot!(code, @"200 OK");
meili_snap::snapshot!(meili_snap::json_string!(response), @r###"
{
"self": "thy",
"remotes": {}
}
"###);
}

View file

@ -5,6 +5,8 @@ use crate::common::Server;
use crate::json;
use crate::search::{SCORE_DOCUMENTS, VECTOR_DOCUMENTS};
mod proxy;
#[actix_rt::test]
async fn search_empty_list() {
let server = Server::new().await;

File diff suppressed because it is too large Load diff

View file

@ -1,3 +1,4 @@
use meili_snap::{json_string, snapshot};
use time::format_description::well_known::Rfc3339;
use time::OffsetDateTime;
@ -74,3 +75,253 @@ async fn stats() {
assert_eq!(response["indexes"]["test"]["fieldDistribution"]["name"], 1);
assert_eq!(response["indexes"]["test"]["fieldDistribution"]["age"], 1);
}
#[actix_rt::test]
async fn add_remove_embeddings() {
let server = Server::new().await;
let index = server.index("doggo");
let (response, code) = index
.update_settings(json!({
"embedders": {
"manual": {
"source": "userProvided",
"dimensions": 3,
},
"handcrafted": {
"source": "userProvided",
"dimensions": 3,
},
},
}))
.await;
snapshot!(code, @"202 Accepted");
server.wait_task(response.uid()).await.succeeded();
// 2 embedded documents for 5 embeddings in total
let documents = json!([
{"id": 0, "name": "kefir", "_vectors": { "manual": [0, 0, 0], "handcrafted": [0, 0, 0] }},
{"id": 1, "name": "echo", "_vectors": { "manual": [1, 1, 1], "handcrafted": [[1, 1, 1], [2, 2, 2]] }},
]);
let (response, code) = index.add_documents(documents, None).await;
snapshot!(code, @"202 Accepted");
index.wait_task(response.uid()).await.succeeded();
let (stats, _code) = index.stats().await;
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 2,
"isIndexing": false,
"numberOfEmbeddings": 5,
"numberOfEmbeddedDocuments": 2,
"fieldDistribution": {
"id": 2,
"name": 2
}
}
"###);
// 2 embedded documents for 3 embeddings in total
let documents = json!([
{"id": 1, "name": "echo", "_vectors": { "manual": [1, 1, 1], "handcrafted": null }},
]);
let (response, code) = index.update_documents(documents, None).await;
snapshot!(code, @"202 Accepted");
index.wait_task(response.uid()).await.succeeded();
let (stats, _code) = index.stats().await;
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 2,
"isIndexing": false,
"numberOfEmbeddings": 3,
"numberOfEmbeddedDocuments": 2,
"fieldDistribution": {
"id": 2,
"name": 2
}
}
"###);
// 2 embedded documents for 2 embeddings in total
let documents = json!([
{"id": 0, "name": "kefir", "_vectors": { "manual": null, "handcrafted": [0, 0, 0] }},
]);
let (response, code) = index.update_documents(documents, None).await;
snapshot!(code, @"202 Accepted");
index.wait_task(response.uid()).await.succeeded();
let (stats, _code) = index.stats().await;
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 2,
"isIndexing": false,
"numberOfEmbeddings": 2,
"numberOfEmbeddedDocuments": 2,
"fieldDistribution": {
"id": 2,
"name": 2
}
}
"###);
// 1 embedded documents for 2 embeddings in total
let documents = json!([
{"id": 0, "name": "kefir", "_vectors": { "manual": [0, 0, 0], "handcrafted": [0, 0, 0] }},
{"id": 1, "name": "echo", "_vectors": { "manual": null, "handcrafted": null }},
]);
let (response, code) = index.update_documents(documents, None).await;
snapshot!(code, @"202 Accepted");
index.wait_task(response.uid()).await.succeeded();
let (stats, _code) = index.stats().await;
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 2,
"isIndexing": false,
"numberOfEmbeddings": 2,
"numberOfEmbeddedDocuments": 1,
"fieldDistribution": {
"id": 2,
"name": 2
}
}
"###);
}
#[actix_rt::test]
async fn add_remove_embedded_documents() {
let server = Server::new().await;
let index = server.index("doggo");
let (response, code) = index
.update_settings(json!({
"embedders": {
"manual": {
"source": "userProvided",
"dimensions": 3,
},
"handcrafted": {
"source": "userProvided",
"dimensions": 3,
},
},
}))
.await;
snapshot!(code, @"202 Accepted");
server.wait_task(response.uid()).await.succeeded();
// 2 embedded documents for 5 embeddings in total
let documents = json!([
{"id": 0, "name": "kefir", "_vectors": { "manual": [0, 0, 0], "handcrafted": [0, 0, 0] }},
{"id": 1, "name": "echo", "_vectors": { "manual": [1, 1, 1], "handcrafted": [[1, 1, 1], [2, 2, 2]] }},
]);
let (response, code) = index.add_documents(documents, None).await;
snapshot!(code, @"202 Accepted");
index.wait_task(response.uid()).await.succeeded();
let (stats, _code) = index.stats().await;
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 2,
"isIndexing": false,
"numberOfEmbeddings": 5,
"numberOfEmbeddedDocuments": 2,
"fieldDistribution": {
"id": 2,
"name": 2
}
}
"###);
// delete one embedded document, remaining 1 embedded documents for 3 embeddings in total
let (response, code) = index.delete_document(0).await;
snapshot!(code, @"202 Accepted");
index.wait_task(response.uid()).await.succeeded();
let (stats, _code) = index.stats().await;
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 1,
"isIndexing": false,
"numberOfEmbeddings": 3,
"numberOfEmbeddedDocuments": 1,
"fieldDistribution": {
"id": 1,
"name": 1
}
}
"###);
}
#[actix_rt::test]
async fn update_embedder_settings() {
let server = Server::new().await;
let index = server.index("doggo");
// 2 embedded documents for 3 embeddings in total
// but no embedders are added in the settings yet so we expect 0 embedded documents for 0 embeddings in total
let documents = json!([
{"id": 0, "name": "kefir", "_vectors": { "manual": [0, 0, 0], "handcrafted": [0, 0, 0] }},
{"id": 1, "name": "echo", "_vectors": { "manual": [1, 1, 1], "handcrafted": null }},
]);
let (response, code) = index.add_documents(documents, None).await;
snapshot!(code, @"202 Accepted");
index.wait_task(response.uid()).await.succeeded();
let (stats, _code) = index.stats().await;
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 2,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"id": 2,
"name": 2
}
}
"###);
// add embedders to the settings
// 2 embedded documents for 3 embeddings in total
let (response, code) = index
.update_settings(json!({
"embedders": {
"manual": {
"source": "userProvided",
"dimensions": 3,
},
"handcrafted": {
"source": "userProvided",
"dimensions": 3,
},
},
}))
.await;
snapshot!(code, @"202 Accepted");
server.wait_task(response.uid()).await.succeeded();
let (stats, _code) = index.stats().await;
snapshot!(json_string!(stats), @r###"
{
"numberOfDocuments": 2,
"isIndexing": false,
"numberOfEmbeddings": 3,
"numberOfEmbeddedDocuments": 2,
"fieldDistribution": {
"id": 2,
"name": 2
}
}
"###);
}

View file

@ -126,14 +126,17 @@ async fn check_the_index_scheduler(server: &Server) {
"#);
// And their metadata are still right
let (stats, _) = server.stats().await;
snapshot!(stats, @r#"
snapshot!(stats, @r###"
{
"databaseSize": 438272,
"usedDatabaseSize": 196608,
"lastUpdate": "2025-01-23T11:36:22.634859166Z",
"indexes": {
"kefir": {
"numberOfDocuments": 1,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"age": 1,
"description": 1,
@ -144,7 +147,7 @@ async fn check_the_index_scheduler(server: &Server) {
}
}
}
"#);
"###);
// Wait until the upgrade has been applied to all indexes to avoid flakyness
let (tasks, _) = server.tasks_filter("types=upgradeDatabase&limit=1").await;
@ -205,14 +208,17 @@ async fn check_the_index_scheduler(server: &Server) {
snapshot!(json_string!(batches, { ".results[0].duration" => "[duration]", ".results[0].enqueuedAt" => "[date]", ".results[0].startedAt" => "[date]", ".results[0].finishedAt" => "[date]" }), name: "batches_filter_afterFinishedAt_equal_2025-01-16T16_47_41");
let (stats, _) = server.stats().await;
snapshot!(stats, @r#"
snapshot!(stats, @r###"
{
"databaseSize": 438272,
"usedDatabaseSize": 196608,
"lastUpdate": "2025-01-23T11:36:22.634859166Z",
"indexes": {
"kefir": {
"numberOfDocuments": 1,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"age": 1,
"description": 1,
@ -223,13 +229,15 @@ async fn check_the_index_scheduler(server: &Server) {
}
}
}
"#);
"###);
let index = server.index("kefir");
let (stats, _) = index.stats().await;
snapshot!(stats, @r#"
snapshot!(stats, @r###"
{
"numberOfDocuments": 1,
"isIndexing": false,
"numberOfEmbeddings": 0,
"numberOfEmbeddedDocuments": 0,
"fieldDistribution": {
"age": 1,
"description": 1,
@ -238,7 +246,7 @@ async fn check_the_index_scheduler(server: &Server) {
"surname": 1
}
}
"#);
"###);
// Delete all the tasks of a specific batch
let (task, _) = server.delete_tasks("batchUids=10").await;

View file

@ -32,7 +32,7 @@ async fn field_unavailable_for_source() {
snapshot!(code, @"400 Bad Request");
snapshot!(response, @r###"
{
"message": "`.embedders.default`: Field `revision` unavailable for source `openAi` (only available for sources: `huggingFace`). Available fields: `source`, `model`, `apiKey`, `documentTemplate`, `dimensions`, `distribution`, `url`, `binaryQuantized`",
"message": "`.embedders.default`: Field `revision` unavailable for source `openAi` (only available for sources: `huggingFace`). Available fields: `source`, `model`, `apiKey`, `documentTemplate`, `documentTemplateMaxBytes`, `dimensions`, `distribution`, `url`, `binaryQuantized`",
"code": "invalid_settings_embedders",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#invalid_settings_embedders"