Pack QueryOrigin memory layout (#885)
Some checks are pending
Book / Book (push) Waiting to run
Book / Deploy (push) Blocked by required conditions
Release-plz / Release-plz release (push) Waiting to run
Release-plz / Release-plz PR (push) Waiting to run
Test / Benchmarks (push) Waiting to run
Test / Test (push) Waiting to run
Test / Miri (push) Waiting to run
Test / Shuttle (push) Waiting to run

* pack `QueryOrigin` into 13 bytes

* nits

Co-authored-by: David Barsky <me@davidbarsky.com>

---------

Co-authored-by: David Barsky <me@davidbarsky.com>
This commit is contained in:
Ibraheem Ahmed 2025-05-29 16:00:50 -04:00 committed by GitHub
parent 40d7844a7a
commit 5750c8448f
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
14 changed files with 293 additions and 113 deletions

View file

@ -11,7 +11,7 @@ use crate::key::DatabaseKeyIndex;
use crate::runtime::Stamp;
use crate::sync::atomic::AtomicBool;
use crate::tracked_struct::{Disambiguator, DisambiguatorMap, IdentityHash, IdentityMap};
use crate::zalsa_local::{QueryEdge, QueryEdges, QueryOrigin, QueryRevisions};
use crate::zalsa_local::{QueryEdge, QueryOrigin, QueryRevisions};
use crate::{Accumulator, IngredientIndex, Revision};
#[derive(Debug)]
@ -193,11 +193,10 @@ impl ActiveQuery {
iteration_count: _,
} = self;
let edges = QueryEdges::new(input_outputs.drain(..));
let origin = if untracked_read {
QueryOrigin::DerivedUntracked(edges)
QueryOrigin::derived_untracked(input_outputs.drain(..))
} else {
QueryOrigin::Derived(edges)
QueryOrigin::derived(input_outputs.drain(..))
};
disambiguator_map.clear();
let accumulated = accumulated

View file

@ -17,7 +17,7 @@ use crate::table::memo::MemoTableTypes;
use crate::table::Table;
use crate::views::DatabaseDownCaster;
use crate::zalsa::{IngredientIndex, MemoIngredientIndex, Zalsa};
use crate::zalsa_local::QueryOrigin;
use crate::zalsa_local::QueryOriginRef;
use crate::{Database, Id, Revision};
mod accumulated;
@ -268,7 +268,7 @@ where
}
}
fn origin(&self, zalsa: &Zalsa, key: Id) -> Option<QueryOrigin> {
fn origin<'db>(&self, zalsa: &'db Zalsa, key: Id) -> Option<QueryOriginRef<'db>> {
self.origin(zalsa, key)
}

View file

