5351: Bring back v1.13.0 changes into main r=irevoire a=Kerollmops

This PR brings back the changes made in v1.13 into the main branch.

Co-authored-by: ManyTheFish <many@meilisearch.com>
Co-authored-by: Kerollmops <clement@meilisearch.com>
Co-authored-by: Louis Dureuil <louis@meilisearch.com>
Co-authored-by: Clémentine <clementine@meilisearch.com>
Co-authored-by: meili-bors[bot] <89034592+meili-bors[bot]@users.noreply.github.com>
Co-authored-by: Tamo <tamo@meilisearch.com>
Co-authored-by: Clément Renault <clement@meilisearch.com>
This commit is contained in:
meili-bors[bot] 2025-02-18 08:05:02 +00:00 committed by GitHub
commit 0f1aeb8eaa
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
101 changed files with 8351 additions and 1518 deletions

View file

@ -2,6 +2,7 @@ use std::collections::HashMap;
use std::io;
use dump::{KindDump, TaskDump, UpdateFile};
use meilisearch_types::batches::{Batch, BatchId};
use meilisearch_types::heed::RwTxn;
use meilisearch_types::milli;
use meilisearch_types::tasks::{Kind, KindWithContent, Status, Task};
@ -14,9 +15,15 @@ pub struct Dump<'a> {
index_scheduler: &'a IndexScheduler,
wtxn: RwTxn<'a>,
batch_to_task_mapping: HashMap<BatchId, RoaringBitmap>,
indexes: HashMap<String, RoaringBitmap>,
statuses: HashMap<Status, RoaringBitmap>,
kinds: HashMap<Kind, RoaringBitmap>,
batch_indexes: HashMap<String, RoaringBitmap>,
batch_statuses: HashMap<Status, RoaringBitmap>,
batch_kinds: HashMap<Kind, RoaringBitmap>,
}
impl<'a> Dump<'a> {
@ -27,12 +34,72 @@ impl<'a> Dump<'a> {
Ok(Dump {
index_scheduler,
wtxn,
batch_to_task_mapping: HashMap::new(),
indexes: HashMap::new(),
statuses: HashMap::new(),
kinds: HashMap::new(),
batch_indexes: HashMap::new(),
batch_statuses: HashMap::new(),
batch_kinds: HashMap::new(),
})
}
/// Register a new batch coming from a dump in the scheduler.
/// By taking a mutable ref we're pretty sure no one will ever import a dump while actix is running.
pub fn register_dumped_batch(&mut self, batch: Batch) -> Result<()> {
self.index_scheduler.queue.batches.all_batches.put(&mut self.wtxn, &batch.uid, &batch)?;
if let Some(enqueued_at) = batch.enqueued_at {
utils::insert_task_datetime(
&mut self.wtxn,
self.index_scheduler.queue.batches.enqueued_at,
enqueued_at.earliest,
batch.uid,
)?;
utils::insert_task_datetime(
&mut self.wtxn,
self.index_scheduler.queue.batches.enqueued_at,
enqueued_at.oldest,
batch.uid,
)?;
}
utils::insert_task_datetime(
&mut self.wtxn,
self.index_scheduler.queue.batches.started_at,
batch.started_at,
batch.uid,
)?;
if let Some(finished_at) = batch.finished_at {
utils::insert_task_datetime(
&mut self.wtxn,
self.index_scheduler.queue.batches.finished_at,
finished_at,
batch.uid,
)?;
}
for index in batch.stats.index_uids.keys() {
match self.batch_indexes.get_mut(index) {
Some(bitmap) => {
bitmap.insert(batch.uid);
}
None => {
let mut bitmap = RoaringBitmap::new();
bitmap.insert(batch.uid);
self.batch_indexes.insert(index.to_string(), bitmap);
}
};
}
for status in batch.stats.status.keys() {
self.batch_statuses.entry(*status).or_default().insert(batch.uid);
}
for kind in batch.stats.types.keys() {
self.batch_kinds.entry(*kind).or_default().insert(batch.uid);
}
Ok(())
}
/// Register a new task coming from a dump in the scheduler.
/// By taking a mutable ref we're pretty sure no one will ever import a dump while actix is running.
pub fn register_dumped_task(
@ -149,6 +216,9 @@ impl<'a> Dump<'a> {
};
self.index_scheduler.queue.tasks.all_tasks.put(&mut self.wtxn, &task.uid, &task)?;
if let Some(batch_id) = task.batch_uid {
self.batch_to_task_mapping.entry(batch_id).or_default().insert(task.uid);
}
for index in task.indexes() {
match self.indexes.get_mut(index) {
@ -198,6 +268,14 @@ impl<'a> Dump<'a> {
/// Commit all the changes and exit the importing dump state
pub fn finish(mut self) -> Result<()> {
for (batch_id, task_ids) in self.batch_to_task_mapping {
self.index_scheduler.queue.batch_to_tasks_mapping.put(
&mut self.wtxn,
&batch_id,
&task_ids,
)?;
}
for (index, bitmap) in self.indexes {
self.index_scheduler.queue.tasks.index_tasks.put(&mut self.wtxn, &index, &bitmap)?;
}
@ -208,6 +286,16 @@ impl<'a> Dump<'a> {
self.index_scheduler.queue.tasks.put_kind(&mut self.wtxn, kind, &bitmap)?;
}
for (index, bitmap) in self.batch_indexes {
self.index_scheduler.queue.batches.index_tasks.put(&mut self.wtxn, &index, &bitmap)?;
}
for (status, bitmap) in self.batch_statuses {
self.index_scheduler.queue.batches.put_status(&mut self.wtxn, status, &bitmap)?;
}
for (kind, bitmap) in self.batch_kinds {
self.index_scheduler.queue.batches.put_kind(&mut self.wtxn, kind, &bitmap)?;
}
self.wtxn.commit()?;
self.index_scheduler.scheduler.wake_up.signal();

View file

@ -109,6 +109,8 @@ pub enum Error {
InvalidIndexUid { index_uid: String },
#[error("Task `{0}` not found.")]
TaskNotFound(TaskId),
#[error("Task `{0}` does not contain any documents. Only `documentAdditionOrUpdate` tasks with the statuses `enqueued` or `processing` contain documents")]
TaskFileNotFound(TaskId),
#[error("Batch `{0}` not found.")]
BatchNotFound(BatchId),
#[error("Query parameters to filter the tasks to delete are missing. Available query parameters are: `uids`, `indexUids`, `statuses`, `types`, `canceledBy`, `beforeEnqueuedAt`, `afterEnqueuedAt`, `beforeStartedAt`, `afterStartedAt`, `beforeFinishedAt`, `afterFinishedAt`.")]
@ -127,8 +129,8 @@ pub enum Error {
_ => format!("{error}")
})]
Milli { error: milli::Error, index_uid: Option<String> },
#[error("An unexpected crash occurred when processing the task.")]
ProcessBatchPanicked,
#[error("An unexpected crash occurred when processing the task: {0}")]
ProcessBatchPanicked(String),
#[error(transparent)]
FileStore(#[from] file_store::Error),
#[error(transparent)]
@ -189,6 +191,7 @@ impl Error {
| Error::InvalidTaskCanceledBy { .. }
| Error::InvalidIndexUid { .. }
| Error::TaskNotFound(_)
| Error::TaskFileNotFound(_)
| Error::BatchNotFound(_)
| Error::TaskDeletionWithEmptyQuery
| Error::TaskCancelationWithEmptyQuery
@ -196,7 +199,7 @@ impl Error {
| Error::Dump(_)
| Error::Heed(_)
| Error::Milli { .. }
| Error::ProcessBatchPanicked
| Error::ProcessBatchPanicked(_)
| Error::FileStore(_)
| Error::IoError(_)
| Error::Persist(_)
@ -250,6 +253,7 @@ impl ErrorCode for Error {
Error::InvalidTaskCanceledBy { .. } => Code::InvalidTaskCanceledBy,
Error::InvalidIndexUid { .. } => Code::InvalidIndexUid,
Error::TaskNotFound(_) => Code::TaskNotFound,
Error::TaskFileNotFound(_) => Code::TaskFileNotFound,
Error::BatchNotFound(_) => Code::BatchNotFound,
Error::TaskDeletionWithEmptyQuery => Code::MissingTaskFilters,
Error::TaskCancelationWithEmptyQuery => Code::MissingTaskFilters,
@ -257,7 +261,7 @@ impl ErrorCode for Error {
Error::NoSpaceLeftInTaskQueue => Code::NoSpaceLeftOnDevice,
Error::Dump(e) => e.error_code(),
Error::Milli { error, .. } => error.error_code(),
Error::ProcessBatchPanicked => Code::Internal,
Error::ProcessBatchPanicked(_) => Code::Internal,
Error::Heed(e) => e.error_code(),
Error::HeedTransaction(e) => e.error_code(),
Error::FileStore(e) => e.error_code(),

View file

@ -1,6 +1,6 @@
use std::sync::{Arc, RwLock};
use meilisearch_types::features::{InstanceTogglableFeatures, RuntimeTogglableFeatures};
use meilisearch_types::features::{InstanceTogglableFeatures, Network, RuntimeTogglableFeatures};
use meilisearch_types::heed::types::{SerdeJson, Str};
use meilisearch_types::heed::{Database, Env, RwTxn};
@ -14,10 +14,16 @@ mod db_name {
pub const EXPERIMENTAL_FEATURES: &str = "experimental-features";
}
mod db_keys {
pub const EXPERIMENTAL_FEATURES: &str = "experimental-features";
pub const NETWORK: &str = "network";
}
#[derive(Clone)]
pub(crate) struct FeatureData {
persisted: Database<Str, SerdeJson<RuntimeTogglableFeatures>>,
runtime: Arc<RwLock<RuntimeTogglableFeatures>>,
network: Arc<RwLock<Network>>,
}
#[derive(Debug, Clone, Copy)]
@ -86,6 +92,32 @@ impl RoFeatures {
.into())
}
}
pub fn check_network(&self, disabled_action: &'static str) -> Result<()> {
if self.runtime.network {
Ok(())
} else {
Err(FeatureNotEnabledError {
disabled_action,
feature: "network",
issue_link: "https://github.com/orgs/meilisearch/discussions/805",
}
.into())
}
}
pub fn check_get_task_documents_route(&self) -> Result<()> {
if self.runtime.get_task_documents_route {
Ok(())
} else {
Err(FeatureNotEnabledError {
disabled_action: "Getting the documents of an enqueued task",
feature: "get task documents route",
issue_link: "https://github.com/orgs/meilisearch/discussions/808",
}
.into())
}
}
}
impl FeatureData {
@ -102,7 +134,7 @@ impl FeatureData {
env.create_database(wtxn, Some(db_name::EXPERIMENTAL_FEATURES))?;
let persisted_features: RuntimeTogglableFeatures =
runtime_features_db.get(wtxn, db_name::EXPERIMENTAL_FEATURES)?.unwrap_or_default();
runtime_features_db.get(wtxn, db_keys::EXPERIMENTAL_FEATURES)?.unwrap_or_default();
let InstanceTogglableFeatures { metrics, logs_route, contains_filter } = instance_features;
let runtime = Arc::new(RwLock::new(RuntimeTogglableFeatures {
metrics: metrics || persisted_features.metrics,
@ -111,7 +143,14 @@ impl FeatureData {
..persisted_features
}));
Ok(Self { persisted: runtime_features_db, runtime })
let network_db = runtime_features_db.remap_data_type::<SerdeJson<Network>>();
let network: Network = network_db.get(wtxn, db_keys::NETWORK)?.unwrap_or_default();
Ok(Self {
persisted: runtime_features_db,
runtime,
network: Arc::new(RwLock::new(network)),
})
}
pub fn put_runtime_features(
@ -119,7 +158,7 @@ impl FeatureData {
mut wtxn: RwTxn,
features: RuntimeTogglableFeatures,
) -> Result<()> {
self.persisted.put(&mut wtxn, db_name::EXPERIMENTAL_FEATURES, &features)?;
self.persisted.put(&mut wtxn, db_keys::EXPERIMENTAL_FEATURES, &features)?;
wtxn.commit()?;
// safe to unwrap, the lock will only fail if:
@ -140,4 +179,21 @@ impl FeatureData {
pub fn features(&self) -> RoFeatures {
RoFeatures::new(self)
}
pub fn put_network(&self, mut wtxn: RwTxn, new_network: Network) -> Result<()> {
self.persisted.remap_data_type::<SerdeJson<Network>>().put(
&mut wtxn,
db_keys::NETWORK,
&new_network,
)?;
wtxn.commit()?;
let mut network = self.network.write().unwrap();
*network = new_network;
Ok(())
}
pub fn network(&self) -> Network {
Network::clone(&*self.network.read().unwrap())
}
}

View file

@ -1,5 +1,7 @@
use std::collections::BTreeMap;
use std::env::VarError;
use std::path::Path;
use std::str::FromStr;
use std::time::Duration;
use meilisearch_types::heed::{EnvClosingEvent, EnvFlags, EnvOpenOptions};
@ -304,7 +306,18 @@ fn create_or_open_index(
) -> Result<Index> {
let mut options = EnvOpenOptions::new();
options.map_size(clamp_to_page_size(map_size));
options.max_readers(1024);
// You can find more details about this experimental
// environment variable on the following GitHub discussion:
// <https://github.com/orgs/meilisearch/discussions/806>
let max_readers = match std::env::var("MEILI_EXPERIMENTAL_INDEX_MAX_READERS") {
Ok(value) => u32::from_str(&value).unwrap(),
Err(VarError::NotPresent) => 1024,
Err(VarError::NotUnicode(value)) => panic!(
"Invalid unicode for the `MEILI_EXPERIMENTAL_INDEX_MAX_READERS` env var: {value:?}"
),
};
options.max_readers(max_readers);
if enable_mdb_writemap {
unsafe { options.flags(EnvFlags::WRITE_MAP) };
}

View file

@ -106,6 +106,12 @@ pub struct IndexStats {
/// are not returned to the disk after a deletion, this number is typically larger than
/// `used_database_size` that only includes the size of the used pages.
pub database_size: u64,
/// Number of embeddings in the index.
/// Option: retrocompatible with the stats of the pre-v1.13.0 versions of meilisearch
pub number_of_embeddings: Option<u64>,
/// Number of embedded documents in the index.
/// Option: retrocompatible with the stats of the pre-v1.13.0 versions of meilisearch
pub number_of_embedded_documents: Option<u64>,
/// Size taken by the used pages of the index' DB, in bytes.
///
/// As the DB backend does not return to the disk the pages that are not currently used by the DB,
@ -130,8 +136,11 @@ impl IndexStats {
///
/// - rtxn: a RO transaction for the index, obtained from `Index::read_txn()`.
pub fn new(index: &Index, rtxn: &RoTxn) -> milli::Result<Self> {
let arroy_stats = index.arroy_stats(rtxn)?;
Ok(IndexStats {
number_of_documents: index.number_of_documents(rtxn)?,
number_of_embeddings: Some(arroy_stats.number_of_embeddings),
number_of_embedded_documents: Some(arroy_stats.documents.len()),
database_size: index.on_disk_size()?,
used_database_size: index.used_size()?,
primary_key: index.primary_key(rtxn)?.map(|s| s.to_string()),

View file

@ -1,7 +1,7 @@
use std::collections::BTreeSet;
use std::fmt::Write;
use meilisearch_types::batches::Batch;
use meilisearch_types::batches::{Batch, BatchEnqueuedAt};
use meilisearch_types::heed::types::{SerdeBincode, SerdeJson, Str};
use meilisearch_types::heed::{Database, RoTxn};
use meilisearch_types::milli::{CboRoaringBitmapCodec, RoaringBitmapCodec, BEU32};
@ -341,10 +341,14 @@ 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, progress: _ } = batch;
let Batch { uid, details, stats, started_at, finished_at, progress: _, enqueued_at } = batch;
if let Some(finished_at) = finished_at {
assert!(finished_at > started_at);
}
let BatchEnqueuedAt { earliest, oldest } = enqueued_at.unwrap();
assert!(*started_at > earliest);
assert!(earliest >= oldest);
snap.push('{');
snap.push_str(&format!("uid: {uid}, "));
snap.push_str(&format!("details: {}, ", serde_json::to_string(details).unwrap()));

View file

@ -33,7 +33,7 @@ mod test_utils;
pub mod upgrade;
mod utils;
pub mod uuid_codec;
mod versioning;
pub mod versioning;
pub type Result<T, E = Error> = std::result::Result<T, E>;
pub type TaskId = u32;
@ -51,7 +51,7 @@ pub use features::RoFeatures;
use flate2::bufread::GzEncoder;
use flate2::Compression;
use meilisearch_types::batches::Batch;
use meilisearch_types::features::{InstanceTogglableFeatures, RuntimeTogglableFeatures};
use meilisearch_types::features::{InstanceTogglableFeatures, Network, RuntimeTogglableFeatures};
use meilisearch_types::heed::byteorder::BE;
use meilisearch_types::heed::types::I128;
use meilisearch_types::heed::{self, Env, RoTxn};
@ -770,7 +770,16 @@ impl IndexScheduler {
Ok(())
}
// TODO: consider using a type alias or a struct embedder/template
pub fn put_network(&self, network: Network) -> Result<()> {
let wtxn = self.env.write_txn().map_err(Error::HeedTransaction)?;
self.features.put_network(wtxn, network)?;
Ok(())
}
pub fn network(&self) -> Network {
self.features.network()
}
pub fn embedders(
&self,
index_uid: String,

View file

@ -96,6 +96,7 @@ make_enum_progress! {
StartTheDumpCreation,
DumpTheApiKeys,
DumpTheTasks,
DumpTheBatches,
DumpTheIndexes,
DumpTheExperimentalFeatures,
CompressTheDump,

View file

@ -12,8 +12,8 @@ use time::OffsetDateTime;
use super::{Query, Queue};
use crate::processing::ProcessingTasks;
use crate::utils::{
insert_task_datetime, keep_ids_within_datetimes, map_bound, remove_task_datetime,
ProcessingBatch,
insert_task_datetime, keep_ids_within_datetimes, map_bound,
remove_n_tasks_datetime_earlier_than, remove_task_datetime, ProcessingBatch,
};
use crate::{Error, Result, BEI128};
@ -181,6 +181,7 @@ impl BatchQueue {
stats: batch.stats,
started_at: batch.started_at,
finished_at: batch.finished_at,
enqueued_at: batch.enqueued_at,
},
)?;
@ -234,34 +235,25 @@ impl BatchQueue {
// What we know, though, is that the task date is from before the enqueued_at, and max two timestamps have been written
// to the DB per batches.
if let Some(ref old_batch) = old_batch {
let started_at = old_batch.started_at.unix_timestamp_nanos();
// We have either one or two enqueued at to remove
let mut exit = old_batch.stats.total_nb_tasks.clamp(0, 2);
let mut iterator = self.enqueued_at.rev_iter_mut(wtxn)?;
while let Some(entry) = iterator.next() {
let (key, mut value) = entry?;
if key > started_at {
continue;
}
if value.remove(old_batch.uid) {
exit = exit.saturating_sub(1);
// Safe because the key and value are owned
unsafe {
iterator.put_current(&key, &value)?;
}
if exit == 0 {
break;
}
}
if let Some(enqueued_at) = old_batch.enqueued_at {
remove_task_datetime(wtxn, self.enqueued_at, enqueued_at.earliest, old_batch.uid)?;
remove_task_datetime(wtxn, self.enqueued_at, enqueued_at.oldest, old_batch.uid)?;
} else {
// If we don't have the enqueued at in the batch it means the database comes from the v1.12
// and we still need to find the date by scrolling the database
remove_n_tasks_datetime_earlier_than(
wtxn,
self.enqueued_at,
old_batch.started_at,
old_batch.stats.total_nb_tasks.clamp(1, 2) as usize,
old_batch.uid,
)?;
}
}
if let Some(enqueued_at) = batch.oldest_enqueued_at {
insert_task_datetime(wtxn, self.enqueued_at, enqueued_at, batch.uid)?;
}
if let Some(enqueued_at) = batch.earliest_enqueued_at {
insert_task_datetime(wtxn, self.enqueued_at, enqueued_at, batch.uid)?;
}
// A finished batch MUST contains at least one task and have an enqueued_at
let enqueued_at = batch.enqueued_at.as_ref().unwrap();
insert_task_datetime(wtxn, self.enqueued_at, enqueued_at.earliest, batch.uid)?;
insert_task_datetime(wtxn, self.enqueued_at, enqueued_at.oldest, batch.uid)?;
// Update the started at and finished at
if let Some(ref old_batch) = old_batch {

View file

@ -102,30 +102,33 @@ fn query_batches_simple() {
.unwrap();
assert_eq!(batches.len(), 1);
batches[0].started_at = OffsetDateTime::UNIX_EPOCH;
assert!(batches[0].enqueued_at.is_some());
batches[0].enqueued_at = None;
// Insta cannot snapshot our batches because the batch stats contains an enum as key: https://github.com/mitsuhiko/insta/issues/689
let batch = serde_json::to_string_pretty(&batches[0]).unwrap();
snapshot!(batch, @r#"
{
"uid": 0,
"details": {
"primaryKey": "mouse"
},
"stats": {
"totalNbTasks": 1,
"status": {
"processing": 1
},
"types": {
"indexCreation": 1
},
"indexUids": {
"catto": 1
}
},
"startedAt": "1970-01-01T00:00:00Z",
"finishedAt": null
{
"uid": 0,
"details": {
"primaryKey": "mouse"
},
"stats": {
"totalNbTasks": 1,
"status": {
"processing": 1
},
"types": {
"indexCreation": 1
},
"indexUids": {
"catto": 1
}
"#);
},
"startedAt": "1970-01-01T00:00:00Z",
"finishedAt": null,
"enqueuedAt": null
}
"#);
let query = Query { statuses: Some(vec![Status::Enqueued]), ..Default::default() };
let (batches, _) = index_scheduler

View file

@ -8,6 +8,7 @@ mod tasks_test;
mod test;
use std::collections::BTreeMap;
use std::fs::File as StdFile;
use std::time::Duration;
use file_store::FileStore;
@ -216,6 +217,11 @@ impl Queue {
}
}
/// Open and returns the task's content File.
pub fn update_file(&self, uuid: Uuid) -> file_store::Result<StdFile> {
self.file_store.get_update(uuid)
}
/// Delete a file from the index scheduler.
///
/// Counterpart to the [`create_update_file`](IndexScheduler::create_update_file) method.

View file

@ -326,7 +326,7 @@ fn test_auto_deletion_of_tasks() {
fn test_task_queue_is_full() {
let (index_scheduler, mut handle) = IndexScheduler::test_with_custom_config(vec![], |config| {
// that's the minimum map size possible
config.task_db_size = 1048576;
config.task_db_size = 1048576 * 3;
None
});

View file

@ -166,13 +166,41 @@ impl IndexScheduler {
let processing_batch = &mut processing_batch;
let progress = progress.clone();
std::thread::scope(|s| {
let p = progress.clone();
let handle = std::thread::Builder::new()
.name(String::from("batch-operation"))
.spawn_scoped(s, move || {
cloned_index_scheduler.process_batch(batch, processing_batch, progress)
cloned_index_scheduler.process_batch(batch, processing_batch, p)
})
.unwrap();
handle.join().unwrap_or(Err(Error::ProcessBatchPanicked))
match handle.join() {
Ok(ret) => {
if ret.is_err() {
if let Ok(progress_view) =
serde_json::to_string(&progress.as_progress_view())
{
tracing::warn!("Batch failed while doing: {progress_view}")
}
}
ret
}
Err(panic) => {
if let Ok(progress_view) =
serde_json::to_string(&progress.as_progress_view())
{
tracing::warn!("Batch failed while doing: {progress_view}")
}
let msg = match panic.downcast_ref::<&'static str>() {
Some(s) => *s,
None => match panic.downcast_ref::<String>() {
Some(s) => &s[..],
None => "Box<dyn Any>",
},
};
Err(Error::ProcessBatchPanicked(msg.to_string()))
}
}
})
};

View file

@ -2,7 +2,7 @@ use std::collections::{BTreeSet, HashMap, HashSet};
use std::panic::{catch_unwind, AssertUnwindSafe};
use std::sync::atomic::Ordering;
use meilisearch_types::batches::BatchId;
use meilisearch_types::batches::{BatchEnqueuedAt, BatchId};
use meilisearch_types::heed::{RoTxn, RwTxn};
use meilisearch_types::milli::progress::{Progress, VariableNameStep};
use meilisearch_types::milli::{self};
@ -16,7 +16,10 @@ use crate::processing::{
InnerSwappingTwoIndexes, SwappingTheIndexes, TaskCancelationProgress, TaskDeletionProgress,
UpdateIndexProgress,
};
use crate::utils::{self, swap_index_uid_in_task, ProcessingBatch};
use crate::utils::{
self, remove_n_tasks_datetime_earlier_than, remove_task_datetime, swap_index_uid_in_task,
ProcessingBatch,
};
use crate::{Error, IndexScheduler, Result, TaskId};
impl IndexScheduler {
@ -323,8 +326,17 @@ impl IndexScheduler {
match ret {
Ok(Ok(())) => (),
Ok(Err(e)) => return Err(Error::DatabaseUpgrade(Box::new(e))),
Err(_e) => {
return Err(Error::DatabaseUpgrade(Box::new(Error::ProcessBatchPanicked)));
Err(e) => {
let msg = match e.downcast_ref::<&'static str>() {
Some(s) => *s,
None => match e.downcast_ref::<String>() {
Some(s) => &s[..],
None => "Box<dyn Any>",
},
};
return Err(Error::DatabaseUpgrade(Box::new(Error::ProcessBatchPanicked(
msg.to_string(),
))));
}
}
@ -418,7 +430,6 @@ impl IndexScheduler {
to_delete_tasks -= &enqueued_tasks;
// 2. We now have a list of tasks to delete, delete them
let mut affected_indexes = HashSet::new();
let mut affected_statuses = HashSet::new();
let mut affected_kinds = HashSet::new();
@ -515,9 +526,51 @@ impl IndexScheduler {
tasks -= &to_delete_tasks;
// We must remove the batch entirely
if tasks.is_empty() {
self.queue.batches.all_batches.delete(wtxn, &batch_id)?;
self.queue.batch_to_tasks_mapping.delete(wtxn, &batch_id)?;
if let Some(batch) = self.queue.batches.get_batch(wtxn, batch_id)? {
if let Some(BatchEnqueuedAt { earliest, oldest }) = batch.enqueued_at {
remove_task_datetime(
wtxn,
self.queue.batches.enqueued_at,
earliest,
batch_id,
)?;
remove_task_datetime(
wtxn,
self.queue.batches.enqueued_at,
oldest,
batch_id,
)?;
} else {
// If we don't have the enqueued at in the batch it means the database comes from the v1.12
// and we still need to find the date by scrolling the database
remove_n_tasks_datetime_earlier_than(
wtxn,
self.queue.batches.enqueued_at,
batch.started_at,
batch.stats.total_nb_tasks.clamp(1, 2) as usize,
batch_id,
)?;
}
remove_task_datetime(
wtxn,
self.queue.batches.started_at,
batch.started_at,
batch_id,
)?;
if let Some(finished_at) = batch.finished_at {
remove_task_datetime(
wtxn,
self.queue.batches.finished_at,
finished_at,
batch_id,
)?;
}
self.queue.batches.all_batches.delete(wtxn, &batch_id)?;
self.queue.batch_to_tasks_mapping.delete(wtxn, &batch_id)?;
}
}
// Anyway, we must remove the batch from all its reverse indexes.
// The only way to do that is to check

View file

@ -1,3 +1,4 @@
use std::collections::BTreeMap;
use std::fs::File;
use std::io::BufWriter;
use std::sync::atomic::Ordering;
@ -11,7 +12,9 @@ use meilisearch_types::tasks::{Details, KindWithContent, Status, Task};
use time::macros::format_description;
use time::OffsetDateTime;
use crate::processing::{AtomicDocumentStep, AtomicTaskStep, DumpCreationProgress};
use crate::processing::{
AtomicBatchStep, AtomicDocumentStep, AtomicTaskStep, DumpCreationProgress,
};
use crate::{Error, IndexScheduler, Result};
impl IndexScheduler {
@ -102,7 +105,40 @@ impl IndexScheduler {
}
dump_tasks.flush()?;
// 3. Dump the indexes
// 3. dump the batches
progress.update_progress(DumpCreationProgress::DumpTheBatches);
let mut dump_batches = dump.create_batches_queue()?;
let (atomic_batch_progress, update_batch_progress) =
AtomicBatchStep::new(self.queue.batches.all_batches.len(&rtxn)? as u32);
progress.update_progress(update_batch_progress);
for ret in self.queue.batches.all_batches.iter(&rtxn)? {
if self.scheduler.must_stop_processing.get() {
return Err(Error::AbortedTask);
}
let (_, mut b) = ret?;
// In the case we're dumping ourselves we want to be marked as finished
// to not loop over ourselves indefinitely.
if b.uid == task.uid {
let finished_at = OffsetDateTime::now_utc();
// We're going to fake the date because we don't know if everything is going to go well.
// But we need to dump the task as finished and successful.
// If something fail everything will be set appropriately in the end.
let mut statuses = BTreeMap::new();
statuses.insert(Status::Succeeded, b.stats.total_nb_tasks);
b.stats.status = statuses;
b.finished_at = Some(finished_at);
}
dump_batches.push_batch(&b)?;
atomic_batch_progress.fetch_add(1, Ordering::Relaxed);
}
dump_batches.flush()?;
// 4. Dump the indexes
progress.update_progress(DumpCreationProgress::DumpTheIndexes);
let nb_indexes = self.index_mapper.index_mapping.len(&rtxn)? as u32;
let mut count = 0;
@ -142,7 +178,7 @@ impl IndexScheduler {
let documents = index
.all_documents(&rtxn)
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?;
// 3.1. Dump the documents
// 4.1. Dump the documents
for ret in documents {
if self.scheduler.must_stop_processing.get() {
return Err(Error::AbortedTask);
@ -204,7 +240,7 @@ impl IndexScheduler {
atomic.fetch_add(1, Ordering::Relaxed);
}
// 3.2. Dump the settings
// 4.2. Dump the settings
let settings = meilisearch_types::settings::settings(
index,
&rtxn,
@ -215,10 +251,12 @@ impl IndexScheduler {
Ok(())
})?;
// 4. Dump experimental feature settings
// 5. Dump experimental feature settings
progress.update_progress(DumpCreationProgress::DumpTheExperimentalFeatures);
let features = self.features().runtime_features();
dump.create_experimental_features(features)?;
let network = self.network();
dump.create_network(network)?;
let dump_uid = started_at.format(format_description!(
"[year repr:full][month repr:numerical][day padding:zero]-[hour padding:zero][minute padding:zero][second padding:zero][subsecond digits:3]"

View file

@ -56,16 +56,13 @@ succeeded [1,]
### Batches Index Tasks:
----------------------------------------------------------------------
### 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

@ -54,15 +54,12 @@ succeeded [1,]
### Batches Index Tasks:
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
----------------------------------------------------------------------
### File Store:

View file

@ -7,7 +7,7 @@ snapshot_kind: text
[]
----------------------------------------------------------------------
### All Tasks:
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "An unexpected crash occurred when processing the task.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "An unexpected crash occurred when processing the task: simulated panic", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
----------------------------------------------------------------------
### Status:
enqueued []

View file

@ -87,7 +87,6 @@ doggo [2,3,]
girafo [4,]
----------------------------------------------------------------------
### Batches Enqueued At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
@ -95,7 +94,6 @@ girafo [4,]
[timestamp] [5,]
----------------------------------------------------------------------
### Batches Started At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]
@ -103,7 +101,6 @@ girafo [4,]
[timestamp] [5,]
----------------------------------------------------------------------
### Batches Finished At:
[timestamp] [0,]
[timestamp] [1,]
[timestamp] [2,]
[timestamp] [3,]

View file

@ -903,7 +903,7 @@ fn create_and_list_index() {
index_scheduler.index("kefir").unwrap();
let list = index_scheduler.get_paginated_indexes_stats(&AuthFilter::default(), 0, 20).unwrap();
snapshot!(json_string!(list, { "[1][0][1].created_at" => "[date]", "[1][0][1].updated_at" => "[date]", "[1][0][1].used_database_size" => "[bytes]", "[1][0][1].database_size" => "[bytes]" }), @r#"
snapshot!(json_string!(list, { "[1][0][1].created_at" => "[date]", "[1][0][1].updated_at" => "[date]", "[1][0][1].used_database_size" => "[bytes]", "[1][0][1].database_size" => "[bytes]" }), @r###"
[
1,
[
@ -912,6 +912,8 @@ fn create_and_list_index() {
{
"number_of_documents": 0,
"database_size": "[bytes]",
"number_of_embeddings": 0,
"number_of_embedded_documents": 0,
"used_database_size": "[bytes]",
"primary_key": null,
"field_distribution": {},
@ -921,5 +923,5 @@ fn create_and_list_index() {
]
]
]
"#);
"###);
}

View file

@ -6,8 +6,7 @@ use meili_snap::snapshot;
use meilisearch_types::milli::obkv_to_json;
use meilisearch_types::milli::update::IndexDocumentsMethod::*;
use meilisearch_types::milli::update::Setting;
use meilisearch_types::tasks::Kind;
use meilisearch_types::tasks::KindWithContent;
use meilisearch_types::tasks::{Kind, KindWithContent};
use crate::insta_snapshot::snapshot_index_scheduler;
use crate::test_utils::Breakpoint::*;

View file

@ -3,7 +3,7 @@
use std::collections::{BTreeSet, HashSet};
use std::ops::Bound;
use meilisearch_types::batches::{Batch, BatchId, BatchStats};
use meilisearch_types::batches::{Batch, BatchEnqueuedAt, BatchId, BatchStats};
use meilisearch_types::heed::{Database, RoTxn, RwTxn};
use meilisearch_types::milli::CboRoaringBitmapCodec;
use meilisearch_types::task_view::DetailsView;
@ -30,8 +30,7 @@ pub struct ProcessingBatch {
pub kinds: HashSet<Kind>,
pub indexes: HashSet<String>,
pub canceled_by: HashSet<TaskId>,
pub oldest_enqueued_at: Option<OffsetDateTime>,
pub earliest_enqueued_at: Option<OffsetDateTime>,
pub enqueued_at: Option<BatchEnqueuedAt>,
pub started_at: OffsetDateTime,
pub finished_at: Option<OffsetDateTime>,
}
@ -51,8 +50,7 @@ impl ProcessingBatch {
kinds: HashSet::default(),
indexes: HashSet::default(),
canceled_by: HashSet::default(),
oldest_enqueued_at: None,
earliest_enqueued_at: None,
enqueued_at: None,
started_at: OffsetDateTime::now_utc(),
finished_at: None,
}
@ -80,14 +78,18 @@ impl ProcessingBatch {
if let Some(canceled_by) = task.canceled_by {
self.canceled_by.insert(canceled_by);
}
self.oldest_enqueued_at =
Some(self.oldest_enqueued_at.map_or(task.enqueued_at, |oldest_enqueued_at| {
task.enqueued_at.min(oldest_enqueued_at)
}));
self.earliest_enqueued_at =
Some(self.earliest_enqueued_at.map_or(task.enqueued_at, |earliest_enqueued_at| {
task.enqueued_at.max(earliest_enqueued_at)
}));
match self.enqueued_at.as_mut() {
Some(BatchEnqueuedAt { earliest, oldest }) => {
*oldest = task.enqueued_at.min(*oldest);
*earliest = task.enqueued_at.max(*earliest);
}
None => {
self.enqueued_at = Some(BatchEnqueuedAt {
earliest: task.enqueued_at,
oldest: task.enqueued_at,
});
}
}
}
}
@ -138,6 +140,7 @@ impl ProcessingBatch {
stats: self.stats.clone(),
started_at: self.started_at,
finished_at: self.finished_at,
enqueued_at: self.enqueued_at,
}
}
}
@ -174,6 +177,33 @@ pub(crate) fn remove_task_datetime(
Ok(())
}
pub(crate) fn remove_n_tasks_datetime_earlier_than(
wtxn: &mut RwTxn,
database: Database<BEI128, CboRoaringBitmapCodec>,
earlier_than: OffsetDateTime,
mut count: usize,
task_id: TaskId,
) -> Result<()> {
let earlier_than = earlier_than.unix_timestamp_nanos();
let mut iter = database.rev_range_mut(wtxn, &(..earlier_than))?;
while let Some((current, mut existing)) = iter.next().transpose()? {
count -= existing.remove(task_id) as usize;
if existing.is_empty() {
// safety: We don't keep references to the database
unsafe { iter.del_current()? };
} else {
// safety: We don't keep references to the database
unsafe { iter.put_current(&current, &existing)? };
}
if count == 0 {
break;
}
}
Ok(())
}
pub(crate) fn keep_ids_within_datetimes(
rtxn: &RoTxn,
ids: &mut RoaringBitmap,
@ -329,14 +359,27 @@ impl crate::IndexScheduler {
kind,
} = task;
assert_eq!(uid, task.uid);
if let Some(ref batch) = batch_uid {
if task.status != Status::Enqueued {
let batch_uid = batch_uid.expect("All non enqueued tasks must be part of a batch");
assert!(self
.queue
.batch_to_tasks_mapping
.get(&rtxn, batch)
.get(&rtxn, &batch_uid)
.unwrap()
.unwrap()
.contains(uid));
let batch = self.queue.batches.get_batch(&rtxn, batch_uid).unwrap().unwrap();
assert_eq!(batch.uid, batch_uid);
if task.status == Status::Processing {
assert!(batch.progress.is_some());
} else {
assert!(batch.progress.is_none());
}
assert_eq!(batch.started_at, task.started_at.unwrap());
assert_eq!(batch.finished_at, task.finished_at);
let enqueued_at = batch.enqueued_at.unwrap();
assert!(task.enqueued_at >= enqueued_at.oldest);
assert!(task.enqueued_at <= enqueued_at.earliest);
}
if let Some(task_index_uid) = &task_index_uid {
assert!(self

View file

@ -1,9 +1,10 @@
use crate::{upgrade::upgrade_index_scheduler, Result};
use meilisearch_types::{
heed::{types::Str, Database, Env, RoTxn, RwTxn},
milli::heed_codec::version::VersionCodec,
versioning,
};
use meilisearch_types::heed::types::Str;
use meilisearch_types::heed::{self, Database, Env, RoTxn, RwTxn};
use meilisearch_types::milli::heed_codec::version::VersionCodec;
use meilisearch_types::versioning;
use crate::upgrade::upgrade_index_scheduler;
use crate::Result;
/// The number of database used by queue itself
const NUMBER_OF_DATABASES: u32 = 1;
@ -21,30 +22,38 @@ pub struct Versioning {
}
impl Versioning {
pub(crate) const fn nb_db() -> u32 {
pub const fn nb_db() -> u32 {
NUMBER_OF_DATABASES
}
pub fn get_version(&self, rtxn: &RoTxn) -> Result<Option<(u32, u32, u32)>> {
Ok(self.version.get(rtxn, entry_name::MAIN)?)
pub fn get_version(&self, rtxn: &RoTxn) -> Result<Option<(u32, u32, u32)>, heed::Error> {
self.version.get(rtxn, entry_name::MAIN)
}
pub fn set_version(&self, wtxn: &mut RwTxn, version: (u32, u32, u32)) -> Result<()> {
Ok(self.version.put(wtxn, entry_name::MAIN, &version)?)
pub fn set_version(
&self,
wtxn: &mut RwTxn,
version: (u32, u32, u32),
) -> Result<(), heed::Error> {
self.version.put(wtxn, entry_name::MAIN, &version)
}
pub fn set_current_version(&self, wtxn: &mut RwTxn) -> Result<()> {
pub fn set_current_version(&self, wtxn: &mut RwTxn) -> Result<(), heed::Error> {
let major = versioning::VERSION_MAJOR.parse().unwrap();
let minor = versioning::VERSION_MINOR.parse().unwrap();
let patch = versioning::VERSION_PATCH.parse().unwrap();
self.set_version(wtxn, (major, minor, patch))
}
/// Create an index scheduler and start its run loop.
/// Return `Self` without checking anything about the version
pub fn raw_new(env: &Env, wtxn: &mut RwTxn) -> Result<Self, heed::Error> {
let version = env.create_database(wtxn, Some(db_name::VERSION))?;
Ok(Self { version })
}
pub(crate) fn new(env: &Env, db_version: (u32, u32, u32)) -> Result<Self> {
let mut wtxn = env.write_txn()?;
let version = env.create_database(&mut wtxn, Some(db_name::VERSION))?;
let this = Self { version };
let this = Self::raw_new(env, &mut wtxn)?;
let from = match this.get_version(&wtxn)? {
Some(version) => version,
// fresh DB: use the db version