make scheduler accept multiple batch handlers

This commit is contained in:
ad hoc 2022-05-19 12:43:46 +02:00
parent 6a0231cb28
commit 46cdc17701
No known key found for this signature in database
GPG key ID: 4F00A782990CC643
28 changed files with 484 additions and 374 deletions

View file

@ -1,22 +1,75 @@
use time::OffsetDateTime;
use super::task::Task;
use crate::snapshot::SnapshotJob;
use super::task::{Task, TaskEvent};
pub type BatchId = u64;
#[derive(Debug)]
pub enum BatchContent {
DocumentAddtitionBatch(Vec<Task>),
IndexUpdate(Task),
Dump(Task),
Snapshot(SnapshotJob),
// Symbolizes a empty batch. This can occur when we were woken, but there wasn't any work to do.
Empty,
}
impl BatchContent {
pub fn first(&self) -> Option<&Task> {
match self {
BatchContent::DocumentAddtitionBatch(ts) => ts.first(),
BatchContent::Dump(t) | BatchContent::IndexUpdate(t) => Some(t),
BatchContent::Snapshot(_) | BatchContent::Empty => None,
}
}
pub fn push_event(&mut self, event: TaskEvent) {
match self {
BatchContent::DocumentAddtitionBatch(ts) => {
ts.iter_mut().for_each(|t| t.events.push(event.clone()))
}
BatchContent::IndexUpdate(t) | BatchContent::Dump(t) => t.events.push(event),
BatchContent::Snapshot(_) | BatchContent::Empty => (),
}
}
}
#[derive(Debug)]
pub struct Batch {
pub id: BatchId,
// Only batches that contains a persistant tasks are given an id. Snapshot batches don't have
// an id.
pub id: Option<BatchId>,
pub created_at: OffsetDateTime,
pub tasks: Vec<Task>,
pub content: BatchContent,
}
impl Batch {
pub fn new(id: Option<BatchId>, content: BatchContent) -> Self {
Self {
id,
created_at: OffsetDateTime::now_utc(),
content,
}
}
pub fn len(&self) -> usize {
self.tasks.len()
match self.content {
BatchContent::DocumentAddtitionBatch(ref ts) => ts.len(),
BatchContent::IndexUpdate(_) | BatchContent::Dump(_) | BatchContent::Snapshot(_) => 1,
BatchContent::Empty => 0,
}
}
pub fn is_empty(&self) -> bool {
self.tasks.is_empty()
self.len() == 0
}
pub fn empty() -> Self {
Self {
id: None,
created_at: OffsetDateTime::now_utc(),
content: BatchContent::Empty,
}
}
}

View file

@ -0,0 +1,20 @@
use crate::tasks::batch::{Batch, BatchContent};
use crate::tasks::BatchHandler;
/// A sink handler for empty tasks.
pub struct EmptyBatchHandler;
#[async_trait::async_trait]
impl BatchHandler for EmptyBatchHandler {
fn accept(&self, batch: &Batch) -> bool {
matches!(batch.content, BatchContent::Empty)
}
async fn process_batch(&self, batch: Batch) -> Batch {
batch
}
async fn finish(&self, _: &Batch) {
()
}
}

View file

@ -0,0 +1,58 @@
use time::OffsetDateTime;
use crate::index_resolver::IndexResolver;
use crate::index_resolver::{index_store::IndexStore, meta_store::IndexMetaStore};
use crate::tasks::batch::{Batch, BatchContent};
use crate::tasks::task::TaskEvent;
use crate::tasks::BatchHandler;
#[async_trait::async_trait]
impl<U, I> BatchHandler for IndexResolver<U, I>
where
U: IndexMetaStore + Send + Sync + 'static,
I: IndexStore + Send + Sync + 'static,
{
fn accept(&self, batch: &Batch) -> bool {
match batch.content {
BatchContent::DocumentAddtitionBatch(_) | BatchContent::IndexUpdate(_) => true,
_ => false,
}
}
async fn process_batch(&self, mut batch: Batch) -> Batch {
match batch.content {
BatchContent::DocumentAddtitionBatch(ref mut tasks) => {
*tasks = self
.process_document_addition_batch(std::mem::take(tasks))
.await;
}
BatchContent::IndexUpdate(ref mut task) => match self.process_task(&task).await {
Ok(success) => {
task.events.push(TaskEvent::Succeded {
result: success,
timestamp: OffsetDateTime::now_utc(),
});
}
Err(err) => task.events.push(TaskEvent::Failed {
error: err.into(),
timestamp: OffsetDateTime::now_utc(),
}),
},
_ => unreachable!(),
}
batch
}
async fn finish(&self, batch: &Batch) {
if let BatchContent::DocumentAddtitionBatch(ref tasks) = batch.content {
for task in tasks {
if let Some(content_uuid) = task.get_content_uuid() {
if let Err(e) = self.file_store.delete(content_uuid).await {
log::error!("error deleting update file: {}", e);
}
}
}
}
}
}

View file

@ -0,0 +1,2 @@
pub mod empty_handler;
mod index_resolver_handler;

View file

@ -1,5 +1,6 @@
use async_trait::async_trait;
pub use batch_handlers::empty_handler::EmptyBatchHandler;
pub use scheduler::Scheduler;
pub use task_store::TaskFilter;
@ -11,9 +12,8 @@ pub use task_store::TaskStore;
use batch::Batch;
use error::Result;
use self::task::Job;
pub mod batch;
mod batch_handlers;
pub mod error;
mod scheduler;
pub mod task;
@ -22,12 +22,13 @@ pub mod update_loop;
#[cfg_attr(test, mockall::automock(type Error=test::DebugError;))]
#[async_trait]
pub trait TaskPerformer: Sync + Send + 'static {
pub trait BatchHandler: Sync + Send + 'static {
/// return whether this handler can accept this batch
fn accept(&self, batch: &Batch) -> bool;
/// Processes the `Task` batch returning the batch with the `Task` updated.
async fn process_batch(&self, batch: Batch) -> Batch;
async fn process_job(&self, job: Job);
/// `finish` is called when the result of `process` has been commited to the task store. This
/// method can be used to perform cleanup after the update has been completed for example.
async fn finish(&self, batch: &Batch);

View file

@ -2,6 +2,7 @@ use std::cmp::Ordering;
use std::collections::{hash_map::Entry, BinaryHeap, HashMap, VecDeque};
use std::ops::{Deref, DerefMut};
use std::path::Path;
use std::slice;
use std::sync::Arc;
use std::time::Duration;
@ -11,24 +12,21 @@ use time::OffsetDateTime;
use tokio::sync::{watch, RwLock};
use crate::options::SchedulerConfig;
use crate::snapshot::SnapshotJob;
use crate::update_file_store::UpdateFileStore;
use super::batch::Batch;
use super::batch::{Batch, BatchContent};
use super::error::Result;
use super::task::{Job, Task, TaskContent, TaskEvent, TaskId};
use super::task::{Task, TaskContent, TaskEvent, TaskId};
use super::update_loop::UpdateLoop;
use super::{TaskFilter, TaskPerformer, TaskStore};
use super::{BatchHandler, TaskFilter, TaskStore};
#[derive(Eq, Debug, Clone, Copy)]
enum TaskType {
DocumentAddition {
number: usize,
},
DocumentUpdate {
number: usize,
},
/// Any other kind of task, including Dumps
Other,
DocumentAddition { number: usize },
DocumentUpdate { number: usize },
IndexUpdate,
Dump,
}
/// Two tasks are equal if they have the same type.
@ -166,7 +164,13 @@ impl TaskQueue {
} => TaskType::DocumentUpdate {
number: documents_count,
},
_ => TaskType::Other,
TaskContent::Dump { .. } => TaskType::Dump,
TaskContent::DocumentDeletion(_)
| TaskContent::SettingsUpdate { .. }
| TaskContent::IndexDeletion
| TaskContent::IndexCreation { .. }
| TaskContent::IndexUpdate { .. } => TaskType::IndexUpdate,
_ => unreachable!("unhandled task type"),
};
let task = PendingTask { kind, id };
@ -217,11 +221,12 @@ impl TaskQueue {
}
pub struct Scheduler {
jobs: VecDeque<Job>,
// TODO: currently snapshots are non persistent tasks, and are treated differently.
snapshots: VecDeque<SnapshotJob>,
tasks: TaskQueue,
store: TaskStore,
processing: Vec<TaskId>,
processing: Processing,
next_fetched_task_id: TaskId,
config: SchedulerConfig,
/// Notifies the update loop that a new task was received
@ -229,14 +234,11 @@ pub struct Scheduler {
}
impl Scheduler {
pub fn new<P>(
pub fn new(
store: TaskStore,
performer: Arc<P>,
performers: Vec<Arc<dyn BatchHandler + Sync + Send + 'static>>,
mut config: SchedulerConfig,
) -> Result<Arc<RwLock<Self>>>
where
P: TaskPerformer,
{
) -> Result<Arc<RwLock<Self>>> {
let (notifier, rcv) = watch::channel(());
let debounce_time = config.debounce_duration_sec;
@ -247,11 +249,11 @@ impl Scheduler {
}
let this = Self {
jobs: VecDeque::new(),
snapshots: VecDeque::new(),
tasks: TaskQueue::default(),
store,
processing: Vec::new(),
processing: Processing::Nothing,
next_fetched_task_id: 0,
config,
notifier,
@ -264,7 +266,7 @@ impl Scheduler {
let update_loop = UpdateLoop::new(
this.clone(),
performer,
performers,
debounce_time.filter(|&v| v > 0).map(Duration::from_secs),
rcv,
);
@ -283,9 +285,13 @@ impl Scheduler {
self.tasks.insert(task);
}
pub fn register_snapshot(&mut self, job: SnapshotJob) {
self.snapshots.push_back(job);
}
/// Clears the processing list, this method should be called when the processing of a batch is finished.
pub fn finish(&mut self) {
self.processing.clear();
self.processing = Processing::Nothing;
}
pub fn notify(&self) {
@ -293,13 +299,27 @@ impl Scheduler {
}
fn notify_if_not_empty(&self) {
if !self.jobs.is_empty() || !self.tasks.is_empty() {
if !self.snapshots.is_empty() || !self.tasks.is_empty() {
self.notify();
}
}
pub async fn update_tasks(&self, tasks: Vec<Task>) -> Result<Vec<Task>> {
self.store.update_tasks(tasks).await
pub async fn update_tasks(&self, content: BatchContent) -> Result<BatchContent> {
match content {
BatchContent::DocumentAddtitionBatch(tasks) => {
let tasks = self.store.update_tasks(tasks).await?;
Ok(BatchContent::DocumentAddtitionBatch(tasks))
}
BatchContent::IndexUpdate(t) => {
let mut tasks = self.store.update_tasks(vec![t]).await?;
Ok(BatchContent::IndexUpdate(tasks.remove(0)))
}
BatchContent::Dump(t) => {
let mut tasks = self.store.update_tasks(vec![t]).await?;
Ok(BatchContent::Dump(tasks.remove(0)))
}
other => Ok(other),
}
}
pub async fn get_task(&self, id: TaskId, filter: Option<TaskFilter>) -> Result<Task> {
@ -318,16 +338,16 @@ impl Scheduler {
pub async fn get_processing_tasks(&self) -> Result<Vec<Task>> {
let mut tasks = Vec::new();
for id in self.processing.iter() {
let task = self.store.get_task(*id, None).await?;
for id in self.processing.ids() {
let task = self.store.get_task(id, None).await?;
tasks.push(task);
}
Ok(tasks)
}
pub async fn schedule_job(&mut self, job: Job) {
self.jobs.push_back(job);
pub async fn schedule_snapshot(&mut self, job: SnapshotJob) {
self.snapshots.push_back(job);
self.notify();
}
@ -353,106 +373,163 @@ impl Scheduler {
}
/// Prepare the next batch, and set `processing` to the ids in that batch.
pub async fn prepare(&mut self) -> Result<Pending> {
pub async fn prepare(&mut self) -> Result<Batch> {
// If there is a job to process, do it first.
if let Some(job) = self.jobs.pop_front() {
if let Some(job) = self.snapshots.pop_front() {
// There is more work to do, notify the update loop
self.notify_if_not_empty();
return Ok(Pending::Job(job));
let batch = Batch::new(None, BatchContent::Snapshot(job));
return Ok(batch);
}
// Try to fill the queue with pending tasks.
self.fetch_pending_tasks().await?;
make_batch(&mut self.tasks, &mut self.processing, &self.config);
self.processing = make_batch(&mut self.tasks, &self.config);
log::debug!("prepared batch with {} tasks", self.processing.len());
if !self.processing.is_empty() {
let ids = std::mem::take(&mut self.processing);
if !self.processing.is_nothing() {
let (processing, mut content) = self
.store
.get_processing_tasks(std::mem::take(&mut self.processing))
.await?;
let (ids, mut tasks) = self.store.get_pending_tasks(ids).await?;
// The batch id is the id of the first update it contains
let id = match tasks.first() {
// The batch id is the id of the first update it contains. At this point we must have a
// valid batch that contains at least 1 task.
let id = match content.first() {
Some(Task { id, .. }) => *id,
_ => panic!("invalid batch"),
};
tasks.iter_mut().for_each(|t| {
t.events.push(TaskEvent::Batched {
batch_id: id,
timestamp: OffsetDateTime::now_utc(),
})
content.push_event(TaskEvent::Batched {
batch_id: id,
timestamp: OffsetDateTime::now_utc(),
});
self.processing = ids;
self.processing = processing;
let batch = Batch {
id,
created_at: OffsetDateTime::now_utc(),
tasks,
};
let batch = Batch::new(Some(id), content);
// There is more work to do, notify the update loop
self.notify_if_not_empty();
Ok(Pending::Batch(batch))
Ok(batch)
} else {
Ok(Pending::Nothing)
Ok(Batch::empty())
}
}
}
#[derive(Debug)]
pub enum Pending {
Batch(Batch),
Job(Job),
#[derive(Debug, Default)]
pub enum Processing {
DocumentAdditions(Vec<TaskId>),
IndexUpdate(TaskId),
Dump(TaskId),
/// Variant used when there is nothing to process.
#[default]
Nothing,
}
fn make_batch(tasks: &mut TaskQueue, processing: &mut Vec<TaskId>, config: &SchedulerConfig) {
processing.clear();
enum ProcessingIter<'a> {
Many(slice::Iter<'a, TaskId>),
Single(Option<TaskId>),
}
let mut doc_count = 0;
tasks.head_mut(|list| match list.peek().copied() {
Some(PendingTask {
kind: TaskType::Other,
id,
}) => {
processing.push(id);
list.pop();
impl<'a> Iterator for ProcessingIter<'a> {
type Item = TaskId;
fn next(&mut self) -> Option<Self::Item> {
match self {
ProcessingIter::Many(iter) => iter.next().copied(),
ProcessingIter::Single(val) => val.take(),
}
Some(PendingTask { kind, .. }) => loop {
match list.peek() {
Some(pending) if pending.kind == kind => {
// We always need to process at least one task for the scheduler to make progress.
if processing.len() >= config.max_batch_size.unwrap_or(usize::MAX).max(1) {
break;
}
let pending = list.pop().unwrap();
processing.push(pending.id);
}
}
// We add the number of documents to the count if we are scheduling document additions and
// stop adding if we already have enough.
//
// We check that bound only after adding the current task to the batch, so that a batch contains at least one task.
match pending.kind {
TaskType::DocumentUpdate { number }
| TaskType::DocumentAddition { number } => {
doc_count += number;
impl Processing {
fn is_nothing(&self) -> bool {
matches!(self, Processing::Nothing)
}
if doc_count >= config.max_documents_per_batch.unwrap_or(usize::MAX) {
pub fn ids(&self) -> impl Iterator<Item = TaskId> + '_ {
match self {
Processing::DocumentAdditions(v) => ProcessingIter::Many(v.iter()),
Processing::IndexUpdate(id) | Processing::Dump(id) => ProcessingIter::Single(Some(*id)),
Processing::Nothing => ProcessingIter::Single(None),
}
}
pub fn len(&self) -> usize {
match self {
Processing::DocumentAdditions(v) => v.len(),
Processing::IndexUpdate(_) | Processing::Dump(_) => 1,
Processing::Nothing => 0,
}
}
pub fn is_empty(&self) -> bool {
self.len() == 0
}
}
fn make_batch(tasks: &mut TaskQueue, config: &SchedulerConfig) -> Processing {
let mut doc_count = 0;
tasks
.head_mut(|list| match list.peek().copied() {
Some(PendingTask {
kind: TaskType::IndexUpdate,
id,
}) => {
list.pop();
Processing::IndexUpdate(id)
}
Some(PendingTask {
kind: TaskType::Dump,
id,
}) => {
list.pop();
Processing::Dump(id)
}
Some(PendingTask { kind, .. }) => {
let mut task_list = Vec::new();
loop {
match list.peek() {
Some(pending) if pending.kind == kind => {
// We always need to process at least one task for the scheduler to make progress.
if task_list.len() >= config.max_batch_size.unwrap_or(usize::MAX).max(1)
{
break;
}
let pending = list.pop().unwrap();
task_list.push(pending.id);
// We add the number of documents to the count if we are scheduling document additions and
// stop adding if we already have enough.
//
// We check that bound only after adding the current task to the batch, so that a batch contains at least one task.
match pending.kind {
TaskType::DocumentUpdate { number }
| TaskType::DocumentAddition { number } => {
doc_count += number;
if doc_count
>= config.max_documents_per_batch.unwrap_or(usize::MAX)
{
break;
}
}
_ => (),
}
}
_ => (),
_ => break,
}
}
_ => break,
Processing::DocumentAdditions(task_list)
}
},
None => (),
});
None => Processing::Nothing,
})
.unwrap_or(Processing::Nothing)
}
#[cfg(test)]

View file

@ -4,14 +4,12 @@ use meilisearch_error::ResponseError;
use milli::update::{DocumentAdditionResult, IndexDocumentsMethod};
use serde::{Deserialize, Serialize};
use time::OffsetDateTime;
use tokio::sync::oneshot;
use uuid::Uuid;
use super::batch::BatchId;
use crate::{
index::{Settings, Unchecked},
index_resolver::{error::IndexResolverError, IndexUid},
snapshot::SnapshotJob,
index_resolver::IndexUid,
};
pub type TaskId = u64;
@ -110,33 +108,6 @@ impl Task {
}
}
/// A job is like a volatile priority `Task`.
/// It should be processed as fast as possible and is not stored on disk.
/// This means, when Meilisearch is closed all your unprocessed jobs will disappear.
#[derive(Debug, derivative::Derivative)]
#[derivative(PartialEq)]
pub enum Job {
Dump {
#[derivative(PartialEq = "ignore")]
ret: oneshot::Sender<Result<oneshot::Sender<()>, IndexResolverError>>,
path: PathBuf,
},
Snapshot(#[derivative(PartialEq = "ignore")] SnapshotJob),
Empty,
}
impl Default for Job {
fn default() -> Self {
Self::Empty
}
}
impl Job {
pub fn take(&mut self) -> Self {
std::mem::take(self)
}
}
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
#[cfg_attr(test, derive(proptest_derive::Arbitrary))]
pub enum DocumentDeletion {

View file

@ -9,7 +9,9 @@ use log::debug;
use milli::heed::{Env, RwTxn};
use time::OffsetDateTime;
use super::batch::BatchContent;
use super::error::TaskError;
use super::scheduler::Processing;
use super::task::{Task, TaskContent, TaskId};
use super::Result;
use crate::index_resolver::IndexUid;
@ -122,19 +124,44 @@ impl TaskStore {
}
}
pub async fn get_pending_tasks(&self, ids: Vec<TaskId>) -> Result<(Vec<TaskId>, Vec<Task>)> {
/// This methods takes a `Processing` which contains the next task ids to process, and returns
/// the coresponding tasks along with the ownership to the passed processing.
///
/// We need get_processing_tasks to take ownership over `Processing` because we need it to be
/// valid for 'static.
pub async fn get_processing_tasks(
&self,
processing: Processing,
) -> Result<(Processing, BatchContent)> {
let store = self.store.clone();
let tasks = tokio::task::spawn_blocking(move || -> Result<_> {
let mut tasks = Vec::new();
let txn = store.rtxn()?;
for id in ids.iter() {
let task = store
.get(&txn, *id)?
.ok_or(TaskError::UnexistingTask(*id))?;
tasks.push(task);
}
Ok((ids, tasks))
let content = match processing {
Processing::DocumentAdditions(ref ids) => {
let mut tasks = Vec::new();
for id in ids.iter() {
let task = store
.get(&txn, *id)?
.ok_or(TaskError::UnexistingTask(*id))?;
tasks.push(task);
}
BatchContent::DocumentAddtitionBatch(tasks)
}
Processing::IndexUpdate(id) => {
let task = store.get(&txn, id)?.ok_or(TaskError::UnexistingTask(id))?;
BatchContent::IndexUpdate(task)
}
Processing::Dump(id) => {
let task = store.get(&txn, id)?.ok_or(TaskError::UnexistingTask(id))?;
debug_assert!(matches!(task.content, TaskContent::Dump { .. }));
BatchContent::Dump(task)
}
Processing::Nothing => unreachable!(),
};
Ok((processing, content))
})
.await??;
@ -231,7 +258,7 @@ impl TaskStore {
#[cfg(test)]
pub mod test {
use crate::tasks::task_store::store::test::tmp_env;
use crate::tasks::{scheduler::Processing, task_store::store::test::tmp_env};
use super::*;
@ -280,12 +307,12 @@ pub mod test {
}
}
pub async fn get_pending_tasks(
pub async fn get_processing_tasks(
&self,
tasks: Vec<TaskId>,
) -> Result<(Vec<TaskId>, Vec<Task>)> {
tasks: Processing,
) -> Result<(Processing, BatchContent)> {
match self {
Self::Real(s) => s.get_pending_tasks(tasks).await,
Self::Real(s) => s.get_processing_tasks(tasks).await,
Self::Mock(m) => unsafe { m.get("get_pending_task").call(tasks) },
}
}

View file

@ -7,33 +7,29 @@ use tokio::time::interval_at;
use super::batch::Batch;
use super::error::Result;
use super::scheduler::Pending;
use super::{Scheduler, TaskPerformer};
use super::{BatchHandler, Scheduler};
use crate::tasks::task::TaskEvent;
/// The update loop sequentially performs batches of updates by asking the scheduler for a batch,
/// and handing it to the `TaskPerformer`.
pub struct UpdateLoop<P: TaskPerformer> {
pub struct UpdateLoop {
scheduler: Arc<RwLock<Scheduler>>,
performer: Arc<P>,
performers: Vec<Arc<dyn BatchHandler + Send + Sync + 'static>>,
notifier: Option<watch::Receiver<()>>,
debounce_duration: Option<Duration>,
}
impl<P> UpdateLoop<P>
where
P: TaskPerformer + Send + Sync + 'static,
{
impl UpdateLoop {
pub fn new(
scheduler: Arc<RwLock<Scheduler>>,
performer: Arc<P>,
performers: Vec<Arc<dyn BatchHandler + Send + Sync + 'static>>,
debuf_duration: Option<Duration>,
notifier: watch::Receiver<()>,
) -> Self {
Self {
scheduler,
performer,
performers,
debounce_duration: debuf_duration,
notifier: Some(notifier),
}
@ -59,34 +55,29 @@ where
}
async fn process_next_batch(&self) -> Result<()> {
let pending = { self.scheduler.write().await.prepare().await? };
match pending {
Pending::Batch(mut batch) => {
for task in &mut batch.tasks {
task.events
.push(TaskEvent::Processing(OffsetDateTime::now_utc()));
}
let mut batch = { self.scheduler.write().await.prepare().await? };
let performer = self
.performers
.iter()
.find(|p| p.accept(&batch))
.expect("No performer found for batch")
.clone();
batch.tasks = {
self.scheduler
.read()
.await
.update_tasks(batch.tasks)
.await?
};
batch
.content
.push_event(TaskEvent::Processing(OffsetDateTime::now_utc()));
let performer = self.performer.clone();
batch.content = {
self.scheduler
.read()
.await
.update_tasks(batch.content)
.await?
};
let batch = performer.process_batch(batch).await;
let batch = performer.process_batch(batch).await;
self.handle_batch_result(batch).await?;
}
Pending::Job(job) => {
let performer = self.performer.clone();
performer.process_job(job).await;
}
Pending::Nothing => (),
}
self.handle_batch_result(batch, performer).await?;
Ok(())
}
@ -96,13 +87,17 @@ where
/// When a task is processed, the result of the process is pushed to its event list. The
/// `handle_batch_result` make sure that the new state is saved to the store.
/// The tasks are then removed from the processing queue.
async fn handle_batch_result(&self, mut batch: Batch) -> Result<()> {
async fn handle_batch_result(
&self,
mut batch: Batch,
performer: Arc<dyn BatchHandler + Sync + Send + 'static>,
) -> Result<()> {
let mut scheduler = self.scheduler.write().await;
let tasks = scheduler.update_tasks(batch.tasks).await?;
let content = scheduler.update_tasks(batch.content).await?;
scheduler.finish();
drop(scheduler);
batch.tasks = tasks;
self.performer.finish(&batch).await;
batch.content = content;
performer.finish(&batch).await;
Ok(())
}
}