Solving the ABA Problem in Rust with Tagged Pointers


Introduction

Concurrent programming is challenging, especially when designing lock-free data structures. One subtle issue that can arise is the ABA problem, leading to unexpected behavior in compare-and-swap (CAS) operations. In this post, we’ll explore the ABA problem and how to solve it in Rust using idiomatic and efficient techniques.

πŸ“Ί Series Overview

This blog post is the first in a three-part series on solving the ABA problem in Rust. In this series we will cover:

  1. Part 1: Tagged Pointers with Versioning – In this post, we explain how to solve the ABA problem by pairing pointers with version numbers.
  2. Part 2: Epoch-Based Reclamation – In the next post, we’ll demonstrate a solution using epoch-based reclamation.
  3. Part 3: Hazard Pointers – Finally, we will explore a solution using hazard pointers for safe memory reclamation.

In this first post, our focus is on Tagged Pointers with Versioning.

πŸ‘¨β€πŸ”¬ What Is the ABA Problem?

The ABA problem arises when a thread reads a shared variable (value A), which is preempted while it holds that outdated value. In the meantime, another thread changes that shared variable from A to B and then back to A. Because the first thread’s CAS operation only checks for A, it succeeds, failing to detect that B ever happened. This can lead to data corruption because the underlying object at address A may no longer be the same object as before.

Here is a detailed example of the ABA problem:

Aba Problem

This is the ABA problem in action. Let’s explore solutions.

Our solution is to use version counters with the pointer so that even if the same address appears, a stale version number causes the CAS to fail.

🏷️ Tagged Pointers with Versioning

πŸ‘¨β€πŸ« How It Works

Each pointer is β€œtagged” with a version number. On every update the version is incremented, so that even if the same memory address is reused, a stale pointer (with an old version) will be detected by the CAS operation.

Note: This implementation requires Nightly Rust because it uses the integer_atomics feature.

πŸ¦€ Rust Implementation

Below is a fully functional lock-free stack implementation that leverages tagged pointers to mitigate the ABA problem. Comprehensive inline comments clarify each component of the code.

⚠️ SAFETY NOTES: ⚠️

This implementation uses unsafe code in several places for raw pointer manipulation. Here’s why each unsafe block is safe:

  1. Node Creation/Deletion:
    • Box::into_raw is used only for newly created boxes, ensuring valid pointer creation
    • Box::from_raw is only called on pointers that were created via Box::into_raw
    • The stack maintains exclusive ownership of nodes through version counting
    • No double-frees can occur due to ABA prevention via versioning
  2. Pointer Dereferencing:
    • Pointers are only dereferenced after successful CAS operations
    • Version checking ensures we never use dangling pointers
    • next pointers are only accessed while holding a reference to the current node
    • Null checks are performed before any pointer dereference
  3. MaybeUninit Usage:
    • Values are initialized immediately in push() using MaybeUninit::new()
    • assume_init() is only called in pop() after successful CAS
    • Values are never read before initialization
    • Dropped nodes are properly reconstructed into boxes
  4. Memory Ordering:
    • Acquire ordering on loads ensures visibility of node contents
    • Release ordering on stores ensures all previous writes are visible
    • Relaxed ordering is only used for operations that don’t require synchronization
    • Full memory fence on successful CAS operations maintains proper happens-before relationships
#![feature(integer_atomics)]
#![feature(test)]

//! # Lock-Free Stack with ABA Protection
//!
//! This example demonstrates a lock-free stack implementation that prevents the ABA problem
//! using tagged pointers. The implementation uses version counters to detect and prevent
//! ABA scenarios in concurrent operations.
//!
//! ## The ABA Problem
//!
//! ```text
//! Time β†’
//! Thread 1: Read A         β†’β†’β†’β†’β†’β†’β†’β†’ Try CAS(A->C) - Problem!
//!                                   (assumes A hasn't changed)
//! Thread 2:    Remove A β†’ Add B β†’ Remove B β†’ Add A
//!
//! Stack:   [A] β†’ [A,B] β†’ [B] β†’ [A]
//! ```
//!
//! The ABA problem occurs when:
//! 1. Thread 1 reads value A
//! 2. Thread 2 changes A to B, then back to A
//! 3. Thread 1 assumes A hasn't changed and proceeds with its operation
//!
//! Our solution uses version counting:
//! ```text
//! Time β†’
//! Thread 1: Read A(v1)      β†’β†’β†’β†’β†’β†’β†’β†’ CAS fails! (A has v2)
//! Thread 2:    Remove A(v1) β†’ Add B β†’ Remove B β†’ Add A(v2)
//! ```

extern crate test;

use std::mem::MaybeUninit;
use std::ptr;
use std::sync::atomic::{AtomicU128, Ordering};
use std::sync::Arc;
use std::thread;
use std::time::Duration;

/// A tagged pointer that combines a raw pointer with a version counter to prevent ABA problems.
///
/// # Structure
/// - `ptr`: Raw pointer to the node
/// - `version`: Counter that gets incremented on every modification
///
/// # ABA Prevention
/// When a pointer is updated, its version is incremented even if the same memory
/// address is being written. This ensures that if a thread sees the same pointer
/// value later, it can detect whether the pointer has been modified by checking
/// the version number.
#[derive(Debug, Clone, Copy)]
struct TaggedPtr {
    ptr: *mut Node,
    version: u64, // Version counter to detect ABA changes
}

impl TaggedPtr {
    /// Packs the pointer and version into a single u128.
    ///
    /// # Layout
    /// - Lower 64 bits: pointer value
    /// - Upper 64 bits: version counter
    ///
    /// This allows atomic operations on both the pointer and version simultaneously.
    fn pack(&self) -> u128 {
        let ptr_val = self.ptr.addr() as u64;
        (ptr_val as u128) | ((self.version as u128) << 64)
    }

    /// Unpacks a u128 into separate pointer and version components.
    ///
    /// # Returns
    /// A TaggedPtr containing:
    /// - The pointer value from the lower 64 bits
    /// - The version counter from the upper 64 bits
    fn unpack(value: u128) -> Self {
        let ptr = (value as u64) as *mut Node;
        let version = (value >> 64) as u64;
        TaggedPtr { ptr, version }
    }
}

/// A node in the lock-free stack.
///
/// # Fields
/// - `value`: The stored integer value, wrapped in MaybeUninit for safe initialization
/// - `next`: Pointer to the next node in the stack
struct Node {
    value: MaybeUninit<i32>,
    next: *mut Node,
}

/// Atomic wrapper for TaggedPtr that provides atomic operations with ABA protection.
///
/// This wrapper ensures that all operations on the tagged pointer are atomic,
/// preventing race conditions in concurrent scenarios.
struct AtomicTaggedPtr {
    inner: AtomicU128,
}

impl AtomicTaggedPtr {
    /// Creates a new AtomicTaggedPtr initialized with a null pointer and version 0.
    fn new() -> Self {
        AtomicTaggedPtr {
            inner: AtomicU128::new(
                TaggedPtr {
                    ptr: ptr::null_mut(),
                    version: 0,
                }
                .pack(),
            ),
        }
    }

    /// Atomically loads the current TaggedPtr value.
    ///
    /// # Parameters
    /// - `ordering`: The memory ordering to use for the load operation
    fn load(&self, ordering: Ordering) -> TaggedPtr {
        TaggedPtr::unpack(self.inner.load(ordering))
    }

    /// Performs an atomic compare-and-swap operation with version increment.
    ///
    /// # Parameters
    /// - `current`: The expected current value
    /// - `new_ptr`: The new pointer value to store
    /// - `success_order`: Memory ordering for successful CAS
    /// - `failure_order`: Memory ordering for failed CAS
    ///
    /// # Returns
    /// - `Ok(())` if the CAS succeeded
    /// - `Err(actual)` if the CAS failed, containing the actual value found
    fn compare_and_swap(
        &self,
        current: TaggedPtr,
        new_ptr: *mut Node,
        success_order: Ordering,
        failure_order: Ordering,
    ) -> Result<(), TaggedPtr> {
        let new = TaggedPtr {
            ptr: new_ptr,
            version: current.version.wrapping_add(1),
        };
        self.inner
            .compare_exchange(current.pack(), new.pack(), success_order, failure_order)
            .map(|_| ())
            .map_err(TaggedPtr::unpack)
    }
}

/// A lock-free stack implementation with ABA protection using tagged pointers.
///
/// # Stack Structure
/// ```text
///  HEAD
///   ↓
/// [3|v2] β†’ [2|v1] β†’ [1|v1] β†’ null
///   β”‚        β”‚        β”‚
///   └────────┴────────┴── Each node has a value and points to the next node
/// ```
///
/// # Example
/// ```
/// let stack = LockFreeStack::new();
/// stack.push(1);
/// stack.push(2);
/// assert_eq!(stack.pop(), Some(2)); // LIFO order
/// ```
pub struct LockFreeStack {
    head: AtomicTaggedPtr,
}

impl LockFreeStack {
    /// Creates a new empty lock-free stack.
    pub fn new() -> Self {
        LockFreeStack {
            head: AtomicTaggedPtr::new(),
        }
    }

    /// Pushes a new value onto the top of the stack.
    ///
    /// # Implementation Details
    /// 1. Creates a new node with the given value
    /// 2. Repeatedly tries to update the head pointer until successful:
    ///    - Reads current head
    ///    - Points new node to current head
    ///    - Attempts CAS to update head to new node
    ///
    /// # Parameters
    /// - `value`: The integer value to push onto the stack
    ///
    /// # Thread Safety
    /// This operation is lock-free and thread-safe. Multiple threads can
    /// push simultaneously without blocking each other.
    pub fn push(&self, value: i32) {
        let new_node = Box::into_raw(Box::new(Node {
            value: MaybeUninit::new(value),
            next: ptr::null_mut(),
        }));

        loop {
            let current = self.head.load(Ordering::Relaxed);
            unsafe { (*new_node).next = current.ptr };

            match self.head.compare_and_swap(
                current,
                new_node,
                Ordering::Release,
                Ordering::Relaxed,
            ) {
                Ok(_) => {
                    println!(
                        "[Thread {:?}] Successfully pushed {} (version {})",
                        thread::current().id(),
                        value,
                        current.version
                    );
                    break;
                }
                Err(new_current) => {
                    println!(
                        "[Thread {:?}] Push conflict detected! Version changed from {} to {}",
                        thread::current().id(),
                        current.version,
                        new_current.version
                    );
                    continue;
                }
            }
        }
    }

    /// Pops a value from the top of the stack.
    ///
    /// # Implementation Details
    /// 1. Repeatedly tries to update the head pointer until successful:
    ///    - Reads current head
    ///    - If null, returns None
    ///    - Otherwise, attempts CAS to update head to next node
    /// 2. If successful, returns the value from the popped node
    ///
    /// # Returns
    /// - `Some(value)` if a value was successfully popped
    /// - `None` if the stack is empty
    ///
    /// # Thread Safety
    /// This operation is lock-free and thread-safe. Multiple threads can
    /// pop simultaneously without blocking each other.
    pub fn pop(&self) -> Option<i32> {
        loop {
            let current = self.head.load(Ordering::Acquire);
            if current.ptr.is_null() {
                return None;
            }

            let next = unsafe { (*current.ptr).next };
            match self
                .head
                .compare_and_swap(current, next, Ordering::Release, Ordering::Relaxed)
            {
                Ok(_) => {
                    let node = unsafe { Box::from_raw(current.ptr) };
                    let value = unsafe { node.value.assume_init() };
                    println!(
                        "[Thread {:?}] Successfully popped {} (version {})",
                        thread::current().id(),
                        value,
                        current.version
                    );
                    return Some(value);
                }
                Err(new_current) => {
                    println!(
                        "[Thread {:?}] Pop conflict detected! Version changed from {} to {}",
                        thread::current().id(),
                        current.version,
                        new_current.version
                    );
                    continue;
                }
            }
        }
    }
}

/// Demonstrates the ABA problem and how version counting prevents it.
///
/// # Scenario
/// 1. Thread 1 reads top value (3)
/// 2. Thread 2 makes multiple modifications while Thread 1 is sleeping:
///    - Pops 3
///    - Pops 2
///    - Pushes 3 back
/// 3. Thread 1 wakes up and attempts to modify stack
///    - Will fail due to version mismatch
///
/// This shows how version counting detects that the stack was modified
/// even though the same value (3) is present.
fn _aba_example() {
    println!("\nDemonstrating ABA problem...");
    let stack = Arc::new(LockFreeStack::new());

    // Initial state: Push 1, 2, 3
    stack.push(1);
    stack.push(2);
    stack.push(3);
    println!("Initial stack state: [3] β†’ [2] β†’ [1]");

    let stack_clone1 = Arc::clone(&stack);
    let stack_clone2 = Arc::clone(&stack);

    // Thread 1: Will try to pop 3 and push it back later
    let handle1 = thread::spawn(move || {
        let current = stack_clone1.head.load(Ordering::Acquire);
        println!(
            "Thread 1: Read top value (3) with version {}",
            current.version
        );

        // Simulate some work
        thread::sleep(Duration::from_millis(200));

        println!("Thread 1: Attempting to modify stack...");
        let _ = stack_clone1.head.compare_and_swap(
            current,
            unsafe { (*current.ptr).next },
            Ordering::Release,
            Ordering::Relaxed,
        );
    });

    // Thread 2: Will perform multiple operations while Thread 1 is sleeping
    let handle2 = thread::spawn(move || {
        thread::sleep(Duration::from_millis(50));

        // Pop 3
        let val = stack_clone2.pop();
        println!("Thread 2: Popped {}", val.unwrap());

        // Pop 2
        let val = stack_clone2.pop();
        println!("Thread 2: Popped {}", val.unwrap());

        // Push 3 back
        stack_clone2.push(3);
        println!("Thread 2: Pushed 3 back");
    });

    handle1.join().unwrap();
    handle2.join().unwrap();

    println!("\nFinal stack state:");
    while let Some(val) = stack.pop() {
        println!("Value: {}", val);
    }
}

Now, let’s run the demonstration to see how version counting prevents the ABA problem.


fn main() {
    // First demonstrate the ABA problem
    _aba_example();
    println!("\n-----------------------------------\n");

    // Then run the original demo with ABA protection
    println!("Now running demo with ABA protection...");
    println!("Starting ABA protection demonstration...");
    let stack = Arc::new(LockFreeStack::new());
    let num_threads = 4;
    let operations_per_thread = 3;

    // Spawn push threads
    let push_handles: Vec<_> = (0..num_threads)
        .map(|thread_id| {
            let stack = Arc::clone(&stack);
            thread::spawn(move || {
                println!(
                    "[Thread {:?}] Started pushing operations",
                    thread::current().id()
                );

                for i in 0..operations_per_thread {
                    let value = thread_id * operations_per_thread + i;
                    println!(
                        "[Thread {:?}] Attempting to push value {}",
                        thread::current().id(),
                        value
                    );
                    stack.push(value);
                    thread::sleep(Duration::from_millis(100));
                }
            })
        })
        .collect();

    // Wait for all pushes to complete
    for handle in push_handles {
        handle.join().unwrap();
    }

    println!("\n--- All push operations completed ---\n");

    // Spawn pop threads
    let pop_handles: Vec<_> = (0..num_threads)
        .map(|_| {
            let stack = Arc::clone(&stack);
            thread::spawn(move || {
                println!(
                    "[Thread {:?}] Started popping operations",
                    thread::current().id()
                );

                for _ in 0..operations_per_thread {
                    match stack.pop() {
                        Some(_) => (),
                        None => println!("[Thread {:?}] Stack was empty", thread::current().id()),
                    }
                    thread::sleep(Duration::from_millis(50));
                }
            })
        })
        .collect();

    // Wait for all pops to complete
    for handle in pop_handles {
        handle.join().unwrap();
    }

    println!("\n--- All operations completed ---");
}

πŸ–¨οΈ Expected Output

cargo run
    Finished `dev` profile [unoptimized + debuginfo] target(s) in 0.00s
     Running `target/debug/tagged_pointer_aba_protection`

Demonstrating ABA problem...
[Thread ThreadId(1)] Successfully pushed 1 (version 0)
[Thread ThreadId(1)] Successfully pushed 2 (version 1)
[Thread ThreadId(1)] Successfully pushed 3 (version 2)
Initial stack state: [3] β†’ [2] β†’ [1]
Thread 1: Read top value (3) with version 3
[Thread ThreadId(3)] Successfully popped 3 (version 3)
Thread 2: Popped 3
[Thread ThreadId(3)] Successfully popped 2 (version 4)
Thread 2: Popped 2
[Thread ThreadId(3)] Successfully pushed 3 (version 5)
Thread 2: Pushed 3 back
Thread 1: Attempting to modify stack...

Final stack state:
[Thread ThreadId(1)] Successfully popped 3 (version 6)
Value: 3
[Thread ThreadId(1)] Successfully popped 1 (version 7)
Value: 1

-----------------------------------

Now running demo with ABA protection...
Starting ABA protection demonstration...
[Thread ThreadId(4)] Started pushing operations
[Thread ThreadId(4)] Attempting to push value 0
[Thread ThreadId(4)] Successfully pushed 0 (version 0)
[Thread ThreadId(5)] Started pushing operations
[Thread ThreadId(5)] Attempting to push value 3
[Thread ThreadId(5)] Successfully pushed 3 (version 1)
[Thread ThreadId(6)] Started pushing operations
[Thread ThreadId(6)] Attempting to push value 6
[Thread ThreadId(6)] Successfully pushed 6 (version 2)
[Thread ThreadId(7)] Started pushing operations
[Thread ThreadId(7)] Attempting to push value 9
[Thread ThreadId(7)] Successfully pushed 9 (version 3)
[Thread ThreadId(4)] Attempting to push value 1
[Thread ThreadId(4)] Successfully pushed 1 (version 4)
[Thread ThreadId(5)] Attempting to push value 4
[Thread ThreadId(5)] Successfully pushed 4 (version 5)
[Thread ThreadId(7)] Attempting to push value 10
[Thread ThreadId(7)] Successfully pushed 10 (version 6)
[Thread ThreadId(6)] Attempting to push value 7
[Thread ThreadId(6)] Successfully pushed 7 (version 7)
[Thread ThreadId(7)] Attempting to push value 11
[Thread ThreadId(7)] Successfully pushed 11 (version 8)
[Thread ThreadId(5)] Attempting to push value 5
[Thread ThreadId(5)] Successfully pushed 5 (version 9)
[Thread ThreadId(4)] Attempting to push value 2
[Thread ThreadId(6)] Attempting to push value 8
[Thread ThreadId(6)] Successfully pushed 8 (version 10)
[Thread ThreadId(4)] Successfully pushed 2 (version 11)

--- All push operations completed ---

[Thread ThreadId(8)] Started popping operations
[Thread ThreadId(8)] Successfully popped 2 (version 12)
[Thread ThreadId(9)] Started popping operations
[Thread ThreadId(9)] Successfully popped 8 (version 13)
[Thread ThreadId(10)] Started popping operations
[Thread ThreadId(10)] Successfully popped 5 (version 14)
[Thread ThreadId(11)] Started popping operations
[Thread ThreadId(11)] Successfully popped 11 (version 15)
[Thread ThreadId(9)] Pop conflict detected! Version changed from 16 to 17
[Thread ThreadId(9)] Successfully popped 1 (version 19)
[Thread ThreadId(8)] Successfully popped 7 (version 16)
[Thread ThreadId(10)] Successfully popped 10 (version 17)
[Thread ThreadId(11)] Successfully popped 4 (version 18)
[Thread ThreadId(9)] Successfully popped 9 (version 20)
[Thread ThreadId(8)] Successfully popped 6 (version 21)
[Thread ThreadId(10)] Successfully popped 3 (version 22)
[Thread ThreadId(11)] Successfully popped 0 (version 23)

--- All operations completed ---

βš™οΈ Key Mechanism

  • Versioning: Each CAS operation increments the version counter so that even if the pointer address is reused, a stale version will cause the operation to fail.
  • Atomic 128-bit Operations: Using AtomicU128, we atomically update both the pointer and its version (on x86_64, this uses the cmpxchg16b instruction).
  • Memory Safety: Functions like Box::into_raw and Box::from_raw help manage ownership, while MaybeUninit ensures safe initialization.

🧐 Pros and Cons

πŸ‘ Pros:

  • No need for deferred memory reclamation.
  • Direct control over memory reuse.
  • Minimal overhead in high-contention scenarios.

πŸ‘Ž Cons:

  • Platform-dependent (requires support for 128-bit atomics).
  • More complex implementation compared to higher-level approaches.
  • Potential (though unlikely) version counter overflow.

πŸ§ͺ Tests and Benchmarks

Tests Module

The tests module validates that the stack works correctly in single-threaded and multi-threaded scenarios. It checks that:

  • Push and pop operations behave in a LIFO manner.
  • The stack correctly reports an empty state.
  • Concurrent operations complete as expected.
  • Edge cases (e.g., pushing maximum and minimum values) are handled properly.
#[cfg(test)]
mod tests {
    use super::*;
    use std::collections::HashSet;
    use std::sync::Arc;
    use std::thread;
    use test::Bencher;

