mirror of
https://github.com/meilisearch/MeiliSearch
synced 2025-07-03 03:47:02 +02:00
Merge branch 'main' into all-cpus-in-import-dump
This commit is contained in:
commit
75a7e40a27
74 changed files with 2402 additions and 1726 deletions
|
@ -2,6 +2,7 @@ use std::fmt::Display;
|
|||
|
||||
use meilisearch_types::batches::BatchId;
|
||||
use meilisearch_types::error::{Code, ErrorCode};
|
||||
use meilisearch_types::milli::index::RollbackOutcome;
|
||||
use meilisearch_types::tasks::{Kind, Status};
|
||||
use meilisearch_types::{heed, milli};
|
||||
use thiserror::Error;
|
||||
|
@ -150,8 +151,24 @@ pub enum Error {
|
|||
CorruptedTaskQueue,
|
||||
#[error(transparent)]
|
||||
DatabaseUpgrade(Box<Self>),
|
||||
#[error("Failed to rollback for index `{index}`: {rollback_outcome} ")]
|
||||
RollbackFailed { index: String, rollback_outcome: RollbackOutcome },
|
||||
#[error(transparent)]
|
||||
UnrecoverableError(Box<Self>),
|
||||
#[error("The index scheduler is in version v{}.{}.{}, but Meilisearch is in version v{}.{}.{}.\n - hint: start the correct version of Meilisearch, or consider updating your database. See also <https://www.meilisearch.com/docs/learn/update_and_migration/updating>",
|
||||
index_scheduler_version.0, index_scheduler_version.1, index_scheduler_version.2,
|
||||
package_version.0, package_version.1, package_version.2)]
|
||||
IndexSchedulerVersionMismatch {
|
||||
index_scheduler_version: (u32, u32, u32),
|
||||
package_version: (u32, u32, u32),
|
||||
},
|
||||
#[error("Index `{index}` is in version v{}.{}.{}, but Meilisearch is in version v{}.{}.{}.\n - note: this is an internal error, please consider filing a bug report: <https://github.com/meilisearch/meilisearch/issues/new?template=bug_report.md>",
|
||||
index_version.0, index_version.1, index_version.2, package_version.0, package_version.1, package_version.2)]
|
||||
IndexVersionMismatch {
|
||||
index: String,
|
||||
index_version: (u32, u32, u32),
|
||||
package_version: (u32, u32, u32),
|
||||
},
|
||||
#[error(transparent)]
|
||||
HeedTransaction(heed::Error),
|
||||
|
||||
|
@ -209,6 +226,9 @@ impl Error {
|
|||
| Error::CorruptedTaskQueue
|
||||
| Error::DatabaseUpgrade(_)
|
||||
| Error::UnrecoverableError(_)
|
||||
| Error::IndexSchedulerVersionMismatch { .. }
|
||||
| Error::IndexVersionMismatch { .. }
|
||||
| Error::RollbackFailed { .. }
|
||||
| Error::HeedTransaction(_) => false,
|
||||
#[cfg(test)]
|
||||
Error::PlannedFailure => false,
|
||||
|
@ -274,7 +294,10 @@ impl ErrorCode for Error {
|
|||
Error::CorruptedTaskQueue => Code::Internal,
|
||||
Error::CorruptedDump => Code::Internal,
|
||||
Error::DatabaseUpgrade(_) => Code::Internal,
|
||||
Error::RollbackFailed { .. } => Code::Internal,
|
||||
Error::UnrecoverableError(_) => Code::Internal,
|
||||
Error::IndexSchedulerVersionMismatch { .. } => Code::Internal,
|
||||
Error::IndexVersionMismatch { .. } => Code::Internal,
|
||||
Error::CreateBatch(_) => Code::Internal,
|
||||
|
||||
// This one should never be seen by the end user
|
||||
|
|
|
@ -7,6 +7,7 @@ use meilisearch_types::heed::types::{SerdeJson, Str};
|
|||
use meilisearch_types::heed::{Database, Env, RoTxn, RwTxn, WithoutTls};
|
||||
use meilisearch_types::milli;
|
||||
use meilisearch_types::milli::database_stats::DatabaseStats;
|
||||
use meilisearch_types::milli::index::RollbackOutcome;
|
||||
use meilisearch_types::milli::update::IndexerConfig;
|
||||
use meilisearch_types::milli::{FieldDistribution, Index};
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
@ -431,6 +432,51 @@ impl IndexMapper {
|
|||
Ok(index)
|
||||
}
|
||||
|
||||
pub fn rollback_index(
|
||||
&self,
|
||||
rtxn: &RoTxn,
|
||||
name: &str,
|
||||
to: (u32, u32, u32),
|
||||
) -> Result<RollbackOutcome> {
|
||||
// remove any currently updating index to make sure that we aren't keeping a reference to the index somewhere
|
||||
drop(self.currently_updating_index.write().unwrap().take());
|
||||
|
||||
let uuid = self
|
||||
.index_mapping
|
||||
.get(rtxn, name)?
|
||||
.ok_or_else(|| Error::IndexNotFound(name.to_string()))?;
|
||||
|
||||
// take the lock to make sure noone is messing with the indexes while we rollback
|
||||
// this will block any search or other operation, but we are rollbacking so this is probably acceptable.
|
||||
let mut index_map = self.index_map.write().unwrap();
|
||||
|
||||
'close_index: loop {
|
||||
match index_map.get(&uuid) {
|
||||
Available(_) => {
|
||||
index_map.close_for_resize(&uuid, self.enable_mdb_writemap, 0);
|
||||
// index should now be `Closing`; try again
|
||||
continue;
|
||||
}
|
||||
// index already closed
|
||||
Missing => break 'close_index,
|
||||
// closing requested by this thread or another one; wait for closing to complete, then exit
|
||||
Closing(closing_index) => {
|
||||
if closing_index.wait_timeout(Duration::from_secs(100)).is_none() {
|
||||
// release the lock so it doesn't get poisoned
|
||||
drop(index_map);
|
||||
panic!("cannot close index")
|
||||
}
|
||||
break;
|
||||
}
|
||||
BeingDeleted => return Err(Error::IndexNotFound(name.to_string())),
|
||||
};
|
||||
}
|
||||
|
||||
let index_path = self.base_path.join(uuid.to_string());
|
||||
Index::rollback(milli::heed::EnvOpenOptions::new().read_txn_without_tls(), index_path, to)
|
||||
.map_err(|err| crate::Error::from_milli(err, Some(name.to_string())))
|
||||
}
|
||||
|
||||
/// Attempts `f` for each index that exists in the index mapper.
|
||||
///
|
||||
/// It is preferable to use this function rather than a loop that opens all indexes, as a way to avoid having all indexes opened,
|
||||
|
|
|
@ -41,11 +41,8 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String {
|
|||
let mut snap = String::new();
|
||||
|
||||
let indx_sched_version = version.get_version(&rtxn).unwrap();
|
||||
let latest_version = (
|
||||
versioning::VERSION_MAJOR.parse().unwrap(),
|
||||
versioning::VERSION_MINOR.parse().unwrap(),
|
||||
versioning::VERSION_PATCH.parse().unwrap(),
|
||||
);
|
||||
let latest_version =
|
||||
(versioning::VERSION_MAJOR, versioning::VERSION_MINOR, versioning::VERSION_PATCH);
|
||||
if indx_sched_version != Some(latest_version) {
|
||||
snap.push_str(&format!("index scheduler running on version {indx_sched_version:?}\n"));
|
||||
}
|
||||
|
|
|
@ -398,9 +398,9 @@ impl IndexScheduler {
|
|||
Ok(Ok(TickOutcome::StopProcessingForever)) => break,
|
||||
Ok(Err(e)) => {
|
||||
tracing::error!("{e}");
|
||||
// Wait one second when an irrecoverable error occurs.
|
||||
// Wait when an irrecoverable error occurs.
|
||||
if !e.is_recoverable() {
|
||||
std::thread::sleep(Duration::from_secs(1));
|
||||
std::thread::sleep(Duration::from_secs(10));
|
||||
}
|
||||
}
|
||||
Err(_panic) => {
|
||||
|
|
|
@ -74,6 +74,7 @@ make_enum_progress! {
|
|||
make_enum_progress! {
|
||||
pub enum TaskCancelationProgress {
|
||||
RetrievingTasks,
|
||||
CancelingUpgrade,
|
||||
UpdatingTasks,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -423,7 +423,8 @@ impl IndexScheduler {
|
|||
}
|
||||
|
||||
/// Create the next batch to be processed;
|
||||
/// 1. We get the *last* task to cancel.
|
||||
/// 0. We get the *last* task to cancel.
|
||||
/// 1. We get the tasks to upgrade.
|
||||
/// 2. We get the *next* task to delete.
|
||||
/// 3. We get the *next* snapshot to process.
|
||||
/// 4. We get the *next* dump to process.
|
||||
|
@ -443,7 +444,20 @@ impl IndexScheduler {
|
|||
let count_total_enqueued = enqueued.len();
|
||||
let failed = &self.queue.tasks.get_status(rtxn, Status::Failed)?;
|
||||
|
||||
// 0. The priority over everything is to upgrade the instance
|
||||
// 0. we get the last task to cancel.
|
||||
let to_cancel = self.queue.tasks.get_kind(rtxn, Kind::TaskCancelation)? & enqueued;
|
||||
if let Some(task_id) = to_cancel.max() {
|
||||
let mut task =
|
||||
self.queue.tasks.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?;
|
||||
current_batch.processing(Some(&mut task));
|
||||
current_batch.reason(BatchStopReason::TaskCannotBeBatched {
|
||||
kind: Kind::TaskCancelation,
|
||||
id: task_id,
|
||||
});
|
||||
return Ok(Some((Batch::TaskCancelation { task }, current_batch)));
|
||||
}
|
||||
|
||||
// 1. We upgrade the instance
|
||||
// There shouldn't be multiple upgrade tasks but just in case we're going to batch all of them at the same time
|
||||
let upgrade = self.queue.tasks.get_kind(rtxn, Kind::UpgradeDatabase)? & (enqueued | failed);
|
||||
if !upgrade.is_empty() {
|
||||
|
@ -459,17 +473,21 @@ impl IndexScheduler {
|
|||
return Ok(Some((Batch::UpgradeDatabase { tasks }, current_batch)));
|
||||
}
|
||||
|
||||
// 1. we get the last task to cancel.
|
||||
let to_cancel = self.queue.tasks.get_kind(rtxn, Kind::TaskCancelation)? & enqueued;
|
||||
if let Some(task_id) = to_cancel.max() {
|
||||
let mut task =
|
||||
self.queue.tasks.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?;
|
||||
current_batch.processing(Some(&mut task));
|
||||
current_batch.reason(BatchStopReason::TaskCannotBeBatched {
|
||||
kind: Kind::TaskCancelation,
|
||||
id: task_id,
|
||||
});
|
||||
return Ok(Some((Batch::TaskCancelation { task }, current_batch)));
|
||||
// check the version of the scheduler here.
|
||||
// if the version is not the current, refuse to batch any additional task.
|
||||
let version = self.version.get_version(rtxn)?;
|
||||
let package_version = (
|
||||
meilisearch_types::versioning::VERSION_MAJOR,
|
||||
meilisearch_types::versioning::VERSION_MINOR,
|
||||
meilisearch_types::versioning::VERSION_PATCH,
|
||||
);
|
||||
if version != Some(package_version) {
|
||||
return Err(Error::UnrecoverableError(Box::new(
|
||||
Error::IndexSchedulerVersionMismatch {
|
||||
index_scheduler_version: version.unwrap_or((1, 12, 0)),
|
||||
package_version,
|
||||
},
|
||||
)));
|
||||
}
|
||||
|
||||
// 2. we get the next task to delete
|
||||
|
|
|
@ -6,7 +6,8 @@ use meilisearch_types::batches::{BatchEnqueuedAt, BatchId};
|
|||
use meilisearch_types::heed::{RoTxn, RwTxn};
|
||||
use meilisearch_types::milli::progress::{Progress, VariableNameStep};
|
||||
use meilisearch_types::milli::{self, ChannelCongestion};
|
||||
use meilisearch_types::tasks::{Details, IndexSwap, KindWithContent, Status, Task};
|
||||
use meilisearch_types::tasks::{Details, IndexSwap, Kind, KindWithContent, Status, Task};
|
||||
use meilisearch_types::versioning::{VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH};
|
||||
use milli::update::Settings as MilliSettings;
|
||||
use roaring::RoaringBitmap;
|
||||
|
||||
|
@ -144,11 +145,22 @@ impl IndexScheduler {
|
|||
self.index_mapper.index(&rtxn, &index_uid)?
|
||||
};
|
||||
|
||||
let mut index_wtxn = index.write_txn()?;
|
||||
|
||||
let index_version = index.get_version(&index_wtxn)?.unwrap_or((1, 12, 0));
|
||||
let package_version = (VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH);
|
||||
if index_version != package_version {
|
||||
return Err(Error::IndexVersionMismatch {
|
||||
index: index_uid,
|
||||
index_version,
|
||||
package_version,
|
||||
});
|
||||
}
|
||||
|
||||
// the index operation can take a long time, so save this handle to make it available to the search for the duration of the tick
|
||||
self.index_mapper
|
||||
.set_currently_updating_index(Some((index_uid.clone(), index.clone())));
|
||||
|
||||
let mut index_wtxn = index.write_txn()?;
|
||||
let pre_commit_dabases_sizes = index.database_sizes(&index_wtxn)?;
|
||||
let (tasks, congestion) =
|
||||
self.apply_index_operation(&mut index_wtxn, &index, op, &progress)?;
|
||||
|
@ -353,9 +365,11 @@ impl IndexScheduler {
|
|||
let KindWithContent::UpgradeDatabase { from } = tasks.last().unwrap().kind else {
|
||||
unreachable!();
|
||||
};
|
||||
|
||||
let ret = catch_unwind(AssertUnwindSafe(|| self.process_upgrade(from, progress)));
|
||||
match ret {
|
||||
Ok(Ok(())) => (),
|
||||
Ok(Err(Error::AbortedTask)) => return Err(Error::AbortedTask),
|
||||
Ok(Err(e)) => return Err(Error::DatabaseUpgrade(Box::new(e))),
|
||||
Err(e) => {
|
||||
let msg = match e.downcast_ref::<&'static str>() {
|
||||
|
@ -653,17 +667,79 @@ impl IndexScheduler {
|
|||
progress: &Progress,
|
||||
) -> Result<Vec<Task>> {
|
||||
progress.update_progress(TaskCancelationProgress::RetrievingTasks);
|
||||
let mut tasks_to_cancel = RoaringBitmap::new();
|
||||
|
||||
let enqueued_tasks = &self.queue.tasks.get_status(rtxn, Status::Enqueued)?;
|
||||
|
||||
// 0. Check if any upgrade task was matched.
|
||||
// If so, we cancel all the failed or enqueued upgrade tasks.
|
||||
let upgrade_tasks = &self.queue.tasks.get_kind(rtxn, Kind::UpgradeDatabase)?;
|
||||
let is_canceling_upgrade = !matched_tasks.is_disjoint(upgrade_tasks);
|
||||
if is_canceling_upgrade {
|
||||
let failed_tasks = self.queue.tasks.get_status(rtxn, Status::Failed)?;
|
||||
tasks_to_cancel |= upgrade_tasks & (enqueued_tasks | failed_tasks);
|
||||
}
|
||||
// 1. Remove from this list the tasks that we are not allowed to cancel
|
||||
// Notice that only the _enqueued_ ones are cancelable and we should
|
||||
// have already aborted the indexation of the _processing_ ones
|
||||
let cancelable_tasks = self.queue.tasks.get_status(rtxn, Status::Enqueued)?;
|
||||
let tasks_to_cancel = cancelable_tasks & matched_tasks;
|
||||
tasks_to_cancel |= enqueued_tasks & matched_tasks;
|
||||
|
||||
// 2. If we're canceling an upgrade, attempt the rollback
|
||||
if let Some(latest_upgrade_task) = (&tasks_to_cancel & upgrade_tasks).max() {
|
||||
progress.update_progress(TaskCancelationProgress::CancelingUpgrade);
|
||||
|
||||
let task = self.queue.tasks.get_task(rtxn, latest_upgrade_task)?.unwrap();
|
||||
let Some(Details::UpgradeDatabase { from, to }) = task.details else {
|
||||
unreachable!("wrong details for upgrade task {latest_upgrade_task}")
|
||||
};
|
||||
|
||||
// check that we are rollbacking an upgrade to the current Meilisearch
|
||||
let bin_major: u32 = meilisearch_types::versioning::VERSION_MAJOR;
|
||||
let bin_minor: u32 = meilisearch_types::versioning::VERSION_MINOR;
|
||||
let bin_patch: u32 = meilisearch_types::versioning::VERSION_PATCH;
|
||||
|
||||
if to == (bin_major, bin_minor, bin_patch) {
|
||||
tracing::warn!(
|
||||
"Rollbacking from v{}.{}.{} to v{}.{}.{}",
|
||||
to.0,
|
||||
to.1,
|
||||
to.2,
|
||||
from.0,
|
||||
from.1,
|
||||
from.2
|
||||
);
|
||||
match std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| {
|
||||
self.process_rollback(from, progress)
|
||||
})) {
|
||||
Ok(Ok(())) => {}
|
||||
Ok(Err(err)) => return Err(Error::DatabaseUpgrade(Box::new(err))),
|
||||
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(),
|
||||
))));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
tracing::debug!(
|
||||
"Not rollbacking an upgrade targetting the earlier version v{}.{}.{}",
|
||||
bin_major,
|
||||
bin_minor,
|
||||
bin_patch
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
// 3. We now have a list of tasks to cancel, cancel them
|
||||
let (task_progress, progress_obj) = AtomicTaskStep::new(tasks_to_cancel.len() as u32);
|
||||
progress.update_progress(progress_obj);
|
||||
|
||||
// 2. We now have a list of tasks to cancel, cancel them
|
||||
let mut tasks = self.queue.tasks.get_existing_tasks(
|
||||
rtxn,
|
||||
tasks_to_cancel.iter().inspect(|_| {
|
||||
|
|
|
@ -12,10 +12,14 @@ impl IndexScheduler {
|
|||
#[cfg(test)]
|
||||
self.maybe_fail(crate::test_utils::FailureLocation::ProcessUpgrade)?;
|
||||
|
||||
enum UpgradeIndex {}
|
||||
let indexes = self.index_names()?;
|
||||
|
||||
for (i, uid) in indexes.iter().enumerate() {
|
||||
let must_stop_processing = self.scheduler.must_stop_processing.clone();
|
||||
|
||||
if must_stop_processing.get() {
|
||||
return Err(Error::AbortedTask);
|
||||
}
|
||||
progress.update_progress(VariableNameStep::<UpgradeIndex>::new(
|
||||
format!("Upgrading index `{uid}`"),
|
||||
i as u32,
|
||||
|
@ -27,6 +31,7 @@ impl IndexScheduler {
|
|||
&mut index_wtxn,
|
||||
&index,
|
||||
db_version,
|
||||
|| must_stop_processing.get(),
|
||||
progress.clone(),
|
||||
)
|
||||
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?;
|
||||
|
@ -46,4 +51,42 @@ impl IndexScheduler {
|
|||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn process_rollback(&self, db_version: (u32, u32, u32), progress: &Progress) -> Result<()> {
|
||||
let mut wtxn = self.env.write_txn()?;
|
||||
tracing::info!(?db_version, "roll back index scheduler version");
|
||||
self.version.set_version(&mut wtxn, db_version)?;
|
||||
let db_path = self.scheduler.version_file_path.parent().unwrap();
|
||||
wtxn.commit()?;
|
||||
|
||||
let indexes = self.index_names()?;
|
||||
|
||||
tracing::info!("roll backing all indexes");
|
||||
for (i, uid) in indexes.iter().enumerate() {
|
||||
progress.update_progress(VariableNameStep::<UpgradeIndex>::new(
|
||||
format!("Rollbacking index `{uid}`"),
|
||||
i as u32,
|
||||
indexes.len() as u32,
|
||||
));
|
||||
let index_schd_rtxn = self.env.read_txn()?;
|
||||
|
||||
let rollback_outcome =
|
||||
self.index_mapper.rollback_index(&index_schd_rtxn, uid, db_version)?;
|
||||
if !rollback_outcome.succeeded() {
|
||||
return Err(crate::Error::RollbackFailed { index: uid.clone(), rollback_outcome });
|
||||
}
|
||||
}
|
||||
|
||||
tracing::info!(?db_path, ?db_version, "roll back version file");
|
||||
meilisearch_types::versioning::create_version_file(
|
||||
db_path,
|
||||
db_version.0,
|
||||
db_version.1,
|
||||
db_version.2,
|
||||
)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
enum UpgradeIndex {}
|
||||
|
|
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
|||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, batch_uid: 0, status: succeeded, details: { from: (1, 12, 0), to: (1, 14, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, batch_uid: 0, status: succeeded, details: { from: (1, 12, 0), to: (1, 15, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
3 {uid: 3, batch_uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `doggo` already exists.", error_code: "index_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_already_exists" }, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
|
@ -57,7 +57,7 @@ girafo: { number_of_documents: 0, field_distribution: {} }
|
|||
[timestamp] [4,]
|
||||
----------------------------------------------------------------------
|
||||
### All Batches:
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.14.0"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "a batch of tasks of type `upgradeDatabase` cannot be batched with any other type of task", }
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.15.0"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "a batch of tasks of type `upgradeDatabase` cannot be batched with any other type of task", }
|
||||
1 {uid: 1, details: {"primaryKey":"mouse"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, stop reason: "task with id 1 of type `indexCreation` cannot be batched", }
|
||||
2 {uid: 2, details: {"primaryKey":"bone"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, stop reason: "task with id 2 of type `indexCreation` cannot be batched", }
|
||||
3 {uid: 3, details: {"primaryKey":"bone"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, stop reason: "task with id 3 of type `indexCreation` cannot be batched", }
|
||||
|
|
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
|||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, status: enqueued, details: { from: (1, 12, 0), to: (1, 14, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, status: enqueued, details: { from: (1, 12, 0), to: (1, 15, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
----------------------------------------------------------------------
|
||||
### Status:
|
||||
enqueued [0,]
|
||||
|
|
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
|||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, status: enqueued, details: { from: (1, 12, 0), to: (1, 14, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, status: enqueued, details: { from: (1, 12, 0), to: (1, 15, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
----------------------------------------------------------------------
|
||||
### Status:
|
||||
|
|
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
|||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { from: (1, 12, 0), to: (1, 14, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { from: (1, 12, 0), to: (1, 15, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
----------------------------------------------------------------------
|
||||
### Status:
|
||||
|
@ -37,7 +37,7 @@ catto [1,]
|
|||
[timestamp] [0,]
|
||||
----------------------------------------------------------------------
|
||||
### All Batches:
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.14.0"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "a batch of tasks of type `upgradeDatabase` cannot be batched with any other type of task", }
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.15.0"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "a batch of tasks of type `upgradeDatabase` cannot be batched with any other type of task", }
|
||||
----------------------------------------------------------------------
|
||||
### Batch to tasks mapping:
|
||||
0 [0,]
|
||||
|
|
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
|||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { from: (1, 12, 0), to: (1, 14, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { from: (1, 12, 0), to: (1, 15, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
2 {uid: 2, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
----------------------------------------------------------------------
|
||||
|
@ -40,7 +40,7 @@ doggo [2,]
|
|||
[timestamp] [0,]
|
||||
----------------------------------------------------------------------
|
||||
### All Batches:
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.14.0"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "a batch of tasks of type `upgradeDatabase` cannot be batched with any other type of task", }
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.15.0"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "a batch of tasks of type `upgradeDatabase` cannot be batched with any other type of task", }
|
||||
----------------------------------------------------------------------
|
||||
### Batch to tasks mapping:
|
||||
0 [0,]
|
||||
|
|
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
|||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, batch_uid: 0, status: succeeded, details: { from: (1, 12, 0), to: (1, 14, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, batch_uid: 0, status: succeeded, details: { from: (1, 12, 0), to: (1, 15, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
2 {uid: 2, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
3 {uid: 3, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
|
@ -43,7 +43,7 @@ doggo [2,3,]
|
|||
[timestamp] [0,]
|
||||
----------------------------------------------------------------------
|
||||
### All Batches:
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.14.0"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "a batch of tasks of type `upgradeDatabase` cannot be batched with any other type of task", }
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.15.0"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "a batch of tasks of type `upgradeDatabase` cannot be batched with any other type of task", }
|
||||
----------------------------------------------------------------------
|
||||
### Batch to tasks mapping:
|
||||
0 [0,]
|
||||
|
|
|
@ -114,12 +114,8 @@ impl IndexScheduler {
|
|||
auto_upgrade: true, // Don't cost much and will ensure the happy path works
|
||||
embedding_cache_cap: 10,
|
||||
};
|
||||
let version = configuration(&mut options).unwrap_or_else(|| {
|
||||
(
|
||||
versioning::VERSION_MAJOR.parse().unwrap(),
|
||||
versioning::VERSION_MINOR.parse().unwrap(),
|
||||
versioning::VERSION_PATCH.parse().unwrap(),
|
||||
)
|
||||
let version = configuration(&mut options).unwrap_or({
|
||||
(versioning::VERSION_MAJOR, versioning::VERSION_MINOR, versioning::VERSION_PATCH)
|
||||
});
|
||||
|
||||
std::fs::create_dir_all(&options.auth_path).unwrap();
|
||||
|
|
|
@ -28,12 +28,17 @@ pub fn upgrade_index_scheduler(
|
|||
let current_minor = to.1;
|
||||
let current_patch = to.2;
|
||||
|
||||
let upgrade_functions: &[&dyn UpgradeIndexScheduler] = &[&ToCurrentNoOp {}];
|
||||
let upgrade_functions: &[&dyn UpgradeIndexScheduler] = &[
|
||||
// This is the last upgrade function, it will be called when the index is up to date.
|
||||
// any other upgrade function should be added before this one.
|
||||
&ToCurrentNoOp {},
|
||||
];
|
||||
|
||||
let start = match from {
|
||||
(1, 12, _) => 0,
|
||||
(1, 13, _) => 0,
|
||||
(1, 14, _) => 0,
|
||||
(1, 15, _) => 0,
|
||||
(major, minor, patch) => {
|
||||
if major > current_major
|
||||
|| (major == current_major && minor > current_minor)
|
||||
|
@ -104,10 +109,6 @@ impl UpgradeIndexScheduler for ToCurrentNoOp {
|
|||
}
|
||||
|
||||
fn target_version(&self) -> (u32, u32, u32) {
|
||||
(
|
||||
VERSION_MAJOR.parse().unwrap(),
|
||||
VERSION_MINOR.parse().unwrap(),
|
||||
VERSION_PATCH.parse().unwrap(),
|
||||
)
|
||||
(VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,9 +39,9 @@ impl Versioning {
|
|||
}
|
||||
|
||||
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();
|
||||
let major = versioning::VERSION_MAJOR;
|
||||
let minor = versioning::VERSION_MINOR;
|
||||
let patch = versioning::VERSION_PATCH;
|
||||
self.set_version(wtxn, (major, minor, patch))
|
||||
}
|
||||
|
||||
|
@ -64,9 +64,9 @@ impl Versioning {
|
|||
};
|
||||
wtxn.commit()?;
|
||||
|
||||
let bin_major: u32 = versioning::VERSION_MAJOR.parse().unwrap();
|
||||
let bin_minor: u32 = versioning::VERSION_MINOR.parse().unwrap();
|
||||
let bin_patch: u32 = versioning::VERSION_PATCH.parse().unwrap();
|
||||
let bin_major: u32 = versioning::VERSION_MAJOR;
|
||||
let bin_minor: u32 = versioning::VERSION_MINOR;
|
||||
let bin_patch: u32 = versioning::VERSION_PATCH;
|
||||
let to = (bin_major, bin_minor, bin_patch);
|
||||
|
||||
if from != to {
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue