Merge pull request #5670 from meilisearch/export-and-transfer-route

Introduce a new route to export indexes
This commit is contained in:
Mubelotix 2025-07-01 14:37:02 +00:00 committed by GitHub
commit 8fef48f8ca
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
36 changed files with 990 additions and 27 deletions

View file

@ -4,6 +4,7 @@ use std::io;
use dump::{KindDump, TaskDump, UpdateFile};
use meilisearch_types::batches::{Batch, BatchId};
use meilisearch_types::heed::RwTxn;
use meilisearch_types::index_uid_pattern::IndexUidPattern;
use meilisearch_types::milli;
use meilisearch_types::tasks::{Kind, KindWithContent, Status, Task};
use roaring::RoaringBitmap;
@ -211,6 +212,23 @@ impl<'a> Dump<'a> {
KindWithContent::DumpCreation { keys, instance_uid }
}
KindDump::SnapshotCreation => KindWithContent::SnapshotCreation,
KindDump::Export { url, api_key, payload_size, indexes } => {
KindWithContent::Export {
url,
api_key,
payload_size,
indexes: indexes
.into_iter()
.map(|(pattern, settings)| {
Ok((
IndexUidPattern::try_from(pattern)
.map_err(|_| Error::CorruptedDump)?,
settings,
))
})
.collect::<Result<_, Error>>()?,
}
}
KindDump::UpgradeDatabase { from } => KindWithContent::UpgradeDatabase { from },
},
};

View file

