MeiliSearch/meilisearch-http/src/index_controller/mod.rs

457 lines
15 KiB
Rust
Raw Normal View History

2021-05-10 20:25:09 +02:00
use std::collections::BTreeMap;
2021-03-04 12:03:06 +01:00
use std::path::Path;
2021-03-06 12:57:56 +01:00
use std::sync::Arc;
use std::time::Duration;
2021-03-04 12:03:06 +01:00
2021-06-23 13:55:16 +02:00
use actix_web::web::Bytes;
2021-05-10 20:24:14 +02:00
use chrono::{DateTime, Utc};
2021-03-04 12:38:55 +01:00
use futures::stream::StreamExt;
2021-06-16 14:52:06 +02:00
use log::error;
2021-03-24 11:29:11 +01:00
use log::info;
2021-06-21 16:59:27 +02:00
use milli::FieldDistribution;
2021-03-15 18:11:10 +01:00
use serde::{Deserialize, Serialize};
2021-03-12 17:44:39 +01:00
use tokio::sync::mpsc;
2021-03-06 12:57:56 +01:00
use tokio::time::sleep;
use uuid::Uuid;
2021-05-10 20:25:09 +02:00
use dump_actor::DumpActorHandle;
2021-05-26 20:42:09 +02:00
pub use dump_actor::{DumpInfo, DumpStatus};
2021-03-23 11:00:50 +01:00
use index_actor::IndexActorHandle;
2021-05-26 20:42:09 +02:00
use snapshot::{load_snapshot, SnapshotService};
2021-04-01 16:44:42 +02:00
use update_actor::UpdateActorHandle;
2021-05-26 20:42:09 +02:00
pub use updates::*;
2021-06-15 17:55:27 +02:00
use uuid_resolver::{error::UuidResolverError, UuidResolverHandle};
2021-04-01 16:44:42 +02:00
2021-06-23 14:56:02 +02:00
use crate::extractors::payload::Payload;
2021-05-10 17:30:09 +02:00
use crate::index::{Checked, Document, SearchQuery, SearchResult, Settings};
2021-04-01 16:44:42 +02:00
use crate::option::Opt;
2021-06-23 14:48:33 +02:00
use error::Result;
2021-04-01 16:44:42 +02:00
2021-05-27 14:30:20 +02:00
use self::dump_actor::load_dump;
use self::error::IndexControllerError;
2021-05-27 14:30:20 +02:00
2021-05-26 20:42:09 +02:00
mod dump_actor;
2021-06-15 17:39:07 +02:00
pub mod error;
2021-06-21 18:42:47 +02:00
pub mod index_actor;
2021-04-01 16:44:42 +02:00
mod snapshot;
mod update_actor;
mod updates;
mod uuid_resolver;
2021-02-01 19:51:47 +01:00
2021-02-03 17:44:20 +01:00
#[derive(Debug, Serialize, Deserialize, Clone)]
#[serde(rename_all = "camelCase")]
pub struct IndexMetadata {
2021-04-14 18:55:04 +02:00
#[serde(skip)]
pub uuid: Uuid,
2021-04-01 16:44:42 +02:00
pub uid: String,
2021-03-15 18:35:16 +01:00
name: String,
2021-03-06 20:12:20 +01:00
#[serde(flatten)]
2021-04-01 16:44:42 +02:00
pub meta: index_actor::IndexMeta,
2021-02-03 17:44:20 +01:00
}
2021-02-09 16:08:13 +01:00
#[derive(Clone, Debug)]
pub struct IndexSettings {
2021-03-11 22:47:29 +01:00
pub uid: Option<String>,
pub primary_key: Option<String>,
}
2021-03-04 12:03:06 +01:00
2021-06-23 12:18:34 +02:00
#[derive(Serialize, Debug)]
2021-04-15 19:54:25 +02:00
#[serde(rename_all = "camelCase")]
2021-04-01 16:44:42 +02:00
pub struct IndexStats {
2021-04-14 18:55:04 +02:00
#[serde(skip)]
2021-04-01 16:44:42 +02:00
pub size: u64,
pub number_of_documents: u64,
2021-04-22 10:14:29 +02:00
/// Whether the current index is performing an update. It is initially `None` when the
/// index returns it, since it is the `UpdateStore` that knows what index is currently indexing. It is
/// later set to either true or false, we we retrieve the information from the `UpdateStore`
2021-04-14 18:55:04 +02:00
pub is_indexing: Option<bool>,
2021-06-21 16:59:27 +02:00
pub field_distribution: FieldDistribution,
2021-04-01 16:44:42 +02:00
}
2021-05-10 20:25:09 +02:00
#[derive(Clone)]
2021-03-04 12:03:06 +01:00
pub struct IndexController {
2021-03-23 11:00:50 +01:00
uuid_resolver: uuid_resolver::UuidResolverHandleImpl,
index_handle: index_actor::IndexActorHandleImpl,
update_handle: update_actor::UpdateActorHandleImpl<Bytes>,
2021-05-10 20:25:09 +02:00
dump_handle: dump_actor::DumpActorHandleImpl,
2021-03-04 12:03:06 +01:00
}
2021-06-23 12:18:34 +02:00
#[derive(Serialize, Debug)]
2021-04-15 19:54:25 +02:00
#[serde(rename_all = "camelCase")]
2021-04-14 18:55:04 +02:00
pub struct Stats {
pub database_size: u64,
pub last_update: Option<DateTime<Utc>>,
pub indexes: BTreeMap<String, IndexStats>,
}
2021-03-04 12:03:06 +01:00
impl IndexController {
2021-06-15 17:39:07 +02:00
pub fn new(path: impl AsRef<Path>, options: &Opt) -> anyhow::Result<Self> {
2021-06-16 19:50:15 +02:00
let index_size = options.max_index_size.get_bytes() as usize;
let update_store_size = options.max_index_size.get_bytes() as usize;
2021-03-17 12:01:56 +01:00
2021-03-22 19:19:37 +01:00
if let Some(ref path) = options.import_snapshot {
2021-03-24 11:29:11 +01:00
info!("Loading from snapshot {:?}", path);
2021-03-23 16:37:46 +01:00
load_snapshot(
&options.db_path,
path,
options.ignore_snapshot_if_db_exists,
options.ignore_missing_snapshot,
)?;
2021-05-27 14:30:20 +02:00
} else if let Some(ref src_path) = options.import_dump {
load_dump(
&options.db_path,
src_path,
2021-06-16 19:50:15 +02:00
options.max_index_size.get_bytes() as usize,
2021-05-31 16:40:59 +02:00
options.max_udb_size.get_bytes() as usize,
2021-05-27 14:30:20 +02:00
&options.indexer_options,
)?;
2021-03-22 19:19:37 +01:00
}
2021-03-23 17:23:57 +01:00
std::fs::create_dir_all(&path)?;
2021-03-23 11:00:50 +01:00
let uuid_resolver = uuid_resolver::UuidResolverHandleImpl::new(&path)?;
let index_handle =
index_actor::IndexActorHandleImpl::new(&path, index_size, &options.indexer_options)?;
2021-03-23 11:00:50 +01:00
let update_handle = update_actor::UpdateActorHandleImpl::new(
2021-03-22 16:58:19 +01:00
index_handle.clone(),
&path,
update_store_size,
)?;
2021-05-26 20:42:09 +02:00
let dump_handle = dump_actor::DumpActorHandleImpl::new(
&options.dumps_dir,
uuid_resolver.clone(),
update_handle.clone(),
2021-06-16 19:50:15 +02:00
options.max_index_size.get_bytes() as usize,
2021-05-31 16:40:59 +02:00
options.max_udb_size.get_bytes() as usize,
2021-05-26 20:42:09 +02:00
)?;
2021-03-17 12:01:56 +01:00
if options.schedule_snapshot {
let snapshot_service = SnapshotService::new(
uuid_resolver.clone(),
update_handle.clone(),
Duration::from_secs(options.snapshot_interval_sec),
2021-03-22 10:17:38 +01:00
options.snapshot_dir.clone(),
2021-04-01 16:44:42 +02:00
options
.db_path
.file_name()
.map(|n| n.to_owned().into_string().expect("invalid path"))
.unwrap_or_else(|| String::from("data.ms")),
2021-03-17 12:01:56 +01:00
);
tokio::task::spawn(snapshot_service.run());
}
2021-03-15 18:11:10 +01:00
Ok(Self {
uuid_resolver,
2021-03-17 11:53:23 +01:00
index_handle,
2021-03-15 18:11:10 +01:00
update_handle,
2021-05-10 20:25:09 +02:00
dump_handle,
2021-03-15 18:11:10 +01:00
})
2021-03-04 12:03:06 +01:00
}
pub async fn add_documents(
&self,
2021-03-11 22:47:29 +01:00
uid: String,
2021-03-04 12:03:06 +01:00
method: milli::update::IndexDocumentsMethod,
format: milli::update::UpdateFormat,
payload: Payload,
2021-03-04 12:03:06 +01:00
primary_key: Option<String>,
) -> Result<UpdateStatus> {
2021-03-22 19:25:56 +01:00
let perform_update = |uuid| async move {
let meta = UpdateMeta::DocumentsAddition {
method,
format,
primary_key,
};
let (sender, receiver) = mpsc::channel(10);
2021-03-22 19:25:56 +01:00
// It is necessary to spawn a local task to send the payload to the update handle to
// prevent dead_locking between the update_handle::update that waits for the update to be
// registered and the update_actor that waits for the the payload to be sent to it.
tokio::task::spawn_local(async move {
2021-05-26 20:42:09 +02:00
payload
.for_each(|r| async {
let _ = sender.send(r).await;
})
.await
});
2021-03-04 12:03:06 +01:00
// This must be done *AFTER* spawning the task.
self.update_handle.update(meta, receiver, uuid).await
2021-03-15 18:11:10 +01:00
};
2021-03-04 12:03:06 +01:00
match self.uuid_resolver.get(uid).await {
2021-03-22 19:25:56 +01:00
Ok(uuid) => Ok(perform_update(uuid).await?),
2021-05-24 16:05:43 +02:00
Err(UuidResolverError::UnexistingIndex(name)) => {
let uuid = Uuid::new_v4();
2021-03-22 19:25:56 +01:00
let status = perform_update(uuid).await?;
2021-04-22 10:14:29 +02:00
// ignore if index creation fails now, since it may already have been created
let _ = self.index_handle.create_index(uuid, None).await;
self.uuid_resolver.insert(name, uuid).await?;
Ok(status)
2021-03-04 12:03:06 +01:00
}
Err(e) => Err(e.into()),
}
2021-03-04 12:03:06 +01:00
}
pub async fn clear_documents(&self, uid: String) -> Result<UpdateStatus> {
let uuid = self.uuid_resolver.get(uid).await?;
2021-03-04 16:04:12 +01:00
let meta = UpdateMeta::ClearDocuments;
let (_, receiver) = mpsc::channel(1);
let status = self.update_handle.update(meta, receiver, uuid).await?;
Ok(status)
2021-03-04 12:03:06 +01:00
}
2021-03-15 18:11:10 +01:00
pub async fn delete_documents(
&self,
uid: String,
2021-06-09 17:10:10 +02:00
documents: Vec<String>,
) -> Result<UpdateStatus> {
let uuid = self.uuid_resolver.get(uid).await?;
2021-06-10 15:55:44 +02:00
let meta = UpdateMeta::DeleteDocuments { ids: documents };
2021-06-09 17:10:10 +02:00
let (_, receiver) = mpsc::channel(1);
2021-03-04 15:59:18 +01:00
let status = self.update_handle.update(meta, receiver, uuid).await?;
Ok(status)
2021-03-04 12:03:06 +01:00
}
2021-03-15 18:11:10 +01:00
pub async fn update_settings(
&self,
uid: String,
2021-05-10 17:30:09 +02:00
settings: Settings<Checked>,
2021-03-15 18:11:10 +01:00
create: bool,
) -> Result<UpdateStatus> {
let perform_udpate = |uuid| async move {
let meta = UpdateMeta::Settings(settings.into_unchecked());
// Nothing so send, drop the sender right away, as not to block the update actor.
let (_, receiver) = mpsc::channel(1);
self.update_handle.update(meta, receiver, uuid).await
};
2021-03-04 12:20:14 +01:00
match self.uuid_resolver.get(uid).await {
Ok(uuid) => Ok(perform_udpate(uuid).await?),
2021-05-24 16:05:43 +02:00
Err(UuidResolverError::UnexistingIndex(name)) if create => {
let uuid = Uuid::new_v4();
let status = perform_udpate(uuid).await?;
2021-04-22 10:14:29 +02:00
// ignore if index creation fails now, since it may already have been created
let _ = self.index_handle.create_index(uuid, None).await;
self.uuid_resolver.insert(name, uuid).await?;
Ok(status)
}
Err(e) => Err(e.into()),
}
2021-03-04 12:03:06 +01:00
}
2021-06-15 17:39:07 +02:00
pub async fn create_index(&self, index_settings: IndexSettings) -> Result<IndexMetadata> {
2021-03-15 18:11:10 +01:00
let IndexSettings { uid, primary_key } = index_settings;
let uid = uid.ok_or(IndexControllerError::MissingUid)?;
2021-06-09 11:52:36 +02:00
let uuid = Uuid::new_v4();
2021-03-06 20:12:20 +01:00
let meta = self.index_handle.create_index(uuid, primary_key).await?;
2021-06-09 11:52:36 +02:00
self.uuid_resolver.insert(uid.clone(), uuid).await?;
2021-03-22 10:17:38 +01:00
let meta = IndexMetadata {
2021-04-14 18:55:04 +02:00
uuid,
2021-03-22 10:17:38 +01:00
name: uid.clone(),
uid,
meta,
};
2021-03-06 20:12:20 +01:00
Ok(meta)
2021-03-04 12:03:06 +01:00
}
pub async fn delete_index(&self, uid: String) -> Result<()> {
2021-03-15 18:11:10 +01:00
let uuid = self.uuid_resolver.delete(uid).await?;
2021-06-16 14:52:06 +02:00
// We remove the index from the resolver synchronously, and effectively perform the index
// deletion as a background task.
let update_handle = self.update_handle.clone();
let index_handle = self.index_handle.clone();
tokio::spawn(async move {
if let Err(e) = update_handle.delete(uuid).await {
error!("Error while deleting index: {}", e);
}
if let Err(e) = index_handle.delete(uuid).await {
error!("Error while deleting index: {}", e);
}
});
2021-03-06 12:57:56 +01:00
Ok(())
2021-03-04 12:03:06 +01:00
}
pub async fn update_status(&self, uid: String, id: u64) -> Result<UpdateStatus> {
let uuid = self.uuid_resolver.get(uid).await?;
2021-03-06 10:51:52 +01:00
let result = self.update_handle.update_status(uuid, id).await?;
Ok(result)
2021-03-04 12:03:06 +01:00
}
pub async fn all_update_status(&self, uid: String) -> Result<Vec<UpdateStatus>> {
let uuid = self.uuid_resolver.get(uid).await?;
2021-03-05 18:34:04 +01:00
let result = self.update_handle.get_all_updates_status(uuid).await?;
Ok(result)
2021-03-04 12:03:06 +01:00
}
pub async fn list_indexes(&self) -> Result<Vec<IndexMetadata>> {
2021-03-06 20:12:20 +01:00
let uuids = self.uuid_resolver.list().await?;
let mut ret = Vec::new();
2021-03-11 22:47:29 +01:00
for (uid, uuid) in uuids {
2021-03-15 16:52:05 +01:00
let meta = self.index_handle.get_index_meta(uuid).await?;
2021-03-22 10:17:38 +01:00
let meta = IndexMetadata {
2021-04-14 18:55:04 +02:00
uuid,
2021-03-22 10:17:38 +01:00
name: uid.clone(),
uid,
meta,
};
2021-03-15 16:52:05 +01:00
ret.push(meta);
2021-03-06 20:12:20 +01:00
}
Ok(ret)
2021-03-04 12:03:06 +01:00
}
pub async fn settings(&self, uid: String) -> Result<Settings<Checked>> {
let uuid = self.uuid_resolver.get(uid.clone()).await?;
2021-03-04 12:38:55 +01:00
let settings = self.index_handle.settings(uuid).await?;
Ok(settings)
}
2021-03-04 14:20:19 +01:00
pub async fn documents(
&self,
2021-03-11 22:47:29 +01:00
uid: String,
2021-03-04 14:20:19 +01:00
offset: usize,
limit: usize,
attributes_to_retrieve: Option<Vec<String>>,
) -> Result<Vec<Document>> {
let uuid = self.uuid_resolver.get(uid.clone()).await?;
2021-03-15 18:11:10 +01:00
let documents = self
.index_handle
.documents(uuid, offset, limit, attributes_to_retrieve)
2021-03-08 16:27:29 +01:00
.await?;
2021-03-04 14:20:19 +01:00
Ok(documents)
}
2021-03-04 15:09:00 +01:00
pub async fn document(
&self,
2021-03-11 22:47:29 +01:00
uid: String,
2021-03-04 15:09:00 +01:00
doc_id: String,
attributes_to_retrieve: Option<Vec<String>>,
) -> Result<Document> {
let uuid = self.uuid_resolver.get(uid.clone()).await?;
2021-03-15 18:11:10 +01:00
let document = self
.index_handle
.document(uuid, doc_id, attributes_to_retrieve)
2021-03-08 16:27:29 +01:00
.await?;
2021-03-04 15:09:00 +01:00
Ok(document)
}
2021-03-15 18:11:10 +01:00
pub async fn update_index(
&self,
uid: String,
2021-06-21 13:57:32 +02:00
mut index_settings: IndexSettings,
) -> Result<IndexMetadata> {
2021-03-12 14:48:43 +01:00
if index_settings.uid.is_some() {
2021-06-21 13:57:32 +02:00
index_settings.uid.take();
2021-03-12 14:48:43 +01:00
}
let uuid = self.uuid_resolver.get(uid.clone()).await?;
2021-03-12 14:48:43 +01:00
let meta = self.index_handle.update_index(uuid, index_settings).await?;
2021-03-22 10:17:38 +01:00
let meta = IndexMetadata {
2021-04-14 18:55:04 +02:00
uuid,
2021-03-22 10:17:38 +01:00
name: uid.clone(),
uid,
meta,
};
2021-03-12 14:48:43 +01:00
Ok(meta)
2021-03-04 12:03:06 +01:00
}
pub async fn search(&self, uid: String, query: SearchQuery) -> Result<SearchResult> {
let uuid = self.uuid_resolver.get(uid).await?;
2021-03-04 12:03:06 +01:00
let result = self.index_handle.search(uuid, query).await?;
Ok(result)
}
2021-03-06 20:17:58 +01:00
pub async fn get_index(&self, uid: String) -> Result<IndexMetadata> {
let uuid = self.uuid_resolver.get(uid.clone()).await?;
2021-03-15 18:11:10 +01:00
let meta = self.index_handle.get_index_meta(uuid).await?;
2021-03-22 10:17:38 +01:00
let meta = IndexMetadata {
2021-04-14 18:55:04 +02:00
uuid,
2021-03-22 10:17:38 +01:00
name: uid.clone(),
uid,
meta,
};
2021-03-15 16:52:05 +01:00
Ok(meta)
2021-03-06 20:17:58 +01:00
}
2021-04-01 16:44:42 +02:00
pub async fn get_uuids_size(&self) -> Result<u64> {
2021-04-14 18:55:04 +02:00
Ok(self.uuid_resolver.get_size().await?)
2021-04-01 16:44:42 +02:00
}
pub async fn get_index_stats(&self, uid: String) -> Result<IndexStats> {
2021-04-14 18:55:04 +02:00
let uuid = self.uuid_resolver.get(uid).await?;
let update_infos = self.update_handle.get_info().await?;
let mut stats = self.index_handle.get_index_stats(uuid).await?;
2021-04-22 10:14:29 +02:00
// Check if the currently indexing update is from out index.
stats.is_indexing = Some(Some(uuid) == update_infos.processing);
2021-04-14 18:55:04 +02:00
Ok(stats)
}
pub async fn get_all_stats(&self) -> Result<Stats> {
2021-04-14 18:55:04 +02:00
let update_infos = self.update_handle.get_info().await?;
let mut database_size = self.get_uuids_size().await? + update_infos.size;
let mut last_update: Option<DateTime<_>> = None;
let mut indexes = BTreeMap::new();
for index in self.list_indexes().await? {
let mut index_stats = self.index_handle.get_index_stats(index.uuid).await?;
database_size += index_stats.size;
last_update = last_update.map_or(Some(index.meta.updated_at), |last| {
Some(last.max(index.meta.updated_at))
});
2021-04-22 10:14:29 +02:00
index_stats.is_indexing = Some(Some(index.uuid) == update_infos.processing);
2021-04-14 18:55:04 +02:00
indexes.insert(index.uid, index_stats);
}
Ok(Stats {
database_size,
last_update,
indexes,
})
}
2021-05-10 20:25:09 +02:00
pub async fn create_dump(&self) -> Result<DumpInfo> {
2021-05-10 20:25:09 +02:00
Ok(self.dump_handle.create_dump().await?)
}
pub async fn dump_info(&self, uid: String) -> Result<DumpInfo> {
2021-05-10 20:25:09 +02:00
Ok(self.dump_handle.dump_info(uid).await?)
}
2021-03-04 12:03:06 +01:00
}
2021-03-06 12:57:56 +01:00
pub async fn get_arc_ownership_blocking<T>(mut item: Arc<T>) -> T {
loop {
match Arc::try_unwrap(item) {
Ok(item) => return item,
Err(item_arc) => {
item = item_arc;
sleep(Duration::from_millis(100)).await;
continue;
}
}
}
}
/// Parses the v1 version of the Asc ranking rules `asc(price)`and returns the field name.
pub fn asc_ranking_rule(text: &str) -> Option<&str> {
text.split_once("asc(")
.and_then(|(_, tail)| tail.rsplit_once(")"))
.map(|(field, _)| field)
}
/// Parses the v1 version of the Desc ranking rules `asc(price)`and returns the field name.
pub fn desc_ranking_rule(text: &str) -> Option<&str> {
text.split_once("desc(")
.and_then(|(_, tail)| tail.rsplit_once(")"))
.map(|(field, _)| field)
}