summaryrefslogtreecommitdiffstats
path: root/tokio/src
diff options
context:
space:
mode:
authorCarl Lerche <me@carllerche.com>2020-02-27 13:45:28 -0800
committerGitHub <noreply@github.com>2020-02-27 13:45:28 -0800
commit06bcbe8dcf52a464d8e89866449397df1c794338 (patch)
tree4cbec9a7a4c451e3eb2fd8894e208e81ed30a962 /tokio/src
parentbfdfb46fcd4fca0aa9bf517d3125bfb4e011449a (diff)
sync: refactor intrusive linked list (#2279)
Allow storing the intrusive linked-list pointers in an arbitrary location in the node. This is in preparation for using the linked list in the scheduler. In order to make using the intrusive linked list more flexible, a trait is introduced to abstract mapping an entry to raw pointers and the next / prev pointers. This also pushes more unsafety onto the user.
Diffstat (limited to 'tokio/src')
-rw-r--r--tokio/src/sync/notify.rs72
-rw-r--r--tokio/src/util/linked_list.rs381
2 files changed, 256 insertions, 197 deletions
diff --git a/tokio/src/sync/notify.rs b/tokio/src/sync/notify.rs
index f3c1bda1..ef34ad2d 100644
--- a/tokio/src/sync/notify.rs
+++ b/tokio/src/sync/notify.rs
@@ -2,8 +2,11 @@ use crate::loom::sync::atomic::AtomicU8;
use crate::loom::sync::Mutex;
use crate::util::linked_list::{self, LinkedList};
+use std::cell::UnsafeCell;
use std::future::Future;
+use std::marker::PhantomPinned;
use std::pin::Pin;
+use std::ptr::NonNull;
use std::sync::atomic::Ordering::SeqCst;
use std::task::{Context, Poll, Waker};
@@ -103,11 +106,17 @@ pub struct Notify {
#[derive(Debug)]
struct Waiter {
+ /// Intrusive linked-list pointers
+ pointers: linked_list::Pointers<Waiter>,
+
/// Waiting task's waker
waker: Option<Waker>,
/// `true` if the notification has been assigned to this waiter.
notified: bool,
+
+ /// Should not be `Unpin`.
+ _p: PhantomPinned,
}
/// Future returned from `notified()`
@@ -120,9 +129,12 @@ struct Notified<'a> {
state: State,
/// Entry in the waiter `LinkedList`.
- waiter: linked_list::Entry<Waiter>,
+ waiter: UnsafeCell<Waiter>,
}
+unsafe impl<'a> Send for Notified<'a> {}
+unsafe impl<'a> Sync for Notified<'a> {}
+
#[derive(Debug)]
enum State {
Init,
@@ -189,9 +201,11 @@ impl Notify {
Notified {
notify: self,
state: State::Init,
- waiter: linked_list::Entry::new(Waiter {
+ waiter: UnsafeCell::new(Waiter {
+ pointers: linked_list::Pointers::new(),
waker: None,
notified: false,
+ _p: PhantomPinned,
}),
}
.await
@@ -292,7 +306,10 @@ fn notify_locked(waiters: &mut LinkedList<Waiter>, state: &AtomicU8, curr: u8) -
// transition **out** of `WAITING`.
//
// Get a pending waiter
- let mut waiter = waiters.pop_back().unwrap();
+ let waiter = waiters.pop_back().unwrap();
+
+ // Safety: `waiters` lock is still held.
+ let waiter = unsafe { &mut *waiter };
assert!(!waiter.notified);
@@ -319,9 +336,7 @@ fn notify_locked(waiters: &mut LinkedList<Waiter>, state: &AtomicU8, curr: u8) -
impl Notified<'_> {
/// A custom `project` implementation is used in place of `pin-project-lite`
/// as a custom drop implementation is needed.
- fn project(
- self: Pin<&mut Self>,
- ) -> (&Notify, &mut State, Pin<&mut linked_list::Entry<Waiter>>) {
+ fn project(self: Pin<&mut Self>) -> (&Notify, &mut State, &UnsafeCell<Waiter>) {
unsafe {
// Safety: both `notify` and `state` are `Unpin`.
@@ -329,11 +344,7 @@ impl Notified<'_> {
is_unpin::<AtomicU8>();
let me = self.get_unchecked_mut();
- (
- &me.notify,
- &mut me.state,
- Pin::new_unchecked(&mut me.waiter),
- )
+ (&me.notify, &mut me.state, &me.waiter)
}
}
}
@@ -344,7 +355,7 @@ impl Future for Notified<'_> {
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> {
use State::*;
- let (notify, state, mut waiter) = self.project();
+ let (notify, state, waiter) = self.project();
loop {
match *state {
@@ -408,12 +419,12 @@ impl Future for Notified<'_> {
// Safety: called while locked.
unsafe {
- (*waiter.as_mut().get()).waker = Some(cx.waker().clone());
-
- // Insert the waiter into the linked list
- waiters.push_front(waiter.as_mut());
+ (*waiter.get()).waker = Some(cx.waker().clone());
}
+ // Insert the waiter into the linked list
+ waiters.push_front(waiter.get());
+
*state = Waiting;
}
Waiting => {
@@ -425,7 +436,7 @@ impl Future for Notified<'_> {
let waiters = notify.waiters.lock().unwrap();
// Safety: called while locked
- let w = unsafe { &mut *waiter.as_mut().get() };
+ let w = unsafe { &mut *waiter.get() };
if w.notified {
// Our waker has been notified. Reset the fields and
@@ -463,7 +474,7 @@ impl Drop for Notified<'_> {
use State::*;
// Safety: The type only transitions to a "Waiting" state when pinned.
- let (notify, state, mut waiter) = unsafe { Pin::new_unchecked(self).project() };
+ let (notify, state, waiter) = unsafe { Pin::new_unchecked(self).project() };
// This is where we ensure safety. The `Notified` value is being
// dropped, which means we must ensure that the waiter entry is no
@@ -490,7 +501,7 @@ impl Drop for Notified<'_> {
//
// safety: the waiter is only added to `waiters` by virtue of it
// being the only `LinkedList` available to the type.
- unsafe { waiters.remove(waiter.as_mut()) };
+ unsafe { waiters.remove(NonNull::new_unchecked(waiter.get())) };
if waiters.is_empty() {
notify_state = EMPTY;
@@ -508,7 +519,7 @@ impl Drop for Notified<'_> {
//
// Safety: with the entry removed from the linked list, there can be
// no concurrent access to the entry
- let notified = unsafe { (*waiter.as_mut().get()).notified };
+ let notified = unsafe { (*waiter.get()).notified };
if notified {
if let Some(waker) = notify_locked(&mut waiters, &notify.state, notify_state) {
@@ -520,4 +531,25 @@ impl Drop for Notified<'_> {
}
}
+/// # Safety
+///
+/// `Waiter` is forced to be !Unpin.
+unsafe impl linked_list::Link for Waiter {
+ type Handle = *mut Waiter;
+ type Target = Waiter;
+
+ fn to_raw(handle: *mut Waiter) -> NonNull<Waiter> {
+ debug_assert!(!handle.is_null());
+ unsafe { NonNull::new_unchecked(handle) }
+ }
+
+ unsafe fn from_raw(ptr: NonNull<Waiter>) -> *mut Waiter {
+ ptr.as_ptr()
+ }
+
+ unsafe fn pointers(mut target: NonNull<Waiter>) -> NonNull<linked_list::Pointers<Waiter>> {
+ NonNull::from(&mut target.as_mut().pointers)
+ }
+}
+
fn is_unpin<T: Unpin>() {}
diff --git a/tokio/src/util/linked_list.rs b/tokio/src/util/linked_list.rs
index 2e4d8d2d..57540c4a 100644
--- a/tokio/src/util/linked_list.rs
+++ b/tokio/src/util/linked_list.rs
@@ -4,95 +4,116 @@
//! structure's APIs are `unsafe` as they require the caller to ensure the
//! specified node is actually contained by the list.
-use core::cell::UnsafeCell;
-use core::marker::PhantomPinned;
-use core::pin::Pin;
use core::ptr::NonNull;
-/// An intrusive linked list of nodes, where each node carries associated data
-/// of type `T`.
+/// An intrusive linked list.
+///
+/// Currently, the list is not emptied on drop. It is the caller's
+/// responsibility to ensure the list is empty before dropping it.
#[derive(Debug)]
-pub(crate) struct LinkedList<T> {
- head: Option<NonNull<Entry<T>>>,
- tail: Option<NonNull<Entry<T>>>,
+pub(crate) struct LinkedList<T: Link> {
+ /// Linked list head
+ head: Option<NonNull<T::Target>>,
+
+ /// Linked list tail
+ tail: Option<NonNull<T::Target>>,
}
-unsafe impl<T: Send> Send for LinkedList<T> {}
-unsafe impl<T: Sync> Sync for LinkedList<T> {}
+unsafe impl<T: Link> Send for LinkedList<T> where T::Target: Send {}
+unsafe impl<T: Link> Sync for LinkedList<T> where T::Target: Sync {}
+
+/// Defines how a type is tracked within a linked list.
+///
+/// In order to support storing a single type within multiple lists, accessing
+/// the list pointers is decoupled from the entry type.
+///
+/// # Safety
+///
+/// Implementations must guarantee that `Target` types are pinned in memory. In
+/// other words, when a node is inserted, the value will not be moved as long as
+/// it is stored in the list.
+pub(crate) unsafe trait Link {
+ /// Handle to the list entry.
+ ///
+ /// This is usually a pointer-ish type.
+ type Handle;
+
+ /// Node type
+ type Target;
+
+ /// Convert the handle to a raw pointer
+ ///
+ /// Consumes ownership of the handle.
+ fn to_raw(handle: Self::Handle) -> NonNull<Self::Target>;
+
+ /// Convert the raw pointer to a handle
+ unsafe fn from_raw(ptr: NonNull<Self::Target>) -> Self::Handle;
+
+ /// Return the pointers for a node
+ unsafe fn pointers(target: NonNull<Self::Target>) -> NonNull<Pointers<Self::Target>>;
+}
-/// A node which carries data of type `T` and is stored in an intrusive list.
+/// Previous / next pointers
#[derive(Debug)]
-pub(crate) struct Entry<T> {
+pub(crate) struct Pointers<T> {
/// The previous node in the list. null if there is no previous node.
- prev: Option<NonNull<Entry<T>>>,
+ prev: Option<NonNull<T>>,
/// The next node in the list. null if there is no previous node.
- next: Option<NonNull<Entry<T>>>,
-
- /// The data which is associated to this list item
- data: UnsafeCell<T>,
-
- /// Prevents `Entry`s from being `Unpin`. They may never be moved, since
- /// the list semantics require addresses to be stable.
- _pin: PhantomPinned,
+ next: Option<NonNull<T>>,
}
-unsafe impl<T: Send> Send for Entry<T> {}
-unsafe impl<T: Sync> Sync for Entry<T> {}
+unsafe impl<T: Send> Send for Pointers<T> {}
+unsafe impl<T: Sync> Sync for Pointers<T> {}
-impl<T> LinkedList<T> {
+// ===== impl LinkedList =====
+
+impl<T: Link> LinkedList<T> {
/// Creates an empty linked list
- pub(crate) fn new() -> Self {
+ pub(crate) fn new() -> LinkedList<T> {
LinkedList {
head: None,
tail: None,
}
}
- /// Adds an item to the back of the linked list.
- ///
- /// # Safety
- ///
- /// The function is only safe as long as valid pointers are stored inside
- /// the linked list.
- pub(crate) unsafe fn push_front(&mut self, entry: Pin<&mut Entry<T>>) {
- let mut entry: NonNull<Entry<T>> = entry.get_unchecked_mut().into();
+ /// Adds an element first in the list.
+ pub(crate) fn push_front(&mut self, val: T::Handle) {
+ let ptr = T::to_raw(val);
- entry.as_mut().next = self.head;
- entry.as_mut().prev = None;
+ unsafe {
+ T::pointers(ptr).as_mut().next = self.head;
+ T::pointers(ptr).as_mut().prev = None;
- if let Some(head) = &mut self.head {
- head.as_mut().prev = Some(entry);
- }
+ if let Some(head) = self.head {
+ T::pointers(head).as_mut().prev = Some(ptr);
+ }
- self.head = Some(entry);
+ self.head = Some(ptr);
- if self.tail.is_none() {
- self.tail = Some(entry);
+ if self.tail.is_none() {
+ self.tail = Some(ptr);
+ }
}
}
- /// Removes the first element and returns it, or `None` if the list is empty.
- ///
- /// The function is safe as the lifetime of the entry is bound to `&mut
- /// self`.
- pub(crate) fn pop_back(&mut self) -> Option<Pin<&mut T>> {
+ /// Removes the last element from a list and returns it, or None if it is
+ /// empty.
+ pub(crate) fn pop_back(&mut self) -> Option<T::Handle> {
unsafe {
- let mut last = self.tail?;
- self.tail = last.as_ref().prev;
+ let last = self.tail?;
+ self.tail = T::pointers(last).as_ref().prev;
- if let Some(mut prev) = last.as_mut().prev {
- prev.as_mut().next = None;
+ if let Some(prev) = T::pointers(last).as_ref().prev {
+ T::pointers(prev).as_mut().next = None;
} else {
self.head = None
}
- last.as_mut().prev = None;
- last.as_mut().next = None;
+ T::pointers(last).as_mut().prev = None;
+ T::pointers(last).as_mut().next = None;
- let val = &mut *last.as_mut().data.get();
-
- Some(Pin::new_unchecked(val))
+ Some(T::from_raw(last))
}
}
@@ -106,60 +127,53 @@ impl<T> LinkedList<T> {
true
}
- /// Removes the given item from the linked list.
+ /// Removes the specified node from the list
///
/// # Safety
///
- /// The caller **must** ensure that `entry` is currently contained by
- /// `self`.
- pub(crate) unsafe fn remove(&mut self, entry: Pin<&mut Entry<T>>) -> bool {
- let mut entry = NonNull::from(entry.get_unchecked_mut());
-
- if let Some(mut prev) = entry.as_mut().prev {
- debug_assert_eq!(prev.as_ref().next, Some(entry));
- prev.as_mut().next = entry.as_ref().next;
+ /// The caller **must** ensure that `node` is currently contained by
+ /// `self` or not contained by any other list.
+ pub(crate) unsafe fn remove(&mut self, node: NonNull<T::Target>) -> bool {
+ if let Some(prev) = T::pointers(node).as_ref().prev {
+ debug_assert_eq!(T::pointers(prev).as_ref().next, Some(node));
+ T::pointers(prev).as_mut().next = T::pointers(node).as_ref().next;
} else {
- if self.head != Some(entry) {
+ if self.head != Some(node) {
return false;
}
- self.head = entry.as_ref().next;
+ self.head = T::pointers(node).as_ref().next;
}
- if let Some(mut next) = entry.as_mut().next {
- debug_assert_eq!(next.as_ref().prev, Some(entry));
- next.as_mut().prev = entry.as_ref().prev;
+ if let Some(next) = T::pointers(node).as_ref().next {
+ debug_assert_eq!(T::pointers(next).as_ref().prev, Some(node));
+ T::pointers(next).as_mut().prev = T::pointers(node).as_ref().prev;
} else {
// This might be the last item in the list
- if self.tail != Some(entry) {
+ if self.tail != Some(node) {
return false;
}
- self.tail = entry.as_ref().prev;
+ self.tail = T::pointers(node).as_ref().prev;
}
- entry.as_mut().next = None;
- entry.as_mut().prev = None;
+ T::pointers(node).as_mut().next = None;
+ T::pointers(node).as_mut().prev = None;
true
}
}
-impl<T> Entry<T> {
- /// Creates a new node with the associated data
- pub(crate) fn new(data: T) -> Entry<T> {
- Entry {
+// ===== impl Pointers =====
+
+impl<T> Pointers<T> {
+ /// Create a new set of empty pointers
+ pub(crate) fn new() -> Pointers<T> {
+ Pointers {
prev: None,
next: None,
- data: UnsafeCell::new(data),
- _pin: PhantomPinned,
}
}
-
- /// Get a raw pointer to the inner data
- pub(crate) fn get(&self) -> *mut T {
- self.data.get()
- }
}
#[cfg(test)]
@@ -167,53 +181,84 @@ impl<T> Entry<T> {
mod tests {
use super::*;
- fn collect_list<T: Copy>(list: &mut LinkedList<T>) -> Vec<T> {
+ use std::pin::Pin;
+
+ struct Entry {
+ pointers: Pointers<Entry>,
+ val: i32,
+ }
+
+ unsafe impl<'a> Link for &'a Entry {
+ type Handle = Pin<&'a Entry>;
+ type Target = Entry;
+
+ fn to_raw(handle: Pin<&'_ Entry>) -> NonNull<Entry> {
+ NonNull::from(handle.get_ref())
+ }
+
+ unsafe fn from_raw(ptr: NonNull<Entry>) -> Pin<&'a Entry> {
+ Pin::new(&*ptr.as_ptr())
+ }
+
+ unsafe fn pointers(mut target: NonNull<Entry>) -> NonNull<Pointers<Entry>> {
+ NonNull::from(&mut target.as_mut().pointers)
+ }
+ }
+
+ fn entry(val: i32) -> Pin<Box<Entry>> {
+ Box::pin(Entry {
+ pointers: Pointers::new(),
+ val,
+ })
+ }
+
+ fn ptr(r: &Pin<Box<Entry>>) -> NonNull<Entry> {
+ r.as_ref().get_ref().into()
+ }
+
+ fn collect_list(list: &mut LinkedList<&'_ Entry>) -> Vec<i32> {
let mut ret = vec![];
- while let Some(v) = list.pop_back() {
- ret.push(*v);
+ while let Some(entry) = list.pop_back() {
+ ret.push(entry.val);
}
ret
}
- unsafe fn push_all(list: &mut LinkedList<i32>, entries: &mut [Pin<&mut Entry<i32>>]) {
- for entry in entries.iter_mut() {
- list.push_front(entry.as_mut());
+ fn push_all<'a>(list: &mut LinkedList<&'a Entry>, entries: &[Pin<&'a Entry>]) {
+ for entry in entries.iter() {
+ list.push_front(*entry);
}
}
macro_rules! assert_clean {
($e:ident) => {{
- assert!($e.next.is_none());
- assert!($e.prev.is_none());
+ assert!($e.pointers.next.is_none());
+ assert!($e.pointers.prev.is_none());
}};
}
macro_rules! assert_ptr_eq {
($a:expr, $b:expr) => {{
// Deal with mapping a Pin<&mut T> -> Option<NonNull<T>>
- assert_eq!(Some($a.as_mut().get_unchecked_mut().into()), $b)
+ assert_eq!(Some($a.as_ref().get_ref().into()), $b)
}};
}
#[test]
fn push_and_drain() {
- pin! {
- let a = Entry::new(5);
- let b = Entry::new(7);
- let c = Entry::new(31);
- }
+ let a = entry(5);
+ let b = entry(7);
+ let c = entry(31);
let mut list = LinkedList::new();
assert!(list.is_empty());
- unsafe {
- list.push_front(a);
- assert!(!list.is_empty());
- list.push_front(b);
- list.push_front(c);
- }
+ list.push_front(a.as_ref());
+ assert!(!list.is_empty());
+ list.push_front(b.as_ref());
+ list.push_front(c.as_ref());
let items: Vec<i32> = collect_list(&mut list);
assert_eq!([5, 7, 31].to_vec(), items);
@@ -223,27 +268,21 @@ mod tests {
#[test]
fn push_pop_push_pop() {
- pin! {
- let a = Entry::new(5);
- let b = Entry::new(7);
- }
+ let a = entry(5);
+ let b = entry(7);
- let mut list = LinkedList::new();
+ let mut list = LinkedList::<&Entry>::new();
- unsafe {
- list.push_front(a);
- }
+ list.push_front(a.as_ref());
- let v = list.pop_back().unwrap();
- assert_eq!(5, *v);
+ let entry = list.pop_back().unwrap();
+ assert_eq!(5, entry.val);
assert!(list.is_empty());
- unsafe {
- list.push_front(b);
- }
+ list.push_front(b.as_ref());
- let v = list.pop_back().unwrap();
- assert_eq!(7, *v);
+ let entry = list.pop_back().unwrap();
+ assert_eq!(7, entry.val);
assert!(list.is_empty());
assert!(list.pop_back().is_none());
@@ -251,33 +290,31 @@ mod tests {
#[test]
fn remove_by_address() {
- pin! {
- let a = Entry::new(5);
- let b = Entry::new(7);
- let c = Entry::new(31);
- }
+ let a = entry(5);
+ let b = entry(7);
+ let c = entry(31);
unsafe {
// Remove first
let mut list = LinkedList::new();
- push_all(&mut list, &mut [c.as_mut(), b.as_mut(), a.as_mut()]);
- assert!(list.remove(a.as_mut()));
+ push_all(&mut list, &[c.as_ref(), b.as_ref(), a.as_ref()]);
+ assert!(list.remove(ptr(&a)));
assert_clean!(a);
// `a` should be no longer there and can't be removed twice
- assert!(!list.remove(a.as_mut()));
+ assert!(!list.remove(ptr(&a)));
assert!(!list.is_empty());
- assert!(list.remove(b.as_mut()));
+ assert!(list.remove(ptr(&b)));
assert_clean!(b);
// `b` should be no longer there and can't be removed twice
- assert!(!list.remove(b.as_mut()));
+ assert!(!list.remove(ptr(&b)));
assert!(!list.is_empty());
- assert!(list.remove(c.as_mut()));
+ assert!(list.remove(ptr(&c)));
assert_clean!(c);
// `b` should be no longer there and can't be removed twice
- assert!(!list.remove(c.as_mut()));
+ assert!(!list.remove(ptr(&c)));
assert!(list.is_empty());
}
@@ -285,14 +322,14 @@ mod tests {
// Remove middle
let mut list = LinkedList::new();
- push_all(&mut list, &mut [c.as_mut(), b.as_mut(), a.as_mut()]);
+ push_all(&mut list, &[c.as_ref(), b.as_ref(), a.as_ref()]);
- assert!(list.remove(a.as_mut()));
+ assert!(list.remove(ptr(&a)));
assert_clean!(a);
assert_ptr_eq!(b, list.head);
- assert_ptr_eq!(c, b.next);
- assert_ptr_eq!(b, c.prev);
+ assert_ptr_eq!(c, b.pointers.next);
+ assert_ptr_eq!(b, c.pointers.prev);
let items = collect_list(&mut list);
assert_eq!([31, 7].to_vec(), items);
@@ -302,13 +339,13 @@ mod tests {
// Remove middle
let mut list = LinkedList::new();
- push_all(&mut list, &mut [c.as_mut(), b.as_mut(), a.as_mut()]);
+ push_all(&mut list, &[c.as_ref(), b.as_ref(), a.as_ref()]);
- assert!(list.remove(b.as_mut()));
+ assert!(list.remove(ptr(&b)));
assert_clean!(b);
- assert_ptr_eq!(c, a.next);
- assert_ptr_eq!(a, c.prev);
+ assert_ptr_eq!(c, a.pointers.next);
+ assert_ptr_eq!(a, c.pointers.prev);
let items = collect_list(&mut list);
assert_eq!([31, 5].to_vec(), items);
@@ -319,12 +356,12 @@ mod tests {
// Remove middle
let mut list = LinkedList::new();
- push_all(&mut list, &mut [c.as_mut(), b.as_mut(), a.as_mut()]);
+ push_all(&mut list, &[c.as_ref(), b.as_ref(), a.as_ref()]);
- assert!(list.remove(c.as_mut()));
+ assert!(list.remove(ptr(&c)));
assert_clean!(c);
- assert!(b.next.is_none());
+ assert!(b.pointers.next.is_none());
assert_ptr_eq!(b, list.tail);
let items = collect_list(&mut list);
@@ -335,20 +372,20 @@ mod tests {
// Remove first of two
let mut list = LinkedList::new();
- push_all(&mut list, &mut [b.as_mut(), a.as_mut()]);
+ push_all(&mut list, &[b.as_ref(), a.as_ref()]);
- assert!(list.remove(a.as_mut()));
+ assert!(list.remove(ptr(&a)));
assert_clean!(a);
// a should be no longer there and can't be removed twice
- assert!(!list.remove(a.as_mut()));
+ assert!(!list.remove(ptr(&a)));
assert_ptr_eq!(b, list.head);
assert_ptr_eq!(b, list.tail);
- assert!(b.next.is_none());
- assert!(b.prev.is_none());
+ assert!(b.pointers.next.is_none());
+ assert!(b.pointers.prev.is_none());
let items = collect_list(&mut list);
assert_eq!([7].to_vec(), items);
@@ -358,17 +395,17 @@ mod tests {
// Remove last of two
let mut list = LinkedList::new();
- push_all(&mut list, &mut [b.as_mut(), a.as_mut()]);
+ push_all(&mut list, &[b.as_ref(), a.as_ref()]);
- assert!(list.remove(b.as_mut()));
+ assert!(list.remove(ptr(&b)));
assert_clean!(b);
assert_ptr_eq!(a, list.head);
assert_ptr_eq!(a, list.tail);
- assert!(a.next.is_none());
- assert!(a.prev.is_none());
+ assert!(a.pointers.next.is_none());
+ assert!(a.pointers.prev.is_none());
let items = collect_list(&mut list);
assert_eq!([5].to_vec(), items);
@@ -378,9 +415,9 @@ mod tests {
// Remove last item
let mut list = LinkedList::new();
- push_all(&mut list, &mut [a.as_mut()]);
+ push_all(&mut list, &[a.as_ref()]);
- assert!(list.remove(a.as_mut()));
+ assert!(list.remove(ptr(&a)));
assert_clean!(a);
assert!(list.head.is_none());
@@ -391,12 +428,12 @@ mod tests {
unsafe {
// Remove missing
- let mut list = LinkedList::new();
+ let mut list = LinkedList::<&Entry>::new();
- list.push_front(b.as_mut());
- list.push_front(a.as_mut());
+ list.push_front(b.as_ref());
+ list.push_front(a.as_ref());
- assert!(!list.remove(c.as_mut()));
+ assert!(!list.remove(ptr(&c)));
}
}
@@ -427,23 +464,18 @@ mod tests {
})
.collect::<Vec<_>>();
- let mut next = 0;
- let mut ll = LinkedList::new();
- let mut entries = VecDeque::new();
+ let mut ll = LinkedList::<&Entry>::new();
let mut reference = VecDeque::new();
- for op in ops {
+ let entries: Vec<_> = (0..ops.len()).map(|i| entry(i as i32)).collect();
+
+ for (i, op) in ops.iter().enumerate() {
match op {
Op::Push => {
- let v = next;
- next += 1;
+ reference.push_front(i as i32);
+ assert_eq!(entries[i].val, i as i32);
- reference.push_front(v);
- entries.push_front(Box::pin(Entry::new(v)));
-
- unsafe {
- ll.push_front(entries.front_mut().unwrap().as_mut());
- }
+ ll.push_front(entries[i].as_ref());
}
Op::Pop => {
if reference.is_empty() {
@@ -452,8 +484,7 @@ mod tests {
}
let v = reference.pop_back();
- assert_eq!(v, ll.pop_back().map(|v| *v));
- entries.pop_back();
+ assert_eq!(v, ll.pop_back().map(|v| v.val));
}
Op::Remove(n) => {
if reference.is_empty() {
@@ -462,15 +493,11 @@ mod tests {
}
let idx = n % reference.len();
+ let v = reference.remove(idx).unwrap();
unsafe {
- assert!(ll.remove(entries[idx].as_mut()));
+ assert!(ll.remove(ptr(&entries[v as usize])));
}
-
- let v = reference.remove(idx).unwrap();
- assert_eq!(v, unsafe { *entries[idx].get() });
-
- entries.remove(idx);
}
}
}