@ -151,6 +151,10 @@ pub enum Error {
CorruptedTaskQueue,
#[error(transparent)]
DatabaseUpgrade(Box<Self>),
#[error(transparent)]
Export(Box<Self>),
#[error("Failed to export documents to remote server {code} ({type}): {message} <{link}>")]
FromRemoteWhenExporting { message: String, code: String, r#type: String, link: String },
#[error("Failed to rollback for index `{index}`: {rollback_outcome} ")]
RollbackFailed { index: String, rollback_outcome: RollbackOutcome },
#[error(transparent)]
@ -212,6 +216,7 @@ impl Error {
| Error::BatchNotFound(_)
| Error::TaskDeletionWithEmptyQuery
| Error::TaskCancelationWithEmptyQuery
| Error::FromRemoteWhenExporting { .. }
| Error::AbortedTask
| Error::Dump(_)
| Error::Heed(_)
@ -221,6 +226,7 @@ impl Error {
| Error::IoError(_)
| Error::Persist(_)
| Error::FeatureNotEnabled(_)
| Error::Export(_)
| Error::Anyhow(_) => true,
Error::CreateBatch(_)
| Error::CorruptedTaskQueue
@ -282,6 +288,7 @@ impl ErrorCode for Error {
Error::Dump(e) => e.error_code(),
Error::Milli { error, .. } => error.error_code(),
Error::ProcessBatchPanicked(_) => Code::Internal,
Error::FromRemoteWhenExporting { .. } => Code::Internal,
Error::Heed(e) => e.error_code(),
Error::HeedTransaction(e) => e.error_code(),
Error::FileStore(e) => e.error_code(),
@ -294,6 +301,7 @@ impl ErrorCode for Error {
Error::CorruptedTaskQueue => Code::Internal,
Error::CorruptedDump => Code::Internal,
Error::DatabaseUpgrade(_) => Code::Internal,
Error::Export(_) => Code::Internal,
Error::RollbackFailed { .. } => Code::Internal,
Error::UnrecoverableError(_) => Code::Internal,
Error::IndexSchedulerVersionMismatch { .. } => Code::Internal,

View file

@ -289,6 +289,9 @@ fn snapshot_details(d: &Details) -> String {
Details::IndexSwap { swaps } => {
format!("{{ swaps: {swaps:?} }}")
}
Details::Export { url, api_key, payload_size, indexes } => {
format!("{{ url: {url:?}, api_key: {api_key:?}, payload_size: {payload_size:?}, indexes: {indexes:?} }}")
}
Details::UpgradeDatabase { from, to } => {
format!("{{ from: {from:?}, to: {to:?} }}")
}

View file

@ -175,8 +175,17 @@ make_enum_progress! {
}
}
make_enum_progress! {
pub enum Export {
EnsuringCorrectnessOfTheTarget,
ExportingTheSettings,
ExportingTheDocuments,
}
}
make_atomic_progress!(Task alias AtomicTaskStep => "task" );
make_atomic_progress!(Document alias AtomicDocumentStep => "document" );
make_atomic_progress!(Index alias AtomicIndexStep => "index" );
make_atomic_progress!(Batch alias AtomicBatchStep => "batch" );
make_atomic_progress!(UpdateFile alias AtomicUpdateFileStep => "update file" );

View file

@ -71,6 +71,7 @@ impl From<KindWithContent> for AutobatchKind {
KindWithContent::TaskCancelation { .. }
| KindWithContent::TaskDeletion { .. }
| KindWithContent::DumpCreation { .. }
| KindWithContent::Export { .. }
| KindWithContent::UpgradeDatabase { .. }
| KindWithContent::SnapshotCreation => {
panic!("The autobatcher should never be called with tasks that don't apply to an index.")

View file

@ -48,6 +48,9 @@ pub(crate) enum Batch {
IndexSwap {
task: Task,
},
Export {
task: Task,
},
UpgradeDatabase {
tasks: Vec<Task>,
},
@ -104,6 +107,7 @@ impl Batch {
Batch::TaskCancelation { task, .. }
| Batch::Dump(task)
| Batch::IndexCreation { task, .. }
| Batch::Export { task }
| Batch::IndexUpdate { task, .. } => {
RoaringBitmap::from_sorted_iter(std::iter::once(task.uid)).unwrap()
}
@ -143,6 +147,7 @@ impl Batch {
| TaskDeletions(_)
| SnapshotCreation(_)
| Dump(_)
| Export { .. }
| UpgradeDatabase { .. }
| IndexSwap { .. } => None,
IndexOperation { op, .. } => Some(op.index_uid()),
@ -168,6 +173,7 @@ impl fmt::Display for Batch {
Batch::IndexUpdate { .. } => f.write_str("IndexUpdate")?,
Batch::IndexDeletion { .. } => f.write_str("IndexDeletion")?,
Batch::IndexSwap { .. } => f.write_str("IndexSwap")?,
Batch::Export { .. } => f.write_str("Export")?,
Batch::UpgradeDatabase { .. } => f.write_str("UpgradeDatabase")?,
};
match index_uid {
@ -427,9 +433,10 @@ impl IndexScheduler {
/// 0. We get the *last* task to cancel.
/// 1. We get the tasks to upgrade.
/// 2. We get the *next* task to delete.
/// 3. We get the *next* snapshot to process.
/// 4. We get the *next* dump to process.
/// 5. We get the *next* tasks to process for a specific index.
/// 3. We get the *next* export to process.
/// 4. We get the *next* snapshot to process.
/// 5. We get the *next* dump to process.
/// 6. We get the *next* tasks to process for a specific index.
#[tracing::instrument(level = "trace", skip(self, rtxn), target = "indexing::scheduler")]
pub(crate) fn create_next_batch(
&self,
@ -501,7 +508,17 @@ impl IndexScheduler {
return Ok(Some((Batch::TaskDeletions(tasks), current_batch)));
}
// 3. we batch the snapshot.
// 3. we batch the export.
let to_export = self.queue.tasks.get_kind(rtxn, Kind::Export)? & enqueued;
if !to_export.is_empty() {
let task_id = to_export.iter().next().expect("There must be at least one export task");
let mut task = self.queue.tasks.get_task(rtxn, task_id)?.unwrap();
current_batch.processing([&mut task]);
current_batch.reason(BatchStopReason::TaskKindCannotBeBatched { kind: Kind::Export });
return Ok(Some((Batch::Export { task }, current_batch)));
}
// 4. we batch the snapshot.
let to_snapshot = self.queue.tasks.get_kind(rtxn, Kind::SnapshotCreation)? & enqueued;
if !to_snapshot.is_empty() {
let mut tasks = self.queue.tasks.get_existing_tasks(rtxn, to_snapshot)?;
@ -511,7 +528,7 @@ impl IndexScheduler {
return Ok(Some((Batch::SnapshotCreation(tasks), current_batch)));
}
// 4. we batch the dumps.
// 5. we batch the dumps.
let to_dump = self.queue.tasks.get_kind(rtxn, Kind::DumpCreation)? & enqueued;
if let Some(to_dump) = to_dump.min() {
let mut task =
@ -524,7 +541,7 @@ impl IndexScheduler {
return Ok(Some((Batch::Dump(task), current_batch)));
}
// 5. We make a batch from the unprioritised tasks. Start by taking the next enqueued task.
// 6. We make a batch from the unprioritised tasks. Start by taking the next enqueued task.
let task_id = if let Some(task_id) = enqueued.min() { task_id } else { return Ok(None) };
let mut task =
self.queue.tasks.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?;

View file

@ -4,6 +4,7 @@ mod autobatcher_test;
mod create_batch;
mod process_batch;
mod process_dump_creation;
mod process_export;
mod process_index_operation;
mod process_snapshot_creation;
mod process_upgrade;

View file

@ -368,6 +368,46 @@ impl IndexScheduler {
task.status = Status::Succeeded;
Ok((vec![task], ProcessBatchInfo::default()))
}
Batch::Export { mut task } => {
let KindWithContent::Export { url, api_key, payload_size, indexes } = &task.kind
else {
unreachable!()
};
let ret = catch_unwind(AssertUnwindSafe(|| {
self.process_export(
url,
api_key.as_deref(),
payload_size.as_ref(),
indexes,
progress,
)
}));
let stats = match ret {
Ok(Ok(stats)) => stats,
Ok(Err(Error::AbortedTask)) => return Err(Error::AbortedTask),
Ok(Err(e)) => return Err(Error::Export(Box::new(e))),
Err(e) => {
let msg = match e.downcast_ref::<&'static str>() {
Some(s) => *s,
None => match e.downcast_ref::<String>() {
Some(s) => &s[..],
None => "Box<dyn Any>",
},
};
return Err(Error::Export(Box::new(Error::ProcessBatchPanicked(
msg.to_string(),
))));
}
};
task.status = Status::Succeeded;
if let Some(Details::Export { indexes, .. }) = task.details.as_mut() {
*indexes = stats;
}
Ok((vec![task], ProcessBatchInfo::default()))
}
Batch::UpgradeDatabase { mut tasks } => {
let KindWithContent::UpgradeDatabase { from } = tasks.last().unwrap().kind else {
unreachable!();
@ -715,9 +755,11 @@ impl IndexScheduler {
from.1,
from.2
);
match std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| {
let ret = catch_unwind(std::panic::AssertUnwindSafe(|| {
self.process_rollback(from, progress)
})) {
}));
match ret {
Ok(Ok(())) => {}
Ok(Err(err)) => return Err(Error::DatabaseUpgrade(Box::new(err))),
Err(e) => {

View file

@ -0,0 +1,373 @@
use std::collections::BTreeMap;
use std::io::{self, Write as _};
use std::sync::atomic;
use std::time::Duration;
use backoff::ExponentialBackoff;
use byte_unit::Byte;
use flate2::write::GzEncoder;
use flate2::Compression;
use meilisearch_types::index_uid_pattern::IndexUidPattern;
use meilisearch_types::milli::constants::RESERVED_VECTORS_FIELD_NAME;
use meilisearch_types::milli::progress::{Progress, VariableNameStep};
use meilisearch_types::milli::update::{request_threads, Setting};
use meilisearch_types::milli::vector::parsed_vectors::{ExplicitVectors, VectorOrArrayOfVectors};
use meilisearch_types::milli::{self, obkv_to_json, Filter, InternalError};
use meilisearch_types::settings::{self, SecretPolicy};
use meilisearch_types::tasks::{DetailsExportIndexSettings, ExportIndexSettings};
use serde::Deserialize;
use ureq::{json, Response};
use super::MustStopProcessing;
use crate::processing::AtomicDocumentStep;
use crate::{Error, IndexScheduler, Result};
impl IndexScheduler {
pub(super) fn process_export(
&self,
base_url: &str,
api_key: Option<&str>,
payload_size: Option<&Byte>,
indexes: &BTreeMap<IndexUidPattern, ExportIndexSettings>,
progress: Progress,
) -> Result<BTreeMap<IndexUidPattern, DetailsExportIndexSettings>> {
#[cfg(test)]
self.maybe_fail(crate::test_utils::FailureLocation::ProcessExport)?;
let indexes: Vec<_> = self
.index_names()?
.into_iter()
.flat_map(|uid| {
indexes
.iter()
.find(|(pattern, _)| pattern.matches_str(&uid))
.map(|(pattern, settings)| (pattern, uid, settings))
})
.collect();
let mut output = BTreeMap::new();
let agent = ureq::AgentBuilder::new().timeout(Duration::from_secs(5)).build();
let must_stop_processing = self.scheduler.must_stop_processing.clone();
for (i, (_pattern, uid, export_settings)) in indexes.iter().enumerate() {
if must_stop_processing.get() {
return Err(Error::AbortedTask);
}
progress.update_progress(VariableNameStep::<ExportIndex>::new(
format!("Exporting index `{uid}`"),
i as u32,
indexes.len() as u32,
));
let ExportIndexSettings { filter, override_settings } = export_settings;
let index = self.index(uid)?;
let index_rtxn = index.read_txn()?;
// First, check if the index already exists
let url = format!("{base_url}/indexes/{uid}");
let response = retry(&must_stop_processing, || {
let mut request = agent.get(&url);
if let Some(api_key) = api_key {
request = request.set("Authorization", &format!("Bearer {api_key}"));
}
request.send_bytes(Default::default()).map_err(into_backoff_error)
});
let index_exists = match response {
Ok(response) => response.status() == 200,
Err(Error::FromRemoteWhenExporting { code, .. }) if code == "index_not_found" => {
false
}
Err(e) => return Err(e),
};
let primary_key = index
.primary_key(&index_rtxn)
.map_err(|e| Error::from_milli(e.into(), Some(uid.to_string())))?;
// Create the index
if !index_exists {
let url = format!("{base_url}/indexes");
retry(&must_stop_processing, || {
let mut request = agent.post(&url);
if let Some(api_key) = api_key {
request = request.set("Authorization", &format!("Bearer {api_key}"));
}
let index_param = json!({ "uid": uid, "primaryKey": primary_key });
request.send_json(&index_param).map_err(into_backoff_error)
})?;
}
// Patch the index primary key
if index_exists && *override_settings {
let url = format!("{base_url}/indexes/{uid}");
retry(&must_stop_processing, || {
let mut request = agent.patch(&url);
if let Some(api_key) = api_key {
request = request.set("Authorization", &format!("Bearer {api_key}"));
}
let index_param = json!({ "primaryKey": primary_key });
request.send_json(&index_param).map_err(into_backoff_error)
})?;
}
// Send the index settings
if !index_exists || *override_settings {
let mut settings =
settings::settings(&index, &index_rtxn, SecretPolicy::RevealSecrets)
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?;
// Remove the experimental chat setting if not enabled
if self.features().check_chat_completions("exporting chat settings").is_err() {
settings.chat = Setting::NotSet;
}
// Retry logic for sending settings
let url = format!("{base_url}/indexes/{uid}/settings");
let bearer = api_key.map(|api_key| format!("Bearer {api_key}"));
retry(&must_stop_processing, || {
let mut request = agent.patch(&url);
if let Some(bearer) = bearer.as_ref() {
request = request.set("Authorization", bearer);
}
request.send_json(settings.clone()).map_err(into_backoff_error)
})?;
}
let filter = filter
.as_ref()
.map(Filter::from_json)
.transpose()
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?
.flatten();
let filter_universe = filter
.map(|f| f.evaluate(&index_rtxn, &index))
.transpose()
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?;
let whole_universe = index
.documents_ids(&index_rtxn)
.map_err(|e| Error::from_milli(e.into(), Some(uid.to_string())))?;
let universe = filter_universe.unwrap_or(whole_universe);
let fields_ids_map = index.fields_ids_map(&index_rtxn)?;
let all_fields: Vec<_> = fields_ids_map.iter().map(|(id, _)| id).collect();
let embedding_configs = index
.embedding_configs(&index_rtxn)
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?;
// We don't need to keep this one alive as we will
// spawn many threads to process the documents
drop(index_rtxn);
let total_documents = universe.len() as u32;
let (step, progress_step) = AtomicDocumentStep::new(total_documents);
progress.update_progress(progress_step);
output.insert(
IndexUidPattern::new_unchecked(uid.clone()),
DetailsExportIndexSettings {
settings: (*export_settings).clone(),
matched_documents: Some(total_documents as u64),
},
);
let limit = payload_size.map(|ps| ps.as_u64() as usize).unwrap_or(50 * 1024 * 1024); // defaults to 50 MiB
let documents_url = format!("{base_url}/indexes/{uid}/documents");
request_threads()
.broadcast(|ctx| {
let index_rtxn = index
.read_txn()
.map_err(|e| Error::from_milli(e.into(), Some(uid.to_string())))?;
let mut buffer = Vec::new();
let mut tmp_buffer = Vec::new();
let mut compressed_buffer = Vec::new();
for (i, docid) in universe.iter().enumerate() {
if i % ctx.num_threads() != ctx.index() {
continue;
}
let document = index
.document(&index_rtxn, docid)
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?;
let mut document = obkv_to_json(&all_fields, &fields_ids_map, document)
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?;
// TODO definitely factorize this code
'inject_vectors: {
let embeddings = index
.embeddings(&index_rtxn, docid)
.map_err(|e| Error::from_milli(e, Some(uid.to_string())))?;
if embeddings.is_empty() {
break 'inject_vectors;
}
let vectors = document
.entry(RESERVED_VECTORS_FIELD_NAME)
.or_insert(serde_json::Value::Object(Default::default()));
let serde_json::Value::Object(vectors) = vectors else {
return Err(Error::from_milli(
milli::Error::UserError(
milli::UserError::InvalidVectorsMapType {
document_id: {
if let Ok(Some(Ok(index))) = index
.external_id_of(
&index_rtxn,
std::iter::once(docid),
)
.map(|it| it.into_iter().next())
{
index
} else {
format!("internal docid={docid}")
}
},
value: vectors.clone(),
},
),
Some(uid.to_string()),
));
};
for (embedder_name, embeddings) in embeddings {
let user_provided = embedding_configs
.iter()
.find(|conf| conf.name == embedder_name)
.is_some_and(|conf| conf.user_provided.contains(docid));
let embeddings = ExplicitVectors {
embeddings: Some(
VectorOrArrayOfVectors::from_array_of_vectors(embeddings),
),
regenerate: !user_provided,
};
vectors.insert(
embedder_name,
serde_json::to_value(embeddings).unwrap(),
);
}
}
tmp_buffer.clear();
serde_json::to_writer(&mut tmp_buffer, &document)
.map_err(milli::InternalError::from)
.map_err(|e| Error::from_milli(e.into(), Some(uid.to_string())))?;
// Make sure we put at least one document in the buffer even
// though we might go above the buffer limit before sending
if !buffer.is_empty() && buffer.len() + tmp_buffer.len() > limit {
// We compress the documents before sending them
let mut encoder =
GzEncoder::new(&mut compressed_buffer, Compression::default());
encoder
.write_all(&buffer)
.map_err(|e| Error::from_milli(e.into(), Some(uid.clone())))?;
encoder
.finish()
.map_err(|e| Error::from_milli(e.into(), Some(uid.clone())))?;
retry(&must_stop_processing, || {
let mut request = agent.post(&documents_url);
request = request.set("Content-Type", "application/x-ndjson");
request = request.set("Content-Encoding", "gzip");
if let Some(api_key) = api_key {
request = request
.set("Authorization", &(format!("Bearer {api_key}")));
}
request.send_bytes(&compressed_buffer).map_err(into_backoff_error)
})?;
buffer.clear();
compressed_buffer.clear();
}
buffer.extend_from_slice(&tmp_buffer);
if i % 100 == 0 {
step.fetch_add(100, atomic::Ordering::Relaxed);
}
}
retry(&must_stop_processing, || {
let mut request = agent.post(&documents_url);
request = request.set("Content-Type", "application/x-ndjson");
if let Some(api_key) = api_key {
request = request.set("Authorization", &(format!("Bearer {api_key}")));
}
request.send_bytes(&buffer).map_err(into_backoff_error)
})?;
Ok(())
})
.map_err(|e| {
Error::from_milli(
milli::Error::InternalError(InternalError::PanicInThreadPool(e)),
Some(uid.to_string()),
)
})?;
step.store(total_documents, atomic::Ordering::Relaxed);
}
Ok(output)
}
}
fn retry<F>(must_stop_processing: &MustStopProcessing, send_request: F) -> Result<ureq::Response>
where
F: Fn() -> Result<ureq::Response, backoff::Error<ureq::Error>>,
{
match backoff::retry(ExponentialBackoff::default(), || {
if must_stop_processing.get() {
return Err(backoff::Error::Permanent(ureq::Error::Status(
u16::MAX,
// 444: Connection Closed Without Response
Response::new(444, "Abort", "Aborted task").unwrap(),
)));
}
send_request()
}) {
Ok(response) => Ok(response),
Err(backoff::Error::Permanent(e)) => Err(ureq_error_into_error(e)),
Err(backoff::Error::Transient { err, retry_after: _ }) => Err(ureq_error_into_error(err)),
}
}
fn into_backoff_error(err: ureq::Error) -> backoff::Error<ureq::Error> {
match err {
// Those code status must trigger an automatic retry
// <https://www.restapitutorial.com/advanced/responses/retries>
ureq::Error::Status(408 | 429 | 500 | 502 | 503 | 504, _) => {
backoff::Error::Transient { err, retry_after: None }
}
ureq::Error::Status(_, _) => backoff::Error::Permanent(err),
ureq::Error::Transport(_) => backoff::Error::Transient { err, retry_after: None },
}
}
/// Converts a `ureq::Error` into an `Error`.
fn ureq_error_into_error(error: ureq::Error) -> Error {
#[derive(Deserialize)]
struct MeiliError {
message: String,
code: String,
r#type: String,
link: String,
}
match error {
// This is a workaround to handle task abortion - the error propagation path
// makes it difficult to cleanly surface the abortion at this level.
ureq::Error::Status(u16::MAX, _) => Error::AbortedTask,
ureq::Error::Status(_, response) => match response.into_json() {
Ok(MeiliError { message, code, r#type, link }) => {
Error::FromRemoteWhenExporting { message, code, r#type, link }
}
Err(e) => e.into(),
},
ureq::Error::Transport(transport) => io::Error::new(io::ErrorKind::Other, transport).into(),
}
}
enum ExportIndex {}

View file

@ -732,6 +732,7 @@ fn basic_get_stats() {
"documentDeletion": 0,
"documentEdition": 0,
"dumpCreation": 0,
"export": 0,
"indexCreation": 3,
"indexDeletion": 0,
"indexSwap": 0,
@ -765,6 +766,7 @@ fn basic_get_stats() {
"documentDeletion": 0,
"documentEdition": 0,
"dumpCreation": 0,
"export": 0,
"indexCreation": 3,
"indexDeletion": 0,
"indexSwap": 0,
@ -805,6 +807,7 @@ fn basic_get_stats() {
"documentDeletion": 0,
"documentEdition": 0,
"dumpCreation": 0,
"export": 0,
"indexCreation": 3,
"indexDeletion": 0,
"indexSwap": 0,
@ -846,6 +849,7 @@ fn basic_get_stats() {
"documentDeletion": 0,
"documentEdition": 0,
"dumpCreation": 0,
"export": 0,
"indexCreation": 3,
"indexDeletion": 0,
"indexSwap": 0,

View file

@ -37,6 +37,7 @@ pub(crate) enum FailureLocation {
InsideCreateBatch,
InsideProcessBatch,
PanicInsideProcessBatch,
ProcessExport,
ProcessUpgrade,
AcquiringWtxn,
UpdatingTaskAfterProcessBatchSuccess { task_uid: u32 },

View file

@ -278,6 +278,7 @@ pub fn swap_index_uid_in_task(task: &mut Task, swap: (&str, &str)) {
K::TaskCancelation { .. }
| K::TaskDeletion { .. }
| K::DumpCreation { .. }
| K::Export { .. }
| K::UpgradeDatabase { .. }
| K::SnapshotCreation => (),
};
@ -605,6 +606,9 @@ impl crate::IndexScheduler {
Details::Dump { dump_uid: _ } => {
assert_eq!(kind.as_kind(), Kind::DumpCreation);
}
Details::Export { url: _, api_key: _, payload_size: _, indexes: _ } => {
assert_eq!(kind.as_kind(), Kind::Export);
}
Details::UpgradeDatabase { from: _, to: _ } => {
assert_eq!(kind.as_kind(), Kind::UpgradeDatabase);
}