mirror of
				https://github.com/meilisearch/meilisearch.git
				synced 2025-10-26 13:36:27 +00:00 
			
		
		
		
	start integrating the index-scheduler in the meilisearch codebase
This commit is contained in:
		| @@ -55,6 +55,9 @@ tokio = { version = "1.21.2", features = ["full"] } | ||||
| uuid = { version = "1.1.2", features = ["serde", "v4"] } | ||||
| walkdir = "2.3.2" | ||||
| whoami = { version = "1.2.3", optional = true } | ||||
| index-scheduler = { path = "../index-scheduler" } | ||||
| index = { path = "../index" } | ||||
| file-store = { path = "../file-store" } | ||||
|  | ||||
| [dev-dependencies] | ||||
| actix-rt = "2.7.0" | ||||
|   | ||||
| @@ -1,61 +0,0 @@ | ||||
| use std::error::Error; | ||||
|  | ||||
| use meilisearch_types::error::{Code, ErrorCode}; | ||||
| use meilisearch_types::internal_error; | ||||
| use serde_json::Value; | ||||
|  | ||||
| use crate::{error::MilliError, update_file_store}; | ||||
|  | ||||
| pub type Result<T> = std::result::Result<T, IndexError>; | ||||
|  | ||||
| #[derive(Debug, thiserror::Error)] | ||||
| pub enum IndexError { | ||||
|     #[error("An internal error has occurred. `{0}`.")] | ||||
|     Internal(Box<dyn Error + Send + Sync + 'static>), | ||||
|     #[error("Document `{0}` not found.")] | ||||
|     DocumentNotFound(String), | ||||
|     #[error("{0}")] | ||||
|     Facet(#[from] FacetError), | ||||
|     #[error("{0}")] | ||||
|     Milli(#[from] milli::Error), | ||||
| } | ||||
|  | ||||
| internal_error!( | ||||
|     IndexError: std::io::Error, | ||||
|     milli::heed::Error, | ||||
|     fst::Error, | ||||
|     serde_json::Error, | ||||
|     update_file_store::UpdateFileStoreError, | ||||
|     milli::documents::Error | ||||
| ); | ||||
|  | ||||
| impl ErrorCode for IndexError { | ||||
|     fn error_code(&self) -> Code { | ||||
|         match self { | ||||
|             IndexError::Internal(_) => Code::Internal, | ||||
|             IndexError::DocumentNotFound(_) => Code::DocumentNotFound, | ||||
|             IndexError::Facet(e) => e.error_code(), | ||||
|             IndexError::Milli(e) => MilliError(e).error_code(), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl From<milli::UserError> for IndexError { | ||||
|     fn from(error: milli::UserError) -> IndexError { | ||||
|         IndexError::Milli(error.into()) | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(Debug, thiserror::Error)] | ||||
| pub enum FacetError { | ||||
|     #[error("Invalid syntax for the filter parameter: `expected {}, found: {1}`.", .0.join(", "))] | ||||
|     InvalidExpression(&'static [&'static str], Value), | ||||
| } | ||||
|  | ||||
| impl ErrorCode for FacetError { | ||||
|     fn error_code(&self) -> Code { | ||||
|         match self { | ||||
|             FacetError::InvalidExpression(_, _) => Code::Filter, | ||||
|         } | ||||
|     } | ||||
| } | ||||
| @@ -7,12 +7,8 @@ use tokio::task::JoinError; | ||||
|  | ||||
| use super::DocumentAdditionFormat; | ||||
| use crate::document_formats::DocumentFormatError; | ||||
| use crate::dump::error::DumpError; | ||||
| use crate::index::error::IndexError; | ||||
| use crate::tasks::error::TaskError; | ||||
| use crate::update_file_store::UpdateFileStoreError; | ||||
|  | ||||
| use crate::index_resolver::error::IndexResolverError; | ||||
| // use crate::dump::error::DumpError; | ||||
| use index::error::IndexError; | ||||
|  | ||||
| pub type Result<T> = std::result::Result<T, IndexControllerError>; | ||||
|  | ||||
| @@ -20,17 +16,15 @@ pub type Result<T> = std::result::Result<T, IndexControllerError>; | ||||
| pub enum IndexControllerError { | ||||
|     #[error("Index creation must have an uid")] | ||||
|     MissingUid, | ||||
|     #[error("{0}")] | ||||
|     IndexResolver(#[from] IndexResolverError), | ||||
|     #[error("{0}")] | ||||
|     #[error(transparent)] | ||||
|     IndexResolver(#[from] index_scheduler::Error), | ||||
|     #[error(transparent)] | ||||
|     IndexError(#[from] IndexError), | ||||
|     #[error("An internal error has occurred. `{0}`.")] | ||||
|     Internal(Box<dyn Error + Send + Sync + 'static>), | ||||
|     #[error("{0}")] | ||||
|     TaskError(#[from] TaskError), | ||||
|     #[error("{0}")] | ||||
|     DumpError(#[from] DumpError), | ||||
|     #[error("{0}")] | ||||
|     // #[error("{0}")] | ||||
|     // DumpError(#[from] DumpError), | ||||
|     #[error(transparent)] | ||||
|     DocumentFormatError(#[from] DocumentFormatError), | ||||
|     #[error("A {0} payload is missing.")] | ||||
|     MissingPayload(DocumentAdditionFormat), | ||||
| @@ -38,7 +32,7 @@ pub enum IndexControllerError { | ||||
|     PayloadTooLarge, | ||||
| } | ||||
|  | ||||
| internal_error!(IndexControllerError: JoinError, UpdateFileStoreError); | ||||
| internal_error!(IndexControllerError: JoinError, file_store::Error); | ||||
|  | ||||
| impl From<actix_web::error::PayloadError> for IndexControllerError { | ||||
|     fn from(other: actix_web::error::PayloadError) -> Self { | ||||
| @@ -53,20 +47,20 @@ impl ErrorCode for IndexControllerError { | ||||
|     fn error_code(&self) -> Code { | ||||
|         match self { | ||||
|             IndexControllerError::MissingUid => Code::BadRequest, | ||||
|             IndexControllerError::IndexResolver(e) => e.error_code(), | ||||
|             IndexControllerError::IndexError(e) => e.error_code(), | ||||
|             IndexControllerError::Internal(_) => Code::Internal, | ||||
|             IndexControllerError::TaskError(e) => e.error_code(), | ||||
|             IndexControllerError::DocumentFormatError(e) => e.error_code(), | ||||
|             IndexControllerError::MissingPayload(_) => Code::MissingPayload, | ||||
|             IndexControllerError::PayloadTooLarge => Code::PayloadTooLarge, | ||||
|             IndexControllerError::DumpError(e) => e.error_code(), | ||||
|             IndexControllerError::IndexResolver(_) => todo!(), | ||||
|             IndexControllerError::IndexError(_) => todo!(), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| /* | ||||
| impl From<IndexUidFormatError> for IndexControllerError { | ||||
|     fn from(err: IndexUidFormatError) -> Self { | ||||
|         IndexResolverError::from(err).into() | ||||
|         index_scheduler::Error::from(err).into() | ||||
|     } | ||||
| } | ||||
| */ | ||||
|   | ||||
| @@ -1,4 +1,3 @@ | ||||
| use meilisearch_auth::SearchRules; | ||||
| use std::collections::BTreeMap; | ||||
| use std::fmt; | ||||
| use std::io::Cursor; | ||||
| @@ -9,10 +8,14 @@ use std::time::Duration; | ||||
|  | ||||
| use actix_web::error::PayloadError; | ||||
| use bytes::Bytes; | ||||
| use file_store::FileStore; | ||||
| use futures::Stream; | ||||
| use futures::StreamExt; | ||||
| use index_scheduler::IndexScheduler; | ||||
| use index_scheduler::TaskKind; | ||||
| use meilisearch_auth::SearchRules; | ||||
| use meilisearch_types::index_uid::IndexUid; | ||||
| use milli::update::IndexDocumentsMethod; | ||||
| use milli::update::{IndexDocumentsMethod, IndexerConfig}; | ||||
| use serde::{Deserialize, Serialize}; | ||||
| use time::OffsetDateTime; | ||||
| use tokio::sync::RwLock; | ||||
| @@ -21,32 +24,19 @@ use tokio::time::sleep; | ||||
| use uuid::Uuid; | ||||
|  | ||||
| use crate::document_formats::{read_csv, read_json, read_ndjson}; | ||||
| use crate::dump::{self, load_dump, DumpHandler}; | ||||
| use crate::index::{ | ||||
|     Checked, Document, IndexMeta, IndexStats, SearchQuery, SearchResult, Settings, Unchecked, | ||||
| }; | ||||
| use crate::index_resolver::error::IndexResolverError; | ||||
| // use crate::dump::{self, load_dump, DumpHandler}; | ||||
| use crate::options::{IndexerOpts, SchedulerConfig}; | ||||
| use crate::snapshot::{load_snapshot, SnapshotService}; | ||||
| use crate::tasks::error::TaskError; | ||||
| use crate::tasks::task::{DocumentDeletion, Task, TaskContent, TaskId}; | ||||
| use crate::tasks::{ | ||||
|     BatchHandler, EmptyBatchHandler, Scheduler, SnapshotHandler, TaskFilter, TaskStore, | ||||
| }; | ||||
| use error::Result; | ||||
| use index::{ | ||||
|     Checked, Document, IndexMeta, IndexStats, SearchQuery, SearchResult, Settings, Unchecked, | ||||
| }; | ||||
|  | ||||
| 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}; | ||||
| use crate::update_file_store::UpdateFileStore; | ||||
|  | ||||
| pub mod error; | ||||
| pub mod versioning; | ||||
|  | ||||
| /// Concrete implementation of the IndexController, exposed by meilisearch-lib | ||||
| pub type MeiliSearch = IndexController<HeedMetaStore, MapIndexStore>; | ||||
|  | ||||
| pub type Payload = Box< | ||||
|     dyn Stream<Item = std::result::Result<Bytes, PayloadError>> + Send + Sync + 'static + Unpin, | ||||
| >; | ||||
| @@ -74,23 +64,9 @@ pub struct IndexSettings { | ||||
|     pub primary_key: Option<String>, | ||||
| } | ||||
|  | ||||
| pub struct IndexController<U, I> { | ||||
|     pub index_resolver: Arc<IndexResolver<U, I>>, | ||||
|     scheduler: Arc<RwLock<Scheduler>>, | ||||
|     task_store: TaskStore, | ||||
|     pub 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(), | ||||
|             scheduler: self.scheduler.clone(), | ||||
|             update_file_store: self.update_file_store.clone(), | ||||
|             task_store: self.task_store.clone(), | ||||
|         } | ||||
|     } | ||||
| #[derive(Clone)] | ||||
| pub struct Meilisearch { | ||||
|     index_scheduler: IndexScheduler, | ||||
| } | ||||
|  | ||||
| #[derive(Debug)] | ||||
| @@ -170,7 +146,7 @@ impl IndexControllerBuilder { | ||||
|         db_path: impl AsRef<Path>, | ||||
|         indexer_options: IndexerOpts, | ||||
|         scheduler_config: SchedulerConfig, | ||||
|     ) -> anyhow::Result<MeiliSearch> { | ||||
|     ) -> anyhow::Result<Meilisearch> { | ||||
|         let index_size = self | ||||
|             .max_index_size | ||||
|             .ok_or_else(|| anyhow::anyhow!("Missing index size"))?; | ||||
| @@ -178,6 +154,8 @@ impl IndexControllerBuilder { | ||||
|             .max_task_store_size | ||||
|             .ok_or_else(|| anyhow::anyhow!("Missing update database size"))?; | ||||
|  | ||||
|         /* | ||||
|         TODO: TAMO: enable dumps and snapshots to happens | ||||
|         if let Some(ref path) = self.import_snapshot { | ||||
|             log::info!("Loading from snapshot {:?}", path); | ||||
|             load_snapshot( | ||||
| @@ -203,47 +181,35 @@ impl IndexControllerBuilder { | ||||
|                 versioning::check_version_file(db_path.as_ref())?; | ||||
|             } | ||||
|         } | ||||
|         */ | ||||
|  | ||||
|         std::fs::create_dir_all(db_path.as_ref())?; | ||||
|  | ||||
|         let meta_env = Arc::new(open_meta_env(db_path.as_ref(), task_store_size)?); | ||||
|  | ||||
|         let update_file_store = UpdateFileStore::new(&db_path)?; | ||||
|         let file_store = FileStore::new(&db_path)?; | ||||
|         // Create or overwrite the version file for this DB | ||||
|         versioning::create_version_file(db_path.as_ref())?; | ||||
|  | ||||
|         let index_resolver = Arc::new(create_index_resolver( | ||||
|             &db_path, | ||||
|         let indexer_config = IndexerConfig { | ||||
|             log_every_n: Some(indexer_options.log_every_n), | ||||
|             max_nb_chunks: indexer_options.max_nb_chunks, | ||||
|             documents_chunk_size: None, | ||||
|             // TODO: TAMO: Fix this thing | ||||
|             max_memory: None, // Some(indexer_options.max_indexing_memory.into()), | ||||
|             chunk_compression_type: milli::CompressionType::None, | ||||
|             chunk_compression_level: None, | ||||
|             // TODO: TAMO: do something with the indexing_config.max_indexing_threads | ||||
|             thread_pool: None, | ||||
|             max_positions_per_attributes: None, | ||||
|         }; | ||||
|  | ||||
|         let scheduler = IndexScheduler::new( | ||||
|             db_path.as_ref().to_path_buf(), | ||||
|             index_size, | ||||
|             &indexer_options, | ||||
|             meta_env.clone(), | ||||
|             update_file_store.clone(), | ||||
|         )?); | ||||
|  | ||||
|         let dump_path = self | ||||
|             .dump_dst | ||||
|             .ok_or_else(|| anyhow::anyhow!("Missing dump directory path"))?; | ||||
|  | ||||
|         let dump_handler = Arc::new(DumpHandler::new( | ||||
|             dump_path, | ||||
|             db_path.as_ref().into(), | ||||
|             update_file_store.clone(), | ||||
|             task_store_size, | ||||
|             index_size, | ||||
|             meta_env.clone(), | ||||
|             index_resolver.clone(), | ||||
|         )); | ||||
|         let task_store = TaskStore::new(meta_env)?; | ||||
|  | ||||
|         // register all the batch handlers for use with the scheduler. | ||||
|         let handlers: Vec<Arc<dyn BatchHandler + Sync + Send + 'static>> = vec![ | ||||
|             index_resolver.clone(), | ||||
|             dump_handler, | ||||
|             Arc::new(SnapshotHandler), | ||||
|             // dummy handler to catch all empty batches | ||||
|             Arc::new(EmptyBatchHandler), | ||||
|         ]; | ||||
|         let scheduler = Scheduler::new(task_store.clone(), handlers, scheduler_config)?; | ||||
|             indexer_config, | ||||
|             file_store, | ||||
|         ); | ||||
|  | ||||
|         if self.schedule_snapshot { | ||||
|             let snapshot_period = self | ||||
| @@ -265,11 +231,8 @@ impl IndexControllerBuilder { | ||||
|             tokio::task::spawn_local(snapshot_service.run()); | ||||
|         } | ||||
|  | ||||
|         Ok(IndexController { | ||||
|             index_resolver, | ||||
|             scheduler, | ||||
|             update_file_store, | ||||
|             task_store, | ||||
|         Ok(Meilisearch { | ||||
|             index_scheduler: scheduler, | ||||
|         }) | ||||
|     } | ||||
|  | ||||
| @@ -350,100 +313,13 @@ impl IndexControllerBuilder { | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<U, I> IndexController<U, I> | ||||
| where | ||||
|     U: IndexMetaStore, | ||||
|     I: IndexStore, | ||||
| { | ||||
| impl Meilisearch { | ||||
|     pub fn builder() -> IndexControllerBuilder { | ||||
|         IndexControllerBuilder::default() | ||||
|     } | ||||
|  | ||||
|     pub async fn register_update(&self, uid: String, update: Update) -> Result<Task> { | ||||
|         let index_uid = IndexUid::from_str(&uid).map_err(IndexResolverError::from)?; | ||||
|         let content = match update { | ||||
|             Update::DeleteDocuments(ids) => TaskContent::DocumentDeletion { | ||||
|                 index_uid, | ||||
|                 deletion: DocumentDeletion::Ids(ids), | ||||
|             }, | ||||
|             Update::ClearDocuments => TaskContent::DocumentDeletion { | ||||
|                 index_uid, | ||||
|                 deletion: DocumentDeletion::Clear, | ||||
|             }, | ||||
|             Update::Settings { | ||||
|                 settings, | ||||
|                 is_deletion, | ||||
|                 allow_index_creation, | ||||
|             } => TaskContent::SettingsUpdate { | ||||
|                 settings, | ||||
|                 is_deletion, | ||||
|                 allow_index_creation, | ||||
|                 index_uid, | ||||
|             }, | ||||
|             Update::DocumentAddition { | ||||
|                 mut payload, | ||||
|                 primary_key, | ||||
|                 format, | ||||
|                 method, | ||||
|                 allow_index_creation, | ||||
|             } => { | ||||
|                 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, | ||||
|                     allow_index_creation, | ||||
|                     index_uid, | ||||
|                 } | ||||
|             } | ||||
|             Update::DeleteIndex => TaskContent::IndexDeletion { index_uid }, | ||||
|             Update::CreateIndex { primary_key } => TaskContent::IndexCreation { | ||||
|                 primary_key, | ||||
|                 index_uid, | ||||
|             }, | ||||
|             Update::UpdateIndex { primary_key } => TaskContent::IndexUpdate { | ||||
|                 primary_key, | ||||
|                 index_uid, | ||||
|             }, | ||||
|         }; | ||||
|  | ||||
|         let task = self.task_store.register(content).await?; | ||||
|         self.scheduler.read().await.notify(); | ||||
|  | ||||
|         Ok(task) | ||||
|     } | ||||
|  | ||||
|     pub async fn register_dump_task(&self) -> Result<Task> { | ||||
|         let uid = dump::generate_uid(); | ||||
|         let content = TaskContent::Dump { uid }; | ||||
|         let task = self.task_store.register(content).await?; | ||||
|         self.scheduler.read().await.notify(); | ||||
|         Ok(task) | ||||
|     pub async fn register_task(&self, task: TaskKind) -> Result<Task> { | ||||
|         Ok(self.index_scheduler.register(task).await?) | ||||
|     } | ||||
|  | ||||
|     pub async fn get_task(&self, id: TaskId, filter: Option<TaskFilter>) -> Result<Task> { | ||||
| @@ -652,6 +528,9 @@ fn clamp_to_page_size(size: usize) -> usize { | ||||
|     size / page_size::get() * page_size::get() | ||||
| } | ||||
|  | ||||
| /* | ||||
| TODO: TAMO: uncomment this test | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod test { | ||||
|     use futures::future::ok; | ||||
| @@ -669,22 +548,6 @@ mod test { | ||||
|  | ||||
|     use super::*; | ||||
|  | ||||
|     impl IndexController<MockIndexMetaStore, MockIndexStore> { | ||||
|         pub fn mock( | ||||
|             index_resolver: Arc<IndexResolver<MockIndexMetaStore, MockIndexStore>>, | ||||
|             task_store: TaskStore, | ||||
|             update_file_store: UpdateFileStore, | ||||
|             scheduler: Arc<RwLock<Scheduler>>, | ||||
|         ) -> Self { | ||||
|             IndexController { | ||||
|                 index_resolver, | ||||
|                 task_store, | ||||
|                 update_file_store, | ||||
|                 scheduler, | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     #[actix_rt::test] | ||||
|     async fn test_search_simple() { | ||||
|         let index_uid = "test"; | ||||
| @@ -781,3 +644,4 @@ mod test { | ||||
|         assert_eq!(r, result); | ||||
|     } | ||||
| } | ||||
| */ | ||||
|   | ||||
| @@ -1,71 +0,0 @@ | ||||
| use std::fmt; | ||||
|  | ||||
| use meilisearch_types::error::{Code, ErrorCode}; | ||||
| use meilisearch_types::index_uid::IndexUidFormatError; | ||||
| use meilisearch_types::internal_error; | ||||
| use tokio::sync::mpsc::error::SendError as MpscSendError; | ||||
| use tokio::sync::oneshot::error::RecvError as OneshotRecvError; | ||||
| use uuid::Uuid; | ||||
|  | ||||
| use crate::{error::MilliError, index::error::IndexError, update_file_store::UpdateFileStoreError}; | ||||
|  | ||||
| pub type Result<T> = std::result::Result<T, IndexResolverError>; | ||||
|  | ||||
| #[derive(thiserror::Error, Debug)] | ||||
| pub enum IndexResolverError { | ||||
|     #[error("{0}")] | ||||
|     IndexError(#[from] IndexError), | ||||
|     #[error("Index `{0}` already exists.")] | ||||
|     IndexAlreadyExists(String), | ||||
|     #[error("Index `{0}` not found.")] | ||||
|     UnexistingIndex(String), | ||||
|     #[error("A primary key is already present. It's impossible to update it")] | ||||
|     ExistingPrimaryKey, | ||||
|     #[error("An internal error has occurred. `{0}`.")] | ||||
|     Internal(Box<dyn std::error::Error + Send + Sync + 'static>), | ||||
|     #[error("The creation of the `{0}` index has failed due to `Index uuid is already assigned`.")] | ||||
|     UuidAlreadyExists(Uuid), | ||||
|     #[error("{0}")] | ||||
|     Milli(#[from] milli::Error), | ||||
|     #[error("{0}")] | ||||
|     BadlyFormatted(#[from] IndexUidFormatError), | ||||
| } | ||||
|  | ||||
| impl<T> From<MpscSendError<T>> for IndexResolverError | ||||
| where | ||||
|     T: Send + Sync + 'static + fmt::Debug, | ||||
| { | ||||
|     fn from(other: tokio::sync::mpsc::error::SendError<T>) -> Self { | ||||
|         Self::Internal(Box::new(other)) | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl From<OneshotRecvError> for IndexResolverError { | ||||
|     fn from(other: tokio::sync::oneshot::error::RecvError) -> Self { | ||||
|         Self::Internal(Box::new(other)) | ||||
|     } | ||||
| } | ||||
|  | ||||
| internal_error!( | ||||
|     IndexResolverError: milli::heed::Error, | ||||
|     uuid::Error, | ||||
|     std::io::Error, | ||||
|     tokio::task::JoinError, | ||||
|     serde_json::Error, | ||||
|     UpdateFileStoreError | ||||
| ); | ||||
|  | ||||
| impl ErrorCode for IndexResolverError { | ||||
|     fn error_code(&self) -> Code { | ||||
|         match self { | ||||
|             IndexResolverError::IndexError(e) => e.error_code(), | ||||
|             IndexResolverError::IndexAlreadyExists(_) => Code::IndexAlreadyExists, | ||||
|             IndexResolverError::UnexistingIndex(_) => Code::IndexNotFound, | ||||
|             IndexResolverError::ExistingPrimaryKey => Code::PrimaryKeyAlreadyPresent, | ||||
|             IndexResolverError::Internal(_) => Code::Internal, | ||||
|             IndexResolverError::UuidAlreadyExists(_) => Code::CreateIndex, | ||||
|             IndexResolverError::Milli(e) => MilliError(e).error_code(), | ||||
|             IndexResolverError::BadlyFormatted(_) => Code::InvalidIndexUid, | ||||
|         } | ||||
|     } | ||||
| } | ||||
| @@ -1,223 +0,0 @@ | ||||
| use std::collections::HashSet; | ||||
| use std::fs::{create_dir_all, File}; | ||||
| use std::io::{BufRead, BufReader, Write}; | ||||
| use std::path::{Path, PathBuf}; | ||||
| use std::sync::Arc; | ||||
| use walkdir::WalkDir; | ||||
|  | ||||
| use milli::heed::types::{SerdeBincode, Str}; | ||||
| use milli::heed::{CompactionOption, Database, Env}; | ||||
| use serde::{Deserialize, Serialize}; | ||||
| use uuid::Uuid; | ||||
|  | ||||
| use super::error::{IndexResolverError, Result}; | ||||
| use crate::tasks::task::TaskId; | ||||
|  | ||||
| #[derive(Serialize, Deserialize)] | ||||
| pub struct DumpEntry { | ||||
|     pub uid: String, | ||||
|     pub index_meta: IndexMeta, | ||||
| } | ||||
|  | ||||
| const UUIDS_DB_PATH: &str = "index_uuids"; | ||||
|  | ||||
| #[async_trait::async_trait] | ||||
| #[cfg_attr(test, mockall::automock)] | ||||
| pub trait IndexMetaStore: Sized { | ||||
|     // Create a new entry for `name`. Return an error if `err` and the entry already exists, return | ||||
|     // the uuid otherwise. | ||||
|     async fn get(&self, uid: String) -> Result<(String, Option<IndexMeta>)>; | ||||
|     async fn delete(&self, uid: String) -> Result<Option<IndexMeta>>; | ||||
|     async fn list(&self) -> Result<Vec<(String, IndexMeta)>>; | ||||
|     async fn insert(&self, name: String, meta: IndexMeta) -> Result<()>; | ||||
|     async fn snapshot(&self, path: PathBuf) -> Result<HashSet<Uuid>>; | ||||
|     async fn get_size(&self) -> Result<u64>; | ||||
|     async fn dump(&self, path: PathBuf) -> Result<()>; | ||||
| } | ||||
|  | ||||
| #[derive(Serialize, Deserialize, Debug, Clone)] | ||||
| pub struct IndexMeta { | ||||
|     pub uuid: Uuid, | ||||
|     pub creation_task_id: TaskId, | ||||
| } | ||||
|  | ||||
| #[derive(Clone)] | ||||
| pub struct HeedMetaStore { | ||||
|     env: Arc<Env>, | ||||
|     db: Database<Str, SerdeBincode<IndexMeta>>, | ||||
| } | ||||
|  | ||||
| impl Drop for HeedMetaStore { | ||||
|     fn drop(&mut self) { | ||||
|         if Arc::strong_count(&self.env) == 1 { | ||||
|             self.env.as_ref().clone().prepare_for_closing(); | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl HeedMetaStore { | ||||
|     pub fn new(env: Arc<milli::heed::Env>) -> Result<Self> { | ||||
|         let db = env.create_database(Some("uuids"))?; | ||||
|         Ok(Self { env, db }) | ||||
|     } | ||||
|  | ||||
|     fn get(&self, name: &str) -> Result<Option<IndexMeta>> { | ||||
|         let env = self.env.clone(); | ||||
|         let db = self.db; | ||||
|         let txn = env.read_txn()?; | ||||
|         match db.get(&txn, name)? { | ||||
|             Some(meta) => Ok(Some(meta)), | ||||
|             None => Ok(None), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn delete(&self, uid: String) -> Result<Option<IndexMeta>> { | ||||
|         let env = self.env.clone(); | ||||
|         let db = self.db; | ||||
|         let mut txn = env.write_txn()?; | ||||
|         match db.get(&txn, &uid)? { | ||||
|             Some(meta) => { | ||||
|                 db.delete(&mut txn, &uid)?; | ||||
|                 txn.commit()?; | ||||
|                 Ok(Some(meta)) | ||||
|             } | ||||
|             None => Ok(None), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn list(&self) -> Result<Vec<(String, IndexMeta)>> { | ||||
|         let env = self.env.clone(); | ||||
|         let db = self.db; | ||||
|         let txn = env.read_txn()?; | ||||
|         let mut entries = Vec::new(); | ||||
|         for entry in db.iter(&txn)? { | ||||
|             let (name, meta) = entry?; | ||||
|             entries.push((name.to_string(), meta)) | ||||
|         } | ||||
|         Ok(entries) | ||||
|     } | ||||
|  | ||||
|     pub(crate) fn insert(&self, name: String, meta: IndexMeta) -> Result<()> { | ||||
|         let env = self.env.clone(); | ||||
|         let db = self.db; | ||||
|         let mut txn = env.write_txn()?; | ||||
|  | ||||
|         if db.get(&txn, &name)?.is_some() { | ||||
|             return Err(IndexResolverError::IndexAlreadyExists(name)); | ||||
|         } | ||||
|  | ||||
|         db.put(&mut txn, &name, &meta)?; | ||||
|         txn.commit()?; | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     fn snapshot(&self, mut path: PathBuf) -> Result<HashSet<Uuid>> { | ||||
|         // Write transaction to acquire a lock on the database. | ||||
|         let txn = self.env.write_txn()?; | ||||
|         let mut entries = HashSet::new(); | ||||
|         for entry in self.db.iter(&txn)? { | ||||
|             let (_, IndexMeta { uuid, .. }) = entry?; | ||||
|             entries.insert(uuid); | ||||
|         } | ||||
|  | ||||
|         // only perform snapshot if there are indexes | ||||
|         if !entries.is_empty() { | ||||
|             path.push(UUIDS_DB_PATH); | ||||
|             create_dir_all(&path).unwrap(); | ||||
|             path.push("data.mdb"); | ||||
|             self.env.copy_to_path(path, CompactionOption::Enabled)?; | ||||
|         } | ||||
|         Ok(entries) | ||||
|     } | ||||
|  | ||||
|     fn get_size(&self) -> Result<u64> { | ||||
|         Ok(WalkDir::new(self.env.path()) | ||||
|             .into_iter() | ||||
|             .filter_map(|entry| entry.ok()) | ||||
|             .filter_map(|entry| entry.metadata().ok()) | ||||
|             .filter(|metadata| metadata.is_file()) | ||||
|             .fold(0, |acc, m| acc + m.len())) | ||||
|     } | ||||
|  | ||||
|     pub fn dump(&self, path: PathBuf) -> Result<()> { | ||||
|         let dump_path = path.join(UUIDS_DB_PATH); | ||||
|         create_dir_all(&dump_path)?; | ||||
|         let dump_file_path = dump_path.join("data.jsonl"); | ||||
|         let mut dump_file = File::create(&dump_file_path)?; | ||||
|  | ||||
|         let txn = self.env.read_txn()?; | ||||
|         for entry in self.db.iter(&txn)? { | ||||
|             let (uid, index_meta) = entry?; | ||||
|             let uid = uid.to_string(); | ||||
|  | ||||
|             let entry = DumpEntry { uid, index_meta }; | ||||
|             serde_json::to_writer(&mut dump_file, &entry)?; | ||||
|             dump_file.write_all(b"\n").unwrap(); | ||||
|         } | ||||
|  | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     pub fn load_dump(src: impl AsRef<Path>, env: Arc<milli::heed::Env>) -> Result<()> { | ||||
|         let src_indexes = src.as_ref().join(UUIDS_DB_PATH).join("data.jsonl"); | ||||
|         let indexes = File::open(&src_indexes)?; | ||||
|         let mut indexes = BufReader::new(indexes); | ||||
|         let mut line = String::new(); | ||||
|  | ||||
|         let db = Self::new(env)?; | ||||
|         let mut txn = db.env.write_txn()?; | ||||
|  | ||||
|         loop { | ||||
|             match indexes.read_line(&mut line) { | ||||
|                 Ok(0) => break, | ||||
|                 Ok(_) => { | ||||
|                     let DumpEntry { uid, index_meta } = serde_json::from_str(&line)?; | ||||
|                     db.db.put(&mut txn, &uid, &index_meta)?; | ||||
|                 } | ||||
|                 Err(e) => return Err(e.into()), | ||||
|             } | ||||
|  | ||||
|             line.clear(); | ||||
|         } | ||||
|         txn.commit()?; | ||||
|  | ||||
|         Ok(()) | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[async_trait::async_trait] | ||||
| impl IndexMetaStore for HeedMetaStore { | ||||
|     async fn get(&self, name: String) -> Result<(String, Option<IndexMeta>)> { | ||||
|         let this = self.clone(); | ||||
|         tokio::task::spawn_blocking(move || this.get(&name).map(|res| (name, res))).await? | ||||
|     } | ||||
|  | ||||
|     async fn delete(&self, uid: String) -> Result<Option<IndexMeta>> { | ||||
|         let this = self.clone(); | ||||
|         tokio::task::spawn_blocking(move || this.delete(uid)).await? | ||||
|     } | ||||
|  | ||||
|     async fn list(&self) -> Result<Vec<(String, IndexMeta)>> { | ||||
|         let this = self.clone(); | ||||
|         tokio::task::spawn_blocking(move || this.list()).await? | ||||
|     } | ||||
|  | ||||
|     async fn insert(&self, name: String, meta: IndexMeta) -> Result<()> { | ||||
|         let this = self.clone(); | ||||
|         tokio::task::spawn_blocking(move || this.insert(name, meta)).await? | ||||
|     } | ||||
|  | ||||
|     async fn snapshot(&self, path: PathBuf) -> Result<HashSet<Uuid>> { | ||||
|         let this = self.clone(); | ||||
|         tokio::task::spawn_blocking(move || this.snapshot(path)).await? | ||||
|     } | ||||
|  | ||||
|     async fn get_size(&self) -> Result<u64> { | ||||
|         self.get_size() | ||||
|     } | ||||
|  | ||||
|     async fn dump(&self, path: PathBuf) -> Result<()> { | ||||
|         let this = self.clone(); | ||||
|         Ok(tokio::task::spawn_blocking(move || this.dump(path)).await??) | ||||
|     } | ||||
| } | ||||
| @@ -1,685 +0,0 @@ | ||||
| pub mod error; | ||||
| pub mod index_store; | ||||
| pub mod meta_store; | ||||
|  | ||||
| use std::convert::TryFrom; | ||||
| use std::path::Path; | ||||
| use std::sync::Arc; | ||||
|  | ||||
| use error::{IndexResolverError, Result}; | ||||
| use index_store::{IndexStore, MapIndexStore}; | ||||
| use meilisearch_types::error::ResponseError; | ||||
| use meilisearch_types::index_uid::IndexUid; | ||||
| use meta_store::{HeedMetaStore, IndexMetaStore}; | ||||
| use milli::heed::Env; | ||||
| use milli::update::{DocumentDeletionResult, IndexerConfig}; | ||||
| use time::OffsetDateTime; | ||||
| use tokio::task::spawn_blocking; | ||||
| use uuid::Uuid; | ||||
|  | ||||
| use crate::index::{error::Result as IndexResult, Index}; | ||||
| use crate::options::IndexerOpts; | ||||
| use crate::tasks::task::{DocumentDeletion, Task, TaskContent, TaskEvent, TaskId, TaskResult}; | ||||
| use crate::update_file_store::UpdateFileStore; | ||||
|  | ||||
| use self::meta_store::IndexMeta; | ||||
|  | ||||
| pub type HardStateIndexResolver = IndexResolver<HeedMetaStore, MapIndexStore>; | ||||
|  | ||||
| #[cfg(not(test))] | ||||
| pub use real::IndexResolver; | ||||
|  | ||||
| #[cfg(test)] | ||||
| pub use test::MockIndexResolver as IndexResolver; | ||||
|  | ||||
| pub fn create_index_resolver( | ||||
|     path: impl AsRef<Path>, | ||||
|     index_size: usize, | ||||
|     indexer_opts: &IndexerOpts, | ||||
|     meta_env: Arc<milli::heed::Env>, | ||||
|     file_store: UpdateFileStore, | ||||
| ) -> anyhow::Result<HardStateIndexResolver> { | ||||
|     let uuid_store = HeedMetaStore::new(meta_env)?; | ||||
|     let index_store = MapIndexStore::new(&path, index_size, indexer_opts)?; | ||||
|     Ok(IndexResolver::new(uuid_store, index_store, file_store)) | ||||
| } | ||||
|  | ||||
| mod real { | ||||
|     use super::*; | ||||
|  | ||||
|     pub struct IndexResolver<U, I> { | ||||
|         pub(super) index_uuid_store: U, | ||||
|         pub(super) index_store: I, | ||||
|         pub(super) file_store: UpdateFileStore, | ||||
|     } | ||||
|  | ||||
|     impl IndexResolver<HeedMetaStore, MapIndexStore> { | ||||
|         pub fn load_dump( | ||||
|             src: impl AsRef<Path>, | ||||
|             dst: impl AsRef<Path>, | ||||
|             index_db_size: usize, | ||||
|             env: Arc<Env>, | ||||
|             indexer_opts: &IndexerOpts, | ||||
|         ) -> anyhow::Result<()> { | ||||
|             HeedMetaStore::load_dump(&src, env)?; | ||||
|             let indexes_path = src.as_ref().join("indexes"); | ||||
|             let indexes = indexes_path.read_dir()?; | ||||
|             let indexer_config = IndexerConfig::try_from(indexer_opts)?; | ||||
|             for index in indexes { | ||||
|                 Index::load_dump(&index?.path(), &dst, index_db_size, &indexer_config)?; | ||||
|             } | ||||
|  | ||||
|             Ok(()) | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     impl<U, I> IndexResolver<U, I> | ||||
|     where | ||||
|         U: IndexMetaStore, | ||||
|         I: IndexStore, | ||||
|     { | ||||
|         pub fn new(index_uuid_store: U, index_store: I, file_store: UpdateFileStore) -> Self { | ||||
|             Self { | ||||
|                 index_uuid_store, | ||||
|                 index_store, | ||||
|                 file_store, | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn process_document_addition_batch(&self, tasks: &mut [Task]) { | ||||
|             fn get_content_uuid(task: &Task) -> Uuid { | ||||
|                 match task { | ||||
|                     Task { | ||||
|                         content: TaskContent::DocumentAddition { content_uuid, .. }, | ||||
|                         .. | ||||
|                     } => *content_uuid, | ||||
|                     _ => panic!("unexpected task in the document addition batch"), | ||||
|                 } | ||||
|             } | ||||
|  | ||||
|             let content_uuids = tasks.iter().map(get_content_uuid).collect::<Vec<_>>(); | ||||
|  | ||||
|             match tasks.first() { | ||||
|                 Some(Task { | ||||
|                     id, | ||||
|                     content: | ||||
|                         TaskContent::DocumentAddition { | ||||
|                             merge_strategy, | ||||
|                             primary_key, | ||||
|                             allow_index_creation, | ||||
|                             index_uid, | ||||
|                             .. | ||||
|                         }, | ||||
|                     .. | ||||
|                 }) => { | ||||
|                     let primary_key = primary_key.clone(); | ||||
|                     let method = *merge_strategy; | ||||
|  | ||||
|                     let index = if *allow_index_creation { | ||||
|                         self.get_or_create_index(index_uid.clone(), *id).await | ||||
|                     } else { | ||||
|                         self.get_index(index_uid.as_str().to_string()).await | ||||
|                     }; | ||||
|  | ||||
|                     // If the index doesn't exist and we are not allowed to create it with the first | ||||
|                     // task, we must fails the whole batch. | ||||
|                     let now = OffsetDateTime::now_utc(); | ||||
|                     let index = match index { | ||||
|                         Ok(index) => index, | ||||
|                         Err(e) => { | ||||
|                             let error = ResponseError::from(e); | ||||
|                             for task in tasks.iter_mut() { | ||||
|                                 task.events.push(TaskEvent::Failed { | ||||
|                                     error: error.clone(), | ||||
|                                     timestamp: now, | ||||
|                                 }); | ||||
|                             } | ||||
|  | ||||
|                             return; | ||||
|                         } | ||||
|                     }; | ||||
|  | ||||
|                     let file_store = self.file_store.clone(); | ||||
|                     let result = spawn_blocking(move || { | ||||
|                         index.update_documents( | ||||
|                             method, | ||||
|                             primary_key, | ||||
|                             file_store, | ||||
|                             content_uuids.into_iter(), | ||||
|                         ) | ||||
|                     }) | ||||
|                     .await; | ||||
|  | ||||
|                     match result { | ||||
|                         Ok(Ok(results)) => { | ||||
|                             for (task, result) in tasks.iter_mut().zip(results) { | ||||
|                                 let event = match result { | ||||
|                                     Ok(addition) => { | ||||
|                                         TaskEvent::succeeded(TaskResult::DocumentAddition { | ||||
|                                             indexed_documents: addition.indexed_documents, | ||||
|                                         }) | ||||
|                                     } | ||||
|                                     Err(error) => { | ||||
|                                         TaskEvent::failed(IndexResolverError::from(error)) | ||||
|                                     } | ||||
|                                 }; | ||||
|                                 task.events.push(event); | ||||
|                             } | ||||
|                         } | ||||
|                         Ok(Err(e)) => { | ||||
|                             let event = TaskEvent::failed(e); | ||||
|                             for task in tasks.iter_mut() { | ||||
|                                 task.events.push(event.clone()); | ||||
|                             } | ||||
|                         } | ||||
|                         Err(e) => { | ||||
|                             let event = TaskEvent::failed(IndexResolverError::from(e)); | ||||
|                             for task in tasks.iter_mut() { | ||||
|                                 task.events.push(event.clone()); | ||||
|                             } | ||||
|                         } | ||||
|                     } | ||||
|                 } | ||||
|                 _ => panic!("invalid batch!"), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn delete_content_file(&self, content_uuid: Uuid) -> Result<()> { | ||||
|             self.file_store.delete(content_uuid).await?; | ||||
|             Ok(()) | ||||
|         } | ||||
|  | ||||
|         async fn process_task_inner(&self, task: &Task) -> Result<TaskResult> { | ||||
|             match &task.content { | ||||
|                 TaskContent::DocumentAddition { .. } => { | ||||
|                     panic!("updates should be handled by batch") | ||||
|                 } | ||||
|                 TaskContent::DocumentDeletion { | ||||
|                     deletion: DocumentDeletion::Ids(ids), | ||||
|                     index_uid, | ||||
|                 } => { | ||||
|                     let ids = ids.clone(); | ||||
|                     let index = self.get_index(index_uid.clone().into_inner()).await?; | ||||
|  | ||||
|                     let DocumentDeletionResult { | ||||
|                         deleted_documents, .. | ||||
|                     } = spawn_blocking(move || index.delete_documents(&ids)).await??; | ||||
|  | ||||
|                     Ok(TaskResult::DocumentDeletion { deleted_documents }) | ||||
|                 } | ||||
|                 TaskContent::DocumentDeletion { | ||||
|                     deletion: DocumentDeletion::Clear, | ||||
|                     index_uid, | ||||
|                 } => { | ||||
|                     let index = self.get_index(index_uid.clone().into_inner()).await?; | ||||
|                     let deleted_documents = spawn_blocking(move || -> IndexResult<u64> { | ||||
|                         let number_documents = index.stats()?.number_of_documents; | ||||
|                         index.clear_documents()?; | ||||
|                         Ok(number_documents) | ||||
|                     }) | ||||
|                     .await??; | ||||
|  | ||||
|                     Ok(TaskResult::ClearAll { deleted_documents }) | ||||
|                 } | ||||
|                 TaskContent::SettingsUpdate { | ||||
|                     settings, | ||||
|                     is_deletion, | ||||
|                     allow_index_creation, | ||||
|                     index_uid, | ||||
|                 } => { | ||||
|                     let index = if *is_deletion || !*allow_index_creation { | ||||
|                         self.get_index(index_uid.clone().into_inner()).await? | ||||
|                     } else { | ||||
|                         self.get_or_create_index(index_uid.clone(), task.id).await? | ||||
|                     }; | ||||
|  | ||||
|                     let settings = settings.clone(); | ||||
|                     spawn_blocking(move || index.update_settings(&settings.check())).await??; | ||||
|  | ||||
|                     Ok(TaskResult::Other) | ||||
|                 } | ||||
|                 TaskContent::IndexDeletion { index_uid } => { | ||||
|                     let index = self.delete_index(index_uid.clone().into_inner()).await?; | ||||
|  | ||||
|                     let deleted_documents = spawn_blocking(move || -> IndexResult<u64> { | ||||
|                         Ok(index.stats()?.number_of_documents) | ||||
|                     }) | ||||
|                     .await??; | ||||
|  | ||||
|                     Ok(TaskResult::ClearAll { deleted_documents }) | ||||
|                 } | ||||
|                 TaskContent::IndexCreation { | ||||
|                     primary_key, | ||||
|                     index_uid, | ||||
|                 } => { | ||||
|                     let index = self.create_index(index_uid.clone(), task.id).await?; | ||||
|  | ||||
|                     if let Some(primary_key) = primary_key { | ||||
|                         let primary_key = primary_key.clone(); | ||||
|                         spawn_blocking(move || index.update_primary_key(primary_key)).await??; | ||||
|                     } | ||||
|  | ||||
|                     Ok(TaskResult::Other) | ||||
|                 } | ||||
|                 TaskContent::IndexUpdate { | ||||
|                     primary_key, | ||||
|                     index_uid, | ||||
|                 } => { | ||||
|                     let index = self.get_index(index_uid.clone().into_inner()).await?; | ||||
|  | ||||
|                     if let Some(primary_key) = primary_key { | ||||
|                         let primary_key = primary_key.clone(); | ||||
|                         spawn_blocking(move || index.update_primary_key(primary_key)).await??; | ||||
|                     } | ||||
|  | ||||
|                     Ok(TaskResult::Other) | ||||
|                 } | ||||
|                 _ => unreachable!("Invalid task for index resolver"), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn process_task(&self, task: &mut Task) { | ||||
|             match self.process_task_inner(task).await { | ||||
|                 Ok(res) => task.events.push(TaskEvent::succeeded(res)), | ||||
|                 Err(e) => task.events.push(TaskEvent::failed(e)), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn dump(&self, path: impl AsRef<Path>) -> Result<()> { | ||||
|             for (_, index) in self.list().await? { | ||||
|                 index.dump(&path)?; | ||||
|             } | ||||
|             self.index_uuid_store.dump(path.as_ref().to_owned()).await?; | ||||
|             Ok(()) | ||||
|         } | ||||
|  | ||||
|         async fn create_index(&self, uid: IndexUid, creation_task_id: TaskId) -> Result<Index> { | ||||
|             match self.index_uuid_store.get(uid.into_inner()).await? { | ||||
|                 (uid, Some(_)) => Err(IndexResolverError::IndexAlreadyExists(uid)), | ||||
|                 (uid, None) => { | ||||
|                     let uuid = Uuid::new_v4(); | ||||
|                     let index = self.index_store.create(uuid).await?; | ||||
|                     match self | ||||
|                         .index_uuid_store | ||||
|                         .insert( | ||||
|                             uid, | ||||
|                             IndexMeta { | ||||
|                                 uuid, | ||||
|                                 creation_task_id, | ||||
|                             }, | ||||
|                         ) | ||||
|                         .await | ||||
|                     { | ||||
|                         Err(e) => { | ||||
|                             match self.index_store.delete(uuid).await { | ||||
|                                 Ok(Some(index)) => { | ||||
|                                     index.close(); | ||||
|                                 } | ||||
|                                 Ok(None) => (), | ||||
|                                 Err(e) => log::error!("Error while deleting index: {:?}", e), | ||||
|                             } | ||||
|                             Err(e) | ||||
|                         } | ||||
|                         Ok(()) => Ok(index), | ||||
|                     } | ||||
|                 } | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         /// Get or create an index with name `uid`. | ||||
|         pub async fn get_or_create_index(&self, uid: IndexUid, task_id: TaskId) -> Result<Index> { | ||||
|             match self.create_index(uid, task_id).await { | ||||
|                 Ok(index) => Ok(index), | ||||
|                 Err(IndexResolverError::IndexAlreadyExists(uid)) => self.get_index(uid).await, | ||||
|                 Err(e) => Err(e), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn list(&self) -> Result<Vec<(String, Index)>> { | ||||
|             let uuids = self.index_uuid_store.list().await?; | ||||
|             let mut indexes = Vec::new(); | ||||
|             for (name, IndexMeta { uuid, .. }) in uuids { | ||||
|                 match self.index_store.get(uuid).await? { | ||||
|                     Some(index) => indexes.push((name, index)), | ||||
|                     None => { | ||||
|                         // we found an unexisting index, we remove it from the uuid store | ||||
|                         let _ = self.index_uuid_store.delete(name).await; | ||||
|                     } | ||||
|                 } | ||||
|             } | ||||
|  | ||||
|             Ok(indexes) | ||||
|         } | ||||
|  | ||||
|         pub async fn delete_index(&self, uid: String) -> Result<Index> { | ||||
|             match self.index_uuid_store.delete(uid.clone()).await? { | ||||
|                 Some(IndexMeta { uuid, .. }) => match self.index_store.delete(uuid).await? { | ||||
|                     Some(index) => { | ||||
|                         index.clone().close(); | ||||
|                         Ok(index) | ||||
|                     } | ||||
|                     None => Err(IndexResolverError::UnexistingIndex(uid)), | ||||
|                 }, | ||||
|                 None => Err(IndexResolverError::UnexistingIndex(uid)), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn get_index(&self, uid: String) -> Result<Index> { | ||||
|             match self.index_uuid_store.get(uid).await? { | ||||
|                 (name, Some(IndexMeta { uuid, .. })) => { | ||||
|                     match self.index_store.get(uuid).await? { | ||||
|                         Some(index) => Ok(index), | ||||
|                         None => { | ||||
|                             // For some reason we got a uuid to an unexisting index, we return an error, | ||||
|                             // and remove the uuid from the uuid store. | ||||
|                             let _ = self.index_uuid_store.delete(name.clone()).await; | ||||
|                             Err(IndexResolverError::UnexistingIndex(name)) | ||||
|                         } | ||||
|                     } | ||||
|                 } | ||||
|                 (name, _) => Err(IndexResolverError::UnexistingIndex(name)), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn get_index_creation_task_id(&self, index_uid: String) -> Result<TaskId> { | ||||
|             let (uid, meta) = self.index_uuid_store.get(index_uid).await?; | ||||
|             meta.map( | ||||
|                 |IndexMeta { | ||||
|                      creation_task_id, .. | ||||
|                  }| creation_task_id, | ||||
|             ) | ||||
|             .ok_or(IndexResolverError::UnexistingIndex(uid)) | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod test { | ||||
|     use crate::index::IndexStats; | ||||
|  | ||||
|     use super::index_store::MockIndexStore; | ||||
|     use super::meta_store::MockIndexMetaStore; | ||||
|     use super::*; | ||||
|  | ||||
|     use futures::future::ok; | ||||
|     use milli::FieldDistribution; | ||||
|     use nelson::Mocker; | ||||
|  | ||||
|     pub enum MockIndexResolver<U, I> { | ||||
|         Real(super::real::IndexResolver<U, I>), | ||||
|         Mock(Mocker), | ||||
|     } | ||||
|  | ||||
|     impl MockIndexResolver<HeedMetaStore, MapIndexStore> { | ||||
|         pub fn load_dump( | ||||
|             src: impl AsRef<Path>, | ||||
|             dst: impl AsRef<Path>, | ||||
|             index_db_size: usize, | ||||
|             env: Arc<Env>, | ||||
|             indexer_opts: &IndexerOpts, | ||||
|         ) -> anyhow::Result<()> { | ||||
|             super::real::IndexResolver::load_dump(src, dst, index_db_size, env, indexer_opts) | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     impl<U, I> MockIndexResolver<U, I> | ||||
|     where | ||||
|         U: IndexMetaStore, | ||||
|         I: IndexStore, | ||||
|     { | ||||
|         pub fn new(index_uuid_store: U, index_store: I, file_store: UpdateFileStore) -> Self { | ||||
|             Self::Real(super::real::IndexResolver { | ||||
|                 index_uuid_store, | ||||
|                 index_store, | ||||
|                 file_store, | ||||
|             }) | ||||
|         } | ||||
|  | ||||
|         pub fn mock(mocker: Mocker) -> Self { | ||||
|             Self::Mock(mocker) | ||||
|         } | ||||
|  | ||||
|         pub async fn process_document_addition_batch(&self, tasks: &mut [Task]) { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.process_document_addition_batch(tasks).await, | ||||
|                 IndexResolver::Mock(m) => unsafe { | ||||
|                     m.get("process_document_addition_batch").call(tasks) | ||||
|                 }, | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn process_task(&self, task: &mut Task) { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.process_task(task).await, | ||||
|                 IndexResolver::Mock(m) => unsafe { m.get("process_task").call(task) }, | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn dump(&self, path: impl AsRef<Path>) -> Result<()> { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.dump(path).await, | ||||
|                 IndexResolver::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         /// Get or create an index with name `uid`. | ||||
|         pub async fn get_or_create_index(&self, uid: IndexUid, task_id: TaskId) -> Result<Index> { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.get_or_create_index(uid, task_id).await, | ||||
|                 IndexResolver::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn list(&self) -> Result<Vec<(String, Index)>> { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.list().await, | ||||
|                 IndexResolver::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn delete_index(&self, uid: String) -> Result<Index> { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.delete_index(uid).await, | ||||
|                 IndexResolver::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn get_index(&self, uid: String) -> Result<Index> { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.get_index(uid).await, | ||||
|                 IndexResolver::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn get_index_creation_task_id(&self, index_uid: String) -> Result<TaskId> { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.get_index_creation_task_id(index_uid).await, | ||||
|                 IndexResolver::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn delete_content_file(&self, content_uuid: Uuid) -> Result<()> { | ||||
|             match self { | ||||
|                 IndexResolver::Real(r) => r.delete_content_file(content_uuid).await, | ||||
|                 IndexResolver::Mock(m) => unsafe { | ||||
|                     m.get("delete_content_file").call(content_uuid) | ||||
|                 }, | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     #[actix_rt::test] | ||||
|     async fn test_remove_unknown_index() { | ||||
|         let mut meta_store = MockIndexMetaStore::new(); | ||||
|         meta_store | ||||
|             .expect_delete() | ||||
|             .once() | ||||
|             .returning(|_| Box::pin(ok(None))); | ||||
|  | ||||
|         let index_store = MockIndexStore::new(); | ||||
|  | ||||
|         let mocker = Mocker::default(); | ||||
|         let file_store = UpdateFileStore::mock(mocker); | ||||
|  | ||||
|         let index_resolver = IndexResolver::new(meta_store, index_store, file_store); | ||||
|  | ||||
|         let mut task = Task { | ||||
|             id: 1, | ||||
|             content: TaskContent::IndexDeletion { | ||||
|                 index_uid: IndexUid::new_unchecked("test"), | ||||
|             }, | ||||
|             events: Vec::new(), | ||||
|         }; | ||||
|  | ||||
|         index_resolver.process_task(&mut task).await; | ||||
|  | ||||
|         assert!(matches!(task.events[0], TaskEvent::Failed { .. })); | ||||
|     } | ||||
|  | ||||
|     #[actix_rt::test] | ||||
|     async fn test_remove_index() { | ||||
|         let mut meta_store = MockIndexMetaStore::new(); | ||||
|         meta_store.expect_delete().once().returning(|_| { | ||||
|             Box::pin(ok(Some(IndexMeta { | ||||
|                 uuid: Uuid::new_v4(), | ||||
|                 creation_task_id: 1, | ||||
|             }))) | ||||
|         }); | ||||
|  | ||||
|         let mut index_store = MockIndexStore::new(); | ||||
|         index_store.expect_delete().once().returning(|_| { | ||||
|             let mocker = Mocker::default(); | ||||
|             mocker.when::<(), ()>("close").then(|_| ()); | ||||
|             mocker | ||||
|                 .when::<(), IndexResult<IndexStats>>("stats") | ||||
|                 .then(|_| { | ||||
|                     Ok(IndexStats { | ||||
|                         size: 10, | ||||
|                         number_of_documents: 10, | ||||
|                         is_indexing: None, | ||||
|                         field_distribution: FieldDistribution::default(), | ||||
|                     }) | ||||
|                 }); | ||||
|             Box::pin(ok(Some(Index::mock(mocker)))) | ||||
|         }); | ||||
|  | ||||
|         let mocker = Mocker::default(); | ||||
|         let file_store = UpdateFileStore::mock(mocker); | ||||
|  | ||||
|         let index_resolver = IndexResolver::new(meta_store, index_store, file_store); | ||||
|  | ||||
|         let mut task = Task { | ||||
|             id: 1, | ||||
|             content: TaskContent::IndexDeletion { | ||||
|                 index_uid: IndexUid::new_unchecked("test"), | ||||
|             }, | ||||
|             events: Vec::new(), | ||||
|         }; | ||||
|  | ||||
|         index_resolver.process_task(&mut task).await; | ||||
|  | ||||
|         assert!(matches!(task.events[0], TaskEvent::Succeeded { .. })); | ||||
|     } | ||||
|  | ||||
|     #[actix_rt::test] | ||||
|     async fn test_delete_documents() { | ||||
|         let mut meta_store = MockIndexMetaStore::new(); | ||||
|         meta_store.expect_get().once().returning(|_| { | ||||
|             Box::pin(ok(( | ||||
|                 "test".to_string(), | ||||
|                 Some(IndexMeta { | ||||
|                     uuid: Uuid::new_v4(), | ||||
|                     creation_task_id: 1, | ||||
|                 }), | ||||
|             ))) | ||||
|         }); | ||||
|  | ||||
|         let mut index_store = MockIndexStore::new(); | ||||
|         index_store.expect_get().once().returning(|_| { | ||||
|             let mocker = Mocker::default(); | ||||
|             mocker | ||||
|                 .when::<(), IndexResult<()>>("clear_documents") | ||||
|                 .once() | ||||
|                 .then(|_| Ok(())); | ||||
|             mocker | ||||
|                 .when::<(), IndexResult<IndexStats>>("stats") | ||||
|                 .once() | ||||
|                 .then(|_| { | ||||
|                     Ok(IndexStats { | ||||
|                         size: 10, | ||||
|                         number_of_documents: 10, | ||||
|                         is_indexing: None, | ||||
|                         field_distribution: FieldDistribution::default(), | ||||
|                     }) | ||||
|                 }); | ||||
|             Box::pin(ok(Some(Index::mock(mocker)))) | ||||
|         }); | ||||
|  | ||||
|         let mocker = Mocker::default(); | ||||
|         let file_store = UpdateFileStore::mock(mocker); | ||||
|  | ||||
|         let index_resolver = IndexResolver::new(meta_store, index_store, file_store); | ||||
|  | ||||
|         let mut task = Task { | ||||
|             id: 1, | ||||
|             content: TaskContent::DocumentDeletion { | ||||
|                 deletion: DocumentDeletion::Clear, | ||||
|                 index_uid: IndexUid::new_unchecked("test"), | ||||
|             }, | ||||
|             events: Vec::new(), | ||||
|         }; | ||||
|  | ||||
|         index_resolver.process_task(&mut task).await; | ||||
|  | ||||
|         assert!(matches!(task.events[0], TaskEvent::Succeeded { .. })); | ||||
|     } | ||||
|  | ||||
|     #[actix_rt::test] | ||||
|     async fn test_index_update() { | ||||
|         let mut meta_store = MockIndexMetaStore::new(); | ||||
|         meta_store.expect_get().once().returning(|_| { | ||||
|             Box::pin(ok(( | ||||
|                 "test".to_string(), | ||||
|                 Some(IndexMeta { | ||||
|                     uuid: Uuid::new_v4(), | ||||
|                     creation_task_id: 1, | ||||
|                 }), | ||||
|             ))) | ||||
|         }); | ||||
|  | ||||
|         let mut index_store = MockIndexStore::new(); | ||||
|         index_store.expect_get().once().returning(|_| { | ||||
|             let mocker = Mocker::default(); | ||||
|  | ||||
|             mocker | ||||
|                 .when::<String, IndexResult<crate::index::IndexMeta>>("update_primary_key") | ||||
|                 .once() | ||||
|                 .then(|_| { | ||||
|                     Ok(crate::index::IndexMeta { | ||||
|                         created_at: OffsetDateTime::now_utc(), | ||||
|                         updated_at: OffsetDateTime::now_utc(), | ||||
|                         primary_key: Some("key".to_string()), | ||||
|                     }) | ||||
|                 }); | ||||
|             Box::pin(ok(Some(Index::mock(mocker)))) | ||||
|         }); | ||||
|  | ||||
|         let mocker = Mocker::default(); | ||||
|         let file_store = UpdateFileStore::mock(mocker); | ||||
|  | ||||
|         let index_resolver = IndexResolver::new(meta_store, index_store, file_store); | ||||
|  | ||||
|         let mut task = Task { | ||||
|             id: 1, | ||||
|             content: TaskContent::IndexUpdate { | ||||
|                 primary_key: Some("key".to_string()), | ||||
|                 index_uid: IndexUid::new_unchecked("test"), | ||||
|             }, | ||||
|             events: Vec::new(), | ||||
|         }; | ||||
|  | ||||
|         index_resolver.process_task(&mut task).await; | ||||
|  | ||||
|         assert!(matches!(task.events[0], TaskEvent::Succeeded { .. })); | ||||
|     } | ||||
| } | ||||
| @@ -3,24 +3,23 @@ pub mod error; | ||||
| pub mod options; | ||||
|  | ||||
| mod analytics; | ||||
| mod document_formats; | ||||
| // TODO: TAMO: reenable the dumps | ||||
| #[cfg(todo)] | ||||
| mod dump; | ||||
| pub mod index; | ||||
| pub mod index_controller; | ||||
| mod index_resolver; | ||||
| mod index_controller; | ||||
| mod snapshot; | ||||
| pub mod tasks; | ||||
| mod update_file_store; | ||||
|  | ||||
| use std::env::VarError; | ||||
| use std::ffi::OsStr; | ||||
| use std::path::Path; | ||||
|  | ||||
| pub use index_controller::MeiliSearch; | ||||
| // TODO: TAMO: rename the MeiliSearch in Meilisearch | ||||
| pub use index_controller::Meilisearch as MeiliSearch; | ||||
| pub use milli; | ||||
| pub use milli::heed; | ||||
|  | ||||
| mod compression; | ||||
| pub mod document_formats; | ||||
|  | ||||
| /// Check if a db is empty. It does not provide any information on the | ||||
| /// validity of the data in it. | ||||
|   | ||||
| @@ -15,7 +15,7 @@ use walkdir::WalkDir; | ||||
| use crate::compression::from_tar_gz; | ||||
| use crate::index_controller::open_meta_env; | ||||
| use crate::index_controller::versioning::VERSION_FILE_NAME; | ||||
| use crate::tasks::Scheduler; | ||||
| use index_scheduler::IndexScheduler; | ||||
|  | ||||
| pub struct SnapshotService { | ||||
|     pub(crate) db_path: PathBuf, | ||||
| @@ -23,7 +23,7 @@ pub struct SnapshotService { | ||||
|     pub(crate) snapshot_path: PathBuf, | ||||
|     pub(crate) index_size: usize, | ||||
|     pub(crate) meta_env_size: usize, | ||||
|     pub(crate) scheduler: Arc<RwLock<Scheduler>>, | ||||
|     pub(crate) scheduler: IndexScheduler, | ||||
| } | ||||
|  | ||||
| impl SnapshotService { | ||||
| @@ -39,7 +39,8 @@ impl SnapshotService { | ||||
|                 meta_env_size: self.meta_env_size, | ||||
|                 index_size: self.index_size, | ||||
|             }; | ||||
|             self.scheduler.write().await.schedule_snapshot(snapshot_job); | ||||
|             // TODO: TAMO: reenable the snapshots | ||||
|             // self.scheduler.write().await.schedule_snapshot(snapshot_job); | ||||
|             sleep(self.snapshot_period).await; | ||||
|         } | ||||
|     } | ||||
|   | ||||
| @@ -1,75 +0,0 @@ | ||||
| use time::OffsetDateTime; | ||||
|  | ||||
| use crate::snapshot::SnapshotJob; | ||||
|  | ||||
| use super::task::{Task, TaskEvent}; | ||||
|  | ||||
| pub type BatchId = u32; | ||||
|  | ||||
| #[derive(Debug)] | ||||
| pub enum BatchContent { | ||||
|     DocumentsAdditionBatch(Vec<Task>), | ||||
|     IndexUpdate(Task), | ||||
|     Dump(Task), | ||||
|     Snapshot(SnapshotJob), | ||||
|     // Symbolizes a empty batch. This can occur when we were woken, but there wasn't any work to do. | ||||
|     Empty, | ||||
| } | ||||
|  | ||||
| impl BatchContent { | ||||
|     pub fn first(&self) -> Option<&Task> { | ||||
|         match self { | ||||
|             BatchContent::DocumentsAdditionBatch(ts) => ts.first(), | ||||
|             BatchContent::Dump(t) | BatchContent::IndexUpdate(t) => Some(t), | ||||
|             BatchContent::Snapshot(_) | BatchContent::Empty => None, | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub fn push_event(&mut self, event: TaskEvent) { | ||||
|         match self { | ||||
|             BatchContent::DocumentsAdditionBatch(ts) => { | ||||
|                 ts.iter_mut().for_each(|t| t.events.push(event.clone())) | ||||
|             } | ||||
|             BatchContent::IndexUpdate(t) | BatchContent::Dump(t) => t.events.push(event), | ||||
|             BatchContent::Snapshot(_) | BatchContent::Empty => (), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(Debug)] | ||||
| pub struct Batch { | ||||
|     // Only batches that contains a persistent tasks are given an id. Snapshot batches don't have | ||||
|     // an id. | ||||
|     pub id: Option<BatchId>, | ||||
|     pub created_at: OffsetDateTime, | ||||
|     pub content: BatchContent, | ||||
| } | ||||
|  | ||||
| impl Batch { | ||||
|     pub fn new(id: Option<BatchId>, content: BatchContent) -> Self { | ||||
|         Self { | ||||
|             id, | ||||
|             created_at: OffsetDateTime::now_utc(), | ||||
|             content, | ||||
|         } | ||||
|     } | ||||
|     pub fn len(&self) -> usize { | ||||
|         match self.content { | ||||
|             BatchContent::DocumentsAdditionBatch(ref ts) => ts.len(), | ||||
|             BatchContent::IndexUpdate(_) | BatchContent::Dump(_) | BatchContent::Snapshot(_) => 1, | ||||
|             BatchContent::Empty => 0, | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub fn is_empty(&self) -> bool { | ||||
|         self.len() == 0 | ||||
|     } | ||||
|  | ||||
|     pub fn empty() -> Self { | ||||
|         Self { | ||||
|             id: None, | ||||
|             created_at: OffsetDateTime::now_utc(), | ||||
|             content: BatchContent::Empty, | ||||
|         } | ||||
|     } | ||||
| } | ||||
| @@ -1,34 +0,0 @@ | ||||
| use meilisearch_types::error::{Code, ErrorCode}; | ||||
| use meilisearch_types::internal_error; | ||||
| use tokio::task::JoinError; | ||||
|  | ||||
| use crate::update_file_store::UpdateFileStoreError; | ||||
|  | ||||
| use super::task::TaskId; | ||||
|  | ||||
| pub type Result<T> = std::result::Result<T, TaskError>; | ||||
|  | ||||
| #[derive(Debug, thiserror::Error)] | ||||
| pub enum TaskError { | ||||
|     #[error("Task `{0}` not found.")] | ||||
|     UnexistingTask(TaskId), | ||||
|     #[error("Internal error: {0}")] | ||||
|     Internal(Box<dyn std::error::Error + Send + Sync + 'static>), | ||||
| } | ||||
|  | ||||
| internal_error!( | ||||
|     TaskError: milli::heed::Error, | ||||
|     JoinError, | ||||
|     std::io::Error, | ||||
|     serde_json::Error, | ||||
|     UpdateFileStoreError | ||||
| ); | ||||
|  | ||||
| impl ErrorCode for TaskError { | ||||
|     fn error_code(&self) -> Code { | ||||
|         match self { | ||||
|             TaskError::UnexistingTask(_) => Code::TaskNotFound, | ||||
|             TaskError::Internal(_) => Code::Internal, | ||||
|         } | ||||
|     } | ||||
| } | ||||
| @@ -1,132 +0,0 @@ | ||||
| use crate::dump::DumpHandler; | ||||
| use crate::index_resolver::index_store::IndexStore; | ||||
| use crate::index_resolver::meta_store::IndexMetaStore; | ||||
| use crate::tasks::batch::{Batch, BatchContent}; | ||||
| use crate::tasks::task::{Task, TaskContent, TaskEvent, TaskResult}; | ||||
| use crate::tasks::BatchHandler; | ||||
|  | ||||
| #[async_trait::async_trait] | ||||
| impl<U, I> BatchHandler for DumpHandler<U, I> | ||||
| where | ||||
|     U: IndexMetaStore + Sync + Send + 'static, | ||||
|     I: IndexStore + Sync + Send + 'static, | ||||
| { | ||||
|     fn accept(&self, batch: &Batch) -> bool { | ||||
|         matches!(batch.content, BatchContent::Dump { .. }) | ||||
|     } | ||||
|  | ||||
|     async fn process_batch(&self, mut batch: Batch) -> Batch { | ||||
|         match &batch.content { | ||||
|             BatchContent::Dump(Task { | ||||
|                 content: TaskContent::Dump { uid }, | ||||
|                 .. | ||||
|             }) => { | ||||
|                 match self.run(uid.clone()).await { | ||||
|                     Ok(_) => { | ||||
|                         batch | ||||
|                             .content | ||||
|                             .push_event(TaskEvent::succeeded(TaskResult::Other)); | ||||
|                     } | ||||
|                     Err(e) => batch.content.push_event(TaskEvent::failed(e)), | ||||
|                 } | ||||
|                 batch | ||||
|             } | ||||
|             _ => unreachable!("invalid batch content for dump"), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     async fn finish(&self, _: &Batch) {} | ||||
| } | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod test { | ||||
|     use crate::dump::error::{DumpError, Result as DumpResult}; | ||||
|     use crate::index_resolver::{index_store::MockIndexStore, meta_store::MockIndexMetaStore}; | ||||
|     use crate::tasks::handlers::test::task_to_batch; | ||||
|  | ||||
|     use super::*; | ||||
|  | ||||
|     use nelson::Mocker; | ||||
|     use proptest::prelude::*; | ||||
|  | ||||
|     proptest! { | ||||
|         #[test] | ||||
|         fn finish_does_nothing( | ||||
|             task in any::<Task>(), | ||||
|         ) { | ||||
|             let rt = tokio::runtime::Runtime::new().unwrap(); | ||||
|             let handle = rt.spawn(async { | ||||
|                 let batch = task_to_batch(task); | ||||
|  | ||||
|                 let mocker = Mocker::default(); | ||||
|                 let dump_handler = DumpHandler::<MockIndexMetaStore, MockIndexStore>::mock(mocker); | ||||
|  | ||||
|                 dump_handler.finish(&batch).await; | ||||
|             }); | ||||
|  | ||||
|             rt.block_on(handle).unwrap(); | ||||
|         } | ||||
|  | ||||
|         #[test] | ||||
|         fn test_handle_dump_success( | ||||
|             task in any::<Task>(), | ||||
|         ) { | ||||
|             let rt = tokio::runtime::Runtime::new().unwrap(); | ||||
|             let handle = rt.spawn(async { | ||||
|                 let batch = task_to_batch(task); | ||||
|                 let should_accept = matches!(batch.content, BatchContent::Dump { .. }); | ||||
|  | ||||
|                 let mocker = Mocker::default(); | ||||
|                 if should_accept { | ||||
|                     mocker.when::<String, DumpResult<()>>("run") | ||||
|                     .once() | ||||
|                     .then(|_| Ok(())); | ||||
|                 } | ||||
|  | ||||
|                 let dump_handler = DumpHandler::<MockIndexMetaStore, MockIndexStore>::mock(mocker); | ||||
|  | ||||
|                 let accept = dump_handler.accept(&batch); | ||||
|                 assert_eq!(accept, should_accept); | ||||
|  | ||||
|                 if accept { | ||||
|                     let batch = dump_handler.process_batch(batch).await; | ||||
|                     let last_event = batch.content.first().unwrap().events.last().unwrap(); | ||||
|                     assert!(matches!(last_event, TaskEvent::Succeeded { .. })); | ||||
|                 } | ||||
|             }); | ||||
|  | ||||
|             rt.block_on(handle).unwrap(); | ||||
|         } | ||||
|  | ||||
|         #[test] | ||||
|         fn test_handle_dump_error( | ||||
|             task in any::<Task>(), | ||||
|         ) { | ||||
|             let rt = tokio::runtime::Runtime::new().unwrap(); | ||||
|             let handle = rt.spawn(async { | ||||
|                 let batch = task_to_batch(task); | ||||
|                 let should_accept = matches!(batch.content, BatchContent::Dump { .. }); | ||||
|  | ||||
|                 let mocker = Mocker::default(); | ||||
|                 if should_accept { | ||||
|                     mocker.when::<String, DumpResult<()>>("run") | ||||
|                     .once() | ||||
|                     .then(|_| Err(DumpError::Internal("error".into()))); | ||||
|                 } | ||||
|  | ||||
|                 let dump_handler = DumpHandler::<MockIndexMetaStore, MockIndexStore>::mock(mocker); | ||||
|  | ||||
|                 let accept = dump_handler.accept(&batch); | ||||
|                 assert_eq!(accept, should_accept); | ||||
|  | ||||
|                 if accept { | ||||
|                     let batch = dump_handler.process_batch(batch).await; | ||||
|                     let last_event = batch.content.first().unwrap().events.last().unwrap(); | ||||
|                     assert!(matches!(last_event, TaskEvent::Failed { .. })); | ||||
|                 } | ||||
|             }); | ||||
|  | ||||
|             rt.block_on(handle).unwrap(); | ||||
|         } | ||||
|     } | ||||
| } | ||||
| @@ -1,18 +0,0 @@ | ||||
| use crate::tasks::batch::{Batch, BatchContent}; | ||||
| use crate::tasks::BatchHandler; | ||||
|  | ||||
| /// A sink handler for empty tasks. | ||||
| pub struct EmptyBatchHandler; | ||||
|  | ||||
| #[async_trait::async_trait] | ||||
| impl BatchHandler for EmptyBatchHandler { | ||||
|     fn accept(&self, batch: &Batch) -> bool { | ||||
|         matches!(batch.content, BatchContent::Empty) | ||||
|     } | ||||
|  | ||||
|     async fn process_batch(&self, batch: Batch) -> Batch { | ||||
|         batch | ||||
|     } | ||||
|  | ||||
|     async fn finish(&self, _: &Batch) {} | ||||
| } | ||||
| @@ -1,199 +0,0 @@ | ||||
| use crate::index_resolver::IndexResolver; | ||||
| use crate::index_resolver::{index_store::IndexStore, meta_store::IndexMetaStore}; | ||||
| use crate::tasks::batch::{Batch, BatchContent}; | ||||
| use crate::tasks::BatchHandler; | ||||
|  | ||||
| #[async_trait::async_trait] | ||||
| impl<U, I> BatchHandler for IndexResolver<U, I> | ||||
| where | ||||
|     U: IndexMetaStore + Send + Sync + 'static, | ||||
|     I: IndexStore + Send + Sync + 'static, | ||||
| { | ||||
|     fn accept(&self, batch: &Batch) -> bool { | ||||
|         matches!( | ||||
|             batch.content, | ||||
|             BatchContent::DocumentsAdditionBatch(_) | BatchContent::IndexUpdate(_) | ||||
|         ) | ||||
|     } | ||||
|  | ||||
|     async fn process_batch(&self, mut batch: Batch) -> Batch { | ||||
|         match batch.content { | ||||
|             BatchContent::DocumentsAdditionBatch(ref mut tasks) => { | ||||
|                 self.process_document_addition_batch(tasks).await; | ||||
|             } | ||||
|             BatchContent::IndexUpdate(ref mut task) => { | ||||
|                 self.process_task(task).await; | ||||
|             } | ||||
|             _ => unreachable!(), | ||||
|         } | ||||
|  | ||||
|         batch | ||||
|     } | ||||
|  | ||||
|     async fn finish(&self, batch: &Batch) { | ||||
|         if let BatchContent::DocumentsAdditionBatch(ref tasks) = batch.content { | ||||
|             for task in tasks { | ||||
|                 if let Some(content_uuid) = task.get_content_uuid() { | ||||
|                     if let Err(e) = self.delete_content_file(content_uuid).await { | ||||
|                         log::error!("error deleting update file: {}", e); | ||||
|                     } | ||||
|                 } | ||||
|             } | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod test { | ||||
|     use crate::index_resolver::index_store::MapIndexStore; | ||||
|     use crate::index_resolver::meta_store::HeedMetaStore; | ||||
|     use crate::index_resolver::{ | ||||
|         error::Result as IndexResult, index_store::MockIndexStore, meta_store::MockIndexMetaStore, | ||||
|     }; | ||||
|     use crate::tasks::{ | ||||
|         handlers::test::task_to_batch, | ||||
|         task::{Task, TaskContent}, | ||||
|     }; | ||||
|     use crate::update_file_store::{Result as FileStoreResult, UpdateFileStore}; | ||||
|  | ||||
|     use super::*; | ||||
|     use meilisearch_types::index_uid::IndexUid; | ||||
|     use milli::update::IndexDocumentsMethod; | ||||
|     use nelson::Mocker; | ||||
|     use proptest::prelude::*; | ||||
|     use uuid::Uuid; | ||||
|  | ||||
|     proptest! { | ||||
|         #[test] | ||||
|         fn test_accept_task( | ||||
|             task in any::<Task>(), | ||||
|         ) { | ||||
|             let batch = task_to_batch(task); | ||||
|  | ||||
|             let index_store = MockIndexStore::new(); | ||||
|             let meta_store = MockIndexMetaStore::new(); | ||||
|             let mocker = Mocker::default(); | ||||
|             let update_file_store = UpdateFileStore::mock(mocker); | ||||
|             let index_resolver = IndexResolver::new(meta_store, index_store, update_file_store); | ||||
|  | ||||
|             match batch.content { | ||||
|                 BatchContent::DocumentsAdditionBatch(_) | ||||
|                     | BatchContent::IndexUpdate(_) => assert!(index_resolver.accept(&batch)), | ||||
|                 BatchContent::Dump(_) | ||||
|                     | BatchContent::Snapshot(_) | ||||
|                     | BatchContent::Empty => assert!(!index_resolver.accept(&batch)), | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     #[actix_rt::test] | ||||
|     async fn finisher_called_on_document_update() { | ||||
|         let index_store = MockIndexStore::new(); | ||||
|         let meta_store = MockIndexMetaStore::new(); | ||||
|         let mocker = Mocker::default(); | ||||
|         let content_uuid = Uuid::new_v4(); | ||||
|         mocker | ||||
|             .when::<Uuid, FileStoreResult<()>>("delete") | ||||
|             .once() | ||||
|             .then(move |uuid| { | ||||
|                 assert_eq!(uuid, content_uuid); | ||||
|                 Ok(()) | ||||
|             }); | ||||
|         let update_file_store = UpdateFileStore::mock(mocker); | ||||
|         let index_resolver = IndexResolver::new(meta_store, index_store, update_file_store); | ||||
|  | ||||
|         let task = Task { | ||||
|             id: 1, | ||||
|             content: TaskContent::DocumentAddition { | ||||
|                 content_uuid, | ||||
|                 merge_strategy: IndexDocumentsMethod::ReplaceDocuments, | ||||
|                 primary_key: None, | ||||
|                 documents_count: 100, | ||||
|                 allow_index_creation: true, | ||||
|                 index_uid: IndexUid::new_unchecked("test"), | ||||
|             }, | ||||
|             events: Vec::new(), | ||||
|         }; | ||||
|  | ||||
|         let batch = task_to_batch(task); | ||||
|  | ||||
|         index_resolver.finish(&batch).await; | ||||
|     } | ||||
|  | ||||
|     #[actix_rt::test] | ||||
|     #[should_panic] | ||||
|     async fn panic_when_passed_unsupported_batch() { | ||||
|         let index_store = MockIndexStore::new(); | ||||
|         let meta_store = MockIndexMetaStore::new(); | ||||
|         let mocker = Mocker::default(); | ||||
|         let update_file_store = UpdateFileStore::mock(mocker); | ||||
|         let index_resolver = IndexResolver::new(meta_store, index_store, update_file_store); | ||||
|  | ||||
|         let task = Task { | ||||
|             id: 1, | ||||
|             content: TaskContent::Dump { | ||||
|                 uid: String::from("hello"), | ||||
|             }, | ||||
|             events: Vec::new(), | ||||
|         }; | ||||
|  | ||||
|         let batch = task_to_batch(task); | ||||
|  | ||||
|         index_resolver.process_batch(batch).await; | ||||
|     } | ||||
|  | ||||
|     proptest! { | ||||
|         #[test] | ||||
|         fn index_document_task_deletes_update_file( | ||||
|             task in any::<Task>(), | ||||
|         ) { | ||||
|             let rt = tokio::runtime::Runtime::new().unwrap(); | ||||
|             let handle = rt.spawn(async { | ||||
|                 let mocker = Mocker::default(); | ||||
|  | ||||
|                 if let TaskContent::DocumentAddition{ .. } = task.content { | ||||
|                     mocker.when::<Uuid, IndexResult<()>>("delete_content_file").then(|_| Ok(())); | ||||
|                 } | ||||
|  | ||||
|                 let index_resolver: IndexResolver<HeedMetaStore, MapIndexStore> = IndexResolver::mock(mocker); | ||||
|  | ||||
|                 let batch = task_to_batch(task); | ||||
|  | ||||
|                 index_resolver.finish(&batch).await; | ||||
|             }); | ||||
|  | ||||
|             rt.block_on(handle).unwrap(); | ||||
|         } | ||||
|  | ||||
|         #[test] | ||||
|         fn test_handle_batch(task in any::<Task>()) { | ||||
|             let rt = tokio::runtime::Runtime::new().unwrap(); | ||||
|             let handle = rt.spawn(async { | ||||
|                 let mocker = Mocker::default(); | ||||
|                 match task.content { | ||||
|                     TaskContent::DocumentAddition { .. } => { | ||||
|                         mocker.when::<&mut [Task], ()>("process_document_addition_batch").then(|_| ()); | ||||
|                     } | ||||
|                     TaskContent::Dump { .. } => (), | ||||
|                     _ => { | ||||
|                         mocker.when::<&mut Task, ()>("process_task").then(|_| ()); | ||||
|                     } | ||||
|                 } | ||||
|                 let index_resolver: IndexResolver<HeedMetaStore, MapIndexStore> = IndexResolver::mock(mocker); | ||||
|  | ||||
|  | ||||
|                 let batch = task_to_batch(task); | ||||
|  | ||||
|                 if index_resolver.accept(&batch) { | ||||
|                     index_resolver.process_batch(batch).await; | ||||
|                 } | ||||
|             }); | ||||
|  | ||||
|             if let Err(e) = rt.block_on(handle) { | ||||
|                 if e.is_panic() { | ||||
|                     std::panic::resume_unwind(e.into_panic()); | ||||
|                 } | ||||
|             } | ||||
|         } | ||||
|     } | ||||
| } | ||||
| @@ -1,34 +0,0 @@ | ||||
| pub mod dump_handler; | ||||
| pub mod empty_handler; | ||||
| mod index_resolver_handler; | ||||
| pub mod snapshot_handler; | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod test { | ||||
|     use time::OffsetDateTime; | ||||
|  | ||||
|     use crate::tasks::{ | ||||
|         batch::{Batch, BatchContent}, | ||||
|         task::{Task, TaskContent}, | ||||
|     }; | ||||
|  | ||||
|     pub fn task_to_batch(task: Task) -> Batch { | ||||
|         let content = match task.content { | ||||
|             TaskContent::DocumentAddition { .. } => { | ||||
|                 BatchContent::DocumentsAdditionBatch(vec![task]) | ||||
|             } | ||||
|             TaskContent::DocumentDeletion { .. } | ||||
|             | TaskContent::SettingsUpdate { .. } | ||||
|             | TaskContent::IndexDeletion { .. } | ||||
|             | TaskContent::IndexCreation { .. } | ||||
|             | TaskContent::IndexUpdate { .. } => BatchContent::IndexUpdate(task), | ||||
|             TaskContent::Dump { .. } => BatchContent::Dump(task), | ||||
|         }; | ||||
|  | ||||
|         Batch { | ||||
|             id: Some(1), | ||||
|             created_at: OffsetDateTime::now_utc(), | ||||
|             content, | ||||
|         } | ||||
|     } | ||||
| } | ||||
| @@ -1,26 +0,0 @@ | ||||
| use crate::tasks::batch::{Batch, BatchContent}; | ||||
| use crate::tasks::BatchHandler; | ||||
|  | ||||
| pub struct SnapshotHandler; | ||||
|  | ||||
| #[async_trait::async_trait] | ||||
| impl BatchHandler for SnapshotHandler { | ||||
|     fn accept(&self, batch: &Batch) -> bool { | ||||
|         matches!(batch.content, BatchContent::Snapshot(_)) | ||||
|     } | ||||
|  | ||||
|     async fn process_batch(&self, batch: Batch) -> Batch { | ||||
|         match batch.content { | ||||
|             BatchContent::Snapshot(job) => { | ||||
|                 if let Err(e) = job.run().await { | ||||
|                     log::error!("snapshot error: {e}"); | ||||
|                 } | ||||
|             } | ||||
|             _ => unreachable!(), | ||||
|         } | ||||
|  | ||||
|         Batch::empty() | ||||
|     } | ||||
|  | ||||
|     async fn finish(&self, _: &Batch) {} | ||||
| } | ||||
| @@ -1,56 +0,0 @@ | ||||
| use async_trait::async_trait; | ||||
|  | ||||
| pub use handlers::empty_handler::EmptyBatchHandler; | ||||
| pub use handlers::snapshot_handler::SnapshotHandler; | ||||
| pub use scheduler::Scheduler; | ||||
| pub use task_store::TaskFilter; | ||||
|  | ||||
| #[cfg(test)] | ||||
| pub use task_store::test::MockTaskStore as TaskStore; | ||||
| #[cfg(not(test))] | ||||
| pub use task_store::TaskStore; | ||||
|  | ||||
| use batch::Batch; | ||||
| use error::Result; | ||||
|  | ||||
| pub mod batch; | ||||
| pub mod error; | ||||
| mod handlers; | ||||
| mod scheduler; | ||||
| pub mod task; | ||||
| mod task_store; | ||||
| pub mod update_loop; | ||||
|  | ||||
| #[cfg_attr(test, mockall::automock(type Error=test::DebugError;))] | ||||
| #[async_trait] | ||||
| pub trait BatchHandler: Sync + Send + 'static { | ||||
|     /// return whether this handler can accept this batch | ||||
|     fn accept(&self, batch: &Batch) -> bool; | ||||
|  | ||||
|     /// Processes the `Task` batch returning the batch with the `Task` updated. | ||||
|     /// | ||||
|     /// It is ok for this function to panic if a batch is handed that hasn't been verified by | ||||
|     /// `accept` beforehand. | ||||
|     async fn process_batch(&self, batch: Batch) -> Batch; | ||||
|  | ||||
|     /// `finish` is called when the result of `process` has been committed to the task store. This | ||||
|     /// method can be used to perform cleanup after the update has been completed for example. | ||||
|     async fn finish(&self, batch: &Batch); | ||||
| } | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod test { | ||||
|     use serde::{Deserialize, Serialize}; | ||||
|     use std::fmt::Display; | ||||
|  | ||||
|     #[derive(Debug, Serialize, Deserialize)] | ||||
|     pub struct DebugError; | ||||
|  | ||||
|     impl Display for DebugError { | ||||
|         fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { | ||||
|             f.write_str("an error") | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     impl std::error::Error for DebugError {} | ||||
| } | ||||
| @@ -1,609 +0,0 @@ | ||||
| use std::cmp::Ordering; | ||||
| use std::collections::{hash_map::Entry, BinaryHeap, HashMap, VecDeque}; | ||||
| use std::ops::{Deref, DerefMut}; | ||||
| use std::slice; | ||||
| use std::sync::Arc; | ||||
|  | ||||
| use atomic_refcell::AtomicRefCell; | ||||
| use milli::update::IndexDocumentsMethod; | ||||
| use time::OffsetDateTime; | ||||
| use tokio::sync::{watch, RwLock}; | ||||
|  | ||||
| use crate::options::SchedulerConfig; | ||||
| use crate::snapshot::SnapshotJob; | ||||
|  | ||||
| use super::batch::{Batch, BatchContent}; | ||||
| use super::error::Result; | ||||
| use super::task::{Task, TaskContent, TaskEvent, TaskId}; | ||||
| use super::update_loop::UpdateLoop; | ||||
| use super::{BatchHandler, TaskFilter, TaskStore}; | ||||
|  | ||||
| #[derive(Eq, Debug, Clone, Copy)] | ||||
| enum TaskType { | ||||
|     DocumentAddition { number: usize }, | ||||
|     DocumentUpdate { number: usize }, | ||||
|     IndexUpdate, | ||||
|     Dump, | ||||
| } | ||||
|  | ||||
| /// Two tasks are equal if they have the same type. | ||||
| impl PartialEq for TaskType { | ||||
|     fn eq(&self, other: &Self) -> bool { | ||||
|         matches!( | ||||
|             (self, other), | ||||
|             (Self::DocumentAddition { .. }, Self::DocumentAddition { .. }) | ||||
|                 | (Self::DocumentUpdate { .. }, Self::DocumentUpdate { .. }) | ||||
|         ) | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(Eq, Debug, Clone, Copy)] | ||||
| struct PendingTask { | ||||
|     kind: TaskType, | ||||
|     id: TaskId, | ||||
| } | ||||
|  | ||||
| impl PartialEq for PendingTask { | ||||
|     fn eq(&self, other: &Self) -> bool { | ||||
|         self.id.eq(&other.id) | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl PartialOrd for PendingTask { | ||||
|     fn partial_cmp(&self, other: &Self) -> Option<Ordering> { | ||||
|         Some(self.cmp(other)) | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Ord for PendingTask { | ||||
|     fn cmp(&self, other: &Self) -> Ordering { | ||||
|         self.id.cmp(&other.id).reverse() | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(Debug)] | ||||
| struct TaskList { | ||||
|     id: TaskListIdentifier, | ||||
|     tasks: BinaryHeap<PendingTask>, | ||||
| } | ||||
|  | ||||
| impl Deref for TaskList { | ||||
|     type Target = BinaryHeap<PendingTask>; | ||||
|  | ||||
|     fn deref(&self) -> &Self::Target { | ||||
|         &self.tasks | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl DerefMut for TaskList { | ||||
|     fn deref_mut(&mut self) -> &mut Self::Target { | ||||
|         &mut self.tasks | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl TaskList { | ||||
|     fn new(id: TaskListIdentifier) -> Self { | ||||
|         Self { | ||||
|             id, | ||||
|             tasks: Default::default(), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl PartialEq for TaskList { | ||||
|     fn eq(&self, other: &Self) -> bool { | ||||
|         self.id == other.id | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Eq for TaskList {} | ||||
|  | ||||
| impl Ord for TaskList { | ||||
|     fn cmp(&self, other: &Self) -> Ordering { | ||||
|         match (&self.id, &other.id) { | ||||
|             (TaskListIdentifier::Index(_), TaskListIdentifier::Index(_)) => { | ||||
|                 match (self.peek(), other.peek()) { | ||||
|                     (None, None) => Ordering::Equal, | ||||
|                     (None, Some(_)) => Ordering::Less, | ||||
|                     (Some(_), None) => Ordering::Greater, | ||||
|                     (Some(lhs), Some(rhs)) => lhs.cmp(rhs), | ||||
|                 } | ||||
|             } | ||||
|             (TaskListIdentifier::Index(_), TaskListIdentifier::Dump) => Ordering::Less, | ||||
|             (TaskListIdentifier::Dump, TaskListIdentifier::Index(_)) => Ordering::Greater, | ||||
|             (TaskListIdentifier::Dump, TaskListIdentifier::Dump) => { | ||||
|                 unreachable!("There should be only one Dump task list") | ||||
|             } | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl PartialOrd for TaskList { | ||||
|     fn partial_cmp(&self, other: &Self) -> Option<Ordering> { | ||||
|         Some(self.cmp(other)) | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(PartialEq, Eq, Hash, Debug, Clone)] | ||||
| enum TaskListIdentifier { | ||||
|     Index(String), | ||||
|     Dump, | ||||
| } | ||||
|  | ||||
| impl From<&Task> for TaskListIdentifier { | ||||
|     fn from(task: &Task) -> Self { | ||||
|         match &task.content { | ||||
|             TaskContent::DocumentAddition { index_uid, .. } | ||||
|             | TaskContent::DocumentDeletion { index_uid, .. } | ||||
|             | TaskContent::SettingsUpdate { index_uid, .. } | ||||
|             | TaskContent::IndexDeletion { index_uid } | ||||
|             | TaskContent::IndexCreation { index_uid, .. } | ||||
|             | TaskContent::IndexUpdate { index_uid, .. } => { | ||||
|                 TaskListIdentifier::Index(index_uid.as_str().to_string()) | ||||
|             } | ||||
|             TaskContent::Dump { .. } => TaskListIdentifier::Dump, | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(Default)] | ||||
| struct TaskQueue { | ||||
|     /// Maps index uids to their TaskList, for quick access | ||||
|     index_tasks: HashMap<TaskListIdentifier, Arc<AtomicRefCell<TaskList>>>, | ||||
|     /// A queue that orders TaskList by the priority of their fist update | ||||
|     queue: BinaryHeap<Arc<AtomicRefCell<TaskList>>>, | ||||
| } | ||||
|  | ||||
| impl TaskQueue { | ||||
|     fn insert(&mut self, task: Task) { | ||||
|         let id = task.id; | ||||
|         let uid = TaskListIdentifier::from(&task); | ||||
|  | ||||
|         let kind = match task.content { | ||||
|             TaskContent::DocumentAddition { | ||||
|                 documents_count, | ||||
|                 merge_strategy: IndexDocumentsMethod::ReplaceDocuments, | ||||
|                 .. | ||||
|             } => TaskType::DocumentAddition { | ||||
|                 number: documents_count, | ||||
|             }, | ||||
|             TaskContent::DocumentAddition { | ||||
|                 documents_count, | ||||
|                 merge_strategy: IndexDocumentsMethod::UpdateDocuments, | ||||
|                 .. | ||||
|             } => TaskType::DocumentUpdate { | ||||
|                 number: documents_count, | ||||
|             }, | ||||
|             TaskContent::Dump { .. } => TaskType::Dump, | ||||
|             TaskContent::DocumentDeletion { .. } | ||||
|             | TaskContent::SettingsUpdate { .. } | ||||
|             | TaskContent::IndexDeletion { .. } | ||||
|             | TaskContent::IndexCreation { .. } | ||||
|             | TaskContent::IndexUpdate { .. } => TaskType::IndexUpdate, | ||||
|             _ => unreachable!("unhandled task type"), | ||||
|         }; | ||||
|         let task = PendingTask { kind, id }; | ||||
|  | ||||
|         match self.index_tasks.entry(uid) { | ||||
|             Entry::Occupied(entry) => { | ||||
|                 // A task list already exists for this index, all we have to to is to push the new | ||||
|                 // update to the end of the list. This won't change the order since ids are | ||||
|                 // monotonically increasing. | ||||
|                 let mut list = entry.get().borrow_mut(); | ||||
|  | ||||
|                 // We only need the first element to be lower than the one we want to | ||||
|                 // insert to preserve the order in the queue. | ||||
|                 assert!(list.peek().map(|old_id| id >= old_id.id).unwrap_or(true)); | ||||
|  | ||||
|                 list.push(task); | ||||
|             } | ||||
|             Entry::Vacant(entry) => { | ||||
|                 let mut task_list = TaskList::new(entry.key().clone()); | ||||
|                 task_list.push(task); | ||||
|                 let task_list = Arc::new(AtomicRefCell::new(task_list)); | ||||
|                 entry.insert(task_list.clone()); | ||||
|                 self.queue.push(task_list); | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     /// Passes a context with a view to the task list of the next index to schedule. It is | ||||
|     /// guaranteed that the first id from task list will be the lowest pending task id. | ||||
|     fn head_mut<R>(&mut self, mut f: impl FnMut(&mut TaskList) -> R) -> Option<R> { | ||||
|         let head = self.queue.pop()?; | ||||
|         let result = { | ||||
|             let mut ref_head = head.borrow_mut(); | ||||
|             f(&mut *ref_head) | ||||
|         }; | ||||
|         if !head.borrow().tasks.is_empty() { | ||||
|             // After being mutated, the head is reinserted to the correct position. | ||||
|             self.queue.push(head); | ||||
|         } else { | ||||
|             self.index_tasks.remove(&head.borrow().id); | ||||
|         } | ||||
|  | ||||
|         Some(result) | ||||
|     } | ||||
|  | ||||
|     pub fn is_empty(&self) -> bool { | ||||
|         self.queue.is_empty() && self.index_tasks.is_empty() | ||||
|     } | ||||
| } | ||||
|  | ||||
| pub struct Scheduler { | ||||
|     // TODO: currently snapshots are non persistent tasks, and are treated differently. | ||||
|     snapshots: VecDeque<SnapshotJob>, | ||||
|     tasks: TaskQueue, | ||||
|  | ||||
|     store: TaskStore, | ||||
|     processing: Processing, | ||||
|     next_fetched_task_id: TaskId, | ||||
|     config: SchedulerConfig, | ||||
|     /// Notifies the update loop that a new task was received | ||||
|     notifier: watch::Sender<()>, | ||||
| } | ||||
|  | ||||
| impl Scheduler { | ||||
|     pub fn new( | ||||
|         store: TaskStore, | ||||
|         performers: Vec<Arc<dyn BatchHandler + Sync + Send + 'static>>, | ||||
|         config: SchedulerConfig, | ||||
|     ) -> Result<Arc<RwLock<Self>>> { | ||||
|         let (notifier, rcv) = watch::channel(()); | ||||
|  | ||||
|         let this = Self { | ||||
|             snapshots: VecDeque::new(), | ||||
|             tasks: TaskQueue::default(), | ||||
|  | ||||
|             store, | ||||
|             processing: Processing::Nothing, | ||||
|             next_fetched_task_id: 0, | ||||
|             config, | ||||
|             notifier, | ||||
|         }; | ||||
|  | ||||
|         // Notify update loop to start processing pending updates immediately after startup. | ||||
|         this.notify(); | ||||
|  | ||||
|         let this = Arc::new(RwLock::new(this)); | ||||
|  | ||||
|         let update_loop = UpdateLoop::new(this.clone(), performers, rcv); | ||||
|  | ||||
|         tokio::task::spawn_local(update_loop.run()); | ||||
|  | ||||
|         Ok(this) | ||||
|     } | ||||
|  | ||||
|     fn register_task(&mut self, task: Task) { | ||||
|         assert!(!task.is_finished()); | ||||
|         self.tasks.insert(task); | ||||
|     } | ||||
|  | ||||
|     /// Clears the processing list, this method should be called when the processing of a batch is finished. | ||||
|     pub fn finish(&mut self) { | ||||
|         self.processing = Processing::Nothing; | ||||
|     } | ||||
|  | ||||
|     pub fn notify(&self) { | ||||
|         let _ = self.notifier.send(()); | ||||
|     } | ||||
|  | ||||
|     fn notify_if_not_empty(&self) { | ||||
|         if !self.snapshots.is_empty() || !self.tasks.is_empty() { | ||||
|             self.notify(); | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub async fn update_tasks(&self, content: BatchContent) -> Result<BatchContent> { | ||||
|         match content { | ||||
|             BatchContent::DocumentsAdditionBatch(tasks) => { | ||||
|                 let tasks = self.store.update_tasks(tasks).await?; | ||||
|                 Ok(BatchContent::DocumentsAdditionBatch(tasks)) | ||||
|             } | ||||
|             BatchContent::IndexUpdate(t) => { | ||||
|                 let mut tasks = self.store.update_tasks(vec![t]).await?; | ||||
|                 Ok(BatchContent::IndexUpdate(tasks.remove(0))) | ||||
|             } | ||||
|             BatchContent::Dump(t) => { | ||||
|                 let mut tasks = self.store.update_tasks(vec![t]).await?; | ||||
|                 Ok(BatchContent::Dump(tasks.remove(0))) | ||||
|             } | ||||
|             other => Ok(other), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub async fn get_task(&self, id: TaskId, filter: Option<TaskFilter>) -> Result<Task> { | ||||
|         self.store.get_task(id, filter).await | ||||
|     } | ||||
|  | ||||
|     pub async fn list_tasks( | ||||
|         &self, | ||||
|         offset: Option<TaskId>, | ||||
|         filter: Option<TaskFilter>, | ||||
|         limit: Option<usize>, | ||||
|     ) -> Result<Vec<Task>> { | ||||
|         self.store.list_tasks(offset, filter, limit).await | ||||
|     } | ||||
|  | ||||
|     pub async fn get_processing_tasks(&self) -> Result<Vec<Task>> { | ||||
|         let mut tasks = Vec::new(); | ||||
|  | ||||
|         for id in self.processing.ids() { | ||||
|             let task = self.store.get_task(id, None).await?; | ||||
|             tasks.push(task); | ||||
|         } | ||||
|  | ||||
|         Ok(tasks) | ||||
|     } | ||||
|  | ||||
|     pub fn schedule_snapshot(&mut self, job: SnapshotJob) { | ||||
|         self.snapshots.push_back(job); | ||||
|         self.notify(); | ||||
|     } | ||||
|  | ||||
|     async fn fetch_pending_tasks(&mut self) -> Result<()> { | ||||
|         self.store | ||||
|             .fetch_unfinished_tasks(Some(self.next_fetched_task_id)) | ||||
|             .await? | ||||
|             .into_iter() | ||||
|             .for_each(|t| { | ||||
|                 self.next_fetched_task_id = t.id + 1; | ||||
|                 self.register_task(t); | ||||
|             }); | ||||
|  | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     /// Prepare the next batch, and set `processing` to the ids in that batch. | ||||
|     pub async fn prepare(&mut self) -> Result<Batch> { | ||||
|         // If there is a job to process, do it first. | ||||
|         if let Some(job) = self.snapshots.pop_front() { | ||||
|             // There is more work to do, notify the update loop | ||||
|             self.notify_if_not_empty(); | ||||
|             let batch = Batch::new(None, BatchContent::Snapshot(job)); | ||||
|             return Ok(batch); | ||||
|         } | ||||
|  | ||||
|         // Try to fill the queue with pending tasks. | ||||
|         self.fetch_pending_tasks().await?; | ||||
|  | ||||
|         self.processing = make_batch(&mut self.tasks, &self.config); | ||||
|  | ||||
|         log::debug!("prepared batch with {} tasks", self.processing.len()); | ||||
|  | ||||
|         if !self.processing.is_nothing() { | ||||
|             let (processing, mut content) = self | ||||
|                 .store | ||||
|                 .get_processing_tasks(std::mem::take(&mut self.processing)) | ||||
|                 .await?; | ||||
|  | ||||
|             // The batch id is the id of the first update it contains. At this point we must have a | ||||
|             // valid batch that contains at least 1 task. | ||||
|             let id = match content.first() { | ||||
|                 Some(Task { id, .. }) => *id, | ||||
|                 _ => panic!("invalid batch"), | ||||
|             }; | ||||
|  | ||||
|             content.push_event(TaskEvent::Batched { | ||||
|                 batch_id: id, | ||||
|                 timestamp: OffsetDateTime::now_utc(), | ||||
|             }); | ||||
|  | ||||
|             self.processing = processing; | ||||
|  | ||||
|             let batch = Batch::new(Some(id), content); | ||||
|  | ||||
|             // There is more work to do, notify the update loop | ||||
|             self.notify_if_not_empty(); | ||||
|  | ||||
|             Ok(batch) | ||||
|         } else { | ||||
|             Ok(Batch::empty()) | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[derive(Debug, PartialEq, Eq)] | ||||
| pub enum Processing { | ||||
|     DocumentAdditions(Vec<TaskId>), | ||||
|     IndexUpdate(TaskId), | ||||
|     Dump(TaskId), | ||||
|     /// Variant used when there is nothing to process. | ||||
|     Nothing, | ||||
| } | ||||
|  | ||||
| impl Default for Processing { | ||||
|     fn default() -> Self { | ||||
|         Self::Nothing | ||||
|     } | ||||
| } | ||||
|  | ||||
| enum ProcessingIter<'a> { | ||||
|     Many(slice::Iter<'a, TaskId>), | ||||
|     Single(Option<TaskId>), | ||||
| } | ||||
|  | ||||
| impl<'a> Iterator for ProcessingIter<'a> { | ||||
|     type Item = TaskId; | ||||
|  | ||||
|     fn next(&mut self) -> Option<Self::Item> { | ||||
|         match self { | ||||
|             ProcessingIter::Many(iter) => iter.next().copied(), | ||||
|             ProcessingIter::Single(val) => val.take(), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Processing { | ||||
|     fn is_nothing(&self) -> bool { | ||||
|         matches!(self, Processing::Nothing) | ||||
|     } | ||||
|  | ||||
|     pub fn ids(&self) -> impl Iterator<Item = TaskId> + '_ { | ||||
|         match self { | ||||
|             Processing::DocumentAdditions(v) => ProcessingIter::Many(v.iter()), | ||||
|             Processing::IndexUpdate(id) | Processing::Dump(id) => ProcessingIter::Single(Some(*id)), | ||||
|             Processing::Nothing => ProcessingIter::Single(None), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub fn len(&self) -> usize { | ||||
|         match self { | ||||
|             Processing::DocumentAdditions(v) => v.len(), | ||||
|             Processing::IndexUpdate(_) | Processing::Dump(_) => 1, | ||||
|             Processing::Nothing => 0, | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub fn is_empty(&self) -> bool { | ||||
|         self.len() == 0 | ||||
|     } | ||||
| } | ||||
|  | ||||
| fn make_batch(tasks: &mut TaskQueue, config: &SchedulerConfig) -> Processing { | ||||
|     let mut doc_count = 0; | ||||
|     tasks | ||||
|         .head_mut(|list| match list.peek().copied() { | ||||
|             Some(PendingTask { | ||||
|                 kind: TaskType::IndexUpdate, | ||||
|                 id, | ||||
|             }) => { | ||||
|                 list.pop(); | ||||
|                 Processing::IndexUpdate(id) | ||||
|             } | ||||
|             Some(PendingTask { | ||||
|                 kind: TaskType::Dump, | ||||
|                 id, | ||||
|             }) => { | ||||
|                 list.pop(); | ||||
|                 Processing::Dump(id) | ||||
|             } | ||||
|             Some(PendingTask { kind, .. }) => { | ||||
|                 let mut task_list = Vec::new(); | ||||
|                 loop { | ||||
|                     match list.peek() { | ||||
|                         Some(pending) if pending.kind == kind => { | ||||
|                             // We always need to process at least one task for the scheduler to make progress. | ||||
|                             if config.disable_auto_batching && !task_list.is_empty() { | ||||
|                                 break; | ||||
|                             } | ||||
|                             let pending = list.pop().unwrap(); | ||||
|                             task_list.push(pending.id); | ||||
|  | ||||
|                             // We add the number of documents to the count if we are scheduling document additions. | ||||
|                             match pending.kind { | ||||
|                                 TaskType::DocumentUpdate { number } | ||||
|                                 | TaskType::DocumentAddition { number } => { | ||||
|                                     doc_count += number; | ||||
|                                 } | ||||
|                                 _ => (), | ||||
|                             } | ||||
|                         } | ||||
|                         _ => break, | ||||
|                     } | ||||
|                 } | ||||
|                 Processing::DocumentAdditions(task_list) | ||||
|             } | ||||
|             None => Processing::Nothing, | ||||
|         }) | ||||
|         .unwrap_or(Processing::Nothing) | ||||
| } | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod test { | ||||
|     use meilisearch_types::index_uid::IndexUid; | ||||
|     use milli::update::IndexDocumentsMethod; | ||||
|     use uuid::Uuid; | ||||
|  | ||||
|     use crate::tasks::task::TaskContent; | ||||
|  | ||||
|     use super::*; | ||||
|  | ||||
|     fn gen_task(id: TaskId, content: TaskContent) -> Task { | ||||
|         Task { | ||||
|             id, | ||||
|             content, | ||||
|             events: vec![], | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     #[test] | ||||
|     #[rustfmt::skip] | ||||
|     fn register_updates_multiples_indexes() { | ||||
|         let mut queue = TaskQueue::default(); | ||||
|         queue.insert(gen_task(0, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test1") })); | ||||
|         queue.insert(gen_task(1, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test2") })); | ||||
|         queue.insert(gen_task(2, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test2") })); | ||||
|         queue.insert(gen_task(3, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test2") })); | ||||
|         queue.insert(gen_task(4, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test1") })); | ||||
|         queue.insert(gen_task(5, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test1") })); | ||||
|         queue.insert(gen_task(6, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test2") })); | ||||
|  | ||||
|         let test1_tasks = queue | ||||
|             .head_mut(|tasks| tasks.drain().map(|t| t.id).collect::<Vec<_>>()) | ||||
|             .unwrap(); | ||||
|  | ||||
|         assert_eq!(test1_tasks, &[0, 4, 5]); | ||||
|  | ||||
|         let test2_tasks = queue | ||||
|             .head_mut(|tasks| tasks.drain().map(|t| t.id).collect::<Vec<_>>()) | ||||
|             .unwrap(); | ||||
|  | ||||
|         assert_eq!(test2_tasks, &[1, 2, 3, 6]); | ||||
|  | ||||
|         assert!(queue.index_tasks.is_empty()); | ||||
|         assert!(queue.queue.is_empty()); | ||||
|     } | ||||
|  | ||||
|     fn gen_doc_addition_task_content(index_uid: &str) -> TaskContent { | ||||
|         TaskContent::DocumentAddition { | ||||
|             content_uuid: Uuid::new_v4(), | ||||
|             merge_strategy: IndexDocumentsMethod::ReplaceDocuments, | ||||
|             primary_key: Some("test".to_string()), | ||||
|             documents_count: 0, | ||||
|             allow_index_creation: true, | ||||
|             index_uid: IndexUid::new_unchecked(index_uid), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     #[test] | ||||
|     #[rustfmt::skip] | ||||
|     fn test_make_batch() { | ||||
|         let mut queue = TaskQueue::default(); | ||||
|         queue.insert(gen_task(0, gen_doc_addition_task_content("test1"))); | ||||
|         queue.insert(gen_task(1, gen_doc_addition_task_content("test2"))); | ||||
|         queue.insert(gen_task(2, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test2")})); | ||||
|         queue.insert(gen_task(3, gen_doc_addition_task_content("test2"))); | ||||
|         queue.insert(gen_task(4, gen_doc_addition_task_content("test1"))); | ||||
|         queue.insert(gen_task(5, TaskContent::IndexDeletion { index_uid: IndexUid::new_unchecked("test1")})); | ||||
|         queue.insert(gen_task(6, gen_doc_addition_task_content("test2"))); | ||||
|         queue.insert(gen_task(7, gen_doc_addition_task_content("test1"))); | ||||
|         queue.insert(gen_task(8, TaskContent::Dump { uid: "adump".to_owned() })); | ||||
|  | ||||
|         let config = SchedulerConfig::default(); | ||||
|  | ||||
|         // Make sure that the dump is processed before everybody else. | ||||
|         let batch = make_batch(&mut queue, &config); | ||||
|         assert_eq!(batch, Processing::Dump(8)); | ||||
|  | ||||
|         let batch = make_batch(&mut queue, &config); | ||||
|         assert_eq!(batch, Processing::DocumentAdditions(vec![0, 4])); | ||||
|  | ||||
|         let batch = make_batch(&mut queue, &config); | ||||
|         assert_eq!(batch, Processing::DocumentAdditions(vec![1])); | ||||
|  | ||||
|         let batch = make_batch(&mut queue, &config); | ||||
|         assert_eq!(batch, Processing::IndexUpdate(2)); | ||||
|  | ||||
|         let batch = make_batch(&mut queue, &config); | ||||
|         assert_eq!(batch, Processing::DocumentAdditions(vec![3, 6])); | ||||
|  | ||||
|         let batch = make_batch(&mut queue, &config); | ||||
|         assert_eq!(batch, Processing::IndexUpdate(5)); | ||||
|  | ||||
|         let batch = make_batch(&mut queue, &config); | ||||
|         assert_eq!(batch, Processing::DocumentAdditions(vec![7])); | ||||
|  | ||||
|         assert!(queue.is_empty()); | ||||
|     } | ||||
| } | ||||
| @@ -1,93 +0,0 @@ | ||||
| use std::sync::Arc; | ||||
|  | ||||
| use time::OffsetDateTime; | ||||
| use tokio::sync::{watch, RwLock}; | ||||
|  | ||||
| use super::batch::Batch; | ||||
| use super::error::Result; | ||||
| use super::{BatchHandler, Scheduler}; | ||||
| use crate::tasks::task::TaskEvent; | ||||
|  | ||||
| /// The update loop sequentially performs batches of updates by asking the scheduler for a batch, | ||||
| /// and handing it to the `TaskPerformer`. | ||||
| pub struct UpdateLoop { | ||||
|     scheduler: Arc<RwLock<Scheduler>>, | ||||
|     performers: Vec<Arc<dyn BatchHandler + Send + Sync + 'static>>, | ||||
|  | ||||
|     notifier: Option<watch::Receiver<()>>, | ||||
| } | ||||
|  | ||||
| impl UpdateLoop { | ||||
|     pub fn new( | ||||
|         scheduler: Arc<RwLock<Scheduler>>, | ||||
|         performers: Vec<Arc<dyn BatchHandler + Send + Sync + 'static>>, | ||||
|         notifier: watch::Receiver<()>, | ||||
|     ) -> Self { | ||||
|         Self { | ||||
|             scheduler, | ||||
|             performers, | ||||
|             notifier: Some(notifier), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub async fn run(mut self) { | ||||
|         let mut notifier = self.notifier.take().unwrap(); | ||||
|  | ||||
|         loop { | ||||
|             if notifier.changed().await.is_err() { | ||||
|                 break; | ||||
|             } | ||||
|  | ||||
|             if let Err(e) = self.process_next_batch().await { | ||||
|                 log::error!("an error occurred while processing an update batch: {}", e); | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     async fn process_next_batch(&self) -> Result<()> { | ||||
|         let mut batch = { self.scheduler.write().await.prepare().await? }; | ||||
|         let performer = self | ||||
|             .performers | ||||
|             .iter() | ||||
|             .find(|p| p.accept(&batch)) | ||||
|             .expect("No performer found for batch") | ||||
|             .clone(); | ||||
|  | ||||
|         batch | ||||
|             .content | ||||
|             .push_event(TaskEvent::Processing(OffsetDateTime::now_utc())); | ||||
|  | ||||
|         batch.content = { | ||||
|             self.scheduler | ||||
|                 .read() | ||||
|                 .await | ||||
|                 .update_tasks(batch.content) | ||||
|                 .await? | ||||
|         }; | ||||
|  | ||||
|         let batch = performer.process_batch(batch).await; | ||||
|  | ||||
|         self.handle_batch_result(batch, performer).await?; | ||||
|  | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     /// Handles the result from a processed batch. | ||||
|     /// | ||||
|     /// When a task is processed, the result of the process is pushed to its event list. The | ||||
|     /// `handle_batch_result` make sure that the new state is saved to the store. | ||||
|     /// The tasks are then removed from the processing queue. | ||||
|     async fn handle_batch_result( | ||||
|         &self, | ||||
|         mut batch: Batch, | ||||
|         performer: Arc<dyn BatchHandler + Sync + Send + 'static>, | ||||
|     ) -> Result<()> { | ||||
|         let mut scheduler = self.scheduler.write().await; | ||||
|         let content = scheduler.update_tasks(batch.content).await?; | ||||
|         scheduler.finish(); | ||||
|         drop(scheduler); | ||||
|         batch.content = content; | ||||
|         performer.finish(&batch).await; | ||||
|         Ok(()) | ||||
|     } | ||||
| } | ||||
| @@ -1,258 +0,0 @@ | ||||
| use std::fs::{create_dir_all, File}; | ||||
| use std::io::{self, BufReader, BufWriter, Write}; | ||||
| use std::ops::{Deref, DerefMut}; | ||||
| use std::path::{Path, PathBuf}; | ||||
|  | ||||
| use milli::documents::DocumentsBatchReader; | ||||
| use serde_json::Map; | ||||
| use tempfile::{NamedTempFile, PersistError}; | ||||
| use uuid::Uuid; | ||||
|  | ||||
| #[cfg(not(test))] | ||||
| pub use store::UpdateFileStore; | ||||
| #[cfg(test)] | ||||
| pub use test::MockUpdateFileStore as UpdateFileStore; | ||||
|  | ||||
| const UPDATE_FILES_PATH: &str = "updates/updates_files"; | ||||
|  | ||||
| use crate::document_formats::read_ndjson; | ||||
|  | ||||
| pub struct UpdateFile { | ||||
|     path: PathBuf, | ||||
|     file: NamedTempFile, | ||||
| } | ||||
|  | ||||
| #[derive(Debug, thiserror::Error)] | ||||
| #[error("Error while persisting update to disk: {0}")] | ||||
| pub struct UpdateFileStoreError(Box<dyn std::error::Error + Sync + Send + 'static>); | ||||
|  | ||||
| pub type Result<T> = std::result::Result<T, UpdateFileStoreError>; | ||||
|  | ||||
| macro_rules! into_update_store_error { | ||||
|     ($($other:path),*) => { | ||||
|         $( | ||||
|             impl From<$other> for UpdateFileStoreError { | ||||
|                 fn from(other: $other) -> Self { | ||||
|                     Self(Box::new(other)) | ||||
|                 } | ||||
|             } | ||||
|         )* | ||||
|     }; | ||||
| } | ||||
|  | ||||
| into_update_store_error!( | ||||
|     PersistError, | ||||
|     io::Error, | ||||
|     serde_json::Error, | ||||
|     milli::documents::Error, | ||||
|     milli::documents::DocumentsBatchCursorError | ||||
| ); | ||||
|  | ||||
| impl UpdateFile { | ||||
|     pub fn persist(self) -> Result<()> { | ||||
|         self.file.persist(&self.path)?; | ||||
|         Ok(()) | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Deref for UpdateFile { | ||||
|     type Target = NamedTempFile; | ||||
|  | ||||
|     fn deref(&self) -> &Self::Target { | ||||
|         &self.file | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl DerefMut for UpdateFile { | ||||
|     fn deref_mut(&mut self) -> &mut Self::Target { | ||||
|         &mut self.file | ||||
|     } | ||||
| } | ||||
|  | ||||
| mod store { | ||||
|     use super::*; | ||||
|  | ||||
|     #[derive(Clone, Debug)] | ||||
|     pub struct UpdateFileStore { | ||||
|         path: PathBuf, | ||||
|     } | ||||
|  | ||||
|     impl UpdateFileStore { | ||||
|         pub fn load_dump(src: impl AsRef<Path>, dst: impl AsRef<Path>) -> anyhow::Result<()> { | ||||
|             let src_update_files_path = src.as_ref().join(UPDATE_FILES_PATH); | ||||
|             let dst_update_files_path = dst.as_ref().join(UPDATE_FILES_PATH); | ||||
|  | ||||
|             // No update files to load | ||||
|             if !src_update_files_path.exists() { | ||||
|                 return Ok(()); | ||||
|             } | ||||
|  | ||||
|             create_dir_all(&dst_update_files_path)?; | ||||
|  | ||||
|             let entries = std::fs::read_dir(src_update_files_path)?; | ||||
|  | ||||
|             for entry in entries { | ||||
|                 let entry = entry?; | ||||
|                 let update_file = BufReader::new(File::open(entry.path())?); | ||||
|                 let file_uuid = entry.file_name(); | ||||
|                 let file_uuid = file_uuid | ||||
|                     .to_str() | ||||
|                     .ok_or_else(|| anyhow::anyhow!("invalid update file name"))?; | ||||
|                 let dst_path = dst_update_files_path.join(file_uuid); | ||||
|                 let dst_file = BufWriter::new(File::create(dst_path)?); | ||||
|                 read_ndjson(update_file, dst_file)?; | ||||
|             } | ||||
|  | ||||
|             Ok(()) | ||||
|         } | ||||
|  | ||||
|         pub fn new(path: impl AsRef<Path>) -> Result<Self> { | ||||
|             let path = path.as_ref().join(UPDATE_FILES_PATH); | ||||
|             std::fs::create_dir_all(&path)?; | ||||
|             Ok(Self { path }) | ||||
|         } | ||||
|  | ||||
|         /// Creates a new temporary update file. | ||||
|         /// A call to `persist` is needed to persist the file in the database. | ||||
|         pub fn new_update(&self) -> Result<(Uuid, UpdateFile)> { | ||||
|             let file = NamedTempFile::new_in(&self.path)?; | ||||
|             let uuid = Uuid::new_v4(); | ||||
|             let path = self.path.join(uuid.to_string()); | ||||
|             let update_file = UpdateFile { file, path }; | ||||
|  | ||||
|             Ok((uuid, update_file)) | ||||
|         } | ||||
|  | ||||
|         /// Returns the file corresponding to the requested uuid. | ||||
|         pub fn get_update(&self, uuid: Uuid) -> Result<File> { | ||||
|             let path = self.path.join(uuid.to_string()); | ||||
|             let file = File::open(path)?; | ||||
|             Ok(file) | ||||
|         } | ||||
|  | ||||
|         /// Copies the content of the update file pointed to by `uuid` to the `dst` directory. | ||||
|         pub fn snapshot(&self, uuid: Uuid, dst: impl AsRef<Path>) -> Result<()> { | ||||
|             let src = self.path.join(uuid.to_string()); | ||||
|             let mut dst = dst.as_ref().join(UPDATE_FILES_PATH); | ||||
|             std::fs::create_dir_all(&dst)?; | ||||
|             dst.push(uuid.to_string()); | ||||
|             std::fs::copy(src, dst)?; | ||||
|             Ok(()) | ||||
|         } | ||||
|  | ||||
|         /// Peforms a dump of the given update file uuid into the provided dump path. | ||||
|         pub fn dump(&self, uuid: Uuid, dump_path: impl AsRef<Path>) -> Result<()> { | ||||
|             let uuid_string = uuid.to_string(); | ||||
|             let update_file_path = self.path.join(&uuid_string); | ||||
|             let mut dst = dump_path.as_ref().join(UPDATE_FILES_PATH); | ||||
|             std::fs::create_dir_all(&dst)?; | ||||
|             dst.push(&uuid_string); | ||||
|  | ||||
|             let update_file = File::open(update_file_path)?; | ||||
|             let mut dst_file = NamedTempFile::new_in(&dump_path)?; | ||||
|             let (mut document_cursor, index) = | ||||
|                 DocumentsBatchReader::from_reader(update_file)?.into_cursor_and_fields_index(); | ||||
|  | ||||
|             let mut document_buffer = Map::new(); | ||||
|             // TODO: we need to find a way to do this more efficiently. (create a custom serializer | ||||
|             // for jsonl for example...) | ||||
|             while let Some(document) = document_cursor.next_document()? { | ||||
|                 for (field_id, content) in document.iter() { | ||||
|                     if let Some(field_name) = index.name(field_id) { | ||||
|                         let content = serde_json::from_slice(content)?; | ||||
|                         document_buffer.insert(field_name.to_string(), content); | ||||
|                     } | ||||
|                 } | ||||
|  | ||||
|                 serde_json::to_writer(&mut dst_file, &document_buffer)?; | ||||
|                 dst_file.write_all(b"\n")?; | ||||
|                 document_buffer.clear(); | ||||
|             } | ||||
|  | ||||
|             dst_file.persist(dst)?; | ||||
|  | ||||
|             Ok(()) | ||||
|         } | ||||
|  | ||||
|         pub fn get_size(&self, uuid: Uuid) -> Result<u64> { | ||||
|             Ok(self.get_update(uuid)?.metadata()?.len()) | ||||
|         } | ||||
|  | ||||
|         pub async fn delete(&self, uuid: Uuid) -> Result<()> { | ||||
|             let path = self.path.join(uuid.to_string()); | ||||
|             tokio::fs::remove_file(path).await?; | ||||
|             Ok(()) | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| #[cfg(test)] | ||||
| mod test { | ||||
|     use std::sync::Arc; | ||||
|  | ||||
|     use nelson::Mocker; | ||||
|  | ||||
|     use super::*; | ||||
|  | ||||
|     #[derive(Clone)] | ||||
|     pub enum MockUpdateFileStore { | ||||
|         Real(store::UpdateFileStore), | ||||
|         Mock(Arc<Mocker>), | ||||
|     } | ||||
|  | ||||
|     impl MockUpdateFileStore { | ||||
|         pub fn mock(mocker: Mocker) -> Self { | ||||
|             Self::Mock(Arc::new(mocker)) | ||||
|         } | ||||
|  | ||||
|         pub fn load_dump(src: impl AsRef<Path>, dst: impl AsRef<Path>) -> anyhow::Result<()> { | ||||
|             store::UpdateFileStore::load_dump(src, dst) | ||||
|         } | ||||
|  | ||||
|         pub fn new(path: impl AsRef<Path>) -> Result<Self> { | ||||
|             store::UpdateFileStore::new(path).map(Self::Real) | ||||
|         } | ||||
|  | ||||
|         pub fn new_update(&self) -> Result<(Uuid, UpdateFile)> { | ||||
|             match self { | ||||
|                 MockUpdateFileStore::Real(s) => s.new_update(), | ||||
|                 MockUpdateFileStore::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub fn get_update(&self, uuid: Uuid) -> Result<File> { | ||||
|             match self { | ||||
|                 MockUpdateFileStore::Real(s) => s.get_update(uuid), | ||||
|                 MockUpdateFileStore::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub fn snapshot(&self, uuid: Uuid, dst: impl AsRef<Path>) -> Result<()> { | ||||
|             match self { | ||||
|                 MockUpdateFileStore::Real(s) => s.snapshot(uuid, dst), | ||||
|                 MockUpdateFileStore::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub fn dump(&self, uuid: Uuid, dump_path: impl AsRef<Path>) -> Result<()> { | ||||
|             match self { | ||||
|                 MockUpdateFileStore::Real(s) => s.dump(uuid, dump_path), | ||||
|                 MockUpdateFileStore::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub fn get_size(&self, uuid: Uuid) -> Result<u64> { | ||||
|             match self { | ||||
|                 MockUpdateFileStore::Real(s) => s.get_size(uuid), | ||||
|                 MockUpdateFileStore::Mock(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn delete(&self, uuid: Uuid) -> Result<()> { | ||||
|             match self { | ||||
|                 MockUpdateFileStore::Real(s) => s.delete(uuid).await, | ||||
|                 MockUpdateFileStore::Mock(mocker) => unsafe { mocker.get("delete").call(uuid) }, | ||||
|             } | ||||
|         } | ||||
|     } | ||||
| } | ||||
		Reference in New Issue
	
	Block a user