diff --git a/crates/dump/src/lib.rs b/crates/dump/src/lib.rs index a17fcf941..8bed7f0d4 100644 --- a/crates/dump/src/lib.rs +++ b/crates/dump/src/lib.rs @@ -1,6 +1,7 @@ #![allow(clippy::type_complexity)] #![allow(clippy::wrong_self_convention)] +use meilisearch_types::batches::BatchId; use meilisearch_types::error::ResponseError; use meilisearch_types::keys::Key; use meilisearch_types::milli::update::IndexDocumentsMethod; @@ -57,6 +58,9 @@ pub enum Version { #[serde(rename_all = "camelCase")] pub struct TaskDump { pub uid: TaskId, + // The batch ID were introduced in v1.12, everything prior to this version will be `None`. + #[serde(default)] + pub batch_uid: Option, #[serde(default)] pub index_uid: Option, pub status: Status, @@ -143,6 +147,7 @@ impl From for TaskDump { fn from(task: Task) -> Self { TaskDump { uid: task.uid, + batch_uid: task.batch_uid, index_uid: task.index_uid().map(|uid| uid.to_string()), status: task.status, kind: task.kind.into(), @@ -297,6 +302,7 @@ pub(crate) mod test { ( TaskDump { uid: 0, + batch_uid: Some(0), index_uid: Some(S("doggo")), status: Status::Succeeded, kind: KindDump::DocumentImport { @@ -320,6 +326,7 @@ pub(crate) mod test { ( TaskDump { uid: 1, + batch_uid: None, index_uid: Some(S("doggo")), status: Status::Enqueued, kind: KindDump::DocumentImport { @@ -346,6 +353,7 @@ pub(crate) mod test { ( TaskDump { uid: 5, + batch_uid: None, index_uid: Some(S("catto")), status: Status::Enqueued, kind: KindDump::IndexDeletion, diff --git a/crates/dump/src/reader/compat/v5_to_v6.rs b/crates/dump/src/reader/compat/v5_to_v6.rs index 40a055465..9887c55ba 100644 --- a/crates/dump/src/reader/compat/v5_to_v6.rs +++ b/crates/dump/src/reader/compat/v5_to_v6.rs @@ -70,6 +70,7 @@ impl CompatV5ToV6 { let task = v6::Task { uid: task_view.uid, + batch_uid: None, index_uid: task_view.index_uid, status: match task_view.status { v5::Status::Enqueued => v6::Status::Enqueued, diff --git a/crates/index-scheduler/src/batch.rs b/crates/index-scheduler/src/batch.rs index 903ec1217..0130e9bfe 100644 --- a/crates/index-scheduler/src/batch.rs +++ b/crates/index-scheduler/src/batch.rs @@ -44,7 +44,7 @@ use time::OffsetDateTime; use uuid::Uuid; use crate::autobatcher::{self, BatchKind}; -use crate::utils::{self, swap_index_uid_in_task}; +use crate::utils::{self, swap_index_uid_in_task, ProcessingBatch}; use crate::{Error, IndexScheduler, MustStopProcessing, ProcessingTasks, Result, TaskId}; /// Represents a combination of tasks that can all be processed at the same time. @@ -279,18 +279,24 @@ impl IndexScheduler { rtxn: &RoTxn, index_uid: String, batch: BatchKind, + current_batch: &mut ProcessingBatch, must_create_index: bool, ) -> Result> { match batch { BatchKind::DocumentClear { ids } => Ok(Some(Batch::IndexOperation { op: IndexOperation::DocumentClear { - tasks: self.get_existing_tasks(rtxn, ids)?, + tasks: self.get_existing_tasks_with_processing_batch( + rtxn, + current_batch, + ids, + )?, index_uid, }, must_create_index, })), BatchKind::DocumentEdition { id } => { - let task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?; + let mut task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?; + current_batch.processing(Some(&mut task)); match &task.kind { KindWithContent::DocumentEdition { index_uid, .. } => { Ok(Some(Batch::IndexOperation { @@ -305,7 +311,11 @@ impl IndexScheduler { } } BatchKind::DocumentOperation { method, operation_ids, .. } => { - let tasks = self.get_existing_tasks(rtxn, operation_ids)?; + let tasks = self.get_existing_tasks_with_processing_batch( + rtxn, + current_batch, + operation_ids, + )?; let primary_key = tasks .iter() .find_map(|task| match task.kind { @@ -352,7 +362,11 @@ impl IndexScheduler { })) } BatchKind::DocumentDeletion { deletion_ids, includes_by_filter: _ } => { - let tasks = self.get_existing_tasks(rtxn, deletion_ids)?; + let tasks = self.get_existing_tasks_with_processing_batch( + rtxn, + current_batch, + deletion_ids, + )?; Ok(Some(Batch::IndexOperation { op: IndexOperation::DocumentDeletion { index_uid, tasks }, @@ -360,7 +374,11 @@ impl IndexScheduler { })) } BatchKind::Settings { settings_ids, .. } => { - let tasks = self.get_existing_tasks(rtxn, settings_ids)?; + let tasks = self.get_existing_tasks_with_processing_batch( + rtxn, + current_batch, + settings_ids, + )?; let mut settings = Vec::new(); for task in &tasks { @@ -383,6 +401,7 @@ impl IndexScheduler { rtxn, index_uid, BatchKind::Settings { settings_ids, allow_index_creation }, + current_batch, must_create_index, )? .unwrap() @@ -398,6 +417,7 @@ impl IndexScheduler { rtxn, index_uid, BatchKind::DocumentClear { ids: other }, + current_batch, must_create_index, )? .unwrap() @@ -430,6 +450,7 @@ impl IndexScheduler { rtxn, index_uid.clone(), BatchKind::Settings { settings_ids, allow_index_creation }, + current_batch, must_create_index, )?; @@ -442,6 +463,7 @@ impl IndexScheduler { primary_key, operation_ids, }, + current_batch, must_create_index, )?; @@ -479,7 +501,8 @@ impl IndexScheduler { } } BatchKind::IndexCreation { id } => { - let task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?; + let mut task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?; + current_batch.processing(Some(&mut task)); let (index_uid, primary_key) = match &task.kind { KindWithContent::IndexCreation { index_uid, primary_key } => { (index_uid.clone(), primary_key.clone()) @@ -489,7 +512,8 @@ impl IndexScheduler { Ok(Some(Batch::IndexCreation { index_uid, primary_key, task })) } BatchKind::IndexUpdate { id } => { - let task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?; + let mut task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?; + current_batch.processing(Some(&mut task)); let primary_key = match &task.kind { KindWithContent::IndexUpdate { primary_key, .. } => primary_key.clone(), _ => unreachable!(), @@ -499,10 +523,11 @@ impl IndexScheduler { BatchKind::IndexDeletion { ids } => Ok(Some(Batch::IndexDeletion { index_uid, index_has_been_created: must_create_index, - tasks: self.get_existing_tasks(rtxn, ids)?, + tasks: self.get_existing_tasks_with_processing_batch(rtxn, current_batch, ids)?, })), BatchKind::IndexSwap { id } => { - let task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?; + let mut task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?; + current_batch.processing(Some(&mut task)); Ok(Some(Batch::IndexSwap { task })) } } @@ -515,10 +540,16 @@ impl IndexScheduler { /// 4. We get the *next* dump to process. /// 5. We get the *next* tasks to process for a specific index. #[tracing::instrument(level = "trace", skip(self, rtxn), target = "indexing::scheduler")] - pub(crate) fn create_next_batch(&self, rtxn: &RoTxn) -> Result> { + pub(crate) fn create_next_batch( + &self, + rtxn: &RoTxn, + ) -> Result> { #[cfg(test)] self.maybe_fail(crate::tests::FailureLocation::InsideCreateBatch)?; + let batch_id = self.next_batch_id(rtxn)?; + let mut current_batch = ProcessingBatch::new(batch_id); + let enqueued = &self.get_status(rtxn, Status::Enqueued)?; let to_cancel = self.get_kind(rtxn, Kind::TaskCancelation)? & enqueued; @@ -526,39 +557,53 @@ impl IndexScheduler { if let Some(task_id) = to_cancel.max() { // We retrieve the tasks that were processing before this tasks cancelation started. // We must *not* reset the processing tasks before calling this method. - let ProcessingTasks { started_at, processing } = + // Displaying the `batch_id` would make a strange error message since this task cancelation is going to + // replace the canceled batch. It's better to avoid mentioning it in the error message. + let ProcessingTasks { batch: previous_batch, processing } = &*self.processing_tasks.read().unwrap(); - return Ok(Some(Batch::TaskCancelation { - task: self.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?, - previous_started_at: *started_at, - previous_processing_tasks: processing.clone(), - })); + let mut task = self.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?; + current_batch.processing(Some(&mut task)); + return Ok(Some(( + Batch::TaskCancelation { + task, + // We should never be in a case where we don't have a previous_batch, but let's not crash if it happens + previous_started_at: previous_batch + .as_ref() + .map_or_else(OffsetDateTime::now_utc, |batch| batch.started_at), + previous_processing_tasks: processing.clone(), + }, + current_batch, + ))); } // 2. we get the next task to delete let to_delete = self.get_kind(rtxn, Kind::TaskDeletion)? & enqueued; if !to_delete.is_empty() { - let tasks = self.get_existing_tasks(rtxn, to_delete)?; - return Ok(Some(Batch::TaskDeletions(tasks))); + let mut tasks = self.get_existing_tasks(rtxn, to_delete)?; + current_batch.processing(&mut tasks); + return Ok(Some((Batch::TaskDeletions(tasks), current_batch))); } // 3. we batch the snapshot. let to_snapshot = self.get_kind(rtxn, Kind::SnapshotCreation)? & enqueued; if !to_snapshot.is_empty() { - return Ok(Some(Batch::SnapshotCreation(self.get_existing_tasks(rtxn, to_snapshot)?))); + let mut tasks = self.get_existing_tasks(rtxn, to_snapshot)?; + current_batch.processing(&mut tasks); + return Ok(Some((Batch::SnapshotCreation(tasks), current_batch))); } // 4. we batch the dumps. let to_dump = self.get_kind(rtxn, Kind::DumpCreation)? & enqueued; if let Some(to_dump) = to_dump.min() { - return Ok(Some(Batch::Dump( - self.get_task(rtxn, to_dump)?.ok_or(Error::CorruptedTaskQueue)?, - ))); + let mut task = self.get_task(rtxn, to_dump)?.ok_or(Error::CorruptedTaskQueue)?; + current_batch.processing(Some(&mut task)); + return Ok(Some((Batch::Dump(task), current_batch))); } // 5. We make a batch from the unprioritised tasks. Start by taking the next enqueued task. let task_id = if let Some(task_id) = enqueued.min() { task_id } else { return Ok(None) }; - let task = self.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?; + let mut task = self.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?; + current_batch.processing(Some(&mut task)); // If the task is not associated with any index, verify that it is an index swap and // create the batch directly. Otherwise, get the index name associated with the task @@ -568,7 +613,7 @@ impl IndexScheduler { index_name } else { assert!(matches!(&task.kind, KindWithContent::IndexSwap { swaps } if swaps.is_empty())); - return Ok(Some(Batch::IndexSwap { task })); + return Ok(Some((Batch::IndexSwap { task }, current_batch))); }; let index_already_exists = self.index_mapper.exists(rtxn, index_name)?; @@ -599,12 +644,15 @@ impl IndexScheduler { if let Some((batchkind, create_index)) = autobatcher::autobatch(enqueued, index_already_exists, primary_key.as_deref()) { - return self.create_next_batch_index( - rtxn, - index_name.to_string(), - batchkind, - create_index, - ); + return Ok(self + .create_next_batch_index( + rtxn, + index_name.to_string(), + batchkind, + &mut current_batch, + create_index, + )? + .map(|batch| (batch, current_batch))); } // If we found no tasks then we were notified for something that got autobatched diff --git a/crates/index-scheduler/src/error.rs b/crates/index-scheduler/src/error.rs index 3bd378fd6..336c0b732 100644 --- a/crates/index-scheduler/src/error.rs +++ b/crates/index-scheduler/src/error.rs @@ -1,5 +1,6 @@ use std::fmt::Display; +use meilisearch_types::batches::BatchId; use meilisearch_types::error::{Code, ErrorCode}; use meilisearch_types::tasks::{Kind, Status}; use meilisearch_types::{heed, milli}; @@ -79,7 +80,9 @@ pub enum Error { )] InvalidTaskDate { field: DateField, date: String }, #[error("Task uid `{task_uid}` is invalid. It should only contain numeric characters.")] - InvalidTaskUids { task_uid: String }, + InvalidTaskUid { task_uid: String }, + #[error("Batch uid `{batch_uid}` is invalid. It should only contain numeric characters.")] + InvalidBatchUid { batch_uid: String }, #[error( "Task status `{status}` is invalid. Available task statuses are {}.", enum_iterator::all::() @@ -106,6 +109,8 @@ pub enum Error { InvalidIndexUid { index_uid: String }, #[error("Task `{0}` not found.")] TaskNotFound(TaskId), + #[error("Batch `{0}` not found.")] + BatchNotFound(BatchId), #[error("Query parameters to filter the tasks to delete are missing. Available query parameters are: `uids`, `indexUids`, `statuses`, `types`, `canceledBy`, `beforeEnqueuedAt`, `afterEnqueuedAt`, `beforeStartedAt`, `afterStartedAt`, `beforeFinishedAt`, `afterFinishedAt`.")] TaskDeletionWithEmptyQuery, #[error("Query parameters to filter the tasks to cancel are missing. Available query parameters are: `uids`, `indexUids`, `statuses`, `types`, `canceledBy`, `beforeEnqueuedAt`, `afterEnqueuedAt`, `beforeStartedAt`, `afterStartedAt`, `beforeFinishedAt`, `afterFinishedAt`.")] @@ -172,12 +177,14 @@ impl Error { | Error::SwapIndexesNotFound(_) | Error::CorruptedDump | Error::InvalidTaskDate { .. } - | Error::InvalidTaskUids { .. } + | Error::InvalidTaskUid { .. } + | Error::InvalidBatchUid { .. } | Error::InvalidTaskStatuses { .. } | Error::InvalidTaskTypes { .. } | Error::InvalidTaskCanceledBy { .. } | Error::InvalidIndexUid { .. } | Error::TaskNotFound(_) + | Error::BatchNotFound(_) | Error::TaskDeletionWithEmptyQuery | Error::TaskCancelationWithEmptyQuery | Error::AbortedTask @@ -216,12 +223,14 @@ impl ErrorCode for Error { Error::SwapIndexNotFound(_) => Code::IndexNotFound, Error::SwapIndexesNotFound(_) => Code::IndexNotFound, Error::InvalidTaskDate { field, .. } => (*field).into(), - Error::InvalidTaskUids { .. } => Code::InvalidTaskUids, + Error::InvalidTaskUid { .. } => Code::InvalidTaskUids, + Error::InvalidBatchUid { .. } => Code::InvalidBatchUids, Error::InvalidTaskStatuses { .. } => Code::InvalidTaskStatuses, Error::InvalidTaskTypes { .. } => Code::InvalidTaskTypes, Error::InvalidTaskCanceledBy { .. } => Code::InvalidTaskCanceledBy, Error::InvalidIndexUid { .. } => Code::InvalidIndexUid, Error::TaskNotFound(_) => Code::TaskNotFound, + Error::BatchNotFound(_) => Code::TaskNotFound, Error::TaskDeletionWithEmptyQuery => Code::MissingTaskFilters, Error::TaskCancelationWithEmptyQuery => Code::MissingTaskFilters, // TODO: not sure of the Code to use diff --git a/crates/index-scheduler/src/insta_snapshot.rs b/crates/index-scheduler/src/insta_snapshot.rs index f295e35b6..da0d90baa 100644 --- a/crates/index-scheduler/src/insta_snapshot.rs +++ b/crates/index-scheduler/src/insta_snapshot.rs @@ -1,6 +1,7 @@ use std::collections::BTreeSet; use std::fmt::Write; +use meilisearch_types::batches::Batch; use meilisearch_types::heed::types::{SerdeBincode, SerdeJson, Str}; use meilisearch_types::heed::{Database, RoTxn}; use meilisearch_types::milli::{CboRoaringBitmapCodec, RoaringBitmapCodec, BEU32}; @@ -24,6 +25,9 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String { file_store, env, all_tasks, + all_batches, + batch_to_tasks_mapping, + // task reverse index status, kind, index_tasks, @@ -31,6 +35,16 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String { enqueued_at, started_at, finished_at, + + // batch reverse index + batch_status, + batch_kind, + batch_index_tasks, + batch_canceled_by, + batch_enqueued_at, + batch_started_at, + batch_finished_at, + index_mapper, features: _, max_number_of_tasks: _, @@ -52,10 +66,13 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String { let mut snap = String::new(); - let processing_tasks = processing_tasks.read().unwrap().processing.clone(); + let processing = processing_tasks.read().unwrap().clone(); snap.push_str(&format!("### Autobatching Enabled = {autobatching_enabled}\n")); - snap.push_str("### Processing Tasks:\n"); - snap.push_str(&snapshot_bitmap(&processing_tasks)); + snap.push_str(&format!( + "### Processing batch {:?}:\n", + processing.batch.map(|batch| batch.uid) + )); + snap.push_str(&snapshot_bitmap(&processing.processing)); snap.push_str("\n----------------------------------------------------------------------\n"); snap.push_str("### All Tasks:\n"); @@ -94,6 +111,42 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String { snap.push_str(&snapshot_date_db(&rtxn, *finished_at)); snap.push_str("----------------------------------------------------------------------\n"); + snap.push_str("### All Batches:\n"); + snap.push_str(&snapshot_all_batches(&rtxn, *all_batches)); + snap.push_str("----------------------------------------------------------------------\n"); + + snap.push_str("### Batch to tasks mapping:\n"); + snap.push_str(&snapshot_batches_to_tasks_mappings(&rtxn, *batch_to_tasks_mapping)); + snap.push_str("----------------------------------------------------------------------\n"); + + snap.push_str("### Batches Status:\n"); + snap.push_str(&snapshot_status(&rtxn, *batch_status)); + snap.push_str("----------------------------------------------------------------------\n"); + + snap.push_str("### Batches Kind:\n"); + snap.push_str(&snapshot_kind(&rtxn, *batch_kind)); + snap.push_str("----------------------------------------------------------------------\n"); + + snap.push_str("### Batches Index Tasks:\n"); + snap.push_str(&snapshot_index_tasks(&rtxn, *batch_index_tasks)); + snap.push_str("----------------------------------------------------------------------\n"); + + snap.push_str("### Batches Canceled By:\n"); + snap.push_str(&snapshot_canceled_by(&rtxn, *batch_canceled_by)); + snap.push_str("\n----------------------------------------------------------------------\n"); + + snap.push_str("### Batches Enqueued At:\n"); + snap.push_str(&snapshot_date_db(&rtxn, *batch_enqueued_at)); + snap.push_str("----------------------------------------------------------------------\n"); + + snap.push_str("### Batches Started At:\n"); + snap.push_str(&snapshot_date_db(&rtxn, *batch_started_at)); + snap.push_str("----------------------------------------------------------------------\n"); + + snap.push_str("### Batches Finished At:\n"); + snap.push_str(&snapshot_date_db(&rtxn, *batch_finished_at)); + snap.push_str("----------------------------------------------------------------------\n"); + snap.push_str("### File Store:\n"); snap.push_str(&snapshot_file_store(file_store)); snap.push_str("\n----------------------------------------------------------------------\n"); @@ -131,6 +184,29 @@ pub fn snapshot_all_tasks(rtxn: &RoTxn, db: Database>) -> snap } +pub fn snapshot_all_batches(rtxn: &RoTxn, db: Database>) -> String { + let mut snap = String::new(); + let iter = db.iter(rtxn).unwrap(); + for next in iter { + let (batch_id, batch) = next.unwrap(); + snap.push_str(&format!("{batch_id} {}\n", snapshot_batch(&batch))); + } + snap +} + +pub fn snapshot_batches_to_tasks_mappings( + rtxn: &RoTxn, + db: Database, +) -> String { + let mut snap = String::new(); + let iter = db.iter(rtxn).unwrap(); + for next in iter { + let (batch_id, tasks) = next.unwrap(); + snap.push_str(&format!("{batch_id} {}\n", snapshot_bitmap(&tasks))); + } + snap +} + pub fn snapshot_date_db(rtxn: &RoTxn, db: Database) -> String { let mut snap = String::new(); let iter = db.iter(rtxn).unwrap(); @@ -145,6 +221,7 @@ pub fn snapshot_task(task: &Task) -> String { let mut snap = String::new(); let Task { uid, + batch_uid, enqueued_at: _, started_at: _, finished_at: _, @@ -156,6 +233,9 @@ pub fn snapshot_task(task: &Task) -> String { } = task; snap.push('{'); snap.push_str(&format!("uid: {uid}, ")); + if let Some(batch_uid) = batch_uid { + snap.push_str(&format!("batch_uid: {batch_uid}, ")); + } snap.push_str(&format!("status: {status}, ")); if let Some(canceled_by) = canceled_by { snap.push_str(&format!("canceled_by: {canceled_by}, ")); @@ -271,6 +351,19 @@ pub fn snapshot_canceled_by(rtxn: &RoTxn, db: Database String { + let mut snap = String::new(); + let Batch { uid, started_at, finished_at } = batch; + if let Some(finished_at) = finished_at { + assert!(finished_at > started_at); + } + snap.push('{'); + snap.push_str(&format!("uid: {uid}, ")); + snap.push('}'); + snap +} + pub fn snapshot_index_mapper(rtxn: &RoTxn, mapper: &IndexMapper) -> String { let mut s = String::new(); let names = mapper.index_names(rtxn).unwrap(); diff --git a/crates/index-scheduler/src/lib.rs b/crates/index-scheduler/src/lib.rs index e0e2bfb75..05b6722e8 100644 --- a/crates/index-scheduler/src/lib.rs +++ b/crates/index-scheduler/src/lib.rs @@ -48,6 +48,7 @@ pub use features::RoFeatures; use file_store::FileStore; use flate2::bufread::GzEncoder; use flate2::Compression; +use meilisearch_types::batches::{Batch, BatchId}; use meilisearch_types::error::ResponseError; use meilisearch_types::features::{InstanceTogglableFeatures, RuntimeTogglableFeatures}; use meilisearch_types::heed::byteorder::BE; @@ -70,7 +71,7 @@ use utils::{filter_out_references_to_newer_tasks, keep_tasks_within_datetimes, m use uuid::Uuid; use crate::index_mapper::IndexMapper; -use crate::utils::{check_index_swap_validity, clamp_to_page_size}; +use crate::utils::{check_index_swap_validity, clamp_to_page_size, ProcessingBatch}; pub(crate) type BEI128 = I128; @@ -84,6 +85,10 @@ pub struct Query { pub limit: Option, /// The minimum [task id](`meilisearch_types::tasks::Task::uid`) to be matched pub from: Option, + /// The [task ids](`meilisearch_types::tasks::Task::uid`) to be matched + pub uids: Option>, + /// The [batch ids](`meilisearch_types::batches::Batch::uid`) to be matched + pub batch_uids: Option>, /// The allowed [statuses](`meilisearch_types::tasks::Task::status`) of the matched tasls pub statuses: Option>, /// The allowed [kinds](meilisearch_types::tasks::Kind) of the matched tasks. @@ -98,8 +103,6 @@ pub struct Query { pub types: Option>, /// The allowed [index ids](meilisearch_types::tasks::Task::index_uid) of the matched tasks pub index_uids: Option>, - /// The [task ids](`meilisearch_types::tasks::Task::uid`) to be matched - pub uids: Option>, /// 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>, @@ -126,10 +129,11 @@ impl Query { Query { limit: None, from: None, + uids: None, + batch_uids: None, statuses: None, types: None, index_uids: None, - uids: None, canceled_by: None, before_enqueued_at: None, after_enqueued_at: None, @@ -157,8 +161,7 @@ impl Query { #[derive(Debug, Clone)] struct ProcessingTasks { - /// The date and time at which the indexation started. - started_at: OffsetDateTime, + batch: Option, /// The list of tasks ids that are currently running. processing: RoaringBitmap, } @@ -166,18 +169,21 @@ struct ProcessingTasks { impl ProcessingTasks { /// Creates an empty `ProcessingAt` struct. fn new() -> ProcessingTasks { - ProcessingTasks { started_at: OffsetDateTime::now_utc(), processing: RoaringBitmap::new() } + ProcessingTasks { batch: None, processing: RoaringBitmap::new() } } /// Stores the currently processing tasks, and the date time at which it started. - fn start_processing_at(&mut self, started_at: OffsetDateTime, processing: RoaringBitmap) { - self.started_at = started_at; + fn start_processing(&mut self, processing_batch: ProcessingBatch, processing: RoaringBitmap) { + self.batch = Some(processing_batch); self.processing = processing; } /// Set the processing tasks to an empty list - fn stop_processing(&mut self) -> RoaringBitmap { - std::mem::take(&mut self.processing) + fn stop_processing(&mut self) -> Self { + Self { + batch: std::mem::take(&mut self.batch), + processing: std::mem::take(&mut self.processing), + } } /// Returns `true` if there, at least, is one task that is currently processing that we must stop. @@ -206,6 +212,8 @@ impl MustStopProcessing { /// Database const names for the `IndexScheduler`. mod db_name { pub const ALL_TASKS: &str = "all-tasks"; + pub const ALL_BATCHES: &str = "all-batches"; + pub const BATCH_TO_TASKS_MAPPING: &str = "batch-to-tasks-mapping"; pub const STATUS: &str = "status"; pub const KIND: &str = "kind"; pub const INDEX_TASKS: &str = "index-tasks"; @@ -213,6 +221,14 @@ mod db_name { pub const ENQUEUED_AT: &str = "enqueued-at"; pub const STARTED_AT: &str = "started-at"; pub const FINISHED_AT: &str = "finished-at"; + + pub const BATCH_STATUS: &str = "batch-status"; + pub const BATCH_KIND: &str = "batch-kind"; + pub const BATCH_INDEX_TASKS: &str = "batch-index-tasks"; + pub const BATCH_CANCELED_BY: &str = "batch-canceled_by"; + pub const BATCH_ENQUEUED_AT: &str = "batch-enqueued-at"; + pub const BATCH_STARTED_AT: &str = "batch-started-at"; + pub const BATCH_FINISHED_AT: &str = "batch-finished-at"; } #[cfg(test)] @@ -294,9 +310,15 @@ pub struct IndexScheduler { /// The list of files referenced by the tasks pub(crate) file_store: FileStore, - // The main database, it contains all the tasks accessible by their Id. + /// The main database, it contains all the tasks accessible by their Id. pub(crate) all_tasks: Database>, + /// Contains all the batches accessible by their Id. + pub(crate) all_batches: Database>, + + /// Matches a batch id with the associated task ids. + pub(crate) batch_to_tasks_mapping: Database, + /// All the tasks ids grouped by their status. // TODO we should not be able to serialize a `Status::Processing` in this database. pub(crate) status: Database, RoaringBitmapCodec>, @@ -304,19 +326,30 @@ pub struct IndexScheduler { pub(crate) kind: Database, RoaringBitmapCodec>, /// Store the tasks associated to an index. pub(crate) index_tasks: Database, - /// Store the tasks that were canceled by a task uid pub(crate) canceled_by: Database, - /// Store the task ids of tasks which were enqueued at a specific date pub(crate) enqueued_at: Database, - /// Store the task ids of finished tasks which started being processed at a specific date pub(crate) started_at: Database, - /// Store the task ids of tasks which finished at a specific date pub(crate) finished_at: Database, + /// All the batches containing a task matching the selected status. + pub(crate) batch_status: Database, RoaringBitmapCodec>, + /// All the batches ids grouped by the kind of their task. + pub(crate) batch_kind: Database, RoaringBitmapCodec>, + /// Store the batches associated to an index. + pub(crate) batch_index_tasks: Database, + /// Store the batches containing a task canceled by a task uid + pub(crate) batch_canceled_by: Database, + /// Store the batches containing tasks which were enqueued at a specific date + pub(crate) batch_enqueued_at: Database, + /// Store the batches containing finished tasks started at a specific date + pub(crate) batch_started_at: Database, + /// Store the batches containing tasks finished at a specific date + pub(crate) batch_finished_at: Database, + /// In charge of creating, opening, storing and returning indexes. pub(crate) index_mapper: IndexMapper, @@ -385,6 +418,10 @@ impl IndexScheduler { processing_tasks: self.processing_tasks.clone(), file_store: self.file_store.clone(), all_tasks: self.all_tasks, + all_batches: self.all_batches, + batch_to_tasks_mapping: self.batch_to_tasks_mapping, + + // Tasks reverse index status: self.status, kind: self.kind, index_tasks: self.index_tasks, @@ -392,6 +429,16 @@ impl IndexScheduler { enqueued_at: self.enqueued_at, started_at: self.started_at, finished_at: self.finished_at, + + // Batches reverse index + batch_status: self.batch_status, + batch_kind: self.batch_kind, + batch_index_tasks: self.batch_index_tasks, + batch_canceled_by: self.batch_canceled_by, + batch_enqueued_at: self.batch_enqueued_at, + batch_started_at: self.batch_started_at, + batch_finished_at: self.batch_finished_at, + index_mapper: self.index_mapper.clone(), wake_up: self.wake_up.clone(), autobatching_enabled: self.autobatching_enabled, @@ -451,7 +498,7 @@ impl IndexScheduler { let env = unsafe { heed::EnvOpenOptions::new() - .max_dbs(11) + .max_dbs(20) .map_size(budget.task_db_size) .open(options.tasks_path) }?; @@ -462,6 +509,10 @@ impl IndexScheduler { let mut wtxn = env.write_txn()?; let all_tasks = env.create_database(&mut wtxn, Some(db_name::ALL_TASKS))?; + let all_batches = env.create_database(&mut wtxn, Some(db_name::ALL_BATCHES))?; + let batch_to_tasks_mapping = + env.create_database(&mut wtxn, Some(db_name::BATCH_TO_TASKS_MAPPING))?; + 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))?; @@ -469,6 +520,14 @@ impl IndexScheduler { 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))?; + + let batch_status = env.create_database(&mut wtxn, Some(db_name::BATCH_STATUS))?; + let batch_kind = env.create_database(&mut wtxn, Some(db_name::BATCH_KIND))?; + let batch_index_tasks = env.create_database(&mut wtxn, Some(db_name::BATCH_INDEX_TASKS))?; + let batch_canceled_by = env.create_database(&mut wtxn, Some(db_name::BATCH_CANCELED_BY))?; + let batch_enqueued_at = env.create_database(&mut wtxn, Some(db_name::BATCH_ENQUEUED_AT))?; + let batch_started_at = env.create_database(&mut wtxn, Some(db_name::BATCH_STARTED_AT))?; + let batch_finished_at = env.create_database(&mut wtxn, Some(db_name::BATCH_FINISHED_AT))?; wtxn.commit()?; // allow unreachable_code to get rids of the warning in the case of a test build. @@ -477,6 +536,9 @@ impl IndexScheduler { processing_tasks: Arc::new(RwLock::new(ProcessingTasks::new())), file_store, all_tasks, + all_batches, + batch_to_tasks_mapping, + // Task reverse indexes status, kind, index_tasks, @@ -484,6 +546,16 @@ impl IndexScheduler { enqueued_at, started_at, finished_at, + + // Batch reverse indexes + batch_status, + batch_kind, + batch_index_tasks, + batch_canceled_by, + batch_enqueued_at, + batch_started_at, + batch_finished_at, + index_mapper: IndexMapper::new( &env, options.indexes_path, @@ -699,17 +771,44 @@ impl IndexScheduler { /// 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 { - let ProcessingTasks { - started_at: started_at_processing, processing: processing_tasks, .. - } = self.processing_tasks.read().unwrap().clone(); + let ProcessingTasks { batch: processing_batch, processing: processing_tasks } = + self.processing_tasks.read().unwrap().clone(); + let Query { + limit, + from, + uids, + batch_uids, + statuses, + types, + index_uids, + canceled_by, + before_enqueued_at, + after_enqueued_at, + before_started_at, + after_started_at, + before_finished_at, + after_finished_at, + } = query; let mut tasks = self.all_task_ids(rtxn)?; - if let Some(from) = &query.from { + if let Some(from) = from { tasks.remove_range(from.saturating_add(1)..); } - if let Some(status) = &query.statuses { + if let Some(batch_uids) = batch_uids { + let mut batch_tasks = RoaringBitmap::new(); + for batch_uid in batch_uids { + if Some(*batch_uid) == processing_batch.as_ref().map(|batch| batch.uid) { + batch_tasks |= &processing_tasks; + } else { + batch_tasks |= self.tasks_in_batch(rtxn, *batch_uid)?; + } + } + tasks &= batch_tasks; + } + + if let Some(status) = statuses { let mut status_tasks = RoaringBitmap::new(); for status in status { match status { @@ -726,12 +825,12 @@ impl IndexScheduler { tasks &= status_tasks; } - if let Some(uids) = &query.uids { + if let Some(uids) = uids { let uids = RoaringBitmap::from_iter(uids); tasks &= &uids; } - if let Some(canceled_by) = &query.canceled_by { + if let Some(canceled_by) = canceled_by { let mut all_canceled_tasks = RoaringBitmap::new(); for cancel_task_uid in canceled_by { if let Some(canceled_by_uid) = self.canceled_by.get(rtxn, cancel_task_uid)? { @@ -748,7 +847,7 @@ impl IndexScheduler { } } - if let Some(kind) = &query.types { + if let Some(kind) = types { let mut kind_tasks = RoaringBitmap::new(); for kind in kind { kind_tasks |= self.get_kind(rtxn, *kind)?; @@ -756,7 +855,7 @@ impl IndexScheduler { tasks &= &kind_tasks; } - if let Some(index) = &query.index_uids { + if let Some(index) = index_uids { let mut index_tasks = RoaringBitmap::new(); for index in index { index_tasks |= self.index_tasks(rtxn, index)?; @@ -774,37 +873,40 @@ impl IndexScheduler { // special case for Processing tasks // A closure that clears the filtered_processing_tasks if their started_at date falls outside the given bounds - let mut clear_filtered_processing_tasks = + let clear_filtered_processing_tasks = |start: Bound, end: Bound| { 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(), + &processing_batch + .map_or_else(OffsetDateTime::now_utc, |batch| batch.started_at) + .unix_timestamp_nanos(), ); if !is_within_dates { filtered_processing_tasks.clear(); } }; - match (query.after_started_at, query.before_started_at) { + match (after_started_at, before_started_at) { (None, None) => (), (None, Some(before)) => { - clear_filtered_processing_tasks(Bound::Unbounded, Bound::Excluded(before)) + clear_filtered_processing_tasks(Bound::Unbounded, Bound::Excluded(*before)) } (Some(after), None) => { - clear_filtered_processing_tasks(Bound::Excluded(after), Bound::Unbounded) - } - (Some(after), Some(before)) => { - clear_filtered_processing_tasks(Bound::Excluded(after), Bound::Excluded(before)) + clear_filtered_processing_tasks(Bound::Excluded(*after), Bound::Unbounded) } + (Some(after), Some(before)) => clear_filtered_processing_tasks( + Bound::Excluded(*after), + Bound::Excluded(*before), + ), }; keep_tasks_within_datetimes( rtxn, &mut filtered_non_processing_tasks, self.started_at, - query.after_started_at, - query.before_started_at, + *after_started_at, + *before_started_at, )?; filtered_non_processing_tasks | filtered_processing_tasks }; @@ -813,23 +915,199 @@ impl IndexScheduler { rtxn, &mut tasks, self.enqueued_at, - query.after_enqueued_at, - query.before_enqueued_at, + *after_enqueued_at, + *before_enqueued_at, )?; keep_tasks_within_datetimes( rtxn, &mut tasks, self.finished_at, + *after_finished_at, + *before_finished_at, + )?; + + if let Some(limit) = *limit { + tasks = tasks.into_iter().rev().take(limit as usize).collect(); + } + + Ok(tasks) + } + + /// Return the batch ids matched by the given query from the index scheduler's point of view. + pub(crate) fn get_batch_ids( + &self, + rtxn: &RoTxn, + processing: &ProcessingTasks, + query: &Query, + ) -> Result { + let mut batches = self.all_batch_ids(rtxn)?; + if let Some(batch_id) = processing.batch.as_ref().map(|batch| batch.uid) { + batches.insert(batch_id); + } + + if let Some(from) = &query.from { + batches.remove_range(from.saturating_add(1)..); + } + + if let Some(batch_uids) = &query.batch_uids { + let batches_uids = RoaringBitmap::from_iter(batch_uids); + batches &= batches_uids; + } + + if let Some(status) = &query.statuses { + let mut status_batches = RoaringBitmap::new(); + for status in status { + match status { + // special case for Processing batches + Status::Processing => { + if let Some(batch_id) = processing.batch.as_ref().map(|batch| batch.uid) { + status_batches.insert(batch_id); + } + } + // Enqueued tasks are not stored in batches + Status::Enqueued => (), + status => status_batches |= &self.get_batch_status(rtxn, *status)?, + }; + } + if !status.contains(&Status::Processing) { + if let Some(ref batch) = processing.batch { + batches.remove(batch.uid); + } + } + batches &= status_batches; + } + + if let Some(task_uids) = &query.uids { + let mut batches_by_task_uids = RoaringBitmap::new(); + for task_uid in task_uids { + if let Some(task) = self.get_task(rtxn, *task_uid)? { + if let Some(batch_uid) = task.batch_uid { + batches_by_task_uids.insert(batch_uid); + } + } + } + batches &= batches_by_task_uids; + } + + if let Some(canceled_by) = &query.canceled_by { + let mut all_canceled_batches = RoaringBitmap::new(); + for cancel_uid in canceled_by { + if let Some(canceled_by_uid) = self.batch_canceled_by.get(rtxn, cancel_uid)? { + all_canceled_batches |= canceled_by_uid; + } + } + + // if the canceled_by has been specified but no batch + // matches then we prefer matching zero than all tasks. + if all_canceled_batches.is_empty() { + return Ok(RoaringBitmap::new()); + } else { + batches &= all_canceled_batches; + } + } + + if let Some(kind) = &query.types { + let mut kind_batches = RoaringBitmap::new(); + for kind in kind { + kind_batches |= self.get_batch_kind(rtxn, *kind)?; + if let Some(uid) = processing + .batch + .as_ref() + .and_then(|batch| batch.kinds.contains(kind).then_some(batch.uid)) + { + kind_batches.insert(uid); + } + } + batches &= &kind_batches; + } + + if let Some(index) = &query.index_uids { + let mut index_batches = RoaringBitmap::new(); + for index in index { + index_batches |= self.index_batches(rtxn, index)?; + if let Some(uid) = processing + .batch + .as_ref() + .and_then(|batch| batch.indexes.contains(index).then_some(batch.uid)) + { + index_batches.insert(uid); + } + } + batches &= &index_batches; + } + + // For the started_at filter, we need to treat the part of the batches that are processing from the part of the + // batches that are not processing. The non-processing ones are filtered normally while the processing ones + // are entirely removed unless the in-memory startedAt variable falls within the date filter. + // Once we have filtered the two subsets, we put them back together and assign it back to `batches`. + batches = { + let (mut filtered_non_processing_batches, mut filtered_processing_batches) = + (&batches - &processing.processing, &batches & &processing.processing); + + // special case for Processing batches + // A closure that clears the filtered_processing_batches if their started_at date falls outside the given bounds + let mut clear_filtered_processing_batches = + |start: Bound, end: Bound| { + let start = map_bound(start, |b| b.unix_timestamp_nanos()); + let end = map_bound(end, |b| b.unix_timestamp_nanos()); + let is_within_dates = RangeBounds::contains( + &(start, end), + &processing + .batch + .as_ref() + .map_or_else(OffsetDateTime::now_utc, |batch| batch.started_at) + .unix_timestamp_nanos(), + ); + if !is_within_dates { + filtered_processing_batches.clear(); + } + }; + match (query.after_started_at, query.before_started_at) { + (None, None) => (), + (None, Some(before)) => { + clear_filtered_processing_batches(Bound::Unbounded, Bound::Excluded(before)) + } + (Some(after), None) => { + clear_filtered_processing_batches(Bound::Excluded(after), Bound::Unbounded) + } + (Some(after), Some(before)) => clear_filtered_processing_batches( + Bound::Excluded(after), + Bound::Excluded(before), + ), + }; + + keep_tasks_within_datetimes( + rtxn, + &mut filtered_non_processing_batches, + self.batch_started_at, + query.after_started_at, + query.before_started_at, + )?; + filtered_non_processing_batches | filtered_processing_batches + }; + + keep_tasks_within_datetimes( + rtxn, + &mut batches, + self.batch_enqueued_at, + query.after_enqueued_at, + query.before_enqueued_at, + )?; + + keep_tasks_within_datetimes( + rtxn, + &mut batches, + self.batch_finished_at, query.after_finished_at, query.before_finished_at, )?; if let Some(limit) = query.limit { - tasks = tasks.into_iter().rev().take(limit as usize).collect(); + batches = batches.into_iter().rev().take(limit as usize).collect(); } - Ok(tasks) + Ok(batches) } /// The returned structure contains: @@ -934,6 +1212,70 @@ impl IndexScheduler { Ok((tasks, total_tasks.len())) } + /// Return the batch ids matching the query along with the total number of batches + /// by ignoring the from and limit parameters from the user's point of view. + /// + /// There are two differences between an internal query and a query executed by + /// the user. + /// + /// 1. IndexSwap tasks are not publicly associated with any index, but they are associated + /// with many indexes internally. + /// 2. The user may not have the rights to access the tasks (internally) associated with all indexes. + pub fn get_batch_ids_from_authorized_indexes( + &self, + rtxn: &RoTxn, + query: &Query, + filters: &meilisearch_auth::AuthFilter, + ) -> Result<(RoaringBitmap, u64)> { + let processing = self.processing_tasks.read().unwrap().clone(); + + // compute all batches matching the filter by ignoring the limits, to find the number of batches matching + // the filter. + // As this causes us to compute the filter twice it is slightly inefficient, but doing it this way spares + // us from modifying the underlying implementation, and the performance remains sufficient. + // Should this change, we would modify `get_batch_ids` to directly return the number of matching batches. + let total_batches = + self.get_batch_ids(rtxn, &processing, &query.clone().without_limits())?; + let mut batches = self.get_batch_ids(rtxn, &processing, query)?; + + // If the query contains a list of index uid or there is a finite list of authorized indexes, + // then we must exclude all the batches that only contains tasks associated to multiple indexes. + // This works because we don't autobatch tasks associated to multiple indexes with tasks associated + // to a single index. e.g: IndexSwap cannot be batched with IndexCreation. + if query.index_uids.is_some() || !filters.all_indexes_authorized() { + for kind in enum_iterator::all::().filter(|kind| !kind.related_to_one_index()) { + batches -= self.get_kind(rtxn, kind)?; + if let Some(batch) = processing.batch.as_ref() { + if batch.kinds.contains(&kind) { + batches.remove(batch.uid); + } + } + } + } + + // Any task that is internally associated with a non-authorized index + // must be discarded. + // This works because currently batches cannot contains tasks from multiple indexes at the same time. + if !filters.all_indexes_authorized() { + let all_indexes_iter = self.batch_index_tasks.iter(rtxn)?; + for result in all_indexes_iter { + let (index, index_tasks) = result?; + if !filters.is_index_authorized(index) { + batches -= index_tasks; + } + } + if let Some(batch) = processing.batch.as_ref() { + for index in &batch.indexes { + if !filters.is_index_authorized(index) { + batches.remove(batch.uid); + } + } + } + } + + Ok((batches, total_batches.len())) + } + /// Return the tasks matching the query from the user's point of view along /// with the total number of tasks matching the query, ignoring from and limit. /// @@ -956,17 +1298,24 @@ impl IndexScheduler { tasks.into_iter().rev().take(query.limit.unwrap_or(u32::MAX) as usize), )?; - let ProcessingTasks { started_at, processing, .. } = + let ProcessingTasks { batch, processing } = self.processing_tasks.read().map_err(|_| Error::CorruptedTaskQueue)?.clone(); let ret = tasks.into_iter(); - if processing.is_empty() { + if processing.is_empty() || batch.is_none() { Ok((ret.collect(), total)) } else { + // Safe because we ensured there was a batch in the previous branch + let batch = batch.unwrap(); Ok(( ret.map(|task| { if processing.contains(task.uid) { - Task { status: Status::Processing, started_at: Some(started_at), ..task } + Task { + status: Status::Processing, + batch_uid: Some(batch.uid), + started_at: Some(batch.started_at), + ..task + } } else { task } @@ -977,6 +1326,32 @@ impl IndexScheduler { } } + /// Return the batches matching the query from the user's point of view along + /// with the total number of batches matching the query, ignoring from and limit. + /// + /// There are two differences between an internal query and a query executed by + /// the user. + /// + /// 1. IndexSwap tasks are not publicly associated with any index, but they are associated + /// with many indexes internally. + /// 2. The user may not have the rights to access the tasks (internally) associated with all indexes. + pub fn get_batches_from_authorized_indexes( + &self, + query: Query, + filters: &meilisearch_auth::AuthFilter, + ) -> Result<(Vec, u64)> { + let rtxn = self.env.read_txn()?; + + let (batches, total) = + self.get_batch_ids_from_authorized_indexes(&rtxn, &query, filters)?; + let batches = self.get_existing_batches( + &rtxn, + batches.into_iter().rev().take(query.limit.unwrap_or(u32::MAX) as usize), + )?; + + Ok((batches, total)) + } + /// Register a new task in the scheduler. /// /// If it fails and data was associated with the task, it tries to delete the associated data. @@ -1005,6 +1380,8 @@ impl IndexScheduler { let mut task = Task { uid: task_id.unwrap_or(next_task_id), + // The batch is defined once we starts processing the task + batch_uid: None, enqueued_at: OffsetDateTime::now_utc(), started_at: None, finished_at: None, @@ -1140,7 +1517,7 @@ impl IndexScheduler { } let rtxn = self.env.read_txn().map_err(Error::HeedTransaction)?; - let batch = + let (batch, mut processing_batch) = match self.create_next_batch(&rtxn).map_err(|e| Error::CreateBatch(Box::new(e)))? { Some(batch) => batch, None => return Ok(TickOutcome::WaitForSignal), @@ -1151,11 +1528,14 @@ impl IndexScheduler { // 1. store the starting date with the bitmap of processing tasks. let ids = batch.ids(); let processed_tasks = ids.len(); - let started_at = OffsetDateTime::now_utc(); // We reset the must_stop flag to be sure that we don't stop processing tasks self.must_stop_processing.reset(); - self.processing_tasks.write().unwrap().start_processing_at(started_at, ids.clone()); + self.processing_tasks + .write() + .unwrap() + // We can clone the processing batch here because we don't want its modification to affect the view of the processing batches + .start_processing(processing_batch.clone(), ids.clone()); #[cfg(test)] self.breakpoint(Breakpoint::BatchCreated); @@ -1189,7 +1569,7 @@ impl IndexScheduler { #[allow(unused_variables)] for (i, mut task) in tasks.into_iter().enumerate() { - task.started_at = Some(started_at); + task.started_at = Some(processing_batch.started_at); task.finished_at = Some(finished_at); #[cfg(test)] @@ -1206,6 +1586,7 @@ impl IndexScheduler { self.update_task(&mut wtxn, &task) .map_err(|e| Error::TaskDatabaseUpdate(Box::new(e)))?; + processing_batch.update(&task); } tracing::info!("A batch of tasks was successfully completed with {success} successful tasks and {failure} failed tasks."); } @@ -1254,7 +1635,8 @@ impl IndexScheduler { .get_task(&wtxn, id) .map_err(|e| Error::TaskDatabaseUpdate(Box::new(e)))? .ok_or(Error::CorruptedTaskQueue)?; - task.started_at = Some(started_at); + task.batch_uid = Some(processing_batch.uid); + task.started_at = Some(processing_batch.started_at); task.finished_at = Some(finished_at); task.status = Status::Failed; task.error = Some(error.clone()); @@ -1267,12 +1649,15 @@ impl IndexScheduler { self.update_task(&mut wtxn, &task) .map_err(|e| Error::TaskDatabaseUpdate(Box::new(e)))?; + processing_batch.update(&task); } } } let processed = self.processing_tasks.write().unwrap().stop_processing(); + self.write_batch(&mut wtxn, processing_batch, &processed.processing, finished_at)?; + #[cfg(test)] self.maybe_fail(tests::FailureLocation::CommittingWtxn)?; @@ -1301,7 +1686,7 @@ impl IndexScheduler { })?; // We shouldn't crash the tick function if we can't send data to the webhook. - let _ = self.notify_webhook(&processed); + let _ = self.notify_webhook(&processed.processing); #[cfg(test)] self.breakpoint(Breakpoint::AfterProcessing); @@ -1585,6 +1970,7 @@ impl<'a> Dump<'a> { let task = Task { uid: task.uid, + batch_uid: task.batch_uid, enqueued_at: task.enqueued_at, started_at: task.started_at, finished_at: task.finished_at, @@ -3761,6 +4147,438 @@ mod tests { snapshot!(snapshot_bitmap(&tasks), @"[1,]"); } + #[test] + fn query_batches_from_and_limit() { + let (index_scheduler, mut handle) = IndexScheduler::test(true, vec![]); + + let kind = index_creation_task("doggo", "bone"); + let _task = index_scheduler.register(kind, None, false).unwrap(); + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_first_task"); + let kind = index_creation_task("whalo", "plankton"); + let _task = index_scheduler.register(kind, None, false).unwrap(); + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_second_task"); + let kind = index_creation_task("catto", "his_own_vomit"); + let _task = index_scheduler.register(kind, None, false).unwrap(); + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "registered_the_third_task"); + + handle.advance_n_successful_batches(3); + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "processed_all_tasks"); + + let rtxn = index_scheduler.env.read_txn().unwrap(); + let query = Query { limit: Some(0), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[]"); + + let query = Query { limit: Some(1), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[2,]"); + + let query = Query { limit: Some(2), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[1,2,]"); + + let query = Query { from: Some(1), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[0,1,]"); + + let query = Query { from: Some(2), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[0,1,2,]"); + + let query = Query { from: Some(1), limit: Some(1), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[1,]"); + + let query = Query { from: Some(1), limit: Some(2), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[0,1,]"); + } + + #[test] + fn query_batches_simple() { + let start_time = OffsetDateTime::now_utc(); + + let (index_scheduler, mut handle) = + IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]); + + let kind = index_creation_task("catto", "mouse"); + let _task = index_scheduler.register(kind, None, false).unwrap(); + let kind = index_creation_task("doggo", "sheep"); + let _task = index_scheduler.register(kind, None, false).unwrap(); + let kind = index_creation_task("whalo", "fish"); + let _task = index_scheduler.register(kind, None, false).unwrap(); + + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start"); + + handle.advance_till([Start, BatchCreated]); + + let rtxn = index_scheduler.env.read_txn().unwrap(); + + let query = Query { statuses: Some(vec![Status::Processing]), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[0,]"); // only the processing batch in the first tick + + let query = Query { statuses: Some(vec![Status::Enqueued]), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[]"); // The batches don't contains any enqueued tasks + + let query = Query { + statuses: Some(vec![Status::Enqueued, Status::Processing]), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + snapshot!(snapshot_bitmap(&batches), @"[0,]"); // both enqueued and processing tasks in the first tick + + let query = Query { + statuses: Some(vec![Status::Enqueued, Status::Processing]), + after_started_at: Some(start_time), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // both enqueued and processing tasks in the first tick, but limited to those with a started_at + // that comes after the start of the test, which should excludes the enqueued tasks + snapshot!(snapshot_bitmap(&batches), @"[0,]"); + + let query = Query { + statuses: Some(vec![Status::Enqueued, Status::Processing]), + before_started_at: Some(start_time), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // both enqueued and processing tasks in the first tick, but limited to those with a started_at + // that comes before the start of the test, which should excludes all of them + snapshot!(snapshot_bitmap(&batches), @"[]"); + + let query = Query { + statuses: Some(vec![Status::Enqueued, Status::Processing]), + after_started_at: Some(start_time), + before_started_at: Some(start_time + Duration::minutes(1)), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // both enqueued and processing tasks in the first tick, but limited to those with a started_at + // that comes after the start of the test and before one minute after the start of the test, + // which should exclude the enqueued tasks and include the only processing task + snapshot!(snapshot_bitmap(&batches), @"[0,]"); + + handle.advance_till([ + InsideProcessBatch, + InsideProcessBatch, + ProcessBatchSucceeded, + AfterProcessing, + Start, + BatchCreated, + ]); + + let rtxn = index_scheduler.env.read_txn().unwrap(); + + let second_start_time = OffsetDateTime::now_utc(); + + let query = Query { + statuses: Some(vec![Status::Succeeded, Status::Processing]), + after_started_at: Some(start_time), + before_started_at: Some(start_time + Duration::minutes(1)), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // both succeeded and processing tasks in the first tick, but limited to those with a started_at + // that comes after the start of the test and before one minute after the start of the test, + // which should include all tasks + snapshot!(snapshot_bitmap(&batches), @"[0,1,]"); + + let query = Query { + statuses: Some(vec![Status::Succeeded, Status::Processing]), + before_started_at: Some(start_time), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // both succeeded and processing tasks in the first tick, but limited to those with a started_at + // that comes before the start of the test, which should exclude all tasks + snapshot!(snapshot_bitmap(&batches), @"[]"); + + let query = Query { + statuses: Some(vec![Status::Enqueued, Status::Succeeded, Status::Processing]), + after_started_at: Some(second_start_time), + before_started_at: Some(second_start_time + Duration::minutes(1)), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // both succeeded and processing tasks in the first tick, but limited to those with a started_at + // that comes after the start of the second part of the test and before one minute after the + // second start of the test, which should exclude all tasks + snapshot!(snapshot_bitmap(&batches), @"[]"); + + // now we make one more batch, the started_at field of the new tasks will be past `second_start_time` + handle.advance_till([ + InsideProcessBatch, + InsideProcessBatch, + ProcessBatchSucceeded, + AfterProcessing, + Start, + BatchCreated, + ]); + + let rtxn = index_scheduler.env.read_txn().unwrap(); + + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // we run the same query to verify that, and indeed find that the last task is matched + snapshot!(snapshot_bitmap(&batches), @"[2,]"); + + let query = Query { + statuses: Some(vec![Status::Enqueued, Status::Succeeded, Status::Processing]), + after_started_at: Some(second_start_time), + before_started_at: Some(second_start_time + Duration::minutes(1)), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // enqueued, succeeded, or processing tasks started after the second part of the test, should + // again only return the last task + snapshot!(snapshot_bitmap(&batches), @"[2,]"); + + handle.advance_till([ProcessBatchFailed, AfterProcessing]); + let rtxn = index_scheduler.read_txn().unwrap(); + + // now the last task should have failed + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "end"); + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // so running the last query should return nothing + snapshot!(snapshot_bitmap(&batches), @"[]"); + + let query = Query { + statuses: Some(vec![Status::Failed]), + after_started_at: Some(second_start_time), + before_started_at: Some(second_start_time + Duration::minutes(1)), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // but the same query on failed tasks should return the last task + snapshot!(snapshot_bitmap(&batches), @"[2,]"); + + let query = Query { + statuses: Some(vec![Status::Failed]), + after_started_at: Some(second_start_time), + before_started_at: Some(second_start_time + Duration::minutes(1)), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // but the same query on failed tasks should return the last task + snapshot!(snapshot_bitmap(&batches), @"[2,]"); + + let query = Query { + statuses: Some(vec![Status::Failed]), + uids: Some(vec![1]), + after_started_at: Some(second_start_time), + before_started_at: Some(second_start_time + Duration::minutes(1)), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // same query but with an invalid uid + snapshot!(snapshot_bitmap(&batches), @"[]"); + + let query = Query { + statuses: Some(vec![Status::Failed]), + uids: Some(vec![2]), + after_started_at: Some(second_start_time), + before_started_at: Some(second_start_time + Duration::minutes(1)), + ..Default::default() + }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // same query but with a valid uid + snapshot!(snapshot_bitmap(&batches), @"[2,]"); + } + + #[test] + fn query_batches_special_rules() { + let (index_scheduler, mut handle) = + IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]); + + let kind = index_creation_task("catto", "mouse"); + let _task = index_scheduler.register(kind, None, false).unwrap(); + let kind = index_creation_task("doggo", "sheep"); + let _task = index_scheduler.register(kind, None, false).unwrap(); + let kind = KindWithContent::IndexSwap { + swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "doggo".to_owned()) }], + }; + let _task = index_scheduler.register(kind, None, false).unwrap(); + let kind = KindWithContent::IndexSwap { + swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "whalo".to_owned()) }], + }; + let _task = index_scheduler.register(kind, None, false).unwrap(); + + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start"); + + handle.advance_till([Start, BatchCreated]); + + let rtxn = index_scheduler.env.read_txn().unwrap(); + + let query = Query { index_uids: Some(vec!["catto".to_owned()]), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // only the first task associated with catto is returned, the indexSwap tasks are excluded! + snapshot!(snapshot_bitmap(&batches), @"[0,]"); + + let query = Query { index_uids: Some(vec!["catto".to_owned()]), ..Default::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes( + &rtxn, + &query, + &AuthFilter::with_allowed_indexes( + vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(), + ), + ) + .unwrap(); + // we have asked for only the tasks associated with catto, but are only authorized to retrieve the tasks + // associated with doggo -> empty result + snapshot!(snapshot_bitmap(&batches), @"[]"); + + drop(rtxn); + // We're going to advance and process all the batches for the next query to actually hit the db + handle.advance_till([ + InsideProcessBatch, + InsideProcessBatch, + ProcessBatchSucceeded, + AfterProcessing, + ]); + handle.advance_one_successful_batch(); + handle.advance_n_failed_batches(2); + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "after-processing-everything"); + let rtxn = index_scheduler.env.read_txn().unwrap(); + + let query = Query::default(); + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes( + &rtxn, + &query, + &AuthFilter::with_allowed_indexes( + vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(), + ), + ) + .unwrap(); + // we asked for all the tasks, but we are only authorized to retrieve the doggo tasks + // -> only the index creation of doggo should be returned + snapshot!(snapshot_bitmap(&batches), @"[1,]"); + + let query = Query::default(); + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes( + &rtxn, + &query, + &AuthFilter::with_allowed_indexes( + vec![ + IndexUidPattern::new_unchecked("catto"), + IndexUidPattern::new_unchecked("doggo"), + ] + .into_iter() + .collect(), + ), + ) + .unwrap(); + // we asked for all the tasks, but we are only authorized to retrieve the doggo and catto tasks + // -> all tasks except the swap of catto with whalo are returned + snapshot!(snapshot_bitmap(&batches), @"[0,1,]"); + + let query = Query::default(); + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // we asked for all the tasks with all index authorized -> all tasks returned + snapshot!(snapshot_bitmap(&batches), @"[0,1,2,3,]"); + } + + #[test] + fn query_batches_canceled_by() { + let (index_scheduler, mut handle) = + IndexScheduler::test(true, vec![(3, FailureLocation::InsideProcessBatch)]); + + let kind = index_creation_task("catto", "mouse"); + let _ = index_scheduler.register(kind, None, false).unwrap(); + let kind = index_creation_task("doggo", "sheep"); + let _ = index_scheduler.register(kind, None, false).unwrap(); + let kind = KindWithContent::IndexSwap { + swaps: vec![IndexSwap { indexes: ("catto".to_owned(), "doggo".to_owned()) }], + }; + let _task = index_scheduler.register(kind, None, false).unwrap(); + + handle.advance_n_successful_batches(1); + let kind = KindWithContent::TaskCancelation { + query: "test_query".to_string(), + tasks: [0, 1, 2, 3].into_iter().collect(), + }; + let task_cancelation = index_scheduler.register(kind, None, false).unwrap(); + handle.advance_n_successful_batches(1); + + snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start"); + + let rtxn = index_scheduler.read_txn().unwrap(); + let query = Query { canceled_by: Some(vec![task_cancelation.uid]), ..Query::default() }; + let (batches, _) = index_scheduler + .get_batch_ids_from_authorized_indexes(&rtxn, &query, &AuthFilter::default()) + .unwrap(); + // 0 is not returned because it was not canceled, 3 is not returned because it is the uid of the + // taskCancelation itself + snapshot!(snapshot_bitmap(&batches), @"[1,2,]"); + + let query = Query { canceled_by: Some(vec![task_cancelation.uid]), ..Query::default() }; + let (batches, _) = index_scheduler + .get_task_ids_from_authorized_indexes( + &rtxn, + &query, + &AuthFilter::with_allowed_indexes( + vec![IndexUidPattern::new_unchecked("doggo")].into_iter().collect(), + ), + ) + .unwrap(); + // Return only 1 because the user is not authorized to see task 2 + snapshot!(snapshot_bitmap(&batches), @"[1,]"); + } + #[test] fn fail_in_process_batch_for_index_creation() { let (index_scheduler, mut handle) = @@ -5053,7 +5871,7 @@ mod tests { snapshot!(task.uid, @"0"); snapshot!(snapshot_index_scheduler(&index_scheduler), @r###" ### Autobatching Enabled = true - ### Processing Tasks: + ### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -5076,6 +5894,25 @@ mod tests { ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- + ### All Batches: + ---------------------------------------------------------------------- + ### Batch to tasks mapping: + ---------------------------------------------------------------------- + ### Batches Status: + ---------------------------------------------------------------------- + ### Batches Kind: + ---------------------------------------------------------------------- + ### Batches Index Tasks: + ---------------------------------------------------------------------- + ### Batches Canceled By: + + ---------------------------------------------------------------------- + ### Batches Enqueued At: + ---------------------------------------------------------------------- + ### Batches Started At: + ---------------------------------------------------------------------- + ### Batches Finished At: + ---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- @@ -5086,7 +5923,7 @@ mod tests { snapshot!(task.uid, @"12"); snapshot!(snapshot_index_scheduler(&index_scheduler), @r###" ### Autobatching Enabled = true - ### Processing Tasks: + ### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -5109,6 +5946,25 @@ mod tests { ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- + ### All Batches: + ---------------------------------------------------------------------- + ### Batch to tasks mapping: + ---------------------------------------------------------------------- + ### Batches Status: + ---------------------------------------------------------------------- + ### Batches Kind: + ---------------------------------------------------------------------- + ### Batches Index Tasks: + ---------------------------------------------------------------------- + ### Batches Canceled By: + + ---------------------------------------------------------------------- + ### Batches Enqueued At: + ---------------------------------------------------------------------- + ### Batches Started At: + ---------------------------------------------------------------------- + ### Batches Finished At: + ---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_enqueued_task/initial_tasks_enqueued.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_enqueued_task/initial_tasks_enqueued.snap index 3d3830b29..d1938e82d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_enqueued_task/initial_tasks_enqueued.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_enqueued_task/initial_tasks_enqueued.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,8 +33,26 @@ catto [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/aborted_indexation.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/aborted_indexation.snap index 112cd597b..1784652c2 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/aborted_indexation.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/aborted_indexation.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(1): [1,] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "beavero", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "wolfo", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { matched_tasks: 3, canceled_tasks: None, original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0, 1, 2]> }} @@ -44,9 +44,35 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/first_task_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/first_task_processed.snap index 16be0e4d5..d3671cb4d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/first_task_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/first_task_processed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "beavero", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "wolfo", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- @@ -40,9 +40,35 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/processing_second_task_cancel_enqueued.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/processing_second_task_cancel_enqueued.snap index 46ccfcb8b..92e484c93 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/processing_second_task_cancel_enqueued.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_mix_of_tasks/processing_second_task_cancel_enqueued.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(1): [1,] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "beavero", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "wolfo", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { matched_tasks: 3, canceled_tasks: None, original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0, 1, 2]> }} @@ -43,9 +43,35 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_dump/after_dump_register.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_dump/after_dump_register.snap index ce0343975..8c6b6be65 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_dump/after_dump_register.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_dump/after_dump_register.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -29,7 +29,25 @@ enqueued [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_dump/cancel_registered.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_dump/cancel_registered.snap index 72ae58e00..7406a373b 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_dump/cancel_registered.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_dump/cancel_registered.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -32,7 +32,25 @@ enqueued [0,1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/aborted_indexation.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/aborted_indexation.snap index a2187ab2b..38925a23d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/aborted_indexation.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/aborted_indexation.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -34,8 +34,26 @@ catto: { number_of_documents: 0, field_distribution: {} } ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/cancel_task_registered.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/cancel_task_registered.snap index f65b5ee67..3b8a9a623 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/cancel_task_registered.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/cancel_task_registered.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -33,8 +33,26 @@ catto [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/initial_task_processing.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/initial_task_processing.snap index 8a3ad4661..5c1c3a765 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/initial_task_processing.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/initial_task_processing.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ catto [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/registered_the_first_task.snap index 6873f8b56..6b6af0667 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_processing_task/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ catto [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/cancel_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/cancel_processed.snap index 36e47f569..6c8ead6b4 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/cancel_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/cancel_processed.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { matched_tasks: 1, canceled_tasks: Some(0), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0]> }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { matched_tasks: 1, canceled_tasks: Some(0), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0]> }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -40,7 +40,39 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +"taskCancelation" [1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/initial_task_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/initial_task_processed.snap index 028e03446..ab62eb146 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/initial_task_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/initial_task_processed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -34,7 +34,33 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/registered_the_first_task.snap index 6873f8b56..6b6af0667 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/cancel_succeeded_task/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ catto [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/do_not_batch_task_of_different_indexes/all_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/do_not_batch_task_of_different_indexes/all_tasks_processed.snap index c9a12c327..3b4d6db02 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/do_not_batch_task_of_different_indexes/all_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/do_not_batch_task_of_different_indexes/all_tasks_processed.snap @@ -1,17 +1,17 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "cattos", primary_key: None }} -2 {uid: 2, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "girafos", primary_key: None }} -3 {uid: 3, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} -4 {uid: 4, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "cattos" }} -5 {uid: 5, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "girafos" }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "cattos", primary_key: None }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "girafos", primary_key: None }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "cattos" }} +5 {uid: 5, batch_uid: 5, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "girafos" }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -59,7 +59,61 @@ girafos: { number_of_documents: 0, field_distribution: {} } [timestamp] [4,] [timestamp] [5,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,5,] +---------------------------------------------------------------------- +### Batches Kind: +"documentDeletion" [3,4,5,] +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +cattos [1,4,] +doggos [0,3,] +girafos [2,5,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition/after_register.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition/after_register.snap index c1869a475..07ab97def 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition/after_register.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition/after_register.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition/after_the_batch_creation.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition/after_the_batch_creation.snap index 33d58eab6..03484afe0 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition/after_the_batch_creation.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition/after_the_batch_creation.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition/once_everything_is_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition/once_everything_is_processed.snap index b4a6da3af..c194af741 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition/once_everything_is_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition/once_everything_is_processed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -34,7 +34,33 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/after_processing_the_batch.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/after_processing_the_batch.snap index b27288a0f..668f9d5bf 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/after_processing_the_batch.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/after_processing_the_batch.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_document_ids: 2, deleted_documents: Some(2) }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1", "2"] }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} +1 {uid: 1, batch_uid: 0, status: succeeded, details: { received_document_ids: 2, deleted_documents: Some(2) }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1", "2"] }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -37,7 +37,35 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } ### Finished At: [timestamp] [0,1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +"documentDeletion" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/registered_the_first_task.snap index d26e62bff..a26cb23aa 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/registered_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/registered_the_second_task.snap index e0f371120..9765289fb 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/registered_the_second_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_document_deletion/registered_the_second_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,8 +33,26 @@ doggos [0,1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/before_index_creation.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/before_index_creation.snap index 97ba419a8..f8642736c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/before_index_creation.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/before_index_creation.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { deleted_documents: None }, kind: IndexDeletion { index_uid: "doggos" }} ---------------------------------------------------------------------- @@ -40,8 +40,34 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/both_task_succeeded.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/both_task_succeeded.snap index fd96ee974..deafc4c81 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/both_task_succeeded.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/both_task_succeeded.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { deleted_documents: Some(0) }, kind: IndexDeletion { index_uid: "doggos" }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { deleted_documents: Some(0) }, kind: IndexDeletion { index_uid: "doggos" }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -41,7 +41,41 @@ doggos [0,1,2,] [timestamp] [0,] [timestamp] [1,2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"indexCreation" [0,] +"indexDeletion" [1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_first_task.snap index 1a29c1ac6..198956185 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_second_task.snap index 68ded1f0d..4e9b84e52 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_second_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_second_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,8 +33,26 @@ doggos [0,1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_third_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_third_task.snap index 1601a6b25..84e037b05 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_third_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion/registered_the_third_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -36,8 +36,26 @@ doggos [0,1,2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion_on_unexisting_index/1.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion_on_unexisting_index/1.snap index c200ce402..8f2bed791 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion_on_unexisting_index/1.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion_on_unexisting_index/1.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,8 +33,26 @@ doggos [0,1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion_on_unexisting_index/2.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion_on_unexisting_index/2.snap index 7884dfe08..ade21179b 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion_on_unexisting_index/2.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_addition_and_index_deletion_on_unexisting_index/2.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { deleted_documents: Some(0) }, kind: IndexDeletion { index_uid: "doggos" }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 0, status: succeeded, details: { deleted_documents: Some(0) }, kind: IndexDeletion { index_uid: "doggos" }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -36,7 +36,35 @@ doggos [0,1,] ### Finished At: [timestamp] [0,1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +"indexDeletion" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/after_failing_the_deletion.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/after_failing_the_deletion.snap index 1d4aa24e2..9a7b4686c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/after_failing_the_deletion.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/after_failing_the_deletion.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_document_ids: 2, deleted_documents: Some(0) }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1", "2"] }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_document_ids: 2, deleted_documents: Some(0) }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1", "2"] }} 1 {uid: 1, status: enqueued, details: { received_documents: 3, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -36,8 +36,34 @@ doggos [0,1,] ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentDeletion" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/after_last_successful_addition.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/after_last_successful_addition.snap index 0f9dfd3e6..60f71df9e 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/after_last_successful_addition.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/after_last_successful_addition.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_document_ids: 2, deleted_documents: Some(0) }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1", "2"] }} -1 {uid: 1, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_document_ids: 2, deleted_documents: Some(0) }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1", "2"] }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -40,7 +40,40 @@ doggos: { number_of_documents: 3, field_distribution: {"catto": 1, "doggo": 2, " [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [1,] +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"documentDeletion" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/registered_the_first_task.snap index 5753db7e6..acfdb5e51 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/registered_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/registered_the_second_task.snap index 0b6191f9e..9748964e3 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/registered_the_second_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/document_deletion_and_document_addition/registered_the_second_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,8 +33,26 @@ doggos [0,1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/document_addition_batch_created.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/document_addition_batch_created.snap index 33d58eab6..03484afe0 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/document_addition_batch_created.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/document_addition_batch_created.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/document_addition_failed.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/document_addition_failed.snap index e66ffa567..19881a91c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/document_addition_failed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/document_addition_failed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -33,7 +33,33 @@ doggos [0,] ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/registered_the_first_task.snap index c1869a475..07ab97def 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_addition/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_adding_the_documents.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_adding_the_documents.snap index 62e634bc5..74e001fba 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_adding_the_documents.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_adding_the_documents.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -39,6 +39,39 @@ doggos: { number_of_documents: 3, field_distribution: {"catto": 1, "doggo": 2, " [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_adding_the_settings.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_adding_the_settings.snap index 45065d8b1..19d7a0c0a 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_adding_the_settings.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_adding_the_settings.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 3, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -37,6 +37,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_removing_the_documents.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_removing_the_documents.snap index 82748751e..a6a036eee 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_removing_the_documents.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/after_removing_the_documents.snap @@ -1,17 +1,17 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_document_ids: 1, deleted_documents: Some(1) }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1"] }} -3 {uid: 3, status: failed, error: ResponseError { code: 200, message: "Invalid type for filter subexpression: expected: String, Array, found: true.", error_code: "invalid_document_filter", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#invalid_document_filter" }, details: { original_filter: true, deleted_documents: Some(0) }, kind: DocumentDeletionByFilter { index_uid: "doggos", filter_expr: Bool(true) }} -4 {uid: 4, status: failed, error: ResponseError { code: 200, message: "Attribute `id` is not filterable. Available filterable attributes are: `catto`.\n1:3 id = 2", error_code: "invalid_document_filter", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#invalid_document_filter" }, details: { original_filter: "id = 2", deleted_documents: Some(0) }, kind: DocumentDeletionByFilter { index_uid: "doggos", filter_expr: String("id = 2") }} -5 {uid: 5, status: succeeded, details: { original_filter: "catto EXISTS", deleted_documents: Some(1) }, kind: DocumentDeletionByFilter { index_uid: "doggos", filter_expr: String("catto EXISTS") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_document_ids: 1, deleted_documents: Some(1) }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1"] }} +3 {uid: 3, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Invalid type for filter subexpression: expected: String, Array, found: true.", error_code: "invalid_document_filter", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#invalid_document_filter" }, details: { original_filter: true, deleted_documents: Some(0) }, kind: DocumentDeletionByFilter { index_uid: "doggos", filter_expr: Bool(true) }} +4 {uid: 4, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Attribute `id` is not filterable. Available filterable attributes are: `catto`.\n1:3 id = 2", error_code: "invalid_document_filter", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#invalid_document_filter" }, details: { original_filter: "id = 2", deleted_documents: Some(0) }, kind: DocumentDeletionByFilter { index_uid: "doggos", filter_expr: String("id = 2") }} +5 {uid: 5, batch_uid: 2, status: succeeded, details: { original_filter: "catto EXISTS", deleted_documents: Some(1) }, kind: DocumentDeletionByFilter { index_uid: "doggos", filter_expr: String("catto EXISTS") }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -51,6 +51,47 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } [timestamp] [1,] [timestamp] [2,3,4,5,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,3,4,5,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,] +failed [2,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"documentDeletion" [2,] +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/registered_the_document_deletions.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/registered_the_document_deletions.snap index 502ff0806..093e693d1 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/registered_the_document_deletions.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/registered_the_document_deletions.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: Set({"catto"}), sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: NotSet, search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 3, indexed_documents: Some(3) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 3, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_document_ids: 1, deleted_documents: None }, kind: DocumentDeletion { index_uid: "doggos", documents_ids: ["1"] }} 3 {uid: 3, status: enqueued, details: { original_filter: true, deleted_documents: None }, kind: DocumentDeletionByFilter { index_uid: "doggos", filter_expr: Bool(true) }} 4 {uid: 4, status: enqueued, details: { original_filter: "id = 2", deleted_documents: None }, kind: DocumentDeletionByFilter { index_uid: "doggos", filter_expr: String("id = 2") }} @@ -48,6 +48,39 @@ doggos: { number_of_documents: 3, field_distribution: {"catto": 1, "doggo": 2, " [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/registered_the_setting_and_document_addition.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/registered_the_setting_and_document_addition.snap index f7e5c35d3..f6ff0721e 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/registered_the_setting_and_document_addition.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_document_deletion/registered_the_setting_and_document_addition.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,6 +33,25 @@ doggos [0,1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_index_creation/after_register.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_index_creation/after_register.snap index f252cc8ef..03d239939 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_index_creation/after_register.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_index_creation/after_register.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ catto [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_index_creation/index_creation_failed.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_index_creation/index_creation_failed.snap index 578b94061..154b1f35b 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_index_creation/index_creation_failed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_process_batch_for_index_creation/index_creation_failed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -33,7 +33,33 @@ catto [0,] ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/after_batch_succeeded.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/after_batch_succeeded.snap index 9c31fd318..177307427 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/after_batch_succeeded.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/after_batch_succeeded.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -31,8 +31,26 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/after_failing_to_commit.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/after_failing_to_commit.snap index 9c31fd318..177307427 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/after_failing_to_commit.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/after_failing_to_commit.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -31,8 +31,26 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/document_addition_succeeded_but_index_scheduler_not_updated.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/document_addition_succeeded_but_index_scheduler_not_updated.snap index c1869a475..07ab97def 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/document_addition_succeeded_but_index_scheduler_not_updated.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/document_addition_succeeded_but_index_scheduler_not_updated.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/registered_the_first_task.snap index c1869a475..07ab97def 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,8 +30,26 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/task_successfully_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/task_successfully_processed.snap index b4a6da3af..c194af741 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/task_successfully_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/fail_in_update_task_after_process_batch_success_for_document_addition/task_successfully_processed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -34,7 +34,33 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/Intel to kefir succeeds.snap b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/Intel to kefir succeeds.snap index 41cfcfdab..e2cdc8d46 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/Intel to kefir succeeds.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/Intel to kefir succeeds.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -43,6 +43,44 @@ doggos: { number_of_documents: 1, field_distribution: {"_vectors": 1, "breed": 1 [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,2,] +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/Intel to kefir.snap b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/Intel to kefir.snap index e6d0d8232..859e3b95d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/Intel to kefir.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/Intel to kefir.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -41,6 +41,39 @@ doggos: { number_of_documents: 1, field_distribution: {"_vectors": 1, "breed": 1 [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 diff --git a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/adding Intel succeeds.snap b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/adding Intel succeeds.snap index bd4cf0c09..44b25b149 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/adding Intel succeeds.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/adding Intel succeeds.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -39,6 +39,39 @@ doggos: { number_of_documents: 1, field_distribution: {"_vectors": 1, "breed": 1 [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/after adding Intel.snap b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/after adding Intel.snap index 746c7c870..4eefe83c2 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/after adding Intel.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/after adding Intel.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -37,6 +37,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 diff --git a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/after_registering_settings_task_vectors.snap b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/after_registering_settings_task_vectors.snap index 15cfd732a..96d060a92 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/after_registering_settings_task_vectors.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/after_registering_settings_task_vectors.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,6 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/settings_update_processed_vectors.snap b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/settings_update_processed_vectors.snap index 9b5c6ce4c..044791d61 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/settings_update_processed_vectors.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/import_vectors/settings_update_processed_vectors.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"A_fakerest": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(384), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet }), "B_small_hf": Set(EmbeddingSettings { source: Set(HuggingFace), model: Set("sentence-transformers/all-MiniLM-L6-v2"), revision: Set("e4ce9877abf3edfe10b0d82785e83bdcb973e22e"), api_key: NotSet, dimensions: NotSet, binary_quantized: NotSet, document_template: Set("{{doc.doggo}} the {{doc.breed}} best doggo"), document_template_max_bytes: NotSet, url: NotSet, request: NotSet, response: NotSet, headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -34,6 +34,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/after_batch_creation.snap b/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/after_batch_creation.snap index 3f34ca57b..eabfaed29 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/after_batch_creation.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/after_batch_creation.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ index_a [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_first_task.snap index f17bfe38f..05bad8454 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ index_a [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_second_task.snap index 75d5d8760..bf009e414 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_second_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_second_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -33,7 +33,25 @@ index_b [1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_third_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_third_task.snap index a69818dfa..9c86652db 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_third_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/insert_task_while_another_task_is_processing/registered_the_third_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [0,] ---------------------------------------------------------------------- ### All Tasks: @@ -36,7 +36,25 @@ index_b [1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/panic_in_process_batch_for_index_creation/index_creation_failed.snap b/crates/index-scheduler/src/snapshots/lib.rs/panic_in_process_batch_for_index_creation/index_creation_failed.snap index 9100e5075..17ed67aaf 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/panic_in_process_batch_for_index_creation/index_creation_failed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/panic_in_process_batch_for_index_creation/index_creation_failed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "An unexpected crash occurred when processing the task.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "An unexpected crash occurred when processing the task.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -33,7 +33,33 @@ catto [0,] ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/panic_in_process_batch_for_index_creation/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/panic_in_process_batch_for_index_creation/registered_the_first_task.snap index f252cc8ef..03d239939 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/panic_in_process_batch_for_index_creation/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/panic_in_process_batch_for_index_creation/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ catto [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_first_task.snap index a34531f2a..31fb88530 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_first_task.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} 1 {uid: 1, status: enqueued, details: { primary_key: None }, kind: IndexCreation { index_uid: "cattos", primary_key: None }} 2 {uid: 2, status: enqueued, details: { deleted_documents: None }, kind: IndexDeletion { index_uid: "doggos" }} ---------------------------------------------------------------------- @@ -40,7 +40,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_second_task.snap index f163ee3d7..7bafb0848 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_second_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_second_task.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "cattos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "cattos", primary_key: None }} 2 {uid: 2, status: enqueued, details: { deleted_documents: None }, kind: IndexDeletion { index_uid: "doggos" }} ---------------------------------------------------------------------- ### Status: @@ -43,7 +43,39 @@ doggos: { number_of_documents: 0, field_distribution: {} } [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +cattos [1,] +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_third_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_third_task.snap index 7e8db762f..b2967a78e 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_third_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/processed_the_third_task.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "cattos", primary_key: None }} -2 {uid: 2, status: succeeded, details: { deleted_documents: Some(0) }, kind: IndexDeletion { index_uid: "doggos" }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "cattos", primary_key: None }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { deleted_documents: Some(0) }, kind: IndexDeletion { index_uid: "doggos" }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -44,7 +44,45 @@ cattos: { number_of_documents: 0, field_distribution: {} } [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,] +"indexDeletion" [2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +cattos [1,] +doggos [0,2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_first_task.snap index 1a29c1ac6..198956185 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_second_task.snap index 4020c2db0..5460e7476 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_second_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_second_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,7 +33,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_third_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_third_task.snap index f25280bc1..610282102 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_third_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_inserted_without_new_signal/registered_the_third_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -36,7 +36,25 @@ doggos [0,2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/first.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/first.snap index d67c659c6..915a39cb0 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/first.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/first.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} 1 {uid: 1, status: enqueued, details: { deleted_documents: None }, kind: DocumentClear { index_uid: "doggos" }} 2 {uid: 2, status: enqueued, details: { deleted_documents: None }, kind: DocumentClear { index_uid: "doggos" }} 3 {uid: 3, status: enqueued, details: { deleted_documents: None }, kind: DocumentClear { index_uid: "doggos" }} @@ -41,7 +41,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/fourth.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/fourth.snap index 3d5c28e2c..dfe92d4b3 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/fourth.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/fourth.snap @@ -1,15 +1,15 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} -2 {uid: 2, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} -3 {uid: 3, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -47,7 +47,49 @@ doggos: { number_of_documents: 0, field_distribution: {} } [timestamp] [2,] [timestamp] [3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Kind: +"documentDeletion" [1,2,3,] +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_first_task.snap index c2ad519fa..d92fa144c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_fourth_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_fourth_task.snap index a9eb16d21..c75e0c550 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_fourth_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_fourth_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -37,7 +37,25 @@ doggos [0,1,2,3,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_second_task.snap index ad269dd7a..87f7406dd 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_second_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_second_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,7 +33,25 @@ doggos [0,1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_third_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_third_task.snap index 6267a840f..86cb34f85 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_third_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/registered_the_third_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -35,7 +35,25 @@ doggos [0,1,2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/second.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/second.snap index d45867c49..597e6704d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/second.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/second.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} 2 {uid: 2, status: enqueued, details: { deleted_documents: None }, kind: DocumentClear { index_uid: "doggos" }} 3 {uid: 3, status: enqueued, details: { deleted_documents: None }, kind: DocumentClear { index_uid: "doggos" }} ---------------------------------------------------------------------- @@ -43,7 +43,39 @@ doggos: { number_of_documents: 0, field_distribution: {} } [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentDeletion" [1,] +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/third.snap b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/third.snap index 9705c5321..85cf963be 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/third.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/process_tasks_without_autobatching/third.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} -2 {uid: 2, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { deleted_documents: Some(0) }, kind: DocumentClear { index_uid: "doggos" }} 3 {uid: 3, status: enqueued, details: { deleted_documents: None }, kind: DocumentClear { index_uid: "doggos" }} ---------------------------------------------------------------------- ### Status: @@ -45,7 +45,44 @@ doggos: { number_of_documents: 0, field_distribution: {} } [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,] +---------------------------------------------------------------------- +### Batches Kind: +"documentDeletion" [1,2,] +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/processed_all_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/processed_all_tasks.snap new file mode 100644 index 000000000..d22441b0b --- /dev/null +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/processed_all_tasks.snap @@ -0,0 +1,90 @@ +--- +source: crates/index-scheduler/src/lib.rs +--- +### Autobatching Enabled = true +### Processing batch None: +[] +---------------------------------------------------------------------- +### All Tasks: +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("his_own_vomit") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("his_own_vomit") }} +---------------------------------------------------------------------- +### Status: +enqueued [] +succeeded [0,1,2,] +---------------------------------------------------------------------- +### Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Index Tasks: +catto [2,] +doggo [0,] +whalo [1,] +---------------------------------------------------------------------- +### Index Mapper: +catto: { number_of_documents: 0, field_distribution: {} } +doggo: { number_of_documents: 0, field_distribution: {} } +whalo: { number_of_documents: 0, field_distribution: {} } + +---------------------------------------------------------------------- +### Canceled By: + +---------------------------------------------------------------------- +### Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [2,] +doggo [0,] +whalo [1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### File Store: + +---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_first_task.snap new file mode 100644 index 000000000..c6bf88bb0 --- /dev/null +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_first_task.snap @@ -0,0 +1,54 @@ +--- +source: crates/index-scheduler/src/lib.rs +--- +### Autobatching Enabled = true +### Processing batch None: +[] +---------------------------------------------------------------------- +### All Tasks: +0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }} +---------------------------------------------------------------------- +### Status: +enqueued [0,] +---------------------------------------------------------------------- +### Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Index Tasks: +doggo [0,] +---------------------------------------------------------------------- +### Index Mapper: + +---------------------------------------------------------------------- +### Canceled By: + +---------------------------------------------------------------------- +### Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Started At: +---------------------------------------------------------------------- +### Finished At: +---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- +### File Store: + +---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_second_task.snap new file mode 100644 index 000000000..cae2c4806 --- /dev/null +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_second_task.snap @@ -0,0 +1,57 @@ +--- +source: crates/index-scheduler/src/lib.rs +--- +### Autobatching Enabled = true +### Processing batch None: +[] +---------------------------------------------------------------------- +### All Tasks: +0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }} +1 {uid: 1, status: enqueued, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }} +---------------------------------------------------------------------- +### Status: +enqueued [0,1,] +---------------------------------------------------------------------- +### Kind: +"indexCreation" [0,1,] +---------------------------------------------------------------------- +### Index Tasks: +doggo [0,] +whalo [1,] +---------------------------------------------------------------------- +### Index Mapper: + +---------------------------------------------------------------------- +### Canceled By: + +---------------------------------------------------------------------- +### Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Started At: +---------------------------------------------------------------------- +### Finished At: +---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- +### File Store: + +---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_third_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_third_task.snap new file mode 100644 index 000000000..2346e5e68 --- /dev/null +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_from_and_limit/registered_the_third_task.snap @@ -0,0 +1,60 @@ +--- +source: crates/index-scheduler/src/lib.rs +--- +### Autobatching Enabled = true +### Processing batch None: +[] +---------------------------------------------------------------------- +### All Tasks: +0 {uid: 0, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }} +1 {uid: 1, status: enqueued, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }} +2 {uid: 2, status: enqueued, details: { primary_key: Some("his_own_vomit") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("his_own_vomit") }} +---------------------------------------------------------------------- +### Status: +enqueued [0,1,2,] +---------------------------------------------------------------------- +### Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Index Tasks: +catto [2,] +doggo [0,] +whalo [1,] +---------------------------------------------------------------------- +### Index Mapper: + +---------------------------------------------------------------------- +### Canceled By: + +---------------------------------------------------------------------- +### Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Started At: +---------------------------------------------------------------------- +### Finished At: +---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- +### File Store: + +---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_batches_simple/end.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_simple/end.snap new file mode 100644 index 000000000..b87a06b2b --- /dev/null +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_simple/end.snap @@ -0,0 +1,91 @@ +--- +source: crates/index-scheduler/src/lib.rs +--- +### Autobatching Enabled = true +### Processing batch None: +[] +---------------------------------------------------------------------- +### All Tasks: +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }} +2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }} +---------------------------------------------------------------------- +### Status: +enqueued [] +succeeded [0,1,] +failed [2,] +---------------------------------------------------------------------- +### Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Index Tasks: +catto [0,] +doggo [1,] +whalo [2,] +---------------------------------------------------------------------- +### Index Mapper: +catto: { number_of_documents: 0, field_distribution: {} } +doggo: { number_of_documents: 0, field_distribution: {} } + +---------------------------------------------------------------------- +### Canceled By: + +---------------------------------------------------------------------- +### Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +failed [2,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +doggo [1,] +whalo [2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### File Store: + +---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_batches_simple/start.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_simple/start.snap new file mode 100644 index 000000000..7428155e9 --- /dev/null +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_simple/start.snap @@ -0,0 +1,60 @@ +--- +source: crates/index-scheduler/src/lib.rs +--- +### Autobatching Enabled = true +### Processing batch None: +[] +---------------------------------------------------------------------- +### All Tasks: +0 {uid: 0, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} +1 {uid: 1, status: enqueued, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }} +2 {uid: 2, status: enqueued, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }} +---------------------------------------------------------------------- +### Status: +enqueued [0,1,2,] +---------------------------------------------------------------------- +### Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Index Tasks: +catto [0,] +doggo [1,] +whalo [2,] +---------------------------------------------------------------------- +### Index Mapper: + +---------------------------------------------------------------------- +### Canceled By: + +---------------------------------------------------------------------- +### Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Started At: +---------------------------------------------------------------------- +### Finished At: +---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- +### File Store: + +---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_batches_special_rules/after-processing-everything.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_special_rules/after-processing-everything.snap new file mode 100644 index 000000000..625017e91 --- /dev/null +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_special_rules/after-processing-everything.snap @@ -0,0 +1,102 @@ +--- +source: crates/index-scheduler/src/lib.rs +--- +### Autobatching Enabled = true +### Processing batch None: +[] +---------------------------------------------------------------------- +### All Tasks: +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }} +2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }} +3 {uid: 3, batch_uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `whalo` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }} +---------------------------------------------------------------------- +### Status: +enqueued [] +succeeded [0,1,] +failed [2,3,] +---------------------------------------------------------------------- +### Kind: +"indexCreation" [0,1,] +"indexSwap" [2,3,] +---------------------------------------------------------------------- +### Index Tasks: +catto [0,2,3,] +doggo [1,2,] +whalo [3,] +---------------------------------------------------------------------- +### Index Mapper: +catto: { number_of_documents: 0, field_distribution: {} } +doggo: { number_of_documents: 0, field_distribution: {} } + +---------------------------------------------------------------------- +### Canceled By: + +---------------------------------------------------------------------- +### Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +failed [2,3,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,] +"indexSwap" [2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,2,3,] +doggo [1,2,] +whalo [3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### File Store: + +---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_batches_special_rules/start.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_special_rules/start.snap new file mode 100644 index 000000000..f5a544f18 --- /dev/null +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_batches_special_rules/start.snap @@ -0,0 +1,63 @@ +--- +source: crates/index-scheduler/src/lib.rs +--- +### Autobatching Enabled = true +### Processing batch None: +[] +---------------------------------------------------------------------- +### All Tasks: +0 {uid: 0, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} +1 {uid: 1, status: enqueued, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }} +2 {uid: 2, status: enqueued, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }} +3 {uid: 3, status: enqueued, details: { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "whalo") }] }} +---------------------------------------------------------------------- +### Status: +enqueued [0,1,2,3,] +---------------------------------------------------------------------- +### Kind: +"indexCreation" [0,1,] +"indexSwap" [2,3,] +---------------------------------------------------------------------- +### Index Tasks: +catto [0,2,3,] +doggo [1,2,] +whalo [3,] +---------------------------------------------------------------------- +### Index Mapper: + +---------------------------------------------------------------------- +### Canceled By: + +---------------------------------------------------------------------- +### Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Started At: +---------------------------------------------------------------------- +### Finished At: +---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- +### File Store: + +---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/processed_all_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/processed_all_tasks.snap index a07c46427..d22441b0b 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/processed_all_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/processed_all_tasks.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("his_own_vomit") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("his_own_vomit") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("plankton") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("plankton") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("his_own_vomit") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("his_own_vomit") }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -46,7 +46,45 @@ whalo: { number_of_documents: 0, field_distribution: {} } [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [2,] +doggo [0,] +whalo [1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_first_task.snap index 8eb784c9b..c6bf88bb0 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ doggo [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_second_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_second_task.snap index 776d699e1..cae2c4806 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_second_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_second_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,7 +33,25 @@ whalo [1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_third_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_third_task.snap index 1a3fc5b53..2346e5e68 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_third_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_from_and_limit/registered_the_third_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -36,7 +36,25 @@ whalo [1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_simple/end.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_simple/end.snap index 6f59e49a2..b87a06b2b 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_simple/end.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_simple/end.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }} -2 {uid: 2, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }} +2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("fish") }, kind: IndexCreation { index_uid: "whalo", primary_key: Some("fish") }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -46,7 +46,46 @@ doggo: { number_of_documents: 0, field_distribution: {} } [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +failed [2,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +doggo [1,] +whalo [2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_simple/start.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_simple/start.snap index 1fa3fa1ab..7428155e9 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_simple/start.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_simple/start.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -36,7 +36,25 @@ whalo [2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_special_rules/start.snap b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_special_rules/start.snap index 4baa9d87a..f5a544f18 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_special_rules/start.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/query_tasks_special_rules/start.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -39,7 +39,25 @@ whalo [3,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/register/everything_is_successfully_registered.snap b/crates/index-scheduler/src/snapshots/lib.rs/register/everything_is_successfully_registered.snap index 62796a929..62a7ae13a 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/register/everything_is_successfully_registered.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/register/everything_is_successfully_registered.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -38,10 +38,28 @@ doggo [3,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_a.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_a.snap index 47a6dc6a7..91c15c3ee 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_a.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_a.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} 1 {uid: 1, status: enqueued, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} 2 {uid: 2, status: enqueued, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} 3 {uid: 3, status: enqueued, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} @@ -43,7 +43,33 @@ a: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_b.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_b.snap index 13b783feb..63e254e35 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_b.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_b.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} 2 {uid: 2, status: enqueued, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} 3 {uid: 3, status: enqueued, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} ---------------------------------------------------------------------- @@ -46,7 +46,39 @@ b: { number_of_documents: 0, field_distribution: {} } [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,] +b [1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_c.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_c.snap index 6b4bdb756..529fa818e 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_c.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_c.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} 3 {uid: 3, status: enqueued, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} ---------------------------------------------------------------------- ### Status: @@ -49,7 +49,45 @@ c: { number_of_documents: 0, field_distribution: {} } [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,] +b [1,] +c [2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_d.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_d.snap index 038903c59..9f47fbebd 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_d.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/create_d.snap @@ -1,15 +1,15 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -52,7 +52,51 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [2,] [timestamp] [3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,] +b [1,] +c [2,] +d [3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/first_swap_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/first_swap_processed.snap index c7ab30a36..74bf714b1 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/first_swap_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/first_swap_processed.snap @@ -1,16 +1,16 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -4 {uid: 4, status: succeeded, details: { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "d") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "d") }] }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "d") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "d") }] }} 5 {uid: 5, status: enqueued, details: { swaps: [IndexSwap { indexes: ("a", "c") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "c") }] }} ---------------------------------------------------------------------- ### Status: @@ -59,7 +59,57 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [3,] [timestamp] [4,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +"indexSwap" [4,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,4,] +b [1,4,] +c [2,4,] +d [3,4,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/first_swap_registered.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/first_swap_registered.snap index c0727d6e9..23cb14ac1 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/first_swap_registered.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/first_swap_registered.snap @@ -1,15 +1,15 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} 4 {uid: 4, status: enqueued, details: { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "d") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "d") }] }} ---------------------------------------------------------------------- ### Status: @@ -55,7 +55,51 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [2,] [timestamp] [3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,] +b [1,] +c [2,] +d [3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/second_swap_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/second_swap_processed.snap index 07acbe474..a279d967f 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/second_swap_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/second_swap_processed.snap @@ -1,17 +1,17 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -4 {uid: 4, status: succeeded, details: { swaps: [IndexSwap { indexes: ("c", "b") }, IndexSwap { indexes: ("a", "d") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("c", "b") }, IndexSwap { indexes: ("a", "d") }] }} -5 {uid: 5, status: succeeded, details: { swaps: [IndexSwap { indexes: ("a", "c") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "c") }] }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { swaps: [IndexSwap { indexes: ("c", "b") }, IndexSwap { indexes: ("a", "d") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("c", "b") }, IndexSwap { indexes: ("a", "d") }] }} +5 {uid: 5, batch_uid: 5, status: succeeded, details: { swaps: [IndexSwap { indexes: ("a", "c") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "c") }] }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -61,7 +61,62 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [4,] [timestamp] [5,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,5,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +"indexSwap" [4,5,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,4,5,] +b [1,4,] +c [2,4,5,] +d [3,4,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/third_empty_swap_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/third_empty_swap_processed.snap index aeeba29f8..e3be49476 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/third_empty_swap_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/third_empty_swap_processed.snap @@ -1,18 +1,18 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -4 {uid: 4, status: succeeded, details: { swaps: [IndexSwap { indexes: ("c", "b") }, IndexSwap { indexes: ("a", "d") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("c", "b") }, IndexSwap { indexes: ("a", "d") }] }} -5 {uid: 5, status: succeeded, details: { swaps: [IndexSwap { indexes: ("a", "c") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "c") }] }} -6 {uid: 6, status: succeeded, details: { swaps: [] }, kind: IndexSwap { swaps: [] }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { swaps: [IndexSwap { indexes: ("c", "b") }, IndexSwap { indexes: ("a", "d") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("c", "b") }, IndexSwap { indexes: ("a", "d") }] }} +5 {uid: 5, batch_uid: 5, status: succeeded, details: { swaps: [IndexSwap { indexes: ("a", "c") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "c") }] }} +6 {uid: 6, batch_uid: 6, status: succeeded, details: { swaps: [] }, kind: IndexSwap { swaps: [] }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -65,7 +65,67 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [5,] [timestamp] [6,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +6 {uid: 6, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +6 [6,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,5,6,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +"indexSwap" [4,5,6,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,4,5,] +b [1,4,] +c [2,4,5,] +d [3,4,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/two_swaps_registered.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/two_swaps_registered.snap index d1847fed1..2014150bb 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/two_swaps_registered.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes/two_swaps_registered.snap @@ -1,15 +1,15 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} 4 {uid: 4, status: enqueued, details: { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "d") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "d") }] }} 5 {uid: 5, status: enqueued, details: { swaps: [IndexSwap { indexes: ("a", "c") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "c") }] }} ---------------------------------------------------------------------- @@ -57,7 +57,51 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [2,] [timestamp] [3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,] +b [1,] +c [2,] +d [3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/after_the_index_creation.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/after_the_index_creation.snap index 038903c59..9f47fbebd 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/after_the_index_creation.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/after_the_index_creation.snap @@ -1,15 +1,15 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -52,7 +52,51 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [2,] [timestamp] [3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,] +b [1,] +c [2,] +d [3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/first_swap_failed.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/first_swap_failed.snap index 4632820f0..df676be58 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/first_swap_failed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/first_swap_failed.snap @@ -1,16 +1,16 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} -4 {uid: 4, status: failed, error: ResponseError { code: 200, message: "Indexes `e`, `f` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "e") }, IndexSwap { indexes: ("d", "f") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "e") }, IndexSwap { indexes: ("d", "f") }] }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +4 {uid: 4, batch_uid: 4, status: failed, error: ResponseError { code: 200, message: "Indexes `e`, `f` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "e") }, IndexSwap { indexes: ("d", "f") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("a", "b") }, IndexSwap { indexes: ("c", "e") }, IndexSwap { indexes: ("d", "f") }] }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -60,7 +60,60 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [3,] [timestamp] [4,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,] +failed [4,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +"indexSwap" [4,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,4,] +b [1,4,] +c [2,4,] +d [3,4,] +e [4,] +f [4,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/initial_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/initial_tasks_processed.snap index 038903c59..9f47fbebd 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/initial_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/swap_indexes_errors/initial_tasks_processed.snap @@ -1,15 +1,15 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} -1 {uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} -2 {uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} -3 {uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "a", primary_key: Some("id") }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "b", primary_key: Some("id") }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "c", primary_key: Some("id") }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { primary_key: Some("id") }, kind: IndexCreation { index_uid: "d", primary_key: Some("id") }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -52,7 +52,51 @@ d: { number_of_documents: 0, field_distribution: {} } [timestamp] [2,] [timestamp] [3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +a [0,] +b [1,] +c [2,] +d [3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/initial_tasks_enqueued.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/initial_tasks_enqueued.snap index 4b4a50bfb..88e7db81c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/initial_tasks_enqueued.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/initial_tasks_enqueued.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,9 +33,27 @@ doggo [1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/initial_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/initial_tasks_processed.snap index 6e3a6e8ed..50d9a972a 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/initial_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/initial_tasks_processed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggo", primary_key: Some("bone"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -37,8 +37,34 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/task_deletion_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/task_deletion_processed.snap index 83cfcdf07..72dc1bfae 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/task_deletion_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_delete_same_task_twice/task_deletion_processed.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggo", primary_key: Some("bone"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { matched_tasks: 1, deleted_tasks: Some(1), original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0]> }} -3 {uid: 3, status: succeeded, details: { matched_tasks: 1, deleted_tasks: Some(0), original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0]> }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { matched_tasks: 1, deleted_tasks: Some(1), original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0]> }} +3 {uid: 3, batch_uid: 1, status: succeeded, details: { matched_tasks: 1, deleted_tasks: Some(0), original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0]> }} ---------------------------------------------------------------------- ### Status: enqueued [1,] @@ -39,8 +39,41 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [2,3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [2,3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +"taskDeletion" [1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/after_registering_the_task_deletion.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/after_registering_the_task_deletion.snap index 302fd5f5e..42d528498 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/after_registering_the_task_deletion.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/after_registering_the_task_deletion.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggo", primary_key: Some("bone"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { matched_tasks: 1, deleted_tasks: None, original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0]> }} ---------------------------------------------------------------------- @@ -40,8 +40,34 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/initial_tasks_enqueued.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/initial_tasks_enqueued.snap index 4b4a50bfb..88e7db81c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/initial_tasks_enqueued.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/initial_tasks_enqueued.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -33,9 +33,27 @@ doggo [1,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/initial_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/initial_tasks_processed.snap index 6e3a6e8ed..50d9a972a 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/initial_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/initial_tasks_processed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggo", primary_key: Some("bone"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -37,8 +37,34 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/task_deletion_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/task_deletion_processed.snap index cf64406b8..71e834518 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/task_deletion_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_deleteable/task_deletion_processed.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggo", primary_key: Some("bone"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { matched_tasks: 1, deleted_tasks: Some(1), original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0]> }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { matched_tasks: 1, deleted_tasks: Some(1), original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0]> }} ---------------------------------------------------------------------- ### Status: enqueued [1,] @@ -37,8 +37,40 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} } ### Finished At: [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +"taskDeletion" [1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +catto [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/initial_tasks_enqueued.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/initial_tasks_enqueued.snap index 25e2deadc..4be93852c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/initial_tasks_enqueued.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/initial_tasks_enqueued.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -36,9 +36,27 @@ doggo [2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_done.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_done.snap index 743cd615f..834dd2e74 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_done.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_done.snap @@ -1,15 +1,15 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: 0 {uid: 0, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "catto", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggo", primary_key: Some("bone"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { matched_tasks: 2, deleted_tasks: Some(0), original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0, 1]> }} +3 {uid: 3, batch_uid: 0, status: succeeded, details: { matched_tasks: 2, deleted_tasks: Some(0), original_filter: "test_query" }, kind: TaskDeletion { query: "test_query", tasks: RoaringBitmap<[0, 1]> }} ---------------------------------------------------------------------- ### Status: enqueued [0,1,2,] @@ -42,9 +42,34 @@ doggo [2,] ### Finished At: [timestamp] [3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"taskDeletion" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_enqueued.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_enqueued.snap index 5c4d9be04..f5d93baa3 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_enqueued.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_enqueued.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -39,9 +39,27 @@ doggo [2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_processing.snap b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_processing.snap index 49df62cb7..d9e7733d4 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_processing.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/task_deletion_undeleteable/task_deletion_processing.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch Some(0): [3,] ---------------------------------------------------------------------- ### All Tasks: @@ -39,9 +39,27 @@ doggo [2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/after_the_second_task_deletion.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/after_the_second_task_deletion.snap index 59948c58c..03213fbb0 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/after_the_second_task_deletion.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/after_the_second_task_deletion.snap @@ -1,9 +1,10 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- [ { "uid": 3, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -24,6 +25,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 5, + "batchUid": 4, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/everything_has_been_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/everything_has_been_processed.snap index 0f2f366e9..cc38f69a0 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/everything_has_been_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/everything_has_been_processed.snap @@ -1,9 +1,10 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- [ { "uid": 6, + "batchUid": 6, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_deletion_have_been_enqueued.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_deletion_have_been_enqueued.snap index dc6b03517..3400d8950 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_deletion_have_been_enqueued.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_deletion_have_been_enqueued.snap @@ -1,9 +1,10 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- [ { "uid": 0, + "batchUid": 0, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -24,6 +25,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 1, + "batchUid": 1, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -49,6 +51,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 2, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -69,6 +72,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 3, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -89,6 +93,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 4, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_deletion_have_been_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_deletion_have_been_processed.snap index 0200f7f4a..ab4210bed 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_deletion_have_been_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_deletion_have_been_processed.snap @@ -1,9 +1,10 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- [ { "uid": 2, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -24,6 +25,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 3, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -44,6 +46,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 4, + "batchUid": 2, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_queue_is_full.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_queue_is_full.snap index 988df76ec..8b69b1cc2 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_queue_is_full.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_auto_deletion_of_tasks/task_queue_is_full.snap @@ -1,9 +1,10 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- [ { "uid": 0, + "batchUid": 0, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -24,6 +25,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 1, + "batchUid": 1, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -49,6 +51,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 2, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -69,6 +72,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 3, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_disable_auto_deletion_of_tasks/task_deletion_have_not_been_enqueued.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_disable_auto_deletion_of_tasks/task_deletion_have_not_been_enqueued.snap index 988df76ec..8b69b1cc2 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_disable_auto_deletion_of_tasks/task_deletion_have_not_been_enqueued.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_disable_auto_deletion_of_tasks/task_deletion_have_not_been_enqueued.snap @@ -1,9 +1,10 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- [ { "uid": 0, + "batchUid": 0, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -24,6 +25,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 1, + "batchUid": 1, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -49,6 +51,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 2, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -69,6 +72,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 3, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_disable_auto_deletion_of_tasks/task_queue_is_full.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_disable_auto_deletion_of_tasks/task_queue_is_full.snap index 988df76ec..8b69b1cc2 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_disable_auto_deletion_of_tasks/task_queue_is_full.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_disable_auto_deletion_of_tasks/task_queue_is_full.snap @@ -1,9 +1,10 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- [ { "uid": 0, + "batchUid": 0, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -24,6 +25,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 1, + "batchUid": 1, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -49,6 +51,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 2, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", @@ -69,6 +72,7 @@ source: index-scheduler/src/lib.rs }, { "uid": 3, + "batchUid": null, "enqueuedAt": "[date]", "startedAt": "[date]", "finishedAt": "[date]", diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/after_processing_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/after_processing_the_10_tasks.snap index cb6ec63de..10149200c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/after_processing_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/after_processing_the_10_tasks.snap @@ -1,22 +1,22 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} -10 {uid: 10, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: false }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} +3 {uid: 3, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} +4 {uid: 4, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} +5 {uid: 5, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} +6 {uid: 6, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} +7 {uid: 7, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} +8 {uid: 8, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} +9 {uid: 9, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} +10 {uid: 10, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: false }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -57,7 +57,40 @@ doggos: { number_of_documents: 10, field_distribution: {"doggo": 10, "id": 10} } [timestamp] [0,] [timestamp] [1,2,3,4,5,6,7,8,9,10,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,2,3,4,5,6,7,8,9,10,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/after_registering_the_10_tasks.snap index 2e4845fb4..fdbef5f31 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/after_registering_the_10_tasks.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} @@ -55,6 +55,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -68,4 +95,3 @@ doggos: { number_of_documents: 0, field_distribution: {} } 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/processed_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/processed_the_first_task.snap index e046ed386..0260edea5 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/processed_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/processed_the_first_task.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -34,7 +34,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/registered_the_first_task.snap index 1a29c1ac6..198956185 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/after_registering_the_10_tasks.snap index e2e2133f5..6e4179250 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/after_registering_the_10_tasks.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} @@ -55,6 +55,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -68,4 +95,3 @@ doggos: { number_of_documents: 0, field_distribution: {} } 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/all_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/all_tasks_processed.snap index 970587ba2..c66a50d47 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/all_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/all_tasks_processed.snap @@ -1,22 +1,22 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} -10 {uid: 10, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: false }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} +5 {uid: 5, batch_uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} +6 {uid: 6, batch_uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} +7 {uid: 7, batch_uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} +8 {uid: 8, batch_uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} +9 {uid: 9, batch_uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} +10 {uid: 10, batch_uid: 10, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: false }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -75,7 +75,84 @@ doggos: { number_of_documents: 10, field_distribution: {"doggo": 10, "id": 10} } [timestamp] [9,] [timestamp] [10,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +6 {uid: 6, } +7 {uid: 7, } +8 {uid: 8, } +9 {uid: 9, } +10 {uid: 10, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +6 [6,] +7 [7,] +8 [8,] +9 [9,] +10 [10,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,5,6,7,8,9,10,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,2,3,4,5,6,7,8,9,10,] +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,5,6,7,8,9,10,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +[timestamp] [10,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +[timestamp] [10,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +[timestamp] [10,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/five_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/five_tasks_processed.snap index 5de505cf2..8440c7113 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/five_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/five_tasks_processed.snap @@ -1,17 +1,17 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} +5 {uid: 5, batch_uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} 6 {uid: 6, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} 7 {uid: 7, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} 8 {uid: 8, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} @@ -65,6 +65,59 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "id": 5} } [timestamp] [4,] [timestamp] [5,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,5,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,2,3,4,5,] +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,5,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000005 00000000-0000-0000-0000-000000000006 @@ -73,4 +126,3 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "id": 5} } 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/processed_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/processed_the_first_task.snap index 606367737..45eb1b027 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/processed_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/processed_the_first_task.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -34,7 +34,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/registered_the_first_task.snap index c2ad519fa..d92fa144c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_with_index_without_autobatching/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index/after_processing_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index/after_processing_the_10_tasks.snap index 5a6ea9100..9600955a0 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index/after_processing_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index/after_processing_the_10_tasks.snap @@ -1,21 +1,21 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} -2 {uid: 2, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} -3 {uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} -4 {uid: 4, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} -5 {uid: 5, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} -6 {uid: 6, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} -7 {uid: 7, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} -8 {uid: 8, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} -9 {uid: 9, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: false }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +1 {uid: 1, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} +2 {uid: 2, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} +3 {uid: 3, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} +4 {uid: 4, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} +5 {uid: 5, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} +6 {uid: 6, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} +7 {uid: 7, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} +8 {uid: 8, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} +9 {uid: 9, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: false }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -51,7 +51,34 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ### Finished At: [timestamp] [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index/after_registering_the_10_tasks.snap index e2217e1f4..06cc66a60 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index/after_registering_the_10_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -48,6 +48,25 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -61,4 +80,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/after_registering_the_10_tasks.snap index 02d0b1988..74111aac3 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/after_registering_the_10_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -48,6 +48,25 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -61,4 +80,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/all_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/all_tasks_processed.snap index 8164394c8..2fe601741 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/all_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/all_tasks_processed.snap @@ -1,21 +1,21 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} -2 {uid: 2, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} -3 {uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} -4 {uid: 4, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} -5 {uid: 5, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} -6 {uid: 6, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} -7 {uid: 7, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} -8 {uid: 8, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} -9 {uid: 9, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: false }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} +2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} +3 {uid: 3, batch_uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} +4 {uid: 4, batch_uid: 4, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} +5 {uid: 5, batch_uid: 5, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} +6 {uid: 6, batch_uid: 6, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} +7 {uid: 7, batch_uid: 7, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} +8 {uid: 8, batch_uid: 8, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} +9 {uid: 9, batch_uid: 9, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: false }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -69,7 +69,78 @@ doggos [0,1,2,3,4,5,6,7,8,9,] [timestamp] [8,] [timestamp] [9,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +6 {uid: 6, } +7 {uid: 7, } +8 {uid: 8, } +9 {uid: 9, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +6 [6,] +7 [7,] +8 [8,] +9 [9,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/five_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/five_tasks_processed.snap index 4e0bb97ab..bc6fecb2a 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/five_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_cant_create_index_without_index_without_autobatching/five_tasks_processed.snap @@ -1,16 +1,16 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} -2 {uid: 2, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} -3 {uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} -4 {uid: 4, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: false }} +2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} +3 {uid: 3, batch_uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: false }} +4 {uid: 4, batch_uid: 4, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} 5 {uid: 5, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: false }} 6 {uid: 6, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} 7 {uid: 7, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: false }} @@ -59,6 +59,53 @@ doggos [0,1,2,3,4,5,6,7,8,9,] [timestamp] [3,] [timestamp] [4,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000005 00000000-0000-0000-0000-000000000006 @@ -67,4 +114,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/after_registering_the_10_tasks.snap index f98802f21..8ee0a9b12 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/after_registering_the_10_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -48,6 +48,25 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -61,4 +80,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/all_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/all_tasks_processed.snap index ea6ef400a..0e225ea8d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/all_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/all_tasks_processed.snap @@ -1,21 +1,21 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} +3 {uid: 3, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} +5 {uid: 5, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +6 {uid: 6, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} +7 {uid: 7, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} +8 {uid: 8, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} +9 {uid: 9, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -55,7 +55,40 @@ doggos: { number_of_documents: 9, field_distribution: {"doggo": 9, "id": 9} } [timestamp] [0,] [timestamp] [1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [1,] +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/only_first_task_failed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/only_first_task_failed.snap index c27a1b5cb..ac3c29fcb 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/only_first_task_failed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_right_without_index_starts_with_cant_create/only_first_task_failed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Index `doggos` not found.", error_code: "index_not_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_not_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} @@ -51,6 +51,33 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 @@ -63,4 +90,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/after_registering_the_10_tasks.snap index 68599f03e..1a0f4a54d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/after_registering_the_10_tasks.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} @@ -55,6 +55,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -68,4 +95,3 @@ doggos: { number_of_documents: 0, field_distribution: {} } 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/all_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/all_tasks_processed.snap index b442a9c2f..feec34884 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/all_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/all_tasks_processed.snap @@ -1,22 +1,22 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} -10 {uid: 10, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: false }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: false }} +4 {uid: 4, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +5 {uid: 5, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: false }} +6 {uid: 6, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +7 {uid: 7, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: false }} +8 {uid: 8, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} +9 {uid: 9, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: false }} +10 {uid: 10, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -57,7 +57,40 @@ doggos: { number_of_documents: 10, field_distribution: {"doggo": 10, "id": 10} } [timestamp] [0,] [timestamp] [1,2,3,4,5,6,7,8,9,10,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,2,3,4,5,6,7,8,9,10,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [1,] +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/processed_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/processed_the_first_task.snap index e046ed386..0260edea5 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/processed_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/processed_the_first_task.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: None }, kind: IndexCreation { index_uid: "doggos", primary_key: None }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -34,7 +34,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"indexCreation" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/registered_the_first_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/registered_the_first_task.snap index 1a29c1ac6..198956185 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/registered_the_first_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_mixed_rights_with_index/registered_the_first_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/after_registering_the_5_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/after_registering_the_5_tasks.snap index fde8e3451..f0f74b328 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/after_registering_the_5_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/after_registering_the_5_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -38,6 +38,25 @@ doggos [0,1,2,3,4,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -46,4 +65,3 @@ doggos [0,1,2,3,4,] 00000000-0000-0000-0000-000000000004 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/fifth_task_succeeds.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/fifth_task_succeeds.snap index 85d7ba460..953297977 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/fifth_task_succeeds.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/fifth_task_succeeds.snap @@ -1,16 +1,16 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":0,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":0,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -49,7 +49,50 @@ doggos: { number_of_documents: 2, field_distribution: {"doggo": 2, "id": 2} } [timestamp] [3,] [timestamp] [4,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,] +1 [2,] +2 [3,] +3 [4,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [1,3,] +failed [0,2,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/first_and_second_task_fails.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/first_and_second_task_fails.snap index ab788cc36..502fefbfd 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/first_and_second_task_fails.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/first_and_second_task_fails.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":0,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":0,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} 4 {uid: 4, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} @@ -42,10 +42,37 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000002 00000000-0000-0000-0000-000000000003 00000000-0000-0000-0000-000000000004 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/fourth_task_fails.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/fourth_task_fails.snap index 2a8748657..95c76085b 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/fourth_task_fails.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/fourth_task_fails.snap @@ -1,15 +1,15 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":0,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":0,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} 4 {uid: 4, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -47,8 +47,46 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } [timestamp] [2,] [timestamp] [3,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,] +1 [2,] +2 [3,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [1,] +failed [0,2,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000004 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/third_task_succeeds.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/third_task_succeeds.snap index 58cdbe432..e29c5e90c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/third_task_succeeds.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_bad_primary_key/third_task_succeeds.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":0,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":0,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"id\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} 4 {uid: 4, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- @@ -45,9 +45,42 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } [timestamp] [0,1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,] +1 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [1,] +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000003 00000000-0000-0000-0000-000000000004 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/after_registering_the_3_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/after_registering_the_3_tasks.snap index 5699c58c4..3ea317c43 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/after_registering_the_3_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/after_registering_the_3_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -34,10 +34,28 @@ doggos [0,1,2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/only_first_task_succeed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/only_first_task_succeed.snap index 85689fa4d..651b3cf5b 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/only_first_task_succeed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/only_first_task_succeed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bloup"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- @@ -38,9 +38,35 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/second_task_fails.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/second_task_fails.snap index 18db64ca4..6fca659b1 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/second_task_fails.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/second_task_fails.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bloup"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -41,8 +41,40 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +failed [1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/third_task_fails.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/third_task_fails.snap index 4a9edc602..18baf7207 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/third_task_fails.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key/third_task_fails.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bloup"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bloup"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -43,7 +43,44 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +failed [1,2,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/after_registering_the_3_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/after_registering_the_3_tasks.snap index 1187159b7..9df0f214d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/after_registering_the_3_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/after_registering_the_3_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -34,10 +34,28 @@ doggos [0,1,2,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/only_first_task_succeed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/only_first_task_succeed.snap index 22b91bb60..cfe528340 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/only_first_task_succeed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/only_first_task_succeed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- @@ -38,9 +38,35 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/second_and_third_tasks_fails.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/second_and_third_tasks_fails.snap index 05eca0af1..7ed185b9b 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/second_and_third_tasks_fails.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_multiple_primary_key_batch_wrong_key/second_and_third_tasks_fails.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `id`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: @@ -41,8 +41,40 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} } [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +failed [1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000002 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/after_registering_the_6_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/after_registering_the_6_tasks.snap index 47b0994ec..1e6cff26c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/after_registering_the_6_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/after_registering_the_6_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -40,6 +40,25 @@ doggos [0,1,2,3,4,5,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -49,4 +68,3 @@ doggos [0,1,2,3,4,5,] 00000000-0000-0000-0000-000000000005 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/all_other_tasks_succeeds.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/all_other_tasks_succeeds.snap index a7464685a..577938932 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/all_other_tasks_succeeds.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/all_other_tasks_succeeds.snap @@ -1,17 +1,17 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "The primary key inference failed as the engine did not find any field ending with `id` in its name. Please specify the primary key manually using the `primaryKey` query parameter.", error_code: "index_primary_key_no_candidate_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_no_candidate_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"paw\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "The primary key inference failed as the engine did not find any field ending with `id` in its name. Please specify the primary key manually using the `primaryKey` query parameter.", error_code: "index_primary_key_no_candidate_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_no_candidate_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"paw\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +5 {uid: 5, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -51,7 +51,50 @@ doggos: { number_of_documents: 4, field_distribution: {"doggo": 4, "paw": 4} } [timestamp] [2,] [timestamp] [3,4,5,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,4,5,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [2,3,] +failed [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/first_task_fails.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/first_task_fails.snap index 5c7f7e8d8..e807035a9 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/first_task_fails.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/first_task_fails.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "The primary key inference failed as the engine did not find any field ending with `id` in its name. Please specify the primary key manually using the `primaryKey` query parameter.", error_code: "index_primary_key_no_candidate_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_no_candidate_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "The primary key inference failed as the engine did not find any field ending with `id` in its name. Please specify the primary key manually using the `primaryKey` query parameter.", error_code: "index_primary_key_no_candidate_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_no_candidate_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} @@ -44,6 +44,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 @@ -52,4 +79,3 @@ doggos: { number_of_documents: 0, field_distribution: {} } 00000000-0000-0000-0000-000000000005 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/second_task_fails.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/second_task_fails.snap index 2c255aeba..9e718473f 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/second_task_fails.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/second_task_fails.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "The primary key inference failed as the engine did not find any field ending with `id` in its name. Please specify the primary key manually using the `primaryKey` query parameter.", error_code: "index_primary_key_no_candidate_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_no_candidate_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"paw\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "The primary key inference failed as the engine did not find any field ending with `id` in its name. Please specify the primary key manually using the `primaryKey` query parameter.", error_code: "index_primary_key_no_candidate_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_no_candidate_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"paw\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} 4 {uid: 4, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} @@ -46,6 +46,38 @@ doggos: { number_of_documents: 0, field_distribution: {} } [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +failed [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000002 00000000-0000-0000-0000-000000000003 @@ -53,4 +85,3 @@ doggos: { number_of_documents: 0, field_distribution: {} } 00000000-0000-0000-0000-000000000005 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/third_task_succeeds.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/third_task_succeeds.snap index 2f944adcf..ec4449e29 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/third_task_succeeds.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key/third_task_succeeds.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: failed, error: ResponseError { code: 200, message: "The primary key inference failed as the engine did not find any field ending with `id` in its name. Please specify the primary key manually using the `primaryKey` query parameter.", error_code: "index_primary_key_no_candidate_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_no_candidate_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"paw\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "The primary key inference failed as the engine did not find any field ending with `id` in its name. Please specify the primary key manually using the `primaryKey` query parameter.", error_code: "index_primary_key_no_candidate_found", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_no_candidate_found" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Document doesn't have a `bork` attribute: `{\"paw\":1,\"doggo\":\"jean bob\"}`.", error_code: "missing_document_id", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#missing_document_id" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} 4 {uid: 4, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} 5 {uid: 5, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("paw"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} @@ -49,10 +49,47 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "paw": 1} } [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [2,] +failed [0,1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000003 00000000-0000-0000-0000-000000000004 00000000-0000-0000-0000-000000000005 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/after_registering_the_6_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/after_registering_the_6_tasks.snap index c89abf033..de01fce5a 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/after_registering_the_6_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/after_registering_the_6_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -40,6 +40,25 @@ doggos [0,1,2,3,4,5,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -49,4 +68,3 @@ doggos [0,1,2,3,4,5,] 00000000-0000-0000-0000-000000000005 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/all_other_tasks_succeeds.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/all_other_tasks_succeeds.snap index 1f55e8f6d..b4d54f680 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/all_other_tasks_succeeds.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/all_other_tasks_succeeds.snap @@ -1,17 +1,17 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `doggoid`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `doggoid`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +5 {uid: 5, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -51,7 +51,50 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "doggoid": 5} [timestamp] [2,] [timestamp] [3,4,5,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,4,5,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,2,3,] +failed [1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/first_task_succeed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/first_task_succeed.snap index 532cef143..8f4ceb94e 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/first_task_succeed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/first_task_succeed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} 1 {uid: 1, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} @@ -44,6 +44,33 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "doggoid": 1} ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000001 00000000-0000-0000-0000-000000000002 @@ -52,4 +79,3 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "doggoid": 1} 00000000-0000-0000-0000-000000000005 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/second_task_fails.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/second_task_fails.snap index 2920c3b2e..350ace217 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/second_task_fails.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/second_task_fails.snap @@ -1,13 +1,13 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `doggoid`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `doggoid`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} 2 {uid: 2, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} 4 {uid: 4, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} @@ -47,6 +47,39 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "doggoid": 1} [timestamp] [0,] [timestamp] [1,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +failed [1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000002 00000000-0000-0000-0000-000000000003 @@ -54,4 +87,3 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "doggoid": 1} 00000000-0000-0000-0000-000000000005 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/third_task_succeeds.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/third_task_succeeds.snap index 8981dd8c3..f908e7a8c 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/third_task_succeeds.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_addition_with_set_and_null_primary_key_inference_works/third_task_succeeds.snap @@ -1,14 +1,14 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `doggoid`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: failed, error: ResponseError { code: 200, message: "Index already has a primary key: `doggoid`.", error_code: "index_primary_key_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_primary_key_already_exists" }, details: { received_documents: 1, indexed_documents: Some(0) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("bork"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} 3 {uid: 3, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} 4 {uid: 4, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: None, method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} 5 {uid: 5, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("doggoid"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} @@ -49,10 +49,47 @@ doggos: { number_of_documents: 2, field_distribution: {"doggo": 2, "doggoid": 2} [timestamp] [1,] [timestamp] [2,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,2,] +failed [1,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000003 00000000-0000-0000-0000-000000000004 00000000-0000-0000-0000-000000000005 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace/1.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace/1.snap index 21581fe78..355ef8821 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace/1.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace/1.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -48,6 +48,25 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -61,4 +80,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace/2.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace/2.snap index fabf764bc..c3ff75f58 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace/2.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace/2.snap @@ -1,21 +1,21 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +5 {uid: 5, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +6 {uid: 6, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} +7 {uid: 7, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} +8 {uid: 8, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} +9 {uid: 9, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -52,7 +52,34 @@ doggos: { number_of_documents: 10, field_distribution: {"doggo": 10, "id": 10} } ### Finished At: [timestamp] [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/after_registering_the_10_tasks.snap index 1beba7264..88d9addb8 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/after_registering_the_10_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -48,6 +48,25 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -61,4 +80,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/all_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/all_tasks_processed.snap index a0b121320..40b51ed14 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/all_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/all_tasks_processed.snap @@ -1,21 +1,21 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +5 {uid: 5, batch_uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +6 {uid: 6, batch_uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} +7 {uid: 7, batch_uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} +8 {uid: 8, batch_uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} +9 {uid: 9, batch_uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -70,7 +70,78 @@ doggos: { number_of_documents: 10, field_distribution: {"doggo": 10, "id": 10} } [timestamp] [8,] [timestamp] [9,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +6 {uid: 6, } +7 {uid: 7, } +8 {uid: 8, } +9 {uid: 9, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +6 [6,] +7 [7,] +8 [8,] +9 [9,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/five_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/five_tasks_processed.snap index 88bf59f8e..62ea510d1 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/five_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_replace_without_autobatching/five_tasks_processed.snap @@ -1,16 +1,16 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} 5 {uid: 5, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} 6 {uid: 6, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} 7 {uid: 7, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} @@ -60,6 +60,53 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "id": 5} } [timestamp] [3,] [timestamp] [4,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000005 00000000-0000-0000-0000-000000000006 @@ -68,4 +115,3 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "id": 5} } 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update/1.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update/1.snap index 53f0fbc9b..7aeb7da17 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update/1.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update/1.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -48,6 +48,25 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -61,4 +80,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update/2.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update/2.snap index d8383818e..a922b0595 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update/2.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update/2.snap @@ -1,21 +1,21 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +5 {uid: 5, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +6 {uid: 6, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} +7 {uid: 7, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} +8 {uid: 8, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} +9 {uid: 9, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -52,7 +52,34 @@ doggos: { number_of_documents: 10, field_distribution: {"doggo": 10, "id": 10} } ### Finished At: [timestamp] [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/after_registering_the_10_tasks.snap index 6785786e3..fea75fb83 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/after_registering_the_10_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -48,6 +48,25 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -61,4 +80,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/all_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/all_tasks_processed.snap index 4c8fd7ea6..2f8a5e4d8 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/all_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/all_tasks_processed.snap @@ -1,21 +1,21 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +5 {uid: 5, batch_uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +6 {uid: 6, batch_uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} +7 {uid: 7, batch_uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} +8 {uid: 8, batch_uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} +9 {uid: 9, batch_uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -70,7 +70,78 @@ doggos: { number_of_documents: 10, field_distribution: {"doggo": 10, "id": 10} } [timestamp] [8,] [timestamp] [9,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +6 {uid: 6, } +7 {uid: 7, } +8 {uid: 8, } +9 {uid: 9, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +6 [6,] +7 [7,] +8 [8,] +9 [9,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/five_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/five_tasks_processed.snap index f184bb3cb..ca1f46581 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/five_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_document_update_without_autobatching/five_tasks_processed.snap @@ -1,16 +1,16 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = false -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} 5 {uid: 5, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} 6 {uid: 6, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} 7 {uid: 7, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} @@ -60,6 +60,53 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "id": 5} } [timestamp] [3,] [timestamp] [4,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000005 00000000-0000-0000-0000-000000000006 @@ -68,4 +115,3 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "id": 5} } 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/after_registering_the_10_tasks.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/after_registering_the_10_tasks.snap index 9095f031b..6575b835d 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/after_registering_the_10_tasks.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/after_registering_the_10_tasks.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -48,6 +48,25 @@ doggos [0,1,2,3,4,5,6,7,8,9,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000000 00000000-0000-0000-0000-000000000001 @@ -61,4 +80,3 @@ doggos [0,1,2,3,4,5,6,7,8,9,] 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/all_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/all_tasks_processed.snap index 6c6fba517..453ff8c61 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/all_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/all_tasks_processed.snap @@ -1,21 +1,21 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} -5 {uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} -6 {uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} -7 {uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} -8 {uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} -9 {uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +5 {uid: 5, batch_uid: 5, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} +6 {uid: 6, batch_uid: 6, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} +7 {uid: 7, batch_uid: 7, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} +8 {uid: 8, batch_uid: 8, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000008, documents_count: 1, allow_index_creation: true }} +9 {uid: 9, batch_uid: 9, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000009, documents_count: 1, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -70,7 +70,78 @@ doggos: { number_of_documents: 10, field_distribution: {"doggo": 10, "id": 10} } [timestamp] [8,] [timestamp] [9,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +5 {uid: 5, } +6 {uid: 6, } +7 {uid: 7, } +8 {uid: 8, } +9 {uid: 9, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +5 [5,] +6 [6,] +7 [7,] +8 [8,] +9 [9,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,5,6,7,8,9,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +[timestamp] [5,] +[timestamp] [6,] +[timestamp] [7,] +[timestamp] [8,] +[timestamp] [9,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/five_tasks_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/five_tasks_processed.snap index 8aa7cfbea..1e06b6dce 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/five_tasks_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_mixed_document_addition/five_tasks_processed.snap @@ -1,16 +1,16 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} -1 {uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} -2 {uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} -3 {uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} -4 {uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000000, documents_count: 1, allow_index_creation: true }} +1 {uid: 1, batch_uid: 1, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000001, documents_count: 1, allow_index_creation: true }} +2 {uid: 2, batch_uid: 2, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000002, documents_count: 1, allow_index_creation: true }} +3 {uid: 3, batch_uid: 3, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000003, documents_count: 1, allow_index_creation: true }} +4 {uid: 4, batch_uid: 4, status: succeeded, details: { received_documents: 1, indexed_documents: Some(1) }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000004, documents_count: 1, allow_index_creation: true }} 5 {uid: 5, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000005, documents_count: 1, allow_index_creation: true }} 6 {uid: 6, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: UpdateDocuments, content_file: 00000000-0000-0000-0000-000000000006, documents_count: 1, allow_index_creation: true }} 7 {uid: 7, status: enqueued, details: { received_documents: 1, indexed_documents: None }, kind: DocumentAdditionOrUpdate { index_uid: "doggos", primary_key: Some("id"), method: ReplaceDocuments, content_file: 00000000-0000-0000-0000-000000000007, documents_count: 1, allow_index_creation: true }} @@ -60,6 +60,53 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "id": 5} } [timestamp] [3,] [timestamp] [4,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +1 {uid: 1, } +2 {uid: 2, } +3 {uid: 3, } +4 {uid: 4, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +1 [1,] +2 [2,] +3 [3,] +4 [4,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Kind: +"documentAdditionOrUpdate" [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,1,2,3,4,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +[timestamp] [1,] +[timestamp] [2,] +[timestamp] [3,] +[timestamp] [4,] +---------------------------------------------------------------------- ### File Store: 00000000-0000-0000-0000-000000000005 00000000-0000-0000-0000-000000000006 @@ -68,4 +115,3 @@ doggos: { number_of_documents: 5, field_distribution: {"doggo": 5, "id": 5} } 00000000-0000-0000-0000-000000000009 ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_settings_update/after_registering_settings_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_settings_update/after_registering_settings_task.snap index 37f0a062d..dd81f33e7 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_settings_update/after_registering_settings_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_settings_update/after_registering_settings_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,6 +30,25 @@ doggos [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_settings_update/settings_update_processed.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_settings_update/settings_update_processed.snap index 3906fc6fc..d889e5706 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_settings_update/settings_update_processed.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_settings_update/settings_update_processed.snap @@ -1,12 +1,12 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: -0 {uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"default": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(4), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"default": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(4), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} +0 {uid: 0, batch_uid: 0, status: succeeded, details: { settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"default": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(4), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData } }, kind: SettingsUpdate { index_uid: "doggos", new_settings: Settings { displayed_attributes: WildcardSetting(NotSet), searchable_attributes: WildcardSetting(NotSet), filterable_attributes: NotSet, sortable_attributes: NotSet, ranking_rules: NotSet, stop_words: NotSet, non_separator_tokens: NotSet, separator_tokens: NotSet, dictionary: NotSet, synonyms: NotSet, distinct_attribute: NotSet, proximity_precision: NotSet, typo_tolerance: NotSet, faceting: NotSet, pagination: NotSet, embedders: Set({"default": Set(EmbeddingSettings { source: Set(Rest), model: NotSet, revision: NotSet, api_key: Set("My super secret"), dimensions: Set(4), binary_quantized: NotSet, document_template: NotSet, document_template_max_bytes: NotSet, url: Set("http://localhost:7777"), request: Set(String("{{text}}")), response: Set(String("{{embedding}}")), headers: NotSet, distribution: NotSet })}), search_cutoff_ms: NotSet, localized_attributes: NotSet, _kind: PhantomData }, is_deletion: false, allow_index_creation: true }} ---------------------------------------------------------------------- ### Status: enqueued [] @@ -34,6 +34,33 @@ doggos: { number_of_documents: 0, field_distribution: {} } ### Finished At: [timestamp] [0,] ---------------------------------------------------------------------- +### All Batches: +0 {uid: 0, } +---------------------------------------------------------------------- +### Batch to tasks mapping: +0 [0,] +---------------------------------------------------------------------- +### Batches Status: +succeeded [0,] +---------------------------------------------------------------------- +### Batches Kind: +"settingsUpdate" [0,] +---------------------------------------------------------------------- +### Batches Index Tasks: +doggos [0,] +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Started At: +[timestamp] [0,] +---------------------------------------------------------------------- +### Batches Finished At: +[timestamp] [0,] +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- diff --git a/crates/index-scheduler/src/snapshots/lib.rs/test_task_is_processing/registered_a_task.snap b/crates/index-scheduler/src/snapshots/lib.rs/test_task_is_processing/registered_a_task.snap index f17bfe38f..05bad8454 100644 --- a/crates/index-scheduler/src/snapshots/lib.rs/test_task_is_processing/registered_a_task.snap +++ b/crates/index-scheduler/src/snapshots/lib.rs/test_task_is_processing/registered_a_task.snap @@ -1,8 +1,8 @@ --- -source: index-scheduler/src/lib.rs +source: crates/index-scheduler/src/lib.rs --- ### Autobatching Enabled = true -### Processing Tasks: +### Processing batch None: [] ---------------------------------------------------------------------- ### All Tasks: @@ -30,7 +30,25 @@ index_a [0,] ---------------------------------------------------------------------- ### Finished At: ---------------------------------------------------------------------- +### All Batches: +---------------------------------------------------------------------- +### Batch to tasks mapping: +---------------------------------------------------------------------- +### Batches Status: +---------------------------------------------------------------------- +### Batches Kind: +---------------------------------------------------------------------- +### Batches Index Tasks: +---------------------------------------------------------------------- +### Batches Canceled By: + +---------------------------------------------------------------------- +### Batches Enqueued At: +---------------------------------------------------------------------- +### Batches Started At: +---------------------------------------------------------------------- +### Batches Finished At: +---------------------------------------------------------------------- ### File Store: ---------------------------------------------------------------------- - diff --git a/crates/index-scheduler/src/utils.rs b/crates/index-scheduler/src/utils.rs index 788a70fb8..8ffef39f3 100644 --- a/crates/index-scheduler/src/utils.rs +++ b/crates/index-scheduler/src/utils.rs @@ -3,6 +3,7 @@ use std::collections::{BTreeSet, HashSet}; use std::ops::Bound; +use meilisearch_types::batches::{Batch, BatchId}; use meilisearch_types::heed::types::DecodeIgnore; use meilisearch_types::heed::{Database, RoTxn, RwTxn}; use meilisearch_types::milli::CboRoaringBitmapCodec; @@ -12,11 +13,75 @@ use time::OffsetDateTime; use crate::{Error, IndexScheduler, Result, Task, TaskId, BEI128}; +/// This structure contains all the information required to write a batch in the database without reading the tasks. +/// It'll stay in RAM so it must be small. +#[derive(Debug, Clone)] +pub(crate) struct ProcessingBatch { + pub uid: BatchId, + pub statuses: HashSet, + pub kinds: HashSet, + pub indexes: HashSet, + pub canceled_by: HashSet, + pub oldest_enqueued_at: Option, + pub earliest_enqueued_at: Option, + pub started_at: OffsetDateTime, +} + +impl ProcessingBatch { + pub fn new(uid: BatchId) -> Self { + // At the beginning, all the tasks are processing + let mut statuses = HashSet::default(); + statuses.insert(Status::Processing); + + Self { + uid, + statuses, + kinds: HashSet::default(), + indexes: HashSet::default(), + canceled_by: HashSet::default(), + oldest_enqueued_at: None, + earliest_enqueued_at: None, + started_at: OffsetDateTime::now_utc(), + } + } + + /// Remove the Processing status and update the real statuses of the tasks. + pub fn update(&mut self, task: &Task) { + self.statuses.clear(); + self.statuses.insert(task.status); + } + + /// Update itself with the content of the task and update the batch id in the task. + pub fn processing<'a>(&mut self, tasks: impl IntoIterator) { + for task in tasks.into_iter() { + task.batch_uid = Some(self.uid); + // We don't store the statuses since they're all enqueued. + self.kinds.insert(task.kind.as_kind()); + self.indexes.extend(task.indexes().iter().map(|s| s.to_string())); + if let Some(canceled_by) = task.canceled_by { + self.canceled_by.insert(canceled_by); + } + self.oldest_enqueued_at = + Some(self.oldest_enqueued_at.map_or(task.enqueued_at, |oldest_enqueued_at| { + task.enqueued_at.min(oldest_enqueued_at) + })); + self.earliest_enqueued_at = + Some(self.earliest_enqueued_at.map_or(task.enqueued_at, |earliest_enqueued_at| { + task.enqueued_at.max(earliest_enqueued_at) + })); + } + } +} + impl IndexScheduler { pub(crate) fn all_task_ids(&self, rtxn: &RoTxn) -> Result { enum_iterator::all().map(|s| self.get_status(rtxn, s)).union() } + pub(crate) fn all_batch_ids(&self, rtxn: &RoTxn) -> Result { + enum_iterator::all().map(|s| self.get_batch_status(rtxn, s)).union() + } + pub(crate) fn last_task_id(&self, rtxn: &RoTxn) -> Result> { Ok(self.all_tasks.remap_data_type::().last(rtxn)?.map(|(k, _)| k + 1)) } @@ -25,10 +90,86 @@ impl IndexScheduler { Ok(self.last_task_id(rtxn)?.unwrap_or_default()) } + pub(crate) fn next_batch_id(&self, rtxn: &RoTxn) -> Result { + Ok(self + .all_batches + .remap_data_type::() + .last(rtxn)? + .map(|(k, _)| k + 1) + .unwrap_or_default()) + } + pub(crate) fn get_task(&self, rtxn: &RoTxn, task_id: TaskId) -> Result> { Ok(self.all_tasks.get(rtxn, &task_id)?) } + pub(crate) fn get_batch(&self, rtxn: &RoTxn, batch_id: BatchId) -> Result> { + Ok(self.all_batches.get(rtxn, &batch_id)?) + } + + pub(crate) fn write_batch( + &self, + wtxn: &mut RwTxn, + batch: ProcessingBatch, + tasks: &RoaringBitmap, + finished_at: OffsetDateTime, + ) -> Result<()> { + self.all_batches.put( + wtxn, + &batch.uid, + &Batch { uid: batch.uid, started_at: batch.started_at, finished_at: Some(finished_at) }, + )?; + self.batch_to_tasks_mapping.put(wtxn, &batch.uid, tasks)?; + + for status in batch.statuses { + self.update_batch_status(wtxn, status, |bitmap| { + bitmap.insert(batch.uid); + })?; + } + + for kind in batch.kinds { + self.update_batch_kind(wtxn, kind, |bitmap| { + bitmap.insert(batch.uid); + })?; + } + + for index in batch.indexes { + self.update_batch_index(wtxn, &index, |bitmap| { + bitmap.insert(batch.uid); + })?; + } + if let Some(enqueued_at) = batch.oldest_enqueued_at { + insert_task_datetime(wtxn, self.batch_enqueued_at, enqueued_at, batch.uid)?; + } + if let Some(enqueued_at) = batch.earliest_enqueued_at { + insert_task_datetime(wtxn, self.batch_enqueued_at, enqueued_at, batch.uid)?; + } + insert_task_datetime(wtxn, self.batch_started_at, batch.started_at, batch.uid)?; + insert_task_datetime(wtxn, self.batch_finished_at, finished_at, batch.uid)?; + + Ok(()) + } + + /// Convert an iterator to a `Vec` of tasks. The tasks MUST exist or a + /// `CorruptedTaskQueue` error will be throwed. + pub(crate) fn get_existing_tasks_with_processing_batch( + &self, + rtxn: &RoTxn, + processing_batch: &mut ProcessingBatch, + tasks: impl IntoIterator, + ) -> Result> { + tasks + .into_iter() + .map(|task_id| { + let mut task = self + .get_task(rtxn, task_id) + .and_then(|task| task.ok_or(Error::CorruptedTaskQueue)); + processing_batch.processing(&mut task); + task + }) + .collect::>() + } + /// Convert an iterator to a `Vec` of tasks. The tasks MUST exist or a /// `CorruptedTaskQueue` error will be throwed. pub(crate) fn get_existing_tasks( @@ -44,11 +185,29 @@ impl IndexScheduler { .collect::>() } + /// Convert an iterator to a `Vec` of batches. The batches MUST exist or a + /// `CorruptedTaskQueue` error will be throwed. + pub(crate) fn get_existing_batches( + &self, + rtxn: &RoTxn, + tasks: impl IntoIterator, + ) -> Result> { + tasks + .into_iter() + .map(|batch_id| { + self.get_batch(rtxn, batch_id) + .and_then(|task| task.ok_or(Error::CorruptedTaskQueue)) + }) + .collect::>() + } + pub(crate) fn update_task(&self, wtxn: &mut RwTxn, task: &Task) -> Result<()> { let old_task = self.get_task(wtxn, task.uid)?.ok_or(Error::CorruptedTaskQueue)?; debug_assert_eq!(old_task.uid, task.uid); + debug_assert!(old_task.batch_uid.is_none() && task.batch_uid.is_some()); + // TODO: This shouldn't ever happen, we should assert it if old_task == *task { return Ok(()); } @@ -92,6 +251,11 @@ impl IndexScheduler { Ok(()) } + /// Returns the whole set of tasks that belongs to this batch. + pub(crate) fn tasks_in_batch(&self, rtxn: &RoTxn, batch_id: BatchId) -> Result { + Ok(self.batch_to_tasks_mapping.get(rtxn, &batch_id)?.unwrap_or_default()) + } + /// Returns the whole set of tasks that belongs to this index. pub(crate) fn index_tasks(&self, rtxn: &RoTxn, index: &str) -> Result { Ok(self.index_tasks.get(rtxn, index)?.unwrap_or_default()) @@ -114,6 +278,28 @@ impl IndexScheduler { Ok(()) } + /// Returns the whole set of batches that belongs to this index. + pub(crate) fn index_batches(&self, rtxn: &RoTxn, index: &str) -> Result { + Ok(self.batch_index_tasks.get(rtxn, index)?.unwrap_or_default()) + } + + pub(crate) fn update_batch_index( + &self, + wtxn: &mut RwTxn, + index: &str, + f: impl Fn(&mut RoaringBitmap), + ) -> Result<()> { + let mut batches = self.index_batches(wtxn, index)?; + f(&mut batches); + if batches.is_empty() { + self.batch_index_tasks.delete(wtxn, index)?; + } else { + self.batch_index_tasks.put(wtxn, index, &batches)?; + } + + Ok(()) + } + pub(crate) fn get_status(&self, rtxn: &RoTxn, status: Status) -> Result { Ok(self.status.get(rtxn, &status)?.unwrap_or_default()) } @@ -140,6 +326,32 @@ impl IndexScheduler { Ok(()) } + pub(crate) fn get_batch_status(&self, rtxn: &RoTxn, status: Status) -> Result { + Ok(self.batch_status.get(rtxn, &status)?.unwrap_or_default()) + } + + pub(crate) fn put_batch_status( + &self, + wtxn: &mut RwTxn, + status: Status, + bitmap: &RoaringBitmap, + ) -> Result<()> { + Ok(self.batch_status.put(wtxn, &status, bitmap)?) + } + + pub(crate) fn update_batch_status( + &self, + wtxn: &mut RwTxn, + status: Status, + f: impl Fn(&mut RoaringBitmap), + ) -> Result<()> { + let mut tasks = self.get_batch_status(wtxn, status)?; + f(&mut tasks); + self.put_batch_status(wtxn, status, &tasks)?; + + Ok(()) + } + pub(crate) fn get_kind(&self, rtxn: &RoTxn, kind: Kind) -> Result { Ok(self.kind.get(rtxn, &kind)?.unwrap_or_default()) } @@ -165,6 +377,32 @@ impl IndexScheduler { Ok(()) } + + pub(crate) fn get_batch_kind(&self, rtxn: &RoTxn, kind: Kind) -> Result { + Ok(self.batch_kind.get(rtxn, &kind)?.unwrap_or_default()) + } + + pub(crate) fn put_batch_kind( + &self, + wtxn: &mut RwTxn, + kind: Kind, + bitmap: &RoaringBitmap, + ) -> Result<()> { + Ok(self.batch_kind.put(wtxn, &kind, bitmap)?) + } + + pub(crate) fn update_batch_kind( + &self, + wtxn: &mut RwTxn, + kind: Kind, + f: impl Fn(&mut RoaringBitmap), + ) -> Result<()> { + let mut tasks = self.get_batch_kind(wtxn, kind)?; + f(&mut tasks); + self.put_batch_kind(wtxn, kind, &tasks)?; + + Ok(()) + } } pub(crate) fn insert_task_datetime( @@ -199,6 +437,7 @@ pub(crate) fn remove_task_datetime( Ok(()) } +// TODO: Rename the function since it also applies to batches pub(crate) fn keep_tasks_within_datetimes( rtxn: &RoTxn, tasks: &mut RoaringBitmap, @@ -342,6 +581,7 @@ impl IndexScheduler { let Task { uid, + batch_uid, enqueued_at, started_at, finished_at, @@ -352,6 +592,14 @@ impl IndexScheduler { kind, } = task; assert_eq!(uid, task.uid); + if let Some(ref batch) = batch_uid { + assert!(self + .batch_to_tasks_mapping + .get(&rtxn, batch) + .unwrap() + .unwrap() + .contains(uid)); + } if let Some(task_index_uid) = &task_index_uid { assert!(self .index_tasks diff --git a/crates/meilisearch-types/src/batch_view.rs b/crates/meilisearch-types/src/batch_view.rs new file mode 100644 index 000000000..15e1f8ab9 --- /dev/null +++ b/crates/meilisearch-types/src/batch_view.rs @@ -0,0 +1,30 @@ +use serde::Serialize; +use time::{Duration, OffsetDateTime}; + +use crate::{ + batches::{Batch, BatchId}, + tasks::serialize_duration, +}; + +#[derive(Debug, Clone, PartialEq, Eq, Serialize)] +#[serde(rename_all = "camelCase")] +pub struct BatchView { + pub uid: BatchId, + #[serde(serialize_with = "serialize_duration", default)] + pub duration: Option, + #[serde(with = "time::serde::rfc3339", default)] + pub started_at: OffsetDateTime, + #[serde(with = "time::serde::rfc3339::option", default)] + pub finished_at: Option, +} + +impl BatchView { + pub fn from_batch(batch: &Batch) -> Self { + Self { + uid: batch.uid, + duration: batch.finished_at.map(|finished_at| finished_at - batch.started_at), + started_at: batch.started_at, + finished_at: batch.finished_at, + } + } +} diff --git a/crates/meilisearch-types/src/batches.rs b/crates/meilisearch-types/src/batches.rs new file mode 100644 index 000000000..8a439c59f --- /dev/null +++ b/crates/meilisearch-types/src/batches.rs @@ -0,0 +1,18 @@ +use serde::{Deserialize, Serialize}; +use time::OffsetDateTime; + +pub type BatchId = u32; + +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(rename_all = "camelCase")] +pub struct Batch { + pub uid: BatchId, + + #[serde(with = "time::serde::rfc3339")] + pub started_at: OffsetDateTime, + #[serde(with = "time::serde::rfc3339::option")] + pub finished_at: Option, + // pub details: Option
, + + // pub status: Status, +} diff --git a/crates/meilisearch-types/src/error.rs b/crates/meilisearch-types/src/error.rs index 514ed18c3..e4266b422 100644 --- a/crates/meilisearch-types/src/error.rs +++ b/crates/meilisearch-types/src/error.rs @@ -321,6 +321,7 @@ InvalidTaskLimit , InvalidRequest , BAD_REQUEST ; InvalidTaskStatuses , InvalidRequest , BAD_REQUEST ; InvalidTaskTypes , InvalidRequest , BAD_REQUEST ; InvalidTaskUids , InvalidRequest , BAD_REQUEST ; +InvalidBatchUids , InvalidRequest , BAD_REQUEST ; IoError , System , UNPROCESSABLE_ENTITY; FeatureNotEnabled , InvalidRequest , BAD_REQUEST ; MalformedPayload , InvalidRequest , BAD_REQUEST ; @@ -342,6 +343,7 @@ NoSpaceLeftOnDevice , System , UNPROCESSABLE_ENT PayloadTooLarge , InvalidRequest , PAYLOAD_TOO_LARGE ; TooManySearchRequests , System , SERVICE_UNAVAILABLE ; TaskNotFound , InvalidRequest , NOT_FOUND ; +BatchNotFound , InvalidRequest , NOT_FOUND ; TooManyOpenFiles , System , UNPROCESSABLE_ENTITY ; TooManyVectors , InvalidRequest , BAD_REQUEST ; UnretrievableDocument , Internal , BAD_REQUEST ; diff --git a/crates/meilisearch-types/src/lib.rs b/crates/meilisearch-types/src/lib.rs index d6049e667..a1a57b7e6 100644 --- a/crates/meilisearch-types/src/lib.rs +++ b/crates/meilisearch-types/src/lib.rs @@ -1,3 +1,5 @@ +pub mod batch_view; +pub mod batches; pub mod compression; pub mod deserr; pub mod document_formats; diff --git a/crates/meilisearch-types/src/task_view.rs b/crates/meilisearch-types/src/task_view.rs index 3075fa899..0cc127c0d 100644 --- a/crates/meilisearch-types/src/task_view.rs +++ b/crates/meilisearch-types/src/task_view.rs @@ -2,6 +2,7 @@ use milli::Object; use serde::Serialize; use time::{Duration, OffsetDateTime}; +use crate::batches::BatchId; use crate::error::ResponseError; use crate::settings::{Settings, Unchecked}; use crate::tasks::{serialize_duration, Details, IndexSwap, Kind, Status, Task, TaskId}; @@ -10,6 +11,7 @@ use crate::tasks::{serialize_duration, Details, IndexSwap, Kind, Status, Task, T #[serde(rename_all = "camelCase")] pub struct TaskView { pub uid: TaskId, + pub batch_uid: Option, #[serde(default)] pub index_uid: Option, pub status: Status, @@ -33,6 +35,7 @@ impl TaskView { pub fn from_task(task: &Task) -> TaskView { TaskView { uid: task.uid, + batch_uid: task.batch_uid, index_uid: task.index_uid().map(ToOwned::to_owned), status: task.status, kind: task.kind.as_kind(), diff --git a/crates/meilisearch-types/src/tasks.rs b/crates/meilisearch-types/src/tasks.rs index 1dd6d3fbf..f5875225a 100644 --- a/crates/meilisearch-types/src/tasks.rs +++ b/crates/meilisearch-types/src/tasks.rs @@ -11,6 +11,7 @@ use serde::{Deserialize, Serialize, Serializer}; use time::{Duration, OffsetDateTime}; use uuid::Uuid; +use crate::batches::BatchId; use crate::error::ResponseError; use crate::keys::Key; use crate::settings::{Settings, Unchecked}; @@ -22,6 +23,7 @@ pub type TaskId = u32; #[serde(rename_all = "camelCase")] pub struct Task { pub uid: TaskId, + pub batch_uid: Option, #[serde(with = "time::serde::rfc3339")] pub enqueued_at: OffsetDateTime, diff --git a/crates/meilisearch/src/routes/batches.rs b/crates/meilisearch/src/routes/batches.rs new file mode 100644 index 000000000..9e432bcc1 --- /dev/null +++ b/crates/meilisearch/src/routes/batches.rs @@ -0,0 +1,80 @@ +use actix_web::{ + web::{self, Data}, + HttpResponse, +}; +use deserr::actix_web::AwebQueryParameter; +use index_scheduler::{IndexScheduler, Query}; +use meilisearch_types::{ + batch_view::BatchView, batches::BatchId, deserr::DeserrQueryParamError, error::ResponseError, + keys::actions, +}; +use serde::Serialize; + +use crate::extractors::{authentication::GuardedData, sequential_extractor::SeqHandler}; + +use super::{tasks::TasksFilterQuery, ActionPolicy}; + +pub fn configure(cfg: &mut web::ServiceConfig) { + cfg.service(web::resource("").route(web::get().to(SeqHandler(get_batches)))) + .service(web::resource("/{batch_id}").route(web::get().to(SeqHandler(get_batch)))); +} + +async fn get_batch( + index_scheduler: GuardedData, Data>, + batch_uid: web::Path, +) -> Result { + let batch_uid_string = batch_uid.into_inner(); + + let batch_uid: BatchId = match batch_uid_string.parse() { + Ok(id) => id, + Err(_e) => { + return Err( + index_scheduler::Error::InvalidBatchUid { batch_uid: batch_uid_string }.into() + ) + } + }; + + let query = index_scheduler::Query { uids: Some(vec![batch_uid]), ..Query::default() }; + let filters = index_scheduler.filters(); + let (batches, _) = index_scheduler.get_batches_from_authorized_indexes(query, filters)?; + + if let Some(batch) = batches.first() { + let task_view = BatchView::from_batch(batch); + Ok(HttpResponse::Ok().json(task_view)) + } else { + Err(index_scheduler::Error::TaskNotFound(batch_uid).into()) + } +} + +#[derive(Debug, Serialize)] +pub struct AllBatches { + results: Vec, + total: u64, + limit: u32, + from: Option, + next: Option, +} + +async fn get_batches( + index_scheduler: GuardedData, Data>, + params: AwebQueryParameter, +) -> Result { + let mut params = params.into_inner(); + // We +1 just to know if there is more after this "page" or not. + params.limit.0 = params.limit.0.saturating_add(1); + let limit = params.limit.0; + let query = params.into_query(); + + let filters = index_scheduler.filters(); + let (tasks, total) = index_scheduler.get_batches_from_authorized_indexes(query, filters)?; + let mut results: Vec<_> = tasks.iter().map(BatchView::from_batch).collect(); + + // If we were able to fetch the number +1 tasks we asked + // it means that there is more to come. + let next = if results.len() == limit as usize { results.pop().map(|t| t.uid) } else { None }; + + let from = results.first().map(|t| t.uid); + let tasks = AllBatches { results, limit: limit.saturating_sub(1), total, from, next }; + + Ok(HttpResponse::Ok().json(tasks)) +} diff --git a/crates/meilisearch/src/routes/mod.rs b/crates/meilisearch/src/routes/mod.rs index b7260ea08..91237b707 100644 --- a/crates/meilisearch/src/routes/mod.rs +++ b/crates/meilisearch/src/routes/mod.rs @@ -19,6 +19,7 @@ use crate::Opt; const PAGINATION_DEFAULT_LIMIT: usize = 20; mod api_key; +pub mod batches; mod dump; pub mod features; pub mod indexes; @@ -32,6 +33,7 @@ pub mod tasks; pub fn configure(cfg: &mut web::ServiceConfig) { cfg.service(web::scope("/tasks").configure(tasks::configure)) + .service(web::scope("/batches").configure(batches::configure)) .service(web::resource("/health").route(web::get().to(get_health))) .service(web::scope("/logs").configure(logs::configure)) .service(web::scope("/keys").configure(api_key::configure)) diff --git a/crates/meilisearch/src/routes/tasks.rs b/crates/meilisearch/src/routes/tasks.rs index 95959d6d5..a22f3bdf9 100644 --- a/crates/meilisearch/src/routes/tasks.rs +++ b/crates/meilisearch/src/routes/tasks.rs @@ -3,6 +3,7 @@ use actix_web::{web, HttpRequest, HttpResponse}; use deserr::actix_web::AwebQueryParameter; use deserr::Deserr; use index_scheduler::{IndexScheduler, Query, TaskId}; +use meilisearch_types::batches::BatchId; use meilisearch_types::deserr::query_params::Param; use meilisearch_types::deserr::DeserrQueryParamError; use meilisearch_types::error::deserr_codes::*; @@ -17,15 +18,13 @@ use time::macros::format_description; use time::{Date, Duration, OffsetDateTime, Time}; use tokio::task; -use super::{get_task_id, is_dry_run, SummarizedTaskView}; +use super::{get_task_id, is_dry_run, SummarizedTaskView, PAGINATION_DEFAULT_LIMIT}; use crate::analytics::{Aggregate, AggregateMethod, Analytics}; use crate::extractors::authentication::policies::*; use crate::extractors::authentication::GuardedData; use crate::extractors::sequential_extractor::SeqHandler; use crate::{aggregate_methods, Opt}; -const DEFAULT_LIMIT: u32 = 20; - pub fn configure(cfg: &mut web::ServiceConfig) { cfg.service( web::resource("") @@ -35,14 +34,18 @@ pub fn configure(cfg: &mut web::ServiceConfig) { .service(web::resource("/cancel").route(web::post().to(SeqHandler(cancel_tasks)))) .service(web::resource("/{task_id}").route(web::get().to(SeqHandler(get_task)))); } + #[derive(Debug, Deserr)] #[deserr(error = DeserrQueryParamError, rename_all = camelCase, deny_unknown_fields)] pub struct TasksFilterQuery { - #[deserr(default = Param(DEFAULT_LIMIT), error = DeserrQueryParamError)] + #[deserr(default = Param(PAGINATION_DEFAULT_LIMIT as u32), error = DeserrQueryParamError)] pub limit: Param, #[deserr(default, error = DeserrQueryParamError)] pub from: Option>, + #[deserr(default, error = DeserrQueryParamError)] + pub batch_uids: OptionStarOrList, + #[deserr(default, error = DeserrQueryParamError)] pub uids: OptionStarOrList, #[deserr(default, error = DeserrQueryParamError)] @@ -69,10 +72,11 @@ pub struct TasksFilterQuery { } impl TasksFilterQuery { - fn into_query(self) -> Query { + pub(crate) fn into_query(self) -> Query { Query { limit: Some(self.limit.0), from: self.from.as_deref().copied(), + batch_uids: self.batch_uids.merge_star_and_none(), statuses: self.statuses.merge_star_and_none(), types: self.types.merge_star_and_none(), index_uids: self.index_uids.map(|x| x.to_string()).merge_star_and_none(), @@ -94,6 +98,7 @@ impl TaskDeletionOrCancelationQuery { self, TaskDeletionOrCancelationQuery { uids: OptionStarOrList::None, + batch_uids: OptionStarOrList::None, canceled_by: OptionStarOrList::None, types: OptionStarOrList::None, statuses: OptionStarOrList::None, @@ -113,9 +118,11 @@ impl TaskDeletionOrCancelationQuery { #[deserr(error = DeserrQueryParamError, rename_all = camelCase, deny_unknown_fields)] pub struct TaskDeletionOrCancelationQuery { #[deserr(default, error = DeserrQueryParamError)] - pub uids: OptionStarOrList, + pub uids: OptionStarOrList, + #[deserr(default, error = DeserrQueryParamError)] + pub batch_uids: OptionStarOrList, #[deserr(default, error = DeserrQueryParamError)] - pub canceled_by: OptionStarOrList, + pub canceled_by: OptionStarOrList, #[deserr(default, error = DeserrQueryParamError)] pub types: OptionStarOrList, #[deserr(default, error = DeserrQueryParamError)] @@ -142,6 +149,7 @@ impl TaskDeletionOrCancelationQuery { Query { limit: None, from: None, + batch_uids: self.batch_uids.merge_star_and_none(), statuses: self.statuses.merge_star_and_none(), types: self.types.merge_star_and_none(), index_uids: self.index_uids.map(|x| x.to_string()).merge_star_and_none(), @@ -359,7 +367,7 @@ async fn get_task( let task_uid: TaskId = match task_uid_string.parse() { Ok(id) => id, Err(_e) => { - return Err(index_scheduler::Error::InvalidTaskUids { task_uid: task_uid_string }.into()) + return Err(index_scheduler::Error::InvalidTaskUid { task_uid: task_uid_string }.into()) } }; @@ -481,7 +489,7 @@ mod tests { // Stars are allowed in date fields as well let params = "afterEnqueuedAt=*&beforeStartedAt=*&afterFinishedAt=*&beforeFinishedAt=*&afterStartedAt=*&beforeEnqueuedAt=*"; let query = deserr_query_params::(params).unwrap(); - snapshot!(format!("{:?}", query), @"TaskDeletionOrCancelationQuery { uids: None, canceled_by: None, types: None, statuses: None, index_uids: None, after_enqueued_at: Star, before_enqueued_at: Star, after_started_at: Star, before_started_at: Star, after_finished_at: Star, before_finished_at: Star }"); + snapshot!(format!("{:?}", query), @"TaskDeletionOrCancelationQuery { uids: None, batch_uids: None, canceled_by: None, types: None, statuses: None, index_uids: None, after_enqueued_at: Star, before_enqueued_at: Star, after_started_at: Star, before_started_at: Star, after_finished_at: Star, before_finished_at: Star }"); } { let params = "afterFinishedAt=2021"; @@ -701,20 +709,20 @@ mod tests { { let params = "from=12&limit=15&indexUids=toto,tata-78&statuses=succeeded,enqueued&afterEnqueuedAt=2012-04-23&uids=1,2,3"; let query = deserr_query_params::(params).unwrap(); - snapshot!(format!("{:?}", query), @r###"TasksFilterQuery { limit: Param(15), from: Some(Param(12)), uids: List([1, 2, 3]), canceled_by: None, types: None, statuses: List([Succeeded, Enqueued]), index_uids: List([IndexUid("toto"), IndexUid("tata-78")]), after_enqueued_at: Other(2012-04-24 0:00:00.0 +00:00:00), before_enqueued_at: None, after_started_at: None, before_started_at: None, after_finished_at: None, before_finished_at: None }"###); + snapshot!(format!("{:?}", query), @r###"TasksFilterQuery { limit: Param(15), from: Some(Param(12)), batch_uids: None, uids: List([1, 2, 3]), canceled_by: None, types: None, statuses: List([Succeeded, Enqueued]), index_uids: List([IndexUid("toto"), IndexUid("tata-78")]), after_enqueued_at: Other(2012-04-24 0:00:00.0 +00:00:00), before_enqueued_at: None, after_started_at: None, before_started_at: None, after_finished_at: None, before_finished_at: None }"###); } { // Stars should translate to `None` in the query // Verify value of the default limit let params = "indexUids=*&statuses=succeeded,*&afterEnqueuedAt=2012-04-23&uids=1,2,3"; let query = deserr_query_params::(params).unwrap(); - snapshot!(format!("{:?}", query), @"TasksFilterQuery { limit: Param(20), from: None, uids: List([1, 2, 3]), canceled_by: None, types: None, statuses: Star, index_uids: Star, after_enqueued_at: Other(2012-04-24 0:00:00.0 +00:00:00), before_enqueued_at: None, after_started_at: None, before_started_at: None, after_finished_at: None, before_finished_at: None }"); + snapshot!(format!("{:?}", query), @"TasksFilterQuery { limit: Param(20), from: None, batch_uids: None, uids: List([1, 2, 3]), canceled_by: None, types: None, statuses: Star, index_uids: Star, after_enqueued_at: Other(2012-04-24 0:00:00.0 +00:00:00), before_enqueued_at: None, after_started_at: None, before_started_at: None, after_finished_at: None, before_finished_at: None }"); } { // Stars should also translate to `None` in task deletion/cancelation queries let params = "indexUids=*&statuses=succeeded,*&afterEnqueuedAt=2012-04-23&uids=1,2,3"; let query = deserr_query_params::(params).unwrap(); - snapshot!(format!("{:?}", query), @"TaskDeletionOrCancelationQuery { uids: List([1, 2, 3]), canceled_by: None, types: None, statuses: Star, index_uids: Star, after_enqueued_at: Other(2012-04-24 0:00:00.0 +00:00:00), before_enqueued_at: None, after_started_at: None, before_started_at: None, after_finished_at: None, before_finished_at: None }"); + snapshot!(format!("{:?}", query), @"TaskDeletionOrCancelationQuery { uids: List([1, 2, 3]), batch_uids: None, canceled_by: None, types: None, statuses: Star, index_uids: Star, after_enqueued_at: Other(2012-04-24 0:00:00.0 +00:00:00), before_enqueued_at: None, after_started_at: None, before_started_at: None, after_finished_at: None, before_finished_at: None }"); } { // Star in from not allowed @@ -735,7 +743,7 @@ mod tests { let err = deserr_query_params::(params).unwrap_err(); snapshot!(meili_snap::json_string!(err), @r###" { - "message": "Unknown parameter `from`: expected one of `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `from`: expected one of `uids`, `batchUids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" @@ -748,7 +756,7 @@ mod tests { let err = deserr_query_params::(params).unwrap_err(); snapshot!(meili_snap::json_string!(err), @r###" { - "message": "Unknown parameter `limit`: expected one of `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `limit`: expected one of `uids`, `batchUids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" @@ -768,7 +776,7 @@ mod tests { let params = "statuses=*"; let query = deserr_query_params::(params).unwrap(); assert!(!query.is_empty()); - snapshot!(format!("{query:?}"), @"TaskDeletionOrCancelationQuery { uids: None, canceled_by: None, types: None, statuses: Star, index_uids: None, after_enqueued_at: None, before_enqueued_at: None, after_started_at: None, before_started_at: None, after_finished_at: None, before_finished_at: None }"); + snapshot!(format!("{query:?}"), @"TaskDeletionOrCancelationQuery { uids: None, batch_uids: None, canceled_by: None, types: None, statuses: Star, index_uids: None, after_enqueued_at: None, before_enqueued_at: None, after_started_at: None, before_started_at: None, after_finished_at: None, before_finished_at: None }"); } } } diff --git a/crates/meilisearch/tests/common/mod.rs b/crates/meilisearch/tests/common/mod.rs index a0137931d..3aae2fe80 100644 --- a/crates/meilisearch/tests/common/mod.rs +++ b/crates/meilisearch/tests/common/mod.rs @@ -99,6 +99,7 @@ impl Display for Value { "{}", json_string!(self, { ".uid" => "[uid]", + ".batchUid" => "[batch_uid]", ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", diff --git a/crates/meilisearch/tests/documents/add_documents.rs b/crates/meilisearch/tests/documents/add_documents.rs index c37b3a5e3..cec6f0791 100644 --- a/crates/meilisearch/tests/documents/add_documents.rs +++ b/crates/meilisearch/tests/documents/add_documents.rs @@ -293,6 +293,7 @@ async fn add_csv_document() { snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###" { "uid": 0, + "batchUid": 0, "indexUid": "pets", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -357,6 +358,7 @@ async fn add_csv_document_with_types() { snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###" { "uid": 0, + "batchUid": 0, "indexUid": "pets", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -432,6 +434,7 @@ async fn add_csv_document_with_custom_delimiter() { snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###" { "uid": 0, + "batchUid": 0, "indexUid": "pets", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -985,6 +988,7 @@ async fn add_documents_no_index_creation() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1063,6 +1067,7 @@ async fn document_addition_with_primary_key() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1111,6 +1116,7 @@ async fn document_addition_with_huge_int_primary_key() { @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1183,6 +1189,7 @@ async fn replace_document() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1271,6 +1278,7 @@ async fn error_add_documents_bad_document_id() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1312,6 +1320,7 @@ async fn error_add_documents_missing_document_id() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1403,6 +1412,7 @@ async fn error_document_field_limit_reached_over_multiple_documents() { @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1437,6 +1447,7 @@ async fn error_document_field_limit_reached_over_multiple_documents() { @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1486,6 +1497,7 @@ async fn error_document_field_limit_reached_in_one_nested_document() { @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1529,6 +1541,7 @@ async fn error_document_field_limit_reached_over_multiple_documents_with_nested_ @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1564,6 +1577,7 @@ async fn error_document_field_limit_reached_over_multiple_documents_with_nested_ @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1611,6 +1625,7 @@ async fn add_documents_with_geo_field() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1651,6 +1666,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 2, + "batchUid": 2, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1688,6 +1704,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 3, + "batchUid": 3, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1725,6 +1742,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 4, + "batchUid": 4, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1762,6 +1780,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 5, + "batchUid": 5, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1799,6 +1818,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 6, + "batchUid": 6, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1836,6 +1856,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 7, + "batchUid": 7, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1873,6 +1894,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 8, + "batchUid": 8, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1910,6 +1932,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 9, + "batchUid": 9, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1947,6 +1970,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 10, + "batchUid": 10, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1984,6 +2008,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 11, + "batchUid": 11, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -2021,6 +2046,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 12, + "batchUid": 12, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -2058,6 +2084,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 13, + "batchUid": 13, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -2096,6 +2123,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 14, + "batchUid": 14, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -2132,6 +2160,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 15, + "batchUid": 15, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -2168,6 +2197,7 @@ async fn add_documents_invalid_geo_field() { @r###" { "uid": 16, + "batchUid": 16, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -2210,6 +2240,7 @@ async fn add_invalid_geo_and_then_settings() { snapshot!(ret, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -2232,6 +2263,7 @@ async fn add_invalid_geo_and_then_settings() { snapshot!(ret, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "failed", "type": "settingsUpdate", @@ -2303,6 +2335,7 @@ async fn error_primary_key_inference() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -2343,6 +2376,7 @@ async fn error_primary_key_inference() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": "test", "status": "failed", "type": "documentAdditionOrUpdate", @@ -2381,6 +2415,7 @@ async fn error_primary_key_inference() { @r###" { "uid": 2, + "batchUid": 2, "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/documents/delete_documents.rs b/crates/meilisearch/tests/documents/delete_documents.rs index 5a15e95ff..c50823183 100644 --- a/crates/meilisearch/tests/documents/delete_documents.rs +++ b/crates/meilisearch/tests/documents/delete_documents.rs @@ -184,6 +184,7 @@ async fn delete_document_by_filter() { snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###" { "uid": 2, + "batchUid": 2, "indexUid": "doggo", "status": "succeeded", "type": "documentDeletion", @@ -249,6 +250,7 @@ async fn delete_document_by_filter() { snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###" { "uid": 3, + "batchUid": 3, "indexUid": "doggo", "status": "succeeded", "type": "documentDeletion", @@ -333,6 +335,7 @@ async fn delete_document_by_complex_filter() { snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###" { "uid": 2, + "batchUid": 2, "indexUid": "doggo", "status": "succeeded", "type": "documentDeletion", @@ -391,6 +394,7 @@ async fn delete_document_by_complex_filter() { snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###" { "uid": 3, + "batchUid": 3, "indexUid": "doggo", "status": "succeeded", "type": "documentDeletion", diff --git a/crates/meilisearch/tests/documents/errors.rs b/crates/meilisearch/tests/documents/errors.rs index 4c644ae98..c90b9ed49 100644 --- a/crates/meilisearch/tests/documents/errors.rs +++ b/crates/meilisearch/tests/documents/errors.rs @@ -563,6 +563,7 @@ async fn delete_document_by_filter() { snapshot!(response, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "DOES_NOT_EXISTS", "status": "failed", "type": "documentDeletion", @@ -592,6 +593,7 @@ async fn delete_document_by_filter() { snapshot!(response, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "EMPTY_INDEX", "status": "failed", "type": "documentDeletion", @@ -623,6 +625,7 @@ async fn delete_document_by_filter() { snapshot!(response, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "SHARED_DOCUMENTS", "status": "failed", "type": "documentDeletion", diff --git a/crates/meilisearch/tests/dumps/mod.rs b/crates/meilisearch/tests/dumps/mod.rs index 1c1a8e649..c7d157b00 100644 --- a/crates/meilisearch/tests/dumps/mod.rs +++ b/crates/meilisearch/tests/dumps/mod.rs @@ -88,7 +88,7 @@ async fn import_dump_v1_movie_raw() { snapshot!(code, @"200 OK"); assert_eq!( tasks, - json!({ "results": [{"uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31968 }, "error": null, "duration": "PT9.317060500S", "enqueuedAt": "2021-09-08T09:08:45.153219Z", "startedAt": "2021-09-08T09:08:45.3961665Z", "finishedAt": "2021-09-08T09:08:54.713227Z" }], "total": 1, "limit": 20, "from": 0, "next": null }) + json!({ "results": [{"uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31968 }, "error": null, "duration": "PT9.317060500S", "enqueuedAt": "2021-09-08T09:08:45.153219Z", "startedAt": "2021-09-08T09:08:45.3961665Z", "finishedAt": "2021-09-08T09:08:54.713227Z" }], "total": 1, "limit": 20, "from": 0, "next": null }) ); // finally we're just going to check that we can still get a few documents by id @@ -251,7 +251,7 @@ async fn import_dump_v1_movie_with_settings() { snapshot!(code, @"200 OK"); assert_eq!( tasks, - json!({ "results": [{ "uid": 1, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["genres", "id", "overview", "poster", "release_date", "title"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "sortableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT7.288826907S", "enqueuedAt": "2021-09-08T09:34:40.882977Z", "startedAt": "2021-09-08T09:34:40.883073093Z", "finishedAt": "2021-09-08T09:34:48.1719Z"}, { "uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31968 }, "error": null, "duration": "PT9.090735774S", "enqueuedAt": "2021-09-08T09:34:16.036101Z", "startedAt": "2021-09-08T09:34:16.261191226Z", "finishedAt": "2021-09-08T09:34:25.351927Z" }], "total": 2, "limit": 20, "from": 1, "next": null }) + json!({ "results": [{ "uid": 1, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["genres", "id", "overview", "poster", "release_date", "title"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "sortableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT7.288826907S", "enqueuedAt": "2021-09-08T09:34:40.882977Z", "startedAt": "2021-09-08T09:34:40.883073093Z", "finishedAt": "2021-09-08T09:34:48.1719Z"}, { "uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31968 }, "error": null, "duration": "PT9.090735774S", "enqueuedAt": "2021-09-08T09:34:16.036101Z", "startedAt": "2021-09-08T09:34:16.261191226Z", "finishedAt": "2021-09-08T09:34:25.351927Z" }], "total": 2, "limit": 20, "from": 1, "next": null }) ); // finally we're just going to check that we can still get a few documents by id @@ -400,7 +400,7 @@ async fn import_dump_v1_rubygems_with_settings() { snapshot!(code, @"200 OK"); assert_eq!( tasks["results"][0], - json!({"uid": 92, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT1.487793839S", "enqueuedAt": "2021-09-08T09:27:01.465296Z", "startedAt": "2021-09-08T09:28:44.882177161Z", "finishedAt": "2021-09-08T09:28:46.369971Z"}) + json!({"uid": 92, "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT1.487793839S", "enqueuedAt": "2021-09-08T09:27:01.465296Z", "startedAt": "2021-09-08T09:28:44.882177161Z", "finishedAt": "2021-09-08T09:28:46.369971Z"}) ); // finally we're just going to check that we can still get a few documents by id @@ -535,7 +535,7 @@ async fn import_dump_v2_movie_raw() { snapshot!(code, @"200 OK"); assert_eq!( tasks, - json!({ "results": [{"uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit": 20, "from": 0, "next": null }) + json!({ "results": [{"uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit": 20, "from": 0, "next": null }) ); // finally we're just going to check that we can still get a few documents by id @@ -682,7 +682,7 @@ async fn import_dump_v2_movie_with_settings() { snapshot!(code, @"200 OK"); assert_eq!( tasks, - json!({ "results": [{ "uid": 1, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null }) + json!({ "results": [{ "uid": 1, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null }) ); // finally we're just going to check that we can still get a few documents by id @@ -828,7 +828,7 @@ async fn import_dump_v2_rubygems_with_settings() { snapshot!(code, @"200 OK"); assert_eq!( tasks["results"][0], - json!({"uid": 92, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"}) + json!({"uid": 92, "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"}) ); // finally we're just going to check that we can still get a few documents by id @@ -963,7 +963,7 @@ async fn import_dump_v3_movie_raw() { snapshot!(code, @"200 OK"); assert_eq!( tasks, - json!({ "results": [{"uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit": 20, "from": 0, "next": null }) + json!({ "results": [{"uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit": 20, "from": 0, "next": null }) ); // finally we're just going to check that we can still get a few documents by id @@ -1110,7 +1110,7 @@ async fn import_dump_v3_movie_with_settings() { snapshot!(code, @"200 OK"); assert_eq!( tasks, - json!({ "results": [{ "uid": 1, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null }) + json!({ "results": [{ "uid": 1, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null }) ); // finally we're just going to check that we can["results"] still get a few documents by id @@ -1256,7 +1256,7 @@ async fn import_dump_v3_rubygems_with_settings() { snapshot!(code, @"200 OK"); assert_eq!( tasks["results"][0], - json!({"uid": 92, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"}) + json!({"uid": 92, "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"}) ); // finally we're just going to check that we can still get a few documents by id @@ -1391,7 +1391,7 @@ async fn import_dump_v4_movie_raw() { snapshot!(code, @"200 OK"); assert_eq!( tasks, - json!({ "results": [{"uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit" : 20, "from": 0, "next": null }) + json!({ "results": [{"uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit" : 20, "from": 0, "next": null }) ); // finally we're just going to check that we can still get a few documents by id @@ -1538,7 +1538,7 @@ async fn import_dump_v4_movie_with_settings() { snapshot!(code, @"200 OK"); assert_eq!( tasks, - json!({ "results": [{ "uid": 1, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null }) + json!({ "results": [{ "uid": 1, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null }) ); // finally we're just going to check that we can still get a few documents by id @@ -1684,7 +1684,7 @@ async fn import_dump_v4_rubygems_with_settings() { snapshot!(code, @"200 OK"); assert_eq!( tasks["results"][0], - json!({ "uid": 92, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"}) + json!({ "uid": 92, "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"}) ); // finally we're just going to check that we can still get a few documents by id diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/generate_and_import_dump_containing_vectors/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/generate_and_import_dump_containing_vectors/1.snap index 2004eb036..a9e7be539 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/generate_and_import_dump_containing_vectors/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/generate_and_import_dump_containing_vectors/1.snap @@ -1,8 +1,9 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "pets", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/generate_and_import_dump_containing_vectors/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/generate_and_import_dump_containing_vectors/2.snap index 8405a0461..ed457d9a6 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/generate_and_import_dump_containing_vectors/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/generate_and_import_dump_containing_vectors/2.snap @@ -1,8 +1,9 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "pets", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/1.snap index 7227fd4cc..b79ce068c 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/2.snap index 7227fd4cc..b79ce068c 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/3.snap index 7227fd4cc..b79ce068c 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/4.snap index 7227fd4cc..b79ce068c 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/5.snap index 7227fd4cc..b79ce068c 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/6.snap index 7227fd4cc..b79ce068c 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/7.snap index 7227fd4cc..b79ce068c 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_raw/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/1.snap index d22c13b72..bbda115f4 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/2.snap index d22c13b72..bbda115f4 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/3.snap index b49f68a59..612a922d2 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/4.snap index b49f68a59..612a922d2 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/5.snap index b49f68a59..612a922d2 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/6.snap index b49f68a59..612a922d2 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/7.snap index b49f68a59..612a922d2 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_movie_with_settings/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/1.snap index beb25ce95..e1887a1f8 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/2.snap index add41439f..b085d78a4 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/3.snap index c9e368061..b793e4eb9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/4.snap index c9e368061..b793e4eb9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/5.snap index c9e368061..b793e4eb9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/6.snap index c9e368061..b793e4eb9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/7.snap index c9e368061..b793e4eb9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v1_rubygems_with_settings/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/1.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/2.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/3.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/4.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/5.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/6.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/7.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_raw/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/1.snap index 6acf1f564..ec5c69801 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/2.snap index 6acf1f564..ec5c69801 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/3.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/4.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/5.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/6.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/7.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_movie_with_settings/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/1.snap index 92613ce34..6ecf9200e 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/2.snap index bfef698e1..3ed8929c1 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/3.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/4.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/5.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/6.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/7.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v2_rubygems_with_settings/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/1.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/2.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/3.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/4.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/5.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/6.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/7.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_raw/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/1.snap index 6acf1f564..ec5c69801 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/2.snap index 6acf1f564..ec5c69801 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/3.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/4.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/5.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/6.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/7.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_movie_with_settings/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/1.snap index 92613ce34..6ecf9200e 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/2.snap index bfef698e1..3ed8929c1 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/3.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/4.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/5.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/6.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/7.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v3_rubygems_with_settings/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/1.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/2.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/3.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/4.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/5.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/6.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/7.snap index 33f1b46c5..93a2938a9 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_raw/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/1.snap index 6acf1f564..ec5c69801 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/2.snap index 6acf1f564..ec5c69801 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/3.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/4.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/5.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/6.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/7.snap index 22fad9f58..0b8da54bf 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_movie_with_settings/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/1.snap index 92613ce34..6ecf9200e 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/2.snap index bfef698e1..3ed8929c1 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/3.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/4.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/5.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/6.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/7.snap index 3fa18f512..4005cdfed 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v4_rubygems_with_settings/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 92, + "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/1.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/1.snap index b8c0d07b1..2de02953d 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/1.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/1.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 0, + "batchUid": null, "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/2.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/2.snap index b31fb744f..dac7f08f7 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/2.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/2.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 1, + "batchUid": null, "indexUid": "test2", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/3.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/3.snap index 082cc97ee..319b6b229 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/3.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/3.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 4, + "batchUid": 0, "indexUid": null, "status": "succeeded", "type": "dumpCreation", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/4.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/4.snap index 54e8f84bf..19c8fc8aa 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/4.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/4.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 2, + "batchUid": null, "indexUid": "test", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/5.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/5.snap index 082cc97ee..319b6b229 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/5.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/5.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 4, + "batchUid": 0, "indexUid": null, "status": "succeeded", "type": "dumpCreation", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/6.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/6.snap index 082cc97ee..319b6b229 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/6.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/6.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 4, + "batchUid": 0, "indexUid": null, "status": "succeeded", "type": "dumpCreation", diff --git a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/7.snap b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/7.snap index 082cc97ee..319b6b229 100644 --- a/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/7.snap +++ b/crates/meilisearch/tests/dumps/snapshots/mod.rs/import_dump_v5/7.snap @@ -1,10 +1,11 @@ --- -source: meilisearch/tests/dumps/mod.rs +source: crates/meilisearch/tests/dumps/mod.rs --- { "results": [ { "uid": 4, + "batchUid": 0, "indexUid": null, "status": "succeeded", "type": "dumpCreation", diff --git a/crates/meilisearch/tests/search/snapshots/distinct.rs/distinct_at_search_time/succeed.snap b/crates/meilisearch/tests/search/snapshots/distinct.rs/distinct_at_search_time/succeed.snap index 64e48b6a5..ea55d9c61 100644 --- a/crates/meilisearch/tests/search/snapshots/distinct.rs/distinct_at_search_time/succeed.snap +++ b/crates/meilisearch/tests/search/snapshots/distinct.rs/distinct_at_search_time/succeed.snap @@ -1,8 +1,9 @@ --- -source: meilisearch/tests/search/distinct.rs +source: crates/meilisearch/tests/search/distinct.rs --- { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "tamo", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/snapshot/mod.rs b/crates/meilisearch/tests/snapshot/mod.rs index 0008993fe..b4454a7f2 100644 --- a/crates/meilisearch/tests/snapshot/mod.rs +++ b/crates/meilisearch/tests/snapshot/mod.rs @@ -151,6 +151,7 @@ async fn perform_on_demand_snapshot() { snapshot!(json_string!(task, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###" { "uid": 4, + "batchUid": 4, "indexUid": null, "status": "succeeded", "type": "snapshotCreation", diff --git a/crates/meilisearch/tests/swap_indexes/mod.rs b/crates/meilisearch/tests/swap_indexes/mod.rs index 7f2b1623c..646f773d8 100644 --- a/crates/meilisearch/tests/swap_indexes/mod.rs +++ b/crates/meilisearch/tests/swap_indexes/mod.rs @@ -24,6 +24,7 @@ async fn swap_indexes() { "results": [ { "uid": 1, + "batchUid": 1, "indexUid": "b", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -40,6 +41,7 @@ async fn swap_indexes() { }, { "uid": 0, + "batchUid": 0, "indexUid": "a", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -76,6 +78,7 @@ async fn swap_indexes() { "results": [ { "uid": 2, + "batchUid": 2, "indexUid": null, "status": "succeeded", "type": "indexSwap", @@ -98,6 +101,7 @@ async fn swap_indexes() { }, { "uid": 1, + "batchUid": 1, "indexUid": "a", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -114,6 +118,7 @@ async fn swap_indexes() { }, { "uid": 0, + "batchUid": 0, "indexUid": "b", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -164,6 +169,7 @@ async fn swap_indexes() { "results": [ { "uid": 4, + "batchUid": 4, "indexUid": "d", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -180,6 +186,7 @@ async fn swap_indexes() { }, { "uid": 3, + "batchUid": 3, "indexUid": "c", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -226,6 +233,7 @@ async fn swap_indexes() { "results": [ { "uid": 5, + "batchUid": 5, "indexUid": null, "status": "succeeded", "type": "indexSwap", @@ -254,6 +262,7 @@ async fn swap_indexes() { }, { "uid": 4, + "batchUid": 4, "indexUid": "c", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -270,6 +279,7 @@ async fn swap_indexes() { }, { "uid": 3, + "batchUid": 3, "indexUid": "d", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -286,6 +296,7 @@ async fn swap_indexes() { }, { "uid": 2, + "batchUid": 2, "indexUid": null, "status": "succeeded", "type": "indexSwap", @@ -308,6 +319,7 @@ async fn swap_indexes() { }, { "uid": 1, + "batchUid": 1, "indexUid": "b", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -324,6 +336,7 @@ async fn swap_indexes() { }, { "uid": 0, + "batchUid": 0, "indexUid": "a", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/tasks/errors.rs b/crates/meilisearch/tests/tasks/errors.rs index 42ec42997..fc11a748f 100644 --- a/crates/meilisearch/tests/tasks/errors.rs +++ b/crates/meilisearch/tests/tasks/errors.rs @@ -222,7 +222,7 @@ async fn task_bad_limit() { snapshot!(code, @"400 Bad Request"); snapshot!(json_string!(response), @r###" { - "message": "Unknown parameter `limit`: expected one of `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `limit`: expected one of `uids`, `batchUids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" @@ -233,7 +233,7 @@ async fn task_bad_limit() { snapshot!(code, @"400 Bad Request"); snapshot!(json_string!(response), @r###" { - "message": "Unknown parameter `limit`: expected one of `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `limit`: expected one of `uids`, `batchUids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" @@ -260,7 +260,7 @@ async fn task_bad_from() { snapshot!(code, @"400 Bad Request"); snapshot!(json_string!(response), @r###" { - "message": "Unknown parameter `from`: expected one of `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `from`: expected one of `uids`, `batchUids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" @@ -271,7 +271,7 @@ async fn task_bad_from() { snapshot!(code, @"400 Bad Request"); snapshot!(json_string!(response), @r###" { - "message": "Unknown parameter `from`: expected one of `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `from`: expected one of `uids`, `batchUids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" diff --git a/crates/meilisearch/tests/tasks/mod.rs b/crates/meilisearch/tests/tasks/mod.rs index c59313885..be4ba4a96 100644 --- a/crates/meilisearch/tests/tasks/mod.rs +++ b/crates/meilisearch/tests/tasks/mod.rs @@ -201,7 +201,7 @@ async fn get_task_filter_error() { assert_eq!(code, 400, "{}", response); meili_snap::snapshot!(meili_snap::json_string!(response), @r###" { - "message": "Unknown parameter `lol`: expected one of `limit`, `from`, `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `lol`: expected one of `limit`, `from`, `batchUids`, `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" @@ -261,7 +261,7 @@ async fn delete_task_filter_error() { assert_eq!(code, 400, "{}", response); meili_snap::snapshot!(meili_snap::json_string!(response), @r###" { - "message": "Unknown parameter `lol`: expected one of `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `lol`: expected one of `uids`, `batchUids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" @@ -299,7 +299,7 @@ async fn cancel_task_filter_error() { assert_eq!(code, 400, "{}", response); meili_snap::snapshot!(meili_snap::json_string!(response), @r###" { - "message": "Unknown parameter `lol`: expected one of `uids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", + "message": "Unknown parameter `lol`: expected one of `uids`, `batchUids`, `canceledBy`, `types`, `statuses`, `indexUids`, `afterEnqueuedAt`, `beforeEnqueuedAt`, `afterStartedAt`, `beforeStartedAt`, `afterFinishedAt`, `beforeFinishedAt`", "code": "bad_request", "type": "invalid_request", "link": "https://docs.meilisearch.com/errors#bad_request" @@ -373,6 +373,7 @@ async fn test_summarized_document_addition_or_update() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -397,6 +398,7 @@ async fn test_summarized_document_addition_or_update() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -426,6 +428,7 @@ async fn test_summarized_delete_documents_by_batch() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "failed", "type": "documentDeletion", @@ -457,6 +460,7 @@ async fn test_summarized_delete_documents_by_batch() { @r###" { "uid": 2, + "batchUid": 2, "indexUid": "test", "status": "succeeded", "type": "documentDeletion", @@ -488,6 +492,7 @@ async fn test_summarized_delete_documents_by_filter() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "failed", "type": "documentDeletion", @@ -519,6 +524,7 @@ async fn test_summarized_delete_documents_by_filter() { @r###" { "uid": 2, + "batchUid": 2, "indexUid": "test", "status": "failed", "type": "documentDeletion", @@ -550,6 +556,7 @@ async fn test_summarized_delete_documents_by_filter() { @r###" { "uid": 4, + "batchUid": 4, "indexUid": "test", "status": "succeeded", "type": "documentDeletion", @@ -580,6 +587,7 @@ async fn test_summarized_delete_document_by_id() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "failed", "type": "documentDeletion", @@ -611,6 +619,7 @@ async fn test_summarized_delete_document_by_id() { @r###" { "uid": 2, + "batchUid": 2, "indexUid": "test", "status": "succeeded", "type": "documentDeletion", @@ -653,6 +662,7 @@ async fn test_summarized_settings_update() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "succeeded", "type": "settingsUpdate", @@ -691,6 +701,7 @@ async fn test_summarized_index_creation() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "succeeded", "type": "indexCreation", @@ -714,6 +725,7 @@ async fn test_summarized_index_creation() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": "test", "status": "failed", "type": "indexCreation", @@ -745,6 +757,7 @@ async fn test_summarized_index_deletion() { @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "failed", "type": "indexDeletion", @@ -775,6 +788,7 @@ async fn test_summarized_index_deletion() { @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -797,6 +811,7 @@ async fn test_summarized_index_deletion() { @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "succeeded", "type": "indexDeletion", @@ -819,6 +834,7 @@ async fn test_summarized_index_deletion() { @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "test", "status": "failed", "type": "indexDeletion", @@ -853,6 +869,7 @@ async fn test_summarized_index_update() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": "test", "status": "failed", "type": "indexUpdate", @@ -881,6 +898,7 @@ async fn test_summarized_index_update() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": "test", "status": "failed", "type": "indexUpdate", @@ -912,6 +930,7 @@ async fn test_summarized_index_update() { @r###" { "uid": 3, + "batchUid": 3, "indexUid": "test", "status": "succeeded", "type": "indexUpdate", @@ -935,6 +954,7 @@ async fn test_summarized_index_update() { @r###" { "uid": 4, + "batchUid": 4, "indexUid": "test", "status": "succeeded", "type": "indexUpdate", @@ -966,6 +986,7 @@ async fn test_summarized_index_swap() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": null, "status": "failed", "type": "indexSwap", @@ -1007,6 +1028,7 @@ async fn test_summarized_index_swap() { @r###" { "uid": 3, + "batchUid": 3, "indexUid": null, "status": "succeeded", "type": "indexSwap", @@ -1045,6 +1067,7 @@ async fn test_summarized_task_cancelation() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": null, "status": "succeeded", "type": "taskCancelation", @@ -1078,6 +1101,7 @@ async fn test_summarized_task_deletion() { @r###" { "uid": 1, + "batchUid": 1, "indexUid": null, "status": "succeeded", "type": "taskDeletion", @@ -1107,6 +1131,7 @@ async fn test_summarized_dump_creation() { @r###" { "uid": 0, + "batchUid": 0, "indexUid": null, "status": "succeeded", "type": "dumpCreation", diff --git a/crates/meilisearch/tests/tasks/webhook.rs b/crates/meilisearch/tests/tasks/webhook.rs index b01ef3d5a..b18002eb7 100644 --- a/crates/meilisearch/tests/tasks/webhook.rs +++ b/crates/meilisearch/tests/tasks/webhook.rs @@ -8,12 +8,12 @@ use actix_http::body::MessageBody; use actix_web::dev::{ServiceFactory, ServiceResponse}; use actix_web::web::{Bytes, Data}; use actix_web::{post, App, HttpRequest, HttpResponse, HttpServer}; -use meili_snap::{json_string, snapshot}; +use meili_snap::snapshot; use meilisearch::Opt; use tokio::sync::mpsc; use url::Url; -use crate::common::{default_settings, Server}; +use crate::common::{self, default_settings, Server}; use crate::json; #[post("/")] @@ -97,11 +97,11 @@ async fn test_basic_webhook() { } nb_tasks += 1; let json: serde_json::Value = serde_json::from_str(json).unwrap(); - snapshot!( - json_string!(json, { ".uid" => "[uid]", ".duration" => "[duration]", ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]" }), + snapshot!(common::Value(json), @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "tamo", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/vector/binary_quantized.rs b/crates/meilisearch/tests/vector/binary_quantized.rs index d3fe3c824..560c4e2f2 100644 --- a/crates/meilisearch/tests/vector/binary_quantized.rs +++ b/crates/meilisearch/tests/vector/binary_quantized.rs @@ -303,6 +303,7 @@ async fn try_to_disable_binary_quantization() { snapshot!(ret, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/vector/mod.rs b/crates/meilisearch/tests/vector/mod.rs index 47d0c1051..828f72c5e 100644 --- a/crates/meilisearch/tests/vector/mod.rs +++ b/crates/meilisearch/tests/vector/mod.rs @@ -240,6 +240,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -269,6 +270,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -299,6 +301,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -328,6 +331,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -357,6 +361,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -386,6 +391,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -427,6 +433,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -455,6 +462,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -483,6 +491,7 @@ async fn user_provided_embeddings_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -523,6 +532,7 @@ async fn user_provided_vectors_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -552,6 +562,7 @@ async fn user_provided_vectors_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -581,6 +592,7 @@ async fn user_provided_vectors_error() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/vector/openai.rs b/crates/meilisearch/tests/vector/openai.rs index 04c068c40..9fae810f9 100644 --- a/crates/meilisearch/tests/vector/openai.rs +++ b/crates/meilisearch/tests/vector/openai.rs @@ -365,6 +365,7 @@ async fn it_works() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -595,6 +596,7 @@ async fn tokenize_long_text() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -651,6 +653,7 @@ async fn bad_api_key() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -683,6 +686,7 @@ async fn bad_api_key() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -727,6 +731,7 @@ async fn bad_api_key() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -796,6 +801,7 @@ async fn bad_model() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -873,6 +879,7 @@ async fn bad_dimensions() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -980,6 +987,7 @@ async fn smaller_dimensions() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1211,6 +1219,7 @@ async fn small_embedding_model() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1441,6 +1450,7 @@ async fn legacy_embedding_model() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1672,6 +1682,7 @@ async fn it_still_works() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/vector/rest.rs b/crates/meilisearch/tests/vector/rest.rs index 2748d0846..2bf6f2a46 100644 --- a/crates/meilisearch/tests/vector/rest.rs +++ b/crates/meilisearch/tests/vector/rest.rs @@ -922,6 +922,7 @@ async fn bad_settings() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -967,6 +968,7 @@ async fn bad_settings() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1016,6 +1018,7 @@ async fn add_vector_and_user_provided() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1112,6 +1115,7 @@ async fn server_returns_bad_request() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -1152,6 +1156,7 @@ async fn server_returns_bad_request() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "settingsUpdate", @@ -1185,6 +1190,7 @@ async fn server_returns_bad_request() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "documentAdditionOrUpdate", @@ -1229,6 +1235,7 @@ async fn server_returns_bad_response() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -1282,6 +1289,7 @@ async fn server_returns_bad_response() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -1337,6 +1345,7 @@ async fn server_returns_bad_response() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -1392,6 +1401,7 @@ async fn server_returns_bad_response() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -1457,6 +1467,7 @@ async fn server_returns_bad_response() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -1524,6 +1535,7 @@ async fn server_returns_multiple() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1629,6 +1641,7 @@ async fn server_single_input_returns_in_array() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1734,6 +1747,7 @@ async fn server_raw() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -1831,6 +1845,7 @@ async fn server_custom_header() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -1870,6 +1885,7 @@ async fn server_custom_header() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", @@ -1912,6 +1928,7 @@ async fn server_custom_header() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "settingsUpdate", @@ -1957,6 +1974,7 @@ async fn searchable_reindex() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "settingsUpdate", @@ -1993,6 +2011,7 @@ async fn searchable_reindex() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", @@ -2021,6 +2040,7 @@ async fn searchable_reindex() { snapshot!(task, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "failed", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/vector/settings.rs b/crates/meilisearch/tests/vector/settings.rs index ed45913a8..027c55219 100644 --- a/crates/meilisearch/tests/vector/settings.rs +++ b/crates/meilisearch/tests/vector/settings.rs @@ -91,6 +91,7 @@ async fn update_embedder() { snapshot!(ret, @r###" { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "settingsUpdate", diff --git a/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/document-added.snap b/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/document-added.snap index c4f1c0b25..709dfeae0 100644 --- a/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/document-added.snap +++ b/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/document-added.snap @@ -1,8 +1,9 @@ --- -source: meilisearch/tests/vector/mod.rs +source: crates/meilisearch/tests/vector/mod.rs --- { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/document-deleted.snap b/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/document-deleted.snap index c4f1c0b25..709dfeae0 100644 --- a/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/document-deleted.snap +++ b/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/document-deleted.snap @@ -1,8 +1,9 @@ --- -source: meilisearch/tests/vector/mod.rs +source: crates/meilisearch/tests/vector/mod.rs --- { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "documentAdditionOrUpdate", diff --git a/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/settings-processed.snap b/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/settings-processed.snap index 08dbe3ee0..c8bad0070 100644 --- a/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/settings-processed.snap +++ b/crates/meilisearch/tests/vector/snapshots/mod.rs/add_remove_one_vector_4588/settings-processed.snap @@ -1,8 +1,9 @@ --- -source: meilisearch/tests/vector/mod.rs +source: crates/meilisearch/tests/vector/mod.rs --- { "uid": "[uid]", + "batchUid": "[batch_uid]", "indexUid": "doggo", "status": "succeeded", "type": "settingsUpdate",