mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-03 11:57:07 +02:00
Merge #5351
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:
commit
0f1aeb8eaa
101 changed files with 8351 additions and 1518 deletions
|
@ -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();
|
||||
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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())
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) };
|
||||
}
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -96,6 +96,7 @@ make_enum_progress! {
|
|||
StartTheDumpCreation,
|
||||
DumpTheApiKeys,
|
||||
DumpTheTasks,
|
||||
DumpTheBatches,
|
||||
DumpTheIndexes,
|
||||
DumpTheExperimentalFeatures,
|
||||
CompressTheDump,
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
});
|
||||
|
||||
|
|
|
@ -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()))
|
||||
}
|
||||
}
|
||||
})
|
||||
};
|
||||
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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]"
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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 []
|
||||
|
|
|
@ -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,]
|
||||
|
|
|
@ -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() {
|
|||
]
|
||||
]
|
||||
]
|
||||
"#);
|
||||
"###);
|
||||
}
|
||||
|
|
|
@ -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::*;
|
||||
|
|
|
@ -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(¤t, &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
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue