Split the index-scheduler in ~500 loc modules

This commit is contained in:
Tamo 2025-01-02 14:13:51 +01:00
parent 7f1071943e
commit cb82b0798a
No known key found for this signature in database
GPG key ID: 20CD8020AFA88D69
251 changed files with 9431 additions and 9079 deletions

View file

@ -0,0 +1,537 @@
use std::ops::{Bound, RangeBounds};
use meilisearch_types::batches::{Batch, BatchId};
use meilisearch_types::heed::types::{DecodeIgnore, SerdeBincode, SerdeJson, Str};
use meilisearch_types::heed::{Database, Env, RoTxn, RwTxn};
use meilisearch_types::milli::{CboRoaringBitmapCodec, RoaringBitmapCodec, BEU32};
use meilisearch_types::tasks::{Kind, Status};
use roaring::{MultiOps, RoaringBitmap};
use time::OffsetDateTime;
use super::{Query, Queue};
use crate::processing::ProcessingTasks;
use crate::utils::{insert_task_datetime, keep_ids_within_datetimes, map_bound, ProcessingBatch};
use crate::{Error, Result, BEI128};
/// Database const names for the `IndexScheduler`.
mod db_name {
pub const ALL_BATCHES: &str = "all-batches";
pub const BATCH_STATUS: &str = "batch-status";
pub const BATCH_KIND: &str = "batch-kind";
pub const BATCH_INDEX_TASKS: &str = "batch-index-tasks";
pub const BATCH_ENQUEUED_AT: &str = "batch-enqueued-at";
pub const BATCH_STARTED_AT: &str = "batch-started-at";
pub const BATCH_FINISHED_AT: &str = "batch-finished-at";
}
pub struct BatchQueue {
/// Contains all the batches accessible by their Id.
pub(crate) all_batches: Database<BEU32, SerdeJson<Batch>>,
/// All the batches containing a task matching the selected status.
pub(crate) status: Database<SerdeBincode<Status>, RoaringBitmapCodec>,
/// All the batches ids grouped by the kind of their task.
pub(crate) kind: Database<SerdeBincode<Kind>, RoaringBitmapCodec>,
/// Store the batches associated to an index.
pub(crate) index_tasks: Database<Str, RoaringBitmapCodec>,
/// Store the batches containing tasks which were enqueued at a specific date
pub(crate) enqueued_at: Database<BEI128, CboRoaringBitmapCodec>,
/// Store the batches containing finished tasks started at a specific date
pub(crate) started_at: Database<BEI128, CboRoaringBitmapCodec>,
/// Store the batches containing tasks finished at a specific date
pub(crate) finished_at: Database<BEI128, CboRoaringBitmapCodec>,
}
impl BatchQueue {
pub(crate) fn private_clone(&self) -> BatchQueue {
BatchQueue {
all_batches: self.all_batches,
status: self.status,
kind: self.kind,
index_tasks: self.index_tasks,
enqueued_at: self.enqueued_at,
started_at: self.started_at,
finished_at: self.finished_at,
}
}
pub(super) fn new(env: &Env, wtxn: &mut RwTxn) -> Result<Self> {
Ok(Self {
all_batches: env.create_database(wtxn, Some(db_name::ALL_BATCHES))?,
status: env.create_database(wtxn, Some(db_name::BATCH_STATUS))?,
kind: env.create_database(wtxn, Some(db_name::BATCH_KIND))?,
index_tasks: env.create_database(wtxn, Some(db_name::BATCH_INDEX_TASKS))?,
enqueued_at: env.create_database(wtxn, Some(db_name::BATCH_ENQUEUED_AT))?,
started_at: env.create_database(wtxn, Some(db_name::BATCH_STARTED_AT))?,
finished_at: env.create_database(wtxn, Some(db_name::BATCH_FINISHED_AT))?,
})
}
pub(crate) fn all_batch_ids(&self, rtxn: &RoTxn) -> Result<RoaringBitmap> {
enum_iterator::all().map(|s| self.get_status(rtxn, s)).union()
}
pub(crate) fn next_batch_id(&self, rtxn: &RoTxn) -> Result<BatchId> {
Ok(self
.all_batches
.remap_data_type::<DecodeIgnore>()
.last(rtxn)?
.map(|(k, _)| k + 1)
.unwrap_or_default())
}
pub(crate) fn get_batch(&self, rtxn: &RoTxn, batch_id: BatchId) -> Result<Option<Batch>> {
Ok(self.all_batches.get(rtxn, &batch_id)?)
}
/// Returns the whole set of batches that belongs to this index.
pub(crate) fn index_batches(&self, rtxn: &RoTxn, index: &str) -> Result<RoaringBitmap> {
Ok(self.index_tasks.get(rtxn, index)?.unwrap_or_default())
}
pub(crate) fn update_index(
&self,
wtxn: &mut RwTxn,
index: &str,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut batches = self.index_batches(wtxn, index)?;
f(&mut batches);
if batches.is_empty() {
self.index_tasks.delete(wtxn, index)?;
} else {
self.index_tasks.put(wtxn, index, &batches)?;
}
Ok(())
}
pub(crate) fn get_status(&self, rtxn: &RoTxn, status: Status) -> Result<RoaringBitmap> {
Ok(self.status.get(rtxn, &status)?.unwrap_or_default())
}
pub(crate) fn put_status(
&self,
wtxn: &mut RwTxn,
status: Status,
bitmap: &RoaringBitmap,
) -> Result<()> {
Ok(self.status.put(wtxn, &status, bitmap)?)
}
pub(crate) fn update_status(
&self,
wtxn: &mut RwTxn,
status: Status,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.get_status(wtxn, status)?;
f(&mut tasks);
self.put_status(wtxn, status, &tasks)?;
Ok(())
}
pub(crate) fn get_kind(&self, rtxn: &RoTxn, kind: Kind) -> Result<RoaringBitmap> {
Ok(self.kind.get(rtxn, &kind)?.unwrap_or_default())
}
pub(crate) fn put_kind(
&self,
wtxn: &mut RwTxn,
kind: Kind,
bitmap: &RoaringBitmap,
) -> Result<()> {
Ok(self.kind.put(wtxn, &kind, bitmap)?)
}
pub(crate) fn update_kind(
&self,
wtxn: &mut RwTxn,
kind: Kind,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.get_kind(wtxn, kind)?;
f(&mut tasks);
self.put_kind(wtxn, kind, &tasks)?;
Ok(())
}
pub(crate) fn write_batch(&self, wtxn: &mut RwTxn, batch: ProcessingBatch) -> Result<()> {
self.all_batches.put(
wtxn,
&batch.uid,
&Batch {
uid: batch.uid,
progress: None,
details: batch.details,
stats: batch.stats,
started_at: batch.started_at,
finished_at: batch.finished_at,
},
)?;
for status in batch.statuses {
self.update_status(wtxn, status, |bitmap| {
bitmap.insert(batch.uid);
})?;
}
for kind in batch.kinds {
self.update_kind(wtxn, kind, |bitmap| {
bitmap.insert(batch.uid);
})?;
}
for index in batch.indexes {
self.update_index(wtxn, &index, |bitmap| {
bitmap.insert(batch.uid);
})?;
}
if let Some(enqueued_at) = batch.oldest_enqueued_at {
insert_task_datetime(wtxn, self.enqueued_at, enqueued_at, batch.uid)?;
}
if let Some(enqueued_at) = batch.earliest_enqueued_at {
insert_task_datetime(wtxn, self.enqueued_at, enqueued_at, batch.uid)?;
}
insert_task_datetime(wtxn, self.started_at, batch.started_at, batch.uid)?;
insert_task_datetime(wtxn, self.finished_at, batch.finished_at.unwrap(), batch.uid)?;
Ok(())
}
/// Convert an iterator to a `Vec` of batches. The batches MUST exist or a
/// `CorruptedTaskQueue` error will be thrown.
pub(crate) fn get_existing_batches(
&self,
rtxn: &RoTxn,
tasks: impl IntoIterator<Item = BatchId>,
processing: &ProcessingTasks,
) -> Result<Vec<Batch>> {
tasks
.into_iter()
.map(|batch_id| {
if Some(batch_id) == processing.batch.as_ref().map(|batch| batch.uid) {
let mut batch = processing.batch.as_ref().unwrap().to_batch();
batch.progress = processing.get_progress_view();
Ok(batch)
} else {
self.get_batch(rtxn, batch_id)
.and_then(|task| task.ok_or(Error::CorruptedTaskQueue))
}
})
.collect::<Result<_>>()
}
}
impl Queue {
/// Return the batch ids matched by the given query from the index scheduler's point of view.
pub(crate) fn get_batch_ids(
&self,
rtxn: &RoTxn,
query: &Query,
processing: &ProcessingTasks,
) -> Result<RoaringBitmap> {
let Query {
limit,
from,
reverse,
uids,
batch_uids,
statuses,
types,
index_uids,
canceled_by,
before_enqueued_at,
after_enqueued_at,
before_started_at,
after_started_at,
before_finished_at,
after_finished_at,
} = query;
let mut batches = self.batches.all_batch_ids(rtxn)?;
if let Some(batch_id) = processing.batch.as_ref().map(|batch| batch.uid) {
batches.insert(batch_id);
}
if let Some(from) = from {
let range = if reverse.unwrap_or_default() {
u32::MIN..*from
} else {
from.saturating_add(1)..u32::MAX
};
batches.remove_range(range);
}
if let Some(batch_uids) = &batch_uids {
let batches_uids = RoaringBitmap::from_iter(batch_uids);
batches &= batches_uids;
}
if let Some(status) = &statuses {
let mut status_batches = RoaringBitmap::new();
for status in status {
match status {
// special case for Processing batches
Status::Processing => {
if let Some(batch_id) = processing.batch.as_ref().map(|batch| batch.uid) {
status_batches.insert(batch_id);
}
}
// Enqueued tasks are not stored in batches
Status::Enqueued => (),
status => status_batches |= &self.batches.get_status(rtxn, *status)?,
};
}
if !status.contains(&Status::Processing) {
if let Some(ref batch) = processing.batch {
batches.remove(batch.uid);
}
}
batches &= status_batches;
}
if let Some(task_uids) = &uids {
let mut batches_by_task_uids = RoaringBitmap::new();
for task_uid in task_uids {
if let Some(task) = self.tasks.get_task(rtxn, *task_uid)? {
if let Some(batch_uid) = task.batch_uid {
batches_by_task_uids.insert(batch_uid);
}
}
}
batches &= batches_by_task_uids;
}
// There is no database for this query, we must retrieve the task queried by the client and ensure it's valid
if let Some(canceled_by) = &canceled_by {
let mut all_canceled_batches = RoaringBitmap::new();
for cancel_uid in canceled_by {
if let Some(task) = self.tasks.get_task(rtxn, *cancel_uid)? {
if task.kind.as_kind() == Kind::TaskCancelation
&& task.status == Status::Succeeded
{
if let Some(batch_uid) = task.batch_uid {
all_canceled_batches.insert(batch_uid);
}
}
}
}
// if the canceled_by has been specified but no batch
// matches then we prefer matching zero than all batches.
if all_canceled_batches.is_empty() {
return Ok(RoaringBitmap::new());
} else {
batches &= all_canceled_batches;
}
}
if let Some(kind) = &types {
let mut kind_batches = RoaringBitmap::new();
for kind in kind {
kind_batches |= self.batches.get_kind(rtxn, *kind)?;
if let Some(uid) = processing
.batch
.as_ref()
.and_then(|batch| batch.kinds.contains(kind).then_some(batch.uid))
{
kind_batches.insert(uid);
}
}
batches &= &kind_batches;
}
if let Some(index) = &index_uids {
let mut index_batches = RoaringBitmap::new();
for index in index {
index_batches |= self.batches.index_batches(rtxn, index)?;
if let Some(uid) = processing
.batch
.as_ref()
.and_then(|batch| batch.indexes.contains(index).then_some(batch.uid))
{
index_batches.insert(uid);
}
}
batches &= &index_batches;
}
// For the started_at filter, we need to treat the part of the batches that are processing from the part of the
// batches that are not processing. The non-processing ones are filtered normally while the processing ones
// are entirely removed unless the in-memory startedAt variable falls within the date filter.
// Once we have filtered the two subsets, we put them back together and assign it back to `batches`.
batches = {
let (mut filtered_non_processing_batches, mut filtered_processing_batches) =
(&batches - &*processing.processing, &batches & &*processing.processing);
// special case for Processing batches
// A closure that clears the filtered_processing_batches if their started_at date falls outside the given bounds
let mut clear_filtered_processing_batches =
|start: Bound<OffsetDateTime>, end: Bound<OffsetDateTime>| {
let start = map_bound(start, |b| b.unix_timestamp_nanos());
let end = map_bound(end, |b| b.unix_timestamp_nanos());
let is_within_dates = RangeBounds::contains(
&(start, end),
&processing
.batch
.as_ref()
.map_or_else(OffsetDateTime::now_utc, |batch| batch.started_at)
.unix_timestamp_nanos(),
);
if !is_within_dates {
filtered_processing_batches.clear();
}
};
match (after_started_at, before_started_at) {
(None, None) => (),
(None, Some(before)) => {
clear_filtered_processing_batches(Bound::Unbounded, Bound::Excluded(*before))
}
(Some(after), None) => {
clear_filtered_processing_batches(Bound::Excluded(*after), Bound::Unbounded)
}
(Some(after), Some(before)) => clear_filtered_processing_batches(
Bound::Excluded(*after),
Bound::Excluded(*before),
),
};
keep_ids_within_datetimes(
rtxn,
&mut filtered_non_processing_batches,
self.batches.started_at,
*after_started_at,
*before_started_at,
)?;
filtered_non_processing_batches | filtered_processing_batches
};
keep_ids_within_datetimes(
rtxn,
&mut batches,
self.batches.enqueued_at,
*after_enqueued_at,
*before_enqueued_at,
)?;
keep_ids_within_datetimes(
rtxn,
&mut batches,
self.batches.finished_at,
*after_finished_at,
*before_finished_at,
)?;
if let Some(limit) = limit {
batches = if query.reverse.unwrap_or_default() {
batches.into_iter().take(*limit as usize).collect()
} else {
batches.into_iter().rev().take(*limit as usize).collect()
};
}
Ok(batches)
}
/// Return the batch ids matching the query along with the total number of batches
/// by ignoring the from and limit parameters from the user's point of view.
///
/// There are two differences between an internal query and a query executed by
/// the user.
///
/// 1. IndexSwap tasks are not publicly associated with any index, but they are associated
/// with many indexes internally.
/// 2. The user may not have the rights to access the tasks (internally) associated with all indexes.
pub(crate) fn get_batch_ids_from_authorized_indexes(
&self,
rtxn: &RoTxn,
query: &Query,
filters: &meilisearch_auth::AuthFilter,
processing: &ProcessingTasks,
) -> Result<(RoaringBitmap, u64)> {
// compute all batches matching the filter by ignoring the limits, to find the number of batches matching
// the filter.
// As this causes us to compute the filter twice it is slightly inefficient, but doing it this way spares
// us from modifying the underlying implementation, and the performance remains sufficient.
// Should this change, we would modify `get_batch_ids` to directly return the number of matching batches.
let total_batches =
self.get_batch_ids(rtxn, &query.clone().without_limits(), processing)?;
let mut batches = self.get_batch_ids(rtxn, query, processing)?;
// If the query contains a list of index uid or there is a finite list of authorized indexes,
// then we must exclude all the batches that only contains tasks associated to multiple indexes.
// This works because we don't autobatch tasks associated to multiple indexes with tasks associated
// to a single index. e.g: IndexSwap cannot be batched with IndexCreation.
if query.index_uids.is_some() || !filters.all_indexes_authorized() {
for kind in enum_iterator::all::<Kind>().filter(|kind| !kind.related_to_one_index()) {
batches -= self.tasks.get_kind(rtxn, kind)?;
if let Some(batch) = processing.batch.as_ref() {
if batch.kinds.contains(&kind) {
batches.remove(batch.uid);
}
}
}
}
// Any batch that is internally associated with at least one authorized index
// must be returned.
if !filters.all_indexes_authorized() {
let mut valid_indexes = RoaringBitmap::new();
let mut forbidden_indexes = RoaringBitmap::new();
let all_indexes_iter = self.batches.index_tasks.iter(rtxn)?;
for result in all_indexes_iter {
let (index, index_tasks) = result?;
if filters.is_index_authorized(index) {
valid_indexes |= index_tasks;
} else {
forbidden_indexes |= index_tasks;
}
}
if let Some(batch) = processing.batch.as_ref() {
for index in &batch.indexes {
if filters.is_index_authorized(index) {
valid_indexes.insert(batch.uid);
} else {
forbidden_indexes.insert(batch.uid);
}
}
}
// If a batch had ONE valid task then it should be returned
let invalid_batches = forbidden_indexes - valid_indexes;
batches -= invalid_batches;
}
Ok((batches, total_batches.len()))
}
pub(crate) fn get_batches_from_authorized_indexes(
&self,
rtxn: &RoTxn,
query: &Query,
filters: &meilisearch_auth::AuthFilter,
processing: &ProcessingTasks,
) -> Result<(Vec<Batch>, u64)> {
let (batches, total) =
self.get_batch_ids_from_authorized_indexes(rtxn, query, filters, processing)?;
let batches = if query.reverse.unwrap_or_default() {
Box::new(batches.into_iter()) as Box<dyn Iterator<Item = u32>>
} else {
Box::new(batches.into_iter().rev()) as Box<dyn Iterator<Item = u32>>
};
let batches = self.batches.get_existing_batches(
rtxn,
batches.take(query.limit.unwrap_or(u32::MAX) as usize),
processing,
)?;
Ok((batches, total))
}
}

View file

@ -0,0 +1,473 @@
use meili_snap::snapshot;
use meilisearch_auth::AuthFilter;
use meilisearch_types::index_uid_pattern::IndexUidPattern;
use meilisearch_types::tasks::{IndexSwap, KindWithContent, Status};
use time::{Duration, OffsetDateTime};
use crate::insta_snapshot::{snapshot_bitmap, snapshot_index_scheduler};
use crate::test_utils::Breakpoint::*;
use crate::test_utils::{index_creation_task, FailureLocation};
use crate::{IndexScheduler, Query};
#[test]
fn query_batches_from_and_limit() {
let (index_scheduler, mut handle) = IndexScheduler::test(true, vec![]);
let kind = index_creation_task("doggo", "bone");
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_first_task");
let kind = index_creation_task("whalo", "plankton");
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_second_task");
let kind = index_creation_task("catto", "his_own_vomit");
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_third_task");
handle.advance_n_successful_batches(3);
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "processed_all_tasks");
let proc = index_scheduler.processing_tasks.read().unwrap().clone();
let rtxn = index_scheduler.env.read_txn().unwrap();
let query = Query { limit: Some(0), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[]");
let query = Query { limit: Some(1), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[2,]");
let query = Query { limit: Some(2), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[1,2,]");
let query = Query { from: Some(1), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[0,1,]");
let query = Query { from: Some(2), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[0,1,2,]");
let query = Query { from: Some(1), limit: Some(1), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[1,]");
let query = Query { from: Some(1), limit: Some(2), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[0,1,]");
}
#[test]
fn query_batches_simple() {
let start_time = OffsetDateTime::now_utc();
let (index_scheduler, mut handle) =
IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]);
let kind = index_creation_task("catto", "mouse");
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = index_creation_task("doggo", "sheep");
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = index_creation_task("whalo", "fish");
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start");
handle.advance_till([Start, BatchCreated]);
let query = Query { statuses: Some(vec![Status::Processing]), ..Default::default() };
let (mut batches, _) = index_scheduler
.get_batches_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
assert_eq!(batches.len(), 1);
batches[0].started_at = OffsetDateTime::UNIX_EPOCH;
// Insta cannot snapshot our batches because the batch stats contains an enum as key: https://github.com/mitsuhiko/insta/issues/689
let batch = serde_json::to_string_pretty(&batches[0]).unwrap();
snapshot!(batch, @r#"
{
"uid": 0,
"details": {
"primaryKey": "mouse"
},
"stats": {
"totalNbTasks": 1,
"status": {
"processing": 1
},
"types": {
"indexCreation": 1
},
"indexUids": {
"catto": 1
}
},
"startedAt": "1970-01-01T00:00:00Z",
"finishedAt": null
}
"#);
let query = Query { statuses: Some(vec![Status::Enqueued]), ..Default::default() };
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[]"); // The batches don't contains any enqueued tasks
let query =
Query { statuses: Some(vec![Status::Enqueued, Status::Processing]), ..Default::default() };
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
snapshot!(snapshot_bitmap(&batches), @"[0,]"); // both enqueued and processing tasks in the first tick
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Processing]),
after_started_at: Some(start_time),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both enqueued and processing tasks in the first tick, but limited to those with a started_at
// that comes after the start of the test, which should excludes the enqueued tasks
snapshot!(snapshot_bitmap(&batches), @"[0,]");
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Processing]),
before_started_at: Some(start_time),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both enqueued and processing tasks in the first tick, but limited to those with a started_at
// that comes before the start of the test, which should excludes all of them
snapshot!(snapshot_bitmap(&batches), @"[]");
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Processing]),
after_started_at: Some(start_time),
before_started_at: Some(start_time + Duration::minutes(1)),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both enqueued and processing tasks in the first tick, but limited to those with a started_at
// that comes after the start of the test and before one minute after the start of the test,
// which should exclude the enqueued tasks and include the only processing task
snapshot!(snapshot_bitmap(&batches), @"[0,]");
handle.advance_till([
InsideProcessBatch,
InsideProcessBatch,
ProcessBatchSucceeded,
AfterProcessing,
Start,
BatchCreated,
]);
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "after-advancing-a-bit");
let second_start_time = OffsetDateTime::now_utc();
let query = Query {
statuses: Some(vec![Status::Succeeded, Status::Processing]),
after_started_at: Some(start_time),
before_started_at: Some(start_time + Duration::minutes(1)),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both succeeded and processing tasks in the first tick, but limited to those with a started_at
// that comes after the start of the test and before one minute after the start of the test,
// which should include all tasks
snapshot!(snapshot_bitmap(&batches), @"[0,1,]");
let query = Query {
statuses: Some(vec![Status::Succeeded, Status::Processing]),
before_started_at: Some(start_time),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both succeeded and processing tasks in the first tick, but limited to those with a started_at
// that comes before the start of the test, which should exclude all tasks
snapshot!(snapshot_bitmap(&batches), @"[]");
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Succeeded, Status::Processing]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both succeeded and processing tasks in the first tick, but limited to those with a started_at
// that comes after the start of the second part of the test and before one minute after the
// second start of the test, which should exclude all tasks
snapshot!(snapshot_bitmap(&batches), @"[]");
// now we make one more batch, the started_at field of the new tasks will be past `second_start_time`
handle.advance_till([
InsideProcessBatch,
InsideProcessBatch,
ProcessBatchSucceeded,
AfterProcessing,
Start,
BatchCreated,
]);
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// we run the same query to verify that, and indeed find that the last task is matched
snapshot!(snapshot_bitmap(&batches), @"[2,]");
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Succeeded, Status::Processing]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// enqueued, succeeded, or processing tasks started after the second part of the test, should
// again only return the last task
snapshot!(snapshot_bitmap(&batches), @"[2,]");
handle.advance_till([ProcessBatchFailed, AfterProcessing]);
// now the last task should have failed
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "end");
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// so running the last query should return nothing
snapshot!(snapshot_bitmap(&batches), @"[]");
let query = Query {
statuses: Some(vec![Status::Failed]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// but the same query on failed tasks should return the last task
snapshot!(snapshot_bitmap(&batches), @"[2,]");
let query = Query {
statuses: Some(vec![Status::Failed]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// but the same query on failed tasks should return the last task
snapshot!(snapshot_bitmap(&batches), @"[2,]");
let query = Query {
statuses: Some(vec![Status::Failed]),
uids: Some(vec![1]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// same query but with an invalid uid
snapshot!(snapshot_bitmap(&batches), @"[]");
let query = Query {
statuses: Some(vec![Status::Failed]),
uids: Some(vec![2]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// same query but with a valid uid
snapshot!(snapshot_bitmap(&batches), @"[2,]");
}
#[test]
fn query_batches_special_rules() {
let (index_scheduler, mut handle) =
IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]);
let kind = index_creation_task("catto", "mouse");
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = index_creation_task("doggo", "sheep");
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = KindWithContent::IndexSwap {
swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "doggo".to_owned()) }],
};
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = KindWithContent::IndexSwap {
swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "whalo".to_owned()) }],
};
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start");
handle.advance_till([Start, BatchCreated]);
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap().clone();
let query = Query { index_uids: Some(vec!["catto".to_owned()]), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
// only the first task associated with catto is returned, the indexSwap tasks are excluded!
snapshot!(snapshot_bitmap(&batches), @"[0,]");
let query = Query { index_uids: Some(vec!["catto".to_owned()]), ..Default::default() };
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(
&rtxn,
&query,
&AuthFilter::with_allowed_indexes(
vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(),
),
&proc,
)
.unwrap();
// we have asked for only the tasks associated with catto, but are only authorized to retrieve the tasks
// associated with doggo -> empty result
snapshot!(snapshot_bitmap(&batches), @"[]");
drop(rtxn);
// We're going to advance and process all the batches for the next query to actually hit the db
handle.advance_till([
InsideProcessBatch,
InsideProcessBatch,
ProcessBatchSucceeded,
AfterProcessing,
]);
handle.advance_one_successful_batch();
handle.advance_n_failed_batches(2);
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "after-processing-everything");
let rtxn = index_scheduler.env.read_txn().unwrap();
let query = Query::default();
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(
&rtxn,
&query,
&AuthFilter::with_allowed_indexes(
vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(),
),
&proc,
)
.unwrap();
// we asked for all the tasks, but we are only authorized to retrieve the doggo tasks
// -> only the index creation of doggo should be returned
snapshot!(snapshot_bitmap(&batches), @"[1,]");
let query = Query::default();
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(
&rtxn,
&query,
&AuthFilter::with_allowed_indexes(
vec![
IndexUidPattern::new_unchecked("catto"),
IndexUidPattern::new_unchecked("doggo"),
]
.into_iter()
.collect(),
),
&proc,
)
.unwrap();
// we asked for all the tasks, but we are only authorized to retrieve the doggo and catto tasks
// -> all tasks except the swap of catto with whalo are returned
snapshot!(snapshot_bitmap(&batches), @"[0,1,]");
let query = Query::default();
let (batches, _) = index_scheduler
.queue
.get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
// we asked for all the tasks with all index authorized -> all tasks returned
snapshot!(snapshot_bitmap(&batches), @"[0,1,2,3,]");
}
#[test]
fn query_batches_canceled_by() {
let (index_scheduler, mut handle) =
IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]);
let kind = index_creation_task("catto", "mouse");
let _ = index_scheduler.register(kind, None, false).unwrap();
let kind = index_creation_task("doggo", "sheep");
let _ = index_scheduler.register(kind, None, false).unwrap();
let kind = KindWithContent::IndexSwap {
swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "doggo".to_owned()) }],
};
let _task = index_scheduler.register(kind, None, false).unwrap();
handle.advance_n_successful_batches(1);
let kind = KindWithContent::TaskCancelation {
query: "test_query".to_string(),
tasks: [0, 1, 2, 3].into_iter().collect(),
};
let task_cancelation = index_scheduler.register(kind, None, false).unwrap();
handle.advance_n_successful_batches(1);
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start");
let query = Query { canceled_by: Some(vec![task_cancelation.uid]), ..Query::default() };
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// The batch zero was the index creation task, the 1 is the task cancellation
snapshot!(snapshot_bitmap(&batches), @"[1,]");
let query = Query { canceled_by: Some(vec![task_cancelation.uid]), ..Query::default() };
let (batches, _) = index_scheduler
.get_batch_ids_from_authorized_indexes(
&query,
&AuthFilter::with_allowed_indexes(
vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(),
),
)
.unwrap();
// Return only 1 because the user is not authorized to see task 2
snapshot!(snapshot_bitmap(&batches), @"[1,]");
}