@ -3,7 +3,7 @@ use crate::accumulator::{self};
use crate::function::{Configuration, IngredientImpl};
use crate::hash::FxHashSet;
use crate::zalsa::ZalsaDatabase;
use crate::zalsa_local::QueryOrigin;
use crate::zalsa_local::QueryOriginRef;
use crate::{AsDynDatabase, DatabaseKeyIndex, Id};
impl<C> IngredientImpl<C>
@ -73,8 +73,9 @@ where
continue;
};
if let QueryOrigin::Derived(edges) | QueryOrigin::DerivedUntracked(edges) = &origin {
stack.reserve(edges.input_outputs.len());
if let QueryOriginRef::Derived(edges) | QueryOriginRef::DerivedUntracked(edges) = origin
{
stack.reserve(edges.len());
}
stack.extend(

View file

@ -28,6 +28,7 @@ where
let mut old_outputs: FxIndexSet<_> = old_memo
.revisions
.origin
.as_ref()
.outputs()
.map(|a| (a.ingredient_index(), a.key_index().index()))
.collect();
@ -38,7 +39,7 @@ where
// Iterate over the outputs of the current query
// and remove elements from `old_outputs` when we find them
for new_output in revisions.origin.outputs() {
for new_output in revisions.origin.as_ref().outputs() {
old_outputs.swap_remove(&(
new_output.ingredient_index(),
new_output.key_index().index(),

View file

@ -1,15 +1,15 @@
use crate::function::{Configuration, IngredientImpl};
use crate::zalsa::Zalsa;
use crate::zalsa_local::QueryOrigin;
use crate::zalsa_local::QueryOriginRef;
use crate::Id;
impl<C> IngredientImpl<C>
where
C: Configuration,
{
pub(super) fn origin(&self, zalsa: &Zalsa, key: Id) -> Option<QueryOrigin> {
pub(super) fn origin<'db>(&self, zalsa: &'db Zalsa, key: Id) -> Option<QueryOriginRef<'db>> {
let memo_ingredient_index = self.memo_ingredient_index(zalsa, key);
self.get_memo_from_table_for(zalsa, key, memo_ingredient_index)
.map(|m| m.revisions.origin.clone())
.map(|m| m.revisions.origin.as_ref())
}
}

View file

@ -7,7 +7,7 @@ use crate::key::DatabaseKeyIndex;
use crate::plumbing::ZalsaLocal;
use crate::sync::atomic::Ordering;
use crate::zalsa::{MemoIngredientIndex, Zalsa, ZalsaDatabase};
use crate::zalsa_local::{QueryEdge, QueryOrigin};
use crate::zalsa_local::{QueryEdge, QueryOriginRef};
use crate::{AsDynDatabase as _, Id, Revision};
/// Result of memo validation.
@ -353,8 +353,8 @@ where
return VerifyResult::unchanged();
}
match &old_memo.revisions.origin {
QueryOrigin::Assigned(_) => {
match old_memo.revisions.origin.as_ref() {
QueryOriginRef::Assigned(_) => {
// If the value was assigned by another query,
// and that query were up-to-date,
// then we would have updated the `verified_at` field already.
@ -372,15 +372,15 @@ where
// when we hit the cycle. Any dependencies accessed when creating the fixpoint initial
// are tracked by the outer query. Nothing should have changed assuming that the
// fixpoint initial function is deterministic.
QueryOrigin::FixpointInitial => {
QueryOriginRef::FixpointInitial => {
cycle_heads.push_initial(database_key_index);
VerifyResult::unchanged()
}
QueryOrigin::DerivedUntracked(_) => {
QueryOriginRef::DerivedUntracked(_) => {
// Untracked inputs? Have to assume that it changed.
VerifyResult::Changed
}
QueryOrigin::Derived(edges) => {
QueryOriginRef::Derived(edges) => {
let is_provisional = old_memo.may_be_provisional();
// If the value is from the same revision but is still provisional, consider it changed
@ -398,7 +398,7 @@ where
// they executed. It's possible that if the value of some input I0 is no longer
// valid, then some later input I1 might never have executed at all, so verifying
// it is still up to date is meaningless.
for &edge in edges.input_outputs.iter() {
for &edge in edges {
match edge {
QueryEdge::Input(dependency_index) => {
match dependency_index.maybe_changed_after(

View file

@ -10,7 +10,7 @@ use crate::revision::AtomicRevision;
use crate::sync::atomic::Ordering;
use crate::table::memo::MemoTableWithTypesMut;
use crate::zalsa::{MemoIngredientIndex, Zalsa};
use crate::zalsa_local::{QueryOrigin, QueryRevisions};
use crate::zalsa_local::{QueryOriginRef, QueryRevisions};
use crate::{Event, EventKind, Id, Revision};
impl<C: Configuration> IngredientImpl<C> {
@ -64,16 +64,16 @@ impl<C: Configuration> IngredientImpl<C> {
memo_ingredient_index: MemoIngredientIndex,
) {
let map = |memo: &mut Memo<C::Output<'static>>| {
match &memo.revisions.origin {
QueryOrigin::Assigned(_)
| QueryOrigin::DerivedUntracked(_)
| QueryOrigin::FixpointInitial => {
match memo.revisions.origin.as_ref() {
QueryOriginRef::Assigned(_)
| QueryOriginRef::DerivedUntracked(_)
| QueryOriginRef::FixpointInitial => {
// Careful: Cannot evict memos whose values were
// assigned as output of another query
// or those with untracked inputs
// as their values cannot be reconstructed.
}
QueryOrigin::Derived(_) => {
QueryOriginRef::Derived(_) => {
// Set the memo value to `None`.
memo.value = None;
}
@ -101,7 +101,7 @@ pub struct Memo<V> {
#[cfg(not(feature = "shuttle"))]
#[cfg(target_pointer_width = "64")]
const _: [(); std::mem::size_of::<Memo<std::num::NonZeroUsize>>()] =
[(); std::mem::size_of::<[usize; 13]>()];
[(); std::mem::size_of::<[usize; 11]>()];
impl<V> Memo<V> {
pub(super) fn new(value: Option<V>, revision_now: Revision, revisions: QueryRevisions) -> Self {
@ -230,7 +230,7 @@ impl<V> Memo<V> {
zalsa: &Zalsa,
database_key_index: DatabaseKeyIndex,
) {
for output in self.revisions.origin.outputs() {
for output in self.revisions.origin.as_ref().outputs() {
output.mark_validated_output(zalsa, database_key_index);
}
}
@ -262,7 +262,7 @@ impl<V> Memo<V> {
}
impl<V: Send + Sync + Any> crate::table::memo::Memo for Memo<V> {
fn origin(&self) -> &QueryOrigin {
&self.revisions.origin
fn origin(&self) -> QueryOriginRef<'_> {
self.revisions.origin.as_ref()
}
}

View file

@ -5,7 +5,7 @@ use crate::revision::AtomicRevision;
use crate::sync::atomic::AtomicBool;
use crate::tracked_struct::TrackedStructInDb;
use crate::zalsa::{Zalsa, ZalsaDatabase};
use crate::zalsa_local::{QueryOrigin, QueryRevisions};
use crate::zalsa_local::{QueryOrigin, QueryOriginRef, QueryRevisions};
use crate::{DatabaseKeyIndex, Id};
impl<C> IngredientImpl<C>
@ -65,7 +65,7 @@ where
let mut revisions = QueryRevisions {
changed_at: current_deps.changed_at,
durability: current_deps.durability,
origin: QueryOrigin::Assigned(active_query_key),
origin: QueryOrigin::assigned(active_query_key),
tracked_struct_ids: Default::default(),
accumulated: Default::default(),
accumulated_inputs: Default::default(),
@ -116,8 +116,8 @@ where
// If we are marking this as validated, it must be a value that was
// assigned by `executor`.
match memo.revisions.origin {
QueryOrigin::Assigned(by_query) => assert_eq!(by_query, executor),
match memo.revisions.origin.as_ref() {
QueryOriginRef::Assigned(by_query) => assert_eq!(by_query, executor),
_ => panic!(
"expected a query assigned by `{:?}`, not `{:?}`",
executor, memo.revisions.origin,

View file

@ -9,7 +9,7 @@ use crate::sync::Arc;
use crate::table::memo::MemoTableTypes;
use crate::table::Table;
use crate::zalsa::{transmute_data_mut_ptr, transmute_data_ptr, IngredientIndex, Zalsa};
use crate::zalsa_local::QueryOrigin;
use crate::zalsa_local::QueryOriginRef;
use crate::{Database, DatabaseKeyIndex, Id, Revision};
/// A "jar" is a group of ingredients that are added atomically.
@ -150,7 +150,7 @@ pub trait Ingredient: Any + std::fmt::Debug + Send + Sync {
}
/// What were the inputs (if any) that were used to create the value at `key_index`.
fn origin(&self, zalsa: &Zalsa, key_index: Id) -> Option<QueryOrigin> {
fn origin<'db>(&self, zalsa: &'db Zalsa, key_index: Id) -> Option<QueryOriginRef<'db>> {
let _ = (zalsa, key_index);
unreachable!("only function ingredients have origins")
}

View file

@ -77,7 +77,7 @@ impl NewMemoIngredientIndices for MemoIngredientIndices {
};
let mut indices = Vec::new();
indices.resize(
last.as_usize() + 1,
(last.as_u32() as usize) + 1,
MemoIngredientIndex::from_usize((u32::MAX - 1) as usize),
);
for &struct_ingredient in &struct_indices.indices {
@ -88,7 +88,7 @@ impl NewMemoIngredientIndices for MemoIngredientIndices {
let mi = zalsa.next_memo_ingredient_index(struct_ingredient, ingredient);
memo_types.set(mi, &memo_type);
indices[struct_ingredient.as_usize()] = mi;
indices[struct_ingredient.as_u32() as usize] = mi;
}
MemoIngredientIndices {
indices: indices.into_boxed_slice(),
@ -125,7 +125,7 @@ impl MemoIngredientMap for MemoIngredientIndices {
}
#[inline(always)]
fn get(&self, index: IngredientIndex) -> MemoIngredientIndex {
self.indices[index.as_usize()]
self.indices[index.as_u32() as usize]
}
}

View file

@ -9,7 +9,7 @@ use thin_vec::ThinVec;
use crate::sync::atomic::{AtomicPtr, Ordering};
use crate::sync::{OnceLock, RwLock};
use crate::{zalsa::MemoIngredientIndex, zalsa_local::QueryOrigin};
use crate::{zalsa::MemoIngredientIndex, zalsa_local::QueryOriginRef};
/// The "memo table" stores the memoized results of tracked function calls.
/// Every tracked function must take a salsa struct as its first argument
@ -21,7 +21,7 @@ pub(crate) struct MemoTable {
pub trait Memo: Any + Send + Sync {
/// Returns the `origin` of this memo
fn origin(&self) -> &QueryOrigin;
fn origin(&self) -> QueryOriginRef<'_>;
}
/// Data for a memoized entry.
@ -103,7 +103,7 @@ impl MemoEntryType {
struct DummyMemo {}
impl Memo for DummyMemo {
fn origin(&self) -> &QueryOrigin {
fn origin(&self) -> QueryOriginRef<'_> {
unreachable!("should not get here")
}
}

View file

@ -447,7 +447,7 @@ where
// 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",
"leaking tracked struct {:?} due to generation overflow",
self.database_key_index(id)
);
@ -547,7 +547,7 @@ where
// the previous slot and allocate a new value.
if id.generation() == u32::MAX {
tracing::info!(
"leaking tracked struct {:?} due to generation oveflow",
"leaking tracked struct {:?} due to generation overflow",
self.database_key_index(id)
);

View file

@ -83,9 +83,9 @@ impl IngredientIndex {
Self(v as u32)
}
/// Convert the ingredient index back into a usize.
pub(crate) fn as_usize(self) -> usize {
self.0 as usize
/// Convert the ingredient index back into a `u32`.
pub(crate) fn as_u32(self) -> u32 {
self.0
}
pub fn successor(self, index: usize) -> Self {
@ -202,7 +202,7 @@ impl Zalsa {
#[inline]
pub(crate) fn lookup_ingredient(&self, index: IngredientIndex) -> &dyn Ingredient {
let index = index.as_usize();
let index = index.as_u32() as usize;
self.ingredients_vec
.get(index)
.unwrap_or_else(|| panic!("index `{index}` is uninitialized"))
@ -214,7 +214,7 @@ impl Zalsa {
struct_ingredient_index: IngredientIndex,
memo_ingredient_index: MemoIngredientIndex,
) -> IngredientIndex {
self.memo_ingredient_indices.read()[struct_ingredient_index.as_usize()]
self.memo_ingredient_indices.read()[struct_ingredient_index.as_u32() as usize]
[memo_ingredient_index.as_usize()]
}
@ -240,7 +240,7 @@ impl Zalsa {
ingredient_index: IngredientIndex,
) -> MemoIngredientIndex {
let mut memo_ingredients = self.memo_ingredient_indices.write();
let idx = struct_ingredient_index.as_usize();
let idx = struct_ingredient_index.as_u32() as usize;
let memo_ingredients = if let Some(memo_ingredients) = memo_ingredients.get_mut(idx) {
memo_ingredients
} else {
@ -305,7 +305,7 @@ impl Zalsa {
let actual_index = self.ingredients_vec.push(ingredient);
assert_eq!(
expected_index.as_usize(),
expected_index.as_u32() as usize,
actual_index,
"ingredient `{:?}` was predicted to have index `{:?}` but actually has index `{:?}`",
self.ingredients_vec[actual_index],
@ -328,7 +328,7 @@ impl Zalsa {
&mut self,
index: IngredientIndex,
) -> (&mut dyn Ingredient, &mut Runtime) {
let index = index.as_usize();
let index = index.as_u32() as usize;
let ingredient = self
.ingredients_vec
.get_mut(index)
@ -358,7 +358,7 @@ impl Zalsa {
let _span = tracing::debug_span!("new_revision", ?new_revision).entered();
for (_, index) in self.ingredients_requiring_reset.iter() {
let index = index.as_usize();
let index = index.as_u32() as usize;
let ingredient = self
.ingredients_vec
.get_mut(index)
@ -375,7 +375,7 @@ impl Zalsa {
pub fn evict_lru(&mut self) {
let _span = tracing::debug_span!("evict_lru").entered();
for (_, index) in self.ingredients_requiring_reset.iter() {
let index = index.as_usize();
let index = index.as_u32() as usize;
self.ingredients_vec
.get_mut(index)
.unwrap_or_else(|| panic!("index `{index}` is uninitialized"))

View file

@ -1,5 +1,6 @@
use std::cell::RefCell;
use std::panic::UnwindSafe;
use std::ptr::{self, NonNull};
use rustc_hash::FxHashMap;
use tracing::debug;
@ -366,12 +367,16 @@ pub(crate) struct QueryRevisions {
pub(super) cycle_heads: CycleHeads,
}
#[cfg(not(feature = "shuttle"))]
#[cfg(target_pointer_width = "64")]
const _: [(); std::mem::size_of::<QueryRevisions>()] = [(); std::mem::size_of::<[usize; 9]>()];
impl QueryRevisions {
pub(crate) fn fixpoint_initial(query: DatabaseKeyIndex) -> Self {
Self {
changed_at: Revision::start(),
durability: Durability::MAX,
origin: QueryOrigin::FixpointInitial,
origin: QueryOrigin::fixpoint_initial(),
tracked_struct_ids: Default::default(),
accumulated: Default::default(),
accumulated_inputs: Default::default(),
@ -380,10 +385,11 @@ impl QueryRevisions {
}
}
}
/// Tracks the way that a memoized value for a query was created.
///
/// This is a read-only reference to a `PackedQueryOrigin`.
#[derive(Debug, Clone)]
pub enum QueryOrigin {
pub enum QueryOriginRef<'a> {
/// The value was assigned as the output of another query (e.g., using `specify`).
/// The `DatabaseKeyIndex` is the identity of the assigning query.
Assigned(DatabaseKeyIndex),
@ -391,69 +397,244 @@ pub enum QueryOrigin {
/// The value was derived by executing a function
/// and we were able to track ALL of that function's inputs.
/// Those inputs are described in [`QueryEdges`].
Derived(QueryEdges),
Derived(&'a [QueryEdge]),
/// The value was derived by executing a function
/// but that function also reported that it read untracked inputs.
/// The [`QueryEdges`] argument contains a listing of all the inputs we saw
/// (but we know there were more).
DerivedUntracked(QueryEdges),
DerivedUntracked(&'a [QueryEdge]),
/// The value is an initial provisional value for a query that supports fixpoint iteration.
FixpointInitial,
}
impl QueryOrigin {
impl<'a> QueryOriginRef<'a> {
/// Indices for queries *read* by this query
pub(crate) fn inputs(&self) -> impl DoubleEndedIterator<Item = DatabaseKeyIndex> + '_ {
let opt_edges = match self {
QueryOrigin::Derived(edges) | QueryOrigin::DerivedUntracked(edges) => Some(edges),
QueryOrigin::Assigned(_) | QueryOrigin::FixpointInitial => None,
QueryOriginRef::Derived(edges) | QueryOriginRef::DerivedUntracked(edges) => Some(edges),
QueryOriginRef::Assigned(_) | QueryOriginRef::FixpointInitial => None,
};
opt_edges.into_iter().flat_map(|edges| edges.inputs())
opt_edges.into_iter().flat_map(|edges| input_edges(edges))
}
/// Indices for queries *written* by this query (if any)
pub(crate) fn outputs(&self) -> impl DoubleEndedIterator<Item = DatabaseKeyIndex> + '_ {
let opt_edges = match self {
QueryOrigin::Derived(edges) | QueryOrigin::DerivedUntracked(edges) => Some(edges),
QueryOrigin::Assigned(_) | QueryOrigin::FixpointInitial => None,
QueryOriginRef::Derived(edges) | QueryOriginRef::DerivedUntracked(edges) => Some(edges),
QueryOriginRef::Assigned(_) | QueryOriginRef::FixpointInitial => None,
};
opt_edges.into_iter().flat_map(|edges| edges.outputs())
opt_edges.into_iter().flat_map(|edges| output_edges(edges))
}
pub(crate) fn edges(&self) -> &[QueryEdge] {
pub(crate) fn edges(&self) -> &'a [QueryEdge] {
let opt_edges = match self {
QueryOrigin::Derived(edges) | QueryOrigin::DerivedUntracked(edges) => Some(edges),
QueryOrigin::Assigned(_) | QueryOrigin::FixpointInitial => None,
QueryOriginRef::Derived(edges) | QueryOriginRef::DerivedUntracked(edges) => Some(edges),
QueryOriginRef::Assigned(_) | QueryOriginRef::FixpointInitial => None,
};
opt_edges
.map(|edges| &*edges.input_outputs)
.unwrap_or_default()
opt_edges.copied().unwrap_or_default()
}
}
/// The edges between a memoized value and other queries in the dependency graph.
/// These edges include both dependency edges
/// e.g., when creating the memoized value for Q0 executed another function Q1)
/// and output edges
/// (e.g., when Q0 specified the value for another query Q2).
#[derive(Debug, Clone)]
pub struct QueryEdges {
/// The list of outgoing edges from this node.
/// This list combines *both* inputs and outputs.
#[derive(Clone, Copy)]
enum QueryOriginKind {
/// The value was assigned as the output of another query.
///
/// This can, for example, can occur when `specify` is used.
Assigned,
/// The value was derived by executing a function
/// _and_ Salsa was able to track all of said function's inputs.
Derived,
/// The value was derived by executing a function
/// but that function also reported that it read untracked inputs.
DerivedUntracked,
/// An initial provisional value.
///
/// This will occur occur in queries that support fixpoint iteration.
FixpointInitial,
}
/// Tracks how a memoized value for a given query was created.
///
/// This type is a manual enum packed to 13 bytes to reduce the size of `QueryRevisions`.
#[repr(Rust, packed)]
pub struct QueryOrigin {
/// The tag of this enum.
///
/// Note that this tag only requires two bits and could likely be packed into
/// some other field. However, we get this byte for free thanks to alignment.
kind: QueryOriginKind,
/// The data portion of this enum.
data: QueryOriginData,
/// The metadata of this enum.
///
/// For `QueryOriginKind::Derived` and `QueryOriginKind::DerivedUntracked`, this
/// is the length of the `input_outputs` allocation.
///
/// For `QueryOriginKind::Assigned`, this is the `IngredientIndex` of assigning query.
/// Combined with the `Id` data, this forms a complete `DatabaseKeyIndex`.
///
/// For `QueryOriginKind::FixpointInitial`, this field is zero.
metadata: u32,
}
/// The data portion of `PackedQueryOrigin`.
union QueryOriginData {
/// Query edges for `QueryOriginKind::Derived` or `QueryOriginKind::DerivedUntracked`.
///
/// The query edges are between a memoized value and other queries in the dependency graph,
/// including both dependency edges (e.g., when creating the memoized value for Q0
/// executed another function Q1) and output edges (e.g., when Q0 specified the value
/// for another query Q2).
///
/// Note that we always track input dependencies even when there are untracked reads.
/// Untracked reads mean that we can't verify values, so we don't use the list of inputs for that,
/// but we still use it for finding the transitive inputs to an accumulator.
/// Untracked reads mean that Salsa can't verify values, so the list of inputs is unused.
/// However, Salsa still uses these edges to find the transitive inputs to an accumulator.
///
/// You can access the input/output list via the methods [`inputs`] and [`outputs`] respectively.
///
/// Important:
///
/// * The inputs must be in **execution order** for the red-green algorithm to work.
// pub input_outputs: ThinBox<[DependencyEdge]>, once that is a thing
pub input_outputs: Box<[QueryEdge]>,
input_outputs: NonNull<QueryEdge>,
/// The identity of the assigning query for `QueryOriginKind::Assigned`.
index: Id,
/// `QueryOriginKind::FixpointInitial` holds no data.
empty: (),
}
/// SAFETY: The `input_outputs` pointer is owned and not accessed or shared concurrently.
unsafe impl Send for QueryOriginData {}
/// SAFETY: Same as above.
unsafe impl Sync for QueryOriginData {}
impl QueryOrigin {
/// Create a query origin of type `QueryOriginKind::FixpointInitial`.
pub fn fixpoint_initial() -> QueryOrigin {
QueryOrigin {
kind: QueryOriginKind::FixpointInitial,
metadata: 0,
data: QueryOriginData { empty: () },
}
}
/// Create a query origin of type `QueryOriginKind::Derived`, with the given edges.
pub fn derived(input_outputs: impl IntoIterator<Item = QueryEdge>) -> QueryOrigin {
let input_outputs = input_outputs.into_iter().collect::<Box<[_]>>();
// Exceeding `u32::MAX` query edges should never happen in real-world usage.
let length = u32::try_from(input_outputs.len())
.expect("exceeded more than `u32::MAX` query edges; this should never happen.");
// SAFETY: `Box::into_raw` returns a non-null pointer.
let input_outputs =
unsafe { NonNull::new_unchecked(Box::into_raw(input_outputs).cast::<QueryEdge>()) };
QueryOrigin {
kind: QueryOriginKind::Derived,
metadata: length,
data: QueryOriginData { input_outputs },
}
}
/// Create a query origin of type `QueryOriginKind::DerivedUntracked`, with the given edges.
pub fn derived_untracked(input_outputs: impl IntoIterator<Item = QueryEdge>) -> QueryOrigin {
let mut origin = QueryOrigin::derived(input_outputs);
origin.kind = QueryOriginKind::DerivedUntracked;
origin
}
/// Create a query origin of type `QueryOriginKind::Assigned`, with the given key.
pub fn assigned(key: DatabaseKeyIndex) -> QueryOrigin {
QueryOrigin {
kind: QueryOriginKind::Assigned,
metadata: key.ingredient_index().as_u32(),
data: QueryOriginData {
index: key.key_index(),
},
}
}
/// Return a read-only reference to this query origin.
pub fn as_ref(&self) -> QueryOriginRef<'_> {
match self.kind {
QueryOriginKind::Assigned => {
// SAFETY: `data.index` is initialized when the tag is `QueryOriginKind::Assigned`.
let index = unsafe { self.data.index };
let ingredient_index = IngredientIndex::from(self.metadata as usize);
QueryOriginRef::Assigned(DatabaseKeyIndex::new(ingredient_index, index))
}
QueryOriginKind::Derived => {
// SAFETY: `data.input_outputs` is initialized when the tag is `QueryOriginKind::Derived`.
let input_outputs = unsafe { self.data.input_outputs };
let length = self.metadata as usize;
// SAFETY: `input_outputs` and `self.metadata` form a valid slice when the
// tag is `QueryOriginKind::Derived`.
let input_outputs =
unsafe { std::slice::from_raw_parts(input_outputs.as_ptr(), length) };
QueryOriginRef::Derived(input_outputs)
}
QueryOriginKind::DerivedUntracked => {
// SAFETY: `data.input_outputs` is initialized when the tag is `QueryOriginKind::DerivedUntracked`.
let input_outputs = unsafe { self.data.input_outputs };
let length = self.metadata as usize;
// SAFETY: `input_outputs` and `self.metadata` form a valid slice when the
// tag is `QueryOriginKind::DerivedUntracked`.
let input_outputs =
unsafe { std::slice::from_raw_parts(input_outputs.as_ptr(), length) };
QueryOriginRef::DerivedUntracked(input_outputs)
}
QueryOriginKind::FixpointInitial => QueryOriginRef::FixpointInitial,
}
}
}
impl Drop for QueryOrigin {
fn drop(&mut self) {
match self.kind {
QueryOriginKind::Derived | QueryOriginKind::DerivedUntracked => {
// SAFETY: `data.input_outputs` is initialized when the tag is `QueryOriginKind::Derived`
// or `QueryOriginKind::DerivedUntracked`.
let input_outputs = unsafe { self.data.input_outputs };
let length = self.metadata as usize;
// SAFETY: `input_outputs` and `self.metadata` form a valid slice when the
// tag is `QueryOriginKind::DerivedUntracked` or `QueryOriginKind::DerivedUntracked`,
// and we have `&mut self`.
let _input_outputs: Box<[QueryEdge]> = unsafe {
Box::from_raw(ptr::slice_from_raw_parts_mut(
input_outputs.as_ptr(),
length,
))
};
}
// The data stored for this variants is `Copy`.
QueryOriginKind::FixpointInitial | QueryOriginKind::Assigned => {}
}
}
}
impl std::fmt::Debug for QueryOrigin {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
self.as_ref().fmt(f)
}
}
#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)]
@ -462,33 +643,28 @@ pub enum QueryEdge {
Output(DatabaseKeyIndex),
}
impl QueryEdges {
/// Returns the (tracked) inputs that were executed in computing this memoized value.
///
/// These will always be in execution order.
pub(crate) fn inputs(&self) -> impl DoubleEndedIterator<Item = DatabaseKeyIndex> + '_ {
self.input_outputs.iter().filter_map(|&edge| match edge {
QueryEdge::Input(dependency_index) => Some(dependency_index),
QueryEdge::Output(_) => None,
})
}
/// Returns the (tracked) inputs that were executed in computing this memoized value.
///
/// These will always be in execution order.
pub(crate) fn input_edges(
input_outputs: &[QueryEdge],
) -> impl DoubleEndedIterator<Item = DatabaseKeyIndex> + '_ {
input_outputs.iter().filter_map(|&edge| match edge {
QueryEdge::Input(dependency_index) => Some(dependency_index),
QueryEdge::Output(_) => None,
})
}
/// Returns the (tracked) outputs that were executed in computing this memoized value.
///
/// These will always be in execution order.
pub(crate) fn outputs(&self) -> impl DoubleEndedIterator<Item = DatabaseKeyIndex> + '_ {
self.input_outputs.iter().filter_map(|&edge| match edge {
QueryEdge::Output(dependency_index) => Some(dependency_index),
QueryEdge::Input(_) => None,
})
}
/// Creates a new `QueryEdges`; the values given for each field must meet struct invariants.
pub(crate) fn new(input_outputs: impl IntoIterator<Item = QueryEdge>) -> Self {
Self {
input_outputs: input_outputs.into_iter().collect(),
}
}
/// Returns the (tracked) outputs that were executed in computing this memoized value.
///
/// These will always be in execution order.
pub(crate) fn output_edges(
input_outputs: &[QueryEdge],
) -> impl DoubleEndedIterator<Item = DatabaseKeyIndex> + '_ {
input_outputs.iter().filter_map(|&edge| match edge {
QueryEdge::Output(dependency_index) => Some(dependency_index),
QueryEdge::Input(_) => None,
})
}
/// When a query is pushed onto the `active_query` stack, this guard
@ -520,8 +696,11 @@ impl ActiveQueryGuard<'_> {
pub(crate) fn seed_iteration(&self, previous: &QueryRevisions) {
let durability = previous.durability;
let changed_at = previous.changed_at;
let edges = previous.origin.edges();
let untracked_read = matches!(previous.origin, QueryOrigin::DerivedUntracked(_));
let edges = previous.origin.as_ref().edges();
let untracked_read = matches!(
previous.origin.as_ref(),
QueryOriginRef::DerivedUntracked(_)
);
self.local_state.with_query_stack_mut(|stack| {
#[cfg(debug_assertions)]