diff --git a/Cargo.toml b/Cargo.toml index 804e09426..3e9602b19 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -14,12 +14,11 @@ salsa-macros = { version = "0.22.0", path = "components/salsa-macros", optional boxcar = { version = "0.2.12" } crossbeam-queue = "0.3.11" -dashmap = { version = "6", features = ["raw-api"] } -# the version of hashbrown used by dashmap -hashbrown_14 = { version = "0.14", package = "hashbrown" } +crossbeam-utils = "0.8.21" hashbrown = "0.15" hashlink = "0.10" indexmap = "2" +intrusive-collections = "0.9.7" parking_lot = "0.12" portable-atomic = "1" rustc-hash = "2" @@ -52,6 +51,7 @@ salsa-macros = { version = "=0.22.0", path = "components/salsa-macros" } [dev-dependencies] # examples crossbeam-channel = "0.5.14" +dashmap = { version = "6", features = ["raw-api"] } eyre = "0.6.8" notify-debouncer-mini = "0.4.1" ordered-float = "4.2.1" diff --git a/src/durability.rs b/src/durability.rs index 26fdf5320..2f2b384d8 100644 --- a/src/durability.rs +++ b/src/durability.rs @@ -87,14 +87,6 @@ impl Durability { pub(crate) fn index(self) -> usize { self.0 as usize } - - pub(crate) fn as_u8(self) -> u8 { - self.0 as u8 - } - - pub(crate) fn from_u8(value: u8) -> Self { - Self(DurabilityVal::from(value)) - } } impl Default for Durability { diff --git a/src/event.rs b/src/event.rs index 284164d11..e046eec95 100644 --- a/src/event.rs +++ b/src/event.rs @@ -107,8 +107,17 @@ pub enum EventKind { revision: Revision, }, + /// Indicates that a value was interned by reusing an existing slot. + DidReuseInternedValue { + // The key of the interned value. + key: DatabaseKeyIndex, + + // The revision the value was interned in. + revision: Revision, + }, + /// Indicates that a previously interned value was read in a new revision. - DidReinternValue { + DidValidateInternedValue { // The key of the interned value. key: DatabaseKeyIndex, diff --git a/src/function/memo.rs b/src/function/memo.rs index 9b8e2bb85..9057a5c29 100644 --- a/src/function/memo.rs +++ b/src/function/memo.rs @@ -99,6 +99,7 @@ pub struct Memo { // Memo's are stored a lot, make sure their size is doesn't randomly increase. #[cfg(not(feature = "shuttle"))] +#[cfg(target_pointer_width = "64")] const _: [(); std::mem::size_of::>()] = [(); std::mem::size_of::<[usize; 13]>()]; diff --git a/src/interned.rs b/src/interned.rs index 98525f59c..72f790dde 100644 --- a/src/interned.rs +++ b/src/interned.rs @@ -1,29 +1,31 @@ -#![allow(clippy::undocumented_unsafe_blocks)] // TODO(#697) document safety - use std::any::TypeId; -use std::cell::Cell; +use std::cell::{Cell, UnsafeCell}; use std::fmt; use std::hash::{BuildHasher, Hash, Hasher}; use std::marker::PhantomData; use std::path::{Path, PathBuf}; -use dashmap::SharedValue; +use crossbeam_utils::CachePadded; +use intrusive_collections::{intrusive_adapter, LinkedList, LinkedListLink, UnsafeRef}; +use rustc_hash::FxBuildHasher; use crate::cycle::CycleHeads; use crate::durability::Durability; use crate::function::VerifyResult; use crate::id::{AsId, FromId}; use crate::ingredient::Ingredient; -use crate::plumbing::{IngredientIndices, Jar}; +use crate::plumbing::{IngredientIndices, Jar, ZalsaLocal}; use crate::revision::AtomicRevision; -use crate::sync::atomic::{AtomicU8, Ordering}; -use crate::sync::Arc; -use crate::sync::FxDashMap; -use crate::table::memo::{MemoTable, MemoTableTypes}; +use crate::sync::{Arc, Mutex, OnceLock}; +use crate::table::memo::{MemoTable, MemoTableTypes, MemoTableWithTypesMut}; use crate::table::Slot; use crate::zalsa::{IngredientIndex, Zalsa}; use crate::{Database, DatabaseKeyIndex, Event, EventKind, Id, Revision}; +/// Trait that defines the key properties of an interned struct. +/// +/// Implemented by the `#[salsa::interned]` macro when applied to +/// a struct. pub trait Configuration: Sized + 'static { const LOCATION: crate::ingredient::Location; @@ -43,60 +45,136 @@ pub struct JarImpl { phantom: PhantomData, } -/// The interned ingredient hashes values of type `Data` to produce an `Id`. +/// The interned ingredient hashes values of type `C::Fields` to produce an `Id`. /// -/// It used to store interned structs but also to store the id fields of a tracked struct. -/// Interned values endure until they are explicitly removed in some way. +/// It used to store interned structs but also to store the ID fields of a tracked struct. +/// Interned values are garbage collected and their memory reused based on an LRU heuristic. pub struct IngredientImpl { - /// Index of this ingredient in the database (used to construct database-ids, etc). + /// Index of this ingredient in the database (used to construct database-IDs, etc). ingredient_index: IngredientIndex, - /// Maps from data to the existing interned id for that data. - /// - /// This doesn't hold the fields themselves to save memory, instead it points to the slot ID. - /// - /// Deadlock requirement: We access `value_map` while holding lock on `key_map`, but not vice versa. - key_map: FxDashMap, + /// A hasher for the sharded ID maps. + hasher: FxBuildHasher, + + /// A shift used to determine the shard for a given hash. + shift: u32, + + /// Sharded data that can only be accessed through a lock. + shards: Box<[CachePadded>>]>, + + /// A queue of recent revisions in which values were interned. + revision_queue: RevisionQueue, memo_table_types: Arc, _marker: PhantomData C>, } +struct IngredientShard { + /// Maps from data to the existing interned ID for that data. + /// + /// This doesn't hold the fields themselves to save memory, instead it points + /// to the slot ID. + key_map: hashbrown::HashTable, + + /// An intrusive linked list for LRU. + lru: LinkedList>, +} + +impl Default for IngredientShard { + fn default() -> Self { + Self { + lru: LinkedList::default(), + key_map: hashbrown::HashTable::new(), + } + } +} + +// SAFETY: `LinkedListLink` is `!Sync`, however, the linked list is only accessed through the +// ingredient lock, and values are only ever linked to a single list on the ingredient. +unsafe impl Sync for Value {} + +intrusive_adapter!(ValueAdapter = UnsafeRef>: Value { link: LinkedListLink } where C: Configuration); + /// Struct storing the interned fields. pub struct Value where C: Configuration, { - fields: C::Fields<'static>, - memos: MemoTable, + /// The index of the shard containing this value. + shard: u16, - /// The revision the value was first interned in. - first_interned_at: Revision, + /// An intrusive linked list for LRU. + link: LinkedListLink, - /// The most recent interned revision. - last_interned_at: AtomicRevision, + /// The interned fields for this value. + /// + /// These are valid for read-only access as long as the lock is held + /// or the value has been validated in the current revision. + fields: UnsafeCell>, + + /// Memos attached to this interned value. + /// + /// This is valid for read-only access as long as the lock is held + /// or the value has been validated in the current revision. + memos: UnsafeCell, + + /// Data that can only be accessed while holding the lock for the + /// `key_map` shard containing the value ID. + shared: UnsafeCell, +} + +/// Shared value data can only be read through the lock. +struct ValueShared { + /// The interned ID for this value. + /// + /// Storing this on the value itself is necessary to identify slots + /// from the LRU list, as well as keep track of the generation. + /// + /// Values that are reused increment the ID generation, as if they had + /// allocated a new slot. This eliminates the need for dependency edges + /// on queries that *read* from an interned value, as any memos dependent + /// on the previous value will not match the new ID. + /// + /// However, reusing a slot invalidates the previous ID, so dependency edges + /// on queries that *create* an interned value are still required to ensure + /// the value is re-interned with a new ID. + id: Id, + + /// The revision the value was most-recently interned in. + last_interned_at: Revision, /// The minimum durability of all inputs consumed by the creator - /// query prior to creating this tracked struct. If any of those + /// query prior to creating this interned struct. If any of those /// inputs changes, then the creator query may create this struct /// with different values. - durability: AtomicU8, + durability: Durability, +} + +impl ValueShared { + /// Returns `true` if this value slot can be reused when interning, and should be added to the LRU. + fn is_reusable(&self) -> bool { + // Collecting higher durability values requires invalidating the revision for their + // durability (see `Database::synthetic_write`, which requires a mutable reference to + // the database) to avoid short-circuiting calls to `maybe_changed_after`. This is + // necessary because `maybe_changed_after` for interned values is not "pure"; it updates + // the `last_interned_at` field before validating a given value to ensure that it is not + // reused after read in the current revision. + self.durability == Durability::LOW + } } impl Value where C: Configuration, { - // Loads the durability of this interned struct. - fn durability(&self) -> Durability { - Durability::from_u8(self.durability.load(Ordering::Acquire)) - } - /// Fields of this interned struct. #[cfg(feature = "salsa_unstable")] pub fn fields(&self) -> &C::Fields<'static> { - &self.fields + // SAFETY: The fact that this function is safe is technically unsound. However, interned + // values are only exposed if they have been validated in the current revision, which + // ensures that they are not reused while being accessed. + unsafe { &*self.fields.get() } } } @@ -127,19 +205,47 @@ where C: Configuration, { pub fn new(ingredient_index: IngredientIndex) -> Self { + static SHARDS: OnceLock = OnceLock::new(); + let shards = *SHARDS.get_or_init(|| { + let num_cpus = std::thread::available_parallelism() + .map(usize::from) + .unwrap_or(1); + + (num_cpus * 4).next_power_of_two() + }); + Self { ingredient_index, - key_map: Default::default(), + hasher: FxBuildHasher, memo_table_types: Arc::new(MemoTableTypes::default()), + revision_queue: RevisionQueue::default(), + shift: usize::BITS - shards.trailing_zeros(), + shards: (0..shards).map(|_| Default::default()).collect(), _marker: PhantomData, } } + /// Returns the shard for a given hash. + /// + /// Note that this value is guaranteed to be in-bounds for `self.shards`. + #[inline] + fn shard(&self, hash: u64) -> usize { + // https://github.com/xacrimon/dashmap/blob/366ce7e7872866a06de66eb95002fa6cf2c117a7/src/lib.rs#L421 + ((hash as usize) << 7) >> self.shift + } + + /// # Safety + /// + /// The `from_internal_data` function must be called to restore the correct lifetime + /// before access. unsafe fn to_internal_data<'db>(&'db self, data: C::Fields<'db>) -> C::Fields<'static> { + // SAFETY: Guaranteed by caller. unsafe { std::mem::transmute(data) } } - unsafe fn from_internal_data<'db>(data: &'db C::Fields<'static>) -> &'db C::Fields<'db> { + fn from_internal_data<'db>(data: &'db C::Fields<'static>) -> &'db C::Fields<'db> { + // SAFETY: It's sound to go from `Data<'static>` to `Data<'db>`. We shrink the + // lifetime here to use a single lifetime in `Lookup::eq(&StructKey<'db>, &C::Data<'db>)` unsafe { std::mem::transmute(data) } } @@ -147,6 +253,7 @@ where /// /// If `key` is already interned, returns the existing [`Id`] for the interned data without /// invoking `assemble`. + /// /// Otherwise, invokes `assemble` with the given `key` and the [`Id`] to be allocated for this /// interned value. The resulting [`C::Data`] will then be interned. /// @@ -169,6 +276,7 @@ where /// /// If `key` is already interned, returns the existing [`Id`] for the interned data without /// invoking `assemble`. + /// /// Otherwise, invokes `assemble` with the given `key` and the [`Id`] to be allocated for this /// interned value. The resulting [`C::Data`] will then be interned. /// @@ -189,176 +297,421 @@ where C::Fields<'db>: HashEqLike, { let (zalsa, zalsa_local) = db.zalsas(); + + // Record the current revision as active. let current_revision = zalsa.current_revision(); - let table = zalsa.table(); + self.revision_queue.record(current_revision); + + // Hash the value before acquiring the lock. + let hash = self.hasher.hash_one(&key); + + let shard_index = self.shard(hash); + // SAFETY: `shard_index` is guaranteed to be in-bounds for `self.shards`. + let shard = unsafe { &mut *self.shards.get_unchecked(shard_index).lock() }; - // Optimization to only get read lock on the map if the data has already been interned. - let data_hash = self.key_map.hasher().hash_one(&key); - let shard = &self.key_map.shards()[self.key_map.determine_shard(data_hash as _)]; let found_value = Cell::new(None); - let eq = |(id, _): &_| { - let data = table.get::>(*id); - found_value.set(Some(data)); - // SAFETY: it's safe to go from Data<'static> to Data<'db> - // shrink lifetime here to use a single lifetime in Lookup::eq(&StructKey<'db>, &C::Data<'db>) - let data = unsafe { - std::mem::transmute::<&C::Fields<'static>, &C::Fields<'db>>(&data.fields) - }; - HashEqLike::eq(data, &key) - }; + // SAFETY: We hold the lock for the shard containing the value. + let eq = |id: &_| unsafe { Self::value_eq(*id, &key, zalsa, &found_value) }; - { - let lock = shard.read(); - if let Some(bucket) = lock.find(data_hash, eq) { - // SAFETY: Read lock on map is held during this block - let id = unsafe { bucket.as_ref().0 }; - - let value = found_value - .get() - .expect("found the interned, so `found_value` should be set"); - - let index = self.database_key_index(id); - - // Sync the value's revision. - if value.last_interned_at.load() < current_revision { - value.last_interned_at.store(current_revision); - zalsa.event(&|| { - Event::new(EventKind::DidReinternValue { - key: index, - revision: current_revision, - }) - }); - } + // Attempt a fast-path lookup of already interned data. + if let Some(&id) = shard.key_map.find(hash, eq) { + let value = found_value + .get() + .expect("found the interned value, so `found_value` should be set"); - let durability = if let Some((_, stamp)) = zalsa_local.active_query() { - // Record the maximum durability across all queries that intern this value. - let previous_durability = value - .durability - .fetch_max(stamp.durability.as_u8(), Ordering::AcqRel); + let index = self.database_key_index(id); - Durability::from_u8(previous_durability).max(stamp.durability) - } else { - value.durability() - }; + // SAFETY: We hold the lock for the shard containing the value. + let value_shared = unsafe { &mut *value.shared.get() }; - // Record a dependency on this value. - zalsa_local.report_tracked_read_simple(index, durability, value.first_interned_at); + // Validate the value in this revision to avoid reuse. + if value_shared.last_interned_at < current_revision { + value_shared.last_interned_at = current_revision; - return id; + zalsa.event(&|| { + Event::new(EventKind::DidValidateInternedValue { + key: index, + revision: current_revision, + }) + }); + + if value_shared.is_reusable() { + // Move the value to the front of the LRU list. + // + // SAFETY: We hold the lock for the shard containing the value, and `value` is + // a reusable value that was previously interned, so is in the list. + unsafe { shard.lru.cursor_mut_from_ptr(value).remove() }; + + // SAFETY: The value pointer is valid for the lifetime of the database + // and never accessed mutably directly. + unsafe { shard.lru.push_front(UnsafeRef::from_raw(value)) }; + } + } + + // Record the maximum durability across all queries that intern this value. + if let Some((_, stamp)) = zalsa_local.active_query() { + value_shared.durability = std::cmp::max(value_shared.durability, stamp.durability); } + + // Record a dependency on the value. + // + // Note that we can use `Revision::start()` here as the ID with the given generation + // is unique to this use of the interned slot. + zalsa_local.report_tracked_read_simple( + index, + value_shared.durability, + Revision::start(), + ); + + return value_shared.id; } - let mut lock = shard.write(); - match lock.find_or_find_insert_slot(data_hash, eq, |(id, _)| { - // This closure is only called if the table is resized. So while it's expensive to lookup all values, - // it will only happen rarely. - self.key_map - .hasher() - .hash_one(&table.get::>(*id).fields) - }) { - // Data has been interned by a racing call, use that ID instead - Ok(slot) => { - let id = unsafe { slot.as_ref().0 }; - let value = zalsa.table().get::>(id); - let index = self.database_key_index(id); - - // Sync the value's revision. - if value.last_interned_at.load() < current_revision { - value.last_interned_at.store(current_revision); - zalsa.event(&|| { - Event::new(EventKind::DidReinternValue { - key: index, - revision: current_revision, - }) - }); - } + // Fill up the table for the first few revisions without attempting garbage collection. + if !self.revision_queue.is_primed() { + return self.intern_id_cold( + db, + key, + zalsa, + zalsa_local, + assemble, + shard, + shard_index, + hash, + ); + } + + // Otherwise, try to reuse a stale slot. + let mut cursor = shard.lru.back_mut(); + + while let Some(value) = cursor.get() { + // SAFETY: We hold the lock for the shard containing the value. + let value_shared = unsafe { &mut *value.shared.get() }; - let durability = if let Some((_, stamp)) = zalsa_local.active_query() { - // Record the maximum durability across all queries that intern this value. - let previous_durability = value - .durability - .fetch_max(stamp.durability.as_u8(), Ordering::AcqRel); + // The value must not have been read in the current revision to be collected + // soundly, but we also do not want to collect values that have been read recently. + // + // Note that the list is sorted by LRU, so if the tail of the list is not stale, we + // will not find any stale slots. + if !self.revision_queue.is_stale(value_shared.last_interned_at) { + break; + } + + // We should never reuse a value that was accessed in the current revision. + debug_assert!(value_shared.last_interned_at < current_revision); + + // Record the durability of the current query on the interned value. + let (durability, last_interned_at) = zalsa_local + .active_query() + .map(|(_, stamp)| (stamp.durability, current_revision)) + // If there is no active query this durability does not actually matter. + // `last_interned_at` needs to be `Revision::MAX`, see the `intern_access_in_different_revision` test. + .unwrap_or((Durability::MAX, Revision::max())); - Durability::from_u8(previous_durability).max(stamp.durability) - } else { - value.durability() - }; + let old_id = value_shared.id; - // Record a dependency on this value. - zalsa_local.report_tracked_read_simple(index, durability, value.first_interned_at); + // Increment the generation of the ID, as if we allocated a new slot. + // + // If the ID is at its maximum generation, we are forced to leak the slot. + let Some(new_id) = value_shared.id.next_generation() else { + // Remove the value from the LRU list. + cursor.remove().unwrap(); + + // Retry with the previous element. + cursor = shard.lru.back_mut(); + + continue; + }; - id + // Mark the slot as reused. + *value_shared = ValueShared { + id: new_id, + durability, + last_interned_at, + }; + + // Record a dependency on the new value. + // + // Note that we can use `Revision::start()` here as we just incremented the ID generation, + // so it as if a new input has been created. + let index = self.database_key_index(value_shared.id); + zalsa_local.report_tracked_read_simple( + index, + value_shared.durability, + Revision::start(), + ); + + zalsa.event(&|| { + Event::new(EventKind::DidReuseInternedValue { + key: index, + revision: current_revision, + }) + }); + + // Remove the value from the LRU list. + // + // SAFETY: The value pointer is valid for the lifetime of the database. + let value = unsafe { &*UnsafeRef::into_raw(cursor.remove().unwrap()) }; + + // SAFETY: We hold the lock for the shard containing the value, and the + // value has not been interned in the current revision, so no references to + // it can exist. + let old_fields = unsafe { &mut *value.fields.get() }; + + // Remove the previous value from the ID map. + // + // Note that while the ID stays the same when a slot is reused, the fields, + // and thus the hash, will change, so we need to re-insert the value into the + // map. Crucially, we know that the hashes for the old and new fields both map + // to the same shard, because we determined the initial shard based on the new + // fields and only accessed the LRU list for that shard. + let old_hash = self.hasher.hash_one(&*old_fields); + shard + .key_map + .find_entry(old_hash, |found_id: &Id| *found_id == old_id) + .expect("interned value in LRU so must be in key_map") + .remove(); + + // Update the fields. + // + // SAFETY: We call `from_internal_data` to restore the correct lifetime before access. + *old_fields = unsafe { self.to_internal_data(assemble(new_id, key)) }; + + // SAFETY: We hold the lock for the shard containing the value. + let hasher = |id: &_| unsafe { self.value_hash(*id, zalsa) }; + + // Insert the new value into the ID map. + shard.key_map.insert_unique(hash, new_id, hasher); + + // Free the memos associated with the previous interned value. + // + // SAFETY: We hold the lock for the shard containing the value, and the + // value has not been interned in the current revision, so no references to + // it can exist. + let mut memo_table = unsafe { std::mem::take(&mut *value.memos.get()) }; + + // SAFETY: The memo table belongs to a value that we allocated, so it has the + // correct type. + unsafe { self.clear_memos(zalsa, &mut memo_table, new_id) }; + + if value_shared.is_reusable() { + // Move the value to the front of the LRU list. + // + // SAFETY: The value pointer is valid for the lifetime of the database. + // and never accessed mutably directly. + shard.lru.push_front(unsafe { UnsafeRef::from_raw(value) }); } - // We won any races so should intern the data - Err(slot) => { - // Record the durability of the current query on the interned value. - let (durability, last_interned_at) = zalsa_local - .active_query() - .map(|(_, stamp)| (stamp.durability, current_revision)) - // If there is no active query this durability does not actually matter. - // `last_interned_at` needs to be `Revision::MAX`, see the intern_access_in_different_revision test. - .unwrap_or((Durability::MAX, Revision::max())); - - let id = zalsa_local.allocate(zalsa, self.ingredient_index, |id| Value:: { - fields: unsafe { self.to_internal_data(assemble(id, key)) }, - memos: Default::default(), - durability: AtomicU8::new(durability.as_u8()), - // Record the revision we are interning in. - first_interned_at: current_revision, - last_interned_at: AtomicRevision::from(last_interned_at), - }); + return new_id; + } - let value = zalsa.table().get::>(id); + // If we could not find any stale slots, we are forced to allocate a new one. + self.intern_id_cold( + db, + key, + zalsa, + zalsa_local, + assemble, + shard, + shard_index, + hash, + ) + } - unsafe { lock.insert_in_slot(data_hash, slot, (id, SharedValue::new(()))) }; + /// The cold path for interning a value, allocating a new slot. + /// + /// Returns `true` if the current thread interned the value. + #[allow(clippy::too_many_arguments)] + fn intern_id_cold<'db, Key>( + &'db self, + _db: &'db dyn crate::Database, + key: Key, + zalsa: &Zalsa, + zalsa_local: &ZalsaLocal, + assemble: impl FnOnce(Id, Key) -> C::Fields<'db>, + shard: &mut IngredientShard, + shard_index: usize, + hash: u64, + ) -> crate::Id + where + Key: Hash, + C::Fields<'db>: HashEqLike, + { + let current_revision = zalsa.current_revision(); - debug_assert_eq!( - data_hash, - self.key_map - .hasher() - .hash_one(zalsa.table().get::>(id).fields.clone()) - ); + // Record the durability of the current query on the interned value. + let (durability, last_interned_at) = zalsa_local + .active_query() + .map(|(_, stamp)| (stamp.durability, current_revision)) + // If there is no active query this durability does not actually matter. + // `last_interned_at` needs to be `Revision::MAX`, see the `intern_access_in_different_revision` test. + .unwrap_or((Durability::MAX, Revision::max())); + + // Allocate the value slot. + let id = zalsa_local.allocate(zalsa, self.ingredient_index, |id| Value:: { + shard: shard_index as u16, + link: LinkedListLink::new(), + memos: UnsafeCell::new(MemoTable::default()), + // SAFETY: We call `from_internal_data` to restore the correct lifetime before access. + fields: UnsafeCell::new(unsafe { self.to_internal_data(assemble(id, key)) }), + shared: UnsafeCell::new(ValueShared { + id, + durability, + last_interned_at, + }), + }); - // Record a dependency on this value. - let index = self.database_key_index(id); - zalsa_local.report_tracked_read_simple(index, durability, value.first_interned_at); + let value = zalsa.table().get::>(id); + // SAFETY: We hold the lock for the shard containing the value. + let value_shared = unsafe { &mut *value.shared.get() }; - zalsa.event(&|| { - Event::new(EventKind::DidInternValue { - key: index, - revision: current_revision, - }) - }); + if value_shared.is_reusable() { + // Add the value to the front of the LRU list. + // + // SAFETY: The value pointer is valid for the lifetime of the database + // and never accessed mutably directly. + shard.lru.push_front(unsafe { UnsafeRef::from_raw(value) }); + } + + // SAFETY: We hold the lock for the shard containing the value. + let hasher = |id: &_| unsafe { self.value_hash(*id, zalsa) }; + + // Insert the value into the ID map. + shard.key_map.insert_unique(hash, id, hasher); + + debug_assert_eq!(hash, { + let value = zalsa.table().get::>(id); + + // SAFETY: We hold the lock for the shard containing the value. + unsafe { self.hasher.hash_one(&*value.fields.get()) } + }); + + let index = self.database_key_index(id); + + // Record a dependency on the newly interned value. + zalsa_local.report_tracked_read_simple(index, durability, Revision::start()); + + zalsa.event(&|| { + Event::new(EventKind::DidInternValue { + key: index, + revision: current_revision, + }) + }); + + id + } - id + /// Clears the given memo table. + /// + /// # Safety + /// + /// The `MemoTable` must belong to a `Value` of the correct type. + pub(crate) unsafe fn clear_memos(&self, zalsa: &Zalsa, memo_table: &mut MemoTable, id: Id) { + // SAFETY: The caller guarantees this is the correct types table. + let table = unsafe { self.memo_table_types.attach_memos_mut(memo_table) }; + + // `Database::salsa_event` is a user supplied callback which may panic + // in that case we need a drop guard to free the memo table + struct TableDropGuard<'a>(MemoTableWithTypesMut<'a>); + + impl Drop for TableDropGuard<'_> { + fn drop(&mut self) { + // SAFETY: We have `&mut MemoTable`, so no more references to these memos exist and we are good + // to drop them. + unsafe { self.0.drop() }; } } + + let mut table_guard = TableDropGuard(table); + + // SAFETY: We have `&mut MemoTable`, so no more references to these memos exist and we are good + // to drop them. + unsafe { + table_guard.0.take_memos(|memo_ingredient_index, memo| { + let ingredient_index = + zalsa.ingredient_index_for_memo(self.ingredient_index, memo_ingredient_index); + + let executor = DatabaseKeyIndex::new(ingredient_index, id); + + zalsa.event(&|| Event::new(EventKind::DidDiscard { key: executor })); + + for stale_output in memo.origin().outputs() { + stale_output.remove_stale_output(zalsa, executor); + } + }) + }; + + std::mem::forget(table_guard); + } + + // Hashes the value by its fields. + // + // # Safety + // + // The lock must be held. + unsafe fn value_hash<'db>(&'db self, id: Id, zalsa: &'db Zalsa) -> u64 { + // This closure is only called if the table is resized. So while it's expensive + // to lookup all values, it will only happen rarely. + let value = zalsa.table().get::>(id); + + // SAFETY: We hold the lock for the shard containing the value. + unsafe { self.hasher.hash_one(&*value.fields.get()) } + } + + // Compares the value by its fields to the given key. + // + // # Safety + // + // The lock must be held. + unsafe fn value_eq<'db, Key>( + id: Id, + key: &Key, + zalsa: &'db Zalsa, + found_value: &Cell>>, + ) -> bool + where + C::Fields<'db>: HashEqLike, + { + let value = zalsa.table().get::>(id); + found_value.set(Some(value)); + + // SAFETY: We hold the lock for the shard containing the value. + let fields = unsafe { &*value.fields.get() }; + + HashEqLike::eq(Self::from_internal_data(fields), key) } /// Returns the database key index for an interned value with the given id. + #[inline] pub fn database_key_index(&self, id: Id) -> DatabaseKeyIndex { DatabaseKeyIndex::new(self.ingredient_index, id) } - /// Lookup the data for an interned value based on its id. - /// Rarely used since end-users generally carry a struct with a pointer directly - /// to the interned item. + /// Lookup the data for an interned value based on its ID. pub fn data<'db>(&'db self, db: &'db dyn Database, id: Id) -> &'db C::Fields<'db> { let zalsa = db.zalsa(); - let internal_data = zalsa.table().get::>(id); - let last_changed_revision = zalsa.last_changed_revision(internal_data.durability()); + let value = zalsa.table().get::>(id); - assert!( - internal_data.last_interned_at.load() >= last_changed_revision, - "Data {:?} was not interned in the latest revision for its durability.", - self.database_key_index(id) + debug_assert!( + { + let _shard = self.shards[value.shard as usize].lock(); + + // SAFETY: We hold the lock for the shard containing the value. + let value_shared = unsafe { &mut *value.shared.get() }; + + let last_changed_revision = zalsa.last_changed_revision(value_shared.durability); + value_shared.last_interned_at >= last_changed_revision + }, + "Data was not interned in the latest revision for its durability." ); - unsafe { Self::from_internal_data(&internal_data.fields) } + // SAFETY: Interned values are only exposed if they have been validated in the + // current revision, as checked by the assertion above, which ensures that they + // are not reused while being accessed. + unsafe { Self::from_internal_data(&*value.fields.get()) } } /// Lookup the fields from an interned struct. + /// /// Note that this is not "leaking" since no dependency edge is required. pub fn fields<'db>(&'db self, db: &'db dyn Database, s: C::Struct<'db>) -> &'db C::Fields<'db> { self.data(db, AsId::as_id(&s)) @@ -366,8 +719,11 @@ where pub fn reset(&mut self, db: &mut dyn Database) { _ = db.zalsa_mut(); - // We can clear the key_map now that we have cancelled all other handles. - self.key_map.clear(); + + for shard in self.shards.iter() { + // We can clear the key maps now that we have cancelled all other handles. + shard.lock().key_map.clear(); + } } #[cfg(feature = "salsa_unstable")] @@ -396,33 +752,41 @@ where &self, db: &dyn Database, input: Id, - revision: Revision, + _revision: Revision, _cycle_heads: &mut CycleHeads, ) -> VerifyResult { let zalsa = db.zalsa(); + + // Record the current revision as active. + let current_revision = zalsa.current_revision(); + self.revision_queue.record(current_revision); + let value = zalsa.table().get::>(input); - if value.first_interned_at > revision { - // The slot was reused. + + // SAFETY: `value.shard` is guaranteed to be in-bounds for `self.shards`. + let _shard = unsafe { self.shards.get_unchecked(value.shard as usize) }.lock(); + + // SAFETY: We hold the lock for the shard containing the value. + let value_shared = unsafe { &mut *value.shared.get() }; + + // The slot was reused. + if value_shared.id.generation() > input.generation() { return VerifyResult::Changed; } - // The slot is valid in this revision but we have to sync the value's revision. - let current_revision = zalsa.current_revision(); - // No `if` to be branchless. - value.last_interned_at.store(std::cmp::max( - current_revision, - value.last_interned_at.load(), - )); + // Validate the value for the current revision to avoid reuse. + value_shared.last_interned_at = current_revision; zalsa.event(&|| { let index = self.database_key_index(input); - Event::new(EventKind::DidReinternValue { + Event::new(EventKind::DidValidateInternedValue { key: index, revision: current_revision, }) }); + // Any change to an interned value results in a new ID generation. VerifyResult::unchanged() } @@ -452,12 +816,85 @@ where { #[inline(always)] unsafe fn memos(&self, _current_revision: Revision) -> &MemoTable { - &self.memos + // SAFETY: The fact that we have a reference to the `Value` means it must + // have been interned, and thus validated, in the current revision. + unsafe { &*self.memos.get() } } #[inline(always)] fn memos_mut(&mut self) -> &mut MemoTable { - &mut self.memos + self.memos.get_mut() + } +} + +#[cfg(not(test))] +const REVS: usize = 3; + +#[cfg(test)] // Aggressively reuse slots in tests. +const REVS: usize = 1; + +/// Keep track of revisions in which interned values were read, to determine staleness. +/// +/// An interned value is considered stale if it has not been read in the past `REVS` +/// revisions. However, we only consider revisions in which interned values were actually +/// read, as revisions may be created in bursts. +struct RevisionQueue { + lock: Mutex<()>, + revisions: [AtomicRevision; REVS], +} + +impl Default for RevisionQueue { + fn default() -> RevisionQueue { + RevisionQueue { + lock: Mutex::new(()), + revisions: [const { AtomicRevision::start() }; REVS], + } + } +} + +impl RevisionQueue { + /// Record the given revision as active. + #[inline] + fn record(&self, revision: Revision) { + // Fast-path: We already recorded this revision. + if self.revisions[0].load() >= revision { + return; + } + + self.record_cold(revision); + } + + #[cold] + fn record_cold(&self, revision: Revision) { + let _lock = self.lock.lock(); + + // Otherwise, update the queue, maintaining sorted order. + // + // Note that this should only happen once per revision. + for i in (1..REVS).rev() { + self.revisions[i].store(self.revisions[i - 1].load()); + } + + self.revisions[0].store(revision); + } + + /// Returns `true` if the given revision is old enough to be considered stale. + #[inline] + fn is_stale(&self, revision: Revision) -> bool { + let oldest = self.revisions[REVS - 1].load(); + + // If we have not recorded `REVS` revisions yet, nothing can be stale. + if oldest == Revision::start() { + return false; + } + + revision < oldest + } + + /// Returns `true` if `REVS` revisions have been recorded as active. + #[inline] + fn is_primed(&self) -> bool { + self.revisions[REVS - 1].load() > Revision::start() } } @@ -590,6 +1027,7 @@ impl Lookup for &str { self.to_owned() } } + impl HashEqLike<&str> for String { fn hash(&self, h: &mut H) { Hash::hash(self, &mut *h) @@ -609,6 +1047,7 @@ impl> HashEqLike<&[A]> for Vec { self.len() == data.len() && data.iter().enumerate().all(|(i, a)| &self[i] == a) } } + impl + Clone + Lookup, T> Lookup> for &[A] { fn into_owned(self) -> Vec { self.iter().map(|a| Lookup::into_owned(a.clone())).collect() @@ -624,6 +1063,7 @@ impl> HashEqLike<[A; N]> for Vec< self.len() == data.len() && data.iter().enumerate().all(|(i, a)| &self[i] == a) } } + impl + Clone + Lookup, T> Lookup> for [A; N] { fn into_owned(self) -> Vec { self.into_iter() @@ -641,6 +1081,7 @@ impl HashEqLike<&Path> for PathBuf { self == data } } + impl Lookup for &Path { fn into_owned(self) -> PathBuf { self.to_owned() diff --git a/src/revision.rs b/src/revision.rs index bbec511b2..11cfd149c 100644 --- a/src/revision.rs +++ b/src/revision.rs @@ -24,8 +24,11 @@ impl Revision { } #[inline] - pub(crate) fn start() -> Self { - Self::from(START) + pub(crate) const fn start() -> Self { + Self { + // SAFETY: `START` is non-zero. + generation: unsafe { NonZeroUsize::new_unchecked(START) }, + } } #[inline] @@ -46,7 +49,7 @@ impl Revision { } #[inline] - fn as_usize(self) -> usize { + pub(crate) fn as_usize(self) -> usize { self.generation.get() } } @@ -71,6 +74,12 @@ impl From for AtomicRevision { } impl AtomicRevision { + pub(crate) const fn start() -> Self { + Self { + data: AtomicUsize::new(START), + } + } + pub(crate) fn load(&self) -> Revision { Revision { // SAFETY: We know that the value is non-zero because we only ever store `START` which 1, or a diff --git a/src/sync.rs b/src/sync.rs index 8cc381c1b..8db618710 100644 --- a/src/sync.rs +++ b/src/sync.rs @@ -5,35 +5,6 @@ pub mod shim { pub use shuttle::sync::*; pub use shuttle::{thread, thread_local}; - /// A polyfill for `dashmap::DashMap`. - pub struct FxDashMap(RwLock>, crate::hash::FxHasher); - - type HashTable = hashbrown_14::raw::RawTable<(K, dashmap::SharedValue)>; - - impl Default for FxDashMap { - fn default() -> FxDashMap { - FxDashMap(RwLock::default(), crate::hash::FxHasher::default()) - } - } - - impl FxDashMap { - pub fn shards(&self) -> &[RwLock>] { - std::slice::from_ref(&self.0) - } - - pub fn determine_shard(&self, _hash: usize) -> usize { - 0 - } - - pub fn hasher(&self) -> &crate::hash::FxHasher { - &self.1 - } - - pub fn clear(&self) { - self.0.write().clear(); - } - } - /// A wrapper around shuttle's `Mutex` to mirror parking-lot's API. #[derive(Default, Debug)] pub struct Mutex(shuttle::sync::Mutex); @@ -157,8 +128,6 @@ pub mod shim { pub use std::sync::*; pub use std::{thread, thread_local}; - pub(crate) type FxDashMap = dashmap::DashMap; - pub mod atomic { pub use portable_atomic::AtomicU64; pub use std::sync::atomic::*; diff --git a/src/tracked_struct.rs b/src/tracked_struct.rs index 44d0a6062..0449a8e3a 100644 --- a/src/tracked_struct.rs +++ b/src/tracked_struct.rs @@ -28,6 +28,7 @@ pub mod tracked_field; // ANCHOR: Configuration /// Trait that defines the key properties of a tracked struct. +/// /// Implemented by the `#[salsa::tracked]` macro when applied /// to a struct. pub trait Configuration: Sized + 'static { @@ -145,10 +146,14 @@ pub trait TrackedStructInDb: SalsaStructInDb { /// This ingredient only stores the "id" fields. It is a kind of "dressed up" interner; /// the active query + values of id fields are hashed to create the tracked /// struct id. The value fields are stored in [`crate::function::IngredientImpl`] -/// instances keyed by the tracked struct id. Unlike normal interners, tracked -/// struct indices can be deleted and reused aggressively: when a tracked -/// function re-executes, any tracked structs that it created before but did -/// not create this time can be deleted. +/// instances keyed by the tracked struct id. +/// +/// Unlike normal interned values, tracked struct indices can be deleted and reused aggressively +/// without dependency edges on the creating query. When a tracked function is collected, +/// any tracked structs it created can be deleted. Additionally, when a tracked function +/// re-executes but does not create a tracked struct that was previously created, it can +/// be deleted. No dependency edge is required as the lifetime of a tracked struct is tied +/// directly to the query that created it. pub struct IngredientImpl where C: Configuration, diff --git a/tests/cycle_output.rs b/tests/cycle_output.rs index 49340ea46..677e474df 100644 --- a/tests/cycle_output.rs +++ b/tests/cycle_output.rs @@ -150,7 +150,7 @@ fn revalidate_no_changes() { [ "salsa_event(DidSetCancellationFlag)", "salsa_event(DidValidateMemoizedValue { database_key: read_value(Id(400)) })", - "salsa_event(DidReinternValue { key: query_d::interned_arguments(Id(800)), revision: R2 })", + "salsa_event(DidValidateInternedValue { key: query_d::interned_arguments(Id(800)), revision: R2 })", "salsa_event(DidValidateMemoizedValue { database_key: query_d(Id(800)) })", "salsa_event(DidValidateMemoizedValue { database_key: read_value(Id(401)) })", "salsa_event(DidValidateMemoizedValue { database_key: read_value(Id(402)) })", @@ -180,9 +180,9 @@ fn revalidate_with_change_after_output_read() { [ "salsa_event(DidSetCancellationFlag)", "salsa_event(DidValidateMemoizedValue { database_key: read_value(Id(400)) })", - "salsa_event(DidReinternValue { key: query_d::interned_arguments(Id(800)), revision: R2 })", + "salsa_event(DidValidateInternedValue { key: query_d::interned_arguments(Id(800)), revision: R2 })", "salsa_event(WillExecute { database_key: query_b(Id(0)) })", - "salsa_event(DidReinternValue { key: query_d::interned_arguments(Id(800)), revision: R2 })", + "salsa_event(DidValidateInternedValue { key: query_d::interned_arguments(Id(800)), revision: R2 })", "salsa_event(WillExecute { database_key: query_a(Id(0)) })", "salsa_event(WillExecute { database_key: query_d(Id(800)) })", "salsa_event(WillDiscardStaleOutput { execute_key: query_a(Id(0)), output_key: Output(Id(403)) })", diff --git a/tests/interned-revisions.rs b/tests/interned-revisions.rs index 80dae4e38..567f27688 100644 --- a/tests/interned-revisions.rs +++ b/tests/interned-revisions.rs @@ -13,28 +13,46 @@ struct Input { } #[salsa::interned] +#[derive(Debug)] struct Interned<'db> { - field1: usize, + field1: BadHash, +} + +// Use a consistent hash value to ensure that interned value sharding +// does not interefere with garbage collection. +#[derive(PartialEq, Eq, PartialOrd, Ord, Debug, Clone)] +struct BadHash(usize); + +impl std::hash::Hash for BadHash { + fn hash(&self, state: &mut H) { + state.write_i16(0); + } +} + +#[salsa::interned] +#[derive(Debug)] +struct NestedInterned<'db> { + interned: Interned<'db>, } #[test] fn test_intern_new() { #[salsa::tracked] fn function<'db>(db: &'db dyn Database, input: Input) -> Interned<'db> { - Interned::new(db, input.field1(db)) + Interned::new(db, BadHash(input.field1(db))) } let mut db = common::EventLoggerDatabase::default(); let input = Input::new(&db, 0); let result_in_rev_1 = function(&db, input); - assert_eq!(result_in_rev_1.field1(&db), 0); + assert_eq!(result_in_rev_1.field1(&db).0, 0); // Modify the input to force a new value to be created. input.set_field1(&mut db).to(1); let result_in_rev_2 = function(&db, input); - assert_eq!(result_in_rev_2.field1(&db), 1); + assert_eq!(result_in_rev_2.field1(&db).0, 1); db.assert_logs(expect![[r#" [ @@ -53,7 +71,7 @@ fn test_reintern() { #[salsa::tracked] fn function(db: &dyn Database, input: Input) -> Interned<'_> { let _ = input.field1(db); - Interned::new(db, 0) + Interned::new(db, BadHash(0)) } let mut db = common::EventLoggerDatabase::default(); @@ -67,7 +85,7 @@ fn test_reintern() { "DidInternValue { key: Interned(Id(400)), revision: R1 }", ]"#]]); - assert_eq!(result_in_rev_1.field1(&db), 0); + assert_eq!(result_in_rev_1.field1(&db).0, 0); // Modify the input to force the value to be re-interned. input.set_field1(&mut db).to(1); @@ -78,31 +96,31 @@ fn test_reintern() { "DidSetCancellationFlag", "WillCheckCancellation", "WillExecute { database_key: function(Id(0)) }", - "DidReinternValue { key: Interned(Id(400)), revision: R2 }", + "DidValidateInternedValue { key: Interned(Id(400)), revision: R2 }", ]"#]]); - assert_eq!(result_in_rev_2.field1(&db), 0); + assert_eq!(result_in_rev_2.field1(&db).0, 0); } #[test] fn test_durability() { #[salsa::tracked] fn function<'db>(db: &'db dyn Database, _input: Input) -> Interned<'db> { - Interned::new(db, 0) + Interned::new(db, BadHash(0)) } let mut db = common::EventLoggerDatabase::default(); let input = Input::new(&db, 0); let result_in_rev_1 = function(&db, input); - assert_eq!(result_in_rev_1.field1(&db), 0); + assert_eq!(result_in_rev_1.field1(&db).0, 0); // Modify the input to bump the revision without re-interning the value, as there // is no read dependency. input.set_field1(&mut db).to(1); let result_in_rev_2 = function(&db, input); - assert_eq!(result_in_rev_2.field1(&db), 0); + assert_eq!(result_in_rev_2.field1(&db).0, 0); db.assert_logs(expect![[r#" [ @@ -114,3 +132,213 @@ fn test_durability() { "DidValidateMemoizedValue { database_key: function(Id(0)) }", ]"#]]); } + +#[test] +fn test_reuse() { + #[salsa::tracked] + fn function<'db>(db: &'db dyn Database, input: Input) -> Interned<'db> { + Interned::new(db, BadHash(input.field1(db))) + } + + let mut db = common::EventLoggerDatabase::default(); + let input = Input::new(&db, 0); + + let result = function(&db, input); + assert_eq!(result.field1(&db).0, 0); + + // Modify the input to bump the revision and intern a new value. + // + // The slot will not be reused for the first few revisions, but after + // that we should not allocate any more slots. + for i in 1..10 { + input.set_field1(&mut db).to(i); + + let result = function(&db, input); + assert_eq!(result.field1(&db).0, i); + } + + // Values that have been reused should be re-interned. + for i in 1..10 { + let result = function(&db, Input::new(&db, i)); + assert_eq!(result.field1(&db).0, i); + } + + db.assert_logs(expect![[r#" + [ + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidInternValue { key: Interned(Id(400)), revision: R1 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidInternValue { key: Interned(Id(401)), revision: R2 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidInternValue { key: Interned(Id(402)), revision: R3 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidReuseInternedValue { key: Interned(Id(400g1)), revision: R4 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidReuseInternedValue { key: Interned(Id(401g1)), revision: R5 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidReuseInternedValue { key: Interned(Id(402g1)), revision: R6 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidReuseInternedValue { key: Interned(Id(400g2)), revision: R7 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidReuseInternedValue { key: Interned(Id(401g2)), revision: R8 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidReuseInternedValue { key: Interned(Id(402g2)), revision: R9 }", + "DidSetCancellationFlag", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(0)) }", + "DidReuseInternedValue { key: Interned(Id(400g3)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(1)) }", + "DidInternValue { key: Interned(Id(403)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(2)) }", + "DidInternValue { key: Interned(Id(404)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(3)) }", + "DidInternValue { key: Interned(Id(405)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(4)) }", + "DidInternValue { key: Interned(Id(406)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(5)) }", + "DidInternValue { key: Interned(Id(407)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(6)) }", + "DidInternValue { key: Interned(Id(408)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(7)) }", + "DidValidateInternedValue { key: Interned(Id(401g2)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(8)) }", + "DidValidateInternedValue { key: Interned(Id(402g2)), revision: R10 }", + "WillCheckCancellation", + "WillExecute { database_key: function(Id(9)) }", + ]"#]]); +} + +#[test] +fn test_reuse_interned_input() { + // A query that creates an interned value. + #[salsa::tracked] + fn create_interned<'db>(db: &'db dyn Database, input: Input) -> Interned<'db> { + Interned::new(db, BadHash(input.field1(db))) + } + + #[salsa::tracked] + fn use_interned<'db>(db: &'db dyn Database, interned: Interned<'db>) -> usize { + interned.field1(db).0 + } + + let mut db = common::EventLoggerDatabase::default(); + let input = Input::new(&db, 0); + + // Create and use I0 in R0. + let interned = create_interned(&db, input); + let result = use_interned(&db, interned); + assert_eq!(result, 0); + + // Create and use I1 in a number of revisions, marking I0 as stale. + input.set_field1(&mut db).to(1); + for _ in 0..10 { + let interned = create_interned(&db, input); + let result = use_interned(&db, interned); + assert_eq!(result, 1); + + // Trigger a new revision. + input.set_field1(&mut db).to(1); + } + + // Create I2, reusing the stale slot of I0. + input.set_field1(&mut db).to(2); + let interned = create_interned(&db, input); + + // Use I2. The function should not be memoized with the value of I0, despite I2 and I0 + // sharing the same slot. + let result = use_interned(&db, interned); + assert_eq!(result, 2); +} + +#[test] +fn test_reuse_multiple_interned_input() { + // A query that creates an interned value. + #[salsa::tracked] + fn create_interned<'db>(db: &'db dyn Database, input: Input) -> Interned<'db> { + Interned::new(db, BadHash(input.field1(db))) + } + + // A query that creates an interned value. + #[salsa::tracked] + fn create_nested_interned<'db>( + db: &'db dyn Database, + interned: Interned<'db>, + ) -> NestedInterned<'db> { + NestedInterned::new(db, interned) + } + + #[salsa::tracked] + fn use_interned<'db>(db: &'db dyn Database, interned: Interned<'db>) -> usize { + interned.field1(db).0 + } + + // A query that reads an interned value. + #[salsa::tracked] + fn use_nested_interned<'db>( + db: &'db dyn Database, + nested_interned: NestedInterned<'db>, + ) -> usize { + nested_interned.interned(db).field1(db).0 + } + + let mut db = common::EventLoggerDatabase::default(); + let input = Input::new(&db, 0); + + // Create and use NI0, which wraps I0, in R0. + let interned = create_interned(&db, input); + let i0_id = salsa::plumbing::AsId::as_id(&interned); + let nested_interned = create_nested_interned(&db, interned); + let result = use_nested_interned(&db, nested_interned); + assert_eq!(result, 0); + + // Create and use I1 in a number of revisions, marking I0 as stale. + input.set_field1(&mut db).to(1); + for _ in 0..10 { + let interned = create_interned(&db, input); + let result = use_interned(&db, interned); + assert_eq!(result, 1); + + // Trigger a new revision. + input.set_field1(&mut db).to(1); + } + + // Create I2, reusing the stale slot of I0. + input.set_field1(&mut db).to(2); + let interned = create_interned(&db, input); + + let i2_id = salsa::plumbing::AsId::as_id(&interned); + assert_ne!(i0_id, i2_id); + + // Create NI1 wrapping I2 instead of I0. + let nested_interned = create_nested_interned(&db, interned); + + // Use NI1. The function should not be memoized with the value of NI0, + // despite I2 and I0 sharing the same ID. + let result = use_nested_interned(&db, nested_interned); + assert_eq!(result, 2); +} diff --git a/tests/preverify-struct-with-leaked-data-2.rs b/tests/preverify-struct-with-leaked-data-2.rs index df3bdf941..d7e3f8f9f 100644 --- a/tests/preverify-struct-with-leaked-data-2.rs +++ b/tests/preverify-struct-with-leaked-data-2.rs @@ -83,7 +83,7 @@ fn test_leaked_inputs_ignored() { [ "DidSetCancellationFlag", "WillCheckCancellation", - "DidReinternValue { key: counter_field::interned_arguments(Id(800)), revision: R2 }", + "DidValidateInternedValue { key: counter_field::interned_arguments(Id(800)), revision: R2 }", "WillCheckCancellation", "WillExecute { database_key: counter_field(Id(800)) }", "WillExecute { database_key: function(Id(0)) }",