View file

@ -0,0 +1,379 @@
mod batches;
#[cfg(test)]
mod batches_test;
mod tasks;
#[cfg(test)]
mod tasks_test;
#[cfg(test)]
mod test;
use std::collections::BTreeMap;
use std::time::Duration;
use file_store::FileStore;
use meilisearch_types::batches::BatchId;
use meilisearch_types::heed::{Database, Env, RoTxn, RwTxn};
use meilisearch_types::milli::{CboRoaringBitmapCodec, BEU32};
use meilisearch_types::tasks::{Kind, KindWithContent, Status, Task};
use roaring::RoaringBitmap;
use time::format_description::well_known::Rfc3339;
use time::OffsetDateTime;
use uuid::Uuid;
use self::batches::BatchQueue;
use self::tasks::TaskQueue;
use crate::processing::ProcessingTasks;
use crate::utils::{
check_index_swap_validity, filter_out_references_to_newer_tasks, ProcessingBatch,
};
use crate::{Error, IndexSchedulerOptions, Result, TaskId};
/// Database const names for the `IndexScheduler`.
mod db_name {
pub const BATCH_TO_TASKS_MAPPING: &str = "batch-to-tasks-mapping";
}
/// Defines a subset of tasks to be retrieved from the [`IndexScheduler`].
///
/// An empty/default query (where each field is set to `None`) matches all tasks.
/// Each non-null field restricts the set of tasks further.
#[derive(Default, Debug, Clone, PartialEq, Eq)]
pub struct Query {
/// The maximum number of tasks to be matched
pub limit: Option<u32>,
/// The minimum [task id](`meilisearch_types::tasks::Task::uid`) to be matched
pub from: Option<u32>,
/// The order used to return the tasks. By default the newest tasks are returned first and the boolean is `false`.
pub reverse: Option<bool>,
/// The [task ids](`meilisearch_types::tasks::Task::uid`) to be matched
pub uids: Option<Vec<TaskId>>,
/// The [batch ids](`meilisearch_types::batches::Batch::uid`) to be matched
pub batch_uids: Option<Vec<BatchId>>,
/// The allowed [statuses](`meilisearch_types::tasks::Task::status`) of the matched tasls
pub statuses: Option<Vec<Status>>,
/// The allowed [kinds](meilisearch_types::tasks::Kind) of the matched tasks.
///
/// The kind of a task is given by:
/// ```
/// # use meilisearch_types::tasks::{Task, Kind};
/// # fn doc_func(task: Task) -> Kind {
/// task.kind.as_kind()
/// # }
/// ```
pub types: Option<Vec<Kind>>,
/// The allowed [index ids](meilisearch_types::tasks::Task::index_uid) of the matched tasks
pub index_uids: Option<Vec<String>>,
/// The [task ids](`meilisearch_types::tasks::Task::uid`) of the [`TaskCancelation`](meilisearch_types::tasks::Task::Kind::TaskCancelation) tasks
/// that canceled the matched tasks.
pub canceled_by: Option<Vec<TaskId>>,
/// Exclusive upper bound of the matched tasks' [`enqueued_at`](meilisearch_types::tasks::Task::enqueued_at) field.
pub before_enqueued_at: Option<OffsetDateTime>,
/// Exclusive lower bound of the matched tasks' [`enqueued_at`](meilisearch_types::tasks::Task::enqueued_at) field.
pub after_enqueued_at: Option<OffsetDateTime>,
/// Exclusive upper bound of the matched tasks' [`started_at`](meilisearch_types::tasks::Task::started_at) field.
pub before_started_at: Option<OffsetDateTime>,
/// Exclusive lower bound of the matched tasks' [`started_at`](meilisearch_types::tasks::Task::started_at) field.
pub after_started_at: Option<OffsetDateTime>,
/// Exclusive upper bound of the matched tasks' [`finished_at`](meilisearch_types::tasks::Task::finished_at) field.
pub before_finished_at: Option<OffsetDateTime>,
/// Exclusive lower bound of the matched tasks' [`finished_at`](meilisearch_types::tasks::Task::finished_at) field.
pub after_finished_at: Option<OffsetDateTime>,
}
impl Query {
/// Return `true` if every field of the query is set to `None`, such that the query
/// matches all tasks.
pub fn is_empty(&self) -> bool {
matches!(
self,
Query {
limit: None,
from: None,
reverse: None,
uids: None,
batch_uids: None,
statuses: None,
types: None,
index_uids: None,
canceled_by: None,
before_enqueued_at: None,
after_enqueued_at: None,
before_started_at: None,
after_started_at: None,
before_finished_at: None,
after_finished_at: None,
}
)
}
/// Add an [index id](meilisearch_types::tasks::Task::index_uid) to the list of permitted indexes.
pub fn with_index(self, index_uid: String) -> Self {
let mut index_vec = self.index_uids.unwrap_or_default();
index_vec.push(index_uid);
Self { index_uids: Some(index_vec), ..self }
}
// Removes the `from` and `limit` restrictions from the query.
// Useful to get the total number of tasks matching a filter.
pub fn without_limits(self) -> Self {
Query { limit: None, from: None, ..self }
}
}
/// Structure which holds meilisearch's indexes and schedules the tasks
/// to be performed on them.
pub struct Queue {
pub(crate) tasks: tasks::TaskQueue,
pub(crate) batches: batches::BatchQueue,
/// Matches a batch id with the associated task ids.
pub(crate) batch_to_tasks_mapping: Database<BEU32, CboRoaringBitmapCodec>,
/// The list of files referenced by the tasks.
pub(crate) file_store: FileStore,
/// The max number of tasks allowed before the scheduler starts to delete
/// the finished tasks automatically.
pub(crate) max_number_of_tasks: usize,
}
impl Queue {
pub(crate) fn private_clone(&self) -> Queue {
Queue {
tasks: self.tasks.private_clone(),
batches: self.batches.private_clone(),
batch_to_tasks_mapping: self.batch_to_tasks_mapping,
file_store: self.file_store.clone(),
max_number_of_tasks: self.max_number_of_tasks,
}
}
/// Create an index scheduler and start its run loop.
pub(crate) fn new(
env: &Env,
wtxn: &mut RwTxn,
options: &IndexSchedulerOptions,
) -> Result<Self> {
// allow unreachable_code to get rids of the warning in the case of a test build.
Ok(Self {
file_store: FileStore::new(&options.update_file_path)?,
batch_to_tasks_mapping: env
.create_database(wtxn, Some(db_name::BATCH_TO_TASKS_MAPPING))?,
tasks: TaskQueue::new(env, wtxn)?,
batches: BatchQueue::new(env, wtxn)?,
max_number_of_tasks: options.max_number_of_tasks,
})
}
/// Returns the whole set of tasks that belongs to this batch.
pub(crate) fn tasks_in_batch(&self, rtxn: &RoTxn, batch_id: BatchId) -> Result<RoaringBitmap> {
Ok(self.batch_to_tasks_mapping.get(rtxn, &batch_id)?.unwrap_or_default())
}
/// Convert an iterator to a `Vec` of tasks and edit the `ProcessingBatch` to add the given tasks.
///
/// The tasks MUST exist, or a `CorruptedTaskQueue` error will be thrown.
pub(crate) fn get_existing_tasks_for_processing_batch(
&self,
rtxn: &RoTxn,
processing_batch: &mut ProcessingBatch,
tasks: impl IntoIterator<Item = TaskId>,
) -> Result<Vec<Task>> {
tasks
.into_iter()
.map(|task_id| {
let mut task = self
.tasks
.get_task(rtxn, task_id)
.and_then(|task| task.ok_or(Error::CorruptedTaskQueue));
processing_batch.processing(&mut task);
task
})
.collect::<Result<_>>()
}
pub(crate) fn write_batch(
&self,
wtxn: &mut RwTxn,
batch: ProcessingBatch,
tasks: &RoaringBitmap,
) -> Result<()> {
self.batch_to_tasks_mapping.put(wtxn, &batch.uid, tasks)?;
self.batches.write_batch(wtxn, batch)?;
Ok(())
}
pub(crate) fn delete_persisted_task_data(&self, task: &Task) -> Result<()> {
match task.content_uuid() {
Some(content_file) => self.delete_update_file(content_file),
None => Ok(()),
}
}
/// Delete a file from the index scheduler.
///
/// Counterpart to the [`create_update_file`](IndexScheduler::create_update_file) method.
pub fn delete_update_file(&self, uuid: Uuid) -> Result<()> {
Ok(self.file_store.delete(uuid)?)
}
/// Create a file and register it in the index scheduler.
///
/// The returned file and uuid can be used to associate
/// some data to a task. The file will be kept until
/// the task has been fully processed.
pub fn create_update_file(&self, dry_run: bool) -> Result<(Uuid, file_store::File)> {
if dry_run {
Ok((Uuid::nil(), file_store::File::dry_file()?))
} else {
Ok(self.file_store.new_update()?)
}
}
#[cfg(test)]
pub fn create_update_file_with_uuid(&self, uuid: u128) -> Result<(Uuid, file_store::File)> {
Ok(self.file_store.new_update_with_uuid(uuid)?)
}
/// The size on disk taken by all the updates files contained in the `IndexScheduler`, in bytes.
pub fn compute_update_file_size(&self) -> Result<u64> {
Ok(self.file_store.compute_total_size()?)
}
pub fn register(
&self,
wtxn: &mut RwTxn,
kind: &KindWithContent,
task_id: Option<TaskId>,
dry_run: bool,
) -> Result<Task> {
let next_task_id = self.tasks.next_task_id(wtxn)?;
if let Some(uid) = task_id {
if uid < next_task_id {
return Err(Error::BadTaskId { received: uid, expected: next_task_id });
}
}
let mut task = Task {
uid: task_id.unwrap_or(next_task_id),
// The batch is defined once we starts processing the task
batch_uid: None,
enqueued_at: OffsetDateTime::now_utc(),
started_at: None,
finished_at: None,
error: None,
canceled_by: None,
details: kind.default_details(),
status: Status::Enqueued,
kind: kind.clone(),
};
// For deletion and cancelation tasks, we want to make extra sure that they
// don't attempt to delete/cancel tasks that are newer than themselves.
filter_out_references_to_newer_tasks(&mut task);
// If the register task is an index swap task, verify that it is well-formed
// (that it does not contain duplicate indexes).
check_index_swap_validity(&task)?;
// At this point the task is going to be registered and no further checks will be done
if dry_run {
return Ok(task);
}
// Get rid of the mutability.
let task = task;
self.tasks.register(wtxn, &task)?;
Ok(task)
}
/// Register a task to cleanup the task queue if needed
pub fn cleanup_task_queue(&self, wtxn: &mut RwTxn) -> Result<()> {
let nb_tasks = self.tasks.all_task_ids(wtxn)?.len();
// if we have less than 1M tasks everything is fine
if nb_tasks < self.max_number_of_tasks as u64 {
return Ok(());
}
let finished = self.tasks.status.get(wtxn, &Status::Succeeded)?.unwrap_or_default()
| self.tasks.status.get(wtxn, &Status::Failed)?.unwrap_or_default()
| self.tasks.status.get(wtxn, &Status::Canceled)?.unwrap_or_default();
let to_delete = RoaringBitmap::from_iter(finished.into_iter().rev().take(100_000));
// /!\ the len must be at least 2 or else we might enter an infinite loop where we only delete
// the deletion tasks we enqueued ourselves.
if to_delete.len() < 2 {
tracing::warn!("The task queue is almost full, but no task can be deleted yet.");
// the only thing we can do is hope that the user tasks are going to finish
return Ok(());
}
tracing::info!(
"The task queue is almost full. Deleting the oldest {} finished tasks.",
to_delete.len()
);
// it's safe to unwrap here because we checked the len above
let newest_task_id = to_delete.iter().last().unwrap();
let last_task_to_delete =
self.tasks.get_task(wtxn, newest_task_id)?.ok_or(Error::CorruptedTaskQueue)?;
// increase time by one nanosecond so that the enqueuedAt of the last task to delete is also lower than that date.
let delete_before = last_task_to_delete.enqueued_at + Duration::from_nanos(1);
self.register(
wtxn,
&KindWithContent::TaskDeletion {
query: format!(
"?beforeEnqueuedAt={}&statuses=succeeded,failed,canceled",
delete_before.format(&Rfc3339).map_err(|_| Error::CorruptedTaskQueue)?,
),
tasks: to_delete,
},
None,
false,
)?;
Ok(())
}
pub fn get_stats(
&self,
rtxn: &RoTxn,
processing: &ProcessingTasks,
) -> Result<BTreeMap<String, BTreeMap<String, u64>>> {
let mut res = BTreeMap::new();
let processing_tasks = processing.processing.len();
res.insert(
"statuses".to_string(),
enum_iterator::all::<Status>()
.map(|s| {
let tasks = self.tasks.get_status(rtxn, s)?.len();
match s {
Status::Enqueued => Ok((s.to_string(), tasks - processing_tasks)),
Status::Processing => Ok((s.to_string(), processing_tasks)),
s => Ok((s.to_string(), tasks)),
}
})
.collect::<Result<BTreeMap<String, u64>>>()?,
);
res.insert(
"types".to_string(),
enum_iterator::all::<Kind>()
.map(|s| Ok((s.to_string(), self.tasks.get_kind(rtxn, s)?.len())))
.collect::<Result<BTreeMap<String, u64>>>()?,
);
res.insert(
"indexes".to_string(),
self.tasks
.index_tasks
.iter(rtxn)?
.map(|res| Ok(res.map(|(name, bitmap)| (name.to_string(), bitmap.len()))?))
.collect::<Result<BTreeMap<String, u64>>>()?,
);
Ok(res)
}
}

