mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-03 03:47:02 +02:00
Split the index-scheduler in ~500 loc modules
This commit is contained in:
parent
7f1071943e
commit
cb82b0798a
251 changed files with 9431 additions and 9079 deletions
537
crates/index-scheduler/src/queue/batches.rs
Normal file
537
crates/index-scheduler/src/queue/batches.rs
Normal 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))
|
||||
}
|
||||
}
|
473
crates/index-scheduler/src/queue/batches_test.rs
Normal file
473
crates/index-scheduler/src/queue/batches_test.rs
Normal 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,]");
|
||||
}
|
379
crates/index-scheduler/src/queue/mod.rs
Normal file
379
crates/index-scheduler/src/queue/mod.rs
Normal 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)
|
||||
}
|
||||
}
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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:
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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
|
||||
|
||||
----------------------------------------------------------------------
|
|
@ -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
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
|
@ -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
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
|
@ -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
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
|
@ -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
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
514
crates/index-scheduler/src/queue/tasks.rs
Normal file
514
crates/index-scheduler/src/queue/tasks.rs
Normal 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,
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
441
crates/index-scheduler/src/queue/tasks_test.rs
Normal file
441
crates/index-scheduler/src/queue/tasks_test.rs
Normal 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,]");
|
||||
}
|
395
crates/index-scheduler/src/queue/test.rs
Normal file
395
crates/index-scheduler/src/queue/test.rs
Normal 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();
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue