mirror of
https://github.com/meilisearch/meilisearch.git
synced 2025-07-26 16:21:07 +00:00
Split the index-scheduler in ~500 loc modules
This commit is contained in:
@ -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()
|
||||
|
Reference in New Issue
Block a user