View file

@ -0,0 +1,87 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, batch_uid: 1, status: canceled, canceled_by: 3, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, batch_uid: 1, status: canceled, canceled_by: 3, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }}
3 {uid: 3, batch_uid: 1, status: succeeded, details: { matched_tasks: 3, canceled_tasks: Some(2), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0, 1, 2]> }}
----------------------------------------------------------------------
### Status:
enqueued []
succeeded [0,3,]
canceled [1,2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,]
"indexSwap" [2,]
"taskCancelation" [3,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,2,]
doggo [1,2,]
----------------------------------------------------------------------
### Index Mapper:
catto: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Canceled By:
3 [1,2,]
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,2,3,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,2,3,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, details: {"primaryKey":"mouse"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, }
1 {uid: 1, details: {"primaryKey":"sheep","matchedTasks":3,"canceledTasks":2,"originalFilter":"test_query","swaps":[{"indexes":["catto","doggo"]}]}, stats: {"totalNbTasks":3,"status":{"succeeded":1,"canceled":2},"types":{"indexCreation":1,"indexSwap":1,"taskCancelation":1},"indexUids":{"doggo":1}}, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [1,2,3,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,]
canceled [1,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,1,]
"indexSwap" [1,]
"taskCancelation" [1,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,1,]
doggo [1,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,88 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }}
2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("his_own_vomit") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("his_own_vomit") }}
----------------------------------------------------------------------
### Status:
enqueued []
succeeded [0,1,2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [2,]
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Index Mapper:
catto: { number_of_documents: 0, field_distribution: {} }
doggo: { number_of_documents: 0, field_distribution: {} }
whalo: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, details: {"primaryKey":"bone"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, }
1 {uid: 1, details: {"primaryKey":"plankton"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"whalo":1}}, }
2 {uid: 2, details: {"primaryKey":"his_own_vomit"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [1,]
2 [2,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,2,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [2,]
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,52 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
----------------------------------------------------------------------
### Status:
enqueued [0,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,]
----------------------------------------------------------------------
### Index Tasks:
doggo [0,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,55 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,]
----------------------------------------------------------------------
### Index Tasks:
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,58 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }}
2 {uid: 2, status: enqueued, details: { primary_key: Some("his_own_vomit") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("his_own_vomit") }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [2,]
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,71 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(1):
[1,]
{uid: 1, details: {"primaryKey":"sheep"}, stats: {"totalNbTasks":1,"status":{"processing":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, }
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, status: enqueued, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }}
----------------------------------------------------------------------
### Status:
enqueued [1,2,]
succeeded [0,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Index Mapper:
catto: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, details: {"primaryKey":"mouse"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,89 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }}
----------------------------------------------------------------------
### Status:
enqueued []
succeeded [0,1,]
failed [2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Index Mapper:
catto: { number_of_documents: 0, field_distribution: {} }
doggo: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, details: {"primaryKey":"mouse"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, }
1 {uid: 1, details: {"primaryKey":"sheep"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, }
2 {uid: 2, details: {"primaryKey":"fish"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"indexCreation":1},"indexUids":{"whalo":1}}, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [1,]
2 [2,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,]
failed [2,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,58 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, status: enqueued, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,100 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }}
3 {uid: 3, batch_uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `whalo` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }}
----------------------------------------------------------------------
### Status:
enqueued []
succeeded [0,1,]
failed [2,3,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,]
"indexSwap" [2,3,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,2,3,]
doggo [1,2,]
whalo [3,]
----------------------------------------------------------------------
### Index Mapper:
catto: { number_of_documents: 0, field_distribution: {} }
doggo: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, details: {"primaryKey":"mouse"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, }
1 {uid: 1, details: {"primaryKey":"sheep"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, }
2 {uid: 2, details: {"swaps":[{"indexes":["catto","doggo"]}]}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"indexSwap":1},"indexUids":{}}, }
3 {uid: 3, details: {"swaps":[{"indexes":["catto","whalo"]}]}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"indexSwap":1},"indexUids":{}}, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [1,]
2 [2,]
3 [3,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,]
failed [2,3,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,1,]
"indexSwap" [2,3,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,2,3,]
doggo [1,2,]
whalo [3,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,61 @@
---
source: crates/index-scheduler/src/queue/batches_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, status: enqueued, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }}
3 {uid: 3, status: enqueued, details: { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,2,3,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,]
"indexSwap" [2,3,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,2,3,]
doggo [1,2,]
whalo [3,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,87 @@
---
source: crates/index-scheduler/src/queue/tasks_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, batch_uid: 1, status: canceled, canceled_by: 3, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, batch_uid: 1, status: canceled, canceled_by: 3, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }}
3 {uid: 3, batch_uid: 1, status: succeeded, details: { matched_tasks: 3, canceled_tasks: Some(2), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0, 1, 2]> }}
----------------------------------------------------------------------
### Status:
enqueued []
succeeded [0,3,]
canceled [1,2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,]
"indexSwap" [2,]
"taskCancelation" [3,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,2,]
doggo [1,2,]
----------------------------------------------------------------------
### Index Mapper:
catto: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Canceled By:
3 [1,2,]
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,2,3,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,2,3,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, details: {"primaryKey":"mouse"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, }
1 {uid: 1, details: {"primaryKey":"sheep","matchedTasks":3,"canceledTasks":2,"originalFilter":"test_query","swaps":[{"indexes":["catto","doggo"]}]}, stats: {"totalNbTasks":3,"status":{"succeeded":1,"canceled":2},"types":{"indexCreation":1,"indexSwap":1,"taskCancelation":1},"indexUids":{"doggo":1}}, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [1,2,3,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,]
canceled [1,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,1,]
"indexSwap" [1,]
"taskCancelation" [1,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,1,]
doggo [1,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,88 @@
---
source: crates/index-scheduler/src/queue/tasks_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }}
2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("his_own_vomit") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("his_own_vomit") }}
----------------------------------------------------------------------
### Status:
enqueued []
succeeded [0,1,2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [2,]
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Index Mapper:
catto: { number_of_documents: 0, field_distribution: {} }
doggo: { number_of_documents: 0, field_distribution: {} }
whalo: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, details: {"primaryKey":"bone"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, }
1 {uid: 1, details: {"primaryKey":"plankton"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"whalo":1}}, }
2 {uid: 2, details: {"primaryKey":"his_own_vomit"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [1,]
2 [2,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,2,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [2,]
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,52 @@
---
source: crates/index-scheduler/src/queue/tasks_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
----------------------------------------------------------------------
### Status:
enqueued [0,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,]
----------------------------------------------------------------------
### Index Tasks:
doggo [0,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,55 @@
---
source: crates/index-scheduler/src/queue/tasks_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,]
----------------------------------------------------------------------
### Index Tasks:
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,58 @@
---
source: crates/index-scheduler/src/queue/tasks_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }}
2 {uid: 2, status: enqueued, details: { primary_key: Some("his_own_vomit") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("his_own_vomit") }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [2,]
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,89 @@
---
source: crates/index-scheduler/src/queue/tasks_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }}
----------------------------------------------------------------------
### Status:
enqueued []
succeeded [0,1,]
failed [2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Index Mapper:
catto: { number_of_documents: 0, field_distribution: {} }
doggo: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, details: {"primaryKey":"mouse"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, }
1 {uid: 1, details: {"primaryKey":"sheep"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, }
2 {uid: 2, details: {"primaryKey":"fish"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"indexCreation":1},"indexUids":{"whalo":1}}, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [1,]
2 [2,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,]
failed [2,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,58 @@
---
source: crates/index-scheduler/src/queue/tasks_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, status: enqueued, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,2,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,2,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,61 @@
---
source: crates/index-scheduler/src/queue/tasks_test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, status: enqueued, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, status: enqueued, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }}
3 {uid: 3, status: enqueued, details: { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,2,3,]
----------------------------------------------------------------------
### Kind:
"indexCreation" [0,1,]
"indexSwap" [2,3,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,2,3,]
doggo [1,2,]
whalo [3,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View file

@ -0,0 +1,63 @@
---
source: crates/index-scheduler/src/queue/test.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, status: enqueued, details: { received_documents: 12, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 12, allow_index_creation: true }}
2 {uid: 2, status: enqueued, details: { received_documents: 50, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 50, allow_index_creation: true }}
3 {uid: 3, status: enqueued, details: { received_documents: 5000, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggo", primary_key: Some("bone"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 5000, allow_index_creation: true }}
----------------------------------------------------------------------
### Status:
enqueued [0,1,2,3,]
----------------------------------------------------------------------
### Kind:
"documentAdditionOrUpdate" [1,2,3,]
"indexCreation" [0,]
----------------------------------------------------------------------
### Index Tasks:
catto [0,1,2,]
doggo [3,]
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
00000000-0000-0000-0000-000000000000
00000000-0000-0000-0000-000000000001
00000000-0000-0000-0000-000000000002
----------------------------------------------------------------------

View file

@ -0,0 +1,71 @@
---
source: crates/index-scheduler/src/queue/test.rs
snapshot_kind: text
---
[
{
"uid": 3,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 5,
"batchUid": 4,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"TaskDeletion": {
"matched_tasks": 2,
"deleted_tasks": 2,
"original_filter": "[filter]"
}
},
"status": "succeeded",
"kind": {
"taskDeletion": {
"query": "[query]",
"tasks": [
58,
48,
0,
0,
1,
0,
0,
0,
0,
0,
1,
0,
16,
0,
0,
0,
2,
0,
4,
0
]
}
}
}
]

View file

@ -0,0 +1,50 @@
---
source: crates/index-scheduler/src/queue/test.rs
snapshot_kind: text
---
[
{
"uid": 6,
"batchUid": 6,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"TaskDeletion": {
"matched_tasks": 2,
"deleted_tasks": 2,
"original_filter": "[filter]"
}
},
"status": "succeeded",
"kind": {
"taskDeletion": {
"query": "[query]",
"tasks": [
58,
48,
0,
0,
1,
0,
0,
0,
0,
0,
1,
0,
16,
0,
0,
0,
3,
0,
5,
0
]
}
}
}
]

View file

@ -0,0 +1,139 @@
---
source: crates/index-scheduler/src/queue/test.rs
snapshot_kind: text
---
[
{
"uid": 0,
"batchUid": 0,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "succeeded",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 1,
"batchUid": 1,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": {
"message": "Index `doggo` already exists.",
"code": "index_already_exists",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#index_already_exists"
},
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "failed",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 2,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 3,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 4,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"TaskDeletion": {
"matched_tasks": 2,
"deleted_tasks": null,
"original_filter": "[filter]"
}
},
"status": "enqueued",
"kind": {
"taskDeletion": {
"query": "[query]",
"tasks": [
58,
48,
0,
0,
1,
0,
0,
0,
0,
0,
1,
0,
16,
0,
0,
0,
0,
0,
1,
0
]
}
}
}
]

View file

@ -0,0 +1,92 @@
---
source: crates/index-scheduler/src/queue/test.rs
snapshot_kind: text
---
[
{
"uid": 2,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 3,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 4,
"batchUid": 2,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"TaskDeletion": {
"matched_tasks": 2,
"deleted_tasks": 2,
"original_filter": "[filter]"
}
},
"status": "succeeded",
"kind": {
"taskDeletion": {
"query": "[query]",
"tasks": [
58,
48,
0,
0,
1,
0,
0,
0,
0,
0,
1,
0,
16,
0,
0,
0,
0,
0,
1,
0
]
}
}
}
]

View file

@ -0,0 +1,95 @@
---
source: crates/index-scheduler/src/queue/test.rs
snapshot_kind: text
---
[
{
"uid": 0,
"batchUid": 0,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "succeeded",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 1,
"batchUid": 1,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": {
"message": "Index `doggo` already exists.",
"code": "index_already_exists",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#index_already_exists"
},
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "failed",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 2,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 3,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
}
]

View file

@ -0,0 +1,95 @@
---
source: crates/index-scheduler/src/queue/test.rs
snapshot_kind: text
---
[
{
"uid": 0,
"batchUid": 0,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "succeeded",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 1,
"batchUid": 1,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": {
"message": "Index `doggo` already exists.",
"code": "index_already_exists",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#index_already_exists"
},
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "failed",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 2,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 3,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
}
]

View file

@ -0,0 +1,95 @@
---
source: crates/index-scheduler/src/queue/test.rs
snapshot_kind: text
---
[
{
"uid": 0,
"batchUid": 0,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "succeeded",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 1,
"batchUid": 1,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": {
"message": "Index `doggo` already exists.",
"code": "index_already_exists",
"type": "invalid_request",
"link": "https://docs.meilisearch.com/errors#index_already_exists"
},
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "failed",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 2,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
},
{
"uid": 3,
"batchUid": null,
"enqueuedAt": "[date]",
"startedAt": "[date]",
"finishedAt": "[date]",
"error": null,
"canceledBy": null,
"details": {
"IndexInfo": {
"primary_key": null
}
},
"status": "enqueued",
"kind": {
"indexCreation": {
"index_uid": "doggo",
"primary_key": null
}
}
}
]

View file

