mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-04 12:27:13 +02:00
add the batch_id to the tasks
This commit is contained in:
parent
057fcb3993
commit
6062914654
126 changed files with 755 additions and 158 deletions
|
@ -48,6 +48,7 @@ pub use features::RoFeatures;
|
|||
use file_store::FileStore;
|
||||
use flate2::bufread::GzEncoder;
|
||||
use flate2::Compression;
|
||||
use meilisearch_types::batches::{Batch, BatchId};
|
||||
use meilisearch_types::error::ResponseError;
|
||||
use meilisearch_types::features::{InstanceTogglableFeatures, RuntimeTogglableFeatures};
|
||||
use meilisearch_types::heed::byteorder::BE;
|
||||
|
@ -162,6 +163,8 @@ impl Query {
|
|||
struct ProcessingTasks {
|
||||
/// The date and time at which the indexation started.
|
||||
started_at: OffsetDateTime,
|
||||
/// The id of the batch processing
|
||||
batch_id: Option<BatchId>,
|
||||
/// The list of tasks ids that are currently running.
|
||||
processing: RoaringBitmap,
|
||||
}
|
||||
|
@ -169,17 +172,28 @@ struct ProcessingTasks {
|
|||
impl ProcessingTasks {
|
||||
/// Creates an empty `ProcessingAt` struct.
|
||||
fn new() -> ProcessingTasks {
|
||||
ProcessingTasks { started_at: OffsetDateTime::now_utc(), processing: RoaringBitmap::new() }
|
||||
ProcessingTasks {
|
||||
started_at: OffsetDateTime::now_utc(),
|
||||
batch_id: None,
|
||||
processing: RoaringBitmap::new(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Stores the currently processing tasks, and the date time at which it started.
|
||||
fn start_processing_at(&mut self, started_at: OffsetDateTime, processing: RoaringBitmap) {
|
||||
fn start_processing(
|
||||
&mut self,
|
||||
started_at: OffsetDateTime,
|
||||
batch_id: BatchId,
|
||||
processing: RoaringBitmap,
|
||||
) {
|
||||
self.started_at = started_at;
|
||||
self.batch_id = Some(batch_id);
|
||||
self.processing = processing;
|
||||
}
|
||||
|
||||
/// Set the processing tasks to an empty list
|
||||
fn stop_processing(&mut self) -> RoaringBitmap {
|
||||
self.batch_id = None;
|
||||
std::mem::take(&mut self.processing)
|
||||
}
|
||||
|
||||
|
@ -209,6 +223,7 @@ impl MustStopProcessing {
|
|||
/// Database const names for the `IndexScheduler`.
|
||||
mod db_name {
|
||||
pub const ALL_TASKS: &str = "all-tasks";
|
||||
pub const ALL_BATCHES: &str = "all-batches";
|
||||
pub const STATUS: &str = "status";
|
||||
pub const KIND: &str = "kind";
|
||||
pub const INDEX_TASKS: &str = "index-tasks";
|
||||
|
@ -216,6 +231,14 @@ mod db_name {
|
|||
pub const ENQUEUED_AT: &str = "enqueued-at";
|
||||
pub const STARTED_AT: &str = "started-at";
|
||||
pub const FINISHED_AT: &str = "finished-at";
|
||||
|
||||
pub const BATCH_STATUS: &str = "batch-status";
|
||||
pub const BATCH_KIND: &str = "batch-kind";
|
||||
pub const BATCH_INDEX_TASKS: &str = "batch-index-tasks";
|
||||
pub const BATCH_CANCELED_BY: &str = "batch-canceled_by";
|
||||
pub const BATCH_ENQUEUED_AT: &str = "batch-enqueued-at";
|
||||
pub const BATCH_STARTED_AT: &str = "batch-started-at";
|
||||
pub const BATCH_FINISHED_AT: &str = "batch-finished-at";
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
|
@ -300,6 +323,28 @@ pub struct IndexScheduler {
|
|||
// The main database, it contains all the tasks accessible by their Id.
|
||||
pub(crate) all_tasks: Database<BEU32, SerdeJson<Task>>,
|
||||
|
||||
// Contains all the batches accessible by their Id.
|
||||
pub(crate) all_batches: Database<BEU32, SerdeJson<Batch>>,
|
||||
|
||||
/// All the batches containing a task matching the selected status.
|
||||
pub(crate) batch_status: Database<SerdeBincode<Status>, RoaringBitmapCodec>,
|
||||
/// All the batches ids grouped by the kind of their task.
|
||||
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
|
||||
pub(crate) batch_started_at: Database<BEI128, CboRoaringBitmapCodec>,
|
||||
|
||||
/// Store the batches containing tasks finished at a specific date
|
||||
pub(crate) batch_finished_at: Database<BEI128, CboRoaringBitmapCodec>,
|
||||
|
||||
/// All the tasks ids grouped by their status.
|
||||
// TODO we should not be able to serialize a `Status::Processing` in this database.
|
||||
pub(crate) status: Database<SerdeBincode<Status>, RoaringBitmapCodec>,
|
||||
|
@ -388,6 +433,9 @@ impl IndexScheduler {
|
|||
processing_tasks: self.processing_tasks.clone(),
|
||||
file_store: self.file_store.clone(),
|
||||
all_tasks: self.all_tasks,
|
||||
all_batches: self.all_batches,
|
||||
|
||||
// Tasks reverse index
|
||||
status: self.status,
|
||||
kind: self.kind,
|
||||
index_tasks: self.index_tasks,
|
||||
|
@ -395,6 +443,16 @@ impl IndexScheduler {
|
|||
enqueued_at: self.enqueued_at,
|
||||
started_at: self.started_at,
|
||||
finished_at: self.finished_at,
|
||||
|
||||
// Batches reverse index
|
||||
batch_status: self.batch_status,
|
||||
batch_kind: self.batch_kind,
|
||||
batch_index_tasks: self.batch_index_tasks,
|
||||
batch_canceled_by: self.batch_canceled_by,
|
||||
batch_enqueued_at: self.batch_enqueued_at,
|
||||
batch_started_at: self.batch_started_at,
|
||||
batch_finished_at: self.batch_finished_at,
|
||||
|
||||
index_mapper: self.index_mapper.clone(),
|
||||
wake_up: self.wake_up.clone(),
|
||||
autobatching_enabled: self.autobatching_enabled,
|
||||
|
@ -454,7 +512,7 @@ impl IndexScheduler {
|
|||
|
||||
let env = unsafe {
|
||||
heed::EnvOpenOptions::new()
|
||||
.max_dbs(11)
|
||||
.max_dbs(19)
|
||||
.map_size(budget.task_db_size)
|
||||
.open(options.tasks_path)
|
||||
}?;
|
||||
|
@ -465,6 +523,7 @@ impl IndexScheduler {
|
|||
|
||||
let mut wtxn = env.write_txn()?;
|
||||
let all_tasks = env.create_database(&mut wtxn, Some(db_name::ALL_TASKS))?;
|
||||
let all_batches = env.create_database(&mut wtxn, Some(db_name::ALL_BATCHES))?;
|
||||
let status = env.create_database(&mut wtxn, Some(db_name::STATUS))?;
|
||||
let kind = env.create_database(&mut wtxn, Some(db_name::KIND))?;
|
||||
let index_tasks = env.create_database(&mut wtxn, Some(db_name::INDEX_TASKS))?;
|
||||
|
@ -472,6 +531,14 @@ impl IndexScheduler {
|
|||
let enqueued_at = env.create_database(&mut wtxn, Some(db_name::ENQUEUED_AT))?;
|
||||
let started_at = env.create_database(&mut wtxn, Some(db_name::STARTED_AT))?;
|
||||
let finished_at = env.create_database(&mut wtxn, Some(db_name::FINISHED_AT))?;
|
||||
|
||||
let batch_status = env.create_database(&mut wtxn, Some(db_name::STATUS))?;
|
||||
let batch_kind = env.create_database(&mut wtxn, Some(db_name::KIND))?;
|
||||
let batch_index_tasks = env.create_database(&mut wtxn, Some(db_name::INDEX_TASKS))?;
|
||||
let batch_canceled_by = env.create_database(&mut wtxn, Some(db_name::CANCELED_BY))?;
|
||||
let batch_enqueued_at = env.create_database(&mut wtxn, Some(db_name::ENQUEUED_AT))?;
|
||||
let batch_started_at = env.create_database(&mut wtxn, Some(db_name::STARTED_AT))?;
|
||||
let batch_finished_at = env.create_database(&mut wtxn, Some(db_name::FINISHED_AT))?;
|
||||
wtxn.commit()?;
|
||||
|
||||
// allow unreachable_code to get rids of the warning in the case of a test build.
|
||||
|
@ -480,6 +547,8 @@ impl IndexScheduler {
|
|||
processing_tasks: Arc::new(RwLock::new(ProcessingTasks::new())),
|
||||
file_store,
|
||||
all_tasks,
|
||||
all_batches,
|
||||
// Task reverse indexes
|
||||
status,
|
||||
kind,
|
||||
index_tasks,
|
||||
|
@ -487,6 +556,16 @@ impl IndexScheduler {
|
|||
enqueued_at,
|
||||
started_at,
|
||||
finished_at,
|
||||
|
||||
// Batch reverse indexes
|
||||
batch_status,
|
||||
batch_kind,
|
||||
batch_index_tasks,
|
||||
batch_canceled_by,
|
||||
batch_enqueued_at,
|
||||
batch_started_at,
|
||||
batch_finished_at,
|
||||
|
||||
index_mapper: IndexMapper::new(
|
||||
&env,
|
||||
options.indexes_path,
|
||||
|
@ -946,6 +1025,52 @@ impl IndexScheduler {
|
|||
Ok((tasks, total_tasks.len()))
|
||||
}
|
||||
|
||||
/// Return the batch ids matching the query along with the total number of batches
|
||||
/// by ignoring the from and limit parameters from the user's point of view.
|
||||
///
|
||||
/// There are two differences between an internal query and a query executed by
|
||||
/// the user.
|
||||
///
|
||||
/// 1. IndexSwap tasks are not publicly associated with any index, but they are associated
|
||||
/// with many indexes internally.
|
||||
/// 2. The user may not have the rights to access the tasks (internally) associated with all indexes.
|
||||
pub fn get_task_ids_from_authorized_indexes(
|
||||
&self,
|
||||
rtxn: &RoTxn,
|
||||
query: &Query,
|
||||
filters: &meilisearch_auth::AuthFilter,
|
||||
) -> Result<(RoaringBitmap, u64)> {
|
||||
// compute all batches matching the filter by ignoring the limits, to find the number of batches matching
|
||||
// the filter.
|
||||
// As this causes us to compute the filter twice it is slightly inefficient, but doing it this way spares
|
||||
// us from modifying the underlying implementation, and the performance remains sufficient.
|
||||
// Should this change, we would modify `get_batch_ids` to directly return the number of matching batches.
|
||||
let total_batches = self.get_batch_ids(rtxn, &query.clone().without_limits())?;
|
||||
let mut batches = self.get_batch_ids(rtxn, query)?;
|
||||
|
||||
// If the query contains a list of index uid or there is a finite list of authorized indexes,
|
||||
// then we must exclude all the kinds that aren't associated to one and only one index.
|
||||
if query.index_uids.is_some() || !filters.all_indexes_authorized() {
|
||||
for kind in enum_iterator::all::<Kind>().filter(|kind| !kind.related_to_one_index()) {
|
||||
batches -= self.get_kind(rtxn, kind)?;
|
||||
}
|
||||
}
|
||||
|
||||
// Any task that is internally associated with a non-authorized index
|
||||
// must be discarded.
|
||||
if !filters.all_indexes_authorized() {
|
||||
let all_indexes_iter = self.index_tasks.iter(rtxn)?;
|
||||
for result in all_indexes_iter {
|
||||
let (index, index_tasks) = result?;
|
||||
if !filters.is_index_authorized(index) {
|
||||
tasks -= index_tasks;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok((tasks, total_tasks.len()))
|
||||
}
|
||||
|
||||
/// Return the tasks matching the query from the user's point of view along
|
||||
/// with the total number of tasks matching the query, ignoring from and limit.
|
||||
///
|
||||
|
@ -971,7 +1096,7 @@ impl IndexScheduler {
|
|||
let tasks =
|
||||
self.get_existing_tasks(&rtxn, tasks.take(query.limit.unwrap_or(u32::MAX) as usize))?;
|
||||
|
||||
let ProcessingTasks { started_at, processing, .. } =
|
||||
let ProcessingTasks { started_at, batch_id, processing } =
|
||||
self.processing_tasks.read().map_err(|_| Error::CorruptedTaskQueue)?.clone();
|
||||
|
||||
let ret = tasks.into_iter();
|
||||
|
@ -981,7 +1106,60 @@ impl IndexScheduler {
|
|||
Ok((
|
||||
ret.map(|task| {
|
||||
if processing.contains(task.uid) {
|
||||
Task { status: Status::Processing, started_at: Some(started_at), ..task }
|
||||
Task {
|
||||
status: Status::Processing,
|
||||
batch_uid: batch_id,
|
||||
started_at: Some(started_at),
|
||||
..task
|
||||
}
|
||||
} else {
|
||||
task
|
||||
}
|
||||
})
|
||||
.collect(),
|
||||
total,
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
/// Return the batches matching the query from the user's point of view along
|
||||
/// with the total number of batches matching the query, ignoring from and limit.
|
||||
///
|
||||
/// There are two differences between an internal query and a query executed by
|
||||
/// the user.
|
||||
///
|
||||
/// 1. IndexSwap tasks are not publicly associated with any index, but they are associated
|
||||
/// with many indexes internally.
|
||||
/// 2. The user may not have the rights to access the tasks (internally) associated with all indexes.
|
||||
pub fn get_batches_from_authorized_indexes(
|
||||
&self,
|
||||
query: Query,
|
||||
filters: &meilisearch_auth::AuthFilter,
|
||||
) -> Result<(Vec<Batch>, u64)> {
|
||||
let rtxn = self.env.read_txn()?;
|
||||
|
||||
let (tasks, total) = self.get_batch_ids_from_authorized_indexes(&rtxn, &query, filters)?;
|
||||
let tasks = self.get_existing_batches(
|
||||
&rtxn,
|
||||
tasks.into_iter().rev().take(query.limit.unwrap_or(u32::MAX) as usize),
|
||||
)?;
|
||||
|
||||
let ProcessingTasks { started_at, batch_id, processing } =
|
||||
self.processing_tasks.read().map_err(|_| Error::CorruptedTaskQueue)?.clone();
|
||||
|
||||
let ret = tasks.into_iter();
|
||||
if processing.is_empty() {
|
||||
Ok((ret.collect(), total))
|
||||
} else {
|
||||
Ok((
|
||||
ret.map(|task| {
|
||||
if processing.contains(task.uid) {
|
||||
Task {
|
||||
status: Status::Processing,
|
||||
batch_uid: batch_id,
|
||||
started_at: Some(started_at),
|
||||
..task
|
||||
}
|
||||
} else {
|
||||
task
|
||||
}
|
||||
|
@ -1020,6 +1198,8 @@ impl IndexScheduler {
|
|||
|
||||
let mut task = Task {
|
||||
uid: task_id.unwrap_or(next_task_id),
|
||||
// The batch is defined once we starts processing the task
|
||||
batch_uid: None,
|
||||
enqueued_at: OffsetDateTime::now_utc(),
|
||||
started_at: None,
|
||||
finished_at: None,
|
||||
|
@ -1155,7 +1335,7 @@ impl IndexScheduler {
|
|||
}
|
||||
|
||||
let rtxn = self.env.read_txn().map_err(Error::HeedTransaction)?;
|
||||
let batch =
|
||||
let (batch, batch_id) =
|
||||
match self.create_next_batch(&rtxn).map_err(|e| Error::CreateBatch(Box::new(e)))? {
|
||||
Some(batch) => batch,
|
||||
None => return Ok(TickOutcome::WaitForSignal),
|
||||
|
@ -1170,7 +1350,7 @@ 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.write().unwrap().start_processing_at(started_at, ids.clone());
|
||||
self.processing_tasks.write().unwrap().start_processing(started_at, batch_id, ids.clone());
|
||||
|
||||
#[cfg(test)]
|
||||
self.breakpoint(Breakpoint::BatchCreated);
|
||||
|
@ -1268,7 +1448,8 @@ impl IndexScheduler {
|
|||
let mut task = self
|
||||
.get_task(&wtxn, id)
|
||||
.map_err(|e| Error::TaskDatabaseUpdate(Box::new(e)))?
|
||||
.ok_or(Error::CorruptedTaskQueue)?;
|
||||
.ok_or(Error::CorruptedTaskQueue)?
|
||||
.with_batch_id(batch_id);
|
||||
task.started_at = Some(started_at);
|
||||
task.finished_at = Some(finished_at);
|
||||
task.status = Status::Failed;
|
||||
|
@ -1286,6 +1467,12 @@ impl IndexScheduler {
|
|||
}
|
||||
}
|
||||
|
||||
self.all_batches.put(
|
||||
&mut wtxn,
|
||||
&batch_id,
|
||||
&Batch { uid: batch_id, started_at, finished_at: Some(finished_at) },
|
||||
)?;
|
||||
|
||||
let processed = self.processing_tasks.write().unwrap().stop_processing();
|
||||
|
||||
#[cfg(test)]
|
||||
|
@ -1600,6 +1787,7 @@ impl<'a> Dump<'a> {
|
|||
|
||||
let task = Task {
|
||||
uid: task.uid,
|
||||
batch_uid: task.batch_uid,
|
||||
enqueued_at: task.enqueued_at,
|
||||
started_at: task.started_at,
|
||||
finished_at: task.finished_at,
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue