gst-plugins-rs/generic/threadshare/src/runtime/executor/task.rs
François Laignel 6163589ac7 ts/executor: replace tokio with smol-like implementation
The threadshare executor was based on a modified version of tokio
which implemented the throttling strategy in the BasicScheduler.
Upstream tokio codebase has significantly diverged from what it
was when the throttling strategy was implemented making it hard
to follow. This means that we can hardly get updates from the
upstream project and when we cherry pick fixes, we can't reflect
the state of the project on our fork's version. As a consequence,
tools such as cargo-deny can't check for RUSTSEC fixes in our fork.

The smol ecosystem makes it quite easy to implement and maintain
a custom async executor. This MR imports the smol parts that
need modifications to comply with the threadshare model and implements
a throttling executor in place of the tokio fork.

Networking tokio specific types are replaced with Async wrappers
in the spirit of [smol-rs/async-io]. Note however that the Async
wrappers needed modifications in order to use the per thread
Reactor model. This means that higher level upstream networking
crates such as [async-net] can not be used with our Async
implementation.

Based on the example benchmark with ts-udpsrc, performances seem on par
with what we achieved using the tokio fork.

Fixes https://gitlab.freedesktop.org/gstreamer/gst-plugins-rs/-/issues/118

Related to https://gitlab.freedesktop.org/gstreamer/gst-plugins-rs/-/merge_requests/604
2021-12-25 11:25:56 +00:00

300 lines
8.1 KiB
Rust

// Copyright (C) 2018-2020 Sebastian Dröge <sebastian@centricular.com>
// Copyright (C) 2019-2021 François Laignel <fengalin@free.fr>
//
// Take a look at the license at the top of the repository in the LICENSE file.
use async_task::Runnable;
use concurrent_queue::ConcurrentQueue;
use futures::future::BoxFuture;
use futures::prelude::*;
use gst::{gst_log, gst_trace, gst_warning};
use pin_project_lite::pin_project;
use slab::Slab;
use std::cell::Cell;
use std::collections::VecDeque;
use std::fmt;
use std::pin::Pin;
use std::sync::{Arc, Mutex};
use std::task::Poll;
use super::CallOnDrop;
use crate::runtime::RUNTIME_CAT;
thread_local! {
static CURRENT_TASK_ID: Cell<Option<TaskId>> = Cell::new(None);
}
#[derive(Clone, Copy, Eq, PartialEq, Hash, Debug)]
pub struct TaskId(pub(super) usize);
impl TaskId {
pub(super) fn current() -> Option<TaskId> {
CURRENT_TASK_ID.try_with(Cell::get).ok().flatten()
}
}
pub type SubTaskOutput = Result<(), gst::FlowError>;
pin_project! {
pub(super) struct TaskFuture<F: Future> {
id: TaskId,
#[pin]
future: F,
}
}
impl<F: Future> Future for TaskFuture<F> {
type Output = F::Output;
fn poll(self: Pin<&mut Self>, cx: &mut std::task::Context<'_>) -> Poll<Self::Output> {
struct TaskIdGuard {
prev_task_id: Option<TaskId>,
}
impl Drop for TaskIdGuard {
fn drop(&mut self) {
let _ = CURRENT_TASK_ID.try_with(|cur| cur.replace(self.prev_task_id.take()));
}
}
let task_id = self.id;
let project = self.project();
let _guard = TaskIdGuard {
prev_task_id: CURRENT_TASK_ID.with(|cur| cur.replace(Some(task_id))),
};
project.future.poll(cx)
}
}
struct Task {
id: TaskId,
sub_tasks: VecDeque<BoxFuture<'static, SubTaskOutput>>,
}
impl Task {
fn new(id: TaskId) -> Self {
Task {
id,
sub_tasks: VecDeque::new(),
}
}
fn add_sub_task<T>(&mut self, sub_task: T)
where
T: Future<Output = SubTaskOutput> + Send + 'static,
{
self.sub_tasks.push_back(sub_task.boxed());
}
fn drain_sub_tasks(&mut self) -> VecDeque<BoxFuture<'static, SubTaskOutput>> {
std::mem::take(&mut self.sub_tasks)
}
}
impl fmt::Debug for Task {
fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result {
fmt.debug_struct("Task")
.field("id", &self.id)
.field("sub_tasks len", &self.sub_tasks.len())
.finish()
}
}
#[derive(Debug)]
pub(super) struct TaskQueue {
runnables: Arc<ConcurrentQueue<Runnable>>,
// FIXME good point about using a slab is that it's probably faster than a HashMap
// However since we reuse the vacant entries, we get the same TaskId
// which can harm debugging. If this is not acceptable, I'll switch back to using
// a HashMap.
tasks: Arc<Mutex<Slab<Task>>>,
context_name: Arc<str>,
}
impl TaskQueue {
pub fn new(context_name: Arc<str>) -> Self {
TaskQueue {
runnables: Arc::new(ConcurrentQueue::unbounded()),
tasks: Arc::new(Mutex::new(Slab::new())),
context_name,
}
}
pub fn add<F>(&self, future: F) -> (TaskId, async_task::Task<<F as Future>::Output>)
where
F: Future + Send + 'static,
F::Output: Send + 'static,
{
let tasks_clone = Arc::clone(&self.tasks);
let mut tasks = self.tasks.lock().unwrap();
let task_id = TaskId(tasks.vacant_entry().key());
let context_name = Arc::clone(&self.context_name);
let task_fut = async move {
gst_trace!(
RUNTIME_CAT,
"Running {:?} on context {}",
task_id,
context_name
);
let _guard = CallOnDrop::new(move || {
if let Some(task) = tasks_clone.lock().unwrap().try_remove(task_id.0) {
if !task.sub_tasks.is_empty() {
gst_warning!(
RUNTIME_CAT,
"Task {:?} on context {} has {} pending sub tasks",
task_id,
context_name,
task.sub_tasks.len(),
);
}
}
gst_trace!(
RUNTIME_CAT,
"Done {:?} on context {}",
task_id,
context_name
);
});
TaskFuture {
id: task_id,
future,
}
.await
};
let runnables = Arc::clone(&self.runnables);
let (runnable, task) = async_task::spawn(task_fut, move |runnable| {
runnables.push(runnable).unwrap();
});
tasks.insert(Task::new(task_id));
drop(tasks);
runnable.schedule();
(task_id, task)
}
pub fn add_sync<F, O>(&self, f: F) -> async_task::Task<O>
where
F: FnOnce() -> O + Send + 'static,
O: Send + 'static,
{
let tasks_clone = Arc::clone(&self.tasks);
let mut tasks = self.tasks.lock().unwrap();
let task_id = TaskId(tasks.vacant_entry().key());
let context_name = Arc::clone(&self.context_name);
let task_fut = async move {
gst_trace!(
RUNTIME_CAT,
"Executing sync function on context {} as {:?}",
context_name,
task_id,
);
let _guard = CallOnDrop::new(move || {
let _ = tasks_clone.lock().unwrap().try_remove(task_id.0);
gst_trace!(
RUNTIME_CAT,
"Done executing sync function on context {} as {:?}",
context_name,
task_id,
);
});
f()
};
let runnables = Arc::clone(&self.runnables);
let (runnable, task) = async_task::spawn(task_fut, move |runnable| {
runnables.push(runnable).unwrap();
});
tasks.insert(Task::new(task_id));
drop(tasks);
runnable.schedule();
task
}
pub fn pop_runnable(&self) -> Result<Runnable, concurrent_queue::PopError> {
self.runnables.pop()
}
pub fn has_sub_tasks(&self, task_id: TaskId) -> bool {
self.tasks
.lock()
.unwrap()
.get(task_id.0)
.map(|t| !t.sub_tasks.is_empty())
.unwrap_or(false)
}
pub fn add_sub_task<T>(&self, task_id: TaskId, sub_task: T) -> Result<(), T>
where
T: Future<Output = SubTaskOutput> + Send + 'static,
{
match self.tasks.lock().unwrap().get_mut(task_id.0) {
Some(task) => {
gst_trace!(
RUNTIME_CAT,
"Adding subtask to {:?} on context {}",
task_id,
self.context_name
);
task.add_sub_task(sub_task);
Ok(())
}
None => {
gst_trace!(RUNTIME_CAT, "Task was removed in the meantime");
Err(sub_task)
}
}
}
pub fn drain_sub_tasks(
&self,
task_id: TaskId,
) -> impl Future<Output = SubTaskOutput> + Send + 'static {
let sub_tasks = self
.tasks
.lock()
.unwrap()
.get_mut(task_id.0)
.map(|task| (task.drain_sub_tasks(), Arc::clone(&self.context_name)));
async move {
if let Some((mut sub_tasks, context_name)) = sub_tasks {
if !sub_tasks.is_empty() {
gst_log!(
RUNTIME_CAT,
"Scheduling draining {} sub tasks from {:?} on '{}'",
sub_tasks.len(),
task_id,
&context_name,
);
for sub_task in sub_tasks.drain(..) {
sub_task.await?;
}
}
}
Ok(())
}
}
}