diff --git a/crates/index-scheduler/src/error.rs b/crates/index-scheduler/src/error.rs index 280127d04..cb798b385 100644 --- a/crates/index-scheduler/src/error.rs +++ b/crates/index-scheduler/src/error.rs @@ -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), + #[error("Failed to rollback for index `{index}`: {rollback_outcome} ")] + RollbackFailed { index: String, rollback_outcome: RollbackOutcome }, #[error(transparent)] UnrecoverableError(Box), + #[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 ", + 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: ", + 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 diff --git a/crates/index-scheduler/src/index_mapper/mod.rs b/crates/index-scheduler/src/index_mapper/mod.rs index c1f6ff472..86fb17ca7 100644 --- a/crates/index-scheduler/src/index_mapper/mod.rs +++ b/crates/index-scheduler/src/index_mapper/mod.rs @@ -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 { + // 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, diff --git a/crates/index-scheduler/src/insta_snapshot.rs b/crates/index-scheduler/src/insta_snapshot.rs index ee271b5df..89e615132 100644 --- a/crates/index-scheduler/src/insta_snapshot.rs +++ b/crates/index-scheduler/src/insta_snapshot.rs @@ -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")); } diff --git a/crates/index-scheduler/src/lib.rs b/crates/index-scheduler/src/lib.rs index a5afdcbf9..4f1109348 100644 --- a/crates/index-scheduler/src/lib.rs +++ b/crates/index-scheduler/src/lib.rs @@ -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) => { diff --git a/crates/index-scheduler/src/processing.rs b/crates/index-scheduler/src/processing.rs index 09ce46884..f23b811e5 100644 --- a/crates/index-scheduler/src/processing.rs +++ b/crates/index-scheduler/src/processing.rs @@ -74,6 +74,7 @@ make_enum_progress! { make_enum_progress! { pub enum TaskCancelationProgress { RetrievingTasks, + CancelingUpgrade, UpdatingTasks, } } diff --git a/crates/index-scheduler/src/scheduler/create_batch.rs b/crates/index-scheduler/src/scheduler/create_batch.rs index 29d352fe8..e3763881b 100644 --- a/crates/index-scheduler/src/scheduler/create_batch.rs +++ b/crates/index-scheduler/src/scheduler/create_batch.rs @@ -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 diff --git a/crates/index-scheduler/src/scheduler/process_batch.rs b/crates/index-scheduler/src/scheduler/process_batch.rs index 42de1d137..c349f90ad 100644 --- a/crates/index-scheduler/src/scheduler/process_batch.rs +++ b/crates/index-scheduler/src/scheduler/process_batch.rs @@ -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> { 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::() { + Some(s) => &s[..], + None => "Box", + }, + }; + 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(|_| { diff --git a/crates/index-scheduler/src/scheduler/process_upgrade/mod.rs b/crates/index-scheduler/src/scheduler/process_upgrade/mod.rs index 4feebabc4..6fbdb0b58 100644 --- a/crates/index-scheduler/src/scheduler/process_upgrade/mod.rs +++ b/crates/index-scheduler/src/scheduler/process_upgrade/mod.rs @@ -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::::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::::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 {} diff --git a/crates/index-scheduler/src/test_utils.rs b/crates/index-scheduler/src/test_utils.rs index 5c04a66ff..0d44b3c81 100644 --- a/crates/index-scheduler/src/test_utils.rs +++ b/crates/index-scheduler/src/test_utils.rs @@ -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(); diff --git a/crates/index-scheduler/src/upgrade/mod.rs b/crates/index-scheduler/src/upgrade/mod.rs index 4a3cb2f75..74c8ee696 100644 --- a/crates/index-scheduler/src/upgrade/mod.rs +++ b/crates/index-scheduler/src/upgrade/mod.rs @@ -104,10 +104,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) } } diff --git a/crates/index-scheduler/src/versioning.rs b/crates/index-scheduler/src/versioning.rs index 107b8e0ba..b0cb7fdb5 100644 --- a/crates/index-scheduler/src/versioning.rs +++ b/crates/index-scheduler/src/versioning.rs @@ -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 { diff --git a/crates/meilisearch-types/src/tasks.rs b/crates/meilisearch-types/src/tasks.rs index d96a45992..6e10f2606 100644 --- a/crates/meilisearch-types/src/tasks.rs +++ b/crates/meilisearch-types/src/tasks.rs @@ -272,9 +272,9 @@ impl KindWithContent { KindWithContent::UpgradeDatabase { from } => Some(Details::UpgradeDatabase { from: (from.0, from.1, from.2), to: ( - versioning::VERSION_MAJOR.parse().unwrap(), - versioning::VERSION_MINOR.parse().unwrap(), - versioning::VERSION_PATCH.parse().unwrap(), + versioning::VERSION_MAJOR, + versioning::VERSION_MINOR, + versioning::VERSION_PATCH, ), }), } @@ -338,9 +338,9 @@ impl KindWithContent { KindWithContent::UpgradeDatabase { from } => Some(Details::UpgradeDatabase { from: *from, to: ( - versioning::VERSION_MAJOR.parse().unwrap(), - versioning::VERSION_MINOR.parse().unwrap(), - versioning::VERSION_PATCH.parse().unwrap(), + versioning::VERSION_MAJOR, + versioning::VERSION_MINOR, + versioning::VERSION_PATCH, ), }), } @@ -386,9 +386,9 @@ impl From<&KindWithContent> for Option
{ KindWithContent::UpgradeDatabase { from } => Some(Details::UpgradeDatabase { from: *from, to: ( - versioning::VERSION_MAJOR.parse().unwrap(), - versioning::VERSION_MINOR.parse().unwrap(), - versioning::VERSION_PATCH.parse().unwrap(), + versioning::VERSION_MAJOR, + versioning::VERSION_MINOR, + versioning::VERSION_PATCH, ), }), } diff --git a/crates/meilisearch-types/src/versioning.rs b/crates/meilisearch-types/src/versioning.rs index 07e42c2ce..b2124c04e 100644 --- a/crates/meilisearch-types/src/versioning.rs +++ b/crates/meilisearch-types/src/versioning.rs @@ -8,9 +8,7 @@ use tempfile::NamedTempFile; /// The name of the file that contains the version of the database. pub const VERSION_FILE_NAME: &str = "VERSION"; -pub static VERSION_MAJOR: &str = env!("CARGO_PKG_VERSION_MAJOR"); -pub static VERSION_MINOR: &str = env!("CARGO_PKG_VERSION_MINOR"); -pub static VERSION_PATCH: &str = env!("CARGO_PKG_VERSION_PATCH"); +pub use milli::constants::{VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH}; /// Persists the version of the current Meilisearch binary to a VERSION file pub fn create_current_version_file(db_path: &Path) -> anyhow::Result<()> { @@ -19,9 +17,9 @@ pub fn create_current_version_file(db_path: &Path) -> anyhow::Result<()> { pub fn create_version_file( db_path: &Path, - major: &str, - minor: &str, - patch: &str, + major: u32, + minor: u32, + patch: u32, ) -> anyhow::Result<()> { let version_path = db_path.join(VERSION_FILE_NAME); // In order to persist the file later we must create it in the `data.ms` and not in `/tmp` diff --git a/crates/meilisearch/src/lib.rs b/crates/meilisearch/src/lib.rs index 761726d83..40d318140 100644 --- a/crates/meilisearch/src/lib.rs +++ b/crates/meilisearch/src/lib.rs @@ -235,10 +235,7 @@ pub fn setup_meilisearch(opt: &Opt) -> anyhow::Result<(Arc, Arc< auto_upgrade: opt.experimental_dumpless_upgrade, embedding_cache_cap: opt.experimental_embedding_cache_entries, }; - let bin_major: u32 = VERSION_MAJOR.parse().unwrap(); - let bin_minor: u32 = VERSION_MINOR.parse().unwrap(); - let bin_patch: u32 = VERSION_PATCH.parse().unwrap(); - let binary_version = (bin_major, bin_minor, bin_patch); + let binary_version = (VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH); let empty_db = is_empty_db(&opt.db_path); let (index_scheduler, auth_controller) = if let Some(ref snapshot_path) = opt.import_snapshot { diff --git a/crates/meilisearch/tests/upgrade/mod.rs b/crates/meilisearch/tests/upgrade/mod.rs index ae6bcff40..0055a3ee1 100644 --- a/crates/meilisearch/tests/upgrade/mod.rs +++ b/crates/meilisearch/tests/upgrade/mod.rs @@ -54,7 +54,7 @@ async fn version_requires_downgrade() { std::fs::create_dir_all(&db_path).unwrap(); let major = meilisearch_types::versioning::VERSION_MAJOR; let minor = meilisearch_types::versioning::VERSION_MINOR; - let patch = meilisearch_types::versioning::VERSION_PATCH.parse::().unwrap() + 1; + let patch = meilisearch_types::versioning::VERSION_PATCH + 1; std::fs::write(db_path.join("VERSION"), format!("{major}.{minor}.{patch}")).unwrap(); let options = Opt { experimental_dumpless_upgrade: true, ..default_settings }; let err = Server::new_with_options(options).await.map(|_| ()).unwrap_err(); diff --git a/crates/meilitool/src/upgrade/mod.rs b/crates/meilitool/src/upgrade/mod.rs index 82a57317c..87ce00772 100644 --- a/crates/meilitool/src/upgrade/mod.rs +++ b/crates/meilitool/src/upgrade/mod.rs @@ -49,15 +49,10 @@ impl OfflineUpgrade { const LAST_SUPPORTED_UPGRADE_TO_VERSION: &str = "1.12.7"; let upgrade_list = [ - ( - v1_9_to_v1_10 as fn(&Path, u32, u32, u32) -> Result<(), anyhow::Error>, - "1", - "10", - "0", - ), - (v1_10_to_v1_11, "1", "11", "0"), - (v1_11_to_v1_12, "1", "12", "0"), - (v1_12_to_v1_12_3, "1", "12", "3"), + (v1_9_to_v1_10 as fn(&Path, u32, u32, u32) -> Result<(), anyhow::Error>, 1, 10, 0), + (v1_10_to_v1_11, 1, 11, 0), + (v1_11_to_v1_12, 1, 12, 0), + (v1_12_to_v1_12_3, 1, 12, 3), ]; let no_upgrade: usize = upgrade_list.len(); @@ -95,13 +90,8 @@ impl OfflineUpgrade { if start_at == no_upgrade { println!("No upgrade operation to perform, writing VERSION file"); - create_version_file( - &self.db_path, - &target_major.to_string(), - &target_minor.to_string(), - &target_patch.to_string(), - ) - .context("while writing VERSION file after the upgrade")?; + create_version_file(&self.db_path, target_major, target_minor, target_patch) + .context("while writing VERSION file after the upgrade")?; println!("Success"); return Ok(()); } diff --git a/crates/milli/src/constants.rs b/crates/milli/src/constants.rs index 39b449661..dc88bdb37 100644 --- a/crates/milli/src/constants.rs +++ b/crates/milli/src/constants.rs @@ -1,6 +1,13 @@ -pub static VERSION_MAJOR: &str = env!("CARGO_PKG_VERSION_MAJOR"); -pub static VERSION_MINOR: &str = env!("CARGO_PKG_VERSION_MINOR"); -pub static VERSION_PATCH: &str = env!("CARGO_PKG_VERSION_PATCH"); +pub const VERSION_MAJOR: u32 = parse_u32(env!("CARGO_PKG_VERSION_MAJOR")); +pub const VERSION_MINOR: u32 = parse_u32(env!("CARGO_PKG_VERSION_MINOR")); +pub const VERSION_PATCH: u32 = parse_u32(env!("CARGO_PKG_VERSION_PATCH")); + +const fn parse_u32(s: &str) -> u32 { + match u32::from_str_radix(s, 10) { + Ok(version) => version, + Err(_) => panic!("could not parse as u32"), + } +} pub const RESERVED_VECTORS_FIELD_NAME: &str = "_vectors"; pub const RESERVED_GEO_FIELD_NAME: &str = "_geo"; diff --git a/crates/milli/src/index.rs b/crates/milli/src/index.rs index d87f8715f..d0cd5c862 100644 --- a/crates/milli/src/index.rs +++ b/crates/milli/src/index.rs @@ -3,8 +3,8 @@ use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::fs::File; use std::path::Path; -use heed::{types::*, DatabaseStat, WithoutTls}; -use heed::{CompactionOption, Database, RoTxn, RwTxn, Unspecified}; +use heed::types::*; +use heed::{CompactionOption, Database, DatabaseStat, RoTxn, RwTxn, Unspecified, WithoutTls}; use indexmap::IndexMap; use roaring::RoaringBitmap; use rstar::RTree; @@ -108,6 +108,7 @@ pub mod db_name { pub const VECTOR_ARROY: &str = "vector-arroy"; pub const DOCUMENTS: &str = "documents"; } +const NUMBER_OF_DBS: u32 = 25; #[derive(Clone)] pub struct Index { @@ -187,7 +188,7 @@ impl Index { ) -> Result { use db_name::*; - options.max_dbs(25); + options.max_dbs(NUMBER_OF_DBS); let env = unsafe { options.open(path) }?; let mut wtxn = env.write_txn()?; @@ -262,11 +263,7 @@ impl Index { if this.get_version(&wtxn)?.is_none() && creation { this.put_version( &mut wtxn, - ( - constants::VERSION_MAJOR.parse().unwrap(), - constants::VERSION_MINOR.parse().unwrap(), - constants::VERSION_PATCH.parse().unwrap(), - ), + (constants::VERSION_MAJOR, constants::VERSION_MINOR, constants::VERSION_PATCH), )?; } wtxn.commit()?; @@ -285,6 +282,76 @@ impl Index { Self::new_with_creation_dates(options, path, now, now, creation) } + /// Attempts to rollback the index at `path` to the version specified by `requested_version`. + pub fn rollback>( + mut options: heed::EnvOpenOptions, + path: P, + requested_version: (u32, u32, u32), + ) -> Result { + options.max_dbs(NUMBER_OF_DBS); + + // optimistically check if the index is already at the requested version. + let env = unsafe { options.open(path.as_ref()) }?; + let rtxn = env.read_txn()?; + let Some(main) = env.database_options().name(db_name::MAIN).open(&rtxn)? else { + return Err(crate::Error::InternalError(crate::InternalError::DatabaseMissingEntry { + db_name: db_name::MAIN, + key: None, + })); + }; + let rollback_version = + main.remap_types::().get(&rtxn, main_key::VERSION_KEY)?; + if rollback_version == Some(requested_version) { + return Ok(RollbackOutcome::NoRollback); + } + + // explicitly drop the environment before reopening it. + drop(rtxn); + drop(env); + + // really need to rollback then... + unsafe { options.flags(heed::EnvFlags::PREV_SNAPSHOT) }; + let env = unsafe { options.open(path) }?; + let mut wtxn = env.write_txn()?; + let Some(main) = env.database_options().name(db_name::MAIN).open(&wtxn)? else { + return Err(crate::Error::InternalError(crate::InternalError::DatabaseMissingEntry { + db_name: db_name::MAIN, + key: None, + })); + }; + + let main = main.remap_key_type::(); + + let Some(rollback_version) = + main.remap_data_type::().get(&wtxn, main_key::VERSION_KEY)? + else { + return Ok(RollbackOutcome::VersionMismatch { + requested_version, + rollback_version: None, + }); + }; + + if requested_version != rollback_version { + return Ok(RollbackOutcome::VersionMismatch { + requested_version, + rollback_version: Some(rollback_version), + }); + } + + // this is a bit of a trick to force a change in the index + // which is necessary to actually discard the next snapshot, replacing it with this transaction. + let now = time::OffsetDateTime::now_utc(); + main.remap_data_type::>().put( + &mut wtxn, + main_key::UPDATED_AT_KEY, + &OffsetDateTime(now), + )?; + + wtxn.commit()?; + + Ok(RollbackOutcome::Rollback) + } + fn set_creation_dates( env: &heed::Env, main: Database, @@ -375,7 +442,7 @@ impl Index { } /// Get the version of the database. `None` if it was never set. - pub(crate) fn get_version(&self, rtxn: &RoTxn<'_>) -> heed::Result> { + pub fn get_version(&self, rtxn: &RoTxn<'_>) -> heed::Result> { self.main.remap_types::().get(rtxn, main_key::VERSION_KEY) } @@ -1865,1419 +1932,39 @@ pub enum PrefixSearch { Disabled, } +#[derive(Debug)] +pub enum RollbackOutcome { + VersionMismatch { + requested_version: (u32, u32, u32), + rollback_version: Option<(u32, u32, u32)>, + }, + Rollback, + NoRollback, +} + +impl RollbackOutcome { + pub fn succeeded(&self) -> bool { + matches!(self, RollbackOutcome::Rollback | RollbackOutcome::NoRollback) + } +} + +impl std::fmt::Display for RollbackOutcome { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + RollbackOutcome::VersionMismatch { requested_version, rollback_version: Some(rollback_version) } => write!(f, "cannot rollback to the requested version\n - note: requested version is v{}.{}.{}\n - note: only possible to rollback to v{}.{}.{}", + requested_version.0, requested_version.1, requested_version.2, rollback_version.0, rollback_version.1, rollback_version.2), + RollbackOutcome::VersionMismatch { requested_version, rollback_version: None } => write!(f, "cannot rollback to the requested version\n - note: requested version is v{}.{}.{}\n - note: only possible to rollback to an unknown version", + requested_version.0, requested_version.1, requested_version.2), + RollbackOutcome::Rollback => f.write_str("rollback complete"), + RollbackOutcome::NoRollback => f.write_str("no rollback necessary"), + } + } +} + #[derive(Serialize, Deserialize)] #[serde(transparent)] struct OffsetDateTime(#[serde(with = "time::serde::rfc3339")] time::OffsetDateTime); #[cfg(test)] -pub(crate) mod tests { - use std::collections::HashSet; - use std::ops::Deref; - - use big_s::S; - use bumpalo::Bump; - use heed::{EnvOpenOptions, RwTxn}; - use maplit::btreemap; - use memmap2::Mmap; - use tempfile::TempDir; - - use crate::constants::RESERVED_GEO_FIELD_NAME; - use crate::error::{Error, InternalError}; - use crate::index::{DEFAULT_MIN_WORD_LEN_ONE_TYPO, DEFAULT_MIN_WORD_LEN_TWO_TYPOS}; - use crate::progress::Progress; - use crate::update::new::indexer; - use crate::update::settings::InnerIndexSettings; - use crate::update::{ - self, IndexDocumentsConfig, IndexDocumentsMethod, IndexerConfig, Setting, Settings, - }; - use crate::vector::settings::{EmbedderSource, EmbeddingSettings}; - use crate::vector::EmbeddingConfigs; - use crate::{ - db_snap, obkv_to_json, Filter, FilterableAttributesRule, Index, Search, SearchResult, - ThreadPoolNoAbortBuilder, - }; - - pub(crate) struct TempIndex { - pub inner: Index, - pub indexer_config: IndexerConfig, - pub index_documents_config: IndexDocumentsConfig, - _tempdir: TempDir, - } - - impl Deref for TempIndex { - type Target = Index; - - fn deref(&self) -> &Self::Target { - &self.inner - } - } - - impl TempIndex { - /// Creates a temporary index - pub fn new_with_map_size(size: usize) -> Self { - let options = EnvOpenOptions::new(); - let mut options = options.read_txn_without_tls(); - options.map_size(size); - let _tempdir = TempDir::new_in(".").unwrap(); - let inner = Index::new(options, _tempdir.path(), true).unwrap(); - let indexer_config = IndexerConfig::default(); - let index_documents_config = IndexDocumentsConfig::default(); - Self { inner, indexer_config, index_documents_config, _tempdir } - } - /// Creates a temporary index, with a default `4096 * 2000` size. This should be enough for - /// most tests. - pub fn new() -> Self { - Self::new_with_map_size(4096 * 2000) - } - - pub fn add_documents_using_wtxn<'t>( - &'t self, - wtxn: &mut RwTxn<'t>, - documents: Mmap, - ) -> Result<(), crate::error::Error> { - let local_pool; - let indexer_config = &self.indexer_config; - let pool = match &indexer_config.thread_pool { - Some(pool) => pool, - None => { - local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap(); - &local_pool - } - }; - - let rtxn = self.inner.read_txn()?; - let db_fields_ids_map = self.inner.fields_ids_map(&rtxn)?; - let mut new_fields_ids_map = db_fields_ids_map.clone(); - - let embedders = - InnerIndexSettings::from_index(&self.inner, &rtxn, None)?.embedding_configs; - let mut indexer = indexer::DocumentOperation::new(); - match self.index_documents_config.update_method { - IndexDocumentsMethod::ReplaceDocuments => { - indexer.replace_documents(&documents).unwrap() - } - IndexDocumentsMethod::UpdateDocuments => { - indexer.update_documents(&documents).unwrap() - } - } - - let indexer_alloc = Bump::new(); - let (document_changes, operation_stats, primary_key) = indexer.into_changes( - &indexer_alloc, - &self.inner, - &rtxn, - None, - &mut new_fields_ids_map, - &|| false, - Progress::default(), - )?; - - if let Some(error) = operation_stats.into_iter().find_map(|stat| stat.error) { - return Err(error.into()); - } - - pool.install(|| { - indexer::index( - wtxn, - &self.inner, - &crate::ThreadPoolNoAbortBuilder::new().build().unwrap(), - indexer_config.grenad_parameters(), - &db_fields_ids_map, - new_fields_ids_map, - primary_key, - &document_changes, - embedders, - &|| false, - &Progress::default(), - ) - }) - .unwrap()?; - - Ok(()) - } - - pub fn add_documents(&self, documents: Mmap) -> Result<(), crate::error::Error> { - let mut wtxn = self.write_txn().unwrap(); - self.add_documents_using_wtxn(&mut wtxn, documents)?; - wtxn.commit().unwrap(); - Ok(()) - } - - pub fn update_settings( - &self, - update: impl Fn(&mut Settings<'_, '_, '_>), - ) -> Result<(), crate::error::Error> { - let mut wtxn = self.write_txn().unwrap(); - self.update_settings_using_wtxn(&mut wtxn, update)?; - wtxn.commit().unwrap(); - Ok(()) - } - - pub fn update_settings_using_wtxn<'t>( - &'t self, - wtxn: &mut RwTxn<'t>, - update: impl Fn(&mut Settings<'_, '_, '_>), - ) -> Result<(), crate::error::Error> { - let mut builder = update::Settings::new(wtxn, &self.inner, &self.indexer_config); - update(&mut builder); - builder.execute(drop, || false)?; - Ok(()) - } - - pub fn delete_documents_using_wtxn<'t>( - &'t self, - wtxn: &mut RwTxn<'t>, - external_document_ids: Vec, - ) -> Result<(), crate::error::Error> { - let local_pool; - let indexer_config = &self.indexer_config; - let pool = match &indexer_config.thread_pool { - Some(pool) => pool, - None => { - local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap(); - &local_pool - } - }; - - let rtxn = self.inner.read_txn()?; - let db_fields_ids_map = self.inner.fields_ids_map(&rtxn)?; - let mut new_fields_ids_map = db_fields_ids_map.clone(); - - let embedders = - InnerIndexSettings::from_index(&self.inner, &rtxn, None)?.embedding_configs; - - let mut indexer = indexer::DocumentOperation::new(); - let external_document_ids: Vec<_> = - external_document_ids.iter().map(AsRef::as_ref).collect(); - indexer.delete_documents(external_document_ids.as_slice()); - - let indexer_alloc = Bump::new(); - let (document_changes, operation_stats, primary_key) = indexer.into_changes( - &indexer_alloc, - &self.inner, - &rtxn, - None, - &mut new_fields_ids_map, - &|| false, - Progress::default(), - )?; - - if let Some(error) = operation_stats.into_iter().find_map(|stat| stat.error) { - return Err(error.into()); - } - - pool.install(|| { - indexer::index( - wtxn, - &self.inner, - &crate::ThreadPoolNoAbortBuilder::new().build().unwrap(), - indexer_config.grenad_parameters(), - &db_fields_ids_map, - new_fields_ids_map, - primary_key, - &document_changes, - embedders, - &|| false, - &Progress::default(), - ) - }) - .unwrap()?; - - Ok(()) - } - - pub fn delete_documents(&self, external_document_ids: Vec) { - let mut wtxn = self.write_txn().unwrap(); - - self.delete_documents_using_wtxn(&mut wtxn, external_document_ids).unwrap(); - - wtxn.commit().unwrap(); - } - - pub fn delete_document(&self, external_document_id: &str) { - self.delete_documents(vec![external_document_id.to_string()]) - } - } - - #[test] - fn aborting_indexation() { - use std::sync::atomic::AtomicBool; - use std::sync::atomic::Ordering::Relaxed; - - let index = TempIndex::new(); - let mut wtxn = index.inner.write_txn().unwrap(); - let should_abort = AtomicBool::new(false); - - let local_pool; - let indexer_config = &index.indexer_config; - let pool = match &indexer_config.thread_pool { - Some(pool) => pool, - None => { - local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap(); - &local_pool - } - }; - - let rtxn = index.inner.read_txn().unwrap(); - let db_fields_ids_map = index.inner.fields_ids_map(&rtxn).unwrap(); - let mut new_fields_ids_map = db_fields_ids_map.clone(); - - let embedders = EmbeddingConfigs::default(); - let mut indexer = indexer::DocumentOperation::new(); - let payload = documents!([ - { "id": 1, "name": "kevin" }, - { "id": 2, "name": "bob", "age": 20 }, - { "id": 2, "name": "bob", "age": 20 }, - ]); - indexer.replace_documents(&payload).unwrap(); - - let indexer_alloc = Bump::new(); - let (document_changes, _operation_stats, primary_key) = indexer - .into_changes( - &indexer_alloc, - &index.inner, - &rtxn, - None, - &mut new_fields_ids_map, - &|| false, - Progress::default(), - ) - .unwrap(); - - should_abort.store(true, Relaxed); - - let err = pool - .install(|| { - indexer::index( - &mut wtxn, - &index.inner, - &crate::ThreadPoolNoAbortBuilder::new().build().unwrap(), - indexer_config.grenad_parameters(), - &db_fields_ids_map, - new_fields_ids_map, - primary_key, - &document_changes, - embedders, - &|| should_abort.load(Relaxed), - &Progress::default(), - ) - }) - .unwrap() - .unwrap_err(); - - assert!(matches!(err, Error::InternalError(InternalError::AbortedIndexation))); - } - - #[test] - fn initial_field_distribution() { - let index = TempIndex::new(); - index - .add_documents(documents!([ - { "id": 1, "name": "kevin" }, - { "id": 2, "name": "bob", "age": 20 }, - { "id": 2, "name": "bob", "age": 20 }, - ])) - .unwrap(); - - db_snap!(index, field_distribution, @r###" - age 1 | - id 2 | - name 2 | - "###); - - db_snap!(index, word_docids, - @r###" - 1 [0, ] - 2 [1, ] - 20 [1, ] - bob [1, ] - kevin [0, ] - "### - ); - - // we add all the documents a second time. we are supposed to get the same - // field_distribution in the end - index - .add_documents(documents!([ - { "id": 1, "name": "kevin" }, - { "id": 2, "name": "bob", "age": 20 }, - { "id": 2, "name": "bob", "age": 20 }, - ])) - .unwrap(); - - db_snap!(index, field_distribution, - @r###" - age 1 | - id 2 | - name 2 | - "### - ); - - // then we update a document by removing one field and another by adding one field - index - .add_documents(documents!([ - { "id": 1, "name": "kevin", "has_dog": true }, - { "id": 2, "name": "bob" } - ])) - .unwrap(); - - db_snap!(index, field_distribution, - @r###" - has_dog 1 | - id 2 | - name 2 | - "### - ); - } - - #[test] - fn put_and_retrieve_disable_typo() { - let index = TempIndex::new(); - let mut txn = index.write_txn().unwrap(); - // default value is true - assert!(index.authorize_typos(&txn).unwrap()); - // set to false - index.put_authorize_typos(&mut txn, false).unwrap(); - txn.commit().unwrap(); - - let txn = index.read_txn().unwrap(); - assert!(!index.authorize_typos(&txn).unwrap()); - } - - #[test] - fn set_min_word_len_for_typos() { - let index = TempIndex::new(); - let mut txn = index.write_txn().unwrap(); - - assert_eq!(index.min_word_len_one_typo(&txn).unwrap(), DEFAULT_MIN_WORD_LEN_ONE_TYPO); - assert_eq!(index.min_word_len_two_typos(&txn).unwrap(), DEFAULT_MIN_WORD_LEN_TWO_TYPOS); - - index.put_min_word_len_one_typo(&mut txn, 3).unwrap(); - index.put_min_word_len_two_typos(&mut txn, 15).unwrap(); - - txn.commit().unwrap(); - - let txn = index.read_txn().unwrap(); - assert_eq!(index.min_word_len_one_typo(&txn).unwrap(), 3); - assert_eq!(index.min_word_len_two_typos(&txn).unwrap(), 15); - } - - #[test] - fn add_documents_and_set_searchable_fields() { - let index = TempIndex::new(); - index - .add_documents(documents!([ - { "id": 1, "doggo": "kevin" }, - { "id": 2, "doggo": { "name": "bob", "age": 20 } }, - { "id": 3, "name": "jean", "age": 25 }, - ])) - .unwrap(); - index - .update_settings(|settings| { - settings.set_searchable_fields(vec![S("doggo"), S("name")]); - }) - .unwrap(); - - // ensure we get the right real searchable fields + user defined searchable fields - let rtxn = index.read_txn().unwrap(); - - let real = index.searchable_fields(&rtxn).unwrap(); - assert_eq!(real, &["doggo", "name", "doggo.name", "doggo.age"]); - - let user_defined = index.user_defined_searchable_fields(&rtxn).unwrap().unwrap(); - assert_eq!(user_defined, &["doggo", "name"]); - } - - #[test] - fn set_searchable_fields_and_add_documents() { - let index = TempIndex::new(); - - index - .update_settings(|settings| { - settings.set_searchable_fields(vec![S("doggo"), S("name")]); - }) - .unwrap(); - - // ensure we get the right real searchable fields + user defined searchable fields - let rtxn = index.read_txn().unwrap(); - - let real = index.searchable_fields(&rtxn).unwrap(); - assert!(real.is_empty()); - let user_defined = index.user_defined_searchable_fields(&rtxn).unwrap().unwrap(); - assert_eq!(user_defined, &["doggo", "name"]); - - index - .add_documents(documents!([ - { "id": 1, "doggo": "kevin" }, - { "id": 2, "doggo": { "name": "bob", "age": 20 } }, - { "id": 3, "name": "jean", "age": 25 }, - ])) - .unwrap(); - - // ensure we get the right real searchable fields + user defined searchable fields - let rtxn = index.read_txn().unwrap(); - - let real = index.searchable_fields(&rtxn).unwrap(); - assert_eq!(real, &["doggo", "name", "doggo.name", "doggo.age"]); - - let user_defined = index.user_defined_searchable_fields(&rtxn).unwrap().unwrap(); - assert_eq!(user_defined, &["doggo", "name"]); - } - - #[test] - fn test_basic_geo_bounding_box() { - let index = TempIndex::new(); - - index - .update_settings(|settings| { - settings.set_filterable_fields(vec![FilterableAttributesRule::Field( - RESERVED_GEO_FIELD_NAME.to_string(), - )]); - }) - .unwrap(); - index - .add_documents(documents!([ - { "id": 0, RESERVED_GEO_FIELD_NAME: { "lat": "0", "lng": "0" } }, - { "id": 1, RESERVED_GEO_FIELD_NAME: { "lat": 0, "lng": "-175" } }, - { "id": 2, RESERVED_GEO_FIELD_NAME: { "lat": "0", "lng": 175 } }, - { "id": 3, RESERVED_GEO_FIELD_NAME: { "lat": 85, "lng": 0 } }, - { "id": 4, RESERVED_GEO_FIELD_NAME: { "lat": "-85", "lng": "0" } }, - ])) - .unwrap(); - - // ensure we get the right real searchable fields + user defined searchable fields - let rtxn = index.read_txn().unwrap(); - let mut search = index.search(&rtxn); - - // exact match a document - let search_result = search - .filter(Filter::from_str("_geoBoundingBox([0, 0], [0, 0])").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0]>"); - - // match a document in the middle of the rectangle - let search_result = search - .filter(Filter::from_str("_geoBoundingBox([10, 10], [-10, -10])").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0]>"); - - // select everything - let search_result = search - .filter(Filter::from_str("_geoBoundingBox([90, 180], [-90, -180])").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0, 1, 2, 3, 4]>"); - - // go on the edge of the longitude - let search_result = search - .filter(Filter::from_str("_geoBoundingBox([0, -170], [0, 180])").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[1]>"); - - // go on the other edge of the longitude - let search_result = search - .filter(Filter::from_str("_geoBoundingBox([0, -180], [0, 170])").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[2]>"); - - // wrap around the longitude - let search_result = search - .filter(Filter::from_str("_geoBoundingBox([0, -170], [0, 170])").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[1, 2]>"); - - // go on the edge of the latitude - let search_result = search - .filter(Filter::from_str("_geoBoundingBox([90, 0], [80, 0])").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[3]>"); - - // go on the edge of the latitude - let search_result = search - .filter(Filter::from_str("_geoBoundingBox([-80, 0], [-90, 0])").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[4]>"); - - // the requests that don't make sense - - // try to wrap around the latitude - let error = search - .filter(Filter::from_str("_geoBoundingBox([-80, 0], [80, 0])").unwrap().unwrap()) - .execute() - .unwrap_err(); - insta::assert_snapshot!( - error, - @r###" - The top latitude `-80` is below the bottom latitude `80`. - 32:33 _geoBoundingBox([-80, 0], [80, 0]) - "### - ); - - // send a top latitude lower than the bottow latitude - let error = search - .filter(Filter::from_str("_geoBoundingBox([-10, 0], [10, 0])").unwrap().unwrap()) - .execute() - .unwrap_err(); - insta::assert_snapshot!( - error, - @r###" - The top latitude `-10` is below the bottom latitude `10`. - 32:33 _geoBoundingBox([-10, 0], [10, 0]) - "### - ); - } - - #[test] - fn test_contains() { - let index = TempIndex::new(); - - index - .update_settings(|settings| { - settings.set_filterable_fields(vec![FilterableAttributesRule::Field( - "doggo".to_string(), - )]); - }) - .unwrap(); - index - .add_documents(documents!([ - { "id": 0, "doggo": "kefir" }, - { "id": 1, "doggo": "kefirounet" }, - { "id": 2, "doggo": "kefkef" }, - { "id": 3, "doggo": "fifir" }, - { "id": 4, "doggo": "boubou" }, - { "id": 5 }, - ])) - .unwrap(); - - let rtxn = index.read_txn().unwrap(); - let mut search = index.search(&rtxn); - let search_result = search - .filter(Filter::from_str("doggo CONTAINS kefir").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0, 1]>"); - let mut search = index.search(&rtxn); - let search_result = search - .filter(Filter::from_str("doggo CONTAINS KEF").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0, 1, 2]>"); - let mut search = index.search(&rtxn); - let search_result = search - .filter(Filter::from_str("doggo NOT CONTAINS fir").unwrap().unwrap()) - .execute() - .unwrap(); - insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[2, 4, 5]>"); - } - - #[test] - fn replace_documents_external_ids_and_soft_deletion_check() { - let index = TempIndex::new(); - - index - .update_settings(|settings| { - settings.set_primary_key("id".to_owned()); - settings.set_filterable_fields(vec![FilterableAttributesRule::Field( - "doggo".to_string(), - )]); - }) - .unwrap(); - - let mut docs = vec![]; - for i in 0..4 { - docs.push(serde_json::json!( - { "id": i, "doggo": i } - )); - } - index.add_documents(documents!(docs)).unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, 3, ]"); - db_snap!(index, external_documents_ids, 1, @r###" - docids: - 0 0 - 1 1 - 2 2 - 3 3 - "###); - db_snap!(index, facet_id_f64_docids, 1, @r###" - 1 0 0 1 [0, ] - 1 0 1 1 [1, ] - 1 0 2 1 [2, ] - 1 0 3 1 [3, ] - "###); - - let mut docs = vec![]; - for i in 0..3 { - docs.push(serde_json::json!( - { "id": i, "doggo": i + 1 } - )); - } - index.add_documents(documents!(docs)).unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, 3, ]"); - db_snap!(index, external_documents_ids, 2, @r###" - docids: - 0 0 - 1 1 - 2 2 - 3 3 - "###); - db_snap!(index, facet_id_f64_docids, 2, @r###" - 1 0 1 1 [0, ] - 1 0 2 1 [1, ] - 1 0 3 1 [2, 3, ] - "###); - - index - .add_documents(documents!([{ "id": 3, "doggo": 4 }, { "id": 3, "doggo": 5 },{ "id": 3, "doggo": 4 }])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, 3, ]"); - db_snap!(index, external_documents_ids, 3, @r###" - docids: - 0 0 - 1 1 - 2 2 - 3 3 - "###); - db_snap!(index, facet_id_f64_docids, 3, @r###" - 1 0 1 1 [0, ] - 1 0 2 1 [1, ] - 1 0 3 1 [2, ] - 1 0 4 1 [3, ] - "###); - - index - .update_settings(|settings| { - settings.set_distinct_field("id".to_owned()); - }) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, 3, ]"); - db_snap!(index, external_documents_ids, 3, @r###" - docids: - 0 0 - 1 1 - 2 2 - 3 3 - "###); - db_snap!(index, facet_id_f64_docids, 3, @r###" - 0 0 0 1 [0, ] - 0 0 1 1 [1, ] - 0 0 2 1 [2, ] - 0 0 3 1 [3, ] - 1 0 1 1 [0, ] - 1 0 2 1 [1, ] - 1 0 3 1 [2, ] - 1 0 4 1 [3, ] - "###); - } - - #[test] - fn bug_3021_first() { - // https://github.com/meilisearch/meilisearch/issues/3021 - let mut index = TempIndex::new(); - index.index_documents_config.update_method = IndexDocumentsMethod::ReplaceDocuments; - - index - .update_settings(|settings| { - settings.set_primary_key("primary_key".to_owned()); - }) - .unwrap(); - - index - .add_documents(documents!([ - { "primary_key": 38 }, - { "primary_key": 34 } - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, ]"); - db_snap!(index, external_documents_ids, 1, @r###" - docids: - 34 1 - 38 0 - "###); - - index.delete_document("34"); - - db_snap!(index, documents_ids, @"[0, ]"); - db_snap!(index, external_documents_ids, 2, @r###" - docids: - 38 0 - "###); - - index - .update_settings(|s| { - s.set_searchable_fields(vec![]); - }) - .unwrap(); - - // The key point of the test is to verify that the external documents ids - // do not contain any entry for previously soft-deleted document ids - db_snap!(index, documents_ids, @"[0, ]"); - db_snap!(index, external_documents_ids, 3, @r###" - docids: - 38 0 - "###); - - // So that this document addition works correctly now. - // It would be wrongly interpreted as a replacement before - index.add_documents(documents!({ "primary_key": 34 })).unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, ]"); - db_snap!(index, external_documents_ids, 4, @r###" - docids: - 34 1 - 38 0 - "###); - - // We do the test again, but deleting the document with id 0 instead of id 1 now - index.delete_document("38"); - - db_snap!(index, documents_ids, @"[1, ]"); - db_snap!(index, external_documents_ids, 5, @r###" - docids: - 34 1 - "###); - - index - .update_settings(|s| { - s.set_searchable_fields(vec!["primary_key".to_owned()]); - }) - .unwrap(); - - db_snap!(index, documents_ids, @"[1, ]"); - db_snap!(index, external_documents_ids, 6, @r###" - docids: - 34 1 - "###); - - // And adding lots of documents afterwards instead of just one. - // These extra subtests don't add much, but it's better than nothing. - index - .add_documents(documents!([ - { "primary_key": 38 }, - { "primary_key": 39 }, - { "primary_key": 41 }, - { "primary_key": 40 }, - { "primary_key": 41 }, - { "primary_key": 42 }, - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, 3, 4, 5, ]"); - db_snap!(index, external_documents_ids, 7, @r###" - docids: - 34 1 - 38 0 - 39 2 - 40 4 - 41 3 - 42 5 - "###); - } - - #[test] - fn simple_delete() { - let mut index = TempIndex::new(); - index.index_documents_config.update_method = IndexDocumentsMethod::UpdateDocuments; - index - .add_documents(documents!([ - { "id": 30 }, - { "id": 34 } - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, ]"); - db_snap!(index, external_documents_ids, 1, @r###" - docids: - 30 0 - 34 1"###); - - index.delete_document("34"); - - db_snap!(index, documents_ids, @"[0, ]"); - db_snap!(index, external_documents_ids, 2, @r###" - docids: - 30 0 - "###); - } - - #[test] - fn bug_3021_second() { - // https://github.com/meilisearch/meilisearch/issues/3021 - let mut index = TempIndex::new(); - index.index_documents_config.update_method = IndexDocumentsMethod::UpdateDocuments; - - index - .update_settings(|settings| { - settings.set_primary_key("primary_key".to_owned()); - }) - .unwrap(); - - index - .add_documents(documents!([ - { "primary_key": 30 }, - { "primary_key": 34 } - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, ]"); - db_snap!(index, external_documents_ids, 1, @r###" - docids: - 30 0 - 34 1 - "###); - - index.delete_document("34"); - - db_snap!(index, documents_ids, @"[0, ]"); - db_snap!(index, external_documents_ids, 2, @r###" - docids: - 30 0 - "###); - - index - .update_settings(|s| { - s.set_searchable_fields(vec![]); - }) - .unwrap(); - - // The key point of the test is to verify that the external documents ids - // do not contain any entry for previously soft-deleted document ids - db_snap!(index, documents_ids, @"[0, ]"); - db_snap!(index, external_documents_ids, 3, @r###" - docids: - 30 0 - "###); - - // So that when we add a new document - index.add_documents(documents!({ "primary_key": 35, "b": 2 })).unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, ]"); - // The external documents ids don't have several external ids pointing to the same - // internal document id - db_snap!(index, external_documents_ids, 4, @r###" - docids: - 30 0 - 35 1 - "###); - - // And when we add 34 again, we don't replace document 35 - index.add_documents(documents!({ "primary_key": 34, "a": 1 })).unwrap(); - - // And document 35 still exists, is not deleted - db_snap!(index, documents_ids, @"[0, 1, 2, ]"); - db_snap!(index, external_documents_ids, 5, @r###" - docids: - 30 0 - 34 2 - 35 1 - "###); - - let rtxn = index.read_txn().unwrap(); - let (_docid, obkv) = index.documents(&rtxn, [0]).unwrap()[0]; - let json = obkv_to_json(&[0, 1, 2], &index.fields_ids_map(&rtxn).unwrap(), obkv).unwrap(); - insta::assert_debug_snapshot!(json, @r###" - { - "primary_key": Number(30), - } - "###); - - // Furthermore, when we retrieve document 34, it is not the result of merging 35 with 34 - let (_docid, obkv) = index.documents(&rtxn, [2]).unwrap()[0]; - let json = obkv_to_json(&[0, 1, 2], &index.fields_ids_map(&rtxn).unwrap(), obkv).unwrap(); - insta::assert_debug_snapshot!(json, @r###" - { - "primary_key": Number(34), - "a": Number(1), - } - "###); - - drop(rtxn); - - // Add new documents again - index - .add_documents( - documents!([{ "primary_key": 37 }, { "primary_key": 38 }, { "primary_key": 39 }]), - ) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, 3, 4, 5, ]"); - db_snap!(index, external_documents_ids, 6, @r###" - docids: - 30 0 - 34 2 - 35 1 - 37 3 - 38 4 - 39 5 - "###); - } - - #[test] - fn bug_3021_third() { - // https://github.com/meilisearch/meilisearch/issues/3021 - let mut index = TempIndex::new(); - index.index_documents_config.update_method = IndexDocumentsMethod::UpdateDocuments; - - index - .update_settings(|settings| { - settings.set_primary_key("primary_key".to_owned()); - }) - .unwrap(); - - index - .add_documents(documents!([ - { "primary_key": 3 }, - { "primary_key": 4 }, - { "primary_key": 5 } - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, ]"); - db_snap!(index, external_documents_ids, 1, @r###" - docids: - 3 0 - 4 1 - 5 2 - "###); - - index.delete_document("3"); - - db_snap!(index, documents_ids, @"[1, 2, ]"); - db_snap!(index, external_documents_ids, 2, @r###" - docids: - 4 1 - 5 2 - "###); - - index.add_documents(documents!([{ "primary_key": "4", "a": 2 }])).unwrap(); - - db_snap!(index, documents_ids, @"[1, 2, ]"); - db_snap!(index, external_documents_ids, 2, @r###" - docids: - 4 1 - 5 2 - "###); - - index - .add_documents(documents!([ - { "primary_key": "3" }, - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, ]"); - db_snap!(index, external_documents_ids, 2, @r###" - docids: - 3 0 - 4 1 - 5 2 - "###); - } - - #[test] - fn bug_3021_fourth() { - // https://github.com/meilisearch/meilisearch/issues/3021 - let mut index = TempIndex::new(); - index.index_documents_config.update_method = IndexDocumentsMethod::UpdateDocuments; - - index - .update_settings(|settings| { - settings.set_primary_key("primary_key".to_owned()); - }) - .unwrap(); - - index - .add_documents(documents!([ - { "primary_key": 11 }, - { "primary_key": 4 }, - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, ]"); - db_snap!(index, external_documents_ids, @r###" - docids: - 11 0 - 4 1 - "###); - db_snap!(index, fields_ids_map, @r###" - 0 primary_key | - "###); - db_snap!(index, searchable_fields, @r###"["primary_key"]"###); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - 0 0 | - "###); - - index - .add_documents(documents!([ - { "primary_key": 4, "a": 0 }, - { "primary_key": 1 }, - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, ]"); - db_snap!(index, external_documents_ids, @r###" - docids: - 1 2 - 11 0 - 4 1 - "###); - db_snap!(index, fields_ids_map, @r###" - 0 primary_key | - 1 a | - "###); - db_snap!(index, searchable_fields, @r###"["primary_key", "a"]"###); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - 0 0 | - 1 0 | - "###); - - index.delete_documents(Default::default()); - - db_snap!(index, documents_ids, @"[0, 1, 2, ]"); - db_snap!(index, external_documents_ids, @r###" - docids: - 1 2 - 11 0 - 4 1 - "###); - db_snap!(index, fields_ids_map, @r###" - 0 primary_key | - 1 a | - "###); - db_snap!(index, searchable_fields, @r###"["primary_key", "a"]"###); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - 0 0 | - 1 0 | - "###); - - index - .add_documents(documents!([ - { "primary_key": 4, "a": 1 }, - { "primary_key": 1, "a": 0 }, - ])) - .unwrap(); - - db_snap!(index, documents_ids, @"[0, 1, 2, ]"); - db_snap!(index, external_documents_ids, @r###" - docids: - 1 2 - 11 0 - 4 1 - "###); - db_snap!(index, fields_ids_map, @r###" - 0 primary_key | - 1 a | - "###); - db_snap!(index, searchable_fields, @r###"["primary_key", "a"]"###); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - 0 0 | - 1 0 | - "###); - - let rtxn = index.read_txn().unwrap(); - let search = Search::new(&rtxn, &index); - let SearchResult { - matching_words: _, - candidates: _, - document_scores: _, - mut documents_ids, - degraded: _, - used_negative_operator: _, - } = search.execute().unwrap(); - let primary_key_id = index.fields_ids_map(&rtxn).unwrap().id("primary_key").unwrap(); - documents_ids.sort_unstable(); - let docs = index.documents(&rtxn, documents_ids).unwrap(); - let mut all_ids = HashSet::new(); - for (_docid, obkv) in docs { - let id = obkv.get(primary_key_id).unwrap(); - assert!(all_ids.insert(id)); - } - } - - #[test] - fn bug_3007() { - // https://github.com/meilisearch/meilisearch/issues/3007 - - use crate::error::{GeoError, UserError}; - let index = TempIndex::new(); - - // Given is an index with a geo field NOT contained in the sortable_fields of the settings - index - .update_settings(|settings| { - settings.set_primary_key("id".to_string()); - settings.set_filterable_fields(vec![FilterableAttributesRule::Field( - RESERVED_GEO_FIELD_NAME.to_string(), - )]); - }) - .unwrap(); - - // happy path - index - .add_documents( - documents!({ "id" : 5, RESERVED_GEO_FIELD_NAME: {"lat": 12.0, "lng": 11.0}}), - ) - .unwrap(); - - db_snap!(index, geo_faceted_documents_ids); - - // both are unparseable, we expect GeoError::BadLatitudeAndLongitude - let err1 = index - .add_documents( - documents!({ "id" : 6, RESERVED_GEO_FIELD_NAME: {"lat": "unparseable", "lng": "unparseable"}}), - ) - .unwrap_err(); - match err1 { - Error::UserError(UserError::InvalidGeoField(err)) => match *err { - GeoError::BadLatitudeAndLongitude { .. } => (), - otherwise => { - panic!("err1 is not a BadLatitudeAndLongitude error but rather a {otherwise:?}") - } - }, - _ => panic!("err1 is not a BadLatitudeAndLongitude error but rather a {err1:?}"), - } - - db_snap!(index, geo_faceted_documents_ids); // ensure that no more document was inserted - } - - #[test] - fn unexpected_extra_fields_in_geo_field() { - let index = TempIndex::new(); - - index - .update_settings(|settings| { - settings.set_primary_key("id".to_string()); - settings.set_filterable_fields(vec![FilterableAttributesRule::Field( - RESERVED_GEO_FIELD_NAME.to_string(), - )]); - }) - .unwrap(); - - let err = index - .add_documents( - documents!({ "id" : "doggo", RESERVED_GEO_FIELD_NAME: { "lat": 1, "lng": 2, "doggo": "are the best" }}), - ) - .unwrap_err(); - insta::assert_snapshot!(err, @r###"The `_geo` field in the document with the id: `"doggo"` contains the following unexpected fields: `{"doggo":"are the best"}`."###); - - db_snap!(index, geo_faceted_documents_ids); // ensure that no documents were inserted - - // multiple fields and complex values - let err = index - .add_documents( - documents!({ "id" : "doggo", RESERVED_GEO_FIELD_NAME: { "lat": 1, "lng": 2, "doggo": "are the best", "and": { "all": ["cats", { "are": "beautiful" } ] } } }), - ) - .unwrap_err(); - insta::assert_snapshot!(err, @r###"The `_geo` field in the document with the id: `"doggo"` contains the following unexpected fields: `{"and":{"all":["cats",{"are":"beautiful"}]},"doggo":"are the best"}`."###); - - db_snap!(index, geo_faceted_documents_ids); // ensure that no documents were inserted - } - - #[test] - fn swapping_searchable_attributes() { - // See https://github.com/meilisearch/meilisearch/issues/4484 - - let index = TempIndex::new(); - - index - .update_settings(|settings| { - settings.set_searchable_fields(vec![S("name")]); - settings.set_filterable_fields(vec![FilterableAttributesRule::Field( - "age".to_string(), - )]); - }) - .unwrap(); - - index - .add_documents(documents!({ "id": 1, "name": "Many", "age": 28, "realName": "Maxime" })) - .unwrap(); - db_snap!(index, fields_ids_map, @r###" - 0 id | - 1 name | - 2 age | - 3 realName | - "###); - db_snap!(index, searchable_fields, @r###"["name"]"###); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - 1 0 | - "###); - - index - .update_settings(|settings| { - settings.set_searchable_fields(vec![S("name"), S("realName")]); - settings.set_filterable_fields(vec![FilterableAttributesRule::Field( - "age".to_string(), - )]); - }) - .unwrap(); - - // The order of the field id map shouldn't change - db_snap!(index, fields_ids_map, @r###" - 0 id | - 1 name | - 2 age | - 3 realName | - "###); - db_snap!(index, searchable_fields, @r###"["name", "realName"]"###); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - 1 0 | - 3 1 | - "###); - } - - #[test] - fn attribute_weights_after_swapping_searchable_attributes() { - // See https://github.com/meilisearch/meilisearch/issues/4484 - - let index = TempIndex::new(); - - index - .update_settings(|settings| { - settings.set_searchable_fields(vec![S("name"), S("beverage")]); - }) - .unwrap(); - - index - .add_documents(documents!([ - { "id": 0, "name": "kefir", "beverage": "water" }, - { "id": 1, "name": "tamo", "beverage": "kefir" } - ])) - .unwrap(); - - let rtxn = index.read_txn().unwrap(); - let mut search = index.search(&rtxn); - let results = search.query("kefir").execute().unwrap(); - - // We should find kefir the dog first - insta::assert_debug_snapshot!(results.documents_ids, @r###" - [ - 0, - 1, - ] - "###); - - index - .update_settings(|settings| { - settings.set_searchable_fields(vec![S("beverage"), S("name")]); - }) - .unwrap(); - - let rtxn = index.read_txn().unwrap(); - let mut search = index.search(&rtxn); - let results = search.query("kefir").execute().unwrap(); - - // We should find tamo first - insta::assert_debug_snapshot!(results.documents_ids, @r###" - [ - 1, - 0, - ] - "###); - } - - #[test] - fn vectors_are_never_indexed_as_searchable_or_filterable() { - let index = TempIndex::new(); - - index - .add_documents(documents!([ - { "id": 0, "_vectors": { "doggo": [2345] } }, - { "id": 1, "_vectors": { "doggo": [6789] } }, - ])) - .unwrap(); - - db_snap!(index, fields_ids_map, @r###" - 0 id | - 1 _vectors | - "###); - db_snap!(index, searchable_fields, @r###"["id"]"###); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - 0 0 | - "###); - - let rtxn = index.read_txn().unwrap(); - let mut search = index.search(&rtxn); - let results = search.query("2345").execute().unwrap(); - assert!(results.candidates.is_empty()); - drop(rtxn); - - index - .update_settings(|settings| { - settings.set_searchable_fields(vec![S("_vectors"), S("_vectors.doggo")]); - settings.set_filterable_fields(vec![ - FilterableAttributesRule::Field("_vectors".to_string()), - FilterableAttributesRule::Field("_vectors.doggo".to_string()), - ]); - }) - .unwrap(); - - db_snap!(index, fields_ids_map, @r###" - 0 id | - 1 _vectors | - "###); - db_snap!(index, searchable_fields, @"[]"); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - "###); - - let rtxn = index.read_txn().unwrap(); - let mut search = index.search(&rtxn); - let results = search.query("2345").execute().unwrap(); - assert!(results.candidates.is_empty()); - - let mut search = index.search(&rtxn); - let results = search - .filter(Filter::from_str("_vectors.doggo = 6789").unwrap().unwrap()) - .execute() - .unwrap(); - assert!(results.candidates.is_empty()); - - index - .update_settings(|settings| { - settings.set_embedder_settings(btreemap! { - S("doggo") => Setting::Set(EmbeddingSettings { - dimensions: Setting::Set(1), - source: Setting::Set(EmbedderSource::UserProvided), - ..EmbeddingSettings::default()}), - }); - }) - .unwrap(); - - db_snap!(index, fields_ids_map, @r###" - 0 id | - 1 _vectors | - "###); - db_snap!(index, searchable_fields, @"[]"); - db_snap!(index, fieldids_weights_map, @r###" - fid weight - "###); - - let rtxn = index.read_txn().unwrap(); - let mut search = index.search(&rtxn); - let results = search.query("2345").execute().unwrap(); - assert!(results.candidates.is_empty()); - - let mut search = index.search(&rtxn); - let results = search - .filter(Filter::from_str("_vectors.doggo = 6789").unwrap().unwrap()) - .execute() - .unwrap(); - assert!(results.candidates.is_empty()); - } -} +#[path = "test_index.rs"] +pub(crate) mod tests; diff --git a/crates/milli/src/snapshots/index.rs/bug_3007/geo_faceted_documents_ids.snap b/crates/milli/src/snapshots/index.rs/bug_3007/geo_faceted_documents_ids.snap deleted file mode 100644 index f9ebc0c20..000000000 --- a/crates/milli/src/snapshots/index.rs/bug_3007/geo_faceted_documents_ids.snap +++ /dev/null @@ -1,4 +0,0 @@ ---- -source: milli/src/index.rs ---- -[0, ] diff --git a/crates/milli/src/snapshots/index.rs/unexpected_extra_fields_in_geo_field/geo_faceted_documents_ids.snap b/crates/milli/src/snapshots/index.rs/unexpected_extra_fields_in_geo_field/geo_faceted_documents_ids.snap deleted file mode 100644 index 89fb1856a..000000000 --- a/crates/milli/src/snapshots/index.rs/unexpected_extra_fields_in_geo_field/geo_faceted_documents_ids.snap +++ /dev/null @@ -1,4 +0,0 @@ ---- -source: milli/src/index.rs ---- -[] diff --git a/crates/milli/src/snapshots/test_index.rs/bug_3007/geo_faceted_documents_ids.snap b/crates/milli/src/snapshots/test_index.rs/bug_3007/geo_faceted_documents_ids.snap new file mode 100644 index 000000000..28f66783a --- /dev/null +++ b/crates/milli/src/snapshots/test_index.rs/bug_3007/geo_faceted_documents_ids.snap @@ -0,0 +1,4 @@ +--- +source: crates/milli/src/test_index.rs +--- +[0, ] diff --git a/crates/milli/src/snapshots/test_index.rs/unexpected_extra_fields_in_geo_field/geo_faceted_documents_ids.snap b/crates/milli/src/snapshots/test_index.rs/unexpected_extra_fields_in_geo_field/geo_faceted_documents_ids.snap new file mode 100644 index 000000000..72a8cceb6 --- /dev/null +++ b/crates/milli/src/snapshots/test_index.rs/unexpected_extra_fields_in_geo_field/geo_faceted_documents_ids.snap @@ -0,0 +1,4 @@ +--- +source: crates/milli/src/test_index.rs +--- +[] diff --git a/crates/milli/src/test_index.rs b/crates/milli/src/test_index.rs new file mode 100644 index 000000000..7759b3e18 --- /dev/null +++ b/crates/milli/src/test_index.rs @@ -0,0 +1,1399 @@ +use std::collections::HashSet; +use std::ops::Deref; + +use big_s::S; +use bumpalo::Bump; +use heed::{EnvOpenOptions, RwTxn}; +use maplit::btreemap; +use memmap2::Mmap; +use tempfile::TempDir; + +use crate::constants::RESERVED_GEO_FIELD_NAME; +use crate::error::{Error, InternalError}; +use crate::index::{DEFAULT_MIN_WORD_LEN_ONE_TYPO, DEFAULT_MIN_WORD_LEN_TWO_TYPOS}; +use crate::progress::Progress; +use crate::update::new::indexer; +use crate::update::settings::InnerIndexSettings; +use crate::update::{ + self, IndexDocumentsConfig, IndexDocumentsMethod, IndexerConfig, Setting, Settings, +}; +use crate::vector::settings::{EmbedderSource, EmbeddingSettings}; +use crate::vector::EmbeddingConfigs; +use crate::{ + db_snap, obkv_to_json, Filter, FilterableAttributesRule, Index, Search, SearchResult, + ThreadPoolNoAbortBuilder, +}; + +pub(crate) struct TempIndex { + pub inner: Index, + pub indexer_config: IndexerConfig, + pub index_documents_config: IndexDocumentsConfig, + _tempdir: TempDir, +} + +impl Deref for TempIndex { + type Target = Index; + + fn deref(&self) -> &Self::Target { + &self.inner + } +} + +impl TempIndex { + /// Creates a temporary index + pub fn new_with_map_size(size: usize) -> Self { + let options = EnvOpenOptions::new(); + let mut options = options.read_txn_without_tls(); + options.map_size(size); + let _tempdir = TempDir::new_in(".").unwrap(); + let inner = Index::new(options, _tempdir.path(), true).unwrap(); + let indexer_config = IndexerConfig::default(); + let index_documents_config = IndexDocumentsConfig::default(); + Self { inner, indexer_config, index_documents_config, _tempdir } + } + /// Creates a temporary index, with a default `4096 * 2000` size. This should be enough for + /// most tests. + pub fn new() -> Self { + Self::new_with_map_size(4096 * 2000) + } + + pub fn add_documents_using_wtxn<'t>( + &'t self, + wtxn: &mut RwTxn<'t>, + documents: Mmap, + ) -> Result<(), crate::error::Error> { + let local_pool; + let indexer_config = &self.indexer_config; + let pool = match &indexer_config.thread_pool { + Some(pool) => pool, + None => { + local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap(); + &local_pool + } + }; + + let rtxn = self.inner.read_txn()?; + let db_fields_ids_map = self.inner.fields_ids_map(&rtxn)?; + let mut new_fields_ids_map = db_fields_ids_map.clone(); + + let embedders = InnerIndexSettings::from_index(&self.inner, &rtxn, None)?.embedding_configs; + let mut indexer = indexer::DocumentOperation::new(); + match self.index_documents_config.update_method { + IndexDocumentsMethod::ReplaceDocuments => { + indexer.replace_documents(&documents).unwrap() + } + IndexDocumentsMethod::UpdateDocuments => indexer.update_documents(&documents).unwrap(), + } + + let indexer_alloc = Bump::new(); + let (document_changes, operation_stats, primary_key) = indexer.into_changes( + &indexer_alloc, + &self.inner, + &rtxn, + None, + &mut new_fields_ids_map, + &|| false, + Progress::default(), + )?; + + if let Some(error) = operation_stats.into_iter().find_map(|stat| stat.error) { + return Err(error.into()); + } + + pool.install(|| { + indexer::index( + wtxn, + &self.inner, + &crate::ThreadPoolNoAbortBuilder::new().build().unwrap(), + indexer_config.grenad_parameters(), + &db_fields_ids_map, + new_fields_ids_map, + primary_key, + &document_changes, + embedders, + &|| false, + &Progress::default(), + ) + }) + .unwrap()?; + + Ok(()) + } + + pub fn add_documents(&self, documents: Mmap) -> Result<(), crate::error::Error> { + let mut wtxn = self.write_txn().unwrap(); + self.add_documents_using_wtxn(&mut wtxn, documents)?; + wtxn.commit().unwrap(); + Ok(()) + } + + pub fn update_settings( + &self, + update: impl Fn(&mut Settings<'_, '_, '_>), + ) -> Result<(), crate::error::Error> { + let mut wtxn = self.write_txn().unwrap(); + self.update_settings_using_wtxn(&mut wtxn, update)?; + wtxn.commit().unwrap(); + Ok(()) + } + + pub fn update_settings_using_wtxn<'t>( + &'t self, + wtxn: &mut RwTxn<'t>, + update: impl Fn(&mut Settings<'_, '_, '_>), + ) -> Result<(), crate::error::Error> { + let mut builder = update::Settings::new(wtxn, &self.inner, &self.indexer_config); + update(&mut builder); + builder.execute(drop, || false)?; + Ok(()) + } + + pub fn delete_documents_using_wtxn<'t>( + &'t self, + wtxn: &mut RwTxn<'t>, + external_document_ids: Vec, + ) -> Result<(), crate::error::Error> { + let local_pool; + let indexer_config = &self.indexer_config; + let pool = match &indexer_config.thread_pool { + Some(pool) => pool, + None => { + local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap(); + &local_pool + } + }; + + let rtxn = self.inner.read_txn()?; + let db_fields_ids_map = self.inner.fields_ids_map(&rtxn)?; + let mut new_fields_ids_map = db_fields_ids_map.clone(); + + let embedders = InnerIndexSettings::from_index(&self.inner, &rtxn, None)?.embedding_configs; + + let mut indexer = indexer::DocumentOperation::new(); + let external_document_ids: Vec<_> = + external_document_ids.iter().map(AsRef::as_ref).collect(); + indexer.delete_documents(external_document_ids.as_slice()); + + let indexer_alloc = Bump::new(); + let (document_changes, operation_stats, primary_key) = indexer.into_changes( + &indexer_alloc, + &self.inner, + &rtxn, + None, + &mut new_fields_ids_map, + &|| false, + Progress::default(), + )?; + + if let Some(error) = operation_stats.into_iter().find_map(|stat| stat.error) { + return Err(error.into()); + } + + pool.install(|| { + indexer::index( + wtxn, + &self.inner, + &crate::ThreadPoolNoAbortBuilder::new().build().unwrap(), + indexer_config.grenad_parameters(), + &db_fields_ids_map, + new_fields_ids_map, + primary_key, + &document_changes, + embedders, + &|| false, + &Progress::default(), + ) + }) + .unwrap()?; + + Ok(()) + } + + pub fn delete_documents(&self, external_document_ids: Vec) { + let mut wtxn = self.write_txn().unwrap(); + + self.delete_documents_using_wtxn(&mut wtxn, external_document_ids).unwrap(); + + wtxn.commit().unwrap(); + } + + pub fn delete_document(&self, external_document_id: &str) { + self.delete_documents(vec![external_document_id.to_string()]) + } +} + +#[test] +fn aborting_indexation() { + use std::sync::atomic::AtomicBool; + use std::sync::atomic::Ordering::Relaxed; + + let index = TempIndex::new(); + let mut wtxn = index.inner.write_txn().unwrap(); + let should_abort = AtomicBool::new(false); + + let local_pool; + let indexer_config = &index.indexer_config; + let pool = match &indexer_config.thread_pool { + Some(pool) => pool, + None => { + local_pool = ThreadPoolNoAbortBuilder::new().build().unwrap(); + &local_pool + } + }; + + let rtxn = index.inner.read_txn().unwrap(); + let db_fields_ids_map = index.inner.fields_ids_map(&rtxn).unwrap(); + let mut new_fields_ids_map = db_fields_ids_map.clone(); + + let embedders = EmbeddingConfigs::default(); + let mut indexer = indexer::DocumentOperation::new(); + let payload = documents!([ + { "id": 1, "name": "kevin" }, + { "id": 2, "name": "bob", "age": 20 }, + { "id": 2, "name": "bob", "age": 20 }, + ]); + indexer.replace_documents(&payload).unwrap(); + + let indexer_alloc = Bump::new(); + let (document_changes, _operation_stats, primary_key) = indexer + .into_changes( + &indexer_alloc, + &index.inner, + &rtxn, + None, + &mut new_fields_ids_map, + &|| false, + Progress::default(), + ) + .unwrap(); + + should_abort.store(true, Relaxed); + + let err = pool + .install(|| { + indexer::index( + &mut wtxn, + &index.inner, + &crate::ThreadPoolNoAbortBuilder::new().build().unwrap(), + indexer_config.grenad_parameters(), + &db_fields_ids_map, + new_fields_ids_map, + primary_key, + &document_changes, + embedders, + &|| should_abort.load(Relaxed), + &Progress::default(), + ) + }) + .unwrap() + .unwrap_err(); + + assert!(matches!(err, Error::InternalError(InternalError::AbortedIndexation))); +} + +#[test] +fn initial_field_distribution() { + let index = TempIndex::new(); + index + .add_documents(documents!([ + { "id": 1, "name": "kevin" }, + { "id": 2, "name": "bob", "age": 20 }, + { "id": 2, "name": "bob", "age": 20 }, + ])) + .unwrap(); + + db_snap!(index, field_distribution, @r###" + age 1 | + id 2 | + name 2 | + "###); + + db_snap!(index, word_docids, + @r###" + 1 [0, ] + 2 [1, ] + 20 [1, ] + bob [1, ] + kevin [0, ] + "### + ); + + // we add all the documents a second time. we are supposed to get the same + // field_distribution in the end + index + .add_documents(documents!([ + { "id": 1, "name": "kevin" }, + { "id": 2, "name": "bob", "age": 20 }, + { "id": 2, "name": "bob", "age": 20 }, + ])) + .unwrap(); + + db_snap!(index, field_distribution, + @r###" + age 1 | + id 2 | + name 2 | + "### + ); + + // then we update a document by removing one field and another by adding one field + index + .add_documents(documents!([ + { "id": 1, "name": "kevin", "has_dog": true }, + { "id": 2, "name": "bob" } + ])) + .unwrap(); + + db_snap!(index, field_distribution, + @r###" + has_dog 1 | + id 2 | + name 2 | + "### + ); +} + +#[test] +fn put_and_retrieve_disable_typo() { + let index = TempIndex::new(); + let mut txn = index.write_txn().unwrap(); + // default value is true + assert!(index.authorize_typos(&txn).unwrap()); + // set to false + index.put_authorize_typos(&mut txn, false).unwrap(); + txn.commit().unwrap(); + + let txn = index.read_txn().unwrap(); + assert!(!index.authorize_typos(&txn).unwrap()); +} + +#[test] +fn set_min_word_len_for_typos() { + let index = TempIndex::new(); + let mut txn = index.write_txn().unwrap(); + + assert_eq!(index.min_word_len_one_typo(&txn).unwrap(), DEFAULT_MIN_WORD_LEN_ONE_TYPO); + assert_eq!(index.min_word_len_two_typos(&txn).unwrap(), DEFAULT_MIN_WORD_LEN_TWO_TYPOS); + + index.put_min_word_len_one_typo(&mut txn, 3).unwrap(); + index.put_min_word_len_two_typos(&mut txn, 15).unwrap(); + + txn.commit().unwrap(); + + let txn = index.read_txn().unwrap(); + assert_eq!(index.min_word_len_one_typo(&txn).unwrap(), 3); + assert_eq!(index.min_word_len_two_typos(&txn).unwrap(), 15); +} + +#[test] +fn add_documents_and_set_searchable_fields() { + let index = TempIndex::new(); + index + .add_documents(documents!([ + { "id": 1, "doggo": "kevin" }, + { "id": 2, "doggo": { "name": "bob", "age": 20 } }, + { "id": 3, "name": "jean", "age": 25 }, + ])) + .unwrap(); + index + .update_settings(|settings| { + settings.set_searchable_fields(vec![S("doggo"), S("name")]); + }) + .unwrap(); + + // ensure we get the right real searchable fields + user defined searchable fields + let rtxn = index.read_txn().unwrap(); + + let real = index.searchable_fields(&rtxn).unwrap(); + assert_eq!(real, &["doggo", "name", "doggo.name", "doggo.age"]); + + let user_defined = index.user_defined_searchable_fields(&rtxn).unwrap().unwrap(); + assert_eq!(user_defined, &["doggo", "name"]); +} + +#[test] +fn set_searchable_fields_and_add_documents() { + let index = TempIndex::new(); + + index + .update_settings(|settings| { + settings.set_searchable_fields(vec![S("doggo"), S("name")]); + }) + .unwrap(); + + // ensure we get the right real searchable fields + user defined searchable fields + let rtxn = index.read_txn().unwrap(); + + let real = index.searchable_fields(&rtxn).unwrap(); + assert!(real.is_empty()); + let user_defined = index.user_defined_searchable_fields(&rtxn).unwrap().unwrap(); + assert_eq!(user_defined, &["doggo", "name"]); + + index + .add_documents(documents!([ + { "id": 1, "doggo": "kevin" }, + { "id": 2, "doggo": { "name": "bob", "age": 20 } }, + { "id": 3, "name": "jean", "age": 25 }, + ])) + .unwrap(); + + // ensure we get the right real searchable fields + user defined searchable fields + let rtxn = index.read_txn().unwrap(); + + let real = index.searchable_fields(&rtxn).unwrap(); + assert_eq!(real, &["doggo", "name", "doggo.name", "doggo.age"]); + + let user_defined = index.user_defined_searchable_fields(&rtxn).unwrap().unwrap(); + assert_eq!(user_defined, &["doggo", "name"]); +} + +#[test] +fn test_basic_geo_bounding_box() { + let index = TempIndex::new(); + + index + .update_settings(|settings| { + settings.set_filterable_fields(vec![FilterableAttributesRule::Field( + RESERVED_GEO_FIELD_NAME.to_string(), + )]); + }) + .unwrap(); + index + .add_documents(documents!([ + { "id": 0, RESERVED_GEO_FIELD_NAME: { "lat": "0", "lng": "0" } }, + { "id": 1, RESERVED_GEO_FIELD_NAME: { "lat": 0, "lng": "-175" } }, + { "id": 2, RESERVED_GEO_FIELD_NAME: { "lat": "0", "lng": 175 } }, + { "id": 3, RESERVED_GEO_FIELD_NAME: { "lat": 85, "lng": 0 } }, + { "id": 4, RESERVED_GEO_FIELD_NAME: { "lat": "-85", "lng": "0" } }, + ])) + .unwrap(); + + // ensure we get the right real searchable fields + user defined searchable fields + let rtxn = index.read_txn().unwrap(); + let mut search = index.search(&rtxn); + + // exact match a document + let search_result = search + .filter(Filter::from_str("_geoBoundingBox([0, 0], [0, 0])").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0]>"); + + // match a document in the middle of the rectangle + let search_result = search + .filter(Filter::from_str("_geoBoundingBox([10, 10], [-10, -10])").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0]>"); + + // select everything + let search_result = search + .filter(Filter::from_str("_geoBoundingBox([90, 180], [-90, -180])").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0, 1, 2, 3, 4]>"); + + // go on the edge of the longitude + let search_result = search + .filter(Filter::from_str("_geoBoundingBox([0, -170], [0, 180])").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[1]>"); + + // go on the other edge of the longitude + let search_result = search + .filter(Filter::from_str("_geoBoundingBox([0, -180], [0, 170])").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[2]>"); + + // wrap around the longitude + let search_result = search + .filter(Filter::from_str("_geoBoundingBox([0, -170], [0, 170])").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[1, 2]>"); + + // go on the edge of the latitude + let search_result = search + .filter(Filter::from_str("_geoBoundingBox([90, 0], [80, 0])").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[3]>"); + + // go on the edge of the latitude + let search_result = search + .filter(Filter::from_str("_geoBoundingBox([-80, 0], [-90, 0])").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[4]>"); + + // the requests that don't make sense + + // try to wrap around the latitude + let error = search + .filter(Filter::from_str("_geoBoundingBox([-80, 0], [80, 0])").unwrap().unwrap()) + .execute() + .unwrap_err(); + insta::assert_snapshot!( + error, + @r###" + The top latitude `-80` is below the bottom latitude `80`. + 32:33 _geoBoundingBox([-80, 0], [80, 0]) + "### + ); + + // send a top latitude lower than the bottow latitude + let error = search + .filter(Filter::from_str("_geoBoundingBox([-10, 0], [10, 0])").unwrap().unwrap()) + .execute() + .unwrap_err(); + insta::assert_snapshot!( + error, + @r###" + The top latitude `-10` is below the bottom latitude `10`. + 32:33 _geoBoundingBox([-10, 0], [10, 0]) + "### + ); +} + +#[test] +fn test_contains() { + let index = TempIndex::new(); + + index + .update_settings(|settings| { + settings + .set_filterable_fields(vec![FilterableAttributesRule::Field("doggo".to_string())]); + }) + .unwrap(); + index + .add_documents(documents!([ + { "id": 0, "doggo": "kefir" }, + { "id": 1, "doggo": "kefirounet" }, + { "id": 2, "doggo": "kefkef" }, + { "id": 3, "doggo": "fifir" }, + { "id": 4, "doggo": "boubou" }, + { "id": 5 }, + ])) + .unwrap(); + + let rtxn = index.read_txn().unwrap(); + let mut search = index.search(&rtxn); + let search_result = search + .filter(Filter::from_str("doggo CONTAINS kefir").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0, 1]>"); + let mut search = index.search(&rtxn); + let search_result = + search.filter(Filter::from_str("doggo CONTAINS KEF").unwrap().unwrap()).execute().unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[0, 1, 2]>"); + let mut search = index.search(&rtxn); + let search_result = search + .filter(Filter::from_str("doggo NOT CONTAINS fir").unwrap().unwrap()) + .execute() + .unwrap(); + insta::assert_debug_snapshot!(search_result.candidates, @"RoaringBitmap<[2, 4, 5]>"); +} + +#[test] +fn replace_documents_external_ids_and_soft_deletion_check() { + let index = TempIndex::new(); + + index + .update_settings(|settings| { + settings.set_primary_key("id".to_owned()); + settings + .set_filterable_fields(vec![FilterableAttributesRule::Field("doggo".to_string())]); + }) + .unwrap(); + + let mut docs = vec![]; + for i in 0..4 { + docs.push(serde_json::json!( + { "id": i, "doggo": i } + )); + } + index.add_documents(documents!(docs)).unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, 3, ]"); + db_snap!(index, external_documents_ids, 1, @r###" + docids: + 0 0 + 1 1 + 2 2 + 3 3 + "###); + db_snap!(index, facet_id_f64_docids, 1, @r###" + 1 0 0 1 [0, ] + 1 0 1 1 [1, ] + 1 0 2 1 [2, ] + 1 0 3 1 [3, ] + "###); + + let mut docs = vec![]; + for i in 0..3 { + docs.push(serde_json::json!( + { "id": i, "doggo": i + 1 } + )); + } + index.add_documents(documents!(docs)).unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, 3, ]"); + db_snap!(index, external_documents_ids, 2, @r###" + docids: + 0 0 + 1 1 + 2 2 + 3 3 + "###); + db_snap!(index, facet_id_f64_docids, 2, @r###" + 1 0 1 1 [0, ] + 1 0 2 1 [1, ] + 1 0 3 1 [2, 3, ] + "###); + + index + .add_documents( + documents!([{ "id": 3, "doggo": 4 }, { "id": 3, "doggo": 5 },{ "id": 3, "doggo": 4 }]), + ) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, 3, ]"); + db_snap!(index, external_documents_ids, 3, @r###" + docids: + 0 0 + 1 1 + 2 2 + 3 3 + "###); + db_snap!(index, facet_id_f64_docids, 3, @r###" + 1 0 1 1 [0, ] + 1 0 2 1 [1, ] + 1 0 3 1 [2, ] + 1 0 4 1 [3, ] + "###); + + index + .update_settings(|settings| { + settings.set_distinct_field("id".to_owned()); + }) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, 3, ]"); + db_snap!(index, external_documents_ids, 3, @r###" + docids: + 0 0 + 1 1 + 2 2 + 3 3 + "###); + db_snap!(index, facet_id_f64_docids, 3, @r###" + 0 0 0 1 [0, ] + 0 0 1 1 [1, ] + 0 0 2 1 [2, ] + 0 0 3 1 [3, ] + 1 0 1 1 [0, ] + 1 0 2 1 [1, ] + 1 0 3 1 [2, ] + 1 0 4 1 [3, ] + "###); +} + +#[test] +fn bug_3021_first() { + // https://github.com/meilisearch/meilisearch/issues/3021 + let mut index = TempIndex::new(); + index.index_documents_config.update_method = IndexDocumentsMethod::ReplaceDocuments; + + index + .update_settings(|settings| { + settings.set_primary_key("primary_key".to_owned()); + }) + .unwrap(); + + index + .add_documents(documents!([ + { "primary_key": 38 }, + { "primary_key": 34 } + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, ]"); + db_snap!(index, external_documents_ids, 1, @r###" + docids: + 34 1 + 38 0 + "###); + + index.delete_document("34"); + + db_snap!(index, documents_ids, @"[0, ]"); + db_snap!(index, external_documents_ids, 2, @r###" + docids: + 38 0 + "###); + + index + .update_settings(|s| { + s.set_searchable_fields(vec![]); + }) + .unwrap(); + + // The key point of the test is to verify that the external documents ids + // do not contain any entry for previously soft-deleted document ids + db_snap!(index, documents_ids, @"[0, ]"); + db_snap!(index, external_documents_ids, 3, @r###" + docids: + 38 0 + "###); + + // So that this document addition works correctly now. + // It would be wrongly interpreted as a replacement before + index.add_documents(documents!({ "primary_key": 34 })).unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, ]"); + db_snap!(index, external_documents_ids, 4, @r###" + docids: + 34 1 + 38 0 + "###); + + // We do the test again, but deleting the document with id 0 instead of id 1 now + index.delete_document("38"); + + db_snap!(index, documents_ids, @"[1, ]"); + db_snap!(index, external_documents_ids, 5, @r###" + docids: + 34 1 + "###); + + index + .update_settings(|s| { + s.set_searchable_fields(vec!["primary_key".to_owned()]); + }) + .unwrap(); + + db_snap!(index, documents_ids, @"[1, ]"); + db_snap!(index, external_documents_ids, 6, @r###" + docids: + 34 1 + "###); + + // And adding lots of documents afterwards instead of just one. + // These extra subtests don't add much, but it's better than nothing. + index + .add_documents(documents!([ + { "primary_key": 38 }, + { "primary_key": 39 }, + { "primary_key": 41 }, + { "primary_key": 40 }, + { "primary_key": 41 }, + { "primary_key": 42 }, + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, 3, 4, 5, ]"); + db_snap!(index, external_documents_ids, 7, @r###" + docids: + 34 1 + 38 0 + 39 2 + 40 4 + 41 3 + 42 5 + "###); +} + +#[test] +fn simple_delete() { + let mut index = TempIndex::new(); + index.index_documents_config.update_method = IndexDocumentsMethod::UpdateDocuments; + index + .add_documents(documents!([ + { "id": 30 }, + { "id": 34 } + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, ]"); + db_snap!(index, external_documents_ids, 1, @r###" + docids: + 30 0 + 34 1"###); + + index.delete_document("34"); + + db_snap!(index, documents_ids, @"[0, ]"); + db_snap!(index, external_documents_ids, 2, @r###" + docids: + 30 0 + "###); +} + +#[test] +fn bug_3021_second() { + // https://github.com/meilisearch/meilisearch/issues/3021 + let mut index = TempIndex::new(); + index.index_documents_config.update_method = IndexDocumentsMethod::UpdateDocuments; + + index + .update_settings(|settings| { + settings.set_primary_key("primary_key".to_owned()); + }) + .unwrap(); + + index + .add_documents(documents!([ + { "primary_key": 30 }, + { "primary_key": 34 } + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, ]"); + db_snap!(index, external_documents_ids, 1, @r###" + docids: + 30 0 + 34 1 + "###); + + index.delete_document("34"); + + db_snap!(index, documents_ids, @"[0, ]"); + db_snap!(index, external_documents_ids, 2, @r###" + docids: + 30 0 + "###); + + index + .update_settings(|s| { + s.set_searchable_fields(vec![]); + }) + .unwrap(); + + // The key point of the test is to verify that the external documents ids + // do not contain any entry for previously soft-deleted document ids + db_snap!(index, documents_ids, @"[0, ]"); + db_snap!(index, external_documents_ids, 3, @r###" + docids: + 30 0 + "###); + + // So that when we add a new document + index.add_documents(documents!({ "primary_key": 35, "b": 2 })).unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, ]"); + // The external documents ids don't have several external ids pointing to the same + // internal document id + db_snap!(index, external_documents_ids, 4, @r###" + docids: + 30 0 + 35 1 + "###); + + // And when we add 34 again, we don't replace document 35 + index.add_documents(documents!({ "primary_key": 34, "a": 1 })).unwrap(); + + // And document 35 still exists, is not deleted + db_snap!(index, documents_ids, @"[0, 1, 2, ]"); + db_snap!(index, external_documents_ids, 5, @r###" + docids: + 30 0 + 34 2 + 35 1 + "###); + + let rtxn = index.read_txn().unwrap(); + let (_docid, obkv) = index.documents(&rtxn, [0]).unwrap()[0]; + let json = obkv_to_json(&[0, 1, 2], &index.fields_ids_map(&rtxn).unwrap(), obkv).unwrap(); + insta::assert_debug_snapshot!(json, @r###" + { + "primary_key": Number(30), + } + "###); + + // Furthermore, when we retrieve document 34, it is not the result of merging 35 with 34 + let (_docid, obkv) = index.documents(&rtxn, [2]).unwrap()[0]; + let json = obkv_to_json(&[0, 1, 2], &index.fields_ids_map(&rtxn).unwrap(), obkv).unwrap(); + insta::assert_debug_snapshot!(json, @r###" + { + "primary_key": Number(34), + "a": Number(1), + } + "###); + + drop(rtxn); + + // Add new documents again + index + .add_documents( + documents!([{ "primary_key": 37 }, { "primary_key": 38 }, { "primary_key": 39 }]), + ) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, 3, 4, 5, ]"); + db_snap!(index, external_documents_ids, 6, @r###" + docids: + 30 0 + 34 2 + 35 1 + 37 3 + 38 4 + 39 5 + "###); +} + +#[test] +fn bug_3021_third() { + // https://github.com/meilisearch/meilisearch/issues/3021 + let mut index = TempIndex::new(); + index.index_documents_config.update_method = IndexDocumentsMethod::UpdateDocuments; + + index + .update_settings(|settings| { + settings.set_primary_key("primary_key".to_owned()); + }) + .unwrap(); + + index + .add_documents(documents!([ + { "primary_key": 3 }, + { "primary_key": 4 }, + { "primary_key": 5 } + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, ]"); + db_snap!(index, external_documents_ids, 1, @r###" + docids: + 3 0 + 4 1 + 5 2 + "###); + + index.delete_document("3"); + + db_snap!(index, documents_ids, @"[1, 2, ]"); + db_snap!(index, external_documents_ids, 2, @r###" + docids: + 4 1 + 5 2 + "###); + + index.add_documents(documents!([{ "primary_key": "4", "a": 2 }])).unwrap(); + + db_snap!(index, documents_ids, @"[1, 2, ]"); + db_snap!(index, external_documents_ids, 2, @r###" + docids: + 4 1 + 5 2 + "###); + + index + .add_documents(documents!([ + { "primary_key": "3" }, + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, ]"); + db_snap!(index, external_documents_ids, 2, @r###" + docids: + 3 0 + 4 1 + 5 2 + "###); +} + +#[test] +fn bug_3021_fourth() { + // https://github.com/meilisearch/meilisearch/issues/3021 + let mut index = TempIndex::new(); + index.index_documents_config.update_method = IndexDocumentsMethod::UpdateDocuments; + + index + .update_settings(|settings| { + settings.set_primary_key("primary_key".to_owned()); + }) + .unwrap(); + + index + .add_documents(documents!([ + { "primary_key": 11 }, + { "primary_key": 4 }, + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, ]"); + db_snap!(index, external_documents_ids, @r###" + docids: + 11 0 + 4 1 + "###); + db_snap!(index, fields_ids_map, @r###" + 0 primary_key | + "###); + db_snap!(index, searchable_fields, @r###"["primary_key"]"###); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + 0 0 | + "###); + + index + .add_documents(documents!([ + { "primary_key": 4, "a": 0 }, + { "primary_key": 1 }, + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, ]"); + db_snap!(index, external_documents_ids, @r###" + docids: + 1 2 + 11 0 + 4 1 + "###); + db_snap!(index, fields_ids_map, @r###" + 0 primary_key | + 1 a | + "###); + db_snap!(index, searchable_fields, @r###"["primary_key", "a"]"###); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + 0 0 | + 1 0 | + "###); + + index.delete_documents(Default::default()); + + db_snap!(index, documents_ids, @"[0, 1, 2, ]"); + db_snap!(index, external_documents_ids, @r###" + docids: + 1 2 + 11 0 + 4 1 + "###); + db_snap!(index, fields_ids_map, @r###" + 0 primary_key | + 1 a | + "###); + db_snap!(index, searchable_fields, @r###"["primary_key", "a"]"###); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + 0 0 | + 1 0 | + "###); + + index + .add_documents(documents!([ + { "primary_key": 4, "a": 1 }, + { "primary_key": 1, "a": 0 }, + ])) + .unwrap(); + + db_snap!(index, documents_ids, @"[0, 1, 2, ]"); + db_snap!(index, external_documents_ids, @r###" + docids: + 1 2 + 11 0 + 4 1 + "###); + db_snap!(index, fields_ids_map, @r###" + 0 primary_key | + 1 a | + "###); + db_snap!(index, searchable_fields, @r###"["primary_key", "a"]"###); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + 0 0 | + 1 0 | + "###); + + let rtxn = index.read_txn().unwrap(); + let search = Search::new(&rtxn, &index); + let SearchResult { + matching_words: _, + candidates: _, + document_scores: _, + mut documents_ids, + degraded: _, + used_negative_operator: _, + } = search.execute().unwrap(); + let primary_key_id = index.fields_ids_map(&rtxn).unwrap().id("primary_key").unwrap(); + documents_ids.sort_unstable(); + let docs = index.documents(&rtxn, documents_ids).unwrap(); + let mut all_ids = HashSet::new(); + for (_docid, obkv) in docs { + let id = obkv.get(primary_key_id).unwrap(); + assert!(all_ids.insert(id)); + } +} + +#[test] +fn bug_3007() { + // https://github.com/meilisearch/meilisearch/issues/3007 + + use crate::error::{GeoError, UserError}; + let index = TempIndex::new(); + + // Given is an index with a geo field NOT contained in the sortable_fields of the settings + index + .update_settings(|settings| { + settings.set_primary_key("id".to_string()); + settings.set_filterable_fields(vec![FilterableAttributesRule::Field( + RESERVED_GEO_FIELD_NAME.to_string(), + )]); + }) + .unwrap(); + + // happy path + index + .add_documents(documents!({ "id" : 5, RESERVED_GEO_FIELD_NAME: {"lat": 12.0, "lng": 11.0}})) + .unwrap(); + + db_snap!(index, geo_faceted_documents_ids); + + // both are unparseable, we expect GeoError::BadLatitudeAndLongitude + let err1 = index + .add_documents( + documents!({ "id" : 6, RESERVED_GEO_FIELD_NAME: {"lat": "unparseable", "lng": "unparseable"}}), + ) + .unwrap_err(); + match err1 { + Error::UserError(UserError::InvalidGeoField(err)) => match *err { + GeoError::BadLatitudeAndLongitude { .. } => (), + otherwise => { + panic!("err1 is not a BadLatitudeAndLongitude error but rather a {otherwise:?}") + } + }, + _ => panic!("err1 is not a BadLatitudeAndLongitude error but rather a {err1:?}"), + } + + db_snap!(index, geo_faceted_documents_ids); // ensure that no more document was inserted +} + +#[test] +fn unexpected_extra_fields_in_geo_field() { + let index = TempIndex::new(); + + index + .update_settings(|settings| { + settings.set_primary_key("id".to_string()); + settings.set_filterable_fields(vec![FilterableAttributesRule::Field( + RESERVED_GEO_FIELD_NAME.to_string(), + )]); + }) + .unwrap(); + + let err = index + .add_documents( + documents!({ "id" : "doggo", RESERVED_GEO_FIELD_NAME: { "lat": 1, "lng": 2, "doggo": "are the best" }}), + ) + .unwrap_err(); + insta::assert_snapshot!(err, @r###"The `_geo` field in the document with the id: `"doggo"` contains the following unexpected fields: `{"doggo":"are the best"}`."###); + + db_snap!(index, geo_faceted_documents_ids); // ensure that no documents were inserted + + // multiple fields and complex values + let err = index + .add_documents( + documents!({ "id" : "doggo", RESERVED_GEO_FIELD_NAME: { "lat": 1, "lng": 2, "doggo": "are the best", "and": { "all": ["cats", { "are": "beautiful" } ] } } }), + ) + .unwrap_err(); + insta::assert_snapshot!(err, @r###"The `_geo` field in the document with the id: `"doggo"` contains the following unexpected fields: `{"and":{"all":["cats",{"are":"beautiful"}]},"doggo":"are the best"}`."###); + + db_snap!(index, geo_faceted_documents_ids); // ensure that no documents were inserted +} + +#[test] +fn swapping_searchable_attributes() { + // See https://github.com/meilisearch/meilisearch/issues/4484 + + let index = TempIndex::new(); + + index + .update_settings(|settings| { + settings.set_searchable_fields(vec![S("name")]); + settings + .set_filterable_fields(vec![FilterableAttributesRule::Field("age".to_string())]); + }) + .unwrap(); + + index + .add_documents(documents!({ "id": 1, "name": "Many", "age": 28, "realName": "Maxime" })) + .unwrap(); + db_snap!(index, fields_ids_map, @r###" + 0 id | + 1 name | + 2 age | + 3 realName | + "###); + db_snap!(index, searchable_fields, @r###"["name"]"###); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + 1 0 | + "###); + + index + .update_settings(|settings| { + settings.set_searchable_fields(vec![S("name"), S("realName")]); + settings + .set_filterable_fields(vec![FilterableAttributesRule::Field("age".to_string())]); + }) + .unwrap(); + + // The order of the field id map shouldn't change + db_snap!(index, fields_ids_map, @r###" + 0 id | + 1 name | + 2 age | + 3 realName | + "###); + db_snap!(index, searchable_fields, @r###"["name", "realName"]"###); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + 1 0 | + 3 1 | + "###); +} + +#[test] +fn attribute_weights_after_swapping_searchable_attributes() { + // See https://github.com/meilisearch/meilisearch/issues/4484 + + let index = TempIndex::new(); + + index + .update_settings(|settings| { + settings.set_searchable_fields(vec![S("name"), S("beverage")]); + }) + .unwrap(); + + index + .add_documents(documents!([ + { "id": 0, "name": "kefir", "beverage": "water" }, + { "id": 1, "name": "tamo", "beverage": "kefir" } + ])) + .unwrap(); + + let rtxn = index.read_txn().unwrap(); + let mut search = index.search(&rtxn); + let results = search.query("kefir").execute().unwrap(); + + // We should find kefir the dog first + insta::assert_debug_snapshot!(results.documents_ids, @r###" + [ + 0, + 1, + ] + "###); + + index + .update_settings(|settings| { + settings.set_searchable_fields(vec![S("beverage"), S("name")]); + }) + .unwrap(); + + let rtxn = index.read_txn().unwrap(); + let mut search = index.search(&rtxn); + let results = search.query("kefir").execute().unwrap(); + + // We should find tamo first + insta::assert_debug_snapshot!(results.documents_ids, @r###" + [ + 1, + 0, + ] + "###); +} + +#[test] +fn vectors_are_never_indexed_as_searchable_or_filterable() { + let index = TempIndex::new(); + + index + .add_documents(documents!([ + { "id": 0, "_vectors": { "doggo": [2345] } }, + { "id": 1, "_vectors": { "doggo": [6789] } }, + ])) + .unwrap(); + + db_snap!(index, fields_ids_map, @r###" + 0 id | + 1 _vectors | + "###); + db_snap!(index, searchable_fields, @r###"["id"]"###); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + 0 0 | + "###); + + let rtxn = index.read_txn().unwrap(); + let mut search = index.search(&rtxn); + let results = search.query("2345").execute().unwrap(); + assert!(results.candidates.is_empty()); + drop(rtxn); + + index + .update_settings(|settings| { + settings.set_searchable_fields(vec![S("_vectors"), S("_vectors.doggo")]); + settings.set_filterable_fields(vec![ + FilterableAttributesRule::Field("_vectors".to_string()), + FilterableAttributesRule::Field("_vectors.doggo".to_string()), + ]); + }) + .unwrap(); + + db_snap!(index, fields_ids_map, @r###" + 0 id | + 1 _vectors | + "###); + db_snap!(index, searchable_fields, @"[]"); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + "###); + + let rtxn = index.read_txn().unwrap(); + let mut search = index.search(&rtxn); + let results = search.query("2345").execute().unwrap(); + assert!(results.candidates.is_empty()); + + let mut search = index.search(&rtxn); + let results = search + .filter(Filter::from_str("_vectors.doggo = 6789").unwrap().unwrap()) + .execute() + .unwrap(); + assert!(results.candidates.is_empty()); + + index + .update_settings(|settings| { + settings.set_embedder_settings(btreemap! { + S("doggo") => Setting::Set(EmbeddingSettings { + dimensions: Setting::Set(1), + source: Setting::Set(EmbedderSource::UserProvided), + ..EmbeddingSettings::default()}), + }); + }) + .unwrap(); + + db_snap!(index, fields_ids_map, @r###" + 0 id | + 1 _vectors | + "###); + db_snap!(index, searchable_fields, @"[]"); + db_snap!(index, fieldids_weights_map, @r###" + fid weight + "###); + + let rtxn = index.read_txn().unwrap(); + let mut search = index.search(&rtxn); + let results = search.query("2345").execute().unwrap(); + assert!(results.candidates.is_empty()); + + let mut search = index.search(&rtxn); + let results = search + .filter(Filter::from_str("_vectors.doggo = 6789").unwrap().unwrap()) + .execute() + .unwrap(); + assert!(results.candidates.is_empty()); +} diff --git a/crates/milli/src/update/upgrade/mod.rs b/crates/milli/src/update/upgrade/mod.rs index d471107ec..f9d971017 100644 --- a/crates/milli/src/update/upgrade/mod.rs +++ b/crates/milli/src/update/upgrade/mod.rs @@ -24,12 +24,16 @@ trait UpgradeIndex { } /// Return true if the cached stats of the index must be regenerated -pub fn upgrade( +pub fn upgrade( wtxn: &mut RwTxn, index: &Index, db_version: (u32, u32, u32), + must_stop_processing: MSP, progress: Progress, -) -> Result { +) -> Result +where + MSP: Fn() -> bool + Sync, +{ let from = index.get_version(wtxn)?.unwrap_or(db_version); let upgrade_functions: &[&dyn UpgradeIndex] = &[ &V1_12_To_V1_12_3 {}, @@ -59,6 +63,9 @@ pub fn upgrade( let mut current_version = from; let mut regenerate_stats = false; for (i, upgrade) in upgrade_path.iter().enumerate() { + if (must_stop_processing)() { + return Err(crate::Error::InternalError(InternalError::AbortedIndexation)); + } let target = upgrade.target_version(); progress.update_progress(VariableNameStep::::new( format!( diff --git a/crates/milli/src/update/upgrade/v1_13.rs b/crates/milli/src/update/upgrade/v1_13.rs index 8e5e052bd..7f6608970 100644 --- a/crates/milli/src/update/upgrade/v1_13.rs +++ b/crates/milli/src/update/upgrade/v1_13.rs @@ -1,7 +1,6 @@ use heed::RwTxn; use super::UpgradeIndex; -use crate::constants::{VERSION_MAJOR, VERSION_MINOR, VERSION_PATCH}; use crate::database_stats::DatabaseStats; use crate::progress::Progress; use crate::{make_enum_progress, Index, Result}; @@ -51,10 +50,6 @@ impl UpgradeIndex for V1_13_1_To_Latest_V1_13 { } fn target_version(&self) -> (u32, u32, u32) { - ( - VERSION_MAJOR.parse().unwrap(), - VERSION_MINOR.parse().unwrap(), - VERSION_PATCH.parse().unwrap(), - ) + (1, 13, 3) } }