mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-04 20:37:15 +02:00
feat(all): Task store
implements: https://github.com/meilisearch/specifications/blob/develop/text/0060-refashion-updates-apis.md linked PR: - #1889 - #1891 - #1892 - #1902 - #1906 - #1911 - #1914 - #1915 - #1916 - #1918 - #1924 - #1925 - #1926 - #1930 - #1936 - #1937 - #1942 - #1944 - #1945 - #1946 - #1947 - #1950 - #1951 - #1957 - #1959 - #1960 - #1961 - #1962 - #1964 - https://github.com/meilisearch/milli/pull/414 - https://github.com/meilisearch/milli/pull/409 - https://github.com/meilisearch/milli/pull/406 - https://github.com/meilisearch/milli/pull/418 - close #1687 - close #1786 - close #1940 - close #1948 - close #1949 - close #1932 - close #1956
This commit is contained in:
parent
c9f3726447
commit
a30e02c18c
88 changed files with 5553 additions and 4496 deletions
22
meilisearch-lib/src/tasks/batch.rs
Normal file
22
meilisearch-lib/src/tasks/batch.rs
Normal file
|
@ -0,0 +1,22 @@
|
|||
use chrono::{DateTime, Utc};
|
||||
|
||||
use super::{task::Task, task_store::Pending};
|
||||
|
||||
pub type BatchId = u32;
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct Batch {
|
||||
pub id: BatchId,
|
||||
pub created_at: DateTime<Utc>,
|
||||
pub tasks: Vec<Pending<Task>>,
|
||||
}
|
||||
|
||||
impl Batch {
|
||||
pub fn len(&self) -> usize {
|
||||
self.tasks.len()
|
||||
}
|
||||
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.tasks.is_empty()
|
||||
}
|
||||
}
|
33
meilisearch-lib/src/tasks/error.rs
Normal file
33
meilisearch-lib/src/tasks/error.rs
Normal file
|
@ -0,0 +1,33 @@
|
|||
use meilisearch_error::{Code, ErrorCode};
|
||||
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: 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,
|
||||
}
|
||||
}
|
||||
}
|
60
meilisearch-lib/src/tasks/mod.rs
Normal file
60
meilisearch-lib/src/tasks/mod.rs
Normal file
|
@ -0,0 +1,60 @@
|
|||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
|
||||
use async_trait::async_trait;
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
#[cfg(test)]
|
||||
pub use task_store::test::MockTaskStore as TaskStore;
|
||||
#[cfg(not(test))]
|
||||
pub use task_store::TaskStore;
|
||||
|
||||
pub use task_store::{Pending, TaskFilter};
|
||||
|
||||
use batch::Batch;
|
||||
use error::Result;
|
||||
use scheduler::Scheduler;
|
||||
|
||||
pub mod batch;
|
||||
pub mod error;
|
||||
pub mod scheduler;
|
||||
pub mod task;
|
||||
mod task_store;
|
||||
|
||||
#[cfg_attr(test, mockall::automock(type Error=test::DebugError;))]
|
||||
#[async_trait]
|
||||
pub trait TaskPerformer: Sync + Send + 'static {
|
||||
type Error: Serialize + for<'de> Deserialize<'de> + std::error::Error + Sync + Send + 'static;
|
||||
/// Processes the `Task` batch returning the batch with the `Task` updated.
|
||||
async fn process(&self, batch: Batch) -> Batch;
|
||||
/// `finish` is called when the result of `process` has been commited 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);
|
||||
}
|
||||
|
||||
pub fn create_task_store<P>(env: heed::Env, performer: Arc<P>) -> Result<TaskStore>
|
||||
where
|
||||
P: TaskPerformer,
|
||||
{
|
||||
let task_store = TaskStore::new(env)?;
|
||||
let scheduler = Scheduler::new(task_store.clone(), performer, Duration::from_millis(1));
|
||||
tokio::task::spawn_local(scheduler.run());
|
||||
Ok(task_store)
|
||||
}
|
||||
|
||||
#[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 {}
|
||||
}
|
253
meilisearch-lib/src/tasks/scheduler.rs
Normal file
253
meilisearch-lib/src/tasks/scheduler.rs
Normal file
|
@ -0,0 +1,253 @@
|
|||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
|
||||
use chrono::Utc;
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use super::batch::Batch;
|
||||
use super::error::Result;
|
||||
#[cfg(test)]
|
||||
use super::task_store::test::MockTaskStore as TaskStore;
|
||||
use super::task_store::Pending;
|
||||
#[cfg(not(test))]
|
||||
use super::task_store::TaskStore;
|
||||
use super::TaskPerformer;
|
||||
use crate::tasks::task::TaskEvent;
|
||||
|
||||
/// The scheduler roles is to perform batches of tasks one at a time. It will monitor the TaskStore
|
||||
/// for new tasks, put them in a batch, and process the batch as soon as possible.
|
||||
///
|
||||
/// When a batch is currently processing, the scheduler is just waiting.
|
||||
pub struct Scheduler<P: TaskPerformer> {
|
||||
store: TaskStore,
|
||||
performer: Arc<P>,
|
||||
|
||||
/// The interval at which the the `TaskStore` should be checked for new updates
|
||||
task_store_check_interval: Duration,
|
||||
}
|
||||
|
||||
impl<P> Scheduler<P>
|
||||
where
|
||||
P: TaskPerformer + Send + Sync + 'static,
|
||||
P::Error: Serialize + for<'de> Deserialize<'de> + Send + Sync + 'static,
|
||||
{
|
||||
pub fn new(store: TaskStore, performer: Arc<P>, task_store_check_interval: Duration) -> Self {
|
||||
Self {
|
||||
store,
|
||||
performer,
|
||||
task_store_check_interval,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn run(self) {
|
||||
loop {
|
||||
if let Err(e) = self.process_next_batch().await {
|
||||
log::error!("an error occured while processing an update batch: {}", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn process_next_batch(&self) -> Result<()> {
|
||||
match self.prepare_batch().await? {
|
||||
Some(mut batch) => {
|
||||
for task in &mut batch.tasks {
|
||||
match task {
|
||||
Pending::Task(task) => task.events.push(TaskEvent::Processing(Utc::now())),
|
||||
Pending::Job(_) => (),
|
||||
}
|
||||
}
|
||||
|
||||
// the jobs are ignored
|
||||
batch.tasks = self.store.update_tasks(batch.tasks).await?;
|
||||
|
||||
let performer = self.performer.clone();
|
||||
let batch_result = performer.process(batch).await;
|
||||
self.handle_batch_result(batch_result).await?;
|
||||
}
|
||||
None => {
|
||||
// No update found to create a batch we wait a bit before we retry.
|
||||
tokio::time::sleep(self.task_store_check_interval).await;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Checks for pending tasks and groups them in a batch. If there are no pending update,
|
||||
/// return Ok(None)
|
||||
///
|
||||
/// Until batching is properly implemented, the batches contain only one task.
|
||||
async fn prepare_batch(&self) -> Result<Option<Batch>> {
|
||||
match self.store.peek_pending_task().await {
|
||||
Some(Pending::Task(next_task_id)) => {
|
||||
let mut task = self.store.get_task(next_task_id, None).await?;
|
||||
|
||||
task.events.push(TaskEvent::Batched {
|
||||
timestamp: Utc::now(),
|
||||
batch_id: 0,
|
||||
});
|
||||
|
||||
let batch = Batch {
|
||||
id: 0,
|
||||
// index_uid: task.index_uid.clone(),
|
||||
created_at: Utc::now(),
|
||||
tasks: vec![Pending::Task(task)],
|
||||
};
|
||||
Ok(Some(batch))
|
||||
}
|
||||
Some(Pending::Job(job)) => Ok(Some(Batch {
|
||||
id: 0,
|
||||
created_at: Utc::now(),
|
||||
tasks: vec![Pending::Job(job)],
|
||||
})),
|
||||
None => Ok(None),
|
||||
}
|
||||
}
|
||||
|
||||
/// Handles the result from a batch processing.
|
||||
///
|
||||
/// When a task is processed, the result of the processing is pushed to its event list. The
|
||||
/// handle batch result make sure that the new state is save into its store.
|
||||
/// The tasks are then removed from the processing queue.
|
||||
async fn handle_batch_result(&self, mut batch: Batch) -> Result<()> {
|
||||
let tasks = self.store.update_tasks(batch.tasks).await?;
|
||||
batch.tasks = tasks;
|
||||
self.store.delete_pending(&batch.tasks[0]).await;
|
||||
self.performer.finish(&batch).await;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use nelson::Mocker;
|
||||
|
||||
use crate::index_resolver::IndexUid;
|
||||
use crate::tasks::task::Task;
|
||||
use crate::tasks::task_store::TaskFilter;
|
||||
|
||||
use super::super::task::{TaskContent, TaskEvent, TaskId, TaskResult};
|
||||
use super::super::MockTaskPerformer;
|
||||
use super::*;
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_prepare_batch_full() {
|
||||
let mocker = Mocker::default();
|
||||
|
||||
mocker
|
||||
.when::<(TaskId, Option<TaskFilter>), Result<Option<Task>>>("get_task")
|
||||
.once()
|
||||
.then(|(id, _filter)| {
|
||||
let task = Task {
|
||||
id,
|
||||
index_uid: IndexUid::new("Test".to_string()).unwrap(),
|
||||
content: TaskContent::IndexDeletion,
|
||||
events: vec![TaskEvent::Created(Utc::now())],
|
||||
};
|
||||
Ok(Some(task))
|
||||
});
|
||||
|
||||
mocker
|
||||
.when::<(), Option<Pending<TaskId>>>("peek_pending_task")
|
||||
.then(|()| Some(Pending::Task(1)));
|
||||
|
||||
let store = TaskStore::mock(mocker);
|
||||
let performer = Arc::new(MockTaskPerformer::new());
|
||||
|
||||
let scheduler = Scheduler {
|
||||
store,
|
||||
performer,
|
||||
task_store_check_interval: Duration::from_millis(1),
|
||||
};
|
||||
|
||||
let batch = scheduler.prepare_batch().await.unwrap().unwrap();
|
||||
|
||||
assert_eq!(batch.tasks.len(), 1);
|
||||
assert!(
|
||||
matches!(batch.tasks[0], Pending::Task(Task { id: 1, .. })),
|
||||
"{:?}",
|
||||
batch.tasks[0]
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_prepare_batch_empty() {
|
||||
let mocker = Mocker::default();
|
||||
mocker
|
||||
.when::<(), Option<Pending<TaskId>>>("peek_pending_task")
|
||||
.then(|()| None);
|
||||
|
||||
let store = TaskStore::mock(mocker);
|
||||
let performer = Arc::new(MockTaskPerformer::new());
|
||||
|
||||
let scheduler = Scheduler {
|
||||
store,
|
||||
performer,
|
||||
task_store_check_interval: Duration::from_millis(1),
|
||||
};
|
||||
|
||||
assert!(scheduler.prepare_batch().await.unwrap().is_none());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_loop_run_normal() {
|
||||
let mocker = Mocker::default();
|
||||
let mut id = Some(1);
|
||||
mocker
|
||||
.when::<(), Option<Pending<TaskId>>>("peek_pending_task")
|
||||
.then(move |()| id.take().map(Pending::Task));
|
||||
mocker
|
||||
.when::<(TaskId, Option<TaskFilter>), Result<Task>>("get_task")
|
||||
.once()
|
||||
.then(|(id, _)| {
|
||||
let task = Task {
|
||||
id,
|
||||
index_uid: IndexUid::new("Test".to_string()).unwrap(),
|
||||
content: TaskContent::IndexDeletion,
|
||||
events: vec![TaskEvent::Created(Utc::now())],
|
||||
};
|
||||
Ok(task)
|
||||
});
|
||||
|
||||
mocker
|
||||
.when::<Vec<Pending<Task>>, Result<Vec<Pending<Task>>>>("update_tasks")
|
||||
.times(2)
|
||||
.then(|tasks| {
|
||||
assert_eq!(tasks.len(), 1);
|
||||
Ok(tasks)
|
||||
});
|
||||
|
||||
mocker.when::<(), ()>("delete_pending").once().then(|_| ());
|
||||
|
||||
let store = TaskStore::mock(mocker);
|
||||
|
||||
let mut performer = MockTaskPerformer::new();
|
||||
performer.expect_process().once().returning(|mut batch| {
|
||||
batch.tasks.iter_mut().for_each(|t| match t {
|
||||
Pending::Task(Task { ref mut events, .. }) => events.push(TaskEvent::Succeded {
|
||||
result: TaskResult::Other,
|
||||
timestamp: Utc::now(),
|
||||
}),
|
||||
_ => panic!("expected a task, found a job"),
|
||||
});
|
||||
|
||||
batch
|
||||
});
|
||||
|
||||
performer.expect_finish().once().returning(|_| ());
|
||||
|
||||
let performer = Arc::new(performer);
|
||||
|
||||
let scheduler = Scheduler {
|
||||
store,
|
||||
performer,
|
||||
task_store_check_interval: Duration::from_millis(1),
|
||||
};
|
||||
|
||||
let handle = tokio::spawn(scheduler.run());
|
||||
|
||||
if let Ok(r) = tokio::time::timeout(Duration::from_millis(100), handle).await {
|
||||
r.unwrap();
|
||||
}
|
||||
}
|
||||
}
|
169
meilisearch-lib/src/tasks/task.rs
Normal file
169
meilisearch-lib/src/tasks/task.rs
Normal file
|
@ -0,0 +1,169 @@
|
|||
use std::path::PathBuf;
|
||||
|
||||
use chrono::{DateTime, Utc};
|
||||
use meilisearch_error::ResponseError;
|
||||
use milli::update::{DocumentAdditionResult, IndexDocumentsMethod};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use tokio::sync::oneshot;
|
||||
use uuid::Uuid;
|
||||
|
||||
use super::batch::BatchId;
|
||||
use crate::{
|
||||
index::{Settings, Unchecked},
|
||||
index_resolver::{error::IndexResolverError, IndexUid},
|
||||
snapshot::SnapshotJob,
|
||||
};
|
||||
|
||||
pub type TaskId = u64;
|
||||
|
||||
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq)]
|
||||
#[cfg_attr(test, derive(proptest_derive::Arbitrary))]
|
||||
pub enum TaskResult {
|
||||
DocumentAddition { indexed_documents: u64 },
|
||||
DocumentDeletion { deleted_documents: u64 },
|
||||
ClearAll { deleted_documents: u64 },
|
||||
Other,
|
||||
}
|
||||
|
||||
impl From<DocumentAdditionResult> for TaskResult {
|
||||
fn from(other: DocumentAdditionResult) -> Self {
|
||||
Self::DocumentAddition {
|
||||
indexed_documents: other.indexed_documents,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq)]
|
||||
#[cfg_attr(test, derive(proptest_derive::Arbitrary))]
|
||||
pub enum TaskEvent {
|
||||
Created(#[cfg_attr(test, proptest(strategy = "test::datetime_strategy()"))] DateTime<Utc>),
|
||||
Batched {
|
||||
#[cfg_attr(test, proptest(strategy = "test::datetime_strategy()"))]
|
||||
timestamp: DateTime<Utc>,
|
||||
batch_id: BatchId,
|
||||
},
|
||||
Processing(#[cfg_attr(test, proptest(strategy = "test::datetime_strategy()"))] DateTime<Utc>),
|
||||
Succeded {
|
||||
result: TaskResult,
|
||||
#[cfg_attr(test, proptest(strategy = "test::datetime_strategy()"))]
|
||||
timestamp: DateTime<Utc>,
|
||||
},
|
||||
Failed {
|
||||
error: ResponseError,
|
||||
#[cfg_attr(test, proptest(strategy = "test::datetime_strategy()"))]
|
||||
timestamp: DateTime<Utc>,
|
||||
},
|
||||
}
|
||||
|
||||
/// A task represents an operation that Meilisearch must do.
|
||||
/// It's stored on disk and executed from the lowest to highest Task id.
|
||||
/// Everytime a new task is created it has a higher Task id than the previous one.
|
||||
/// See also `Job`.
|
||||
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
|
||||
#[cfg_attr(test, derive(proptest_derive::Arbitrary))]
|
||||
pub struct Task {
|
||||
pub id: TaskId,
|
||||
pub index_uid: IndexUid,
|
||||
pub content: TaskContent,
|
||||
pub events: Vec<TaskEvent>,
|
||||
}
|
||||
|
||||
impl Task {
|
||||
/// Return true when a task is finished.
|
||||
/// A task is finished when its last state is either `Succeeded` or `Failed`.
|
||||
pub fn is_finished(&self) -> bool {
|
||||
self.events.last().map_or(false, |event| {
|
||||
matches!(event, TaskEvent::Succeded { .. } | TaskEvent::Failed { .. })
|
||||
})
|
||||
}
|
||||
|
||||
/// Return the content_uuid of the `Task` if there is one.
|
||||
pub fn get_content_uuid(&self) -> Option<Uuid> {
|
||||
match self {
|
||||
Task {
|
||||
content: TaskContent::DocumentAddition { content_uuid, .. },
|
||||
..
|
||||
} => Some(*content_uuid),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A job is like a volatile priority `Task`.
|
||||
/// It should be processed as fast as possible and is not stored on disk.
|
||||
/// This means, when Meilisearch is closed all your unprocessed jobs will disappear.
|
||||
#[derive(Debug, derivative::Derivative)]
|
||||
#[derivative(PartialEq)]
|
||||
pub enum Job {
|
||||
Dump {
|
||||
#[derivative(PartialEq = "ignore")]
|
||||
ret: oneshot::Sender<Result<(), IndexResolverError>>,
|
||||
path: PathBuf,
|
||||
},
|
||||
Snapshot(#[derivative(PartialEq = "ignore")] SnapshotJob),
|
||||
Empty,
|
||||
}
|
||||
|
||||
impl Default for Job {
|
||||
fn default() -> Self {
|
||||
Self::Empty
|
||||
}
|
||||
}
|
||||
|
||||
impl Job {
|
||||
pub fn take(&mut self) -> Self {
|
||||
std::mem::take(self)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
|
||||
#[cfg_attr(test, derive(proptest_derive::Arbitrary))]
|
||||
pub enum DocumentDeletion {
|
||||
Clear,
|
||||
Ids(Vec<String>),
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
|
||||
#[cfg_attr(test, derive(proptest_derive::Arbitrary))]
|
||||
#[allow(clippy::large_enum_variant)]
|
||||
pub enum TaskContent {
|
||||
DocumentAddition {
|
||||
#[cfg_attr(test, proptest(value = "Uuid::new_v4()"))]
|
||||
content_uuid: Uuid,
|
||||
#[cfg_attr(test, proptest(strategy = "test::index_document_method_strategy()"))]
|
||||
merge_strategy: IndexDocumentsMethod,
|
||||
primary_key: Option<String>,
|
||||
documents_count: usize,
|
||||
},
|
||||
DocumentDeletion(DocumentDeletion),
|
||||
SettingsUpdate {
|
||||
settings: Settings<Unchecked>,
|
||||
/// Indicates whether the task was a deletion
|
||||
is_deletion: bool,
|
||||
},
|
||||
IndexDeletion,
|
||||
IndexCreation {
|
||||
primary_key: Option<String>,
|
||||
},
|
||||
IndexUpdate {
|
||||
primary_key: Option<String>,
|
||||
},
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use proptest::prelude::*;
|
||||
|
||||
use super::*;
|
||||
|
||||
pub(super) fn index_document_method_strategy() -> impl Strategy<Value = IndexDocumentsMethod> {
|
||||
prop_oneof![
|
||||
Just(IndexDocumentsMethod::ReplaceDocuments),
|
||||
Just(IndexDocumentsMethod::UpdateDocuments),
|
||||
]
|
||||
}
|
||||
|
||||
pub(super) fn datetime_strategy() -> impl Strategy<Value = DateTime<Utc>> {
|
||||
Just(Utc::now())
|
||||
}
|
||||
}
|
480
meilisearch-lib/src/tasks/task_store/mod.rs
Normal file
480
meilisearch-lib/src/tasks/task_store/mod.rs
Normal file
|
@ -0,0 +1,480 @@
|
|||
mod store;
|
||||
|
||||
use std::cmp::Ordering;
|
||||
use std::collections::{BinaryHeap, HashSet};
|
||||
use std::io::{BufWriter, Write};
|
||||
use std::path::Path;
|
||||
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::Result;
|
||||
use crate::index_resolver::IndexUid;
|
||||
use crate::tasks::task::TaskEvent;
|
||||
use crate::update_file_store::UpdateFileStore;
|
||||
|
||||
#[cfg(test)]
|
||||
pub use store::test::MockStore as Store;
|
||||
#[cfg(not(test))]
|
||||
pub use store::Store;
|
||||
|
||||
/// Defines constraints to be applied when querying for Tasks from the store.
|
||||
#[derive(Default, Debug)]
|
||||
pub struct TaskFilter {
|
||||
indexes: Option<HashSet<String>>,
|
||||
}
|
||||
|
||||
impl TaskFilter {
|
||||
fn pass(&self, task: &Task) -> bool {
|
||||
self.indexes
|
||||
.as_ref()
|
||||
.map(|indexes| indexes.contains(&*task.index_uid))
|
||||
.unwrap_or(true)
|
||||
}
|
||||
|
||||
/// Adds an index to the filter, so the filter must match this index.
|
||||
pub fn filter_index(&mut self, index: String) {
|
||||
self.indexes
|
||||
.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 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: 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)),
|
||||
})
|
||||
}
|
||||
|
||||
pub async fn register(&self, index_uid: IndexUid, content: TaskContent) -> Result<Task> {
|
||||
debug!("registering update: {:?}", content);
|
||||
let store = self.store.clone();
|
||||
let task = tokio::task::spawn_blocking(move || -> Result<Task> {
|
||||
let mut txn = store.wtxn()?;
|
||||
let next_task_id = store.next_task_id(&mut txn)?;
|
||||
let created_at = TaskEvent::Created(Utc::now());
|
||||
let task = Task {
|
||||
id: next_task_id,
|
||||
index_uid,
|
||||
content,
|
||||
events: vec![created_at],
|
||||
};
|
||||
|
||||
store.put(&mut txn, &task)?;
|
||||
txn.commit()?;
|
||||
|
||||
Ok(task)
|
||||
})
|
||||
.await??;
|
||||
|
||||
self.pending_queue
|
||||
.write()
|
||||
.await
|
||||
.push(Pending::Task(task.id));
|
||||
|
||||
Ok(task)
|
||||
}
|
||||
|
||||
pub fn register_raw_update(&self, wtxn: &mut RwTxn, task: &Task) -> Result<()> {
|
||||
self.store.put(wtxn, task)?;
|
||||
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<_> {
|
||||
let txn = store.rtxn()?;
|
||||
let task = store.get(&txn, id)?;
|
||||
Ok(task)
|
||||
})
|
||||
.await??
|
||||
.ok_or(TaskError::UnexistingTask(id))?;
|
||||
|
||||
match filter {
|
||||
Some(filter) => filter
|
||||
.pass(&task)
|
||||
.then(|| task)
|
||||
.ok_or(TaskError::UnexistingTask(id)),
|
||||
None => Ok(task),
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn update_tasks(&self, tasks: Vec<Pending<Task>>) -> Result<Vec<Pending<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(_) => (),
|
||||
}
|
||||
}
|
||||
|
||||
txn.commit()?;
|
||||
|
||||
Ok(tasks)
|
||||
})
|
||||
.await??;
|
||||
|
||||
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>,
|
||||
filter: Option<TaskFilter>,
|
||||
limit: Option<usize>,
|
||||
) -> Result<Vec<Task>> {
|
||||
let store = self.store.clone();
|
||||
|
||||
tokio::task::spawn_blocking(move || {
|
||||
let txn = store.rtxn()?;
|
||||
let tasks = store.list_tasks(&txn, offset, filter, limit)?;
|
||||
Ok(tasks)
|
||||
})
|
||||
.await?
|
||||
}
|
||||
|
||||
pub async fn dump(
|
||||
&self,
|
||||
dir_path: impl AsRef<Path>,
|
||||
update_file_store: UpdateFileStore,
|
||||
) -> Result<()> {
|
||||
let update_dir = dir_path.as_ref().join("updates");
|
||||
let updates_file = update_dir.join("data.jsonl");
|
||||
let tasks = self.list_tasks(None, None, None).await?;
|
||||
|
||||
let dir_path = dir_path.as_ref().to_path_buf();
|
||||
tokio::task::spawn_blocking(move || -> Result<()> {
|
||||
std::fs::create_dir(&update_dir)?;
|
||||
let updates_file = std::fs::File::create(updates_file)?;
|
||||
let mut updates_file = BufWriter::new(updates_file);
|
||||
|
||||
for task in tasks {
|
||||
serde_json::to_writer(&mut updates_file, &task)?;
|
||||
updates_file.write_all(b"\n")?;
|
||||
|
||||
if !task.is_finished() {
|
||||
if let Some(content_uuid) = task.get_content_uuid() {
|
||||
update_file_store.dump(content_uuid, &dir_path)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
updates_file.flush()?;
|
||||
Ok(())
|
||||
})
|
||||
.await??;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn load_dump(src: impl AsRef<Path>, env: Env) -> anyhow::Result<()> {
|
||||
// create a dummy update field store, since it is not needed right now.
|
||||
let store = Self::new(env.clone())?;
|
||||
|
||||
let src_update_path = src.as_ref().join("updates");
|
||||
let update_data = std::fs::File::open(&src_update_path.join("data.jsonl"))?;
|
||||
let update_data = std::io::BufReader::new(update_data);
|
||||
|
||||
let stream = serde_json::Deserializer::from_reader(update_data).into_iter::<Task>();
|
||||
|
||||
let mut wtxn = env.write_txn()?;
|
||||
for entry in stream {
|
||||
store.register_raw_update(&mut wtxn, &entry?)?;
|
||||
}
|
||||
wtxn.commit()?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub mod test {
|
||||
use crate::tasks::task_store::store::test::tmp_env;
|
||||
|
||||
use super::*;
|
||||
|
||||
use nelson::Mocker;
|
||||
use proptest::{
|
||||
strategy::Strategy,
|
||||
test_runner::{Config, TestRunner},
|
||||
};
|
||||
|
||||
pub enum MockTaskStore {
|
||||
Real(TaskStore),
|
||||
Mock(Arc<Mocker>),
|
||||
}
|
||||
|
||||
impl Clone for MockTaskStore {
|
||||
fn clone(&self) -> Self {
|
||||
match self {
|
||||
Self::Real(x) => Self::Real(x.clone()),
|
||||
Self::Mock(x) => Self::Mock(x.clone()),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl MockTaskStore {
|
||||
pub fn new(env: heed::Env) -> Result<Self> {
|
||||
Ok(Self::Real(TaskStore::new(env)?))
|
||||
}
|
||||
|
||||
pub fn mock(mocker: Mocker) -> Self {
|
||||
Self::Mock(Arc::new(mocker))
|
||||
}
|
||||
|
||||
pub async fn update_tasks(&self, tasks: Vec<Pending<Task>>) -> Result<Vec<Pending<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)
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
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,
|
||||
Self::Mock(m) => unsafe { m.get::<_, Result<Task>>("get_task").call((id, filter)) },
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn get_processing_task(&self) -> Result<Option<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(())
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn list_tasks(
|
||||
&self,
|
||||
from: Option<TaskId>,
|
||||
filter: Option<TaskFilter>,
|
||||
limit: Option<usize>,
|
||||
) -> Result<Vec<Task>> {
|
||||
match self {
|
||||
Self::Real(s) => s.list_tasks(from, filter, limit).await,
|
||||
Self::Mock(_m) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn dump(&self, path: &Path, update_file_store: UpdateFileStore) -> Result<()> {
|
||||
match self {
|
||||
Self::Real(s) => s.dump(path, update_file_store).await,
|
||||
Self::Mock(_m) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn register(&self, index_uid: IndexUid, content: TaskContent) -> Result<Task> {
|
||||
match self {
|
||||
Self::Real(s) => s.register(index_uid, content).await,
|
||||
Self::Mock(_m) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn register_raw_update(&self, wtxn: &mut RwTxn, task: &Task) -> Result<()> {
|
||||
match self {
|
||||
Self::Real(s) => s.register_raw_update(wtxn, task),
|
||||
Self::Mock(_m) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
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: Env) -> anyhow::Result<()> {
|
||||
TaskStore::load_dump(path, env)
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_increment_task_id() {
|
||||
let tmp = tmp_env();
|
||||
let store = Store::new(tmp.env()).unwrap();
|
||||
|
||||
let mut txn = store.wtxn().unwrap();
|
||||
assert_eq!(store.next_task_id(&mut txn).unwrap(), 0);
|
||||
txn.abort().unwrap();
|
||||
|
||||
let gen_task = |id: TaskId| Task {
|
||||
id,
|
||||
index_uid: IndexUid::new_unchecked("test"),
|
||||
content: TaskContent::IndexCreation { primary_key: None },
|
||||
events: Vec::new(),
|
||||
};
|
||||
|
||||
let mut runner = TestRunner::new(Config::default());
|
||||
runner
|
||||
.run(&(0..100u64).prop_map(gen_task), |task| {
|
||||
let mut txn = store.wtxn().unwrap();
|
||||
let previous_id = store.next_task_id(&mut txn).unwrap();
|
||||
|
||||
store.put(&mut txn, &task).unwrap();
|
||||
|
||||
let next_id = store.next_task_id(&mut txn).unwrap();
|
||||
|
||||
// if we put a task whose task_id is less than the next_id, then the next_id remains
|
||||
// unchanged, otherwise it becomes task.id + 1
|
||||
if task.id < previous_id {
|
||||
assert_eq!(next_id, previous_id)
|
||||
} else {
|
||||
assert_eq!(next_id, task.id + 1);
|
||||
}
|
||||
|
||||
txn.commit().unwrap();
|
||||
|
||||
Ok(())
|
||||
})
|
||||
.unwrap();
|
||||
}
|
||||
}
|
452
meilisearch-lib/src/tasks/task_store/store.rs
Normal file
452
meilisearch-lib/src/tasks/task_store/store.rs
Normal file
|
@ -0,0 +1,452 @@
|
|||
#[allow(clippy::upper_case_acronyms)]
|
||||
type BEU64 = heed::zerocopy::U64<heed::byteorder::BE>;
|
||||
|
||||
const UID_TASK_IDS: &str = "uid_task_id";
|
||||
const TASKS: &str = "tasks";
|
||||
|
||||
use std::borrow::Cow;
|
||||
use std::collections::BinaryHeap;
|
||||
use std::convert::TryInto;
|
||||
use std::mem::size_of;
|
||||
use std::ops::Range;
|
||||
use std::result::Result as StdResult;
|
||||
|
||||
use heed::types::{ByteSlice, OwnedType, SerdeJson, Unit};
|
||||
use heed::{BytesDecode, BytesEncode, Database, Env, RoTxn, RwTxn};
|
||||
|
||||
use crate::tasks::task::{Task, TaskId};
|
||||
|
||||
use super::super::Result;
|
||||
|
||||
use super::{Pending, TaskFilter};
|
||||
|
||||
enum IndexUidTaskIdCodec {}
|
||||
|
||||
impl<'a> BytesEncode<'a> for IndexUidTaskIdCodec {
|
||||
type EItem = (&'a str, TaskId);
|
||||
|
||||
fn bytes_encode((s, id): &'a Self::EItem) -> Option<Cow<'a, [u8]>> {
|
||||
let size = s.len() + std::mem::size_of::<TaskId>() + 1;
|
||||
if size > 512 {
|
||||
return None;
|
||||
}
|
||||
let mut b = Vec::with_capacity(size);
|
||||
b.extend_from_slice(s.as_bytes());
|
||||
// null terminate the string
|
||||
b.push(0);
|
||||
b.extend_from_slice(&id.to_be_bytes());
|
||||
Some(Cow::Owned(b))
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> BytesDecode<'a> for IndexUidTaskIdCodec {
|
||||
type DItem = (&'a str, TaskId);
|
||||
|
||||
fn bytes_decode(bytes: &'a [u8]) -> Option<Self::DItem> {
|
||||
let len = bytes.len();
|
||||
let s_end = len.checked_sub(size_of::<TaskId>())?.checked_sub(1)?;
|
||||
let str_bytes = &bytes[..s_end];
|
||||
let str = std::str::from_utf8(str_bytes).ok()?;
|
||||
let id = TaskId::from_be_bytes(bytes[(len - size_of::<TaskId>())..].try_into().ok()?);
|
||||
Some((str, id))
|
||||
}
|
||||
}
|
||||
|
||||
pub struct Store {
|
||||
env: Env,
|
||||
uids_task_ids: Database<IndexUidTaskIdCodec, Unit>,
|
||||
tasks: Database<OwnedType<BEU64>, SerdeJson<Task>>,
|
||||
}
|
||||
|
||||
impl Store {
|
||||
/// Create a new store from the specified `Path`.
|
||||
/// Be really cautious when calling this function, the returned `Store` may
|
||||
/// be in an invalid state, with dangling processing tasks.
|
||||
/// You want to patch all un-finished tasks and put them in your pending
|
||||
/// queue with the `reset_and_return_unfinished_update` method.
|
||||
pub fn new(env: heed::Env) -> Result<Self> {
|
||||
let uids_task_ids = env.create_database(Some(UID_TASK_IDS))?;
|
||||
let tasks = env.create_database(Some(TASKS))?;
|
||||
|
||||
Ok(Self {
|
||||
env,
|
||||
uids_task_ids,
|
||||
tasks,
|
||||
})
|
||||
}
|
||||
|
||||
/// 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()?)
|
||||
}
|
||||
|
||||
pub fn rtxn(&self) -> Result<RoTxn> {
|
||||
Ok(self.env.read_txn()?)
|
||||
}
|
||||
|
||||
/// Returns the id for the next task.
|
||||
///
|
||||
/// The required `mut txn` acts as a reservation system. It guarantees that as long as you commit
|
||||
/// the task to the store in the same transaction, no one else will hav this task id.
|
||||
pub fn next_task_id(&self, txn: &mut RwTxn) -> Result<TaskId> {
|
||||
let id = self
|
||||
.tasks
|
||||
.lazily_decode_data()
|
||||
.last(txn)?
|
||||
.map(|(id, _)| id.get() + 1)
|
||||
.unwrap_or(0);
|
||||
Ok(id)
|
||||
}
|
||||
|
||||
pub fn put(&self, txn: &mut RwTxn, task: &Task) -> Result<()> {
|
||||
self.tasks.put(txn, &BEU64::new(task.id), task)?;
|
||||
self.uids_task_ids
|
||||
.put(txn, &(&task.index_uid, task.id), &())?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn get(&self, txn: &RoTxn, id: TaskId) -> Result<Option<Task>> {
|
||||
let task = self.tasks.get(txn, &BEU64::new(id))?;
|
||||
Ok(task)
|
||||
}
|
||||
|
||||
pub fn list_tasks<'a>(
|
||||
&self,
|
||||
txn: &'a RoTxn,
|
||||
from: Option<TaskId>,
|
||||
filter: Option<TaskFilter>,
|
||||
limit: Option<usize>,
|
||||
) -> Result<Vec<Task>> {
|
||||
let from = from.unwrap_or_default();
|
||||
let range = from..limit
|
||||
.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) => {
|
||||
let iter = self
|
||||
.compute_candidates(txn, filter, range)?
|
||||
.into_iter()
|
||||
.filter_map(|id| self.tasks.get(txn, &BEU64::new(id)).transpose());
|
||||
|
||||
Box::new(iter)
|
||||
}
|
||||
None => Box::new(
|
||||
self.tasks
|
||||
.rev_range(txn, &(BEU64::new(range.start)..BEU64::new(range.end)))?
|
||||
.map(|r| r.map(|(_, t)| t)),
|
||||
),
|
||||
};
|
||||
|
||||
// Collect 'limit' task if it exists or all of them.
|
||||
let tasks = iter
|
||||
.take(limit.unwrap_or(usize::MAX))
|
||||
.try_fold::<_, _, StdResult<_, heed::Error>>(Vec::new(), |mut v, task| {
|
||||
v.push(task?);
|
||||
Ok(v)
|
||||
})?;
|
||||
|
||||
Ok(tasks)
|
||||
}
|
||||
|
||||
fn compute_candidates(
|
||||
&self,
|
||||
txn: &heed::RoTxn,
|
||||
filter: TaskFilter,
|
||||
range: Range<TaskId>,
|
||||
) -> Result<BinaryHeap<TaskId>> {
|
||||
let mut candidates = BinaryHeap::new();
|
||||
if let Some(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
|
||||
// prefix, i.e test and test1.
|
||||
let mut index_uid = index.as_bytes().to_vec();
|
||||
index_uid.push(0);
|
||||
|
||||
self.uids_task_ids
|
||||
.remap_key_type::<ByteSlice>()
|
||||
.rev_prefix_iter(txn, &index_uid)?
|
||||
.map(|entry| -> StdResult<_, heed::Error> {
|
||||
let (key, _) = entry?;
|
||||
let (_, id) =
|
||||
IndexUidTaskIdCodec::bytes_decode(key).ok_or(heed::Error::Decoding)?;
|
||||
Ok(id)
|
||||
})
|
||||
.skip_while(|entry| {
|
||||
entry
|
||||
.as_ref()
|
||||
.ok()
|
||||
// we skip all elements till we enter in the range
|
||||
.map(|key| !range.contains(key))
|
||||
// if we encounter an error we returns true to collect it later
|
||||
.unwrap_or(true)
|
||||
})
|
||||
.take_while(|entry| {
|
||||
entry
|
||||
.as_ref()
|
||||
.ok()
|
||||
// as soon as we are out of the range we exit
|
||||
.map(|key| range.contains(key))
|
||||
// if we encounter an error we returns true to collect it later
|
||||
.unwrap_or(true)
|
||||
})
|
||||
.try_for_each::<_, StdResult<(), heed::Error>>(|id| {
|
||||
candidates.push(id?);
|
||||
Ok(())
|
||||
})?;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(candidates)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub mod test {
|
||||
use heed::EnvOpenOptions;
|
||||
use itertools::Itertools;
|
||||
use nelson::Mocker;
|
||||
use proptest::collection::vec;
|
||||
use proptest::prelude::*;
|
||||
use tempfile::TempDir;
|
||||
|
||||
use crate::index_resolver::IndexUid;
|
||||
use crate::tasks::task::TaskContent;
|
||||
|
||||
use super::*;
|
||||
|
||||
/// TODO: use this mock to test the task store properly.
|
||||
#[allow(dead_code)]
|
||||
pub enum MockStore {
|
||||
Real(Store),
|
||||
Fake(Mocker),
|
||||
}
|
||||
|
||||
pub struct TmpEnv(TempDir, heed::Env);
|
||||
|
||||
impl TmpEnv {
|
||||
pub fn env(&self) -> heed::Env {
|
||||
self.1.clone()
|
||||
}
|
||||
}
|
||||
|
||||
pub fn tmp_env() -> TmpEnv {
|
||||
let tmp = tempfile::tempdir().unwrap();
|
||||
|
||||
let mut options = EnvOpenOptions::new();
|
||||
options.map_size(4096 * 100000);
|
||||
options.max_dbs(1000);
|
||||
let env = options.open(tmp.path()).unwrap();
|
||||
|
||||
TmpEnv(tmp, env)
|
||||
}
|
||||
|
||||
impl MockStore {
|
||||
pub fn new(env: heed::Env) -> Result<Self> {
|
||||
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(),
|
||||
MockStore::Fake(_) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn rtxn(&self) -> Result<RoTxn> {
|
||||
match self {
|
||||
MockStore::Real(index) => index.rtxn(),
|
||||
MockStore::Fake(_) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn next_task_id(&self, txn: &mut RwTxn) -> Result<TaskId> {
|
||||
match self {
|
||||
MockStore::Real(index) => index.next_task_id(txn),
|
||||
MockStore::Fake(_) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn put(&self, txn: &mut RwTxn, task: &Task) -> Result<()> {
|
||||
match self {
|
||||
MockStore::Real(index) => index.put(txn, task),
|
||||
MockStore::Fake(_) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn get(&self, txn: &RoTxn, id: TaskId) -> Result<Option<Task>> {
|
||||
match self {
|
||||
MockStore::Real(index) => index.get(txn, id),
|
||||
MockStore::Fake(_) => todo!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn list_tasks<'a>(
|
||||
&self,
|
||||
txn: &'a RoTxn,
|
||||
from: Option<TaskId>,
|
||||
filter: Option<TaskFilter>,
|
||||
limit: Option<usize>,
|
||||
) -> Result<Vec<Task>> {
|
||||
match self {
|
||||
MockStore::Real(index) => index.list_tasks(txn, from, filter, limit),
|
||||
MockStore::Fake(_) => todo!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_ordered_filtered_updates() {
|
||||
let tmp = tmp_env();
|
||||
let store = Store::new(tmp.env()).unwrap();
|
||||
|
||||
let tasks = (0..100)
|
||||
.map(|_| Task {
|
||||
id: rand::random(),
|
||||
index_uid: IndexUid::new_unchecked("test".to_string()),
|
||||
content: TaskContent::IndexDeletion,
|
||||
events: vec![],
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let mut txn = store.env.write_txn().unwrap();
|
||||
tasks
|
||||
.iter()
|
||||
.try_for_each(|t| store.put(&mut txn, t))
|
||||
.unwrap();
|
||||
|
||||
let mut filter = TaskFilter::default();
|
||||
filter.filter_index("test".into());
|
||||
|
||||
let tasks = store.list_tasks(&txn, None, Some(filter), None).unwrap();
|
||||
|
||||
assert!(tasks
|
||||
.iter()
|
||||
.map(|t| t.id)
|
||||
.tuple_windows()
|
||||
.all(|(a, b)| a > b));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_filter_same_index_prefix() {
|
||||
let tmp = tmp_env();
|
||||
let store = Store::new(tmp.env()).unwrap();
|
||||
|
||||
let task_1 = Task {
|
||||
id: 1,
|
||||
index_uid: IndexUid::new_unchecked("test".to_string()),
|
||||
content: TaskContent::IndexDeletion,
|
||||
events: vec![],
|
||||
};
|
||||
|
||||
let task_2 = Task {
|
||||
id: 0,
|
||||
index_uid: IndexUid::new_unchecked("test1".to_string()),
|
||||
content: TaskContent::IndexDeletion,
|
||||
events: vec![],
|
||||
};
|
||||
|
||||
let mut txn = store.wtxn().unwrap();
|
||||
store.put(&mut txn, &task_1).unwrap();
|
||||
store.put(&mut txn, &task_2).unwrap();
|
||||
|
||||
let mut filter = TaskFilter::default();
|
||||
filter.filter_index("test".into());
|
||||
|
||||
let tasks = store.list_tasks(&txn, None, Some(filter), None).unwrap();
|
||||
|
||||
txn.abort().unwrap();
|
||||
assert_eq!(tasks.len(), 1);
|
||||
assert_eq!(&*tasks.first().unwrap().index_uid, "test");
|
||||
|
||||
// same thing but invert the ids
|
||||
let task_1 = Task {
|
||||
id: 0,
|
||||
index_uid: IndexUid::new_unchecked("test".to_string()),
|
||||
content: TaskContent::IndexDeletion,
|
||||
events: vec![],
|
||||
};
|
||||
let task_2 = Task {
|
||||
id: 1,
|
||||
index_uid: IndexUid::new_unchecked("test1".to_string()),
|
||||
content: TaskContent::IndexDeletion,
|
||||
events: vec![],
|
||||
};
|
||||
|
||||
let mut txn = store.wtxn().unwrap();
|
||||
store.put(&mut txn, &task_1).unwrap();
|
||||
store.put(&mut txn, &task_2).unwrap();
|
||||
|
||||
let mut filter = TaskFilter::default();
|
||||
filter.filter_index("test".into());
|
||||
|
||||
let tasks = store.list_tasks(&txn, None, Some(filter), None).unwrap();
|
||||
|
||||
assert_eq!(tasks.len(), 1);
|
||||
assert_eq!(&*tasks.first().unwrap().index_uid, "test");
|
||||
}
|
||||
|
||||
proptest! {
|
||||
#[test]
|
||||
fn encode_decode_roundtrip(index_uid in any::<IndexUid>(), task_id in 0..TaskId::MAX) {
|
||||
let value = (index_uid.as_ref(), task_id);
|
||||
let bytes = IndexUidTaskIdCodec::bytes_encode(&value).unwrap();
|
||||
let (index, id) = IndexUidTaskIdCodec::bytes_decode(bytes.as_ref()).unwrap();
|
||||
assert_eq!(&*index_uid, index);
|
||||
assert_eq!(task_id, id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn encode_doesnt_crash(index_uid in "\\PC*", task_id in 0..TaskId::MAX) {
|
||||
let value = (index_uid.as_ref(), task_id);
|
||||
IndexUidTaskIdCodec::bytes_encode(&value);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn decode_doesnt_crash(bytes in vec(any::<u8>(), 0..1000)) {
|
||||
IndexUidTaskIdCodec::bytes_decode(&bytes);
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue