Marin Postma 2021-12-02 16:03:26 +01:00
parent c9f3726447
commit a30e02c18c
88 changed files with 5553 additions and 4496 deletions

View file

@ -1,5 +1,6 @@
use std::collections::BTreeMap;
use std::fmt;
use std::io::Cursor;
use std::path::{Path, PathBuf};
use std::sync::Arc;
use std::time::Duration;
@ -8,44 +9,39 @@ use actix_web::error::PayloadError;
use bytes::Bytes;
use chrono::{DateTime, Utc};
use futures::Stream;
use log::info;
use futures::StreamExt;
use milli::update::IndexDocumentsMethod;
use serde::{Deserialize, Serialize};
use tokio::sync::mpsc;
use tokio::task::spawn_blocking;
use tokio::time::sleep;
use uuid::Uuid;
use dump_actor::DumpActorHandle;
pub use dump_actor::{DumpInfo, DumpStatus};
use snapshot::load_snapshot;
use crate::index::error::Result as IndexResult;
use crate::document_formats::{read_csv, read_json, read_ndjson};
use crate::index::{
Checked, Document, IndexMeta, IndexStats, SearchQuery, SearchResult, Settings, Unchecked,
};
use crate::index_controller::index_resolver::create_index_resolver;
use crate::index_controller::snapshot::SnapshotService;
use crate::index_controller::dump_actor::{load_dump, DumpActor, DumpActorHandleImpl};
use crate::options::IndexerOpts;
use crate::snapshot::{load_snapshot, SnapshotService};
use crate::tasks::create_task_store;
use crate::tasks::error::TaskError;
use crate::tasks::task::{DocumentDeletion, Task, TaskContent, TaskId};
use crate::tasks::{TaskFilter, TaskStore};
use error::Result;
use self::dump_actor::load_dump;
use self::index_resolver::error::IndexResolverError;
use self::index_resolver::index_store::{IndexStore, MapIndexStore};
use self::index_resolver::uuid_store::{HeedUuidStore, UuidStore};
use self::index_resolver::IndexResolver;
use self::updates::status::UpdateStatus;
use self::updates::UpdateMsg;
use self::dump_actor::{DumpActorHandle, DumpInfo};
use self::error::IndexControllerError;
use crate::index_resolver::index_store::{IndexStore, MapIndexStore};
use crate::index_resolver::meta_store::{HeedMetaStore, IndexMetaStore};
use crate::index_resolver::{create_index_resolver, IndexResolver, IndexUid};
use crate::update_file_store::UpdateFileStore;
mod dump_actor;
pub mod error;
mod index_resolver;
mod snapshot;
pub mod update_file_store;
pub mod updates;
/// Concrete implementation of the IndexController, exposed by meilisearch-lib
pub type MeiliSearch =
IndexController<HeedUuidStore, MapIndexStore, dump_actor::DumpActorHandleImpl>;
pub type MeiliSearch = IndexController<HeedMetaStore, MapIndexStore>;
pub type Payload = Box<
dyn Stream<Item = std::result::Result<Bytes, PayloadError>> + Send + Sync + 'static + Unpin,
@ -68,6 +64,25 @@ pub struct IndexSettings {
pub primary_key: Option<String>,
}
pub struct IndexController<U, I> {
index_resolver: Arc<IndexResolver<U, I>>,
task_store: TaskStore,
dump_handle: dump_actor::DumpActorHandleImpl,
update_file_store: UpdateFileStore,
}
/// Need a custom implementation for clone because deriving require that U and I are clone.
impl<U, I> Clone for IndexController<U, I> {
fn clone(&self) -> Self {
Self {
index_resolver: self.index_resolver.clone(),
task_store: self.task_store.clone(),
dump_handle: self.dump_handle.clone(),
update_file_store: self.update_file_store.clone(),
}
}
}
#[derive(Debug)]
pub enum DocumentAdditionFormat {
Json,
@ -99,7 +114,11 @@ pub struct Stats {
pub enum Update {
DeleteDocuments(Vec<String>),
ClearDocuments,
Settings(Settings<Unchecked>),
Settings {
settings: Settings<Unchecked>,
/// Indicates whether the update was a deletion
is_deletion: bool,
},
DocumentAddition {
#[derivative(Debug = "ignore")]
payload: Payload,
@ -107,12 +126,19 @@ pub enum Update {
method: IndexDocumentsMethod,
format: DocumentAdditionFormat,
},
DeleteIndex,
CreateIndex {
primary_key: Option<String>,
},
UpdateIndex {
primary_key: Option<String>,
},
}
#[derive(Default, Debug)]
pub struct IndexControllerBuilder {
max_index_size: Option<usize>,
max_update_store_size: Option<usize>,
max_task_store_size: Option<usize>,
snapshot_dir: Option<PathBuf>,
import_snapshot: Option<PathBuf>,
snapshot_interval: Option<Duration>,
@ -132,12 +158,12 @@ impl IndexControllerBuilder {
let index_size = self
.max_index_size
.ok_or_else(|| anyhow::anyhow!("Missing index size"))?;
let update_store_size = self
.max_index_size
let task_store_size = self
.max_task_store_size
.ok_or_else(|| anyhow::anyhow!("Missing update database size"))?;
if let Some(ref path) = self.import_snapshot {
info!("Loading from snapshot {:?}", path);
log::info!("Loading from snapshot {:?}", path);
load_snapshot(
db_path.as_ref(),
path,
@ -149,67 +175,84 @@ impl IndexControllerBuilder {
db_path.as_ref(),
src_path,
index_size,
update_store_size,
task_store_size,
&indexer_options,
)?;
}
std::fs::create_dir_all(db_path.as_ref())?;
let mut options = heed::EnvOpenOptions::new();
options.map_size(task_store_size);
options.max_dbs(20);
let meta_env = options.open(&db_path)?;
let update_file_store = UpdateFileStore::new(&db_path)?;
let index_resolver = Arc::new(create_index_resolver(
&db_path,
index_size,
&indexer_options,
meta_env.clone(),
update_file_store.clone(),
)?);
#[allow(unreachable_code)]
let update_sender =
updates::create_update_handler(index_resolver.clone(), &db_path, update_store_size)?;
let task_store =
create_task_store(meta_env, index_resolver.clone()).map_err(|e| anyhow::anyhow!(e))?;
let dump_path = self
.dump_dst
.ok_or_else(|| anyhow::anyhow!("Missing dump directory path"))?;
let analytics_path = db_path.as_ref().join("instance-uid");
let dump_handle = dump_actor::DumpActorHandleImpl::new(
dump_path,
analytics_path,
index_resolver.clone(),
update_sender.clone(),
index_size,
update_store_size,
)?;
let dump_handle = {
let analytics_path = &db_path;
let (sender, receiver) = mpsc::channel(10);
let actor = DumpActor::new(
receiver,
update_file_store.clone(),
task_store.clone(),
dump_path,
analytics_path,
index_size,
task_store_size,
);
let dump_handle = Arc::new(dump_handle);
tokio::task::spawn(actor.run());
DumpActorHandleImpl { sender }
};
if self.schedule_snapshot {
let snapshot_service = SnapshotService::new(
index_resolver.clone(),
update_sender.clone(),
self.snapshot_interval
.ok_or_else(|| anyhow::anyhow!("Snapshot interval not provided."))?,
self.snapshot_dir
.ok_or_else(|| anyhow::anyhow!("Snapshot path not provided."))?,
db_path.as_ref().into(),
db_path
.as_ref()
.file_name()
.map(|n| n.to_owned().into_string().expect("invalid path"))
.unwrap_or_else(|| String::from("data.ms")),
);
let snapshot_period = self
.snapshot_interval
.ok_or_else(|| anyhow::anyhow!("Snapshot interval not provided."))?;
let snapshot_path = self
.snapshot_dir
.ok_or_else(|| anyhow::anyhow!("Snapshot path not provided."))?;
let snapshot_service = SnapshotService {
db_path: db_path.as_ref().to_path_buf(),
snapshot_period,
snapshot_path,
index_size,
meta_env_size: task_store_size,
task_store: task_store.clone(),
};
tokio::task::spawn(snapshot_service.run());
}
Ok(IndexController {
index_resolver,
update_sender,
task_store,
dump_handle,
update_file_store,
})
}
/// Set the index controller builder's max update store size.
pub fn set_max_update_store_size(&mut self, max_update_store_size: usize) -> &mut Self {
self.max_update_store_size.replace(max_update_store_size);
pub fn set_max_task_store_size(&mut self, max_update_store_size: usize) -> &mut Self {
self.max_task_store_size.replace(max_update_store_size);
self
}
@ -270,61 +313,133 @@ impl IndexControllerBuilder {
}
}
// We are using derivative here to derive Clone, because U, I and D do not necessarily implement
// Clone themselves.
#[derive(derivative::Derivative)]
#[derivative(Clone(bound = ""))]
pub struct IndexController<U, I, D> {
index_resolver: Arc<IndexResolver<U, I>>,
update_sender: updates::UpdateSender,
dump_handle: Arc<D>,
}
impl<U, I, D> IndexController<U, I, D>
impl<U, I> IndexController<U, I>
where
U: UuidStore + Sync + Send + 'static,
I: IndexStore + Sync + Send + 'static,
D: DumpActorHandle + Send + Sync,
U: IndexMetaStore,
I: IndexStore,
{
pub fn builder() -> IndexControllerBuilder {
IndexControllerBuilder::default()
}
pub async fn register_update(
&self,
uid: String,
update: Update,
create_index: bool,
) -> Result<UpdateStatus> {
match self.index_resolver.get_uuid(uid).await {
Ok(uuid) => {
let update_result = UpdateMsg::update(&self.update_sender, uuid, update).await?;
Ok(update_result)
pub async fn register_update(&self, uid: String, update: Update) -> Result<Task> {
let uid = IndexUid::new(uid)?;
let content = match update {
Update::DeleteDocuments(ids) => {
TaskContent::DocumentDeletion(DocumentDeletion::Ids(ids))
}
Err(IndexResolverError::UnexistingIndex(name)) => {
if create_index {
let index = self.index_resolver.create_index(name, None).await?;
let update_result =
UpdateMsg::update(&self.update_sender, index.uuid(), update).await?;
Ok(update_result)
} else {
Err(IndexResolverError::UnexistingIndex(name).into())
Update::ClearDocuments => TaskContent::DocumentDeletion(DocumentDeletion::Clear),
Update::Settings {
settings,
is_deletion,
} => TaskContent::SettingsUpdate {
settings,
is_deletion,
},
Update::DocumentAddition {
mut payload,
primary_key,
format,
method,
} => {
let mut buffer = Vec::new();
while let Some(bytes) = payload.next().await {
let bytes = bytes?;
buffer.extend_from_slice(&bytes);
}
let (content_uuid, mut update_file) = self.update_file_store.new_update()?;
let documents_count = tokio::task::spawn_blocking(move || -> Result<_> {
// check if the payload is empty, and return an error
if buffer.is_empty() {
return Err(IndexControllerError::MissingPayload(format));
}
let reader = Cursor::new(buffer);
let count = match format {
DocumentAdditionFormat::Json => read_json(reader, &mut *update_file)?,
DocumentAdditionFormat::Csv => read_csv(reader, &mut *update_file)?,
DocumentAdditionFormat::Ndjson => read_ndjson(reader, &mut *update_file)?,
};
update_file.persist()?;
Ok(count)
})
.await??;
TaskContent::DocumentAddition {
content_uuid,
merge_strategy: method,
primary_key,
documents_count,
}
}
Err(e) => Err(e.into()),
Update::DeleteIndex => TaskContent::IndexDeletion,
Update::CreateIndex { primary_key } => TaskContent::IndexCreation { primary_key },
Update::UpdateIndex { primary_key } => TaskContent::IndexUpdate { primary_key },
};
let task = self.task_store.register(uid, content).await?;
Ok(task)
}
pub async fn get_task(&self, id: TaskId, filter: Option<TaskFilter>) -> Result<Task> {
let task = self.task_store.get_task(id, filter).await?;
Ok(task)
}
pub async fn get_index_task(&self, index_uid: String, task_id: TaskId) -> Result<Task> {
let creation_task_id = self
.index_resolver
.get_index_creation_task_id(index_uid.clone())
.await?;
if task_id < creation_task_id {
return Err(TaskError::UnexistingTask(task_id).into());
}
let mut filter = TaskFilter::default();
filter.filter_index(index_uid);
let task = self.task_store.get_task(task_id, Some(filter)).await?;
Ok(task)
}
pub async fn update_status(&self, uid: String, id: u64) -> Result<UpdateStatus> {
let uuid = self.index_resolver.get_uuid(uid).await?;
let result = UpdateMsg::get_update(&self.update_sender, uuid, id).await?;
Ok(result)
pub async fn list_tasks(
&self,
filter: Option<TaskFilter>,
limit: Option<usize>,
offset: Option<TaskId>,
) -> Result<Vec<Task>> {
let tasks = self.task_store.list_tasks(offset, filter, limit).await?;
Ok(tasks)
}
pub async fn all_update_status(&self, uid: String) -> Result<Vec<UpdateStatus>> {
let uuid = self.index_resolver.get_uuid(uid).await?;
let result = UpdateMsg::list_updates(&self.update_sender, uuid).await?;
Ok(result)
pub async fn list_index_task(
&self,
index_uid: String,
limit: Option<usize>,
offset: Option<TaskId>,
) -> Result<Vec<Task>> {
let task_id = self
.index_resolver
.get_index_creation_task_id(index_uid.clone())
.await?;
let mut filter = TaskFilter::default();
filter.filter_index(index_uid);
let tasks = self
.task_store
.list_tasks(
Some(offset.unwrap_or_default() + task_id),
Some(filter),
limit,
)
.await?;
Ok(tasks)
}
pub async fn list_indexes(&self) -> Result<Vec<IndexMetadata>> {
@ -377,28 +492,8 @@ where
Ok(document)
}
pub async fn update_index(
&self,
uid: String,
mut index_settings: IndexSettings,
) -> Result<IndexMetadata> {
index_settings.uid.take();
let index = self.index_resolver.get_index(uid.clone()).await?;
let uuid = index.uuid();
let meta =
spawn_blocking(move || index.update_primary_key(index_settings.primary_key)).await??;
let meta = IndexMetadata {
uuid,
name: uid.clone(),
uid,
meta,
};
Ok(meta)
}
pub async fn search(&self, uid: String, query: SearchQuery) -> Result<SearchResult> {
let index = self.index_resolver.get_index(uid.clone()).await?;
let index = self.index_resolver.get_index(uid).await?;
let result = spawn_blocking(move || index.perform_search(query)).await??;
Ok(result)
}
@ -417,45 +512,50 @@ where
}
pub async fn get_index_stats(&self, uid: String) -> Result<IndexStats> {
let update_infos = UpdateMsg::get_info(&self.update_sender).await?;
let index = self.index_resolver.get_index(uid).await?;
let uuid = index.uuid();
let mut stats = spawn_blocking(move || index.stats()).await??;
let last_task = self.task_store.get_processing_task().await?;
// Check if the currently indexing update is from our index.
stats.is_indexing = Some(Some(uuid) == update_infos.processing);
let is_indexing = last_task
.map(|task| task.index_uid.into_inner() == uid)
.unwrap_or_default();
let index = self.index_resolver.get_index(uid).await?;
let mut stats = spawn_blocking(move || index.stats()).await??;
stats.is_indexing = Some(is_indexing);
Ok(stats)
}
pub async fn get_all_stats(&self) -> Result<Stats> {
let update_infos = UpdateMsg::get_info(&self.update_sender).await?;
let mut database_size = self.index_resolver.get_uuids_size().await? + update_infos.size;
let mut last_update: Option<DateTime<_>> = None;
let mut last_task: Option<DateTime<_>> = None;
let mut indexes = BTreeMap::new();
let mut database_size = 0;
let processing_task = self.task_store.get_processing_task().await?;
for (index_uid, index) in self.index_resolver.list().await? {
let uuid = index.uuid();
let (mut stats, meta) = spawn_blocking::<_, IndexResult<_>>(move || {
let stats = index.stats()?;
let meta = index.meta()?;
Ok((stats, meta))
})
.await??;
let (mut stats, meta) =
spawn_blocking::<_, Result<(IndexStats, IndexMeta)>>(move || {
Ok((index.stats()?, index.meta()?))
})
.await??;
database_size += stats.size;
last_update = last_update.map_or(Some(meta.updated_at), |last| {
last_task = last_task.map_or(Some(meta.updated_at), |last| {
Some(last.max(meta.updated_at))
});
// Check if the currently indexing update is from our index.
stats.is_indexing = Some(Some(uuid) == update_infos.processing);
stats.is_indexing = processing_task
.as_ref()
.map(|p| p.index_uid.as_str() == index_uid)
.or(Some(false));
indexes.insert(index_uid, stats);
}
Ok(Stats {
database_size,
last_update,
last_update: last_task,
indexes,
})
}
@ -467,41 +567,6 @@ where
pub async fn dump_info(&self, uid: String) -> Result<DumpInfo> {
Ok(self.dump_handle.dump_info(uid).await?)
}
pub async fn create_index(
&self,
uid: String,
primary_key: Option<String>,
) -> Result<IndexMetadata> {
let index = self
.index_resolver
.create_index(uid.clone(), primary_key)
.await?;
let meta = spawn_blocking(move || -> IndexResult<_> {
let meta = index.meta()?;
let meta = IndexMetadata {
uuid: index.uuid(),
uid: uid.clone(),
name: uid,
meta,
};
Ok(meta)
})
.await??;
Ok(meta)
}
pub async fn delete_index(&self, uid: String) -> Result<()> {
let uuid = self.index_resolver.delete_index(uid).await?;
let update_sender = self.update_sender.clone();
tokio::spawn(async move {
let _ = UpdateMsg::delete(&update_sender, uuid).await;
});
Ok(())
}
}
pub async fn get_arc_ownership_blocking<T>(mut item: Arc<T>) -> T {
@ -521,28 +586,28 @@ pub async fn get_arc_ownership_blocking<T>(mut item: Arc<T>) -> T {
mod test {
use futures::future::ok;
use mockall::predicate::eq;
use tokio::sync::mpsc;
use nelson::Mocker;
use crate::index::error::Result as IndexResult;
use crate::index::test::Mocker;
use crate::index::Index;
use crate::index_controller::dump_actor::MockDumpActorHandle;
use crate::index_controller::index_resolver::index_store::MockIndexStore;
use crate::index_controller::index_resolver::uuid_store::MockUuidStore;
use crate::index_resolver::index_store::MockIndexStore;
use crate::index_resolver::meta_store::MockIndexMetaStore;
use crate::index_resolver::IndexResolver;
use super::updates::UpdateSender;
use super::*;
impl<D: DumpActorHandle> IndexController<MockUuidStore, MockIndexStore, D> {
impl IndexController<MockIndexMetaStore, MockIndexStore> {
pub fn mock(
index_resolver: IndexResolver<MockUuidStore, MockIndexStore>,
update_sender: UpdateSender,
dump_handle: D,
index_resolver: IndexResolver<MockIndexMetaStore, MockIndexStore>,
task_store: TaskStore,
update_file_store: UpdateFileStore,
dump_handle: DumpActorHandleImpl,
) -> Self {
IndexController {
index_resolver: Arc::new(index_resolver),
update_sender,
dump_handle: Arc::new(dump_handle),
task_store,
dump_handle,
update_file_store,
}
}
}
@ -577,11 +642,19 @@ mod test {
exhaustive_facets_count: Some(true),
};
let mut uuid_store = MockUuidStore::new();
let mut uuid_store = MockIndexMetaStore::new();
uuid_store
.expect_get_uuid()
.expect_get()
.with(eq(index_uid.to_owned()))
.returning(move |s| Box::pin(ok((s, Some(index_uuid)))));
.returning(move |s| {
Box::pin(ok((
s,
Some(crate::index_resolver::meta_store::IndexMeta {
uuid: index_uuid,
creation_task_id: 0,
}),
)))
});
let mut index_store = MockIndexStore::new();
let result_clone = result.clone();
@ -600,14 +673,20 @@ mod test {
assert_eq!(&q, &query);
Ok(result.clone())
});
let index = Index::faux(mocker);
let index = Index::mock(mocker);
Box::pin(ok(Some(index)))
});
let index_resolver = IndexResolver::new(uuid_store, index_store);
let (update_sender, _) = mpsc::channel(1);
let dump_actor = MockDumpActorHandle::new();
let index_controller = IndexController::mock(index_resolver, update_sender, dump_actor);
let task_store_mocker = nelson::Mocker::default();
let mocker = Mocker::default();
let update_file_store = UpdateFileStore::mock(mocker);
let index_resolver = IndexResolver::new(uuid_store, index_store, update_file_store.clone());
let task_store = TaskStore::mock(task_store_mocker);
// let dump_actor = MockDumpActorHandle::new();
let (sender, _) = mpsc::channel(1);
let dump_handle = DumpActorHandleImpl { sender };
let index_controller =
IndexController::mock(index_resolver, task_store, update_file_store, dump_handle);
let r = index_controller
.search(index_uid.to_owned(), query.clone())