2022-10-20 16:25:34 +08:00
/*!
This crate defines the index scheduler , which is responsible for :
1. Keeping references to meilisearch ' s indexes and mapping them to their
user - defined names .
2. Scheduling tasks given by the user and executing them , in batch if possible .
When an ` IndexScheduler ` is created , a new thread containing a reference to the
scheduler is created . This thread runs the scheduler ' s run loop , where the
scheduler waits to be woken up to process new tasks . It wakes up when :
1. it is launched for the first time
2. a new task is registered
3. a batch of tasks has been processed
It is only within this thread that the scheduler is allowed to process tasks .
On the other hand , the publicly accessible methods of the scheduler can be
called asynchronously from any thread . These methods can either query the
content of the scheduler or enqueue new tasks .
* /
2022-09-09 18:16:19 +08:00
mod autobatcher ;
2022-09-07 06:10:14 +08:00
mod batch ;
2022-09-06 22:43:59 +08:00
pub mod error ;
2022-09-14 18:35:33 +08:00
mod index_mapper ;
2022-10-10 18:57:17 +08:00
#[ cfg(test) ]
2022-10-25 16:23:14 +08:00
mod insta_snapshot ;
2023-01-12 00:30:50 +08:00
mod lru ;
2022-09-07 06:10:14 +08:00
mod utils ;
2022-10-31 14:55:19 +08:00
mod uuid_codec ;
2022-09-06 22:43:59 +08:00
2022-09-07 05:49:19 +08:00
pub type Result < T > = std ::result ::Result < T , Error > ;
2022-09-06 22:43:59 +08:00
pub type TaskId = u32 ;
2023-03-29 20:44:15 +08:00
use std ::collections ::HashMap ;
2022-10-24 19:32:46 +08:00
use std ::ops ::{ Bound , RangeBounds } ;
2023-02-15 19:30:46 +08:00
use std ::path ::{ Path , PathBuf } ;
2022-10-21 00:00:07 +08:00
use std ::sync ::atomic ::AtomicBool ;
use std ::sync ::atomic ::Ordering ::Relaxed ;
2022-10-03 22:15:10 +08:00
use std ::sync ::{ Arc , RwLock } ;
2022-10-24 20:16:14 +08:00
use std ::time ::Duration ;
2022-10-03 22:15:10 +08:00
2022-10-25 16:59:06 +08:00
use dump ::{ KindDump , TaskDump , UpdateFile } ;
pub use error ::Error ;
2022-10-16 07:39:01 +08:00
use file_store ::FileStore ;
2022-10-05 22:48:43 +08:00
use meilisearch_types ::error ::ResponseError ;
2022-10-21 00:00:07 +08:00
use meilisearch_types ::heed ::types ::{ OwnedType , SerdeBincode , SerdeJson , Str } ;
2023-03-29 20:27:40 +08:00
use meilisearch_types ::heed ::{ self , Database , Env , RoTxn , RwTxn } ;
2022-10-25 16:59:06 +08:00
use meilisearch_types ::milli ::documents ::DocumentsBatchBuilder ;
2022-10-21 00:00:07 +08:00
use meilisearch_types ::milli ::update ::IndexerConfig ;
2023-02-24 02:31:57 +08:00
use meilisearch_types ::milli ::{ self , CboRoaringBitmapCodec , Index , RoaringBitmapCodec , BEU32 } ;
2022-10-25 16:59:06 +08:00
use meilisearch_types ::tasks ::{ Kind , KindWithContent , Status , Task } ;
2022-10-03 22:15:10 +08:00
use roaring ::RoaringBitmap ;
use synchronoise ::SignalEvent ;
2023-04-26 19:55:02 +08:00
use time ::format_description ::well_known ::Rfc3339 ;
2022-10-03 22:15:10 +08:00
use time ::OffsetDateTime ;
2022-10-25 22:10:14 +08:00
use utils ::{ filter_out_references_to_newer_tasks , keep_tasks_within_datetimes , map_bound } ;
2022-10-03 22:15:10 +08:00
use uuid ::Uuid ;
use crate ::index_mapper ::IndexMapper ;
2022-11-29 17:38:27 +08:00
use crate ::utils ::{ check_index_swap_validity , clamp_to_page_size } ;
2022-10-03 22:15:10 +08:00
2022-10-20 19:11:50 +08:00
pub ( crate ) type BEI128 =
meilisearch_types ::heed ::zerocopy ::I128 < meilisearch_types ::heed ::byteorder ::BE > ;
2022-10-19 18:59:12 +08:00
2022-10-20 16:25:34 +08:00
/// 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.
2022-10-19 22:07:04 +08:00
#[ derive(Default, Debug, Clone, PartialEq, Eq) ]
2022-10-03 22:15:10 +08:00
pub struct Query {
2022-10-20 16:25:34 +08:00
/// The maximum number of tasks to be matched
2022-10-13 18:48:23 +08:00
pub limit : Option < u32 > ,
2022-10-20 16:25:34 +08:00
/// The minimum [task id](`meilisearch_types::tasks::Task::uid`) to be matched
2022-10-03 22:15:10 +08:00
pub from : Option < u32 > ,
2022-10-20 16:25:34 +08:00
/// The allowed [statuses](`meilisearch_types::tasks::Task::status`) of the matched tasls
2022-11-28 23:27:41 +08:00
pub statuses : Option < Vec < Status > > ,
2022-10-20 16:25:34 +08:00
/// 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()
/// # }
/// ```
2022-11-28 23:27:41 +08:00
pub types : Option < Vec < Kind > > ,
2022-10-20 16:25:34 +08:00
/// The allowed [index ids](meilisearch_types::tasks::Task::index_uid) of the matched tasks
2022-11-28 23:27:41 +08:00
pub index_uids : Option < Vec < String > > ,
2022-10-20 16:25:34 +08:00
/// The [task ids](`meilisearch_types::tasks::Task::uid`) to be matched
2022-11-28 23:27:41 +08:00
pub uids : Option < Vec < TaskId > > ,
/// 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 > > ,
2022-10-20 16:25:34 +08:00
/// Exclusive upper bound of the matched tasks' [`enqueued_at`](meilisearch_types::tasks::Task::enqueued_at) field.
2022-10-19 18:59:12 +08:00
pub before_enqueued_at : Option < OffsetDateTime > ,
2022-10-20 16:25:34 +08:00
/// Exclusive lower bound of the matched tasks' [`enqueued_at`](meilisearch_types::tasks::Task::enqueued_at) field.
2022-10-19 18:59:12 +08:00
pub after_enqueued_at : Option < OffsetDateTime > ,
2022-10-20 16:25:34 +08:00
/// Exclusive upper bound of the matched tasks' [`started_at`](meilisearch_types::tasks::Task::started_at) field.
2022-10-19 18:59:12 +08:00
pub before_started_at : Option < OffsetDateTime > ,
2022-10-20 16:25:34 +08:00
/// Exclusive lower bound of the matched tasks' [`started_at`](meilisearch_types::tasks::Task::started_at) field.
2022-10-19 18:59:12 +08:00
pub after_started_at : Option < OffsetDateTime > ,
2022-10-20 16:25:34 +08:00
/// Exclusive upper bound of the matched tasks' [`finished_at`](meilisearch_types::tasks::Task::finished_at) field.
2022-10-19 18:59:12 +08:00
pub before_finished_at : Option < OffsetDateTime > ,
2022-10-20 16:25:34 +08:00
/// Exclusive lower bound of the matched tasks' [`finished_at`](meilisearch_types::tasks::Task::finished_at) field.
2022-10-19 18:59:12 +08:00
pub after_finished_at : Option < OffsetDateTime > ,
2022-10-03 22:15:10 +08:00
}
impl Query {
2022-10-27 22:23:50 +08:00
/// Return `true` if every field of the query is set to `None`, such that the query
2022-10-20 16:25:34 +08:00
/// matches all tasks.
2022-10-15 17:17:06 +08:00
pub fn is_empty ( & self ) -> bool {
matches! (
self ,
Query {
limit : None ,
from : None ,
2022-11-28 23:27:41 +08:00
statuses : None ,
types : None ,
index_uids : None ,
uids : None ,
canceled_by : None ,
2022-10-19 18:59:12 +08:00
before_enqueued_at : None ,
after_enqueued_at : None ,
before_started_at : None ,
after_started_at : None ,
before_finished_at : None ,
after_finished_at : None ,
2022-10-15 17:17:06 +08:00
}
)
}
2022-10-03 22:15:10 +08:00
2022-10-20 16:25:34 +08:00
/// Add an [index id](meilisearch_types::tasks::Task::index_uid) to the list of permitted indexes.
2022-10-03 22:15:10 +08:00
pub fn with_index ( self , index_uid : String ) -> Self {
2022-11-28 23:27:41 +08:00
let mut index_vec = self . index_uids . unwrap_or_default ( ) ;
2022-10-03 22:15:10 +08:00
index_vec . push ( index_uid ) ;
2022-11-28 23:27:41 +08:00
Self { index_uids : Some ( index_vec ) , .. self }
2022-10-03 22:15:10 +08:00
}
}
2022-10-17 19:54:35 +08:00
#[ derive(Debug, Clone) ]
struct ProcessingTasks {
/// The date and time at which the indexation started.
started_at : OffsetDateTime ,
/// The list of tasks ids that are currently running.
processing : RoaringBitmap ,
}
impl ProcessingTasks {
2022-10-19 17:26:55 +08:00
/// Creates an empty `ProcessingAt` struct.
fn new ( ) -> ProcessingTasks {
2022-10-21 00:00:07 +08:00
ProcessingTasks { started_at : OffsetDateTime ::now_utc ( ) , processing : RoaringBitmap ::new ( ) }
2022-10-19 17:26:55 +08:00
}
2022-10-19 17:22:59 +08:00
/// Stores the currently processing tasks, and the date time at which it started.
2022-10-17 19:54:35 +08:00
fn start_processing_at ( & mut self , started_at : OffsetDateTime , processing : RoaringBitmap ) {
self . started_at = started_at ;
self . processing = processing ;
}
2022-11-28 23:27:41 +08:00
/// Set the processing tasks to an empty list
fn stop_processing ( & mut self ) {
2022-10-17 19:54:35 +08:00
self . processing = RoaringBitmap ::new ( ) ;
}
2022-11-28 23:27:41 +08:00
/// Returns `true` if there, at least, is one task that is currently processing that we must stop.
2022-10-19 17:22:59 +08:00
fn must_cancel_processing_tasks ( & self , canceled_tasks : & RoaringBitmap ) -> bool {
! self . processing . is_disjoint ( canceled_tasks )
}
}
#[ derive(Default, Clone, Debug) ]
struct MustStopProcessing ( Arc < AtomicBool > ) ;
impl MustStopProcessing {
fn get ( & self ) -> bool {
self . 0. load ( Relaxed )
}
fn must_stop ( & self ) {
self . 0. store ( true , Relaxed ) ;
}
fn reset ( & self ) {
self . 0. store ( false , Relaxed ) ;
2022-10-17 19:54:35 +08:00
}
}
2022-10-03 22:15:10 +08:00
/// 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 " ;
2022-11-28 23:27:41 +08:00
pub const CANCELED_BY : & str = " canceled_by " ;
2022-10-19 18:59:12 +08:00
pub const ENQUEUED_AT : & str = " enqueued-at " ;
pub const STARTED_AT : & str = " started-at " ;
pub const FINISHED_AT : & str = " finished-at " ;
2022-10-03 22:15:10 +08:00
}
2022-10-26 17:41:59 +08:00
#[ cfg(test) ]
#[ derive(Debug, Clone, Copy, PartialEq, Eq) ]
pub enum Breakpoint {
2022-11-28 23:27:41 +08:00
// this state is only encountered while creating the scheduler in the test suite.
Init ,
2022-10-26 17:41:59 +08:00
Start ,
BatchCreated ,
BeforeProcessing ,
AfterProcessing ,
AbortedIndexation ,
ProcessBatchSucceeded ,
ProcessBatchFailed ,
InsideProcessBatch ,
}
#[ derive(Debug) ]
pub struct IndexSchedulerOptions {
/// The path to the version file of Meilisearch.
2022-10-26 17:47:49 +08:00
pub version_file_path : PathBuf ,
2022-10-26 17:41:59 +08:00
/// The path to the folder containing the auth LMDB env.
2022-10-26 17:47:49 +08:00
pub auth_path : PathBuf ,
2022-10-26 17:41:59 +08:00
/// The path to the folder containing the task databases.
2022-10-26 17:47:49 +08:00
pub tasks_path : PathBuf ,
2022-10-26 17:41:59 +08:00
/// The path to the file store containing the files associated to the tasks.
2022-10-26 17:47:49 +08:00
pub update_file_path : PathBuf ,
2022-10-26 17:41:59 +08:00
/// The path to the folder containing meilisearch's indexes.
2022-10-26 17:47:49 +08:00
pub indexes_path : PathBuf ,
2022-10-26 17:41:59 +08:00
/// The path to the folder containing the snapshots.
2022-10-26 17:47:49 +08:00
pub snapshots_path : PathBuf ,
2022-10-26 17:41:59 +08:00
/// The path to the folder containing the dumps.
2022-10-26 17:47:49 +08:00
pub dumps_path : PathBuf ,
2022-12-26 18:41:31 +08:00
/// The maximum size, in bytes, of the task index.
2022-10-26 17:47:49 +08:00
pub task_db_size : usize ,
2023-01-12 00:34:46 +08:00
/// The size, in bytes, with which a meilisearch index is opened the first time of each meilisearch index.
pub index_base_map_size : usize ,
2023-05-15 17:23:58 +08:00
/// Weither we open a meilisearch index with the MDB_WRITEMAP option or not.
pub enable_mdb_writemap : bool ,
2023-01-12 00:34:46 +08:00
/// The size, in bytes, by which the map size of an index is increased when it resized due to being full.
pub index_growth_amount : usize ,
/// The number of indexes that can be concurrently opened in memory.
pub index_count : usize ,
2022-10-26 17:41:59 +08:00
/// Configuration used during indexing for each meilisearch index.
2022-10-26 17:47:49 +08:00
pub indexer_config : IndexerConfig ,
2022-10-26 17:41:59 +08:00
/// Set to `true` iff the index scheduler is allowed to automatically
/// batch tasks together, to process multiple tasks at once.
2022-10-26 17:47:49 +08:00
pub autobatching_enabled : bool ,
2023-04-25 23:26:34 +08:00
/// The maximum number of tasks stored in the task queue before starting
/// to auto schedule task deletions.
pub max_number_of_tasks : usize ,
2022-10-26 17:41:59 +08:00
}
2022-10-20 16:25:34 +08:00
/// Structure which holds meilisearch's indexes and schedules the tasks
/// to be performed on them.
2022-10-03 22:15:10 +08:00
pub struct IndexScheduler {
/// The LMDB environment which the DBs are associated with.
pub ( crate ) env : Env ,
2022-10-19 17:22:59 +08:00
/// A boolean that can be set to true to stop the currently processing tasks.
pub ( crate ) must_stop_processing : MustStopProcessing ,
2022-10-20 16:25:34 +08:00
/// The list of tasks currently processing
2022-10-17 19:54:35 +08:00
pub ( crate ) processing_tasks : Arc < RwLock < ProcessingTasks > > ,
2022-10-20 16:25:34 +08:00
/// The list of files referenced by the tasks
2022-10-17 19:54:35 +08:00
pub ( crate ) file_store : FileStore ,
2022-10-03 22:15:10 +08:00
// The main database, it contains all the tasks accessible by their Id.
pub ( crate ) all_tasks : Database < OwnedType < BEU32 > , SerdeJson < Task > > ,
/// All the tasks ids grouped by their status.
2022-10-17 23:19:17 +08:00
// TODO we should not be able to serialize a `Status::Processing` in this database.
2022-10-03 22:15:10 +08:00
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 > ,
2022-11-28 23:27:41 +08:00
/// Store the tasks that were canceled by a task uid
pub ( crate ) canceled_by : Database < OwnedType < BEU32 > , RoaringBitmapCodec > ,
2022-10-19 18:59:12 +08:00
/// Store the task ids of tasks which were enqueued at a specific date
pub ( crate ) enqueued_at : Database < OwnedType < BEI128 > , CboRoaringBitmapCodec > ,
/// Store the task ids of finished tasks which started being processed at a specific date
pub ( crate ) started_at : Database < OwnedType < BEI128 > , CboRoaringBitmapCodec > ,
/// Store the task ids of tasks which finished at a specific date
pub ( crate ) finished_at : Database < OwnedType < BEI128 > , CboRoaringBitmapCodec > ,
2022-10-03 22:15:10 +08:00
/// In charge of creating, opening, storing and returning indexes.
pub ( crate ) index_mapper : IndexMapper ,
/// Get a signal when a batch needs to be processed.
pub ( crate ) wake_up : Arc < SignalEvent > ,
2022-10-17 19:54:35 +08:00
/// Whether auto-batching is enabled or not.
2022-10-10 23:00:56 +08:00
pub ( crate ) autobatching_enabled : bool ,
2023-04-25 23:26:34 +08:00
/// The max number of tasks allowed before the scheduler starts to delete
/// the finished tasks automatically.
pub ( crate ) max_number_of_tasks : usize ,
2022-10-13 21:02:59 +08:00
/// The path used to create the dumps.
pub ( crate ) dumps_path : PathBuf ,
2022-10-25 16:53:25 +08:00
/// The path used to create the snapshots.
pub ( crate ) snapshots_path : PathBuf ,
2022-10-25 20:35:10 +08:00
/// The path to the folder containing the auth LMDB env.
pub ( crate ) auth_path : PathBuf ,
2022-10-25 21:06:28 +08:00
/// The path to the version file of Meilisearch.
pub ( crate ) version_file_path : PathBuf ,
2022-10-03 22:15:10 +08:00
// ================= test
2022-10-25 15:48:51 +08:00
// The next entry is dedicated to the tests.
/// Provide a way to set a breakpoint in multiple part of the scheduler.
///
/// See [self.breakpoint()](`IndexScheduler::breakpoint`) for an explanation.
2022-10-03 22:15:10 +08:00
#[ cfg(test) ]
2022-10-20 23:11:44 +08:00
test_breakpoint_sdr : crossbeam ::channel ::Sender < ( Breakpoint , bool ) > ,
/// A list of planned failures within the [`tick`](IndexScheduler::tick) method of the index scheduler.
///
/// The first field is the iteration index and the second field identifies a location in the code.
2022-10-26 23:31:23 +08:00
#[ cfg(test) ]
2022-10-20 23:11:44 +08:00
planned_failures : Vec < ( usize , tests ::FailureLocation ) > ,
/// A counter that is incremented before every call to [`tick`](IndexScheduler::tick)
2022-10-26 23:31:23 +08:00
#[ cfg(test) ]
2022-10-20 23:11:44 +08:00
run_loop_iteration : Arc < RwLock < usize > > ,
}
2022-10-26 17:41:59 +08:00
2022-10-20 23:11:44 +08:00
impl IndexScheduler {
2022-10-25 16:53:25 +08:00
fn private_clone ( & self ) -> IndexScheduler {
IndexScheduler {
2022-10-20 23:11:44 +08:00
env : self . env . clone ( ) ,
must_stop_processing : self . must_stop_processing . clone ( ) ,
processing_tasks : self . processing_tasks . clone ( ) ,
file_store : self . file_store . clone ( ) ,
2022-10-25 16:53:29 +08:00
all_tasks : self . all_tasks ,
status : self . status ,
kind : self . kind ,
index_tasks : self . index_tasks ,
2022-11-28 23:27:41 +08:00
canceled_by : self . canceled_by ,
2022-10-25 16:53:29 +08:00
enqueued_at : self . enqueued_at ,
started_at : self . started_at ,
finished_at : self . finished_at ,
2022-10-20 23:11:44 +08:00
index_mapper : self . index_mapper . clone ( ) ,
wake_up : self . wake_up . clone ( ) ,
2022-10-25 16:53:29 +08:00
autobatching_enabled : self . autobatching_enabled ,
2023-04-25 23:26:34 +08:00
max_number_of_tasks : self . max_number_of_tasks ,
2022-10-25 16:53:25 +08:00
snapshots_path : self . snapshots_path . clone ( ) ,
2022-10-20 23:11:44 +08:00
dumps_path : self . dumps_path . clone ( ) ,
2022-10-25 20:35:10 +08:00
auth_path : self . auth_path . clone ( ) ,
2022-10-25 21:06:28 +08:00
version_file_path : self . version_file_path . clone ( ) ,
2022-10-20 23:11:44 +08:00
#[ cfg(test) ]
test_breakpoint_sdr : self . test_breakpoint_sdr . clone ( ) ,
#[ cfg(test) ]
planned_failures : self . planned_failures . clone ( ) ,
#[ cfg(test) ]
run_loop_iteration : self . run_loop_iteration . clone ( ) ,
}
}
2022-10-03 22:15:10 +08:00
}
impl IndexScheduler {
2022-10-20 16:25:34 +08:00
/// Create an index scheduler and start its run loop.
2022-10-03 22:15:10 +08:00
pub fn new (
2022-10-26 17:41:59 +08:00
options : IndexSchedulerOptions ,
2022-10-20 23:11:44 +08:00
#[ cfg(test) ] test_breakpoint_sdr : crossbeam ::channel ::Sender < ( Breakpoint , bool ) > ,
#[ cfg(test) ] planned_failures : Vec < ( usize , tests ::FailureLocation ) > ,
2022-10-03 22:15:10 +08:00
) -> Result < Self > {
2022-10-26 17:41:59 +08:00
std ::fs ::create_dir_all ( & options . tasks_path ) ? ;
std ::fs ::create_dir_all ( & options . update_file_path ) ? ;
std ::fs ::create_dir_all ( & options . indexes_path ) ? ;
std ::fs ::create_dir_all ( & options . dumps_path ) ? ;
2022-10-03 22:15:10 +08:00
2023-05-15 17:23:58 +08:00
if cfg! ( windows ) & & options . enable_mdb_writemap {
panic! ( " Windows doesn't support the MDB_WRITEMAP LMDB option " ) ;
}
2023-02-15 19:30:46 +08:00
let task_db_size = clamp_to_page_size ( options . task_db_size ) ;
2023-02-15 19:31:14 +08:00
let budget = if options . indexer_config . skip_index_budget {
IndexBudget {
2023-02-15 19:30:46 +08:00
map_size : options . index_base_map_size ,
index_count : options . index_count ,
task_db_size ,
2023-02-15 19:31:14 +08:00
}
} else {
Self ::index_budget (
& options . tasks_path ,
options . index_base_map_size ,
task_db_size ,
options . index_count ,
)
} ;
2023-02-15 19:30:46 +08:00
2022-10-26 17:41:59 +08:00
let env = heed ::EnvOpenOptions ::new ( )
2022-11-28 23:27:41 +08:00
. max_dbs ( 10 )
2023-02-15 19:30:46 +08:00
. map_size ( budget . task_db_size )
2022-10-26 17:41:59 +08:00
. open ( options . tasks_path ) ? ;
let file_store = FileStore ::new ( & options . update_file_path ) ? ;
2022-10-03 22:15:10 +08:00
2023-05-15 16:15:33 +08:00
let mut wtxn = env . write_txn ( ) ? ;
let all_tasks = env . create_database ( & mut wtxn , Some ( db_name ::ALL_TASKS ) ) ? ;
let status = env . create_database ( & mut wtxn , Some ( db_name ::STATUS ) ) ? ;
let kind = env . create_database ( & mut wtxn , Some ( db_name ::KIND ) ) ? ;
let index_tasks = env . create_database ( & mut wtxn , Some ( db_name ::INDEX_TASKS ) ) ? ;
let canceled_by = env . create_database ( & mut wtxn , Some ( db_name ::CANCELED_BY ) ) ? ;
let enqueued_at = env . create_database ( & mut wtxn , Some ( db_name ::ENQUEUED_AT ) ) ? ;
let started_at = env . create_database ( & mut wtxn , Some ( db_name ::STARTED_AT ) ) ? ;
let finished_at = env . create_database ( & mut wtxn , Some ( db_name ::FINISHED_AT ) ) ? ;
wtxn . commit ( ) ? ;
2022-10-03 22:15:10 +08:00
// allow unreachable_code to get rids of the warning in the case of a test build.
let this = Self {
2022-10-19 17:22:59 +08:00
must_stop_processing : MustStopProcessing ::default ( ) ,
2022-10-19 17:26:55 +08:00
processing_tasks : Arc ::new ( RwLock ::new ( ProcessingTasks ::new ( ) ) ) ,
2022-10-03 22:15:10 +08:00
file_store ,
2023-05-15 16:15:33 +08:00
all_tasks ,
status ,
kind ,
index_tasks ,
canceled_by ,
enqueued_at ,
started_at ,
finished_at ,
2022-10-26 17:41:59 +08:00
index_mapper : IndexMapper ::new (
& env ,
options . indexes_path ,
2023-02-15 19:30:46 +08:00
budget . map_size ,
2023-01-12 00:34:46 +08:00
options . index_growth_amount ,
2023-02-15 19:30:46 +08:00
budget . index_count ,
2023-05-15 17:23:58 +08:00
options . enable_mdb_writemap ,
2022-10-26 17:41:59 +08:00
options . indexer_config ,
) ? ,
2022-10-03 22:15:10 +08:00
env ,
// we want to start the loop right away in case meilisearch was ctrl+Ced while processing things
wake_up : Arc ::new ( SignalEvent ::auto ( true ) ) ,
2022-10-26 17:41:59 +08:00
autobatching_enabled : options . autobatching_enabled ,
2023-04-25 23:26:34 +08:00
max_number_of_tasks : options . max_number_of_tasks ,
2022-10-26 17:41:59 +08:00
dumps_path : options . dumps_path ,
snapshots_path : options . snapshots_path ,
auth_path : options . auth_path ,
version_file_path : options . version_file_path ,
2022-10-03 22:15:10 +08:00
#[ cfg(test) ]
test_breakpoint_sdr ,
2022-10-20 23:11:44 +08:00
#[ cfg(test) ]
planned_failures ,
#[ cfg(test) ]
run_loop_iteration : Arc ::new ( RwLock ::new ( 0 ) ) ,
2022-10-03 22:15:10 +08:00
} ;
this . run ( ) ;
Ok ( this )
}
2023-04-06 19:38:47 +08:00
/// Return `Ok(())` if the index scheduler is able to access one of its database.
pub fn health ( & self ) -> Result < ( ) > {
let rtxn = self . env . read_txn ( ) ? ;
self . all_tasks . first ( & rtxn ) ? ;
Ok ( ( ) )
}
2023-02-15 19:30:46 +08:00
fn index_budget (
tasks_path : & Path ,
base_map_size : usize ,
mut task_db_size : usize ,
max_index_count : usize ,
) -> IndexBudget {
2023-02-28 00:13:07 +08:00
#[ cfg(windows) ]
const DEFAULT_BUDGET : usize = 6 * 1024 * 1024 * 1024 * 1024 ; // 6 TiB, 1 index
#[ cfg(not(windows)) ]
const DEFAULT_BUDGET : usize = 80 * 1024 * 1024 * 1024 * 1024 ; // 80 TiB, 18 indexes
let budget = if Self ::is_good_heed ( tasks_path , DEFAULT_BUDGET ) {
DEFAULT_BUDGET
} else {
log ::debug! ( " determining budget with dichotomic search " ) ;
utils ::dichotomic_search ( DEFAULT_BUDGET / 2 , | map_size | {
Self ::is_good_heed ( tasks_path , map_size )
} )
} ;
2023-02-15 19:30:46 +08:00
log ::debug! ( " memmap budget: {budget}B " ) ;
let mut budget = budget / 2 ;
if task_db_size > ( budget / 2 ) {
task_db_size = clamp_to_page_size ( budget * 2 / 5 ) ;
log ::debug! (
" Decreasing max size of task DB to {task_db_size}B due to constrained memory space "
) ;
}
budget - = task_db_size ;
// won't be mutated again
let budget = budget ;
let task_db_size = task_db_size ;
log ::debug! ( " index budget: {budget}B " ) ;
let mut index_count = budget / base_map_size ;
if index_count < 2 {
// take a bit less than half than the budget to make sure we can always afford to open an index
let map_size = ( budget * 2 ) / 5 ;
// single index of max budget
log ::debug! ( " 1 index of {map_size}B can be opened simultaneously. " ) ;
return IndexBudget { map_size , index_count : 1 , task_db_size } ;
}
// give us some space for an additional index when the cache is already full
// decrement is OK because index_count >= 2.
index_count - = 1 ;
if index_count > max_index_count {
index_count = max_index_count ;
}
log ::debug! ( " Up to {index_count} indexes of {base_map_size}B opened simultaneously. " ) ;
IndexBudget { map_size : base_map_size , index_count , task_db_size }
}
fn is_good_heed ( tasks_path : & Path , map_size : usize ) -> bool {
if let Ok ( env ) =
heed ::EnvOpenOptions ::new ( ) . map_size ( clamp_to_page_size ( map_size ) ) . open ( tasks_path )
{
env . prepare_for_closing ( ) . wait ( ) ;
true
} else {
// We're treating all errors equally here, not only allocation errors.
// This means there's a possiblity for the budget to lower due to errors different from allocation errors.
// For persistent errors, this is OK as long as the task db is then reopened normally without ignoring the error this time.
// For transient errors, this could lead to an instance with too low a budget.
// However transient errors are: 1) less likely than persistent errors 2) likely to cause other issues down the line anyway.
false
}
}
2022-10-27 17:17:50 +08:00
pub fn read_txn ( & self ) -> Result < RoTxn > {
self . env . read_txn ( ) . map_err ( | e | e . into ( ) )
}
2022-10-20 16:25:34 +08:00
/// Start the run loop for the given index scheduler.
///
/// This function will execute in a different thread and must be called
/// only once per index scheduler.
2022-10-03 22:15:10 +08:00
fn run ( & self ) {
2022-10-20 23:11:44 +08:00
let run = self . private_clone ( ) ;
2022-11-28 23:27:41 +08:00
std ::thread ::Builder ::new ( )
. name ( String ::from ( " scheduler " ) )
. spawn ( move | | {
#[ cfg(test) ]
run . breakpoint ( Breakpoint ::Init ) ;
2023-01-10 02:30:29 +08:00
run . wake_up . wait ( ) ;
2022-11-28 23:27:41 +08:00
2023-01-10 02:30:29 +08:00
loop {
2022-11-28 23:27:41 +08:00
match run . tick ( ) {
2023-01-10 02:30:29 +08:00
Ok ( TickOutcome ::TickAgain ( _ ) ) = > ( ) ,
Ok ( TickOutcome ::WaitForSignal ) = > run . wake_up . wait ( ) ,
2022-11-28 23:27:41 +08:00
Err ( e ) = > {
log ::error! ( " {} " , e ) ;
// Wait one second when an irrecoverable error occurs.
2023-04-04 03:08:47 +08:00
if ! e . is_recoverable ( ) {
2022-11-28 23:27:41 +08:00
std ::thread ::sleep ( Duration ::from_secs ( 1 ) ) ;
}
}
2022-10-24 20:16:14 +08:00
}
}
2022-11-28 23:27:41 +08:00
} )
. unwrap ( ) ;
2022-10-03 22:15:10 +08:00
}
2022-10-16 07:39:01 +08:00
pub fn indexer_config ( & self ) -> & IndexerConfig {
& self . index_mapper . indexer_config
}
2023-01-24 23:17:23 +08:00
pub fn size ( & self ) -> Result < u64 > {
Ok ( self . env . real_disk_size ( ) ? )
}
2022-10-20 16:25:34 +08:00
/// Return the index corresponding to the name.
///
/// * If the index wasn't opened before, the index will be opened.
/// * If the index doesn't exist on disk, the `IndexNotFoundError` is thrown.
2023-02-20 23:42:54 +08:00
///
/// ### Note
///
/// As an `Index` requires a large swath of the virtual memory address space, correct usage of an `Index` does not
/// keep its handle for too long.
///
/// Some configurations also can't reasonably open multiple indexes at once.
/// If you need to fetch information from or perform an action on all indexes,
/// see the `try_for_each_index` function.
2022-10-03 22:15:10 +08:00
pub fn index ( & self , name : & str ) -> Result < Index > {
let rtxn = self . env . read_txn ( ) ? ;
self . index_mapper . index ( & rtxn , name )
}
2023-02-20 23:42:54 +08:00
/// Return the name of all indexes without opening them.
2023-02-24 02:31:57 +08:00
pub fn index_names ( & self ) -> Result < Vec < String > > {
2023-02-20 23:42:54 +08:00
let rtxn = self . env . read_txn ( ) ? ;
self . index_mapper . index_names ( & rtxn )
}
/// Attempts `f` for each index that exists known to the index scheduler.
///
/// It is preferable to use this function rather than a loop that opens all indexes, as a way to avoid having all indexes opened,
/// which is unsupported in general.
///
/// Since `f` is allowed to return a result, and `Index` is cloneable, it is still possible to wrongly build e.g. a vector of
/// all the indexes, but this function makes it harder and so less likely to do accidentally.
///
/// If many indexes exist, this operation can take time to complete (in the order of seconds for a 1000 of indexes) as it needs to open
/// all the indexes.
pub fn try_for_each_index < U , V > ( & self , f : impl FnMut ( & str , & Index ) -> Result < U > ) -> Result < V >
where
V : FromIterator < U > ,
{
2022-10-03 22:15:10 +08:00
let rtxn = self . env . read_txn ( ) ? ;
2023-02-20 23:42:54 +08:00
self . index_mapper . try_for_each_index ( & rtxn , f )
2022-10-03 22:15:10 +08:00
}
2022-10-27 17:17:50 +08:00
/// 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 ) -> Result < RoaringBitmap > {
2022-11-28 23:27:41 +08:00
let ProcessingTasks {
started_at : started_at_processing , processing : processing_tasks , ..
} = self . processing_tasks . read ( ) . unwrap ( ) . clone ( ) ;
2022-10-03 22:15:10 +08:00
2022-10-27 19:00:30 +08:00
let mut tasks = self . all_task_ids ( rtxn ) ? ;
2022-10-03 22:15:10 +08:00
2022-10-26 18:56:01 +08:00
if let Some ( from ) = & query . from {
tasks . remove_range ( from . saturating_add ( 1 ) .. ) ;
}
2022-11-28 23:27:41 +08:00
if let Some ( status ) = & query . statuses {
2022-10-03 22:15:10 +08:00
let mut status_tasks = RoaringBitmap ::new ( ) ;
for status in status {
2022-10-24 19:32:46 +08:00
match status {
// special case for Processing tasks
Status ::Processing = > {
2022-10-25 15:48:51 +08:00
status_tasks | = & processing_tasks ;
2022-10-24 19:32:46 +08:00
}
2022-10-27 19:00:30 +08:00
status = > status_tasks | = & self . get_status ( rtxn , * status ) ? ,
2022-10-24 19:32:46 +08:00
} ;
}
2022-10-25 15:48:51 +08:00
if ! status . contains ( & Status ::Processing ) {
tasks - = & processing_tasks ;
2022-10-03 22:15:10 +08:00
}
tasks & = status_tasks ;
}
2022-11-28 23:27:41 +08:00
if let Some ( uids ) = & query . uids {
2022-10-24 19:32:46 +08:00
let uids = RoaringBitmap ::from_iter ( uids ) ;
tasks & = & uids ;
}
2022-11-28 23:27:41 +08:00
if let Some ( canceled_by ) = & query . canceled_by {
2023-01-18 22:25:27 +08:00
let mut all_canceled_tasks = RoaringBitmap ::new ( ) ;
2022-11-28 23:27:41 +08:00
for cancel_task_uid in canceled_by {
if let Some ( canceled_by_uid ) =
self . canceled_by . get ( rtxn , & BEU32 ::new ( * cancel_task_uid ) ) ?
{
2023-01-18 22:25:27 +08:00
all_canceled_tasks | = canceled_by_uid ;
2022-11-28 23:27:41 +08:00
}
}
2023-01-18 22:25:27 +08:00
// 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 ;
}
2022-11-28 23:27:41 +08:00
}
if let Some ( kind ) = & query . types {
2022-10-03 22:15:10 +08:00
let mut kind_tasks = RoaringBitmap ::new ( ) ;
for kind in kind {
2022-10-27 19:00:30 +08:00
kind_tasks | = self . get_kind ( rtxn , * kind ) ? ;
2022-10-03 22:15:10 +08:00
}
2022-10-24 19:32:46 +08:00
tasks & = & kind_tasks ;
2022-10-03 22:15:10 +08:00
}
2022-11-28 23:27:41 +08:00
if let Some ( index ) = & query . index_uids {
2022-10-03 22:15:10 +08:00
let mut index_tasks = RoaringBitmap ::new ( ) ;
for index in index {
2022-10-27 19:00:30 +08:00
index_tasks | = self . index_tasks ( rtxn , index ) ? ;
2022-10-03 22:15:10 +08:00
}
2022-10-24 19:32:46 +08:00
tasks & = & index_tasks ;
2022-10-03 22:15:10 +08:00
}
2022-10-17 22:30:18 +08:00
2022-10-24 19:32:46 +08:00
// 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 ) =
2022-10-25 15:48:51 +08:00
( & tasks - & processing_tasks , & tasks & & processing_tasks ) ;
2022-10-24 19:32:46 +08:00
// special case for Processing tasks
2022-10-25 15:48:51 +08:00
// 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 ) ,
& started_at_processing . unix_timestamp_nanos ( ) ,
) ;
if ! is_within_dates {
filtered_processing_tasks . clear ( ) ;
2022-10-24 19:32:46 +08:00
}
} ;
2022-10-25 15:48:51 +08:00
match ( query . after_started_at , query . before_started_at ) {
( None , None ) = > ( ) ,
( None , Some ( before ) ) = > {
clear_filtered_processing_tasks ( Bound ::Unbounded , Bound ::Excluded ( before ) )
2022-10-24 19:32:46 +08:00
}
2022-10-25 15:48:51 +08:00
( 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 ) )
}
} ;
2022-10-24 19:32:46 +08:00
keep_tasks_within_datetimes (
2022-10-27 19:00:30 +08:00
rtxn ,
2022-10-24 19:32:46 +08:00
& mut filtered_non_processing_tasks ,
self . started_at ,
query . after_started_at ,
query . before_started_at ,
) ? ;
filtered_non_processing_tasks | filtered_processing_tasks
} ;
2022-10-19 18:59:12 +08:00
keep_tasks_within_datetimes (
2022-10-27 19:00:30 +08:00
rtxn ,
2022-10-19 18:59:12 +08:00
& mut tasks ,
self . enqueued_at ,
query . after_enqueued_at ,
query . before_enqueued_at ,
) ? ;
keep_tasks_within_datetimes (
2022-10-27 19:00:30 +08:00
rtxn ,
2022-10-19 18:59:12 +08:00
& mut tasks ,
self . finished_at ,
query . after_finished_at ,
query . before_finished_at ,
) ? ;
2022-10-26 18:56:01 +08:00
if let Some ( limit ) = query . limit {
tasks = tasks . into_iter ( ) . rev ( ) . take ( limit as usize ) . collect ( ) ;
}
2022-10-13 17:09:00 +08:00
Ok ( tasks )
}
2022-10-27 17:17:50 +08:00
/// Return true iff there is at least one task associated with this index
/// that is processing.
pub fn is_index_processing ( & self , index : & str ) -> Result < bool > {
2022-10-13 19:04:49 +08:00
let rtxn = self . env . read_txn ( ) ? ;
2022-10-27 17:17:50 +08:00
let processing_tasks = self . processing_tasks . read ( ) . unwrap ( ) . processing . clone ( ) ;
let index_tasks = self . index_tasks ( & rtxn , index ) ? ;
let nbr_index_processing_tasks = processing_tasks . intersection_len ( & index_tasks ) ;
Ok ( nbr_index_processing_tasks > 0 )
}
/// Return the task ids matching the query 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.
2022-10-27 22:23:50 +08:00
/// 2. The user may not have the rights to access the tasks (internally) associated with all indexes.
2022-10-27 17:17:50 +08:00
pub fn get_task_ids_from_authorized_indexes (
& self ,
rtxn : & RoTxn ,
query : & Query ,
2023-02-19 21:40:25 +08:00
filters : & meilisearch_auth ::AuthFilter ,
2022-10-27 17:17:50 +08:00
) -> Result < RoaringBitmap > {
2022-10-27 19:00:30 +08:00
let mut tasks = self . get_task_ids ( rtxn , query ) ? ;
2022-10-27 17:17:50 +08:00
2022-11-28 23:27:41 +08:00
// 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.
2023-02-19 21:40:25 +08:00
if query . index_uids . is_some ( ) | | ! filters . all_indexes_authorized ( ) {
2022-10-28 00:00:04 +08:00
for kind in enum_iterator ::all ::< Kind > ( ) . filter ( | kind | ! kind . related_to_one_index ( ) ) {
tasks - = self . get_kind ( rtxn , kind ) ? ;
}
2022-10-27 17:17:50 +08:00
}
// Any task that is internally associated with a non-authorized index
// must be discarded.
2023-02-19 21:40:25 +08:00
if ! filters . all_indexes_authorized ( ) {
2022-10-27 17:17:50 +08:00
let all_indexes_iter = self . index_tasks . iter ( rtxn ) ? ;
2022-10-27 22:23:50 +08:00
for result in all_indexes_iter {
let ( index , index_tasks ) = result ? ;
2023-02-19 21:40:25 +08:00
if ! filters . is_index_authorized ( index ) {
2022-10-27 17:17:50 +08:00
tasks - = index_tasks ;
}
}
}
Ok ( tasks )
}
/// Return the tasks matching the query 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.
2022-10-27 22:23:50 +08:00
/// 2. The user may not have the rights to access the tasks (internally) associated with all indexes.
2022-10-27 17:17:50 +08:00
pub fn get_tasks_from_authorized_indexes (
& self ,
query : Query ,
2023-02-19 21:40:25 +08:00
filters : & meilisearch_auth ::AuthFilter ,
2022-10-27 17:17:50 +08:00
) -> Result < Vec < Task > > {
let rtxn = self . env . read_txn ( ) ? ;
2023-02-19 21:40:25 +08:00
let tasks = self . get_task_ids_from_authorized_indexes ( & rtxn , & query , filters ) ? ;
2022-10-03 22:15:10 +08:00
2022-10-13 18:48:23 +08:00
let tasks = self . get_existing_tasks (
& rtxn ,
2022-10-21 00:00:07 +08:00
tasks . into_iter ( ) . rev ( ) . take ( query . limit . unwrap_or ( u32 ::MAX ) as usize ) ,
2022-10-13 18:48:23 +08:00
) ? ;
2022-10-21 00:00:07 +08:00
let ProcessingTasks { started_at , processing , .. } =
self . processing_tasks . read ( ) . map_err ( | _ | Error ::CorruptedTaskQueue ) ? . clone ( ) ;
2022-10-03 22:15:10 +08:00
2022-10-12 09:21:25 +08:00
let ret = tasks . into_iter ( ) ;
2022-10-03 22:15:10 +08:00
if processing . is_empty ( ) {
Ok ( ret . collect ( ) )
} else {
Ok ( ret
. map ( | task | match processing . contains ( task . uid ) {
2022-10-21 00:00:07 +08:00
true = > {
Task { status : Status ::Processing , started_at : Some ( started_at ) , .. task }
}
2022-10-03 22:15:10 +08:00
false = > task ,
} )
. collect ( ) )
}
}
2022-10-20 16:25:34 +08:00
/// Register a new task in the scheduler.
///
/// If it fails and data was associated with the task, it tries to delete the associated data.
2022-10-17 23:19:17 +08:00
pub fn register ( & self , kind : KindWithContent ) -> Result < Task > {
2022-10-03 22:15:10 +08:00
let mut wtxn = self . env . write_txn ( ) ? ;
2023-04-13 00:46:24 +08:00
// if the task doesn't delete anything and 50% of the task queue is full, we must refuse to enqueue the incomming task
2023-04-07 00:26:27 +08:00
if ! matches! ( & kind , KindWithContent ::TaskDeletion { tasks , .. } if ! tasks . is_empty ( ) )
2023-04-13 23:07:44 +08:00
& & ( self . env . non_free_pages_size ( ) ? * 100 ) / self . env . map_size ( ) ? as u64 > 50
2023-04-07 00:26:27 +08:00
{
return Err ( Error ::NoSpaceLeftInTaskQueue ) ;
}
2022-10-25 22:10:14 +08:00
let mut task = Task {
2022-10-03 22:15:10 +08:00
uid : self . next_task_id ( & wtxn ) ? ,
2022-12-22 18:46:17 +08:00
enqueued_at : OffsetDateTime ::now_utc ( ) ,
2022-10-03 22:15:10 +08:00
started_at : None ,
finished_at : None ,
error : None ,
2022-10-18 19:57:58 +08:00
canceled_by : None ,
2022-10-17 23:19:17 +08:00
details : kind . default_details ( ) ,
2022-10-03 22:15:10 +08:00
status : Status ::Enqueued ,
2022-10-17 23:19:17 +08:00
kind : kind . clone ( ) ,
2022-10-03 22:15:10 +08:00
} ;
2022-10-25 22:10:14 +08:00
// 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 ) ;
2022-10-27 15:41:32 +08:00
// 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 ) ? ;
2022-10-25 22:10:14 +08:00
// Get rid of the mutability.
let task = task ;
2022-10-21 00:00:07 +08:00
self . all_tasks . append ( & mut wtxn , & BEU32 ::new ( task . uid ) , & task ) ? ;
2022-10-03 22:15:10 +08:00
2022-10-25 16:26:51 +08:00
for index in task . indexes ( ) {
self . update_index ( & mut wtxn , index , | bitmap | {
bitmap . insert ( task . uid ) ;
} ) ? ;
2022-10-03 22:15:10 +08:00
}
self . update_status ( & mut wtxn , Status ::Enqueued , | bitmap | {
bitmap . insert ( task . uid ) ;
} ) ? ;
self . update_kind ( & mut wtxn , task . kind . as_kind ( ) , | bitmap | {
2022-10-26 23:31:23 +08:00
bitmap . insert ( task . uid ) ;
2022-10-03 22:15:10 +08:00
} ) ? ;
2022-10-19 18:59:12 +08:00
utils ::insert_task_datetime ( & mut wtxn , self . enqueued_at , task . enqueued_at , task . uid ) ? ;
2022-10-19 17:31:08 +08:00
if let Err ( e ) = wtxn . commit ( ) {
self . delete_persisted_task_data ( & task ) ? ;
return Err ( e . into ( ) ) ;
2022-10-03 22:15:10 +08:00
}
2022-10-17 23:19:17 +08:00
// If the registered task is a task cancelation
// we inform the processing tasks to stop (if necessary).
if let KindWithContent ::TaskCancelation { tasks , .. } = kind {
let tasks_to_cancel = RoaringBitmap ::from_iter ( tasks ) ;
2022-10-21 00:00:07 +08:00
if self . processing_tasks . read ( ) . unwrap ( ) . must_cancel_processing_tasks ( & tasks_to_cancel )
2022-10-19 17:22:59 +08:00
{
self . must_stop_processing . must_stop ( ) ;
}
2022-10-17 23:19:17 +08:00
}
2022-10-10 22:20:35 +08:00
// notify the scheduler loop to execute a new tick
self . wake_up . signal ( ) ;
2022-10-03 22:15:10 +08:00
2022-10-12 09:21:25 +08:00
Ok ( task )
2022-10-03 22:15:10 +08:00
}
2023-01-09 16:36:00 +08:00
/// Register a new task coming from a dump in the scheduler.
/// By taking a mutable ref we're pretty sure no one will ever import a dump while actix is running.
2023-03-29 20:27:40 +08:00
pub fn register_dumped_task ( & mut self ) -> Result < Dump > {
Dump ::new ( self )
2022-10-16 07:39:01 +08:00
}
/// Create a new index without any associated task.
2022-12-21 21:28:00 +08:00
pub fn create_raw_index (
& self ,
name : & str ,
2022-12-22 18:46:17 +08:00
date : Option < ( OffsetDateTime , OffsetDateTime ) > ,
2022-12-21 21:28:00 +08:00
) -> Result < Index > {
2022-10-26 20:19:56 +08:00
let wtxn = self . env . write_txn ( ) ? ;
2022-12-16 15:11:12 +08:00
let index = self . index_mapper . create_index ( wtxn , name , date ) ? ;
2022-10-16 09:14:01 +08:00
Ok ( index )
2022-10-16 07:39:01 +08:00
}
2022-10-20 16:25:34 +08:00
/// 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.
2022-10-16 07:39:01 +08:00
pub fn create_update_file ( & self ) -> Result < ( Uuid , file_store ::File ) > {
2022-10-03 22:15:10 +08:00
Ok ( self . file_store . new_update ( ) ? )
}
2022-10-17 19:11:12 +08:00
2022-10-10 21:51:28 +08:00
#[ cfg(test) ]
2022-10-16 07:39:01 +08:00
pub fn create_update_file_with_uuid ( & self , uuid : u128 ) -> Result < ( Uuid , file_store ::File ) > {
2022-10-11 15:55:03 +08:00
Ok ( self . file_store . new_update_with_uuid ( uuid ) ? )
2022-10-10 21:51:28 +08:00
}
2022-10-03 22:15:10 +08:00
2023-01-25 18:04:29 +08:00
/// The size on disk taken by all the updates files contained in the `IndexScheduler`, in bytes.
2023-01-25 18:20:15 +08:00
pub fn compute_update_file_size ( & self ) -> Result < u64 > {
Ok ( self . file_store . compute_total_size ( ) ? )
2023-01-24 23:17:23 +08:00
}
2022-10-20 16:25:34 +08:00
/// Delete a file from the index scheduler.
///
/// Counterpart to the [`create_update_file`](IndexScheduler::create_update_file) method.
2022-10-03 22:15:10 +08:00
pub fn delete_update_file ( & self , uuid : Uuid ) -> Result < ( ) > {
Ok ( self . file_store . delete ( uuid ) ? )
}
2022-10-20 16:25:34 +08:00
/// Perform one iteration of the run loop.
///
2023-04-25 02:04:50 +08:00
/// 1. See if we need to cleanup the task queue
/// 2. Find the next batch of tasks to be processed.
/// 3. Update the information of these tasks following the start of their processing.
/// 4. Update the in-memory list of processed tasks accordingly.
/// 5. Process the batch:
2022-10-20 16:25:34 +08:00
/// - perform the actions of each batched task
/// - update the information of each batched task following the end
/// of their processing.
2023-04-25 02:04:50 +08:00
/// 6. Reset the in-memory list of processed tasks.
2022-10-10 22:19:23 +08:00
///
/// Returns the number of processed tasks.
2023-01-10 02:30:29 +08:00
fn tick ( & self ) -> Result < TickOutcome > {
2022-10-03 22:15:10 +08:00
#[ cfg(test) ]
2022-10-20 23:11:44 +08:00
{
* self . run_loop_iteration . write ( ) . unwrap ( ) + = 1 ;
self . breakpoint ( Breakpoint ::Start ) ;
}
2022-10-03 22:15:10 +08:00
2023-04-25 02:04:50 +08:00
self . cleanup_task_queue ( ) ? ;
2022-10-24 20:16:14 +08:00
let rtxn = self . env . read_txn ( ) . map_err ( Error ::HeedTransaction ) ? ;
let batch =
match self . create_next_batch ( & rtxn ) . map_err ( | e | Error ::CreateBatch ( Box ::new ( e ) ) ) ? {
Some ( batch ) = > batch ,
2023-01-10 02:30:29 +08:00
None = > return Ok ( TickOutcome ::WaitForSignal ) ,
2022-10-24 20:16:14 +08:00
} ;
2023-01-09 16:36:00 +08:00
let index_uid = batch . index_uid ( ) . map ( ToOwned ::to_owned ) ;
2022-10-19 13:34:10 +08:00
drop ( rtxn ) ;
2022-10-03 22:15:10 +08:00
// 1. store the starting date with the bitmap of processing tasks.
let mut ids = batch . ids ( ) ;
ids . sort_unstable ( ) ;
2022-10-10 22:19:23 +08:00
let processed_tasks = ids . len ( ) ;
2022-10-03 22:15:10 +08:00
let processing_tasks = RoaringBitmap ::from_sorted_iter ( ids . iter ( ) . copied ( ) ) . unwrap ( ) ;
let started_at = OffsetDateTime ::now_utc ( ) ;
2022-10-19 17:22:59 +08:00
// We reset the must_stop flag to be sure that we don't stop processing tasks
self . must_stop_processing . reset ( ) ;
2022-10-21 00:00:07 +08:00
self . processing_tasks . write ( ) . unwrap ( ) . start_processing_at ( started_at , processing_tasks ) ;
2022-10-03 22:15:10 +08:00
#[ cfg(test) ]
2022-10-20 23:11:44 +08:00
self . breakpoint ( Breakpoint ::BatchCreated ) ;
2022-10-03 22:15:10 +08:00
// 2. Process the tasks
2022-10-20 23:11:44 +08:00
let res = {
let cloned_index_scheduler = self . private_clone ( ) ;
2022-11-28 23:27:41 +08:00
let handle = std ::thread ::Builder ::new ( )
. name ( String ::from ( " batch-operation " ) )
. spawn ( move | | cloned_index_scheduler . process_batch ( batch ) )
. unwrap ( ) ;
2022-10-25 16:53:29 +08:00
handle . join ( ) . unwrap_or ( Err ( Error ::ProcessBatchPanicked ) )
2022-10-20 23:11:44 +08:00
} ;
#[ cfg(test) ]
self . maybe_fail ( tests ::FailureLocation ::AcquiringWtxn ) ? ;
2022-10-24 20:16:14 +08:00
let mut wtxn = self . env . write_txn ( ) . map_err ( Error ::HeedTransaction ) ? ;
2022-10-17 22:30:18 +08:00
2022-10-03 22:15:10 +08:00
let finished_at = OffsetDateTime ::now_utc ( ) ;
match res {
Ok ( tasks ) = > {
2022-10-24 20:16:14 +08:00
#[ cfg(test) ]
self . breakpoint ( Breakpoint ::ProcessBatchSucceeded ) ;
2022-11-28 23:27:41 +08:00
2022-10-20 23:11:44 +08:00
#[ allow(unused_variables) ]
for ( i , mut task ) in tasks . into_iter ( ) . enumerate ( ) {
2022-10-03 22:15:10 +08:00
task . started_at = Some ( started_at ) ;
task . finished_at = Some ( finished_at ) ;
2022-10-20 23:11:44 +08:00
#[ cfg(test) ]
self . maybe_fail (
tests ::FailureLocation ::UpdatingTaskAfterProcessBatchSuccess {
task_uid : i as u32 ,
} ,
) ? ;
2022-10-24 20:16:14 +08:00
self . update_task ( & mut wtxn , & task )
. map_err ( | e | Error ::TaskDatabaseUpdate ( Box ::new ( e ) ) ) ? ;
if let Err ( e ) = self . delete_persisted_task_data ( & task ) {
log ::error! ( " Failure to delete the content files associated with task {}. Error: {e} " , task . uid ) ;
}
2022-10-03 22:15:10 +08:00
}
2022-10-13 21:02:59 +08:00
log ::info! ( " A batch of tasks was successfully completed. " ) ;
2022-10-03 22:15:10 +08:00
}
2022-10-17 23:19:17 +08:00
// If we have an abortion error we must stop the tick here and re-schedule tasks.
Err ( Error ::Milli ( milli ::Error ::InternalError (
milli ::InternalError ::AbortedIndexation ,
) ) ) = > {
2022-10-24 20:16:14 +08:00
#[ cfg(test) ]
self . breakpoint ( Breakpoint ::AbortedIndexation ) ;
wtxn . abort ( ) . map_err ( Error ::HeedTransaction ) ? ;
2022-11-28 23:27:41 +08:00
// We make sure that we don't call `stop_processing` on the `processing_tasks`,
// this is because we want to let the next tick call `create_next_batch` and keep
// the `started_at` date times and `processings` of the current processing tasks.
// This date time is used by the task cancelation to store the right `started_at`
// date in the task on disk.
2023-01-10 02:30:29 +08:00
return Ok ( TickOutcome ::TickAgain ( 0 ) ) ;
2022-10-17 23:19:17 +08:00
}
2023-01-09 16:36:00 +08:00
// If an index said it was full, we need to:
// 1. identify which index is full
// 2. close the associated environment
// 3. resize it
// 4. re-schedule tasks
Err ( Error ::Milli ( milli ::Error ::UserError (
milli ::UserError ::MaxDatabaseSizeReached ,
) ) ) if index_uid . is_some ( ) = > {
// fixme: add index_uid to match to avoid the unwrap
let index_uid = index_uid . unwrap ( ) ;
// fixme: handle error more gracefully? not sure when this could happen
self . index_mapper . resize_index ( & wtxn , & index_uid ) ? ;
wtxn . abort ( ) . map_err ( Error ::HeedTransaction ) ? ;
return Ok ( TickOutcome ::TickAgain ( 0 ) ) ;
2022-10-17 23:19:17 +08:00
}
2022-10-03 22:15:10 +08:00
// In case of a failure we must get back and patch all the tasks with the error.
2022-10-05 22:48:43 +08:00
Err ( err ) = > {
2022-10-24 20:16:14 +08:00
#[ cfg(test) ]
self . breakpoint ( Breakpoint ::ProcessBatchFailed ) ;
2022-10-05 22:48:43 +08:00
let error : ResponseError = err . into ( ) ;
2022-10-03 22:15:10 +08:00
for id in ids {
2022-10-24 20:16:14 +08:00
let mut task = self
. get_task ( & wtxn , id )
. map_err ( | e | Error ::TaskDatabaseUpdate ( Box ::new ( e ) ) ) ?
. ok_or ( Error ::CorruptedTaskQueue ) ? ;
2022-10-03 22:15:10 +08:00
task . started_at = Some ( started_at ) ;
task . finished_at = Some ( finished_at ) ;
task . status = Status ::Failed ;
2022-10-05 22:48:43 +08:00
task . error = Some ( error . clone ( ) ) ;
2022-11-28 23:27:41 +08:00
task . details = task . details . map ( | d | d . to_failed ( ) ) ;
2022-10-03 22:15:10 +08:00
2022-10-20 23:11:44 +08:00
#[ cfg(test) ]
self . maybe_fail ( tests ::FailureLocation ::UpdatingTaskAfterProcessBatchFailure ) ? ;
2022-10-24 20:16:14 +08:00
if let Err ( e ) = self . delete_persisted_task_data ( & task ) {
log ::error! ( " Failure to delete the content files associated with task {}. Error: {e} " , task . uid ) ;
}
self . update_task ( & mut wtxn , & task )
. map_err ( | e | Error ::TaskDatabaseUpdate ( Box ::new ( e ) ) ) ? ;
2022-10-03 22:15:10 +08:00
}
}
}
2022-10-26 23:31:23 +08:00
2022-11-28 23:27:41 +08:00
self . processing_tasks . write ( ) . unwrap ( ) . stop_processing ( ) ;
2022-10-20 23:11:44 +08:00
#[ cfg(test) ]
self . maybe_fail ( tests ::FailureLocation ::CommittingWtxn ) ? ;
2022-10-24 20:16:14 +08:00
wtxn . commit ( ) . map_err ( Error ::HeedTransaction ) ? ;
2022-10-03 22:15:10 +08:00
#[ cfg(test) ]
2022-10-20 23:11:44 +08:00
self . breakpoint ( Breakpoint ::AfterProcessing ) ;
2022-10-03 22:15:10 +08:00
2023-01-10 02:30:29 +08:00
Ok ( TickOutcome ::TickAgain ( processed_tasks ) )
2022-10-03 22:15:10 +08:00
}
2022-10-18 21:04:14 +08:00
2023-04-25 02:04:50 +08:00
/// Register a task to cleanup the task queue if needed
fn cleanup_task_queue ( & self ) -> Result < ( ) > {
2023-04-25 23:26:34 +08:00
let rtxn = self . env . read_txn ( ) . map_err ( Error ::HeedTransaction ) ? ;
let nb_tasks = self . all_task_ids ( & rtxn ) ? . len ( ) ;
// if we have less than 1M tasks everything is fine
if nb_tasks < self . max_number_of_tasks as u64 {
2023-04-25 02:04:50 +08:00
return Ok ( ( ) ) ;
}
let finished = self . status . get ( & rtxn , & Status ::Succeeded ) ? . unwrap_or_default ( )
| self . status . get ( & rtxn , & Status ::Failed ) ? . unwrap_or_default ( )
| self . status . get ( & rtxn , & Status ::Canceled ) ? . unwrap_or_default ( ) ;
2023-04-25 23:26:34 +08:00
let to_delete = RoaringBitmap ::from_iter ( finished . into_iter ( ) . rev ( ) . take ( 100_000 ) ) ;
2023-04-25 02:04:50 +08:00
// /!\ 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 {
2023-04-25 19:11:58 +08:00
log ::warn! ( " The task queue is almost full, but no task can be deleted yet. " ) ;
2023-04-25 02:04:50 +08:00
// the only thing we can do is hope that the user tasks are going to finish
return Ok ( ( ) ) ;
}
2023-04-25 19:11:58 +08:00
log ::info! (
2023-04-26 18:02:06 +08:00
" The task queue is almost full. Deleting the oldest {} finished tasks. " ,
2023-04-25 19:11:58 +08:00
to_delete . len ( )
) ;
2023-04-26 19:55:02 +08:00
// it's safe to unwrap here because we checked the len above
let newest_task_id = to_delete . iter ( ) . last ( ) . unwrap ( ) ;
2023-05-04 15:56:48 +08:00
let last_task_to_delete =
self . get_task ( & rtxn , newest_task_id ) ? . ok_or ( Error ::CorruptedTaskQueue ) ? ;
2023-04-26 19:55:02 +08:00
drop ( rtxn ) ;
2023-05-04 15:56:48 +08:00
// 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 ) ;
2023-04-25 02:04:50 +08:00
self . register ( KindWithContent ::TaskDeletion {
query : format ! (
2023-05-04 16:07:59 +08:00
" ?beforeEnqueuedAt={}&statuses=succeeded,failed,canceled " ,
2023-05-04 15:56:48 +08:00
delete_before . format ( & Rfc3339 ) . map_err ( | _ | Error ::CorruptedTaskQueue ) ? ,
2023-04-25 02:04:50 +08:00
) ,
tasks : to_delete ,
} ) ? ;
Ok ( ( ) )
}
2023-02-24 02:31:57 +08:00
pub fn index_stats ( & self , index_uid : & str ) -> Result < IndexStats > {
let is_indexing = self . is_index_processing ( index_uid ) ? ;
let rtxn = self . read_txn ( ) ? ;
let index_stats = self . index_mapper . stats_of ( & rtxn , index_uid ) ? ;
Ok ( IndexStats { is_indexing , inner_stats : index_stats } )
}
2022-10-18 21:04:14 +08:00
pub ( crate ) fn delete_persisted_task_data ( & self , task : & Task ) -> Result < ( ) > {
2022-10-19 17:33:05 +08:00
match task . content_uuid ( ) {
2022-10-25 20:09:01 +08:00
Some ( content_file ) = > self . delete_update_file ( content_file ) ,
2022-10-19 17:33:05 +08:00
None = > Ok ( ( ) ) ,
2022-10-18 21:04:14 +08:00
}
}
2022-10-20 23:11:44 +08:00
2022-10-25 15:48:51 +08:00
/// Blocks the thread until the test handle asks to progress to/through this breakpoint.
///
/// Two messages are sent through the channel for each breakpoint.
/// The first message is `(b, false)` and the second message is `(b, true)`.
///
/// Since the channel has a capacity of zero, the `send` and `recv` calls wait for each other.
/// So when the index scheduler calls `test_breakpoint_sdr.send(b, false)`, it blocks
/// the thread until the test catches up by calling `test_breakpoint_rcv.recv()` enough.
/// From the test side, we call `recv()` repeatedly until we find the message `(breakpoint, false)`.
/// As soon as we find it, the index scheduler is unblocked but then wait again on the call to
/// `test_breakpoint_sdr.send(b, true)`. This message will only be able to send once the
/// test asks to progress to the next `(b2, false)`.
2022-10-20 23:11:44 +08:00
#[ cfg(test) ]
fn breakpoint ( & self , b : Breakpoint ) {
// We send two messages. The first one will sync with the call
// to `handle.wait_until(b)`. The second one will block until the
// the next call to `handle.wait_until(..)`.
self . test_breakpoint_sdr . send ( ( b , false ) ) . unwrap ( ) ;
// This one will only be able to be sent if the test handle stays alive.
// If it fails, then it means that we have exited the test.
// By crashing with `unwrap`, we kill the run loop.
self . test_breakpoint_sdr . send ( ( b , true ) ) . unwrap ( ) ;
}
2022-10-03 22:15:10 +08:00
}
2023-03-29 20:27:40 +08:00
pub struct Dump < ' a > {
index_scheduler : & ' a IndexScheduler ,
wtxn : RwTxn < ' a , ' a > ,
2023-03-29 20:44:15 +08:00
indexes : HashMap < String , RoaringBitmap > ,
statuses : HashMap < Status , RoaringBitmap > ,
kinds : HashMap < Kind , RoaringBitmap > ,
2023-03-29 20:27:40 +08:00
}
impl < ' a > Dump < ' a > {
pub ( crate ) fn new ( index_scheduler : & ' a mut IndexScheduler ) -> Result < Self > {
// While loading a dump no one should be able to access the scheduler thus I can block everything.
let wtxn = index_scheduler . env . write_txn ( ) ? ;
2023-03-29 20:44:15 +08:00
Ok ( Dump {
index_scheduler ,
wtxn ,
indexes : HashMap ::new ( ) ,
statuses : HashMap ::new ( ) ,
kinds : HashMap ::new ( ) ,
} )
2023-03-29 20:27:40 +08:00
}
/// Register a new task coming from a dump in the scheduler.
/// By taking a mutable ref we're pretty sure no one will ever import a dump while actix is running.
pub fn register_dumped_task (
& mut self ,
task : TaskDump ,
content_file : Option < Box < UpdateFile > > ,
) -> Result < Task > {
let content_uuid = match content_file {
Some ( content_file ) if task . status = = Status ::Enqueued = > {
let ( uuid , mut file ) = self . index_scheduler . create_update_file ( ) ? ;
let mut builder = DocumentsBatchBuilder ::new ( file . as_file_mut ( ) ) ;
for doc in content_file {
builder . append_json_object ( & doc ? ) ? ;
}
builder . into_inner ( ) ? ;
file . persist ( ) ? ;
Some ( uuid )
}
// If the task isn't `Enqueued` then just generate a recognisable `Uuid`
// in case we try to open it later.
_ if task . status ! = Status ::Enqueued = > Some ( Uuid ::nil ( ) ) ,
_ = > None ,
} ;
let task = Task {
uid : task . uid ,
enqueued_at : task . enqueued_at ,
started_at : task . started_at ,
finished_at : task . finished_at ,
error : task . error ,
canceled_by : task . canceled_by ,
details : task . details ,
status : task . status ,
kind : match task . kind {
KindDump ::DocumentImport {
primary_key ,
method ,
documents_count ,
allow_index_creation ,
} = > KindWithContent ::DocumentAdditionOrUpdate {
index_uid : task . index_uid . ok_or ( Error ::CorruptedDump ) ? ,
primary_key ,
method ,
content_file : content_uuid . ok_or ( Error ::CorruptedDump ) ? ,
documents_count ,
allow_index_creation ,
} ,
KindDump ::DocumentDeletion { documents_ids } = > KindWithContent ::DocumentDeletion {
documents_ids ,
index_uid : task . index_uid . ok_or ( Error ::CorruptedDump ) ? ,
} ,
2023-03-07 17:02:04 +08:00
KindDump ::DocumentDeletionByFilter { filter } = > {
KindWithContent ::DocumentDeletionByFilter {
2023-05-03 04:36:56 +08:00
filter_expr : filter ,
2023-03-07 17:02:04 +08:00
index_uid : task . index_uid . ok_or ( Error ::CorruptedDump ) ? ,
}
}
2023-03-29 20:27:40 +08:00
KindDump ::DocumentClear = > KindWithContent ::DocumentClear {
index_uid : task . index_uid . ok_or ( Error ::CorruptedDump ) ? ,
} ,
KindDump ::Settings { settings , is_deletion , allow_index_creation } = > {
KindWithContent ::SettingsUpdate {
index_uid : task . index_uid . ok_or ( Error ::CorruptedDump ) ? ,
new_settings : settings ,
is_deletion ,
allow_index_creation ,
}
}
KindDump ::IndexDeletion = > KindWithContent ::IndexDeletion {
index_uid : task . index_uid . ok_or ( Error ::CorruptedDump ) ? ,
} ,
KindDump ::IndexCreation { primary_key } = > KindWithContent ::IndexCreation {
index_uid : task . index_uid . ok_or ( Error ::CorruptedDump ) ? ,
primary_key ,
} ,
KindDump ::IndexUpdate { primary_key } = > KindWithContent ::IndexUpdate {
index_uid : task . index_uid . ok_or ( Error ::CorruptedDump ) ? ,
primary_key ,
} ,
KindDump ::IndexSwap { swaps } = > KindWithContent ::IndexSwap { swaps } ,
KindDump ::TaskCancelation { query , tasks } = > {
KindWithContent ::TaskCancelation { query , tasks }
}
KindDump ::TasksDeletion { query , tasks } = > {
KindWithContent ::TaskDeletion { query , tasks }
}
KindDump ::DumpCreation { keys , instance_uid } = > {
KindWithContent ::DumpCreation { keys , instance_uid }
}
KindDump ::SnapshotCreation = > KindWithContent ::SnapshotCreation ,
} ,
} ;
self . index_scheduler . all_tasks . put ( & mut self . wtxn , & BEU32 ::new ( task . uid ) , & task ) ? ;
for index in task . indexes ( ) {
2023-03-29 20:44:15 +08:00
match self . indexes . get_mut ( index ) {
Some ( bitmap ) = > {
bitmap . insert ( task . uid ) ;
}
None = > {
let mut bitmap = RoaringBitmap ::new ( ) ;
bitmap . insert ( task . uid ) ;
self . indexes . insert ( index . to_string ( ) , bitmap ) ;
}
} ;
2023-03-29 20:27:40 +08:00
}
2023-04-05 17:32:14 +08:00
utils ::insert_task_datetime (
& mut self . wtxn ,
self . index_scheduler . enqueued_at ,
task . enqueued_at ,
task . uid ,
) ? ;
// we can't override the started_at & finished_at, so we must only set it if the tasks is finished and won't change
if matches! ( task . status , Status ::Succeeded | Status ::Failed | Status ::Canceled ) {
if let Some ( started_at ) = task . started_at {
utils ::insert_task_datetime (
& mut self . wtxn ,
self . index_scheduler . started_at ,
started_at ,
task . uid ,
) ? ;
}
if let Some ( finished_at ) = task . finished_at {
utils ::insert_task_datetime (
& mut self . wtxn ,
self . index_scheduler . finished_at ,
finished_at ,
task . uid ,
) ? ;
}
}
2023-03-29 20:44:15 +08:00
self . statuses . entry ( task . status ) . or_insert ( RoaringBitmap ::new ( ) ) . insert ( task . uid ) ;
self . kinds . entry ( task . kind . as_kind ( ) ) . or_insert ( RoaringBitmap ::new ( ) ) . insert ( task . uid ) ;
2023-03-29 20:27:40 +08:00
Ok ( task )
}
/// Commit all the changes and exit the importing dump state
2023-03-29 20:44:15 +08:00
pub fn finish ( mut self ) -> Result < ( ) > {
for ( index , bitmap ) in self . indexes {
self . index_scheduler . index_tasks . put ( & mut self . wtxn , & index , & bitmap ) ? ;
}
for ( status , bitmap ) in self . statuses {
self . index_scheduler . put_status ( & mut self . wtxn , status , & bitmap ) ? ;
}
for ( kind , bitmap ) in self . kinds {
self . index_scheduler . put_kind ( & mut self . wtxn , kind , & bitmap ) ? ;
}
2023-03-29 20:27:40 +08:00
self . wtxn . commit ( ) ? ;
self . index_scheduler . wake_up . signal ( ) ;
2023-03-29 20:44:15 +08:00
2023-03-29 20:27:40 +08:00
Ok ( ( ) )
}
}
2023-01-10 02:30:29 +08:00
/// The outcome of calling the [`IndexScheduler::tick`] function.
pub enum TickOutcome {
/// The scheduler should immediately attempt another `tick`.
///
/// The `usize` field contains the number of processed tasks.
TickAgain ( usize ) ,
/// The scheduler should wait for an external signal before attempting another `tick`.
WaitForSignal ,
}
2023-02-15 19:30:46 +08:00
/// How many indexes we can afford to have open simultaneously.
struct IndexBudget {
/// Map size of an index.
map_size : usize ,
/// Maximum number of simultaneously opened indexes.
index_count : usize ,
/// For very constrained systems we might need to reduce the base task_db_size so we can accept at least one index.
task_db_size : usize ,
}
2023-02-28 22:24:31 +08:00
/// The statistics that can be computed from an `Index` object and the scheduler.
///
/// Compared with `index_mapper::IndexStats`, it adds the scheduling status.
2023-02-24 02:31:57 +08:00
#[ derive(Debug) ]
pub struct IndexStats {
2023-02-28 22:24:31 +08:00
/// Whether this index is currently performing indexation, according to the scheduler.
2023-02-24 02:31:57 +08:00
pub is_indexing : bool ,
2023-02-28 22:24:31 +08:00
/// Internal stats computed from the index.
2023-02-24 02:31:57 +08:00
pub inner_stats : index_mapper ::IndexStats ,
}
2022-09-15 18:23:41 +08:00
#[ cfg(test) ]
mod tests {
2022-11-30 17:59:06 +08:00
use std ::io ::{ BufWriter , Seek , Write } ;
2022-10-24 20:16:14 +08:00
use std ::time ::Instant ;
2022-10-03 22:15:10 +08:00
use big_s ::S ;
2022-11-28 23:27:41 +08:00
use crossbeam ::channel ::RecvTimeoutError ;
2022-10-18 01:24:06 +08:00
use file_store ::File ;
2023-04-26 19:55:02 +08:00
use meili_snap ::{ json_string , snapshot } ;
2023-02-19 21:40:25 +08:00
use meilisearch_auth ::AuthFilter ;
2022-11-30 00:03:22 +08:00
use meilisearch_types ::document_formats ::DocumentFormatError ;
2023-04-25 02:04:50 +08:00
use meilisearch_types ::error ::ErrorCode ;
2023-02-19 21:40:25 +08:00
use meilisearch_types ::index_uid_pattern ::IndexUidPattern ;
2022-10-24 23:29:17 +08:00
use meilisearch_types ::milli ::obkv_to_json ;
use meilisearch_types ::milli ::update ::IndexDocumentsMethod ::{
ReplaceDocuments , UpdateDocuments ,
} ;
2022-10-26 18:57:29 +08:00
use meilisearch_types ::tasks ::IndexSwap ;
2022-10-25 21:51:15 +08:00
use meilisearch_types ::VERSION_FILE_NAME ;
2022-11-30 17:59:06 +08:00
use tempfile ::{ NamedTempFile , TempDir } ;
2022-10-24 19:32:46 +08:00
use time ::Duration ;
2022-10-03 22:15:10 +08:00
use uuid ::Uuid ;
2022-11-28 23:27:41 +08:00
use Breakpoint ::* ;
2022-10-03 22:15:10 +08:00
use super ::* ;
2022-10-25 16:23:14 +08:00
use crate ::insta_snapshot ::{ snapshot_bitmap , snapshot_index_scheduler } ;
2022-10-20 23:11:44 +08:00
#[ derive(Debug, Clone, Copy, PartialEq, Eq) ]
pub enum FailureLocation {
InsideCreateBatch ,
InsideProcessBatch ,
PanicInsideProcessBatch ,
AcquiringWtxn ,
UpdatingTaskAfterProcessBatchSuccess { task_uid : u32 } ,
UpdatingTaskAfterProcessBatchFailure ,
CommittingWtxn ,
}
impl IndexScheduler {
pub fn test (
2022-10-26 17:41:59 +08:00
autobatching_enabled : bool ,
2022-10-20 23:11:44 +08:00
planned_failures : Vec < ( usize , FailureLocation ) > ,
2023-04-25 23:26:34 +08:00
) -> ( Self , IndexSchedulerHandle ) {
Self ::test_with_custom_config ( planned_failures , | config | {
config . autobatching_enabled = autobatching_enabled ;
} )
}
pub fn test_with_custom_config (
planned_failures : Vec < ( usize , FailureLocation ) > ,
configuration : impl Fn ( & mut IndexSchedulerOptions ) ,
2022-10-20 23:11:44 +08:00
) -> ( Self , IndexSchedulerHandle ) {
let tempdir = TempDir ::new ( ) . unwrap ( ) ;
let ( sender , receiver ) = crossbeam ::channel ::bounded ( 0 ) ;
2023-02-15 19:31:14 +08:00
let indexer_config = IndexerConfig { skip_index_budget : true , .. Default ::default ( ) } ;
2023-04-25 23:26:34 +08:00
let mut options = IndexSchedulerOptions {
2022-10-26 17:41:59 +08:00
version_file_path : tempdir . path ( ) . join ( VERSION_FILE_NAME ) ,
auth_path : tempdir . path ( ) . join ( " auth " ) ,
tasks_path : tempdir . path ( ) . join ( " db_path " ) ,
update_file_path : tempdir . path ( ) . join ( " file_store " ) ,
indexes_path : tempdir . path ( ) . join ( " indexes " ) ,
snapshots_path : tempdir . path ( ) . join ( " snapshots " ) ,
dumps_path : tempdir . path ( ) . join ( " dumps " ) ,
2022-11-29 22:26:24 +08:00
task_db_size : 1000 * 1000 , // 1 MB, we don't use MiB on purpose.
2023-01-12 00:34:46 +08:00
index_base_map_size : 1000 * 1000 , // 1 MB, we don't use MiB on purpose.
2023-05-15 17:23:58 +08:00
enable_mdb_writemap : false ,
2023-01-12 00:34:46 +08:00
index_growth_amount : 1000 * 1000 , // 1 MB
index_count : 5 ,
2023-02-15 19:31:14 +08:00
indexer_config ,
2023-04-25 23:26:34 +08:00
autobatching_enabled : true ,
max_number_of_tasks : 1_000_000 ,
2022-10-26 17:41:59 +08:00
} ;
2023-04-25 23:26:34 +08:00
configuration ( & mut options ) ;
2022-10-26 17:41:59 +08:00
let index_scheduler = Self ::new ( options , sender , planned_failures ) . unwrap ( ) ;
2022-10-20 23:11:44 +08:00
2022-11-28 23:27:41 +08:00
// To be 100% consistent between all test we're going to start the scheduler right now
// and ensure it's in the expected starting state.
let breakpoint = match receiver . recv_timeout ( std ::time ::Duration ::from_secs ( 1 ) ) {
Ok ( b ) = > b ,
Err ( RecvTimeoutError ::Timeout ) = > {
panic! ( " The scheduler seems to be waiting for a new task while your test is waiting for a breakpoint. " )
}
Err ( RecvTimeoutError ::Disconnected ) = > panic! ( " The scheduler crashed. " ) ,
} ;
assert_eq! ( breakpoint , ( Init , false ) ) ;
let index_scheduler_handle = IndexSchedulerHandle {
_tempdir : tempdir ,
test_breakpoint_rcv : receiver ,
last_breakpoint : breakpoint . 0 ,
} ;
2022-10-20 23:11:44 +08:00
( index_scheduler , index_scheduler_handle )
}
2023-04-04 03:08:47 +08:00
/// Return a [`PlannedFailure`](Error::PlannedFailure) error if a failure is planned
2022-10-20 23:11:44 +08:00
/// for the given location and current run loop iteration.
pub fn maybe_fail ( & self , location : FailureLocation ) -> Result < ( ) > {
if self . planned_failures . contains ( & ( * self . run_loop_iteration . read ( ) . unwrap ( ) , location ) )
{
match location {
FailureLocation ::PanicInsideProcessBatch = > {
panic! ( " simulated panic " )
}
2023-04-04 03:08:47 +08:00
_ = > Err ( Error ::PlannedFailure ) ,
2022-10-20 23:11:44 +08:00
}
} else {
Ok ( ( ) )
}
}
}
2022-10-03 22:15:10 +08:00
2022-10-10 23:02:28 +08:00
/// Return a `KindWithContent::IndexCreation` task
fn index_creation_task ( index : & 'static str , primary_key : & 'static str ) -> KindWithContent {
2022-10-21 00:00:07 +08:00
KindWithContent ::IndexCreation { index_uid : S ( index ) , primary_key : Some ( S ( primary_key ) ) }
2022-10-10 23:02:28 +08:00
}
/// Create a `KindWithContent::DocumentImport` task that imports documents.
///
/// - `index_uid` is given as parameter
/// - `primary_key` is given as parameter
/// - `method` is set to `ReplaceDocuments`
/// - `content_file` is given as parameter
/// - `documents_count` is given as parameter
/// - `allow_index_creation` is set to `true`
fn replace_document_import_task (
index : & 'static str ,
primary_key : Option < & 'static str > ,
content_file_uuid : u128 ,
documents_count : u64 ,
) -> KindWithContent {
2022-10-22 00:03:10 +08:00
KindWithContent ::DocumentAdditionOrUpdate {
2022-10-10 23:02:28 +08:00
index_uid : S ( index ) ,
primary_key : primary_key . map ( ToOwned ::to_owned ) ,
method : ReplaceDocuments ,
content_file : Uuid ::from_u128 ( content_file_uuid ) ,
2022-10-22 22:35:42 +08:00
documents_count ,
2022-10-10 23:02:28 +08:00
allow_index_creation : true ,
}
}
2022-11-30 00:03:22 +08:00
/// Adapting to the new json reading interface
2022-11-30 17:59:06 +08:00
pub fn read_json (
bytes : & [ u8 ] ,
write : impl Write + Seek ,
2022-12-13 22:10:51 +08:00
) -> std ::result ::Result < u64 , DocumentFormatError > {
2022-11-30 00:03:22 +08:00
let temp_file = NamedTempFile ::new ( ) . unwrap ( ) ;
let mut buffer = BufWriter ::new ( temp_file . reopen ( ) . unwrap ( ) ) ;
2022-12-03 22:48:38 +08:00
buffer . write_all ( bytes ) . unwrap ( ) ;
2022-11-30 00:03:22 +08:00
buffer . flush ( ) . unwrap ( ) ;
meilisearch_types ::document_formats ::read_json ( temp_file . as_file ( ) , write )
}
2022-10-10 23:02:28 +08:00
/// Create an update file with the given file uuid.
///
/// The update file contains just one simple document whose id is given by `document_id`.
///
/// The uuid of the file and its documents count is returned.
fn sample_documents (
index_scheduler : & IndexScheduler ,
file_uuid : u128 ,
document_id : usize ,
) -> ( File , u64 ) {
let content = format! (
r #"
{ {
" id " : " {document_id} "
} } " #
) ;
2022-10-21 00:00:07 +08:00
let ( _uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( file_uuid ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-10 23:02:28 +08:00
( file , documents_count )
}
2022-10-03 22:15:10 +08:00
pub struct IndexSchedulerHandle {
_tempdir : TempDir ,
2022-10-20 23:11:44 +08:00
test_breakpoint_rcv : crossbeam ::channel ::Receiver < ( Breakpoint , bool ) > ,
2022-11-28 23:27:41 +08:00
last_breakpoint : Breakpoint ,
2022-10-03 22:15:10 +08:00
}
impl IndexSchedulerHandle {
2022-11-28 23:27:41 +08:00
/// Advance the scheduler to the next tick.
/// Panic
/// * If the scheduler is waiting for a task to be registered.
/// * If the breakpoint queue is in a bad state.
#[ track_caller ]
fn advance ( & mut self ) -> Breakpoint {
let ( breakpoint_1 , b ) = match self
. test_breakpoint_rcv
. recv_timeout ( std ::time ::Duration ::from_secs ( 5 ) )
{
Ok ( b ) = > b ,
Err ( RecvTimeoutError ::Timeout ) = > {
panic! ( " The scheduler seems to be waiting for a new task while your test is waiting for a breakpoint. " )
}
Err ( RecvTimeoutError ::Disconnected ) = > panic! ( " The scheduler crashed. " ) ,
} ;
// if we've already encountered a breakpoint we're supposed to be stuck on the false
// and we expect the same variant with the true to come now.
assert_eq! (
( breakpoint_1 , b ) ,
( self . last_breakpoint , true ) ,
" Internal error in the test suite. In the previous iteration I got `({:?}, false)` and now I got `({:?}, {:?})`. " ,
self . last_breakpoint ,
breakpoint_1 ,
b ,
) ;
let ( breakpoint_2 , b ) = match self
. test_breakpoint_rcv
. recv_timeout ( std ::time ::Duration ::from_secs ( 5 ) )
{
Ok ( b ) = > b ,
Err ( RecvTimeoutError ::Timeout ) = > {
panic! ( " The scheduler seems to be waiting for a new task while your test is waiting for a breakpoint. " )
}
Err ( RecvTimeoutError ::Disconnected ) = > panic! ( " The scheduler crashed. " ) ,
} ;
assert! ( ! b , " Found the breakpoint handle in a bad state. Check your test suite " ) ;
self . last_breakpoint = breakpoint_2 ;
breakpoint_2
2022-10-03 22:15:10 +08:00
}
2022-10-24 23:29:17 +08:00
2022-11-28 23:27:41 +08:00
/// Advance the scheduler until all the provided breakpoints are reached in order.
#[ track_caller ]
fn advance_till ( & mut self , breakpoints : impl IntoIterator < Item = Breakpoint > ) {
for breakpoint in breakpoints {
let b = self . advance ( ) ;
assert_eq! (
b , breakpoint ,
" Was expecting the breakpoint `{:?}` but instead got `{:?}`. " ,
breakpoint , b
) ;
}
}
/// Wait for `n` successful batches.
#[ track_caller ]
fn advance_n_successful_batches ( & mut self , n : usize ) {
2022-10-24 23:29:17 +08:00
for _ in 0 .. n {
2022-11-28 23:27:41 +08:00
self . advance_one_successful_batch ( ) ;
}
}
/// Wait for `n` failed batches.
#[ track_caller ]
fn advance_n_failed_batches ( & mut self , n : usize ) {
for _ in 0 .. n {
self . advance_one_failed_batch ( ) ;
}
}
// Wait for one successful batch.
#[ track_caller ]
fn advance_one_successful_batch ( & mut self ) {
self . advance_till ( [ Start , BatchCreated ] ) ;
loop {
match self . advance ( ) {
// the process_batch function can call itself recursively, thus we need to
// accept as may InsideProcessBatch as possible before moving to the next state.
InsideProcessBatch = > ( ) ,
// the batch went successfully, we can stop the loop and go on with the next states.
ProcessBatchSucceeded = > break ,
AbortedIndexation = > panic! ( " The batch was aborted. " ) ,
ProcessBatchFailed = > panic! ( " The batch failed. " ) ,
breakpoint = > panic! ( " Encountered an impossible breakpoint ` {:?} `, this is probably an issue with the test suite. " , breakpoint ) ,
}
}
self . advance_till ( [ AfterProcessing ] ) ;
}
// Wait for one failed batch.
#[ track_caller ]
fn advance_one_failed_batch ( & mut self ) {
self . advance_till ( [ Start , BatchCreated ] ) ;
loop {
match self . advance ( ) {
// the process_batch function can call itself recursively, thus we need to
// accept as may InsideProcessBatch as possible before moving to the next state.
InsideProcessBatch = > ( ) ,
// the batch went failed, we can stop the loop and go on with the next states.
ProcessBatchFailed = > break ,
ProcessBatchSucceeded = > panic! ( " The batch succeeded. (and it wasn't supposed to sorry) " ) ,
AbortedIndexation = > panic! ( " The batch was aborted. " ) ,
breakpoint = > panic! ( " Encountered an impossible breakpoint ` {:?} `, this is probably an issue with the test suite. " , breakpoint ) ,
}
2022-10-24 23:29:17 +08:00
}
2022-11-28 23:27:41 +08:00
self . advance_till ( [ AfterProcessing ] ) ;
2022-10-24 23:29:17 +08:00
}
2022-10-03 22:15:10 +08:00
}
#[ test ]
fn register ( ) {
2022-10-10 23:02:28 +08:00
// In this test, the handle doesn't make any progress, we only check that the tasks are registered
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut _handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-03 22:15:10 +08:00
let kinds = [
2022-10-10 23:02:28 +08:00
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 ) ,
2022-10-03 22:15:10 +08:00
] ;
2022-10-20 19:11:50 +08:00
let ( _ , file ) = index_scheduler . create_update_file_with_uuid ( 0 ) . unwrap ( ) ;
file . persist ( ) . unwrap ( ) ;
let ( _ , file ) = index_scheduler . create_update_file_with_uuid ( 1 ) . unwrap ( ) ;
file . persist ( ) . unwrap ( ) ;
let ( _ , file ) = index_scheduler . create_update_file_with_uuid ( 2 ) . unwrap ( ) ;
file . persist ( ) . unwrap ( ) ;
2022-10-03 22:15:10 +08:00
for ( idx , kind ) in kinds . into_iter ( ) . enumerate ( ) {
let k = kind . as_kind ( ) ;
let task = index_scheduler . register ( kind ) . unwrap ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-03 22:15:10 +08:00
assert_eq! ( task . uid , idx as u32 ) ;
assert_eq! ( task . status , Status ::Enqueued ) ;
2022-10-12 09:21:25 +08:00
assert_eq! ( task . kind . as_kind ( ) , k ) ;
2022-10-03 22:15:10 +08:00
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " everything_is_succesfully_registered " ) ;
2022-10-03 22:15:10 +08:00
}
#[ test ]
fn insert_task_while_another_task_is_processing ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-03 22:15:10 +08:00
2022-10-21 00:00:07 +08:00
index_scheduler . register ( index_creation_task ( " index_a " , " id " ) ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start , BatchCreated ] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_batch_creation " ) ;
2022-10-20 19:11:50 +08:00
2022-10-03 22:15:10 +08:00
// while the task is processing can we register another task?
2022-10-21 00:00:07 +08:00
index_scheduler . register ( index_creation_task ( " index_b " , " id " ) ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_second_task " ) ;
2022-10-20 19:11:50 +08:00
2022-10-17 22:30:18 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::IndexDeletion { index_uid : S ( " index_a " ) } )
2022-10-03 22:15:10 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_third_task " ) ;
2022-10-03 22:15:10 +08:00
}
2022-10-10 22:18:35 +08:00
/// We send a lot of tasks but notify the tasks scheduler only once as
/// we send them very fast, we must make sure that they are all processed.
#[ test ]
fn process_tasks_inserted_without_new_signal ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-10 22:18:35 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggos " ) , primary_key : None } )
2022-10-10 22:18:35 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-20 19:11:50 +08:00
2022-10-10 22:18:35 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::IndexCreation { index_uid : S ( " cattos " ) , primary_key : None } )
2022-10-10 22:18:35 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_second_task " ) ;
2022-10-20 19:11:50 +08:00
2022-10-10 22:18:35 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::IndexDeletion { index_uid : S ( " doggos " ) } )
2022-10-10 22:18:35 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_third_task " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " processed_the_first_task " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " processed_the_second_task " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " processed_the_third_task " ) ;
2022-10-10 22:18:35 +08:00
}
2022-10-10 23:00:56 +08:00
#[ test ]
fn process_tasks_without_autobatching ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( false , vec! [ ] ) ;
2022-10-10 23:00:56 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggos " ) , primary_key : None } )
2022-10-10 23:00:56 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-20 19:11:50 +08:00
2022-10-10 23:00:56 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::DocumentClear { index_uid : S ( " doggos " ) } )
2022-10-10 23:00:56 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_second_task " ) ;
2022-10-20 19:11:50 +08:00
2022-10-10 23:00:56 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::DocumentClear { index_uid : S ( " doggos " ) } )
2022-10-10 23:00:56 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_third_task " ) ;
2022-10-20 19:11:50 +08:00
2022-10-10 23:00:56 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::DocumentClear { index_uid : S ( " doggos " ) } )
2022-10-10 23:00:56 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_fourth_task " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " first " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " second " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " third " ) ;
2022-10-10 23:00:56 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " fourth " ) ;
2022-10-10 23:00:56 +08:00
}
2022-10-06 22:53:21 +08:00
#[ test ]
2022-10-10 23:02:28 +08:00
fn task_deletion_undeleteable ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-06 22:53:21 +08:00
2022-10-20 19:11:50 +08:00
let ( file0 , documents_count0 ) = sample_documents ( & index_scheduler , 0 , 0 ) ;
let ( file1 , documents_count1 ) = sample_documents ( & index_scheduler , 1 , 1 ) ;
file0 . persist ( ) . unwrap ( ) ;
file1 . persist ( ) . unwrap ( ) ;
2022-10-06 22:53:21 +08:00
let to_enqueue = [
2022-10-10 23:02:28 +08:00
index_creation_task ( " catto " , " mouse " ) ,
2022-10-20 19:11:50 +08:00
replace_document_import_task ( " catto " , None , 0 , documents_count0 ) ,
replace_document_import_task ( " doggo " , Some ( " bone " ) , 1 , documents_count1 ) ,
2022-10-06 22:53:21 +08:00
] ;
2022-10-20 19:11:50 +08:00
2022-10-06 22:53:21 +08:00
for task in to_enqueue {
let _ = index_scheduler . register ( task ) . unwrap ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-06 22:53:21 +08:00
}
2022-10-10 21:51:28 +08:00
2022-10-10 23:02:28 +08:00
// here we have registered all the tasks, but the index scheduler
// has not progressed at all
2022-10-13 17:09:00 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " initial_tasks_enqueued " ) ;
2022-10-06 22:53:21 +08:00
2022-10-10 23:02:28 +08:00
index_scheduler
2022-10-13 17:09:00 +08:00
. register ( KindWithContent ::TaskDeletion {
2022-10-10 23:02:28 +08:00
query : " test_query " . to_owned ( ) ,
2022-10-25 16:53:29 +08:00
tasks : RoaringBitmap ::from_iter ( [ 0 , 1 ] ) ,
2022-10-10 23:02:28 +08:00
} )
. unwrap ( ) ;
// again, no progress made at all, but one more task is registered
2022-10-13 17:09:00 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " task_deletion_enqueued " ) ;
2022-10-06 22:53:21 +08:00
2022-10-10 23:02:28 +08:00
// now we create the first batch
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start , BatchCreated ] ) ;
2022-10-06 22:53:21 +08:00
2022-10-10 23:02:28 +08:00
// the task deletion should now be "processing"
2022-10-13 17:09:00 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " task_deletion_processing " ) ;
2022-10-06 22:53:21 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ InsideProcessBatch , ProcessBatchSucceeded , AfterProcessing ] ) ;
2022-10-10 23:02:28 +08:00
// after the task deletion is processed, no task should actually have been deleted,
// because the tasks with ids 0 and 1 were still "enqueued", and thus undeleteable
// the "task deletion" task should be marked as "succeeded" and, in its details, the
// number of deleted tasks should be 0
2022-10-13 17:09:00 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " task_deletion_done " ) ;
2022-10-10 23:02:28 +08:00
}
#[ test ]
fn task_deletion_deleteable ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-10 23:02:28 +08:00
let ( file0 , documents_count0 ) = sample_documents ( & index_scheduler , 0 , 0 ) ;
let ( file1 , documents_count1 ) = sample_documents ( & index_scheduler , 1 , 1 ) ;
2022-10-20 19:11:50 +08:00
file0 . persist ( ) . unwrap ( ) ;
file1 . persist ( ) . unwrap ( ) ;
2022-10-10 23:02:28 +08:00
let to_enqueue = [
replace_document_import_task ( " catto " , None , 0 , documents_count0 ) ,
replace_document_import_task ( " doggo " , Some ( " bone " ) , 1 , documents_count1 ) ,
] ;
for task in to_enqueue {
let _ = index_scheduler . register ( task ) . unwrap ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-10 23:02:28 +08:00
}
2022-10-13 17:09:00 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " initial_tasks_enqueued " ) ;
2022-10-10 23:02:28 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-10 23:02:28 +08:00
// first addition of documents should be successful
2022-10-13 17:09:00 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " initial_tasks_processed " ) ;
2022-10-10 23:02:28 +08:00
// Now we delete the first task
index_scheduler
2022-10-13 17:09:00 +08:00
. register ( KindWithContent ::TaskDeletion {
2022-10-10 23:02:28 +08:00
query : " test_query " . to_owned ( ) ,
2022-10-25 16:53:29 +08:00
tasks : RoaringBitmap ::from_iter ( [ 0 ] ) ,
2022-10-10 23:02:28 +08:00
} )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_task_deletion " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-13 17:09:00 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " task_deletion_processed " ) ;
2022-10-06 22:53:21 +08:00
}
2022-10-15 17:38:43 +08:00
#[ test ]
fn task_deletion_delete_same_task_twice ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-15 17:38:43 +08:00
let ( file0 , documents_count0 ) = sample_documents ( & index_scheduler , 0 , 0 ) ;
let ( file1 , documents_count1 ) = sample_documents ( & index_scheduler , 1 , 1 ) ;
2022-10-20 19:11:50 +08:00
file0 . persist ( ) . unwrap ( ) ;
file1 . persist ( ) . unwrap ( ) ;
2022-10-15 17:38:43 +08:00
let to_enqueue = [
replace_document_import_task ( " catto " , None , 0 , documents_count0 ) ,
replace_document_import_task ( " doggo " , Some ( " bone " ) , 1 , documents_count1 ) ,
] ;
for task in to_enqueue {
let _ = index_scheduler . register ( task ) . unwrap ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-15 17:38:43 +08:00
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " initial_tasks_enqueued " ) ;
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-15 17:38:43 +08:00
// first addition of documents should be successful
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " initial_tasks_processed " ) ;
// Now we delete the first task multiple times in a row
for _ in 0 .. 2 {
index_scheduler
. register ( KindWithContent ::TaskDeletion {
query : " test_query " . to_owned ( ) ,
2022-10-25 16:53:29 +08:00
tasks : RoaringBitmap ::from_iter ( [ 0 ] ) ,
2022-10-15 17:38:43 +08:00
} )
. unwrap ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-15 17:38:43 +08:00
}
for _ in 0 .. 2 {
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-15 17:38:43 +08:00
}
2022-10-20 19:11:50 +08:00
2022-10-15 17:38:43 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " task_deletion_processed " ) ;
}
2022-10-03 22:15:10 +08:00
#[ test ]
fn document_addition ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-03 22:15:10 +08:00
let content = r #"
{
" id " : 1 ,
" doggo " : " bob "
} " #;
2022-10-10 21:51:28 +08:00
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( 0 ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-20 19:11:50 +08:00
file . persist ( ) . unwrap ( ) ;
2022-10-03 22:15:10 +08:00
index_scheduler
2022-10-22 00:03:10 +08:00
. register ( KindWithContent ::DocumentAdditionOrUpdate {
2022-10-03 22:15:10 +08:00
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_register " ) ;
2022-10-20 19:11:50 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start , BatchCreated ] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_the_batch_creation " ) ;
2022-10-03 22:15:10 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ InsideProcessBatch , ProcessBatchSucceeded , AfterProcessing ] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " once_everything_is_processed " ) ;
2022-10-03 22:15:10 +08:00
}
2022-10-19 22:44:42 +08:00
#[ test ]
fn document_addition_and_index_deletion ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-19 22:44:42 +08:00
let content = r #"
{
" id " : 1 ,
" doggo " : " bob "
} " #;
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggos " ) , primary_key : None } )
2022-10-19 22:44:42 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-19 22:44:42 +08:00
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( 0 ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-19 22:44:42 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
2022-10-22 00:03:10 +08:00
. register ( KindWithContent ::DocumentAdditionOrUpdate {
2022-10-19 22:44:42 +08:00
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_second_task " ) ;
2022-10-20 19:11:50 +08:00
2022-10-19 22:44:42 +08:00
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::IndexDeletion { index_uid : S ( " doggos " ) } )
2022-10-19 22:44:42 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_third_task " ) ;
2022-10-19 22:44:42 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ; // The index creation.
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " before_index_creation " ) ;
handle . advance_one_successful_batch ( ) ; // // after the execution of the two tasks in a single batch.
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " both_task_succeeded " ) ;
2022-10-19 22:44:42 +08:00
}
2022-10-13 16:57:33 +08:00
#[ test ]
fn do_not_batch_task_of_different_indexes ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-13 16:57:33 +08:00
let index_names = [ " doggos " , " cattos " , " girafos " ] ;
for name in index_names {
index_scheduler
. register ( KindWithContent ::IndexCreation {
index_uid : name . to_string ( ) ,
primary_key : None ,
} )
. unwrap ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-13 16:57:33 +08:00
}
for name in index_names {
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::DocumentClear { index_uid : name . to_string ( ) } )
2022-10-13 16:57:33 +08:00
. unwrap ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-13 16:57:33 +08:00
}
for _ in 0 .. ( index_names . len ( ) * 2 ) {
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-20 19:11:50 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-13 16:57:33 +08:00
}
2022-10-27 17:17:50 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_tasks_processed " ) ;
2022-10-13 16:57:33 +08:00
}
2022-10-17 22:30:18 +08:00
#[ test ]
fn swap_indexes ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-17 22:30:18 +08:00
let to_enqueue = [
index_creation_task ( " a " , " id " ) ,
index_creation_task ( " b " , " id " ) ,
index_creation_task ( " c " , " id " ) ,
index_creation_task ( " d " , " id " ) ,
] ;
for task in to_enqueue {
let _ = index_scheduler . register ( task ) . unwrap ( ) ;
2022-10-24 14:12:03 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-17 22:30:18 +08:00
}
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " create_a " ) ;
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " create_b " ) ;
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " create_c " ) ;
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " create_d " ) ;
2022-10-17 22:30:18 +08:00
index_scheduler
. register ( KindWithContent ::IndexSwap {
2022-10-26 18:57:29 +08:00
swaps : vec ! [
IndexSwap { indexes : ( " a " . to_owned ( ) , " b " . to_owned ( ) ) } ,
IndexSwap { indexes : ( " c " . to_owned ( ) , " d " . to_owned ( ) ) } ,
] ,
2022-10-17 22:30:18 +08:00
} )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " first_swap_registered " ) ;
2022-10-17 22:30:18 +08:00
index_scheduler
2022-10-26 18:57:29 +08:00
. register ( KindWithContent ::IndexSwap {
swaps : vec ! [ IndexSwap { indexes : ( " a " . to_owned ( ) , " c " . to_owned ( ) ) } ] ,
} )
2022-10-17 22:30:18 +08:00
. unwrap ( ) ;
2022-10-27 15:41:32 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " two_swaps_registered " ) ;
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-27 15:41:32 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " first_swap_processed " ) ;
2022-10-24 14:12:03 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-17 22:30:18 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " second_swap_processed " ) ;
2022-10-25 16:26:51 +08:00
index_scheduler . register ( KindWithContent ::IndexSwap { swaps : vec ! [ ] } ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-25 16:26:51 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " third_empty_swap_processed " ) ;
2022-10-17 22:30:18 +08:00
}
2022-10-27 15:41:32 +08:00
#[ test ]
fn swap_indexes_errors ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-27 15:41:32 +08:00
let to_enqueue = [
index_creation_task ( " a " , " id " ) ,
index_creation_task ( " b " , " id " ) ,
index_creation_task ( " c " , " id " ) ,
index_creation_task ( " d " , " id " ) ,
] ;
for task in to_enqueue {
let _ = index_scheduler . register ( task ) . unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 4 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_the_index_creation " ) ;
2022-10-27 15:41:32 +08:00
let first_snap = snapshot_index_scheduler ( & index_scheduler ) ;
snapshot! ( first_snap , name : " initial_tasks_processed " ) ;
let err = index_scheduler
. register ( KindWithContent ::IndexSwap {
swaps : vec ! [
IndexSwap { indexes : ( " a " . to_owned ( ) , " b " . to_owned ( ) ) } ,
IndexSwap { indexes : ( " b " . to_owned ( ) , " a " . to_owned ( ) ) } ,
] ,
} )
. unwrap_err ( ) ;
snapshot! ( format! ( " {err} " ) , @ " Indexes must be declared only once during a swap. `a`, `b` were specified several times. " ) ;
let second_snap = snapshot_index_scheduler ( & index_scheduler ) ;
assert_eq! ( first_snap , second_snap ) ;
// Index `e` does not exist, but we don't check its existence yet
index_scheduler
. register ( KindWithContent ::IndexSwap {
swaps : vec ! [
IndexSwap { indexes : ( " a " . to_owned ( ) , " b " . to_owned ( ) ) } ,
IndexSwap { indexes : ( " c " . to_owned ( ) , " e " . to_owned ( ) ) } ,
IndexSwap { indexes : ( " d " . to_owned ( ) , " f " . to_owned ( ) ) } ,
] ,
} )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
handle . advance_one_failed_batch ( ) ;
2022-10-27 15:41:32 +08:00
// Now the first swap should have an error message saying `e` and `f` do not exist
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " first_swap_failed " ) ;
}
2022-10-20 19:18:25 +08:00
#[ test ]
fn document_addition_and_index_deletion_on_unexisting_index ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-20 19:18:25 +08:00
let content = r #"
{
" id " : 1 ,
" doggo " : " bob "
} " #;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( 0 ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-20 19:18:25 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
2022-10-22 00:03:10 +08:00
. register ( KindWithContent ::DocumentAdditionOrUpdate {
2022-10-20 19:18:25 +08:00
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
index_scheduler
2022-10-21 00:00:07 +08:00
. register ( KindWithContent ::IndexDeletion { index_uid : S ( " doggos " ) } )
2022-10-20 19:18:25 +08:00
. unwrap ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) ) ;
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 1 ) ;
2022-10-20 19:18:25 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) ) ;
}
2022-10-25 17:42:14 +08:00
#[ test ]
fn cancel_enqueued_task ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-25 17:42:14 +08:00
let ( file0 , documents_count0 ) = sample_documents ( & index_scheduler , 0 , 0 ) ;
file0 . persist ( ) . unwrap ( ) ;
let to_enqueue = [
replace_document_import_task ( " catto " , None , 0 , documents_count0 ) ,
KindWithContent ::TaskCancelation {
query : " test_query " . to_owned ( ) ,
tasks : RoaringBitmap ::from_iter ( [ 0 ] ) ,
} ,
] ;
for task in to_enqueue {
let _ = index_scheduler . register ( task ) . unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " initial_tasks_enqueued " ) ;
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-25 17:42:14 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " cancel_processed " ) ;
}
#[ test ]
fn cancel_succeeded_task ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-25 17:42:14 +08:00
let ( file0 , documents_count0 ) = sample_documents ( & index_scheduler , 0 , 0 ) ;
file0 . persist ( ) . unwrap ( ) ;
let _ = index_scheduler
. register ( replace_document_import_task ( " catto " , None , 0 , documents_count0 ) )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-25 17:42:14 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-25 17:42:14 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " initial_task_processed " ) ;
index_scheduler
. register ( KindWithContent ::TaskCancelation {
query : " test_query " . to_owned ( ) ,
tasks : RoaringBitmap ::from_iter ( [ 0 ] ) ,
} )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-25 17:42:14 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " cancel_processed " ) ;
}
#[ test ]
fn cancel_processing_task ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-25 17:42:14 +08:00
let ( file0 , documents_count0 ) = sample_documents ( & index_scheduler , 0 , 0 ) ;
file0 . persist ( ) . unwrap ( ) ;
let _ = index_scheduler
. register ( replace_document_import_task ( " catto " , None , 0 , documents_count0 ) )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-25 17:42:14 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start , BatchCreated , InsideProcessBatch ] ) ;
2022-10-25 17:42:14 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " initial_task_processing " ) ;
index_scheduler
. register ( KindWithContent ::TaskCancelation {
query : " test_query " . to_owned ( ) ,
tasks : RoaringBitmap ::from_iter ( [ 0 ] ) ,
} )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " cancel_task_registered " ) ;
2022-10-25 17:42:14 +08:00
// Now we check that we can reach the AbortedIndexation error handling
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ AbortedIndexation ] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " aborted_indexation " ) ;
2022-10-25 17:42:14 +08:00
2022-11-28 23:27:41 +08:00
// handle.advance_till([Start, BatchCreated, BeforeProcessing, AfterProcessing]);
handle . advance_one_successful_batch ( ) ;
2022-10-25 17:42:14 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " cancel_processed " ) ;
}
#[ test ]
fn cancel_mix_of_tasks ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-25 17:42:14 +08:00
let ( file0 , documents_count0 ) = sample_documents ( & index_scheduler , 0 , 0 ) ;
file0 . persist ( ) . unwrap ( ) ;
let ( file1 , documents_count1 ) = sample_documents ( & index_scheduler , 1 , 1 ) ;
file1 . persist ( ) . unwrap ( ) ;
let ( file2 , documents_count2 ) = sample_documents ( & index_scheduler , 2 , 2 ) ;
file2 . persist ( ) . unwrap ( ) ;
let to_enqueue = [
replace_document_import_task ( " catto " , None , 0 , documents_count0 ) ,
replace_document_import_task ( " beavero " , None , 1 , documents_count1 ) ,
replace_document_import_task ( " wolfo " , None , 2 , documents_count2 ) ,
] ;
for task in to_enqueue {
let _ = index_scheduler . register ( task ) . unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-25 17:42:14 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " first_task_processed " ) ;
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start , BatchCreated , InsideProcessBatch ] ) ;
2022-10-25 17:42:14 +08:00
index_scheduler
. register ( KindWithContent ::TaskCancelation {
query : " test_query " . to_owned ( ) ,
tasks : RoaringBitmap ::from_iter ( [ 0 , 1 , 2 ] ) ,
} )
. unwrap ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " processing_second_task_cancel_enqueued " ) ;
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ AbortedIndexation ] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " aborted_indexation " ) ;
2022-10-25 17:42:14 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
2022-10-25 17:42:14 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " cancel_processed " ) ;
}
2022-10-24 23:29:17 +08:00
#[ test ]
fn test_document_replace ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-24 23:29:17 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-24 23:29:17 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-24 23:29:17 +08:00
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) ) ;
// everything should be batched together.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 1 ) ;
2022-10-24 23:29:17 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) ) ;
// has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-24 23:29:17 +08:00
}
#[ test ]
fn test_document_update ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-24 23:29:17 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-24 23:29:17 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : UpdateDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-24 23:29:17 +08:00
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) ) ;
// everything should be batched together.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 1 ) ;
2022-10-24 23:29:17 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) ) ;
// has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-24 23:29:17 +08:00
}
#[ test ]
fn test_mixed_document_addition ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-24 23:29:17 +08:00
for i in 0 .. 10 {
let method = if i % 2 = = 0 { UpdateDocuments } else { ReplaceDocuments } ;
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-24 23:29:17 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-24 23:29:17 +08:00
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-24 23:29:17 +08:00
// Only half of the task should've been processed since we can't autobatch replace and update together.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " five_tasks_processed " ) ;
2022-10-24 23:29:17 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_tasks_processed " ) ;
2022-10-24 23:29:17 +08:00
// has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-24 23:29:17 +08:00
}
#[ test ]
fn test_document_replace_without_autobatching ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( false , vec! [ ] ) ;
2022-10-24 23:29:17 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-24 23:29:17 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-24 23:29:17 +08:00
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-24 23:29:17 +08:00
// Nothing should be batched thus half of the tasks are processed.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " five_tasks_processed " ) ;
2022-10-24 23:29:17 +08:00
// Everything is processed.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_tasks_processed " ) ;
2022-10-24 23:29:17 +08:00
// has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-24 23:29:17 +08:00
}
#[ test ]
fn test_document_update_without_autobatching ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( false , vec! [ ] ) ;
2022-10-24 23:29:17 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-24 23:29:17 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : UpdateDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-24 23:29:17 +08:00
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-24 23:29:17 +08:00
// Nothing should be batched thus half of the tasks are processed.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " five_tasks_processed " ) ;
2022-10-24 23:29:17 +08:00
// Everything is processed.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_tasks_processed " ) ;
2022-10-24 23:29:17 +08:00
// has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
2022-10-25 18:30:56 +08:00
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-25 18:30:56 +08:00
}
2022-10-25 21:30:36 +08:00
#[ macro_export ]
macro_rules ! debug_snapshot {
( $value :expr , @ $snapshot :literal ) = > { {
let value = format! ( " {:?} " , $value ) ;
meili_snap ::snapshot! ( value , @ $snapshot ) ;
} } ;
}
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
#[ test ]
fn simple_new ( ) {
crate ::IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-25 18:30:56 +08:00
}
2022-10-26 18:56:01 +08:00
#[ test ]
fn query_tasks_from_and_limit ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-26 18:56:01 +08:00
let kind = index_creation_task ( " doggo " , " bone " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-26 18:56:01 +08:00
let kind = index_creation_task ( " whalo " , " plankton " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_second_task " ) ;
2022-10-26 18:56:01 +08:00
let kind = index_creation_task ( " catto " , " his_own_vomit " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_third_task " ) ;
2022-10-26 18:56:01 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 3 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " processed_all_tasks " ) ;
2022-10-26 18:56:01 +08:00
2022-10-27 17:17:50 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
2022-10-26 18:56:01 +08:00
let query = Query { limit : Some ( 0 ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-26 18:56:01 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [] " ) ;
let query = Query { limit : Some ( 1 ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-26 18:56:01 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [2,] " ) ;
let query = Query { limit : Some ( 2 ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-26 18:56:01 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [1,2,] " ) ;
let query = Query { from : Some ( 1 ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-26 18:56:01 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [0,1,] " ) ;
let query = Query { from : Some ( 2 ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-26 18:56:01 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [0,1,2,] " ) ;
let query = Query { from : Some ( 1 ) , limit : Some ( 1 ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-26 18:56:01 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [1,] " ) ;
let query = Query { from : Some ( 1 ) , limit : Some ( 2 ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-26 18:56:01 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [0,1,] " ) ;
}
2022-10-25 18:30:56 +08:00
#[ test ]
2022-10-27 17:17:50 +08:00
fn query_tasks_simple ( ) {
2022-10-25 21:30:36 +08:00
let start_time = OffsetDateTime ::now_utc ( ) ;
2022-10-25 18:30:56 +08:00
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) =
2022-10-25 21:30:36 +08:00
IndexScheduler ::test ( true , vec! [ ( 3 , FailureLocation ::InsideProcessBatch ) ] ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
let kind = index_creation_task ( " catto " , " mouse " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
let kind = index_creation_task ( " doggo " , " sheep " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
let kind = index_creation_task ( " whalo " , " fish " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " start " ) ;
2022-10-25 18:30:56 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start , BatchCreated ] ) ;
2022-10-25 18:30:56 +08:00
2022-10-27 17:17:50 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
let query = Query { statuses : Some ( vec! [ Status ::Processing ] ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [0,] " ) ; // only the processing tasks in the first tick
2022-10-25 18:30:56 +08:00
2022-11-28 23:27:41 +08:00
let query = Query { statuses : Some ( vec! [ Status ::Enqueued ] ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [1,2,] " ) ; // only the enqueued tasks in the first tick
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Enqueued , Status ::Processing ] ) ,
2022-10-25 21:30:36 +08:00
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [0,1,2,] " ) ; // both enqueued and processing tasks in the first tick
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Enqueued , Status ::Processing ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( start_time ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// 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,] " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Enqueued , Status ::Processing ] ) ,
2022-10-25 21:30:36 +08:00
before_started_at : Some ( start_time ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// 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 ) , @ " [] " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Enqueued , Status ::Processing ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( start_time ) ,
before_started_at : Some ( start_time + Duration ::minutes ( 1 ) ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// 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,] " ) ;
2022-10-25 18:30:56 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [
InsideProcessBatch ,
InsideProcessBatch ,
ProcessBatchSucceeded ,
AfterProcessing ,
Start ,
BatchCreated ,
] ) ;
2022-10-25 18:30:56 +08:00
2022-10-27 17:17:50 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
2022-10-25 21:30:36 +08:00
let second_start_time = OffsetDateTime ::now_utc ( ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Succeeded , Status ::Processing ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( start_time ) ,
before_started_at : Some ( start_time + Duration ::minutes ( 1 ) ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// 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,] " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Succeeded , Status ::Processing ] ) ,
2022-10-25 21:30:36 +08:00
before_started_at : Some ( start_time ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// 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 ) , @ " [] " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Enqueued , Status ::Succeeded , Status ::Processing ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( second_start_time ) ,
before_started_at : Some ( second_start_time + Duration ::minutes ( 1 ) ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// 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 ) , @ " [] " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
// now we make one more batch, the started_at field of the new tasks will be past `second_start_time`
2022-11-28 23:27:41 +08:00
handle . advance_till ( [
InsideProcessBatch ,
InsideProcessBatch ,
ProcessBatchSucceeded ,
AfterProcessing ,
Start ,
BatchCreated ,
] ) ;
2022-10-27 17:17:50 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// 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 {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Enqueued , Status ::Succeeded , Status ::Processing ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( second_start_time ) ,
before_started_at : Some ( second_start_time + Duration ::minutes ( 1 ) ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// 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,] " ) ;
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ ProcessBatchFailed , AfterProcessing ] ) ;
2022-10-27 17:17:50 +08:00
let rtxn = index_scheduler . read_txn ( ) . unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// now the last task should have failed
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " end " ) ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// so running the last query should return nothing
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [] " ) ;
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Failed ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( second_start_time ) ,
before_started_at : Some ( second_start_time + Duration ::minutes ( 1 ) ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// but the same query on failed tasks should return the last task
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [2,] " ) ;
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Failed ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( second_start_time ) ,
before_started_at : Some ( second_start_time + Duration ::minutes ( 1 ) ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// but the same query on failed tasks should return the last task
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [2,] " ) ;
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Failed ] ) ,
uids : Some ( vec! [ 1 ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( second_start_time ) ,
before_started_at : Some ( second_start_time + Duration ::minutes ( 1 ) ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// same query but with an invalid uid
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [] " ) ;
let query = Query {
2022-11-28 23:27:41 +08:00
statuses : Some ( vec! [ Status ::Failed ] ) ,
uids : Some ( vec! [ 2 ] ) ,
2022-10-25 21:30:36 +08:00
after_started_at : Some ( second_start_time ) ,
before_started_at : Some ( second_start_time + Duration ::minutes ( 1 ) ) ,
.. Default ::default ( )
} ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
// same query but with a valid uid
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [2,] " ) ;
}
2022-10-25 18:30:56 +08:00
2022-10-27 17:17:50 +08:00
#[ test ]
fn query_tasks_special_rules ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) =
2022-10-27 17:17:50 +08:00
IndexScheduler ::test ( true , vec! [ ( 3 , FailureLocation ::InsideProcessBatch ) ] ) ;
let kind = index_creation_task ( " catto " , " mouse " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
let kind = index_creation_task ( " doggo " , " sheep " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
let kind = KindWithContent ::IndexSwap {
swaps : vec ! [ IndexSwap { indexes : ( " catto " . to_owned ( ) , " doggo " . to_owned ( ) ) } ] ,
} ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
let kind = KindWithContent ::IndexSwap {
swaps : vec ! [ IndexSwap { indexes : ( " catto " . to_owned ( ) , " whalo " . to_owned ( ) ) } ] ,
} ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " start " ) ;
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start , BatchCreated ] ) ;
2022-10-27 17:17:50 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
let query = Query { index_uids : Some ( vec! [ " catto " . to_owned ( ) ] ) , .. Default ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-27 17:17:50 +08:00
// only the first task associated with catto is returned, the indexSwap tasks are excluded!
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [0,] " ) ;
2022-11-28 23:27:41 +08:00
let query = Query { index_uids : Some ( vec! [ " catto " . to_owned ( ) ] ) , .. Default ::default ( ) } ;
2022-10-27 17:17:50 +08:00
let tasks = index_scheduler
2023-02-02 01:21:45 +08:00
. get_task_ids_from_authorized_indexes (
& rtxn ,
& query ,
2023-02-19 21:40:25 +08:00
& AuthFilter ::with_allowed_indexes (
vec! [ IndexUidPattern ::new_unchecked ( " doggo " ) ] . into_iter ( ) . collect ( ) ,
) ,
2023-02-02 01:21:45 +08:00
)
2022-10-27 17:17:50 +08:00
. 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
2023-02-02 01:21:45 +08:00
. get_task_ids_from_authorized_indexes (
& rtxn ,
& query ,
2023-02-19 21:40:25 +08:00
& AuthFilter ::with_allowed_indexes (
vec! [ IndexUidPattern ::new_unchecked ( " doggo " ) ] . into_iter ( ) . collect ( ) ,
) ,
2023-02-02 01:21:45 +08:00
)
2022-10-27 17:17:50 +08:00
. 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
. get_task_ids_from_authorized_indexes (
& rtxn ,
& query ,
2023-02-19 21:40:25 +08:00
& AuthFilter ::with_allowed_indexes (
vec! [
IndexUidPattern ::new_unchecked ( " catto " ) ,
IndexUidPattern ::new_unchecked ( " doggo " ) ,
]
. into_iter ( )
. collect ( ) ,
) ,
2022-10-27 17:17:50 +08:00
)
. 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
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [0,1,] " ) ;
2022-10-27 17:17:50 +08:00
let query = Query ::default ( ) ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-10-27 17:17:50 +08:00
// we asked for all the tasks with all index authorized -> all tasks returned
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [0,1,2,3,] " ) ;
}
2022-10-25 18:30:56 +08:00
#[ test ]
2022-11-28 23:27:41 +08:00
fn query_tasks_canceled_by ( ) {
let ( index_scheduler , mut handle ) =
IndexScheduler ::test ( true , vec! [ ( 3 , FailureLocation ::InsideProcessBatch ) ] ) ;
2022-10-25 18:30:56 +08:00
2022-11-28 23:27:41 +08:00
let kind = index_creation_task ( " catto " , " mouse " ) ;
let _ = index_scheduler . register ( kind ) . unwrap ( ) ;
let kind = index_creation_task ( " doggo " , " sheep " ) ;
let _ = index_scheduler . register ( kind ) . unwrap ( ) ;
let kind = KindWithContent ::IndexSwap {
swaps : vec ! [ IndexSwap { indexes : ( " catto " . to_owned ( ) , " doggo " . to_owned ( ) ) } ] ,
} ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
2022-10-25 21:30:36 +08:00
2022-11-28 23:27:41 +08:00
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 ) . unwrap ( ) ;
handle . advance_n_successful_batches ( 1 ) ;
2022-10-25 21:30:36 +08:00
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " start " ) ;
let rtxn = index_scheduler . read_txn ( ) . unwrap ( ) ;
let query = Query { canceled_by : Some ( vec! [ task_cancelation . uid ] ) , .. Query ::default ( ) } ;
2023-02-19 21:40:25 +08:00
let tasks = index_scheduler
. get_task_ids_from_authorized_indexes ( & rtxn , & query , & AuthFilter ::default ( ) )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
// 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
2023-02-02 01:21:45 +08:00
. get_task_ids_from_authorized_indexes (
& rtxn ,
& query ,
2023-02-19 21:40:25 +08:00
& AuthFilter ::with_allowed_indexes (
vec! [ IndexUidPattern ::new_unchecked ( " doggo " ) ] . into_iter ( ) . collect ( ) ,
) ,
2023-02-02 01:21:45 +08:00
)
2022-11-28 23:27:41 +08:00
. unwrap ( ) ;
// Return only 1 because the user is not authorized to see task 2
snapshot! ( snapshot_bitmap ( & tasks ) , @ " [1,] " ) ;
2022-10-25 21:30:36 +08:00
}
#[ test ]
fn fail_in_process_batch_for_index_creation ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) =
2022-10-25 21:30:36 +08:00
IndexScheduler ::test ( true , vec! [ ( 1 , FailureLocation ::InsideProcessBatch ) ] ) ;
let kind = index_creation_task ( " catto " , " mouse " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_register " ) ;
2022-10-25 21:30:36 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_one_failed_batch ( ) ;
2022-10-25 21:30:36 +08:00
// Still in the first iteration
assert_eq! ( * index_scheduler . run_loop_iteration . read ( ) . unwrap ( ) , 1 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " index_creation_failed " ) ;
}
#[ test ]
fn fail_in_process_batch_for_document_addition ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) =
2022-10-25 21:30:36 +08:00
IndexScheduler ::test ( true , vec! [ ( 1 , FailureLocation ::InsideProcessBatch ) ] ) ;
let content = r #"
{
" id " : 1 ,
" doggo " : " bob "
} " #;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( 0 ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-25 21:30:36 +08:00
file . persist ( ) . unwrap ( ) ;
2022-10-25 18:30:56 +08:00
index_scheduler
2022-10-25 21:30:36 +08:00
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
2022-10-25 18:30:56 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
handle . advance_till ( [ Start , BatchCreated ] ) ;
2022-10-25 21:30:36 +08:00
snapshot! (
snapshot_index_scheduler ( & index_scheduler ) ,
name : " document_addition_batch_created "
) ;
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ ProcessBatchFailed , AfterProcessing ] ) ;
2022-10-25 21:30:36 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " document_addition_failed " ) ;
}
#[ test ]
fn fail_in_update_task_after_process_batch_success_for_document_addition ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test (
2022-10-25 21:30:36 +08:00
true ,
vec! [ ( 1 , FailureLocation ::UpdatingTaskAfterProcessBatchSuccess { task_uid : 0 } ) ] ,
) ;
let content = r #"
{
" id " : 1 ,
" doggo " : " bob "
} " #;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( 0 ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-25 21:30:36 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-25 21:30:36 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start ] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " document_addition_succeeded_but_index_scheduler_not_updated " ) ;
2022-10-25 21:30:36 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ BatchCreated , InsideProcessBatch , ProcessBatchSucceeded ] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_batch_succeeded " ) ;
2022-10-25 21:30:36 +08:00
2022-11-28 23:27:41 +08:00
// At this point the next time the scheduler will try to progress it should encounter
// a critical failure and have to wait for 1s before retrying anything.
2022-10-25 21:30:36 +08:00
2022-11-28 23:27:41 +08:00
let before_failure = Instant ::now ( ) ;
handle . advance_till ( [ Start ] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_failing_to_commit " ) ;
let failure_duration = before_failure . elapsed ( ) ;
assert! ( failure_duration . as_millis ( ) > = 1000 ) ;
2022-10-25 21:30:36 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [
BatchCreated ,
InsideProcessBatch ,
ProcessBatchSucceeded ,
AfterProcessing ,
] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " task_successfully_processed " ) ;
2022-10-25 21:30:36 +08:00
}
#[ test ]
fn test_document_addition_cant_create_index_without_index ( ) {
// We're going to autobatch multiple document addition that don't have
// the right to create an index while there is no index currently.
// Thus, everything should be batched together and a IndexDoesNotExists
// error should be throwed.
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-25 18:30:56 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-25 18:30:56 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : false ,
} )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-25 18:30:56 +08:00
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
// Everything should be batched together.
2022-11-28 23:27:41 +08:00
handle . advance_till ( [
Start ,
BatchCreated ,
InsideProcessBatch ,
ProcessBatchFailed ,
AfterProcessing ,
] ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_processing_the_10_tasks " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
// The index should not exists.
snapshot! ( format! ( " {} " , index_scheduler . index ( " doggos " ) . map ( | _ | ( ) ) . unwrap_err ( ) ) , @ " Index `doggos` not found. " ) ;
2022-10-25 18:30:56 +08:00
}
#[ test ]
2022-10-25 21:30:36 +08:00
fn test_document_addition_cant_create_index_without_index_without_autobatching ( ) {
// We're going to execute multiple document addition that don't have
// the right to create an index while there is no index currently.
// Since the autobatching is disabled, every tasks should be processed
// sequentially and throw an IndexDoesNotExists.
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( false , vec! [ ] ) ;
2022-10-25 18:30:56 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-25 18:30:56 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
2022-10-25 21:30:36 +08:00
allow_index_creation : false ,
2022-10-25 18:30:56 +08:00
} )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-25 18:30:56 +08:00
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
// Nothing should be batched thus half of the tasks are processed.
2022-11-28 23:27:41 +08:00
handle . advance_n_failed_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " five_tasks_processed " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
// Everything is processed.
2022-11-28 23:27:41 +08:00
handle . advance_n_failed_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_tasks_processed " ) ;
2022-10-25 21:30:36 +08:00
// The index should not exists.
snapshot! ( format! ( " {} " , index_scheduler . index ( " doggos " ) . map ( | _ | ( ) ) . unwrap_err ( ) ) , @ " Index `doggos` not found. " ) ;
2022-10-25 18:30:56 +08:00
}
#[ test ]
2022-10-25 21:30:36 +08:00
fn test_document_addition_cant_create_index_with_index ( ) {
// We're going to autobatch multiple document addition that don't have
// the right to create an index while there is already an index.
// Thus, everything should be batched together and no error should be
// throwed.
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
// Create the index.
index_scheduler
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggos " ) , primary_key : None } )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " processed_the_first_task " ) ;
2022-10-25 21:30:36 +08:00
2022-10-25 18:30:56 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-25 18:30:56 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
2022-10-25 21:30:36 +08:00
allow_index_creation : false ,
2022-10-25 18:30:56 +08:00
} )
. unwrap ( ) ;
2022-10-25 21:30:36 +08:00
index_scheduler . assert_internally_consistent ( ) ;
2022-10-25 18:30:56 +08:00
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-25 18:30:56 +08:00
2022-10-25 21:30:36 +08:00
// Everything should be batched together.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 1 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_processing_the_10_tasks " ) ;
2022-10-25 18:30:56 +08:00
// Has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
2022-10-24 23:29:17 +08:00
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-24 23:29:17 +08:00
}
2022-09-21 18:01:46 +08:00
#[ test ]
2022-10-25 21:30:36 +08:00
fn test_document_addition_cant_create_index_with_index_without_autobatching ( ) {
// We're going to execute multiple document addition that don't have
// the right to create an index while there is no index currently.
// Since the autobatching is disabled, every tasks should be processed
// sequentially and throw an IndexDoesNotExists.
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( false , vec! [ ] ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
// Create the index.
index_scheduler
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggos " ) , primary_key : None } )
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " processed_the_first_task " ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-25 21:30:36 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : false ,
} )
. unwrap ( ) ;
2022-10-20 23:11:44 +08:00
index_scheduler . assert_internally_consistent ( ) ;
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
// Nothing should be batched thus half of the tasks are processed.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " five_tasks_processed " ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
// Everything is processed.
2022-11-28 23:27:41 +08:00
handle . advance_n_successful_batches ( 5 ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_tasks_processed " ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
// Has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-20 23:11:44 +08:00
}
#[ test ]
2022-10-25 21:30:36 +08:00
fn test_document_addition_mixed_rights_with_index ( ) {
// We're going to autobatch multiple document addition.
// - The index already exists
// - The first document addition don't have the right to create an index
// can it batch with the other one?
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
// Create the index.
2022-10-20 23:11:44 +08:00
index_scheduler
2022-10-25 21:30:36 +08:00
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggos " ) , primary_key : None } )
2022-10-20 23:11:44 +08:00
. unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " processed_the_first_task " ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
2022-10-26 21:14:46 +08:00
let allow_index_creation = i % 2 ! = 0 ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-25 21:30:36 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation ,
} )
. unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-25 21:30:36 +08:00
// Everything should be batched together.
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_tasks_processed " ) ;
2022-10-25 21:30:36 +08:00
// Has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-20 23:11:44 +08:00
}
#[ test ]
2022-10-25 21:30:36 +08:00
fn test_document_addition_mixed_right_without_index_starts_with_cant_create ( ) {
// We're going to autobatch multiple document addition.
// - The index does not exists
// - The first document addition don't have the right to create an index
// - The second do. They should not batch together.
// - The second should batch with everything else as it's going to create an index.
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
for i in 0 .. 10 {
let content = format! (
r #" {{
" id " : { } ,
" doggo " : " bob {} "
} } " #,
i , i
) ;
2022-10-26 21:14:46 +08:00
let allow_index_creation = i % 2 ! = 0 ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
let ( uuid , mut file ) = index_scheduler . create_update_file_with_uuid ( i ) . unwrap ( ) ;
2022-12-19 21:12:26 +08:00
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
2022-10-25 21:30:36 +08:00
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( " id " ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation ,
} )
. unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_10_tasks " ) ;
2022-10-24 20:16:14 +08:00
2022-10-25 21:30:36 +08:00
// A first batch should be processed with only the first documentAddition that's going to fail.
2022-11-28 23:27:41 +08:00
handle . advance_one_failed_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " only_first_task_failed " ) ;
2022-10-20 23:11:44 +08:00
2022-10-25 21:30:36 +08:00
// Everything else should be batched together.
2022-11-28 23:27:41 +08:00
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_tasks_processed " ) ;
2022-10-25 21:30:36 +08:00
// Has everything being pushed successfully in milli?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
2023-01-24 00:32:13 +08:00
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
}
#[ test ]
fn test_document_addition_with_multiple_primary_key ( ) {
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
for ( id , primary_key ) in [ " id " , " bork " , " bloup " ] . iter ( ) . enumerate ( ) {
let content = format! (
r #" {{
" id " : { id } ,
" doggo " : " jean bob "
} } " #,
) ;
let ( uuid , mut file ) =
index_scheduler . create_update_file_with_uuid ( id as u128 ) . unwrap ( ) ;
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
assert_eq! ( documents_count , 1 ) ;
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( primary_key ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_3_tasks " ) ;
// A first batch should be processed with only the first documentAddition.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " only_first_task_succeed " ) ;
// The second batch should fail.
handle . advance_one_failed_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " second_task_fails " ) ;
// The second batch should fail.
handle . advance_one_failed_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " third_task_fails " ) ;
// Is the primary key still what we expect?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) . unwrap ( ) ;
snapshot! ( primary_key , @ " id " ) ;
// Is the document still the one we expect?.
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
}
#[ test ]
fn test_document_addition_with_multiple_primary_key_batch_wrong_key ( ) {
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
for ( id , primary_key ) in [ " id " , " bork " , " bork " ] . iter ( ) . enumerate ( ) {
let content = format! (
r #" {{
" id " : { id } ,
" doggo " : " jean bob "
} } " #,
) ;
let ( uuid , mut file ) =
index_scheduler . create_update_file_with_uuid ( id as u128 ) . unwrap ( ) ;
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
assert_eq! ( documents_count , 1 ) ;
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( primary_key ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_3_tasks " ) ;
// A first batch should be processed with only the first documentAddition.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " only_first_task_succeed " ) ;
// The second batch should fail and contains two tasks.
handle . advance_one_failed_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " second_and_third_tasks_fails " ) ;
// Is the primary key still what we expect?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) . unwrap ( ) ;
snapshot! ( primary_key , @ " id " ) ;
// Is the document still the one we expect?.
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
}
#[ test ]
fn test_document_addition_with_bad_primary_key ( ) {
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
for ( id , primary_key ) in [ " bork " , " bork " , " id " , " bork " , " id " ] . iter ( ) . enumerate ( ) {
let content = format! (
r #" {{
" id " : { id } ,
" doggo " : " jean bob "
} } " #,
) ;
let ( uuid , mut file ) =
index_scheduler . create_update_file_with_uuid ( id as u128 ) . unwrap ( ) ;
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
assert_eq! ( documents_count , 1 ) ;
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : Some ( S ( primary_key ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_5_tasks " ) ;
// A first batch should be processed with only the first two documentAddition.
// it should fails because the documents don't contains any `bork` field.
// NOTE: it's marked as successful because the batch didn't fails, it's the individual tasks that failed.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " first_and_second_task_fails " ) ;
// The primary key should be set to none since we failed the batch.
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) ;
snapshot! ( primary_key . is_none ( ) , @ " true " ) ;
// The second batch should succeed and only contains one task.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " third_task_succeeds " ) ;
// The primary key should be set to `id` since this batch succeeded.
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) . unwrap ( ) ;
snapshot! ( primary_key , @ " id " ) ;
// We're trying to `bork` again, but now there is already a primary key set for this index.
handle . advance_one_failed_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " fourth_task_fails " ) ;
// Finally the last task should succeed since its primary key is the same as the valid one.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " fifth_task_succeeds " ) ;
// Is the primary key still what we expect?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) . unwrap ( ) ;
snapshot! ( primary_key , @ " id " ) ;
// Is the document still the one we expect?.
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
}
#[ test ]
fn test_document_addition_with_set_and_null_primary_key ( ) {
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
for ( id , primary_key ) in
[ None , Some ( " bork " ) , Some ( " paw " ) , None , None , Some ( " paw " ) ] . into_iter ( ) . enumerate ( )
{
let content = format! (
r #" {{
" paw " : { id } ,
" doggo " : " jean bob "
} } " #,
) ;
let ( uuid , mut file ) =
index_scheduler . create_update_file_with_uuid ( id as u128 ) . unwrap ( ) ;
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
assert_eq! ( documents_count , 1 ) ;
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : primary_key . map ( | pk | pk . to_string ( ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_6_tasks " ) ;
// A first batch should contains only one task that fails because we can't infer the primary key.
// NOTE: it's marked as successful because the batch didn't fails, it's the individual tasks that failed.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " first_task_fails " ) ;
// The second batch should contains only one task that fails because we bork is not a valid primary key.
// NOTE: it's marked as successful because the batch didn't fails, it's the individual tasks that failed.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " second_task_fails " ) ;
// No primary key should be set at this point.
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) ;
snapshot! ( primary_key . is_none ( ) , @ " true " ) ;
// The third batch should succeed and only contains one task.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " third_task_succeeds " ) ;
// The primary key should be set to `id` since this batch succeeded.
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) . unwrap ( ) ;
snapshot! ( primary_key , @ " paw " ) ;
// We should be able to batch together the next two tasks that don't specify any primary key
2023-01-24 03:16:16 +08:00
// + the last task that matches the current primary-key. Everything should succeed.
2023-01-24 00:32:13 +08:00
handle . advance_one_successful_batch ( ) ;
2023-01-24 03:16:16 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_other_tasks_succeeds " ) ;
2023-01-24 00:32:13 +08:00
// Is the primary key still what we expect?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) . unwrap ( ) ;
snapshot! ( primary_key , @ " paw " ) ;
// Is the document still the one we expect?.
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
}
#[ test ]
fn test_document_addition_with_set_and_null_primary_key_inference_works ( ) {
let ( index_scheduler , mut handle ) = IndexScheduler ::test ( true , vec! [ ] ) ;
for ( id , primary_key ) in [ None , Some ( " bork " ) , Some ( " doggoid " ) , None , None , Some ( " doggoid " ) ]
. into_iter ( )
. enumerate ( )
{
let content = format! (
r #" {{
" doggoid " : { id } ,
" doggo " : " jean bob "
} } " #,
) ;
let ( uuid , mut file ) =
index_scheduler . create_update_file_with_uuid ( id as u128 ) . unwrap ( ) ;
let documents_count = read_json ( content . as_bytes ( ) , file . as_file_mut ( ) ) . unwrap ( ) ;
assert_eq! ( documents_count , 1 ) ;
file . persist ( ) . unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::DocumentAdditionOrUpdate {
index_uid : S ( " doggos " ) ,
primary_key : primary_key . map ( | pk | pk . to_string ( ) ) ,
method : ReplaceDocuments ,
content_file : uuid ,
documents_count ,
allow_index_creation : true ,
} )
. unwrap ( ) ;
index_scheduler . assert_internally_consistent ( ) ;
}
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " after_registering_the_6_tasks " ) ;
// A first batch should contains only one task that succeed and sets the primary key to `doggoid`.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " first_task_succeed " ) ;
// Checking the primary key.
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) ;
snapshot! ( primary_key . is_none ( ) , @ " false " ) ;
// The second batch should contains only one task that fails because it tries to update the primary key to `bork`.
handle . advance_one_failed_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " second_task_fails " ) ;
// The third batch should succeed and only contains one task.
handle . advance_one_successful_batch ( ) ;
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " third_task_succeeds " ) ;
// We should be able to batch together the next two tasks that don't specify any primary key
2023-01-24 03:16:16 +08:00
// + the last task that matches the current primary-key. Everything should succeed.
2023-01-24 00:32:13 +08:00
handle . advance_one_successful_batch ( ) ;
2023-01-24 03:16:16 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " all_other_tasks_succeeds " ) ;
2023-01-24 00:32:13 +08:00
// Is the primary key still what we expect?
let index = index_scheduler . index ( " doggos " ) . unwrap ( ) ;
let rtxn = index . read_txn ( ) . unwrap ( ) ;
let primary_key = index . primary_key ( & rtxn ) . unwrap ( ) . unwrap ( ) ;
snapshot! ( primary_key , @ " doggoid " ) ;
// Is the document still the one we expect?.
let field_ids_map = index . fields_ids_map ( & rtxn ) . unwrap ( ) ;
let field_ids = field_ids_map . ids ( ) . collect ::< Vec < _ > > ( ) ;
let documents = index
. all_documents ( & rtxn )
. unwrap ( )
. map ( | ret | obkv_to_json ( & field_ids , & field_ids_map , ret . unwrap ( ) . 1 ) . unwrap ( ) )
. collect ::< Vec < _ > > ( ) ;
snapshot! ( serde_json ::to_string_pretty ( & documents ) . unwrap ( ) , name : " documents " ) ;
2022-10-20 23:11:44 +08:00
}
#[ test ]
fn panic_in_process_batch_for_index_creation ( ) {
2022-11-28 23:27:41 +08:00
let ( index_scheduler , mut handle ) =
2022-10-20 23:11:44 +08:00
IndexScheduler ::test ( true , vec! [ ( 1 , FailureLocation ::PanicInsideProcessBatch ) ] ) ;
let kind = index_creation_task ( " catto " , " mouse " ) ;
let _task = index_scheduler . register ( kind ) . unwrap ( ) ;
2022-11-28 23:27:41 +08:00
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " registered_the_first_task " ) ;
2022-10-20 23:11:44 +08:00
2022-11-28 23:27:41 +08:00
handle . advance_till ( [ Start , BatchCreated , ProcessBatchFailed , AfterProcessing ] ) ;
2022-10-20 23:11:44 +08:00
// Still in the first iteration
assert_eq! ( * index_scheduler . run_loop_iteration . read ( ) . unwrap ( ) , 1 ) ;
// No matter what happens in process_batch, the index_scheduler should be internally consistent
snapshot! ( snapshot_index_scheduler ( & index_scheduler ) , name : " index_creation_failed " ) ;
2022-09-21 18:01:46 +08:00
}
2023-04-25 02:04:50 +08:00
#[ test ]
2023-04-25 23:26:34 +08:00
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 } )
2023-04-25 02:04:50 +08:00
. unwrap ( ) ;
2023-04-25 23:26:34 +08:00
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 ,
} ) ;
if result . is_err ( ) {
break ;
}
handle . advance_one_failed_batch ( ) ;
2023-04-25 02:04:50 +08:00
}
index_scheduler . assert_internally_consistent ( ) ;
2023-04-25 23:26:34 +08:00
// at this point the task DB shoud have reached its limit and we should not be able to register new tasks
2023-04-25 02:04:50 +08:00
let result = index_scheduler
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggo " ) , primary_key : None } )
. unwrap_err ( ) ;
2023-04-25 23:26:34 +08:00
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 " ) ;
2023-04-25 02:04:50 +08:00
2023-04-25 23:26:34 +08:00
// 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 ( ) ,
} )
. unwrap_err ( ) ;
2023-04-25 02:04:50 +08:00
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 " ) ;
2023-04-25 23:26:34 +08:00
// But a task deletion that delete something should works
index_scheduler
2023-05-04 06:05:39 +08:00
. register ( KindWithContent ::TaskDeletion { query : S ( " test " ) , tasks : ( 0 .. 100 ) . collect ( ) } )
2023-04-25 02:04:50 +08:00
. unwrap ( ) ;
2023-04-25 23:26:34 +08:00
handle . advance_one_successful_batch ( ) ;
2023-04-25 02:04:50 +08:00
2023-04-25 23:26:34 +08:00
// Now we should be able to enqueue a few tasks again
index_scheduler
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggo " ) , primary_key : None } )
2023-04-25 02:04:50 +08:00
. unwrap ( ) ;
2023-04-25 23:26:34 +08:00
handle . advance_one_failed_batch ( ) ;
}
#[ 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 ;
} ) ;
2023-04-25 02:04:50 +08:00
2023-04-25 23:26:34 +08:00
index_scheduler
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggo " ) , primary_key : None } )
. unwrap ( ) ;
2023-04-25 02:04:50 +08:00
handle . advance_one_successful_batch ( ) ;
2023-04-25 23:26:34 +08:00
index_scheduler
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggo " ) , primary_key : None } )
2023-04-25 02:04:50 +08:00
. unwrap ( ) ;
2023-04-25 23:26:34 +08:00
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 } )
. unwrap ( ) ;
index_scheduler
. register ( KindWithContent ::IndexCreation { index_uid : S ( " doggo " ) , primary_key : None } )
. unwrap ( ) ;
2023-04-26 19:55:02 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
let tasks = index_scheduler . get_task_ids ( & rtxn , & Query { .. Default ::default ( ) } ) . unwrap ( ) ;
let tasks = index_scheduler . get_existing_tasks ( & rtxn , tasks ) . unwrap ( ) ;
snapshot! ( json_string! ( tasks , { " [].enqueuedAt " = > " [date] " , " [].startedAt " = > " [date] " , " [].finishedAt " = > " [date] " } ) , name : " task_queue_is_full " ) ;
drop ( rtxn ) ;
2023-04-25 23:26:34 +08:00
2023-04-26 19:55:02 +08:00
// now we're above the max number of tasks
2023-04-25 23:26:34 +08:00
// and if we try to advance in the tick function a new task deletion should be enqueued
handle . advance_till ( [ Start , BatchCreated ] ) ;
2023-04-26 19:55:02 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
let tasks = index_scheduler . get_task_ids ( & rtxn , & Query { .. Default ::default ( ) } ) . unwrap ( ) ;
let tasks = index_scheduler . 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 ) ;
2023-04-25 23:26:34 +08:00
handle . advance_till ( [ InsideProcessBatch , ProcessBatchSucceeded , AfterProcessing ] ) ;
2023-04-26 19:55:02 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
let tasks = index_scheduler . get_task_ids ( & rtxn , & Query { .. Default ::default ( ) } ) . unwrap ( ) ;
let tasks = index_scheduler . 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 ) ;
2023-04-25 23:26:34 +08:00
handle . advance_one_failed_batch ( ) ;
// a new task deletion has been enqueued
handle . advance_one_successful_batch ( ) ;
2023-04-26 19:55:02 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
let tasks = index_scheduler . get_task_ids ( & rtxn , & Query { .. Default ::default ( ) } ) . unwrap ( ) ;
let tasks = index_scheduler . 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 ) ;
2023-04-25 23:26:34 +08:00
handle . advance_one_failed_batch ( ) ;
handle . advance_one_successful_batch ( ) ;
2023-04-26 19:55:02 +08:00
let rtxn = index_scheduler . env . read_txn ( ) . unwrap ( ) ;
let tasks = index_scheduler . get_task_ids ( & rtxn , & Query { .. Default ::default ( ) } ) . unwrap ( ) ;
let tasks = index_scheduler . 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 ) ;
2023-04-25 02:04:50 +08:00
}
2022-09-15 18:23:41 +08:00
}