    #[test]
    fn test_push_and_pop_single_threaded() {
        let stack = LockFreeStack::new();
        stack.push(1);
        stack.push(2);
        stack.push(3);

        assert_eq!(stack.pop(), Some(3));
        assert_eq!(stack.pop(), Some(2));
        assert_eq!(stack.pop(), Some(1));
        assert_eq!(stack.pop(), None);
    }

    #[test]
    fn test_empty_stack() {
        let stack = LockFreeStack::new();
        assert_eq!(stack.pop(), None);
    }

    #[test]
    fn test_multiple_threads_push() {
        let stack = Arc::new(LockFreeStack::new());
        let thread_count = 4;
        let values_per_thread = 100;

        let handles: Vec<_> = (0..thread_count)
            .map(|thread_id| {
                let stack = Arc::clone(&stack);
                thread::spawn(move || {
                    for i in 0..values_per_thread {
                        stack.push(thread_id * values_per_thread + i);
                    }
                })
            })
            .collect();

        for handle in handles {
            handle.join().unwrap();
        }

        // Verify that we can pop all values
        let mut popped_count = 0;
        while stack.pop().is_some() {
            popped_count += 1;
        }

        assert_eq!(popped_count, thread_count * values_per_thread);
    }

    #[test]
    fn test_concurrent_push_and_pop() {
        let stack = Arc::new(LockFreeStack::new());
        let push_thread_count: usize = 3;
        let pop_thread_count: usize = 2;
        let values_per_thread: usize = 100;

        // Spawn push threads
        let push_handles: Vec<_> = (0..push_thread_count)
            .map(|thread_id| {
                let stack = Arc::clone(&stack);
                thread::spawn(move || {
                    for i in 0..values_per_thread {
                        stack.push(i32::try_from(thread_id * values_per_thread + i).unwrap());
                    }
                })
            })
            .collect();

        // Spawn pop threads
        let pop_handles: Vec<_> = (0..pop_thread_count)
            .map(|_| {
                let stack = Arc::clone(&stack);
                thread::spawn(move || {
                    let mut popped_values = HashSet::new();
                    let target_count = (values_per_thread * push_thread_count) / pop_thread_count;
                    while popped_values.len() < target_count {
                        if let Some(value) = stack.pop() {
                            popped_values.insert(value);
                        }
                        thread::yield_now();
                    }
                    popped_values
                })
            })
            .collect();

        // Wait for push threads
        for handle in push_handles {
            handle.join().unwrap();
        }

        // Collect results from pop threads
        let mut all_popped = HashSet::new();
        for handle in pop_handles {
            let thread_values = handle.join().unwrap();
            all_popped.extend(thread_values);
        }

        // Verify that all remaining values can be popped
        while let Some(value) = stack.pop() {
            all_popped.insert(value);
        }

        assert_eq!(
            all_popped.len(),
            values_per_thread * push_thread_count,
            "All pushed values should be popped exactly once"
        );
    }

    #[test]
    fn test_aba_prevention() {
        let stack = Arc::new(LockFreeStack::new());

        // Push initial values
        stack.push(1);
        stack.push(2);
        stack.push(3);

        let stack_clone = Arc::clone(&stack);

        // Thread 1: Try to pop and push back after delay
        let handle1 = thread::spawn(move || {
            // Pop the top value (3)
            let value = stack_clone.pop().unwrap();
            assert_eq!(value, 3);

            // Delay to allow other thread to modify stack
            thread::sleep(Duration::from_millis(100));

            // Push the value back
            stack_clone.push(value);
        });

        let stack_clone = Arc::clone(&stack);

        // Thread 2: Perform multiple operations while Thread 1 is delayed
        let handle2 = thread::spawn(move || {
            // Pop value (2)
            let _value2 = stack_clone.pop().unwrap();
            // Push new value
            stack_clone.push(4);
        });

        handle1.join().unwrap();
        handle2.join().unwrap();

        // The final state should reflect all operations with version tracking
        let mut values = Vec::new();
        while let Some(value) = stack.pop() {
            values.push(value);
        }

        // Values should be in LIFO order
        assert!(values.len() >= 2, "Stack should have at least 2 values");
    }

    #[test]
    fn test_integer_conversion_edge_cases() {
        let stack = LockFreeStack::new();

        // Test maximum i32 value
        stack.push(i32::MAX);
        assert_eq!(stack.pop(), Some(i32::MAX));

        // Test minimum i32 value
        stack.push(i32::MIN);
        assert_eq!(stack.pop(), Some(i32::MIN));

        // Test zero
        stack.push(0);
        assert_eq!(stack.pop(), Some(0));
    }

    #[test]
    fn test_stack_operations_visualization() {
        let stack = LockFreeStack::new();
        println!("Empty stack: null");

        stack.push(1);
        println!("After push(1): [1] β†’ null");

        stack.push(2);
        println!("After push(2): [2] β†’ [1] β†’ null");

        stack.pop();
        println!("After pop():   [1] β†’ null");

        assert_eq!(stack.pop(), Some(1));
        println!("After pop():   null");
    }
}

Benchmarks Module

The benchmarks module measures the performance of our stack operations. Let me explain these benchmark results:

  • bench_single_threaded_push_pop:
    • Time: ~186 nanoseconds per iteration
    • Explanation: This benchmark measures how long it takes to perform a push followed by a pop in a single thread. The higher time is expected because it performs both operations sequentially. The standard deviation (Β±5.62 ns) indicates that the actual time can vary by about 4 nanoseconds between runs.
  • bench_concurrent_push_pop :
    • Time: ~5 nanoseconds per iteration
    • Explanation: This benchmark measures only the pop operation while a separate thread continuously pushes values onto the stack. It’s much faster because:
      • Only the pop operation is measured.
      • The push thread ensures the stack is always populated.
      • There is less contention since the push operation happens in a separate thread.
      • The lower standard deviation (Β±1.02 ns) indicates more consistent timing between runs.

Note: The benchmarks are run using the nightly toolchain to enable the integer_atomics feature. You numbers may vary depending on your hardware and system load.

    #[bench]
    fn bench_single_threaded_push_pop(b: &mut Bencher) {
        let stack = LockFreeStack::new();
        b.iter(|| {
            stack.push(1);
            stack.pop()
        });
    }

    #[bench]
    fn bench_concurrent_push_pop(b: &mut Bencher) {
        let stack = Arc::new(LockFreeStack::new());
        let running = Arc::new(AtomicU128::new(1));
        let running_clone = Arc::clone(&running);
        let stack_clone = Arc::clone(&stack);

        let push_thread = thread::spawn(move || {
            while running_clone.load(Ordering::Relaxed) == 1 {
                stack_clone.push(1);
                thread::yield_now();
            }
        });

        b.iter(|| stack.pop());

        // Signal the push thread to stop
        running.store(0, Ordering::Relaxed);
        push_thread.join().unwrap();
    }

πŸ–¨οΈ Expected Output

 πŸš€ cargo +nightly bench
   Compiling tagged_pointer_aba_protection v0.1.0 (../tagged_pointer_aba_protection)
    Finished `bench` profile [optimized] target(s) in 0.44s
     Running unittests src/main.rs (target/release/deps/tagged_pointer_aba_protection-cfc6cdec2e76d2d6)

running 9 tests
test tests::test_aba_prevention ... ignored
test tests::test_concurrent_push_and_pop ... ignored
test tests::test_empty_stack ... ignored
test tests::test_integer_conversion_edge_cases ... ignored
test tests::test_multiple_threads_push ... ignored
test tests::test_push_and_pop_single_threaded ... ignored
test tests::test_stack_operations_visualization ... ignored
test tests::bench_concurrent_push_pop      ... bench:           5.02 ns/iter (+/- 1.02)
test tests::bench_single_threaded_push_pop ... bench:         186.58 ns/iter (+/- 5.62)

test result: ok. 0 passed; 0 failed; 7 ignored; 2 measured; 0 filtered out; finished in 8.83s

πŸ”— Resources


πŸ€” Final Thoughts

Tagged pointers with versioning effectively solve the ABA problem by attaching a version identifier to each pointer. Whenever the pointer is reused, its version tag is incremented, ensuring that any stale references to the same memory address are promptly detected. Although this method can be more complex to implement and may rely on specific hardware capabilities, it excels in performance-critical environments by minimizing synchronization overhead and safeguarding data integrity.

Stay tuned for Part 2: Epoch-Based Reclamation and Part 3: Hazard Pointers, where we’ll explore alternative approaches to safely reclaim memory in lock-free data structures.