mirror of
				https://github.com/meilisearch/meilisearch.git
				synced 2025-10-26 13:36:27 +00:00 
			
		
		
		
	feat(lib): auto-batching
This commit is contained in:
		| @@ -1,7 +1,6 @@ | ||||
| mod store; | ||||
|  | ||||
| use std::cmp::Ordering; | ||||
| use std::collections::{BinaryHeap, HashSet}; | ||||
| use std::collections::HashSet; | ||||
| use std::io::{BufWriter, Write}; | ||||
| use std::path::Path; | ||||
| use std::sync::Arc; | ||||
| @@ -9,11 +8,9 @@ use std::sync::Arc; | ||||
| use chrono::Utc; | ||||
| use heed::{Env, RwTxn}; | ||||
| use log::debug; | ||||
| use tokio::sync::RwLock; | ||||
| use uuid::Uuid; | ||||
|  | ||||
| use super::error::TaskError; | ||||
| use super::task::{Job, Task, TaskContent, TaskId}; | ||||
| use super::task::{Task, TaskContent, TaskId}; | ||||
| use super::Result; | ||||
| use crate::index_resolver::IndexUid; | ||||
| use crate::tasks::task::TaskEvent; | ||||
| @@ -25,9 +22,10 @@ pub use store::test::MockStore as Store; | ||||
| pub use store::Store; | ||||
|  | ||||
| /// Defines constraints to be applied when querying for Tasks from the store. | ||||
| #[derive(Default, Debug)] | ||||
| #[derive(Default)] | ||||
| pub struct TaskFilter { | ||||
|     indexes: Option<HashSet<String>>, | ||||
|     filter_fn: Option<Box<dyn Fn(&Task) -> bool + Sync + Send + 'static>>, | ||||
| } | ||||
|  | ||||
| impl TaskFilter { | ||||
| @@ -44,85 +42,28 @@ impl TaskFilter { | ||||
|             .get_or_insert_with(Default::default) | ||||
|             .insert(index); | ||||
|     } | ||||
| } | ||||
|  | ||||
| /// You can't clone a job because of its volatile nature. | ||||
| /// If you need to take the `Job` with you though. You can call the method | ||||
| /// `Pending::take`. It'll return the `Pending` as-is but `Empty` the original. | ||||
| #[derive(Debug, PartialEq)] | ||||
| pub enum Pending<T> { | ||||
|     /// A task stored on disk that must be processed. | ||||
|     Task(T), | ||||
|     /// Job always have a higher priority over normal tasks and are not stored on disk. | ||||
|     /// It can be refered as `Volatile job`. | ||||
|     Job(Job), | ||||
| } | ||||
|  | ||||
| impl Pending<TaskId> { | ||||
|     /// Makes a copy of the task or take the content of the volatile job. | ||||
|     pub(crate) fn take(&mut self) -> Self { | ||||
|         match self { | ||||
|             Self::Task(id) => Self::Task(*id), | ||||
|             Self::Job(job) => Self::Job(job.take()), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Eq for Pending<TaskId> {} | ||||
|  | ||||
| impl PartialOrd for Pending<TaskId> { | ||||
|     fn partial_cmp(&self, other: &Self) -> Option<Ordering> { | ||||
|         match (self, other) { | ||||
|             // in case of two tasks we want to return the lowest taskId first. | ||||
|             (Pending::Task(lhs), Pending::Task(rhs)) => Some(lhs.cmp(rhs).reverse()), | ||||
|             // A job is always better than a task. | ||||
|             (Pending::Task(_), Pending::Job(_)) => Some(Ordering::Less), | ||||
|             (Pending::Job(_), Pending::Task(_)) => Some(Ordering::Greater), | ||||
|             // When there is two jobs we consider them equals. | ||||
|             (Pending::Job(_), Pending::Job(_)) => Some(Ordering::Equal), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Pending<Task> { | ||||
|     pub fn get_content_uuid(&self) -> Option<Uuid> { | ||||
|         match self { | ||||
|             Pending::Task(task) => task.get_content_uuid(), | ||||
|             _ => None, | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl Ord for Pending<TaskId> { | ||||
|     fn cmp(&self, other: &Self) -> Ordering { | ||||
|         self.partial_cmp(other).unwrap() | ||||
|     pub fn filter_fn(&mut self, f: impl Fn(&Task) -> bool + Sync + Send + 'static) { | ||||
|         self.filter_fn.replace(Box::new(f)); | ||||
|     } | ||||
| } | ||||
|  | ||||
| pub struct TaskStore { | ||||
|     store: Arc<Store>, | ||||
|     pending_queue: Arc<RwLock<BinaryHeap<Pending<TaskId>>>>, | ||||
| } | ||||
|  | ||||
| impl Clone for TaskStore { | ||||
|     fn clone(&self) -> Self { | ||||
|         Self { | ||||
|             store: self.store.clone(), | ||||
|             pending_queue: self.pending_queue.clone(), | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl TaskStore { | ||||
|     pub fn new(env: Arc<heed::Env>) -> Result<Self> { | ||||
|         let mut store = Store::new(env)?; | ||||
|         let unfinished_tasks = store.reset_and_return_unfinished_tasks()?; | ||||
|         let store = Arc::new(store); | ||||
|  | ||||
|         Ok(Self { | ||||
|             store, | ||||
|             pending_queue: Arc::new(RwLock::new(unfinished_tasks)), | ||||
|         }) | ||||
|         let store = Arc::new(Store::new(env)?); | ||||
|         Ok(Self { store }) | ||||
|     } | ||||
|  | ||||
|     pub async fn register(&self, index_uid: IndexUid, content: TaskContent) -> Result<Task> { | ||||
| @@ -146,11 +87,6 @@ impl TaskStore { | ||||
|         }) | ||||
|         .await??; | ||||
|  | ||||
|         self.pending_queue | ||||
|             .write() | ||||
|             .await | ||||
|             .push(Pending::Task(task.id)); | ||||
|  | ||||
|         Ok(task) | ||||
|     } | ||||
|  | ||||
| @@ -159,35 +95,6 @@ impl TaskStore { | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     /// Register an update that applies on multiple indexes. | ||||
|     /// Currently the update is considered as a priority. | ||||
|     pub async fn register_job(&self, content: Job) { | ||||
|         debug!("registering a job: {:?}", content); | ||||
|         self.pending_queue.write().await.push(Pending::Job(content)); | ||||
|     } | ||||
|  | ||||
|     /// Returns the next task to process. | ||||
|     pub async fn peek_pending_task(&self) -> Option<Pending<TaskId>> { | ||||
|         let mut pending_queue = self.pending_queue.write().await; | ||||
|         loop { | ||||
|             match pending_queue.peek()? { | ||||
|                 Pending::Job(Job::Empty) => drop(pending_queue.pop()), | ||||
|                 _ => return Some(pending_queue.peek_mut()?.take()), | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     /// Returns the next task to process if there is one. | ||||
|     pub async fn get_processing_task(&self) -> Result<Option<Task>> { | ||||
|         match self.peek_pending_task().await { | ||||
|             Some(Pending::Task(tid)) => { | ||||
|                 let task = self.get_task(tid, None).await?; | ||||
|                 Ok(matches!(task.events.last(), Some(TaskEvent::Processing(_))).then(|| task)) | ||||
|             } | ||||
|             _ => Ok(None), | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub async fn get_task(&self, id: TaskId, filter: Option<TaskFilter>) -> Result<Task> { | ||||
|         let store = self.store.clone(); | ||||
|         let task = tokio::task::spawn_blocking(move || -> Result<_> { | ||||
| @@ -207,17 +114,33 @@ impl TaskStore { | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub async fn update_tasks(&self, tasks: Vec<Pending<Task>>) -> Result<Vec<Pending<Task>>> { | ||||
|     pub async fn get_pending_tasks(&self, ids: Vec<TaskId>) -> Result<(Vec<TaskId>, Vec<Task>)> { | ||||
|         let store = self.store.clone(); | ||||
|         let tasks = tokio::task::spawn_blocking(move || -> Result<_> { | ||||
|             let mut tasks = Vec::new(); | ||||
|             let txn = store.rtxn()?; | ||||
|  | ||||
|             for id in ids.iter() { | ||||
|                 let task = store | ||||
|                     .get(&txn, *id)? | ||||
|                     .ok_or(TaskError::UnexistingTask(*id))?; | ||||
|                 tasks.push(task); | ||||
|             } | ||||
|             Ok((ids, tasks)) | ||||
|         }) | ||||
|         .await??; | ||||
|  | ||||
|         Ok(tasks) | ||||
|     } | ||||
|  | ||||
|     pub async fn update_tasks(&self, tasks: Vec<Task>) -> Result<Vec<Task>> { | ||||
|         let store = self.store.clone(); | ||||
|  | ||||
|         let tasks = tokio::task::spawn_blocking(move || -> Result<_> { | ||||
|             let mut txn = store.wtxn()?; | ||||
|  | ||||
|             for task in &tasks { | ||||
|                 match task { | ||||
|                     Pending::Task(task) => store.put(&mut txn, task)?, | ||||
|                     Pending::Job(_) => (), | ||||
|                 } | ||||
|                 store.put(&mut txn, task)?; | ||||
|             } | ||||
|  | ||||
|             txn.commit()?; | ||||
| @@ -229,21 +152,6 @@ impl TaskStore { | ||||
|         Ok(tasks) | ||||
|     } | ||||
|  | ||||
|     /// Delete one task from the queue and remove all `Empty` job. | ||||
|     pub async fn delete_pending(&self, to_delete: &Pending<Task>) { | ||||
|         if let Pending::Task(Task { id: pending_id, .. }) = to_delete { | ||||
|             let mut pending_queue = self.pending_queue.write().await; | ||||
|             *pending_queue = std::mem::take(&mut *pending_queue) | ||||
|                 .into_iter() | ||||
|                 .filter(|pending| match pending { | ||||
|                     Pending::Job(Job::Empty) => false, | ||||
|                     Pending::Task(id) => pending_id != id, | ||||
|                     _ => true, | ||||
|                 }) | ||||
|                 .collect::<BinaryHeap<Pending<TaskId>>>(); | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub async fn list_tasks( | ||||
|         &self, | ||||
|         offset: Option<TaskId>, | ||||
| @@ -348,23 +256,15 @@ pub mod test { | ||||
|             Self::Mock(Arc::new(mocker)) | ||||
|         } | ||||
|  | ||||
|         pub async fn update_tasks(&self, tasks: Vec<Pending<Task>>) -> Result<Vec<Pending<Task>>> { | ||||
|         pub async fn update_tasks(&self, tasks: Vec<Task>) -> Result<Vec<Task>> { | ||||
|             match self { | ||||
|                 Self::Real(s) => s.update_tasks(tasks).await, | ||||
|                 Self::Mock(m) => unsafe { | ||||
|                     m.get::<_, Result<Vec<Pending<Task>>>>("update_tasks") | ||||
|                         .call(tasks) | ||||
|                     m.get::<_, Result<Vec<Task>>>("update_tasks").call(tasks) | ||||
|                 }, | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn delete_pending(&self, to_delete: &Pending<Task>) { | ||||
|             match self { | ||||
|                 Self::Real(s) => s.delete_pending(to_delete).await, | ||||
|                 Self::Mock(m) => unsafe { m.get("delete_pending").call(to_delete) }, | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn get_task(&self, id: TaskId, filter: Option<TaskFilter>) -> Result<Task> { | ||||
|             match self { | ||||
|                 Self::Real(s) => s.get_task(id, filter).await, | ||||
| @@ -372,23 +272,13 @@ pub mod test { | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn get_processing_task(&self) -> Result<Option<Task>> { | ||||
|         pub async fn get_pending_tasks( | ||||
|             &self, | ||||
|             tasks: Vec<TaskId>, | ||||
|         ) -> Result<(Vec<TaskId>, Vec<Task>)> { | ||||
|             match self { | ||||
|                 Self::Real(s) => s.get_processing_task().await, | ||||
|                 Self::Mock(m) => unsafe { | ||||
|                     m.get::<_, Result<Option<Task>>>("get_pending_task") | ||||
|                         .call(()) | ||||
|                 }, | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn peek_pending_task(&self) -> Option<Pending<TaskId>> { | ||||
|             match self { | ||||
|                 Self::Real(s) => s.peek_pending_task().await, | ||||
|                 Self::Mock(m) => unsafe { | ||||
|                     m.get::<_, Option<Pending<TaskId>>>("peek_pending_task") | ||||
|                         .call(()) | ||||
|                 }, | ||||
|                 Self::Real(s) => s.get_pending_tasks(tasks).await, | ||||
|                 Self::Mock(m) => unsafe { m.get("get_pending_task").call(tasks) }, | ||||
|             } | ||||
|         } | ||||
|  | ||||
| @@ -400,14 +290,18 @@ pub mod test { | ||||
|         ) -> Result<Vec<Task>> { | ||||
|             match self { | ||||
|                 Self::Real(s) => s.list_tasks(from, filter, limit).await, | ||||
|                 Self::Mock(_m) => todo!(), | ||||
|                 Self::Mock(m) => unsafe { m.get("list_tasks").call((from, filter, limit)) }, | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn dump(&self, path: &Path, update_file_store: UpdateFileStore) -> Result<()> { | ||||
|         pub async fn dump( | ||||
|             &self, | ||||
|             path: impl AsRef<Path>, | ||||
|             update_file_store: UpdateFileStore, | ||||
|         ) -> Result<()> { | ||||
|             match self { | ||||
|                 Self::Real(s) => s.dump(path, update_file_store).await, | ||||
|                 Self::Mock(_m) => todo!(), | ||||
|                 Self::Mock(m) => unsafe { m.get("dump").call((path, update_file_store)) }, | ||||
|             } | ||||
|         } | ||||
|  | ||||
| @@ -425,13 +319,6 @@ pub mod test { | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub async fn register_job(&self, content: Job) { | ||||
|             match self { | ||||
|                 Self::Real(s) => s.register_job(content).await, | ||||
|                 Self::Mock(_m) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub fn load_dump(path: impl AsRef<Path>, env: Arc<Env>) -> anyhow::Result<()> { | ||||
|             TaskStore::load_dump(path, env) | ||||
|         } | ||||
|   | ||||
| @@ -19,7 +19,7 @@ use crate::tasks::task::{Task, TaskId}; | ||||
|  | ||||
| use super::super::Result; | ||||
|  | ||||
| use super::{Pending, TaskFilter}; | ||||
| use super::TaskFilter; | ||||
|  | ||||
| enum IndexUidTaskIdCodec {} | ||||
|  | ||||
| @@ -84,41 +84,6 @@ impl Store { | ||||
|         }) | ||||
|     } | ||||
|  | ||||
|     /// This function should be called *right after* creating the store. | ||||
|     /// It put back all unfinished update in the `Created` state. This | ||||
|     /// allow us to re-enqueue an update that didn't had the time to finish | ||||
|     /// when Meilisearch closed. | ||||
|     pub fn reset_and_return_unfinished_tasks(&mut self) -> Result<BinaryHeap<Pending<TaskId>>> { | ||||
|         let mut unfinished_tasks: BinaryHeap<Pending<TaskId>> = BinaryHeap::new(); | ||||
|  | ||||
|         let mut wtxn = self.wtxn()?; | ||||
|         let mut iter = self.tasks.rev_iter_mut(&mut wtxn)?; | ||||
|  | ||||
|         while let Some(entry) = iter.next() { | ||||
|             let entry = entry?; | ||||
|             let (id, mut task): (BEU64, Task) = entry; | ||||
|  | ||||
|             // Since all tasks are ordered, we can stop iterating when we encounter our first non-finished task. | ||||
|             if task.is_finished() { | ||||
|                 break; | ||||
|             } | ||||
|  | ||||
|             // we only keep the first state. It’s supposed to be a `Created` state. | ||||
|             task.events.drain(1..); | ||||
|             unfinished_tasks.push(Pending::Task(id.get())); | ||||
|  | ||||
|             // Since we own the id and the task this is a safe operation. | ||||
|             unsafe { | ||||
|                 iter.put_current(&id, &task)?; | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         drop(iter); | ||||
|         wtxn.commit()?; | ||||
|  | ||||
|         Ok(unfinished_tasks) | ||||
|     } | ||||
|  | ||||
|     pub fn wtxn(&self) -> Result<RwTxn> { | ||||
|         Ok(self.env.write_txn()?) | ||||
|     } | ||||
| @@ -166,7 +131,11 @@ impl Store { | ||||
|             .map(|limit| (limit as u64).saturating_add(from)) | ||||
|             .unwrap_or(u64::MAX); | ||||
|         let iter: Box<dyn Iterator<Item = StdResult<_, heed::Error>>> = match filter { | ||||
|             Some(filter) => { | ||||
|             Some( | ||||
|                 ref filter @ TaskFilter { | ||||
|                     indexes: Some(_), .. | ||||
|                 }, | ||||
|             ) => { | ||||
|                 let iter = self | ||||
|                     .compute_candidates(txn, filter, range)? | ||||
|                     .into_iter() | ||||
| @@ -174,15 +143,24 @@ impl Store { | ||||
|  | ||||
|                 Box::new(iter) | ||||
|             } | ||||
|             None => Box::new( | ||||
|             _ => Box::new( | ||||
|                 self.tasks | ||||
|                     .rev_range(txn, &(BEU64::new(range.start)..BEU64::new(range.end)))? | ||||
|                     .map(|r| r.map(|(_, t)| t)), | ||||
|             ), | ||||
|         }; | ||||
|  | ||||
|         let apply_fitler = |task: &StdResult<_, heed::Error>| match task { | ||||
|             Ok(ref t) => filter | ||||
|                 .as_ref() | ||||
|                 .and_then(|filter| filter.filter_fn.as_ref()) | ||||
|                 .map(|f| f(t)) | ||||
|                 .unwrap_or(true), | ||||
|             Err(_) => true, | ||||
|         }; | ||||
|         // Collect 'limit' task if it exists or all of them. | ||||
|         let tasks = iter | ||||
|             .filter(apply_fitler) | ||||
|             .take(limit.unwrap_or(usize::MAX)) | ||||
|             .try_fold::<_, _, StdResult<_, heed::Error>>(Vec::new(), |mut v, task| { | ||||
|                 v.push(task?); | ||||
| @@ -195,11 +173,11 @@ impl Store { | ||||
|     fn compute_candidates( | ||||
|         &self, | ||||
|         txn: &heed::RoTxn, | ||||
|         filter: TaskFilter, | ||||
|         filter: &TaskFilter, | ||||
|         range: Range<TaskId>, | ||||
|     ) -> Result<BinaryHeap<TaskId>> { | ||||
|         let mut candidates = BinaryHeap::new(); | ||||
|         if let Some(indexes) = filter.indexes { | ||||
|         if let Some(ref indexes) = filter.indexes { | ||||
|             for index in indexes { | ||||
|                 // We need to prefix search the null terminated string to make sure that we only | ||||
|                 // get exact matches for the index, and not other uids that would share the same | ||||
| @@ -290,13 +268,6 @@ pub mod test { | ||||
|             Ok(Self::Real(Store::new(env)?)) | ||||
|         } | ||||
|  | ||||
|         pub fn reset_and_return_unfinished_tasks(&mut self) -> Result<BinaryHeap<Pending<TaskId>>> { | ||||
|             match self { | ||||
|                 MockStore::Real(index) => index.reset_and_return_unfinished_tasks(), | ||||
|                 MockStore::Fake(_) => todo!(), | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         pub fn wtxn(&self) -> Result<RwTxn> { | ||||
|             match self { | ||||
|                 MockStore::Real(index) => index.wtxn(), | ||||
|   | ||||
		Reference in New Issue
	
	Block a user