diff options
Diffstat (limited to 'compiler/rustc_query_system/src')
| -rw-r--r-- | compiler/rustc_query_system/src/dep_graph/graph.rs | 52 | ||||
| -rw-r--r-- | compiler/rustc_query_system/src/dep_graph/serialized.rs | 5 | ||||
| -rw-r--r-- | compiler/rustc_query_system/src/query/job.rs | 2 | ||||
| -rw-r--r-- | compiler/rustc_query_system/src/query/plumbing.rs | 365 |
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(), - ¤t_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:?}"), }); |
