plug the cluster thingy into meilisearch

This commit is contained in:
Tamo 2023-03-16 18:53:22 +01:00
parent 6cc14feb51
commit de1b939ca0
9 changed files with 254 additions and 50 deletions

View file

@ -1430,18 +1430,20 @@ impl IndexScheduler {
Ok(match batch {
CBatch::TaskCancelation { task, previous_started_at, previous_processing_tasks } => {
Batch::TaskCancelation {
task: self.get_existing_tasks(rtxn, Some(task))?[0],
task: self.get_existing_tasks(rtxn, Some(task))?[0].clone(),
previous_started_at,
previous_processing_tasks,
}
}
CBatch::TaskDeletion(task) => {
Batch::TaskDeletion(self.get_existing_tasks(rtxn, Some(task))?[0])
Batch::TaskDeletion(self.get_existing_tasks(rtxn, Some(task))?[0].clone())
}
CBatch::SnapshotCreation(tasks) => {
Batch::SnapshotCreation(self.get_existing_tasks(rtxn, tasks)?)
}
CBatch::Dump(task) => Batch::Dump(self.get_existing_tasks(rtxn, Some(task))?[0]),
CBatch::Dump(task) => {
Batch::Dump(self.get_existing_tasks(rtxn, Some(task))?[0].clone())
}
CBatch::IndexOperation { op, must_create_index } => Batch::IndexOperation {
op: self.get_index_op_from_cluster_index_op(rtxn, op)?,
must_create_index,
@ -1449,12 +1451,12 @@ impl IndexScheduler {
CBatch::IndexCreation { index_uid, primary_key, task } => Batch::IndexCreation {
index_uid,
primary_key,
task: self.get_existing_tasks(rtxn, Some(task))?[0],
task: self.get_existing_tasks(rtxn, Some(task))?[0].clone(),
},
CBatch::IndexUpdate { index_uid, primary_key, task } => Batch::IndexUpdate {
index_uid,
primary_key,
task: self.get_existing_tasks(rtxn, Some(task))?[0],
task: self.get_existing_tasks(rtxn, Some(task))?[0].clone(),
},
CBatch::IndexDeletion { index_uid, tasks, index_has_been_created } => {
Batch::IndexDeletion {
@ -1464,7 +1466,7 @@ impl IndexScheduler {
}
}
CBatch::IndexSwap { task } => {
Batch::IndexSwap { task: self.get_existing_tasks(rtxn, Some(task))?[0] }
Batch::IndexSwap { task: self.get_existing_tasks(rtxn, Some(task))?[0].clone() }
}
})
}
@ -1559,10 +1561,20 @@ impl From<Batch> for cluster::batch::Batch {
Batch::IndexOperation { op, must_create_index } => {
CBatch::IndexOperation { op: op.into(), must_create_index }
}
Batch::IndexCreation { index_uid, primary_key, task } => todo!(),
Batch::IndexUpdate { index_uid, primary_key, task } => todo!(),
Batch::IndexDeletion { index_uid, tasks, index_has_been_created } => todo!(),
Batch::IndexSwap { task } => todo!(),
Batch::IndexCreation { index_uid, primary_key, task } => {
CBatch::IndexCreation { index_uid, primary_key, task: task.uid }
}
Batch::IndexUpdate { index_uid, primary_key, task } => {
CBatch::IndexUpdate { index_uid, primary_key, task: task.uid }
}
Batch::IndexDeletion { index_uid, tasks, index_has_been_created } => {
CBatch::IndexDeletion {
index_uid,
tasks: tasks.into_iter().map(|task| task.uid).collect(),
index_has_been_created,
}
}
Batch::IndexSwap { task } => CBatch::IndexSwap { task: task.uid },
}
}
}

View file

@ -33,6 +33,7 @@ pub fn snapshot_index_scheduler(scheduler: &IndexScheduler) -> String {
snapshots_path: _,
auth_path: _,
version_file_path: _,
cluster: _,
test_breakpoint_sdr: _,
planned_failures: _,
run_loop_iteration: _,

View file

@ -31,6 +31,7 @@ mod uuid_codec;
pub type Result<T> = std::result::Result<T, Error>;
pub type TaskId = u32;
use std::io::Write;
use std::ops::{Bound, RangeBounds};
use std::path::{Path, PathBuf};
use std::sync::atomic::AtomicBool;
@ -39,7 +40,7 @@ use std::sync::{Arc, RwLock};
use std::time::Duration;
use batch::Batch;
use cluster::{Consistency, Follower, Leader};
use cluster::{Follower, Leader};
use dump::{KindDump, TaskDump, UpdateFile};
pub use error::Error;
use file_store::FileStore;
@ -52,6 +53,7 @@ use meilisearch_types::milli::update::IndexerConfig;
use meilisearch_types::milli::{CboRoaringBitmapCodec, Index, RoaringBitmapCodec, BEU32};
use meilisearch_types::tasks::{Kind, KindWithContent, Status, Task};
use roaring::RoaringBitmap;
use serde::Deserialize;
use synchronoise::SignalEvent;
use time::OffsetDateTime;
use utils::{filter_out_references_to_newer_tasks, keep_tasks_within_datetimes, map_bound};
@ -326,9 +328,28 @@ pub struct IndexScheduler {
run_loop_iteration: Arc<RwLock<usize>>,
}
enum Cluster {
Leader(RwLock<Leader>),
Follower(RwLock<Follower>),
#[derive(Debug, Clone, Copy, PartialEq, Eq, Deserialize)]
pub enum ClusterMode {
Leader,
Follower,
}
impl std::str::FromStr for ClusterMode {
type Err = ();
fn from_str(s: &str) -> std::result::Result<Self, Self::Err> {
match s {
"leader" => Ok(ClusterMode::Leader),
"follower" => Ok(ClusterMode::Follower),
_ => Err(()),
}
}
}
#[derive(Clone)]
pub enum Cluster {
Leader(Arc<RwLock<Leader>>),
Follower(Arc<RwLock<Follower>>),
}
impl IndexScheduler {
@ -368,6 +389,7 @@ impl IndexScheduler {
/// Create an index scheduler and start its run loop.
pub fn new(
options: IndexSchedulerOptions,
cluster: Option<Cluster>,
#[cfg(test)] test_breakpoint_sdr: crossbeam::channel::Sender<(Breakpoint, bool)>,
#[cfg(test)] planned_failures: Vec<(usize, tests::FailureLocation)>,
) -> Result<Self> {
@ -427,7 +449,7 @@ impl IndexScheduler {
snapshots_path: options.snapshots_path,
auth_path: options.auth_path,
version_file_path: options.version_file_path,
cluster: None,
cluster,
#[cfg(test)]
test_breakpoint_sdr,
@ -520,6 +542,16 @@ impl IndexScheduler {
/// only once per index scheduler.
fn run(&self) {
let run = self.private_clone();
// if we're a follower we starts a thread to register the tasks coming from the leader
if let Some(Cluster::Follower(follower)) = self.cluster.clone() {
let this = self.private_clone();
std::thread::spawn(move || loop {
let (task, content) = follower.write().unwrap().get_new_task();
this.register_raw_task(task, content);
});
}
std::thread::Builder::new()
.name(String::from("scheduler"))
.spawn(move || {
@ -877,6 +909,16 @@ impl IndexScheduler {
return Err(e.into());
}
if let Some(Cluster::Leader(leader)) = &self.cluster {
let update_file = if let Some(uuid) = task.content_uuid() {
let path = self.file_store.get_update_path(uuid);
Some(std::fs::read(path).unwrap())
} else {
None
};
leader.write().unwrap().register_new_task(task.clone(), update_file);
}
// If the registered task is a task cancelation
// we inform the processing tasks to stop (if necessary).
if let KindWithContent::TaskCancelation { tasks, .. } = kind {
@ -1006,6 +1048,44 @@ impl IndexScheduler {
Ok(task)
}
/// /!\ should only be used when you're a follower in cluster mode
pub fn register_raw_task(&self, task: Task, content_file: Option<Vec<u8>>) {
if let Some(content) = content_file {
let uuid = task.content_uuid().expect("bad task");
let (_, mut file) = self.file_store.new_update_with_uuid(uuid.as_u128()).unwrap();
file.write_all(&content).unwrap();
file.persist().unwrap();
}
let mut wtxn = self.env.write_txn().unwrap();
self.all_tasks.put(&mut wtxn, &BEU32::new(task.uid), &task).unwrap();
for index in task.indexes() {
self.update_index(&mut wtxn, index, |bitmap| {
bitmap.insert(task.uid);
})
.unwrap();
}
self.update_status(&mut wtxn, task.status, |bitmap| {
bitmap.insert(task.uid);
})
.unwrap();
self.update_kind(&mut wtxn, task.kind.as_kind(), |bitmap| {
(bitmap.insert(task.uid));
})
.unwrap();
utils::insert_task_datetime(&mut wtxn, self.enqueued_at, task.enqueued_at, task.uid)
.unwrap();
wtxn.commit().unwrap();
self.wake_up.signal();
}
/// Create a new index without any associated task.
pub fn create_raw_index(
&self,
@ -1335,7 +1415,7 @@ mod tests {
autobatching_enabled,
};
let index_scheduler = Self::new(options, sender, planned_failures).unwrap();
let index_scheduler = Self::new(options, None, sender, planned_failures).unwrap();
// To be 100% consistent between all test we're going to start the scheduler right now
// and ensure it's in the expected starting state.