@ -0,0 +1,514 @@
use std::ops::{Bound, RangeBounds};
use meilisearch_types::heed::types::{DecodeIgnore, SerdeBincode, SerdeJson, Str};
use meilisearch_types::heed::{Database, Env, RoTxn, RwTxn};
use meilisearch_types::milli::{CboRoaringBitmapCodec, RoaringBitmapCodec, BEU32};
use meilisearch_types::tasks::{Kind, Status, Task};
use roaring::{MultiOps, RoaringBitmap};
use time::OffsetDateTime;
use super::{Query, Queue};
use crate::processing::ProcessingTasks;
use crate::utils::{self, insert_task_datetime, keep_ids_within_datetimes, map_bound};
use crate::{Error, Result, TaskId, BEI128};
/// Database const names for the `IndexScheduler`.
mod db_name {
pub const ALL_TASKS: &str = "all-tasks";
pub const STATUS: &str = "status";
pub const KIND: &str = "kind";
pub const INDEX_TASKS: &str = "index-tasks";
pub const CANCELED_BY: &str = "canceled_by";
pub const ENQUEUED_AT: &str = "enqueued-at";
pub const STARTED_AT: &str = "started-at";
pub const FINISHED_AT: &str = "finished-at";
}
pub struct TaskQueue {
/// The main database, it contains all the tasks accessible by their Id.
pub(crate) all_tasks: Database<BEU32, SerdeJson<Task>>,
/// All the tasks ids grouped by their status.
// TODO we should not be able to serialize a `Status::Processing` in this database.
pub(crate) status: Database<SerdeBincode<Status>, RoaringBitmapCodec>,
/// All the tasks ids grouped by their kind.
pub(crate) kind: Database<SerdeBincode<Kind>, RoaringBitmapCodec>,
/// Store the tasks associated to an index.
pub(crate) index_tasks: Database<Str, RoaringBitmapCodec>,
/// Store the tasks that were canceled by a task uid
pub(crate) canceled_by: Database<BEU32, RoaringBitmapCodec>,
/// Store the task ids of tasks which were enqueued at a specific date
pub(crate) enqueued_at: Database<BEI128, CboRoaringBitmapCodec>,
/// Store the task ids of finished tasks which started being processed at a specific date
pub(crate) started_at: Database<BEI128, CboRoaringBitmapCodec>,
/// Store the task ids of tasks which finished at a specific date
pub(crate) finished_at: Database<BEI128, CboRoaringBitmapCodec>,
}
impl TaskQueue {
pub(crate) fn private_clone(&self) -> TaskQueue {
TaskQueue {
all_tasks: self.all_tasks,
status: self.status,
kind: self.kind,
index_tasks: self.index_tasks,
canceled_by: self.canceled_by,
enqueued_at: self.enqueued_at,
started_at: self.started_at,
finished_at: self.finished_at,
}
}
pub(super) fn new(env: &Env, wtxn: &mut RwTxn) -> Result<Self> {
Ok(Self {
all_tasks: env.create_database(wtxn, Some(db_name::ALL_TASKS))?,
status: env.create_database(wtxn, Some(db_name::STATUS))?,
kind: env.create_database(wtxn, Some(db_name::KIND))?,
index_tasks: env.create_database(wtxn, Some(db_name::INDEX_TASKS))?,
canceled_by: env.create_database(wtxn, Some(db_name::CANCELED_BY))?,
enqueued_at: env.create_database(wtxn, Some(db_name::ENQUEUED_AT))?,
started_at: env.create_database(wtxn, Some(db_name::STARTED_AT))?,
finished_at: env.create_database(wtxn, Some(db_name::FINISHED_AT))?,
})
}
pub(crate) fn last_task_id(&self, rtxn: &RoTxn) -> Result<Option<TaskId>> {
Ok(self.all_tasks.remap_data_type::<DecodeIgnore>().last(rtxn)?.map(|(k, _)| k + 1))
}
pub(crate) fn next_task_id(&self, rtxn: &RoTxn) -> Result<TaskId> {
Ok(self.last_task_id(rtxn)?.unwrap_or_default())
}
pub(crate) fn all_task_ids(&self, rtxn: &RoTxn) -> Result<RoaringBitmap> {
enum_iterator::all().map(|s| self.get_status(rtxn, s)).union()
}
pub(crate) fn get_task(&self, rtxn: &RoTxn, task_id: TaskId) -> Result<Option<Task>> {
Ok(self.all_tasks.get(rtxn, &task_id)?)
}
pub(crate) fn update_task(&self, wtxn: &mut RwTxn, task: &Task) -> Result<()> {
let old_task = self.get_task(wtxn, task.uid)?.ok_or(Error::CorruptedTaskQueue)?;
debug_assert!(old_task != *task);
debug_assert_eq!(old_task.uid, task.uid);
debug_assert!(old_task.batch_uid.is_none() && task.batch_uid.is_some());
if old_task.status != task.status {
self.update_status(wtxn, old_task.status, |bitmap| {
bitmap.remove(task.uid);
})?;
self.update_status(wtxn, task.status, |bitmap| {
bitmap.insert(task.uid);
})?;
}
if old_task.kind.as_kind() != task.kind.as_kind() {
self.update_kind(wtxn, old_task.kind.as_kind(), |bitmap| {
bitmap.remove(task.uid);
})?;
self.update_kind(wtxn, task.kind.as_kind(), |bitmap| {
bitmap.insert(task.uid);
})?;
}
assert_eq!(
old_task.enqueued_at, task.enqueued_at,
"Cannot update a task's enqueued_at time"
);
if old_task.started_at != task.started_at {
assert!(old_task.started_at.is_none(), "Cannot update a task's started_at time");
if let Some(started_at) = task.started_at {
insert_task_datetime(wtxn, self.started_at, started_at, task.uid)?;
}
}
if old_task.finished_at != task.finished_at {
assert!(old_task.finished_at.is_none(), "Cannot update a task's finished_at time");
if let Some(finished_at) = task.finished_at {
insert_task_datetime(wtxn, self.finished_at, finished_at, task.uid)?;
}
}
self.all_tasks.put(wtxn, &task.uid, task)?;
Ok(())
}
/// Returns the whole set of tasks that belongs to this index.
pub(crate) fn index_tasks(&self, rtxn: &RoTxn, index: &str) -> Result<RoaringBitmap> {
Ok(self.index_tasks.get(rtxn, index)?.unwrap_or_default())
}
pub(crate) fn update_index(
&self,
wtxn: &mut RwTxn,
index: &str,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.index_tasks(wtxn, index)?;
f(&mut tasks);
if tasks.is_empty() {
self.index_tasks.delete(wtxn, index)?;
} else {
self.index_tasks.put(wtxn, index, &tasks)?;
}
Ok(())
}
pub(crate) fn get_status(&self, rtxn: &RoTxn, status: Status) -> Result<RoaringBitmap> {
Ok(self.status.get(rtxn, &status)?.unwrap_or_default())
}
pub(crate) fn put_status(
&self,
wtxn: &mut RwTxn,
status: Status,
bitmap: &RoaringBitmap,
) -> Result<()> {
Ok(self.status.put(wtxn, &status, bitmap)?)
}
pub(crate) fn update_status(
&self,
wtxn: &mut RwTxn,
status: Status,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.get_status(wtxn, status)?;
f(&mut tasks);
self.put_status(wtxn, status, &tasks)?;
Ok(())
}
pub(crate) fn get_kind(&self, rtxn: &RoTxn, kind: Kind) -> Result<RoaringBitmap> {
Ok(self.kind.get(rtxn, &kind)?.unwrap_or_default())
}
pub(crate) fn put_kind(
&self,
wtxn: &mut RwTxn,
kind: Kind,
bitmap: &RoaringBitmap,
) -> Result<()> {
Ok(self.kind.put(wtxn, &kind, bitmap)?)
}
pub(crate) fn update_kind(
&self,
wtxn: &mut RwTxn,
kind: Kind,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.get_kind(wtxn, kind)?;
f(&mut tasks);
self.put_kind(wtxn, kind, &tasks)?;
Ok(())
}
/// Convert an iterator to a `Vec` of tasks. The tasks MUST exist or a
/// `CorruptedTaskQueue` error will be thrown.
pub(crate) fn get_existing_tasks(
&self,
rtxn: &RoTxn,
tasks: impl IntoIterator<Item = TaskId>,
) -> Result<Vec<Task>> {
tasks
.into_iter()
.map(|task_id| {
self.get_task(rtxn, task_id).and_then(|task| task.ok_or(Error::CorruptedTaskQueue))
})
.collect::<Result<_>>()
}
pub(crate) fn register(&self, wtxn: &mut RwTxn, task: &Task) -> Result<()> {
self.all_tasks.put(wtxn, &task.uid, task)?;
for index in task.indexes() {
self.update_index(wtxn, index, |bitmap| {
bitmap.insert(task.uid);
})?;
}
self.update_status(wtxn, Status::Enqueued, |bitmap| {
bitmap.insert(task.uid);
})?;
self.update_kind(wtxn, task.kind.as_kind(), |bitmap| {
bitmap.insert(task.uid);
})?;
utils::insert_task_datetime(wtxn, self.enqueued_at, task.enqueued_at, task.uid)?;
Ok(())
}
}
impl Queue {
/// Return the task ids matched by the given query from the index scheduler's point of view.
pub(crate) fn get_task_ids(
&self,
rtxn: &RoTxn,
query: &Query,
processing_tasks: &ProcessingTasks,
) -> Result<RoaringBitmap> {
let ProcessingTasks { batch: processing_batch, processing: processing_tasks, progress: _ } =
processing_tasks;
let Query {
limit,
from,
reverse,
uids,
batch_uids,
statuses,
types,
index_uids,
canceled_by,
before_enqueued_at,
after_enqueued_at,
before_started_at,
after_started_at,
before_finished_at,
after_finished_at,
} = query;
let mut tasks = self.tasks.all_task_ids(rtxn)?;
if let Some(from) = from {
let range = if reverse.unwrap_or_default() {
u32::MIN..*from
} else {
from.saturating_add(1)..u32::MAX
};
tasks.remove_range(range);
}
if let Some(batch_uids) = batch_uids {
let mut batch_tasks = RoaringBitmap::new();
for batch_uid in batch_uids {
if processing_batch.as_ref().map_or(false, |batch| batch.uid == *batch_uid) {
batch_tasks |= &**processing_tasks;
} else {
batch_tasks |= self.tasks_in_batch(rtxn, *batch_uid)?;
}
}
tasks &= batch_tasks;
}
if let Some(status) = statuses {
let mut status_tasks = RoaringBitmap::new();
for status in status {
match status {
// special case for Processing tasks
Status::Processing => {
status_tasks |= &**processing_tasks;
}
status => status_tasks |= &self.tasks.get_status(rtxn, *status)?,
};
}
if !status.contains(&Status::Processing) {
tasks -= &**processing_tasks;
}
tasks &= status_tasks;
}
if let Some(uids) = uids {
let uids = RoaringBitmap::from_iter(uids);
tasks &= &uids;
}
if let Some(canceled_by) = canceled_by {
let mut all_canceled_tasks = RoaringBitmap::new();
for cancel_task_uid in canceled_by {
if let Some(canceled_by_uid) = self.tasks.canceled_by.get(rtxn, cancel_task_uid)? {
all_canceled_tasks |= canceled_by_uid;
}
}
// if the canceled_by has been specified but no task
// matches then we prefer matching zero than all tasks.
if all_canceled_tasks.is_empty() {
return Ok(RoaringBitmap::new());
} else {
tasks &= all_canceled_tasks;
}
}
if let Some(kind) = types {
let mut kind_tasks = RoaringBitmap::new();
for kind in kind {
kind_tasks |= self.tasks.get_kind(rtxn, *kind)?;
}
tasks &= &kind_tasks;
}
if let Some(index) = index_uids {
let mut index_tasks = RoaringBitmap::new();
for index in index {
index_tasks |= self.tasks.index_tasks(rtxn, index)?;
}
tasks &= &index_tasks;
}
// For the started_at filter, we need to treat the part of the tasks that are processing from the part of the
// tasks that are not processing. The non-processing ones are filtered normally while the processing ones
// are entirely removed unless the in-memory startedAt variable falls within the date filter.
// Once we have filtered the two subsets, we put them back together and assign it back to `tasks`.
tasks = {
let (mut filtered_non_processing_tasks, mut filtered_processing_tasks) =
(&tasks - &**processing_tasks, &tasks & &**processing_tasks);
// special case for Processing tasks
// A closure that clears the filtered_processing_tasks if their started_at date falls outside the given bounds
let mut clear_filtered_processing_tasks =
|start: Bound<OffsetDateTime>, end: Bound<OffsetDateTime>| {
let start = map_bound(start, |b| b.unix_timestamp_nanos());
let end = map_bound(end, |b| b.unix_timestamp_nanos());
let is_within_dates = RangeBounds::contains(
&(start, end),
&processing_batch
.as_ref()
.map_or_else(OffsetDateTime::now_utc, |batch| batch.started_at)
.unix_timestamp_nanos(),
);
if !is_within_dates {
filtered_processing_tasks.clear();
}
};
match (after_started_at, before_started_at) {
(None, None) => (),
(None, Some(before)) => {
clear_filtered_processing_tasks(Bound::Unbounded, Bound::Excluded(*before))
}
(Some(after), None) => {
clear_filtered_processing_tasks(Bound::Excluded(*after), Bound::Unbounded)
}
(Some(after), Some(before)) => clear_filtered_processing_tasks(
Bound::Excluded(*after),
Bound::Excluded(*before),
),
};
keep_ids_within_datetimes(
rtxn,
&mut filtered_non_processing_tasks,
self.tasks.started_at,
*after_started_at,
*before_started_at,
)?;
filtered_non_processing_tasks | filtered_processing_tasks
};
keep_ids_within_datetimes(
rtxn,
&mut tasks,
self.tasks.enqueued_at,
*after_enqueued_at,
*before_enqueued_at,
)?;
keep_ids_within_datetimes(
rtxn,
&mut tasks,
self.tasks.finished_at,
*after_finished_at,
*before_finished_at,
)?;
if let Some(limit) = limit {
tasks = if query.reverse.unwrap_or_default() {
tasks.into_iter().take(*limit as usize).collect()
} else {
tasks.into_iter().rev().take(*limit as usize).collect()
};
}
Ok(tasks)
}
pub(crate) fn get_task_ids_from_authorized_indexes(
&self,
rtxn: &RoTxn,
query: &Query,
filters: &meilisearch_auth::AuthFilter,
processing_tasks: &ProcessingTasks,
) -> Result<(RoaringBitmap, u64)> {
// compute all tasks matching the filter by ignoring the limits, to find the number of tasks matching
// the filter.
// As this causes us to compute the filter twice it is slightly inefficient, but doing it this way spares
// us from modifying the underlying implementation, and the performance remains sufficient.
// Should this change, we would modify `get_task_ids` to directly return the number of matching tasks.
let total_tasks =
self.get_task_ids(rtxn, &query.clone().without_limits(), processing_tasks)?;
let mut tasks = self.get_task_ids(rtxn, query, processing_tasks)?;
// If the query contains a list of index uid or there is a finite list of authorized indexes,
// then we must exclude all the kinds that aren't associated to one and only one index.
if query.index_uids.is_some() || !filters.all_indexes_authorized() {
for kind in enum_iterator::all::<Kind>().filter(|kind| !kind.related_to_one_index()) {
tasks -= self.tasks.get_kind(rtxn, kind)?;
}
}
// Any task that is internally associated with a non-authorized index
// must be discarded.
if !filters.all_indexes_authorized() {
let all_indexes_iter = self.tasks.index_tasks.iter(rtxn)?;
for result in all_indexes_iter {
let (index, index_tasks) = result?;
if !filters.is_index_authorized(index) {
tasks -= index_tasks;
}
}
}
Ok((tasks, total_tasks.len()))
}
pub(crate) fn get_tasks_from_authorized_indexes(
&self,
rtxn: &RoTxn,
query: &Query,
filters: &meilisearch_auth::AuthFilter,
processing_tasks: &ProcessingTasks,
) -> Result<(Vec<Task>, u64)> {
let (tasks, total) =
self.get_task_ids_from_authorized_indexes(rtxn, query, filters, processing_tasks)?;
let tasks = if query.reverse.unwrap_or_default() {
Box::new(tasks.into_iter()) as Box<dyn Iterator<Item = u32>>
} else {
Box::new(tasks.into_iter().rev()) as Box<dyn Iterator<Item = u32>>
};
let tasks = self
.tasks
.get_existing_tasks(rtxn, tasks.take(query.limit.unwrap_or(u32::MAX) as usize))?;
let ProcessingTasks { batch, processing, progress: _ } = processing_tasks;
let ret = tasks.into_iter();
if processing.is_empty() || batch.is_none() {
Ok((ret.collect(), total))
} else {
// Safe because we ensured there was a batch in the previous branch
let batch = batch.as_ref().unwrap();
Ok((
ret.map(|task| {
if processing.contains(task.uid) {
Task {
status: Status::Processing,
batch_uid: Some(batch.uid),
started_at: Some(batch.started_at),
..task
}
} else {
task
}
})
.collect(),
total,
))
}
}
}

