mirror of
https://github.com/esp-rs/esp-hal.git
synced 2025-09-27 20:30:35 +00:00
esp-preempt: don't switch to sleeping tasks (#4081)
* Separate the alloc and run lists * Replace circular task list with ready queue * Remove separate SCHEDULER_STATE static * Move scheduler to new file * Reorganize, allow restarting scheduler * Fix InternalMemory polyfill * Use SingleShotTimer internally * Implement a simple timer queue * Extract run queue, wake tasks, store reason of scheduler event * Add inherent function to get current task ptr * Reimplement usleep with the timer queue * Store current task in timer queue * Sleep in timer queue task * Remove ability to sleep arbitrary tasks * More logging * Clear timer interrupt in timer handler * Even more logging * Merge mutexes into semaphores
This commit is contained in:
parent
e888f574fb
commit
779f7a874f
@ -49,6 +49,10 @@ esp-hal = { version = "1.0.0-rc.0", path = "../esp-hal", features = ["unstable"]
|
||||
default = ["esp-alloc"]
|
||||
|
||||
## Enable the use of the `esp-alloc` crate for dynamic memory allocation.
|
||||
##
|
||||
## If you opt-out, you need to provide implementations for the following functions:
|
||||
## - `pub extern "C" fn malloc_internal(size: usize) -> *mut u8`
|
||||
## - `pub extern "C" fn free_internal(ptr: *mut u8)`
|
||||
esp-alloc = ["dep:esp-alloc"]
|
||||
|
||||
#! ### Chip selection
|
||||
|
@ -23,27 +23,24 @@ extern crate alloc;
|
||||
// MUST be the first module
|
||||
mod fmt;
|
||||
|
||||
mod mutex;
|
||||
mod queue;
|
||||
mod run_queue;
|
||||
mod scheduler;
|
||||
mod semaphore;
|
||||
mod task;
|
||||
mod timer;
|
||||
mod timer_queue;
|
||||
|
||||
use core::ffi::c_void;
|
||||
|
||||
use allocator_api2::boxed::Box;
|
||||
pub(crate) use esp_alloc::InternalMemory;
|
||||
use esp_hal::{
|
||||
Blocking,
|
||||
time::{Duration, Instant, Rate},
|
||||
timer::{AnyTimer, PeriodicTimer},
|
||||
timer::{AnyTimer, OneShotTimer},
|
||||
};
|
||||
use esp_radio_preempt_driver::semaphore::{SemaphoreImplementation, SemaphorePtr};
|
||||
use esp_sync::NonReentrantMutex;
|
||||
pub(crate) use scheduler::SCHEDULER;
|
||||
|
||||
use crate::{semaphore::Semaphore, task::Context, timer::TIMER};
|
||||
use crate::timer::TimeDriver;
|
||||
|
||||
type TimeBase = PeriodicTimer<'static, Blocking>;
|
||||
type TimeBase = OneShotTimer<'static, Blocking>;
|
||||
|
||||
// Polyfill the InternalMemory allocator
|
||||
#[cfg(not(feature = "esp-alloc"))]
|
||||
@ -52,32 +49,28 @@ mod esp_alloc {
|
||||
|
||||
use allocator_api2::alloc::{AllocError, Allocator};
|
||||
|
||||
unsafe extern "C" {
|
||||
fn malloc_internal(size: usize) -> *mut u8;
|
||||
|
||||
fn free_internal(ptr: *mut u8);
|
||||
}
|
||||
|
||||
/// An allocator that uses internal memory only.
|
||||
pub struct InternalMemory;
|
||||
|
||||
unsafe impl Allocator for InternalMemory {
|
||||
fn allocate(&self, layout: Layout) -> Result<NonNull<[u8]>, AllocError> {
|
||||
unsafe extern "C" {
|
||||
fn esp_radio_allocate_from_internal_ram(size: usize) -> *mut u8;
|
||||
}
|
||||
let raw_ptr = unsafe { esp_radio_allocate_from_internal_ram(layout.size()) };
|
||||
let raw_ptr = unsafe { malloc_internal(layout.size()) };
|
||||
let ptr = NonNull::new(raw_ptr).ok_or(AllocError)?;
|
||||
Ok(NonNull::slice_from_raw_parts(ptr, layout.size()))
|
||||
}
|
||||
|
||||
unsafe fn deallocate(&self, ptr: NonNull<u8>, _layout: Layout) {
|
||||
unsafe extern "C" {
|
||||
fn esp_radio_deallocate_internal_ram(ptr: *mut u8);
|
||||
}
|
||||
unsafe {
|
||||
esp_radio_deallocate_internal_ram(ptr.as_ptr());
|
||||
}
|
||||
unsafe { free_internal(ptr.as_ptr()) };
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) use esp_alloc::InternalMemory;
|
||||
|
||||
/// A trait to allow better UX for initializing esp-preempt.
|
||||
///
|
||||
/// This trait is meant to be used only for the `init` function.
|
||||
@ -107,174 +100,6 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
struct SchedulerState {
|
||||
/// Pointer to the current task.
|
||||
///
|
||||
/// Tasks are stored in a circular linked list. CTX_NOW points to the
|
||||
/// current task.
|
||||
current_task: *mut Context,
|
||||
|
||||
/// Pointer to the task that is scheduled for deletion.
|
||||
to_delete: *mut Context,
|
||||
}
|
||||
|
||||
unsafe impl Send for SchedulerState {}
|
||||
|
||||
impl SchedulerState {
|
||||
const fn new() -> Self {
|
||||
Self {
|
||||
current_task: core::ptr::null_mut(),
|
||||
to_delete: core::ptr::null_mut(),
|
||||
}
|
||||
}
|
||||
|
||||
fn delete_task(&mut self, task: *mut Context) {
|
||||
let mut current_task = self.current_task;
|
||||
// Save the first pointer so we can prevent an accidental infinite loop.
|
||||
let initial = current_task;
|
||||
loop {
|
||||
// We don't have the previous pointer, so we need to walk forward in the circle
|
||||
// even if we need to delete the first task.
|
||||
|
||||
// If the next task is the one we want to delete, we need to remove it from the
|
||||
// list, then drop it.
|
||||
let next_task = unsafe { (*current_task).next };
|
||||
if core::ptr::eq(next_task, task) {
|
||||
unsafe {
|
||||
(*current_task).next = (*next_task).next;
|
||||
|
||||
core::ptr::drop_in_place(task);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// If the next task is the first task, we can stop. If we needed to delete the
|
||||
// first task, we have already handled it in the above case. If we needed to
|
||||
// delete another task, it has already been deleted in a previous iteration.
|
||||
if core::ptr::eq(next_task, initial) {
|
||||
break;
|
||||
}
|
||||
|
||||
// Move to the next task.
|
||||
current_task = next_task;
|
||||
}
|
||||
}
|
||||
|
||||
fn delete_marked_tasks(&mut self) {
|
||||
while !self.to_delete.is_null() {
|
||||
let task_to_delete = core::mem::take(&mut self.to_delete);
|
||||
self.to_delete = unsafe { (*task_to_delete).next_to_delete };
|
||||
self.delete_task(task_to_delete);
|
||||
}
|
||||
}
|
||||
|
||||
fn select_next_task(&mut self) -> Option<*mut Context> {
|
||||
let mut current = self.current_task;
|
||||
loop {
|
||||
let next_task = unsafe { (*current).next };
|
||||
|
||||
if next_task == self.current_task {
|
||||
// We didn't find a new task to switch to.
|
||||
// TODO: mark the current task as Running
|
||||
// Once we have actual task states, yield should marked the current task as Ready,
|
||||
// other stuff as Waiting.
|
||||
return None;
|
||||
}
|
||||
|
||||
if unsafe { (*next_task).state }.is_ready() {
|
||||
// TODO: mark the selected task as Running
|
||||
return Some(next_task);
|
||||
}
|
||||
current = next_task;
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(xtensa)]
|
||||
fn switch_task(&mut self, trap_frame: &mut esp_hal::trapframe::TrapFrame) {
|
||||
self.delete_marked_tasks();
|
||||
|
||||
let Some(next_task) = self.select_next_task() else {
|
||||
return;
|
||||
};
|
||||
|
||||
task::save_task_context(unsafe { &mut *self.current_task }, trap_frame);
|
||||
|
||||
self.current_task = next_task;
|
||||
|
||||
task::restore_task_context(unsafe { &mut *self.current_task }, trap_frame);
|
||||
}
|
||||
|
||||
#[cfg(riscv)]
|
||||
fn switch_task(&mut self) {
|
||||
self.delete_marked_tasks();
|
||||
|
||||
let Some(next_task) = self.select_next_task() else {
|
||||
return;
|
||||
};
|
||||
|
||||
let old_ctx = unsafe { &raw mut (*self.current_task).trap_frame };
|
||||
let new_ctx = unsafe { &raw mut (*next_task).trap_frame };
|
||||
|
||||
if crate::task::arch_specific::task_switch(old_ctx, new_ctx) {
|
||||
unsafe { self.current_task = (*self.current_task).next };
|
||||
}
|
||||
}
|
||||
|
||||
fn schedule_task_deletion(&mut self, mut task_to_delete: *mut Context) -> bool {
|
||||
if task_to_delete.is_null() {
|
||||
task_to_delete = self.current_task;
|
||||
}
|
||||
let is_current = core::ptr::eq(task_to_delete, self.current_task);
|
||||
|
||||
unsafe { (*task_to_delete).next_to_delete = self.to_delete };
|
||||
self.to_delete = task_to_delete;
|
||||
|
||||
is_current
|
||||
}
|
||||
}
|
||||
|
||||
fn usleep(us: u32) {
|
||||
trace!("usleep");
|
||||
unsafe extern "C" {
|
||||
fn esp_rom_delay_us(us: u32);
|
||||
}
|
||||
|
||||
const MIN_YIELD_TIME: u32 = 1_000_000 / TICK_RATE;
|
||||
if us < MIN_YIELD_TIME {
|
||||
// Short wait, just sleep
|
||||
unsafe { esp_rom_delay_us(us) };
|
||||
} else {
|
||||
const MIN_YIELD_DURATION: Duration = Duration::from_micros(MIN_YIELD_TIME as u64);
|
||||
let sleep_for = Duration::from_micros(us as u64);
|
||||
let start = Instant::now();
|
||||
loop {
|
||||
// Yield to other tasks
|
||||
timer::yield_task();
|
||||
|
||||
let elapsed = start.elapsed();
|
||||
if elapsed.as_micros() > us as u64 {
|
||||
break;
|
||||
}
|
||||
|
||||
let remaining = sleep_for - elapsed;
|
||||
|
||||
if remaining < MIN_YIELD_DURATION {
|
||||
// If the remaining time is less than the minimum yield time, we can just sleep
|
||||
// for the remaining time.
|
||||
unsafe { esp_rom_delay_us(remaining.as_micros() as u32) };
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static SCHEDULER_STATE: NonReentrantMutex<SchedulerState> =
|
||||
NonReentrantMutex::new(SchedulerState::new());
|
||||
|
||||
struct Scheduler {}
|
||||
|
||||
esp_radio_preempt_driver::scheduler_impl!(static SCHEDULER: Scheduler = Scheduler {});
|
||||
|
||||
/// Initializes the scheduler.
|
||||
///
|
||||
/// # The `timer` argument
|
||||
@ -288,99 +113,7 @@ esp_radio_preempt_driver::scheduler_impl!(static SCHEDULER: Scheduler = Schedule
|
||||
///
|
||||
/// For an example, see the [crate-level documentation][self].
|
||||
pub fn init(timer: impl TimerSource) {
|
||||
timer::setup_timebase(timer.timer());
|
||||
SCHEDULER.with(move |scheduler| scheduler.set_time_driver(TimeDriver::new(timer.timer())))
|
||||
}
|
||||
|
||||
const TICK_RATE: u32 = esp_config::esp_config_int!(u32, "ESP_PREEMPT_CONFIG_TICK_RATE_HZ");
|
||||
const TIMESLICE_FREQUENCY: Rate = Rate::from_hz(TICK_RATE);
|
||||
|
||||
impl esp_radio_preempt_driver::Scheduler for Scheduler {
|
||||
fn initialized(&self) -> bool {
|
||||
timer::initialized()
|
||||
}
|
||||
|
||||
fn enable(&self) {
|
||||
// allocate the main task
|
||||
task::allocate_main_task();
|
||||
timer::setup_multitasking();
|
||||
timer_queue::create_timer_task();
|
||||
|
||||
TIMER.with(|t| {
|
||||
let t = unwrap!(t.as_mut());
|
||||
unwrap!(t.start(TIMESLICE_FREQUENCY.as_duration()));
|
||||
});
|
||||
}
|
||||
|
||||
fn disable(&self) {
|
||||
timer::disable_timebase();
|
||||
timer::disable_multitasking();
|
||||
task::delete_all_tasks();
|
||||
}
|
||||
|
||||
fn yield_task(&self) {
|
||||
timer::yield_task()
|
||||
}
|
||||
|
||||
fn yield_task_from_isr(&self) {
|
||||
self.yield_task();
|
||||
}
|
||||
|
||||
fn max_task_priority(&self) -> u32 {
|
||||
255
|
||||
}
|
||||
|
||||
fn task_create(
|
||||
&self,
|
||||
task: extern "C" fn(*mut c_void),
|
||||
param: *mut c_void,
|
||||
_priority: u32,
|
||||
_pin_to_core: Option<u32>,
|
||||
task_stack_size: usize,
|
||||
) -> *mut c_void {
|
||||
let task = Box::new_in(Context::new(task, param, task_stack_size), InternalMemory);
|
||||
let task_ptr = Box::into_raw(task);
|
||||
|
||||
SCHEDULER_STATE.with(|state| unsafe {
|
||||
let current_task = state.current_task;
|
||||
debug_assert!(
|
||||
!current_task.is_null(),
|
||||
"Tried to allocate a task before allocating the main task"
|
||||
);
|
||||
// Insert the new task at the next position.
|
||||
let next = (*current_task).next;
|
||||
(*task_ptr).next = next;
|
||||
(*current_task).next = task_ptr;
|
||||
});
|
||||
|
||||
task_ptr as *mut c_void
|
||||
}
|
||||
|
||||
fn current_task(&self) -> *mut c_void {
|
||||
task::current_task() as *mut c_void
|
||||
}
|
||||
|
||||
fn schedule_task_deletion(&self, task_handle: *mut c_void) {
|
||||
task::schedule_task_deletion(task_handle as *mut Context)
|
||||
}
|
||||
|
||||
fn current_task_thread_semaphore(&self) -> SemaphorePtr {
|
||||
task::with_current_task(|task| {
|
||||
if task.thread_semaphore.is_none() {
|
||||
task.thread_semaphore = Some(Semaphore::create(1, 0));
|
||||
}
|
||||
|
||||
unwrap!(task.thread_semaphore)
|
||||
})
|
||||
}
|
||||
|
||||
fn usleep(&self, us: u32) {
|
||||
usleep(us)
|
||||
}
|
||||
|
||||
fn now(&self) -> u64 {
|
||||
// FIXME: this function needs to return the timestamp of the scheduler's timer
|
||||
esp_hal::time::Instant::now()
|
||||
.duration_since_epoch()
|
||||
.as_micros()
|
||||
}
|
||||
}
|
||||
|
@ -1,123 +0,0 @@
|
||||
use alloc::boxed::Box;
|
||||
use core::ptr::NonNull;
|
||||
|
||||
use esp_hal::time::{Duration, Instant};
|
||||
use esp_radio_preempt_driver::{
|
||||
current_task,
|
||||
mutex::{MutexImplementation, MutexPtr},
|
||||
register_mutex_implementation,
|
||||
yield_task,
|
||||
};
|
||||
use esp_sync::NonReentrantMutex;
|
||||
|
||||
struct MutexInner {
|
||||
recursive: bool,
|
||||
owner: usize,
|
||||
lock_counter: u32,
|
||||
}
|
||||
|
||||
impl MutexInner {
|
||||
fn try_lock(&mut self) -> bool {
|
||||
let current = current_task() as usize;
|
||||
|
||||
if self.owner == 0 || (self.owner == current && self.recursive) {
|
||||
self.owner = current;
|
||||
self.lock_counter += 1;
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
fn unlock(&mut self) -> bool {
|
||||
let current = current_task() as usize;
|
||||
|
||||
if self.owner == current && self.lock_counter > 0 {
|
||||
self.lock_counter -= 1;
|
||||
if self.lock_counter == 0 {
|
||||
self.owner = 0;
|
||||
}
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct Mutex {
|
||||
inner: NonReentrantMutex<MutexInner>,
|
||||
}
|
||||
|
||||
impl Mutex {
|
||||
pub fn new(recursive: bool) -> Self {
|
||||
Mutex {
|
||||
inner: NonReentrantMutex::new(MutexInner {
|
||||
recursive,
|
||||
owner: 0,
|
||||
lock_counter: 0,
|
||||
}),
|
||||
}
|
||||
}
|
||||
|
||||
unsafe fn from_ptr<'a>(ptr: MutexPtr) -> &'a Self {
|
||||
unsafe { ptr.cast::<Self>().as_ref() }
|
||||
}
|
||||
|
||||
fn yield_loop_with_timeout(timeout_us: Option<u32>, cb: impl Fn() -> bool) -> bool {
|
||||
let start = if timeout_us.is_some() {
|
||||
Instant::now()
|
||||
} else {
|
||||
Instant::EPOCH
|
||||
};
|
||||
|
||||
let timeout = timeout_us
|
||||
.map(|us| Duration::from_micros(us as u64))
|
||||
.unwrap_or(Duration::MAX);
|
||||
|
||||
loop {
|
||||
if cb() {
|
||||
return true;
|
||||
}
|
||||
|
||||
if timeout_us.is_some() && start.elapsed() > timeout {
|
||||
return false;
|
||||
}
|
||||
|
||||
yield_task();
|
||||
}
|
||||
}
|
||||
|
||||
pub fn lock(&self, timeout_us: Option<u32>) -> bool {
|
||||
Self::yield_loop_with_timeout(timeout_us, || self.inner.with(|mutex| mutex.try_lock()))
|
||||
}
|
||||
|
||||
pub fn unlock(&self) -> bool {
|
||||
self.inner.with(|mutex| mutex.unlock())
|
||||
}
|
||||
}
|
||||
|
||||
impl MutexImplementation for Mutex {
|
||||
fn create(recursive: bool) -> MutexPtr {
|
||||
let mutex = Box::new(Mutex::new(recursive));
|
||||
NonNull::from(Box::leak(mutex)).cast()
|
||||
}
|
||||
|
||||
unsafe fn delete(mutex: MutexPtr) {
|
||||
let mutex = unsafe { Box::from_raw(mutex.cast::<Mutex>().as_ptr()) };
|
||||
core::mem::drop(mutex);
|
||||
}
|
||||
|
||||
unsafe fn lock(mutex: MutexPtr, timeout_us: Option<u32>) -> bool {
|
||||
let mutex = unsafe { Mutex::from_ptr(mutex) };
|
||||
|
||||
mutex.lock(timeout_us)
|
||||
}
|
||||
|
||||
unsafe fn unlock(mutex: MutexPtr) -> bool {
|
||||
let mutex = unsafe { Mutex::from_ptr(mutex) };
|
||||
|
||||
mutex.unlock()
|
||||
}
|
||||
}
|
||||
|
||||
register_mutex_implementation!(Mutex);
|
40
esp-preempt/src/run_queue.rs
Normal file
40
esp-preempt/src/run_queue.rs
Normal file
@ -0,0 +1,40 @@
|
||||
use crate::task::{TaskPtr, TaskQueue, TaskReadyQueueElement, TaskState};
|
||||
|
||||
pub(crate) struct RunQueue {
|
||||
// TODO: one queue per priority level
|
||||
pub(crate) ready_tasks: TaskQueue<TaskReadyQueueElement>,
|
||||
}
|
||||
|
||||
impl RunQueue {
|
||||
pub(crate) const PRIORITY_LEVELS: usize = 32;
|
||||
|
||||
pub(crate) const fn new() -> Self {
|
||||
Self {
|
||||
ready_tasks: TaskQueue::new(),
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn mark_same_priority_task_ready(&mut self, ready_task: TaskPtr) {
|
||||
self.mark_task_ready(ready_task);
|
||||
}
|
||||
|
||||
pub(crate) fn mark_task_ready(&mut self, mut ready_task: TaskPtr) {
|
||||
// TODO: this will need to track max ready priority.
|
||||
unsafe { ready_task.as_mut().state = TaskState::Ready };
|
||||
self.ready_tasks.push(ready_task);
|
||||
}
|
||||
|
||||
pub(crate) fn pop(&mut self) -> Option<TaskPtr> {
|
||||
// TODO: on current prio level
|
||||
self.ready_tasks.pop()
|
||||
}
|
||||
|
||||
pub(crate) fn is_empty(&self) -> bool {
|
||||
// TODO: on current prio level
|
||||
self.ready_tasks.is_empty()
|
||||
}
|
||||
|
||||
pub(crate) fn remove(&mut self, to_delete: TaskPtr) {
|
||||
self.ready_tasks.remove(to_delete);
|
||||
}
|
||||
}
|
329
esp-preempt/src/scheduler.rs
Normal file
329
esp-preempt/src/scheduler.rs
Normal file
@ -0,0 +1,329 @@
|
||||
use core::{ffi::c_void, ptr::NonNull};
|
||||
|
||||
use allocator_api2::boxed::Box;
|
||||
use esp_hal::time::{Duration, Instant};
|
||||
use esp_radio_preempt_driver::semaphore::{SemaphoreImplementation, SemaphoreKind, SemaphorePtr};
|
||||
use esp_sync::NonReentrantMutex;
|
||||
|
||||
use crate::{
|
||||
InternalMemory,
|
||||
run_queue::RunQueue,
|
||||
semaphore::Semaphore,
|
||||
task::{self, Context, TaskAllocListElement, TaskDeleteListElement, TaskList, TaskPtr},
|
||||
timer::TimeDriver,
|
||||
timer_queue,
|
||||
};
|
||||
|
||||
#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
|
||||
// Due to how our interrupts work, we may have a timer event and a yield at the same time. Their
|
||||
// order of processing is an implementation detail in esp-hal. We need to be able to store multiple
|
||||
// events to not miss any.
|
||||
pub(crate) struct SchedulerEvent(usize);
|
||||
|
||||
impl SchedulerEvent {
|
||||
// If this is NOT set, the event is a cooperative yield of some sort (time slicing, self-yield).
|
||||
const TASK_BLOCKED: usize = 1 << 0;
|
||||
|
||||
// If this is set, the timer queue should be processed.
|
||||
const TIMER_EVENT: usize = 1 << 1;
|
||||
|
||||
fn contains(self, bit: usize) -> bool {
|
||||
self.0 & bit != 0
|
||||
}
|
||||
fn set(&mut self, bit: usize) {
|
||||
self.0 |= bit;
|
||||
}
|
||||
|
||||
pub(crate) fn set_blocked(&mut self) {
|
||||
self.set(Self::TASK_BLOCKED)
|
||||
}
|
||||
|
||||
pub(crate) fn set_timer_event(&mut self) {
|
||||
self.set(Self::TIMER_EVENT)
|
||||
}
|
||||
|
||||
pub(crate) fn is_cooperative_yield(self) -> bool {
|
||||
!self.contains(Self::TASK_BLOCKED)
|
||||
}
|
||||
|
||||
pub(crate) fn is_timer_event(self) -> bool {
|
||||
self.contains(Self::TIMER_EVENT)
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct SchedulerState {
|
||||
/// Pointer to the current task.
|
||||
pub(crate) current_task: Option<TaskPtr>,
|
||||
|
||||
/// A list of all allocated tasks
|
||||
pub(crate) all_tasks: TaskList<TaskAllocListElement>,
|
||||
|
||||
/// A list of tasks ready to run
|
||||
pub(crate) run_queue: RunQueue,
|
||||
|
||||
/// Pointer to the task that is scheduled for deletion.
|
||||
pub(crate) to_delete: TaskList<TaskDeleteListElement>,
|
||||
|
||||
pub(crate) time_driver: Option<TimeDriver>,
|
||||
|
||||
pub(crate) event: SchedulerEvent,
|
||||
}
|
||||
|
||||
unsafe impl Send for SchedulerState {}
|
||||
|
||||
impl SchedulerState {
|
||||
const fn new() -> Self {
|
||||
Self {
|
||||
current_task: None,
|
||||
all_tasks: TaskList::new(),
|
||||
run_queue: RunQueue::new(),
|
||||
to_delete: TaskList::new(),
|
||||
|
||||
time_driver: None,
|
||||
event: SchedulerEvent(0),
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn set_time_driver(&mut self, driver: TimeDriver) {
|
||||
self.time_driver = Some(driver);
|
||||
}
|
||||
|
||||
fn delete_marked_tasks(&mut self) {
|
||||
while let Some(to_delete) = self.to_delete.pop() {
|
||||
trace!("delete_marked_tasks {:?}", to_delete);
|
||||
|
||||
self.all_tasks.remove(to_delete);
|
||||
self.run_queue.remove(to_delete);
|
||||
unwrap!(self.time_driver.as_mut())
|
||||
.timer_queue
|
||||
.remove(to_delete);
|
||||
|
||||
unsafe {
|
||||
let task = Box::from_raw_in(to_delete.as_ptr(), InternalMemory);
|
||||
core::mem::drop(task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn select_next_task(&mut self, currently_active_task: TaskPtr) -> Option<TaskPtr> {
|
||||
// At least one task must be ready to run. If there are none, we can't do anything - we
|
||||
// can't just WFI from an interrupt handler. We should create an idle task that WFIs for us,
|
||||
// and can be replaced with auto light sleep.
|
||||
let next = unwrap!(self.run_queue.pop(), "There are no tasks ready to run.");
|
||||
|
||||
if next == currently_active_task {
|
||||
return None;
|
||||
}
|
||||
|
||||
Some(next)
|
||||
}
|
||||
|
||||
fn run_scheduler(&mut self, task_switch: impl FnOnce(TaskPtr, TaskPtr)) {
|
||||
self.delete_marked_tasks();
|
||||
let current_task = unwrap!(self.current_task);
|
||||
|
||||
let event = core::mem::take(&mut self.event);
|
||||
if event.is_cooperative_yield() {
|
||||
// Current task is still ready, mark it as such.
|
||||
self.run_queue.mark_same_priority_task_ready(current_task);
|
||||
}
|
||||
|
||||
if event.is_timer_event() {
|
||||
unwrap!(self.time_driver.as_mut()).handle_alarm(|ready_task| {
|
||||
debug_assert_eq!(ready_task.state, task::TaskState::Sleeping);
|
||||
|
||||
debug!("Task {:?} is ready", ready_task as *const _);
|
||||
|
||||
self.run_queue.mark_task_ready(NonNull::from(ready_task));
|
||||
});
|
||||
}
|
||||
|
||||
if let Some(next_task) = self.select_next_task(current_task) {
|
||||
debug_assert_eq!(unsafe { next_task.as_ref().state }, task::TaskState::Ready);
|
||||
|
||||
trace!("Switching task {:?} -> {:?}", current_task, next_task);
|
||||
|
||||
task_switch(current_task, next_task);
|
||||
self.current_task = Some(next_task);
|
||||
}
|
||||
|
||||
// TODO maybe we don't need to do this every time?
|
||||
self.arm_time_slice_alarm();
|
||||
}
|
||||
|
||||
#[cfg(xtensa)]
|
||||
pub(crate) fn switch_task(&mut self, trap_frame: &mut esp_hal::trapframe::TrapFrame) {
|
||||
self.run_scheduler(|mut current_task, mut next_task| {
|
||||
task::save_task_context(unsafe { current_task.as_mut() }, trap_frame);
|
||||
task::restore_task_context(unsafe { next_task.as_mut() }, trap_frame);
|
||||
});
|
||||
}
|
||||
|
||||
#[cfg(riscv)]
|
||||
pub(crate) fn switch_task(&mut self) {
|
||||
self.run_scheduler(|mut current_task, mut next_task| {
|
||||
let old_ctx = unsafe { &raw mut current_task.as_mut().trap_frame };
|
||||
let new_ctx = unsafe { &raw mut next_task.as_mut().trap_frame };
|
||||
|
||||
crate::task::arch_specific::task_switch(old_ctx, new_ctx);
|
||||
});
|
||||
}
|
||||
|
||||
fn arm_time_slice_alarm(&mut self) {
|
||||
let ready_tasks = !self.run_queue.is_empty();
|
||||
unwrap!(self.time_driver.as_mut()).arm_next_wakeup(ready_tasks);
|
||||
}
|
||||
|
||||
pub(crate) fn schedule_task_deletion(&mut self, task_to_delete: *mut Context) -> bool {
|
||||
let current_task = unwrap!(self.current_task);
|
||||
let task_to_delete = NonNull::new(task_to_delete).unwrap_or(current_task);
|
||||
let is_current = task_to_delete == current_task;
|
||||
|
||||
self.to_delete.push(task_to_delete);
|
||||
|
||||
is_current
|
||||
}
|
||||
|
||||
pub(crate) fn sleep_until(&mut self, at: Instant) {
|
||||
let current_task = unwrap!(self.current_task);
|
||||
let timer_queue = unwrap!(self.time_driver.as_mut());
|
||||
timer_queue.schedule_wakeup(current_task, at);
|
||||
|
||||
self.event.set_blocked();
|
||||
task::yield_task();
|
||||
}
|
||||
|
||||
pub(crate) fn resume_task(&mut self, task: TaskPtr) {
|
||||
let timer_queue = unwrap!(self.time_driver.as_mut());
|
||||
timer_queue.timer_queue.remove(task);
|
||||
|
||||
self.run_queue.mark_task_ready(task);
|
||||
|
||||
// if task.priority > current_task.priority
|
||||
task::yield_task();
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct Scheduler {
|
||||
inner: NonReentrantMutex<SchedulerState>,
|
||||
}
|
||||
|
||||
impl Scheduler {
|
||||
pub(crate) fn with<R>(&self, cb: impl FnOnce(&mut SchedulerState) -> R) -> R {
|
||||
self.inner.with(cb)
|
||||
}
|
||||
|
||||
pub(crate) fn yield_task(&self) {
|
||||
task::yield_task();
|
||||
}
|
||||
|
||||
pub(crate) fn current_task(&self) -> TaskPtr {
|
||||
task::current_task()
|
||||
}
|
||||
|
||||
pub(crate) fn create_task(
|
||||
&self,
|
||||
task: extern "C" fn(*mut c_void),
|
||||
param: *mut c_void,
|
||||
task_stack_size: usize,
|
||||
) -> TaskPtr {
|
||||
let task = Box::new_in(Context::new(task, param, task_stack_size), InternalMemory);
|
||||
let task_ptr = NonNull::from(Box::leak(task));
|
||||
|
||||
SCHEDULER.with(|state| {
|
||||
state.all_tasks.push(task_ptr);
|
||||
state.run_queue.mark_task_ready(task_ptr);
|
||||
});
|
||||
|
||||
debug!("Task created: {:?}", task_ptr);
|
||||
|
||||
task_ptr
|
||||
}
|
||||
|
||||
pub(crate) fn sleep_until(&self, wake_at: Instant) {
|
||||
SCHEDULER.with(|scheduler| scheduler.sleep_until(wake_at))
|
||||
}
|
||||
}
|
||||
|
||||
esp_radio_preempt_driver::scheduler_impl!(pub(crate) static SCHEDULER: Scheduler = Scheduler {
|
||||
inner: NonReentrantMutex::new(SchedulerState::new())
|
||||
});
|
||||
|
||||
impl esp_radio_preempt_driver::Scheduler for Scheduler {
|
||||
fn initialized(&self) -> bool {
|
||||
self.with(|scheduler| scheduler.time_driver.is_some())
|
||||
}
|
||||
|
||||
fn enable(&self) {
|
||||
// allocate the main task
|
||||
task::allocate_main_task();
|
||||
task::setup_multitasking();
|
||||
timer_queue::create_timer_task();
|
||||
|
||||
self.with(|scheduler| unwrap!(scheduler.time_driver.as_mut()).start());
|
||||
}
|
||||
|
||||
fn disable(&self) {
|
||||
self.with(|scheduler| unwrap!(scheduler.time_driver.as_mut()).stop());
|
||||
task::disable_multitasking();
|
||||
task::delete_all_tasks();
|
||||
}
|
||||
|
||||
fn yield_task(&self) {
|
||||
self.yield_task();
|
||||
}
|
||||
|
||||
fn yield_task_from_isr(&self) {
|
||||
self.yield_task();
|
||||
}
|
||||
|
||||
fn max_task_priority(&self) -> u32 {
|
||||
RunQueue::PRIORITY_LEVELS as u32 - 1
|
||||
}
|
||||
|
||||
fn task_create(
|
||||
&self,
|
||||
task: extern "C" fn(*mut c_void),
|
||||
param: *mut c_void,
|
||||
_priority: u32,
|
||||
_pin_to_core: Option<u32>,
|
||||
task_stack_size: usize,
|
||||
) -> *mut c_void {
|
||||
self.create_task(task, param, task_stack_size)
|
||||
.as_ptr()
|
||||
.cast()
|
||||
}
|
||||
|
||||
fn current_task(&self) -> *mut c_void {
|
||||
self.current_task().as_ptr().cast()
|
||||
}
|
||||
|
||||
fn schedule_task_deletion(&self, task_handle: *mut c_void) {
|
||||
task::schedule_task_deletion(task_handle as *mut Context)
|
||||
}
|
||||
|
||||
fn current_task_thread_semaphore(&self) -> SemaphorePtr {
|
||||
task::with_current_task(|task| {
|
||||
if task.thread_semaphore.is_none() {
|
||||
task.thread_semaphore = Some(Semaphore::create(SemaphoreKind::Counting {
|
||||
max: 1,
|
||||
initial: 0,
|
||||
}));
|
||||
}
|
||||
|
||||
unwrap!(task.thread_semaphore)
|
||||
})
|
||||
}
|
||||
|
||||
fn usleep(&self, us: u32) {
|
||||
SCHEDULER.sleep_until(Instant::now() + Duration::from_micros(us as u64))
|
||||
}
|
||||
|
||||
fn now(&self) -> u64 {
|
||||
// We're using a SingleShotTimer as the time driver, which lets us use the system timer's
|
||||
// timestamps.
|
||||
esp_hal::time::Instant::now()
|
||||
.duration_since_epoch()
|
||||
.as_micros()
|
||||
}
|
||||
}
|
@ -4,37 +4,86 @@ use core::ptr::NonNull;
|
||||
use esp_hal::time::{Duration, Instant};
|
||||
use esp_radio_preempt_driver::{
|
||||
register_semaphore_implementation,
|
||||
semaphore::{SemaphoreImplementation, SemaphorePtr},
|
||||
semaphore::{SemaphoreImplementation, SemaphoreKind, SemaphorePtr},
|
||||
yield_task,
|
||||
};
|
||||
use esp_sync::NonReentrantMutex;
|
||||
|
||||
struct SemaphoreInner {
|
||||
use crate::task::{TaskPtr, current_task};
|
||||
|
||||
enum SemaphoreInner {
|
||||
Counting {
|
||||
current: u32,
|
||||
max: u32,
|
||||
},
|
||||
RecursiveMutex {
|
||||
owner: Option<TaskPtr>,
|
||||
lock_counter: u32,
|
||||
},
|
||||
}
|
||||
|
||||
impl SemaphoreInner {
|
||||
fn try_take(&mut self) -> bool {
|
||||
if self.current > 0 {
|
||||
self.current -= 1;
|
||||
match self {
|
||||
SemaphoreInner::Counting { current, .. } => {
|
||||
if *current > 0 {
|
||||
*current -= 1;
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
SemaphoreInner::RecursiveMutex {
|
||||
owner,
|
||||
lock_counter,
|
||||
} => {
|
||||
let current = current_task();
|
||||
if owner.is_none() || owner.unwrap() == current {
|
||||
*lock_counter += 1;
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn try_give(&mut self) -> bool {
|
||||
if self.current < self.max {
|
||||
self.current += 1;
|
||||
match self {
|
||||
SemaphoreInner::Counting { current, max } => {
|
||||
if *current < *max {
|
||||
*current += 1;
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
SemaphoreInner::RecursiveMutex {
|
||||
owner,
|
||||
lock_counter,
|
||||
} => {
|
||||
let current = current_task();
|
||||
|
||||
if *owner == Some(current) && *lock_counter > 0 {
|
||||
*lock_counter -= 1;
|
||||
if *lock_counter == 0 {
|
||||
*owner = None;
|
||||
}
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn current_count(&mut self) -> u32 {
|
||||
self.current
|
||||
match self {
|
||||
SemaphoreInner::Counting { current, .. } => *current,
|
||||
SemaphoreInner::RecursiveMutex { .. } => {
|
||||
panic!("RecursiveMutex does not support current_count")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -43,12 +92,19 @@ pub struct Semaphore {
|
||||
}
|
||||
|
||||
impl Semaphore {
|
||||
pub fn new(max: u32, initial: u32) -> Self {
|
||||
Semaphore {
|
||||
inner: NonReentrantMutex::new(SemaphoreInner {
|
||||
pub fn new(kind: SemaphoreKind) -> Self {
|
||||
let inner = match kind {
|
||||
SemaphoreKind::Counting { initial, max } => SemaphoreInner::Counting {
|
||||
current: initial,
|
||||
max,
|
||||
}),
|
||||
},
|
||||
SemaphoreKind::RecursiveMutex => SemaphoreInner::RecursiveMutex {
|
||||
owner: None,
|
||||
lock_counter: 0,
|
||||
},
|
||||
};
|
||||
Semaphore {
|
||||
inner: NonReentrantMutex::new(inner),
|
||||
}
|
||||
}
|
||||
|
||||
@ -98,8 +154,8 @@ impl Semaphore {
|
||||
}
|
||||
|
||||
impl SemaphoreImplementation for Semaphore {
|
||||
fn create(max: u32, initial: u32) -> SemaphorePtr {
|
||||
let sem = Box::new(Semaphore::new(max, initial));
|
||||
fn create(kind: SemaphoreKind) -> SemaphorePtr {
|
||||
let sem = Box::new(Semaphore::new(kind));
|
||||
NonNull::from(Box::leak(sem)).cast()
|
||||
}
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
#[cfg_attr(xtensa, path = "xtensa.rs")]
|
||||
pub(crate) mod arch_specific;
|
||||
|
||||
use core::{ffi::c_void, mem::MaybeUninit};
|
||||
use core::{ffi::c_void, marker::PhantomData, mem::MaybeUninit, ptr::NonNull};
|
||||
|
||||
use allocator_api2::boxed::Box;
|
||||
#[cfg(riscv)]
|
||||
@ -12,16 +12,160 @@ pub(crate) use arch_specific::*;
|
||||
use esp_hal::trapframe::TrapFrame;
|
||||
use esp_radio_preempt_driver::semaphore::{SemaphoreHandle, SemaphorePtr};
|
||||
|
||||
use crate::{InternalMemory, SCHEDULER_STATE, task, timer};
|
||||
use crate::{InternalMemory, SCHEDULER, run_queue::RunQueue};
|
||||
|
||||
#[derive(Clone, Copy)]
|
||||
#[derive(Clone, Copy, PartialEq, Debug)]
|
||||
#[cfg_attr(feature = "defmt", derive(defmt::Format))]
|
||||
pub(crate) enum TaskState {
|
||||
Ready,
|
||||
Sleeping,
|
||||
}
|
||||
|
||||
impl TaskState {
|
||||
pub fn is_ready(self) -> bool {
|
||||
matches!(self, Self::Ready)
|
||||
pub(crate) type TaskPtr = NonNull<Context>;
|
||||
pub(crate) type TaskListItem = Option<TaskPtr>;
|
||||
|
||||
/// An abstraction that allows the task to contain multiple different queue pointers.
|
||||
pub(crate) trait TaskListElement: Default {
|
||||
fn next(task: TaskPtr) -> Option<TaskPtr>;
|
||||
fn set_next(task: TaskPtr, next: Option<TaskPtr>);
|
||||
}
|
||||
|
||||
macro_rules! task_list_item {
|
||||
($struct:ident, $field:ident) => {
|
||||
#[derive(Default)]
|
||||
pub(crate) struct $struct;
|
||||
impl TaskListElement for $struct {
|
||||
fn next(task: TaskPtr) -> Option<TaskPtr> {
|
||||
unsafe { task.as_ref().$field }
|
||||
}
|
||||
|
||||
fn set_next(mut task: TaskPtr, next: Option<TaskPtr>) {
|
||||
unsafe {
|
||||
task.as_mut().$field = next;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
task_list_item!(TaskAllocListElement, alloc_list_item);
|
||||
task_list_item!(TaskReadyQueueElement, ready_queue_item);
|
||||
task_list_item!(TaskDeleteListElement, delete_list_item);
|
||||
task_list_item!(TaskTimerQueueElement, timer_queue_item);
|
||||
|
||||
/// Extension trait for common task operations. These should be inherent methods but we can't
|
||||
/// implement stuff for NonNull.
|
||||
pub(crate) trait TaskExt {
|
||||
fn resume(self);
|
||||
}
|
||||
|
||||
impl TaskExt for TaskPtr {
|
||||
fn resume(self) {
|
||||
SCHEDULER.with(|scheduler| scheduler.resume_task(self))
|
||||
}
|
||||
}
|
||||
|
||||
/// A singly linked list of tasks.
|
||||
///
|
||||
/// Use this where you don't care about the order of list elements.
|
||||
///
|
||||
/// The `E` type parameter is used to access the data in the task object that belongs to this list.
|
||||
#[derive(Default)]
|
||||
pub(crate) struct TaskList<E> {
|
||||
head: Option<TaskPtr>,
|
||||
_item: PhantomData<E>,
|
||||
}
|
||||
|
||||
impl<E: TaskListElement> TaskList<E> {
|
||||
pub const fn new() -> Self {
|
||||
Self {
|
||||
head: None,
|
||||
_item: PhantomData,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn push(&mut self, task: TaskPtr) {
|
||||
E::set_next(task, self.head);
|
||||
self.head = Some(task);
|
||||
}
|
||||
|
||||
pub fn pop(&mut self) -> Option<TaskPtr> {
|
||||
let popped = self.head.take();
|
||||
|
||||
if let Some(task) = popped {
|
||||
self.head = E::next(task);
|
||||
}
|
||||
|
||||
popped
|
||||
}
|
||||
|
||||
pub fn remove(&mut self, task: TaskPtr) {
|
||||
// TODO: maybe this (and TaskQueue::remove) may prove too expensive.
|
||||
let mut list = core::mem::take(self);
|
||||
while let Some(popped) = list.pop() {
|
||||
if popped != task {
|
||||
self.push(popped);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A singly linked queue of tasks.
|
||||
///
|
||||
/// Use this where you care about the order of list elements. Elements are popped from the front,
|
||||
/// and pushed to the back.
|
||||
///
|
||||
/// The `E` type parameter is used to access the data in the task object that belongs to this list.
|
||||
#[derive(Default)]
|
||||
pub(crate) struct TaskQueue<E> {
|
||||
head: Option<TaskPtr>,
|
||||
tail: Option<TaskPtr>,
|
||||
_item: PhantomData<E>,
|
||||
}
|
||||
|
||||
impl<E: TaskListElement> TaskQueue<E> {
|
||||
pub const fn new() -> Self {
|
||||
Self {
|
||||
head: None,
|
||||
tail: None,
|
||||
_item: PhantomData,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn push(&mut self, task: TaskPtr) {
|
||||
E::set_next(task, None);
|
||||
if let Some(tail) = self.tail {
|
||||
E::set_next(tail, Some(task));
|
||||
} else {
|
||||
self.head = Some(task);
|
||||
}
|
||||
self.tail = Some(task);
|
||||
}
|
||||
|
||||
pub fn pop(&mut self) -> Option<TaskPtr> {
|
||||
let popped = self.head.take();
|
||||
|
||||
if let Some(task) = popped {
|
||||
self.head = E::next(task);
|
||||
if self.head.is_none() {
|
||||
self.tail = None;
|
||||
}
|
||||
}
|
||||
|
||||
popped
|
||||
}
|
||||
|
||||
pub fn remove(&mut self, task: TaskPtr) {
|
||||
let mut list = core::mem::take(self);
|
||||
while let Some(popped) = list.pop() {
|
||||
if popped != task {
|
||||
self.push(popped);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn is_empty(&self) -> bool {
|
||||
self.head.is_none()
|
||||
}
|
||||
}
|
||||
|
||||
@ -32,10 +176,23 @@ pub(crate) struct Context {
|
||||
#[cfg(xtensa)]
|
||||
pub trap_frame: TrapFrame,
|
||||
pub thread_semaphore: Option<SemaphorePtr>,
|
||||
pub next: *mut Context,
|
||||
pub next_to_delete: *mut Context,
|
||||
pub state: TaskState,
|
||||
pub _allocated_stack: Box<[MaybeUninit<u8>], InternalMemory>,
|
||||
|
||||
pub wakeup_at: u64,
|
||||
|
||||
// Lists a task can be in:
|
||||
/// The list of all allocated tasks
|
||||
pub alloc_list_item: TaskListItem,
|
||||
|
||||
/// The list of ready tasks
|
||||
pub ready_queue_item: TaskListItem,
|
||||
|
||||
/// The timer queue
|
||||
pub timer_queue_item: TaskListItem,
|
||||
|
||||
/// The list of tasks scheduled for deletion
|
||||
pub delete_list_item: TaskListItem,
|
||||
}
|
||||
|
||||
impl Context {
|
||||
@ -51,18 +208,24 @@ impl Context {
|
||||
let stack_top = unsafe { stack.as_mut_ptr().add(task_stack_size).cast() };
|
||||
|
||||
Context {
|
||||
trap_frame: task::new_task_context(task_fn, param, stack_top),
|
||||
trap_frame: new_task_context(task_fn, param, stack_top),
|
||||
thread_semaphore: None,
|
||||
next: core::ptr::null_mut(),
|
||||
next_to_delete: core::ptr::null_mut(),
|
||||
state: TaskState::Ready,
|
||||
_allocated_stack: stack,
|
||||
|
||||
wakeup_at: 0,
|
||||
|
||||
alloc_list_item: TaskListItem::None,
|
||||
ready_queue_item: TaskListItem::None,
|
||||
timer_queue_item: TaskListItem::None,
|
||||
delete_list_item: TaskListItem::None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for Context {
|
||||
fn drop(&mut self) {
|
||||
debug!("Dropping task: {:?}", self as *mut Context);
|
||||
if let Some(sem) = self.thread_semaphore {
|
||||
let sem = unsafe { SemaphoreHandle::from_ptr(sem) };
|
||||
core::mem::drop(sem)
|
||||
@ -72,94 +235,80 @@ impl Drop for Context {
|
||||
|
||||
pub(super) fn allocate_main_task() {
|
||||
// This context will be filled out by the first context switch.
|
||||
let context = Box::new_in(
|
||||
let task = Box::new_in(
|
||||
Context {
|
||||
#[cfg(riscv)]
|
||||
trap_frame: Registers::default(),
|
||||
#[cfg(xtensa)]
|
||||
trap_frame: TrapFrame::default(),
|
||||
thread_semaphore: None,
|
||||
next: core::ptr::null_mut(),
|
||||
next_to_delete: core::ptr::null_mut(),
|
||||
state: TaskState::Ready,
|
||||
_allocated_stack: Box::<[u8], _>::new_uninit_slice_in(0, InternalMemory),
|
||||
|
||||
wakeup_at: 0,
|
||||
|
||||
alloc_list_item: TaskListItem::None,
|
||||
ready_queue_item: TaskListItem::None,
|
||||
timer_queue_item: TaskListItem::None,
|
||||
delete_list_item: TaskListItem::None,
|
||||
},
|
||||
InternalMemory,
|
||||
);
|
||||
let main_task_ptr = NonNull::from(Box::leak(task));
|
||||
debug!("Main task created: {:?}", main_task_ptr);
|
||||
|
||||
let context_ptr = Box::into_raw(context);
|
||||
unsafe {
|
||||
// The first task loops back to itself.
|
||||
(*context_ptr).next = context_ptr;
|
||||
}
|
||||
|
||||
SCHEDULER_STATE.with(|state| {
|
||||
SCHEDULER.with(|state| {
|
||||
debug_assert!(
|
||||
state.current_task.is_null(),
|
||||
state.current_task.is_none(),
|
||||
"Tried to allocate main task multiple times"
|
||||
);
|
||||
state.current_task = context_ptr;
|
||||
|
||||
// The main task is already running, no need to add it to the ready queue.
|
||||
state.all_tasks.push(main_task_ptr);
|
||||
state.current_task = Some(main_task_ptr);
|
||||
})
|
||||
}
|
||||
|
||||
pub(super) fn delete_all_tasks() {
|
||||
let first_task = SCHEDULER_STATE.with(|state| {
|
||||
// Remove all tasks from the list. We will drop them outside of the critical
|
||||
// section.
|
||||
core::mem::take(&mut state.current_task)
|
||||
trace!("delete_all_tasks");
|
||||
let mut all_tasks = SCHEDULER.with(|state| {
|
||||
// Since we delete all tasks, we walk through the allocation list - we just need to clear
|
||||
// the lists.
|
||||
state.to_delete = TaskList::new();
|
||||
state.run_queue = RunQueue::new();
|
||||
|
||||
// Clear the current task.
|
||||
state.current_task = None;
|
||||
|
||||
// Take the allocation list
|
||||
core::mem::take(&mut state.all_tasks)
|
||||
});
|
||||
|
||||
if first_task.is_null() {
|
||||
return;
|
||||
while let Some(task) = all_tasks.pop() {
|
||||
unsafe {
|
||||
let task = Box::from_raw_in(task.as_ptr(), InternalMemory);
|
||||
core::mem::drop(task);
|
||||
}
|
||||
|
||||
let mut task_to_delete = first_task;
|
||||
|
||||
loop {
|
||||
let next_task = unsafe {
|
||||
// SAFETY: Tasks are in a circular linked list. We are guaranteed that the next
|
||||
// task is a valid pointer, or the first task that may already have been
|
||||
// deleted. In the second case, we will not move on to the next
|
||||
// iteration, so the loop will not try to free a task twice.
|
||||
let next_task = (*task_to_delete).next;
|
||||
core::ptr::drop_in_place(task_to_delete);
|
||||
next_task
|
||||
};
|
||||
|
||||
if core::ptr::eq(next_task, first_task) {
|
||||
break;
|
||||
}
|
||||
|
||||
task_to_delete = next_task;
|
||||
}
|
||||
}
|
||||
|
||||
pub(super) fn with_current_task<R>(mut cb: impl FnMut(&mut Context) -> R) -> R {
|
||||
SCHEDULER_STATE.with(|state| cb(unsafe { &mut *state.current_task }))
|
||||
SCHEDULER.with(|state| cb(unsafe { unwrap!(state.current_task).as_mut() }))
|
||||
}
|
||||
|
||||
pub(super) fn current_task() -> *mut Context {
|
||||
with_current_task(|task| task as *mut Context)
|
||||
pub(super) fn current_task() -> TaskPtr {
|
||||
with_current_task(|task| NonNull::from(task))
|
||||
}
|
||||
|
||||
pub(super) fn schedule_task_deletion(task: *mut Context) {
|
||||
let deleting_current = SCHEDULER_STATE.with(|state| state.schedule_task_deletion(task));
|
||||
trace!("schedule_task_deletion {:?}", task);
|
||||
let deleting_current = SCHEDULER.with(|state| state.schedule_task_deletion(task));
|
||||
|
||||
// Tasks are deleted during context switches, so we need to yield if we are
|
||||
// deleting the current task.
|
||||
if deleting_current {
|
||||
loop {
|
||||
timer::yield_task();
|
||||
SCHEDULER.yield_task();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(riscv)]
|
||||
pub(crate) fn task_switch() {
|
||||
SCHEDULER_STATE.with(|state| state.switch_task());
|
||||
}
|
||||
|
||||
#[cfg(xtensa)]
|
||||
pub(crate) fn task_switch(trap_frame: &mut TrapFrame) {
|
||||
SCHEDULER_STATE.with(|state| state.switch_task(trap_frame));
|
||||
}
|
||||
|
@ -1,5 +1,13 @@
|
||||
use core::ffi::c_void;
|
||||
|
||||
use esp_hal::{
|
||||
interrupt::{self, software::SoftwareInterrupt},
|
||||
peripherals::Interrupt,
|
||||
riscv::register,
|
||||
};
|
||||
|
||||
use crate::SCHEDULER;
|
||||
|
||||
unsafe extern "C" {
|
||||
fn sys_switch();
|
||||
}
|
||||
@ -118,52 +126,34 @@ pub(crate) fn new_task_context(
|
||||
/// We save MEPC as the current task's PC and change MEPC to an assembly function
|
||||
/// which will save the current CPU state for the current task (excluding PC) and
|
||||
/// restoring the CPU state from the next task.
|
||||
pub fn task_switch(old_ctx: *mut Registers, new_ctx: *mut Registers) -> bool {
|
||||
// Check that there isn't a switch "in-progress"
|
||||
//
|
||||
// While this shouldn't happen: from observation it does!
|
||||
//
|
||||
// This happens if the timer tick interrupt gets asserted while the task switch is in
|
||||
// progress (i.e. the sw-int is served).
|
||||
//
|
||||
// In that case returning via `mret` will first service the pending interrupt before actually
|
||||
// ending up in `sys_switch`.
|
||||
//
|
||||
// Setting MPIE to 0 _should_ prevent that from happening.
|
||||
if !_NEXT_CTX_PTR
|
||||
pub fn task_switch(old_ctx: *mut Registers, new_ctx: *mut Registers) {
|
||||
debug_assert!(
|
||||
_NEXT_CTX_PTR
|
||||
.load(portable_atomic::Ordering::SeqCst)
|
||||
.is_null()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
);
|
||||
_CURRENT_CTX_PTR.store(old_ctx, portable_atomic::Ordering::SeqCst);
|
||||
_NEXT_CTX_PTR.store(new_ctx, portable_atomic::Ordering::SeqCst);
|
||||
|
||||
let old = esp_hal::riscv::register::mepc::read();
|
||||
unsafe {
|
||||
(*old_ctx).pc = old;
|
||||
(*old_ctx).pc = register::mepc::read();
|
||||
}
|
||||
|
||||
// set MSTATUS for the switched to task
|
||||
// MIE will be set from MPIE
|
||||
// MPP will be used to determine the privilege-level
|
||||
let mstatus = esp_hal::riscv::register::mstatus::read().bits();
|
||||
let mstatus = register::mstatus::read().bits();
|
||||
unsafe {
|
||||
(*new_ctx).mstatus = mstatus;
|
||||
}
|
||||
|
||||
unsafe {
|
||||
// set MPIE in MSTATUS to 0 to disable interrupts while task switching
|
||||
esp_hal::riscv::register::mstatus::write(
|
||||
esp_hal::riscv::register::mstatus::Mstatus::from_bits(mstatus & !(1 << 7)),
|
||||
);
|
||||
register::mstatus::write(register::mstatus::Mstatus::from_bits(mstatus & !(1 << 7)));
|
||||
|
||||
// load address of sys_switch into MEPC - will run after all registers are restored
|
||||
esp_hal::riscv::register::mepc::write(sys_switch as usize);
|
||||
register::mepc::write(sys_switch as usize);
|
||||
}
|
||||
|
||||
true
|
||||
}
|
||||
|
||||
core::arch::global_asm!(
|
||||
@ -278,3 +268,38 @@ sys_switch:
|
||||
_CURRENT_CTX_PTR = sym _CURRENT_CTX_PTR,
|
||||
_NEXT_CTX_PTR = sym _NEXT_CTX_PTR,
|
||||
);
|
||||
|
||||
pub(crate) fn setup_multitasking() {
|
||||
// Register the interrupt handler without nesting to satisfy the requirements of the task
|
||||
// switching code
|
||||
let swint2_handler = esp_hal::interrupt::InterruptHandler::new_not_nested(
|
||||
unsafe { core::mem::transmute::<*const (), extern "C" fn()>(swint2_handler as *const ()) },
|
||||
esp_hal::interrupt::Priority::Priority1,
|
||||
);
|
||||
|
||||
unsafe { SoftwareInterrupt::<2>::steal() }.set_interrupt_handler(swint2_handler);
|
||||
}
|
||||
|
||||
pub(crate) fn disable_multitasking() {
|
||||
interrupt::disable(esp_hal::system::Cpu::ProCpu, Interrupt::FROM_CPU_INTR2);
|
||||
}
|
||||
|
||||
#[esp_hal::ram]
|
||||
extern "C" fn swint2_handler() {
|
||||
let swi = unsafe { SoftwareInterrupt::<2>::steal() };
|
||||
swi.reset();
|
||||
|
||||
SCHEDULER.with(|scheduler| scheduler.switch_task());
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn yield_task() {
|
||||
let swi = unsafe { SoftwareInterrupt::<2>::steal() };
|
||||
swi.raise();
|
||||
|
||||
// It takes a bit for the software interrupt to be serviced.
|
||||
esp_hal::riscv::asm::nop();
|
||||
esp_hal::riscv::asm::nop();
|
||||
esp_hal::riscv::asm::nop();
|
||||
esp_hal::riscv::asm::nop();
|
||||
}
|
||||
|
@ -1,8 +1,9 @@
|
||||
use core::ffi::c_void;
|
||||
|
||||
use esp_hal::trapframe::TrapFrame;
|
||||
pub(crate) use esp_hal::trapframe::TrapFrame;
|
||||
use esp_hal::{xtensa_lx, xtensa_lx_rt};
|
||||
|
||||
use crate::task::Context;
|
||||
use crate::{SCHEDULER, task::Context};
|
||||
|
||||
pub(crate) fn new_task_context(
|
||||
task_fn: extern "C" fn(*mut c_void),
|
||||
@ -40,3 +41,40 @@ pub(crate) fn restore_task_context(ctx: &mut Context, trap_frame: &mut TrapFrame
|
||||
pub(crate) fn save_task_context(ctx: &mut Context, trap_frame: &TrapFrame) {
|
||||
ctx.trap_frame = *trap_frame;
|
||||
}
|
||||
|
||||
// ESP32 uses Software1 (priority 3) for task switching, because it reserves
|
||||
// Software0 for the Bluetooth stack.
|
||||
const SW_INTERRUPT: u32 = if cfg!(esp32) { 1 << 29 } else { 1 << 7 };
|
||||
|
||||
pub(crate) fn setup_multitasking() {
|
||||
unsafe {
|
||||
let enabled = xtensa_lx::interrupt::disable();
|
||||
xtensa_lx::interrupt::enable_mask(
|
||||
SW_INTERRUPT
|
||||
| xtensa_lx_rt::interrupt::CpuInterruptLevel::Level2.mask()
|
||||
| xtensa_lx_rt::interrupt::CpuInterruptLevel::Level6.mask()
|
||||
| enabled,
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn disable_multitasking() {
|
||||
xtensa_lx::interrupt::disable_mask(SW_INTERRUPT);
|
||||
}
|
||||
|
||||
#[allow(non_snake_case)]
|
||||
#[esp_hal::ram]
|
||||
#[cfg_attr(not(esp32), unsafe(export_name = "Software0"))]
|
||||
#[cfg_attr(esp32, unsafe(export_name = "Software1"))]
|
||||
fn task_switch_interrupt(context: &mut TrapFrame) {
|
||||
let intr = SW_INTERRUPT;
|
||||
unsafe { core::arch::asm!("wsr.intclear {0}", in(reg) intr, options(nostack)) };
|
||||
|
||||
SCHEDULER.with(|scheduler| scheduler.switch_task(context));
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn yield_task() {
|
||||
let intr = SW_INTERRUPT;
|
||||
unsafe { core::arch::asm!("wsr.intset {0}", in(reg) intr, options(nostack)) };
|
||||
}
|
||||
|
@ -1,50 +1,186 @@
|
||||
#[cfg_attr(riscv, path = "riscv.rs")]
|
||||
#[cfg_attr(xtensa, path = "xtensa.rs")]
|
||||
mod arch_specific;
|
||||
use esp_hal::{
|
||||
interrupt::{InterruptHandler, Priority},
|
||||
time::{Duration, Instant, Rate},
|
||||
};
|
||||
|
||||
pub(crate) use arch_specific::*;
|
||||
use esp_hal::interrupt::{InterruptHandler, Priority};
|
||||
use esp_sync::NonReentrantMutex;
|
||||
use crate::{
|
||||
SCHEDULER,
|
||||
TICK_RATE,
|
||||
TimeBase,
|
||||
task::{Context, TaskPtr, TaskQueue, TaskState, TaskTimerQueueElement},
|
||||
};
|
||||
|
||||
use crate::TimeBase;
|
||||
const TIMESLICE_DURATION: Duration = Rate::from_hz(TICK_RATE).as_duration();
|
||||
|
||||
/// The timer responsible for time slicing.
|
||||
pub(crate) static TIMER: NonReentrantMutex<Option<TimeBase>> = NonReentrantMutex::new(None);
|
||||
|
||||
pub(crate) fn initialized() -> bool {
|
||||
TIMER.with(|timer| timer.is_some())
|
||||
pub(crate) struct TimerQueue {
|
||||
queue: TaskQueue<TaskTimerQueueElement>,
|
||||
next_wakeup: u64,
|
||||
}
|
||||
|
||||
pub(crate) fn setup_timebase(mut timer: TimeBase) {
|
||||
impl Default for TimerQueue {
|
||||
fn default() -> Self {
|
||||
// Can't derive Default, the default implementation must start with no wakeup timestamp
|
||||
Self::new()
|
||||
}
|
||||
}
|
||||
|
||||
impl TimerQueue {
|
||||
pub const fn new() -> Self {
|
||||
Self {
|
||||
queue: TaskQueue::new(),
|
||||
next_wakeup: u64::MAX,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn push(&mut self, task: TaskPtr, wakeup_at: u64) {
|
||||
self.queue.push(task);
|
||||
|
||||
self.next_wakeup = self.next_wakeup.min(wakeup_at);
|
||||
}
|
||||
|
||||
pub fn pop(&mut self) -> Option<TaskPtr> {
|
||||
// We can allow waking up sooner than necessary, so this function doesn't need to
|
||||
// re-calculate the next wakeup time.
|
||||
self.queue.pop()
|
||||
}
|
||||
|
||||
pub fn remove(&mut self, task: TaskPtr) {
|
||||
// We can allow waking up sooner than necessary, so this function doesn't need to
|
||||
// re-calculate the next wakeup time.
|
||||
self.queue.remove(task);
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct TimeDriver {
|
||||
timer: TimeBase,
|
||||
pub(crate) timer_queue: TimerQueue,
|
||||
}
|
||||
|
||||
impl TimeDriver {
|
||||
pub(crate) fn new(mut timer: TimeBase) -> Self {
|
||||
// The timer needs to tick at Priority 1 to prevent accidentally interrupting
|
||||
// priority 1 limited locks.
|
||||
// priority limited locks.
|
||||
let timer_priority = Priority::Priority1;
|
||||
|
||||
let cb: extern "C" fn() = unsafe { core::mem::transmute(timer_tick_handler as *const ()) };
|
||||
|
||||
// Register the interrupt handler without nesting to satisfy the requirements of the task
|
||||
// switching code
|
||||
#[cfg(riscv)]
|
||||
let handler = InterruptHandler::new_not_nested(cb, Priority::Priority1);
|
||||
|
||||
#[cfg(xtensa)]
|
||||
let handler = InterruptHandler::new(cb, Priority::Priority1);
|
||||
cfg_if::cfg_if! {
|
||||
if #[cfg(riscv)] {
|
||||
// Register the interrupt handler without nesting to satisfy the requirements of the
|
||||
// task switching code
|
||||
let handler = InterruptHandler::new_not_nested(cb, timer_priority);
|
||||
} else {
|
||||
let handler = InterruptHandler::new(cb, timer_priority);
|
||||
}
|
||||
};
|
||||
|
||||
timer.set_interrupt_handler(handler);
|
||||
TIMER.with(|t| {
|
||||
timer.listen();
|
||||
t.replace(timer);
|
||||
});
|
||||
|
||||
Self {
|
||||
timer,
|
||||
timer_queue: TimerQueue::new(),
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn start(&mut self) {
|
||||
self.timer.listen();
|
||||
}
|
||||
|
||||
pub(crate) fn stop(&mut self) {
|
||||
self.timer.unlisten();
|
||||
self.timer.stop();
|
||||
}
|
||||
|
||||
pub(crate) fn handle_alarm(&mut self, mut on_task_ready: impl FnMut(&mut Context)) {
|
||||
let mut timer_queue = core::mem::take(&mut self.timer_queue);
|
||||
|
||||
let now = Instant::now().duration_since_epoch().as_micros();
|
||||
|
||||
while let Some(mut task_ptr) = timer_queue.pop() {
|
||||
let task = unsafe { task_ptr.as_mut() };
|
||||
|
||||
let wakeup_at = task.wakeup_at;
|
||||
let ready = wakeup_at <= now;
|
||||
|
||||
if ready {
|
||||
on_task_ready(task);
|
||||
} else {
|
||||
self.timer_queue.push(task_ptr, wakeup_at);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn arm_next_wakeup(&mut self, with_time_slice: bool) {
|
||||
let wakeup_at = self.timer_queue.next_wakeup;
|
||||
|
||||
if wakeup_at != u64::MAX {
|
||||
let now = Instant::now().duration_since_epoch().as_micros();
|
||||
let sleep_duration = wakeup_at.saturating_sub(now);
|
||||
|
||||
let timeout = if with_time_slice {
|
||||
sleep_duration.min(TIMESLICE_DURATION.as_micros())
|
||||
} else {
|
||||
// assume 52-bit underlying timer. it's not a big deal to sleep for a shorter time
|
||||
sleep_duration & ((1 << 52) - 1)
|
||||
};
|
||||
|
||||
trace!("Arming timer for {:?}", timeout);
|
||||
unwrap!(self.timer.schedule(Duration::from_micros(timeout)));
|
||||
} else if with_time_slice {
|
||||
trace!("Arming timer for {:?}", TIMESLICE_DURATION);
|
||||
unwrap!(self.timer.schedule(TIMESLICE_DURATION));
|
||||
} else {
|
||||
trace!("Stopping timer");
|
||||
self.timer.stop();
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn schedule_wakeup(&mut self, mut current_task: TaskPtr, at: Instant) {
|
||||
unsafe { debug_assert_eq!(current_task.as_mut().state, TaskState::Ready) };
|
||||
unsafe { current_task.as_mut().state = TaskState::Sleeping };
|
||||
|
||||
if at == Instant::EPOCH + Duration::MAX {
|
||||
debug!("Suspending task: {:?}", current_task);
|
||||
return;
|
||||
}
|
||||
|
||||
let timestamp = at.duration_since_epoch().as_micros();
|
||||
debug!(
|
||||
"Scheduling wakeup for task {:?} at timestamp {}",
|
||||
current_task, timestamp
|
||||
);
|
||||
self.timer_queue.push(current_task, timestamp);
|
||||
|
||||
unsafe { current_task.as_mut().wakeup_at = timestamp };
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn clear_timer_interrupt() {
|
||||
TIMER.with(|timer| {
|
||||
unwrap!(timer.as_mut()).clear_interrupt();
|
||||
});
|
||||
}
|
||||
#[esp_hal::ram]
|
||||
extern "C" fn timer_tick_handler(#[cfg(xtensa)] _context: &mut esp_hal::trapframe::TrapFrame) {
|
||||
SCHEDULER.with(|scheduler| {
|
||||
unwrap!(scheduler.time_driver.as_mut())
|
||||
.timer
|
||||
.clear_interrupt();
|
||||
|
||||
pub(crate) fn disable_timebase() {
|
||||
TIMER.with(|timer| {
|
||||
let mut timer = unwrap!(timer.take());
|
||||
timer.unlisten();
|
||||
unwrap!(timer.cancel());
|
||||
scheduler.event.set_timer_event();
|
||||
|
||||
// `Scheduler::switch_task` must be called on a single interrupt priority level only.
|
||||
// To ensure this, we call yield_task to pend the software interrupt.
|
||||
//
|
||||
// RISC-V: esp-hal's interrupt handler can process multiple interrupts before handing
|
||||
// control back to the interrupted context. This can result in two task switches
|
||||
// before the first one's context save could run. To prevent this, here we only
|
||||
// trigger the software interrupt which will then run the scheduler.
|
||||
//
|
||||
// ESP32: Because on ESP32 the software interrupt is triggered at priority 3 but
|
||||
// the timer interrupt is triggered at priority 1, we need to trigger the
|
||||
// software interrupt manually.
|
||||
cfg_if::cfg_if! {
|
||||
if #[cfg(any(riscv, esp32))] {
|
||||
SCHEDULER.yield_task();
|
||||
} else {
|
||||
scheduler.switch_task(_context)
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -1,43 +0,0 @@
|
||||
use esp_hal::{
|
||||
interrupt::{self, software::SoftwareInterrupt},
|
||||
peripherals::Interrupt,
|
||||
};
|
||||
|
||||
use crate::task::task_switch;
|
||||
|
||||
pub(crate) fn setup_multitasking() {
|
||||
// Register the interrupt handler without nesting to satisfy the requirements of the task
|
||||
// switching code
|
||||
let swint2_handler = esp_hal::interrupt::InterruptHandler::new_not_nested(
|
||||
unsafe { core::mem::transmute::<*const (), extern "C" fn()>(swint2_handler as *const ()) },
|
||||
esp_hal::interrupt::Priority::Priority1,
|
||||
);
|
||||
|
||||
unsafe { SoftwareInterrupt::<2>::steal() }.set_interrupt_handler(swint2_handler);
|
||||
}
|
||||
|
||||
pub(crate) fn disable_multitasking() {
|
||||
interrupt::disable(esp_hal::system::Cpu::ProCpu, Interrupt::FROM_CPU_INTR2);
|
||||
}
|
||||
|
||||
#[esp_hal::ram]
|
||||
extern "C" fn swint2_handler() {
|
||||
// clear FROM_CPU_INTR2
|
||||
let swi = unsafe { SoftwareInterrupt::<2>::steal() };
|
||||
swi.reset();
|
||||
|
||||
task_switch();
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn yield_task() {
|
||||
// clear FROM_CPU_INTR2
|
||||
let swi = unsafe { SoftwareInterrupt::<2>::steal() };
|
||||
swi.raise();
|
||||
}
|
||||
|
||||
#[esp_hal::ram]
|
||||
pub(crate) extern "C" fn timer_tick_handler() {
|
||||
super::clear_timer_interrupt();
|
||||
crate::task::task_switch();
|
||||
}
|
@ -1,55 +0,0 @@
|
||||
use esp_hal::{trapframe::TrapFrame, xtensa_lx, xtensa_lx_rt};
|
||||
|
||||
use crate::task::task_switch;
|
||||
|
||||
// ESP32 uses Software1 (priority 3) for task switching, because it reserves
|
||||
// Software0 for the Bluetooth stack.
|
||||
const SW_INTERRUPT: u32 = if cfg!(esp32) { 1 << 29 } else { 1 << 7 };
|
||||
|
||||
pub(crate) fn setup_multitasking() {
|
||||
unsafe {
|
||||
let enabled = xtensa_lx::interrupt::disable();
|
||||
xtensa_lx::interrupt::enable_mask(
|
||||
SW_INTERRUPT
|
||||
| xtensa_lx_rt::interrupt::CpuInterruptLevel::Level2.mask()
|
||||
| xtensa_lx_rt::interrupt::CpuInterruptLevel::Level6.mask()
|
||||
| enabled,
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn disable_multitasking() {
|
||||
xtensa_lx::interrupt::disable_mask(SW_INTERRUPT);
|
||||
}
|
||||
|
||||
#[allow(non_snake_case)]
|
||||
#[esp_hal::ram]
|
||||
#[cfg_attr(not(esp32), unsafe(export_name = "Software0"))]
|
||||
#[cfg_attr(esp32, unsafe(export_name = "Software1"))]
|
||||
fn task_switch_interrupt(context: &mut TrapFrame) {
|
||||
let intr = SW_INTERRUPT;
|
||||
unsafe { core::arch::asm!("wsr.intclear {0}", in(reg) intr, options(nostack)) };
|
||||
|
||||
task_switch(context);
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn yield_task() {
|
||||
let intr = SW_INTERRUPT;
|
||||
unsafe { core::arch::asm!("wsr.intset {0}", in(reg) intr, options(nostack)) };
|
||||
}
|
||||
|
||||
#[esp_hal::ram]
|
||||
pub(crate) extern "C" fn timer_tick_handler(_context: &mut TrapFrame) {
|
||||
super::clear_timer_interrupt();
|
||||
|
||||
// `task_switch` must be called on a single interrupt priority level only.
|
||||
// Because on ESP32 the software interrupt is triggered at priority 3 but
|
||||
// the timer interrupt is triggered at priority 1, we need to trigger the
|
||||
// software interrupt manually.
|
||||
if cfg!(esp32) {
|
||||
yield_task();
|
||||
} else {
|
||||
crate::task::task_switch(_context);
|
||||
}
|
||||
}
|
@ -5,6 +5,7 @@ use core::{
|
||||
ptr::NonNull,
|
||||
};
|
||||
|
||||
use esp_hal::time::{Duration, Instant};
|
||||
use esp_radio_preempt_driver::{
|
||||
Scheduler,
|
||||
register_timer_implementation,
|
||||
@ -12,7 +13,10 @@ use esp_radio_preempt_driver::{
|
||||
};
|
||||
use esp_sync::NonReentrantMutex;
|
||||
|
||||
use crate::{SCHEDULER, timer::yield_task};
|
||||
use crate::{
|
||||
SCHEDULER,
|
||||
task::{TaskExt, TaskPtr},
|
||||
};
|
||||
|
||||
static TIMER_QUEUE: TimerQueue = TimerQueue::new();
|
||||
|
||||
@ -20,6 +24,7 @@ struct TimerQueueInner {
|
||||
// A linked list of active timers
|
||||
head: Option<NonNull<Timer>>,
|
||||
next_wakeup: u64,
|
||||
task: Option<TaskPtr>,
|
||||
}
|
||||
|
||||
unsafe impl Send for TimerQueueInner {}
|
||||
@ -29,6 +34,7 @@ impl TimerQueueInner {
|
||||
Self {
|
||||
head: None,
|
||||
next_wakeup: 0,
|
||||
task: None,
|
||||
}
|
||||
}
|
||||
|
||||
@ -46,6 +52,7 @@ impl TimerQueueInner {
|
||||
|
||||
if due < self.next_wakeup {
|
||||
self.next_wakeup = due;
|
||||
unwrap!(self.task).resume();
|
||||
}
|
||||
}
|
||||
|
||||
@ -97,6 +104,7 @@ impl TimerQueue {
|
||||
/// The timer queue needs to be re-processed when a new timer is armed, because the new timer
|
||||
/// may need to be triggered before the next scheduled wakeup.
|
||||
fn process(&self) {
|
||||
debug!("Processing timers");
|
||||
let mut timers = self.inner.with(|q| {
|
||||
q.next_wakeup = u64::MAX;
|
||||
q.head.take()
|
||||
@ -113,11 +121,19 @@ impl TimerQueue {
|
||||
timers = props.next.take();
|
||||
|
||||
if !props.is_active || props.drop {
|
||||
debug!(
|
||||
"Timer {:x} is inactive or dropped",
|
||||
current_timer as *const _ as usize
|
||||
);
|
||||
return false;
|
||||
}
|
||||
|
||||
if props.next_due > SCHEDULER.now() {
|
||||
// Not our time yet.
|
||||
debug!(
|
||||
"Timer {:x} is not due yet",
|
||||
current_timer as *const _ as usize
|
||||
);
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -156,10 +172,12 @@ impl TimerQueue {
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
fn next_wakeup(&self) -> u64 {
|
||||
self.inner.with(|q| q.next_wakeup)
|
||||
self.inner.with(|q| {
|
||||
let next_wakeup = q.next_wakeup;
|
||||
debug!("next_wakeup: {}", next_wakeup);
|
||||
SCHEDULER.sleep_until(Instant::EPOCH + Duration::from_micros(next_wakeup));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@ -221,17 +239,12 @@ impl Timer {
|
||||
props.next_due = next_due;
|
||||
props.period = timeout;
|
||||
props.periodic = periodic;
|
||||
debug!(
|
||||
"Arming timer: {:x} @ {} ({})",
|
||||
self as *const _ as usize, next_due, timeout
|
||||
);
|
||||
|
||||
q.enqueue(self);
|
||||
}
|
||||
|
||||
fn disarm(&self, q: &mut TimerQueueInner) {
|
||||
self.properties(q).is_active = false;
|
||||
debug!("Disarming timer: {:x}", self as *const _ as usize);
|
||||
|
||||
// We don't dequeue the timer - processing the queue will just skip it. If we re-arm,
|
||||
// the timer may already be in the queue.
|
||||
@ -260,7 +273,9 @@ impl TimerImplementation for Timer {
|
||||
let mut callback = CCallback { func, data };
|
||||
|
||||
let timer = Box::new(Timer::new(Box::new(move || unsafe { callback.call() })));
|
||||
NonNull::from(Box::leak(timer)).cast()
|
||||
let ptr = NonNull::from(Box::leak(timer)).cast();
|
||||
debug!("Created timer: {:x}", ptr.addr());
|
||||
ptr
|
||||
}
|
||||
|
||||
unsafe fn delete(timer: TimerPtr) {
|
||||
@ -284,11 +299,16 @@ impl TimerImplementation for Timer {
|
||||
}
|
||||
|
||||
unsafe fn arm(timer: TimerPtr, timeout: u64, periodic: bool) {
|
||||
debug!(
|
||||
"Arming {:?} for {} us, periodic = {:?}",
|
||||
timer, timeout, periodic
|
||||
);
|
||||
let timer = unsafe { Timer::from_ptr(timer) };
|
||||
TIMER_QUEUE.inner.with(|q| timer.arm(q, timeout, periodic))
|
||||
}
|
||||
|
||||
unsafe fn disarm(timer: TimerPtr) {
|
||||
debug!("Disarming {:?}", timer);
|
||||
let timer = unsafe { Timer::from_ptr(timer) };
|
||||
TIMER_QUEUE.inner.with(|q| timer.disarm(q))
|
||||
}
|
||||
@ -298,18 +318,17 @@ register_timer_implementation!(Timer);
|
||||
|
||||
/// Initializes the `timer` task for the Wi-Fi driver.
|
||||
pub(crate) fn create_timer_task() {
|
||||
// schedule the timer task
|
||||
SCHEDULER.task_create(timer_task, core::ptr::null_mut(), 1, None, 8192);
|
||||
// create the timer task
|
||||
TIMER_QUEUE.inner.with(|q| {
|
||||
let task_ptr = SCHEDULER.create_task(timer_task, core::ptr::null_mut(), 8192);
|
||||
q.task = Some(task_ptr);
|
||||
});
|
||||
}
|
||||
|
||||
/// Entry point for the timer task responsible for handling scheduled timer
|
||||
/// events.
|
||||
pub(crate) extern "C" fn timer_task(_: *mut c_void) {
|
||||
loop {
|
||||
debug!("Timer task");
|
||||
TIMER_QUEUE.process();
|
||||
while SCHEDULER.now() < TIMER_QUEUE.next_wakeup() {
|
||||
yield_task();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -13,7 +13,6 @@
|
||||
//! capabilities:
|
||||
//!
|
||||
//! - A preemptive task scheduler: [`Scheduler`]
|
||||
//! - Mutexes: [`mutex::MutexImplementation`]
|
||||
//! - Semaphores: [`semaphore::SemaphoreImplementation`]
|
||||
//! - Queues: [`queue::QueueImplementation`]
|
||||
//! - Timers (functions that are executed at a specific time): [`timer::TimerImplementation`]
|
||||
@ -22,7 +21,6 @@
|
||||
|
||||
#![no_std]
|
||||
|
||||
pub mod mutex;
|
||||
pub mod queue;
|
||||
pub mod semaphore;
|
||||
pub mod timer;
|
||||
@ -61,8 +59,8 @@ unsafe extern "Rust" {
|
||||
/// See the [module documentation][crate] for an example.
|
||||
#[macro_export]
|
||||
macro_rules! scheduler_impl {
|
||||
(static $name:ident: $t: ty = $val:expr) => {
|
||||
static $name: $t = $val;
|
||||
($vis:vis static $name:ident: $t: ty = $val:expr) => {
|
||||
$vis static $name: $t = $val;
|
||||
|
||||
#[unsafe(no_mangle)]
|
||||
#[inline]
|
||||
|
@ -1,210 +0,0 @@
|
||||
//! # Mutexes (mutual exclusion)
|
||||
//!
|
||||
//! Mutexes are a synchronization primitive used to protect shared data from concurrent access.
|
||||
//! They allow only one thread at a time to access a critical section of code or data.
|
||||
//!
|
||||
//! ## Implementation
|
||||
//!
|
||||
//! Implement the `MutexImplementation` trait for an object, and use the
|
||||
//! `register_mutex_implementation` to register that implementation for esp-radio.
|
||||
//!
|
||||
//! See the [`MutexImplementation`] documentation for more information.
|
||||
//!
|
||||
//! ## Usage
|
||||
//!
|
||||
//! Users should use [`MutexHandle`] to interact with mutexes created by the driver implementation.
|
||||
//!
|
||||
//! > Note that the only expected user of this crate is esp-radio.
|
||||
|
||||
use core::ptr::NonNull;
|
||||
|
||||
/// Pointer to an opaque mutex created by the driver implementation.
|
||||
pub type MutexPtr = NonNull<()>;
|
||||
|
||||
unsafe extern "Rust" {
|
||||
fn esp_preempt_mutex_create(recursive: bool) -> MutexPtr;
|
||||
fn esp_preempt_mutex_delete(mutex: MutexPtr);
|
||||
|
||||
fn esp_preempt_mutex_lock(mutex: MutexPtr, timeout_us: Option<u32>) -> bool;
|
||||
fn esp_preempt_mutex_unlock(mutex: MutexPtr) -> bool;
|
||||
}
|
||||
|
||||
/// A mutex (mutual exclusion) primitive.
|
||||
///
|
||||
/// The following snippet demonstrates the boilerplate necessary to implement a mutex using the
|
||||
/// `MutexImplementation` trait:
|
||||
///
|
||||
/// ```rust,no_run
|
||||
/// use esp_radio_preempt_driver::{
|
||||
/// mutex::{MutexImplementation, MutexPtr},
|
||||
/// register_mutex_implementation,
|
||||
/// };
|
||||
///
|
||||
/// struct MyMutex {
|
||||
/// // Mutex implementation details
|
||||
/// }
|
||||
///
|
||||
/// impl MutexImplementation for MyMutex {
|
||||
/// fn create(recursive: bool) -> MutexPtr {
|
||||
/// unimplemented!()
|
||||
/// }
|
||||
///
|
||||
/// unsafe fn delete(mutex: MutexPtr) {
|
||||
/// unimplemented!()
|
||||
/// }
|
||||
///
|
||||
/// unsafe fn lock(mutex: MutexPtr, timeout_us: Option<u32>) -> bool {
|
||||
/// unimplemented!()
|
||||
/// }
|
||||
///
|
||||
/// unsafe fn unlock(mutex: MutexPtr) -> bool {
|
||||
/// unimplemented!()
|
||||
/// }
|
||||
/// }
|
||||
///
|
||||
/// register_mutex_implementation!(MyMutex);
|
||||
/// ```
|
||||
pub trait MutexImplementation {
|
||||
/// Creates a new mutex instance.
|
||||
///
|
||||
/// The mutex should start in the unlocked state.
|
||||
///
|
||||
/// If `recursive` is `true`, the mutex should support recursive locking (i.e. the mutex owner
|
||||
/// can lock the mutex multiple times).
|
||||
fn create(recursive: bool) -> MutexPtr;
|
||||
|
||||
/// Deletes a mutex instance.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// `mutex` must be a pointer returned from [`Self::create`].
|
||||
unsafe fn delete(mutex: MutexPtr);
|
||||
|
||||
/// Locks the mutex.
|
||||
///
|
||||
/// If a timeout is given, this function should block until either the mutex could be locked,
|
||||
/// or the timeout has been reached. If no timeout is specified, the function should block
|
||||
/// indefinitely.
|
||||
///
|
||||
/// The timeout is specified in microseconds.
|
||||
///
|
||||
/// This function returns `true` if the mutex was locked, `false` if the timeout was reached.
|
||||
///
|
||||
/// Recursive mutexes can be re-locked by the mutex owner without blocking.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// `mutex` must be a pointer returned from [`Self::create`].
|
||||
unsafe fn lock(mutex: MutexPtr, timeout_us: Option<u32>) -> bool;
|
||||
|
||||
/// Unlocks a mutex.
|
||||
///
|
||||
/// This function returns `true` if the mutex was unlocked, `false` if the mutex wasn't locked,
|
||||
/// or the unlocking task was not the mutex owner. Unlocking a recursive mutex also returns
|
||||
/// `true` if the mutex's lock counter is successfully decremented.
|
||||
///
|
||||
/// Recursive mutexes are released only when `unlock` has been called for each preceding `lock`.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// `mutex` must be a pointer returned from [`Self::create`].
|
||||
unsafe fn unlock(mutex: MutexPtr) -> bool;
|
||||
}
|
||||
|
||||
#[macro_export]
|
||||
macro_rules! register_mutex_implementation {
|
||||
($t: ty) => {
|
||||
#[unsafe(no_mangle)]
|
||||
#[inline]
|
||||
fn esp_preempt_mutex_create(recursive: bool) -> $crate::mutex::MutexPtr {
|
||||
<$t as $crate::mutex::MutexImplementation>::create(recursive)
|
||||
}
|
||||
|
||||
#[unsafe(no_mangle)]
|
||||
#[inline]
|
||||
fn esp_preempt_mutex_delete(mutex: $crate::mutex::MutexPtr) {
|
||||
unsafe { <$t as $crate::mutex::MutexImplementation>::delete(mutex) }
|
||||
}
|
||||
|
||||
#[unsafe(no_mangle)]
|
||||
#[inline]
|
||||
fn esp_preempt_mutex_lock(mutex: $crate::mutex::MutexPtr, timeout_us: Option<u32>) -> bool {
|
||||
unsafe { <$t as $crate::mutex::MutexImplementation>::lock(mutex, timeout_us) }
|
||||
}
|
||||
|
||||
#[unsafe(no_mangle)]
|
||||
#[inline]
|
||||
fn esp_preempt_mutex_unlock(mutex: $crate::mutex::MutexPtr) -> bool {
|
||||
unsafe { <$t as $crate::mutex::MutexImplementation>::unlock(mutex) }
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/// Mutex handle.
|
||||
///
|
||||
/// This handle is used to interact with mutexes created by the driver implementation.
|
||||
#[repr(transparent)]
|
||||
pub struct MutexHandle(MutexPtr);
|
||||
impl MutexHandle {
|
||||
/// Creates a new mutex instance.
|
||||
///
|
||||
/// Recursive mutexes can be locked multiple times by the same thread.
|
||||
pub fn new(recursive: bool) -> Self {
|
||||
let ptr = unsafe { esp_preempt_mutex_create(recursive) };
|
||||
Self(ptr)
|
||||
}
|
||||
|
||||
/// Converts this object into a pointer without dropping it.
|
||||
pub fn leak(self) -> MutexPtr {
|
||||
let ptr = self.0;
|
||||
core::mem::forget(self);
|
||||
ptr
|
||||
}
|
||||
|
||||
/// Recovers the object from a leaked pointer.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// - The caller must only use pointers created using [`Self::leak`].
|
||||
/// - The caller must ensure the pointer is not shared.
|
||||
pub unsafe fn from_ptr(ptr: MutexPtr) -> Self {
|
||||
Self(ptr)
|
||||
}
|
||||
|
||||
/// Creates a reference to this object from a leaked pointer.
|
||||
///
|
||||
/// This function is used in the esp-radio code to interact with the mutex.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// - The caller must only use pointers created using [`Self::leak`].
|
||||
pub unsafe fn ref_from_ptr(ptr: &MutexPtr) -> &Self {
|
||||
unsafe { core::mem::transmute(ptr) }
|
||||
}
|
||||
|
||||
/// Locks a mutex.
|
||||
///
|
||||
/// If a timeout is given, this function blocks until either a mutex could be locked, or the
|
||||
/// timeout has been reached. If no timeout is given, this function blocks until the operation
|
||||
/// succeeds.
|
||||
///
|
||||
/// This function returns `true` if the mutex was taken, `false` if the timeout was reached.
|
||||
pub fn lock(&self, timeout_us: Option<u32>) -> bool {
|
||||
unsafe { esp_preempt_mutex_lock(self.0, timeout_us) }
|
||||
}
|
||||
|
||||
/// Unlocks a mutex.
|
||||
///
|
||||
/// If a timeout is given, this function blocks until either a mutex could be unlocked.
|
||||
///
|
||||
/// This function returns `true` if the mutex was given, `false` if the timeout was reached.
|
||||
pub fn unlock(&self) -> bool {
|
||||
unsafe { esp_preempt_mutex_unlock(self.0) }
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for MutexHandle {
|
||||
fn drop(&mut self) {
|
||||
unsafe { esp_preempt_mutex_delete(self.0) };
|
||||
}
|
||||
}
|
@ -1,9 +1,12 @@
|
||||
//! Counting Semaphores
|
||||
//! Semaphores
|
||||
//!
|
||||
//! Semaphores are synchronization primitives that allow threads to coordinate their execution.
|
||||
//! They are used to control access to a shared resource by limiting the number of threads that can
|
||||
//! access it simultaneously.
|
||||
//!
|
||||
//! esp-radio sometimes mixes up semaphores and mutexes (FreeRTOS allows this), so this crate
|
||||
//! exposes a single interface to work with both.
|
||||
//!
|
||||
//! ## Implementation
|
||||
//!
|
||||
//! Implement the `SemaphoreImplementation` trait for an object, and use the
|
||||
@ -14,7 +17,7 @@
|
||||
//! ## Usage
|
||||
//!
|
||||
//! Users should use [`SemaphoreHandle`] to interact with semaphores created by the driver
|
||||
//! implementation.
|
||||
//! implementation. Use [`SemaphoreKind`] to specify the type of semaphore or mutex to create.
|
||||
//!
|
||||
//! > Note that the only expected user of this crate is esp-radio.
|
||||
|
||||
@ -23,8 +26,20 @@ use core::ptr::NonNull;
|
||||
/// Pointer to an opaque semaphore created by the driver implementation.
|
||||
pub type SemaphorePtr = NonNull<()>;
|
||||
|
||||
/// The type of semaphore or mutex to create.
|
||||
pub enum SemaphoreKind {
|
||||
/// Counting semaphore or non-recursive mutex.
|
||||
///
|
||||
/// To obtain a non-recursive mutex, use [`SemaphoreKind::Counting`] with maximum and initial
|
||||
/// counts of 1.
|
||||
Counting { max: u32, initial: u32 },
|
||||
|
||||
/// Recursive mutex.
|
||||
RecursiveMutex,
|
||||
}
|
||||
|
||||
unsafe extern "Rust" {
|
||||
fn esp_preempt_semaphore_create(max: u32, initial: u32) -> SemaphorePtr;
|
||||
fn esp_preempt_semaphore_create(kind: SemaphoreKind) -> SemaphorePtr;
|
||||
fn esp_preempt_semaphore_delete(semaphore: SemaphorePtr);
|
||||
|
||||
fn esp_preempt_semaphore_take(semaphore: SemaphorePtr, timeout_us: Option<u32>) -> bool;
|
||||
@ -42,7 +57,7 @@ unsafe extern "Rust" {
|
||||
) -> bool;
|
||||
}
|
||||
|
||||
/// A counting semaphore primitive.
|
||||
/// A semaphore primitive.
|
||||
///
|
||||
/// The following snippet demonstrates the boilerplate necessary to implement a semaphore using the
|
||||
/// `SemaphoreImplementation` trait:
|
||||
@ -50,7 +65,7 @@ unsafe extern "Rust" {
|
||||
/// ```rust,no_run
|
||||
/// use esp_radio_preempt_driver::{
|
||||
/// register_semaphore_implementation,
|
||||
/// semaphore::{SemaphoreImplementation, SemaphorePtr},
|
||||
/// semaphore::{SemaphoreImplementation, SemaphoreKind, SemaphorePtr},
|
||||
/// };
|
||||
///
|
||||
/// struct MySemaphore {
|
||||
@ -58,7 +73,7 @@ unsafe extern "Rust" {
|
||||
/// }
|
||||
///
|
||||
/// impl SemaphoreImplementation for MySemaphore {
|
||||
/// fn create(max: u32, initial: u32) -> SemaphorePtr {
|
||||
/// fn create(kind: SemaphoreKind) -> SemaphorePtr {
|
||||
/// unimplemented!()
|
||||
/// }
|
||||
///
|
||||
@ -101,7 +116,12 @@ unsafe extern "Rust" {
|
||||
/// ```
|
||||
pub trait SemaphoreImplementation {
|
||||
/// Creates a new semaphore instance.
|
||||
fn create(max: u32, initial: u32) -> SemaphorePtr;
|
||||
///
|
||||
/// `kind` specifies the type of semaphore to create.
|
||||
///
|
||||
/// - `SemaphoreKind::Counting` should create counting, non-recursive semaphores/mutexes.
|
||||
/// - `SemaphoreKind::RecursiveMutex` should create recursive mutexes.
|
||||
fn create(kind: SemaphoreKind) -> SemaphorePtr;
|
||||
|
||||
/// Deletes a semaphore instance.
|
||||
///
|
||||
@ -116,6 +136,8 @@ pub trait SemaphoreImplementation {
|
||||
/// or the timeout has been reached. If no timeout is specified, the function should block
|
||||
/// indefinitely.
|
||||
///
|
||||
/// Recursive mutexes can be repeatedly taken by the same task.
|
||||
///
|
||||
/// The timeout is specified in microseconds.
|
||||
///
|
||||
/// This function returns `true` if the semaphore was taken, `false` if the timeout was reached.
|
||||
@ -130,6 +152,8 @@ pub trait SemaphoreImplementation {
|
||||
/// This function returns `true` if the semaphore was given, `false` if the counter is at
|
||||
/// its maximum.
|
||||
///
|
||||
/// Recursive mutexes can not be given by a task other than the one that first locked it.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// `semaphore` must be a pointer returned from [`Self::create`].
|
||||
@ -188,8 +212,10 @@ macro_rules! register_semaphore_implementation {
|
||||
($t: ty) => {
|
||||
#[unsafe(no_mangle)]
|
||||
#[inline]
|
||||
fn esp_preempt_semaphore_create(max: u32, initial: u32) -> $crate::semaphore::SemaphorePtr {
|
||||
<$t as $crate::semaphore::SemaphoreImplementation>::create(max, initial)
|
||||
fn esp_preempt_semaphore_create(
|
||||
kind: $crate::semaphore::SemaphoreKind,
|
||||
) -> $crate::semaphore::SemaphorePtr {
|
||||
<$t as $crate::semaphore::SemaphoreImplementation>::create(kind)
|
||||
}
|
||||
|
||||
#[unsafe(no_mangle)]
|
||||
@ -265,9 +291,12 @@ pub struct SemaphoreHandle(SemaphorePtr);
|
||||
impl SemaphoreHandle {
|
||||
/// Creates a new semaphore instance.
|
||||
///
|
||||
/// The semaphore will have the specified initial and maximum values.
|
||||
pub fn new(initial: u32, max: u32) -> Self {
|
||||
let ptr = unsafe { esp_preempt_semaphore_create(initial, max) };
|
||||
/// `kind` specifies the type of semaphore to create.
|
||||
///
|
||||
/// - Use `SemaphoreKind::Counting` to create counting semaphores and non-recursive mutexes.
|
||||
/// - Use `SemaphoreKind::RecursiveMutex` to create recursive mutexes.
|
||||
pub fn new(kind: SemaphoreKind) -> Self {
|
||||
let ptr = unsafe { esp_preempt_semaphore_create(kind) };
|
||||
Self(ptr)
|
||||
}
|
||||
|
||||
|
@ -112,7 +112,7 @@ esp32s3 = [
|
||||
|
||||
## Use `esp-alloc` with the `compat` feature for dynamic allocations.
|
||||
##
|
||||
## If you opt-out you need to provide implementations for the following functions:
|
||||
## If you opt-out, you need to provide implementations for the following functions:
|
||||
## - `pub extern "C" fn malloc(size: usize) -> *mut u8`
|
||||
## - `pub extern "C" fn malloc_internal(size: usize) -> *mut u8`
|
||||
## - `pub extern "C" fn free(ptr: *mut u8)`
|
||||
|
@ -1,30 +1,42 @@
|
||||
use esp_radio_preempt_driver::semaphore::SemaphoreKind;
|
||||
use esp_wifi_sys::c_types::c_void;
|
||||
|
||||
use crate::preempt::mutex::{MutexHandle, MutexPtr};
|
||||
use crate::preempt::semaphore::{SemaphoreHandle, SemaphorePtr};
|
||||
|
||||
pub(crate) fn mutex_create(recursive: bool) -> *mut c_void {
|
||||
MutexHandle::new(recursive).leak().as_ptr().cast()
|
||||
let ptr = SemaphoreHandle::new(if recursive {
|
||||
SemaphoreKind::RecursiveMutex
|
||||
} else {
|
||||
SemaphoreKind::Counting { max: 1, initial: 1 }
|
||||
})
|
||||
.leak()
|
||||
.as_ptr()
|
||||
.cast();
|
||||
|
||||
trace!("mutex_create -> {:?}", ptr);
|
||||
|
||||
ptr
|
||||
}
|
||||
|
||||
pub(crate) fn mutex_delete(mutex: *mut c_void) {
|
||||
let ptr = unwrap!(MutexPtr::new(mutex.cast()), "mutex is null");
|
||||
let ptr = unwrap!(SemaphorePtr::new(mutex.cast()), "mutex is null");
|
||||
|
||||
let handle = unsafe { MutexHandle::from_ptr(ptr) };
|
||||
let handle = unsafe { SemaphoreHandle::from_ptr(ptr) };
|
||||
core::mem::drop(handle);
|
||||
}
|
||||
|
||||
pub(crate) fn mutex_lock(mutex: *mut c_void) -> i32 {
|
||||
let ptr = unwrap!(MutexPtr::new(mutex.cast()), "mutex is null");
|
||||
let ptr = unwrap!(SemaphorePtr::new(mutex.cast()), "mutex is null");
|
||||
|
||||
let handle = unsafe { MutexHandle::ref_from_ptr(&ptr) };
|
||||
let handle = unsafe { SemaphoreHandle::ref_from_ptr(&ptr) };
|
||||
|
||||
handle.lock(None) as i32
|
||||
handle.take(None) as i32
|
||||
}
|
||||
|
||||
pub(crate) fn mutex_unlock(mutex: *mut c_void) -> i32 {
|
||||
let ptr = unwrap!(MutexPtr::new(mutex.cast()), "mutex is null");
|
||||
let ptr = unwrap!(SemaphorePtr::new(mutex.cast()), "mutex is null");
|
||||
|
||||
let handle = unsafe { MutexHandle::ref_from_ptr(&ptr) };
|
||||
let handle = unsafe { SemaphoreHandle::ref_from_ptr(&ptr) };
|
||||
|
||||
handle.unlock() as i32
|
||||
handle.give() as i32
|
||||
}
|
||||
|
@ -1,3 +1,4 @@
|
||||
use esp_radio_preempt_driver::semaphore::SemaphoreKind;
|
||||
use esp_wifi_sys::c_types::c_void;
|
||||
|
||||
use crate::{
|
||||
@ -5,11 +6,19 @@ use crate::{
|
||||
preempt::semaphore::{SemaphoreHandle, SemaphorePtr},
|
||||
};
|
||||
|
||||
pub(crate) fn sem_create(max: u32, init: u32) -> *mut c_void {
|
||||
SemaphoreHandle::new(max, init).leak().as_ptr().cast()
|
||||
pub(crate) fn sem_create(max: u32, initial: u32) -> *mut c_void {
|
||||
let ptr = SemaphoreHandle::new(SemaphoreKind::Counting { max, initial })
|
||||
.leak()
|
||||
.as_ptr()
|
||||
.cast();
|
||||
|
||||
trace!("sem_create -> {:?}", ptr);
|
||||
|
||||
ptr
|
||||
}
|
||||
|
||||
pub(crate) fn sem_delete(semphr: *mut c_void) {
|
||||
trace!("sem_delete: {:?}", semphr);
|
||||
let ptr = unwrap!(SemaphorePtr::new(semphr.cast()), "semphr is null");
|
||||
|
||||
let handle = unsafe { SemaphoreHandle::from_ptr(ptr) };
|
||||
|
@ -261,7 +261,7 @@ esp-metadata-generated = { path = "../esp-metadata-generated", features = ["buil
|
||||
default = []
|
||||
unstable = ["esp-hal/unstable"]
|
||||
|
||||
defmt = ["dep:defmt-rtt", "esp-hal/defmt", "embedded-test/defmt", "esp-radio?/defmt", "esp-hal-embassy?/defmt"]
|
||||
defmt = ["dep:defmt-rtt", "esp-hal/defmt", "embedded-test/defmt", "esp-radio?/defmt", "esp-hal-embassy?/defmt", "esp-preempt?/defmt"]
|
||||
esp-radio = ["dep:esp-radio", "dep:esp-preempt"]
|
||||
|
||||
# Device support (required!):
|
||||
|
Loading…
x
Reference in New Issue
Block a user