Split the index-scheduler in ~500 loc modules

This commit is contained in:
Tamo
2025-01-02 14:13:51 +01:00
parent 7f1071943e
commit cb82b0798a
251 changed files with 9431 additions and 9079 deletions

View File

@ -4,15 +4,14 @@ use std::collections::{BTreeSet, HashSet};
use std::ops::Bound;
use meilisearch_types::batches::{Batch, BatchId, BatchStats};
use meilisearch_types::heed::types::DecodeIgnore;
use meilisearch_types::heed::{Database, RoTxn, RwTxn};
use meilisearch_types::milli::CboRoaringBitmapCodec;
use meilisearch_types::task_view::DetailsView;
use meilisearch_types::tasks::{Details, IndexSwap, Kind, KindWithContent, Status};
use roaring::{MultiOps, RoaringBitmap};
use roaring::RoaringBitmap;
use time::OffsetDateTime;
use crate::{Error, IndexScheduler, ProcessingTasks, Result, Task, TaskId, BEI128};
use crate::{Error, Result, Task, TaskId, BEI128};
/// This structure contains all the information required to write a batch in the database without reading the tasks.
/// It'll stay in RAM so it must be small.
@ -22,7 +21,7 @@ use crate::{Error, IndexScheduler, ProcessingTasks, Result, Task, TaskId, BEI128
/// 3. Call `finished` once the batch has been processed.
/// 4. Call `update` on all the tasks.
#[derive(Debug, Clone)]
pub(crate) struct ProcessingBatch {
pub struct ProcessingBatch {
pub uid: BatchId,
pub details: DetailsView,
pub stats: BatchStats,
@ -143,349 +142,6 @@ impl ProcessingBatch {
}
}
impl IndexScheduler {
pub(crate) fn all_task_ids(&self, rtxn: &RoTxn) -> Result<RoaringBitmap> {
enum_iterator::all().map(|s| self.get_status(rtxn, s)).union()
}
pub(crate) fn all_batch_ids(&self, rtxn: &RoTxn) -> Result<RoaringBitmap> {
enum_iterator::all().map(|s| self.get_batch_status(rtxn, s)).union()
}
pub(crate) fn last_task_id(&self, rtxn: &RoTxn) -> Result<Option<TaskId>> {
Ok(self.all_tasks.remap_data_type::<DecodeIgnore>().last(rtxn)?.map(|(k, _)| k + 1))
}
pub(crate) fn next_task_id(&self, rtxn: &RoTxn) -> Result<TaskId> {
Ok(self.last_task_id(rtxn)?.unwrap_or_default())
}
pub(crate) fn next_batch_id(&self, rtxn: &RoTxn) -> Result<BatchId> {
Ok(self
.all_batches
.remap_data_type::<DecodeIgnore>()
.last(rtxn)?
.map(|(k, _)| k + 1)
.unwrap_or_default())
}
pub(crate) fn get_task(&self, rtxn: &RoTxn, task_id: TaskId) -> Result<Option<Task>> {
Ok(self.all_tasks.get(rtxn, &task_id)?)
}
pub(crate) fn get_batch(&self, rtxn: &RoTxn, batch_id: BatchId) -> Result<Option<Batch>> {
Ok(self.all_batches.get(rtxn, &batch_id)?)
}
pub(crate) fn write_batch(
&self,
wtxn: &mut RwTxn,
batch: ProcessingBatch,
tasks: &RoaringBitmap,
) -> Result<()> {
self.all_batches.put(
wtxn,
&batch.uid,
&Batch {
uid: batch.uid,
progress: None,
details: batch.details,
stats: batch.stats,
started_at: batch.started_at,
finished_at: batch.finished_at,
},
)?;
self.batch_to_tasks_mapping.put(wtxn, &batch.uid, tasks)?;
for status in batch.statuses {
self.update_batch_status(wtxn, status, |bitmap| {
bitmap.insert(batch.uid);
})?;
}
for kind in batch.kinds {
self.update_batch_kind(wtxn, kind, |bitmap| {
bitmap.insert(batch.uid);
})?;
}
for index in batch.indexes {
self.update_batch_index(wtxn, &index, |bitmap| {
bitmap.insert(batch.uid);
})?;
}
if let Some(enqueued_at) = batch.oldest_enqueued_at {
insert_task_datetime(wtxn, self.batch_enqueued_at, enqueued_at, batch.uid)?;
}
if let Some(enqueued_at) = batch.earliest_enqueued_at {
insert_task_datetime(wtxn, self.batch_enqueued_at, enqueued_at, batch.uid)?;
}
insert_task_datetime(wtxn, self.batch_started_at, batch.started_at, batch.uid)?;
insert_task_datetime(wtxn, self.batch_finished_at, batch.finished_at.unwrap(), batch.uid)?;
Ok(())
}
/// Convert an iterator to a `Vec` of tasks and edit the `ProcessingBatch` to add the given tasks.
///
/// The tasks MUST exist, or a `CorruptedTaskQueue` error will be thrown.
pub(crate) fn get_existing_tasks_for_processing_batch(
&self,
rtxn: &RoTxn,
processing_batch: &mut ProcessingBatch,
tasks: impl IntoIterator<Item = TaskId>,
) -> Result<Vec<Task>> {
tasks
.into_iter()
.map(|task_id| {
let mut task = self
.get_task(rtxn, task_id)
.and_then(|task| task.ok_or(Error::CorruptedTaskQueue));
processing_batch.processing(&mut task);
task
})
.collect::<Result<_>>()
}
/// Convert an iterator to a `Vec` of tasks. The tasks MUST exist or a
/// `CorruptedTaskQueue` error will be thrown.
pub(crate) fn get_existing_tasks(
&self,
rtxn: &RoTxn,
tasks: impl IntoIterator<Item = TaskId>,
) -> Result<Vec<Task>> {
tasks
.into_iter()
.map(|task_id| {
self.get_task(rtxn, task_id).and_then(|task| task.ok_or(Error::CorruptedTaskQueue))
})
.collect::<Result<_>>()
}
/// Convert an iterator to a `Vec` of batches. The batches MUST exist or a
/// `CorruptedTaskQueue` error will be thrown.
pub(crate) fn get_existing_batches(
&self,
rtxn: &RoTxn,
processing: &ProcessingTasks,
tasks: impl IntoIterator<Item = BatchId>,
) -> Result<Vec<Batch>> {
tasks
.into_iter()
.map(|batch_id| {
if Some(batch_id) == processing.batch.as_ref().map(|batch| batch.uid) {
let mut batch = processing.batch.as_ref().unwrap().to_batch();
batch.progress = processing.get_progress_view();
Ok(batch)
} else {
self.get_batch(rtxn, batch_id)
.and_then(|task| task.ok_or(Error::CorruptedTaskQueue))
}
})
.collect::<Result<_>>()
}
pub(crate) fn update_task(&self, wtxn: &mut RwTxn, task: &Task) -> Result<()> {
let old_task = self.get_task(wtxn, task.uid)?.ok_or(Error::CorruptedTaskQueue)?;
debug_assert!(old_task != *task);
debug_assert_eq!(old_task.uid, task.uid);
debug_assert!(old_task.batch_uid.is_none() && task.batch_uid.is_some());
if old_task.status != task.status {
self.update_status(wtxn, old_task.status, |bitmap| {
bitmap.remove(task.uid);
})?;
self.update_status(wtxn, task.status, |bitmap| {
bitmap.insert(task.uid);
})?;
}
if old_task.kind.as_kind() != task.kind.as_kind() {
self.update_kind(wtxn, old_task.kind.as_kind(), |bitmap| {
bitmap.remove(task.uid);
})?;
self.update_kind(wtxn, task.kind.as_kind(), |bitmap| {
bitmap.insert(task.uid);
})?;
}
assert_eq!(
old_task.enqueued_at, task.enqueued_at,
"Cannot update a task's enqueued_at time"
);
if old_task.started_at != task.started_at {
assert!(old_task.started_at.is_none(), "Cannot update a task's started_at time");
if let Some(started_at) = task.started_at {
insert_task_datetime(wtxn, self.started_at, started_at, task.uid)?;
}
}
if old_task.finished_at != task.finished_at {
assert!(old_task.finished_at.is_none(), "Cannot update a task's finished_at time");
if let Some(finished_at) = task.finished_at {
insert_task_datetime(wtxn, self.finished_at, finished_at, task.uid)?;
}
}
self.all_tasks.put(wtxn, &task.uid, task)?;
Ok(())
}
/// Returns the whole set of tasks that belongs to this batch.
pub(crate) fn tasks_in_batch(&self, rtxn: &RoTxn, batch_id: BatchId) -> Result<RoaringBitmap> {
Ok(self.batch_to_tasks_mapping.get(rtxn, &batch_id)?.unwrap_or_default())
}
/// Returns the whole set of tasks that belongs to this index.
pub(crate) fn index_tasks(&self, rtxn: &RoTxn, index: &str) -> Result<RoaringBitmap> {
Ok(self.index_tasks.get(rtxn, index)?.unwrap_or_default())
}
pub(crate) fn update_index(
&self,
wtxn: &mut RwTxn,
index: &str,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.index_tasks(wtxn, index)?;
f(&mut tasks);
if tasks.is_empty() {
self.index_tasks.delete(wtxn, index)?;
} else {
self.index_tasks.put(wtxn, index, &tasks)?;
}
Ok(())
}
/// Returns the whole set of batches that belongs to this index.
pub(crate) fn index_batches(&self, rtxn: &RoTxn, index: &str) -> Result<RoaringBitmap> {
Ok(self.batch_index_tasks.get(rtxn, index)?.unwrap_or_default())
}
pub(crate) fn update_batch_index(
&self,
wtxn: &mut RwTxn,
index: &str,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut batches = self.index_batches(wtxn, index)?;
f(&mut batches);
if batches.is_empty() {
self.batch_index_tasks.delete(wtxn, index)?;
} else {
self.batch_index_tasks.put(wtxn, index, &batches)?;
}
Ok(())
}
pub(crate) fn get_status(&self, rtxn: &RoTxn, status: Status) -> Result<RoaringBitmap> {
Ok(self.status.get(rtxn, &status)?.unwrap_or_default())
}
pub(crate) fn put_status(
&self,
wtxn: &mut RwTxn,
status: Status,
bitmap: &RoaringBitmap,
) -> Result<()> {
Ok(self.status.put(wtxn, &status, bitmap)?)
}
pub(crate) fn update_status(
&self,
wtxn: &mut RwTxn,
status: Status,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.get_status(wtxn, status)?;
f(&mut tasks);
self.put_status(wtxn, status, &tasks)?;
Ok(())
}
pub(crate) fn get_batch_status(&self, rtxn: &RoTxn, status: Status) -> Result<RoaringBitmap> {
Ok(self.batch_status.get(rtxn, &status)?.unwrap_or_default())
}
pub(crate) fn put_batch_status(
&self,
wtxn: &mut RwTxn,
status: Status,
bitmap: &RoaringBitmap,
) -> Result<()> {
Ok(self.batch_status.put(wtxn, &status, bitmap)?)
}
pub(crate) fn update_batch_status(
&self,
wtxn: &mut RwTxn,
status: Status,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.get_batch_status(wtxn, status)?;
f(&mut tasks);
self.put_batch_status(wtxn, status, &tasks)?;
Ok(())
}
pub(crate) fn get_kind(&self, rtxn: &RoTxn, kind: Kind) -> Result<RoaringBitmap> {
Ok(self.kind.get(rtxn, &kind)?.unwrap_or_default())
}
pub(crate) fn put_kind(
&self,
wtxn: &mut RwTxn,
kind: Kind,
bitmap: &RoaringBitmap,
) -> Result<()> {
Ok(self.kind.put(wtxn, &kind, bitmap)?)
}
pub(crate) fn update_kind(
&self,
wtxn: &mut RwTxn,
kind: Kind,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.get_kind(wtxn, kind)?;
f(&mut tasks);
self.put_kind(wtxn, kind, &tasks)?;
Ok(())
}
pub(crate) fn get_batch_kind(&self, rtxn: &RoTxn, kind: Kind) -> Result<RoaringBitmap> {
Ok(self.batch_kind.get(rtxn, &kind)?.unwrap_or_default())
}
pub(crate) fn put_batch_kind(
&self,
wtxn: &mut RwTxn,
kind: Kind,
bitmap: &RoaringBitmap,
) -> Result<()> {
Ok(self.batch_kind.put(wtxn, &kind, bitmap)?)
}
pub(crate) fn update_batch_kind(
&self,
wtxn: &mut RwTxn,
kind: Kind,
f: impl Fn(&mut RoaringBitmap),
) -> Result<()> {
let mut tasks = self.get_batch_kind(wtxn, kind)?;
f(&mut tasks);
self.put_batch_kind(wtxn, kind, &tasks)?;
Ok(())
}
}
pub(crate) fn insert_task_datetime(
wtxn: &mut RwTxn,
database: Database<BEI128, CboRoaringBitmapCodec>,
@ -651,11 +307,11 @@ pub fn clamp_to_page_size(size: usize) -> usize {
}
#[cfg(test)]
impl IndexScheduler {
impl crate::IndexScheduler {
/// Asserts that the index scheduler's content is internally consistent.
pub fn assert_internally_consistent(&self) {
let rtxn = self.env.read_txn().unwrap();
for task in self.all_tasks.iter(&rtxn).unwrap() {
for task in self.queue.tasks.all_tasks.iter(&rtxn).unwrap() {
let (task_id, task) = task.unwrap();
let task_index_uid = task.index_uid().map(ToOwned::to_owned);
@ -674,6 +330,7 @@ impl IndexScheduler {
assert_eq!(uid, task.uid);
if let Some(ref batch) = batch_uid {
assert!(self
.queue
.batch_to_tasks_mapping
.get(&rtxn, batch)
.unwrap()
@ -682,17 +339,26 @@ impl IndexScheduler {
}
if let Some(task_index_uid) = &task_index_uid {
assert!(self
.queue
.tasks
.index_tasks
.get(&rtxn, task_index_uid.as_str())
.unwrap()
.unwrap()
.contains(task.uid));
}
let db_enqueued_at =
self.enqueued_at.get(&rtxn, &enqueued_at.unix_timestamp_nanos()).unwrap().unwrap();
let db_enqueued_at = self
.queue
.tasks
.enqueued_at
.get(&rtxn, &enqueued_at.unix_timestamp_nanos())
.unwrap()
.unwrap();
assert!(db_enqueued_at.contains(task_id));
if let Some(started_at) = started_at {
let db_started_at = self
.queue
.tasks
.started_at
.get(&rtxn, &started_at.unix_timestamp_nanos())
.unwrap()
@ -701,6 +367,8 @@ impl IndexScheduler {
}
if let Some(finished_at) = finished_at {
let db_finished_at = self
.queue
.tasks
.finished_at
.get(&rtxn, &finished_at.unix_timestamp_nanos())
.unwrap()
@ -708,9 +376,11 @@ impl IndexScheduler {
assert!(db_finished_at.contains(task_id));
}
if let Some(canceled_by) = canceled_by {
let db_canceled_tasks = self.get_status(&rtxn, Status::Canceled).unwrap();
let db_canceled_tasks =
self.queue.tasks.get_status(&rtxn, Status::Canceled).unwrap();
assert!(db_canceled_tasks.contains(uid));
let db_canceling_task = self.get_task(&rtxn, canceled_by).unwrap().unwrap();
let db_canceling_task =
self.queue.tasks.get_task(&rtxn, canceled_by).unwrap().unwrap();
assert_eq!(db_canceling_task.status, Status::Succeeded);
match db_canceling_task.kind {
KindWithContent::TaskCancelation { query: _, tasks } => {
@ -770,7 +440,9 @@ impl IndexScheduler {
Details::IndexInfo { primary_key: pk1 } => match &kind {
KindWithContent::IndexCreation { index_uid, primary_key: pk2 }
| KindWithContent::IndexUpdate { index_uid, primary_key: pk2 } => {
self.index_tasks
self.queue
.tasks
.index_tasks
.get(&rtxn, index_uid.as_str())
.unwrap()
.unwrap()
@ -878,23 +550,24 @@ impl IndexScheduler {
}
}
assert!(self.get_status(&rtxn, status).unwrap().contains(uid));
assert!(self.get_kind(&rtxn, kind.as_kind()).unwrap().contains(uid));
assert!(self.queue.tasks.get_status(&rtxn, status).unwrap().contains(uid));
assert!(self.queue.tasks.get_kind(&rtxn, kind.as_kind()).unwrap().contains(uid));
if let KindWithContent::DocumentAdditionOrUpdate { content_file, .. } = kind {
match status {
Status::Enqueued | Status::Processing => {
assert!(self
.file_store
.queue.file_store
.all_uuids()
.unwrap()
.any(|uuid| uuid.as_ref().unwrap() == &content_file),
"Could not find uuid `{content_file}` in the file_store. Available uuids are {:?}.",
self.file_store.all_uuids().unwrap().collect::<std::result::Result<Vec<_>, file_store::Error>>().unwrap(),
self.queue.file_store.all_uuids().unwrap().collect::<std::result::Result<Vec<_>, file_store::Error>>().unwrap(),
);
}
Status::Succeeded | Status::Failed | Status::Canceled => {
assert!(self
.queue
.file_store
.all_uuids()
.unwrap()