From 4becb1ce16dfe119bf6625a6743fba2687a1ad1c Mon Sep 17 00:00:00 2001 From: james7132 Date: Thu, 28 Aug 2025 03:28:39 -0700 Subject: [PATCH 01/11] Polymorphization --- examples/with-metadata.rs | 4 +- src/header.rs | 206 +++++++++++++++++++++++++++++-- src/raw.rs | 254 +++++++++++++++++--------------------- src/runnable.rs | 27 ++-- src/task.rs | 161 ++++-------------------- 5 files changed, 344 insertions(+), 308 deletions(-) diff --git a/examples/with-metadata.rs b/examples/with-metadata.rs index 1bd1bc7..7a71f50 100644 --- a/examples/with-metadata.rs +++ b/examples/with-metadata.rs @@ -72,9 +72,7 @@ thread_local! { static QUEUE: RefCell> = RefCell::new(BinaryHeap::new()); } -fn make_future_fn<'a, F>( - future: F, -) -> impl (FnOnce(&'a DurationMetadata) -> MeasureRuntime<'a, F>) { +fn make_future_fn<'a, F>(future: F) -> impl FnOnce(&'a DurationMetadata) -> MeasureRuntime<'a, F> { move |duration_meta| MeasureRuntime { f: future, duration: &duration_meta.inner, diff --git a/src/header.rs b/src/header.rs index ee84035..df4e8c1 100644 --- a/src/header.rs +++ b/src/header.rs @@ -1,6 +1,6 @@ use core::cell::UnsafeCell; use core::fmt; -use core::task::Waker; +use core::task::{RawWaker, RawWakerVTable, Waker}; #[cfg(not(feature = "portable-atomic"))] use core::sync::atomic::AtomicUsize; @@ -10,12 +10,16 @@ use portable_atomic::AtomicUsize; use crate::raw::TaskVTable; use crate::state::*; +use crate::utils::abort; use crate::utils::abort_on_panic; +use crate::ScheduleInfo; -/// The header of a task. -/// -/// This header is stored in memory at the beginning of the heap-allocated task. -pub(crate) struct Header { +pub(crate) enum Action { + Schedule, + Destroy, +} + +pub(crate) struct Header { /// Current state of the task. /// /// Contains flags representing the current state and the reference count. @@ -32,17 +36,12 @@ pub(crate) struct Header { /// methods necessary for bookkeeping the heap-allocated task. pub(crate) vtable: &'static TaskVTable, - /// Metadata associated with the task. - /// - /// This metadata may be provided to the user. - pub(crate) metadata: M, - /// Whether or not a panic that occurs in the task should be propagated. #[cfg(feature = "std")] pub(crate) propagate_panic: bool, } -impl Header { +impl Header { /// Notifies the awaiter blocked on this task. /// /// If the awaiter is the same as the current waker, it will not be notified. @@ -157,11 +156,192 @@ impl Header { abort_on_panic(|| w.wake()); } } + + /// Clones a waker. + pub(crate) unsafe fn clone_waker(&self, vtable: &'static RawWakerVTable) -> RawWaker { + let ptr: *const Header = self; + let ptr: *const () = ptr.cast(); + + // Increment the reference count. With any kind of reference-counted data structure, + // relaxed ordering is appropriate when incrementing the counter. + let state = self.state.fetch_add(REFERENCE, Ordering::Relaxed); + + // If the reference count overflowed, abort. + if state > isize::MAX as usize { + abort(); + } + + RawWaker::new(ptr, vtable) + } + + #[inline(never)] + pub(crate) unsafe fn drop_waker(&self) -> Option { + // Decrement the reference count. + let new = self.state.fetch_sub(REFERENCE, Ordering::AcqRel) - REFERENCE; + + // If this was the last reference to the task and the `Task` has been dropped too, + // then we need to decide how to destroy the task. + if new & !(REFERENCE - 1) == 0 && new & TASK == 0 { + if new & (COMPLETED | CLOSED) == 0 { + // If the task was not completed nor closed, close it and schedule one more time so + // that its future gets dropped by the executor. + self.state + .store(SCHEDULED | CLOSED | REFERENCE, Ordering::Release); + Some(Action::Schedule) + } else { + // Otherwise, destroy the task right away. + Some(Action::Destroy) + } + } else { + None + } + } + + /// Puts the task in detached state. + #[inline(never)] + pub(crate) fn set_detached(&self) -> Option<*const ()> { + let ptr: *const Header = self; + let ptr: *const () = ptr.cast(); + + unsafe { + // A place where the output will be stored in case it needs to be dropped. + let mut output = None; + + // Optimistically assume the `Task` is being detached just after creating the task. + // This is a common case so if the `Task` is datached, the overhead of it is only one + // compare-exchange operation. + if let Err(mut state) = self.state.compare_exchange_weak( + SCHEDULED | TASK | REFERENCE, + SCHEDULED | REFERENCE, + Ordering::AcqRel, + Ordering::Acquire, + ) { + loop { + // If the task has been completed but not yet closed, that means its output + // must be dropped. + if state & COMPLETED != 0 && state & CLOSED == 0 { + // Mark the task as closed in order to grab its output. + match self.state.compare_exchange_weak( + state, + state | CLOSED, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // Read the output. + output = Some(self.vtable.get_output(ptr)); + + // Update the state variable because we're continuing the loop. + state |= CLOSED; + } + Err(s) => state = s, + } + } else { + // If this is the last reference to the task and it's not closed, then + // close it and schedule one more time so that its future gets dropped by + // the executor. + let new = if state & (!(REFERENCE - 1) | CLOSED) == 0 { + SCHEDULED | CLOSED | REFERENCE + } else { + state & !TASK + }; + + // Unset the `TASK` flag. + match self.state.compare_exchange_weak( + state, + new, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // If this is the last reference to the task, we need to either + // schedule dropping its future or destroy it. + if state & !(REFERENCE - 1) == 0 { + if state & CLOSED == 0 { + (self.vtable.schedule)(ptr, ScheduleInfo::new(false)); + } else { + (self.vtable.destroy)(ptr); + } + } + + break; + } + Err(s) => state = s, + } + } + } + } + + output + } + } + + /// Puts the task in canceled state. + #[inline(never)] + pub(crate) fn set_canceled(&self) { + let ptr: *const Header = self; + let ptr: *const () = ptr.cast(); + + unsafe { + let mut state = self.state.load(Ordering::Acquire); + + loop { + // If the task has been completed or closed, it can't be canceled. + if state & (COMPLETED | CLOSED) != 0 { + break; + } + + // If the task is not scheduled nor running, we'll need to schedule it. + let new = if state & (SCHEDULED | RUNNING) == 0 { + (state | SCHEDULED | CLOSED) + REFERENCE + } else { + state | CLOSED + }; + + // Mark the task as closed. + match self.state.compare_exchange_weak( + state, + new, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // If the task is not scheduled nor running, schedule it one more time so + // that its future gets dropped by the executor. + if state & (SCHEDULED | RUNNING) == 0 { + (self.vtable.schedule)(ptr, ScheduleInfo::new(false)); + } + + // Notify the awaiter that the task has been closed. + if state & AWAITER != 0 { + self.notify(None); + } + + break; + } + Err(s) => state = s, + } + } + } + } +} + +/// The header of a task. +/// +/// This header is stored in memory at the beginning of the heap-allocated task. +#[repr(C)] +pub(crate) struct HeaderWithMetadata { + pub(crate) header: Header, + + /// Metadata associated with the task. + /// + /// This metadata may be provided to the user. + pub(crate) metadata: M, } -impl fmt::Debug for Header { +impl fmt::Debug for HeaderWithMetadata { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let state = self.state.load(Ordering::SeqCst); + let state = self.header.state.load(Ordering::SeqCst); f.debug_struct("Header") .field("scheduled", &(state & SCHEDULED != 0)) diff --git a/src/raw.rs b/src/raw.rs index 7a45dad..1a6c679 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -12,7 +12,7 @@ use core::sync::atomic::Ordering; #[cfg(feature = "portable-atomic")] use portable_atomic::AtomicUsize; -use crate::header::Header; +use crate::header::{Action, Header, HeaderWithMetadata}; use crate::runnable::{Schedule, ScheduleInfo}; use crate::state::*; use crate::utils::{abort, abort_on_panic, max, Layout}; @@ -26,17 +26,13 @@ pub(crate) type Panic = core::convert::Infallible; /// The vtable for a task. pub(crate) struct TaskVTable { + pub(crate) raw_waker_vtable: &'static RawWakerVTable, + /// Schedules the task. pub(crate) schedule: unsafe fn(*const (), ScheduleInfo), /// Drops the future inside the task. - pub(crate) drop_future: unsafe fn(*const ()), - - /// Returns a pointer to the output stored after completion. - pub(crate) get_output: unsafe fn(*const ()) -> *const (), - - /// Drops the task reference (`Runnable` or `Waker`). - pub(crate) drop_ref: unsafe fn(ptr: *const ()), + pub(crate) drop_future: unsafe fn(*const (), &TaskLayout), /// Destroys the task. pub(crate) destroy: unsafe fn(*const ()), @@ -44,9 +40,6 @@ pub(crate) struct TaskVTable { /// Runs the task. pub(crate) run: unsafe fn(*const ()) -> bool, - /// Creates a new waker associated with the task. - pub(crate) clone_waker: unsafe fn(ptr: *const ()) -> RawWaker, - /// The memory layout of the task. This information enables /// debuggers to decode raw task memory blobs. Do not remove /// the field, even if it appears to be unused. @@ -54,6 +47,13 @@ pub(crate) struct TaskVTable { pub(crate) layout_info: &'static TaskLayout, } +impl TaskVTable { + /// Returns a pointer to the output inside a task. + pub(crate) unsafe fn get_output(&self, ptr: *const ()) -> *const () { + ptr.byte_add(self.layout_info.offset_r) + } +} + /// Memory layout of a task. /// /// This struct contains the following information: @@ -78,7 +78,7 @@ pub(crate) struct TaskLayout { /// Raw pointers to the fields inside a task. pub(crate) struct RawTask { /// The task header. - pub(crate) header: *const Header, + pub(crate) header: *const HeaderWithMetadata, /// The schedule function. pub(crate) schedule: *const S, @@ -105,7 +105,7 @@ impl RawTask { #[inline] const fn eval_task_layout() -> TaskLayout { // Compute the layouts for `Header`, `S`, `F`, and `T`. - let layout_header = Layout::new::>(); + let layout_header = Layout::new::>(); let layout_s = Layout::new::(); let layout_f = Layout::new::(); let layout_r = Layout::new::>(); @@ -158,11 +158,9 @@ where // Compute the layout of the task for allocation. Abort if the computation fails. // // n.b. notgull: task_layout now automatically aborts instead of panicking - let task_layout = Self::task_layout(); - unsafe { // Allocate enough space for the entire task. - let ptr = match NonNull::new(alloc::alloc::alloc(task_layout.layout) as *mut ()) { + let ptr = match NonNull::new(alloc::alloc::alloc(Self::TASK_LAYOUT.layout) as *mut ()) { None => abort(), Some(p) => p, }; @@ -176,22 +174,22 @@ where } = builder; // Write the header as the first field of the task. - (raw.header as *mut Header).write(Header { - state: AtomicUsize::new(SCHEDULED | TASK | REFERENCE), - awaiter: UnsafeCell::new(None), - vtable: &TaskVTable { - schedule: Self::schedule, - drop_future: Self::drop_future, - get_output: Self::get_output, - drop_ref: Self::drop_ref, - destroy: Self::destroy, - run: Self::run, - clone_waker: Self::clone_waker, - layout_info: &Self::TASK_LAYOUT, + (raw.header as *mut HeaderWithMetadata).write(HeaderWithMetadata { + header: Header { + state: AtomicUsize::new(SCHEDULED | TASK | REFERENCE), + awaiter: UnsafeCell::new(None), + vtable: &TaskVTable { + raw_waker_vtable: &Self::RAW_WAKER_VTABLE, + schedule: Self::schedule, + drop_future: drop_future::, + destroy: Self::destroy, + run: Self::run, + layout_info: &Self::TASK_LAYOUT, + }, + #[cfg(feature = "std")] + propagate_panic, }, metadata, - #[cfg(feature = "std")] - propagate_panic, }); // Write the schedule function as the third field of the task. @@ -210,24 +208,21 @@ where /// Creates a `RawTask` from a raw task pointer. #[inline] pub(crate) fn from_ptr(ptr: *const ()) -> Self { - let task_layout = Self::task_layout(); - let p = ptr as *const u8; - unsafe { Self { - header: p as *const Header, - schedule: p.add(task_layout.offset_s) as *const S, - future: p.add(task_layout.offset_f) as *mut F, - output: p.add(task_layout.offset_r) as *mut Result, + header: ptr as *const HeaderWithMetadata, + schedule: ptr.byte_add(Self::TASK_LAYOUT.offset_s) as *const S, + future: ptr.byte_add(Self::TASK_LAYOUT.offset_f) as *mut F, + output: ptr.byte_add(Self::TASK_LAYOUT.offset_r) as *mut Result, } } } - /// Returns the layout of the task. - #[inline] - fn task_layout() -> TaskLayout { - Self::TASK_LAYOUT + fn header<'a>(ptr: *const ()) -> &'a Header { + let header = ptr as *const Header; + unsafe { &*header } } + /// Wakes a waker. unsafe fn wake(ptr: *const ()) { // This is just an optimization. If the schedule function has captured variables, then @@ -238,9 +233,9 @@ where return; } - let raw = Self::from_ptr(ptr); + let header = Self::header(ptr); - let mut state = (*raw.header).state.load(Ordering::Acquire); + let mut state = header.state.load(Ordering::Acquire); loop { // If the task is completed or closed, it can't be woken up. @@ -254,7 +249,7 @@ where // will run the task by "publishing" our current view of the memory. if state & SCHEDULED != 0 { // Update the state without actually modifying it. - match (*raw.header).state.compare_exchange_weak( + match header.state.compare_exchange_weak( state, state, Ordering::AcqRel, @@ -269,7 +264,7 @@ where } } else { // Mark the task as scheduled. - match (*raw.header).state.compare_exchange_weak( + match header.state.compare_exchange_weak( state, state | SCHEDULED, Ordering::AcqRel, @@ -297,8 +292,9 @@ where /// Wakes a waker by reference. unsafe fn wake_by_ref(ptr: *const ()) { let raw = Self::from_ptr(ptr); + let header = Self::header(ptr); - let mut state = (*raw.header).state.load(Ordering::Acquire); + let mut state = header.state.load(Ordering::Acquire); loop { // If the task is completed or closed, it can't be woken up. @@ -310,7 +306,7 @@ where // will run the task by "publishing" our current view of the memory. if state & SCHEDULED != 0 { // Update the state without actually modifying it. - match (*raw.header).state.compare_exchange_weak( + match header.state.compare_exchange_weak( state, state, Ordering::AcqRel, @@ -328,7 +324,7 @@ where }; // Mark the task as scheduled. - match (*raw.header).state.compare_exchange_weak( + match header.state.compare_exchange_weak( state, new, Ordering::AcqRel, @@ -359,18 +355,7 @@ where /// Clones a waker. unsafe fn clone_waker(ptr: *const ()) -> RawWaker { - let raw = Self::from_ptr(ptr); - - // Increment the reference count. With any kind of reference-counted data structure, - // relaxed ordering is appropriate when incrementing the counter. - let state = (*raw.header).state.fetch_add(REFERENCE, Ordering::Relaxed); - - // If the reference count overflowed, abort. - if state > isize::MAX as usize { - abort(); - } - - RawWaker::new(ptr, &Self::RAW_WAKER_VTABLE) + Self::header(ptr).clone_waker(&Self::RAW_WAKER_VTABLE) } /// Drops a waker. @@ -380,43 +365,10 @@ where /// scheduled one more time so that its future gets dropped by the executor. #[inline] unsafe fn drop_waker(ptr: *const ()) { - let raw = Self::from_ptr(ptr); - - // Decrement the reference count. - let new = (*raw.header).state.fetch_sub(REFERENCE, Ordering::AcqRel) - REFERENCE; - - // If this was the last reference to the task and the `Task` has been dropped too, - // then we need to decide how to destroy the task. - if new & !(REFERENCE - 1) == 0 && new & TASK == 0 { - if new & (COMPLETED | CLOSED) == 0 { - // If the task was not completed nor closed, close it and schedule one more time so - // that its future gets dropped by the executor. - (*raw.header) - .state - .store(SCHEDULED | CLOSED | REFERENCE, Ordering::Release); - Self::schedule(ptr, ScheduleInfo::new(false)); - } else { - // Otherwise, destroy the task right away. - Self::destroy(ptr); - } - } - } - - /// Drops a task reference (`Runnable` or `Waker`). - /// - /// This function will decrement the reference count. If it drops down to zero and the - /// associated `Task` handle has been dropped too, then the task gets destroyed. - #[inline] - unsafe fn drop_ref(ptr: *const ()) { - let raw = Self::from_ptr(ptr); - - // Decrement the reference count. - let new = (*raw.header).state.fetch_sub(REFERENCE, Ordering::AcqRel) - REFERENCE; - - // If this was the last reference to the task and the `Task` has been dropped too, - // then destroy the task. - if new & !(REFERENCE - 1) == 0 && new & TASK == 0 { - Self::destroy(ptr); + match Self::header(ptr).drop_waker() { + Some(Action::Schedule) => Self::schedule(ptr, ScheduleInfo::new(false)), + Some(Action::Destroy) => Self::destroy(ptr), + None => {} } } @@ -426,35 +378,19 @@ where /// its schedule function. unsafe fn schedule(ptr: *const (), info: ScheduleInfo) { let raw = Self::from_ptr(ptr); + let header = Self::header(ptr); // If the schedule function has captured variables, create a temporary waker that prevents // the task from getting deallocated while the function is being invoked. let _waker; if mem::size_of::() > 0 { - _waker = Waker::from_raw(Self::clone_waker(ptr)); + _waker = Waker::from_raw(header.clone_waker(&Self::RAW_WAKER_VTABLE)); } let task = Runnable::from_raw(NonNull::new_unchecked(ptr as *mut ())); (*raw.schedule).schedule(task, info); } - /// Drops the future inside a task. - #[inline] - unsafe fn drop_future(ptr: *const ()) { - let raw = Self::from_ptr(ptr); - - // We need a safeguard against panics because the destructor can panic. - abort_on_panic(|| { - raw.future.drop_in_place(); - }) - } - - /// Returns a pointer to the output inside a task. - unsafe fn get_output(ptr: *const ()) -> *const () { - let raw = Self::from_ptr(ptr); - raw.output as *const () - } - /// Cleans up task's resources and deallocates it. /// /// The schedule function will be dropped, and the task will then get deallocated. @@ -462,19 +398,18 @@ where #[inline] unsafe fn destroy(ptr: *const ()) { let raw = Self::from_ptr(ptr); - let task_layout = Self::task_layout(); // We need a safeguard against panics because destructors can panic. abort_on_panic(|| { // Drop the header along with the metadata. - (raw.header as *mut Header).drop_in_place(); + (raw.header as *mut HeaderWithMetadata).drop_in_place(); // Drop the schedule function. (raw.schedule as *mut S).drop_in_place(); }); // Finally, deallocate the memory reserved by the task. - alloc::alloc::dealloc(ptr as *mut u8, task_layout.layout); + alloc::alloc::dealloc(ptr as *mut u8, Self::TASK_LAYOUT.layout); } /// Runs a task. @@ -483,31 +418,32 @@ where /// the caller. unsafe fn run(ptr: *const ()) -> bool { let raw = Self::from_ptr(ptr); + let header = Self::header(ptr); // Create a context from the raw task pointer and the vtable inside the its header. let waker = ManuallyDrop::new(Waker::from_raw(RawWaker::new(ptr, &Self::RAW_WAKER_VTABLE))); let cx = &mut Context::from_waker(&waker); - let mut state = (*raw.header).state.load(Ordering::Acquire); + let mut state = header.state.load(Ordering::Acquire); // Update the task's state before polling its future. loop { // If the task has already been closed, drop the task reference and return. if state & CLOSED != 0 { // Drop the future. - Self::drop_future(ptr); + drop_future::(ptr, &Self::TASK_LAYOUT); // Mark the task as unscheduled. - let state = (*raw.header).state.fetch_and(!SCHEDULED, Ordering::AcqRel); + let state = header.state.fetch_and(!SCHEDULED, Ordering::AcqRel); // Take the awaiter out. let mut awaiter = None; if state & AWAITER != 0 { - awaiter = (*raw.header).take(None); + awaiter = header.take(None); } // Drop the task reference. - Self::drop_ref(ptr); + drop_ref(ptr); // Notify the awaiter that the future has been dropped. if let Some(w) = awaiter { @@ -517,7 +453,7 @@ where } // Mark the task as unscheduled and running. - match (*raw.header).state.compare_exchange_weak( + match header.state.compare_exchange_weak( state, (state & !SCHEDULED) | RUNNING, Ordering::AcqRel, @@ -544,7 +480,7 @@ where #[cfg(feature = "std")] let poll = { // Check if we should propagate panics. - if (*raw.header).propagate_panic { + if header.propagate_panic { // Use catch_unwind to catch the panic. match std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| { ::poll(Pin::new_unchecked(&mut *raw.future), cx) @@ -563,7 +499,7 @@ where match poll { Poll::Ready(out) => { // Replace the future with its output. - Self::drop_future(ptr); + drop_future::(ptr, &Self::TASK_LAYOUT); raw.output.write(out); // The task is now completed. @@ -576,7 +512,7 @@ where }; // Mark the task as not running and completed. - match (*raw.header).state.compare_exchange_weak( + match header.state.compare_exchange_weak( state, new, Ordering::AcqRel, @@ -593,11 +529,11 @@ where // Take the awaiter out. let mut awaiter = None; if state & AWAITER != 0 { - awaiter = (*raw.header).take(None); + awaiter = header.take(None); } // Drop the task reference. - Self::drop_ref(ptr); + drop_ref(ptr); // Notify the awaiter that the future has been dropped. if let Some(w) = awaiter { @@ -625,12 +561,12 @@ where if state & CLOSED != 0 && !future_dropped { // The thread that closed the task didn't drop the future because it was // running so now it's our responsibility to do so. - Self::drop_future(ptr); + drop_future::(ptr, &Self::TASK_LAYOUT); future_dropped = true; } // Mark the task as not running. - match (*raw.header).state.compare_exchange_weak( + match header.state.compare_exchange_weak( state, new, Ordering::AcqRel, @@ -644,11 +580,11 @@ where // Take the awaiter out. let mut awaiter = None; if state & AWAITER != 0 { - awaiter = (*raw.header).take(None); + awaiter = header.take(None); } // Drop the task reference. - Self::drop_ref(ptr); + drop_ref(ptr); // Notify the awaiter that the future has been dropped. if let Some(w) = awaiter { @@ -661,7 +597,7 @@ where return true; } else { // Drop the task reference. - Self::drop_ref(ptr); + drop_ref(ptr); } break; } @@ -687,9 +623,11 @@ where fn drop(&mut self) { let raw = self.0; let ptr = raw.header as *const (); + let header = raw.header as *const Header; unsafe { - let mut state = (*raw.header).state.load(Ordering::Acquire); + let header = &*header; + let mut state = header.state.load(Ordering::Acquire); loop { // If the task was closed while running, then unschedule it, drop its @@ -697,21 +635,21 @@ where if state & CLOSED != 0 { // The thread that closed the task didn't drop the future because it // was running so now it's our responsibility to do so. - RawTask::::drop_future(ptr); + drop_future::(ptr, &RawTask::::TASK_LAYOUT); // Mark the task as not running and not scheduled. - (*raw.header) + header .state .fetch_and(!RUNNING & !SCHEDULED, Ordering::AcqRel); // Take the awaiter out. let mut awaiter = None; if state & AWAITER != 0 { - awaiter = (*raw.header).take(None); + awaiter = header.take(None); } // Drop the task reference. - RawTask::::drop_ref(ptr); + drop_ref(ptr); // Notify the awaiter that the future has been dropped. if let Some(w) = awaiter { @@ -721,7 +659,7 @@ where } // Mark the task as not running, not scheduled, and closed. - match (*raw.header).state.compare_exchange_weak( + match header.state.compare_exchange_weak( state, (state & !RUNNING & !SCHEDULED) | CLOSED, Ordering::AcqRel, @@ -729,16 +667,16 @@ where ) { Ok(state) => { // Drop the future because the task is now closed. - RawTask::::drop_future(ptr); + drop_future::(ptr, &RawTask::::TASK_LAYOUT); // Take the awaiter out. let mut awaiter = None; if state & AWAITER != 0 { - awaiter = (*raw.header).take(None); + awaiter = header.take(None); } // Drop the task reference. - RawTask::::drop_ref(ptr); + drop_ref(ptr); // Notify the awaiter that the future has been dropped. if let Some(w) = awaiter { @@ -754,3 +692,33 @@ where } } } + +/// Drops the future inside a task. +#[inline] +unsafe fn drop_future(ptr: *const (), task_layout: &TaskLayout) { + let future_ptr = ptr.byte_add(task_layout.offset_f) as *mut F; + + // We need a safeguard against panics because the destructor can panic. + abort_on_panic(|| { + future_ptr.drop_in_place(); + }) +} + +/// Drops a task reference (`Runnable` or `Waker`). +/// +/// This function will decrement the reference count. If it drops down to zero and the +/// associated `Task` handle has been dropped too, then the task gets destroyed. +#[inline] +pub(crate) unsafe fn drop_ref(ptr: *const ()) { + let header = ptr as *const Header; + let header = &*header; + + // Decrement the reference count. + let new = header.state.fetch_sub(REFERENCE, Ordering::AcqRel) - REFERENCE; + + // If this was the last reference to the task and the `Task` has been dropped too, + // then destroy the task. + if new & !(REFERENCE - 1) == 0 && new & TASK == 0 { + (header.vtable.destroy)(ptr); + } +} diff --git a/src/runnable.rs b/src/runnable.rs index 17945b2..cef1635 100644 --- a/src/runnable.rs +++ b/src/runnable.rs @@ -9,6 +9,8 @@ use core::task::Waker; use alloc::boxed::Box; use crate::header::Header; +use crate::header::HeaderWithMetadata; +use crate::raw::drop_ref; use crate::raw::RawTask; use crate::state::*; use crate::Task; @@ -713,7 +715,7 @@ impl Runnable { /// Tasks can be created with a metadata object associated with them; by default, this /// is a `()` value. See the [`Builder::metadata()`] method for more information. pub fn metadata(&self) -> &M { - &self.header().metadata + &self.header_with_metadata().metadata } /// Schedules the task. @@ -737,7 +739,7 @@ impl Runnable { /// ``` pub fn schedule(self) { let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; + let header = ptr as *const Header; mem::forget(self); unsafe { @@ -775,7 +777,7 @@ impl Runnable { /// ``` pub fn run(self) -> bool { let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; + let header = ptr as *const Header; mem::forget(self); unsafe { ((*header).vtable.run)(ptr) } @@ -805,17 +807,20 @@ impl Runnable { /// assert_eq!(r.len(), 1); /// ``` pub fn waker(&self) -> Waker { - let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; + let header = self.header(); unsafe { - let raw_waker = ((*header).vtable.clone_waker)(ptr); + let raw_waker = header.clone_waker(header.vtable.raw_waker_vtable); Waker::from_raw(raw_waker) } } - fn header(&self) -> &Header { - unsafe { &*(self.ptr.as_ptr() as *const Header) } + fn header(&self) -> &Header { + unsafe { &*(self.ptr.as_ptr() as *const Header) } + } + + fn header_with_metadata(&self) -> &HeaderWithMetadata { + unsafe { &*(self.ptr.as_ptr() as *const HeaderWithMetadata) } } /// Converts this task into a raw pointer. @@ -917,7 +922,7 @@ impl Drop for Runnable { } // Drop the future. - (header.vtable.drop_future)(ptr); + (header.vtable.drop_future)(ptr, &*header.vtable.layout_info); // Mark the task as unscheduled. let state = header.state.fetch_and(!SCHEDULED, Ordering::AcqRel); @@ -928,7 +933,7 @@ impl Drop for Runnable { } // Drop the task reference. - (header.vtable.drop_ref)(ptr); + drop_ref(ptr); } } } @@ -936,7 +941,7 @@ impl Drop for Runnable { impl fmt::Debug for Runnable { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; + let header = ptr as *const HeaderWithMetadata; f.debug_struct("Runnable") .field("header", unsafe { &(*header) }) diff --git a/src/task.rs b/src/task.rs index a3dfd17..ddc8c8e 100644 --- a/src/task.rs +++ b/src/task.rs @@ -7,9 +7,8 @@ use core::ptr::NonNull; use core::sync::atomic::Ordering; use core::task::{Context, Poll}; -use crate::header::Header; +use crate::header::{Header, HeaderWithMetadata}; use crate::raw::Panic; -use crate::runnable::ScheduleInfo; use crate::state::*; /// A spawned task. @@ -125,8 +124,8 @@ impl Task { /// }); /// ``` pub async fn cancel(self) -> Option { - let mut this = self; - this.set_canceled(); + let this = self; + this.header().set_canceled(); this.fallible().await } @@ -179,133 +178,11 @@ impl Task { FallibleTask { task: self } } - /// Puts the task in canceled state. - fn set_canceled(&mut self) { - let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; - - unsafe { - let mut state = (*header).state.load(Ordering::Acquire); - - loop { - // If the task has been completed or closed, it can't be canceled. - if state & (COMPLETED | CLOSED) != 0 { - break; - } - - // If the task is not scheduled nor running, we'll need to schedule it. - let new = if state & (SCHEDULED | RUNNING) == 0 { - (state | SCHEDULED | CLOSED) + REFERENCE - } else { - state | CLOSED - }; - - // Mark the task as closed. - match (*header).state.compare_exchange_weak( - state, - new, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // If the task is not scheduled nor running, schedule it one more time so - // that its future gets dropped by the executor. - if state & (SCHEDULED | RUNNING) == 0 { - ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)); - } - - // Notify the awaiter that the task has been closed. - if state & AWAITER != 0 { - (*header).notify(None); - } - - break; - } - Err(s) => state = s, - } - } - } - } - /// Puts the task in detached state. fn set_detached(&mut self) -> Option> { - let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; - - unsafe { - // A place where the output will be stored in case it needs to be dropped. - let mut output = None; - - // Optimistically assume the `Task` is being detached just after creating the task. - // This is a common case so if the `Task` is datached, the overhead of it is only one - // compare-exchange operation. - if let Err(mut state) = (*header).state.compare_exchange_weak( - SCHEDULED | TASK | REFERENCE, - SCHEDULED | REFERENCE, - Ordering::AcqRel, - Ordering::Acquire, - ) { - loop { - // If the task has been completed but not yet closed, that means its output - // must be dropped. - if state & COMPLETED != 0 && state & CLOSED == 0 { - // Mark the task as closed in order to grab its output. - match (*header).state.compare_exchange_weak( - state, - state | CLOSED, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // Read the output. - output = Some( - (((*header).vtable.get_output)(ptr) as *mut Result) - .read(), - ); - - // Update the state variable because we're continuing the loop. - state |= CLOSED; - } - Err(s) => state = s, - } - } else { - // If this is the last reference to the task and it's not closed, then - // close it and schedule one more time so that its future gets dropped by - // the executor. - let new = if state & (!(REFERENCE - 1) | CLOSED) == 0 { - SCHEDULED | CLOSED | REFERENCE - } else { - state & !TASK - }; - - // Unset the `TASK` flag. - match (*header).state.compare_exchange_weak( - state, - new, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // If this is the last reference to the task, we need to either - // schedule dropping its future or destroy it. - if state & !(REFERENCE - 1) == 0 { - if state & CLOSED == 0 { - ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)); - } else { - ((*header).vtable.destroy)(ptr); - } - } - - break; - } - Err(s) => state = s, - } - } - } - } - - output - } + self.header() + .set_detached() + .map(|ptr| unsafe { (ptr as *mut Result).read() }) } /// Polls the task to retrieve its output. @@ -320,7 +197,7 @@ impl Task { /// 4. It is completed and the `Task` gets dropped. fn poll_task(&mut self, cx: &mut Context<'_>) -> Poll> { let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; + let header = ptr as *const Header; unsafe { let mut state = (*header).state.load(Ordering::Acquire); @@ -386,7 +263,7 @@ impl Task { } // Take the output from the task. - let output = ((*header).vtable.get_output)(ptr) as *mut Result; + let output = (*header).vtable.get_output(ptr) as *mut Result; let output = output.read(); // Propagate the panic if the task panicked. @@ -410,9 +287,15 @@ impl Task { } } - fn header(&self) -> &Header { + fn header(&self) -> &Header { let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; + let header = ptr as *const Header; + unsafe { &*header } + } + + fn header_with_metadata(&self) -> &HeaderWithMetadata { + let ptr = self.ptr.as_ptr(); + let header = ptr as *const HeaderWithMetadata; unsafe { &*header } } @@ -421,7 +304,7 @@ impl Task { /// Note that in a multithreaded environment, this task can change finish immediately after calling this function. pub fn is_finished(&self) -> bool { let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; + let header = ptr as *const Header; unsafe { let state = (*header).state.load(Ordering::Acquire); @@ -434,13 +317,15 @@ impl Task { /// Tasks can be created with a metadata object associated with them; by default, this /// is a `()` value. See the [`Builder::metadata()`] method for more information. pub fn metadata(&self) -> &M { - &self.header().metadata + let ptr = self.ptr.as_ptr(); + let header = ptr as *const HeaderWithMetadata; + &unsafe { &*header }.metadata } } impl Drop for Task { fn drop(&mut self) { - self.set_canceled(); + self.header().set_canceled(); self.set_detached(); } } @@ -459,7 +344,7 @@ impl Future for Task { impl fmt::Debug for Task { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Task") - .field("header", self.header()) + .field("header", self.header_with_metadata()) .finish() } } @@ -560,7 +445,7 @@ impl Future for FallibleTask { impl fmt::Debug for FallibleTask { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("FallibleTask") - .field("header", self.task.header()) + .field("header", self.task.header_with_metadata()) .finish() } } From e0806bf933f478044a802bf04a60eb8072e930a0 Mon Sep 17 00:00:00 2001 From: james7132 Date: Thu, 28 Aug 2025 13:02:54 -0700 Subject: [PATCH 02/11] Fix some Miri-reported UB --- Cargo.toml | 2 +- src/header.rs | 148 +++--------------------------- src/raw.rs | 235 +++++++++++++++++++++++++----------------------- src/runnable.rs | 6 +- src/task.rs | 144 +++++++++++++++++++++++++++-- 5 files changed, 277 insertions(+), 258 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 7779a6a..362baae 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -6,7 +6,7 @@ name = "async-task" version = "4.7.1" authors = ["Stjepan Glavina "] edition = "2021" -rust-version = "1.57" +rust-version = "1.75" license = "Apache-2.0 OR MIT" repository = "https://github.com/smol-rs/async-task" description = "Task abstraction for building executors" diff --git a/src/header.rs b/src/header.rs index df4e8c1..57e2b4f 100644 --- a/src/header.rs +++ b/src/header.rs @@ -12,7 +12,6 @@ use crate::raw::TaskVTable; use crate::state::*; use crate::utils::abort; use crate::utils::abort_on_panic; -use crate::ScheduleInfo; pub(crate) enum Action { Schedule, @@ -158,13 +157,12 @@ impl Header { } /// Clones a waker. - pub(crate) unsafe fn clone_waker(&self, vtable: &'static RawWakerVTable) -> RawWaker { - let ptr: *const Header = self; - let ptr: *const () = ptr.cast(); + pub(crate) unsafe fn clone_waker(ptr: *const (), vtable: &'static RawWakerVTable) -> RawWaker { + let header = ptr as *const Header; // Increment the reference count. With any kind of reference-counted data structure, // relaxed ordering is appropriate when incrementing the counter. - let state = self.state.fetch_add(REFERENCE, Ordering::Relaxed); + let state = (*header).state.fetch_add(REFERENCE, Ordering::Relaxed); // If the reference count overflowed, abort. if state > isize::MAX as usize { @@ -175,9 +173,11 @@ impl Header { } #[inline(never)] - pub(crate) unsafe fn drop_waker(&self) -> Option { + pub(crate) unsafe fn drop_waker(ptr: *const ()) -> Option { + let header = ptr as *const Header; + // Decrement the reference count. - let new = self.state.fetch_sub(REFERENCE, Ordering::AcqRel) - REFERENCE; + let new = (*header).state.fetch_sub(REFERENCE, Ordering::AcqRel) - REFERENCE; // If this was the last reference to the task and the `Task` has been dropped too, // then we need to decide how to destroy the task. @@ -185,7 +185,8 @@ impl Header { if new & (COMPLETED | CLOSED) == 0 { // If the task was not completed nor closed, close it and schedule one more time so // that its future gets dropped by the executor. - self.state + (*header) + .state .store(SCHEDULED | CLOSED | REFERENCE, Ordering::Release); Some(Action::Schedule) } else { @@ -196,136 +197,11 @@ impl Header { None } } - - /// Puts the task in detached state. - #[inline(never)] - pub(crate) fn set_detached(&self) -> Option<*const ()> { - let ptr: *const Header = self; - let ptr: *const () = ptr.cast(); - - unsafe { - // A place where the output will be stored in case it needs to be dropped. - let mut output = None; - - // Optimistically assume the `Task` is being detached just after creating the task. - // This is a common case so if the `Task` is datached, the overhead of it is only one - // compare-exchange operation. - if let Err(mut state) = self.state.compare_exchange_weak( - SCHEDULED | TASK | REFERENCE, - SCHEDULED | REFERENCE, - Ordering::AcqRel, - Ordering::Acquire, - ) { - loop { - // If the task has been completed but not yet closed, that means its output - // must be dropped. - if state & COMPLETED != 0 && state & CLOSED == 0 { - // Mark the task as closed in order to grab its output. - match self.state.compare_exchange_weak( - state, - state | CLOSED, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // Read the output. - output = Some(self.vtable.get_output(ptr)); - - // Update the state variable because we're continuing the loop. - state |= CLOSED; - } - Err(s) => state = s, - } - } else { - // If this is the last reference to the task and it's not closed, then - // close it and schedule one more time so that its future gets dropped by - // the executor. - let new = if state & (!(REFERENCE - 1) | CLOSED) == 0 { - SCHEDULED | CLOSED | REFERENCE - } else { - state & !TASK - }; - - // Unset the `TASK` flag. - match self.state.compare_exchange_weak( - state, - new, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // If this is the last reference to the task, we need to either - // schedule dropping its future or destroy it. - if state & !(REFERENCE - 1) == 0 { - if state & CLOSED == 0 { - (self.vtable.schedule)(ptr, ScheduleInfo::new(false)); - } else { - (self.vtable.destroy)(ptr); - } - } - - break; - } - Err(s) => state = s, - } - } - } - } - - output - } - } - - /// Puts the task in canceled state. - #[inline(never)] - pub(crate) fn set_canceled(&self) { - let ptr: *const Header = self; - let ptr: *const () = ptr.cast(); - - unsafe { - let mut state = self.state.load(Ordering::Acquire); - - loop { - // If the task has been completed or closed, it can't be canceled. - if state & (COMPLETED | CLOSED) != 0 { - break; - } - - // If the task is not scheduled nor running, we'll need to schedule it. - let new = if state & (SCHEDULED | RUNNING) == 0 { - (state | SCHEDULED | CLOSED) + REFERENCE - } else { - state | CLOSED - }; - - // Mark the task as closed. - match self.state.compare_exchange_weak( - state, - new, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // If the task is not scheduled nor running, schedule it one more time so - // that its future gets dropped by the executor. - if state & (SCHEDULED | RUNNING) == 0 { - (self.vtable.schedule)(ptr, ScheduleInfo::new(false)); - } - - // Notify the awaiter that the task has been closed. - if state & AWAITER != 0 { - self.notify(None); - } - - break; - } - Err(s) => state = s, - } - } - } - } } +// SAFETY: repr(C) is explicitly used here so that casts between `Header` and `HeaderWithMetadata` +// can be done safely without additional offsets. +// /// The header of a task. /// /// This header is stored in memory at the beginning of the heap-allocated task. diff --git a/src/raw.rs b/src/raw.rs index 1a6c679..6807bd0 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -29,13 +29,13 @@ pub(crate) struct TaskVTable { pub(crate) raw_waker_vtable: &'static RawWakerVTable, /// Schedules the task. - pub(crate) schedule: unsafe fn(*const (), ScheduleInfo), + pub(crate) schedule: unsafe fn(*const (), ScheduleInfo, &TaskVTable), /// Drops the future inside the task. pub(crate) drop_future: unsafe fn(*const (), &TaskLayout), /// Destroys the task. - pub(crate) destroy: unsafe fn(*const ()), + pub(crate) destroy: unsafe fn(*const (), &TaskLayout), /// Runs the task. pub(crate) run: unsafe fn(*const ()) -> bool, @@ -148,7 +148,7 @@ where /// It is assumed that initially only the `Runnable` and the `Task` exist. pub(crate) fn allocate<'a, Gen: FnOnce(&'a M) -> F>( future: Gen, - schedule: S, + schedule_fn: S, builder: crate::Builder, ) -> NonNull<()> where @@ -180,9 +180,9 @@ where awaiter: UnsafeCell::new(None), vtable: &TaskVTable { raw_waker_vtable: &Self::RAW_WAKER_VTABLE, - schedule: Self::schedule, + schedule: schedule::, drop_future: drop_future::, - destroy: Self::destroy, + destroy: destroy::, run: Self::run, layout_info: &Self::TASK_LAYOUT, }, @@ -193,7 +193,7 @@ where }); // Write the schedule function as the third field of the task. - (raw.schedule as *mut S).write(schedule); + (raw.schedule as *mut S).write(schedule_fn); // Generate the future, now that the metadata has been pinned in place. let future = abort_on_panic(|| future(&(*raw.header).metadata)); @@ -275,7 +275,7 @@ where // time to schedule it. if state & RUNNING == 0 { // Schedule the task. - Self::schedule(ptr, ScheduleInfo::new(false)); + schedule::(ptr, ScheduleInfo::new(false), header.vtable); } else { // Drop the waker. Self::drop_waker(ptr); @@ -291,71 +291,12 @@ where /// Wakes a waker by reference. unsafe fn wake_by_ref(ptr: *const ()) { - let raw = Self::from_ptr(ptr); - let header = Self::header(ptr); - - let mut state = header.state.load(Ordering::Acquire); - - loop { - // If the task is completed or closed, it can't be woken up. - if state & (COMPLETED | CLOSED) != 0 { - break; - } - - // If the task is already scheduled, we just need to synchronize with the thread that - // will run the task by "publishing" our current view of the memory. - if state & SCHEDULED != 0 { - // Update the state without actually modifying it. - match header.state.compare_exchange_weak( - state, - state, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => break, - Err(s) => state = s, - } - } else { - // If the task is not running, we can schedule right away. - let new = if state & RUNNING == 0 { - (state | SCHEDULED) + REFERENCE - } else { - state | SCHEDULED - }; - - // Mark the task as scheduled. - match header.state.compare_exchange_weak( - state, - new, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // If the task is not running, now is the time to schedule. - if state & RUNNING == 0 { - // If the reference count overflowed, abort. - if state > isize::MAX as usize { - abort(); - } - - // Schedule the task. There is no need to call `Self::schedule(ptr)` - // because the schedule function cannot be destroyed while the waker is - // still alive. - let task = Runnable::from_raw(NonNull::new_unchecked(ptr as *mut ())); - (*raw.schedule).schedule(task, ScheduleInfo::new(false)); - } - - break; - } - Err(s) => state = s, - } - } - } + wake_by_ref::(ptr, &Self::TASK_LAYOUT); } /// Clones a waker. unsafe fn clone_waker(ptr: *const ()) -> RawWaker { - Self::header(ptr).clone_waker(&Self::RAW_WAKER_VTABLE) + Header::clone_waker(ptr, &Self::RAW_WAKER_VTABLE) } /// Drops a waker. @@ -365,51 +306,14 @@ where /// scheduled one more time so that its future gets dropped by the executor. #[inline] unsafe fn drop_waker(ptr: *const ()) { - match Self::header(ptr).drop_waker() { - Some(Action::Schedule) => Self::schedule(ptr, ScheduleInfo::new(false)), - Some(Action::Destroy) => Self::destroy(ptr), - None => {} - } - } - - /// Schedules a task for running. - /// - /// This function doesn't modify the state of the task. It only passes the task reference to - /// its schedule function. - unsafe fn schedule(ptr: *const (), info: ScheduleInfo) { - let raw = Self::from_ptr(ptr); let header = Self::header(ptr); - - // If the schedule function has captured variables, create a temporary waker that prevents - // the task from getting deallocated while the function is being invoked. - let _waker; - if mem::size_of::() > 0 { - _waker = Waker::from_raw(header.clone_waker(&Self::RAW_WAKER_VTABLE)); + match Header::drop_waker(ptr) { + Some(Action::Schedule) => { + schedule::(ptr, ScheduleInfo::new(false), header.vtable) + } + Some(Action::Destroy) => destroy::(ptr, &Self::TASK_LAYOUT), + None => {} } - - let task = Runnable::from_raw(NonNull::new_unchecked(ptr as *mut ())); - (*raw.schedule).schedule(task, info); - } - - /// Cleans up task's resources and deallocates it. - /// - /// The schedule function will be dropped, and the task will then get deallocated. - /// The task must be closed before this function is called. - #[inline] - unsafe fn destroy(ptr: *const ()) { - let raw = Self::from_ptr(ptr); - - // We need a safeguard against panics because destructors can panic. - abort_on_panic(|| { - // Drop the header along with the metadata. - (raw.header as *mut HeaderWithMetadata).drop_in_place(); - - // Drop the schedule function. - (raw.schedule as *mut S).drop_in_place(); - }); - - // Finally, deallocate the memory reserved by the task. - alloc::alloc::dealloc(ptr as *mut u8, Self::TASK_LAYOUT.layout); } /// Runs a task. @@ -593,7 +497,7 @@ where } else if state & SCHEDULED != 0 { // The thread that woke the task up didn't reschedule it because // it was running so now it's our responsibility to do so. - Self::schedule(ptr, ScheduleInfo::new(true)); + schedule::(ptr, ScheduleInfo::new(true), header.vtable); return true; } else { // Drop the task reference. @@ -693,6 +597,24 @@ where } } +/// Schedules a task for running. +/// +/// This function doesn't modify the state of the task. It only passes the task reference to +/// its schedule function. +unsafe fn schedule, M>(ptr: *const (), info: ScheduleInfo, vtable: &TaskVTable) { + let schedule = ptr.byte_add(vtable.layout_info.offset_s) as *mut S; + + // If the schedule function has captured variables, create a temporary waker that prevents + // the task from getting deallocated while the function is being invoked. + let _waker; + if mem::size_of::() > 0 { + _waker = Waker::from_raw(Header::clone_waker(ptr, vtable.raw_waker_vtable)); + } + + let task = Runnable::from_raw(NonNull::new_unchecked(ptr as *mut ())); + (*schedule).schedule(task, info); +} + /// Drops the future inside a task. #[inline] unsafe fn drop_future(ptr: *const (), task_layout: &TaskLayout) { @@ -704,6 +626,93 @@ unsafe fn drop_future(ptr: *const (), task_layout: &TaskLayout) { }) } +/// Wakes a waker by reference. +unsafe fn wake_by_ref, M>(ptr: *const (), task_layout: &TaskLayout) { + let header = ptr as *const Header; + let header = &*header; + + let mut state = header.state.load(Ordering::Acquire); + + loop { + // If the task is completed or closed, it can't be woken up. + if state & (COMPLETED | CLOSED) != 0 { + break; + } + + // If the task is already scheduled, we just need to synchronize with the thread that + // will run the task by "publishing" our current view of the memory. + if state & SCHEDULED != 0 { + // Update the state without actually modifying it. + match header.state.compare_exchange_weak( + state, + state, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => break, + Err(s) => state = s, + } + } else { + // If the task is not running, we can schedule right away. + let new = if state & RUNNING == 0 { + (state | SCHEDULED) + REFERENCE + } else { + state | SCHEDULED + }; + + // Mark the task as scheduled. + match header.state.compare_exchange_weak( + state, + new, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // If the task is not running, now is the time to schedule. + if state & RUNNING == 0 { + // If the reference count overflowed, abort. + if state > isize::MAX as usize { + abort(); + } + + let schedule = ptr.byte_add(task_layout.offset_s) as *mut S; + + // Schedule the task. There is no need to call `Self::schedule(ptr)` + // because the schedule function cannot be destroyed while the waker is + // still alive. + let task = Runnable::from_raw(NonNull::new_unchecked(ptr as *mut ())); + (*schedule).schedule(task, ScheduleInfo::new(false)); + } + + break; + } + Err(s) => state = s, + } + } + } +} + +/// Cleans up task's resources and deallocates it. +/// +/// The schedule function will be dropped, and the task will then get deallocated. +/// The task must be closed before this function is called. +#[inline] +unsafe fn destroy(ptr: *const (), task_layout: &TaskLayout) { + let schedule = ptr.byte_add(task_layout.offset_s); + + // We need a safeguard against panics because destructors can panic. + abort_on_panic(|| { + // Drop the header along with the metadata. + (ptr as *mut HeaderWithMetadata).drop_in_place(); + + // Drop the schedule function. + (schedule as *mut S).drop_in_place(); + }); + + // Finally, deallocate the memory reserved by the task. + alloc::alloc::dealloc(ptr as *mut u8, task_layout.layout); +} + /// Drops a task reference (`Runnable` or `Waker`). /// /// This function will decrement the reference count. If it drops down to zero and the @@ -719,6 +728,6 @@ pub(crate) unsafe fn drop_ref(ptr: *const ()) { // If this was the last reference to the task and the `Task` has been dropped too, // then destroy the task. if new & !(REFERENCE - 1) == 0 && new & TASK == 0 { - (header.vtable.destroy)(ptr); + (header.vtable.destroy)(ptr, header.vtable.layout_info); } } diff --git a/src/runnable.rs b/src/runnable.rs index cef1635..9135f67 100644 --- a/src/runnable.rs +++ b/src/runnable.rs @@ -743,7 +743,7 @@ impl Runnable { mem::forget(self); unsafe { - ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)); + ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false), (*header).vtable); } } @@ -810,7 +810,7 @@ impl Runnable { let header = self.header(); unsafe { - let raw_waker = header.clone_waker(header.vtable.raw_waker_vtable); + let raw_waker = Header::clone_waker(self.ptr.as_ptr(), header.vtable.raw_waker_vtable); Waker::from_raw(raw_waker) } } @@ -922,7 +922,7 @@ impl Drop for Runnable { } // Drop the future. - (header.vtable.drop_future)(ptr, &*header.vtable.layout_info); + (header.vtable.drop_future)(ptr, header.vtable.layout_info); // Mark the task as unscheduled. let state = header.state.fetch_and(!SCHEDULED, Ordering::AcqRel); diff --git a/src/task.rs b/src/task.rs index ddc8c8e..d32623a 100644 --- a/src/task.rs +++ b/src/task.rs @@ -10,6 +10,7 @@ use core::task::{Context, Poll}; use crate::header::{Header, HeaderWithMetadata}; use crate::raw::Panic; use crate::state::*; +use crate::ScheduleInfo; /// A spawned task. /// @@ -125,7 +126,7 @@ impl Task { /// ``` pub async fn cancel(self) -> Option { let this = self; - this.header().set_canceled(); + set_canceled(this.ptr.as_ptr()); this.fallible().await } @@ -180,9 +181,90 @@ impl Task { /// Puts the task in detached state. fn set_detached(&mut self) -> Option> { - self.header() - .set_detached() - .map(|ptr| unsafe { (ptr as *mut Result).read() }) + let ptr = self.ptr.as_ptr(); + let header = ptr as *const Header; + + unsafe { + // A place where the output will be stored in case it needs to be dropped. + let mut output = None; + + // Optimistically assume the `Task` is being detached just after creating the task. + // This is a common case so if the `Task` is datached, the overhead of it is only one + // compare-exchange operation. + if let Err(mut state) = (*header).state.compare_exchange_weak( + SCHEDULED | TASK | REFERENCE, + SCHEDULED | REFERENCE, + Ordering::AcqRel, + Ordering::Acquire, + ) { + loop { + // If the task has been completed but not yet closed, that means its output + // must be dropped. + if state & COMPLETED != 0 && state & CLOSED == 0 { + // Mark the task as closed in order to grab its output. + match (*header).state.compare_exchange_weak( + state, + state | CLOSED, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // Read the output. + output = Some( + ((*header).vtable.get_output(ptr) as *mut Result) + .read(), + ); + + // Update the state variable because we're continuing the loop. + state |= CLOSED; + } + Err(s) => state = s, + } + } else { + // If this is the last reference to the task and it's not closed, then + // close it and schedule one more time so that its future gets dropped by + // the executor. + let new = if state & (!(REFERENCE - 1) | CLOSED) == 0 { + SCHEDULED | CLOSED | REFERENCE + } else { + state & !TASK + }; + + // Unset the `TASK` flag. + match (*header).state.compare_exchange_weak( + state, + new, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // If this is the last reference to the task, we need to either + // schedule dropping its future or destroy it. + if state & !(REFERENCE - 1) == 0 { + if state & CLOSED == 0 { + ((*header).vtable.schedule)( + ptr, + ScheduleInfo::new(false), + (*header).vtable, + ); + } else { + ((*header).vtable.destroy)( + ptr, + (*header).vtable.layout_info, + ); + } + } + + break; + } + Err(s) => state = s, + } + } + } + } + + output + } } /// Polls the task to retrieve its output. @@ -323,9 +405,61 @@ impl Task { } } +/// Puts the task in canceled state. +#[inline(never)] +fn set_canceled(ptr: *const ()) { + let header = ptr as *const Header; + + unsafe { + let mut state = (*header).state.load(Ordering::Acquire); + + loop { + // If the task has been completed or closed, it can't be canceled. + if state & (COMPLETED | CLOSED) != 0 { + break; + } + + // If the task is not scheduled nor running, we'll need to schedule it. + let new = if state & (SCHEDULED | RUNNING) == 0 { + (state | SCHEDULED | CLOSED) + REFERENCE + } else { + state | CLOSED + }; + + // Mark the task as closed. + match (*header).state.compare_exchange_weak( + state, + new, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // If the task is not scheduled nor running, schedule it one more time so + // that its future gets dropped by the executor. + if state & (SCHEDULED | RUNNING) == 0 { + ((*header).vtable.schedule)( + ptr, + ScheduleInfo::new(false), + (*header).vtable, + ); + } + + // Notify the awaiter that the task has been closed. + if state & AWAITER != 0 { + (*header).notify(None); + } + + break; + } + Err(s) => state = s, + } + } + } +} + impl Drop for Task { fn drop(&mut self) { - self.header().set_canceled(); + set_canceled(self.ptr.as_ptr()); self.set_detached(); } } From b3abfa1811646eb06f82427ea74d9e77cb75a1ee Mon Sep 17 00:00:00 2001 From: James Liu Date: Mon, 1 Sep 2025 01:19:07 -0700 Subject: [PATCH 03/11] Clippy fixes --- src/task.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/task.rs b/src/task.rs index d32623a..3e0bbcd 100644 --- a/src/task.rs +++ b/src/task.rs @@ -369,12 +369,6 @@ impl Task { } } - fn header(&self) -> &Header { - let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; - unsafe { &*header } - } - fn header_with_metadata(&self) -> &HeaderWithMetadata { let ptr = self.ptr.as_ptr(); let header = ptr as *const HeaderWithMetadata; From 6f4de1e06e4d6cc9c5f17a28efdca507f31a6436 Mon Sep 17 00:00:00 2001 From: james7132 Date: Sat, 6 Sep 2025 18:07:30 -0700 Subject: [PATCH 04/11] Polymorphize more functions --- src/header.rs | 6 +- src/raw.rs | 344 +++++++++++++++++++++++------------------------- src/runnable.rs | 6 +- src/task.rs | 17 +-- src/utils.rs | 4 + 5 files changed, 180 insertions(+), 197 deletions(-) diff --git a/src/header.rs b/src/header.rs index 57e2b4f..150df0e 100644 --- a/src/header.rs +++ b/src/header.rs @@ -1,6 +1,6 @@ use core::cell::UnsafeCell; use core::fmt; -use core::task::{RawWaker, RawWakerVTable, Waker}; +use core::task::{RawWaker, Waker}; #[cfg(not(feature = "portable-atomic"))] use core::sync::atomic::AtomicUsize; @@ -157,7 +157,7 @@ impl Header { } /// Clones a waker. - pub(crate) unsafe fn clone_waker(ptr: *const (), vtable: &'static RawWakerVTable) -> RawWaker { + pub(crate) unsafe fn clone_waker(ptr: *const ()) -> RawWaker { let header = ptr as *const Header; // Increment the reference count. With any kind of reference-counted data structure, @@ -169,7 +169,7 @@ impl Header { abort(); } - RawWaker::new(ptr, vtable) + RawWaker::new(ptr, (*header).vtable.raw_waker_vtable) } #[inline(never)] diff --git a/src/raw.rs b/src/raw.rs index 6807bd0..5459ffc 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -1,6 +1,7 @@ use alloc::alloc::Layout as StdLayout; use core::cell::UnsafeCell; use core::future::Future; +use core::marker::PhantomData; use core::mem::{self, ManuallyDrop}; use core::pin::Pin; use core::ptr::NonNull; @@ -29,13 +30,13 @@ pub(crate) struct TaskVTable { pub(crate) raw_waker_vtable: &'static RawWakerVTable, /// Schedules the task. - pub(crate) schedule: unsafe fn(*const (), ScheduleInfo, &TaskVTable), + pub(crate) schedule: unsafe fn(*const (), ScheduleInfo), /// Drops the future inside the task. pub(crate) drop_future: unsafe fn(*const (), &TaskLayout), /// Destroys the task. - pub(crate) destroy: unsafe fn(*const (), &TaskLayout), + pub(crate) destroy: unsafe fn(*const ()), /// Runs the task. pub(crate) run: unsafe fn(*const ()) -> bool, @@ -137,10 +138,10 @@ where S: Schedule, { const RAW_WAKER_VTABLE: RawWakerVTable = RawWakerVTable::new( - Self::clone_waker, - Self::wake, - Self::wake_by_ref, - Self::drop_waker, + Header::clone_waker, + wake::, + wake_by_ref::, + drop_waker, ); /// Allocates a task with the given `future` and `schedule` function. @@ -223,99 +224,6 @@ where unsafe { &*header } } - /// Wakes a waker. - unsafe fn wake(ptr: *const ()) { - // This is just an optimization. If the schedule function has captured variables, then - // we'll do less reference counting if we wake the waker by reference and then drop it. - if mem::size_of::() > 0 { - Self::wake_by_ref(ptr); - Self::drop_waker(ptr); - return; - } - - let header = Self::header(ptr); - - let mut state = header.state.load(Ordering::Acquire); - - loop { - // If the task is completed or closed, it can't be woken up. - if state & (COMPLETED | CLOSED) != 0 { - // Drop the waker. - Self::drop_waker(ptr); - break; - } - - // If the task is already scheduled, we just need to synchronize with the thread that - // will run the task by "publishing" our current view of the memory. - if state & SCHEDULED != 0 { - // Update the state without actually modifying it. - match header.state.compare_exchange_weak( - state, - state, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // Drop the waker. - Self::drop_waker(ptr); - break; - } - Err(s) => state = s, - } - } else { - // Mark the task as scheduled. - match header.state.compare_exchange_weak( - state, - state | SCHEDULED, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // If the task is not yet scheduled and isn't currently running, now is the - // time to schedule it. - if state & RUNNING == 0 { - // Schedule the task. - schedule::(ptr, ScheduleInfo::new(false), header.vtable); - } else { - // Drop the waker. - Self::drop_waker(ptr); - } - - break; - } - Err(s) => state = s, - } - } - } - } - - /// Wakes a waker by reference. - unsafe fn wake_by_ref(ptr: *const ()) { - wake_by_ref::(ptr, &Self::TASK_LAYOUT); - } - - /// Clones a waker. - unsafe fn clone_waker(ptr: *const ()) -> RawWaker { - Header::clone_waker(ptr, &Self::RAW_WAKER_VTABLE) - } - - /// Drops a waker. - /// - /// This function will decrement the reference count. If it drops down to zero, the associated - /// `Task` has been dropped too, and the task has not been completed, then it will get - /// scheduled one more time so that its future gets dropped by the executor. - #[inline] - unsafe fn drop_waker(ptr: *const ()) { - let header = Self::header(ptr); - match Header::drop_waker(ptr) { - Some(Action::Schedule) => { - schedule::(ptr, ScheduleInfo::new(false), header.vtable) - } - Some(Action::Destroy) => destroy::(ptr, &Self::TASK_LAYOUT), - None => {} - } - } - /// Runs a task. /// /// If polling its future panics, the task will be closed and the panic will be propagated into @@ -375,7 +283,7 @@ where // Poll the inner future, but surround it with a guard that closes the task in case polling // panics. // If available, we should also try to catch the panic so that it is propagated correctly. - let guard = Guard(raw); + let guard = Guard::(raw.header.cast(), &Self::TASK_LAYOUT, PhantomData); // Panic propagation is not available for no_std. #[cfg(not(feature = "std"))] @@ -497,7 +405,7 @@ where } else if state & SCHEDULED != 0 { // The thread that woke the task up didn't reschedule it because // it was running so now it's our responsibility to do so. - schedule::(ptr, ScheduleInfo::new(true), header.vtable); + schedule::(ptr, ScheduleInfo::new(true)); return true; } else { // Drop the task reference. @@ -512,85 +420,83 @@ where } return false; + } +} - /// A guard that closes the task if polling its future panics. - struct Guard(RawTask) - where - F: Future, - S: Schedule; - - impl Drop for Guard - where - F: Future, - S: Schedule, - { - fn drop(&mut self) { - let raw = self.0; - let ptr = raw.header as *const (); - let header = raw.header as *const Header; - - unsafe { - let header = &*header; - let mut state = header.state.load(Ordering::Acquire); - - loop { - // If the task was closed while running, then unschedule it, drop its - // future, and drop the task reference. - if state & CLOSED != 0 { - // The thread that closed the task didn't drop the future because it - // was running so now it's our responsibility to do so. - drop_future::(ptr, &RawTask::::TASK_LAYOUT); - - // Mark the task as not running and not scheduled. - header - .state - .fetch_and(!RUNNING & !SCHEDULED, Ordering::AcqRel); +/// A guard that closes the task if polling its future panics. +struct Guard(*const (), &'static TaskLayout, PhantomData F>) +where + F: Future; - // Take the awaiter out. - let mut awaiter = None; - if state & AWAITER != 0 { - awaiter = header.take(None); - } +impl Drop for Guard +where + F: Future, +{ + fn drop(&mut self) { + let ptr = self.0; + let task_layout = self.1; + let header = ptr as *const Header; - // Drop the task reference. - drop_ref(ptr); + unsafe { + let header = &*header; + let mut state = header.state.load(Ordering::Acquire); + + loop { + // If the task was closed while running, then unschedule it, drop its + // future, and drop the task reference. + if state & CLOSED != 0 { + // The thread that closed the task didn't drop the future because it + // was running so now it's our responsibility to do so. + drop_future::(ptr, task_layout); + + // Mark the task as not running and not scheduled. + header + .state + .fetch_and(!RUNNING & !SCHEDULED, Ordering::AcqRel); + + // Take the awaiter out. + let mut awaiter = None; + if state & AWAITER != 0 { + awaiter = header.take(None); + } - // Notify the awaiter that the future has been dropped. - if let Some(w) = awaiter { - abort_on_panic(|| w.wake()); - } - break; - } + // Drop the task reference. + drop_ref(ptr); - // Mark the task as not running, not scheduled, and closed. - match header.state.compare_exchange_weak( - state, - (state & !RUNNING & !SCHEDULED) | CLOSED, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(state) => { - // Drop the future because the task is now closed. - drop_future::(ptr, &RawTask::::TASK_LAYOUT); + // Notify the awaiter that the future has been dropped. + if let Some(w) = awaiter { + abort_on_panic(|| w.wake()); + } + break; + } - // Take the awaiter out. - let mut awaiter = None; - if state & AWAITER != 0 { - awaiter = header.take(None); - } + // Mark the task as not running, not scheduled, and closed. + match header.state.compare_exchange_weak( + state, + (state & !RUNNING & !SCHEDULED) | CLOSED, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(state) => { + // Drop the future because the task is now closed. + drop_future::(ptr, task_layout); + + // Take the awaiter out. + let mut awaiter = None; + if state & AWAITER != 0 { + awaiter = header.take(None); + } - // Drop the task reference. - drop_ref(ptr); + // Drop the task reference. + drop_ref(ptr); - // Notify the awaiter that the future has been dropped. - if let Some(w) = awaiter { - abort_on_panic(|| w.wake()); - } - break; - } - Err(s) => state = s, + // Notify the awaiter that the future has been dropped. + if let Some(w) = awaiter { + abort_on_panic(|| w.wake()); } + break; } + Err(s) => state = s, } } } @@ -601,20 +507,37 @@ where /// /// This function doesn't modify the state of the task. It only passes the task reference to /// its schedule function. -unsafe fn schedule, M>(ptr: *const (), info: ScheduleInfo, vtable: &TaskVTable) { - let schedule = ptr.byte_add(vtable.layout_info.offset_s) as *mut S; +unsafe fn schedule, M>(ptr: *const (), info: ScheduleInfo) { + let header = ptr as *const Header; + let task_layout = (*header).vtable.layout_info; + let schedule = ptr.byte_add(task_layout.offset_s) as *mut S; // If the schedule function has captured variables, create a temporary waker that prevents // the task from getting deallocated while the function is being invoked. let _waker; if mem::size_of::() > 0 { - _waker = Waker::from_raw(Header::clone_waker(ptr, vtable.raw_waker_vtable)); + _waker = Waker::from_raw(Header::clone_waker(ptr)); } let task = Runnable::from_raw(NonNull::new_unchecked(ptr as *mut ())); (*schedule).schedule(task, info); } +/// Drops a waker. +/// +/// This function will decrement the reference count. If it drops down to zero, the associated +/// `Task` has been dropped too, and the task has not been completed, then it will get +/// scheduled one more time so that its future gets dropped by the executor. +#[inline] +unsafe fn drop_waker(ptr: *const ()) { + let header = ptr as *const Header; + match Header::drop_waker(ptr) { + Some(Action::Schedule) => ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)), + Some(Action::Destroy) => (((*header).vtable.destroy))(ptr), + None => {} + } +} + /// Drops the future inside a task. #[inline] unsafe fn drop_future(ptr: *const (), task_layout: &TaskLayout) { @@ -626,10 +549,77 @@ unsafe fn drop_future(ptr: *const (), task_layout: &TaskLayout) { }) } +/// Wakes a waker. +unsafe fn wake, M>(ptr: *const ()) { + // This is just an optimization. If the schedule function has captured variables, then + // we'll do less reference counting if we wake the waker by reference and then drop it. + if mem::size_of::() > 0 { + wake_by_ref::(ptr); + drop_waker(ptr); + return; + } + + let header = ptr as *const Header; + + let mut state = (*header).state.load(Ordering::Acquire); + + loop { + // If the task is completed or closed, it can't be woken up. + if state & (COMPLETED | CLOSED) != 0 { + // Drop the waker. + drop_waker(ptr); + break; + } + + // If the task is already scheduled, we just need to synchronize with the thread that + // will run the task by "publishing" our current view of the memory. + if state & SCHEDULED != 0 { + // Update the state without actually modifying it. + match (*header).state.compare_exchange_weak( + state, + state, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // Drop the waker. + drop_waker(ptr); + break; + } + Err(s) => state = s, + } + } else { + // Mark the task as scheduled. + match (*header).state.compare_exchange_weak( + state, + state | SCHEDULED, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // If the task is not yet scheduled and isn't currently running, now is the + // time to schedule it. + if state & RUNNING == 0 { + // Schedule the task. + schedule::(ptr, ScheduleInfo::new(false)); + } else { + // Drop the waker. + drop_waker(ptr); + } + + break; + } + Err(s) => state = s, + } + } + } +} + /// Wakes a waker by reference. -unsafe fn wake_by_ref, M>(ptr: *const (), task_layout: &TaskLayout) { +unsafe fn wake_by_ref, M>(ptr: *const ()) { let header = ptr as *const Header; let header = &*header; + let task_layout = (*header).vtable.layout_info; let mut state = header.state.load(Ordering::Acquire); @@ -697,7 +687,9 @@ unsafe fn wake_by_ref, M>(ptr: *const (), task_layout: &TaskLayou /// The schedule function will be dropped, and the task will then get deallocated. /// The task must be closed before this function is called. #[inline] -unsafe fn destroy(ptr: *const (), task_layout: &TaskLayout) { +unsafe fn destroy(ptr: *const ()) { + let header = ptr as *const Header; + let task_layout = (*header).vtable.layout_info; let schedule = ptr.byte_add(task_layout.offset_s); // We need a safeguard against panics because destructors can panic. @@ -728,6 +720,6 @@ pub(crate) unsafe fn drop_ref(ptr: *const ()) { // If this was the last reference to the task and the `Task` has been dropped too, // then destroy the task. if new & !(REFERENCE - 1) == 0 && new & TASK == 0 { - (header.vtable.destroy)(ptr, header.vtable.layout_info); + (header.vtable.destroy)(ptr); } } diff --git a/src/runnable.rs b/src/runnable.rs index 9135f67..5af4bf3 100644 --- a/src/runnable.rs +++ b/src/runnable.rs @@ -743,7 +743,7 @@ impl Runnable { mem::forget(self); unsafe { - ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false), (*header).vtable); + ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)); } } @@ -807,10 +807,8 @@ impl Runnable { /// assert_eq!(r.len(), 1); /// ``` pub fn waker(&self) -> Waker { - let header = self.header(); - unsafe { - let raw_waker = Header::clone_waker(self.ptr.as_ptr(), header.vtable.raw_waker_vtable); + let raw_waker = Header::clone_waker(self.ptr.as_ptr()); Waker::from_raw(raw_waker) } } diff --git a/src/task.rs b/src/task.rs index 3e0bbcd..c251ca2 100644 --- a/src/task.rs +++ b/src/task.rs @@ -242,16 +242,9 @@ impl Task { // schedule dropping its future or destroy it. if state & !(REFERENCE - 1) == 0 { if state & CLOSED == 0 { - ((*header).vtable.schedule)( - ptr, - ScheduleInfo::new(false), - (*header).vtable, - ); + ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)); } else { - ((*header).vtable.destroy)( - ptr, - (*header).vtable.layout_info, - ); + ((*header).vtable.destroy)(ptr); } } @@ -431,11 +424,7 @@ fn set_canceled(ptr: *const ()) { // If the task is not scheduled nor running, schedule it one more time so // that its future gets dropped by the executor. if state & (SCHEDULED | RUNNING) == 0 { - ((*header).vtable.schedule)( - ptr, - ScheduleInfo::new(false), - (*header).vtable, - ); + ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)); } // Notify the awaiter that the task has been closed. diff --git a/src/utils.rs b/src/utils.rs index 5c2170c..b9b250f 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -4,6 +4,7 @@ use core::mem; /// Aborts the process. /// /// To abort, this function simply panics while panicking. +#[cfg(not(feature = "std"))] pub(crate) fn abort() -> ! { struct Panic; @@ -17,6 +18,9 @@ pub(crate) fn abort() -> ! { panic!("aborting the process"); } +#[cfg(feature = "std")] +pub use std::process::abort; + /// Calls a function and aborts if it panics. /// /// This is useful in unsafe code where we can't recover from panics. From b204ae70354ec5ec7f95ce61e404ec0624616d72 Mon Sep 17 00:00:00 2001 From: james7132 Date: Sat, 6 Sep 2025 18:37:54 -0700 Subject: [PATCH 05/11] Shut up clippy --- src/raw.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/raw.rs b/src/raw.rs index 5459ffc..466c53d 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -419,7 +419,7 @@ where } } - return false; + false } } @@ -533,7 +533,7 @@ unsafe fn drop_waker(ptr: *const ()) { let header = ptr as *const Header; match Header::drop_waker(ptr) { Some(Action::Schedule) => ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)), - Some(Action::Destroy) => (((*header).vtable.destroy))(ptr), + Some(Action::Destroy) => ((*header).vtable.destroy)(ptr), None => {} } } @@ -619,7 +619,7 @@ unsafe fn wake, M>(ptr: *const ()) { unsafe fn wake_by_ref, M>(ptr: *const ()) { let header = ptr as *const Header; let header = &*header; - let task_layout = (*header).vtable.layout_info; + let task_layout = header.vtable.layout_info; let mut state = header.state.load(Ordering::Acquire); From bd57a3f1640126a966d4b06bb086244bd0c89a42 Mon Sep 17 00:00:00 2001 From: james7132 Date: Sun, 7 Sep 2025 13:58:46 -0700 Subject: [PATCH 06/11] Document DropWakerAction --- src/header.rs | 15 ++++++++++----- src/raw.rs | 8 ++++---- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/src/header.rs b/src/header.rs index 150df0e..cf74471 100644 --- a/src/header.rs +++ b/src/header.rs @@ -13,9 +13,14 @@ use crate::state::*; use crate::utils::abort; use crate::utils::abort_on_panic; -pub(crate) enum Action { +/// Actions to take upon calling [`Header::drop_waker`]. +pub(crate) enum DropWakerAction { + /// Re-schedule the task Schedule, + /// Destroy the task. Destroy, + /// Do nothing. + None, } pub(crate) struct Header { @@ -173,7 +178,7 @@ impl Header { } #[inline(never)] - pub(crate) unsafe fn drop_waker(ptr: *const ()) -> Option { + pub(crate) unsafe fn drop_waker(ptr: *const ()) -> DropWakerAction { let header = ptr as *const Header; // Decrement the reference count. @@ -188,13 +193,13 @@ impl Header { (*header) .state .store(SCHEDULED | CLOSED | REFERENCE, Ordering::Release); - Some(Action::Schedule) + DropWakerAction::Schedule } else { // Otherwise, destroy the task right away. - Some(Action::Destroy) + DropWakerAction::Destroy } } else { - None + DropWakerAction::None } } } diff --git a/src/raw.rs b/src/raw.rs index 466c53d..9175f85 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -13,7 +13,7 @@ use core::sync::atomic::Ordering; #[cfg(feature = "portable-atomic")] use portable_atomic::AtomicUsize; -use crate::header::{Action, Header, HeaderWithMetadata}; +use crate::header::{DropWakerAction, Header, HeaderWithMetadata}; use crate::runnable::{Schedule, ScheduleInfo}; use crate::state::*; use crate::utils::{abort, abort_on_panic, max, Layout}; @@ -532,9 +532,9 @@ unsafe fn schedule, M>(ptr: *const (), info: ScheduleInfo) { unsafe fn drop_waker(ptr: *const ()) { let header = ptr as *const Header; match Header::drop_waker(ptr) { - Some(Action::Schedule) => ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)), - Some(Action::Destroy) => ((*header).vtable.destroy)(ptr), - None => {} + DropWakerAction::Schedule => ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)), + DropWakerAction::Destroy => ((*header).vtable.destroy)(ptr), + DropWakerAction::None => {} } } From 2f8b13b6c23e41293c94a3a8c45b3b1647f371ee Mon Sep 17 00:00:00 2001 From: james7132 Date: Wed, 10 Sep 2025 19:22:30 -0700 Subject: [PATCH 07/11] Polymorphize some Task functions --- src/task.rs | 382 ++++++++++++++++++++++++++-------------------------- 1 file changed, 191 insertions(+), 191 deletions(-) diff --git a/src/task.rs b/src/task.rs index c251ca2..c174b7b 100644 --- a/src/task.rs +++ b/src/task.rs @@ -85,8 +85,8 @@ impl Task { /// .detach(); /// ``` pub fn detach(self) { - let mut this = self; - let _out = this.set_detached(); + let this = self; + let _out = set_detached::(this.ptr.as_ptr()); mem::forget(this); } @@ -179,189 +179,6 @@ impl Task { FallibleTask { task: self } } - /// Puts the task in detached state. - fn set_detached(&mut self) -> Option> { - let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; - - unsafe { - // A place where the output will be stored in case it needs to be dropped. - let mut output = None; - - // Optimistically assume the `Task` is being detached just after creating the task. - // This is a common case so if the `Task` is datached, the overhead of it is only one - // compare-exchange operation. - if let Err(mut state) = (*header).state.compare_exchange_weak( - SCHEDULED | TASK | REFERENCE, - SCHEDULED | REFERENCE, - Ordering::AcqRel, - Ordering::Acquire, - ) { - loop { - // If the task has been completed but not yet closed, that means its output - // must be dropped. - if state & COMPLETED != 0 && state & CLOSED == 0 { - // Mark the task as closed in order to grab its output. - match (*header).state.compare_exchange_weak( - state, - state | CLOSED, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // Read the output. - output = Some( - ((*header).vtable.get_output(ptr) as *mut Result) - .read(), - ); - - // Update the state variable because we're continuing the loop. - state |= CLOSED; - } - Err(s) => state = s, - } - } else { - // If this is the last reference to the task and it's not closed, then - // close it and schedule one more time so that its future gets dropped by - // the executor. - let new = if state & (!(REFERENCE - 1) | CLOSED) == 0 { - SCHEDULED | CLOSED | REFERENCE - } else { - state & !TASK - }; - - // Unset the `TASK` flag. - match (*header).state.compare_exchange_weak( - state, - new, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // If this is the last reference to the task, we need to either - // schedule dropping its future or destroy it. - if state & !(REFERENCE - 1) == 0 { - if state & CLOSED == 0 { - ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)); - } else { - ((*header).vtable.destroy)(ptr); - } - } - - break; - } - Err(s) => state = s, - } - } - } - } - - output - } - } - - /// Polls the task to retrieve its output. - /// - /// Returns `Some` if the task has completed or `None` if it was closed. - /// - /// A task becomes closed in the following cases: - /// - /// 1. It gets canceled by `Runnable::drop()`, `Task::drop()`, or `Task::cancel()`. - /// 2. Its output gets awaited by the `Task`. - /// 3. It panics while polling the future. - /// 4. It is completed and the `Task` gets dropped. - fn poll_task(&mut self, cx: &mut Context<'_>) -> Poll> { - let ptr = self.ptr.as_ptr(); - let header = ptr as *const Header; - - unsafe { - let mut state = (*header).state.load(Ordering::Acquire); - - loop { - // If the task has been closed, notify the awaiter and return `None`. - if state & CLOSED != 0 { - // If the task is scheduled or running, we need to wait until its future is - // dropped. - if state & (SCHEDULED | RUNNING) != 0 { - // Replace the waker with one associated with the current task. - (*header).register(cx.waker()); - - // Reload the state after registering. It is possible changes occurred just - // before registration so we need to check for that. - state = (*header).state.load(Ordering::Acquire); - - // If the task is still scheduled or running, we need to wait because its - // future is not dropped yet. - if state & (SCHEDULED | RUNNING) != 0 { - return Poll::Pending; - } - } - - // Even though the awaiter is most likely the current task, it could also be - // another task. - (*header).notify(Some(cx.waker())); - return Poll::Ready(None); - } - - // If the task is not completed, register the current task. - if state & COMPLETED == 0 { - // Replace the waker with one associated with the current task. - (*header).register(cx.waker()); - - // Reload the state after registering. It is possible that the task became - // completed or closed just before registration so we need to check for that. - state = (*header).state.load(Ordering::Acquire); - - // If the task has been closed, restart. - if state & CLOSED != 0 { - continue; - } - - // If the task is still not completed, we're blocked on it. - if state & COMPLETED == 0 { - return Poll::Pending; - } - } - - // Since the task is now completed, mark it as closed in order to grab its output. - match (*header).state.compare_exchange( - state, - state | CLOSED, - Ordering::AcqRel, - Ordering::Acquire, - ) { - Ok(_) => { - // Notify the awaiter. Even though the awaiter is most likely the current - // task, it could also be another task. - if state & AWAITER != 0 { - (*header).notify(Some(cx.waker())); - } - - // Take the output from the task. - let output = (*header).vtable.get_output(ptr) as *mut Result; - let output = output.read(); - - // Propagate the panic if the task panicked. - let output = match output { - Ok(output) => output, - #[allow(unreachable_patterns)] - Err(panic) => { - #[cfg(feature = "std")] - std::panic::resume_unwind(panic); - - #[cfg(not(feature = "std"))] - match panic {} - } - }; - - return Poll::Ready(Some(output)); - } - Err(s) => state = s, - } - } - } - } - fn header_with_metadata(&self) -> &HeaderWithMetadata { let ptr = self.ptr.as_ptr(); let header = ptr as *const HeaderWithMetadata; @@ -392,6 +209,87 @@ impl Task { } } +/// Puts the task in detached state. +#[inline(never)] +fn set_detached(ptr: *const ()) -> Option> { + let header = ptr as *const Header; + + unsafe { + // A place where the output will be stored in case it needs to be dropped. + let mut output = None; + + // Optimistically assume the `Task` is being detached just after creating the task. + // This is a common case so if the `Task` is datached, the overhead of it is only one + // compare-exchange operation. + if let Err(mut state) = (*header).state.compare_exchange_weak( + SCHEDULED | TASK | REFERENCE, + SCHEDULED | REFERENCE, + Ordering::AcqRel, + Ordering::Acquire, + ) { + loop { + // If the task has been completed but not yet closed, that means its output + // must be dropped. + if state & COMPLETED != 0 && state & CLOSED == 0 { + // Mark the task as closed in order to grab its output. + match (*header).state.compare_exchange_weak( + state, + state | CLOSED, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // Read the output. + output = Some( + ((*header).vtable.get_output(ptr) as *mut Result) + .read(), + ); + + // Update the state variable because we're continuing the loop. + state |= CLOSED; + } + Err(s) => state = s, + } + } else { + // If this is the last reference to the task and it's not closed, then + // close it and schedule one more time so that its future gets dropped by + // the executor. + let new = if state & (!(REFERENCE - 1) | CLOSED) == 0 { + SCHEDULED | CLOSED | REFERENCE + } else { + state & !TASK + }; + + // Unset the `TASK` flag. + match (*header).state.compare_exchange_weak( + state, + new, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // If this is the last reference to the task, we need to either + // schedule dropping its future or destroy it. + if state & !(REFERENCE - 1) == 0 { + if state & CLOSED == 0 { + ((*header).vtable.schedule)(ptr, ScheduleInfo::new(false)); + } else { + ((*header).vtable.destroy)(ptr); + } + } + + break; + } + Err(s) => state = s, + } + } + } + } + + output + } +} + /// Puts the task in canceled state. #[inline(never)] fn set_canceled(ptr: *const ()) { @@ -440,18 +338,120 @@ fn set_canceled(ptr: *const ()) { } } +/// Polls the task to retrieve its output. +/// +/// Returns `Some` if the task has completed or `None` if it was closed. +/// +/// A task becomes closed in the following cases: +/// +/// 1. It gets canceled by `Runnable::drop()`, `Task::drop()`, or `Task::cancel()`. +/// 2. Its output gets awaited by the `Task`. +/// 3. It panics while polling the future. +/// 4. It is completed and the `Task` gets dropped. +fn poll_task(ptr: *const (), cx: &mut Context<'_>) -> Poll> { + let header = ptr as *const Header; + + unsafe { + let mut state = (*header).state.load(Ordering::Acquire); + + loop { + // If the task has been closed, notify the awaiter and return `None`. + if state & CLOSED != 0 { + // If the task is scheduled or running, we need to wait until its future is + // dropped. + if state & (SCHEDULED | RUNNING) != 0 { + // Replace the waker with one associated with the current task. + (*header).register(cx.waker()); + + // Reload the state after registering. It is possible changes occurred just + // before registration so we need to check for that. + state = (*header).state.load(Ordering::Acquire); + + // If the task is still scheduled or running, we need to wait because its + // future is not dropped yet. + if state & (SCHEDULED | RUNNING) != 0 { + return Poll::Pending; + } + } + + // Even though the awaiter is most likely the current task, it could also be + // another task. + (*header).notify(Some(cx.waker())); + return Poll::Ready(None); + } + + // If the task is not completed, register the current task. + if state & COMPLETED == 0 { + // Replace the waker with one associated with the current task. + (*header).register(cx.waker()); + + // Reload the state after registering. It is possible that the task became + // completed or closed just before registration so we need to check for that. + state = (*header).state.load(Ordering::Acquire); + + // If the task has been closed, restart. + if state & CLOSED != 0 { + continue; + } + + // If the task is still not completed, we're blocked on it. + if state & COMPLETED == 0 { + return Poll::Pending; + } + } + + // Since the task is now completed, mark it as closed in order to grab its output. + match (*header).state.compare_exchange( + state, + state | CLOSED, + Ordering::AcqRel, + Ordering::Acquire, + ) { + Ok(_) => { + // Notify the awaiter. Even though the awaiter is most likely the current + // task, it could also be another task. + if state & AWAITER != 0 { + (*header).notify(Some(cx.waker())); + } + + // Take the output from the task. + let output = (*header).vtable.get_output(ptr) as *mut Result; + let output = output.read(); + + // Propagate the panic if the task panicked. + let output = match output { + Ok(output) => output, + #[allow(unreachable_patterns)] + Err(panic) => { + #[cfg(feature = "std")] + std::panic::resume_unwind(panic); + + #[cfg(not(feature = "std"))] + match panic {} + } + }; + + return Poll::Ready(Some(output)); + } + Err(s) => state = s, + } + } + } +} + impl Drop for Task { fn drop(&mut self) { - set_canceled(self.ptr.as_ptr()); - self.set_detached(); + let ptr = self.ptr.as_ptr(); + set_canceled(ptr); + set_detached::(ptr); } } impl Future for Task { type Output = T; - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - match self.poll_task(cx) { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match poll_task::(self.ptr.as_ptr(), cx) { Poll::Ready(t) => Poll::Ready(t.expect("Task polled after completion")), Poll::Pending => Poll::Pending, } @@ -554,8 +554,8 @@ impl FallibleTask { impl Future for FallibleTask { type Output = Option; - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - self.task.poll_task(cx) + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + poll_task::(self.task.ptr.as_ptr(), cx) } } From 0601e63502f99cef9d8d7d5b5207235ef0145d77 Mon Sep 17 00:00:00 2001 From: james7132 Date: Fri, 12 Sep 2025 23:26:56 -0700 Subject: [PATCH 08/11] Fix build --- master | Bin 0 -> 15512 bytes merged | Bin 0 -> 10684 bytes merged-with-shared-schedule | Bin 0 -> 10684 bytes no-box | Bin 0 -> 11696 bytes polymorphization | Bin 0 -> 14202 bytes polymorphization-shared-schedule | Bin 0 -> 14428 bytes src/raw.rs | 4 ++-- src/utils.rs | 1 - 8 files changed, 2 insertions(+), 3 deletions(-) create mode 100644 master create mode 100644 merged create mode 100644 merged-with-shared-schedule create mode 100644 no-box create mode 100644 polymorphization create mode 100644 polymorphization-shared-schedule diff --git a/master b/master new file mode 100644 index 0000000000000000000000000000000000000000..0819eb6cdad7a902ab9e4678e288c822d7d5a31f GIT binary patch literal 15512 zcmds8T~8ZF6rJZv{SSVrL{y5f`7n-9B$9~qp;4+p)JMy~fv|dKH$6`-4FgMzc~F_bP5~bJv z)fDgXh~M&b{#xRopI9V%$kAXC3Fd2a5r;h-ZR1GuRf@ze)-lUnW^Cfl@pB0eC&;_) zAx54UW%h_MUR$xl_c!2=S|V#@U7WiS*7XuV%sBP+eestW3;f85hg+u6Fcx?;MWmFwu`pBt%{10 zrJp=op3iYRLF9<%kBA-_bPNnn@b^7X5L(B*OACc#B;yiCEOBO56yWC^S24gU(V0RY9d4L>-9L)G4qln3 z4Sa*5rG$cJTs}o?uYnPA+-+oUAJCag_B_86!-k8kIexfK{n`S~L;fh+%WA^v z{@Q1nJo^|~e~%+tHv?m9<+=74zVT=leL#$yc&3sS4 z5!RWQ&-&O$x(^vg$;Bg z8(`@pei`wQ_xi}N+QWDoi^RC4)-ohk1Qx+Ndqm*s>i*=f{ zUB8Z*5wuZ3s^*~utc9O}fT@q|V}G>omF)pkZI=p=j7|+gPZ(e2&5>u6n0a!%x$D=N zQY>esE8eV^A=S3(7d>J98u@x)_Lk8W>&dcTO&Ikpaw-akyZp-U^>CLH`NUAwYFcBe zMRtj;Q{A4b;IIOzqCWSS`T)eS4^FSH978OydP~fB^*TFLdk8@_E43ilwA`7$>RWGv z=zA)svFDK+|CyXI1?IvPq*Q(@sH|= zk*~AHkVJfre}+Dm)|J%bQR#@O6K)b@QAw4oiP=zhN~jKozQ&T}bq5gnq5$*zuYQ#B zcGXi`Lyve)dMY6Fv?-<@K|RD7<8;vQuKGkW7VDTC-r0-a zN8b^4d3AHw2gW(Q6dhxpP}i@noZ^YC3v;JaNLs>rbcx9>=Mw2L>lHuao~M2vtE%U+s2G=(6x<$SdLI)AnXir!H|x~uW3-%S z28Xj+i-kPocEmi~~zbReZ5^C4C*v{cXxeO<rmI9!cunFaU^N2F>!3I>AKPmm9E*mN|~4ypHchWcRKs{n`hLNThimhfO`ZXSIA0L zkFmSoE8rroE4(kJ_|a~%>P;#7G6#yMqAy`%I4PC6Xt%SAW&Xu_v+SQ`wd5U@?)n!IHJPTpTS}QSMKO)qx{8U1|5eY>yj`%6NTx@d{mu5@!pSiZK+iQsLi}p*8MKer9~|5lI^Ne&#jYG zO&i8IUVX5zSdcKA&yd*l+e&^`Rp5M*rJYaKBZL;#o4J^E^g6F2r~Wy|_?bR-skb=e zA!Z1C^)u?QJ@LmJ&V0WiQ{^N0eSsa2(}BFZ)Rq5Z%rL2@iEHY*S@Il??DAYM$z4|$ zLv7+6Mu|^C>NB~FV)dPPF2#~ zJ+tQvcsG&U!h77j2g-Yn+!K7iYJog;u2@~iytEl|ZN}1$6)&K)s2kqQ6?Ij7)XEhJMy# z#PEzd9#mN*okmoxlDfBgo$L)&?8b;!;6MJqB71Pb;W1v%fHY2MIhB(C3zpeF)h_SX Hs|NZ90zOGX literal 0 HcmV?d00001 diff --git a/merged b/merged new file mode 100644 index 0000000000000000000000000000000000000000..c202dceaa4c6edd0d771f4bfa356addf099805fa GIT binary patch literal 10684 zcmd5?T~8ZV5WUZp`XBrf2~>)(&4*oCNHh`YLn9=h%A*B1K!w4`hLR}CUvGQP+?}0! z*L!!@td&-l!>)b3J9FmDnYpw1=kJc|xZmBM?!kR>3mjL!-nyB4#Oms`$L<-|yK+xh zIm1y0?+bU2U;XHQkURYB^i_SoPQH%o(`)r=%YXHbeYfX^Zs7VASMk>$xSrduUA5uP z-3i`DKyNemtLJubp8@{uxD6Z~xNThRjoZSxoo&k1``M~K*6e3%I;lz z@&-N;jdC05RX<-dl6zNM`5uh8fYlSH79Qg2^rr)!%<#=8v6Spu$k7C!@r>TS%Z|M_ z`i=S1bGx~vw11F!!)S+SmiyBM8llf=%!wkIeCei{y$e{#NQCHiVa?EW;n@a`wsGXK z9S9p}o5!|?Z^*_CFk~;Vx-T(cM7SSf_afiWJz*Vl zLs#vKC&qyCoV+gRI-_B+w2 z1>@;g)km-yt9qh`%){6wa@`OqkL|u>j^=7QTP-J4y~d7cK4nY~F`}CJ?4v!Z%tPU? zc~3n2Dpg}*bbwdL$p1>4n<7u`Rpc$7sfLDWIr6Nr>(8nwG$4!xtK+c5LRJ~8faQwg zEmXCP>zr0~;>28@p*x~h7tfjN{EkY&8ac-MGd{b;`GdQWPhRlv1^R;6ZtX#c5?SoE zK&xX_o(-gG?dEwFYQZCHor{&mV`?Sq%@kHJ-$&4It%@maP4nZh;0KjS8Df5DU!{JO zDZcl^GB{PQ!oJG-DE3g=KP})DE9*r5(#u;MO$wHVTqZX>;m%ODG;gqG! z&I`#_uT{Sl&YAId_^fD`DNSrw9ml<0hI7iUWsb)dXho-Hn+4pT)%BRZJFqTqbQMZar(A^K?zci!dKdS8l3xXZ7ayn@5R1l{#kGQ8cxJ@LaSChe-5L#yk{y? zI5K4Q$u7Ce-c~g&_NTrtBgd^T3uDm|2y7P0XxHtg0UC=3YBLOodnHUigdV`ucy>sAW#KL)LOX z#Hm?Uue16WGu8GmG4HLaSJh)%Wlsq>V_@EKzQ&m!ClS;SeqOx0?GRSIdptV{F*W5* zO)O}g&L@f%?Ix7VJWb(2i>wyCR=FO~ow75z&U(9M>ebF_t2imEZ4$tlqIJ>D^|hUa zwu@RV($!CTi0xm(P5*>+8=l<)`+Gdci=X5%=IuPNnpXC_uKo2*xzn?b>M2574^82k zX+d-D*ev%oDq)!$a;KT{rt+bx|5&aZMf*9Q=B$R^R8ME(S(@R>F3oqR+@oJ%pU2&n zf1W{H7+0%>rl_&8?m0qE{4x) z_>9BPYwFnC&UVwp@57pph|d#z{Qz%SEhl*MuT9n08=G=PZI=~g5iCCKS)(&4*oCNHh`YLn9=h%A*B1K!w4`hLR}CUvGQP+?}0! z*L!!@td&-l!>)b3J9FmDnYpw1=kJc|xZmBM?!kR>3mjL!-nyB4#Oms`$L<-|yK+xh zIm1y0?+bU2U;XHQkURYB^i_SoPQH%o(`)r=%YXHbeYfX^Zs7VASMk>$xSrduUA5uP z-3i`DKyNemtLJubp8@{uxD6Z~xNThRjoZSxoo&k1``M~K*6e3%I;lz z@&-N;jdC05RX<-dl6zNM`5uh8fYlSH79Qg2^rr)!%<#=8v6Spu$k7C!@r>TS%Z|M_ z`i=S1bGx~vw11F!!)S+SmiyBM8llf=%!wkIeCei{y$e{#NQCHiVa?EW;n@a`wsGXK z9S9p}o5!|?Z^*_CFk~;Vx-T(cM7SSf_afiWJz*Vl zLs#vKC&qyCoV+gRI-_B+w2 z1>@;g)km-yt9qh`%){6wa@`OqkL|u>j^=7QTP-J4y~d7cK4nY~F`}CJ?4v!Z%tPU? zc~3n2Dpg}*bbwdL$p1>4n<7u`Rpc$7sfLDWIr6Nr>(8nwG$4!xtK+c5LRJ~8faQwg zEmXCP>zr0~;>28@p*x~h7tfjN{EkY&8ac-MGd{b;`GdQWPhRlv1^R;6ZtX#c5?SoE zK&xX_o(-gG?dEwFYQZCHor{&mV`?Sq%@kHJ-$&4It%@maP4nZh;0KjS8Df5DU!{JO zDZcl^GB{PQ!oJG-DE3g=KP})DE9*r5(#u;MO$wHVTqZX>;m%ODG;gqG! z&I`#_uT{Sl&YAId_^fD`DNSrw9ml<0hI7iUWsb)dXho-Hn+4pT)%BRZJFqTqbQMZar(A^K?zci!dKdS8l3xXZ7ayn@5R1l{#kGQ8cxJ@LaSChe-5L#yk{y? zI5K4Q$u7Ce-c~g&_NTrtBgd^T3uDm|2y7P0XxHtg0UC=3YBLOodnHUigdV`ucy>sAW#KL)LOX z#Hm?Uue16WGu8GmG4HLaSJh)%Wlsq>V_@EKzQ&m!ClS;SeqOx0?GRSIdptV{F*W5* zO)O}g&L@f%?Ix7VJWb(2i>wyCR=FO~ow75z&U(9M>ebF_t2imEZ4$tlqIJ>D^|hUa zwu@RV($!CTi0xm(P5*>+8=l<)`+Gdci=X5%=IuPNnpXC_uKo2*xzn?b>M2574^82k zX+d-D*ev%oDq)!$a;KT{rt+bx|5&aZMf*9Q=B$R^R8ME(S(@R>F3oqR+@oJ%pU2&n zf1W{H7+0%>rl_&8?m0qE{4x) z_>9BPYwFnC&UVwp@57pph|d#z{Qz%SEhl*MuT9n08=G=PZI=~g5iCCKSB1VMhi?Kv|X z%`U~INQ*5}7=lS@DbD50nX^Ov{Z}va!Y|>Ea2w9U1CHzeUWL1GkJYQzz6l@jjc4Hz zYwvK>!|Np6;JRu6Q`*v4zG_Ir4bKK}OcS9d>t z_97gIcX%Blh8qjt`e76XVHjS@iXD8{7LLCQn>hEgjpEw}#ky^*+AUV~V5gqxsuSeq zlSFianDXp8{D{9fj-TY{y&N4EpN+#C-0c{zBfNeM2gPb;#iG$e97edeTG@wXTX;vU z+=V52R&OO&^>K}^8q3|gvXlP31&&VO~(9kM|T?tF*{?7?ZZZ`3&U7-bo-Kp{RVat-Q=0mFe~uz z00gi?V${=`?ILcphxlb>vfi87!)SYayNTNaaC`-;r$9Wd_xSatPIn~FjF{CZ`6aDU zqIg@@(LY}~Sm1+^&e{7x*m_pf52MJsdIa*w;1h}DE%^K_v5mZ;7$jq{Rv61+!`zZ3 z^(w_c&bD?uXB9p{=Cy|AiOnw;+)SI9Nwqo6Yp&YM4BnN^?5U5!RG%Vidr~{b(O%_4 zL9179-dbXYedXVYFqt=ICDxHu7yF)dw4r3$t2l2j8JOL&OMA~yQ$#V@v>*Kj#-&qTJ1Q#04qG=#b`S-u!CDaVyq3xBT*b~4fV`!QKinN#QdJDG%o7~$#d9w zD}DEK_zbpSCV^_1>SI?I&{A zGgSXfEV;oEHEtX`a;4T4ztg%WmD?7qitoKwtTVNvkt0sMu2$chN@R6J2FQt9#TaNr z*s1((IUG{a#fq$(j9vnD+6Rcjd%|M&c%<7|b%ZVWENj}pnroh?+pV^AkN~(IpP-O-j#Ul&_`4{PUQr}s5- zGL_{FcCZr0=oF^da~z&W$wA4J-6#j%8<_*Ly>{wcZ&6RY59F9M-i`b2n>%mYbw9u_ zvgcI((@)j;DPH1UAiRw1jW4W4Xi$k5{ zOb|6!!A9U)`C!1iT z=XYxT4sue87u%&Sw0*kG*-`l$-YD{cxn_{a_b&&*te%9@CO(1JnQvOv?Jdu1_v&cc z4!){GJiq_0?~!-IwrujXdE(66zOv#gnltX#%2~@5yDsj%KH^9A>HaTHoP4oTo{q-q2cNgvXB;8f7$Aq`gz*Gqrh1P1uhU;4I ziYi8hv)QUi5~I94Kcek_O2qYd7h8z`6@0tFv%d6cA9GvwfwO$eNBu;$JU6I%rq{(C zTnRSASo3aH-r=4p7ha)4IG;#$Vso_=XEMJ(?LIE91=^!BzM_f! zWp41_2dF-1$_>lwbU$XA+ga`&oW{u?chAap$2gwLQ(VriZFZpZX)4zHg4<~8KYB?R APXGV_ literal 0 HcmV?d00001 diff --git a/polymorphization b/polymorphization new file mode 100644 index 0000000000000000000000000000000000000000..4f7e3a00c05c437b129285fda0cde9804c2844fa GIT binary patch literal 14202 zcmdU0T~8ZF6rJZv{SSVLL{y3b*w~H`5=lh*&?r?P%A@7rBt#*AOj3fX{`I!!%=Pu1 zncZ2>vTi@M9ERTV`=l$${@_YR1^wGRyZ@G@^o7U>@RbO?_ksG=_x9c7>{toxp!J}l|hC6Z3 z@jQeM?=NNRyKOx7{5~75k6pL${+`>!FWb@m1qxlagH^-DDk6|0Sv7@iZoNh)&?%2T zxQBSm@&3lYd*R=mEWX=!N7(HI&tp8Fxr4=OT1B<74J?L|@3ye3prn?z@a;WFB|5K1 zIl!v^a@D@yy&30Ic*YdCXHGF5UQXS{p{Mmo8dDaQQjwu_acveqZ0L? zzg;(4=zX-nMy(Ga%N8Uwd3sCxsHVBDhqFVu*Q58TJICV^@=akqi-8`{4&4|U>^b_$ z7LG;#Lr2Yy1H1!Bu;bU!GtHlNy=3$*S}KS4vya5{g+{0Dwb%3~H^J)#{$AnC3&iT_ z(mRNaJ2F-_{EpO~=yHtx&Fa*e#L9-pqB87yAGUQlZ05r^K=A?knE>A{{?ZrT28edB zYv_lg1)`DPiN34ennOg*F~&-xN)efdZNo!C42%(KW#sK=(&n{7%UWdRwZd2>Yrcl> zNK8CJ?7YX(499cCkH+=2x8IGor10(;R-AfW&I0{0#x*(^5A-j=&e*5jR_TxHOvXHR zWu!27H*(t7ZxOj9Pw17IHbIHx5v2aF+|9ED!7yS?ENS9vtF=HrB9^_xk7W&IqdQf)!7v6V)X%icW)V^)xC+tJR{^p ztzrg1E162xT_X$4{9cxCKKMP~`A;Q3k})QTxeHV_#3N_S;5QRQBr_;iSR@jq<{zTe z-N#F|FqFEEey(|H;3KvTtC4>x3+s~G1PNI|ZDmN9+hY&h!!g&f`cRwlvv6H~J@nS= z#%gQY?g)B%j^`>Pz!F>kz?W`l_?)Ua~G9Cz=JcPMaW` zvl0b4RZdo8(OmtIRHhkf*LZFA{}CJz1PR#yMDtTvS#e zgmTtrs=TZRs4PNjZ3FRm$;-44c|)uf)|#}J@wetsk=NFR@@ijBEX+-p$fN9|(qrZ> ze#gwmYI1_-7kqYx;|urBe{zGr?0{nAAv4CvW45S^Q0a^;x9xir+GAT5CfYFLPLSWG zU~9=Hhd8>%`#I!$3ACD*j~8p$$*-#9R%NYy6vdKg&7#>>Vu+H&oKY7;B8jcbFBX^g~AI$s1YPZzuw~-084vzI7vsS&Y5TiwMT;s$?PmxtF z(PQ#cVzX~2;9>G@oVTrOrxzQtSS!?C39_b2vzobIc#gH1j!baY*#R@B88gPxnLcyA zAsxTXB+_ibJWd_|;rp_Eyw~&p-_umbkJUYW01W3tZQs~wNCk~yqnI=#h%n~s283mwS-R2 z+p+tq-wMBP+s|x9NZXz*oKQc}?)J;w&ThgL_is&{RpM=|=ZxF_UvCTTT<25a_iZit zJ?Pt5D<9qL32a?^eaB~f3)WBlFPz!Mb(P6UebiNrY2Oh_rz}o3YMvj^S6HdUS%_Cz zwa(Ug1!o;$|I&=hdWJZg#z)m@1+N$Ab)Fd0PEh-I$D|s{)GajN(=DS~_X%ziw?rUwSevTnEul}U7JVsc~=S`}-^o$LwJSLbV zu)5~;9D3M1xuV0}suMwpu=cyP*L8RX?}E`gs;I^1!ExNHg;Nxb3ZC%_I%YRRPfHoD zyPV`jnH-0x%X}(4gYywybw7u+MExQ-mO0^O5FcFeHGHtDFYB~pmMirO(^tOI3@v#A zsd?v@Ts`MJtd3*ucITADi_OTpN)+0&%mX&OboFTq{vCAZj|C5NbV=0MM`ui8`0Hfy_^-3)VE9$B{sc9o>dWFqw0a<2Ve*{bTXklX|I_PQ?tI*W=Fj{rKfVJ` ze06;?)ZPEYJ-Y5-^NBL=Drfz)t7+?ZhAgySS+>55r&bwheG-^%Sp|sy@v0DKd93Q? z&I9T;uJ~W0wtVNWXE4Ig5J!w~-hrH+n?`-wgS6GY%?-Ye_FqBDPmu5#Bqo#1@Z_sq zdH)e4th*AD)`C?(>uM{Tk*v>FO?ByyAwl_n6)^62=kG_v)%s<#eyhE8!ZUbp5+_ucTWI^jp{bDGec0NETq&UrPtRsQ!)u5z$T;8kv|AO8c<(wOJ~ literal 0 HcmV?d00001 diff --git a/polymorphization-shared-schedule b/polymorphization-shared-schedule new file mode 100644 index 0000000000000000000000000000000000000000..3dfff644fc26a25effec882450a9eae91c13eeac GIT binary patch literal 14428 zcmdU0T~8ZF6rJZv{SSVLMAXy@27-wY5=lh*&?r?P%A@4~2~h|j10|^HUvGQP+`Yat zvpf62ZbB+#@UCaQ_v74iKW6#&pOG86-`yv7=8oMZ-q-&;cjxW`tDDyDxodpmp}WG` zbG#ejHFsZd-OuhPe~(|C{?y-bv|Pvab!+we%768qn{LZ(xK+IBzK5T!VkLi?)~&b$ z_ZqKV*zn;}w{={<>YljAxPHaoaqOnPx)^PrDv1;V>NLJ0^n_F+u z8End<&+aLH7I=T--@Wng4vNpV-5&0CfY&}=uiQ?tnqEK33MOh z_XB7pHqdn{7guqe$SkY2{oU*3dE9jU9FM5) zi{icLqo1foBj~UOON-s9!CGO@O`IJ=C)0j@{zd#{lj&`$e-f4ohFZA zNiyyVG%`)Lpxqc}%o9YBL_+AXjV0QRV>Ft0eZU3~R@F*gB1djU;OF9wDxWh=w-a5vCWb(;s9=(Teyn9u%b6I@MmZVr~=fgz>42 zcM88eMed&?qAmMK_RtvRy){E_dF(cG^#-m8GwCHb?#SQE*zB{v=SP25*!R$#fbVGI z8~>SHaRl7V3V!Aai$q$8u}J*JQZjFZ8Img}ejPti3-RhO^Hk@!4_+dxmuRs)$oLws zL!g_-l%L2K&gR)rO2&zxj#82##E;A>A>UQ!y(d1(%;tG?D&J}DojJ;WRj^nn5i^!a zqzt^3+=+Q+QmoTzf|74~Ynw#(Ra>-+qIE=fm9ESvWB|=J)tPeBBi^#T@htD2V>qdS zShwFqNLXLmlSokh*#bJ#h}P%}E1*r!QzpwWQAON_*s!ci88y3!jCSu_WwGgXF>K_~ zTISM_p~@9RJ8#!XZ(Nv{p1fsRhKz85NLu)4xA-D!vO+sW{4?qq3C$TW)U%q2^SbUJ zbGGpoJBBoamop!`u8;xyNES*ZJ(}{eR{C4M!ZYjUGFK{9=mqVyXdzbY=g3~HjniXh z41UIZ!y0ae*H@f7!SR{<=+E5X_Z;l%_EKoyWVw&h zhIwv=>VFQlmTa<%qf5MBK)<(;RJ%zvZkRPAqIf!Rdtcb;rQr%=dd-%@~S1y0A_t%WBrlvF_BfkX5wpSd=oh=_43N z$A~TB=Y@|d9=-CFkgGeWf=(y2w5AL4BB61n3XH3iQK zpXu5eIF4KvXKL$1X?69yNEY{@dC4p$>Z-VMu72bB%X%T10}R>IvBzG|*ga;QHrnOz zZ&SJ(*qyVA89Z={--4PD3G6712V;91zvWOaU9}-5%?+$2==DE*@3W1+)f^wz^jx1F z?|B5v{Km!$dAai!O$-oOQ_*^I}p&S$|88$tSdVY-MKAa=cNp zxRK_vRy5%)naZkA6l`G4Z7>!oE15l3lTr0{$)N-r@G=##%yR8UMKlq z=epgpM$>nNpLgwy9z~MIS6?oxmTNHKaZ&YFPkQUVM<->pOZTh0ND3Ntma}SHbl3TCA75W2F7yzB!MVp6%)&~7E~#^H_UzM_HtPch;wv&RGwQfPP8+K zWts$?N`INuLb>c|b1tIe9c4J#(fH&Q?_gv0T*Q3J`{8cd{p!i? zoBQvl(3Voa365oU`6!4~uJ|4pEZfWG0e7>P@nF7^4n27Qt@(tJ?4xr=SH*UBdqPLD zD4(p7{nWl1g}80UDl1;Q>Zg>-m1a7FA|k4DR<#AC$Kez_&Z3-+^s|AnLQbN#lMy1j zy3ll+ zZ`@Tf>y1MUwX3x564B*sGMom+x~$-Y^SBerIWs#ldA^3Wjk~q?=nFpj9W%1?3HXM- z;T~RkZHmpbf9z^~>mT_%hSoqiyH-6(y`M~_We-*M?3z$Lob=xZJMZA5EBNOX{KXu4 zgctwXb0|amr%%V-#fxbdQjU_ozWNrL)OAF|YhYdOaA<4)e-mVlXJCdeV1*w`x6m4V zevngLO8XY|ralq&yf`0sW$k;D(9`)j(C~J#$~)AuU3LnuXe0m7++ng?Ub;YL;r|J7 xE+zTN{#Pfxe>yE}n_J53gda7#)Jbyy$rivk_m9Y}^51=uP1v9EDG=>k{sXFLw literal 0 HcmV?d00001 diff --git a/src/raw.rs b/src/raw.rs index 6bf00a0..415bb34 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -149,13 +149,13 @@ macro_rules! allocate_task { } = $builder; // Write the header as the first field of the task. - (raw.header as *mut HeaderWithMetadata).write(HeaderWithMetadata { + ($raw.header as *mut HeaderWithMetadata<$m>).write(HeaderWithMetadata { header: Header { #[cfg(not(feature = "portable-atomic"))] state: core::sync::atomic::AtomicUsize::new(SCHEDULED | TASK | REFERENCE), #[cfg(feature = "portable-atomic")] state: portable_atomic::AtomicUsize::new(SCHEDULED | TASK | REFERENCE), - awaiter: UnsafeCell::new(None), + awaiter: core::cell::UnsafeCell::new(None), vtable: &RawTask::<$f, <$f as Future>::Output, $s, $m>::TASK_VTABLE, #[cfg(feature = "std")] propagate_panic, diff --git a/src/utils.rs b/src/utils.rs index cd40cf8..0e8da5f 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -4,7 +4,6 @@ use core::mem; /// Aborts the process. /// /// To abort, this function simply panics while panicking. -#[cfg(not(feature = "std"))] pub(crate) fn abort() -> ! { struct Panic; From 8d1a1b5211b7e25ffe9f72972874e1f46f918df5 Mon Sep 17 00:00:00 2001 From: james7132 Date: Fri, 12 Sep 2025 23:41:37 -0700 Subject: [PATCH 09/11] Review fix --- src/raw.rs | 27 +++++++++++---------------- 1 file changed, 11 insertions(+), 16 deletions(-) diff --git a/src/raw.rs b/src/raw.rs index 415bb34..5f66025 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -216,24 +216,19 @@ where } } - fn header<'a>(ptr: *const ()) -> &'a Header { - let header = ptr as *const Header; - unsafe { &*header } - } - /// Runs a task. /// /// If polling its future panics, the task will be closed and the panic will be propagated into /// the caller. unsafe fn run(ptr: *const ()) -> bool { let raw = Self::from_ptr(ptr); - let header = Self::header(ptr); + let header = ptr as *const Header; // Create a context from the raw task pointer and the vtable inside the its header. let waker = ManuallyDrop::new(Waker::from_raw(RawWaker::new(ptr, &Self::RAW_WAKER_VTABLE))); let cx = &mut Context::from_waker(&waker); - let mut state = header.state.load(Ordering::Acquire); + let mut state = (*header).state.load(Ordering::Acquire); // Update the task's state before polling its future. loop { @@ -243,12 +238,12 @@ where drop_future::(ptr, &Self::TASK_LAYOUT); // Mark the task as unscheduled. - let state = header.state.fetch_and(!SCHEDULED, Ordering::AcqRel); + let state = (*header).state.fetch_and(!SCHEDULED, Ordering::AcqRel); // Take the awaiter out. let mut awaiter = None; if state & AWAITER != 0 { - awaiter = header.take(None); + awaiter = (*header).take(None); } // Drop the task reference. @@ -262,7 +257,7 @@ where } // Mark the task as unscheduled and running. - match header.state.compare_exchange_weak( + match (*header).state.compare_exchange_weak( state, (state & !SCHEDULED) | RUNNING, Ordering::AcqRel, @@ -280,7 +275,7 @@ where // Poll the inner future, but surround it with a guard that closes the task in case polling // panics. // If available, we should also try to catch the panic so that it is propagated correctly. - let guard = Guard::(raw.header.cast(), &Self::TASK_LAYOUT, PhantomData); + let guard = Guard::(ptr, &Self::TASK_LAYOUT, PhantomData); // Panic propagation is not available for no_std. #[cfg(not(feature = "std"))] @@ -289,7 +284,7 @@ where #[cfg(feature = "std")] let poll = { // Check if we should propagate panics. - if header.propagate_panic { + if (*header).propagate_panic { // Use catch_unwind to catch the panic. match std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| { ::poll(Pin::new_unchecked(&mut *raw.future), cx) @@ -321,7 +316,7 @@ where }; // Mark the task as not running and completed. - match header.state.compare_exchange_weak( + match (*header).state.compare_exchange_weak( state, new, Ordering::AcqRel, @@ -338,7 +333,7 @@ where // Take the awaiter out. let mut awaiter = None; if state & AWAITER != 0 { - awaiter = header.take(None); + awaiter = (*header).take(None); } // Drop the task reference. @@ -375,7 +370,7 @@ where } // Mark the task as not running. - match header.state.compare_exchange_weak( + match (*header).state.compare_exchange_weak( state, new, Ordering::AcqRel, @@ -389,7 +384,7 @@ where // Take the awaiter out. let mut awaiter = None; if state & AWAITER != 0 { - awaiter = header.take(None); + awaiter = (*header).take(None); } // Drop the task reference. From 23383f3895b5c730facb8aaf38c838fad23de0e0 Mon Sep 17 00:00:00 2001 From: james7132 Date: Fri, 12 Sep 2025 23:42:04 -0700 Subject: [PATCH 10/11] Formatting --- src/task.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/task.rs b/src/task.rs index c174b7b..1c6c4b8 100644 --- a/src/task.rs +++ b/src/task.rs @@ -241,8 +241,7 @@ fn set_detached(ptr: *const ()) -> Option> { Ok(_) => { // Read the output. output = Some( - ((*header).vtable.get_output(ptr) as *mut Result) - .read(), + ((*header).vtable.get_output(ptr) as *mut Result).read(), ); // Update the state variable because we're continuing the loop. From c2ee67a33051b53efa99436aaf916ad353150574 Mon Sep 17 00:00:00 2001 From: james7132 Date: Sat, 13 Sep 2025 01:08:38 -0700 Subject: [PATCH 11/11] Revert use of byte_add --- Cargo.toml | 2 +- master | Bin 15512 -> 0 bytes merged | Bin 10684 -> 0 bytes merged-with-shared-schedule | Bin 10684 -> 0 bytes no-box | Bin 11696 -> 0 bytes polymorphization | Bin 14202 -> 0 bytes polymorphization-shared-schedule | Bin 14428 -> 0 bytes src/lib.rs | 2 +- src/raw.rs | 47 +++++++++++++++++++++++++------ src/task.rs | 1 - 10 files changed, 40 insertions(+), 12 deletions(-) delete mode 100644 master delete mode 100644 merged delete mode 100644 merged-with-shared-schedule delete mode 100644 no-box delete mode 100644 polymorphization delete mode 100644 polymorphization-shared-schedule diff --git a/Cargo.toml b/Cargo.toml index 362baae..7779a6a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -6,7 +6,7 @@ name = "async-task" version = "4.7.1" authors = ["Stjepan Glavina "] edition = "2021" -rust-version = "1.75" +rust-version = "1.57" license = "Apache-2.0 OR MIT" repository = "https://github.com/smol-rs/async-task" description = "Task abstraction for building executors" diff --git a/master b/master deleted file mode 100644 index 0819eb6cdad7a902ab9e4678e288c822d7d5a31f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 15512 zcmds8T~8ZF6rJZv{SSVrL{y5f`7n-9B$9~qp;4+p)JMy~fv|dKH$6`-4FgMzc~F_bP5~bJv z)fDgXh~M&b{#xRopI9V%$kAXC3Fd2a5r;h-ZR1GuRf@ze)-lUnW^Cfl@pB0eC&;_) zAx54UW%h_MUR$xl_c!2=S|V#@U7WiS*7XuV%sBP+eestW3;f85hg+u6Fcx?;MWmFwu`pBt%{10 zrJp=op3iYRLF9<%kBA-_bPNnn@b^7X5L(B*OACc#B;yiCEOBO56yWC^S24gU(V0RY9d4L>-9L)G4qln3 z4Sa*5rG$cJTs}o?uYnPA+-+oUAJCag_B_86!-k8kIexfK{n`S~L;fh+%WA^v z{@Q1nJo^|~e~%+tHv?m9<+=74zVT=leL#$yc&3sS4 z5!RWQ&-&O$x(^vg$;Bg z8(`@pei`wQ_xi}N+QWDoi^RC4)-ohk1Qx+Ndqm*s>i*=f{ zUB8Z*5wuZ3s^*~utc9O}fT@q|V}G>omF)pkZI=p=j7|+gPZ(e2&5>u6n0a!%x$D=N zQY>esE8eV^A=S3(7d>J98u@x)_Lk8W>&dcTO&Ikpaw-akyZp-U^>CLH`NUAwYFcBe zMRtj;Q{A4b;IIOzqCWSS`T)eS4^FSH978OydP~fB^*TFLdk8@_E43ilwA`7$>RWGv z=zA)svFDK+|CyXI1?IvPq*Q(@sH|= zk*~AHkVJfre}+Dm)|J%bQR#@O6K)b@QAw4oiP=zhN~jKozQ&T}bq5gnq5$*zuYQ#B zcGXi`Lyve)dMY6Fv?-<@K|RD7<8;vQuKGkW7VDTC-r0-a zN8b^4d3AHw2gW(Q6dhxpP}i@noZ^YC3v;JaNLs>rbcx9>=Mw2L>lHuao~M2vtE%U+s2G=(6x<$SdLI)AnXir!H|x~uW3-%S z28Xj+i-kPocEmi~~zbReZ5^C4C*v{cXxeO<rmI9!cunFaU^N2F>!3I>AKPmm9E*mN|~4ypHchWcRKs{n`hLNThimhfO`ZXSIA0L zkFmSoE8rroE4(kJ_|a~%>P;#7G6#yMqAy`%I4PC6Xt%SAW&Xu_v+SQ`wd5U@?)n!IHJPTpTS}QSMKO)qx{8U1|5eY>yj`%6NTx@d{mu5@!pSiZK+iQsLi}p*8MKer9~|5lI^Ne&#jYG zO&i8IUVX5zSdcKA&yd*l+e&^`Rp5M*rJYaKBZL;#o4J^E^g6F2r~Wy|_?bR-skb=e zA!Z1C^)u?QJ@LmJ&V0WiQ{^N0eSsa2(}BFZ)Rq5Z%rL2@iEHY*S@Il??DAYM$z4|$ zLv7+6Mu|^C>NB~FV)dPPF2#~ zJ+tQvcsG&U!h77j2g-Yn+!K7iYJog;u2@~iytEl|ZN}1$6)&K)s2kqQ6?Ij7)XEhJMy# z#PEzd9#mN*okmoxlDfBgo$L)&?8b;!;6MJqB71Pb;W1v%fHY2MIhB(C3zpeF)h_SX Hs|NZ90zOGX diff --git a/merged b/merged deleted file mode 100644 index c202dceaa4c6edd0d771f4bfa356addf099805fa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10684 zcmd5?T~8ZV5WUZp`XBrf2~>)(&4*oCNHh`YLn9=h%A*B1K!w4`hLR}CUvGQP+?}0! z*L!!@td&-l!>)b3J9FmDnYpw1=kJc|xZmBM?!kR>3mjL!-nyB4#Oms`$L<-|yK+xh zIm1y0?+bU2U;XHQkURYB^i_SoPQH%o(`)r=%YXHbeYfX^Zs7VASMk>$xSrduUA5uP z-3i`DKyNemtLJubp8@{uxD6Z~xNThRjoZSxoo&k1``M~K*6e3%I;lz z@&-N;jdC05RX<-dl6zNM`5uh8fYlSH79Qg2^rr)!%<#=8v6Spu$k7C!@r>TS%Z|M_ z`i=S1bGx~vw11F!!)S+SmiyBM8llf=%!wkIeCei{y$e{#NQCHiVa?EW;n@a`wsGXK z9S9p}o5!|?Z^*_CFk~;Vx-T(cM7SSf_afiWJz*Vl zLs#vKC&qyCoV+gRI-_B+w2 z1>@;g)km-yt9qh`%){6wa@`OqkL|u>j^=7QTP-J4y~d7cK4nY~F`}CJ?4v!Z%tPU? zc~3n2Dpg}*bbwdL$p1>4n<7u`Rpc$7sfLDWIr6Nr>(8nwG$4!xtK+c5LRJ~8faQwg zEmXCP>zr0~;>28@p*x~h7tfjN{EkY&8ac-MGd{b;`GdQWPhRlv1^R;6ZtX#c5?SoE zK&xX_o(-gG?dEwFYQZCHor{&mV`?Sq%@kHJ-$&4It%@maP4nZh;0KjS8Df5DU!{JO zDZcl^GB{PQ!oJG-DE3g=KP})DE9*r5(#u;MO$wHVTqZX>;m%ODG;gqG! z&I`#_uT{Sl&YAId_^fD`DNSrw9ml<0hI7iUWsb)dXho-Hn+4pT)%BRZJFqTqbQMZar(A^K?zci!dKdS8l3xXZ7ayn@5R1l{#kGQ8cxJ@LaSChe-5L#yk{y? zI5K4Q$u7Ce-c~g&_NTrtBgd^T3uDm|2y7P0XxHtg0UC=3YBLOodnHUigdV`ucy>sAW#KL)LOX z#Hm?Uue16WGu8GmG4HLaSJh)%Wlsq>V_@EKzQ&m!ClS;SeqOx0?GRSIdptV{F*W5* zO)O}g&L@f%?Ix7VJWb(2i>wyCR=FO~ow75z&U(9M>ebF_t2imEZ4$tlqIJ>D^|hUa zwu@RV($!CTi0xm(P5*>+8=l<)`+Gdci=X5%=IuPNnpXC_uKo2*xzn?b>M2574^82k zX+d-D*ev%oDq)!$a;KT{rt+bx|5&aZMf*9Q=B$R^R8ME(S(@R>F3oqR+@oJ%pU2&n zf1W{H7+0%>rl_&8?m0qE{4x) z_>9BPYwFnC&UVwp@57pph|d#z{Qz%SEhl*MuT9n08=G=PZI=~g5iCCKS)(&4*oCNHh`YLn9=h%A*B1K!w4`hLR}CUvGQP+?}0! z*L!!@td&-l!>)b3J9FmDnYpw1=kJc|xZmBM?!kR>3mjL!-nyB4#Oms`$L<-|yK+xh zIm1y0?+bU2U;XHQkURYB^i_SoPQH%o(`)r=%YXHbeYfX^Zs7VASMk>$xSrduUA5uP z-3i`DKyNemtLJubp8@{uxD6Z~xNThRjoZSxoo&k1``M~K*6e3%I;lz z@&-N;jdC05RX<-dl6zNM`5uh8fYlSH79Qg2^rr)!%<#=8v6Spu$k7C!@r>TS%Z|M_ z`i=S1bGx~vw11F!!)S+SmiyBM8llf=%!wkIeCei{y$e{#NQCHiVa?EW;n@a`wsGXK z9S9p}o5!|?Z^*_CFk~;Vx-T(cM7SSf_afiWJz*Vl zLs#vKC&qyCoV+gRI-_B+w2 z1>@;g)km-yt9qh`%){6wa@`OqkL|u>j^=7QTP-J4y~d7cK4nY~F`}CJ?4v!Z%tPU? zc~3n2Dpg}*bbwdL$p1>4n<7u`Rpc$7sfLDWIr6Nr>(8nwG$4!xtK+c5LRJ~8faQwg zEmXCP>zr0~;>28@p*x~h7tfjN{EkY&8ac-MGd{b;`GdQWPhRlv1^R;6ZtX#c5?SoE zK&xX_o(-gG?dEwFYQZCHor{&mV`?Sq%@kHJ-$&4It%@maP4nZh;0KjS8Df5DU!{JO zDZcl^GB{PQ!oJG-DE3g=KP})DE9*r5(#u;MO$wHVTqZX>;m%ODG;gqG! z&I`#_uT{Sl&YAId_^fD`DNSrw9ml<0hI7iUWsb)dXho-Hn+4pT)%BRZJFqTqbQMZar(A^K?zci!dKdS8l3xXZ7ayn@5R1l{#kGQ8cxJ@LaSChe-5L#yk{y? zI5K4Q$u7Ce-c~g&_NTrtBgd^T3uDm|2y7P0XxHtg0UC=3YBLOodnHUigdV`ucy>sAW#KL)LOX z#Hm?Uue16WGu8GmG4HLaSJh)%Wlsq>V_@EKzQ&m!ClS;SeqOx0?GRSIdptV{F*W5* zO)O}g&L@f%?Ix7VJWb(2i>wyCR=FO~ow75z&U(9M>ebF_t2imEZ4$tlqIJ>D^|hUa zwu@RV($!CTi0xm(P5*>+8=l<)`+Gdci=X5%=IuPNnpXC_uKo2*xzn?b>M2574^82k zX+d-D*ev%oDq)!$a;KT{rt+bx|5&aZMf*9Q=B$R^R8ME(S(@R>F3oqR+@oJ%pU2&n zf1W{H7+0%>rl_&8?m0qE{4x) z_>9BPYwFnC&UVwp@57pph|d#z{Qz%SEhl*MuT9n08=G=PZI=~g5iCCKSB1VMhi?Kv|X z%`U~INQ*5}7=lS@DbD50nX^Ov{Z}va!Y|>Ea2w9U1CHzeUWL1GkJYQzz6l@jjc4Hz zYwvK>!|Np6;JRu6Q`*v4zG_Ir4bKK}OcS9d>t z_97gIcX%Blh8qjt`e76XVHjS@iXD8{7LLCQn>hEgjpEw}#ky^*+AUV~V5gqxsuSeq zlSFianDXp8{D{9fj-TY{y&N4EpN+#C-0c{zBfNeM2gPb;#iG$e97edeTG@wXTX;vU z+=V52R&OO&^>K}^8q3|gvXlP31&&VO~(9kM|T?tF*{?7?ZZZ`3&U7-bo-Kp{RVat-Q=0mFe~uz z00gi?V${=`?ILcphxlb>vfi87!)SYayNTNaaC`-;r$9Wd_xSatPIn~FjF{CZ`6aDU zqIg@@(LY}~Sm1+^&e{7x*m_pf52MJsdIa*w;1h}DE%^K_v5mZ;7$jq{Rv61+!`zZ3 z^(w_c&bD?uXB9p{=Cy|AiOnw;+)SI9Nwqo6Yp&YM4BnN^?5U5!RG%Vidr~{b(O%_4 zL9179-dbXYedXVYFqt=ICDxHu7yF)dw4r3$t2l2j8JOL&OMA~yQ$#V@v>*Kj#-&qTJ1Q#04qG=#b`S-u!CDaVyq3xBT*b~4fV`!QKinN#QdJDG%o7~$#d9w zD}DEK_zbpSCV^_1>SI?I&{A zGgSXfEV;oEHEtX`a;4T4ztg%WmD?7qitoKwtTVNvkt0sMu2$chN@R6J2FQt9#TaNr z*s1((IUG{a#fq$(j9vnD+6Rcjd%|M&c%<7|b%ZVWENj}pnroh?+pV^AkN~(IpP-O-j#Ul&_`4{PUQr}s5- zGL_{FcCZr0=oF^da~z&W$wA4J-6#j%8<_*Ly>{wcZ&6RY59F9M-i`b2n>%mYbw9u_ zvgcI((@)j;DPH1UAiRw1jW4W4Xi$k5{ zOb|6!!A9U)`C!1iT z=XYxT4sue87u%&Sw0*kG*-`l$-YD{cxn_{a_b&&*te%9@CO(1JnQvOv?Jdu1_v&cc z4!){GJiq_0?~!-IwrujXdE(66zOv#gnltX#%2~@5yDsj%KH^9A>HaTHoP4oTo{q-q2cNgvXB;8f7$Aq`gz*Gqrh1P1uhU;4I ziYi8hv)QUi5~I94Kcek_O2qYd7h8z`6@0tFv%d6cA9GvwfwO$eNBu;$JU6I%rq{(C zTnRSASo3aH-r=4p7ha)4IG;#$Vso_=XEMJ(?LIE91=^!BzM_f! zWp41_2dF-1$_>lwbU$XA+ga`&oW{u?chAap$2gwLQ(VriZFZpZX)4zHg4<~8KYB?R APXGV_ diff --git a/polymorphization b/polymorphization deleted file mode 100644 index 4f7e3a00c05c437b129285fda0cde9804c2844fa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 14202 zcmdU0T~8ZF6rJZv{SSVLL{y3b*w~H`5=lh*&?r?P%A@7rBt#*AOj3fX{`I!!%=Pu1 zncZ2>vTi@M9ERTV`=l$${@_YR1^wGRyZ@G@^o7U>@RbO?_ksG=_x9c7>{toxp!J}l|hC6Z3 z@jQeM?=NNRyKOx7{5~75k6pL${+`>!FWb@m1qxlagH^-DDk6|0Sv7@iZoNh)&?%2T zxQBSm@&3lYd*R=mEWX=!N7(HI&tp8Fxr4=OT1B<74J?L|@3ye3prn?z@a;WFB|5K1 zIl!v^a@D@yy&30Ic*YdCXHGF5UQXS{p{Mmo8dDaQQjwu_acveqZ0L? zzg;(4=zX-nMy(Ga%N8Uwd3sCxsHVBDhqFVu*Q58TJICV^@=akqi-8`{4&4|U>^b_$ z7LG;#Lr2Yy1H1!Bu;bU!GtHlNy=3$*S}KS4vya5{g+{0Dwb%3~H^J)#{$AnC3&iT_ z(mRNaJ2F-_{EpO~=yHtx&Fa*e#L9-pqB87yAGUQlZ05r^K=A?knE>A{{?ZrT28edB zYv_lg1)`DPiN34ennOg*F~&-xN)efdZNo!C42%(KW#sK=(&n{7%UWdRwZd2>Yrcl> zNK8CJ?7YX(499cCkH+=2x8IGor10(;R-AfW&I0{0#x*(^5A-j=&e*5jR_TxHOvXHR zWu!27H*(t7ZxOj9Pw17IHbIHx5v2aF+|9ED!7yS?ENS9vtF=HrB9^_xk7W&IqdQf)!7v6V)X%icW)V^)xC+tJR{^p ztzrg1E162xT_X$4{9cxCKKMP~`A;Q3k})QTxeHV_#3N_S;5QRQBr_;iSR@jq<{zTe z-N#F|FqFEEey(|H;3KvTtC4>x3+s~G1PNI|ZDmN9+hY&h!!g&f`cRwlvv6H~J@nS= z#%gQY?g)B%j^`>Pz!F>kz?W`l_?)Ua~G9Cz=JcPMaW` zvl0b4RZdo8(OmtIRHhkf*LZFA{}CJz1PR#yMDtTvS#e zgmTtrs=TZRs4PNjZ3FRm$;-44c|)uf)|#}J@wetsk=NFR@@ijBEX+-p$fN9|(qrZ> ze#gwmYI1_-7kqYx;|urBe{zGr?0{nAAv4CvW45S^Q0a^;x9xir+GAT5CfYFLPLSWG zU~9=Hhd8>%`#I!$3ACD*j~8p$$*-#9R%NYy6vdKg&7#>>Vu+H&oKY7;B8jcbFBX^g~AI$s1YPZzuw~-084vzI7vsS&Y5TiwMT;s$?PmxtF z(PQ#cVzX~2;9>G@oVTrOrxzQtSS!?C39_b2vzobIc#gH1j!baY*#R@B88gPxnLcyA zAsxTXB+_ibJWd_|;rp_Eyw~&p-_umbkJUYW01W3tZQs~wNCk~yqnI=#h%n~s283mwS-R2 z+p+tq-wMBP+s|x9NZXz*oKQc}?)J;w&ThgL_is&{RpM=|=ZxF_UvCTTT<25a_iZit zJ?Pt5D<9qL32a?^eaB~f3)WBlFPz!Mb(P6UebiNrY2Oh_rz}o3YMvj^S6HdUS%_Cz zwa(Ug1!o;$|I&=hdWJZg#z)m@1+N$Ab)Fd0PEh-I$D|s{)GajN(=DS~_X%ziw?rUwSevTnEul}U7JVsc~=S`}-^o$LwJSLbV zu)5~;9D3M1xuV0}suMwpu=cyP*L8RX?}E`gs;I^1!ExNHg;Nxb3ZC%_I%YRRPfHoD zyPV`jnH-0x%X}(4gYywybw7u+MExQ-mO0^O5FcFeHGHtDFYB~pmMirO(^tOI3@v#A zsd?v@Ts`MJtd3*ucITADi_OTpN)+0&%mX&OboFTq{vCAZj|C5NbV=0MM`ui8`0Hfy_^-3)VE9$B{sc9o>dWFqw0a<2Ve*{bTXklX|I_PQ?tI*W=Fj{rKfVJ` ze06;?)ZPEYJ-Y5-^NBL=Drfz)t7+?ZhAgySS+>55r&bwheG-^%Sp|sy@v0DKd93Q? z&I9T;uJ~W0wtVNWXE4Ig5J!w~-hrH+n?`-wgS6GY%?-Ye_FqBDPmu5#Bqo#1@Z_sq zdH)e4th*AD)`C?(>uM{Tk*v>FO?ByyAwl_n6)^62=kG_v)%s<#eyhE8!ZUbp5+_ucTWI^jp{bDGec0NETq&UrPtRsQ!)u5z$T;8kv|AO8c<(wOJ~ diff --git a/polymorphization-shared-schedule b/polymorphization-shared-schedule deleted file mode 100644 index 3dfff644fc26a25effec882450a9eae91c13eeac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 14428 zcmdU0T~8ZF6rJZv{SSVLMAXy@27-wY5=lh*&?r?P%A@4~2~h|j10|^HUvGQP+`Yat zvpf62ZbB+#@UCaQ_v74iKW6#&pOG86-`yv7=8oMZ-q-&;cjxW`tDDyDxodpmp}WG` zbG#ejHFsZd-OuhPe~(|C{?y-bv|Pvab!+we%768qn{LZ(xK+IBzK5T!VkLi?)~&b$ z_ZqKV*zn;}w{={<>YljAxPHaoaqOnPx)^PrDv1;V>NLJ0^n_F+u z8End<&+aLH7I=T--@Wng4vNpV-5&0CfY&}=uiQ?tnqEK33MOh z_XB7pHqdn{7guqe$SkY2{oU*3dE9jU9FM5) zi{icLqo1foBj~UOON-s9!CGO@O`IJ=C)0j@{zd#{lj&`$e-f4ohFZA zNiyyVG%`)Lpxqc}%o9YBL_+AXjV0QRV>Ft0eZU3~R@F*gB1djU;OF9wDxWh=w-a5vCWb(;s9=(Teyn9u%b6I@MmZVr~=fgz>42 zcM88eMed&?qAmMK_RtvRy){E_dF(cG^#-m8GwCHb?#SQE*zB{v=SP25*!R$#fbVGI z8~>SHaRl7V3V!Aai$q$8u}J*JQZjFZ8Img}ejPti3-RhO^Hk@!4_+dxmuRs)$oLws zL!g_-l%L2K&gR)rO2&zxj#82##E;A>A>UQ!y(d1(%;tG?D&J}DojJ;WRj^nn5i^!a zqzt^3+=+Q+QmoTzf|74~Ynw#(Ra>-+qIE=fm9ESvWB|=J)tPeBBi^#T@htD2V>qdS zShwFqNLXLmlSokh*#bJ#h}P%}E1*r!QzpwWQAON_*s!ci88y3!jCSu_WwGgXF>K_~ zTISM_p~@9RJ8#!XZ(Nv{p1fsRhKz85NLu)4xA-D!vO+sW{4?qq3C$TW)U%q2^SbUJ zbGGpoJBBoamop!`u8;xyNES*ZJ(}{eR{C4M!ZYjUGFK{9=mqVyXdzbY=g3~HjniXh z41UIZ!y0ae*H@f7!SR{<=+E5X_Z;l%_EKoyWVw&h zhIwv=>VFQlmTa<%qf5MBK)<(;RJ%zvZkRPAqIf!Rdtcb;rQr%=dd-%@~S1y0A_t%WBrlvF_BfkX5wpSd=oh=_43N z$A~TB=Y@|d9=-CFkgGeWf=(y2w5AL4BB61n3XH3iQK zpXu5eIF4KvXKL$1X?69yNEY{@dC4p$>Z-VMu72bB%X%T10}R>IvBzG|*ga;QHrnOz zZ&SJ(*qyVA89Z={--4PD3G6712V;91zvWOaU9}-5%?+$2==DE*@3W1+)f^wz^jx1F z?|B5v{Km!$dAai!O$-oOQ_*^I}p&S$|88$tSdVY-MKAa=cNp zxRK_vRy5%)naZkA6l`G4Z7>!oE15l3lTr0{$)N-r@G=##%yR8UMKlq z=epgpM$>nNpLgwy9z~MIS6?oxmTNHKaZ&YFPkQUVM<->pOZTh0ND3Ntma}SHbl3TCA75W2F7yzB!MVp6%)&~7E~#^H_UzM_HtPch;wv&RGwQfPP8+K zWts$?N`INuLb>c|b1tIe9c4J#(fH&Q?_gv0T*Q3J`{8cd{p!i? zoBQvl(3Voa365oU`6!4~uJ|4pEZfWG0e7>P@nF7^4n27Qt@(tJ?4xr=SH*UBdqPLD zD4(p7{nWl1g}80UDl1;Q>Zg>-m1a7FA|k4DR<#AC$Kez_&Z3-+^s|AnLQbN#lMy1j zy3ll+ zZ`@Tf>y1MUwX3x564B*sGMom+x~$-Y^SBerIWs#ldA^3Wjk~q?=nFpj9W%1?3HXM- z;T~RkZHmpbf9z^~>mT_%hSoqiyH-6(y`M~_We-*M?3z$Lob=xZJMZA5EBNOX{KXu4 zgctwXb0|amr%%V-#fxbdQjU_ozWNrL)OAF|YhYdOaA<4)e-mVlXJCdeV1*w`x6m4V zevngLO8XY|ralq&yf`0sW$k;D(9`)j(C~J#$~)AuU3LnuXe0m7++ng?Ub;YL;r|J7 xE+zTN{#Pfxe>yE}n_J53gda7#)Jbyy$rivk_m9Y}^51=uP1v9EDG=>k{sXFLw diff --git a/src/lib.rs b/src/lib.rs index 58b2211..2f97ae1 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -73,7 +73,7 @@ #![no_std] #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms)] #![doc(test(attr(deny(rust_2018_idioms, warnings))))] -#![doc(test(attr(allow(unused_extern_crates, unused_variables))))] +#![doc(test(attr(allow(unused_variables))))] #![doc( html_favicon_url = "https://raw.githubusercontent.com/smol-rs/smol/master/assets/images/logo_fullsize_transparent.png" )] diff --git a/src/raw.rs b/src/raw.rs index 5f66025..2994332 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -39,14 +39,13 @@ pub(crate) struct TaskVTable { /// The memory layout of the task. This information enables /// debuggers to decode raw task memory blobs. Do not remove /// the field, even if it appears to be unused. - #[allow(unused)] pub(crate) layout_info: &'static TaskLayout, } impl TaskVTable { /// Returns a pointer to the output inside a task. pub(crate) unsafe fn get_output(&self, ptr: *const ()) -> *const () { - ptr.byte_add(self.layout_info.offset_r) + ptr.add_byte(self.layout_info.offset_r) } } @@ -209,9 +208,9 @@ where unsafe { Self { header: ptr as *const HeaderWithMetadata, - schedule: ptr.byte_add(Self::TASK_LAYOUT.offset_s) as *const S, - future: ptr.byte_add(Self::TASK_LAYOUT.offset_f) as *mut F, - output: ptr.byte_add(Self::TASK_LAYOUT.offset_r) as *mut Result, + schedule: ptr.add_byte(Self::TASK_LAYOUT.offset_s) as *const S, + future: ptr.add_byte(Self::TASK_LAYOUT.offset_f) as *mut F, + output: ptr.add_byte(Self::TASK_LAYOUT.offset_r) as *mut Result, } } } @@ -502,7 +501,7 @@ where unsafe fn schedule, M>(ptr: *const (), info: ScheduleInfo) { let header = ptr as *const Header; let task_layout = (*header).vtable.layout_info; - let schedule = ptr.byte_add(task_layout.offset_s) as *mut S; + let schedule = ptr.add_byte(task_layout.offset_s) as *mut S; // If the schedule function has captured variables, create a temporary waker that prevents // the task from getting deallocated while the function is being invoked. @@ -533,7 +532,7 @@ unsafe fn drop_waker(ptr: *const ()) { /// Drops the future inside a task. #[inline] unsafe fn drop_future(ptr: *const (), task_layout: &TaskLayout) { - let future_ptr = ptr.byte_add(task_layout.offset_f) as *mut F; + let future_ptr = ptr.add_byte(task_layout.offset_f) as *mut F; // We need a safeguard against panics because the destructor can panic. abort_on_panic(|| { @@ -657,7 +656,7 @@ unsafe fn wake_by_ref, M>(ptr: *const ()) { abort(); } - let schedule = ptr.byte_add(task_layout.offset_s) as *mut S; + let schedule = ptr.add_byte(task_layout.offset_s) as *mut S; // Schedule the task. There is no need to call `Self::schedule(ptr)` // because the schedule function cannot be destroyed while the waker is @@ -682,7 +681,7 @@ unsafe fn wake_by_ref, M>(ptr: *const ()) { unsafe fn destroy(ptr: *const ()) { let header = ptr as *const Header; let task_layout = (*header).vtable.layout_info; - let schedule = ptr.byte_add(task_layout.offset_s); + let schedule = ptr.add_byte(task_layout.offset_s); // We need a safeguard against panics because destructors can panic. abort_on_panic(|| { @@ -715,3 +714,33 @@ pub(crate) unsafe fn drop_ref(ptr: *const ()) { (header.vtable.destroy)(ptr); } } + +trait PointerPolyfill { + // Polyfill for `byte_add`. + // TODO: Replace this with `byte_add` once the MSRV should be bumped past 1.75 + /// Adds an unsigned offset in bytes to a pointer. + /// + /// `count` is in units of bytes. + /// + /// This is purely a convenience for casting to a `u8` pointer and + /// using [add][pointer::add] on it. See that method for documentation + /// and safety requirements. + /// + /// # Safety + /// If any of the following conditions are violated, the result is Undefined Behavior: + /// + /// - The offset in bytes, count * size_of::(), computed on mathematical integers + /// (without “wrapping around”), must fit in an isize. + /// - If the computed offset is non-zero, then self must be derived from a pointer to + /// some allocation, and the entire memory range between self and the result must be + /// in bounds of that allocation. In particular, this range must not “wrap around” + /// the edge of the address space. + unsafe fn add_byte(self, size: usize) -> Self; +} + +impl PointerPolyfill for *const T { + #[inline] + unsafe fn add_byte(self, size: usize) -> Self { + (self.cast::().add(size)).cast::() + } +} diff --git a/src/task.rs b/src/task.rs index 1c6c4b8..d92172f 100644 --- a/src/task.rs +++ b/src/task.rs @@ -420,7 +420,6 @@ fn poll_task(ptr: *const (), cx: &mut Context<'_>) -> Poll> { // Propagate the panic if the task panicked. let output = match output { Ok(output) => output, - #[allow(unreachable_patterns)] Err(panic) => { #[cfg(feature = "std")] std::panic::resume_unwind(panic);