inital implementation of the progress

This commit is contained in:
Tamo 2024-12-10 16:30:48 +01:00
parent 8c19cb0a0b
commit df9b68f8ed
No known key found for this signature in database
GPG key ID: 20CD8020AFA88D69
29 changed files with 585 additions and 414 deletions

View file

@ -22,8 +22,6 @@ use std::ffi::OsStr;
use std::fmt;
use std::fs::{self, File};
use std::io::BufWriter;
use std::sync::atomic::{self, AtomicU64};
use std::time::Duration;
use bumpalo::collections::CollectIn;
use bumpalo::Bump;
@ -32,6 +30,7 @@ use meilisearch_types::batches::BatchId;
use meilisearch_types::heed::{RoTxn, RwTxn};
use meilisearch_types::milli::documents::{obkv_to_object, DocumentsBatchReader, PrimaryKey};
use meilisearch_types::milli::heed::CompactionOption;
use meilisearch_types::milli::progress::Progress;
use meilisearch_types::milli::update::new::indexer::{self, UpdateByFunction};
use meilisearch_types::milli::update::{
DocumentAdditionResult, IndexDocumentsMethod, Settings as MilliSettings,
@ -41,9 +40,7 @@ use meilisearch_types::milli::vector::parsed_vectors::{
};
use meilisearch_types::milli::{self, Filter, ThreadPoolNoAbortBuilder};
use meilisearch_types::settings::{apply_settings_to_builder, Settings, Unchecked};
use meilisearch_types::tasks::{
Details, IndexSwap, Kind, KindWithContent, Status, Task, TaskProgress,
};
use meilisearch_types::tasks::{Details, IndexSwap, Kind, KindWithContent, Status, Task};
use meilisearch_types::{compression, Index, VERSION_FILE_NAME};
use roaring::RoaringBitmap;
use time::macros::format_description;
@ -561,11 +558,12 @@ impl IndexScheduler {
/// The list of tasks that were processed. The metadata of each task in the returned
/// 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()))]
#[tracing::instrument(level = "trace", skip(self, batch, progress), target = "indexing::scheduler", fields(batch=batch.to_string()))]
pub(crate) fn process_batch(
&self,
batch: Batch,
current_batch: &mut ProcessingBatch,
progress: Progress,
) -> Result<Vec<Task>> {
#[cfg(test)]
{
@ -953,7 +951,7 @@ impl IndexScheduler {
.set_currently_updating_index(Some((index_uid.clone(), index.clone())));
let mut index_wtxn = index.write_txn()?;
let tasks = self.apply_index_operation(&mut index_wtxn, &index, op)?;
let tasks = self.apply_index_operation(&mut index_wtxn, &index, op, progress)?;
{
let span = tracing::trace_span!(target: "indexing::scheduler", "commit");
@ -996,6 +994,7 @@ impl IndexScheduler {
self.process_batch(
Batch::IndexUpdate { index_uid, primary_key, task },
current_batch,
progress,
)
}
Batch::IndexUpdate { index_uid, primary_key, mut task } => {
@ -1168,7 +1167,7 @@ impl IndexScheduler {
/// The list of processed tasks.
#[tracing::instrument(
level = "trace",
skip(self, index_wtxn, index),
skip(self, index_wtxn, index, progress),
target = "indexing::scheduler"
)]
fn apply_index_operation<'i>(
@ -1176,44 +1175,12 @@ impl IndexScheduler {
index_wtxn: &mut RwTxn<'i>,
index: &'i Index,
operation: IndexOperation,
progress: Progress,
) -> Result<Vec<Task>> {
let indexer_alloc = Bump::new();
let started_processing_at = std::time::Instant::now();
let secs_since_started_processing_at = AtomicU64::new(0);
const PRINT_SECS_DELTA: u64 = 5;
let processing_tasks = self.processing_tasks.clone();
let must_stop_processing = self.must_stop_processing.clone();
let send_progress = |progress| {
let now = std::time::Instant::now();
let elapsed = secs_since_started_processing_at.load(atomic::Ordering::Relaxed);
let previous = started_processing_at + Duration::from_secs(elapsed);
let elapsed = now - previous;
if elapsed.as_secs() < PRINT_SECS_DELTA {
return;
}
secs_since_started_processing_at
.store((now - started_processing_at).as_secs(), atomic::Ordering::Relaxed);
let TaskProgress {
current_step,
finished_steps,
total_steps,
finished_substeps,
total_substeps,
} = processing_tasks.write().unwrap().update_progress(progress);
tracing::info!(
current_step,
finished_steps,
total_steps,
finished_substeps,
total_substeps
);
};
match operation {
IndexOperation::DocumentClear { index_uid, mut tasks } => {
@ -1308,7 +1275,7 @@ impl IndexScheduler {
primary_key.as_deref(),
&mut new_fields_ids_map,
&|| must_stop_processing.get(),
&send_progress,
progress.clone(),
)
.map_err(|e| Error::from_milli(e, Some(index_uid.clone())))?;
@ -1356,7 +1323,7 @@ impl IndexScheduler {
&document_changes,
embedders,
&|| must_stop_processing.get(),
&send_progress,
&progress,
)
.map_err(|e| Error::from_milli(e, Some(index_uid.clone())))?;
@ -1470,7 +1437,7 @@ impl IndexScheduler {
&document_changes,
embedders,
&|| must_stop_processing.get(),
&send_progress,
&progress,
)
.map_err(|err| Error::from_milli(err, Some(index_uid.clone())))?;
@ -1621,7 +1588,7 @@ impl IndexScheduler {
&document_changes,
embedders,
&|| must_stop_processing.get(),
&send_progress,
&progress,
)
.map_err(|err| Error::from_milli(err, Some(index_uid.clone())))?;
@ -1673,12 +1640,14 @@ impl IndexScheduler {
index_uid: index_uid.clone(),
tasks: cleared_tasks,
},
progress.clone(),
)?;
let settings_tasks = self.apply_index_operation(
index_wtxn,
index,
IndexOperation::Settings { index_uid, settings, tasks: settings_tasks },
progress,
)?;
let mut tasks = settings_tasks;
@ -1702,8 +1671,8 @@ impl IndexScheduler {
let all_task_ids = self.all_task_ids(wtxn)?;
let mut to_delete_tasks = all_task_ids & matched_tasks;
to_delete_tasks -= processing_tasks;
to_delete_tasks -= enqueued_tasks;
to_delete_tasks -= &**processing_tasks;
to_delete_tasks -= &enqueued_tasks;
// 2. We now have a list of tasks to delete, delete them

View file

@ -353,7 +353,7 @@ pub fn snapshot_canceled_by(rtxn: &RoTxn, db: Database<BEU32, RoaringBitmapCodec
pub fn snapshot_batch(batch: &Batch) -> String {
let mut snap = String::new();
let Batch { uid, details, stats, started_at, finished_at } = batch;
let Batch { uid, details, stats, started_at, finished_at, progress: _ } = batch;
if let Some(finished_at) = finished_at {
assert!(finished_at > started_at);
}

View file

@ -26,6 +26,7 @@ mod index_mapper;
#[cfg(test)]
mod insta_snapshot;
mod lru;
mod processing;
mod utils;
pub mod uuid_codec;
@ -56,12 +57,12 @@ use meilisearch_types::heed::types::{SerdeBincode, SerdeJson, Str, I128};
use meilisearch_types::heed::{self, Database, Env, PutFlags, RoTxn, RwTxn};
use meilisearch_types::milli::documents::DocumentsBatchBuilder;
use meilisearch_types::milli::index::IndexEmbeddingConfig;
use meilisearch_types::milli::update::new::indexer::document_changes::Progress;
use meilisearch_types::milli::update::IndexerConfig;
use meilisearch_types::milli::vector::{Embedder, EmbedderOptions, EmbeddingConfigs};
use meilisearch_types::milli::{self, CboRoaringBitmapCodec, Index, RoaringBitmapCodec, BEU32};
use meilisearch_types::task_view::TaskView;
use meilisearch_types::tasks::{Kind, KindWithContent, Status, Task, TaskProgress};
use meilisearch_types::tasks::{Kind, KindWithContent, Status, Task};
use processing::ProcessingTasks;
use rayon::current_num_threads;
use rayon::prelude::{IntoParallelIterator, ParallelIterator};
use roaring::RoaringBitmap;
@ -72,7 +73,8 @@ use utils::{filter_out_references_to_newer_tasks, keep_ids_within_datetimes, map
use uuid::Uuid;
use crate::index_mapper::IndexMapper;
use crate::utils::{check_index_swap_validity, clamp_to_page_size, ProcessingBatch};
use crate::processing::{AtomicTaskStep, BatchProgress};
use crate::utils::{check_index_swap_validity, clamp_to_page_size};
pub(crate) type BEI128 = I128<BE>;
@ -163,48 +165,6 @@ impl Query {
}
}
#[derive(Debug, Clone)]
pub struct ProcessingTasks {
batch: Option<ProcessingBatch>,
/// The list of tasks ids that are currently running.
processing: RoaringBitmap,
/// The progress on processing tasks
progress: Option<TaskProgress>,
}
impl ProcessingTasks {
/// Creates an empty `ProcessingAt` struct.
fn new() -> ProcessingTasks {
ProcessingTasks { batch: None, processing: RoaringBitmap::new(), progress: None }
}
/// Stores the currently processing tasks, and the date time at which it started.
fn start_processing(&mut self, processing_batch: ProcessingBatch, processing: RoaringBitmap) {
self.batch = Some(processing_batch);
self.processing = processing;
}
fn update_progress(&mut self, progress: Progress) -> TaskProgress {
self.progress.get_or_insert_with(TaskProgress::default).update(progress)
}
/// Set the processing tasks to an empty list
fn stop_processing(&mut self) -> Self {
self.progress = None;
Self {
batch: std::mem::take(&mut self.batch),
processing: std::mem::take(&mut self.processing),
progress: None,
}
}
/// Returns `true` if there, at least, is one task that is currently processing that we must stop.
fn must_cancel_processing_tasks(&self, canceled_tasks: &RoaringBitmap) -> bool {
!self.processing.is_disjoint(canceled_tasks)
}
}
#[derive(Default, Clone, Debug)]
struct MustStopProcessing(Arc<AtomicBool>);
@ -813,7 +773,7 @@ impl IndexScheduler {
let mut batch_tasks = RoaringBitmap::new();
for batch_uid in batch_uids {
if processing_batch.as_ref().map_or(false, |batch| batch.uid == *batch_uid) {
batch_tasks |= &processing_tasks;
batch_tasks |= &*processing_tasks;
} else {
batch_tasks |= self.tasks_in_batch(rtxn, *batch_uid)?;
}
@ -827,13 +787,13 @@ impl IndexScheduler {
match status {
// special case for Processing tasks
Status::Processing => {
status_tasks |= &processing_tasks;
status_tasks |= &*processing_tasks;
}
status => status_tasks |= &self.get_status(rtxn, *status)?,
};
}
if !status.contains(&Status::Processing) {
tasks -= &processing_tasks;
tasks -= &*processing_tasks;
}
tasks &= status_tasks;
}
@ -882,7 +842,7 @@ impl IndexScheduler {
// Once we have filtered the two subsets, we put them back together and assign it back to `tasks`.
tasks = {
let (mut filtered_non_processing_tasks, mut filtered_processing_tasks) =
(&tasks - &processing_tasks, &tasks & &processing_tasks);
(&tasks - &*processing_tasks, &tasks & &*processing_tasks);
// special case for Processing tasks
// A closure that clears the filtered_processing_tasks if their started_at date falls outside the given bounds
@ -1090,7 +1050,7 @@ impl IndexScheduler {
// 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);
(&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
@ -1606,7 +1566,8 @@ impl IndexScheduler {
// We reset the must_stop flag to be sure that we don't stop processing tasks
self.must_stop_processing.reset();
self.processing_tasks
let progress = 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
@ -1619,11 +1580,12 @@ impl IndexScheduler {
let res = {
let cloned_index_scheduler = self.private_clone();
let processing_batch = &mut processing_batch;
let progress = progress.clone();
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)
cloned_index_scheduler.process_batch(batch, processing_batch, progress)
})
.unwrap();
handle.join().unwrap_or(Err(Error::ProcessBatchPanicked))
@ -1636,6 +1598,7 @@ impl IndexScheduler {
#[cfg(test)]
self.maybe_fail(tests::FailureLocation::AcquiringWtxn)?;
progress.update_progress(BatchProgress::WritingTasksToDisk);
processing_batch.finished();
let mut wtxn = self.env.write_txn().map_err(Error::HeedTransaction)?;
let mut canceled = RoaringBitmap::new();
@ -1645,12 +1608,15 @@ impl IndexScheduler {
#[cfg(test)]
self.breakpoint(Breakpoint::ProcessBatchSucceeded);
let (task_progress, task_progress_obj) = AtomicTaskStep::new(tasks.len() as u32);
progress.update_progress(task_progress_obj);
let mut success = 0;
let mut failure = 0;
let mut canceled_by = None;
#[allow(unused_variables)]
for (i, mut task) in tasks.into_iter().enumerate() {
task_progress.fetch_add(1, Ordering::Relaxed);
processing_batch.update(&mut task);
if task.status == Status::Canceled {
canceled.insert(task.uid);
@ -1718,8 +1684,12 @@ impl IndexScheduler {
Err(err) => {
#[cfg(test)]
self.breakpoint(Breakpoint::ProcessBatchFailed);
let (task_progress, task_progress_obj) = AtomicTaskStep::new(ids.len() as u32);
progress.update_progress(task_progress_obj);
let error: ResponseError = err.into();
for id in ids.iter() {
task_progress.fetch_add(1, Ordering::Relaxed);
let mut task = self
.get_task(&wtxn, id)
.map_err(|e| Error::TaskDatabaseUpdate(Box::new(e)))?

View file

@ -0,0 +1,205 @@
use crate::utils::ProcessingBatch;
use meilisearch_types::milli::progress::{AtomicSubStep, NamedStep, Progress, ProgressView, Step};
use roaring::RoaringBitmap;
use std::{borrow::Cow, sync::Arc};
#[derive(Clone)]
pub struct ProcessingTasks {
pub batch: Option<Arc<ProcessingBatch>>,
/// The list of tasks ids that are currently running.
pub processing: Arc<RoaringBitmap>,
/// The progress on processing tasks
pub progress: Option<Progress>,
}
impl ProcessingTasks {
/// Creates an empty `ProcessingAt` struct.
pub fn new() -> ProcessingTasks {
ProcessingTasks { batch: None, processing: Arc::new(RoaringBitmap::new()), progress: None }
}
pub fn get_progress_view(&self) -> Option<ProgressView> {
Some(self.progress.as_ref()?.as_progress_view())
}
/// Stores the currently processing tasks, and the date time at which it started.
pub fn start_processing(
&mut self,
processing_batch: ProcessingBatch,
processing: RoaringBitmap,
) -> Progress {
self.batch = Some(Arc::new(processing_batch));
self.processing = Arc::new(processing);
let progress = Progress::default();
progress.update_progress(BatchProgress::ProcessingTasks);
self.progress = Some(progress.clone());
progress
}
/// Set the processing tasks to an empty list
pub fn stop_processing(&mut self) -> Self {
self.progress = None;
Self {
batch: std::mem::take(&mut self.batch),
processing: std::mem::take(&mut self.processing),
progress: None,
}
}
/// Returns `true` if there, at least, is one task that is currently processing that we must stop.
pub fn must_cancel_processing_tasks(&self, canceled_tasks: &RoaringBitmap) -> bool {
!self.processing.is_disjoint(canceled_tasks)
}
}
#[repr(u8)]
#[derive(Copy, Clone)]
pub enum BatchProgress {
ProcessingTasks,
WritingTasksToDisk,
}
impl Step for BatchProgress {
fn name(&self) -> Cow<'static, str> {
match self {
BatchProgress::ProcessingTasks => Cow::Borrowed("processing tasks"),
BatchProgress::WritingTasksToDisk => Cow::Borrowed("writing tasks to disk"),
}
}
fn current(&self) -> u32 {
*self as u8 as u32
}
fn total(&self) -> u32 {
2
}
}
#[derive(Default)]
pub struct Task {}
impl NamedStep for Task {
fn name(&self) -> &'static str {
"task"
}
}
pub type AtomicTaskStep = AtomicSubStep<Task>;
#[cfg(test)]
mod test {
use std::sync::atomic::Ordering;
use meili_snap::{json_string, snapshot};
use super::*;
#[test]
fn one_level() {
let mut processing = ProcessingTasks::new();
processing.start_processing(ProcessingBatch::new(0), RoaringBitmap::new());
snapshot!(json_string!(processing.get_progress_view()), @r#"
{
"steps": [
{
"name": "processing tasks",
"finished": 0,
"total": 2
}
],
"percentage": 0.0
}
"#);
processing.progress.as_ref().unwrap().update_progress(BatchProgress::WritingTasksToDisk);
snapshot!(json_string!(processing.get_progress_view()), @r#"
{
"steps": [
{
"name": "writing tasks to disk",
"finished": 1,
"total": 2
}
],
"percentage": 50.0
}
"#);
}
#[test]
fn task_progress() {
let mut processing = ProcessingTasks::new();
processing.start_processing(ProcessingBatch::new(0), RoaringBitmap::new());
let (atomic, tasks) = AtomicTaskStep::new(10);
processing.progress.as_ref().unwrap().update_progress(tasks);
snapshot!(json_string!(processing.get_progress_view()), @r#"
{
"steps": [
{
"name": "processing tasks",
"finished": 0,
"total": 2
},
{
"name": "task",
"finished": 0,
"total": 10
}
],
"percentage": 0.0
}
"#);
atomic.fetch_add(6, Ordering::Relaxed);
snapshot!(json_string!(processing.get_progress_view()), @r#"
{
"steps": [
{
"name": "processing tasks",
"finished": 0,
"total": 2
},
{
"name": "task",
"finished": 6,
"total": 10
}
],
"percentage": 30.000002
}
"#);
processing.progress.as_ref().unwrap().update_progress(BatchProgress::WritingTasksToDisk);
snapshot!(json_string!(processing.get_progress_view()), @r#"
{
"steps": [
{
"name": "writing tasks to disk",
"finished": 1,
"total": 2
}
],
"percentage": 50.0
}
"#);
let (atomic, tasks) = AtomicTaskStep::new(5);
processing.progress.as_ref().unwrap().update_progress(tasks);
atomic.fetch_add(4, Ordering::Relaxed);
snapshot!(json_string!(processing.get_progress_view()), @r#"
{
"steps": [
{
"name": "writing tasks to disk",
"finished": 1,
"total": 2
},
{
"name": "task",
"finished": 4,
"total": 5
}
],
"percentage": 90.0
}
"#);
}
}

View file

@ -134,6 +134,7 @@ impl ProcessingBatch {
pub fn to_batch(&self) -> Batch {
Batch {
uid: self.uid,
progress: None,
details: self.details.clone(),
stats: self.stats.clone(),
started_at: self.started_at,
@ -187,6 +188,7 @@ impl IndexScheduler {
&batch.uid,
&Batch {
uid: batch.uid,
progress: None,
details: batch.details,
stats: batch.stats,
started_at: batch.started_at,
@ -273,7 +275,10 @@ impl IndexScheduler {
.into_iter()
.map(|batch_id| {
if Some(batch_id) == processing.batch.as_ref().map(|batch| batch.uid) {
Ok(processing.batch.as_ref().unwrap().to_batch())
let mut batch = processing.batch.as_ref().unwrap().to_batch();
println!("here with progress: {}", processing.progress.is_some());
batch.progress = processing.get_progress_view();
Ok(batch)
} else {
self.get_batch(rtxn, batch_id)
.and_then(|task| task.ok_or(Error::CorruptedTaskQueue))