Use generational identifiers for tracked structs (#864)

* use generational identifiers for tracked structs

* increase ID generations to 32-bits

* remove `created_at` field from tracked structs

* clean up tracked struct IDs to handle overflow

* log tracing message for leaked tracked structs
This commit is contained in:
Ibraheem Ahmed 2025-05-23 10:18:03 -04:00 committed by GitHub
parent a12bf313a4
commit f7b08562ed
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
8 changed files with 293 additions and 167 deletions

View file

@ -1,16 +1,17 @@
use std::fmt::Debug;
use std::hash::Hash;
use std::num::NonZeroU32;
use std::num::NonZeroU64;
use crate::zalsa::Zalsa;
/// The `Id` of a salsa struct in the database [`Table`](`crate::table::Table`).
///
/// The higher-order bits of an `Id` identify a [`Page`](`crate::table::Page`)
/// and the low-order bits identify a slot within the page.
/// The high-order bits of an `Id` store a 32-bit generation counter, while
/// the low-order bits pack a [`PageIndex`](`crate::table::PageIndex`) and
/// [`SlotIndex`](`crate::table::SlotIndex`) within the page.
///
/// An Id is a newtype'd u32 ranging from `0..Id::MAX_U32`.
/// The maximum range is smaller than a standard u32 to leave
/// The low-order bits of `Id` are a `u32` ranging from `0..Id::MAX_U32`.
/// The maximum range is smaller than a standard `u32` to leave
/// room for niches; currently there is only one niche, so that
/// `Option<Id>` is the same size as an `Id`.
///
@ -18,15 +19,15 @@ use crate::zalsa::Zalsa;
/// it is wrapped in new types.
#[derive(Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)]
pub struct Id {
value: NonZeroU32,
value: NonZeroU64,
}
impl Id {
pub const MAX_U32: u32 = u32::MAX - 0xFF;
pub const MAX_USIZE: usize = Self::MAX_U32 as usize;
/// Create a `salsa::Id` from a u32 value. This value should
/// be less than [`Self::MAX_U32`].
/// Create a `salsa::Id` from a u32 value, without a generation. This
/// value should be less than [`Self::MAX_U32`].
///
/// In general, you should not need to create salsa ids yourself,
/// but it can be useful if you are using the type as a general
@ -38,23 +39,90 @@ impl Id {
#[doc(hidden)]
#[track_caller]
#[inline]
pub const unsafe fn from_u32(v: u32) -> Self {
pub const unsafe fn from_index(v: u32) -> Self {
debug_assert!(v < Self::MAX_U32);
Id {
// SAFETY: Caller obligation
value: unsafe { NonZeroU32::new_unchecked(v + 1) },
// SAFETY: Caller obligation.
value: unsafe { NonZeroU64::new_unchecked((v + 1) as u64) },
}
}
/// Create a `salsa::Id` from a u64 value.
///
/// This should only be used to recreate an `Id` together with `Id::as_u64`.
///
/// # Safety
///
/// The data bits of the supplied value must represent a valid `Id` returned
/// by `Id::as_u64`.
#[doc(hidden)]
#[track_caller]
#[inline]
pub const fn as_u32(self) -> u32 {
self.value.get() - 1
pub const unsafe fn from_bits(v: u64) -> Self {
Id {
// SAFETY: Caller obligation.
value: unsafe { NonZeroU64::new_unchecked(v) },
}
}
/// Returns a new `Id` with same index, but the generation incremented by one.
///
/// Returns `None` if the generation would overflow, i.e. the current generation
/// is `u32::MAX`.
#[inline]
pub fn next_generation(self) -> Option<Id> {
self.generation()
.checked_add(1)
.map(|generation| self.with_generation(generation))
}
/// Mark the `Id` with a generation.
///
/// This `Id` will refer to the same page and slot in the database,
/// but will differ from other identifiers of the slot based on the
/// provided generation.
#[inline]
pub const fn with_generation(self, generation: u32) -> Id {
let mut value = self.value.get();
value &= 0xFFFFFFFF;
value |= (generation as u64) << 32;
Id {
// SAFETY: The niche of `value` is in the lower bits, which we did not touch.
value: unsafe { NonZeroU64::new_unchecked(value) },
}
}
/// Return the index portion of this `Id`.
#[inline]
pub const fn index(self) -> u32 {
// Truncate the high-order bits.
(self.value.get() as u32) - 1
}
/// Return the generation of this `Id`.
#[inline]
pub const fn generation(self) -> u32 {
// Shift away the low-order bits.
(self.value.get() >> 32) as u32
}
/// Return the internal `u64` representation of this `Id`.
#[inline]
pub const fn as_bits(self) -> u64 {
self.value.get()
}
}
impl Debug for Id {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "Id({:x})", self.as_u32())
if self.generation() == 0 {
write!(f, "Id({:x})", self.index())
} else {
write!(f, "Id({:x}g{:x})", self.index(), self.generation())
}
}
}

View file

@ -1,4 +1,4 @@
use crate::loom::sync::atomic::{AtomicU32, Ordering};
use crate::loom::sync::atomic::{AtomicU64, Ordering};
use crate::Id;
mod sealed {
@ -11,7 +11,7 @@ pub trait SingletonChoice: sealed::Sealed + Default {
}
pub struct Singleton {
index: AtomicU32,
index: AtomicU64,
}
impl sealed::Sealed for Singleton {}
impl SingletonChoice for Singleton {
@ -22,7 +22,7 @@ impl SingletonChoice for Singleton {
let id = cb();
if self
.index
.compare_exchange(0, id.as_u32() + 1, Ordering::AcqRel, Ordering::Acquire)
.compare_exchange(0, id.as_bits(), Ordering::AcqRel, Ordering::Acquire)
.is_err()
{
panic!("singleton struct may not be duplicated");
@ -33,8 +33,9 @@ impl SingletonChoice for Singleton {
fn index(&self) -> Option<Id> {
match self.index.load(Ordering::Acquire) {
0 => None,
// SAFETY: Our u32 is derived from an ID and thus safe to convert back.
id => Some(unsafe { Id::from_u32(id - 1) }),
// SAFETY: Our u64 is derived from an ID and thus safe to convert back.
id => Some(unsafe { Id::from_bits(id) }),
}
}
}
@ -42,7 +43,7 @@ impl SingletonChoice for Singleton {
impl Default for Singleton {
fn default() -> Self {
Self {
index: AtomicU32::new(0),
index: AtomicU64::new(0),
}
}
}

View file

@ -12,8 +12,9 @@ use crate::{Database, Id};
/// only for inserting into maps and the like.
#[derive(Copy, Clone, PartialEq, Eq, Hash)]
pub struct DatabaseKeyIndex {
key_index: u32,
key_generation: u32,
ingredient_index: IngredientIndex,
key_index: Id,
}
// ANCHOR_END: DatabaseKeyIndex
@ -21,17 +22,19 @@ impl DatabaseKeyIndex {
#[inline]
pub(crate) fn new(ingredient_index: IngredientIndex, key_index: Id) -> Self {
Self {
key_index,
key_index: key_index.index(),
key_generation: key_index.generation(),
ingredient_index,
}
}
pub fn ingredient_index(self) -> IngredientIndex {
pub const fn ingredient_index(self) -> IngredientIndex {
self.ingredient_index
}
pub fn key_index(self) -> Id {
self.key_index
pub const fn key_index(self) -> Id {
// SAFETY: `self.key_index` was returned by `Id::data`.
unsafe { Id::from_index(self.key_index) }.with_generation(self.key_generation)
}
pub(crate) fn maybe_changed_after(
@ -44,15 +47,15 @@ impl DatabaseKeyIndex {
// SAFETY: The `db` belongs to the ingredient
unsafe {
zalsa
.lookup_ingredient(self.ingredient_index)
.maybe_changed_after(db, self.key_index, last_verified_at, cycle_heads)
.lookup_ingredient(self.ingredient_index())
.maybe_changed_after(db, self.key_index(), last_verified_at, cycle_heads)
}
}
pub(crate) fn remove_stale_output(&self, zalsa: &Zalsa, executor: DatabaseKeyIndex) {
zalsa
.lookup_ingredient(self.ingredient_index)
.remove_stale_output(zalsa, executor, self.key_index)
.lookup_ingredient(self.ingredient_index())
.remove_stale_output(zalsa, executor, self.key_index())
}
pub(crate) fn mark_validated_output(
@ -61,21 +64,21 @@ impl DatabaseKeyIndex {
database_key_index: DatabaseKeyIndex,
) {
zalsa
.lookup_ingredient(self.ingredient_index)
.mark_validated_output(zalsa, database_key_index, self.key_index)
.lookup_ingredient(self.ingredient_index())
.mark_validated_output(zalsa, database_key_index, self.key_index())
}
}
impl fmt::Debug for DatabaseKeyIndex {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
crate::attach::with_attached_database(|db| {
let ingredient = db.zalsa().lookup_ingredient(self.ingredient_index);
ingredient.fmt_index(self.key_index, f)
let ingredient = db.zalsa().lookup_ingredient(self.ingredient_index());
ingredient.fmt_index(self.key_index(), f)
})
.unwrap_or_else(|| {
f.debug_tuple("DatabaseKeyIndex")
.field(&self.ingredient_index)
.field(&self.key_index)
.field(&self.ingredient_index())
.field(&self.key_index())
.finish()
})
}

View file

@ -400,13 +400,13 @@ fn make_id(page: PageIndex, slot: SlotIndex) -> Id {
let page = page.0 as u32;
let slot = slot.0 as u32;
// SAFETY: `slot` is guaranteed to be small enough that the resulting Id won't be bigger than `Id::MAX_U32`
unsafe { Id::from_u32((page << PAGE_LEN_BITS) | slot) }
unsafe { Id::from_index((page << PAGE_LEN_BITS) | slot) }
}
#[inline]
fn split_id(id: Id) -> (PageIndex, SlotIndex) {
let id = id.as_u32() as usize;
let slot = id & PAGE_LEN_MASK;
let page = id >> PAGE_LEN_BITS;
let index = id.index() as usize;
let slot = index & PAGE_LEN_MASK;
let page = index >> PAGE_LEN_BITS;
(PageIndex::new(page), SlotIndex::new(slot))
}

View file

@ -271,15 +271,6 @@ where
/// with different values.
durability: Durability,
/// The revision in which the tracked struct was first created.
///
/// Unlike `updated_at`, which gets bumped on every read,
/// `created_at` is updated whenever an untracked field is updated.
/// This is necessary to detect reused tracked struct ids _after_
/// they've been freed in a prior revision or tracked structs that have been updated
/// in-place because of a bad `Hash` implementation.
created_at: Revision,
/// The revision when this tracked struct was last updated.
/// This field also acts as a kind of "lock". Once it is equal
/// to `Some(current_revision)`, the fields are locked and
@ -376,7 +367,7 @@ where
pub fn new_struct<'db>(
&'db self,
db: &'db dyn Database,
fields: C::Fields<'db>,
mut fields: C::Fields<'db>,
) -> C::Struct<'db> {
let (zalsa, zalsa_local) = db.zalsas();
@ -394,26 +385,39 @@ where
};
let current_revision = zalsa.current_revision();
match zalsa_local.tracked_struct_id(&identity) {
Some(id) => {
let index = self.database_key_index(id);
tracing::trace!("Reuse tracked struct {id:?}", id = index);
// The struct already exists in the intern map.
zalsa_local.add_output(index);
self.update(zalsa, current_revision, id, &current_deps, fields);
FromId::from_id(id)
}
if let Some(id) = zalsa_local.tracked_struct_id(&identity) {
// The struct already exists in the intern map.
let index = self.database_key_index(id);
tracing::trace!("Reuse tracked struct {id:?}", id = index);
zalsa_local.add_output(index);
None => {
// This is a new tracked struct, so create an entry in the struct map.
let id = self.allocate(zalsa, zalsa_local, current_revision, &current_deps, fields);
let key = self.database_key_index(id);
tracing::trace!("Allocated new tracked struct {key:?}");
zalsa_local.add_output(key);
zalsa_local.store_tracked_struct_id(identity, id);
FromId::from_id(id)
}
// SAFETY: The `id` was present in the interned map, so the value must be initialized.
let update_result =
unsafe { self.update(zalsa, current_revision, id, &current_deps, fields) };
fields = match update_result {
// Overwrite the previous ID if we are reusing the old slot with new fields.
Ok(updated_id) if updated_id != id => {
zalsa_local.store_tracked_struct_id(identity, updated_id);
return FromId::from_id(updated_id);
}
// The id has not changed.
Ok(id) => return FromId::from_id(id),
// Failed to perform the update, we are forced to allocate a new slot.
Err(fields) => fields,
};
}
// We failed to perform the update, or this is a new tracked struct, so allocate a new entry
// in the struct map.
let id = self.allocate(zalsa, zalsa_local, current_revision, &current_deps, fields);
let key = self.database_key_index(id);
tracing::trace!("Allocated new tracked struct {key:?}");
zalsa_local.add_output(key);
zalsa_local.store_tracked_struct_id(identity, id);
FromId::from_id(id)
}
fn allocate<'db>(
@ -425,7 +429,6 @@ where
fields: C::Fields<'db>,
) -> Id {
let value = |_| Value {
created_at: current_revision,
updated_at: OptionalAtomicRevision::new(Some(current_revision)),
durability: current_deps.durability,
// lifetime erase for storage
@ -434,26 +437,38 @@ where
memos: Default::default(),
};
if let Some(id) = self.free_list.pop() {
Self::data_raw(zalsa.table(), id).with_mut(|data_raw| {
let data_raw = data_raw.cast::<Value<C>>();
while let Some(id) = self.free_list.pop() {
// Increment the ID generation before reusing it, as if we have allocated a new
// slot in the table.
//
// If the generation would overflow, we are forced to leak the slot. Note that this
// shouldn't be a problem in general as sufficient bits are reserved for the generation.
let Some(id) = id.next_generation() else {
tracing::info!(
"leaking tracked struct {:?} due to generation oveflow",
self.database_key_index(id)
);
continue;
};
return Self::data_raw(zalsa.table(), id).with_mut(|data_raw| {
let data = unsafe { (*data_raw).assume_init_mut() };
assert!(
unsafe { (*data_raw).updated_at.load().is_none() },
data.updated_at.load().is_none(),
"free list entry for `{id:?}` does not have `None` for `updated_at`"
);
// Overwrite the free-list entry. Use `*foo = ` because the entry
// has been previously initialized and we want to free the old contents.
unsafe {
*data_raw = value(id);
}
*data = value(id);
id
})
} else {
zalsa_local.allocate::<Value<C>>(zalsa, self.ingredient_index, value)
});
}
zalsa_local.allocate::<Value<C>>(zalsa, self.ingredient_index, value)
}
/// Get mutable access to the data for `id` -- this holds a write lock for the duration
@ -463,14 +478,18 @@ where
///
/// * If the value is not present in the map.
/// * If the value is already updated in this revision.
fn update<'db>(
///
/// # Safety
///
/// The value at the given `id` must be initialized.
unsafe fn update<'db>(
&'db self,
zalsa: &'db Zalsa,
current_revision: Revision,
id: Id,
mut id: Id,
current_deps: &StampedValue<()>,
fields: C::Fields<'db>,
) {
) -> Result<Id, C::Fields<'db>> {
let data_raw = Self::data_raw(zalsa.table(), id);
// The protocol is:
@ -507,37 +526,51 @@ where
// during the current revision and thus obtained an `&` reference to those fields
// that is still live.
// UNSAFE: Marking as mut requires exclusive access for the duration of
// the `mut`. We have now *claimed* this data by swapping in `None`,
// any attempt to read concurrently will panic.
let locked = data_raw.with(|data| {
let data = data.cast::<Value<C>>();
let lock_result = data_raw.with(|data_raw| {
// SAFETY: Guaranteed by caller.
let data = unsafe { (*data_raw).assume_init_ref() };
let last_updated_at = unsafe { (*data).updated_at.load() };
let last_updated_at = data.updated_at.load();
assert!(
last_updated_at.is_some(),
"two concurrent writers to {id:?}, should not be possible"
);
if last_updated_at == Some(current_revision) {
// already read-locked
return false;
return Ok(false);
}
// Updating the fields may make it necessary to increment the generation of the ID. In
// the unlikely case that the ID is already at its maximum generation, we are forced to leak
// the previous slot and allocate a new value.
if id.generation() == u32::MAX {
tracing::info!(
"leaking tracked struct {:?} due to generation oveflow",
self.database_key_index(id)
);
return Err(());
}
// Acquire the write-lock. This can only fail if there is a parallel thread
// reading from this same `id`, which can only happen if the user has leaked it.
// Tsk tsk.
let swapped_out = unsafe { (*data).updated_at.swap(None) };
let swapped_out = data.updated_at.swap(None) ;
if swapped_out != last_updated_at {
panic!(
"failed to acquire write lock, id `{id:?}` must have been leaked across threads"
);
panic!("failed to acquire write lock, id `{id:?}` must have been leaked across threads");
}
true
Ok(true)
});
if !locked {
return;
match lock_result {
// We cannot perform the update as the ID is at its maximum generation.
Err(()) => return Err(fields),
// The value is already read-locked, but we can reuse it safely as per above.
Ok(false) => return Ok(id),
// Acquired the write-lock.
Ok(true) => {}
}
data_raw.with_mut(|data| {
@ -551,31 +584,43 @@ where
// from a previous revision. As such, it continues to meet
// its validity invariant and any owned content also continues
// to meet its safety invariant.
unsafe {
if C::update_fields(
let untracked_update = unsafe {
C::update_fields(
current_deps.changed_at,
&mut data.revisions,
mem::transmute::<*mut C::Fields<'static>, *mut C::Fields<'db>>(
std::ptr::addr_of_mut!(data.fields),
),
fields,
) {
// Consider this a new tracked-struct (even though it still uses the same id)
// when any non-tracked field got updated.
// This should be rare and only ever happen if there's a hash collision
// which makes Salsa consider two tracked structs to still be the same
// even though the fields are different.
// See `tracked-struct-id-field-bad-hash` for more details.
data.created_at = current_revision;
}
)
};
if untracked_update {
// Consider this a new tracked-struct when any non-tracked field got updated.
// This should be rare and only ever happen if there's a hash collision.
//
// Note that we hold the lock and have exclusive access to the tracked struct data,
// so there should be no live instances of IDs from the previous generation. We clear
// the memos and return a new ID here as if we have allocated a new slot.
let mut table = data.take_memo_table();
// SAFETY: The memo table belongs to a value that we allocated, so it has the
// correct type.
unsafe { self.clear_memos(zalsa, &mut table, id) };
id = id
.next_generation()
.expect("already verified that generation is not maximum");
}
if current_deps.durability < data.durability {
data.revisions = C::new_revisions(current_deps.changed_at);
data.created_at = current_revision;
}
data.durability = current_deps.durability;
let swapped_out = data.updated_at.swap(Some(current_revision));
assert!(swapped_out.is_none());
Ok(id)
})
}
@ -609,13 +654,12 @@ where
let current_revision = zalsa.current_revision();
let data_raw = Self::data_raw(zalsa.table(), id);
data_raw.with(|data| {
let data = data.cast::<Value<C>>();
data_raw.with(|data_raw| {
let data = unsafe { (*data_raw).assume_init_ref() };
// We want to set `updated_at` to `None`, signalling that other field values
// cannot be read. The current value should be `Some(R0)` for some older revision.
let updated_at = unsafe { &(*data).updated_at };
match updated_at.load() {
match data.updated_at.load() {
None => {
panic!("cannot delete write-locked id `{id:?}`; value leaked across threads");
}
@ -623,7 +667,7 @@ where
"cannot delete read-locked id `{id:?}`; value leaked across threads or user functions not deterministic"
),
Some(r) => {
if updated_at.compare_exchange(Some(r), None).is_err() {
if data.updated_at.compare_exchange(Some(r), None).is_err() {
panic!("race occurred when deleting value `{id:?}`")
}
}
@ -635,20 +679,37 @@ where
unsafe { (*data).assume_init_mut() }.take_memo_table()
});
// SAFETY: We use the correct types table.
let table = unsafe { self.memo_table_types.attach_memos_mut(&mut memo_table) };
// 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, id) };
// now that all cleanup has occurred, make available for re-use
self.free_list.push(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 verified that no more references to these memos exist and so we are good
// 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 verified that no more references to these memos exist and so we are good
// 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| {
@ -664,10 +725,8 @@ where
}
})
};
mem::forget(table_guard);
// now that all cleanup has occurred, make available for re-use
self.free_list.push(id);
mem::forget(table_guard);
}
/// Return reference to the field data ignoring dependency tracking.
@ -719,18 +778,15 @@ where
db: &'db dyn crate::Database,
s: C::Struct<'db>,
) -> &'db C::Fields<'db> {
let (zalsa, zalsa_local) = db.zalsas();
let zalsa = db.zalsa();
let id = AsId::as_id(&s);
let data = Self::data(zalsa.table(), id);
data.read_lock(zalsa.current_revision());
// Add a dependency on the tracked struct itself.
zalsa_local.report_tracked_read_simple(
DatabaseKeyIndex::new(self.ingredient_index, id),
data.durability,
data.created_at,
);
// Note that we do not need to add a dependency on the tracked struct
// as IDs that are reused increment their generation, invalidating any
// dependent queries directly.
data.fields()
}
@ -759,15 +815,13 @@ where
unsafe fn maybe_changed_after(
&self,
db: &dyn Database,
input: Id,
revision: Revision,
_db: &dyn Database,
_input: Id,
_revision: Revision,
_cycle_heads: &mut CycleHeads,
) -> VerifyResult {
let zalsa = db.zalsa();
let data = Self::data(zalsa.table(), input);
VerifyResult::changed_if(data.created_at > revision)
// Any change to a tracked struct results in a new ID generation.
VerifyResult::unchanged()
}
fn mark_validated_output(
@ -791,7 +845,7 @@ where
// `executor` creates a tracked struct `salsa_output_key`,
// but it did not in the current revision.
// In that case, we can delete `stale_output_key` and any data associated with it.
self.delete_entity(zalsa, stale_output_key);
self.delete_entity(zalsa, stale_output_key)
}
fn debug_name(&self) -> &'static str {
@ -960,23 +1014,23 @@ mod tests {
};
// SAFETY: We don't use the IDs within salsa internals so this is fine
unsafe {
assert_eq!(d.insert(i1, Id::from_u32(0)), None);
assert_eq!(d.insert(i2, Id::from_u32(1)), None);
assert_eq!(d.insert(i3, Id::from_u32(2)), None);
assert_eq!(d.insert(i4, Id::from_u32(3)), None);
assert_eq!(d.insert(i5, Id::from_u32(4)), None);
assert_eq!(d.insert(i6, Id::from_u32(5)), None);
assert_eq!(d.insert(i7, Id::from_u32(6)), None);
assert_eq!(d.insert(i8, Id::from_u32(7)), None);
assert_eq!(d.insert(i1, Id::from_index(0)), None);
assert_eq!(d.insert(i2, Id::from_index(1)), None);
assert_eq!(d.insert(i3, Id::from_index(2)), None);
assert_eq!(d.insert(i4, Id::from_index(3)), None);
assert_eq!(d.insert(i5, Id::from_index(4)), None);
assert_eq!(d.insert(i6, Id::from_index(5)), None);
assert_eq!(d.insert(i7, Id::from_index(6)), None);
assert_eq!(d.insert(i8, Id::from_index(7)), None);
assert_eq!(d.get(&i1), Some(Id::from_u32(0)));
assert_eq!(d.get(&i2), Some(Id::from_u32(1)));
assert_eq!(d.get(&i3), Some(Id::from_u32(2)));
assert_eq!(d.get(&i4), Some(Id::from_u32(3)));
assert_eq!(d.get(&i5), Some(Id::from_u32(4)));
assert_eq!(d.get(&i6), Some(Id::from_u32(5)));
assert_eq!(d.get(&i7), Some(Id::from_u32(6)));
assert_eq!(d.get(&i8), Some(Id::from_u32(7)));
assert_eq!(d.get(&i1), Some(Id::from_index(0)));
assert_eq!(d.get(&i2), Some(Id::from_index(1)));
assert_eq!(d.get(&i3), Some(Id::from_index(2)));
assert_eq!(d.get(&i4), Some(Id::from_index(3)));
assert_eq!(d.get(&i5), Some(Id::from_index(4)));
assert_eq!(d.get(&i6), Some(Id::from_index(5)));
assert_eq!(d.get(&i7), Some(Id::from_index(6)));
assert_eq!(d.get(&i8), Some(Id::from_index(7)));
};
}
}

View file

@ -295,11 +295,7 @@ impl ZalsaLocal {
let top_query = stack
.last_mut()
.expect("cannot store a tracked struct ID outside of a tracked function");
let old_id = top_query.tracked_struct_ids.insert(identity, id);
assert!(
old_id.is_none(),
"overwrote a previous id for `{identity:?}`"
);
top_query.tracked_struct_ids.insert(identity, id);
})
}

View file

@ -152,6 +152,9 @@ fn revalidate_no_changes() {
"salsa_event(DidValidateMemoizedValue { database_key: read_value(Id(400)) })",
"salsa_event(DidReinternValue { 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)) })",
"salsa_event(DidValidateMemoizedValue { database_key: read_value(Id(403)) })",
"salsa_event(DidValidateMemoizedValue { database_key: query_b(Id(0)) })",
]"#]]);
}
@ -193,12 +196,12 @@ fn revalidate_with_change_after_output_read() {
"salsa_event(DidDiscard { key: read_value(Id(402)) })",
"salsa_event(WillIterateCycle { database_key: query_b(Id(0)), iteration_count: 1, fell_back: false })",
"salsa_event(WillExecute { database_key: query_a(Id(0)) })",
"salsa_event(WillExecute { database_key: read_value(Id(403)) })",
"salsa_event(WillExecute { database_key: read_value(Id(403g1)) })",
"salsa_event(WillIterateCycle { database_key: query_b(Id(0)), iteration_count: 2, fell_back: false })",
"salsa_event(WillExecute { database_key: query_a(Id(0)) })",
"salsa_event(WillExecute { database_key: read_value(Id(401)) })",
"salsa_event(WillExecute { database_key: read_value(Id(401g1)) })",
"salsa_event(WillIterateCycle { database_key: query_b(Id(0)), iteration_count: 3, fell_back: false })",
"salsa_event(WillExecute { database_key: query_a(Id(0)) })",
"salsa_event(WillExecute { database_key: read_value(Id(402)) })",
"salsa_event(WillExecute { database_key: read_value(Id(402g1)) })",
]"#]]);
}

View file

@ -1,9 +1,10 @@
//! Test for a tracked struct where an untracked field has a
//! very poorly chosen hash impl (always returns 0).
//! This demonstrates that the `untracked fields on a struct
//! can change values and yet the struct can have the same
//! id (because struct ids are based on the *hash* of the
//! untracked fields).
//!
//! This demonstrates that tracked struct ids will always change if
//! untracked fields on a struct change values, because although struct
//! ids are based on the *hash* of the untracked fields, ids are generational
//! based on the field values.
use salsa::{Database as Db, Setter};
use test_log::test;
@ -70,11 +71,11 @@ fn dependent_query() {
assert!(with_tracked(&db, tracked));
input.set_field(&mut db).to(false);
// We now re-run the query that creates the tracked struct.
// Salsa will re-use the `MyTracked` struct from the previous revision
// because it thinks it is unchanged because of `BadHash`'s bad hash function.
// That's why Salsa updates the `MyTracked` struct in-place and the struct
// should be considered re-created even though it still has the same id.
//
// Salsa will re-use the `MyTracked` struct from the previous revision,
// but practically it has been re-created due to generational ids.
let tracked = create_tracked(&db, input);
assert!(!with_tracked(&db, tracked));
}