mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-04 20:37:15 +02:00
feat(all): Task store
implements: https://github.com/meilisearch/specifications/blob/develop/text/0060-refashion-updates-apis.md linked PR: - #1889 - #1891 - #1892 - #1902 - #1906 - #1911 - #1914 - #1915 - #1916 - #1918 - #1924 - #1925 - #1926 - #1930 - #1936 - #1937 - #1942 - #1944 - #1945 - #1946 - #1947 - #1950 - #1951 - #1957 - #1959 - #1960 - #1961 - #1962 - #1964 - https://github.com/meilisearch/milli/pull/414 - https://github.com/meilisearch/milli/pull/409 - https://github.com/meilisearch/milli/pull/406 - https://github.com/meilisearch/milli/pull/418 - close #1687 - close #1786 - close #1940 - close #1948 - close #1949 - close #1932 - close #1956
This commit is contained in:
parent
c9f3726447
commit
a30e02c18c
88 changed files with 5553 additions and 4496 deletions
|
@ -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())
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue