mirror of
https://github.com/meilisearch/meilisearch.git
synced 2025-09-06 04:36:32 +00:00
Merge branch 'main' into fragment-filters
This commit is contained in:
@ -85,7 +85,7 @@ impl RoFeatures {
|
||||
Ok(())
|
||||
} else {
|
||||
Err(FeatureNotEnabledError {
|
||||
disabled_action: "Using `CONTAINS` or `STARTS WITH` in a filter",
|
||||
disabled_action: "Using `CONTAINS` in a filter",
|
||||
feature: "contains filter",
|
||||
issue_link: "https://github.com/orgs/meilisearch/discussions/763",
|
||||
}
|
||||
@ -182,6 +182,7 @@ impl FeatureData {
|
||||
..persisted_features
|
||||
}));
|
||||
|
||||
// Once this is stabilized, network should be stored along with webhooks in index-scheduler's persisted database
|
||||
let network_db = runtime_features_db.remap_data_type::<SerdeJson<Network>>();
|
||||
let network: Network = network_db.get(wtxn, db_keys::NETWORK)?.unwrap_or_default();
|
||||
|
||||
|
@ -26,11 +26,11 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String {
|
||||
version,
|
||||
queue,
|
||||
scheduler,
|
||||
persisted,
|
||||
|
||||
index_mapper,
|
||||
features: _,
|
||||
webhook_url: _,
|
||||
webhook_authorization_header: _,
|
||||
webhooks: _,
|
||||
test_breakpoint_sdr: _,
|
||||
planned_failures: _,
|
||||
run_loop_iteration: _,
|
||||
@ -62,6 +62,13 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String {
|
||||
}
|
||||
snap.push_str("\n----------------------------------------------------------------------\n");
|
||||
|
||||
let persisted_db_snapshot = snapshot_persisted_db(&rtxn, persisted);
|
||||
if !persisted_db_snapshot.is_empty() {
|
||||
snap.push_str("### Persisted:\n");
|
||||
snap.push_str(&persisted_db_snapshot);
|
||||
snap.push_str("----------------------------------------------------------------------\n");
|
||||
}
|
||||
|
||||
snap.push_str("### All Tasks:\n");
|
||||
snap.push_str(&snapshot_all_tasks(&rtxn, queue.tasks.all_tasks));
|
||||
snap.push_str("----------------------------------------------------------------------\n");
|
||||
@ -200,6 +207,16 @@ pub fn snapshot_date_db(rtxn: &RoTxn, db: Database<BEI128, CboRoaringBitmapCodec
|
||||
snap
|
||||
}
|
||||
|
||||
pub fn snapshot_persisted_db(rtxn: &RoTxn, db: &Database<Str, Str>) -> String {
|
||||
let mut snap = String::new();
|
||||
let iter = db.iter(rtxn).unwrap();
|
||||
for next in iter {
|
||||
let (key, value) = next.unwrap();
|
||||
snap.push_str(&format!("{key}: {value}\n"));
|
||||
}
|
||||
snap
|
||||
}
|
||||
|
||||
pub fn snapshot_task(task: &Task) -> String {
|
||||
let mut snap = String::new();
|
||||
let Task {
|
||||
@ -311,6 +328,7 @@ pub fn snapshot_status(
|
||||
}
|
||||
snap
|
||||
}
|
||||
|
||||
pub fn snapshot_kind(rtxn: &RoTxn, db: Database<SerdeBincode<Kind>, RoaringBitmapCodec>) -> String {
|
||||
let mut snap = String::new();
|
||||
let iter = db.iter(rtxn).unwrap();
|
||||
@ -331,6 +349,7 @@ pub fn snapshot_index_tasks(rtxn: &RoTxn, db: Database<Str, RoaringBitmapCodec>)
|
||||
}
|
||||
snap
|
||||
}
|
||||
|
||||
pub fn snapshot_canceled_by(rtxn: &RoTxn, db: Database<BEU32, RoaringBitmapCodec>) -> String {
|
||||
let mut snap = String::new();
|
||||
let iter = db.iter(rtxn).unwrap();
|
||||
|
@ -65,13 +65,16 @@ use meilisearch_types::milli::vector::{
|
||||
use meilisearch_types::milli::{self, Index};
|
||||
use meilisearch_types::task_view::TaskView;
|
||||
use meilisearch_types::tasks::{KindWithContent, Task};
|
||||
use meilisearch_types::webhooks::{Webhook, WebhooksDumpView, WebhooksView};
|
||||
use milli::vector::db::IndexEmbeddingConfig;
|
||||
use processing::ProcessingTasks;
|
||||
pub use queue::Query;
|
||||
use queue::Queue;
|
||||
use roaring::RoaringBitmap;
|
||||
use scheduler::Scheduler;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use time::OffsetDateTime;
|
||||
use uuid::Uuid;
|
||||
use versioning::Versioning;
|
||||
|
||||
use crate::index_mapper::IndexMapper;
|
||||
@ -80,7 +83,15 @@ use crate::utils::clamp_to_page_size;
|
||||
pub(crate) type BEI128 = I128<BE>;
|
||||
|
||||
const TASK_SCHEDULER_SIZE_THRESHOLD_PERCENT_INT: u64 = 40;
|
||||
const CHAT_SETTINGS_DB_NAME: &str = "chat-settings";
|
||||
|
||||
mod db_name {
|
||||
pub const CHAT_SETTINGS: &str = "chat-settings";
|
||||
pub const PERSISTED: &str = "persisted";
|
||||
}
|
||||
|
||||
mod db_keys {
|
||||
pub const WEBHOOKS: &str = "webhooks";
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct IndexSchedulerOptions {
|
||||
@ -98,10 +109,10 @@ pub struct IndexSchedulerOptions {
|
||||
pub snapshots_path: PathBuf,
|
||||
/// The path to the folder containing the dumps.
|
||||
pub dumps_path: PathBuf,
|
||||
/// The URL on which we must send the tasks statuses
|
||||
pub webhook_url: Option<String>,
|
||||
/// The value we will send into the Authorization HTTP header on the webhook URL
|
||||
pub webhook_authorization_header: Option<String>,
|
||||
/// The webhook url that was set by the CLI.
|
||||
pub cli_webhook_url: Option<String>,
|
||||
/// The Authorization header to send to the webhook URL that was set by the CLI.
|
||||
pub cli_webhook_authorization: Option<String>,
|
||||
/// The maximum size, in bytes, of the task index.
|
||||
pub task_db_size: usize,
|
||||
/// The size, in bytes, with which a meilisearch index is opened the first time of each meilisearch index.
|
||||
@ -171,10 +182,11 @@ pub struct IndexScheduler {
|
||||
/// Whether we should use the old document indexer or the new one.
|
||||
pub(crate) experimental_no_edition_2024_for_dumps: bool,
|
||||
|
||||
/// The webhook url we should send tasks to after processing every batches.
|
||||
pub(crate) webhook_url: Option<String>,
|
||||
/// The Authorization header to send to the webhook URL.
|
||||
pub(crate) webhook_authorization_header: Option<String>,
|
||||
/// A database to store single-keyed data that is persisted across restarts.
|
||||
persisted: Database<Str, Str>,
|
||||
|
||||
/// Webhook, loaded and stored in the `persisted` database
|
||||
webhooks: Arc<Webhooks>,
|
||||
|
||||
/// A map to retrieve the runtime representation of an embedder depending on its configuration.
|
||||
///
|
||||
@ -214,8 +226,9 @@ impl IndexScheduler {
|
||||
index_mapper: self.index_mapper.clone(),
|
||||
cleanup_enabled: self.cleanup_enabled,
|
||||
experimental_no_edition_2024_for_dumps: self.experimental_no_edition_2024_for_dumps,
|
||||
webhook_url: self.webhook_url.clone(),
|
||||
webhook_authorization_header: self.webhook_authorization_header.clone(),
|
||||
persisted: self.persisted,
|
||||
|
||||
webhooks: self.webhooks.clone(),
|
||||
embedders: self.embedders.clone(),
|
||||
#[cfg(test)]
|
||||
test_breakpoint_sdr: self.test_breakpoint_sdr.clone(),
|
||||
@ -234,6 +247,7 @@ impl IndexScheduler {
|
||||
+ IndexMapper::nb_db()
|
||||
+ features::FeatureData::nb_db()
|
||||
+ 1 // chat-prompts
|
||||
+ 1 // persisted
|
||||
}
|
||||
|
||||
/// Create an index scheduler and start its run loop.
|
||||
@ -284,10 +298,18 @@ impl IndexScheduler {
|
||||
let version = versioning::Versioning::new(&env, from_db_version)?;
|
||||
|
||||
let mut wtxn = env.write_txn()?;
|
||||
|
||||
let features = features::FeatureData::new(&env, &mut wtxn, options.instance_features)?;
|
||||
let queue = Queue::new(&env, &mut wtxn, &options)?;
|
||||
let index_mapper = IndexMapper::new(&env, &mut wtxn, &options, budget)?;
|
||||
let chat_settings = env.create_database(&mut wtxn, Some(CHAT_SETTINGS_DB_NAME))?;
|
||||
let chat_settings = env.create_database(&mut wtxn, Some(db_name::CHAT_SETTINGS))?;
|
||||
|
||||
let persisted = env.create_database(&mut wtxn, Some(db_name::PERSISTED))?;
|
||||
let webhooks_db = persisted.remap_data_type::<SerdeJson<Webhooks>>();
|
||||
let mut webhooks = webhooks_db.get(&wtxn, db_keys::WEBHOOKS)?.unwrap_or_default();
|
||||
webhooks
|
||||
.with_cli(options.cli_webhook_url.clone(), options.cli_webhook_authorization.clone());
|
||||
|
||||
wtxn.commit()?;
|
||||
|
||||
// allow unreachable_code to get rids of the warning in the case of a test build.
|
||||
@ -303,8 +325,8 @@ impl IndexScheduler {
|
||||
experimental_no_edition_2024_for_dumps: options
|
||||
.indexer_config
|
||||
.experimental_no_edition_2024_for_dumps,
|
||||
webhook_url: options.webhook_url,
|
||||
webhook_authorization_header: options.webhook_authorization_header,
|
||||
persisted,
|
||||
webhooks: Arc::new(webhooks),
|
||||
embedders: Default::default(),
|
||||
|
||||
#[cfg(test)]
|
||||
@ -752,86 +774,92 @@ impl IndexScheduler {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Once the tasks changes have been committed we must send all the tasks that were updated to our webhook if there is one.
|
||||
fn notify_webhook(&self, updated: &RoaringBitmap) -> Result<()> {
|
||||
if let Some(ref url) = self.webhook_url {
|
||||
struct TaskReader<'a, 'b> {
|
||||
rtxn: &'a RoTxn<'a>,
|
||||
index_scheduler: &'a IndexScheduler,
|
||||
tasks: &'b mut roaring::bitmap::Iter<'b>,
|
||||
buffer: Vec<u8>,
|
||||
written: usize,
|
||||
}
|
||||
/// Once the tasks changes have been committed we must send all the tasks that were updated to our webhooks
|
||||
fn notify_webhooks(&self, updated: RoaringBitmap) {
|
||||
struct TaskReader<'a, 'b> {
|
||||
rtxn: &'a RoTxn<'a>,
|
||||
index_scheduler: &'a IndexScheduler,
|
||||
tasks: &'b mut roaring::bitmap::Iter<'b>,
|
||||
buffer: Vec<u8>,
|
||||
written: usize,
|
||||
}
|
||||
|
||||
impl Read for TaskReader<'_, '_> {
|
||||
fn read(&mut self, mut buf: &mut [u8]) -> std::io::Result<usize> {
|
||||
if self.buffer.is_empty() {
|
||||
match self.tasks.next() {
|
||||
None => return Ok(0),
|
||||
Some(task_id) => {
|
||||
let task = self
|
||||
.index_scheduler
|
||||
.queue
|
||||
.tasks
|
||||
.get_task(self.rtxn, task_id)
|
||||
.map_err(|err| io::Error::new(io::ErrorKind::Other, err))?
|
||||
.ok_or_else(|| {
|
||||
io::Error::new(
|
||||
io::ErrorKind::Other,
|
||||
Error::CorruptedTaskQueue,
|
||||
)
|
||||
})?;
|
||||
impl Read for TaskReader<'_, '_> {
|
||||
fn read(&mut self, mut buf: &mut [u8]) -> std::io::Result<usize> {
|
||||
if self.buffer.is_empty() {
|
||||
match self.tasks.next() {
|
||||
None => return Ok(0),
|
||||
Some(task_id) => {
|
||||
let task = self
|
||||
.index_scheduler
|
||||
.queue
|
||||
.tasks
|
||||
.get_task(self.rtxn, task_id)
|
||||
.map_err(|err| io::Error::new(io::ErrorKind::Other, err))?
|
||||
.ok_or_else(|| {
|
||||
io::Error::new(io::ErrorKind::Other, Error::CorruptedTaskQueue)
|
||||
})?;
|
||||
|
||||
serde_json::to_writer(
|
||||
&mut self.buffer,
|
||||
&TaskView::from_task(&task),
|
||||
)?;
|
||||
self.buffer.push(b'\n');
|
||||
}
|
||||
serde_json::to_writer(&mut self.buffer, &TaskView::from_task(&task))?;
|
||||
self.buffer.push(b'\n');
|
||||
}
|
||||
}
|
||||
|
||||
let mut to_write = &self.buffer[self.written..];
|
||||
let wrote = io::copy(&mut to_write, &mut buf)?;
|
||||
self.written += wrote as usize;
|
||||
|
||||
// we wrote everything and must refresh our buffer on the next call
|
||||
if self.written == self.buffer.len() {
|
||||
self.written = 0;
|
||||
self.buffer.clear();
|
||||
}
|
||||
|
||||
Ok(wrote as usize)
|
||||
}
|
||||
}
|
||||
|
||||
let rtxn = self.env.read_txn()?;
|
||||
let mut to_write = &self.buffer[self.written..];
|
||||
let wrote = io::copy(&mut to_write, &mut buf)?;
|
||||
self.written += wrote as usize;
|
||||
|
||||
let task_reader = TaskReader {
|
||||
rtxn: &rtxn,
|
||||
index_scheduler: self,
|
||||
tasks: &mut updated.into_iter(),
|
||||
buffer: Vec::with_capacity(50), // on average a task is around ~100 bytes
|
||||
written: 0,
|
||||
};
|
||||
// we wrote everything and must refresh our buffer on the next call
|
||||
if self.written == self.buffer.len() {
|
||||
self.written = 0;
|
||||
self.buffer.clear();
|
||||
}
|
||||
|
||||
// let reader = GzEncoder::new(BufReader::new(task_reader), Compression::default());
|
||||
let reader = GzEncoder::new(BufReader::new(task_reader), Compression::default());
|
||||
let request = ureq::post(url)
|
||||
.timeout(Duration::from_secs(30))
|
||||
.set("Content-Encoding", "gzip")
|
||||
.set("Content-Type", "application/x-ndjson");
|
||||
let request = match &self.webhook_authorization_header {
|
||||
Some(header) => request.set("Authorization", header),
|
||||
None => request,
|
||||
};
|
||||
|
||||
if let Err(e) = request.send(reader) {
|
||||
tracing::error!("While sending data to the webhook: {e}");
|
||||
Ok(wrote as usize)
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
let webhooks = self.webhooks.get_all();
|
||||
if webhooks.is_empty() {
|
||||
return;
|
||||
}
|
||||
let this = self.private_clone();
|
||||
// We must take the RoTxn before entering the thread::spawn otherwise another batch may be
|
||||
// processed before we had the time to take our txn.
|
||||
let rtxn = match self.env.clone().static_read_txn() {
|
||||
Ok(rtxn) => rtxn,
|
||||
Err(e) => {
|
||||
tracing::error!("Couldn't get an rtxn to notify the webhook: {e}");
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
std::thread::spawn(move || {
|
||||
for (uuid, Webhook { url, headers }) in webhooks.iter() {
|
||||
let task_reader = TaskReader {
|
||||
rtxn: &rtxn,
|
||||
index_scheduler: &this,
|
||||
tasks: &mut updated.iter(),
|
||||
buffer: Vec::with_capacity(page_size::get()),
|
||||
written: 0,
|
||||
};
|
||||
|
||||
let reader = GzEncoder::new(BufReader::new(task_reader), Compression::default());
|
||||
|
||||
let mut request = ureq::post(url)
|
||||
.timeout(Duration::from_secs(30))
|
||||
.set("Content-Encoding", "gzip")
|
||||
.set("Content-Type", "application/x-ndjson");
|
||||
for (header_name, header_value) in headers.iter() {
|
||||
request = request.set(header_name, header_value);
|
||||
}
|
||||
|
||||
if let Err(e) = request.send(reader) {
|
||||
tracing::error!("While sending data to the webhook {uuid}: {e}");
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
pub fn index_stats(&self, index_uid: &str) -> Result<IndexStats> {
|
||||
@ -862,6 +890,29 @@ impl IndexScheduler {
|
||||
self.features.network()
|
||||
}
|
||||
|
||||
pub fn update_runtime_webhooks(&self, runtime: RuntimeWebhooks) -> Result<()> {
|
||||
let webhooks = Webhooks::from_runtime(runtime);
|
||||
let mut wtxn = self.env.write_txn()?;
|
||||
let webhooks_db = self.persisted.remap_data_type::<SerdeJson<Webhooks>>();
|
||||
webhooks_db.put(&mut wtxn, db_keys::WEBHOOKS, &webhooks)?;
|
||||
wtxn.commit()?;
|
||||
self.webhooks.update_runtime(webhooks.into_runtime());
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn webhooks_dump_view(&self) -> WebhooksDumpView {
|
||||
// We must not dump the cli api key
|
||||
WebhooksDumpView { webhooks: self.webhooks.get_runtime() }
|
||||
}
|
||||
|
||||
pub fn webhooks_view(&self) -> WebhooksView {
|
||||
WebhooksView { webhooks: self.webhooks.get_all() }
|
||||
}
|
||||
|
||||
pub fn retrieve_runtime_webhooks(&self) -> RuntimeWebhooks {
|
||||
self.webhooks.get_runtime()
|
||||
}
|
||||
|
||||
pub fn embedders(
|
||||
&self,
|
||||
index_uid: String,
|
||||
@ -990,3 +1041,72 @@ pub struct IndexStats {
|
||||
/// Internal stats computed from the index.
|
||||
pub inner_stats: index_mapper::IndexStats,
|
||||
}
|
||||
|
||||
/// These structure are not meant to be exposed to the end user, if needed, use the meilisearch-types::webhooks structure instead.
|
||||
/// /!\ Everytime you deserialize this structure you should fill the cli_webhook later on with the `with_cli` method. /!\
|
||||
#[derive(Debug, Serialize, Deserialize, Default)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct Webhooks {
|
||||
// The cli webhook should *never* be stored in a database.
|
||||
// It represent a state that only exists for this execution of meilisearch
|
||||
#[serde(skip)]
|
||||
pub cli: Option<CliWebhook>,
|
||||
|
||||
#[serde(default)]
|
||||
pub runtime: RwLock<RuntimeWebhooks>,
|
||||
}
|
||||
|
||||
type RuntimeWebhooks = BTreeMap<Uuid, Webhook>;
|
||||
|
||||
impl Webhooks {
|
||||
pub fn with_cli(&mut self, url: Option<String>, auth: Option<String>) {
|
||||
if let Some(url) = url {
|
||||
let webhook = CliWebhook { url, auth };
|
||||
self.cli = Some(webhook);
|
||||
}
|
||||
}
|
||||
|
||||
pub fn from_runtime(webhooks: RuntimeWebhooks) -> Self {
|
||||
Self { cli: None, runtime: RwLock::new(webhooks) }
|
||||
}
|
||||
|
||||
pub fn into_runtime(self) -> RuntimeWebhooks {
|
||||
// safe because we own self and it cannot be cloned
|
||||
self.runtime.into_inner().unwrap()
|
||||
}
|
||||
|
||||
pub fn update_runtime(&self, webhooks: RuntimeWebhooks) {
|
||||
*self.runtime.write().unwrap() = webhooks;
|
||||
}
|
||||
|
||||
/// Returns all the webhooks in an unified view. The cli webhook is represented with an uuid set to 0
|
||||
pub fn get_all(&self) -> BTreeMap<Uuid, Webhook> {
|
||||
self.cli
|
||||
.as_ref()
|
||||
.map(|wh| (Uuid::nil(), Webhook::from(wh)))
|
||||
.into_iter()
|
||||
.chain(self.runtime.read().unwrap().iter().map(|(uuid, wh)| (*uuid, wh.clone())))
|
||||
.collect()
|
||||
}
|
||||
|
||||
/// Returns all the runtime webhooks.
|
||||
pub fn get_runtime(&self) -> BTreeMap<Uuid, Webhook> {
|
||||
self.runtime.read().unwrap().iter().map(|(uuid, wh)| (*uuid, wh.clone())).collect()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize, Default, Clone, PartialEq)]
|
||||
struct CliWebhook {
|
||||
pub url: String,
|
||||
pub auth: Option<String>,
|
||||
}
|
||||
|
||||
impl From<&CliWebhook> for Webhook {
|
||||
fn from(webhook: &CliWebhook) -> Self {
|
||||
let mut headers = BTreeMap::new();
|
||||
if let Some(ref auth) = webhook.auth {
|
||||
headers.insert("Authorization".to_string(), auth.to_string());
|
||||
}
|
||||
Self { url: webhook.url.to_string(), headers }
|
||||
}
|
||||
}
|
||||
|
@ -108,6 +108,7 @@ make_enum_progress! {
|
||||
DumpTheBatches,
|
||||
DumpTheIndexes,
|
||||
DumpTheExperimentalFeatures,
|
||||
DumpTheWebhooks,
|
||||
CompressTheDump,
|
||||
}
|
||||
}
|
||||
|
@ -446,8 +446,7 @@ impl IndexScheduler {
|
||||
Ok(())
|
||||
})?;
|
||||
|
||||
// We shouldn't crash the tick function if we can't send data to the webhook.
|
||||
let _ = self.notify_webhook(&ids);
|
||||
self.notify_webhooks(ids);
|
||||
|
||||
#[cfg(test)]
|
||||
self.breakpoint(crate::test_utils::Breakpoint::AfterProcessing);
|
||||
|
@ -270,6 +270,11 @@ impl IndexScheduler {
|
||||
let network = self.network();
|
||||
dump.create_network(network)?;
|
||||
|
||||
// 7. Dump the webhooks
|
||||
progress.update_progress(DumpCreationProgress::DumpTheWebhooks);
|
||||
let webhooks = self.webhooks_dump_view();
|
||||
dump.create_webhooks(webhooks)?;
|
||||
|
||||
let dump_uid = started_at.format(format_description!(
|
||||
"[year repr:full][month repr:numerical][day padding:zero]-[hour padding:zero][minute padding:zero][second padding:zero][subsecond digits:3]"
|
||||
)).unwrap();
|
||||
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
||||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, batch_uid: 0, status: succeeded, details: { from: (1, 12, 0), to: (1, 16, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, batch_uid: 0, status: succeeded, details: { from: (1, 12, 0), to: (1, 17, 1) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, batch_uid: 1, status: succeeded, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
2 {uid: 2, batch_uid: 2, status: succeeded, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
3 {uid: 3, batch_uid: 3, status: failed, error: ResponseError { code: 200, message: "Index `doggo` already exists.", error_code: "index_already_exists", error_type: "invalid_request", error_link: "https://docs.meilisearch.com/errors#index_already_exists" }, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
@ -57,7 +57,7 @@ girafo: { number_of_documents: 0, field_distribution: {} }
|
||||
[timestamp] [4,]
|
||||
----------------------------------------------------------------------
|
||||
### All Batches:
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.16.0"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "stopped after the last task of type `upgradeDatabase` because they cannot be batched with tasks of any other type.", }
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.17.1"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "stopped after the last task of type `upgradeDatabase` because they cannot be batched with tasks of any other type.", }
|
||||
1 {uid: 1, details: {"primaryKey":"mouse"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"catto":1}}, stop reason: "created batch containing only task with id 1 of type `indexCreation` that cannot be batched with any other task.", }
|
||||
2 {uid: 2, details: {"primaryKey":"bone"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, stop reason: "created batch containing only task with id 2 of type `indexCreation` that cannot be batched with any other task.", }
|
||||
3 {uid: 3, details: {"primaryKey":"bone"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"indexCreation":1},"indexUids":{"doggo":1}}, stop reason: "created batch containing only task with id 3 of type `indexCreation` that cannot be batched with any other task.", }
|
||||
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
||||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, status: enqueued, details: { from: (1, 12, 0), to: (1, 16, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, status: enqueued, details: { from: (1, 12, 0), to: (1, 17, 1) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
----------------------------------------------------------------------
|
||||
### Status:
|
||||
enqueued [0,]
|
||||
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
||||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, status: enqueued, details: { from: (1, 12, 0), to: (1, 16, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, status: enqueued, details: { from: (1, 12, 0), to: (1, 17, 1) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
----------------------------------------------------------------------
|
||||
### Status:
|
||||
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
||||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { from: (1, 12, 0), to: (1, 16, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { from: (1, 12, 0), to: (1, 17, 1) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
----------------------------------------------------------------------
|
||||
### Status:
|
||||
@ -37,7 +37,7 @@ catto [1,]
|
||||
[timestamp] [0,]
|
||||
----------------------------------------------------------------------
|
||||
### All Batches:
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.16.0"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "stopped after the last task of type `upgradeDatabase` because they cannot be batched with tasks of any other type.", }
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.17.1"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "stopped after the last task of type `upgradeDatabase` because they cannot be batched with tasks of any other type.", }
|
||||
----------------------------------------------------------------------
|
||||
### Batch to tasks mapping:
|
||||
0 [0,]
|
||||
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
||||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { from: (1, 12, 0), to: (1, 16, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, batch_uid: 0, status: failed, error: ResponseError { code: 200, message: "Planned failure for tests.", error_code: "internal", error_type: "internal", error_link: "https://docs.meilisearch.com/errors#internal" }, details: { from: (1, 12, 0), to: (1, 17, 1) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
2 {uid: 2, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
----------------------------------------------------------------------
|
||||
@ -40,7 +40,7 @@ doggo [2,]
|
||||
[timestamp] [0,]
|
||||
----------------------------------------------------------------------
|
||||
### All Batches:
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.16.0"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "stopped after the last task of type `upgradeDatabase` because they cannot be batched with tasks of any other type.", }
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.17.1"}, stats: {"totalNbTasks":1,"status":{"failed":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "stopped after the last task of type `upgradeDatabase` because they cannot be batched with tasks of any other type.", }
|
||||
----------------------------------------------------------------------
|
||||
### Batch to tasks mapping:
|
||||
0 [0,]
|
||||
|
@ -6,7 +6,7 @@ source: crates/index-scheduler/src/scheduler/test_failure.rs
|
||||
[]
|
||||
----------------------------------------------------------------------
|
||||
### All Tasks:
|
||||
0 {uid: 0, batch_uid: 0, status: succeeded, details: { from: (1, 12, 0), to: (1, 16, 0) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
0 {uid: 0, batch_uid: 0, status: succeeded, details: { from: (1, 12, 0), to: (1, 17, 1) }, kind: UpgradeDatabase { from: (1, 12, 0) }}
|
||||
1 {uid: 1, status: enqueued, details: { primary_key: Some("mouse") }, kind: IndexCreation { index_uid: "catto", primary_key: Some("mouse") }}
|
||||
2 {uid: 2, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
3 {uid: 3, status: enqueued, details: { primary_key: Some("bone") }, kind: IndexCreation { index_uid: "doggo", primary_key: Some("bone") }}
|
||||
@ -43,7 +43,7 @@ doggo [2,3,]
|
||||
[timestamp] [0,]
|
||||
----------------------------------------------------------------------
|
||||
### All Batches:
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.16.0"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "stopped after the last task of type `upgradeDatabase` because they cannot be batched with tasks of any other type.", }
|
||||
0 {uid: 0, details: {"upgradeFrom":"v1.12.0","upgradeTo":"v1.17.1"}, stats: {"totalNbTasks":1,"status":{"succeeded":1},"types":{"upgradeDatabase":1},"indexUids":{}}, stop reason: "stopped after the last task of type `upgradeDatabase` because they cannot be batched with tasks of any other type.", }
|
||||
----------------------------------------------------------------------
|
||||
### Batch to tasks mapping:
|
||||
0 [0,]
|
||||
|
@ -98,8 +98,8 @@ impl IndexScheduler {
|
||||
indexes_path: tempdir.path().join("indexes"),
|
||||
snapshots_path: tempdir.path().join("snapshots"),
|
||||
dumps_path: tempdir.path().join("dumps"),
|
||||
webhook_url: None,
|
||||
webhook_authorization_header: None,
|
||||
cli_webhook_url: None,
|
||||
cli_webhook_authorization: None,
|
||||
task_db_size: 1000 * 1000 * 10, // 10 MB, we don't use MiB on purpose.
|
||||
index_base_map_size: 1000 * 1000, // 1 MB, we don't use MiB on purpose.
|
||||
enable_mdb_writemap: false,
|
||||
|
@ -39,6 +39,7 @@ pub fn upgrade_index_scheduler(
|
||||
(1, 13, _) => 0,
|
||||
(1, 14, _) => 0,
|
||||
(1, 15, _) => 0,
|
||||
(1, 16, _) => 0,
|
||||
(major, minor, patch) => {
|
||||
if major > current_major
|
||||
|| (major == current_major && minor > current_minor)
|
||||
|
Reference in New Issue
Block a user