fix the task cancelation

This commit is contained in:
Tamo 2024-11-19 01:47:42 +01:00
parent 15eefa4fcc
commit 1fcb9526f5
No known key found for this signature in database
GPG Key ID: 20CD8020AFA88D69
180 changed files with 524 additions and 644 deletions

View File

@ -667,7 +667,11 @@ impl IndexScheduler {
/// list is updated accordingly, with the exception of the its date fields
/// [`finished_at`](meilisearch_types::tasks::Task::finished_at) and [`started_at`](meilisearch_types::tasks::Task::started_at).
#[tracing::instrument(level = "trace", skip(self, batch), target = "indexing::scheduler", fields(batch=batch.to_string()))]
pub(crate) fn process_batch(&self, batch: Batch) -> Result<Vec<Task>> {
pub(crate) fn process_batch(
&self,
batch: Batch,
current_batch: &mut ProcessingBatch,
) -> Result<Vec<Task>> {
#[cfg(test)]
{
self.maybe_fail(crate::tests::FailureLocation::InsideProcessBatch)?;
@ -685,47 +689,25 @@ impl IndexScheduler {
unreachable!()
};
let mut wtxn = self.env.write_txn()?;
let canceled_tasks_content_uuids = self.cancel_matched_tasks(
&mut wtxn,
task.uid,
matched_tasks,
previous_started_at,
&previous_processing_tasks,
)?;
let rtxn = self.env.read_txn()?;
let mut canceled_tasks =
self.cancel_matched_tasks(&rtxn, task.uid, current_batch, matched_tasks)?;
task.status = Status::Succeeded;
match &mut task.details {
Some(Details::TaskCancelation {
matched_tasks: _,
canceled_tasks,
canceled_tasks: canceled_tasks_details,
original_filter: _,
}) => {
*canceled_tasks = Some(canceled_tasks_content_uuids.len() as u64);
*canceled_tasks_details = Some(canceled_tasks.len() as u64);
}
_ => unreachable!(),
}
// We must only remove the content files if the transaction is successfully committed
// and if errors occurs when we are deleting files we must do our best to delete
// everything. We do not return the encountered errors when deleting the content
// files as it is not a breaking operation and we can safely continue our job.
match wtxn.commit() {
Ok(()) => {
for content_uuid in canceled_tasks_content_uuids {
if let Err(error) = self.delete_update_file(content_uuid) {
tracing::error!(
file_content_uuid = %content_uuid,
%error,
"Failed deleting content file"
)
}
}
}
Err(e) => return Err(e.into()),
}
canceled_tasks.push(task);
Ok(vec![task])
Ok(canceled_tasks)
}
Batch::TaskDeletions(mut tasks) => {
// 1. Retrieve the tasks that matched the query at enqueue-time.
@ -1090,7 +1072,10 @@ impl IndexScheduler {
}
self.index_mapper.create_index(wtxn, &index_uid, None)?;
self.process_batch(Batch::IndexUpdate { index_uid, primary_key, task })
self.process_batch(
Batch::IndexUpdate { index_uid, primary_key, task },
current_batch,
)
}
Batch::IndexUpdate { index_uid, primary_key, mut task } => {
let rtxn = self.env.read_txn()?;
@ -1744,41 +1729,31 @@ impl IndexScheduler {
/// Cancel each given task from all the databases (if it is cancelable).
///
/// Returns the content files that the transaction owner must delete if the commit is successful.
/// Returns the list of tasks that matched the filter and must be written in the database.
fn cancel_matched_tasks(
&self,
wtxn: &mut RwTxn,
rtxn: &RoTxn,
cancel_task_id: TaskId,
current_batch: &mut ProcessingBatch,
matched_tasks: &RoaringBitmap,
previous_started_at: OffsetDateTime,
previous_processing_tasks: &RoaringBitmap,
) -> Result<Vec<Uuid>> {
let now = OffsetDateTime::now_utc();
) -> Result<Vec<Task>> {
// 1. Remove from this list the tasks that we are not allowed to cancel
// Notice that only the _enqueued_ ones are cancelable and we should
// have already aborted the indexation of the _processing_ ones
let cancelable_tasks = self.get_status(wtxn, Status::Enqueued)?;
let cancelable_tasks = self.get_status(rtxn, Status::Enqueued)?;
let tasks_to_cancel = cancelable_tasks & matched_tasks;
// 2. We now have a list of tasks to cancel, cancel them
let mut content_files_to_delete = Vec::new();
for mut task in self.get_existing_tasks(wtxn, tasks_to_cancel.iter())? {
if let Some(uuid) = task.content_uuid() {
content_files_to_delete.push(uuid);
}
if previous_processing_tasks.contains(task.uid) {
task.started_at = Some(previous_started_at);
}
let mut tasks = self.get_existing_tasks(rtxn, tasks_to_cancel.iter())?;
for task in tasks.iter_mut() {
task.status = Status::Canceled;
task.canceled_by = Some(cancel_task_id);
task.finished_at = Some(now);
task.details = task.details.map(|d| d.to_failed());
self.update_task(wtxn, &task)?;
task.details = task.details.as_ref().map(|d| d.to_failed());
current_batch.processing(Some(task));
}
self.canceled_by.put(wtxn, &cancel_task_id, &tasks_to_cancel)?;
Ok(content_files_to_delete)
Ok(tasks)
}
}

View File

@ -40,7 +40,6 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String {
batch_status,
batch_kind,
batch_index_tasks,
batch_canceled_by,
batch_enqueued_at,
batch_started_at,
batch_finished_at,
@ -131,10 +130,6 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String {
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");

View File

@ -228,7 +228,6 @@ mod db_name {
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";
@ -344,8 +343,6 @@ pub struct IndexScheduler {
pub(crate) batch_kind: Database<SerdeBincode<Kind>, RoaringBitmapCodec>,
/// Store the batches associated to an index.
pub(crate) batch_index_tasks: Database<Str, RoaringBitmapCodec>,
/// Store the batches containing a task canceled by a task uid
pub(crate) batch_canceled_by: Database<BEU32, RoaringBitmapCodec>,
/// Store the batches containing tasks which were enqueued at a specific date
pub(crate) batch_enqueued_at: Database<BEI128, CboRoaringBitmapCodec>,
/// Store the batches containing finished tasks started at a specific date
@ -437,7 +434,6 @@ impl IndexScheduler {
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,
@ -501,7 +497,7 @@ impl IndexScheduler {
let env = unsafe {
heed::EnvOpenOptions::new()
.max_dbs(20)
.max_dbs(19)
.map_size(budget.task_db_size)
.open(options.tasks_path)
}?;
@ -527,7 +523,6 @@ impl IndexScheduler {
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))?;
@ -554,7 +549,6 @@ impl IndexScheduler {
batch_status,
batch_kind,
batch_index_tasks,
batch_canceled_by,
batch_enqueued_at,
batch_started_at,
batch_finished_at,
@ -1003,16 +997,23 @@ impl IndexScheduler {
batches &= batches_by_task_uids;
}
// There is no database for this query, we must retrieve the task queried by the client and ensure it's valid
if let Some(canceled_by) = &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 let Some(task) = self.get_task(rtxn, *cancel_uid)? {
if task.kind.as_kind() == Kind::TaskCancelation
&& task.status == Status::Succeeded
{
if let Some(batch_uid) = task.batch_uid {
all_canceled_batches.insert(batch_uid);
}
}
}
}
// if the canceled_by has been specified but no batch
// matches then we prefer matching zero than all tasks.
// matches then we prefer matching zero than all batches.
if all_canceled_batches.is_empty() {
return Ok(RoaringBitmap::new());
} else {
@ -1266,24 +1267,35 @@ impl IndexScheduler {
}
}
// 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.
// Any batch that is internally associated with at least one authorized index
// must be returned.
if !filters.all_indexes_authorized() {
let mut valid_indexes = RoaringBitmap::new();
let mut forbidden_indexes = RoaringBitmap::new();
let all_indexes_iter = self.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 filters.is_index_authorized(index) {
valid_indexes |= index_tasks;
} else {
forbidden_indexes |= index_tasks;
}
}
if let Some(batch) = processing.batch.as_ref() {
for index in &batch.indexes {
if !filters.is_index_authorized(index) {
batches.remove(batch.uid);
if filters.is_index_authorized(index) {
valid_indexes.insert(batch.uid);
} else {
forbidden_indexes.insert(batch.uid);
}
}
}
// If a batch had ONE valid task then it should be returned
let invalid_batches = forbidden_indexes - valid_indexes;
batches -= invalid_batches;
}
Ok((batches, total_batches.len()))
@ -1559,11 +1571,16 @@ impl IndexScheduler {
// 2. Process the tasks
let res = {
let cloned_index_scheduler = self.private_clone();
let handle = std::thread::Builder::new()
.name(String::from("batch-operation"))
.spawn(move || cloned_index_scheduler.process_batch(batch))
.unwrap();
handle.join().unwrap_or(Err(Error::ProcessBatchPanicked))
let processing_batch = &mut processing_batch;
std::thread::scope(|s| {
let handle = std::thread::Builder::new()
.name(String::from("batch-operation"))
.spawn_scoped(s, move || {
cloned_index_scheduler.process_batch(batch, processing_batch)
})
.unwrap();
handle.join().unwrap_or(Err(Error::ProcessBatchPanicked))
})
};
// Reset the currently updating index to relinquish the index handle
@ -1582,11 +1599,20 @@ impl IndexScheduler {
let mut success = 0;
let mut failure = 0;
let mut canceled_by = None;
let mut canceled = RoaringBitmap::new();
dbg!(&tasks);
#[allow(unused_variables)]
for (i, mut task) in tasks.into_iter().enumerate() {
task.started_at = Some(processing_batch.started_at);
task.finished_at = Some(finished_at);
if task.status != Status::Canceled {
task.started_at = Some(processing_batch.started_at);
task.finished_at = Some(finished_at);
} else {
canceled.insert(task.uid);
canceled_by = task.canceled_by;
}
#[cfg(test)]
self.maybe_fail(
@ -1604,6 +1630,10 @@ impl IndexScheduler {
.map_err(|e| Error::TaskDatabaseUpdate(Box::new(e)))?;
processing_batch.update(&task);
}
if let Some(canceled_by) = canceled_by {
println!("inserting the canceled by {canceled_by}: {canceled:?}");
self.canceled_by.put(&mut wtxn, &canceled_by, &canceled)?;
}
tracing::info!("A batch of tasks was successfully completed with {success} successful tasks and {failure} failed tasks.");
}
// If we have an abortion error we must stop the tick here and re-schedule tasks.
@ -4136,6 +4166,7 @@ mod tests {
tasks: [0, 1, 2, 3].into_iter().collect(),
};
let task_cancelation = index_scheduler.register(kind, None, false).unwrap();
println!("HEEERE");
handle.advance_n_successful_batches(1);
snapshot!(snapshot_index_scheduler(&index_scheduler), name: "start");
@ -4577,13 +4608,12 @@ mod tests {
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,]");
// The batch zero was the index creation task, the 1 is the task cancellation
snapshot!(snapshot_bitmap(&batches), @"[1,]");
let query = Query { canceled_by: Some(vec![task_cancelation.uid]), ..Query::default() };
let (batches, _) = index_scheduler
.get_task_ids_from_authorized_indexes(
.get_batch_ids_from_authorized_indexes(
&rtxn,
&query,
&AuthFilter::with_allowed_indexes(
@ -5885,7 +5915,7 @@ mod tests {
let kind = KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None };
let task = index_scheduler.register(kind, None, true).unwrap();
snapshot!(task.uid, @"0");
snapshot!(snapshot_index_scheduler(&index_scheduler), @r###"
snapshot!(snapshot_index_scheduler(&index_scheduler), @r"
### Autobatching Enabled = true
### Processing batch None:
[]
@ -5919,9 +5949,6 @@ mod tests {
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
@ -5932,12 +5959,12 @@ mod tests {
### File Store:
----------------------------------------------------------------------
"###);
");
let kind = KindWithContent::IndexCreation { index_uid: S("doggo"), primary_key: None };
let task = index_scheduler.register(kind, Some(12), true).unwrap();
snapshot!(task.uid, @"12");
snapshot!(snapshot_index_scheduler(&index_scheduler), @r###"
snapshot!(snapshot_index_scheduler(&index_scheduler), @r"
### Autobatching Enabled = true
### Processing batch None:
[]
@ -5971,9 +5998,6 @@ mod tests {
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------
@ -5984,7 +6008,7 @@ mod tests {
### File Store:
----------------------------------------------------------------------
"###);
");
}
#[test]

View File

@ -1,13 +1,14 @@
---
source: index-scheduler/src/lib.rs
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing Tasks:
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: canceled, canceled_by: 1, details: { received_documents: 1, indexed_documents: Some(0) }, 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(1), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0]> }}
0 {uid: 0, batch_uid: 0, status: canceled, canceled_by: 1, details: { received_documents: 1, indexed_documents: Some(0) }, 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: 0, status: succeeded, details: { matched_tasks: 1, canceled_tasks: Some(1), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0]> }}
----------------------------------------------------------------------
### Status:
enqueued []
@ -36,10 +37,35 @@ catto [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [1,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,]
----------------------------------------------------------------------
### Batches Kind:
"documentAdditionOrUpdate" [0,]
"taskCancelation" [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [0,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
----------------------------------------------------------------------
### File Store:
00000000-0000-0000-0000-000000000000
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ catto [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(1):
@ -58,9 +59,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,15 +1,16 @@
---
source: index-scheduler/src/lib.rs
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### 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: canceled, canceled_by: 3, details: { received_documents: 1, indexed_documents: Some(0) }, 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: canceled, canceled_by: 3, details: { received_documents: 1, indexed_documents: Some(0) }, 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: succeeded, details: { matched_tasks: 3, canceled_tasks: Some(2), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0, 1, 2]> }}
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: canceled, canceled_by: 3, details: { received_documents: 1, indexed_documents: Some(0) }, 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, batch_uid: 1, status: canceled, canceled_by: 3, details: { received_documents: 1, indexed_documents: Some(0) }, 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, batch_uid: 1, status: succeeded, details: { matched_tasks: 3, canceled_tasks: Some(2), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0, 1, 2]> }}
----------------------------------------------------------------------
### Status:
enqueued []
@ -42,15 +43,47 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} }
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [3,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,2,]
[timestamp] [3,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, }
1 {uid: 1, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [3,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,]
----------------------------------------------------------------------
### Batches Kind:
"documentAdditionOrUpdate" [0,1,]
"taskCancelation" [1,]
----------------------------------------------------------------------
### Batches Index Tasks:
beavero [1,]
catto [0,]
wolfo [1,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### File Store:
00000000-0000-0000-0000-000000000001
00000000-0000-0000-0000-000000000002
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -54,9 +55,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(1):
@ -57,9 +58,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -38,9 +39,6 @@ enqueued [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,13 +1,14 @@
---
source: index-scheduler/src/lib.rs
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing Tasks:
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: canceled, canceled_by: 1, details: { dump_uid: None }, kind: DumpCreation { keys: [], instance_uid: None }}
1 {uid: 1, status: succeeded, details: { matched_tasks: 1, canceled_tasks: Some(0), original_filter: "cancel dump" }, kind: TaskCancelation { query: "cancel dump", tasks: RoaringBitmap<[0]> }}
0 {uid: 0, batch_uid: 0, status: canceled, canceled_by: 1, details: { dump_uid: None }, kind: DumpCreation { keys: [], instance_uid: None }}
1 {uid: 1, batch_uid: 0, status: succeeded, details: { matched_tasks: 1, canceled_tasks: Some(1), original_filter: "cancel dump" }, kind: TaskCancelation { query: "cancel dump", tasks: RoaringBitmap<[0]> }}
----------------------------------------------------------------------
### Status:
enqueued []
@ -32,14 +33,36 @@ canceled [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [1,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,]
----------------------------------------------------------------------
### Batches Kind:
"taskCancelation" [0,]
"dumpCreation" [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [0,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -41,9 +42,6 @@ enqueued [0,1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -43,9 +44,6 @@ catto: { number_of_documents: 0, field_distribution: {} }
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,13 +1,14 @@
---
source: index-scheduler/src/lib.rs
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing Tasks:
### Processing batch None:
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, status: canceled, canceled_by: 1, details: { received_documents: 1, indexed_documents: Some(0) }, 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(1), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0]> }}
0 {uid: 0, batch_uid: 0, status: canceled, canceled_by: 1, details: { received_documents: 1, indexed_documents: Some(0) }, 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: 0, status: succeeded, details: { matched_tasks: 1, canceled_tasks: Some(1), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0]> }}
----------------------------------------------------------------------
### Status:
enqueued []
@ -34,14 +35,38 @@ catto: { number_of_documents: 0, field_distribution: {} }
[timestamp] [1,]
----------------------------------------------------------------------
### Started At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [1,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,]
----------------------------------------------------------------------
### Batches Kind:
"documentAdditionOrUpdate" [0,]
"taskCancelation" [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [0,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
----------------------------------------------------------------------
### File Store:
00000000-0000-0000-0000-000000000000
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -42,9 +43,6 @@ catto [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -39,9 +40,6 @@ catto [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ catto [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -25,7 +26,6 @@ catto: { number_of_documents: 1, field_distribution: {"id": 1} }
----------------------------------------------------------------------
### Canceled By:
1 []
----------------------------------------------------------------------
### Enqueued At:
@ -57,9 +57,6 @@ succeeded [0,1,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -48,9 +49,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ catto [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -86,9 +87,6 @@ succeeded [0,1,2,3,4,5,]
cattos [1,4,]
doggos [0,3,]
girafos [2,5,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -48,9 +49,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -52,9 +53,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ doggos [0,1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -54,9 +55,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -59,9 +60,6 @@ succeeded [0,1,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ doggos [0,1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -45,9 +46,6 @@ doggos [0,1,2,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ doggos [0,1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -51,9 +52,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -50,9 +51,6 @@ failed [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -58,9 +59,6 @@ failed [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ doggos [0,1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -47,9 +48,6 @@ failed [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -56,9 +57,6 @@ succeeded [0,1,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -51,9 +52,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -63,7 +64,6 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} }
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,2,]
failed [2,]
----------------------------------------------------------------------
### Batches Kind:
"documentAdditionOrUpdate" [1,]
@ -72,9 +72,6 @@ failed [2,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,2,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -65,9 +66,6 @@ succeeded [0,1,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ doggos [0,1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ catto [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -47,9 +48,6 @@ failed [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -40,9 +41,6 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} }
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -40,9 +41,6 @@ doggos: { number_of_documents: 1, field_distribution: {"doggo": 1, "id": 1} }
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -48,9 +49,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -62,9 +63,6 @@ succeeded [0,1,2,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,2,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -58,9 +59,6 @@ succeeded [0,1,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -56,9 +57,6 @@ succeeded [0,1,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -51,9 +52,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -48,9 +49,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -39,9 +40,6 @@ index_a [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ index_a [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -42,9 +43,6 @@ index_b [1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch Some(0):
@ -45,9 +46,6 @@ index_b [1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -47,9 +48,6 @@ failed [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ catto [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -54,9 +55,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -60,9 +61,6 @@ succeeded [0,1,]
### Batches Index Tasks:
cattos [1,]
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -64,9 +65,6 @@ succeeded [0,1,2,]
### Batches Index Tasks:
cattos [1,]
doggos [0,2,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -45,9 +46,6 @@ doggos [0,2,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = false
### Processing batch None:
@ -55,9 +56,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = false
### Processing batch None:
@ -68,9 +69,6 @@ succeeded [0,1,2,3,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,2,3,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = false
### Processing batch None:
@ -39,9 +40,6 @@ doggos [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = false
### Processing batch None:
@ -46,9 +47,6 @@ doggos [0,1,2,3,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = false
### Processing batch None:
@ -42,9 +43,6 @@ doggos [0,1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = false
### Processing batch None:
@ -44,9 +45,6 @@ doggos [0,1,2,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = false
### Processing batch None:
@ -60,9 +61,6 @@ succeeded [0,1,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = false
### Processing batch None:
@ -64,9 +65,6 @@ succeeded [0,1,2,]
----------------------------------------------------------------------
### Batches Index Tasks:
doggos [0,1,2,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

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

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -66,9 +67,6 @@ succeeded [0,1,2,]
catto [2,]
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggo [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ whalo [1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -45,9 +46,6 @@ whalo [1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -67,9 +68,6 @@ failed [2,]
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -45,9 +46,6 @@ whalo [2,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -75,9 +76,6 @@ failed [2,3,]
catto [0,2,3,]
doggo [1,2,]
whalo [3,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -48,9 +49,6 @@ whalo [3,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,15 +1,16 @@
---
source: index-scheduler/src/lib.rs
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### 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: canceled, canceled_by: 3, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, status: canceled, canceled_by: 3, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }}
3 {uid: 3, status: succeeded, details: { matched_tasks: 3, canceled_tasks: Some(0), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0, 1, 2]> }}
0 {uid: 0, batch_uid: 0, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
1 {uid: 1, batch_uid: 1, status: canceled, canceled_by: 3, details: { primary_key: Some("sheep") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("sheep") }}
2 {uid: 2, batch_uid: 1, status: canceled, canceled_by: 3, details: { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }, kind: IndexSwap { swaps: [IndexSwap { indexes: ("catto", "doggo") }] }}
3 {uid: 3, batch_uid: 1, status: succeeded, details: { matched_tasks: 3, canceled_tasks: Some(2), original_filter: "test_query" }, kind: TaskCancelation { query: "test_query", tasks: RoaringBitmap<[0, 1, 2]> }}
----------------------------------------------------------------------
### Status:
enqueued []
@ -45,10 +46,41 @@ catto: { number_of_documents: 0, field_distribution: {} }
----------------------------------------------------------------------
### Finished At:
[timestamp] [0,]
[timestamp] [1,2,]
[timestamp] [3,]
----------------------------------------------------------------------
### All Batches:
0 {uid: 0, }
1 {uid: 1, }
----------------------------------------------------------------------
### Batch to tasks mapping:
0 [0,]
1 [3,]
----------------------------------------------------------------------
### Batches Status:
succeeded [0,1,]
----------------------------------------------------------------------
### Batches Kind:
"indexCreation" [0,1,]
"indexSwap" [1,]
"taskCancelation" [1,]
----------------------------------------------------------------------
### Batches Index Tasks:
catto [0,1,]
doggo [1,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### File Store:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -66,9 +67,6 @@ succeeded [0,1,2,]
catto [2,]
doggo [0,]
whalo [1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -39,9 +40,6 @@ doggo [0,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -42,9 +43,6 @@ whalo [1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -45,9 +46,6 @@ whalo [1,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -67,9 +68,6 @@ failed [2,]
catto [0,]
doggo [1,]
whalo [2,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -45,9 +46,6 @@ whalo [2,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -48,9 +49,6 @@ whalo [3,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -47,9 +48,6 @@ doggo [3,]
### Batches Kind:
----------------------------------------------------------------------
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
----------------------------------------------------------------------

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -57,9 +58,6 @@ succeeded [0,]
----------------------------------------------------------------------
### Batches Index Tasks:
a [0,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -63,9 +64,6 @@ succeeded [0,1,]
### Batches Index Tasks:
a [0,]
b [1,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

View File

@ -1,5 +1,6 @@
---
source: crates/index-scheduler/src/lib.rs
snapshot_kind: text
---
### Autobatching Enabled = true
### Processing batch None:
@ -69,9 +70,6 @@ succeeded [0,1,2,]
a [0,]
b [1,]
c [2,]
----------------------------------------------------------------------
### Batches Canceled By:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]

Some files were not shown because too many files have changed in this diff Show More