View file

@ -0,0 +1,441 @@
use meili_snap::snapshot;
use meilisearch_auth::AuthFilter;
use meilisearch_types::index_uid_pattern::IndexUidPattern;
use meilisearch_types::tasks::{IndexSwap, KindWithContent, Status};
use time::{Duration, OffsetDateTime};
use crate::insta_snapshot::{snapshot_bitmap, snapshot_index_scheduler};
use crate::test_utils::Breakpoint::*;
use crate::test_utils::{index_creation_task, FailureLocation};
use crate::{IndexScheduler, Query};
#[test]
fn query_tasks_from_and_limit() {
let (index_scheduler, mut handle) = IndexScheduler::test(true, vec![]);
let kind = index_creation_task("doggo", "bone");
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_first_task");
let kind = index_creation_task("whalo", "plankton");
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_second_task");
let kind = index_creation_task("catto", "his_own_vomit");
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_third_task");
handle.advance_n_successful_batches(3);
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "processed_all_tasks");
let rtxn = index_scheduler.env.read_txn().unwrap();
let processing = index_scheduler.processing_tasks.read().unwrap();
let query = Query { limit: Some(0), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &processing)
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[]");
let query = Query { limit: Some(1), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &processing)
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[2,]");
let query = Query { limit: Some(2), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &processing)
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[1,2,]");
let query = Query { from: Some(1), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &processing)
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[0,1,]");
let query = Query { from: Some(2), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &processing)
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[0,1,2,]");
let query = Query { from: Some(1), limit: Some(1), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &processing)
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[1,]");
let query = Query { from: Some(1), limit: Some(2), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &processing)
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[0,1,]");
}
#[test]
fn query_tasks_simple() {
let start_time = OffsetDateTime::now_utc();
let (index_scheduler, mut handle) =
IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]);
let kind = index_creation_task("catto", "mouse");
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = index_creation_task("doggo", "sheep");
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = index_creation_task("whalo", "fish");
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start");
handle.advance_till([Start, BatchCreated]);
let query = Query { statuses: Some(vec![Status::Processing]), ..Default::default() };
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[0,]"); // only the processing tasks in the first tick
let query = Query { statuses: Some(vec![Status::Enqueued]), ..Default::default() };
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[1,2,]"); // only the enqueued tasks in the first tick
let query =
Query { statuses: Some(vec![Status::Enqueued, Status::Processing]), ..Default::default() };
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
snapshot!(snapshot_bitmap(&tasks), @"[0,1,2,]"); // both enqueued and processing tasks in the first tick
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Processing]),
after_started_at: Some(start_time),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both enqueued and processing tasks in the first tick, but limited to those with a started_at
// that comes after the start of the test, which should excludes the enqueued tasks
snapshot!(snapshot_bitmap(&tasks), @"[0,]");
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Processing]),
before_started_at: Some(start_time),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both enqueued and processing tasks in the first tick, but limited to those with a started_at
// that comes before the start of the test, which should excludes all of them
snapshot!(snapshot_bitmap(&tasks), @"[]");
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Processing]),
after_started_at: Some(start_time),
before_started_at: Some(start_time + Duration::minutes(1)),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both enqueued and processing tasks in the first tick, but limited to those with a started_at
// that comes after the start of the test and before one minute after the start of the test,
// which should exclude the enqueued tasks and include the only processing task
snapshot!(snapshot_bitmap(&tasks), @"[0,]");
handle.advance_till([
InsideProcessBatch,
InsideProcessBatch,
ProcessBatchSucceeded,
AfterProcessing,
Start,
BatchCreated,
]);
let second_start_time = OffsetDateTime::now_utc();
let query = Query {
statuses: Some(vec![Status::Succeeded, Status::Processing]),
after_started_at: Some(start_time),
before_started_at: Some(start_time + Duration::minutes(1)),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both succeeded and processing tasks in the first tick, but limited to those with a started_at
// that comes after the start of the test and before one minute after the start of the test,
// which should include all tasks
snapshot!(snapshot_bitmap(&tasks), @"[0,1,]");
let query = Query {
statuses: Some(vec![Status::Succeeded, Status::Processing]),
before_started_at: Some(start_time),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both succeeded and processing tasks in the first tick, but limited to those with a started_at
// that comes before the start of the test, which should exclude all tasks
snapshot!(snapshot_bitmap(&tasks), @"[]");
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Succeeded, Status::Processing]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// both succeeded and processing tasks in the first tick, but limited to those with a started_at
// that comes after the start of the second part of the test and before one minute after the
// second start of the test, which should exclude all tasks
snapshot!(snapshot_bitmap(&tasks), @"[]");
// now we make one more batch, the started_at field of the new tasks will be past `second_start_time`
handle.advance_till([
InsideProcessBatch,
InsideProcessBatch,
ProcessBatchSucceeded,
AfterProcessing,
Start,
BatchCreated,
]);
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// we run the same query to verify that, and indeed find that the last task is matched
snapshot!(snapshot_bitmap(&tasks), @"[2,]");
let query = Query {
statuses: Some(vec![Status::Enqueued, Status::Succeeded, Status::Processing]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// enqueued, succeeded, or processing tasks started after the second part of the test, should
// again only return the last task
snapshot!(snapshot_bitmap(&tasks), @"[2,]");
handle.advance_till([ProcessBatchFailed, AfterProcessing]);
// now the last task should have failed
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "end");
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// so running the last query should return nothing
snapshot!(snapshot_bitmap(&tasks), @"[]");
let query = Query {
statuses: Some(vec![Status::Failed]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// but the same query on failed tasks should return the last task
snapshot!(snapshot_bitmap(&tasks), @"[2,]");
let query = Query {
statuses: Some(vec![Status::Failed]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// but the same query on failed tasks should return the last task
snapshot!(snapshot_bitmap(&tasks), @"[2,]");
let query = Query {
statuses: Some(vec![Status::Failed]),
uids: Some(vec![1]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// same query but with an invalid uid
snapshot!(snapshot_bitmap(&tasks), @"[]");
let query = Query {
statuses: Some(vec![Status::Failed]),
uids: Some(vec![2]),
after_started_at: Some(second_start_time),
before_started_at: Some(second_start_time + Duration::minutes(1)),
..Default::default()
};
let (tasks, _) = index_scheduler
.get_task_ids_from_authorized_indexes(&query, &AuthFilter::default())
.unwrap();
// same query but with a valid uid
snapshot!(snapshot_bitmap(&tasks), @"[2,]");
}
#[test]
fn query_tasks_special_rules() {
let (index_scheduler, mut handle) =
IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]);
let kind = index_creation_task("catto", "mouse");
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = index_creation_task("doggo", "sheep");
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = KindWithContent::IndexSwap {
swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "doggo".to_owned()) }],
};
let _task = index_scheduler.register(kind, None, false).unwrap();
let kind = KindWithContent::IndexSwap {
swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "whalo".to_owned()) }],
};
let _task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start");
handle.advance_till([Start, BatchCreated]);
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let query = Query { index_uids: Some(vec!["catto".to_owned()]), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
// only the first task associated with catto is returned, the indexSwap tasks are excluded!
snapshot!(snapshot_bitmap(&tasks), @"[0,]");
let query = Query { index_uids: Some(vec!["catto".to_owned()]), ..Default::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(
&rtxn,
&query,
&AuthFilter::with_allowed_indexes(
vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(),
),
&proc,
)
.unwrap();
// we have asked for only the tasks associated with catto, but are only authorized to retrieve the tasks
// associated with doggo -> empty result
snapshot!(snapshot_bitmap(&tasks), @"[]");
let query = Query::default();
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(
&rtxn,
&query,
&AuthFilter::with_allowed_indexes(
vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(),
),
&proc,
)
.unwrap();
// we asked for all the tasks, but we are only authorized to retrieve the doggo tasks
// -> only the index creation of doggo should be returned
snapshot!(snapshot_bitmap(&tasks), @"[1,]");
let query = Query::default();
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(
&rtxn,
&query,
&AuthFilter::with_allowed_indexes(
vec![
IndexUidPattern::new_unchecked("catto"),
IndexUidPattern::new_unchecked("doggo"),
]
.into_iter()
.collect(),
),
&proc,
)
.unwrap();
// we asked for all the tasks, but we are only authorized to retrieve the doggo and catto tasks
// -> all tasks except the swap of catto with whalo are returned
snapshot!(snapshot_bitmap(&tasks), @"[0,1,]");
let query = Query::default();
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
// we asked for all the tasks with all index authorized -> all tasks returned
snapshot!(snapshot_bitmap(&tasks), @"[0,1,2,3,]");
}
#[test]
fn query_tasks_canceled_by() {
let (index_scheduler, mut handle) =
IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]);
let kind = index_creation_task("catto", "mouse");
let _ = index_scheduler.register(kind, None, false).unwrap();
let kind = index_creation_task("doggo", "sheep");
let _ = index_scheduler.register(kind, None, false).unwrap();
let kind = KindWithContent::IndexSwap {
swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "doggo".to_owned()) }],
};
let _task = index_scheduler.register(kind, None, false).unwrap();
handle.advance_n_successful_batches(1);
let kind = KindWithContent::TaskCancelation {
query: "test_query".to_string(),
tasks: [0, 1, 2, 3].into_iter().collect(),
};
let task_cancelation = index_scheduler.register(kind, None, false).unwrap();
handle.advance_n_successful_batches(1);
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start");
let rtxn = index_scheduler.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let query = Query { canceled_by: Some(vec![task_cancelation.uid]), ..Query::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default(), &proc)
.unwrap();
// 0 is not returned because it was not canceled, 3 is not returned because it is the uid of the
// taskCancelation itself
snapshot!(snapshot_bitmap(&tasks), @"[1,2,]");
let query = Query { canceled_by: Some(vec![task_cancelation.uid]), ..Query::default() };
let (tasks, _) = index_scheduler
.queue
.get_task_ids_from_authorized_indexes(
&rtxn,
&query,
&AuthFilter::with_allowed_indexes(
vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(),
),
&proc,
)
.unwrap();
// Return only 1 because the user is not authorized to see task 2
snapshot!(snapshot_bitmap(&tasks), @"[1,]");
}

View file

@ -0,0 +1,395 @@
use big_s::S;
use meili_snap::{json_string, snapshot};
use meilisearch_types::error::ErrorCode;
use meilisearch_types::tasks::{KindWithContent, Status};
use roaring::RoaringBitmap;
use crate::insta_snapshot::snapshot_index_scheduler;
use crate::test_utils::Breakpoint::*;
use crate::test_utils::{index_creation_task, replace_document_import_task};
use crate::{IndexScheduler, Query};
#[test]
fn register() {
// In this test, the handle doesn't make any progress, we only check that the tasks are registered
let (index_scheduler, mut _handle) = IndexScheduler::test(true, vec![]);
let kinds = [
index_creation_task("catto", "mouse"),
replace_document_import_task("catto", None, 0, 12),
replace_document_import_task("catto", None, 1, 50),
replace_document_import_task("doggo", Some("bone"), 2, 5000),
];
let (_, file) = index_scheduler.queue.create_update_file_with_uuid(0).unwrap();
file.persist().unwrap();
let (_, file) = index_scheduler.queue.create_update_file_with_uuid(1).unwrap();
file.persist().unwrap();
let (_, file) = index_scheduler.queue.create_update_file_with_uuid(2).unwrap();
file.persist().unwrap();
for (idx, kind) in kinds.into_iter().enumerate() {
let k = kind.as_kind();
let task = index_scheduler.register(kind, None, false).unwrap();
index_scheduler.assert_internally_consistent();
assert_eq!(task.uid, idx as u32);
assert_eq!(task.status, Status::Enqueued);
assert_eq!(task.kind.as_kind(), k);
}
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "everything_is_successfully_registered");
}
#[test]
fn dry_run() {
let (index_scheduler, _handle) = IndexScheduler::test(true, vec![]);
let kind = KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None };
let task = index_scheduler.register(kind, None, true).unwrap();
snapshot!(task.uid, @"0");
snapshot!(snapshot_index_scheduler(&index_scheduler), @r"
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
----------------------------------------------------------------------
### Status:
----------------------------------------------------------------------
### Kind:
----------------------------------------------------------------------
### Index Tasks:
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------
");
let kind = KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None };
let task = index_scheduler.register(kind, Some(12), true).unwrap();
snapshot!(task.uid, @"12");
snapshot!(snapshot_index_scheduler(&index_scheduler), @r"
### Autobatching Enabled = true
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
----------------------------------------------------------------------
### Status:
----------------------------------------------------------------------
### Kind:
----------------------------------------------------------------------
### Index Tasks:
----------------------------------------------------------------------
### Index Mapper:
----------------------------------------------------------------------
### Canceled By:
----------------------------------------------------------------------
### Enqueued At:
----------------------------------------------------------------------
### Started At:
----------------------------------------------------------------------
### Finished At:
----------------------------------------------------------------------
### All Batches:
----------------------------------------------------------------------
### Batch to tasks mapping:
----------------------------------------------------------------------
### Batches Status:
----------------------------------------------------------------------
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
### Batches Started At:
----------------------------------------------------------------------
### Batches Finished At:
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------
");
}
#[test]
fn basic_set_taskid() {
let (index_scheduler, _handle) = IndexScheduler::test(true, vec![]);
let kind = KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None };
let task = index_scheduler.register(kind, None, false).unwrap();
snapshot!(task.uid, @"0");
let kind = KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None };
let task = index_scheduler.register(kind, Some(12), false).unwrap();
snapshot!(task.uid, @"12");
let kind = KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None };
let error = index_scheduler.register(kind, Some(5), false).unwrap_err();
snapshot!(error, @"Received bad task id: 5 should be >= to 13.");
}
#[test]
fn test_disable_auto_deletion_of_tasks() {
let (index_scheduler, mut handle) = IndexScheduler::test_with_custom_config(vec![], |config| {
config.cleanup_enabled = false;
config.max_number_of_tasks = 2;
});
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
handle.advance_one_successful_batch();
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
handle.advance_one_failed_batch();
// at this point the max number of tasks is reached
// we can still enqueue multiple tasks
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let tasks =
index_scheduler.queue.get_task_ids(&rtxn, &Query { ..Default::default() }, &proc).unwrap();
let tasks = index_scheduler.queue.tasks.get_existing_tasks(&rtxn, tasks).unwrap();
snapshot!(json_string!(tasks, { "[].enqueuedAt" => "[date]", "[].startedAt" => "[date]", "[].finishedAt" => "[date]" }), name: "task_queue_is_full");
drop(rtxn);
drop(proc);
// now we're above the max number of tasks
// and if we try to advance in the tick function no new task deletion should be enqueued
handle.advance_till([Start, BatchCreated]);
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let tasks =
index_scheduler.queue.get_task_ids(&rtxn, &Query { ..Default::default() }, &proc).unwrap();
let tasks = index_scheduler.queue.tasks.get_existing_tasks(&rtxn, tasks).unwrap();
snapshot!(json_string!(tasks, { "[].enqueuedAt" => "[date]", "[].startedAt" => "[date]", "[].finishedAt" => "[date]", ".**.original_filter" => "[filter]", ".**.query" => "[query]" }), name: "task_deletion_have_not_been_enqueued");
drop(rtxn);
drop(proc);
}
#[test]
fn test_auto_deletion_of_tasks() {
let (index_scheduler, mut handle) = IndexScheduler::test_with_custom_config(vec![], |config| {
config.max_number_of_tasks = 2;
});
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
handle.advance_one_successful_batch();
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
handle.advance_one_failed_batch();
// at this point the max number of tasks is reached
// we can still enqueue multiple tasks
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let tasks =
index_scheduler.queue.get_task_ids(&rtxn, &Query { ..Default::default() }, &proc).unwrap();
let tasks = index_scheduler.queue.tasks.get_existing_tasks(&rtxn, tasks).unwrap();
snapshot!(json_string!(tasks, { "[].enqueuedAt" => "[date]", "[].startedAt" => "[date]", "[].finishedAt" => "[date]" }), name: "task_queue_is_full");
drop(rtxn);
drop(proc);
// now we're above the max number of tasks
// and if we try to advance in the tick function a new task deletion should be enqueued
handle.advance_till([Start, BatchCreated]);
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let tasks =
index_scheduler.queue.get_task_ids(&rtxn, &Query { ..Default::default() }, &proc).unwrap();
let tasks = index_scheduler.queue.tasks.get_existing_tasks(&rtxn, tasks).unwrap();
snapshot!(json_string!(tasks, { "[].enqueuedAt" => "[date]", "[].startedAt" => "[date]", "[].finishedAt" => "[date]", ".**.original_filter" => "[filter]", ".**.query" => "[query]" }), name: "task_deletion_have_been_enqueued");
drop(rtxn);
drop(proc);
handle.advance_till([InsideProcessBatch, ProcessBatchSucceeded, AfterProcessing]);
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let tasks =
index_scheduler.queue.get_task_ids(&rtxn, &Query { ..Default::default() }, &proc).unwrap();
let tasks = index_scheduler.queue.tasks.get_existing_tasks(&rtxn, tasks).unwrap();
snapshot!(json_string!(tasks, { "[].enqueuedAt" => "[date]", "[].startedAt" => "[date]", "[].finishedAt" => "[date]", ".**.original_filter" => "[filter]", ".**.query" => "[query]" }), name: "task_deletion_have_been_processed");
drop(rtxn);
drop(proc);
handle.advance_one_failed_batch();
// a new task deletion has been enqueued
handle.advance_one_successful_batch();
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let tasks =
index_scheduler.queue.get_task_ids(&rtxn, &Query { ..Default::default() }, &proc).unwrap();
let tasks = index_scheduler.queue.tasks.get_existing_tasks(&rtxn, tasks).unwrap();
snapshot!(json_string!(tasks, { "[].enqueuedAt" => "[date]", "[].startedAt" => "[date]", "[].finishedAt" => "[date]", ".**.original_filter" => "[filter]", ".**.query" => "[query]" }), name: "after_the_second_task_deletion");
drop(rtxn);
drop(proc);
handle.advance_one_failed_batch();
handle.advance_one_successful_batch();
let rtxn = index_scheduler.env.read_txn().unwrap();
let proc = index_scheduler.processing_tasks.read().unwrap();
let tasks =
index_scheduler.queue.get_task_ids(&rtxn, &Query { ..Default::default() }, &proc).unwrap();
let tasks = index_scheduler.queue.tasks.get_existing_tasks(&rtxn, tasks).unwrap();
snapshot!(json_string!(tasks, { "[].enqueuedAt" => "[date]", "[].startedAt" => "[date]", "[].finishedAt" => "[date]", ".**.original_filter" => "[filter]", ".**.query" => "[query]" }), name: "everything_has_been_processed");
drop(rtxn);
drop(proc);
}
#[test]
fn test_task_queue_is_full() {
let (index_scheduler, mut handle) = IndexScheduler::test_with_custom_config(vec![], |config| {
// that's the minimum map size possible
config.task_db_size = 1048576;
});
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
handle.advance_one_successful_batch();
// on average this task takes ~600 bytes
loop {
let result = index_scheduler.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
);
if result.is_err() {
break;
}
handle.advance_one_failed_batch();
}
index_scheduler.assert_internally_consistent();
// at this point the task DB shoud have reached its limit and we should not be able to register new tasks
let result = index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap_err();
snapshot!(result, @"Meilisearch cannot receive write operations because the limit of the task database has been reached. Please delete tasks to continue performing write operations.");
// we won't be able to test this error in an integration test thus as a best effort test I still ensure the error return the expected error code
snapshot!(format!("{:?}", result.error_code()), @"NoSpaceLeftOnDevice");
// Even the task deletion that doesn't delete anything shouldn't be accepted
let result = index_scheduler
.register(
KindWithContent::TaskDeletion { query: S("test"), tasks: RoaringBitmap::new() },
None,
false,
)
.unwrap_err();
snapshot!(result, @"Meilisearch cannot receive write operations because the limit of the task database has been reached. Please delete tasks to continue performing write operations.");
// we won't be able to test this error in an integration test thus as a best effort test I still ensure the error return the expected error code
snapshot!(format!("{:?}", result.error_code()), @"NoSpaceLeftOnDevice");
// But a task deletion that delete something should works
index_scheduler
.register(
KindWithContent::TaskDeletion { query: S("test"), tasks: (0..100).collect() },
None,
false,
)
.unwrap();
handle.advance_one_successful_batch();
// Now we should be able to enqueue a few tasks again
index_scheduler
.register(
KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None },
None,
false,
)
.unwrap();
handle.advance_one_failed_batch();
}