Marin Postma 2021-12-02 16:03:26 +01:00
parent c9f3726447
commit a30e02c18c
88 changed files with 5553 additions and 4496 deletions

View file

@ -0,0 +1,480 @@
mod store;
use std::cmp::Ordering;
use std::collections::{BinaryHeap, HashSet};
use std::io::{BufWriter, Write};
use std::path::Path;
use std::sync::Arc;
use chrono::Utc;
use heed::{Env, RwTxn};
use log::debug;
use tokio::sync::RwLock;
use uuid::Uuid;
use super::error::TaskError;
use super::task::{Job, Task, TaskContent, TaskId};
use super::Result;
use crate::index_resolver::IndexUid;
use crate::tasks::task::TaskEvent;
use crate::update_file_store::UpdateFileStore;
#[cfg(test)]
pub use store::test::MockStore as Store;
#[cfg(not(test))]
pub use store::Store;
/// Defines constraints to be applied when querying for Tasks from the store.
#[derive(Default, Debug)]
pub struct TaskFilter {
indexes: Option<HashSet<String>>,
}
impl TaskFilter {
fn pass(&self, task: &Task) -> bool {
self.indexes
.as_ref()
.map(|indexes| indexes.contains(&*task.index_uid))
.unwrap_or(true)
}
/// Adds an index to the filter, so the filter must match this index.
pub fn filter_index(&mut self, index: String) {
self.indexes
.get_or_insert_with(Default::default)
.insert(index);
}
}
/// You can't clone a job because of its volatile nature.
/// If you need to take the `Job` with you though. You can call the method
/// `Pending::take`. It'll return the `Pending` as-is but `Empty` the original.
#[derive(Debug, PartialEq)]
pub enum Pending<T> {
/// A task stored on disk that must be processed.
Task(T),
/// Job always have a higher priority over normal tasks and are not stored on disk.
/// It can be refered as `Volatile job`.
Job(Job),
}
impl Pending<TaskId> {
/// Makes a copy of the task or take the content of the volatile job.
pub(crate) fn take(&mut self) -> Self {
match self {
Self::Task(id) => Self::Task(*id),
Self::Job(job) => Self::Job(job.take()),
}
}
}
impl Eq for Pending<TaskId> {}
impl PartialOrd for Pending<TaskId> {
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
match (self, other) {
// in case of two tasks we want to return the lowest taskId first.
(Pending::Task(lhs), Pending::Task(rhs)) => Some(lhs.cmp(rhs).reverse()),
// A job is always better than a task.
(Pending::Task(_), Pending::Job(_)) => Some(Ordering::Less),
(Pending::Job(_), Pending::Task(_)) => Some(Ordering::Greater),
// When there is two jobs we consider them equals.
(Pending::Job(_), Pending::Job(_)) => Some(Ordering::Equal),
}
}
}
impl Pending<Task> {
pub fn get_content_uuid(&self) -> Option<Uuid> {
match self {
Pending::Task(task) => task.get_content_uuid(),
_ => None,
}
}
}
impl Ord for Pending<TaskId> {
fn cmp(&self, other: &Self) -> Ordering {
self.partial_cmp(other).unwrap()
}
}
pub struct TaskStore {
store: Arc<Store>,
pending_queue: Arc<RwLock<BinaryHeap<Pending<TaskId>>>>,
}
impl Clone for TaskStore {
fn clone(&self) -> Self {
Self {
store: self.store.clone(),
pending_queue: self.pending_queue.clone(),
}
}
}
impl TaskStore {
pub fn new(env: heed::Env) -> Result<Self> {
let mut store = Store::new(env)?;
let unfinished_tasks = store.reset_and_return_unfinished_tasks()?;
let store = Arc::new(store);
Ok(Self {
store,
pending_queue: Arc::new(RwLock::new(unfinished_tasks)),
})
}
pub async fn register(&self, index_uid: IndexUid, content: TaskContent) -> Result<Task> {
debug!("registering update: {:?}", content);
let store = self.store.clone();
let task = tokio::task::spawn_blocking(move || -> Result<Task> {
let mut txn = store.wtxn()?;
let next_task_id = store.next_task_id(&mut txn)?;
let created_at = TaskEvent::Created(Utc::now());
let task = Task {
id: next_task_id,
index_uid,
content,
events: vec![created_at],
};
store.put(&mut txn, &task)?;
txn.commit()?;
Ok(task)
})
.await??;
self.pending_queue
.write()
.await
.push(Pending::Task(task.id));
Ok(task)
}
pub fn register_raw_update(&self, wtxn: &mut RwTxn, task: &Task) -> Result<()> {
self.store.put(wtxn, task)?;
Ok(())
}
/// Register an update that applies on multiple indexes.
/// Currently the update is considered as a priority.
pub async fn register_job(&self, content: Job) {
debug!("registering a job: {:?}", content);
self.pending_queue.write().await.push(Pending::Job(content));
}
/// Returns the next task to process.
pub async fn peek_pending_task(&self) -> Option<Pending<TaskId>> {
let mut pending_queue = self.pending_queue.write().await;
loop {
match pending_queue.peek()? {
Pending::Job(Job::Empty) => drop(pending_queue.pop()),
_ => return Some(pending_queue.peek_mut()?.take()),
}
}
}
/// Returns the next task to process if there is one.
pub async fn get_processing_task(&self) -> Result<Option<Task>> {
match self.peek_pending_task().await {
Some(Pending::Task(tid)) => {
let task = self.get_task(tid, None).await?;
Ok(matches!(task.events.last(), Some(TaskEvent::Processing(_))).then(|| task))
}
_ => Ok(None),
}
}
pub async fn get_task(&self, id: TaskId, filter: Option<TaskFilter>) -> Result<Task> {
let store = self.store.clone();
let task = tokio::task::spawn_blocking(move || -> Result<_> {
let txn = store.rtxn()?;
let task = store.get(&txn, id)?;
Ok(task)
})
.await??
.ok_or(TaskError::UnexistingTask(id))?;
match filter {
Some(filter) => filter
.pass(&task)
.then(|| task)
.ok_or(TaskError::UnexistingTask(id)),
None => Ok(task),
}
}
pub async fn update_tasks(&self, tasks: Vec<Pending<Task>>) -> Result<Vec<Pending<Task>>> {
let store = self.store.clone();
let tasks = tokio::task::spawn_blocking(move || -> Result<_> {
let mut txn = store.wtxn()?;
for task in &tasks {
match task {
Pending::Task(task) => store.put(&mut txn, task)?,
Pending::Job(_) => (),
}
}
txn.commit()?;
Ok(tasks)
})
.await??;
Ok(tasks)
}
/// Delete one task from the queue and remove all `Empty` job.
pub async fn delete_pending(&self, to_delete: &Pending<Task>) {
if let Pending::Task(Task { id: pending_id, .. }) = to_delete {
let mut pending_queue = self.pending_queue.write().await;
*pending_queue = std::mem::take(&mut *pending_queue)
.into_iter()
.filter(|pending| match pending {
Pending::Job(Job::Empty) => false,
Pending::Task(id) => pending_id != id,
_ => true,
})
.collect::<BinaryHeap<Pending<TaskId>>>();
}
}
pub async fn list_tasks(
&self,
offset: Option<TaskId>,
filter: Option<TaskFilter>,
limit: Option<usize>,
) -> Result<Vec<Task>> {
let store = self.store.clone();
tokio::task::spawn_blocking(move || {
let txn = store.rtxn()?;
let tasks = store.list_tasks(&txn, offset, filter, limit)?;
Ok(tasks)
})
.await?
}
pub async fn dump(
&self,
dir_path: impl AsRef<Path>,
update_file_store: UpdateFileStore,
) -> Result<()> {
let update_dir = dir_path.as_ref().join("updates");
let updates_file = update_dir.join("data.jsonl");
let tasks = self.list_tasks(None, None, None).await?;
let dir_path = dir_path.as_ref().to_path_buf();
tokio::task::spawn_blocking(move || -> Result<()> {
std::fs::create_dir(&update_dir)?;
let updates_file = std::fs::File::create(updates_file)?;
let mut updates_file = BufWriter::new(updates_file);
for task in tasks {
serde_json::to_writer(&mut updates_file, &task)?;
updates_file.write_all(b"\n")?;
if !task.is_finished() {
if let Some(content_uuid) = task.get_content_uuid() {
update_file_store.dump(content_uuid, &dir_path)?;
}
}
}
updates_file.flush()?;
Ok(())
})
.await??;
Ok(())
}
pub fn load_dump(src: impl AsRef<Path>, env: Env) -> anyhow::Result<()> {
// create a dummy update field store, since it is not needed right now.
let store = Self::new(env.clone())?;
let src_update_path = src.as_ref().join("updates");
let update_data = std::fs::File::open(&src_update_path.join("data.jsonl"))?;
let update_data = std::io::BufReader::new(update_data);
let stream = serde_json::Deserializer::from_reader(update_data).into_iter::<Task>();
let mut wtxn = env.write_txn()?;
for entry in stream {
store.register_raw_update(&mut wtxn, &entry?)?;
}
wtxn.commit()?;
Ok(())
}
}
#[cfg(test)]
pub mod test {
use crate::tasks::task_store::store::test::tmp_env;
use super::*;
use nelson::Mocker;
use proptest::{
strategy::Strategy,
test_runner::{Config, TestRunner},
};
pub enum MockTaskStore {
Real(TaskStore),
Mock(Arc<Mocker>),
}
impl Clone for MockTaskStore {
fn clone(&self) -> Self {
match self {
Self::Real(x) => Self::Real(x.clone()),
Self::Mock(x) => Self::Mock(x.clone()),
}
}
}
impl MockTaskStore {
pub fn new(env: heed::Env) -> Result<Self> {
Ok(Self::Real(TaskStore::new(env)?))
}
pub fn mock(mocker: Mocker) -> Self {
Self::Mock(Arc::new(mocker))
}
pub async fn update_tasks(&self, tasks: Vec<Pending<Task>>) -> Result<Vec<Pending<Task>>> {
match self {
Self::Real(s) => s.update_tasks(tasks).await,
Self::Mock(m) => unsafe {
m.get::<_, Result<Vec<Pending<Task>>>>("update_tasks")
.call(tasks)
},
}
}
pub async fn delete_pending(&self, to_delete: &Pending<Task>) {
match self {
Self::Real(s) => s.delete_pending(to_delete).await,
Self::Mock(m) => unsafe { m.get("delete_pending").call(to_delete) },
}
}
pub async fn get_task(&self, id: TaskId, filter: Option<TaskFilter>) -> Result<Task> {
match self {
Self::Real(s) => s.get_task(id, filter).await,
Self::Mock(m) => unsafe { m.get::<_, Result<Task>>("get_task").call((id, filter)) },
}
}
pub async fn get_processing_task(&self) -> Result<Option<Task>> {
match self {
Self::Real(s) => s.get_processing_task().await,
Self::Mock(m) => unsafe {
m.get::<_, Result<Option<Task>>>("get_pending_task")
.call(())
},
}
}
pub async fn peek_pending_task(&self) -> Option<Pending<TaskId>> {
match self {
Self::Real(s) => s.peek_pending_task().await,
Self::Mock(m) => unsafe {
m.get::<_, Option<Pending<TaskId>>>("peek_pending_task")
.call(())
},
}
}
pub async fn list_tasks(
&self,
from: Option<TaskId>,
filter: Option<TaskFilter>,
limit: Option<usize>,
) -> Result<Vec<Task>> {
match self {
Self::Real(s) => s.list_tasks(from, filter, limit).await,
Self::Mock(_m) => todo!(),
}
}
pub async fn dump(&self, path: &Path, update_file_store: UpdateFileStore) -> Result<()> {
match self {
Self::Real(s) => s.dump(path, update_file_store).await,
Self::Mock(_m) => todo!(),
}
}
pub async fn register(&self, index_uid: IndexUid, content: TaskContent) -> Result<Task> {
match self {
Self::Real(s) => s.register(index_uid, content).await,
Self::Mock(_m) => todo!(),
}
}
pub fn register_raw_update(&self, wtxn: &mut RwTxn, task: &Task) -> Result<()> {
match self {
Self::Real(s) => s.register_raw_update(wtxn, task),
Self::Mock(_m) => todo!(),
}
}
pub async fn register_job(&self, content: Job) {
match self {
Self::Real(s) => s.register_job(content).await,
Self::Mock(_m) => todo!(),
}
}
pub fn load_dump(path: impl AsRef<Path>, env: Env) -> anyhow::Result<()> {
TaskStore::load_dump(path, env)
}
}
#[test]
fn test_increment_task_id() {
let tmp = tmp_env();
let store = Store::new(tmp.env()).unwrap();
let mut txn = store.wtxn().unwrap();
assert_eq!(store.next_task_id(&mut txn).unwrap(), 0);
txn.abort().unwrap();
let gen_task = |id: TaskId| Task {
id,
index_uid: IndexUid::new_unchecked("test"),
content: TaskContent::IndexCreation { primary_key: None },
events: Vec::new(),
};
let mut runner = TestRunner::new(Config::default());
runner
.run(&(0..100u64).prop_map(gen_task), |task| {
let mut txn = store.wtxn().unwrap();
let previous_id = store.next_task_id(&mut txn).unwrap();
store.put(&mut txn, &task).unwrap();
let next_id = store.next_task_id(&mut txn).unwrap();
// if we put a task whose task_id is less than the next_id, then the next_id remains
// unchanged, otherwise it becomes task.id + 1
if task.id < previous_id {
assert_eq!(next_id, previous_id)
} else {
assert_eq!(next_id, task.id + 1);
}
txn.commit().unwrap();
Ok(())
})
.unwrap();
}
}

