mirror of
https://github.com/meilisearch/meilisearch.git
synced 2024-11-22 01:57:41 +08:00
add the batch_id to the tasks
This commit is contained in:
parent
057fcb3993
commit
6062914654
@ -1,6 +1,7 @@
|
||||
#![allow(clippy::type_complexity)]
|
||||
#![allow(clippy::wrong_self_convention)]
|
||||
|
||||
use meilisearch_types::batches::BatchId;
|
||||
use meilisearch_types::error::ResponseError;
|
||||
use meilisearch_types::keys::Key;
|
||||
use meilisearch_types::milli::update::IndexDocumentsMethod;
|
||||
@ -57,6 +58,9 @@ pub enum Version {
|
||||
#[serde(rename_all = "camelCase")]
|
||||
pub struct TaskDump {
|
||||
pub uid: TaskId,
|
||||
// The batch ID were introduced in v1.12, everything prior to this version will be `None`.
|
||||
#[serde(default)]
|
||||
pub batch_uid: Option<BatchId>,
|
||||
#[serde(default)]
|
||||
pub index_uid: Option<String>,
|
||||
pub status: Status,
|
||||
@ -143,6 +147,7 @@ impl From<Task> for TaskDump {
|
||||
fn from(task: Task) -> Self {
|
||||
TaskDump {
|
||||
uid: task.uid,
|
||||
batch_uid: task.batch_uid,
|
||||
index_uid: task.index_uid().map(|uid| uid.to_string()),
|
||||
status: task.status,
|
||||
kind: task.kind.into(),
|
||||
@ -297,6 +302,7 @@ pub(crate) mod test {
|
||||
(
|
||||
TaskDump {
|
||||
uid: 0,
|
||||
batch_uid: Some(0),
|
||||
index_uid: Some(S("doggo")),
|
||||
status: Status::Succeeded,
|
||||
kind: KindDump::DocumentImport {
|
||||
@ -320,6 +326,7 @@ pub(crate) mod test {
|
||||
(
|
||||
TaskDump {
|
||||
uid: 1,
|
||||
batch_uid: None,
|
||||
index_uid: Some(S("doggo")),
|
||||
status: Status::Enqueued,
|
||||
kind: KindDump::DocumentImport {
|
||||
@ -346,6 +353,7 @@ pub(crate) mod test {
|
||||
(
|
||||
TaskDump {
|
||||
uid: 5,
|
||||
batch_uid: None,
|
||||
index_uid: Some(S("catto")),
|
||||
status: Status::Enqueued,
|
||||
kind: KindDump::IndexDeletion,
|
||||
|
@ -70,6 +70,7 @@ impl CompatV5ToV6 {
|
||||
|
||||
let task = v6::Task {
|
||||
uid: task_view.uid,
|
||||
batch_uid: None,
|
||||
index_uid: task_view.index_uid,
|
||||
status: match task_view.status {
|
||||
v5::Status::Enqueued => v6::Status::Enqueued,
|
||||
|
@ -24,6 +24,7 @@ use std::fs::{self, File};
|
||||
use std::io::BufWriter;
|
||||
|
||||
use dump::IndexMetadata;
|
||||
use meilisearch_types::batches::BatchId;
|
||||
use meilisearch_types::error::Code;
|
||||
use meilisearch_types::heed::{RoTxn, RwTxn};
|
||||
use meilisearch_types::milli::documents::{obkv_to_object, DocumentsBatchReader};
|
||||
@ -279,18 +280,22 @@ impl IndexScheduler {
|
||||
rtxn: &RoTxn,
|
||||
index_uid: String,
|
||||
batch: BatchKind,
|
||||
batch_id: BatchId,
|
||||
must_create_index: bool,
|
||||
) -> Result<Option<Batch>> {
|
||||
match batch {
|
||||
BatchKind::DocumentClear { ids } => Ok(Some(Batch::IndexOperation {
|
||||
op: IndexOperation::DocumentClear {
|
||||
tasks: self.get_existing_tasks(rtxn, ids)?,
|
||||
tasks: self.get_existing_tasks_with_batch_id(rtxn, batch_id, ids)?,
|
||||
index_uid,
|
||||
},
|
||||
must_create_index,
|
||||
})),
|
||||
BatchKind::DocumentEdition { id } => {
|
||||
let task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?;
|
||||
let task = self
|
||||
.get_task(rtxn, id)?
|
||||
.ok_or(Error::CorruptedTaskQueue)?
|
||||
.with_batch_id(batch_id);
|
||||
match &task.kind {
|
||||
KindWithContent::DocumentEdition { index_uid, .. } => {
|
||||
Ok(Some(Batch::IndexOperation {
|
||||
@ -305,7 +310,7 @@ impl IndexScheduler {
|
||||
}
|
||||
}
|
||||
BatchKind::DocumentOperation { method, operation_ids, .. } => {
|
||||
let tasks = self.get_existing_tasks(rtxn, operation_ids)?;
|
||||
let tasks = self.get_existing_tasks_with_batch_id(rtxn, batch_id, operation_ids)?;
|
||||
let primary_key = tasks
|
||||
.iter()
|
||||
.find_map(|task| match task.kind {
|
||||
@ -352,7 +357,7 @@ impl IndexScheduler {
|
||||
}))
|
||||
}
|
||||
BatchKind::DocumentDeletion { deletion_ids, includes_by_filter: _ } => {
|
||||
let tasks = self.get_existing_tasks(rtxn, deletion_ids)?;
|
||||
let tasks = self.get_existing_tasks_with_batch_id(rtxn, batch_id, deletion_ids)?;
|
||||
|
||||
Ok(Some(Batch::IndexOperation {
|
||||
op: IndexOperation::DocumentDeletion { index_uid, tasks },
|
||||
@ -360,7 +365,7 @@ impl IndexScheduler {
|
||||
}))
|
||||
}
|
||||
BatchKind::Settings { settings_ids, .. } => {
|
||||
let tasks = self.get_existing_tasks(rtxn, settings_ids)?;
|
||||
let tasks = self.get_existing_tasks_with_batch_id(rtxn, batch_id, settings_ids)?;
|
||||
|
||||
let mut settings = Vec::new();
|
||||
for task in &tasks {
|
||||
@ -383,6 +388,7 @@ impl IndexScheduler {
|
||||
rtxn,
|
||||
index_uid,
|
||||
BatchKind::Settings { settings_ids, allow_index_creation },
|
||||
batch_id,
|
||||
must_create_index,
|
||||
)?
|
||||
.unwrap()
|
||||
@ -398,6 +404,7 @@ impl IndexScheduler {
|
||||
rtxn,
|
||||
index_uid,
|
||||
BatchKind::DocumentClear { ids: other },
|
||||
batch_id,
|
||||
must_create_index,
|
||||
)?
|
||||
.unwrap()
|
||||
@ -430,6 +437,7 @@ impl IndexScheduler {
|
||||
rtxn,
|
||||
index_uid.clone(),
|
||||
BatchKind::Settings { settings_ids, allow_index_creation },
|
||||
batch_id,
|
||||
must_create_index,
|
||||
)?;
|
||||
|
||||
@ -442,6 +450,7 @@ impl IndexScheduler {
|
||||
primary_key,
|
||||
operation_ids,
|
||||
},
|
||||
batch_id,
|
||||
must_create_index,
|
||||
)?;
|
||||
|
||||
@ -479,7 +488,10 @@ impl IndexScheduler {
|
||||
}
|
||||
}
|
||||
BatchKind::IndexCreation { id } => {
|
||||
let task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?;
|
||||
let task = self
|
||||
.get_task(rtxn, id)?
|
||||
.ok_or(Error::CorruptedTaskQueue)?
|
||||
.with_batch_id(batch_id);
|
||||
let (index_uid, primary_key) = match &task.kind {
|
||||
KindWithContent::IndexCreation { index_uid, primary_key } => {
|
||||
(index_uid.clone(), primary_key.clone())
|
||||
@ -489,7 +501,10 @@ impl IndexScheduler {
|
||||
Ok(Some(Batch::IndexCreation { index_uid, primary_key, task }))
|
||||
}
|
||||
BatchKind::IndexUpdate { id } => {
|
||||
let task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?;
|
||||
let task = self
|
||||
.get_task(rtxn, id)?
|
||||
.ok_or(Error::CorruptedTaskQueue)?
|
||||
.with_batch_id(batch_id);
|
||||
let primary_key = match &task.kind {
|
||||
KindWithContent::IndexUpdate { primary_key, .. } => primary_key.clone(),
|
||||
_ => unreachable!(),
|
||||
@ -499,10 +514,13 @@ impl IndexScheduler {
|
||||
BatchKind::IndexDeletion { ids } => Ok(Some(Batch::IndexDeletion {
|
||||
index_uid,
|
||||
index_has_been_created: must_create_index,
|
||||
tasks: self.get_existing_tasks(rtxn, ids)?,
|
||||
tasks: self.get_existing_tasks_with_batch_id(rtxn, batch_id, ids)?,
|
||||
})),
|
||||
BatchKind::IndexSwap { id } => {
|
||||
let task = self.get_task(rtxn, id)?.ok_or(Error::CorruptedTaskQueue)?;
|
||||
let task = self
|
||||
.get_task(rtxn, id)?
|
||||
.ok_or(Error::CorruptedTaskQueue)?
|
||||
.with_batch_id(batch_id);
|
||||
Ok(Some(Batch::IndexSwap { task }))
|
||||
}
|
||||
}
|
||||
@ -515,10 +533,11 @@ impl IndexScheduler {
|
||||
/// 4. We get the *next* dump to process.
|
||||
/// 5. 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, rtxn: &RoTxn) -> Result<Option<Batch>> {
|
||||
pub(crate) fn create_next_batch(&self, rtxn: &RoTxn) -> Result<Option<(Batch, BatchId)>> {
|
||||
#[cfg(test)]
|
||||
self.maybe_fail(crate::tests::FailureLocation::InsideCreateBatch)?;
|
||||
|
||||
let batch_id = self.next_batch_id(rtxn)?;
|
||||
let enqueued = &self.get_status(rtxn, Status::Enqueued)?;
|
||||
let to_cancel = self.get_kind(rtxn, Kind::TaskCancelation)? & enqueued;
|
||||
|
||||
@ -526,39 +545,65 @@ impl IndexScheduler {
|
||||
if let Some(task_id) = to_cancel.max() {
|
||||
// We retrieve the tasks that were processing before this tasks cancelation started.
|
||||
// We must *not* reset the processing tasks before calling this method.
|
||||
let ProcessingTasks { started_at, processing } =
|
||||
// Displaying the `batch_id` would make a strange error message since this task cancelation is going to
|
||||
// replace the canceled batch. It's better to avoid mentioning it in the error message.
|
||||
let ProcessingTasks { started_at, batch_id: _, processing } =
|
||||
&*self.processing_tasks.read().unwrap();
|
||||
return Ok(Some(Batch::TaskCancelation {
|
||||
task: self.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?,
|
||||
previous_started_at: *started_at,
|
||||
previous_processing_tasks: processing.clone(),
|
||||
}));
|
||||
return Ok(Some((
|
||||
Batch::TaskCancelation {
|
||||
task: self
|
||||
.get_task(rtxn, task_id)?
|
||||
.ok_or(Error::CorruptedTaskQueue)?
|
||||
.with_batch_id(batch_id),
|
||||
previous_started_at: *started_at,
|
||||
previous_processing_tasks: processing.clone(),
|
||||
},
|
||||
batch_id,
|
||||
)));
|
||||
}
|
||||
|
||||
// 2. we get the next task to delete
|
||||
let to_delete = self.get_kind(rtxn, Kind::TaskDeletion)? & enqueued;
|
||||
if !to_delete.is_empty() {
|
||||
let tasks = self.get_existing_tasks(rtxn, to_delete)?;
|
||||
return Ok(Some(Batch::TaskDeletions(tasks)));
|
||||
let tasks = self
|
||||
.get_existing_tasks(rtxn, to_delete)?
|
||||
.into_iter()
|
||||
.map(|task| task.with_batch_id(batch_id))
|
||||
.collect();
|
||||
return Ok(Some((Batch::TaskDeletions(tasks), batch_id)));
|
||||
}
|
||||
|
||||
// 3. we batch the snapshot.
|
||||
let to_snapshot = self.get_kind(rtxn, Kind::SnapshotCreation)? & enqueued;
|
||||
if !to_snapshot.is_empty() {
|
||||
return Ok(Some(Batch::SnapshotCreation(self.get_existing_tasks(rtxn, to_snapshot)?)));
|
||||
return Ok(Some((
|
||||
Batch::SnapshotCreation(
|
||||
self.get_existing_tasks(rtxn, to_snapshot)?
|
||||
.into_iter()
|
||||
.map(|task| task.with_batch_id(batch_id))
|
||||
.collect(),
|
||||
),
|
||||
batch_id,
|
||||
)));
|
||||
}
|
||||
|
||||
// 4. we batch the dumps.
|
||||
let to_dump = self.get_kind(rtxn, Kind::DumpCreation)? & enqueued;
|
||||
if let Some(to_dump) = to_dump.min() {
|
||||
return Ok(Some(Batch::Dump(
|
||||
self.get_task(rtxn, to_dump)?.ok_or(Error::CorruptedTaskQueue)?,
|
||||
return Ok(Some((
|
||||
Batch::Dump(
|
||||
self.get_task(rtxn, to_dump)?
|
||||
.ok_or(Error::CorruptedTaskQueue)?
|
||||
.with_batch_id(batch_id),
|
||||
),
|
||||
batch_id,
|
||||
)));
|
||||
}
|
||||
|
||||
// 5. 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 task = self.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?;
|
||||
let task =
|
||||
self.get_task(rtxn, task_id)?.ok_or(Error::CorruptedTaskQueue)?.with_batch_id(batch_id);
|
||||
|
||||
// If the task is not associated with any index, verify that it is an index swap and
|
||||
// create the batch directly. Otherwise, get the index name associated with the task
|
||||
@ -568,7 +613,7 @@ impl IndexScheduler {
|
||||
index_name
|
||||
} else {
|
||||
assert!(matches!(&task.kind, KindWithContent::IndexSwap { swaps } if swaps.is_empty()));
|
||||
return Ok(Some(Batch::IndexSwap { task }));
|
||||
return Ok(Some((Batch::IndexSwap { task }, batch_id)));
|
||||
};
|
||||
|
||||
let index_already_exists = self.index_mapper.exists(rtxn, index_name)?;
|
||||
@ -599,12 +644,15 @@ impl IndexScheduler {
|
||||
if let Some((batchkind, create_index)) =
|
||||
autobatcher::autobatch(enqueued, index_already_exists, primary_key.as_deref())
|
||||
{
|
||||
return self.create_next_batch_index(
|
||||
rtxn,
|
||||
index_name.to_string(),
|
||||
batchkind,
|
||||
create_index,
|
||||
);
|
||||
return Ok(self
|
||||
.create_next_batch_index(
|
||||
rtxn,
|
||||
index_name.to_string(),
|
||||
batchkind,
|
||||
batch_id,
|
||||
create_index,
|
||||
)?
|
||||
.map(|batch| (batch, batch_id)));
|
||||
}
|
||||
|
||||
// If we found no tasks then we were notified for something that got autobatched
|
||||
|
@ -79,7 +79,9 @@ pub enum Error {
|
||||
)]
|
||||
InvalidTaskDate { field: DateField, date: String },
|
||||
#[error("Task uid `{task_uid}` is invalid. It should only contain numeric characters.")]
|
||||
InvalidTaskUids { task_uid: String },
|
||||
InvalidTaskUid { task_uid: String },
|
||||
#[error("Batch uid `{batch_uid}` is invalid. It should only contain numeric characters.")]
|
||||
InvalidBatchUid { batch_uid: String },
|
||||
#[error(
|
||||
"Task status `{status}` is invalid. Available task statuses are {}.",
|
||||
enum_iterator::all::<Status>()
|
||||
@ -172,7 +174,8 @@ impl Error {
|
||||
| Error::SwapIndexesNotFound(_)
|
||||
| Error::CorruptedDump
|
||||
| Error::InvalidTaskDate { .. }
|
||||
| Error::InvalidTaskUids { .. }
|
||||
| Error::InvalidTaskUid { .. }
|
||||
| Error::InvalidBatchUid { .. }
|
||||
| Error::InvalidTaskStatuses { .. }
|
||||
| Error::InvalidTaskTypes { .. }
|
||||
| Error::InvalidTaskCanceledBy { .. }
|
||||
@ -216,7 +219,8 @@ impl ErrorCode for Error {
|
||||
Error::SwapIndexNotFound(_) => Code::IndexNotFound,
|
||||
Error::SwapIndexesNotFound(_) => Code::IndexNotFound,
|
||||
Error::InvalidTaskDate { field, .. } => (*field).into(),
|
||||
Error::InvalidTaskUids { .. } => Code::InvalidTaskUids,
|
||||
Error::InvalidTaskUid { .. } => Code::InvalidTaskUids,
|
||||
Error::InvalidBatchUid { .. } => Code::InvalidBatchUids,
|
||||
Error::InvalidTaskStatuses { .. } => Code::InvalidTaskStatuses,
|
||||
Error::InvalidTaskTypes { .. } => Code::InvalidTaskTypes,
|
||||
Error::InvalidTaskCanceledBy { .. } => Code::InvalidTaskCanceledBy,
|
||||
|
@ -24,6 +24,8 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String {
|
||||
file_store,
|
||||
env,
|
||||
all_tasks,
|
||||
all_batches,
|
||||
// task reverse index
|
||||
status,
|
||||
kind,
|
||||
index_tasks,
|
||||
@ -31,6 +33,16 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String {
|
||||
enqueued_at,
|
||||
started_at,
|
||||
finished_at,
|
||||
|
||||
// batch reverse index
|
||||
batch_status,
|
||||
batch_kind,
|
||||
batch_index_tasks,
|
||||
batch_canceled_by,
|
||||
batch_enqueued_at,
|
||||
batch_started_at,
|
||||
batch_finished_at,
|
||||
|
||||
index_mapper,
|
||||
features: _,
|
||||
max_number_of_tasks: _,
|
||||
@ -145,6 +157,7 @@ pub fn snapshot_task(task: &Task) -> String {
|
||||
let mut snap = String::new();
|
||||
let Task {
|
||||
uid,
|
||||
batch_uid,
|
||||
enqueued_at: _,
|
||||
started_at: _,
|
||||
finished_at: _,
|
||||
@ -156,6 +169,9 @@ pub fn snapshot_task(task: &Task) -> String {
|
||||
} = task;
|
||||
snap.push('{');
|
||||
snap.push_str(&format!("uid: {uid}, "));
|
||||
if let Some(batch_uid) = batch_uid {
|
||||
snap.push_str(&format!("batch_uid: {batch_uid}, "));
|
||||
}
|
||||
snap.push_str(&format!("status: {status}, "));
|
||||
if let Some(canceled_by) = canceled_by {
|
||||
snap.push_str(&format!("canceled_by: {canceled_by}, "));
|
||||
|
@ -48,6 +48,7 @@ pub use features::RoFeatures;
|
||||
use file_store::FileStore;
|
||||
use flate2::bufread::GzEncoder;
|
||||
use flate2::Compression;
|
||||
use meilisearch_types::batches::{Batch, BatchId};
|
||||
use meilisearch_types::error::ResponseError;
|
||||
use meilisearch_types::features::{InstanceTogglableFeatures, RuntimeTogglableFeatures};
|
||||
use meilisearch_types::heed::byteorder::BE;
|
||||
@ -162,6 +163,8 @@ impl Query {
|
||||
struct ProcessingTasks {
|
||||
/// The date and time at which the indexation started.
|
||||
started_at: OffsetDateTime,
|
||||
/// The id of the batch processing
|
||||
batch_id: Option<BatchId>,
|
||||
/// The list of tasks ids that are currently running.
|
||||
processing: RoaringBitmap,
|
||||
}
|
||||
@ -169,17 +172,28 @@ struct ProcessingTasks {
|
||||
impl ProcessingTasks {
|
||||
/// Creates an empty `ProcessingAt` struct.
|
||||
fn new() -> ProcessingTasks {
|
||||
ProcessingTasks { started_at: OffsetDateTime::now_utc(), processing: RoaringBitmap::new() }
|
||||
ProcessingTasks {
|
||||
started_at: OffsetDateTime::now_utc(),
|
||||
batch_id: None,
|
||||
processing: RoaringBitmap::new(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Stores the currently processing tasks, and the date time at which it started.
|
||||
fn start_processing_at(&mut self, started_at: OffsetDateTime, processing: RoaringBitmap) {
|
||||
fn start_processing(
|
||||
&mut self,
|
||||
started_at: OffsetDateTime,
|
||||
batch_id: BatchId,
|
||||
processing: RoaringBitmap,
|
||||
) {
|
||||
self.started_at = started_at;
|
||||
self.batch_id = Some(batch_id);
|
||||
self.processing = processing;
|
||||
}
|
||||
|
||||
/// Set the processing tasks to an empty list
|
||||
fn stop_processing(&mut self) -> RoaringBitmap {
|
||||
self.batch_id = None;
|
||||
std::mem::take(&mut self.processing)
|
||||
}
|
||||
|
||||
@ -209,6 +223,7 @@ impl MustStopProcessing {
|
||||
/// Database const names for the `IndexScheduler`.
|
||||
mod db_name {
|
||||
pub const ALL_TASKS: &str = "all-tasks";
|
||||
pub const ALL_BATCHES: &str = "all-batches";
|
||||
pub const STATUS: &str = "status";
|
||||
pub const KIND: &str = "kind";
|
||||
pub const INDEX_TASKS: &str = "index-tasks";
|
||||
@ -216,6 +231,14 @@ mod db_name {
|
||||
pub const ENQUEUED_AT: &str = "enqueued-at";
|
||||
pub const STARTED_AT: &str = "started-at";
|
||||
pub const FINISHED_AT: &str = "finished-at";
|
||||
|
||||
pub const BATCH_STATUS: &str = "batch-status";
|
||||
pub const BATCH_KIND: &str = "batch-kind";
|
||||
pub const BATCH_INDEX_TASKS: &str = "batch-index-tasks";
|
||||
pub const BATCH_CANCELED_BY: &str = "batch-canceled_by";
|
||||
pub const BATCH_ENQUEUED_AT: &str = "batch-enqueued-at";
|
||||
pub const BATCH_STARTED_AT: &str = "batch-started-at";
|
||||
pub const BATCH_FINISHED_AT: &str = "batch-finished-at";
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
@ -300,6 +323,28 @@ pub struct IndexScheduler {
|
||||
// The main database, it contains all the tasks accessible by their Id.
|
||||
pub(crate) all_tasks: Database<BEU32, SerdeJson<Task>>,
|
||||
|
||||
// Contains all the batches accessible by their Id.
|
||||
pub(crate) all_batches: Database<BEU32, SerdeJson<Batch>>,
|
||||
|
||||
/// All the batches containing a task matching the selected status.
|
||||
pub(crate) batch_status: Database<SerdeBincode<Status>, RoaringBitmapCodec>,
|
||||
/// All the batches ids grouped by the kind of their task.
|
||||
pub(crate) batch_kind: Database<SerdeBincode<Kind>, RoaringBitmapCodec>,
|
||||
/// Store the batches associated to an index.
|
||||
pub(crate) batch_index_tasks: Database<Str, RoaringBitmapCodec>,
|
||||
|
||||
/// Store the batches containing a task canceled by a task uid
|
||||
pub(crate) batch_canceled_by: Database<BEU32, RoaringBitmapCodec>,
|
||||
|
||||
/// Store the batches containing tasks which were enqueued at a specific date
|
||||
pub(crate) batch_enqueued_at: Database<BEI128, CboRoaringBitmapCodec>,
|
||||
|
||||
/// Store the batches containing finished tasks started at a specific date
|
||||
pub(crate) batch_started_at: Database<BEI128, CboRoaringBitmapCodec>,
|
||||
|
||||
/// Store the batches containing tasks finished at a specific date
|
||||
pub(crate) batch_finished_at: Database<BEI128, CboRoaringBitmapCodec>,
|
||||
|
||||
/// All the tasks ids grouped by their status.
|
||||
// TODO we should not be able to serialize a `Status::Processing` in this database.
|
||||
pub(crate) status: Database<SerdeBincode<Status>, RoaringBitmapCodec>,
|
||||
@ -388,6 +433,9 @@ impl IndexScheduler {
|
||||
processing_tasks: self.processing_tasks.clone(),
|
||||
file_store: self.file_store.clone(),
|
||||
all_tasks: self.all_tasks,
|
||||
all_batches: self.all_batches,
|
||||
|
||||
// Tasks reverse index
|
||||
status: self.status,
|
||||
kind: self.kind,
|
||||
index_tasks: self.index_tasks,
|
||||
@ -395,6 +443,16 @@ impl IndexScheduler {
|
||||
enqueued_at: self.enqueued_at,
|
||||
started_at: self.started_at,
|
||||
finished_at: self.finished_at,
|
||||
|
||||
// Batches reverse index
|
||||
batch_status: self.batch_status,
|
||||
batch_kind: self.batch_kind,
|
||||
batch_index_tasks: self.batch_index_tasks,
|
||||
batch_canceled_by: self.batch_canceled_by,
|
||||
batch_enqueued_at: self.batch_enqueued_at,
|
||||
batch_started_at: self.batch_started_at,
|
||||
batch_finished_at: self.batch_finished_at,
|
||||
|
||||
index_mapper: self.index_mapper.clone(),
|
||||
wake_up: self.wake_up.clone(),
|
||||
autobatching_enabled: self.autobatching_enabled,
|
||||
@ -454,7 +512,7 @@ impl IndexScheduler {
|
||||
|
||||
let env = unsafe {
|
||||
heed::EnvOpenOptions::new()
|
||||
.max_dbs(11)
|
||||
.max_dbs(19)
|
||||
.map_size(budget.task_db_size)
|
||||
.open(options.tasks_path)
|
||||
}?;
|
||||
@ -465,6 +523,7 @@ impl IndexScheduler {
|
||||
|
||||
let mut wtxn = env.write_txn()?;
|
||||
let all_tasks = env.create_database(&mut wtxn, Some(db_name::ALL_TASKS))?;
|
||||
let all_batches = env.create_database(&mut wtxn, Some(db_name::ALL_BATCHES))?;
|
||||
let status = env.create_database(&mut wtxn, Some(db_name::STATUS))?;
|
||||
let kind = env.create_database(&mut wtxn, Some(db_name::KIND))?;
|
||||
let index_tasks = env.create_database(&mut wtxn, Some(db_name::INDEX_TASKS))?;
|
||||
@ -472,6 +531,14 @@ impl IndexScheduler {
|
||||
let enqueued_at = env.create_database(&mut wtxn, Some(db_name::ENQUEUED_AT))?;
|
||||
let started_at = env.create_database(&mut wtxn, Some(db_name::STARTED_AT))?;
|
||||
let finished_at = env.create_database(&mut wtxn, Some(db_name::FINISHED_AT))?;
|
||||
|
||||
let batch_status = env.create_database(&mut wtxn, Some(db_name::STATUS))?;
|
||||
let batch_kind = env.create_database(&mut wtxn, Some(db_name::KIND))?;
|
||||
let batch_index_tasks = env.create_database(&mut wtxn, Some(db_name::INDEX_TASKS))?;
|
||||
let batch_canceled_by = env.create_database(&mut wtxn, Some(db_name::CANCELED_BY))?;
|
||||
let batch_enqueued_at = env.create_database(&mut wtxn, Some(db_name::ENQUEUED_AT))?;
|
||||
let batch_started_at = env.create_database(&mut wtxn, Some(db_name::STARTED_AT))?;
|
||||
let batch_finished_at = env.create_database(&mut wtxn, Some(db_name::FINISHED_AT))?;
|
||||
wtxn.commit()?;
|
||||
|
||||
// allow unreachable_code to get rids of the warning in the case of a test build.
|
||||
@ -480,6 +547,8 @@ impl IndexScheduler {
|
||||
processing_tasks: Arc::new(RwLock::new(ProcessingTasks::new())),
|
||||
file_store,
|
||||
all_tasks,
|
||||
all_batches,
|
||||
// Task reverse indexes
|
||||
status,
|
||||
kind,
|
||||
index_tasks,
|
||||
@ -487,6 +556,16 @@ impl IndexScheduler {
|
||||
enqueued_at,
|
||||
started_at,
|
||||
finished_at,
|
||||
|
||||
// Batch reverse indexes
|
||||
batch_status,
|
||||
batch_kind,
|
||||
batch_index_tasks,
|
||||
batch_canceled_by,
|
||||
batch_enqueued_at,
|
||||
batch_started_at,
|
||||
batch_finished_at,
|
||||
|
||||
index_mapper: IndexMapper::new(
|
||||
&env,
|
||||
options.indexes_path,
|
||||
@ -946,6 +1025,52 @@ impl IndexScheduler {
|
||||
Ok((tasks, total_tasks.len()))
|
||||
}
|
||||
|
||||
/// Return the batch ids matching the query along with the total number of batches
|
||||
/// by ignoring the from and limit parameters from the user's point of view.
|
||||
///
|
||||
/// There are two differences between an internal query and a query executed by
|
||||
/// the user.
|
||||
///
|
||||
/// 1. IndexSwap tasks are not publicly associated with any index, but they are associated
|
||||
/// with many indexes internally.
|
||||
/// 2. The user may not have the rights to access the tasks (internally) associated with all indexes.
|
||||
pub fn get_task_ids_from_authorized_indexes(
|
||||
&self,
|
||||
rtxn: &RoTxn,
|
||||
query: &Query,
|
||||
filters: &meilisearch_auth::AuthFilter,
|
||||
) -> Result<(RoaringBitmap, u64)> {
|
||||
// compute all batches matching the filter by ignoring the limits, to find the number of batches matching
|
||||
// the filter.
|
||||
// As this causes us to compute the filter twice it is slightly inefficient, but doing it this way spares
|
||||
// us from modifying the underlying implementation, and the performance remains sufficient.
|
||||
// Should this change, we would modify `get_batch_ids` to directly return the number of matching batches.
|
||||
let total_batches = self.get_batch_ids(rtxn, &query.clone().without_limits())?;
|
||||
let mut batches = self.get_batch_ids(rtxn, query)?;
|
||||
|
||||
// If the query contains a list of index uid or there is a finite list of authorized indexes,
|
||||
// then we must exclude all the kinds that aren't associated to one and only one index.
|
||||
if query.index_uids.is_some() || !filters.all_indexes_authorized() {
|
||||
for kind in enum_iterator::all::<Kind>().filter(|kind| !kind.related_to_one_index()) {
|
||||
batches -= self.get_kind(rtxn, kind)?;
|
||||
}
|
||||
}
|
||||
|
||||
// Any task that is internally associated with a non-authorized index
|
||||
// must be discarded.
|
||||
if !filters.all_indexes_authorized() {
|
||||
let all_indexes_iter = self.index_tasks.iter(rtxn)?;
|
||||
for result in all_indexes_iter {
|
||||
let (index, index_tasks) = result?;
|
||||
if !filters.is_index_authorized(index) {
|
||||
tasks -= index_tasks;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok((tasks, total_tasks.len()))
|
||||
}
|
||||
|
||||
/// Return the tasks matching the query from the user's point of view along
|
||||
/// with the total number of tasks matching the query, ignoring from and limit.
|
||||
///
|
||||
@ -971,7 +1096,7 @@ impl IndexScheduler {
|
||||
let tasks =
|
||||
self.get_existing_tasks(&rtxn, tasks.take(query.limit.unwrap_or(u32::MAX) as usize))?;
|
||||
|
||||
let ProcessingTasks { started_at, processing, .. } =
|
||||
let ProcessingTasks { started_at, batch_id, processing } =
|
||||
self.processing_tasks.read().map_err(|_| Error::CorruptedTaskQueue)?.clone();
|
||||
|
||||
let ret = tasks.into_iter();
|
||||
@ -981,7 +1106,60 @@ impl IndexScheduler {
|
||||
Ok((
|
||||
ret.map(|task| {
|
||||
if processing.contains(task.uid) {
|
||||
Task { status: Status::Processing, started_at: Some(started_at), ..task }
|
||||
Task {
|
||||
status: Status::Processing,
|
||||
batch_uid: batch_id,
|
||||
started_at: Some(started_at),
|
||||
..task
|
||||
}
|
||||
} else {
|
||||
task
|
||||
}
|
||||
})
|
||||
.collect(),
|
||||
total,
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
/// Return the batches matching the query from the user's point of view along
|
||||
/// with the total number of batches matching the query, ignoring from and limit.
|
||||
///
|
||||
/// There are two differences between an internal query and a query executed by
|
||||
/// the user.
|
||||
///
|
||||
/// 1. IndexSwap tasks are not publicly associated with any index, but they are associated
|
||||
/// with many indexes internally.
|
||||
/// 2. The user may not have the rights to access the tasks (internally) associated with all indexes.
|
||||
pub fn get_batches_from_authorized_indexes(
|
||||
&self,
|
||||
query: Query,
|
||||
filters: &meilisearch_auth::AuthFilter,
|
||||
) -> Result<(Vec<Batch>, u64)> {
|
||||
let rtxn = self.env.read_txn()?;
|
||||
|
||||
let (tasks, total) = self.get_batch_ids_from_authorized_indexes(&rtxn, &query, filters)?;
|
||||
let tasks = self.get_existing_batches(
|
||||
&rtxn,
|
||||
tasks.into_iter().rev().take(query.limit.unwrap_or(u32::MAX) as usize),
|
||||
)?;
|
||||
|
||||
let ProcessingTasks { started_at, batch_id, processing } =
|
||||
self.processing_tasks.read().map_err(|_| Error::CorruptedTaskQueue)?.clone();
|
||||
|
||||
let ret = tasks.into_iter();
|
||||
if processing.is_empty() {
|
||||
Ok((ret.collect(), total))
|
||||
} else {
|
||||
Ok((
|
||||
ret.map(|task| {
|
||||
if processing.contains(task.uid) {
|
||||
Task {
|
||||
status: Status::Processing,
|
||||
batch_uid: batch_id,
|
||||
started_at: Some(started_at),
|
||||
..task
|
||||
}
|
||||
} else {
|
||||
task
|
||||
}
|
||||
@ -1020,6 +1198,8 @@ impl IndexScheduler {
|
||||
|
||||
let mut task = Task {
|
||||
uid: task_id.unwrap_or(next_task_id),
|
||||
// The batch is defined once we starts processing the task
|
||||
batch_uid: None,
|
||||
enqueued_at: OffsetDateTime::now_utc(),
|
||||
started_at: None,
|
||||
finished_at: None,
|
||||
@ -1155,7 +1335,7 @@ impl IndexScheduler {
|
||||
}
|
||||
|
||||
let rtxn = self.env.read_txn().map_err(Error::HeedTransaction)?;
|
||||
let batch =
|
||||
let (batch, batch_id) =
|
||||
match self.create_next_batch(&rtxn).map_err(|e| Error::CreateBatch(Box::new(e)))? {
|
||||
Some(batch) => batch,
|
||||
None => return Ok(TickOutcome::WaitForSignal),
|
||||
@ -1170,7 +1350,7 @@ impl IndexScheduler {
|
||||
|
||||
// We reset the must_stop flag to be sure that we don't stop processing tasks
|
||||
self.must_stop_processing.reset();
|
||||
self.processing_tasks.write().unwrap().start_processing_at(started_at, ids.clone());
|
||||
self.processing_tasks.write().unwrap().start_processing(started_at, batch_id, ids.clone());
|
||||
|
||||
#[cfg(test)]
|
||||
self.breakpoint(Breakpoint::BatchCreated);
|
||||
@ -1268,7 +1448,8 @@ impl IndexScheduler {
|
||||
let mut task = self
|
||||
.get_task(&wtxn, id)
|
||||
.map_err(|e| Error::TaskDatabaseUpdate(Box::new(e)))?
|
||||
.ok_or(Error::CorruptedTaskQueue)?;
|
||||
.ok_or(Error::CorruptedTaskQueue)?
|
||||
.with_batch_id(batch_id);
|
||||
task.started_at = Some(started_at);
|
||||
task.finished_at = Some(finished_at);
|
||||
task.status = Status::Failed;
|
||||
@ -1286,6 +1467,12 @@ impl IndexScheduler {
|
||||
}
|
||||
}
|
||||
|
||||
self.all_batches.put(
|
||||
&mut wtxn,
|
||||
&batch_id,
|
||||
&Batch { uid: batch_id, started_at, finished_at: Some(finished_at) },
|
||||
)?;
|
||||
|
||||
let processed = self.processing_tasks.write().unwrap().stop_processing();
|
||||
|
||||
#[cfg(test)]
|
||||
@ -1600,6 +1787,7 @@ impl<'a> Dump<'a> {
|
||||
|
||||
let task = Task {
|
||||
uid: task.uid,
|
||||
batch_uid: task.batch_uid,
|
||||
enqueued_at: task.enqueued_at,
|
||||
started_at: task.started_at,
|
||||
finished_at: task.finished_at,
|
||||
|
@ -3,6 +3,7 @@
|
||||
use std::collections::{BTreeSet, HashSet};
|
||||
use std::ops::Bound;
|
||||
|
||||
use meilisearch_types::batches::BatchId;
|
||||
use meilisearch_types::heed::types::DecodeIgnore;
|
||||
use meilisearch_types::heed::{Database, RoTxn, RwTxn};
|
||||
use meilisearch_types::milli::CboRoaringBitmapCodec;
|
||||
@ -25,10 +26,37 @@ impl IndexScheduler {
|
||||
Ok(self.last_task_id(rtxn)?.unwrap_or_default())
|
||||
}
|
||||
|
||||
pub(crate) fn next_batch_id(&self, rtxn: &RoTxn) -> Result<BatchId> {
|
||||
Ok(self
|
||||
.all_batches
|
||||
.remap_data_type::<DecodeIgnore>()
|
||||
.last(rtxn)?
|
||||
.map(|(k, _)| k + 1)
|
||||
.unwrap_or_default())
|
||||
}
|
||||
|
||||
pub(crate) fn get_task(&self, rtxn: &RoTxn, task_id: TaskId) -> Result<Option<Task>> {
|
||||
Ok(self.all_tasks.get(rtxn, &task_id)?)
|
||||
}
|
||||
|
||||
/// Convert an iterator to a `Vec` of tasks. The tasks MUST exist or a
|
||||
/// `CorruptedTaskQueue` error will be throwed.
|
||||
pub(crate) fn get_existing_tasks_with_batch_id(
|
||||
&self,
|
||||
rtxn: &RoTxn,
|
||||
batch_id: BatchId,
|
||||
tasks: impl IntoIterator<Item = TaskId>,
|
||||
) -> Result<Vec<Task>> {
|
||||
tasks
|
||||
.into_iter()
|
||||
.map(|task_id| {
|
||||
self.get_task(rtxn, task_id)
|
||||
.and_then(|task| task.ok_or(Error::CorruptedTaskQueue))
|
||||
.map(|task| task.with_batch_id(batch_id))
|
||||
})
|
||||
.collect::<Result<_>>()
|
||||
}
|
||||
|
||||
/// Convert an iterator to a `Vec` of tasks. The tasks MUST exist or a
|
||||
/// `CorruptedTaskQueue` error will be throwed.
|
||||
pub(crate) fn get_existing_tasks(
|
||||
@ -342,6 +370,8 @@ impl IndexScheduler {
|
||||
|
||||
let Task {
|
||||
uid,
|
||||
/// We should iterate over the list of batch to ensure this task is effectively in the right batch
|
||||
batch_uid,
|
||||
enqueued_at,
|
||||
started_at,
|
||||
finished_at,
|
||||
|
18
crates/meilisearch-types/src/batches.rs
Normal file
18
crates/meilisearch-types/src/batches.rs
Normal file
@ -0,0 +1,18 @@
|
||||
use serde::{Deserialize, Serialize};
|
||||
use time::OffsetDateTime;
|
||||
|
||||
pub type BatchId = u32;
|
||||
|
||||
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
pub struct Batch {
|
||||
pub uid: BatchId,
|
||||
|
||||
#[serde(with = "time::serde::rfc3339")]
|
||||
pub started_at: OffsetDateTime,
|
||||
#[serde(with = "time::serde::rfc3339::option")]
|
||||
pub finished_at: Option<OffsetDateTime>,
|
||||
// pub details: Option<Details>,
|
||||
|
||||
// pub status: Status,
|
||||
}
|
@ -322,6 +322,7 @@ InvalidTaskReverse , InvalidRequest , BAD_REQUEST ;
|
||||
InvalidTaskStatuses , InvalidRequest , BAD_REQUEST ;
|
||||
InvalidTaskTypes , InvalidRequest , BAD_REQUEST ;
|
||||
InvalidTaskUids , InvalidRequest , BAD_REQUEST ;
|
||||
InvalidBatchUids , InvalidRequest , BAD_REQUEST ;
|
||||
IoError , System , UNPROCESSABLE_ENTITY;
|
||||
FeatureNotEnabled , InvalidRequest , BAD_REQUEST ;
|
||||
MalformedPayload , InvalidRequest , BAD_REQUEST ;
|
||||
|
@ -1,3 +1,4 @@
|
||||
pub mod batches;
|
||||
pub mod compression;
|
||||
pub mod deserr;
|
||||
pub mod document_formats;
|
||||
|
@ -2,6 +2,7 @@ use milli::Object;
|
||||
use serde::Serialize;
|
||||
use time::{Duration, OffsetDateTime};
|
||||
|
||||
use crate::batches::BatchId;
|
||||
use crate::error::ResponseError;
|
||||
use crate::settings::{Settings, Unchecked};
|
||||
use crate::tasks::{serialize_duration, Details, IndexSwap, Kind, Status, Task, TaskId};
|
||||
@ -10,6 +11,7 @@ use crate::tasks::{serialize_duration, Details, IndexSwap, Kind, Status, Task, T
|
||||
#[serde(rename_all = "camelCase")]
|
||||
pub struct TaskView {
|
||||
pub uid: TaskId,
|
||||
pub batch_uid: Option<BatchId>,
|
||||
#[serde(default)]
|
||||
pub index_uid: Option<String>,
|
||||
pub status: Status,
|
||||
@ -33,6 +35,7 @@ impl TaskView {
|
||||
pub fn from_task(task: &Task) -> TaskView {
|
||||
TaskView {
|
||||
uid: task.uid,
|
||||
batch_uid: task.batch_uid,
|
||||
index_uid: task.index_uid().map(ToOwned::to_owned),
|
||||
status: task.status,
|
||||
kind: task.kind.as_kind(),
|
||||
|
@ -11,6 +11,7 @@ use serde::{Deserialize, Serialize, Serializer};
|
||||
use time::{Duration, OffsetDateTime};
|
||||
use uuid::Uuid;
|
||||
|
||||
use crate::batches::BatchId;
|
||||
use crate::error::ResponseError;
|
||||
use crate::keys::Key;
|
||||
use crate::settings::{Settings, Unchecked};
|
||||
@ -22,6 +23,7 @@ pub type TaskId = u32;
|
||||
#[serde(rename_all = "camelCase")]
|
||||
pub struct Task {
|
||||
pub uid: TaskId,
|
||||
pub batch_uid: Option<BatchId>,
|
||||
|
||||
#[serde(with = "time::serde::rfc3339")]
|
||||
pub enqueued_at: OffsetDateTime,
|
||||
@ -60,6 +62,11 @@ impl Task {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn with_batch_id(mut self, batch_id: TaskId) -> Self {
|
||||
self.batch_uid = Some(batch_id);
|
||||
self
|
||||
}
|
||||
|
||||
/// Return the list of indexes updated by this tasks.
|
||||
pub fn indexes(&self) -> Vec<&str> {
|
||||
self.kind.indexes()
|
||||
|
48
crates/meilisearch/src/routes/batches.rs
Normal file
48
crates/meilisearch/src/routes/batches.rs
Normal file
@ -0,0 +1,48 @@
|
||||
use actix_web::{
|
||||
web::{self, Data},
|
||||
HttpResponse,
|
||||
};
|
||||
use index_scheduler::{IndexScheduler, Query};
|
||||
use meilisearch_types::{
|
||||
batches::BatchId, error::ResponseError, keys::actions, task_view::TaskView,
|
||||
};
|
||||
|
||||
use crate::extractors::{authentication::GuardedData, sequential_extractor::SeqHandler};
|
||||
|
||||
use super::ActionPolicy;
|
||||
|
||||
pub fn configure(cfg: &mut web::ServiceConfig) {
|
||||
cfg
|
||||
// .service(
|
||||
// web::resource("")
|
||||
// .route(web::get().to(SeqHandler(get_tasks)))
|
||||
// )
|
||||
.service(web::resource("/{batch_id}").route(web::get().to(SeqHandler(get_batch))));
|
||||
}
|
||||
|
||||
async fn get_task(
|
||||
index_scheduler: GuardedData<ActionPolicy<{ actions::TASKS_GET }>, Data<IndexScheduler>>,
|
||||
batch_uid: web::Path<String>,
|
||||
) -> Result<HttpResponse, ResponseError> {
|
||||
let batch_uid_string = batch_uid.into_inner();
|
||||
|
||||
let batch_uid: BatchId = match batch_uid_string.parse() {
|
||||
Ok(id) => id,
|
||||
Err(_e) => {
|
||||
return Err(
|
||||
index_scheduler::Error::InvalidBatchUid { batch_uid: batch_uid_string }.into()
|
||||
)
|
||||
}
|
||||
};
|
||||
|
||||
let query = index_scheduler::Query { uids: Some(vec![batch_uid]), ..Query::default() };
|
||||
let filters = index_scheduler.filters();
|
||||
let (tasks, _) = index_scheduler.get_tasks_from_authorized_indexes(query, filters)?;
|
||||
|
||||
if let Some(task) = tasks.first() {
|
||||
let task_view = TaskView::from_task(task);
|
||||
Ok(HttpResponse::Ok().json(task_view))
|
||||
} else {
|
||||
Err(index_scheduler::Error::TaskNotFound(batch_uid).into())
|
||||
}
|
||||
}
|
@ -19,6 +19,7 @@ use crate::Opt;
|
||||
const PAGINATION_DEFAULT_LIMIT: usize = 20;
|
||||
|
||||
mod api_key;
|
||||
pub mod batches;
|
||||
mod dump;
|
||||
pub mod features;
|
||||
pub mod indexes;
|
||||
|
@ -17,15 +17,13 @@ use time::macros::format_description;
|
||||
use time::{Date, Duration, OffsetDateTime, Time};
|
||||
use tokio::task;
|
||||
|
||||
use super::{get_task_id, is_dry_run, SummarizedTaskView};
|
||||
use super::{get_task_id, is_dry_run, SummarizedTaskView, PAGINATION_DEFAULT_LIMIT};
|
||||
use crate::analytics::{Aggregate, AggregateMethod, Analytics};
|
||||
use crate::extractors::authentication::policies::*;
|
||||
use crate::extractors::authentication::GuardedData;
|
||||
use crate::extractors::sequential_extractor::SeqHandler;
|
||||
use crate::{aggregate_methods, Opt};
|
||||
|
||||
const DEFAULT_LIMIT: u32 = 20;
|
||||
|
||||
pub fn configure(cfg: &mut web::ServiceConfig) {
|
||||
cfg.service(
|
||||
web::resource("")
|
||||
@ -35,10 +33,11 @@ pub fn configure(cfg: &mut web::ServiceConfig) {
|
||||
.service(web::resource("/cancel").route(web::post().to(SeqHandler(cancel_tasks))))
|
||||
.service(web::resource("/{task_id}").route(web::get().to(SeqHandler(get_task))));
|
||||
}
|
||||
|
||||
#[derive(Debug, Deserr)]
|
||||
#[deserr(error = DeserrQueryParamError, rename_all = camelCase, deny_unknown_fields)]
|
||||
pub struct TasksFilterQuery {
|
||||
#[deserr(default = Param(DEFAULT_LIMIT), error = DeserrQueryParamError<InvalidTaskLimit>)]
|
||||
#[deserr(default = Param(PAGINATION_DEFAULT_LIMIT as u32), error = DeserrQueryParamError<InvalidTaskLimit>)]
|
||||
pub limit: Param<u32>,
|
||||
#[deserr(default, error = DeserrQueryParamError<InvalidTaskFrom>)]
|
||||
pub from: Option<Param<TaskId>>,
|
||||
@ -363,7 +362,7 @@ async fn get_task(
|
||||
let task_uid: TaskId = match task_uid_string.parse() {
|
||||
Ok(id) => id,
|
||||
Err(_e) => {
|
||||
return Err(index_scheduler::Error::InvalidTaskUids { task_uid: task_uid_string }.into())
|
||||
return Err(index_scheduler::Error::InvalidTaskUid { task_uid: task_uid_string }.into())
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -99,6 +99,7 @@ impl Display for Value {
|
||||
"{}",
|
||||
json_string!(self, {
|
||||
".uid" => "[uid]",
|
||||
".batchUid" => "[batch_uid]",
|
||||
".enqueuedAt" => "[date]",
|
||||
".startedAt" => "[date]",
|
||||
".finishedAt" => "[date]",
|
||||
|
@ -293,6 +293,7 @@ async fn add_csv_document() {
|
||||
snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###"
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": 0,
|
||||
"indexUid": "pets",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -357,6 +358,7 @@ async fn add_csv_document_with_types() {
|
||||
snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###"
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": 0,
|
||||
"indexUid": "pets",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -432,6 +434,7 @@ async fn add_csv_document_with_custom_delimiter() {
|
||||
snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###"
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": 0,
|
||||
"indexUid": "pets",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -985,6 +988,7 @@ async fn add_documents_no_index_creation() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": 0,
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1063,6 +1067,7 @@ async fn document_addition_with_primary_key() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": 0,
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1111,6 +1116,7 @@ async fn document_addition_with_huge_int_primary_key() {
|
||||
@r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1183,6 +1189,7 @@ async fn replace_document() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": 1,
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1271,6 +1278,7 @@ async fn error_add_documents_bad_document_id() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": 1,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1312,6 +1320,7 @@ async fn error_add_documents_missing_document_id() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": 1,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1403,6 +1412,7 @@ async fn error_document_field_limit_reached_over_multiple_documents() {
|
||||
@r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1437,6 +1447,7 @@ async fn error_document_field_limit_reached_over_multiple_documents() {
|
||||
@r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1486,6 +1497,7 @@ async fn error_document_field_limit_reached_in_one_nested_document() {
|
||||
@r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1529,6 +1541,7 @@ async fn error_document_field_limit_reached_over_multiple_documents_with_nested_
|
||||
@r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1564,6 +1577,7 @@ async fn error_document_field_limit_reached_over_multiple_documents_with_nested_
|
||||
@r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1611,6 +1625,7 @@ async fn add_documents_with_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": 1,
|
||||
"indexUid": "doggo",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1651,6 +1666,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 2,
|
||||
"batchUid": 2,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1688,6 +1704,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 3,
|
||||
"batchUid": 3,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1725,6 +1742,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 4,
|
||||
"batchUid": 4,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1762,6 +1780,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 5,
|
||||
"batchUid": 5,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1799,6 +1818,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 6,
|
||||
"batchUid": 6,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1836,6 +1856,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 7,
|
||||
"batchUid": 7,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1873,6 +1894,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 8,
|
||||
"batchUid": 8,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1910,6 +1932,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 9,
|
||||
"batchUid": 9,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1947,6 +1970,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 10,
|
||||
"batchUid": 10,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -1984,6 +2008,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 11,
|
||||
"batchUid": 11,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2021,6 +2046,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 12,
|
||||
"batchUid": 12,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2058,6 +2084,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 13,
|
||||
"batchUid": 13,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2096,6 +2123,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 14,
|
||||
"batchUid": 14,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2132,6 +2160,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 15,
|
||||
"batchUid": 15,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2168,6 +2197,7 @@ async fn add_documents_invalid_geo_field() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 16,
|
||||
"batchUid": 16,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2210,6 +2240,7 @@ async fn add_invalid_geo_and_then_settings() {
|
||||
snapshot!(ret, @r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2232,6 +2263,7 @@ async fn add_invalid_geo_and_then_settings() {
|
||||
snapshot!(ret, @r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "settingsUpdate",
|
||||
@ -2303,6 +2335,7 @@ async fn error_primary_key_inference() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": 0,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2343,6 +2376,7 @@ async fn error_primary_key_inference() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": 1,
|
||||
"indexUid": "test",
|
||||
"status": "failed",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
@ -2381,6 +2415,7 @@ async fn error_primary_key_inference() {
|
||||
@r###"
|
||||
{
|
||||
"uid": 2,
|
||||
"batchUid": 2,
|
||||
"indexUid": "test",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -184,6 +184,7 @@ async fn delete_document_by_filter() {
|
||||
snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###"
|
||||
{
|
||||
"uid": 2,
|
||||
"batchUid": 2,
|
||||
"indexUid": "doggo",
|
||||
"status": "succeeded",
|
||||
"type": "documentDeletion",
|
||||
@ -249,6 +250,7 @@ async fn delete_document_by_filter() {
|
||||
snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###"
|
||||
{
|
||||
"uid": 3,
|
||||
"batchUid": 3,
|
||||
"indexUid": "doggo",
|
||||
"status": "succeeded",
|
||||
"type": "documentDeletion",
|
||||
@ -333,6 +335,7 @@ async fn delete_document_by_complex_filter() {
|
||||
snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###"
|
||||
{
|
||||
"uid": 2,
|
||||
"batchUid": 2,
|
||||
"indexUid": "doggo",
|
||||
"status": "succeeded",
|
||||
"type": "documentDeletion",
|
||||
@ -391,6 +394,7 @@ async fn delete_document_by_complex_filter() {
|
||||
snapshot!(json_string!(response, { ".enqueuedAt" => "[date]", ".startedAt" => "[date]", ".finishedAt" => "[date]", ".duration" => "[duration]" }), @r###"
|
||||
{
|
||||
"uid": 3,
|
||||
"batchUid": 3,
|
||||
"indexUid": "doggo",
|
||||
"status": "succeeded",
|
||||
"type": "documentDeletion",
|
||||
|
@ -563,6 +563,7 @@ async fn delete_document_by_filter() {
|
||||
snapshot!(response, @r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "DOES_NOT_EXISTS",
|
||||
"status": "failed",
|
||||
"type": "documentDeletion",
|
||||
@ -592,6 +593,7 @@ async fn delete_document_by_filter() {
|
||||
snapshot!(response, @r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "EMPTY_INDEX",
|
||||
"status": "failed",
|
||||
"type": "documentDeletion",
|
||||
@ -623,6 +625,7 @@ async fn delete_document_by_filter() {
|
||||
snapshot!(response, @r###"
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "SHARED_DOCUMENTS",
|
||||
"status": "failed",
|
||||
"type": "documentDeletion",
|
||||
|
@ -88,7 +88,7 @@ async fn import_dump_v1_movie_raw() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks,
|
||||
json!({ "results": [{"uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31968 }, "error": null, "duration": "PT9.317060500S", "enqueuedAt": "2021-09-08T09:08:45.153219Z", "startedAt": "2021-09-08T09:08:45.3961665Z", "finishedAt": "2021-09-08T09:08:54.713227Z" }], "total": 1, "limit": 20, "from": 0, "next": null })
|
||||
json!({ "results": [{"uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31968 }, "error": null, "duration": "PT9.317060500S", "enqueuedAt": "2021-09-08T09:08:45.153219Z", "startedAt": "2021-09-08T09:08:45.3961665Z", "finishedAt": "2021-09-08T09:08:54.713227Z" }], "total": 1, "limit": 20, "from": 0, "next": null })
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -251,7 +251,7 @@ async fn import_dump_v1_movie_with_settings() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks,
|
||||
json!({ "results": [{ "uid": 1, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["genres", "id", "overview", "poster", "release_date", "title"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "sortableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT7.288826907S", "enqueuedAt": "2021-09-08T09:34:40.882977Z", "startedAt": "2021-09-08T09:34:40.883073093Z", "finishedAt": "2021-09-08T09:34:48.1719Z"}, { "uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31968 }, "error": null, "duration": "PT9.090735774S", "enqueuedAt": "2021-09-08T09:34:16.036101Z", "startedAt": "2021-09-08T09:34:16.261191226Z", "finishedAt": "2021-09-08T09:34:25.351927Z" }], "total": 2, "limit": 20, "from": 1, "next": null })
|
||||
json!({ "results": [{ "uid": 1, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["genres", "id", "overview", "poster", "release_date", "title"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "sortableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT7.288826907S", "enqueuedAt": "2021-09-08T09:34:40.882977Z", "startedAt": "2021-09-08T09:34:40.883073093Z", "finishedAt": "2021-09-08T09:34:48.1719Z"}, { "uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31968 }, "error": null, "duration": "PT9.090735774S", "enqueuedAt": "2021-09-08T09:34:16.036101Z", "startedAt": "2021-09-08T09:34:16.261191226Z", "finishedAt": "2021-09-08T09:34:25.351927Z" }], "total": 2, "limit": 20, "from": 1, "next": null })
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -400,7 +400,7 @@ async fn import_dump_v1_rubygems_with_settings() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks["results"][0],
|
||||
json!({"uid": 92, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT1.487793839S", "enqueuedAt": "2021-09-08T09:27:01.465296Z", "startedAt": "2021-09-08T09:28:44.882177161Z", "finishedAt": "2021-09-08T09:28:46.369971Z"})
|
||||
json!({"uid": 92, "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT1.487793839S", "enqueuedAt": "2021-09-08T09:27:01.465296Z", "startedAt": "2021-09-08T09:28:44.882177161Z", "finishedAt": "2021-09-08T09:28:46.369971Z"})
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -535,7 +535,7 @@ async fn import_dump_v2_movie_raw() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks,
|
||||
json!({ "results": [{"uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit": 20, "from": 0, "next": null })
|
||||
json!({ "results": [{"uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit": 20, "from": 0, "next": null })
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -682,7 +682,7 @@ async fn import_dump_v2_movie_with_settings() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks,
|
||||
json!({ "results": [{ "uid": 1, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null })
|
||||
json!({ "results": [{ "uid": 1, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null })
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -828,7 +828,7 @@ async fn import_dump_v2_rubygems_with_settings() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks["results"][0],
|
||||
json!({"uid": 92, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"})
|
||||
json!({"uid": 92, "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"})
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -963,7 +963,7 @@ async fn import_dump_v3_movie_raw() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks,
|
||||
json!({ "results": [{"uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit": 20, "from": 0, "next": null })
|
||||
json!({ "results": [{"uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit": 20, "from": 0, "next": null })
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -1110,7 +1110,7 @@ async fn import_dump_v3_movie_with_settings() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks,
|
||||
json!({ "results": [{ "uid": 1, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null })
|
||||
json!({ "results": [{ "uid": 1, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null })
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can["results"] still get a few documents by id
|
||||
@ -1256,7 +1256,7 @@ async fn import_dump_v3_rubygems_with_settings() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks["results"][0],
|
||||
json!({"uid": 92, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"})
|
||||
json!({"uid": 92, "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"})
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -1391,7 +1391,7 @@ async fn import_dump_v4_movie_raw() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks,
|
||||
json!({ "results": [{"uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit" : 20, "from": 0, "next": null })
|
||||
json!({ "results": [{"uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT41.751156S", "enqueuedAt": "2021-09-08T08:30:30.550282Z", "startedAt": "2021-09-08T08:30:30.553012Z", "finishedAt": "2021-09-08T08:31:12.304168Z" }], "total": 1, "limit" : 20, "from": 0, "next": null })
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -1538,7 +1538,7 @@ async fn import_dump_v4_movie_with_settings() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks,
|
||||
json!({ "results": [{ "uid": 1, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null })
|
||||
json!({ "results": [{ "uid": 1, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "settingsUpdate", "canceledBy": null, "details": { "displayedAttributes": ["title", "genres", "overview", "poster", "release_date"], "searchableAttributes": ["title", "overview"], "filterableAttributes": ["genres"], "stopWords": ["of", "the"] }, "error": null, "duration": "PT37.488777S", "enqueuedAt": "2021-09-08T08:24:02.323444Z", "startedAt": "2021-09-08T08:24:02.324145Z", "finishedAt": "2021-09-08T08:24:39.812922Z" }, { "uid": 0, "batchUid": null, "indexUid": "indexUID", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": { "receivedDocuments": 0, "indexedDocuments": 31944 }, "error": null, "duration": "PT39.941318S", "enqueuedAt": "2021-09-08T08:21:14.742672Z", "startedAt": "2021-09-08T08:21:14.750166Z", "finishedAt": "2021-09-08T08:21:54.691484Z" }], "total": 2, "limit": 20, "from": 1, "next": null })
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
@ -1684,7 +1684,7 @@ async fn import_dump_v4_rubygems_with_settings() {
|
||||
snapshot!(code, @"200 OK");
|
||||
assert_eq!(
|
||||
tasks["results"][0],
|
||||
json!({ "uid": 92, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"})
|
||||
json!({ "uid": 92, "batchUid": null, "indexUid": "rubygems", "status": "succeeded", "type": "documentAdditionOrUpdate", "canceledBy": null, "details": {"receivedDocuments": 0, "indexedDocuments": 1042}, "error": null, "duration": "PT14.034672S", "enqueuedAt": "2021-09-08T08:40:31.390775Z", "startedAt": "2021-09-08T08:51:39.060642Z", "finishedAt": "2021-09-08T08:51:53.095314Z"})
|
||||
);
|
||||
|
||||
// finally we're just going to check that we can still get a few documents by id
|
||||
|
@ -1,8 +1,9 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "pets",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,8 +1,9 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"uid": "[uid]",
|
||||
"batchUid": "[batch_uid]",
|
||||
"indexUid": "pets",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 92,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "documentAdditionOrUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 1,
|
||||
"batchUid": null,
|
||||
"indexUid": "indexUID",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
@ -1,10 +1,11 @@
|
||||
---
|
||||
source: meilisearch/tests/dumps/mod.rs
|
||||
source: crates/meilisearch/tests/dumps/mod.rs
|
||||
---
|
||||
{
|
||||
"results": [
|
||||
{
|
||||
"uid": 0,
|
||||
"batchUid": null,
|
||||
"indexUid": "rubygems",
|
||||
"status": "succeeded",
|
||||
"type": "settingsUpdate",
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user