write the dump export

This commit is contained in:
Tamo 2022-10-13 15:02:59 +02:00 committed by Clément Renault
parent 8954b1bd1d
commit b7f9c94f4a
No known key found for this signature in database
GPG Key ID: 92ADA4E935E71FA4
25 changed files with 686 additions and 184 deletions

View File

@ -16,7 +16,6 @@ time = { version = "0.3.7", features = ["serde-well-known", "formatting", "parsi
tar = "0.4.38"
anyhow = "1.0.65"
log = "0.4.17"
index-scheduler = { path = "../index-scheduler" }
meilisearch-auth = { path = "../meilisearch-auth" }
meilisearch-types = { path = "../meilisearch-types" }
http = "0.2.8"

View File

@ -1,3 +1,4 @@
use meilisearch_types::error::{Code, ErrorCode};
use thiserror::Error;
#[derive(Debug, Error)]
@ -16,3 +17,19 @@ pub enum Error {
#[error(transparent)]
Uuid(#[from] uuid::Error),
}
impl ErrorCode for Error {
fn error_code(&self) -> Code {
match self {
// Are these three really Internal errors?
Error::Io(_) => Code::Internal,
Error::Serde(_) => Code::Internal,
Error::Uuid(_) => Code::Internal,
// all these errors should never be raised when creating a dump, thus no error code should be associated.
Error::DumpV1Unsupported => Code::Internal,
Error::BadIndexName => Code::Internal,
Error::MalformedTask => Code::Internal,
}
}
}

View File

@ -52,12 +52,11 @@ pub(crate) mod test {
};
use big_s::S;
use index_scheduler::task::Details;
use maplit::btreeset;
use meilisearch_auth::{Action, Key};
use meilisearch_types::keys::{Action, Key};
use meilisearch_types::milli::{self, update::Setting};
use meilisearch_types::settings::{Checked, Settings};
use meilisearch_types::tasks::{DetailsView, Kind, Status, TaskView};
use meilisearch_types::tasks::{Kind, Status};
use meilisearch_types::{index_uid::IndexUid, star_or::StarOr};
use serde_json::{json, Map, Value};
use time::{macros::datetime, Duration};
@ -116,7 +115,7 @@ pub(crate) mod test {
settings.check()
}
pub fn create_test_tasks() -> Vec<(TaskView, Option<Vec<Document>>)> {
pub fn create_test_tasks() -> Vec<(Task, Option<Vec<Document>>)> {
vec![
(
TaskView {

View File

@ -124,7 +124,7 @@ impl CompatV5ToV6 {
indexed_documents,
} => v6::Details::DocumentAddition {
received_documents: received_documents as u64,
indexed_documents: indexed_documents.map_or(0, |i| i as u64),
indexed_documents: indexed_documents.map(|i| i as u64),
},
v5::Details::Settings { settings } => v6::Details::Settings {
settings: settings.into(),

202
dump/src/reader/v6/mod.rs Normal file
View File

@ -0,0 +1,202 @@
use std::{
fs::{self, File},
io::{BufRead, BufReader},
path::Path,
str::FromStr,
};
use tempfile::TempDir;
use time::OffsetDateTime;
use uuid::Uuid;
use crate::{Error, IndexMetadata, Result, Version};
mod tasks;
pub use meilisearch_types::milli;
use super::Document;
pub type Metadata = crate::Metadata;
pub type Settings<T> = meilisearch_types::settings::Settings<T>;
pub type Checked = meilisearch_types::settings::Checked;
pub type Unchecked = meilisearch_types::settings::Unchecked;
pub type Task = tasks::TaskDump;
pub type Key = meilisearch_types::keys::Key;
// ===== Other types to clarify the code of the compat module
// everything related to the tasks
pub type Status = meilisearch_types::tasks::Status;
pub type Kind = tasks::KindDump;
pub type Details = meilisearch_types::tasks::Details;
// everything related to the settings
pub type Setting<T> = meilisearch_types::milli::update::Setting<T>;
pub type TypoTolerance = meilisearch_types::settings::TypoSettings;
pub type MinWordSizeForTypos = meilisearch_types::settings::MinWordSizeTyposSetting;
pub type FacetingSettings = meilisearch_types::settings::FacetingSettings;
pub type PaginationSettings = meilisearch_types::settings::PaginationSettings;
// everything related to the api keys
pub type Action = meilisearch_types::keys::Action;
pub type StarOr<T> = meilisearch_types::star_or::StarOr<T>;
pub type IndexUid = meilisearch_types::index_uid::IndexUid;
// everything related to the errors
pub type ResponseError = meilisearch_types::error::ResponseError;
pub type Code = meilisearch_types::error::Code;
pub struct V6Reader {
dump: TempDir,
instance_uid: Uuid,
metadata: Metadata,
tasks: BufReader<File>,
keys: BufReader<File>,
}
impl V6Reader {
pub fn open(dump: TempDir) -> Result<Self> {
let meta_file = fs::read(dump.path().join("metadata.json"))?;
let instance_uid = fs::read_to_string(dump.path().join("instance_uid.uuid"))?;
let instance_uid = Uuid::from_str(&instance_uid)?;
Ok(V6Reader {
metadata: serde_json::from_reader(&*meta_file)?,
instance_uid,
tasks: BufReader::new(File::open(dump.path().join("tasks").join("queue.jsonl"))?),
keys: BufReader::new(File::open(dump.path().join("keys.jsonl"))?),
dump,
})
}
pub fn version(&self) -> Version {
Version::V6
}
pub fn date(&self) -> Option<OffsetDateTime> {
Some(self.metadata.dump_date)
}
pub fn instance_uid(&self) -> Result<Option<Uuid>> {
Ok(Some(self.instance_uid))
}
pub fn indexes(&self) -> Result<Box<dyn Iterator<Item = Result<V6IndexReader>> + '_>> {
let entries = fs::read_dir(self.dump.path().join("indexes"))?;
Ok(Box::new(
entries
.map(|entry| -> Result<Option<_>> {
let entry = entry?;
if entry.file_type()?.is_dir() {
let index = V6IndexReader::new(
entry
.file_name()
.to_str()
.ok_or(Error::BadIndexName)?
.to_string(),
&entry.path(),
)?;
Ok(Some(index))
} else {
Ok(None)
}
})
.filter_map(|entry| entry.transpose()),
))
}
pub fn tasks(
&mut self,
) -> Box<dyn Iterator<Item = Result<(Task, Option<Box<super::UpdateFile>>)>> + '_> {
Box::new((&mut self.tasks).lines().map(|line| -> Result<_> {
let task: Task = serde_json::from_str(&line?)?;
let update_file_path = self
.dump
.path()
.join("tasks")
.join("update_files")
.join(format!("{}.jsonl", task.uid.to_string()));
if update_file_path.exists() {
Ok((
task,
Some(Box::new(UpdateFile::new(&update_file_path)?) as Box<super::UpdateFile>),
))
} else {
Ok((task, None))
}
}))
}
pub fn keys(&mut self) -> Box<dyn Iterator<Item = Result<Key>> + '_> {
Box::new(
(&mut self.keys)
.lines()
.map(|line| -> Result<_> { Ok(serde_json::from_str(&line?)?) }),
)
}
}
pub struct UpdateFile {
reader: BufReader<File>,
}
impl UpdateFile {
fn new(path: &Path) -> Result<Self> {
Ok(UpdateFile {
reader: BufReader::new(File::open(path)?),
})
}
}
impl Iterator for UpdateFile {
type Item = Result<Document>;
fn next(&mut self) -> Option<Self::Item> {
(&mut self.reader)
.lines()
.map(|line| {
line.map_err(Error::from)
.and_then(|line| serde_json::from_str(&line).map_err(Error::from))
})
.next()
}
}
pub struct V6IndexReader {
metadata: IndexMetadata,
documents: BufReader<File>,
settings: BufReader<File>,
}
impl V6IndexReader {
pub fn new(_name: String, path: &Path) -> Result<Self> {
let metadata = File::open(path.join("metadata.json"))?;
let ret = V6IndexReader {
metadata: serde_json::from_reader(metadata)?,
documents: BufReader::new(File::open(path.join("documents.jsonl"))?),
settings: BufReader::new(File::open(path.join("settings.json"))?),
};
Ok(ret)
}
pub fn metadata(&self) -> &IndexMetadata {
&self.metadata
}
pub fn documents(&mut self) -> Result<impl Iterator<Item = Result<Document>> + '_> {
Ok((&mut self.documents)
.lines()
.map(|line| -> Result<_> { Ok(serde_json::from_str(&line?)?) }))
}
pub fn settings(&mut self) -> Result<Settings<Checked>> {
let settings: Settings<Unchecked> = serde_json::from_reader(&mut self.settings)?;
Ok(settings.check())
}
}

View File

@ -0,0 +1,81 @@
use meilisearch_types::{
error::ResponseError,
milli::update::IndexDocumentsMethod,
settings::Unchecked,
tasks::{Details, Status, TaskId},
};
use serde::{Deserialize, Serialize};
use time::OffsetDateTime;
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct TaskDump {
pub uid: TaskId,
#[serde(default)]
pub index_uid: Option<String>,
pub status: Status,
// TODO use our own Kind for the user
#[serde(rename = "type")]
pub kind: KindDump,
#[serde(skip_serializing_if = "Option::is_none")]
pub details: Option<Details>,
#[serde(skip_serializing_if = "Option::is_none")]
pub error: Option<ResponseError>,
#[serde(with = "time::serde::rfc3339")]
pub enqueued_at: OffsetDateTime,
#[serde(
with = "time::serde::rfc3339::option",
skip_serializing_if = "Option::is_none",
default
)]
pub started_at: Option<OffsetDateTime>,
#[serde(
with = "time::serde::rfc3339::option",
skip_serializing_if = "Option::is_none",
default
)]
pub finished_at: Option<OffsetDateTime>,
}
// A `Kind` specific version made for the dump. If modified you may break the dump.
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub enum KindDump {
DocumentImport {
primary_key: Option<String>,
method: IndexDocumentsMethod,
documents_count: u64,
allow_index_creation: bool,
},
DocumentDeletion {
documents_ids: Vec<String>,
},
DocumentClear,
Settings {
settings: meilisearch_types::settings::Settings<Unchecked>,
is_deletion: bool,
allow_index_creation: bool,
},
IndexDeletion,
IndexCreation {
primary_key: Option<String>,
},
IndexUpdate {
primary_key: Option<String>,
},
IndexSwap {
lhs: String,
rhs: String,
},
CancelTask {
tasks: Vec<TaskId>,
},
DeleteTasks {
query: String,
tasks: Vec<TaskId>,
},
DumpExport,
Snapshot,
}

View File

@ -22,13 +22,15 @@ pub struct DumpWriter {
}
impl DumpWriter {
pub fn new(instance_uuid: Uuid) -> Result<DumpWriter> {
pub fn new(instance_uuid: Option<Uuid>) -> Result<DumpWriter> {
let dir = TempDir::new()?;
fs::write(
dir.path().join("instance_uid.uuid"),
&instance_uuid.as_hyphenated().to_string(),
)?;
if let Some(instance_uuid) = instance_uuid {
fs::write(
dir.path().join("instance_uid.uuid"),
&instance_uuid.as_hyphenated().to_string(),
)?;
}
let metadata = Metadata {
dump_version: CURRENT_DUMP_VERSION,
@ -133,7 +135,6 @@ impl UpdateFile {
writer.write_all(b"\n")?;
writer.flush()?;
} else {
dbg!(&self.path);
let file = File::create(&self.path).unwrap();
self.writer = Some(BufWriter::new(file));
self.push_document(document)?;

View File

@ -13,6 +13,7 @@ file-store = { path = "../file-store" }
log = "0.4.14"
meilisearch-types = { path = "../meilisearch-types" }
roaring = { version = "0.10.0", features = ["serde"] }
dump = { path = "../dump" }
serde = { version = "1.0.136", features = ["derive"] }
serde_json = { version = "1.0.85", features = ["preserve_order"] }
tempfile = "3.3.0"

View File

@ -1,7 +1,11 @@
use std::collections::HashSet;
use std::fs::File;
use std::io::BufWriter;
use crate::{autobatcher::BatchKind, Error, IndexScheduler, Result, TaskId};
use dump::IndexMetadata;
use meilisearch_types::milli::documents::obkv_to_object;
use meilisearch_types::tasks::{Details, Kind, KindWithContent, Status, Task};
use log::{debug, info};
@ -25,7 +29,7 @@ pub(crate) enum Batch {
Cancel(Task),
TaskDeletion(Task),
Snapshot(Vec<Task>),
Dump(Vec<Task>),
Dump(Task),
IndexOperation(IndexOperation),
IndexCreation {
index_uid: String,
@ -100,9 +104,10 @@ impl Batch {
match self {
Batch::Cancel(task)
| Batch::TaskDeletion(task)
| Batch::Dump(task)
| Batch::IndexCreation { task, .. }
| Batch::IndexUpdate { task, .. } => vec![task.uid],
Batch::Snapshot(tasks) | Batch::Dump(tasks) | Batch::IndexDeletion { tasks, .. } => {
Batch::Snapshot(tasks) | Batch::IndexDeletion { tasks, .. } => {
tasks.iter().map(|task| task.uid).collect()
}
Batch::IndexOperation(operation) => match operation {
@ -402,8 +407,11 @@ impl IndexScheduler {
// 4. we batch the dumps.
let to_dump = self.get_kind(rtxn, Kind::DumpExport)? & enqueued;
if !to_dump.is_empty() {
return Ok(Some(Batch::Dump(self.get_existing_tasks(rtxn, to_dump)?)));
if let Some(to_dump) = to_dump.min() {
return Ok(Some(Batch::Dump(
self.get_task(rtxn, to_dump)?
.ok_or(Error::CorruptedTaskQueue)?,
)));
}
// 5. We take the next task and try to batch all the tasks associated with this index.
@ -477,7 +485,80 @@ impl IndexScheduler {
Ok(vec![task])
}
Batch::Snapshot(_) => todo!(),
Batch::Dump(_) => todo!(),
Batch::Dump(mut task) => {
let KindWithContent::DumpExport { keys, instance_uid, dump_uid } = &task.kind else {
unreachable!();
};
let dump = dump::DumpWriter::new(instance_uid.clone())?;
let mut d_keys = dump.create_keys()?;
// 1. dump the keys
for key in keys {
d_keys.push_key(key)?;
}
let rtxn = self.env.read_txn()?;
// 2. dump the tasks
let mut tasks = dump.create_tasks_queue()?;
for ret in self.all_tasks.iter(&rtxn)? {
let (_, task) = ret?;
let mut dump_content_file = tasks.push_task(&task)?;
// 2.1. Dump the `content_file` associated with the task if there is one.
if let Some(content_file) = task.content_uuid() {
let content_file = self.file_store.get_update(*content_file)?;
let reader = DocumentsBatchReader::from_reader(content_file)
.map_err(milli::Error::from)?;
let (mut cursor, documents_batch_index) =
reader.into_cursor_and_fields_index();
while let Some(doc) = cursor.next_document().map_err(milli::Error::from)? {
dump_content_file
.push_document(&obkv_to_object(&doc, &documents_batch_index)?)?;
}
}
}
// TODO: maybe `self.indexes` could use this rtxn instead of creating its own
drop(rtxn);
// 3. Dump the indexes
for (uid, index) in self.indexes()? {
let rtxn = index.read_txn()?;
let metadata = IndexMetadata {
uid: uid.clone(),
primary_key: index.primary_key(&rtxn)?.map(String::from),
created_at: index.created_at(&rtxn)?,
updated_at: index.updated_at(&rtxn)?,
};
let mut index_dumper = dump.create_index(&uid, &metadata)?;
let fields_ids_map = index.fields_ids_map(&rtxn)?;
let all_fields: Vec<_> = fields_ids_map.iter().map(|(id, _)| id).collect();
// 3.1. Dump the documents
for ret in index.all_documents(&rtxn)? {
let (_id, doc) = ret?;
let document = milli::obkv_to_json(&all_fields, &fields_ids_map, doc)?;
index_dumper.push_document(&document)?;
}
// 3.2. Dump the settings
let settings = meilisearch_types::settings::settings(&index, &rtxn)?;
index_dumper.settings(&settings)?;
}
let path = self.dumps_path.join(format!("{}.dump", dump_uid));
let file = File::create(path).unwrap();
dump.persist_to(BufWriter::new(file)).unwrap();
task.status = Status::Succeeded;
Ok(vec![task])
}
Batch::IndexOperation(operation) => {
#[rustfmt::skip]
let index = match operation {
@ -679,14 +760,14 @@ impl IndexScheduler {
task.status = Status::Succeeded;
task.details = Some(Details::DocumentAddition {
received_documents: number_of_documents,
indexed_documents,
indexed_documents: Some(indexed_documents),
});
}
Err(error) => {
task.status = Status::Failed;
task.details = Some(Details::DocumentAddition {
received_documents: count,
indexed_documents: count,
indexed_documents: Some(count),
});
task.error = Some(error.into())
}

View File

@ -24,6 +24,8 @@ pub enum Error {
#[error("`{0}` is not a type. Available types are")]
InvalidKind(String),
#[error(transparent)]
Dump(#[from] dump::Error),
#[error(transparent)]
Heed(#[from] heed::Error),
#[error(transparent)]
@ -48,8 +50,9 @@ impl ErrorCode for Error {
Error::InvalidKind(_) => Code::BadRequest,
// TODO: TAMO: are all these errors really internal?
Error::Dump(e) => e.error_code(),
Error::Milli(e) => e.error_code(),
Error::Heed(_) => Code::Internal,
Error::Milli(_) => Code::Internal,
Error::FileStore(_) => Code::Internal,
Error::IoError(_) => Code::Internal,
Error::Anyhow(_) => Code::Internal,

View File

@ -154,6 +154,9 @@ pub struct IndexScheduler {
/// Weither autobatching is enabled or not.
pub(crate) autobatching_enabled: bool,
/// The path used to create the dumps.
pub(crate) dumps_path: PathBuf,
// ================= test
/// The next entry is dedicated to the tests.
/// It provide a way to break in multiple part of the scheduler.
@ -175,6 +178,7 @@ impl IndexScheduler {
tasks_path: PathBuf,
update_file_path: PathBuf,
indexes_path: PathBuf,
dumps_path: PathBuf,
index_size: usize,
indexer_config: IndexerConfig,
autobatching_enabled: bool,
@ -183,6 +187,7 @@ impl IndexScheduler {
std::fs::create_dir_all(&tasks_path)?;
std::fs::create_dir_all(&update_file_path)?;
std::fs::create_dir_all(&indexes_path)?;
std::fs::create_dir_all(&dumps_path)?;
let mut options = heed::EnvOpenOptions::new();
options.max_dbs(6);
@ -205,6 +210,7 @@ impl IndexScheduler {
// we want to start the loop right away in case meilisearch was ctrl+Ced while processing things
wake_up: Arc::new(SignalEvent::auto(true)),
autobatching_enabled,
dumps_path,
#[cfg(test)]
test_breakpoint_sdr,
@ -227,6 +233,7 @@ impl IndexScheduler {
index_mapper: self.index_mapper.clone(),
wake_up: self.wake_up.clone(),
autobatching_enabled: self.autobatching_enabled,
dumps_path: self.dumps_path.clone(),
#[cfg(test)]
test_breakpoint_sdr: self.test_breakpoint_sdr.clone(),
@ -342,7 +349,7 @@ impl IndexScheduler {
started_at: None,
finished_at: None,
error: None,
details: task.default_details(),
details: (&task).into(),
status: Status::Enqueued,
kind: task,
};
@ -367,9 +374,9 @@ impl IndexScheduler {
match wtxn.commit() {
Ok(()) => (),
e @ Err(_) => {
_e @ Err(_) => {
todo!("remove the data associated with the task");
e?;
// _e?;
}
}
@ -436,6 +443,7 @@ impl IndexScheduler {
// TODO the info field should've been set by the process_batch function
self.update_task(&mut wtxn, &task)?;
}
log::info!("A batch of tasks was successfully completed.");
}
// In case of a failure we must get back and patch all the tasks with the error.
Err(err) => {
@ -453,7 +461,6 @@ impl IndexScheduler {
}
*self.processing_tasks.write().unwrap() = (finished_at, RoaringBitmap::new());
wtxn.commit()?;
log::info!("A batch of tasks was successfully completed.");
#[cfg(test)]
self.test_breakpoint_sdr
@ -542,6 +549,7 @@ mod tests {
tempdir.path().join("db_path"),
tempdir.path().join("file_store"),
tempdir.path().join("indexes"),
tempdir.path().join("dumps"),
1024 * 1024,
IndexerConfig::default(),
autobatching, // enable autobatching

View File

@ -1,16 +1,19 @@
use std::{any::Any, sync::Arc};
use actix_web::HttpRequest;
use meilisearch_types::InstanceUid;
use serde_json::Value;
use crate::{routes::indexes::documents::UpdateDocumentsQuery, Opt};
use super::{find_user_id, Analytics};
pub struct MockAnalytics;
pub struct MockAnalytics {
instance_uid: Option<InstanceUid>,
}
#[derive(Default)]
pub struct SearchAggregator {}
pub struct SearchAggregator;
#[allow(dead_code)]
impl SearchAggregator {
@ -23,13 +26,17 @@ impl SearchAggregator {
impl MockAnalytics {
#[allow(clippy::new_ret_no_self)]
pub fn new(opt: &Opt) -> (Arc<dyn Analytics>, String) {
let user = find_user_id(&opt.db_path).unwrap_or_default();
(Arc::new(Self), user)
pub fn new(opt: &Opt) -> Arc<dyn Analytics> {
let instance_uid = find_user_id(&opt.db_path);
Arc::new(Self { instance_uid })
}
}
impl Analytics for MockAnalytics {
fn instance_uid(&self) -> Option<&meilisearch_types::InstanceUid> {
self.instance_uid.as_ref()
}
// These methods are noop and should be optimized out
fn publish(&self, _event_name: String, _send: Value, _request: Option<&HttpRequest>) {}
fn get_search(&self, _aggregate: super::SearchAggregator) {}

View File

@ -5,8 +5,10 @@ mod segment_analytics;
use std::fs;
use std::path::{Path, PathBuf};
use std::str::FromStr;
use actix_web::HttpRequest;
use meilisearch_types::InstanceUid;
use once_cell::sync::Lazy;
use platform_dirs::AppDirs;
use serde_json::Value;
@ -51,13 +53,16 @@ fn config_user_id_path(db_path: &Path) -> Option<PathBuf> {
}
/// Look for the instance-uid in the `data.ms` or in `~/.config/Meilisearch/path-to-db-instance-uid`
fn find_user_id(db_path: &Path) -> Option<String> {
fn find_user_id(db_path: &Path) -> Option<InstanceUid> {
fs::read_to_string(db_path.join("instance-uid"))
.ok()
.or_else(|| fs::read_to_string(&config_user_id_path(db_path)?).ok())
.and_then(|uid| InstanceUid::from_str(&uid).ok())
}
pub trait Analytics: Sync + Send {
fn instance_uid(&self) -> Option<&InstanceUid>;
/// The method used to publish most analytics that do not need to be batched every hours
fn publish(&self, event_name: String, send: Value, request: Option<&HttpRequest>);

View File

@ -37,6 +37,7 @@ pub fn setup_meilisearch(opt: &Opt) -> anyhow::Result<IndexScheduler> {
opt.db_path.join("tasks"),
opt.db_path.join("update_files"),
opt.db_path.join("indexes"),
opt.dumps_dir.clone(),
opt.max_index_size.get_bytes() as usize,
(&opt.indexer_options).try_into()?,
true,

View File

@ -53,15 +53,15 @@ async fn main() -> anyhow::Result<()> {
let auth_controller = AuthController::new(&opt.db_path, &opt.master_key)?;
#[cfg(all(not(debug_assertions), feature = "analytics"))]
let (analytics, user) = if !opt.no_analytics {
let analytics = if !opt.no_analytics {
analytics::SegmentAnalytics::new(&opt, &meilisearch).await
} else {
analytics::MockAnalytics::new(&opt)
};
#[cfg(any(debug_assertions, not(feature = "analytics")))]
let (analytics, user) = analytics::MockAnalytics::new(&opt);
let analytics = analytics::MockAnalytics::new(&opt);
print_launch_resume(&opt, &user, config_read_from);
print_launch_resume(&opt, analytics.clone(), config_read_from);
run_http(index_scheduler, auth_controller, opt, analytics).await?;
@ -133,7 +133,7 @@ async fn run_http(
Ok(())
}
pub fn print_launch_resume(opt: &Opt, user: &str, config_read_from: Option<PathBuf>) {
pub fn print_launch_resume(opt: &Opt, analytics: Arc<dyn Analytics>, config_read_from: Option<PathBuf>) {
let commit_sha = option_env!("VERGEN_GIT_SHA").unwrap_or("unknown");
let commit_date = option_env!("VERGEN_GIT_COMMIT_TIMESTAMP").unwrap_or("unknown");
let protocol = if opt.ssl_cert_path.is_some() && opt.ssl_key_path.is_some() {
@ -186,8 +186,8 @@ Anonymous telemetry:\t\"Enabled\""
}
}
if !user.is_empty() {
eprintln!("Instance UID:\t\t\"{}\"", user);
if let Some(instance_uid) = analytics.instance_uid() {
eprintln!("Instance UID:\t\t\"{}\"", instance_uid.to_string());
}
eprintln!();

View File

@ -2,13 +2,17 @@ use actix_web::web::Data;
use actix_web::{web, HttpRequest, HttpResponse};
use index_scheduler::IndexScheduler;
use log::debug;
use meilisearch_auth::AuthController;
use meilisearch_types::error::ResponseError;
use meilisearch_types::tasks::KindWithContent;
use serde_json::json;
use time::macros::format_description;
use time::OffsetDateTime;
use crate::analytics::Analytics;
use crate::extractors::authentication::{policies::*, GuardedData};
use crate::extractors::sequential_extractor::SeqHandler;
use crate::routes::SummarizedTaskView;
pub fn configure(cfg: &mut web::ServiceConfig) {
cfg.service(web::resource("").route(web::post().to(SeqHandler(create_dump))));
@ -16,16 +20,28 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
pub async fn create_dump(
index_scheduler: GuardedData<ActionPolicy<{ actions::DUMPS_CREATE }>, Data<IndexScheduler>>,
auth_controller: GuardedData<ActionPolicy<{ actions::DUMPS_CREATE }>, AuthController>,
req: HttpRequest,
analytics: web::Data<dyn Analytics>,
) -> Result<HttpResponse, ResponseError> {
analytics.publish("Dump Created".to_string(), json!({}), Some(&req));
let task = KindWithContent::DumpExport {
output: "todo".to_string().into(),
};
let res = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let dump_uid = OffsetDateTime::now_utc()
.format(format_description!(
"[year repr:full][month repr:numerical][day padding:zero]-[hour padding:zero][minute padding:zero][second padding:zero][subsecond digits:3]"
))
.unwrap();
debug!("returns: {:?}", res);
Ok(HttpResponse::Accepted().json(res))
let task = KindWithContent::DumpExport {
keys: auth_controller.list_keys()?,
instance_uid: analytics.instance_uid().cloned(),
dump_uid,
};
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))
}

View File

@ -109,7 +109,10 @@ pub async fn delete_document(
index_uid,
documents_ids: vec![document_id],
};
let task = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))
}
@ -314,7 +317,10 @@ pub async fn delete_documents(
index_uid: path.into_inner(),
documents_ids: ids,
};
let task = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))
@ -327,7 +333,10 @@ pub async fn clear_all_documents(
let task = KindWithContent::DocumentClear {
index_uid: path.into_inner(),
};
let task = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))

View File

@ -13,7 +13,7 @@ use crate::analytics::Analytics;
use crate::extractors::authentication::{policies::*, AuthenticationError, GuardedData};
use crate::extractors::sequential_extractor::SeqHandler;
use super::Pagination;
use super::{Pagination, SummarizedTaskView};
pub mod documents;
pub mod search;
@ -108,7 +108,10 @@ pub async fn create_index(
index_uid: uid,
primary_key,
};
let task = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
Ok(HttpResponse::Accepted().json(task))
} else {
@ -156,7 +159,10 @@ pub async fn update_index(
primary_key: body.primary_key,
};
let task = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))
@ -169,7 +175,10 @@ pub async fn delete_index(
let task = KindWithContent::IndexDeletion {
index_uid: index_uid.into_inner(),
};
let task = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
Ok(HttpResponse::Accepted().json(task))
}

View File

@ -1,27 +1,16 @@
use std::collections::BTreeSet;
use std::marker::PhantomData;
use actix_web::web::Data;
use fst::IntoStreamer;
use log::debug;
use actix_web::{web, HttpRequest, HttpResponse};
use index_scheduler::IndexScheduler;
use meilisearch_types::error::ResponseError;
use meilisearch_types::heed::RoTxn;
use meilisearch_types::milli::update::Setting;
use meilisearch_types::milli::{self, DEFAULT_VALUES_PER_FACET};
use meilisearch_types::settings::{
Checked, FacetingSettings, MinWordSizeTyposSetting, PaginationSettings, Settings, TypoSettings,
Unchecked,
};
use meilisearch_types::settings::{settings, Settings, Unchecked};
use meilisearch_types::tasks::KindWithContent;
use meilisearch_types::Index;
use serde_json::json;
use crate::analytics::Analytics;
use crate::extractors::authentication::{policies::*, GuardedData};
use crate::search::DEFAULT_PAGINATION_MAX_TOTAL_HITS;
use crate::routes::SummarizedTaskView;
#[macro_export]
macro_rules! make_setting_route {
@ -33,14 +22,14 @@ macro_rules! make_setting_route {
use index_scheduler::IndexScheduler;
use meilisearch_types::milli::update::Setting;
use meilisearch_types::settings::Settings;
use meilisearch_types::settings::{settings, Settings};
use meilisearch_types::tasks::KindWithContent;
use meilisearch_types::error::ResponseError;
use $crate::analytics::Analytics;
use $crate::extractors::authentication::{policies::*, GuardedData};
use $crate::extractors::sequential_extractor::SeqHandler;
use $crate::routes::indexes::settings::settings;
use $crate::routes::SummarizedTaskView;
pub async fn delete(
index_scheduler: GuardedData<
@ -61,8 +50,10 @@ macro_rules! make_setting_route {
is_deletion: true,
allow_index_creation,
};
let task =
tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))
@ -97,8 +88,10 @@ macro_rules! make_setting_route {
is_deletion: false,
allow_index_creation,
};
let task =
tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))
@ -459,7 +452,10 @@ pub async fn update_all(
is_deletion: false,
allow_index_creation,
};
let task = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))
@ -489,113 +485,11 @@ pub async fn delete_all(
is_deletion: true,
allow_index_creation,
};
let task = tokio::task::spawn_blocking(move || index_scheduler.register(task)).await??;
let task: SummarizedTaskView =
tokio::task::spawn_blocking(move || index_scheduler.register(task))
.await??
.into();
debug!("returns: {:?}", task);
Ok(HttpResponse::Accepted().json(task))
}
pub fn settings(index: &Index, rtxn: &RoTxn) -> Result<Settings<Checked>, milli::Error> {
let displayed_attributes = index
.displayed_fields(rtxn)?
.map(|fields| fields.into_iter().map(String::from).collect());
let searchable_attributes = index
.user_defined_searchable_fields(rtxn)?
.map(|fields| fields.into_iter().map(String::from).collect());
let filterable_attributes = index.filterable_fields(rtxn)?.into_iter().collect();
let sortable_attributes = index.sortable_fields(rtxn)?.into_iter().collect();
let criteria = index
.criteria(rtxn)?
.into_iter()
.map(|c| c.to_string())
.collect();
let stop_words = index
.stop_words(rtxn)?
.map(|stop_words| -> Result<BTreeSet<_>, milli::Error> {
Ok(stop_words.stream().into_strs()?.into_iter().collect())
})
.transpose()?
.unwrap_or_default();
let distinct_field = index.distinct_field(rtxn)?.map(String::from);
// in milli each word in the synonyms map were split on their separator. Since we lost
// this information we are going to put space between words.
let synonyms = index
.synonyms(rtxn)?
.iter()
.map(|(key, values)| {
(
key.join(" "),
values.iter().map(|value| value.join(" ")).collect(),
)
})
.collect();
let min_typo_word_len = MinWordSizeTyposSetting {
one_typo: Setting::Set(index.min_word_len_one_typo(rtxn)?),
two_typos: Setting::Set(index.min_word_len_two_typos(rtxn)?),
};
let disabled_words = match index.exact_words(rtxn)? {
Some(fst) => fst.into_stream().into_strs()?.into_iter().collect(),
None => BTreeSet::new(),
};
let disabled_attributes = index
.exact_attributes(rtxn)?
.into_iter()
.map(String::from)
.collect();
let typo_tolerance = TypoSettings {
enabled: Setting::Set(index.authorize_typos(rtxn)?),
min_word_size_for_typos: Setting::Set(min_typo_word_len),
disable_on_words: Setting::Set(disabled_words),
disable_on_attributes: Setting::Set(disabled_attributes),
};
let faceting = FacetingSettings {
max_values_per_facet: Setting::Set(
index
.max_values_per_facet(rtxn)?
.unwrap_or(DEFAULT_VALUES_PER_FACET),
),
};
let pagination = PaginationSettings {
max_total_hits: Setting::Set(
index
.pagination_max_total_hits(rtxn)?
.unwrap_or(DEFAULT_PAGINATION_MAX_TOTAL_HITS),
),
};
Ok(Settings {
displayed_attributes: match displayed_attributes {
Some(attrs) => Setting::Set(attrs),
None => Setting::Reset,
},
searchable_attributes: match searchable_attributes {
Some(attrs) => Setting::Set(attrs),
None => Setting::Reset,
},
filterable_attributes: Setting::Set(filterable_attributes),
sortable_attributes: Setting::Set(sortable_attributes),
ranking_rules: Setting::Set(criteria),
stop_words: Setting::Set(stop_words),
distinct_attribute: match distinct_field {
Some(field) => Setting::Set(field),
None => Setting::Reset,
},
synonyms: Setting::Set(synonyms),
typo_tolerance: Setting::Set(typo_tolerance),
faceting: Setting::Set(faceting),
pagination: Setting::Set(pagination),
_kind: PhantomData,
})
}

View File

@ -121,7 +121,7 @@ impl From<Details> for DetailsView {
indexed_documents,
} => DetailsView {
received_documents: Some(received_documents),
indexed_documents: Some(indexed_documents),
indexed_documents,
..DetailsView::default()
},
Details::Settings { settings } => DetailsView {

View File

@ -4,6 +4,7 @@ use std::str::FromStr;
use std::time::Instant;
use either::Either;
use meilisearch_types::settings::DEFAULT_PAGINATION_MAX_TOTAL_HITS;
use meilisearch_types::{milli, Document};
use milli::tokenizer::TokenizerBuilder;
use milli::{
@ -24,10 +25,6 @@ pub const DEFAULT_CROP_MARKER: fn() -> String = || "…".to_string();
pub const DEFAULT_HIGHLIGHT_PRE_TAG: fn() -> String = || "<em>".to_string();
pub const DEFAULT_HIGHLIGHT_POST_TAG: fn() -> String = || "</em>".to_string();
/// The maximimum number of results that the engine
/// will be able to return in one search call.
pub const DEFAULT_PAGINATION_MAX_TOTAL_HITS: usize = 1000;
#[derive(Deserialize, Debug, Clone, PartialEq, Eq)]
#[serde(rename_all = "camelCase", deny_unknown_fields)]
pub struct SearchQuery {

View File

@ -10,6 +10,7 @@ csv = "1.1.6"
either = { version = "1.6.1", features = ["serde"] }
milli = { git = "https://github.com/meilisearch/milli.git", branch = "indexation-abortion", default-features = false }
enum-iterator = "0.7.0"
fst = "0.4.7"
proptest = { version = "1.0.0", optional = true }
proptest-derive = { version = "0.3.0", optional = true }
roaring = { version = "0.10.0", features = ["serde"] }

View File

@ -9,5 +9,7 @@ pub mod tasks;
pub use milli;
pub use milli::heed;
pub use milli::Index;
use uuid::Uuid;
pub type Document = serde_json::Map<String, serde_json::Value>;
pub type InstanceUid = Uuid;

View File

@ -2,9 +2,15 @@ use std::collections::{BTreeMap, BTreeSet};
use std::marker::PhantomData;
use std::num::NonZeroUsize;
use fst::IntoStreamer;
use milli::update::Setting;
use milli::{Index, DEFAULT_VALUES_PER_FACET};
use serde::{Deserialize, Serialize, Serializer};
/// The maximimum number of results that the engine
/// will be able to return in one search call.
pub const DEFAULT_PAGINATION_MAX_TOTAL_HITS: usize = 1000;
fn serialize_with_wildcard<S>(
field: &Setting<Vec<String>>,
s: S,
@ -366,6 +372,114 @@ pub fn apply_settings_to_builder(
}
}
pub fn settings(
index: &Index,
rtxn: &crate::heed::RoTxn,
) -> Result<Settings<Checked>, milli::Error> {
let displayed_attributes = index
.displayed_fields(rtxn)?
.map(|fields| fields.into_iter().map(String::from).collect());
let searchable_attributes = index
.user_defined_searchable_fields(rtxn)?
.map(|fields| fields.into_iter().map(String::from).collect());
let filterable_attributes = index.filterable_fields(rtxn)?.into_iter().collect();
let sortable_attributes = index.sortable_fields(rtxn)?.into_iter().collect();
let criteria = index
.criteria(rtxn)?
.into_iter()
.map(|c| c.to_string())
.collect();
let stop_words = index
.stop_words(rtxn)?
.map(|stop_words| -> Result<BTreeSet<_>, milli::Error> {
Ok(stop_words.stream().into_strs()?.into_iter().collect())
})
.transpose()?
.unwrap_or_default();
let distinct_field = index.distinct_field(rtxn)?.map(String::from);
// in milli each word in the synonyms map were split on their separator. Since we lost
// this information we are going to put space between words.
let synonyms = index
.synonyms(rtxn)?
.iter()
.map(|(key, values)| {
(
key.join(" "),
values.iter().map(|value| value.join(" ")).collect(),
)
})
.collect();
let min_typo_word_len = MinWordSizeTyposSetting {
one_typo: Setting::Set(index.min_word_len_one_typo(rtxn)?),
two_typos: Setting::Set(index.min_word_len_two_typos(rtxn)?),
};
let disabled_words = match index.exact_words(rtxn)? {
Some(fst) => fst.into_stream().into_strs()?.into_iter().collect(),
None => BTreeSet::new(),
};
let disabled_attributes = index
.exact_attributes(rtxn)?
.into_iter()
.map(String::from)
.collect();
let typo_tolerance = TypoSettings {
enabled: Setting::Set(index.authorize_typos(rtxn)?),
min_word_size_for_typos: Setting::Set(min_typo_word_len),
disable_on_words: Setting::Set(disabled_words),
disable_on_attributes: Setting::Set(disabled_attributes),
};
let faceting = FacetingSettings {
max_values_per_facet: Setting::Set(
index
.max_values_per_facet(rtxn)?
.unwrap_or(DEFAULT_VALUES_PER_FACET),
),
};
let pagination = PaginationSettings {
max_total_hits: Setting::Set(
index
.pagination_max_total_hits(rtxn)?
.unwrap_or(DEFAULT_PAGINATION_MAX_TOTAL_HITS),
),
};
Ok(Settings {
displayed_attributes: match displayed_attributes {
Some(attrs) => Setting::Set(attrs),
None => Setting::Reset,
},
searchable_attributes: match searchable_attributes {
Some(attrs) => Setting::Set(attrs),
None => Setting::Reset,
},
filterable_attributes: Setting::Set(filterable_attributes),
sortable_attributes: Setting::Set(sortable_attributes),
ranking_rules: Setting::Set(criteria),
stop_words: Setting::Set(stop_words),
distinct_attribute: match distinct_field {
Some(field) => Setting::Set(field),
None => Setting::Reset,
},
synonyms: Setting::Set(synonyms),
typo_tolerance: Setting::Set(typo_tolerance),
faceting: Setting::Set(faceting),
pagination: Setting::Set(pagination),
_kind: PhantomData,
})
}
#[cfg(test)]
pub(crate) mod test {
use proptest::prelude::*;

View File

@ -3,7 +3,6 @@ use roaring::RoaringBitmap;
use serde::{Deserialize, Serialize, Serializer};
use std::{
fmt::{Display, Write},
path::PathBuf,
str::FromStr,
};
use time::{Duration, OffsetDateTime};
@ -11,7 +10,9 @@ use uuid::Uuid;
use crate::{
error::{Code, ResponseError},
keys::Key,
settings::{Settings, Unchecked},
InstanceUid,
};
pub type TaskId = u32;
@ -71,6 +72,26 @@ impl Task {
IndexSwap { lhs, rhs } => Some(vec![lhs, rhs]),
}
}
/// Return the content-uuid if there is one
pub fn content_uuid(&self) -> Option<&Uuid> {
match self.kind {
KindWithContent::DocumentImport {
ref content_file, ..
} => Some(content_file),
KindWithContent::DocumentDeletion { .. }
| KindWithContent::DocumentClear { .. }
| KindWithContent::Settings { .. }
| KindWithContent::IndexDeletion { .. }
| KindWithContent::IndexCreation { .. }
| KindWithContent::IndexUpdate { .. }
| KindWithContent::IndexSwap { .. }
| KindWithContent::CancelTask { .. }
| KindWithContent::DeleteTasks { .. }
| KindWithContent::DumpExport { .. }
| KindWithContent::Snapshot => None,
}
}
}
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
@ -120,7 +141,9 @@ pub enum KindWithContent {
tasks: RoaringBitmap,
},
DumpExport {
output: PathBuf,
dump_uid: String,
keys: Vec<Key>,
instance_uid: Option<InstanceUid>,
},
Snapshot,
}
@ -167,7 +190,7 @@ impl KindWithContent {
documents_count, ..
} => Some(Details::DocumentAddition {
received_documents: *documents_count,
indexed_documents: 0,
indexed_documents: Some(0),
}),
KindWithContent::DocumentDeletion {
index_uid: _,
@ -204,6 +227,38 @@ impl KindWithContent {
}
}
impl From<&KindWithContent> for Option<Details> {
fn from(kind: &KindWithContent) -> Self {
match kind {
KindWithContent::DocumentImport {
documents_count, ..
} => Some(Details::DocumentAddition {
received_documents: *documents_count,
indexed_documents: None,
}),
KindWithContent::DocumentDeletion { .. } => None,
KindWithContent::DocumentClear { .. } => None,
KindWithContent::Settings { new_settings, .. } => Some(Details::Settings {
settings: new_settings.clone(),
}),
KindWithContent::IndexDeletion { .. } => None,
KindWithContent::IndexCreation { primary_key, .. } => Some(Details::IndexInfo {
primary_key: primary_key.clone(),
}),
KindWithContent::IndexUpdate { primary_key, .. } => Some(Details::IndexInfo {
primary_key: primary_key.clone(),
}),
KindWithContent::IndexSwap { .. } => None,
KindWithContent::CancelTask { .. } => None,
KindWithContent::DeleteTasks { .. } => todo!(),
KindWithContent::DumpExport { dump_uid, .. } => Some(Details::Dump {
dump_uid: dump_uid.clone(),
}),
KindWithContent::Snapshot => None,
}
}
}
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub enum Status {
@ -289,7 +344,7 @@ impl FromStr for Kind {
pub enum Details {
DocumentAddition {
received_documents: u64,
indexed_documents: u64,
indexed_documents: Option<u64>,
},
Settings {
settings: Settings<Unchecked>,