mirror of
				https://github.com/meilisearch/meilisearch.git
				synced 2025-11-04 01:46:28 +00:00 
			
		
		
		
	wip integrating the scheduler in meilisearch-http
This commit is contained in:
		
				
					committed by
					
						
						Clément Renault
					
				
			
			
				
	
			
			
			
						parent
						
							250410495c
						
					
				
				
					commit
					8d51c1f389
				
			
							
								
								
									
										3
									
								
								Cargo.lock
									
									
									
										generated
									
									
									
								
							
							
						
						
									
										3
									
								
								Cargo.lock
									
									
									
										generated
									
									
									
								
							@@ -2332,6 +2332,7 @@ dependencies = [
 | 
			
		||||
 "cargo_toml",
 | 
			
		||||
 "clap 4.0.9",
 | 
			
		||||
 "crossbeam-channel",
 | 
			
		||||
 "document-formats",
 | 
			
		||||
 "either",
 | 
			
		||||
 "env_logger",
 | 
			
		||||
 "flate2",
 | 
			
		||||
@@ -2340,6 +2341,8 @@ dependencies = [
 | 
			
		||||
 "futures-util",
 | 
			
		||||
 "hex",
 | 
			
		||||
 "http",
 | 
			
		||||
 "index",
 | 
			
		||||
 "index-scheduler",
 | 
			
		||||
 "indexmap",
 | 
			
		||||
 "itertools",
 | 
			
		||||
 "jsonwebtoken",
 | 
			
		||||
 
 | 
			
		||||
@@ -30,7 +30,7 @@ pub(crate) enum Batch {
 | 
			
		||||
    },
 | 
			
		||||
    DocumentDeletion {
 | 
			
		||||
        index_uid: String,
 | 
			
		||||
        documents: Vec<DocumentId>,
 | 
			
		||||
        documents: Vec<String>,
 | 
			
		||||
        tasks: Vec<Task>,
 | 
			
		||||
    },
 | 
			
		||||
    DocumentClear {
 | 
			
		||||
 
 | 
			
		||||
@@ -1,3 +1,4 @@
 | 
			
		||||
use meilisearch_types::error::{Code, ErrorCode};
 | 
			
		||||
use milli::heed;
 | 
			
		||||
use thiserror::Error;
 | 
			
		||||
 | 
			
		||||
@@ -13,6 +14,13 @@ pub enum Error {
 | 
			
		||||
    CorruptedTaskQueue,
 | 
			
		||||
    #[error("Task `{0}` not found")]
 | 
			
		||||
    TaskNotFound(TaskId),
 | 
			
		||||
 | 
			
		||||
    // maybe the two next errors are going to move to the frontend
 | 
			
		||||
    #[error("`{0}` is not a status. Available status are")]
 | 
			
		||||
    InvalidStatus(String),
 | 
			
		||||
    #[error("`{0}` is not a type. Available types are")]
 | 
			
		||||
    InvalidKind(String),
 | 
			
		||||
 | 
			
		||||
    #[error(transparent)]
 | 
			
		||||
    Heed(#[from] heed::Error),
 | 
			
		||||
    #[error(transparent)]
 | 
			
		||||
@@ -27,3 +35,22 @@ pub enum Error {
 | 
			
		||||
    #[error(transparent)]
 | 
			
		||||
    Anyhow(#[from] anyhow::Error),
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl ErrorCode for Error {
 | 
			
		||||
    fn error_code(&self) -> Code {
 | 
			
		||||
        match self {
 | 
			
		||||
            Error::IndexNotFound(_) => Code::IndexNotFound,
 | 
			
		||||
            Error::IndexAlreadyExists(_) => Code::IndexAlreadyExists,
 | 
			
		||||
            Error::TaskNotFound(_) => Code::TaskNotFound,
 | 
			
		||||
            Error::InvalidStatus(_) => todo!(),
 | 
			
		||||
            Error::InvalidKind(_) => todo!(),
 | 
			
		||||
            Error::Heed(_) => todo!(),
 | 
			
		||||
            Error::Milli(_) => todo!(),
 | 
			
		||||
            Error::IndexError(_) => todo!(),
 | 
			
		||||
            Error::FileStore(_) => todo!(),
 | 
			
		||||
            Error::IoError(_) => todo!(),
 | 
			
		||||
            Error::Anyhow(_) => Code::Internal,
 | 
			
		||||
            Error::CorruptedTaskQueue => Code::Internal,
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
@@ -1,11 +1,12 @@
 | 
			
		||||
use crate::index_mapper::IndexMapper;
 | 
			
		||||
use crate::task::{Kind, KindWithContent, Status, Task, TaskView};
 | 
			
		||||
use crate::{Error, Result, TaskId};
 | 
			
		||||
use file_store::FileStore;
 | 
			
		||||
use file_store::{File, FileStore};
 | 
			
		||||
use index::Index;
 | 
			
		||||
use milli::update::IndexerConfig;
 | 
			
		||||
use synchronoise::SignalEvent;
 | 
			
		||||
use time::OffsetDateTime;
 | 
			
		||||
use uuid::Uuid;
 | 
			
		||||
 | 
			
		||||
use std::path::PathBuf;
 | 
			
		||||
use std::sync::Arc;
 | 
			
		||||
@@ -24,12 +25,12 @@ const DEFAULT_LIMIT: fn() -> u32 = || 20;
 | 
			
		||||
#[serde(rename_all = "camelCase")]
 | 
			
		||||
pub struct Query {
 | 
			
		||||
    #[serde(default = "DEFAULT_LIMIT")]
 | 
			
		||||
    limit: u32,
 | 
			
		||||
    from: Option<u32>,
 | 
			
		||||
    status: Option<Vec<Status>>,
 | 
			
		||||
    pub limit: u32,
 | 
			
		||||
    pub from: Option<u32>,
 | 
			
		||||
    pub status: Option<Vec<Status>>,
 | 
			
		||||
    #[serde(rename = "type")]
 | 
			
		||||
    kind: Option<Vec<Kind>>,
 | 
			
		||||
    index_uid: Option<Vec<String>>,
 | 
			
		||||
    pub kind: Option<Vec<Kind>>,
 | 
			
		||||
    pub index_uid: Option<Vec<String>>,
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl Default for Query {
 | 
			
		||||
@@ -62,6 +63,15 @@ impl Query {
 | 
			
		||||
            ..self
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    pub fn with_index(self, index_uid: String) -> Self {
 | 
			
		||||
        let mut index_vec = self.index_uid.unwrap_or_default();
 | 
			
		||||
        index_vec.push(index_uid);
 | 
			
		||||
        Self {
 | 
			
		||||
            index_uid: Some(index_vec),
 | 
			
		||||
            ..self
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
pub mod db_name {
 | 
			
		||||
@@ -193,15 +203,6 @@ impl IndexScheduler {
 | 
			
		||||
        Ok(tasks.into_iter().map(|task| task.as_task_view()).collect())
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    /// Returns the tasks corresponding to the query.
 | 
			
		||||
    pub fn task(&self, uid: TaskId) -> Result<TaskView> {
 | 
			
		||||
        let rtxn = self.env.read_txn()?;
 | 
			
		||||
        self.get_task(&rtxn, uid).and_then(|opt| {
 | 
			
		||||
            opt.ok_or(Error::TaskNotFound(uid))
 | 
			
		||||
                .map(|task| task.as_task_view())
 | 
			
		||||
        })
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    /// Register a new task in the scheduler. If it fails and data was associated with the task
 | 
			
		||||
    /// it tries to delete the file.
 | 
			
		||||
    pub fn register(&self, task: KindWithContent) -> Result<TaskView> {
 | 
			
		||||
@@ -251,6 +252,10 @@ impl IndexScheduler {
 | 
			
		||||
        Ok(task.as_task_view())
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    pub fn create_update_file(&self) -> Result<(Uuid, File)> {
 | 
			
		||||
        Ok(self.file_store.new_update()?)
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    /// This worker function must be run in a different thread and must be run only once.
 | 
			
		||||
    pub fn run(&self) -> ! {
 | 
			
		||||
        loop {
 | 
			
		||||
@@ -422,10 +427,8 @@ mod tests {
 | 
			
		||||
            "doggo": "bob"
 | 
			
		||||
        }"#;
 | 
			
		||||
 | 
			
		||||
        let (uuid, mut file) = index_scheduler.file_store.new_update().unwrap();
 | 
			
		||||
        let (uuid, mut file) = index_scheduler.create_update_file().unwrap();
 | 
			
		||||
        document_formats::read_json(content.as_bytes(), file.as_file_mut()).unwrap();
 | 
			
		||||
        file.persist().unwrap();
 | 
			
		||||
 | 
			
		||||
        index_scheduler
 | 
			
		||||
            .register(KindWithContent::DocumentAddition {
 | 
			
		||||
                index_uid: S("doggos"),
 | 
			
		||||
@@ -435,6 +438,7 @@ mod tests {
 | 
			
		||||
                allow_index_creation: true,
 | 
			
		||||
            })
 | 
			
		||||
            .unwrap();
 | 
			
		||||
        file.persist().unwrap();
 | 
			
		||||
 | 
			
		||||
        index_scheduler.tick().unwrap();
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -11,10 +11,7 @@ pub type TaskId = u32;
 | 
			
		||||
 | 
			
		||||
pub use crate::index_scheduler::{IndexScheduler, Query};
 | 
			
		||||
pub use error::Error;
 | 
			
		||||
/// from the exterior you don't need to know there is multiple type of `Kind`
 | 
			
		||||
pub use task::KindWithContent;
 | 
			
		||||
/// from the exterior you don't need to know there is multiple type of `Task`
 | 
			
		||||
pub use task::TaskView;
 | 
			
		||||
pub use task::{Kind, KindWithContent, Status, TaskView};
 | 
			
		||||
 | 
			
		||||
#[cfg(test)]
 | 
			
		||||
mod tests {
 | 
			
		||||
 
 | 
			
		||||
@@ -2,22 +2,12 @@ use anyhow::Result;
 | 
			
		||||
use index::{Settings, Unchecked};
 | 
			
		||||
use meilisearch_types::error::ResponseError;
 | 
			
		||||
 | 
			
		||||
use milli::DocumentId;
 | 
			
		||||
use serde::{Deserialize, Serialize, Serializer};
 | 
			
		||||
use std::{fmt::Write, path::PathBuf};
 | 
			
		||||
use std::{fmt::Write, path::PathBuf, str::FromStr};
 | 
			
		||||
use time::{Duration, OffsetDateTime};
 | 
			
		||||
use uuid::Uuid;
 | 
			
		||||
 | 
			
		||||
use crate::TaskId;
 | 
			
		||||
 | 
			
		||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)]
 | 
			
		||||
#[serde(rename_all = "camelCase")]
 | 
			
		||||
pub enum Status {
 | 
			
		||||
    Enqueued,
 | 
			
		||||
    Processing,
 | 
			
		||||
    Succeeded,
 | 
			
		||||
    Failed,
 | 
			
		||||
}
 | 
			
		||||
use crate::{Error, TaskId};
 | 
			
		||||
 | 
			
		||||
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
 | 
			
		||||
#[serde(rename_all = "camelCase")]
 | 
			
		||||
@@ -98,6 +88,29 @@ impl Task {
 | 
			
		||||
    }
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)]
 | 
			
		||||
#[serde(rename_all = "camelCase")]
 | 
			
		||||
pub enum Status {
 | 
			
		||||
    Enqueued,
 | 
			
		||||
    Processing,
 | 
			
		||||
    Succeeded,
 | 
			
		||||
    Failed,
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl FromStr for Status {
 | 
			
		||||
    type Err = Error;
 | 
			
		||||
 | 
			
		||||
    fn from_str(s: &str) -> Result<Self, Self::Err> {
 | 
			
		||||
        match s {
 | 
			
		||||
            "enqueued" => Ok(Status::Enqueued),
 | 
			
		||||
            "processing" => Ok(Status::Processing),
 | 
			
		||||
            "succeeded" => Ok(Status::Succeeded),
 | 
			
		||||
            "failed" => Ok(Status::Failed),
 | 
			
		||||
            s => Err(Error::InvalidStatus(s.to_string())),
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
#[derive(Debug, Serialize, Deserialize)]
 | 
			
		||||
#[serde(rename_all = "camelCase")]
 | 
			
		||||
pub enum KindWithContent {
 | 
			
		||||
@@ -117,7 +130,7 @@ pub enum KindWithContent {
 | 
			
		||||
    },
 | 
			
		||||
    DocumentDeletion {
 | 
			
		||||
        index_uid: String,
 | 
			
		||||
        documents_ids: Vec<DocumentId>,
 | 
			
		||||
        documents_ids: Vec<String>,
 | 
			
		||||
    },
 | 
			
		||||
    DocumentClear {
 | 
			
		||||
        index_uid: String,
 | 
			
		||||
@@ -261,6 +274,29 @@ pub enum Kind {
 | 
			
		||||
    Snapshot,
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl FromStr for Kind {
 | 
			
		||||
    type Err = Error;
 | 
			
		||||
 | 
			
		||||
    fn from_str(s: &str) -> Result<Self, Self::Err> {
 | 
			
		||||
        match s {
 | 
			
		||||
            "document_addition" => Ok(Kind::DocumentAddition),
 | 
			
		||||
            "document_update" => Ok(Kind::DocumentUpdate),
 | 
			
		||||
            "document_deletion" => Ok(Kind::DocumentDeletion),
 | 
			
		||||
            "document_clear" => Ok(Kind::DocumentClear),
 | 
			
		||||
            "settings" => Ok(Kind::Settings),
 | 
			
		||||
            "index_creation" => Ok(Kind::IndexCreation),
 | 
			
		||||
            "index_deletion" => Ok(Kind::IndexDeletion),
 | 
			
		||||
            "index_update" => Ok(Kind::IndexUpdate),
 | 
			
		||||
            "index_rename" => Ok(Kind::IndexRename),
 | 
			
		||||
            "index_swap" => Ok(Kind::IndexSwap),
 | 
			
		||||
            "cancel_task" => Ok(Kind::CancelTask),
 | 
			
		||||
            "dump_export" => Ok(Kind::DumpExport),
 | 
			
		||||
            "snapshot" => Ok(Kind::Snapshot),
 | 
			
		||||
            s => Err(Error::InvalidKind(s.to_string())),
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)]
 | 
			
		||||
#[serde(untagged)]
 | 
			
		||||
#[allow(clippy::large_enum_variant)]
 | 
			
		||||
 
 | 
			
		||||
@@ -48,6 +48,9 @@ log = "0.4.17"
 | 
			
		||||
meilisearch-auth = { path = "../meilisearch-auth" }
 | 
			
		||||
meilisearch-types = { path = "../meilisearch-types" }
 | 
			
		||||
meilisearch-lib = { path = "../meilisearch-lib", default-features = false }
 | 
			
		||||
index = { path = "../index" }
 | 
			
		||||
index-scheduler = { path = "../index-scheduler" }
 | 
			
		||||
document-formats = { path = "../document-formats" }
 | 
			
		||||
mimalloc = { version = "0.1.29", default-features = false }
 | 
			
		||||
mime = "0.3.16"
 | 
			
		||||
num_cpus = "1.13.1"
 | 
			
		||||
 
 | 
			
		||||
@@ -2,7 +2,6 @@
 | 
			
		||||
#[macro_use]
 | 
			
		||||
pub mod error;
 | 
			
		||||
pub mod analytics;
 | 
			
		||||
pub mod task;
 | 
			
		||||
#[macro_use]
 | 
			
		||||
pub mod extractors;
 | 
			
		||||
pub mod option;
 | 
			
		||||
 
 | 
			
		||||
@@ -1,4 +1,5 @@
 | 
			
		||||
use actix_web::{web, HttpRequest, HttpResponse};
 | 
			
		||||
use index_scheduler::KindWithContent;
 | 
			
		||||
use log::debug;
 | 
			
		||||
use meilisearch_lib::MeiliSearch;
 | 
			
		||||
use meilisearch_types::error::ResponseError;
 | 
			
		||||
@@ -7,7 +8,6 @@ use serde_json::json;
 | 
			
		||||
use crate::analytics::Analytics;
 | 
			
		||||
use crate::extractors::authentication::{policies::*, GuardedData};
 | 
			
		||||
use crate::extractors::sequential_extractor::SeqHandler;
 | 
			
		||||
use crate::task::SummarizedTaskView;
 | 
			
		||||
 | 
			
		||||
pub fn configure(cfg: &mut web::ServiceConfig) {
 | 
			
		||||
    cfg.service(web::resource("").route(web::post().to(SeqHandler(create_dump))));
 | 
			
		||||
@@ -20,7 +20,10 @@ pub async fn create_dump(
 | 
			
		||||
) -> Result<HttpResponse, ResponseError> {
 | 
			
		||||
    analytics.publish("Dump Created".to_string(), json!({}), Some(&req));
 | 
			
		||||
 | 
			
		||||
    let res: SummarizedTaskView = meilisearch.register_dump_task().await?.into();
 | 
			
		||||
    let task = KindWithContent::DumpExport {
 | 
			
		||||
        output: "toto".to_string().into(),
 | 
			
		||||
    };
 | 
			
		||||
    let res = meilisearch.register_task(task).await?;
 | 
			
		||||
 | 
			
		||||
    debug!("returns: {:?}", res);
 | 
			
		||||
    Ok(HttpResponse::Accepted().json(res))
 | 
			
		||||
 
 | 
			
		||||
@@ -4,9 +4,10 @@ use actix_web::web::Bytes;
 | 
			
		||||
use actix_web::HttpMessage;
 | 
			
		||||
use actix_web::{web, HttpRequest, HttpResponse};
 | 
			
		||||
use bstr::ByteSlice;
 | 
			
		||||
use document_formats::PayloadType;
 | 
			
		||||
use futures::{Stream, StreamExt};
 | 
			
		||||
use index_scheduler::{KindWithContent, TaskView};
 | 
			
		||||
use log::debug;
 | 
			
		||||
use meilisearch_lib::index_controller::{DocumentAdditionFormat, Update};
 | 
			
		||||
use meilisearch_lib::milli::update::IndexDocumentsMethod;
 | 
			
		||||
use meilisearch_lib::MeiliSearch;
 | 
			
		||||
use meilisearch_types::error::ResponseError;
 | 
			
		||||
@@ -24,7 +25,6 @@ use crate::extractors::authentication::{policies::*, GuardedData};
 | 
			
		||||
use crate::extractors::payload::Payload;
 | 
			
		||||
use crate::extractors::sequential_extractor::SeqHandler;
 | 
			
		||||
use crate::routes::{fold_star_or, PaginationView};
 | 
			
		||||
use crate::task::SummarizedTaskView;
 | 
			
		||||
 | 
			
		||||
static ACCEPTED_CONTENT_TYPE: Lazy<Vec<String>> = Lazy::new(|| {
 | 
			
		||||
    vec![
 | 
			
		||||
@@ -117,8 +117,11 @@ pub async fn delete_document(
 | 
			
		||||
        document_id,
 | 
			
		||||
        index_uid,
 | 
			
		||||
    } = path.into_inner();
 | 
			
		||||
    let update = Update::DeleteDocuments(vec![document_id]);
 | 
			
		||||
    let task: SummarizedTaskView = meilisearch.register_update(index_uid, update).await?.into();
 | 
			
		||||
    let task = KindWithContent::DocumentDeletion {
 | 
			
		||||
        index_uid,
 | 
			
		||||
        documents_ids: vec![document_id],
 | 
			
		||||
    };
 | 
			
		||||
    let task = meilisearch.register_task(task).await?;
 | 
			
		||||
    debug!("returns: {:?}", task);
 | 
			
		||||
    Ok(HttpResponse::Accepted().json(task))
 | 
			
		||||
}
 | 
			
		||||
@@ -235,14 +238,14 @@ async fn document_addition(
 | 
			
		||||
    body: Payload,
 | 
			
		||||
    method: IndexDocumentsMethod,
 | 
			
		||||
    allow_index_creation: bool,
 | 
			
		||||
) -> Result<SummarizedTaskView, ResponseError> {
 | 
			
		||||
) -> Result<TaskView, ResponseError> {
 | 
			
		||||
    let format = match mime_type
 | 
			
		||||
        .as_ref()
 | 
			
		||||
        .map(|m| (m.type_().as_str(), m.subtype().as_str()))
 | 
			
		||||
    {
 | 
			
		||||
        Some(("application", "json")) => DocumentAdditionFormat::Json,
 | 
			
		||||
        Some(("application", "x-ndjson")) => DocumentAdditionFormat::Ndjson,
 | 
			
		||||
        Some(("text", "csv")) => DocumentAdditionFormat::Csv,
 | 
			
		||||
        Some(("application", "json")) => PayloadType::Json,
 | 
			
		||||
        Some(("application", "x-ndjson")) => PayloadType::Ndjson,
 | 
			
		||||
        Some(("text", "csv")) => PayloadType::Csv,
 | 
			
		||||
        Some((type_, subtype)) => {
 | 
			
		||||
            return Err(MeilisearchHttpError::InvalidContentType(
 | 
			
		||||
                format!("{}/{}", type_, subtype),
 | 
			
		||||
@@ -257,12 +260,16 @@ async fn document_addition(
 | 
			
		||||
        }
 | 
			
		||||
    };
 | 
			
		||||
 | 
			
		||||
    let update = Update::DocumentAddition {
 | 
			
		||||
        payload: Box::new(payload_to_stream(body)),
 | 
			
		||||
    let (file, uuid) = meilisearch.create_update_file()?;
 | 
			
		||||
 | 
			
		||||
    let update = KindWithContent::DocumentAddition {
 | 
			
		||||
        content_file: Box::new(payload_to_stream(body)),
 | 
			
		||||
        documents_count: 0, // TODO: TAMO: get the document count
 | 
			
		||||
        primary_key,
 | 
			
		||||
        method,
 | 
			
		||||
        format,
 | 
			
		||||
        allow_index_creation,
 | 
			
		||||
        index_uid,
 | 
			
		||||
    };
 | 
			
		||||
 | 
			
		||||
    let task = meilisearch.register_update(index_uid, update).await?.into();
 | 
			
		||||
 
 | 
			
		||||
@@ -1,6 +1,6 @@
 | 
			
		||||
use actix_web::{web, HttpRequest, HttpResponse};
 | 
			
		||||
use index_scheduler::KindWithContent;
 | 
			
		||||
use log::debug;
 | 
			
		||||
use meilisearch_lib::index_controller::Update;
 | 
			
		||||
use meilisearch_lib::MeiliSearch;
 | 
			
		||||
use meilisearch_types::error::ResponseError;
 | 
			
		||||
use serde::{Deserialize, Serialize};
 | 
			
		||||
@@ -10,7 +10,7 @@ use time::OffsetDateTime;
 | 
			
		||||
use crate::analytics::Analytics;
 | 
			
		||||
use crate::extractors::authentication::{policies::*, AuthenticationError, GuardedData};
 | 
			
		||||
use crate::extractors::sequential_extractor::SeqHandler;
 | 
			
		||||
use crate::task::SummarizedTaskView;
 | 
			
		||||
use index_scheduler::task::TaskView;
 | 
			
		||||
 | 
			
		||||
use super::Pagination;
 | 
			
		||||
 | 
			
		||||
@@ -48,10 +48,14 @@ pub async fn list_indexes(
 | 
			
		||||
    let nb_indexes = indexes.len();
 | 
			
		||||
    let iter = indexes
 | 
			
		||||
        .into_iter()
 | 
			
		||||
        .filter(|i| search_rules.is_index_authorized(&i.uid));
 | 
			
		||||
        .filter(|index| search_rules.is_index_authorized(&index.name));
 | 
			
		||||
    /*
 | 
			
		||||
    TODO: TAMO: implements me
 | 
			
		||||
    let ret = paginate
 | 
			
		||||
        .into_inner()
 | 
			
		||||
        .auto_paginate_unsized(nb_indexes, iter);
 | 
			
		||||
    */
 | 
			
		||||
    let ret = todo!();
 | 
			
		||||
 | 
			
		||||
    debug!("returns: {:?}", ret);
 | 
			
		||||
    Ok(HttpResponse::Ok().json(ret))
 | 
			
		||||
@@ -70,9 +74,7 @@ pub async fn create_index(
 | 
			
		||||
    req: HttpRequest,
 | 
			
		||||
    analytics: web::Data<dyn Analytics>,
 | 
			
		||||
) -> Result<HttpResponse, ResponseError> {
 | 
			
		||||
    let IndexCreateRequest {
 | 
			
		||||
        primary_key, uid, ..
 | 
			
		||||
    } = body.into_inner();
 | 
			
		||||
    let IndexCreateRequest { primary_key, uid } = body.into_inner();
 | 
			
		||||
 | 
			
		||||
    let allow_index_creation = meilisearch.filters().search_rules.is_index_authorized(&uid);
 | 
			
		||||
    if allow_index_creation {
 | 
			
		||||
@@ -82,8 +84,11 @@ pub async fn create_index(
 | 
			
		||||
            Some(&req),
 | 
			
		||||
        );
 | 
			
		||||
 | 
			
		||||
        let update = Update::CreateIndex { primary_key };
 | 
			
		||||
        let task: SummarizedTaskView = meilisearch.register_update(uid, update).await?.into();
 | 
			
		||||
        let task = KindWithContent::IndexCreation {
 | 
			
		||||
            index_uid: uid,
 | 
			
		||||
            primary_key,
 | 
			
		||||
        };
 | 
			
		||||
        let task = meilisearch.register_task(task).await?;
 | 
			
		||||
 | 
			
		||||
        Ok(HttpResponse::Accepted().json(task))
 | 
			
		||||
    } else {
 | 
			
		||||
@@ -118,7 +123,10 @@ pub async fn get_index(
 | 
			
		||||
) -> Result<HttpResponse, ResponseError> {
 | 
			
		||||
    let meta = meilisearch.get_index(path.into_inner()).await?;
 | 
			
		||||
    debug!("returns: {:?}", meta);
 | 
			
		||||
    Ok(HttpResponse::Ok().json(meta))
 | 
			
		||||
 | 
			
		||||
    // TODO: TAMO: do this as well
 | 
			
		||||
    todo!()
 | 
			
		||||
    // Ok(HttpResponse::Ok().json(meta))
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
pub async fn update_index(
 | 
			
		||||
@@ -136,14 +144,12 @@ pub async fn update_index(
 | 
			
		||||
        Some(&req),
 | 
			
		||||
    );
 | 
			
		||||
 | 
			
		||||
    let update = Update::UpdateIndex {
 | 
			
		||||
    let task = KindWithContent::IndexUpdate {
 | 
			
		||||
        index_uid: path.into_inner(),
 | 
			
		||||
        primary_key: body.primary_key,
 | 
			
		||||
    };
 | 
			
		||||
 | 
			
		||||
    let task: SummarizedTaskView = meilisearch
 | 
			
		||||
        .register_update(path.into_inner(), update)
 | 
			
		||||
        .await?
 | 
			
		||||
        .into();
 | 
			
		||||
    let task = meilisearch.register_task(task).await?;
 | 
			
		||||
 | 
			
		||||
    debug!("returns: {:?}", task);
 | 
			
		||||
    Ok(HttpResponse::Accepted().json(task))
 | 
			
		||||
@@ -153,9 +159,9 @@ pub async fn delete_index(
 | 
			
		||||
    meilisearch: GuardedData<ActionPolicy<{ actions::INDEXES_DELETE }>, MeiliSearch>,
 | 
			
		||||
    path: web::Path<String>,
 | 
			
		||||
) -> Result<HttpResponse, ResponseError> {
 | 
			
		||||
    let uid = path.into_inner();
 | 
			
		||||
    let update = Update::DeleteIndex;
 | 
			
		||||
    let task: SummarizedTaskView = meilisearch.register_update(uid, update).await?.into();
 | 
			
		||||
    let index_uid = path.into_inner();
 | 
			
		||||
    let task = KindWithContent::IndexDeletion { index_uid };
 | 
			
		||||
    let task = meilisearch.register_task(task).await?;
 | 
			
		||||
 | 
			
		||||
    Ok(HttpResponse::Accepted().json(task))
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
@@ -1,11 +1,11 @@
 | 
			
		||||
use actix_web::{web, HttpRequest, HttpResponse};
 | 
			
		||||
use log::debug;
 | 
			
		||||
use meilisearch_auth::IndexSearchRules;
 | 
			
		||||
use meilisearch_lib::index::{
 | 
			
		||||
use index::{
 | 
			
		||||
    MatchingStrategy, SearchQuery, DEFAULT_CROP_LENGTH, DEFAULT_CROP_MARKER,
 | 
			
		||||
    DEFAULT_HIGHLIGHT_POST_TAG, DEFAULT_HIGHLIGHT_PRE_TAG, DEFAULT_SEARCH_LIMIT,
 | 
			
		||||
    DEFAULT_SEARCH_OFFSET,
 | 
			
		||||
};
 | 
			
		||||
use log::debug;
 | 
			
		||||
use meilisearch_auth::IndexSearchRules;
 | 
			
		||||
use meilisearch_lib::MeiliSearch;
 | 
			
		||||
use meilisearch_types::error::ResponseError;
 | 
			
		||||
use serde::Deserialize;
 | 
			
		||||
 
 | 
			
		||||
@@ -1,15 +1,14 @@
 | 
			
		||||
use log::debug;
 | 
			
		||||
 | 
			
		||||
use actix_web::{web, HttpRequest, HttpResponse};
 | 
			
		||||
use meilisearch_lib::index::{Settings, Unchecked};
 | 
			
		||||
use meilisearch_lib::index_controller::Update;
 | 
			
		||||
use index::{Settings, Unchecked};
 | 
			
		||||
use index_scheduler::KindWithContent;
 | 
			
		||||
use meilisearch_lib::MeiliSearch;
 | 
			
		||||
use meilisearch_types::error::ResponseError;
 | 
			
		||||
use serde_json::json;
 | 
			
		||||
 | 
			
		||||
use crate::analytics::Analytics;
 | 
			
		||||
use crate::extractors::authentication::{policies::*, GuardedData};
 | 
			
		||||
use crate::task::SummarizedTaskView;
 | 
			
		||||
 | 
			
		||||
#[macro_export]
 | 
			
		||||
macro_rules! make_setting_route {
 | 
			
		||||
@@ -18,34 +17,33 @@ macro_rules! make_setting_route {
 | 
			
		||||
            use actix_web::{web, HttpRequest, HttpResponse, Resource};
 | 
			
		||||
            use log::debug;
 | 
			
		||||
 | 
			
		||||
            use index::Settings;
 | 
			
		||||
            use index_scheduler::KindWithContent;
 | 
			
		||||
            use meilisearch_lib::milli::update::Setting;
 | 
			
		||||
            use meilisearch_lib::{index::Settings, index_controller::Update, MeiliSearch};
 | 
			
		||||
            use meilisearch_lib::MeiliSearch;
 | 
			
		||||
 | 
			
		||||
            use meilisearch_types::error::ResponseError;
 | 
			
		||||
            use $crate::analytics::Analytics;
 | 
			
		||||
            use $crate::extractors::authentication::{policies::*, GuardedData};
 | 
			
		||||
            use $crate::extractors::sequential_extractor::SeqHandler;
 | 
			
		||||
            use $crate::task::SummarizedTaskView;
 | 
			
		||||
 | 
			
		||||
            pub async fn delete(
 | 
			
		||||
                meilisearch: GuardedData<ActionPolicy<{ actions::SETTINGS_UPDATE }>, MeiliSearch>,
 | 
			
		||||
                index_uid: web::Path<String>,
 | 
			
		||||
            ) -> Result<HttpResponse, ResponseError> {
 | 
			
		||||
                let settings = Settings {
 | 
			
		||||
                let new_settings = Settings {
 | 
			
		||||
                    $attr: Setting::Reset,
 | 
			
		||||
                    ..Default::default()
 | 
			
		||||
                };
 | 
			
		||||
 | 
			
		||||
                let allow_index_creation = meilisearch.filters().allow_index_creation;
 | 
			
		||||
                let update = Update::Settings {
 | 
			
		||||
                    settings,
 | 
			
		||||
                let task = KindWithContent::Settings {
 | 
			
		||||
                    index_uid: index_uid.into_inner(),
 | 
			
		||||
                    new_settings,
 | 
			
		||||
                    is_deletion: true,
 | 
			
		||||
                    allow_index_creation,
 | 
			
		||||
                };
 | 
			
		||||
                let task: SummarizedTaskView = meilisearch
 | 
			
		||||
                    .register_update(index_uid.into_inner(), update)
 | 
			
		||||
                    .await?
 | 
			
		||||
                    .into();
 | 
			
		||||
                let task = meilisearch.register_task(task).await?;
 | 
			
		||||
 | 
			
		||||
                debug!("returns: {:?}", task);
 | 
			
		||||
                Ok(HttpResponse::Accepted().json(task))
 | 
			
		||||
@@ -62,7 +60,7 @@ macro_rules! make_setting_route {
 | 
			
		||||
 | 
			
		||||
                $analytics(&body, &req);
 | 
			
		||||
 | 
			
		||||
                let settings = Settings {
 | 
			
		||||
                let new_settings = Settings {
 | 
			
		||||
                    $attr: match body {
 | 
			
		||||
                        Some(inner_body) => Setting::Set(inner_body),
 | 
			
		||||
                        None => Setting::Reset,
 | 
			
		||||
@@ -71,15 +69,13 @@ macro_rules! make_setting_route {
 | 
			
		||||
                };
 | 
			
		||||
 | 
			
		||||
                let allow_index_creation = meilisearch.filters().allow_index_creation;
 | 
			
		||||
                let update = Update::Settings {
 | 
			
		||||
                    settings,
 | 
			
		||||
                let task = KindWithContent::Settings {
 | 
			
		||||
                    index_uid: index_uid.into_inner(),
 | 
			
		||||
                    new_settings,
 | 
			
		||||
                    is_deletion: false,
 | 
			
		||||
                    allow_index_creation,
 | 
			
		||||
                };
 | 
			
		||||
                let task: SummarizedTaskView = meilisearch
 | 
			
		||||
                    .register_update(index_uid.into_inner(), update)
 | 
			
		||||
                    .await?
 | 
			
		||||
                    .into();
 | 
			
		||||
                let task = meilisearch.register_task(task).await?;
 | 
			
		||||
 | 
			
		||||
                debug!("returns: {:?}", task);
 | 
			
		||||
                Ok(HttpResponse::Accepted().json(task))
 | 
			
		||||
@@ -89,7 +85,9 @@ macro_rules! make_setting_route {
 | 
			
		||||
                meilisearch: GuardedData<ActionPolicy<{ actions::SETTINGS_GET }>, MeiliSearch>,
 | 
			
		||||
                index_uid: actix_web::web::Path<String>,
 | 
			
		||||
            ) -> std::result::Result<HttpResponse, ResponseError> {
 | 
			
		||||
                let settings = meilisearch.settings(index_uid.into_inner()).await?;
 | 
			
		||||
                let index = meilisearch.get_index(index_uid.into_inner()).await?;
 | 
			
		||||
                let settings = index.settings()?;
 | 
			
		||||
 | 
			
		||||
                debug!("returns: {:?}", settings);
 | 
			
		||||
                let mut json = serde_json::json!(&settings);
 | 
			
		||||
                let val = json[$camelcase_attr].take();
 | 
			
		||||
@@ -175,11 +173,11 @@ make_setting_route!(
 | 
			
		||||
make_setting_route!(
 | 
			
		||||
    "/typo-tolerance",
 | 
			
		||||
    patch,
 | 
			
		||||
    meilisearch_lib::index::updates::TypoSettings,
 | 
			
		||||
    index::updates::TypoSettings,
 | 
			
		||||
    typo_tolerance,
 | 
			
		||||
    "typoTolerance",
 | 
			
		||||
    analytics,
 | 
			
		||||
    |setting: &Option<meilisearch_lib::index::updates::TypoSettings>, req: &HttpRequest| {
 | 
			
		||||
    |setting: &Option<index::updates::TypoSettings>, req: &HttpRequest| {
 | 
			
		||||
        use serde_json::json;
 | 
			
		||||
 | 
			
		||||
        analytics.publish(
 | 
			
		||||
@@ -285,11 +283,11 @@ make_setting_route!(
 | 
			
		||||
make_setting_route!(
 | 
			
		||||
    "/faceting",
 | 
			
		||||
    patch,
 | 
			
		||||
    meilisearch_lib::index::updates::FacetingSettings,
 | 
			
		||||
    index::updates::FacetingSettings,
 | 
			
		||||
    faceting,
 | 
			
		||||
    "faceting",
 | 
			
		||||
    analytics,
 | 
			
		||||
    |setting: &Option<meilisearch_lib::index::updates::FacetingSettings>, req: &HttpRequest| {
 | 
			
		||||
    |setting: &Option<index::updates::FacetingSettings>, req: &HttpRequest| {
 | 
			
		||||
        use serde_json::json;
 | 
			
		||||
 | 
			
		||||
        analytics.publish(
 | 
			
		||||
@@ -307,11 +305,11 @@ make_setting_route!(
 | 
			
		||||
make_setting_route!(
 | 
			
		||||
    "/pagination",
 | 
			
		||||
    patch,
 | 
			
		||||
    meilisearch_lib::index::updates::PaginationSettings,
 | 
			
		||||
    index::updates::PaginationSettings,
 | 
			
		||||
    pagination,
 | 
			
		||||
    "pagination",
 | 
			
		||||
    analytics,
 | 
			
		||||
    |setting: &Option<meilisearch_lib::index::updates::PaginationSettings>, req: &HttpRequest| {
 | 
			
		||||
    |setting: &Option<index::updates::PaginationSettings>, req: &HttpRequest| {
 | 
			
		||||
        use serde_json::json;
 | 
			
		||||
 | 
			
		||||
        analytics.publish(
 | 
			
		||||
@@ -361,40 +359,40 @@ pub async fn update_all(
 | 
			
		||||
    req: HttpRequest,
 | 
			
		||||
    analytics: web::Data<dyn Analytics>,
 | 
			
		||||
) -> Result<HttpResponse, ResponseError> {
 | 
			
		||||
    let settings = body.into_inner();
 | 
			
		||||
    let new_settings = body.into_inner();
 | 
			
		||||
 | 
			
		||||
    analytics.publish(
 | 
			
		||||
        "Settings Updated".to_string(),
 | 
			
		||||
        json!({
 | 
			
		||||
           "ranking_rules": {
 | 
			
		||||
                "sort_position": settings.ranking_rules.as_ref().set().map(|sort| sort.iter().position(|s| s == "sort")),
 | 
			
		||||
                "sort_position": new_settings.ranking_rules.as_ref().set().map(|sort| sort.iter().position(|s| s == "sort")),
 | 
			
		||||
            },
 | 
			
		||||
            "searchable_attributes": {
 | 
			
		||||
                "total": settings.searchable_attributes.as_ref().set().map(|searchable| searchable.len()),
 | 
			
		||||
                "total": new_settings.searchable_attributes.as_ref().set().map(|searchable| searchable.len()),
 | 
			
		||||
            },
 | 
			
		||||
           "sortable_attributes": {
 | 
			
		||||
                "total": settings.sortable_attributes.as_ref().set().map(|sort| sort.len()),
 | 
			
		||||
                "has_geo": settings.sortable_attributes.as_ref().set().map(|sort| sort.iter().any(|s| s == "_geo")),
 | 
			
		||||
                "total": new_settings.sortable_attributes.as_ref().set().map(|sort| sort.len()),
 | 
			
		||||
                "has_geo": new_settings.sortable_attributes.as_ref().set().map(|sort| sort.iter().any(|s| s == "_geo")),
 | 
			
		||||
            },
 | 
			
		||||
           "filterable_attributes": {
 | 
			
		||||
                "total": settings.filterable_attributes.as_ref().set().map(|filter| filter.len()),
 | 
			
		||||
                "has_geo": settings.filterable_attributes.as_ref().set().map(|filter| filter.iter().any(|s| s == "_geo")),
 | 
			
		||||
                "total": new_settings.filterable_attributes.as_ref().set().map(|filter| filter.len()),
 | 
			
		||||
                "has_geo": new_settings.filterable_attributes.as_ref().set().map(|filter| filter.iter().any(|s| s == "_geo")),
 | 
			
		||||
            },
 | 
			
		||||
            "typo_tolerance": {
 | 
			
		||||
                "enabled": settings.typo_tolerance
 | 
			
		||||
                "enabled": new_settings.typo_tolerance
 | 
			
		||||
                    .as_ref()
 | 
			
		||||
                    .set()
 | 
			
		||||
                    .and_then(|s| s.enabled.as_ref().set())
 | 
			
		||||
                    .copied(),
 | 
			
		||||
                "disable_on_attributes": settings.typo_tolerance
 | 
			
		||||
                "disable_on_attributes": new_settings.typo_tolerance
 | 
			
		||||
                    .as_ref()
 | 
			
		||||
                    .set()
 | 
			
		||||
                    .and_then(|s| s.disable_on_attributes.as_ref().set().map(|m| !m.is_empty())),
 | 
			
		||||
                "disable_on_words": settings.typo_tolerance
 | 
			
		||||
                "disable_on_words": new_settings.typo_tolerance
 | 
			
		||||
                    .as_ref()
 | 
			
		||||
                    .set()
 | 
			
		||||
                    .and_then(|s| s.disable_on_words.as_ref().set().map(|m| !m.is_empty())),
 | 
			
		||||
                "min_word_size_for_one_typo": settings.typo_tolerance
 | 
			
		||||
                "min_word_size_for_one_typo": new_settings.typo_tolerance
 | 
			
		||||
                    .as_ref()
 | 
			
		||||
                    .set()
 | 
			
		||||
                    .and_then(|s| s.min_word_size_for_typos
 | 
			
		||||
@@ -402,7 +400,7 @@ pub async fn update_all(
 | 
			
		||||
                        .set()
 | 
			
		||||
                        .map(|s| s.one_typo.set()))
 | 
			
		||||
                    .flatten(),
 | 
			
		||||
                "min_word_size_for_two_typos": settings.typo_tolerance
 | 
			
		||||
                "min_word_size_for_two_typos": new_settings.typo_tolerance
 | 
			
		||||
                    .as_ref()
 | 
			
		||||
                    .set()
 | 
			
		||||
                    .and_then(|s| s.min_word_size_for_typos
 | 
			
		||||
@@ -412,13 +410,13 @@ pub async fn update_all(
 | 
			
		||||
                    .flatten(),
 | 
			
		||||
            },
 | 
			
		||||
            "faceting": {
 | 
			
		||||
                "max_values_per_facet": settings.faceting
 | 
			
		||||
                "max_values_per_facet": new_settings.faceting
 | 
			
		||||
                    .as_ref()
 | 
			
		||||
                    .set()
 | 
			
		||||
                    .and_then(|s| s.max_values_per_facet.as_ref().set()),
 | 
			
		||||
            },
 | 
			
		||||
            "pagination": {
 | 
			
		||||
                "max_total_hits": settings.pagination
 | 
			
		||||
                "max_total_hits": new_settings.pagination
 | 
			
		||||
                    .as_ref()
 | 
			
		||||
                    .set()
 | 
			
		||||
                    .and_then(|s| s.max_total_hits.as_ref().set()),
 | 
			
		||||
@@ -428,45 +426,42 @@ pub async fn update_all(
 | 
			
		||||
    );
 | 
			
		||||
 | 
			
		||||
    let allow_index_creation = meilisearch.filters().allow_index_creation;
 | 
			
		||||
    let update = Update::Settings {
 | 
			
		||||
        settings,
 | 
			
		||||
    let task = KindWithContent::Settings {
 | 
			
		||||
        index_uid: index_uid.into_inner(),
 | 
			
		||||
        new_settings,
 | 
			
		||||
        is_deletion: false,
 | 
			
		||||
        allow_index_creation,
 | 
			
		||||
    };
 | 
			
		||||
    let task: SummarizedTaskView = meilisearch
 | 
			
		||||
        .register_update(index_uid.into_inner(), update)
 | 
			
		||||
        .await?
 | 
			
		||||
        .into();
 | 
			
		||||
    let task = meilisearch.register_task(task).await?;
 | 
			
		||||
 | 
			
		||||
    debug!("returns: {:?}", task);
 | 
			
		||||
    Ok(HttpResponse::Accepted().json(task))
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
pub async fn get_all(
 | 
			
		||||
    data: GuardedData<ActionPolicy<{ actions::SETTINGS_GET }>, MeiliSearch>,
 | 
			
		||||
    meilisearch: GuardedData<ActionPolicy<{ actions::SETTINGS_GET }>, MeiliSearch>,
 | 
			
		||||
    index_uid: web::Path<String>,
 | 
			
		||||
) -> Result<HttpResponse, ResponseError> {
 | 
			
		||||
    let settings = data.settings(index_uid.into_inner()).await?;
 | 
			
		||||
    debug!("returns: {:?}", settings);
 | 
			
		||||
    Ok(HttpResponse::Ok().json(settings))
 | 
			
		||||
    let index = meilisearch.get_index(index_uid.into_inner()).await?;
 | 
			
		||||
    let new_settings = index.settings()?;
 | 
			
		||||
    debug!("returns: {:?}", new_settings);
 | 
			
		||||
    Ok(HttpResponse::Ok().json(new_settings))
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
pub async fn delete_all(
 | 
			
		||||
    data: GuardedData<ActionPolicy<{ actions::SETTINGS_UPDATE }>, MeiliSearch>,
 | 
			
		||||
    index_uid: web::Path<String>,
 | 
			
		||||
) -> Result<HttpResponse, ResponseError> {
 | 
			
		||||
    let settings = Settings::cleared().into_unchecked();
 | 
			
		||||
    let new_settings = Settings::cleared().into_unchecked();
 | 
			
		||||
 | 
			
		||||
    let allow_index_creation = data.filters().allow_index_creation;
 | 
			
		||||
    let update = Update::Settings {
 | 
			
		||||
        settings,
 | 
			
		||||
    let task = KindWithContent::Settings {
 | 
			
		||||
        index_uid: index_uid.into_inner(),
 | 
			
		||||
        new_settings,
 | 
			
		||||
        is_deletion: true,
 | 
			
		||||
        allow_index_creation,
 | 
			
		||||
    };
 | 
			
		||||
    let task: SummarizedTaskView = data
 | 
			
		||||
        .register_update(index_uid.into_inner(), update)
 | 
			
		||||
        .await?
 | 
			
		||||
        .into();
 | 
			
		||||
    let task = data.register_task(task).await?;
 | 
			
		||||
 | 
			
		||||
    debug!("returns: {:?}", task);
 | 
			
		||||
    Ok(HttpResponse::Accepted().json(task))
 | 
			
		||||
 
 | 
			
		||||
@@ -5,7 +5,7 @@ use serde::{Deserialize, Serialize};
 | 
			
		||||
use serde_json::json;
 | 
			
		||||
use time::OffsetDateTime;
 | 
			
		||||
 | 
			
		||||
use meilisearch_lib::index::{Settings, Unchecked};
 | 
			
		||||
use index::{Settings, Unchecked};
 | 
			
		||||
use meilisearch_lib::MeiliSearch;
 | 
			
		||||
use meilisearch_types::error::ResponseError;
 | 
			
		||||
use meilisearch_types::star_or::StarOr;
 | 
			
		||||
 
 | 
			
		||||
@@ -1,6 +1,6 @@
 | 
			
		||||
use actix_web::{web, HttpRequest, HttpResponse};
 | 
			
		||||
use meilisearch_lib::tasks::task::{TaskContent, TaskEvent, TaskId};
 | 
			
		||||
use meilisearch_lib::tasks::TaskFilter;
 | 
			
		||||
use index_scheduler::TaskId;
 | 
			
		||||
use index_scheduler::{Kind, Status};
 | 
			
		||||
use meilisearch_lib::MeiliSearch;
 | 
			
		||||
use meilisearch_types::error::ResponseError;
 | 
			
		||||
use meilisearch_types::index_uid::IndexUid;
 | 
			
		||||
@@ -12,7 +12,6 @@ use serde_json::json;
 | 
			
		||||
use crate::analytics::Analytics;
 | 
			
		||||
use crate::extractors::authentication::{policies::*, GuardedData};
 | 
			
		||||
use crate::extractors::sequential_extractor::SeqHandler;
 | 
			
		||||
use crate::task::{TaskListView, TaskStatus, TaskType, TaskView};
 | 
			
		||||
 | 
			
		||||
use super::fold_star_or;
 | 
			
		||||
 | 
			
		||||
@@ -27,8 +26,8 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
 | 
			
		||||
#[serde(rename_all = "camelCase", deny_unknown_fields)]
 | 
			
		||||
pub struct TasksFilterQuery {
 | 
			
		||||
    #[serde(rename = "type")]
 | 
			
		||||
    type_: Option<CS<StarOr<TaskType>>>,
 | 
			
		||||
    status: Option<CS<StarOr<TaskStatus>>>,
 | 
			
		||||
    type_: Option<CS<StarOr<Kind>>>,
 | 
			
		||||
    status: Option<CS<StarOr<Status>>>,
 | 
			
		||||
    index_uid: Option<CS<StarOr<IndexUid>>>,
 | 
			
		||||
    #[serde(default = "DEFAULT_LIMIT")]
 | 
			
		||||
    limit: usize,
 | 
			
		||||
@@ -92,65 +91,43 @@ async fn get_tasks(
 | 
			
		||||
        Some(&req),
 | 
			
		||||
    );
 | 
			
		||||
 | 
			
		||||
    let mut filters = index_scheduler::Query::default();
 | 
			
		||||
 | 
			
		||||
    // Then we filter on potential indexes and make sure that the search filter
 | 
			
		||||
    // restrictions are also applied.
 | 
			
		||||
    let indexes_filters = match index_uid {
 | 
			
		||||
    match index_uid {
 | 
			
		||||
        Some(indexes) => {
 | 
			
		||||
            let mut filters = TaskFilter::default();
 | 
			
		||||
            for name in indexes {
 | 
			
		||||
                if search_rules.is_index_authorized(&name) {
 | 
			
		||||
                    filters.filter_index(name.to_string());
 | 
			
		||||
                    filters = filters.with_index(name.to_string());
 | 
			
		||||
                }
 | 
			
		||||
            }
 | 
			
		||||
            Some(filters)
 | 
			
		||||
        }
 | 
			
		||||
        None => {
 | 
			
		||||
            if search_rules.is_index_authorized("*") {
 | 
			
		||||
                None
 | 
			
		||||
            } else {
 | 
			
		||||
                let mut filters = TaskFilter::default();
 | 
			
		||||
            if !search_rules.is_index_authorized("*") {
 | 
			
		||||
                for (index, _policy) in search_rules.clone() {
 | 
			
		||||
                    filters.filter_index(index);
 | 
			
		||||
                    filters = filters.with_index(index.to_string());
 | 
			
		||||
                }
 | 
			
		||||
                Some(filters)
 | 
			
		||||
            }
 | 
			
		||||
        }
 | 
			
		||||
    };
 | 
			
		||||
 | 
			
		||||
    // Then we complete the task filter with other potential status and types filters.
 | 
			
		||||
    let filters = if type_.is_some() || status.is_some() {
 | 
			
		||||
        let mut filters = indexes_filters.unwrap_or_default();
 | 
			
		||||
        filters.filter_fn(Box::new(move |task| {
 | 
			
		||||
            let matches_type = match &type_ {
 | 
			
		||||
                Some(types) => types
 | 
			
		||||
                    .iter()
 | 
			
		||||
                    .any(|t| task_type_matches_content(t, &task.content)),
 | 
			
		||||
                None => true,
 | 
			
		||||
            };
 | 
			
		||||
    if let Some(kinds) = type_ {
 | 
			
		||||
        for kind in kinds {
 | 
			
		||||
            filters = filters.with_kind(kind);
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
            let matches_status = match &status {
 | 
			
		||||
                Some(statuses) => statuses
 | 
			
		||||
                    .iter()
 | 
			
		||||
                    .any(|t| task_status_matches_events(t, &task.events)),
 | 
			
		||||
                None => true,
 | 
			
		||||
            };
 | 
			
		||||
 | 
			
		||||
            matches_type && matches_status
 | 
			
		||||
        }));
 | 
			
		||||
        Some(filters)
 | 
			
		||||
    } else {
 | 
			
		||||
        indexes_filters
 | 
			
		||||
    };
 | 
			
		||||
    if let Some(statuses) = status {
 | 
			
		||||
        for status in statuses {
 | 
			
		||||
            filters = filters.with_status(status);
 | 
			
		||||
        }
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    // We +1 just to know if there is more after this "page" or not.
 | 
			
		||||
    let limit = limit.saturating_add(1);
 | 
			
		||||
 | 
			
		||||
    let mut tasks_results: Vec<_> = meilisearch
 | 
			
		||||
        .list_tasks(filters, Some(limit), from)
 | 
			
		||||
        .await?
 | 
			
		||||
        .into_iter()
 | 
			
		||||
        .map(TaskView::from)
 | 
			
		||||
        .collect();
 | 
			
		||||
    let mut tasks_results: Vec<_> = meilisearch.list_tasks(filters).await?.into_iter().collect();
 | 
			
		||||
 | 
			
		||||
    // If we were able to fetch the number +1 tasks we asked
 | 
			
		||||
    // it means that there is more to come.
 | 
			
		||||
@@ -162,12 +139,13 @@ async fn get_tasks(
 | 
			
		||||
 | 
			
		||||
    let from = tasks_results.first().map(|t| t.uid);
 | 
			
		||||
 | 
			
		||||
    let tasks = TaskListView {
 | 
			
		||||
        results: tasks_results,
 | 
			
		||||
        limit: limit.saturating_sub(1),
 | 
			
		||||
        from,
 | 
			
		||||
        next,
 | 
			
		||||
    };
 | 
			
		||||
    // TODO: TAMO: define a structure to represent this type
 | 
			
		||||
    let tasks = json!({
 | 
			
		||||
        "results": tasks_results,
 | 
			
		||||
        "limit": limit.saturating_sub(1),
 | 
			
		||||
        "from": from,
 | 
			
		||||
        "next": next,
 | 
			
		||||
    });
 | 
			
		||||
 | 
			
		||||
    Ok(HttpResponse::Ok().json(tasks))
 | 
			
		||||
}
 | 
			
		||||
@@ -185,20 +163,17 @@ async fn get_task(
 | 
			
		||||
    );
 | 
			
		||||
 | 
			
		||||
    let search_rules = &meilisearch.filters().search_rules;
 | 
			
		||||
    let filters = if search_rules.is_index_authorized("*") {
 | 
			
		||||
        None
 | 
			
		||||
    } else {
 | 
			
		||||
        let mut filters = TaskFilter::default();
 | 
			
		||||
    let mut filters = index_scheduler::Query::default();
 | 
			
		||||
    if !search_rules.is_index_authorized("*") {
 | 
			
		||||
        for (index, _policy) in search_rules.clone() {
 | 
			
		||||
            filters.filter_index(index);
 | 
			
		||||
            filters = filters.with_index(index);
 | 
			
		||||
        }
 | 
			
		||||
        Some(filters)
 | 
			
		||||
    };
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    let task: TaskView = meilisearch
 | 
			
		||||
        .get_task(task_id.into_inner(), filters)
 | 
			
		||||
        .await?
 | 
			
		||||
        .into();
 | 
			
		||||
    filters.limit = 1;
 | 
			
		||||
    filters.from = Some(*task_id);
 | 
			
		||||
 | 
			
		||||
    let task = meilisearch.list_tasks(filters).await?;
 | 
			
		||||
 | 
			
		||||
    Ok(HttpResponse::Ok().json(task))
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
@@ -60,6 +60,14 @@ pub struct Meilisearch {
 | 
			
		||||
    index_scheduler: IndexScheduler,
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl std::ops::Deref for Meilisearch {
 | 
			
		||||
    type Target = IndexScheduler;
 | 
			
		||||
 | 
			
		||||
    fn deref(&self) -> &Self::Target {
 | 
			
		||||
        &self.index_scheduler
 | 
			
		||||
    }
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
#[derive(Debug)]
 | 
			
		||||
pub enum DocumentAdditionFormat {
 | 
			
		||||
    Json,
 | 
			
		||||
@@ -317,10 +325,6 @@ impl Meilisearch {
 | 
			
		||||
        )
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    pub async fn get_task(&self, id: TaskId) -> Result<TaskView> {
 | 
			
		||||
        Ok(self.index_scheduler.task(id)?)
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    pub async fn list_tasks(&self, filter: index_scheduler::Query) -> Result<Vec<TaskView>> {
 | 
			
		||||
        Ok(self.index_scheduler.get_tasks(filter)?)
 | 
			
		||||
    }
 | 
			
		||||
 
 | 
			
		||||
		Reference in New Issue
	
	Block a user