feat(auth): API keys

implements:
https://github.com/meilisearch/specifications/blob/develop/text/0085-api-keys.md

- Add tests on API keys management route (meilisearch-http/tests/auth/api_keys.rs)
- Add tests checking authorizations on each meilisearch routes (meilisearch-http/tests/auth/authorization.rs)
- Implement API keys management routes (meilisearch-http/src/routes/api_key.rs)
- Create module to manage API keys and authorizations (meilisearch-auth)
- Reimplement GuardedData to extend authorizations (meilisearch-http/src/extractors/authentication/mod.rs)
- Change X-MEILI-API-KEY by Authorization Bearer (meilisearch-http/src/extractors/authentication/mod.rs)
- Change meilisearch routes to fit to the new authorization feature (meilisearch-http/src/routes/)

- close #1867
This commit is contained in:
many
2021-11-08 18:31:27 +01:00
parent fa196986c2
commit ffefd0caf2
44 changed files with 3155 additions and 361 deletions

34
Cargo.lock generated
View File

@@ -314,6 +314,16 @@ dependencies = [
"stable_deref_trait", "stable_deref_trait",
] ]
[[package]]
name = "assert-json-diff"
version = "2.0.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "50f1c3703dd33532d7f0ca049168930e9099ecac238e23cf932f3a69c42f06da"
dependencies = [
"serde",
"serde_json",
]
[[package]] [[package]]
name = "async-stream" name = "async-stream"
version = "0.3.2" version = "0.3.2"
@@ -1600,12 +1610,33 @@ dependencies = [
"syn 0.15.44", "syn 0.15.44",
] ]
[[package]]
name = "maplit"
version = "1.0.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "3e2e65a1a2e43cfcb47a895c4c8b10d1f4a61097f9f254f183aee60cad9c651d"
[[package]] [[package]]
name = "matches" name = "matches"
version = "0.1.9" version = "0.1.9"
source = "registry+https://github.com/rust-lang/crates.io-index" source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "a3e378b66a060d48947b590737b30a1be76706c8dd7b8ba0f2fe3989c68a853f" checksum = "a3e378b66a060d48947b590737b30a1be76706c8dd7b8ba0f2fe3989c68a853f"
[[package]]
name = "meilisearch-auth"
version = "0.24.0"
dependencies = [
"chrono",
"enum-iterator",
"heed",
"meilisearch-error",
"rand",
"serde",
"serde_json",
"sha2",
"thiserror",
]
[[package]] [[package]]
name = "meilisearch-error" name = "meilisearch-error"
version = "0.24.0" version = "0.24.0"
@@ -1629,6 +1660,7 @@ dependencies = [
"actix-web-static-files", "actix-web-static-files",
"anyhow", "anyhow",
"arc-swap", "arc-swap",
"assert-json-diff",
"async-stream", "async-stream",
"async-trait", "async-trait",
"bstr", "bstr",
@@ -1649,6 +1681,8 @@ dependencies = [
"indexmap", "indexmap",
"itertools", "itertools",
"log", "log",
"maplit",
"meilisearch-auth",
"meilisearch-error", "meilisearch-error",
"meilisearch-lib", "meilisearch-lib",
"meilisearch-tokenizer 0.2.5", "meilisearch-tokenizer 0.2.5",

View File

@@ -3,6 +3,7 @@ members = [
"meilisearch-http", "meilisearch-http",
"meilisearch-error", "meilisearch-error",
"meilisearch-lib", "meilisearch-lib",
"meilisearch-auth",
] ]
resolver = "2" resolver = "2"

View File

@@ -0,0 +1,15 @@
[package]
name = "meilisearch-auth"
version = "0.24.0"
edition = "2018"
[dependencies]
enum-iterator = "0.7.0"
heed = { git = "https://github.com/Kerollmops/heed", tag = "v0.12.1" }
sha2 = "0.9.6"
chrono = { version = "0.4.19", features = ["serde"] }
meilisearch-error = { path = "../meilisearch-error" }
serde_json = { version = "1.0.67", features = ["preserve_order"] }
rand = "0.8.4"
serde = { version = "1.0.130", features = ["derive"] }
thiserror = "1.0.28"

View File

@@ -0,0 +1,104 @@
use enum_iterator::IntoEnumIterator;
use serde::{Deserialize, Serialize};
#[derive(IntoEnumIterator, Copy, Clone, Serialize, Deserialize, Debug, Eq, PartialEq)]
#[repr(u8)]
pub enum Action {
#[serde(rename = "*")]
All = 0,
#[serde(rename = "search")]
Search = actions::SEARCH,
#[serde(rename = "documents.add")]
DocumentsAdd = actions::DOCUMENTS_ADD,
#[serde(rename = "documents.get")]
DocumentsGet = actions::DOCUMENTS_GET,
#[serde(rename = "documents.delete")]
DocumentsDelete = actions::DOCUMENTS_DELETE,
#[serde(rename = "indexes.add")]
IndexesAdd = actions::INDEXES_ADD,
#[serde(rename = "indexes.get")]
IndexesGet = actions::INDEXES_GET,
#[serde(rename = "indexes.update")]
IndexesUpdate = actions::INDEXES_UPDATE,
#[serde(rename = "indexes.delete")]
IndexesDelete = actions::INDEXES_DELETE,
#[serde(rename = "tasks.get")]
TasksGet = actions::TASKS_GET,
#[serde(rename = "settings.get")]
SettingsGet = actions::SETTINGS_GET,
#[serde(rename = "settings.update")]
SettingsUpdate = actions::SETTINGS_UPDATE,
#[serde(rename = "stats.get")]
StatsGet = actions::STATS_GET,
#[serde(rename = "dumps.create")]
DumpsCreate = actions::DUMPS_CREATE,
#[serde(rename = "dumps.get")]
DumpsGet = actions::DUMPS_GET,
#[serde(rename = "version")]
Version = actions::VERSION,
}
impl Action {
pub fn from_repr(repr: u8) -> Option<Self> {
use actions::*;
match repr {
0 => Some(Self::All),
SEARCH => Some(Self::Search),
DOCUMENTS_ADD => Some(Self::DocumentsAdd),
DOCUMENTS_GET => Some(Self::DocumentsGet),
DOCUMENTS_DELETE => Some(Self::DocumentsDelete),
INDEXES_ADD => Some(Self::IndexesAdd),
INDEXES_GET => Some(Self::IndexesGet),
INDEXES_UPDATE => Some(Self::IndexesUpdate),
INDEXES_DELETE => Some(Self::IndexesDelete),
TASKS_GET => Some(Self::TasksGet),
SETTINGS_GET => Some(Self::SettingsGet),
SETTINGS_UPDATE => Some(Self::SettingsUpdate),
STATS_GET => Some(Self::StatsGet),
DUMPS_CREATE => Some(Self::DumpsCreate),
DUMPS_GET => Some(Self::DumpsGet),
VERSION => Some(Self::Version),
_otherwise => None,
}
}
pub fn repr(&self) -> u8 {
use actions::*;
match self {
Self::All => 0,
Self::Search => SEARCH,
Self::DocumentsAdd => DOCUMENTS_ADD,
Self::DocumentsGet => DOCUMENTS_GET,
Self::DocumentsDelete => DOCUMENTS_DELETE,
Self::IndexesAdd => INDEXES_ADD,
Self::IndexesGet => INDEXES_GET,
Self::IndexesUpdate => INDEXES_UPDATE,
Self::IndexesDelete => INDEXES_DELETE,
Self::TasksGet => TASKS_GET,
Self::SettingsGet => SETTINGS_GET,
Self::SettingsUpdate => SETTINGS_UPDATE,
Self::StatsGet => STATS_GET,
Self::DumpsCreate => DUMPS_CREATE,
Self::DumpsGet => DUMPS_GET,
Self::Version => VERSION,
}
}
}
pub mod actions {
pub const SEARCH: u8 = 1;
pub const DOCUMENTS_ADD: u8 = 2;
pub const DOCUMENTS_GET: u8 = 3;
pub const DOCUMENTS_DELETE: u8 = 4;
pub const INDEXES_ADD: u8 = 5;
pub const INDEXES_GET: u8 = 6;
pub const INDEXES_UPDATE: u8 = 7;
pub const INDEXES_DELETE: u8 = 8;
pub const TASKS_GET: u8 = 9;
pub const SETTINGS_GET: u8 = 10;
pub const SETTINGS_UPDATE: u8 = 11;
pub const STATS_GET: u8 = 12;
pub const DUMPS_CREATE: u8 = 13;
pub const DUMPS_GET: u8 = 14;
pub const VERSION: u8 = 15;
}

View File

@@ -0,0 +1,41 @@
use std::error::Error;
use meilisearch_error::ErrorCode;
use meilisearch_error::{internal_error, Code};
use serde_json::Value;
pub type Result<T> = std::result::Result<T, AuthControllerError>;
#[derive(Debug, thiserror::Error)]
pub enum AuthControllerError {
#[error("`{0}` field is mandatory.")]
MissingParameter(&'static str),
#[error("actions field value `{0}` is invalid. It should be an array of string representing action names.")]
InvalidApiKeyActions(Value),
#[error("indexes field value `{0}` is invalid. It should be an array of string representing index names.")]
InvalidApiKeyIndexes(Value),
#[error("expiresAt field value `{0}` is invalid. It should be in ISO-8601 format to represents a date or datetime in the future or specified as a null value. e.g. 'YYYY-MM-DD' or 'YYYY-MM-DDTHH:MM:SS'.")]
InvalidApiKeyExpiresAt(Value),
#[error("description field value `{0}` is invalid. It should be a string or specified as a null value.")]
InvalidApiKeyDescription(Value),
#[error("API key `{0}` not found.")]
ApiKeyNotFound(String),
#[error("Internal error: {0}")]
Internal(Box<dyn Error + Send + Sync + 'static>),
}
internal_error!(AuthControllerError: heed::Error, std::io::Error);
impl ErrorCode for AuthControllerError {
fn error_code(&self) -> Code {
match self {
Self::MissingParameter(_) => Code::MissingParameter,
Self::InvalidApiKeyActions(_) => Code::InvalidApiKeyActions,
Self::InvalidApiKeyIndexes(_) => Code::InvalidApiKeyIndexes,
Self::InvalidApiKeyExpiresAt(_) => Code::InvalidApiKeyExpiresAt,
Self::InvalidApiKeyDescription(_) => Code::InvalidApiKeyDescription,
Self::ApiKeyNotFound(_) => Code::ApiKeyNotFound,
Self::Internal(_) => Code::Internal,
}
}
}

139
meilisearch-auth/src/key.rs Normal file
View File

@@ -0,0 +1,139 @@
use crate::action::Action;
use crate::error::{AuthControllerError, Result};
use crate::store::{KeyId, KEY_ID_LENGTH};
use chrono::{DateTime, Utc};
use rand::Rng;
use serde::{Deserialize, Serialize};
use serde_json::{from_value, Value};
#[derive(Debug, Deserialize, Serialize)]
pub struct Key {
#[serde(skip_serializing_if = "Option::is_none")]
pub description: Option<String>,
pub id: KeyId,
pub actions: Vec<Action>,
pub indexes: Vec<String>,
pub expires_at: Option<DateTime<Utc>>,
pub created_at: DateTime<Utc>,
pub updated_at: DateTime<Utc>,
}
impl Key {
pub fn create_from_value(value: Value) -> Result<Self> {
let description = value
.get("description")
.map(|des| {
from_value(des.clone())
.map_err(|_| AuthControllerError::InvalidApiKeyDescription(des.clone()))
})
.transpose()?;
let id = generate_id();
let actions = value
.get("actions")
.map(|act| {
from_value(act.clone())
.map_err(|_| AuthControllerError::InvalidApiKeyActions(act.clone()))
})
.ok_or(AuthControllerError::MissingParameter("actions"))??;
let indexes = value
.get("indexes")
.map(|ind| {
from_value(ind.clone())
.map_err(|_| AuthControllerError::InvalidApiKeyIndexes(ind.clone()))
})
.ok_or(AuthControllerError::MissingParameter("indexes"))??;
let expires_at = value
.get("expiresAt")
.map(|exp| {
from_value(exp.clone())
.map_err(|_| AuthControllerError::InvalidApiKeyExpiresAt(exp.clone()))
})
.transpose()?;
let created_at = Utc::now();
let updated_at = Utc::now();
Ok(Self {
description,
id,
actions,
indexes,
expires_at,
created_at,
updated_at,
})
}
pub fn update_from_value(&mut self, value: Value) -> Result<()> {
if let Some(des) = value.get("description") {
let des = from_value(des.clone())
.map_err(|_| AuthControllerError::InvalidApiKeyDescription(des.clone()));
self.description = des?;
}
if let Some(act) = value.get("actions") {
let act = from_value(act.clone())
.map_err(|_| AuthControllerError::InvalidApiKeyActions(act.clone()));
self.actions = act?;
}
if let Some(ind) = value.get("indexes") {
let ind = from_value(ind.clone())
.map_err(|_| AuthControllerError::InvalidApiKeyIndexes(ind.clone()));
self.indexes = ind?;
}
if let Some(exp) = value.get("expiresAt") {
let exp = from_value(exp.clone())
.map_err(|_| AuthControllerError::InvalidApiKeyExpiresAt(exp.clone()));
self.expires_at = exp?;
}
self.updated_at = Utc::now();
Ok(())
}
pub(crate) fn default_admin() -> Self {
Self {
description: Some("Default Admin API Key (Use it for all other operations. Caution! Do not use it on a public frontend)".to_string()),
id: generate_id(),
actions: vec![Action::All],
indexes: vec!["*".to_string()],
expires_at: None,
created_at: Utc::now(),
updated_at: Utc::now(),
}
}
pub(crate) fn default_search() -> Self {
Self {
description: Some(
"Default Search API Key (Use it to search from the frontend)".to_string(),
),
id: generate_id(),
actions: vec![Action::Search],
indexes: vec!["*".to_string()],
expires_at: None,
created_at: Utc::now(),
updated_at: Utc::now(),
}
}
}
/// Generate a printable key of 64 characters using thread_rng.
fn generate_id() -> [u8; KEY_ID_LENGTH] {
const CHARSET: &[u8] = b"abcdefghijklmnopqrstuvwxyz0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ";
let mut rng = rand::thread_rng();
let mut bytes = [0; KEY_ID_LENGTH];
for byte in bytes.iter_mut() {
*byte = CHARSET[rng.gen_range(0..CHARSET.len())];
}
bytes
}

136
meilisearch-auth/src/lib.rs Normal file
View File

@@ -0,0 +1,136 @@
mod action;
pub mod error;
mod key;
mod store;
use std::path::Path;
use std::str::from_utf8;
use std::sync::Arc;
use chrono::Utc;
use serde_json::Value;
use sha2::{Digest, Sha256};
pub use action::{actions, Action};
use error::{AuthControllerError, Result};
pub use key::Key;
use store::HeedAuthStore;
#[derive(Clone)]
pub struct AuthController {
store: Arc<HeedAuthStore>,
master_key: Option<String>,
}
impl AuthController {
pub fn new(db_path: impl AsRef<Path>, master_key: &Option<String>) -> Result<Self> {
let store = HeedAuthStore::new(db_path)?;
if store.is_empty()? {
generate_default_keys(&store)?;
}
Ok(Self {
store: Arc::new(store),
master_key: master_key.clone(),
})
}
pub async fn create_key(&self, value: Value) -> Result<Key> {
let key = Key::create_from_value(value)?;
self.store.put_api_key(key)
}
pub async fn update_key(&self, key: impl AsRef<str>, value: Value) -> Result<Key> {
let mut key = self.get_key(key).await?;
key.update_from_value(value)?;
self.store.put_api_key(key)
}
pub async fn get_key(&self, key: impl AsRef<str>) -> Result<Key> {
self.store
.get_api_key(&key)?
.ok_or_else(|| AuthControllerError::ApiKeyNotFound(key.as_ref().to_string()))
}
pub fn get_key_filters(&self, key: impl AsRef<str>) -> Result<AuthFilter> {
let mut filters = AuthFilter::default();
if self
.master_key
.as_ref()
.map_or(false, |master_key| master_key != key.as_ref())
{
let key = self
.store
.get_api_key(&key)?
.ok_or_else(|| AuthControllerError::ApiKeyNotFound(key.as_ref().to_string()))?;
if !key.indexes.iter().any(|i| i.as_str() == "*") {
filters.indexes = Some(key.indexes);
}
}
Ok(filters)
}
pub async fn list_keys(&self) -> Result<Vec<Key>> {
self.store.list_api_keys()
}
pub async fn delete_key(&self, key: impl AsRef<str>) -> Result<()> {
if self.store.delete_api_key(&key)? {
Ok(())
} else {
Err(AuthControllerError::ApiKeyNotFound(
key.as_ref().to_string(),
))
}
}
pub fn get_master_key(&self) -> Option<&String> {
self.master_key.as_ref()
}
pub fn authenticate(&self, token: &[u8], action: Action, index: Option<&[u8]>) -> Result<bool> {
if let Some(master_key) = &self.master_key {
if let Some((id, exp)) = self
.store
// check if the key has access to all indexes.
.get_expiration_date(token, action, None)?
.or(match index {
// else check if the key has access to the requested index.
Some(index) => self.store.get_expiration_date(token, action, Some(index))?,
// or to any index if no index has been requested.
None => self.store.prefix_first_expiration_date(token, action)?,
})
{
let id = from_utf8(&id).map_err(|e| AuthControllerError::Internal(Box::new(e)))?;
if exp.map_or(true, |exp| Utc::now() < exp)
&& generate_key(master_key.as_bytes(), id).as_bytes() == token
{
return Ok(true);
}
}
}
Ok(false)
}
}
#[derive(Default)]
pub struct AuthFilter {
pub indexes: Option<Vec<String>>,
}
pub fn generate_key(master_key: &[u8], uid: &str) -> String {
let key = [uid.as_bytes(), master_key].concat();
let sha = Sha256::digest(&key);
format!("{}{:x}", uid, sha)
}
fn generate_default_keys(store: &HeedAuthStore) -> Result<()> {
store.put_api_key(Key::default_admin())?;
store.put_api_key(Key::default_search())?;
Ok(())
}

View File

@@ -0,0 +1,234 @@
use enum_iterator::IntoEnumIterator;
use std::borrow::Cow;
use std::convert::TryFrom;
use std::convert::TryInto;
use std::fs::create_dir_all;
use std::path::Path;
use std::str;
use chrono::{DateTime, Utc};
use heed::types::{ByteSlice, DecodeIgnore, SerdeJson};
use heed::{Database, Env, EnvOpenOptions, RwTxn};
use super::error::Result;
use super::{Action, Key};
const AUTH_STORE_SIZE: usize = 1_073_741_824; //1GiB
pub const KEY_ID_LENGTH: usize = 8;
const AUTH_DB_PATH: &str = "auth";
const KEY_DB_NAME: &str = "api-keys";
const KEY_ID_ACTION_INDEX_EXPIRATION_DB_NAME: &str = "keyid-action-index-expiration";
pub type KeyId = [u8; KEY_ID_LENGTH];
#[derive(Clone)]
pub struct HeedAuthStore {
env: Env,
keys: Database<ByteSlice, SerdeJson<Key>>,
action_keyid_index_expiration: Database<KeyIdActionCodec, SerdeJson<Option<DateTime<Utc>>>>,
}
impl HeedAuthStore {
pub fn new(path: impl AsRef<Path>) -> Result<Self> {
let path = path.as_ref().join(AUTH_DB_PATH);
create_dir_all(&path)?;
let mut options = EnvOpenOptions::new();
options.map_size(AUTH_STORE_SIZE); // 1GB
options.max_dbs(2);
let env = options.open(path)?;
let keys = env.create_database(Some(KEY_DB_NAME))?;
let action_keyid_index_expiration =
env.create_database(Some(KEY_ID_ACTION_INDEX_EXPIRATION_DB_NAME))?;
Ok(Self {
env,
keys,
action_keyid_index_expiration,
})
}
pub fn is_empty(&self) -> Result<bool> {
let rtxn = self.env.read_txn()?;
Ok(self.keys.len(&rtxn)? == 0)
}
pub fn put_api_key(&self, key: Key) -> Result<Key> {
let mut wtxn = self.env.write_txn()?;
self.keys.put(&mut wtxn, &key.id, &key)?;
let id = key.id;
// delete key from inverted database before refilling it.
self.delete_key_from_inverted_db(&mut wtxn, &id)?;
// create inverted database.
let db = self.action_keyid_index_expiration;
let actions = if key.actions.contains(&Action::All) {
// if key.actions contains All, we iterate over all actions.
Action::into_enum_iter().collect()
} else {
key.actions.clone()
};
let no_index_restriction = key.indexes.contains(&"*".to_owned());
for action in actions {
if no_index_restriction {
// If there is no index restriction we put None.
db.put(&mut wtxn, &(&id, &action, None), &key.expires_at)?;
} else {
// else we create a key for each index.
for index in key.indexes.iter() {
db.put(
&mut wtxn,
&(&id, &action, Some(index.as_bytes())),
&key.expires_at,
)?;
}
}
}
wtxn.commit()?;
Ok(key)
}
pub fn get_api_key(&self, key: impl AsRef<str>) -> Result<Option<Key>> {
let rtxn = self.env.read_txn()?;
match try_split_array_at::<_, KEY_ID_LENGTH>(key.as_ref().as_bytes()) {
Some((id, _)) => self.keys.get(&rtxn, id).map_err(|e| e.into()),
None => Ok(None),
}
}
pub fn delete_api_key(&self, key: impl AsRef<str>) -> Result<bool> {
let mut wtxn = self.env.write_txn()?;
let existing = match try_split_array_at(key.as_ref().as_bytes()) {
Some((id, _)) => {
let existing = self.keys.delete(&mut wtxn, id)?;
self.delete_key_from_inverted_db(&mut wtxn, id)?;
existing
}
None => false,
};
wtxn.commit()?;
Ok(existing)
}
pub fn list_api_keys(&self) -> Result<Vec<Key>> {
let mut list = Vec::new();
let rtxn = self.env.read_txn()?;
for result in self.keys.remap_key_type::<DecodeIgnore>().iter(&rtxn)? {
let (_, content) = result?;
list.push(content);
}
Ok(list)
}
pub fn get_expiration_date(
&self,
key: &[u8],
action: Action,
index: Option<&[u8]>,
) -> Result<Option<(KeyId, Option<DateTime<Utc>>)>> {
let rtxn = self.env.read_txn()?;
match try_split_array_at::<_, KEY_ID_LENGTH>(key) {
Some((id, _)) => {
let tuple = (id, &action, index);
Ok(self
.action_keyid_index_expiration
.get(&rtxn, &tuple)?
.map(|expiration| (*id, expiration)))
}
None => Ok(None),
}
}
pub fn prefix_first_expiration_date(
&self,
key: &[u8],
action: Action,
) -> Result<Option<(KeyId, Option<DateTime<Utc>>)>> {
let rtxn = self.env.read_txn()?;
match try_split_array_at::<_, KEY_ID_LENGTH>(key) {
Some((id, _)) => {
let tuple = (id, &action, None);
Ok(self
.action_keyid_index_expiration
.prefix_iter(&rtxn, &tuple)?
.next()
.transpose()?
.map(|(_, expiration)| (*id, expiration)))
}
None => Ok(None),
}
}
fn delete_key_from_inverted_db(&self, wtxn: &mut RwTxn, key: &KeyId) -> Result<()> {
let mut iter = self
.action_keyid_index_expiration
.remap_types::<ByteSlice, DecodeIgnore>()
.prefix_iter_mut(wtxn, key)?;
while iter.next().transpose()?.is_some() {
// safety: we don't keep references from inside the LMDB database.
unsafe { iter.del_current()? };
}
Ok(())
}
}
/// Codec allowing to retrieve the expiration date of an action,
/// optionnally on a spcific index, for a given key.
pub struct KeyIdActionCodec;
impl<'a> heed::BytesDecode<'a> for KeyIdActionCodec {
type DItem = (KeyId, Action, Option<&'a [u8]>);
fn bytes_decode(bytes: &'a [u8]) -> Option<Self::DItem> {
let (key_id, action_bytes) = try_split_array_at(bytes)?;
let (action_bytes, index) = match try_split_array_at(action_bytes)? {
(action, []) => (action, None),
(action, index) => (action, Some(index)),
};
let action = Action::from_repr(u8::from_be_bytes(*action_bytes))?;
Some((*key_id, action, index))
}
}
impl<'a> heed::BytesEncode<'a> for KeyIdActionCodec {
type EItem = (&'a KeyId, &'a Action, Option<&'a [u8]>);
fn bytes_encode((key_id, action, index): &Self::EItem) -> Option<Cow<[u8]>> {
let mut bytes = Vec::new();
bytes.extend_from_slice(*key_id);
let action_bytes = u8::to_be_bytes(action.repr());
bytes.extend_from_slice(&action_bytes);
if let Some(index) = index {
bytes.extend_from_slice(index);
}
Some(Cow::Owned(bytes))
}
}
/// Divides one slice into two at an index, returns `None` if mid is out of bounds.
pub fn try_split_at<T>(slice: &[T], mid: usize) -> Option<(&[T], &[T])> {
if mid <= slice.len() {
Some(slice.split_at(mid))
} else {
None
}
}
/// Divides one slice into an array and the tail at an index,
/// returns `None` if `N` is out of bounds.
pub fn try_split_array_at<T, const N: usize>(slice: &[T]) -> Option<(&[T; N], &[T])>
where
[T; N]: for<'a> TryFrom<&'a [T]>,
{
let (head, tail) = try_split_at(slice, N)?;
let head = head.try_into().ok()?;
Some((head, tail))
}

View File

@@ -159,6 +159,13 @@ pub enum Code {
MissingContentType, MissingContentType,
MalformedPayload, MalformedPayload,
MissingPayload, MissingPayload,
ApiKeyNotFound,
MissingParameter,
InvalidApiKeyActions,
InvalidApiKeyIndexes,
InvalidApiKeyExpiresAt,
InvalidApiKeyDescription,
} }
impl Code { impl Code {
@@ -249,6 +256,22 @@ impl Code {
ErrCode::invalid("invalid_content_type", StatusCode::UNSUPPORTED_MEDIA_TYPE) ErrCode::invalid("invalid_content_type", StatusCode::UNSUPPORTED_MEDIA_TYPE)
} }
MissingPayload => ErrCode::invalid("missing_payload", StatusCode::BAD_REQUEST), MissingPayload => ErrCode::invalid("missing_payload", StatusCode::BAD_REQUEST),
// error related to keys
ApiKeyNotFound => ErrCode::invalid("api_key_not_found", StatusCode::NOT_FOUND),
MissingParameter => ErrCode::invalid("missing_parameter", StatusCode::BAD_REQUEST),
InvalidApiKeyActions => {
ErrCode::invalid("invalid_api_key_actions", StatusCode::BAD_REQUEST)
}
InvalidApiKeyIndexes => {
ErrCode::invalid("invalid_api_key_indexes", StatusCode::BAD_REQUEST)
}
InvalidApiKeyExpiresAt => {
ErrCode::invalid("invalid_api_key_expires_at", StatusCode::BAD_REQUEST)
}
InvalidApiKeyDescription => {
ErrCode::invalid("invalid_api_key_description", StatusCode::BAD_REQUEST)
}
} }
} }
@@ -316,3 +339,16 @@ mod strategy {
(100..999u16).prop_map(|i| StatusCode::from_u16(i).unwrap()) (100..999u16).prop_map(|i| StatusCode::from_u16(i).unwrap())
} }
} }
#[macro_export]
macro_rules! internal_error {
($target:ty : $($other:path), *) => {
$(
impl From<$other> for $target {
fn from(other: $other) -> Self {
Self::Internal(Box::new(other))
}
}
)*
}
}

View File

@@ -47,6 +47,7 @@ http = "0.2.4"
indexmap = { version = "1.7.0", features = ["serde-1"] } indexmap = { version = "1.7.0", features = ["serde-1"] }
itertools = "0.10.1" itertools = "0.10.1"
log = "0.4.14" log = "0.4.14"
meilisearch-auth = { path = "../meilisearch-auth" }
meilisearch-error = { path = "../meilisearch-error" } meilisearch-error = { path = "../meilisearch-error" }
meilisearch-lib = { path = "../meilisearch-lib" } meilisearch-lib = { path = "../meilisearch-lib" }
meilisearch-tokenizer = { git = "https://github.com/meilisearch/tokenizer.git", tag = "v0.2.5" } meilisearch-tokenizer = { git = "https://github.com/meilisearch/tokenizer.git", tag = "v0.2.5" }
@@ -79,6 +80,8 @@ walkdir = "2.3.2"
[dev-dependencies] [dev-dependencies]
actix-rt = "2.2.0" actix-rt = "2.2.0"
assert-json-diff = "2.0.1"
maplit = "1.0.2"
paste = "1.0.5" paste = "1.0.5"
serde_url_params = "0.2.1" serde_url_params = "0.2.1"
urlencoding = "2.1.0" urlencoding = "2.1.0"

View File

@@ -259,7 +259,7 @@ impl Segment {
} }
async fn tick(&mut self, meilisearch: MeiliSearch) { async fn tick(&mut self, meilisearch: MeiliSearch) {
if let Ok(stats) = meilisearch.get_all_stats().await { if let Ok(stats) = meilisearch.get_all_stats(&None).await {
let _ = self let _ = self
.batcher .batcher
.push(Identify { .push(Identify {

View File

@@ -1,6 +1,3 @@
use std::error::Error;
use std::fmt;
use actix_web as aweb; use actix_web as aweb;
use aweb::error::{JsonPayloadError, QueryPayloadError}; use aweb::error::{JsonPayloadError, QueryPayloadError};
use meilisearch_error::{Code, ErrorCode, ResponseError}; use meilisearch_error::{Code, ErrorCode, ResponseError};
@@ -32,23 +29,18 @@ impl From<MeilisearchHttpError> for aweb::Error {
} }
} }
impl fmt::Display for PayloadError { #[derive(Debug, thiserror::Error)]
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
PayloadError::Json(e) => e.fmt(f),
PayloadError::Query(e) => e.fmt(f),
}
}
}
#[derive(Debug)]
pub enum PayloadError { pub enum PayloadError {
#[error("{0}")]
Json(JsonPayloadError), Json(JsonPayloadError),
#[error("{0}")]
Query(QueryPayloadError), Query(QueryPayloadError),
#[error("The json payload provided is malformed. `{0}`.")]
MalformedPayload(serde_json::error::Error),
#[error("A json payload is missing.")]
MissingPayload,
} }
impl Error for PayloadError {}
impl ErrorCode for PayloadError { impl ErrorCode for PayloadError {
fn error_code(&self) -> Code { fn error_code(&self) -> Code {
match self { match self {
@@ -58,7 +50,8 @@ impl ErrorCode for PayloadError {
JsonPayloadError::Payload(aweb::error::PayloadError::Overflow) => { JsonPayloadError::Payload(aweb::error::PayloadError::Overflow) => {
Code::PayloadTooLarge Code::PayloadTooLarge
} }
JsonPayloadError::Deserialize(_) | JsonPayloadError::Payload(_) => Code::BadRequest, JsonPayloadError::Payload(_) => Code::BadRequest,
JsonPayloadError::Deserialize(_) => Code::BadRequest,
JsonPayloadError::Serialize(_) => Code::Internal, JsonPayloadError::Serialize(_) => Code::Internal,
_ => Code::Internal, _ => Code::Internal,
}, },
@@ -66,13 +59,29 @@ impl ErrorCode for PayloadError {
QueryPayloadError::Deserialize(_) => Code::BadRequest, QueryPayloadError::Deserialize(_) => Code::BadRequest,
_ => Code::Internal, _ => Code::Internal,
}, },
PayloadError::MissingPayload => Code::MissingPayload,
PayloadError::MalformedPayload(_) => Code::MalformedPayload,
} }
} }
} }
impl From<JsonPayloadError> for PayloadError { impl From<JsonPayloadError> for PayloadError {
fn from(other: JsonPayloadError) -> Self { fn from(other: JsonPayloadError) -> Self {
Self::Json(other) match other {
JsonPayloadError::Deserialize(e)
if e.classify() == serde_json::error::Category::Eof
&& e.line() == 1
&& e.column() == 0 =>
{
Self::MissingPayload
}
JsonPayloadError::Deserialize(e)
if e.classify() != serde_json::error::Category::Data =>
{
Self::MalformedPayload(e)
}
_ => Self::Json(other),
}
} }
} }

View File

@@ -2,15 +2,13 @@ use meilisearch_error::{Code, ErrorCode};
#[derive(Debug, thiserror::Error)] #[derive(Debug, thiserror::Error)]
pub enum AuthenticationError { pub enum AuthenticationError {
#[error("The X-MEILI-API-KEY header is missing.")] #[error("The Authorization header is missing. It must use the bearer authorization method.")]
MissingAuthorizationHeader, MissingAuthorizationHeader,
#[error("The provided API key is invalid.")] #[error("The provided API key is invalid.")]
InvalidToken(String), InvalidToken(String),
// Triggered on configuration error. // Triggered on configuration error.
#[error("An internal error has occurred. `Irretrievable state`.")] #[error("An internal error has occurred. `Irretrievable state`.")]
IrretrievableState, IrretrievableState,
#[error("An internal error has occurred. `Unknown authentication policy`.")]
UnknownPolicy,
} }
impl ErrorCode for AuthenticationError { impl ErrorCode for AuthenticationError {
@@ -19,7 +17,6 @@ impl ErrorCode for AuthenticationError {
AuthenticationError::MissingAuthorizationHeader => Code::MissingAuthorizationHeader, AuthenticationError::MissingAuthorizationHeader => Code::MissingAuthorizationHeader,
AuthenticationError::InvalidToken(_) => Code::InvalidToken, AuthenticationError::InvalidToken(_) => Code::InvalidToken,
AuthenticationError::IrretrievableState => Code::Internal, AuthenticationError::IrretrievableState => Code::Internal,
AuthenticationError::UnknownPolicy => Code::Internal,
} }
} }
} }

View File

@@ -1,7 +1,5 @@
mod error; mod error;
use std::any::{Any, TypeId};
use std::collections::HashMap;
use std::marker::PhantomData; use std::marker::PhantomData;
use std::ops::Deref; use std::ops::Deref;
@@ -11,73 +9,20 @@ use futures::future::{ok, Ready};
use meilisearch_error::ResponseError; use meilisearch_error::ResponseError;
use error::AuthenticationError; use error::AuthenticationError;
use meilisearch_auth::{AuthController, AuthFilter};
macro_rules! create_policies {
($($name:ident), *) => {
pub mod policies {
use std::collections::HashSet;
use crate::extractors::authentication::Policy;
$(
#[derive(Debug, Default)]
pub struct $name {
inner: HashSet<Vec<u8>>
}
impl $name {
pub fn new() -> Self {
Self { inner: HashSet::new() }
}
pub fn add(&mut self, token: Vec<u8>) {
self.inner.insert(token);
}
}
impl Policy for $name {
fn authenticate(&self, token: &[u8]) -> bool {
self.inner.contains(token)
}
}
)*
}
};
}
create_policies!(Public, Private, Admin);
/// Instanciate a `Policies`, filled with the given policies.
macro_rules! init_policies {
($($name:ident), *) => {
{
let mut policies = crate::extractors::authentication::Policies::new();
$(
let policy = $name::new();
policies.insert(policy);
)*
policies
}
};
}
/// Adds user to all specified policies.
macro_rules! create_users {
($policies:ident, $($user:expr => { $($policy:ty), * }), *) => {
{
$(
$(
$policies.get_mut::<$policy>().map(|p| p.add($user.to_owned()));
)*
)*
}
};
}
pub struct GuardedData<T, D> { pub struct GuardedData<T, D> {
data: D, data: D,
filters: AuthFilter,
_marker: PhantomData<T>, _marker: PhantomData<T>,
} }
impl<T, D> GuardedData<T, D> {
pub fn filters(&self) -> &AuthFilter {
&self.filters
}
}
impl<T, D> Deref for GuardedData<T, D> { impl<T, D> Deref for GuardedData<T, D> {
type Target = D; type Target = D;
@@ -86,56 +31,6 @@ impl<T, D> Deref for GuardedData<T, D> {
} }
} }
pub trait Policy {
fn authenticate(&self, token: &[u8]) -> bool;
}
#[derive(Debug)]
pub struct Policies {
inner: HashMap<TypeId, Box<dyn Any>>,
}
impl Policies {
pub fn new() -> Self {
Self {
inner: HashMap::new(),
}
}
pub fn insert<S: Policy + 'static>(&mut self, policy: S) {
self.inner.insert(TypeId::of::<S>(), Box::new(policy));
}
pub fn get<S: Policy + 'static>(&self) -> Option<&S> {
self.inner
.get(&TypeId::of::<S>())
.and_then(|p| p.downcast_ref::<S>())
}
pub fn get_mut<S: Policy + 'static>(&mut self) -> Option<&mut S> {
self.inner
.get_mut(&TypeId::of::<S>())
.and_then(|p| p.downcast_mut::<S>())
}
}
impl Default for Policies {
fn default() -> Self {
Self::new()
}
}
pub enum AuthConfig {
NoAuth,
Auth(Policies),
}
impl Default for AuthConfig {
fn default() -> Self {
Self::NoAuth
}
}
impl<P: Policy + 'static, D: 'static + Clone> FromRequest for GuardedData<P, D> { impl<P: Policy + 'static, D: 'static + Clone> FromRequest for GuardedData<P, D> {
type Config = AuthConfig; type Config = AuthConfig;
@@ -152,32 +47,113 @@ impl<P: Policy + 'static, D: 'static + Clone> FromRequest for GuardedData<P, D>
AuthConfig::NoAuth => match req.app_data::<D>().cloned() { AuthConfig::NoAuth => match req.app_data::<D>().cloned() {
Some(data) => ok(Self { Some(data) => ok(Self {
data, data,
filters: AuthFilter::default(),
_marker: PhantomData, _marker: PhantomData,
}), }),
None => err(AuthenticationError::IrretrievableState.into()), None => err(AuthenticationError::IrretrievableState.into()),
}, },
AuthConfig::Auth(policies) => match policies.get::<P>() { AuthConfig::Auth => match req.app_data::<AuthController>().cloned() {
Some(policy) => match req.headers().get("x-meili-api-key") { Some(auth) => match req
Some(token) => { .headers()
if policy.authenticate(token.as_bytes()) { .get("Authorization")
match req.app_data::<D>().cloned() { .map(|type_token| type_token.to_str().unwrap_or_default().splitn(2, ' '))
Some(data) => ok(Self { {
data, Some(mut type_token) => match type_token.next() {
_marker: PhantomData, Some("Bearer") => {
}), // TODO: find a less hardcoded way?
None => err(AuthenticationError::IrretrievableState.into()), let index = req.match_info().get("index_uid");
let token = type_token.next().unwrap_or("unknown");
match P::authenticate(auth, token, index) {
Some(filters) => match req.app_data::<D>().cloned() {
Some(data) => ok(Self {
data,
filters,
_marker: PhantomData,
}),
None => err(AuthenticationError::IrretrievableState.into()),
},
None => {
let token = token.to_string();
err(AuthenticationError::InvalidToken(token).into())
}
} }
} else {
let token = token.to_str().unwrap_or("unknown").to_string();
err(AuthenticationError::InvalidToken(token).into())
} }
} _otherwise => {
err(AuthenticationError::MissingAuthorizationHeader.into())
}
},
None => err(AuthenticationError::MissingAuthorizationHeader.into()), None => err(AuthenticationError::MissingAuthorizationHeader.into()),
}, },
None => err(AuthenticationError::UnknownPolicy.into()), None => err(AuthenticationError::IrretrievableState.into()),
}, },
}, },
None => err(AuthenticationError::IrretrievableState.into()), None => err(AuthenticationError::IrretrievableState.into()),
} }
} }
} }
pub trait Policy {
fn authenticate(auth: AuthController, token: &str, index: Option<&str>) -> Option<AuthFilter>;
}
pub mod policies {
use crate::extractors::authentication::Policy;
use meilisearch_auth::{Action, AuthController, AuthFilter};
// reexport actions in policies in order to be used in routes configuration.
pub use meilisearch_auth::actions;
pub struct MasterPolicy;
impl Policy for MasterPolicy {
fn authenticate(
auth: AuthController,
token: &str,
_index: Option<&str>,
) -> Option<AuthFilter> {
if let Some(master_key) = auth.get_master_key() {
if master_key == token {
return Some(AuthFilter::default());
}
}
None
}
}
pub struct ActionPolicy<const A: u8>;
impl<const A: u8> Policy for ActionPolicy<A> {
fn authenticate(
auth: AuthController,
token: &str,
index: Option<&str>,
) -> Option<AuthFilter> {
// authenticate if token is the master key.
if let Some(master_key) = auth.get_master_key() {
if master_key == token {
return Some(AuthFilter::default());
}
}
// authenticate if token is allowed.
if let Some(action) = Action::from_repr(A) {
let index = index.map(|i| i.as_bytes());
if let Ok(true) = auth.authenticate(token.as_bytes(), action, index) {
return auth.get_key_filters(token).ok();
}
}
None
}
}
}
pub enum AuthConfig {
NoAuth,
Auth,
}
impl Default for AuthConfig {
fn default() -> Self {
Self::NoAuth
}
}

View File

@@ -22,8 +22,8 @@ pub use option::Opt;
use actix_web::{web, HttpRequest}; use actix_web::{web, HttpRequest};
use extractors::authentication::policies::*;
use extractors::payload::PayloadConfig; use extractors::payload::PayloadConfig;
use meilisearch_auth::AuthController;
use meilisearch_lib::MeiliSearch; use meilisearch_lib::MeiliSearch;
use sha2::Digest; use sha2::Digest;
@@ -80,12 +80,14 @@ pub fn setup_meilisearch(opt: &Opt) -> anyhow::Result<MeiliSearch> {
pub fn configure_data( pub fn configure_data(
config: &mut web::ServiceConfig, config: &mut web::ServiceConfig,
data: MeiliSearch, data: MeiliSearch,
auth: AuthController,
opt: &Opt, opt: &Opt,
analytics: Arc<dyn Analytics>, analytics: Arc<dyn Analytics>,
) { ) {
let http_payload_size_limit = opt.http_payload_size_limit.get_bytes() as usize; let http_payload_size_limit = opt.http_payload_size_limit.get_bytes() as usize;
config config
.app_data(data) .app_data(data)
.app_data(auth)
.app_data(web::Data::from(analytics)) .app_data(web::Data::from(analytics))
.app_data( .app_data(
web::JsonConfig::default() web::JsonConfig::default()
@@ -112,30 +114,13 @@ pub fn configure_data(
} }
pub fn configure_auth(config: &mut web::ServiceConfig, opts: &Opt) { pub fn configure_auth(config: &mut web::ServiceConfig, opts: &Opt) {
let mut keys = ApiKeys { let auth_config = if opts.master_key.is_some() {
master: opts.master_key.clone(), AuthConfig::Auth
private: None,
public: None,
};
keys.generate_missing_api_keys();
let auth_config = if let Some(ref master_key) = keys.master {
let private_key = keys.private.as_ref().unwrap();
let public_key = keys.public.as_ref().unwrap();
let mut policies = init_policies!(Public, Private, Admin);
create_users!(
policies,
master_key.as_bytes() => { Admin, Private, Public },
private_key.as_bytes() => { Private, Public },
public_key.as_bytes() => { Public }
);
AuthConfig::Auth(policies)
} else { } else {
AuthConfig::NoAuth AuthConfig::NoAuth
}; };
config.app_data(auth_config).app_data(keys); config.app_data(auth_config);
} }
#[cfg(feature = "mini-dashboard")] #[cfg(feature = "mini-dashboard")]
@@ -177,7 +162,7 @@ pub fn dashboard(config: &mut web::ServiceConfig, _enable_frontend: bool) {
#[macro_export] #[macro_export]
macro_rules! create_app { macro_rules! create_app {
($data:expr, $enable_frontend:expr, $opt:expr, $analytics:expr) => {{ ($data:expr, $auth:expr, $enable_frontend:expr, $opt:expr, $analytics:expr) => {{
use actix_cors::Cors; use actix_cors::Cors;
use actix_web::middleware::TrailingSlash; use actix_web::middleware::TrailingSlash;
use actix_web::App; use actix_web::App;
@@ -188,7 +173,7 @@ macro_rules! create_app {
use meilisearch_http::{configure_auth, configure_data, dashboard}; use meilisearch_http::{configure_auth, configure_data, dashboard};
App::new() App::new()
.configure(|s| configure_data(s, $data.clone(), &$opt, $analytics)) .configure(|s| configure_data(s, $data.clone(), $auth.clone(), &$opt, $analytics))
.configure(|s| configure_auth(s, &$opt)) .configure(|s| configure_auth(s, &$opt))
.configure(routes::configure) .configure(routes::configure)
.configure(|s| dashboard(s, $enable_frontend)) .configure(|s| dashboard(s, $enable_frontend))

View File

@@ -2,6 +2,7 @@ use std::env;
use std::sync::Arc; use std::sync::Arc;
use actix_web::HttpServer; use actix_web::HttpServer;
use meilisearch_auth::AuthController;
use meilisearch_http::analytics; use meilisearch_http::analytics;
use meilisearch_http::analytics::Analytics; use meilisearch_http::analytics::Analytics;
use meilisearch_http::{create_app, setup_meilisearch, Opt}; use meilisearch_http::{create_app, setup_meilisearch, Opt};
@@ -46,6 +47,8 @@ async fn main() -> anyhow::Result<()> {
let meilisearch = setup_meilisearch(&opt)?; let meilisearch = setup_meilisearch(&opt)?;
let auth_controller = AuthController::new(&opt.db_path, &opt.master_key)?;
#[cfg(all(not(debug_assertions), feature = "analytics"))] #[cfg(all(not(debug_assertions), feature = "analytics"))]
let (analytics, user) = if !opt.no_analytics { let (analytics, user) = if !opt.no_analytics {
analytics::SegmentAnalytics::new(&opt, &meilisearch).await analytics::SegmentAnalytics::new(&opt, &meilisearch).await
@@ -57,22 +60,30 @@ async fn main() -> anyhow::Result<()> {
print_launch_resume(&opt, &user); print_launch_resume(&opt, &user);
run_http(meilisearch, opt, analytics).await?; run_http(meilisearch, auth_controller, opt, analytics).await?;
Ok(()) Ok(())
} }
async fn run_http( async fn run_http(
data: MeiliSearch, data: MeiliSearch,
auth_controller: AuthController,
opt: Opt, opt: Opt,
analytics: Arc<dyn Analytics>, analytics: Arc<dyn Analytics>,
) -> anyhow::Result<()> { ) -> anyhow::Result<()> {
let _enable_dashboard = &opt.env == "development"; let _enable_dashboard = &opt.env == "development";
let opt_clone = opt.clone(); let opt_clone = opt.clone();
let http_server = let http_server = HttpServer::new(move || {
HttpServer::new(move || create_app!(data, _enable_dashboard, opt_clone, analytics.clone())) create_app!(
// Disable signals allows the server to terminate immediately when a user enter CTRL-C data,
.disable_signals(); auth_controller,
_enable_dashboard,
opt_clone,
analytics.clone()
)
})
// Disable signals allows the server to terminate immediately when a user enter CTRL-C
.disable_signals();
if let Some(config) = opt.get_ssl_config()? { if let Some(config) = opt.get_ssl_config()? {
http_server http_server

View File

@@ -0,0 +1,126 @@
use std::str;
use actix_web::{web, HttpRequest, HttpResponse};
use chrono::{DateTime, Utc};
use log::debug;
use meilisearch_auth::{generate_key, Action, AuthController, Key};
use serde::{Deserialize, Serialize};
use serde_json::Value;
use crate::extractors::authentication::{policies::*, GuardedData};
use meilisearch_error::ResponseError;
pub fn configure(cfg: &mut web::ServiceConfig) {
cfg.service(
web::resource("")
.route(web::post().to(create_api_key))
.route(web::get().to(list_api_keys)),
)
.service(
web::resource("/{api_key}")
.route(web::get().to(get_api_key))
.route(web::patch().to(patch_api_key))
.route(web::delete().to(delete_api_key)),
);
}
pub async fn create_api_key(
auth_controller: GuardedData<MasterPolicy, AuthController>,
body: web::Json<Value>,
_req: HttpRequest,
) -> Result<HttpResponse, ResponseError> {
let key = auth_controller.create_key(body.into_inner()).await?;
let res = KeyView::from_key(key, auth_controller.get_master_key());
debug!("returns: {:?}", res);
Ok(HttpResponse::Created().json(res))
}
pub async fn list_api_keys(
auth_controller: GuardedData<MasterPolicy, AuthController>,
_req: HttpRequest,
) -> Result<HttpResponse, ResponseError> {
let keys = auth_controller.list_keys().await?;
let res: Vec<_> = keys
.into_iter()
.map(|k| KeyView::from_key(k, auth_controller.get_master_key()))
.collect();
debug!("returns: {:?}", res);
Ok(HttpResponse::Ok().json(res))
}
pub async fn get_api_key(
auth_controller: GuardedData<MasterPolicy, AuthController>,
path: web::Path<AuthParam>,
) -> Result<HttpResponse, ResponseError> {
// keep 8 first characters that are the ID of the API key.
let key = auth_controller.get_key(&path.api_key).await?;
let res = KeyView::from_key(key, auth_controller.get_master_key());
debug!("returns: {:?}", res);
Ok(HttpResponse::Ok().json(res))
}
pub async fn patch_api_key(
auth_controller: GuardedData<MasterPolicy, AuthController>,
body: web::Json<Value>,
path: web::Path<AuthParam>,
) -> Result<HttpResponse, ResponseError> {
let key = auth_controller
// keep 8 first characters that are the ID of the API key.
.update_key(&path.api_key, body.into_inner())
.await?;
let res = KeyView::from_key(key, auth_controller.get_master_key());
debug!("returns: {:?}", res);
Ok(HttpResponse::Ok().json(res))
}
pub async fn delete_api_key(
auth_controller: GuardedData<MasterPolicy, AuthController>,
path: web::Path<AuthParam>,
) -> Result<HttpResponse, ResponseError> {
// keep 8 first characters that are the ID of the API key.
auth_controller.delete_key(&path.api_key).await?;
Ok(HttpResponse::NoContent().json(()))
}
#[derive(Deserialize)]
pub struct AuthParam {
api_key: String,
}
#[derive(Debug, Serialize)]
#[serde(rename_all = "camelCase")]
struct KeyView {
#[serde(skip_serializing_if = "Option::is_none")]
description: Option<String>,
key: String,
actions: Vec<Action>,
indexes: Vec<String>,
expires_at: Option<DateTime<Utc>>,
created_at: DateTime<Utc>,
updated_at: DateTime<Utc>,
}
impl KeyView {
fn from_key(key: Key, master_key: Option<&String>) -> Self {
let key_id = str::from_utf8(&key.id).unwrap();
let generated_key = match master_key {
Some(master_key) => generate_key(master_key.as_bytes(), key_id),
None => generate_key(&[], key_id),
};
KeyView {
description: key.description,
key: generated_key,
actions: key.actions,
indexes: key.indexes,
expires_at: key.expires_at,
created_at: key.created_at,
updated_at: key.updated_at,
}
}
}

View File

@@ -14,7 +14,7 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
} }
pub async fn create_dump( pub async fn create_dump(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DUMPS_CREATE }>, MeiliSearch>,
req: HttpRequest, req: HttpRequest,
analytics: web::Data<dyn Analytics>, analytics: web::Data<dyn Analytics>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
@@ -38,7 +38,7 @@ struct DumpParam {
} }
async fn get_dump_status( async fn get_dump_status(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DUMPS_GET }>, MeiliSearch>,
path: web::Path<DumpParam>, path: web::Path<DumpParam>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let res = meilisearch.dump_info(path.dump_uid.clone()).await?; let res = meilisearch.dump_info(path.dump_uid.clone()).await?;

View File

@@ -86,7 +86,7 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
} }
pub async fn get_document( pub async fn get_document(
meilisearch: GuardedData<Public, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DOCUMENTS_GET }>, MeiliSearch>,
path: web::Path<DocumentParam>, path: web::Path<DocumentParam>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let index = path.index_uid.clone(); let index = path.index_uid.clone();
@@ -99,7 +99,7 @@ pub async fn get_document(
} }
pub async fn delete_document( pub async fn delete_document(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DOCUMENTS_DELETE }>, MeiliSearch>,
path: web::Path<DocumentParam>, path: web::Path<DocumentParam>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let DocumentParam { let DocumentParam {
@@ -121,7 +121,7 @@ pub struct BrowseQuery {
} }
pub async fn get_all_documents( pub async fn get_all_documents(
meilisearch: GuardedData<Public, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DOCUMENTS_GET }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
params: web::Query<BrowseQuery>, params: web::Query<BrowseQuery>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
@@ -156,7 +156,7 @@ pub struct UpdateDocumentsQuery {
} }
pub async fn add_documents( pub async fn add_documents(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DOCUMENTS_ADD }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
params: web::Query<UpdateDocumentsQuery>, params: web::Query<UpdateDocumentsQuery>,
body: Payload, body: Payload,
@@ -187,7 +187,7 @@ pub async fn add_documents(
} }
pub async fn update_documents( pub async fn update_documents(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DOCUMENTS_ADD }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
params: web::Query<UpdateDocumentsQuery>, params: web::Query<UpdateDocumentsQuery>,
body: Payload, body: Payload,
@@ -218,7 +218,7 @@ pub async fn update_documents(
async fn document_addition( async fn document_addition(
mime_type: Option<Mime>, mime_type: Option<Mime>,
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DOCUMENTS_ADD }>, MeiliSearch>,
index_uid: String, index_uid: String,
primary_key: Option<String>, primary_key: Option<String>,
body: Payload, body: Payload,
@@ -259,7 +259,7 @@ async fn document_addition(
} }
pub async fn delete_documents( pub async fn delete_documents(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DOCUMENTS_DELETE }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
body: web::Json<Vec<Value>>, body: web::Json<Vec<Value>>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
@@ -284,7 +284,7 @@ pub async fn delete_documents(
} }
pub async fn clear_all_documents( pub async fn clear_all_documents(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::DOCUMENTS_DELETE }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let update = Update::ClearDocuments; let update = Update::ClearDocuments;

View File

@@ -39,9 +39,17 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
} }
pub async fn list_indexes( pub async fn list_indexes(
data: GuardedData<Private, MeiliSearch>, data: GuardedData<ActionPolicy<{ actions::INDEXES_GET }>, MeiliSearch>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let indexes = data.list_indexes().await?; let filters = data.filters();
let mut indexes = data.list_indexes().await?;
if let Some(indexes_filter) = filters.indexes.as_ref() {
indexes = indexes
.into_iter()
.filter(|i| indexes_filter.contains(&i.uid))
.collect();
}
debug!("returns: {:?}", indexes); debug!("returns: {:?}", indexes);
Ok(HttpResponse::Ok().json(indexes)) Ok(HttpResponse::Ok().json(indexes))
} }
@@ -54,7 +62,7 @@ pub struct IndexCreateRequest {
} }
pub async fn create_index( pub async fn create_index(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::INDEXES_ADD }>, MeiliSearch>,
body: web::Json<IndexCreateRequest>, body: web::Json<IndexCreateRequest>,
req: HttpRequest, req: HttpRequest,
analytics: web::Data<dyn Analytics>, analytics: web::Data<dyn Analytics>,
@@ -94,7 +102,7 @@ pub struct UpdateIndexResponse {
} }
pub async fn get_index( pub async fn get_index(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::INDEXES_GET }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let meta = meilisearch.get_index(path.into_inner()).await?; let meta = meilisearch.get_index(path.into_inner()).await?;
@@ -103,7 +111,7 @@ pub async fn get_index(
} }
pub async fn update_index( pub async fn update_index(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::INDEXES_UPDATE }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
body: web::Json<UpdateIndexRequest>, body: web::Json<UpdateIndexRequest>,
req: HttpRequest, req: HttpRequest,
@@ -131,7 +139,7 @@ pub async fn update_index(
} }
pub async fn delete_index( pub async fn delete_index(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::INDEXES_DELETE }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let uid = path.into_inner(); let uid = path.into_inner();
@@ -142,7 +150,7 @@ pub async fn delete_index(
} }
pub async fn get_index_stats( pub async fn get_index_stats(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::STATS_GET }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let response = meilisearch.get_index_stats(path.into_inner()).await?; let response = meilisearch.get_index_stats(path.into_inner()).await?;

View File

@@ -106,7 +106,7 @@ fn fix_sort_query_parameters(sort_query: &str) -> Vec<String> {
} }
pub async fn search_with_url_query( pub async fn search_with_url_query(
meilisearch: GuardedData<Public, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::SEARCH }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
params: web::Query<SearchQueryGet>, params: web::Query<SearchQueryGet>,
req: HttpRequest, req: HttpRequest,
@@ -134,7 +134,7 @@ pub async fn search_with_url_query(
} }
pub async fn search_with_post( pub async fn search_with_post(
meilisearch: GuardedData<Public, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::SEARCH }>, MeiliSearch>,
path: web::Path<String>, path: web::Path<String>,
params: web::Json<SearchQuery>, params: web::Json<SearchQuery>,
req: HttpRequest, req: HttpRequest,

View File

@@ -27,7 +27,7 @@ macro_rules! make_setting_route {
use meilisearch_error::ResponseError; use meilisearch_error::ResponseError;
pub async fn delete( pub async fn delete(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::SETTINGS_UPDATE }>, MeiliSearch>,
index_uid: web::Path<String>, index_uid: web::Path<String>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let settings = Settings { let settings = Settings {
@@ -48,7 +48,7 @@ macro_rules! make_setting_route {
} }
pub async fn update( pub async fn update(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::SETTINGS_UPDATE }>, MeiliSearch>,
index_uid: actix_web::web::Path<String>, index_uid: actix_web::web::Path<String>,
body: actix_web::web::Json<Option<$type>>, body: actix_web::web::Json<Option<$type>>,
req: HttpRequest, req: HttpRequest,
@@ -80,7 +80,7 @@ macro_rules! make_setting_route {
} }
pub async fn get( pub async fn get(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::SETTINGS_GET }>, MeiliSearch>,
index_uid: actix_web::web::Path<String>, index_uid: actix_web::web::Path<String>,
) -> std::result::Result<HttpResponse, ResponseError> { ) -> std::result::Result<HttpResponse, ResponseError> {
let settings = meilisearch.settings(index_uid.into_inner()).await?; let settings = meilisearch.settings(index_uid.into_inner()).await?;
@@ -243,7 +243,7 @@ generate_configure!(
); );
pub async fn update_all( pub async fn update_all(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::SETTINGS_UPDATE }>, MeiliSearch>,
index_uid: web::Path<String>, index_uid: web::Path<String>,
body: web::Json<Settings<Unchecked>>, body: web::Json<Settings<Unchecked>>,
req: HttpRequest, req: HttpRequest,
@@ -286,7 +286,7 @@ pub async fn update_all(
} }
pub async fn get_all( pub async fn get_all(
data: GuardedData<Private, MeiliSearch>, data: GuardedData<ActionPolicy<{ actions::SETTINGS_GET }>, MeiliSearch>,
index_uid: web::Path<String>, index_uid: web::Path<String>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let settings = data.settings(index_uid.into_inner()).await?; let settings = data.settings(index_uid.into_inner()).await?;
@@ -295,7 +295,7 @@ pub async fn get_all(
} }
pub async fn delete_all( pub async fn delete_all(
data: GuardedData<Private, MeiliSearch>, data: GuardedData<ActionPolicy<{ actions::SETTINGS_UPDATE }>, MeiliSearch>,
index_uid: web::Path<String>, index_uid: web::Path<String>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let settings = Settings::cleared().into_unchecked(); let settings = Settings::cleared().into_unchecked();

View File

@@ -32,7 +32,7 @@ pub struct UpdateParam {
} }
pub async fn get_task_status( pub async fn get_task_status(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::TASKS_GET }>, MeiliSearch>,
index_uid: web::Path<UpdateParam>, index_uid: web::Path<UpdateParam>,
req: HttpRequest, req: HttpRequest,
analytics: web::Data<dyn Analytics>, analytics: web::Data<dyn Analytics>,
@@ -52,7 +52,7 @@ pub async fn get_task_status(
} }
pub async fn get_all_tasks_status( pub async fn get_all_tasks_status(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::TASKS_GET }>, MeiliSearch>,
index_uid: web::Path<String>, index_uid: web::Path<String>,
req: HttpRequest, req: HttpRequest,
analytics: web::Data<dyn Analytics>, analytics: web::Data<dyn Analytics>,

View File

@@ -8,8 +8,8 @@ use meilisearch_lib::index::{Settings, Unchecked};
use meilisearch_lib::MeiliSearch; use meilisearch_lib::MeiliSearch;
use crate::extractors::authentication::{policies::*, GuardedData}; use crate::extractors::authentication::{policies::*, GuardedData};
use crate::ApiKeys;
mod api_key;
mod dump; mod dump;
pub mod indexes; pub mod indexes;
mod tasks; mod tasks;
@@ -17,8 +17,8 @@ mod tasks;
pub fn configure(cfg: &mut web::ServiceConfig) { pub fn configure(cfg: &mut web::ServiceConfig) {
cfg.service(web::scope("/tasks").configure(tasks::configure)) cfg.service(web::scope("/tasks").configure(tasks::configure))
.service(web::resource("/health").route(web::get().to(get_health))) .service(web::resource("/health").route(web::get().to(get_health)))
.service(web::scope("/keys").configure(api_key::configure))
.service(web::scope("/dumps").configure(dump::configure)) .service(web::scope("/dumps").configure(dump::configure))
.service(web::resource("/keys").route(web::get().to(list_keys)))
.service(web::resource("/stats").route(web::get().to(get_stats))) .service(web::resource("/stats").route(web::get().to(get_stats)))
.service(web::resource("/version").route(web::get().to(get_version))) .service(web::resource("/version").route(web::get().to(get_version)))
.service(web::scope("/indexes").configure(indexes::configure)); .service(web::scope("/indexes").configure(indexes::configure));
@@ -125,9 +125,11 @@ pub async fn running() -> HttpResponse {
} }
async fn get_stats( async fn get_stats(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::STATS_GET }>, MeiliSearch>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
let response = meilisearch.get_all_stats().await?; let filters = meilisearch.filters();
let response = meilisearch.get_all_stats(&filters.indexes).await?;
debug!("returns: {:?}", response); debug!("returns: {:?}", response);
Ok(HttpResponse::Ok().json(response)) Ok(HttpResponse::Ok().json(response))
@@ -141,7 +143,9 @@ struct VersionResponse {
pkg_version: String, pkg_version: String,
} }
async fn get_version(_meilisearch: GuardedData<Private, MeiliSearch>) -> HttpResponse { async fn get_version(
_meilisearch: GuardedData<ActionPolicy<{ actions::VERSION }>, MeiliSearch>,
) -> HttpResponse {
let commit_sha = option_env!("VERGEN_GIT_SHA").unwrap_or("unknown"); let commit_sha = option_env!("VERGEN_GIT_SHA").unwrap_or("unknown");
let commit_date = option_env!("VERGEN_GIT_COMMIT_TIMESTAMP").unwrap_or("unknown"); let commit_date = option_env!("VERGEN_GIT_COMMIT_TIMESTAMP").unwrap_or("unknown");
@@ -158,108 +162,6 @@ struct KeysResponse {
public: Option<String>, public: Option<String>,
} }
pub async fn list_keys(meilisearch: GuardedData<Admin, ApiKeys>) -> HttpResponse {
let api_keys = (*meilisearch).clone();
HttpResponse::Ok().json(&KeysResponse {
private: api_keys.private,
public: api_keys.public,
})
}
pub async fn get_health() -> Result<HttpResponse, ResponseError> { pub async fn get_health() -> Result<HttpResponse, ResponseError> {
Ok(HttpResponse::Ok().json(serde_json::json!({ "status": "available" }))) Ok(HttpResponse::Ok().json(serde_json::json!({ "status": "available" })))
} }
#[cfg(test)]
mod test {
use super::*;
use crate::extractors::authentication::GuardedData;
/// A type implemented for a route that uses a authentication policy `Policy`.
///
/// This trait is used for regression testing of route authenticaton policies.
trait Is<Policy, Data, T> {}
macro_rules! impl_is_policy {
($($param:ident)*) => {
impl<Policy, Func, Data, $($param,)* Res> Is<Policy, Data, (($($param,)*), Res)> for Func
where Func: Fn(GuardedData<Policy, Data>, $($param,)*) -> Res {}
};
}
impl_is_policy! {}
impl_is_policy! {A}
impl_is_policy! {A B}
impl_is_policy! {A B C}
impl_is_policy! {A B C D}
impl_is_policy! {A B C D E}
/// Emits a compile error if a route doesn't have the correct authentication policy.
///
/// This works by trying to cast the route function into a Is<Policy, _> type, where Policy it
/// the authentication policy defined for the route.
macro_rules! test_auth_routes {
($($policy:ident => { $($route:expr,)*})*) => {
#[test]
fn test_auth() {
$($(let _: &dyn Is<$policy, _, _> = &$route;)*)*
}
};
}
test_auth_routes! {
Public => {
indexes::search::search_with_url_query,
indexes::search::search_with_post,
indexes::documents::get_document,
indexes::documents::get_all_documents,
}
Private => {
get_stats,
get_version,
indexes::create_index,
indexes::list_indexes,
indexes::get_index_stats,
indexes::delete_index,
indexes::update_index,
indexes::get_index,
dump::create_dump,
indexes::settings::filterable_attributes::get,
indexes::settings::displayed_attributes::get,
indexes::settings::searchable_attributes::get,
indexes::settings::stop_words::get,
indexes::settings::synonyms::get,
indexes::settings::distinct_attribute::get,
indexes::settings::filterable_attributes::update,
indexes::settings::displayed_attributes::update,
indexes::settings::searchable_attributes::update,
indexes::settings::stop_words::update,
indexes::settings::synonyms::update,
indexes::settings::distinct_attribute::update,
indexes::settings::filterable_attributes::delete,
indexes::settings::displayed_attributes::delete,
indexes::settings::searchable_attributes::delete,
indexes::settings::stop_words::delete,
indexes::settings::synonyms::delete,
indexes::settings::distinct_attribute::delete,
indexes::settings::delete_all,
indexes::settings::get_all,
indexes::settings::update_all,
indexes::documents::clear_all_documents,
indexes::documents::delete_documents,
indexes::documents::update_documents,
indexes::documents::add_documents,
indexes::documents::delete_document,
indexes::tasks::get_all_tasks_status,
indexes::tasks::get_task_status,
}
Admin => { list_keys, }
}
}

View File

@@ -14,7 +14,7 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
} }
async fn get_tasks( async fn get_tasks(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::TASKS_GET }>, MeiliSearch>,
req: HttpRequest, req: HttpRequest,
analytics: web::Data<dyn Analytics>, analytics: web::Data<dyn Analytics>,
) -> Result<HttpResponse, ResponseError> { ) -> Result<HttpResponse, ResponseError> {
@@ -36,7 +36,7 @@ async fn get_tasks(
} }
async fn get_task( async fn get_task(
meilisearch: GuardedData<Private, MeiliSearch>, meilisearch: GuardedData<ActionPolicy<{ actions::TASKS_GET }>, MeiliSearch>,
task_id: web::Path<TaskId>, task_id: web::Path<TaskId>,
req: HttpRequest, req: HttpRequest,
analytics: web::Data<dyn Analytics>, analytics: web::Data<dyn Analytics>,

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,412 @@
use crate::common::Server;
use maplit::hashmap;
use once_cell::sync::Lazy;
use serde_json::{json, Value};
use std::collections::{HashMap, HashSet};
static AUTHORIZATIONS: Lazy<HashMap<(&'static str, &'static str), &'static str>> =
Lazy::new(|| {
hashmap! {
("POST", "/indexes/products/search") => "search",
("GET", "/indexes/products/search") => "search",
("POST", "/indexes/products/documents") => "documents.add",
("GET", "/indexes/products/documents") => "documents.get",
("GET", "/indexes/products/documents/0") => "documents.get",
("DELETE", "/indexes/products/documents/0") => "documents.delete",
("GET", "/tasks") => "tasks.get",
("GET", "/indexes/products/tasks") => "tasks.get",
("GET", "/indexes/products/tasks/0") => "tasks.get",
("PUT", "/indexes/products/") => "indexes.update",
("GET", "/indexes/products/") => "indexes.get",
("DELETE", "/indexes/products/") => "indexes.delete",
("POST", "/indexes") => "indexes.add",
("GET", "/indexes") => "indexes.get",
("GET", "/indexes/products/settings") => "settings.get",
("GET", "/indexes/products/settings/displayed-attributes") => "settings.get",
("GET", "/indexes/products/settings/distinct-attribute") => "settings.get",
("GET", "/indexes/products/settings/filterable-attributes") => "settings.get",
("GET", "/indexes/products/settings/ranking-rules") => "settings.get",
("GET", "/indexes/products/settings/searchable-attributes") => "settings.get",
("GET", "/indexes/products/settings/sortable-attributes") => "settings.get",
("GET", "/indexes/products/settings/stop-words") => "settings.get",
("GET", "/indexes/products/settings/synonyms") => "settings.get",
("DELETE", "/indexes/products/settings") => "settings.update",
("POST", "/indexes/products/settings") => "settings.update",
("POST", "/indexes/products/settings/displayed-attributes") => "settings.update",
("POST", "/indexes/products/settings/distinct-attribute") => "settings.update",
("POST", "/indexes/products/settings/filterable-attributes") => "settings.update",
("POST", "/indexes/products/settings/ranking-rules") => "settings.update",
("POST", "/indexes/products/settings/searchable-attributes") => "settings.update",
("POST", "/indexes/products/settings/sortable-attributes") => "settings.update",
("POST", "/indexes/products/settings/stop-words") => "settings.update",
("POST", "/indexes/products/settings/synonyms") => "settings.update",
("GET", "/indexes/products/stats") => "stats.get",
("GET", "/stats") => "stats.get",
("POST", "/dumps") => "dumps.create",
("GET", "/dumps/0/status") => "dumps.get",
("GET", "/version") => "version",
}
});
static ALL_ACTIONS: Lazy<HashSet<&'static str>> =
Lazy::new(|| AUTHORIZATIONS.values().cloned().collect());
static INVALID_RESPONSE: Lazy<Value> = Lazy::new(|| {
json!({"message": "The provided API key is invalid.",
"code": "invalid_api_key",
"type": "auth",
"link": "https://docs.meilisearch.com/errors#invalid_api_key"
})
});
#[actix_rt::test]
async fn error_access_expired_key() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let content = json!({
"indexes": ["products"],
"actions": ALL_ACTIONS.clone(),
"expiresAt": "2020-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
for (method, route) in AUTHORIZATIONS.keys() {
let (response, code) = server.dummy_request(method, route).await;
assert_eq!(response, INVALID_RESPONSE.clone());
assert_eq!(code, 403);
}
}
#[actix_rt::test]
async fn error_access_unauthorized_index() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let content = json!({
"indexes": ["sales"],
"actions": ALL_ACTIONS.clone(),
"expiresAt": "2050-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
for (method, route) in AUTHORIZATIONS
.keys()
// filter `products` index routes
.filter(|(_, route)| route.starts_with("/indexes/products"))
{
let (response, code) = server.dummy_request(method, route).await;
assert_eq!(response, INVALID_RESPONSE.clone());
assert_eq!(code, 403);
}
}
#[actix_rt::test]
async fn error_access_unauthorized_action() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let content = json!({
"indexes": ["products"],
"actions": [],
"expiresAt": "2050-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
for ((method, route), action) in AUTHORIZATIONS.iter() {
server.use_api_key("MASTER_KEY");
// Patch API key letting all rights but the needed one.
let content = json!({
"actions": ALL_ACTIONS.iter().cloned().filter(|a| a != action).collect::<Vec<_>>(),
});
let (_, code) = server.patch_api_key(&key, content).await;
assert_eq!(code, 200);
server.use_api_key(&key);
let (response, code) = server.dummy_request(method, route).await;
assert_eq!(response, INVALID_RESPONSE.clone());
assert_eq!(code, 403);
}
}
#[actix_rt::test]
async fn access_authorized_restricted_index() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let content = json!({
"indexes": ["products"],
"actions": [],
"expiresAt": "2050-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
for ((method, route), action) in AUTHORIZATIONS.iter() {
// Patch API key letting only the needed action.
let content = json!({
"actions": [action],
});
server.use_api_key("MASTER_KEY");
let (_, code) = server.patch_api_key(&key, content).await;
assert_eq!(code, 200);
server.use_api_key(&key);
let (response, code) = server.dummy_request(method, route).await;
assert_ne!(response, INVALID_RESPONSE.clone());
assert_ne!(code, 403);
// Patch API key using action all action.
let content = json!({
"actions": ["*"],
});
server.use_api_key("MASTER_KEY");
let (_, code) = server.patch_api_key(&key, content).await;
assert_eq!(code, 200);
server.use_api_key(&key);
let (response, code) = server.dummy_request(method, route).await;
assert_ne!(response, INVALID_RESPONSE.clone());
assert_ne!(code, 403);
}
}
#[actix_rt::test]
async fn access_authorized_no_index_restriction() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let content = json!({
"indexes": ["*"],
"actions": [],
"expiresAt": "2050-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
for ((method, route), action) in AUTHORIZATIONS.iter() {
server.use_api_key("MASTER_KEY");
// Patch API key letting only the needed action.
let content = json!({
"actions": [action],
});
let (_, code) = server.patch_api_key(&key, content).await;
assert_eq!(code, 200);
server.use_api_key(&key);
let (response, code) = server.dummy_request(method, route).await;
assert_ne!(response, INVALID_RESPONSE.clone());
assert_ne!(code, 403);
// Patch API key using action all action.
let content = json!({
"actions": ["*"],
});
server.use_api_key("MASTER_KEY");
let (_, code) = server.patch_api_key(&key, content).await;
assert_eq!(code, 200);
server.use_api_key(&key);
let (response, code) = server.dummy_request(method, route).await;
assert_ne!(response, INVALID_RESPONSE.clone());
assert_ne!(code, 403);
}
}
#[actix_rt::test]
async fn access_authorized_stats_restricted_index() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
// create index `test`
let index = server.index("test");
let (_, code) = index.create(Some("id")).await;
assert_eq!(code, 202);
// create index `products`
let index = server.index("products");
let (_, code) = index.create(Some("product_id")).await;
assert_eq!(code, 202);
index.wait_task(0).await;
// create key with access on `products` index only.
let content = json!({
"indexes": ["products"],
"actions": ["stats.get"],
"expiresAt": "2050-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
// use created key.
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
let (response, code) = server.stats().await;
assert_eq!(code, 200);
// key should have access on `products` index.
assert!(response["indexes"].get("products").is_some());
// key should not have access on `test` index.
assert!(response["indexes"].get("test").is_none());
}
#[actix_rt::test]
async fn access_authorized_stats_no_index_restriction() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
// create index `test`
let index = server.index("test");
let (_, code) = index.create(Some("id")).await;
assert_eq!(code, 202);
// create index `products`
let index = server.index("products");
let (_, code) = index.create(Some("product_id")).await;
assert_eq!(code, 202);
index.wait_task(0).await;
// create key with access on all indexes.
let content = json!({
"indexes": ["*"],
"actions": ["stats.get"],
"expiresAt": "2050-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
// use created key.
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
let (response, code) = server.stats().await;
assert_eq!(code, 200);
// key should have access on `products` index.
assert!(response["indexes"].get("products").is_some());
// key should have access on `test` index.
assert!(response["indexes"].get("test").is_some());
}
#[actix_rt::test]
async fn list_authorized_indexes_restricted_index() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
// create index `test`
let index = server.index("test");
let (_, code) = index.create(Some("id")).await;
assert_eq!(code, 202);
// create index `products`
let index = server.index("products");
let (_, code) = index.create(Some("product_id")).await;
assert_eq!(code, 202);
index.wait_task(0).await;
// create key with access on `products` index only.
let content = json!({
"indexes": ["products"],
"actions": ["indexes.get"],
"expiresAt": "2050-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
// use created key.
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
let (response, code) = server.list_indexes().await;
assert_eq!(code, 200);
let response = response.as_array().unwrap();
// key should have access on `products` index.
assert!(response.iter().any(|index| index["uid"] == "products"));
// key should not have access on `test` index.
assert!(!response.iter().any(|index| index["uid"] == "test"));
}
#[actix_rt::test]
async fn list_authorized_indexes_no_index_restriction() {
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
// create index `test`
let index = server.index("test");
let (_, code) = index.create(Some("id")).await;
assert_eq!(code, 202);
// create index `products`
let index = server.index("products");
let (_, code) = index.create(Some("product_id")).await;
assert_eq!(code, 202);
index.wait_task(0).await;
// create key with access on all indexes.
let content = json!({
"indexes": ["*"],
"actions": ["indexes.get"],
"expiresAt": "2050-11-13T00:00:00Z"
});
let (response, code) = server.add_api_key(content).await;
assert_eq!(code, 201);
assert!(response["key"].is_string());
// use created key.
let key = response["key"].as_str().unwrap();
server.use_api_key(&key);
let (response, code) = server.list_indexes().await;
assert_eq!(code, 200);
let response = response.as_array().unwrap();
// key should have access on `products` index.
assert!(response.iter().any(|index| index["uid"] == "products"));
// key should have access on `test` index.
assert!(response.iter().any(|index| index["uid"] == "test"));
}

View File

@@ -0,0 +1,71 @@
mod api_keys;
mod authorization;
mod payload;
use crate::common::server::default_settings;
use crate::common::server::TEST_TEMP_DIR;
use crate::common::Server;
use actix_web::http::StatusCode;
use serde_json::{json, Value};
use tempfile::TempDir;
impl Server {
pub async fn new_auth() -> Self {
let dir = TempDir::new().unwrap();
if cfg!(windows) {
std::env::set_var("TMP", TEST_TEMP_DIR.path());
} else {
std::env::set_var("TMPDIR", TEST_TEMP_DIR.path());
}
let mut options = default_settings(dir.path());
options.master_key = Some("MASTER_KEY".to_string());
Self::new_with_options(options).await
}
pub fn use_api_key(&mut self, api_key: impl AsRef<str>) {
self.service.api_key = Some(api_key.as_ref().to_string());
}
pub async fn add_api_key(&self, content: Value) -> (Value, StatusCode) {
let url = "/keys";
self.service.post(url, content).await
}
pub async fn get_api_key(&self, key: impl AsRef<str>) -> (Value, StatusCode) {
let url = format!("/keys/{}", key.as_ref());
self.service.get(url).await
}
pub async fn patch_api_key(&self, key: impl AsRef<str>, content: Value) -> (Value, StatusCode) {
let url = format!("/keys/{}", key.as_ref());
self.service.patch(url, content).await
}
pub async fn list_api_keys(&self) -> (Value, StatusCode) {
let url = "/keys";
self.service.get(url).await
}
pub async fn delete_api_key(&self, key: impl AsRef<str>) -> (Value, StatusCode) {
let url = format!("/keys/{}", key.as_ref());
self.service.delete(url).await
}
pub async fn dummy_request(
&self,
method: impl AsRef<str>,
url: impl AsRef<str>,
) -> (Value, StatusCode) {
match method.as_ref() {
"POST" => self.service.post(url, json!({})).await,
"PUT" => self.service.put(url, json!({})).await,
"PATCH" => self.service.patch(url, json!({})).await,
"GET" => self.service.get(url).await,
"DELETE" => self.service.delete(url).await,
_ => unreachable!(),
}
}
}

View File

@@ -0,0 +1,340 @@
use crate::common::Server;
use actix_web::test;
use meilisearch_http::{analytics, create_app};
use serde_json::{json, Value};
#[actix_rt::test]
async fn error_api_key_bad_content_types() {
let content = json!({
"indexes": ["products"],
"actions": [
"documents.add"
],
"expiresAt": "2050-11-13T00:00:00Z"
});
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let app = test::init_service(create_app!(
&server.service.meilisearch,
&server.service.auth,
true,
&server.service.options,
analytics::MockAnalytics::new(&server.service.options).0
))
.await;
// post
let req = test::TestRequest::post()
.uri("/keys")
.set_payload(content.to_string())
.insert_header(("content-type", "text/plain"))
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 415);
assert_eq!(
response["message"],
json!(
r#"The Content-Type `text/plain` is invalid. Accepted values for the Content-Type header are: `application/json`"#
)
);
assert_eq!(response["code"], "invalid_content_type");
assert_eq!(response["type"], "invalid_request");
assert_eq!(
response["link"],
"https://docs.meilisearch.com/errors#invalid_content_type"
);
// patch
let req = test::TestRequest::patch()
.uri("/keys/d0552b41536279a0ad88bd595327b96f01176a60c2243e906c52ac02375f9bc4")
.set_payload(content.to_string())
.insert_header(("content-type", "text/plain"))
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 415);
assert_eq!(
response["message"],
json!(
r#"The Content-Type `text/plain` is invalid. Accepted values for the Content-Type header are: `application/json`"#
)
);
assert_eq!(response["code"], "invalid_content_type");
assert_eq!(response["type"], "invalid_request");
assert_eq!(
response["link"],
"https://docs.meilisearch.com/errors#invalid_content_type"
);
}
#[actix_rt::test]
async fn error_api_key_empty_content_types() {
let content = json!({
"indexes": ["products"],
"actions": [
"documents.add"
],
"expiresAt": "2050-11-13T00:00:00Z"
});
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let app = test::init_service(create_app!(
&server.service.meilisearch,
&server.service.auth,
true,
&server.service.options,
analytics::MockAnalytics::new(&server.service.options).0
))
.await;
// post
let req = test::TestRequest::post()
.uri("/keys")
.set_payload(content.to_string())
.insert_header(("content-type", ""))
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 415);
assert_eq!(
response["message"],
json!(
r#"The Content-Type `` is invalid. Accepted values for the Content-Type header are: `application/json`"#
)
);
assert_eq!(response["code"], "invalid_content_type");
assert_eq!(response["type"], "invalid_request");
assert_eq!(
response["link"],
"https://docs.meilisearch.com/errors#invalid_content_type"
);
// patch
let req = test::TestRequest::patch()
.uri("/keys/d0552b41536279a0ad88bd595327b96f01176a60c2243e906c52ac02375f9bc4")
.set_payload(content.to_string())
.insert_header(("content-type", ""))
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 415);
assert_eq!(
response["message"],
json!(
r#"The Content-Type `` is invalid. Accepted values for the Content-Type header are: `application/json`"#
)
);
assert_eq!(response["code"], "invalid_content_type");
assert_eq!(response["type"], "invalid_request");
assert_eq!(
response["link"],
"https://docs.meilisearch.com/errors#invalid_content_type"
);
}
#[actix_rt::test]
async fn error_api_key_missing_content_types() {
let content = json!({
"indexes": ["products"],
"actions": [
"documents.add"
],
"expiresAt": "2050-11-13T00:00:00Z"
});
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let app = test::init_service(create_app!(
&server.service.meilisearch,
&server.service.auth,
true,
&server.service.options,
analytics::MockAnalytics::new(&server.service.options).0
))
.await;
// post
let req = test::TestRequest::post()
.uri("/keys")
.set_payload(content.to_string())
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 415);
assert_eq!(
response["message"],
json!(
r#"A Content-Type header is missing. Accepted values for the Content-Type header are: `application/json`"#
)
);
assert_eq!(response["code"], "missing_content_type");
assert_eq!(response["type"], "invalid_request");
assert_eq!(
response["link"],
"https://docs.meilisearch.com/errors#missing_content_type"
);
// patch
let req = test::TestRequest::patch()
.uri("/keys/d0552b41536279a0ad88bd595327b96f01176a60c2243e906c52ac02375f9bc4")
.set_payload(content.to_string())
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 415);
assert_eq!(
response["message"],
json!(
r#"A Content-Type header is missing. Accepted values for the Content-Type header are: `application/json`"#
)
);
assert_eq!(response["code"], "missing_content_type");
assert_eq!(response["type"], "invalid_request");
assert_eq!(
response["link"],
"https://docs.meilisearch.com/errors#missing_content_type"
);
}
#[actix_rt::test]
async fn error_api_key_empty_payload() {
let content = "";
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let app = test::init_service(create_app!(
&server.service.meilisearch,
&server.service.auth,
true,
&server.service.options,
analytics::MockAnalytics::new(&server.service.options).0
))
.await;
// post
let req = test::TestRequest::post()
.uri("/keys")
.set_payload(content)
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.insert_header(("content-type", "application/json"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 400);
assert_eq!(response["code"], json!("missing_payload"));
assert_eq!(response["type"], json!("invalid_request"));
assert_eq!(
response["link"],
json!("https://docs.meilisearch.com/errors#missing_payload")
);
assert_eq!(response["message"], json!(r#"A json payload is missing."#));
// patch
let req = test::TestRequest::patch()
.uri("/keys/d0552b41536279a0ad88bd595327b96f01176a60c2243e906c52ac02375f9bc4")
.set_payload(content)
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.insert_header(("content-type", "application/json"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 400);
assert_eq!(response["code"], json!("missing_payload"));
assert_eq!(response["type"], json!("invalid_request"));
assert_eq!(
response["link"],
json!("https://docs.meilisearch.com/errors#missing_payload")
);
assert_eq!(response["message"], json!(r#"A json payload is missing."#));
}
#[actix_rt::test]
async fn error_api_key_malformed_payload() {
let content = r#"{"malormed": "payload""#;
let mut server = Server::new_auth().await;
server.use_api_key("MASTER_KEY");
let app = test::init_service(create_app!(
&server.service.meilisearch,
&server.service.auth,
true,
&server.service.options,
analytics::MockAnalytics::new(&server.service.options).0
))
.await;
// post
let req = test::TestRequest::post()
.uri("/keys")
.set_payload(content)
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.insert_header(("content-type", "application/json"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 400);
assert_eq!(response["code"], json!("malformed_payload"));
assert_eq!(response["type"], json!("invalid_request"));
assert_eq!(
response["link"],
json!("https://docs.meilisearch.com/errors#malformed_payload")
);
assert_eq!(
response["message"],
json!(
r#"The json payload provided is malformed. `EOF while parsing an object at line 1 column 22`."#
)
);
// patch
let req = test::TestRequest::patch()
.uri("/keys/d0552b41536279a0ad88bd595327b96f01176a60c2243e906c52ac02375f9bc4")
.set_payload(content)
.insert_header(("Authorization", "Bearer MASTER_KEY"))
.insert_header(("content-type", "application/json"))
.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response: Value = serde_json::from_slice(&body).unwrap_or_default();
assert_eq!(status_code, 400);
assert_eq!(response["code"], json!("malformed_payload"));
assert_eq!(response["type"], json!("invalid_request"));
assert_eq!(
response["link"],
json!("https://docs.meilisearch.com/errors#malformed_payload")
);
assert_eq!(
response["message"],
json!(
r#"The json payload provided is malformed. `EOF while parsing an object at line 1 column 22`."#
)
);
}

View File

@@ -2,6 +2,7 @@ use std::path::Path;
use actix_web::http::StatusCode; use actix_web::http::StatusCode;
use byte_unit::{Byte, ByteUnit}; use byte_unit::{Byte, ByteUnit};
use meilisearch_auth::AuthController;
use meilisearch_http::setup_meilisearch; use meilisearch_http::setup_meilisearch;
use meilisearch_lib::options::{IndexerOpts, MaxMemory}; use meilisearch_lib::options::{IndexerOpts, MaxMemory};
use once_cell::sync::Lazy; use once_cell::sync::Lazy;
@@ -19,7 +20,7 @@ pub struct Server {
_dir: Option<TempDir>, _dir: Option<TempDir>,
} }
static TEST_TEMP_DIR: Lazy<TempDir> = Lazy::new(|| TempDir::new().unwrap()); pub static TEST_TEMP_DIR: Lazy<TempDir> = Lazy::new(|| TempDir::new().unwrap());
impl Server { impl Server {
pub async fn new() -> Self { pub async fn new() -> Self {
@@ -34,9 +35,12 @@ impl Server {
let options = default_settings(dir.path()); let options = default_settings(dir.path());
let meilisearch = setup_meilisearch(&options).unwrap(); let meilisearch = setup_meilisearch(&options).unwrap();
let auth = AuthController::new(&options.db_path, &options.master_key).unwrap();
let service = Service { let service = Service {
meilisearch, meilisearch,
auth,
options, options,
api_key: None,
}; };
Server { Server {
@@ -47,9 +51,12 @@ impl Server {
pub async fn new_with_options(options: Opt) -> Self { pub async fn new_with_options(options: Opt) -> Self {
let meilisearch = setup_meilisearch(&options).unwrap(); let meilisearch = setup_meilisearch(&options).unwrap();
let auth = AuthController::new(&options.db_path, &options.master_key).unwrap();
let service = Service { let service = Service {
meilisearch, meilisearch,
auth,
options, options,
api_key: None,
}; };
Server { Server {

View File

@@ -1,4 +1,5 @@
use actix_web::{http::StatusCode, test}; use actix_web::{http::StatusCode, test};
use meilisearch_auth::AuthController;
use meilisearch_lib::MeiliSearch; use meilisearch_lib::MeiliSearch;
use serde_json::Value; use serde_json::Value;
@@ -6,23 +7,27 @@ use meilisearch_http::{analytics, create_app, Opt};
pub struct Service { pub struct Service {
pub meilisearch: MeiliSearch, pub meilisearch: MeiliSearch,
pub auth: AuthController,
pub options: Opt, pub options: Opt,
pub api_key: Option<String>,
} }
impl Service { impl Service {
pub async fn post(&self, url: impl AsRef<str>, body: Value) -> (Value, StatusCode) { pub async fn post(&self, url: impl AsRef<str>, body: Value) -> (Value, StatusCode) {
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&self.meilisearch, &self.meilisearch,
&self.auth,
true, true,
&self.options, &self.options,
analytics::MockAnalytics::new(&self.options).0 analytics::MockAnalytics::new(&self.options).0
)) ))
.await; .await;
let req = test::TestRequest::post() let mut req = test::TestRequest::post().uri(url.as_ref()).set_json(&body);
.uri(url.as_ref()) if let Some(api_key) = &self.api_key {
.set_json(&body) req = req.insert_header(("Authorization", ["Bearer ", api_key].concat()));
.to_request(); }
let req = req.to_request();
let res = test::call_service(&app, req).await; let res = test::call_service(&app, req).await;
let status_code = res.status(); let status_code = res.status();
@@ -39,17 +44,21 @@ impl Service {
) -> (Value, StatusCode) { ) -> (Value, StatusCode) {
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&self.meilisearch, &self.meilisearch,
&self.auth,
true, true,
&self.options, &self.options,
analytics::MockAnalytics::new(&self.options).0 analytics::MockAnalytics::new(&self.options).0
)) ))
.await; .await;
let req = test::TestRequest::post() let mut req = test::TestRequest::post()
.uri(url.as_ref()) .uri(url.as_ref())
.set_payload(body.as_ref().to_string()) .set_payload(body.as_ref().to_string())
.insert_header(("content-type", "application/json")) .insert_header(("content-type", "application/json"));
.to_request(); if let Some(api_key) = &self.api_key {
req = req.insert_header(("Authorization", ["Bearer ", api_key].concat()));
}
let req = req.to_request();
let res = test::call_service(&app, req).await; let res = test::call_service(&app, req).await;
let status_code = res.status(); let status_code = res.status();
@@ -61,13 +70,18 @@ impl Service {
pub async fn get(&self, url: impl AsRef<str>) -> (Value, StatusCode) { pub async fn get(&self, url: impl AsRef<str>) -> (Value, StatusCode) {
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&self.meilisearch, &self.meilisearch,
&self.auth,
true, true,
&self.options, &self.options,
analytics::MockAnalytics::new(&self.options).0 analytics::MockAnalytics::new(&self.options).0
)) ))
.await; .await;
let req = test::TestRequest::get().uri(url.as_ref()).to_request(); let mut req = test::TestRequest::get().uri(url.as_ref());
if let Some(api_key) = &self.api_key {
req = req.insert_header(("Authorization", ["Bearer ", api_key].concat()));
}
let req = req.to_request();
let res = test::call_service(&app, req).await; let res = test::call_service(&app, req).await;
let status_code = res.status(); let status_code = res.status();
@@ -79,16 +93,41 @@ impl Service {
pub async fn put(&self, url: impl AsRef<str>, body: Value) -> (Value, StatusCode) { pub async fn put(&self, url: impl AsRef<str>, body: Value) -> (Value, StatusCode) {
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&self.meilisearch, &self.meilisearch,
&self.auth,
true, true,
&self.options, &self.options,
analytics::MockAnalytics::new(&self.options).0 analytics::MockAnalytics::new(&self.options).0
)) ))
.await; .await;
let req = test::TestRequest::put() let mut req = test::TestRequest::put().uri(url.as_ref()).set_json(&body);
.uri(url.as_ref()) if let Some(api_key) = &self.api_key {
.set_json(&body) req = req.insert_header(("Authorization", ["Bearer ", api_key].concat()));
.to_request(); }
let req = req.to_request();
let res = test::call_service(&app, req).await;
let status_code = res.status();
let body = test::read_body(res).await;
let response = serde_json::from_slice(&body).unwrap_or_default();
(response, status_code)
}
pub async fn patch(&self, url: impl AsRef<str>, body: Value) -> (Value, StatusCode) {
let app = test::init_service(create_app!(
&self.meilisearch,
&self.auth,
true,
&self.options,
analytics::MockAnalytics::new(&self.options).0
))
.await;
let mut req = test::TestRequest::patch().uri(url.as_ref()).set_json(&body);
if let Some(api_key) = &self.api_key {
req = req.insert_header(("Authorization", ["Bearer ", api_key].concat()));
}
let req = req.to_request();
let res = test::call_service(&app, req).await; let res = test::call_service(&app, req).await;
let status_code = res.status(); let status_code = res.status();
@@ -100,13 +139,18 @@ impl Service {
pub async fn delete(&self, url: impl AsRef<str>) -> (Value, StatusCode) { pub async fn delete(&self, url: impl AsRef<str>) -> (Value, StatusCode) {
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&self.meilisearch, &self.meilisearch,
&self.auth,
true, true,
&self.options, &self.options,
analytics::MockAnalytics::new(&self.options).0 analytics::MockAnalytics::new(&self.options).0
)) ))
.await; .await;
let req = test::TestRequest::delete().uri(url.as_ref()).to_request(); let mut req = test::TestRequest::delete().uri(url.as_ref());
if let Some(api_key) = &self.api_key {
req = req.insert_header(("Authorization", ["Bearer ", api_key].concat()));
}
let req = req.to_request();
let res = test::call_service(&app, req).await; let res = test::call_service(&app, req).await;
let status_code = res.status(); let status_code = res.status();

View File

@@ -39,6 +39,7 @@ async fn error_json_bad_content_type() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -118,6 +119,7 @@ async fn extract_actual_content_type() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0

View File

@@ -18,6 +18,7 @@ async fn add_documents_test_json_content_types() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -63,6 +64,7 @@ async fn error_add_documents_test_bad_content_types() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -130,6 +132,7 @@ async fn error_add_documents_test_no_content_type() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -189,6 +192,7 @@ async fn error_add_malformed_csv_documents() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -250,6 +254,7 @@ async fn error_add_malformed_json_documents() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -311,6 +316,7 @@ async fn error_add_malformed_ndjson_documents() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -372,6 +378,7 @@ async fn error_add_missing_payload_csv_documents() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -423,6 +430,7 @@ async fn error_add_missing_payload_json_documents() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0
@@ -474,6 +482,7 @@ async fn error_add_missing_payload_ndjson_documents() {
let server = Server::new().await; let server = Server::new().await;
let app = test::init_service(create_app!( let app = test::init_service(create_app!(
&server.service.meilisearch, &server.service.meilisearch,
&server.service.auth,
true, true,
&server.service.options, &server.service.options,
analytics::MockAnalytics::new(&server.service.options).0 analytics::MockAnalytics::new(&server.service.options).0

View File

@@ -1,3 +1,4 @@
mod auth;
mod common; mod common;
mod dashboard; mod dashboard;
mod documents; mod documents;

View File

@@ -1,7 +1,7 @@
use std::fmt; use std::fmt;
use std::io::{self, BufRead, BufReader, BufWriter, Cursor, Read, Seek, Write}; use std::io::{self, BufRead, BufReader, BufWriter, Cursor, Read, Seek, Write};
use meilisearch_error::{Code, ErrorCode}; use meilisearch_error::{internal_error, Code, ErrorCode};
use milli::documents::DocumentBatchBuilder; use milli::documents::DocumentBatchBuilder;
type Result<T> = std::result::Result<T, DocumentFormatError>; type Result<T> = std::result::Result<T, DocumentFormatError>;

View File

@@ -4,18 +4,6 @@ use std::fmt;
use meilisearch_error::{Code, ErrorCode}; use meilisearch_error::{Code, ErrorCode};
use milli::UserError; use milli::UserError;
macro_rules! internal_error {
($target:ty : $($other:path), *) => {
$(
impl From<$other> for $target {
fn from(other: $other) -> Self {
Self::Internal(Box::new(other))
}
}
)*
}
}
#[derive(Debug)] #[derive(Debug)]
pub struct MilliError<'a>(pub &'a milli::Error); pub struct MilliError<'a>(pub &'a milli::Error);

View File

@@ -1,6 +1,6 @@
use std::error::Error; use std::error::Error;
use meilisearch_error::{Code, ErrorCode}; use meilisearch_error::{internal_error, Code, ErrorCode};
use serde_json::Value; use serde_json::Value;
use crate::error::MilliError; use crate::error::MilliError;

View File

@@ -1,4 +1,4 @@
use meilisearch_error::{Code, ErrorCode}; use meilisearch_error::{internal_error, Code, ErrorCode};
use crate::{index_resolver::error::IndexResolverError, tasks::error::TaskError}; use crate::{index_resolver::error::IndexResolverError, tasks::error::TaskError};
@@ -16,20 +16,8 @@ pub enum DumpActorError {
IndexResolver(#[from] IndexResolverError), IndexResolver(#[from] IndexResolverError),
} }
macro_rules! internal_error {
($($other:path), *) => {
$(
impl From<$other> for DumpActorError {
fn from(other: $other) -> Self {
Self::Internal(Box::new(other))
}
}
)*
}
}
internal_error!( internal_error!(
heed::Error, DumpActorError: heed::Error,
std::io::Error, std::io::Error,
tokio::task::JoinError, tokio::task::JoinError,
tokio::sync::oneshot::error::RecvError, tokio::sync::oneshot::error::RecvError,

View File

@@ -1,7 +1,7 @@
use std::error::Error; use std::error::Error;
use meilisearch_error::Code; use meilisearch_error::Code;
use meilisearch_error::ErrorCode; use meilisearch_error::{internal_error, ErrorCode};
use tokio::task::JoinError; use tokio::task::JoinError;
use super::DocumentAdditionFormat; use super::DocumentAdditionFormat;

View File

@@ -525,13 +525,20 @@ where
Ok(stats) Ok(stats)
} }
pub async fn get_all_stats(&self) -> Result<Stats> { pub async fn get_all_stats(&self, index_filter: &Option<Vec<String>>) -> Result<Stats> {
let mut last_task: Option<DateTime<_>> = None; let mut last_task: Option<DateTime<_>> = None;
let mut indexes = BTreeMap::new(); let mut indexes = BTreeMap::new();
let mut database_size = 0; let mut database_size = 0;
let processing_task = self.task_store.get_processing_task().await?; let processing_task = self.task_store.get_processing_task().await?;
for (index_uid, index) in self.index_resolver.list().await? { for (index_uid, index) in self.index_resolver.list().await? {
if index_filter
.as_ref()
.map_or(false, |filter| !filter.contains(&index_uid))
{
continue;
}
let (mut stats, meta) = let (mut stats, meta) =
spawn_blocking::<_, Result<(IndexStats, IndexMeta)>>(move || { spawn_blocking::<_, Result<(IndexStats, IndexMeta)>>(move || {
Ok((index.stats()?, index.meta()?)) Ok((index.stats()?, index.meta()?))

View File

@@ -1,7 +1,7 @@
use std::error::Error; use std::error::Error;
use std::fmt; use std::fmt;
use meilisearch_error::{Code, ErrorCode}; use meilisearch_error::{internal_error, Code, ErrorCode};
use crate::{ use crate::{
document_formats::DocumentFormatError, document_formats::DocumentFormatError,

View File

@@ -1,6 +1,6 @@
use std::fmt; use std::fmt;
use meilisearch_error::{Code, ErrorCode}; use meilisearch_error::{internal_error, Code, ErrorCode};
use tokio::sync::mpsc::error::SendError as MpscSendError; use tokio::sync::mpsc::error::SendError as MpscSendError;
use tokio::sync::oneshot::error::RecvError as OneshotRecvError; use tokio::sync::oneshot::error::RecvError as OneshotRecvError;
use uuid::Uuid; use uuid::Uuid;

View File

@@ -1,4 +1,4 @@
use meilisearch_error::{Code, ErrorCode}; use meilisearch_error::{internal_error, Code, ErrorCode};
use tokio::task::JoinError; use tokio::task::JoinError;
use crate::update_file_store::UpdateFileStoreError; use crate::update_file_store::UpdateFileStoreError;