View file

@ -0,0 +1,452 @@
#[allow(clippy::upper_case_acronyms)]
type BEU64 = heed::zerocopy::U64<heed::byteorder::BE>;
const UID_TASK_IDS: &str = "uid_task_id";
const TASKS: &str = "tasks";
use std::borrow::Cow;
use std::collections::BinaryHeap;
use std::convert::TryInto;
use std::mem::size_of;
use std::ops::Range;
use std::result::Result as StdResult;
use heed::types::{ByteSlice, OwnedType, SerdeJson, Unit};
use heed::{BytesDecode, BytesEncode, Database, Env, RoTxn, RwTxn};
use crate::tasks::task::{Task, TaskId};
use super::super::Result;
use super::{Pending, TaskFilter};
enum IndexUidTaskIdCodec {}
impl<'a> BytesEncode<'a> for IndexUidTaskIdCodec {
type EItem = (&'a str, TaskId);
fn bytes_encode((s, id): &'a Self::EItem) -> Option<Cow<'a, [u8]>> {
let size = s.len() + std::mem::size_of::<TaskId>() + 1;
if size > 512 {
return None;
}
let mut b = Vec::with_capacity(size);
b.extend_from_slice(s.as_bytes());
// null terminate the string
b.push(0);
b.extend_from_slice(&id.to_be_bytes());
Some(Cow::Owned(b))
}
}
impl<'a> BytesDecode<'a> for IndexUidTaskIdCodec {
type DItem = (&'a str, TaskId);
fn bytes_decode(bytes: &'a [u8]) -> Option<Self::DItem> {
let len = bytes.len();
let s_end = len.checked_sub(size_of::<TaskId>())?.checked_sub(1)?;
let str_bytes = &bytes[..s_end];
let str = std::str::from_utf8(str_bytes).ok()?;
let id = TaskId::from_be_bytes(bytes[(len - size_of::<TaskId>())..].try_into().ok()?);
Some((str, id))
}
}
pub struct Store {
env: Env,
uids_task_ids: Database<IndexUidTaskIdCodec, Unit>,
tasks: Database<OwnedType<BEU64>, SerdeJson<Task>>,
}
impl Store {
/// Create a new store from the specified `Path`.
/// Be really cautious when calling this function, the returned `Store` may
/// be in an invalid state, with dangling processing tasks.
/// You want to patch all un-finished tasks and put them in your pending
/// queue with the `reset_and_return_unfinished_update` method.
pub fn new(env: heed::Env) -> Result<Self> {
let uids_task_ids = env.create_database(Some(UID_TASK_IDS))?;
let tasks = env.create_database(Some(TASKS))?;
Ok(Self {
env,
uids_task_ids,
tasks,
})
}
/// This function should be called *right after* creating the store.
/// It put back all unfinished update in the `Created` state. This
/// allow us to re-enqueue an update that didn't had the time to finish
/// when Meilisearch closed.
pub fn reset_and_return_unfinished_tasks(&mut self) -> Result<BinaryHeap<Pending<TaskId>>> {
let mut unfinished_tasks: BinaryHeap<Pending<TaskId>> = BinaryHeap::new();
let mut wtxn = self.wtxn()?;
let mut iter = self.tasks.rev_iter_mut(&mut wtxn)?;
while let Some(entry) = iter.next() {
let entry = entry?;
let (id, mut task): (BEU64, Task) = entry;
// Since all tasks are ordered, we can stop iterating when we encounter our first non-finished task.
if task.is_finished() {
break;
}
// we only keep the first state. Its supposed to be a `Created` state.
task.events.drain(1..);
unfinished_tasks.push(Pending::Task(id.get()));
// Since we own the id and the task this is a safe operation.
unsafe {
iter.put_current(&id, &task)?;
}
}
drop(iter);
wtxn.commit()?;
Ok(unfinished_tasks)
}
pub fn wtxn(&self) -> Result<RwTxn> {
Ok(self.env.write_txn()?)
}
pub fn rtxn(&self) -> Result<RoTxn> {
Ok(self.env.read_txn()?)
}
/// Returns the id for the next task.
///
/// The required `mut txn` acts as a reservation system. It guarantees that as long as you commit
/// the task to the store in the same transaction, no one else will hav this task id.
pub fn next_task_id(&self, txn: &mut RwTxn) -> Result<TaskId> {
let id = self
.tasks
.lazily_decode_data()
.last(txn)?
.map(|(id, _)| id.get() + 1)
.unwrap_or(0);
Ok(id)
}
pub fn put(&self, txn: &mut RwTxn, task: &Task) -> Result<()> {
self.tasks.put(txn, &BEU64::new(task.id), task)?;
self.uids_task_ids
.put(txn, &(&task.index_uid, task.id), &())?;
Ok(())
}
pub fn get(&self, txn: &RoTxn, id: TaskId) -> Result<Option<Task>> {
let task = self.tasks.get(txn, &BEU64::new(id))?;
Ok(task)
}
pub fn list_tasks<'a>(
&self,
txn: &'a RoTxn,
from: Option<TaskId>,
filter: Option<TaskFilter>,
limit: Option<usize>,
) -> Result<Vec<Task>> {
let from = from.unwrap_or_default();
let range = from..limit
.map(|limit| (limit as u64).saturating_add(from))
.unwrap_or(u64::MAX);
let iter: Box<dyn Iterator<Item = StdResult<_, heed::Error>>> = match filter {
Some(filter) => {
let iter = self
.compute_candidates(txn, filter, range)?
.into_iter()
.filter_map(|id| self.tasks.get(txn, &BEU64::new(id)).transpose());
Box::new(iter)
}
None => Box::new(
self.tasks
.rev_range(txn, &(BEU64::new(range.start)..BEU64::new(range.end)))?
.map(|r| r.map(|(_, t)| t)),
),
};
// Collect 'limit' task if it exists or all of them.
let tasks = iter
.take(limit.unwrap_or(usize::MAX))
.try_fold::<_, _, StdResult<_, heed::Error>>(Vec::new(), |mut v, task| {
v.push(task?);
Ok(v)
})?;
Ok(tasks)
}
fn compute_candidates(
&self,
txn: &heed::RoTxn,
filter: TaskFilter,
range: Range<TaskId>,
) -> Result<BinaryHeap<TaskId>> {
let mut candidates = BinaryHeap::new();
if let Some(indexes) = filter.indexes {
for index in indexes {
// We need to prefix search the null terminated string to make sure that we only
// get exact matches for the index, and not other uids that would share the same
// prefix, i.e test and test1.
let mut index_uid = index.as_bytes().to_vec();
index_uid.push(0);
self.uids_task_ids
.remap_key_type::<ByteSlice>()
.rev_prefix_iter(txn, &index_uid)?
.map(|entry| -> StdResult<_, heed::Error> {
let (key, _) = entry?;
let (_, id) =
IndexUidTaskIdCodec::bytes_decode(key).ok_or(heed::Error::Decoding)?;
Ok(id)
})
.skip_while(|entry| {
entry
.as_ref()
.ok()
// we skip all elements till we enter in the range
.map(|key| !range.contains(key))
// if we encounter an error we returns true to collect it later
.unwrap_or(true)
})
.take_while(|entry| {
entry
.as_ref()
.ok()
// as soon as we are out of the range we exit
.map(|key| range.contains(key))
// if we encounter an error we returns true to collect it later
.unwrap_or(true)
})
.try_for_each::<_, StdResult<(), heed::Error>>(|id| {
candidates.push(id?);
Ok(())
})?;
}
}
Ok(candidates)
}
}
#[cfg(test)]
pub mod test {
use heed::EnvOpenOptions;
use itertools::Itertools;
use nelson::Mocker;
use proptest::collection::vec;
use proptest::prelude::*;
use tempfile::TempDir;
use crate::index_resolver::IndexUid;
use crate::tasks::task::TaskContent;
use super::*;
/// TODO: use this mock to test the task store properly.
#[allow(dead_code)]
pub enum MockStore {
Real(Store),
Fake(Mocker),
}
pub struct TmpEnv(TempDir, heed::Env);
impl TmpEnv {
pub fn env(&self) -> heed::Env {
self.1.clone()
}
}
pub fn tmp_env() -> TmpEnv {
let tmp = tempfile::tempdir().unwrap();
let mut options = EnvOpenOptions::new();
options.map_size(4096 * 100000);
options.max_dbs(1000);
let env = options.open(tmp.path()).unwrap();
TmpEnv(tmp, env)
}
impl MockStore {
pub fn new(env: heed::Env) -> Result<Self> {
Ok(Self::Real(Store::new(env)?))
}
pub fn reset_and_return_unfinished_tasks(&mut self) -> Result<BinaryHeap<Pending<TaskId>>> {
match self {
MockStore::Real(index) => index.reset_and_return_unfinished_tasks(),
MockStore::Fake(_) => todo!(),
}
}
pub fn wtxn(&self) -> Result<RwTxn> {
match self {
MockStore::Real(index) => index.wtxn(),
MockStore::Fake(_) => todo!(),
}
}
pub fn rtxn(&self) -> Result<RoTxn> {
match self {
MockStore::Real(index) => index.rtxn(),
MockStore::Fake(_) => todo!(),
}
}
pub fn next_task_id(&self, txn: &mut RwTxn) -> Result<TaskId> {
match self {
MockStore::Real(index) => index.next_task_id(txn),
MockStore::Fake(_) => todo!(),
}
}
pub fn put(&self, txn: &mut RwTxn, task: &Task) -> Result<()> {
match self {
MockStore::Real(index) => index.put(txn, task),
MockStore::Fake(_) => todo!(),
}
}
pub fn get(&self, txn: &RoTxn, id: TaskId) -> Result<Option<Task>> {
match self {
MockStore::Real(index) => index.get(txn, id),
MockStore::Fake(_) => todo!(),
}
}
pub fn list_tasks<'a>(
&self,
txn: &'a RoTxn,
from: Option<TaskId>,
filter: Option<TaskFilter>,
limit: Option<usize>,
) -> Result<Vec<Task>> {
match self {
MockStore::Real(index) => index.list_tasks(txn, from, filter, limit),
MockStore::Fake(_) => todo!(),
}
}
}
#[test]
fn test_ordered_filtered_updates() {
let tmp = tmp_env();
let store = Store::new(tmp.env()).unwrap();
let tasks = (0..100)
.map(|_| Task {
id: rand::random(),
index_uid: IndexUid::new_unchecked("test".to_string()),
content: TaskContent::IndexDeletion,
events: vec![],
})
.collect::<Vec<_>>();
let mut txn = store.env.write_txn().unwrap();
tasks
.iter()
.try_for_each(|t| store.put(&mut txn, t))
.unwrap();
let mut filter = TaskFilter::default();
filter.filter_index("test".into());
let tasks = store.list_tasks(&txn, None, Some(filter), None).unwrap();
assert!(tasks
.iter()
.map(|t| t.id)
.tuple_windows()
.all(|(a, b)| a > b));
}
#[test]
fn test_filter_same_index_prefix() {
let tmp = tmp_env();
let store = Store::new(tmp.env()).unwrap();
let task_1 = Task {
id: 1,
index_uid: IndexUid::new_unchecked("test".to_string()),
content: TaskContent::IndexDeletion,
events: vec![],
};
let task_2 = Task {
id: 0,
index_uid: IndexUid::new_unchecked("test1".to_string()),
content: TaskContent::IndexDeletion,
events: vec![],
};
let mut txn = store.wtxn().unwrap();
store.put(&mut txn, &task_1).unwrap();
store.put(&mut txn, &task_2).unwrap();
let mut filter = TaskFilter::default();
filter.filter_index("test".into());
let tasks = store.list_tasks(&txn, None, Some(filter), None).unwrap();
txn.abort().unwrap();
assert_eq!(tasks.len(), 1);
assert_eq!(&*tasks.first().unwrap().index_uid, "test");
// same thing but invert the ids
let task_1 = Task {
id: 0,
index_uid: IndexUid::new_unchecked("test".to_string()),
content: TaskContent::IndexDeletion,
events: vec![],
};
let task_2 = Task {
id: 1,
index_uid: IndexUid::new_unchecked("test1".to_string()),
content: TaskContent::IndexDeletion,
events: vec![],
};
let mut txn = store.wtxn().unwrap();
store.put(&mut txn, &task_1).unwrap();
store.put(&mut txn, &task_2).unwrap();
let mut filter = TaskFilter::default();
filter.filter_index("test".into());
let tasks = store.list_tasks(&txn, None, Some(filter), None).unwrap();
assert_eq!(tasks.len(), 1);
assert_eq!(&*tasks.first().unwrap().index_uid, "test");
}
proptest! {
#[test]
fn encode_decode_roundtrip(index_uid in any::<IndexUid>(), task_id in 0..TaskId::MAX) {
let value = (index_uid.as_ref(), task_id);
let bytes = IndexUidTaskIdCodec::bytes_encode(&value).unwrap();
let (index, id) = IndexUidTaskIdCodec::bytes_decode(bytes.as_ref()).unwrap();
assert_eq!(&*index_uid, index);
assert_eq!(task_id, id);
}
#[test]
fn encode_doesnt_crash(index_uid in "\\PC*", task_id in 0..TaskId::MAX) {
let value = (index_uid.as_ref(), task_id);
IndexUidTaskIdCodec::bytes_encode(&value);
}
#[test]
fn decode_doesnt_crash(bytes in vec(any::<u8>(), 0..1000)) {
IndexUidTaskIdCodec::bytes_decode(&bytes);
}
}
}