aboutsummaryrefslogtreecommitdiffhomepage
path: root/pingora-lru/src
diff options
context:
space:
mode:
authorYuchen Wu <[email protected]>2024-02-27 20:25:44 -0800
committerYuchen Wu <[email protected]>2024-02-27 20:25:44 -0800
commit8797329225018c4d0ab990166dd020338ae292dc (patch)
tree1e8d0bf6f3c27e987559f52319d91ff75e4da5cb /pingora-lru/src
parent0bca116c1027a878469b72352e1e9e3916e85dde (diff)
downloadpingora-8797329225018c4d0ab990166dd020338ae292dc.tar.gz
pingora-8797329225018c4d0ab990166dd020338ae292dc.zip
Release Pingora version 0.1.0v0.1.0
Co-authored-by: Andrew Hauck <[email protected]> Co-authored-by: Edward Wang <[email protected]>
Diffstat (limited to 'pingora-lru/src')
-rw-r--r--pingora-lru/src/lib.rs661
-rw-r--r--pingora-lru/src/linked_list.rs439
2 files changed, 1100 insertions, 0 deletions
diff --git a/pingora-lru/src/lib.rs b/pingora-lru/src/lib.rs
new file mode 100644
index 0000000..a2ddf40
--- /dev/null
+++ b/pingora-lru/src/lib.rs
@@ -0,0 +1,661 @@
+// Copyright 2024 Cloudflare, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+//! An implementation of a LRU that focuses on memory efficiency, concurrency and persistence
+//!
+//! Features
+//! - keys can have different sizes
+//! - LRUs are sharded to avoid global locks.
+//! - Memory layout and usage are optimized: small and no memory fragmentation
+
+pub mod linked_list;
+
+use linked_list::{LinkedList, LinkedListIter};
+
+use hashbrown::HashMap;
+use parking_lot::RwLock;
+use std::sync::atomic::{AtomicUsize, Ordering};
+
+/// The LRU with `N` shards
+pub struct Lru<T, const N: usize> {
+ units: [RwLock<LruUnit<T>>; N],
+ weight: AtomicUsize,
+ weight_limit: usize,
+ len: AtomicUsize,
+ evicted_weight: AtomicUsize,
+ evicted_len: AtomicUsize,
+}
+
+impl<T, const N: usize> Lru<T, N> {
+ /// Create an [Lru] with the given weight limit and predicted capacity.
+ ///
+ /// The capacity is per shard (for simplicity). So the total capacity = capacity * N
+ pub fn with_capacity(weight_limit: usize, capacity: usize) -> Self {
+ // use the unsafe code from ArrayVec just to init the array
+ let mut units = arrayvec::ArrayVec::<_, N>::new();
+ for _ in 0..N {
+ units.push(RwLock::new(LruUnit::with_capacity(capacity)));
+ }
+ Lru {
+ // we did init all N elements so safe to unwrap
+ // map_err because unwrap() requires LruUnit to TODO: impl Debug
+ units: units.into_inner().map_err(|_| "").unwrap(),
+ weight: AtomicUsize::new(0),
+ weight_limit,
+ len: AtomicUsize::new(0),
+ evicted_weight: AtomicUsize::new(0),
+ evicted_len: AtomicUsize::new(0),
+ }
+ }
+
+ /// Admit the key value to the [Lru]
+ ///
+ /// Return the shard index which the asset is added to
+ pub fn admit(&self, key: u64, data: T, weight: usize) -> usize {
+ let shard = get_shard(key, N);
+ let unit = &mut self.units[shard].write();
+
+ // Make sure weight is positive otherwise eviction won't work
+ // TODO: Probably should use NonZeroUsize instead
+ let weight = if weight == 0 { 1 } else { weight };
+
+ let old_weight = unit.admit(key, data, weight);
+ if old_weight != weight {
+ self.weight.fetch_add(weight, Ordering::Relaxed);
+ if old_weight > 0 {
+ self.weight.fetch_sub(old_weight, Ordering::Relaxed);
+ } else {
+ // Assume old_weight == 0 means a new item is admitted
+ self.len.fetch_add(1, Ordering::Relaxed);
+ }
+ }
+ shard
+ }
+
+ /// Promote the key to the head of the LRU
+ ///
+ /// Return `true` if the key exist.
+ pub fn promote(&self, key: u64) -> bool {
+ self.units[get_shard(key, N)].write().access(key)
+ }
+
+ /// Promote to the top n of the LRU
+ ///
+ /// This function is a bit more efficient in terms of reducing lock contention because it
+ /// will acquire a write lock only if the key is outside top n but only acquires a read lock
+ /// when the key is already in the top n.
+ ///
+ /// Return false if the item doesn't exist
+ pub fn promote_top_n(&self, key: u64, top: usize) -> bool {
+ let unit = &self.units[get_shard(key, N)];
+ if !unit.read().need_promote(key, top) {
+ return true;
+ }
+ unit.write().access(key)
+ }
+
+ /// Evict at most one item from the given shard
+ ///
+ /// Return the evicted asset and its size if there is anything to evict
+ pub fn evict_shard(&self, shard: u64) -> Option<(T, usize)> {
+ let evicted = self.units[get_shard(shard, N)].write().evict();
+ if let Some((_, weight)) = evicted.as_ref() {
+ self.weight.fetch_sub(*weight, Ordering::Relaxed);
+ self.len.fetch_sub(1, Ordering::Relaxed);
+ self.evicted_weight.fetch_add(*weight, Ordering::Relaxed);
+ self.evicted_len.fetch_add(1, Ordering::Relaxed);
+ }
+ evicted
+ }
+
+ /// Evict the [Lru] until the overall weight is below the limit.
+ ///
+ /// Return a list of evicted items.
+ ///
+ /// The evicted items are randomly selected from all the shards.
+ pub fn evict_to_limit(&self) -> Vec<(T, usize)> {
+ let mut evicted = vec![];
+ let mut initial_weight = self.weight();
+ let mut shard_seed = rand::random(); // start from a random shard
+ let mut empty_shard = 0;
+
+ // Entries can be admitted or removed from the LRU by others during the loop below
+ // Track initial_weight not to over evict due to entries admitted after the loop starts
+ // self.weight() is also used not to over evict due to some entries are removed by others
+ while initial_weight > self.weight_limit
+ && self.weight() > self.weight_limit
+ && empty_shard < N
+ {
+ if let Some(i) = self.evict_shard(shard_seed) {
+ initial_weight -= i.1;
+ evicted.push(i)
+ } else {
+ empty_shard += 1;
+ }
+ // move on to the next shard
+ shard_seed += 1;
+ }
+ evicted
+ }
+
+ /// Remove the given asset
+ pub fn remove(&self, key: u64) -> Option<(T, usize)> {
+ let removed = self.units[get_shard(key, N)].write().remove(key);
+ if let Some((_, weight)) = removed.as_ref() {
+ self.weight.fetch_sub(*weight, Ordering::Relaxed);
+ self.len.fetch_sub(1, Ordering::Relaxed);
+ }
+ removed
+ }
+
+ /// Insert the item to the tail of this LRU
+ ///
+ /// Useful to recreate an LRU in most-to-least order
+ pub fn insert_tail(&self, key: u64, data: T, weight: usize) -> bool {
+ if self.units[get_shard(key, N)]
+ .write()
+ .insert_tail(key, data, weight)
+ {
+ self.weight.fetch_add(weight, Ordering::Relaxed);
+ self.len.fetch_add(1, Ordering::Relaxed);
+ true
+ } else {
+ false
+ }
+ }
+
+ /// Check existence of a key without changing the order in LRU
+ pub fn peek(&self, key: u64) -> bool {
+ self.units[get_shard(key, N)].read().peek(key).is_some()
+ }
+
+ /// Return the current total weight
+ pub fn weight(&self) -> usize {
+ self.weight.load(Ordering::Relaxed)
+ }
+
+ /// Return the total weight of items evicted from this [Lru].
+ pub fn evicted_weight(&self) -> usize {
+ self.evicted_weight.load(Ordering::Relaxed)
+ }
+
+ /// Return the total count of items evicted from this [Lru].
+ pub fn evicted_len(&self) -> usize {
+ self.evicted_len.load(Ordering::Relaxed)
+ }
+
+ /// The number of items inside this [Lru].
+ #[allow(clippy::len_without_is_empty)]
+ pub fn len(&self) -> usize {
+ self.len.load(Ordering::Relaxed)
+ }
+
+ /// Scan a shard with the given function F
+ pub fn iter_for_each<F>(&self, shard: usize, f: F)
+ where
+ F: FnMut((&T, usize)),
+ {
+ assert!(shard < N);
+ self.units[shard].read().iter().for_each(f);
+ }
+
+ /// Get the total number of shards
+ pub const fn shards(&self) -> usize {
+ N
+ }
+
+ /// Get the number of items inside a shard
+ pub fn shard_len(&self, shard: usize) -> usize {
+ self.units[shard].read().len()
+ }
+}
+
+#[inline]
+fn get_shard(key: u64, n_shards: usize) -> usize {
+ (key % n_shards as u64) as usize
+}
+
+struct LruNode<T> {
+ data: T,
+ list_index: usize,
+ weight: usize,
+}
+
+struct LruUnit<T> {
+ lookup_table: HashMap<u64, Box<LruNode<T>>>,
+ order: LinkedList,
+ used_weight: usize,
+}
+
+impl<T> LruUnit<T> {
+ fn with_capacity(capacity: usize) -> Self {
+ LruUnit {
+ lookup_table: HashMap::with_capacity(capacity),
+ order: LinkedList::with_capacity(capacity),
+ used_weight: 0,
+ }
+ }
+
+ pub fn peek(&self, key: u64) -> Option<&T> {
+ self.lookup_table.get(&key).map(|n| &n.data)
+ }
+
+ // admin into LRU, return old weight if there was any
+ pub fn admit(&mut self, key: u64, data: T, weight: usize) -> usize {
+ if let Some(node) = self.lookup_table.get_mut(&key) {
+ let old_weight = node.weight;
+ if weight != old_weight {
+ self.used_weight += weight;
+ self.used_weight -= old_weight;
+ node.weight = weight;
+ }
+ node.data = data;
+ self.order.promote(node.list_index);
+ return old_weight;
+ }
+ self.used_weight += weight;
+ let list_index = self.order.push_head(key);
+ let node = Box::new(LruNode {
+ data,
+ list_index,
+ weight,
+ });
+ self.lookup_table.insert(key, node);
+ 0
+ }
+
+ pub fn access(&mut self, key: u64) -> bool {
+ if let Some(node) = self.lookup_table.get(&key) {
+ self.order.promote(node.list_index);
+ true
+ } else {
+ false
+ }
+ }
+
+ // Check if a key is already in the top n most recently used nodes.
+ // this is a heuristic to reduce write, which requires exclusive locks, for promotion,
+ // especially on very populate nodes
+ // NOTE: O(n) search here so limit needs to be small
+ pub fn need_promote(&self, key: u64, limit: usize) -> bool {
+ !self.order.exist_near_head(key, limit)
+ }
+
+ // try to evict 1 node
+ pub fn evict(&mut self) -> Option<(T, usize)> {
+ self.order.pop_tail().map(|key| {
+ // unwrap is safe because we always insert in both the hashtable and the list
+ let node = self.lookup_table.remove(&key).unwrap();
+ self.used_weight -= node.weight;
+ (node.data, node.weight)
+ })
+ }
+ // TODO: scan the tail up to K elements to decide which ones to evict
+
+ pub fn remove(&mut self, key: u64) -> Option<(T, usize)> {
+ self.lookup_table.remove(&key).map(|node| {
+ let list_key = self.order.remove(node.list_index);
+ assert_eq!(key, list_key);
+ (node.data, node.weight)
+ })
+ }
+
+ pub fn insert_tail(&mut self, key: u64, data: T, weight: usize) -> bool {
+ if self.lookup_table.contains_key(&key) {
+ return false;
+ }
+ let list_index = self.order.push_tail(key);
+ let node = Box::new(LruNode {
+ data,
+ list_index,
+ weight,
+ });
+ self.lookup_table.insert(key, node);
+ true
+ }
+
+ pub fn len(&self) -> usize {
+ assert_eq!(self.lookup_table.len(), self.order.len());
+ self.lookup_table.len()
+ }
+
+ #[cfg(test)]
+ pub fn used_weight(&self) -> usize {
+ self.used_weight
+ }
+
+ pub fn iter(&self) -> LruUnitIter<'_, T> {
+ LruUnitIter {
+ unit: self,
+ iter: self.order.iter(),
+ }
+ }
+}
+
+struct LruUnitIter<'a, T> {
+ unit: &'a LruUnit<T>,
+ iter: LinkedListIter<'a>,
+}
+
+impl<'a, T> Iterator for LruUnitIter<'a, T> {
+ type Item = (&'a T, usize);
+
+ fn next(&mut self) -> Option<Self::Item> {
+ self.iter.next().map(|key| {
+ // safe because we always items in table and list are always 1:1
+ let node = self.unit.lookup_table.get(key).unwrap();
+ (&node.data, node.weight)
+ })
+ }
+
+ fn size_hint(&self) -> (usize, Option<usize>) {
+ self.iter.size_hint()
+ }
+}
+
+impl<'a, T> DoubleEndedIterator for LruUnitIter<'a, T> {
+ fn next_back(&mut self) -> Option<Self::Item> {
+ self.iter.next_back().map(|key| {
+ // safe because we always items in table and list are always 1:1
+ let node = self.unit.lookup_table.get(key).unwrap();
+ (&node.data, node.weight)
+ })
+ }
+}
+
+#[cfg(test)]
+mod test_lru {
+ use super::*;
+
+ fn assert_lru<T: Copy + PartialEq + std::fmt::Debug, const N: usize>(
+ lru: &Lru<T, N>,
+ values: &[T],
+ shard: usize,
+ ) {
+ let mut list_values = vec![];
+ lru.iter_for_each(shard, |(v, _)| list_values.push(*v));
+ assert_eq!(values, &list_values)
+ }
+
+ #[test]
+ fn test_admit() {
+ let lru = Lru::<_, 2>::with_capacity(30, 10);
+ assert_eq!(lru.len(), 0);
+
+ lru.admit(2, 2, 3);
+ assert_eq!(lru.len(), 1);
+ assert_eq!(lru.weight(), 3);
+
+ lru.admit(2, 2, 1);
+ assert_eq!(lru.len(), 1);
+ assert_eq!(lru.weight(), 1);
+
+ lru.admit(2, 2, 2); // admit again with different weight
+ assert_eq!(lru.len(), 1);
+ assert_eq!(lru.weight(), 2);
+
+ lru.admit(3, 3, 3);
+ lru.admit(4, 4, 4);
+
+ assert_eq!(lru.weight(), 2 + 3 + 4);
+ assert_eq!(lru.len(), 3);
+ }
+
+ #[test]
+ fn test_promote() {
+ let lru = Lru::<_, 2>::with_capacity(30, 10);
+
+ lru.admit(2, 2, 2);
+ lru.admit(3, 3, 3);
+ lru.admit(4, 4, 4);
+ lru.admit(5, 5, 5);
+ lru.admit(6, 6, 6);
+ assert_lru(&lru, &[6, 4, 2], 0);
+ assert_lru(&lru, &[5, 3], 1);
+
+ assert!(lru.promote(3));
+ assert_lru(&lru, &[3, 5], 1);
+ assert!(lru.promote(3));
+ assert_lru(&lru, &[3, 5], 1);
+
+ assert!(lru.promote(2));
+ assert_lru(&lru, &[2, 6, 4], 0);
+
+ assert!(!lru.promote(7)); // 7 doesn't exist
+ assert_lru(&lru, &[2, 6, 4], 0);
+ assert_lru(&lru, &[3, 5], 1);
+
+ // promote 2 to top 1, already there
+ assert!(lru.promote_top_n(2, 1));
+ assert_lru(&lru, &[2, 6, 4], 0);
+
+ // promote 4 to top 3, already there
+ assert!(lru.promote_top_n(4, 3));
+ assert_lru(&lru, &[2, 6, 4], 0);
+
+ // promote 4 to top 2
+ assert!(lru.promote_top_n(4, 2));
+ assert_lru(&lru, &[4, 2, 6], 0);
+
+ // promote 2 to top 1
+ assert!(lru.promote_top_n(2, 1));
+ assert_lru(&lru, &[2, 4, 6], 0);
+
+ assert!(!lru.promote_top_n(7, 1)); // 7 doesn't exist
+ }
+
+ #[test]
+ fn test_evict() {
+ let lru = Lru::<_, 2>::with_capacity(14, 10);
+
+ // same weight to make the random eviction less random
+ lru.admit(2, 2, 2);
+ lru.admit(3, 3, 2);
+ lru.admit(4, 4, 4);
+ lru.admit(5, 5, 4);
+ lru.admit(6, 6, 2);
+ lru.admit(7, 7, 2);
+
+ assert_lru(&lru, &[6, 4, 2], 0);
+ assert_lru(&lru, &[7, 5, 3], 1);
+
+ assert_eq!(lru.weight(), 16);
+ assert_eq!(lru.len(), 6);
+
+ let evicted = lru.evict_to_limit();
+ assert_eq!(lru.weight(), 14);
+ assert_eq!(lru.len(), 5);
+ assert_eq!(lru.evicted_weight(), 2);
+ assert_eq!(lru.evicted_len(), 1);
+ assert_eq!(evicted.len(), 1);
+ assert_eq!(evicted[0].1, 2); //weight
+ assert!(evicted[0].0 == 2 || evicted[0].0 == 3); //either 2 or 3 are evicted
+
+ let lru = Lru::<_, 2>::with_capacity(6, 10);
+
+ // same weight random eviction less random
+ lru.admit(2, 2, 2);
+ lru.admit(3, 3, 2);
+ lru.admit(4, 4, 2);
+ lru.admit(5, 5, 2);
+ lru.admit(6, 6, 2);
+ lru.admit(7, 7, 2);
+ assert_eq!(lru.weight(), 12);
+ assert_eq!(lru.len(), 6);
+
+ let evicted = lru.evict_to_limit();
+ // NOTE: there is a low chance this test would fail see the TODO in evict_to_limit
+ assert_eq!(lru.weight(), 6);
+ assert_eq!(lru.len(), 3);
+ assert_eq!(lru.evicted_weight(), 6);
+ assert_eq!(lru.evicted_len(), 3);
+ assert_eq!(evicted.len(), 3);
+ }
+
+ #[test]
+ fn test_remove() {
+ let lru = Lru::<_, 2>::with_capacity(30, 10);
+ lru.admit(2, 2, 2);
+ lru.admit(3, 3, 3);
+ lru.admit(4, 4, 4);
+ lru.admit(5, 5, 5);
+ lru.admit(6, 6, 6);
+
+ assert_eq!(lru.weight(), 2 + 3 + 4 + 5 + 6);
+ assert_eq!(lru.len(), 5);
+ assert_lru(&lru, &[6, 4, 2], 0);
+ assert_lru(&lru, &[5, 3], 1);
+
+ let node = lru.remove(6).unwrap();
+ assert_eq!(node.0, 6); // data
+ assert_eq!(node.1, 6); // weight
+ assert_eq!(lru.weight(), 2 + 3 + 4 + 5);
+ assert_eq!(lru.len(), 4);
+ assert_lru(&lru, &[4, 2], 0);
+
+ let node = lru.remove(3).unwrap();
+ assert_eq!(node.0, 3); // data
+ assert_eq!(node.1, 3); // weight
+ assert_eq!(lru.weight(), 2 + 4 + 5);
+ assert_eq!(lru.len(), 3);
+ assert_lru(&lru, &[5], 1);
+
+ assert!(lru.remove(7).is_none());
+ }
+
+ #[test]
+ fn test_peek() {
+ let lru = Lru::<_, 2>::with_capacity(30, 10);
+ lru.admit(2, 2, 2);
+ lru.admit(3, 3, 3);
+ lru.admit(4, 4, 4);
+
+ assert!(lru.peek(4));
+ assert!(lru.peek(3));
+ assert!(lru.peek(2));
+
+ assert_lru(&lru, &[4, 2], 0);
+ assert_lru(&lru, &[3], 1);
+ }
+
+ #[test]
+ fn test_insert_tail() {
+ let lru = Lru::<_, 2>::with_capacity(30, 10);
+ lru.admit(2, 2, 2);
+ lru.admit(3, 3, 3);
+ lru.admit(4, 4, 4);
+ lru.admit(5, 5, 5);
+ lru.admit(6, 6, 6);
+
+ assert_eq!(lru.weight(), 2 + 3 + 4 + 5 + 6);
+ assert_eq!(lru.len(), 5);
+ assert_lru(&lru, &[6, 4, 2], 0);
+ assert_lru(&lru, &[5, 3], 1);
+
+ assert!(lru.insert_tail(7, 7, 7));
+ assert_eq!(lru.weight(), 2 + 3 + 4 + 5 + 6 + 7);
+ assert_eq!(lru.len(), 6);
+ assert_lru(&lru, &[5, 3, 7], 1);
+
+ // ignore existing ones
+ assert!(!lru.insert_tail(6, 6, 7));
+ }
+}
+
+#[cfg(test)]
+mod test_lru_unit {
+ use super::*;
+
+ fn assert_lru<T: Copy + PartialEq + std::fmt::Debug>(lru: &LruUnit<T>, values: &[T]) {
+ let list_values: Vec<_> = lru.iter().map(|(v, _)| *v).collect();
+ assert_eq!(values, &list_values)
+ }
+
+ #[test]
+ fn test_admit() {
+ let mut lru = LruUnit::with_capacity(10);
+ assert_eq!(lru.len(), 0);
+ assert!(lru.peek(0).is_none());
+
+ lru.admit(2, 2, 1);
+ assert_eq!(lru.len(), 1);
+ assert_eq!(lru.peek(2).unwrap(), &2);
+ assert_eq!(lru.used_weight(), 1);
+
+ lru.admit(2, 2, 2); // admit again with different weight
+ assert_eq!(lru.used_weight(), 2);
+
+ lru.admit(3, 3, 3);
+ lru.admit(4, 4, 4);
+
+ assert_eq!(lru.used_weight(), 2 + 3 + 4);
+ assert_lru(&lru, &[4, 3, 2]);
+ }
+
+ #[test]
+ fn test_access() {
+ let mut lru = LruUnit::with_capacity(10);
+
+ lru.admit(2, 2, 2);
+ lru.admit(3, 3, 3);
+ lru.admit(4, 4, 4);
+ assert_lru(&lru, &[4, 3, 2]);
+
+ assert!(lru.access(3));
+ assert_lru(&lru, &[3, 4, 2]);
+ assert!(lru.access(3));
+ assert_lru(&lru, &[3, 4, 2]);
+ assert!(lru.access(2));
+ assert_lru(&lru, &[2, 3, 4]);
+
+ assert!(!lru.access(5)); // 5 doesn't exist
+ assert_lru(&lru, &[2, 3, 4]);
+
+ assert!(!lru.need_promote(2, 1));
+ assert!(lru.need_promote(3, 1));
+ assert!(!lru.need_promote(4, 9999));
+ }
+
+ #[test]
+ fn test_evict() {
+ let mut lru = LruUnit::with_capacity(10);
+
+ lru.admit(2, 2, 2);
+ lru.admit(3, 3, 3);
+ lru.admit(4, 4, 4);
+ assert_lru(&lru, &[4, 3, 2]);
+
+ assert!(lru.access(3));
+ assert!(lru.access(3));
+ assert!(lru.access(2));
+ assert_lru(&lru, &[2, 3, 4]);
+
+ assert_eq!(lru.used_weight(), 2 + 3 + 4);
+ assert_eq!(lru.evict(), Some((4, 4)));
+ assert_eq!(lru.used_weight(), 2 + 3);
+ assert_lru(&lru, &[2, 3]);
+
+ assert_eq!(lru.evict(), Some((3, 3)));
+ assert_eq!(lru.used_weight(), 2);
+ assert_lru(&lru, &[2]);
+
+ assert_eq!(lru.evict(), Some((2, 2)));
+ assert_eq!(lru.used_weight(), 0);
+ assert_lru(&lru, &[]);
+
+ assert_eq!(lru.evict(), None);
+ assert_eq!(lru.used_weight(), 0);
+ assert_lru(&lru, &[]);
+ }
+}
diff --git a/pingora-lru/src/linked_list.rs b/pingora-lru/src/linked_list.rs
new file mode 100644
index 0000000..7664aaf
--- /dev/null
+++ b/pingora-lru/src/linked_list.rs
@@ -0,0 +1,439 @@
+// Copyright 2024 Cloudflare, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Can't tell people you know Rust until you write a (doubly) linked list
+
+//! Doubly linked list
+//!
+//! Features
+//! - Preallocate consecutive memory, no memory fragmentation.
+//! - No shrink function: for Lru cache that grows to a certain size but never shrink.
+//! - Relatively fast and efficient.
+
+// inspired by clru::FixedSizeList (Élie!)
+
+use std::mem::replace;
+
+type Index = usize;
+const NULL: Index = usize::MAX;
+const HEAD: Index = 0;
+const TAIL: Index = 1;
+const OFFSET: usize = 2;
+
+#[derive(Debug)]
+struct Node {
+ pub(crate) prev: Index,
+ pub(crate) next: Index,
+ pub(crate) data: u64,
+}
+
+// Functionally the same as vec![head, tail, data_nodes...] where head & tail are fixed and
+// the rest data nodes can expand. Both head and tail can be accessed faster than using index
+struct Nodes {
+ // we use these sentinel nodes to guard the head and tail of the list so that list
+ // manipulation is simpler (fewer if-else)
+ head: Node,
+ tail: Node,
+ data_nodes: Vec<Node>,
+}
+
+impl Nodes {
+ fn with_capacity(capacity: usize) -> Self {
+ Nodes {
+ head: Node {
+ prev: NULL,
+ next: TAIL,
+ data: 0,
+ },
+ tail: Node {
+ prev: HEAD,
+ next: NULL,
+ data: 0,
+ },
+ data_nodes: Vec::with_capacity(capacity),
+ }
+ }
+
+ fn new_node(&mut self, data: u64) -> Index {
+ const VEC_EXP_GROWTH_CAP: usize = 65536;
+ let node = Node {
+ prev: NULL,
+ next: NULL,
+ data,
+ };
+ // Constrain the growth of vec: vec always double its capacity when it needs to grow
+ // It could waste too much memory when it is already very large.
+ // Here we limit the memory waste to 10% onces it grows beyond the cap.
+ // The amortized growth cost is O(n) beyond the max of the initial reserved capacity and
+ // the cap. But this list is for limited sized LRU and we recycle released node, so
+ // hopefully insertions are rare beyond certain sizes
+ if self.data_nodes.capacity() > VEC_EXP_GROWTH_CAP
+ && self.data_nodes.capacity() - self.data_nodes.len() < 2
+ {
+ self.data_nodes
+ .reserve_exact(self.data_nodes.capacity() / 10)
+ }
+ self.data_nodes.push(node);
+ self.data_nodes.len() - 1 + OFFSET
+ }
+
+ fn len(&self) -> usize {
+ self.data_nodes.len()
+ }
+
+ fn head(&self) -> &Node {
+ &self.head
+ }
+
+ fn tail(&self) -> &Node {
+ &self.tail
+ }
+}
+
+impl std::ops::Index<usize> for Nodes {
+ type Output = Node;
+
+ fn index(&self, index: usize) -> &Self::Output {
+ match index {
+ HEAD => &self.head,
+ TAIL => &self.tail,
+ _ => &self.data_nodes[index - OFFSET],
+ }
+ }
+}
+
+impl std::ops::IndexMut<usize> for Nodes {
+ fn index_mut(&mut self, index: usize) -> &mut Self::Output {
+ match index {
+ HEAD => &mut self.head,
+ TAIL => &mut self.tail,
+ _ => &mut self.data_nodes[index - OFFSET],
+ }
+ }
+}
+
+/// Doubly linked list
+pub struct LinkedList {
+ nodes: Nodes,
+ free: Vec<Index>, // to keep track of freed node to be used again
+}
+// Panic when index used as parameters are invalid
+// Index returned by push_* are always valid.
+impl LinkedList {
+ /// Create a [LinkedList] with the given predicted capacity.
+ pub fn with_capacity(capacity: usize) -> Self {
+ LinkedList {
+ nodes: Nodes::with_capacity(capacity),
+ free: vec![],
+ }
+ }
+
+ // Allocate a new node and return its index
+ // NOTE: this node is leaked if not used by caller
+ fn new_node(&mut self, data: u64) -> Index {
+ if let Some(index) = self.free.pop() {
+ // have a free node, update its payload and return its index
+ self.nodes[index].data = data;
+ index
+ } else {
+ // create a new node
+ self.nodes.new_node(data)
+ }
+ }
+
+ /// How many nodes in the list
+ #[allow(clippy::len_without_is_empty)]
+ pub fn len(&self) -> usize {
+ // exclude the 2 sentinels
+ self.nodes.len() - self.free.len()
+ }
+
+ fn valid_index(&self, index: Index) -> bool {
+ index != HEAD && index != TAIL && index < self.nodes.len() + OFFSET
+ // TODO: check node prev/next not NULL
+ // TODO: debug_check index not in self.free
+ }
+
+ fn node(&self, index: Index) -> Option<&Node> {
+ if self.valid_index(index) {
+ Some(&self.nodes[index])
+ } else {
+ None
+ }
+ }
+
+ /// Peek into the list
+ pub fn peek(&self, index: Index) -> Option<u64> {
+ self.node(index).map(|n| n.data)
+ }
+
+ // safe because index still needs to be in the range of the vec
+ fn peek_unchecked(&self, index: Index) -> &u64 {
+ &self.nodes[index].data
+ }
+
+ /// Whether the value exists closed (up to search_limit nodes) to the head of the list
+ // It can be done via iter().take().find() but this is cheaper
+ pub fn exist_near_head(&self, value: u64, search_limit: usize) -> bool {
+ let mut current_node = HEAD;
+ for _ in 0..search_limit {
+ current_node = self.nodes[current_node].next;
+ if current_node == TAIL {
+ return false;
+ }
+ if self.nodes[current_node].data == value {
+ return true;
+ }
+ }
+ false
+ }
+
+ // put a node right after the node at `at`
+ fn insert_after(&mut self, node_index: Index, at: Index) {
+ assert!(at != TAIL && at != node_index); // can't insert after tail or to itself
+
+ let next = replace(&mut self.nodes[at].next, node_index);
+
+ let node = &mut self.nodes[node_index];
+ node.next = next;
+ node.prev = at;
+
+ self.nodes[next].prev = node_index;
+ }
+
+ /// Put the data at the head of the list.
+ pub fn push_head(&mut self, data: u64) -> Index {
+ let new_node_index = self.new_node(data);
+ self.insert_after(new_node_index, HEAD);
+ new_node_index
+ }
+
+ /// Put the data at the tail of the list.
+ pub fn push_tail(&mut self, data: u64) -> Index {
+ let new_node_index = self.new_node(data);
+ self.insert_after(new_node_index, self.nodes.tail().prev);
+ new_node_index
+ }
+
+ // lift the node out of the linked list, to either delete it or insert to another place
+ // NOTE: the node is leaked if not used by the caller
+ fn lift(&mut self, index: Index) -> u64 {
+ // can't touch the sentinels
+ assert!(index != HEAD && index != TAIL);
+
+ let node = &mut self.nodes[index];
+
+ // zero out the pointers, useful in case we try to access a freed node
+ let prev = replace(&mut node.prev, NULL);
+ let next = replace(&mut node.next, NULL);
+ let data = node.data;
+
+ // make sure we are accessing a node in the list, not freed already
+ assert!(prev != NULL && next != NULL);
+
+ self.nodes[prev].next = next;
+ self.nodes[next].prev = prev;
+
+ data
+ }
+
+ /// Remove the node at the index, and return the value
+ pub fn remove(&mut self, index: Index) -> u64 {
+ self.free.push(index);
+ self.lift(index)
+ }
+
+ /// Remove the tail of the list
+ pub fn pop_tail(&mut self) -> Option<u64> {
+ let data_tail = self.nodes.tail().prev;
+ if data_tail == HEAD {
+ None // empty list
+ } else {
+ Some(self.remove(data_tail))
+ }
+ }
+
+ /// Put the node at the index to the head
+ pub fn promote(&mut self, index: Index) {
+ if self.nodes.head().next == index {
+ return; // already head
+ }
+ self.lift(index);
+ self.insert_after(index, HEAD);
+ }
+
+ fn next(&self, index: Index) -> Index {
+ self.nodes[index].next
+ }
+
+ fn prev(&self, index: Index) -> Index {
+ self.nodes[index].prev
+ }
+
+ /// Get the head of the list
+ pub fn head(&self) -> Option<Index> {
+ let data_head = self.nodes.head().next;
+ if data_head == TAIL {
+ None
+ } else {
+ Some(data_head)
+ }
+ }
+
+ /// Get the tail of the list
+ pub fn tail(&self) -> Option<Index> {
+ let data_tail = self.nodes.tail().prev;
+ if data_tail == HEAD {
+ None
+ } else {
+ Some(data_tail)
+ }
+ }
+
+ /// Iterate over the list
+ pub fn iter(&self) -> LinkedListIter<'_> {
+ LinkedListIter {
+ list: self,
+ head: HEAD,
+ tail: TAIL,
+ len: self.len(),
+ }
+ }
+}
+
+/// The iter over the list
+pub struct LinkedListIter<'a> {
+ list: &'a LinkedList,
+ head: Index,
+ tail: Index,
+ len: usize,
+}
+
+impl<'a> Iterator for LinkedListIter<'a> {
+ type Item = &'a u64;
+
+ fn next(&mut self) -> Option<Self::Item> {
+ let next_index = self.list.next(self.head);
+ if next_index == TAIL || next_index == NULL {
+ None
+ } else {
+ self.head = next_index;
+ self.len -= 1;
+ Some(self.list.peek_unchecked(next_index))
+ }
+ }
+
+ fn size_hint(&self) -> (usize, Option<usize>) {
+ (self.len, Some(self.len))
+ }
+}
+
+impl<'a> DoubleEndedIterator for LinkedListIter<'a> {
+ fn next_back(&mut self) -> Option<Self::Item> {
+ let prev_index = self.list.prev(self.tail);
+ if prev_index == HEAD || prev_index == NULL {
+ None
+ } else {
+ self.tail = prev_index;
+ self.len -= 1;
+ Some(self.list.peek_unchecked(prev_index))
+ }
+ }
+}
+
+#[cfg(test)]
+mod test {
+ use super::*;
+
+ // assert the list is the same as `values`
+ fn assert_list(list: &LinkedList, values: &[u64]) {
+ let list_values: Vec<_> = list.iter().copied().collect();
+ assert_eq!(values, &list_values)
+ }
+
+ fn assert_list_reverse(list: &LinkedList, values: &[u64]) {
+ let list_values: Vec<_> = list.iter().rev().copied().collect();
+ assert_eq!(values, &list_values)
+ }
+
+ #[test]
+ fn test_insert() {
+ let mut list = LinkedList::with_capacity(10);
+ assert_eq!(list.len(), 0);
+ assert!(list.node(2).is_none());
+ assert_eq!(list.head(), None);
+ assert_eq!(list.tail(), None);
+
+ let index1 = list.push_head(2);
+ assert_eq!(list.len(), 1);
+ assert_eq!(list.peek(index1).unwrap(), 2);
+
+ let index2 = list.push_head(3);
+ assert_eq!(list.head(), Some(index2));
+ assert_eq!(list.tail(), Some(index1));
+
+ let index3 = list.push_tail(4);
+ assert_eq!(list.head(), Some(index2));
+ assert_eq!(list.tail(), Some(index3));
+
+ assert_list(&list, &[3, 2, 4]);
+ assert_list_reverse(&list, &[4, 2, 3]);
+ }
+
+ #[test]
+ fn test_pop() {
+ let mut list = LinkedList::with_capacity(10);
+ list.push_head(2);
+ list.push_head(3);
+ list.push_tail(4);
+ assert_list(&list, &[3, 2, 4]);
+ assert_eq!(list.pop_tail(), Some(4));
+ assert_eq!(list.pop_tail(), Some(2));
+ assert_eq!(list.pop_tail(), Some(3));
+ assert_eq!(list.pop_tail(), None);
+ }
+
+ #[test]
+ fn test_promote() {
+ let mut list = LinkedList::with_capacity(10);
+ let index2 = list.push_head(2);
+ let index3 = list.push_head(3);
+ let index4 = list.push_tail(4);
+ assert_list(&list, &[3, 2, 4]);
+
+ list.promote(index3);
+ assert_list(&list, &[3, 2, 4]);
+
+ list.promote(index2);
+ assert_list(&list, &[2, 3, 4]);
+
+ list.promote(index4);
+ assert_list(&list, &[4, 2, 3]);
+ }
+
+ #[test]
+ fn test_exist_near_head() {
+ let mut list = LinkedList::with_capacity(10);
+ list.push_head(2);
+ list.push_head(3);
+ list.push_tail(4);
+ assert_list(&list, &[3, 2, 4]);
+
+ assert!(!list.exist_near_head(4, 1));
+ assert!(!list.exist_near_head(4, 2));
+ assert!(list.exist_near_head(4, 3));
+ assert!(list.exist_near_head(4, 4));
+ assert!(list.exist_near_head(4, 99999));
+ }
+}