about summary refs log tree commit diff
path: root/compiler/rustc_query_system/src
diff options
context:
space:
mode:
Diffstat (limited to 'compiler/rustc_query_system/src')
-rw-r--r--compiler/rustc_query_system/src/dep_graph/graph.rs52
-rw-r--r--compiler/rustc_query_system/src/dep_graph/serialized.rs5
-rw-r--r--compiler/rustc_query_system/src/query/job.rs2
-rw-r--r--compiler/rustc_query_system/src/query/plumbing.rs365
4 files changed, 199 insertions, 225 deletions
diff --git a/compiler/rustc_query_system/src/dep_graph/graph.rs b/compiler/rustc_query_system/src/dep_graph/graph.rs
index 3dbcc4d2e8a..2ff7de8cb9e 100644
--- a/compiler/rustc_query_system/src/dep_graph/graph.rs
+++ b/compiler/rustc_query_system/src/dep_graph/graph.rs
@@ -535,16 +535,22 @@ impl<K: DepKind> DepGraph<K> {
             // value to an existing node.
             //
             // For sanity, we still check that the loaded stable hash and the new one match.
-            if let Some(dep_node_index) = data.dep_node_index_of_opt(&node) {
-                let _current_fingerprint =
-                    crate::query::incremental_verify_ich(cx, data, result, &node, hash_result);
+            if let Some(prev_index) = data.previous.node_to_index_opt(&node) {
+                let dep_node_index = data.current.prev_index_to_index.lock()[prev_index];
+                if let Some(dep_node_index) = dep_node_index {
+                    crate::query::incremental_verify_ich(cx, data, result, prev_index, hash_result);
 
-                #[cfg(debug_assertions)]
-                if hash_result.is_some() {
-                    data.current.record_edge(dep_node_index, node, _current_fingerprint);
-                }
+                    #[cfg(debug_assertions)]
+                    if hash_result.is_some() {
+                        data.current.record_edge(
+                            dep_node_index,
+                            node,
+                            data.prev_fingerprint_of(prev_index),
+                        );
+                    }
 
-                return dep_node_index;
+                    return dep_node_index;
+                }
             }
 
             let mut edges = SmallVec::new();
@@ -626,13 +632,19 @@ impl<K: DepKind> DepGraphData<K> {
 
     /// Returns true if the given node has been marked as green during the
     /// current compilation session. Used in various assertions
-    pub fn is_green(&self, dep_node: &DepNode<K>) -> bool {
-        self.node_color(dep_node).map_or(false, |c| c.is_green())
+    #[inline]
+    pub fn is_index_green(&self, prev_index: SerializedDepNodeIndex) -> bool {
+        self.colors.get(prev_index).map_or(false, |c| c.is_green())
+    }
+
+    #[inline]
+    pub fn prev_fingerprint_of(&self, prev_index: SerializedDepNodeIndex) -> Fingerprint {
+        self.previous.fingerprint_by_index(prev_index)
     }
 
     #[inline]
-    pub fn prev_fingerprint_of(&self, dep_node: &DepNode<K>) -> Option<Fingerprint> {
-        self.previous.fingerprint_of(dep_node)
+    pub fn prev_node_of(&self, prev_index: SerializedDepNodeIndex) -> DepNode<K> {
+        self.previous.index_to_node(prev_index)
     }
 
     pub fn mark_debug_loaded_from_disk(&self, dep_node: DepNode<K>) {
@@ -643,7 +655,7 @@ impl<K: DepKind> DepGraphData<K> {
 impl<K: DepKind> DepGraph<K> {
     #[inline]
     pub fn dep_node_exists(&self, dep_node: &DepNode<K>) -> bool {
-        self.data.as_ref().and_then(|data| data.dep_node_index_of_opt(dep_node)).is_some()
+        self.data.as_ref().map_or(false, |data| data.dep_node_exists(dep_node))
     }
 
     /// Checks whether a previous work product exists for `v` and, if
@@ -1053,7 +1065,7 @@ pub(super) struct CurrentDepGraph<K: DepKind> {
     /// This is used to verify that fingerprints do not change between the creation of a node
     /// and its recomputation.
     #[cfg(debug_assertions)]
-    fingerprints: Lock<FxHashMap<DepNode<K>, Fingerprint>>,
+    fingerprints: Lock<IndexVec<DepNodeIndex, Option<Fingerprint>>>,
 
     /// Used to trap when a specific edge is added to the graph.
     /// This is used for debug purposes and is only active with `debug_assertions`.
@@ -1139,7 +1151,7 @@ impl<K: DepKind> CurrentDepGraph<K> {
             #[cfg(debug_assertions)]
             forbidden_edge,
             #[cfg(debug_assertions)]
-            fingerprints: Lock::new(Default::default()),
+            fingerprints: Lock::new(IndexVec::from_elem_n(None, new_node_count_estimate)),
             total_read_count: AtomicU64::new(0),
             total_duplicate_read_count: AtomicU64::new(0),
             node_intern_event_id,
@@ -1151,14 +1163,8 @@ impl<K: DepKind> CurrentDepGraph<K> {
         if let Some(forbidden_edge) = &self.forbidden_edge {
             forbidden_edge.index_to_node.lock().insert(dep_node_index, key);
         }
-        match self.fingerprints.lock().entry(key) {
-            Entry::Vacant(v) => {
-                v.insert(fingerprint);
-            }
-            Entry::Occupied(o) => {
-                assert_eq!(*o.get(), fingerprint, "Unstable fingerprints for {:?}", key);
-            }
-        }
+        let previous = *self.fingerprints.lock().get_or_insert_with(dep_node_index, || fingerprint);
+        assert_eq!(previous, fingerprint, "Unstable fingerprints for {:?}", key);
     }
 
     /// Writes the node to the current dep-graph and allocates a `DepNodeIndex` for it.
diff --git a/compiler/rustc_query_system/src/dep_graph/serialized.rs b/compiler/rustc_query_system/src/dep_graph/serialized.rs
index 29513df460f..3d19a84915a 100644
--- a/compiler/rustc_query_system/src/dep_graph/serialized.rs
+++ b/compiler/rustc_query_system/src/dep_graph/serialized.rs
@@ -80,11 +80,6 @@ impl<K: DepKind> SerializedDepGraph<K> {
     }
 
     #[inline]
-    pub fn fingerprint_of(&self, dep_node: &DepNode<K>) -> Option<Fingerprint> {
-        self.index.get(dep_node).map(|&node_index| self.fingerprints[node_index])
-    }
-
-    #[inline]
     pub fn fingerprint_by_index(&self, dep_node_index: SerializedDepNodeIndex) -> Fingerprint {
         self.fingerprints[dep_node_index]
     }
diff --git a/compiler/rustc_query_system/src/query/job.rs b/compiler/rustc_query_system/src/query/job.rs
index d56a5955aff..a534b54070c 100644
--- a/compiler/rustc_query_system/src/query/job.rs
+++ b/compiler/rustc_query_system/src/query/job.rs
@@ -124,8 +124,6 @@ impl<D: DepKind> QueryJob<D> {
 }
 
 impl QueryJobId {
-    #[cold]
-    #[inline(never)]
     #[cfg(not(parallel_compiler))]
     pub(super) fn find_cycle_in_stack<D: DepKind>(
         &self,
diff --git a/compiler/rustc_query_system/src/query/plumbing.rs b/compiler/rustc_query_system/src/query/plumbing.rs
index ba2f859ff0f..519ea5ffed1 100644
--- a/compiler/rustc_query_system/src/query/plumbing.rs
+++ b/compiler/rustc_query_system/src/query/plumbing.rs
@@ -6,20 +6,20 @@ use crate::dep_graph::{DepContext, DepKind, DepNode, DepNodeIndex, DepNodeParams
 use crate::dep_graph::{DepGraphData, HasDepContext};
 use crate::ich::StableHashingContext;
 use crate::query::caches::QueryCache;
+#[cfg(parallel_compiler)]
+use crate::query::job::QueryLatch;
 use crate::query::job::{report_cycle, QueryInfo, QueryJob, QueryJobId, QueryJobInfo};
+use crate::query::SerializedDepNodeIndex;
 use crate::query::{QueryContext, QueryMap, QuerySideEffects, QueryStackFrame};
 use crate::values::Value;
 use crate::HandleCycleError;
 use rustc_data_structures::fingerprint::Fingerprint;
 use rustc_data_structures::fx::FxHashMap;
-#[cfg(parallel_compiler)]
-use rustc_data_structures::profiling::TimingGuard;
-#[cfg(parallel_compiler)]
-use rustc_data_structures::sharded::Sharded;
 use rustc_data_structures::stack::ensure_sufficient_stack;
-use rustc_data_structures::sync::{Lock, LockGuard};
+use rustc_data_structures::sync::Lock;
+#[cfg(parallel_compiler)]
+use rustc_data_structures::{cold_path, sharded::Sharded};
 use rustc_errors::{DiagnosticBuilder, ErrorGuaranteed, FatalError};
-use rustc_session::Session;
 use rustc_span::{Span, DUMMY_SP};
 use std::cell::Cell;
 use std::collections::hash_map::Entry;
@@ -116,7 +116,6 @@ where
 {
     state: &'tcx QueryState<K, D>,
     key: K,
-    id: QueryJobId,
 }
 
 #[cold]
@@ -166,81 +165,6 @@ impl<'tcx, K, D: DepKind> JobOwner<'tcx, K, D>
 where
     K: Eq + Hash + Copy,
 {
-    /// Either gets a `JobOwner` corresponding the query, allowing us to
-    /// start executing the query, or returns with the result of the query.
-    /// This function assumes that `try_get_cached` is already called and returned `lookup`.
-    /// If the query is executing elsewhere, this will wait for it and return the result.
-    /// If the query panicked, this will silently panic.
-    ///
-    /// This function is inlined because that results in a noticeable speed-up
-    /// for some compile-time benchmarks.
-    #[inline(always)]
-    fn try_start<'b, Qcx>(
-        qcx: &'b Qcx,
-        state: &'b QueryState<K, Qcx::DepKind>,
-        mut state_lock: LockGuard<'b, FxHashMap<K, QueryResult<Qcx::DepKind>>>,
-        span: Span,
-        key: K,
-    ) -> TryGetJob<'b, K, D>
-    where
-        Qcx: QueryContext + HasDepContext<DepKind = D>,
-    {
-        let lock = &mut *state_lock;
-        let current_job_id = qcx.current_query_job();
-
-        match lock.entry(key) {
-            Entry::Vacant(entry) => {
-                let id = qcx.next_job_id();
-                let job = QueryJob::new(id, span, current_job_id);
-
-                let key = *entry.key();
-                entry.insert(QueryResult::Started(job));
-
-                let owner = JobOwner { state, id, key };
-                return TryGetJob::NotYetStarted(owner);
-            }
-            Entry::Occupied(mut entry) => {
-                match entry.get_mut() {
-                    #[cfg(not(parallel_compiler))]
-                    QueryResult::Started(job) => {
-                        let id = job.id;
-                        drop(state_lock);
-
-                        // If we are single-threaded we know that we have cycle error,
-                        // so we just return the error.
-                        return TryGetJob::Cycle(id.find_cycle_in_stack(
-                            qcx.try_collect_active_jobs().unwrap(),
-                            &current_job_id,
-                            span,
-                        ));
-                    }
-                    #[cfg(parallel_compiler)]
-                    QueryResult::Started(job) => {
-                        // For parallel queries, we'll block and wait until the query running
-                        // in another thread has completed. Record how long we wait in the
-                        // self-profiler.
-                        let query_blocked_prof_timer = qcx.dep_context().profiler().query_blocked();
-
-                        // Get the latch out
-                        let latch = job.latch();
-
-                        drop(state_lock);
-
-                        // With parallel queries we might just have to wait on some other
-                        // thread.
-                        let result = latch.wait_on(current_job_id, span);
-
-                        match result {
-                            Ok(()) => TryGetJob::JobCompleted(query_blocked_prof_timer),
-                            Err(cycle) => TryGetJob::Cycle(cycle),
-                        }
-                    }
-                    QueryResult::Poisoned => FatalError.raise(),
-                }
-            }
-        }
-    }
-
     /// Completes the query by updating the query cache with the `result`,
     /// signals the waiter and forgets the JobOwner, so it won't poison the query
     fn complete<C>(self, cache: &C, result: C::Value, dep_node_index: DepNodeIndex)
@@ -307,25 +231,6 @@ pub(crate) struct CycleError<D: DepKind> {
     pub cycle: Vec<QueryInfo<D>>,
 }
 
-/// The result of `try_start`.
-enum TryGetJob<'tcx, K, D>
-where
-    K: Eq + Hash + Copy,
-    D: DepKind,
-{
-    /// The query is not yet started. Contains a guard to the cache eventually used to start it.
-    NotYetStarted(JobOwner<'tcx, K, D>),
-
-    /// The query was already completed.
-    /// Returns the result of the query and its dep-node index
-    /// if it succeeded or a cycle error if it failed.
-    #[cfg(parallel_compiler)]
-    JobCompleted(TimingGuard<'tcx>),
-
-    /// Trying to execute the query resulted in a cycle.
-    Cycle(CycleError<D>),
-}
-
 /// Checks if the query is already computed and in the cache.
 /// It returns the shard index and a lock guard to the shard,
 /// which will be used if the query is not in the cache and we need
@@ -346,6 +251,65 @@ where
     }
 }
 
+#[cold]
+#[inline(never)]
+#[cfg(not(parallel_compiler))]
+fn cycle_error<Q, Qcx>(
+    query: Q,
+    qcx: Qcx,
+    try_execute: QueryJobId,
+    span: Span,
+) -> (Q::Value, Option<DepNodeIndex>)
+where
+    Q: QueryConfig<Qcx>,
+    Qcx: QueryContext,
+{
+    let error = try_execute.find_cycle_in_stack(
+        qcx.try_collect_active_jobs().unwrap(),
+        &qcx.current_query_job(),
+        span,
+    );
+    (mk_cycle(qcx, error, query.handle_cycle_error()), None)
+}
+
+#[inline(always)]
+#[cfg(parallel_compiler)]
+fn wait_for_query<Q, Qcx>(
+    query: Q,
+    qcx: Qcx,
+    span: Span,
+    key: Q::Key,
+    latch: QueryLatch<Qcx::DepKind>,
+    current: Option<QueryJobId>,
+) -> (Q::Value, Option<DepNodeIndex>)
+where
+    Q: QueryConfig<Qcx>,
+    Qcx: QueryContext,
+{
+    // For parallel queries, we'll block and wait until the query running
+    // in another thread has completed. Record how long we wait in the
+    // self-profiler.
+    let query_blocked_prof_timer = qcx.dep_context().profiler().query_blocked();
+
+    // With parallel queries we might just have to wait on some other
+    // thread.
+    let result = latch.wait_on(current, span);
+
+    match result {
+        Ok(()) => {
+            let Some((v, index)) = query.query_cache(qcx).lookup(&key) else {
+                cold_path(|| panic!("value must be in cache after waiting"))
+            };
+
+            qcx.dep_context().profiler().query_cache_hit(index.into());
+            query_blocked_prof_timer.finish_with_query_invocation_id(index.into());
+
+            (v, Some(index))
+        }
+        Err(cycle) => (mk_cycle(qcx, cycle, query.handle_cycle_error()), None),
+    }
+}
+
 #[inline(never)]
 fn try_execute_query<Q, Qcx>(
     query: Q,
@@ -360,9 +324,9 @@ where
 {
     let state = query.query_state(qcx);
     #[cfg(parallel_compiler)]
-    let state_lock = state.active.get_shard_by_value(&key).lock();
+    let mut state_lock = state.active.get_shard_by_value(&key).lock();
     #[cfg(not(parallel_compiler))]
-    let state_lock = state.active.lock();
+    let mut state_lock = state.active.lock();
 
     // For the parallel compiler we need to check both the query cache and query state structures
     // while holding the state lock to ensure that 1) the query has not yet completed and 2) the
@@ -377,44 +341,82 @@ where
         }
     }
 
-    match JobOwner::<'_, Q::Key, Qcx::DepKind>::try_start(&qcx, state, state_lock, span, key) {
-        TryGetJob::NotYetStarted(job) => {
-            let (result, dep_node_index) = match qcx.dep_context().dep_graph().data() {
-                None => execute_job_non_incr(query, qcx, key, job.id),
-                Some(data) => execute_job_incr(query, qcx, data, key, dep_node, job.id),
-            };
+    let current_job_id = qcx.current_query_job();
+
+    match state_lock.entry(key) {
+        Entry::Vacant(entry) => {
+            // Nothing has computed or is computing the query, so we start a new job and insert it in the
+            // state map.
+            let id = qcx.next_job_id();
+            let job = QueryJob::new(id, span, current_job_id);
+            entry.insert(QueryResult::Started(job));
 
-            let cache = query.query_cache(qcx);
-            if query.feedable() {
-                // We should not compute queries that also got a value via feeding.
-                // This can't happen, as query feeding adds the very dependencies to the fed query
-                // as its feeding query had. So if the fed query is red, so is its feeder, which will
-                // get evaluated first, and re-feed the query.
-                if let Some((cached_result, _)) = cache.lookup(&key) {
-                    panic!(
-                        "fed query later has its value computed. The already cached value: {cached_result:?}"
-                    );
+            // Drop the lock before we start executing the query
+            drop(state_lock);
+
+            execute_job(query, qcx, state, key, id, dep_node)
+        }
+        Entry::Occupied(mut entry) => {
+            match entry.get_mut() {
+                #[cfg(not(parallel_compiler))]
+                QueryResult::Started(job) => {
+                    let id = job.id;
+                    drop(state_lock);
+
+                    // If we are single-threaded we know that we have cycle error,
+                    // so we just return the error.
+                    cycle_error(query, qcx, id, span)
+                }
+                #[cfg(parallel_compiler)]
+                QueryResult::Started(job) => {
+                    // Get the latch out
+                    let latch = job.latch();
+                    drop(state_lock);
+
+                    wait_for_query(query, qcx, span, key, latch, current_job_id)
                 }
+                QueryResult::Poisoned => FatalError.raise(),
             }
-            job.complete(cache, result, dep_node_index);
-            (result, Some(dep_node_index))
         }
-        TryGetJob::Cycle(error) => {
-            let result = mk_cycle(qcx, error, query.handle_cycle_error());
-            (result, None)
-        }
-        #[cfg(parallel_compiler)]
-        TryGetJob::JobCompleted(query_blocked_prof_timer) => {
-            let Some((v, index)) = query.query_cache(qcx).lookup(&key) else {
-                panic!("value must be in cache after waiting")
-            };
+    }
+}
 
-            qcx.dep_context().profiler().query_cache_hit(index.into());
-            query_blocked_prof_timer.finish_with_query_invocation_id(index.into());
+#[inline(always)]
+fn execute_job<Q, Qcx>(
+    query: Q,
+    qcx: Qcx,
+    state: &QueryState<Q::Key, Qcx::DepKind>,
+    key: Q::Key,
+    id: QueryJobId,
+    dep_node: Option<DepNode<Qcx::DepKind>>,
+) -> (Q::Value, Option<DepNodeIndex>)
+where
+    Q: QueryConfig<Qcx>,
+    Qcx: QueryContext,
+{
+    // Use `JobOwner` so the query will be poisoned if executing it panics.
+    let job_owner = JobOwner { state, key };
 
-            (v, Some(index))
+    let (result, dep_node_index) = match qcx.dep_context().dep_graph().data() {
+        None => execute_job_non_incr(query, qcx, key, id),
+        Some(data) => execute_job_incr(query, qcx, data, key, dep_node, id),
+    };
+
+    let cache = query.query_cache(qcx);
+    if query.feedable() {
+        // We should not compute queries that also got a value via feeding.
+        // This can't happen, as query feeding adds the very dependencies to the fed query
+        // as its feeding query had. So if the fed query is red, so is its feeder, which will
+        // get evaluated first, and re-feed the query.
+        if let Some((cached_result, _)) = cache.lookup(&key) {
+            panic!(
+                "fed query later has its value computed. The already cached value: {cached_result:?}"
+            );
         }
     }
+    job_owner.complete(cache, result, dep_node_index);
+
+    (result, Some(dep_node_index))
 }
 
 // Fast path for when incr. comp. is off.
@@ -537,7 +539,7 @@ where
 
     let (prev_dep_node_index, dep_node_index) = dep_graph_data.try_mark_green(qcx, &dep_node)?;
 
-    debug_assert!(dep_graph_data.is_green(dep_node));
+    debug_assert!(dep_graph_data.is_index_green(prev_dep_node_index));
 
     // First we try to load the result from the on-disk cache.
     // Some things are never cached on disk.
@@ -561,8 +563,7 @@ where
                 dep_graph_data.mark_debug_loaded_from_disk(*dep_node)
             }
 
-            let prev_fingerprint =
-                dep_graph_data.prev_fingerprint_of(dep_node).unwrap_or(Fingerprint::ZERO);
+            let prev_fingerprint = dep_graph_data.prev_fingerprint_of(prev_dep_node_index);
             // If `-Zincremental-verify-ich` is specified, re-hash results from
             // the cache and make sure that they have the expected fingerprint.
             //
@@ -578,7 +579,7 @@ where
                     *qcx.dep_context(),
                     dep_graph_data,
                     &result,
-                    dep_node,
+                    prev_dep_node_index,
                     query.hash_result(),
                 );
             }
@@ -623,7 +624,7 @@ where
         *qcx.dep_context(),
         dep_graph_data,
         &result,
-        dep_node,
+        prev_dep_node_index,
         query.hash_result(),
     );
 
@@ -636,77 +637,50 @@ pub(crate) fn incremental_verify_ich<Tcx, V: Debug>(
     tcx: Tcx,
     dep_graph_data: &DepGraphData<Tcx::DepKind>,
     result: &V,
-    dep_node: &DepNode<Tcx::DepKind>,
+    prev_index: SerializedDepNodeIndex,
     hash_result: Option<fn(&mut StableHashingContext<'_>, &V) -> Fingerprint>,
-) -> Fingerprint
-where
+) where
     Tcx: DepContext,
 {
-    assert!(
-        dep_graph_data.is_green(dep_node),
-        "fingerprint for green query instance not loaded from cache: {dep_node:?}",
-    );
+    if !dep_graph_data.is_index_green(prev_index) {
+        incremental_verify_ich_not_green(tcx, prev_index)
+    }
 
     let new_hash = hash_result.map_or(Fingerprint::ZERO, |f| {
         tcx.with_stable_hashing_context(|mut hcx| f(&mut hcx, result))
     });
 
-    let old_hash = dep_graph_data.prev_fingerprint_of(dep_node);
+    let old_hash = dep_graph_data.prev_fingerprint_of(prev_index);
 
-    if Some(new_hash) != old_hash {
-        incremental_verify_ich_failed(
-            tcx.sess(),
-            DebugArg::from(&dep_node),
-            DebugArg::from(&result),
-        );
+    if new_hash != old_hash {
+        incremental_verify_ich_failed(tcx, prev_index, result);
     }
-
-    new_hash
 }
 
-// This DebugArg business is largely a mirror of std::fmt::ArgumentV1, which is
-// currently not exposed publicly.
-//
-// The PR which added this attempted to use `&dyn Debug` instead, but that
-// showed statistically significant worse compiler performance. It's not
-// actually clear what the cause there was -- the code should be cold. If this
-// can be replaced with `&dyn Debug` with on perf impact, then it probably
-// should be.
-extern "C" {
-    type Opaque;
-}
-
-struct DebugArg<'a> {
-    value: &'a Opaque,
-    fmt: fn(&Opaque, &mut std::fmt::Formatter<'_>) -> std::fmt::Result,
-}
-
-impl<'a, T> From<&'a T> for DebugArg<'a>
+#[cold]
+#[inline(never)]
+fn incremental_verify_ich_not_green<Tcx>(tcx: Tcx, prev_index: SerializedDepNodeIndex)
 where
-    T: std::fmt::Debug,
+    Tcx: DepContext,
 {
-    fn from(value: &'a T) -> DebugArg<'a> {
-        DebugArg {
-            value: unsafe { std::mem::transmute(value) },
-            fmt: unsafe {
-                std::mem::transmute(<T as std::fmt::Debug>::fmt as fn(_, _) -> std::fmt::Result)
-            },
-        }
-    }
-}
-
-impl std::fmt::Debug for DebugArg<'_> {
-    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
-        (self.fmt)(self.value, f)
-    }
+    panic!(
+        "fingerprint for green query instance not loaded from cache: {:?}",
+        tcx.dep_graph().data().unwrap().prev_node_of(prev_index)
+    )
 }
 
-// Note that this is marked #[cold] and intentionally takes the equivalent of
-// `dyn Debug` for its arguments, as we want to avoid generating a bunch of
-// different implementations for LLVM to chew on (and filling up the final
-// binary, too).
+// Note that this is marked #[cold] and intentionally takes `dyn Debug` for `result`,
+// as we want to avoid generating a bunch of different implementations for LLVM to
+// chew on (and filling up the final binary, too).
 #[cold]
-fn incremental_verify_ich_failed(sess: &Session, dep_node: DebugArg<'_>, result: DebugArg<'_>) {
+#[inline(never)]
+fn incremental_verify_ich_failed<Tcx>(
+    tcx: Tcx,
+    prev_index: SerializedDepNodeIndex,
+    result: &dyn Debug,
+) where
+    Tcx: DepContext,
+{
     // When we emit an error message and panic, we try to debug-print the `DepNode`
     // and query result. Unfortunately, this can cause us to run additional queries,
     // which may result in another fingerprint mismatch while we're in the middle
@@ -720,15 +694,16 @@ fn incremental_verify_ich_failed(sess: &Session, dep_node: DebugArg<'_>, result:
     let old_in_panic = INSIDE_VERIFY_PANIC.with(|in_panic| in_panic.replace(true));
 
     if old_in_panic {
-        sess.emit_err(crate::error::Reentrant);
+        tcx.sess().emit_err(crate::error::Reentrant);
     } else {
-        let run_cmd = if let Some(crate_name) = &sess.opts.crate_name {
+        let run_cmd = if let Some(crate_name) = &tcx.sess().opts.crate_name {
             format!("`cargo clean -p {crate_name}` or `cargo clean`")
         } else {
             "`cargo clean`".to_string()
         };
 
-        sess.emit_err(crate::error::IncrementCompilation {
+        let dep_node = tcx.dep_graph().data().unwrap().prev_node_of(prev_index);
+        tcx.sess().emit_err(crate::error::IncrementCompilation {
             run_cmd,
             dep_node: format!("{dep_node:?}"),
         });