start integrating the index-scheduler in the meilisearch codebase

This commit is contained in:
Tamo
2022-09-14 16:16:53 +02:00
committed by Clément Renault
parent b816535e33
commit fc098022c7
28 changed files with 679 additions and 3170 deletions

View File

@ -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()
}
}
*/

View File

@ -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);
}
}
*/