summaryrefslogtreecommitdiffstats
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.rs63
-rw-r--r--compiler/rustc_query_system/src/dep_graph/mod.rs6
-rw-r--r--compiler/rustc_query_system/src/dep_graph/serialized.rs3
-rw-r--r--compiler/rustc_query_system/src/ich/hcx.rs4
-rw-r--r--compiler/rustc_query_system/src/lib.rs6
-rw-r--r--compiler/rustc_query_system/src/query/caches.rs267
-rw-r--r--compiler/rustc_query_system/src/query/config.rs13
-rw-r--r--compiler/rustc_query_system/src/query/mod.rs19
-rw-r--r--compiler/rustc_query_system/src/query/plumbing.rs201
9 files changed, 215 insertions, 367 deletions
diff --git a/compiler/rustc_query_system/src/dep_graph/graph.rs b/compiler/rustc_query_system/src/dep_graph/graph.rs
index 47b2fd8f8..59e0c3597 100644
--- a/compiler/rustc_query_system/src/dep_graph/graph.rs
+++ b/compiler/rustc_query_system/src/dep_graph/graph.rs
@@ -6,6 +6,7 @@ use rustc_data_structures::sharded::{self, Sharded};
use rustc_data_structures::stable_hasher::{HashStable, StableHasher};
use rustc_data_structures::steal::Steal;
use rustc_data_structures::sync::{AtomicU32, AtomicU64, Lock, Lrc, Ordering};
+use rustc_data_structures::OnDrop;
use rustc_index::vec::IndexVec;
use rustc_serialize::opaque::{FileEncodeResult, FileEncoder};
use smallvec::{smallvec, SmallVec};
@@ -47,7 +48,7 @@ impl DepNodeIndex {
}
impl From<DepNodeIndex> for QueryInvocationId {
- #[inline]
+ #[inline(always)]
fn from(dep_node_index: DepNodeIndex) -> Self {
QueryInvocationId(dep_node_index.as_u32())
}
@@ -278,6 +279,7 @@ impl<K: DepKind> DepGraph<K> {
/// `arg` parameter.
///
/// [rustc dev guide]: https://rustc-dev-guide.rust-lang.org/incremental-compilation.html
+ #[inline(always)]
pub fn with_task<Ctxt: HasDepContext<DepKind = K>, A: Debug, R>(
&self,
key: DepNode<K>,
@@ -297,6 +299,7 @@ impl<K: DepKind> DepGraph<K> {
}
}
+ #[inline(always)]
fn with_task_impl<Ctxt: HasDepContext<DepKind = K>, A: Debug, R>(
&self,
key: DepNode<K>,
@@ -597,6 +600,7 @@ impl<K: DepKind> DepGraph<K> {
self.data.is_some() && self.dep_node_index_of_opt(dep_node).is_some()
}
+ #[inline]
pub fn prev_fingerprint_of(&self, dep_node: &DepNode<K>) -> Option<Fingerprint> {
self.data.as_ref().unwrap().previous.fingerprint_of(dep_node)
}
@@ -671,17 +675,24 @@ impl<K: DepKind> DepGraph<K> {
let prev_index = data.previous.node_to_index_opt(dep_node)?;
match data.colors.get(prev_index) {
- Some(DepNodeColor::Green(dep_node_index)) => Some((prev_index, dep_node_index)),
- Some(DepNodeColor::Red) => None,
- None => {
- // This DepNode and the corresponding query invocation existed
- // in the previous compilation session too, so we can try to
- // mark it as green by recursively marking all of its
- // dependencies green.
- self.try_mark_previous_green(qcx, data, prev_index, &dep_node)
- .map(|dep_node_index| (prev_index, dep_node_index))
- }
+ Some(DepNodeColor::Green(dep_node_index)) => return Some((prev_index, dep_node_index)),
+ Some(DepNodeColor::Red) => return None,
+ None => {}
}
+
+ let backtrace = backtrace_printer(qcx.dep_context().sess(), data, prev_index);
+
+ // This DepNode and the corresponding query invocation existed
+ // in the previous compilation session too, so we can try to
+ // mark it as green by recursively marking all of its
+ // dependencies green.
+ let ret = self
+ .try_mark_previous_green(qcx, data, prev_index, &dep_node)
+ .map(|dep_node_index| (prev_index, dep_node_index));
+
+ // We succeeded, no backtrace.
+ backtrace.disable();
+ return ret;
}
#[instrument(skip(self, qcx, data, parent_dep_node_index), level = "debug")]
@@ -794,7 +805,10 @@ impl<K: DepKind> DepGraph<K> {
let prev_deps = data.previous.edge_targets_from(prev_dep_node_index);
for &dep_dep_node_index in prev_deps {
- self.try_mark_parent_green(qcx, data, dep_dep_node_index, dep_node)?
+ let backtrace = backtrace_printer(qcx.dep_context().sess(), data, dep_dep_node_index);
+ let success = self.try_mark_parent_green(qcx, data, dep_dep_node_index, dep_node);
+ backtrace.disable();
+ success?;
}
// If we got here without hitting a `return` that means that all
@@ -1116,6 +1130,7 @@ impl<K: DepKind> CurrentDepGraph<K> {
/// Writes the node to the current dep-graph and allocates a `DepNodeIndex` for it.
/// Assumes that this is a node that has no equivalent in the previous dep-graph.
+ #[inline(always)]
fn intern_new_node(
&self,
profiler: &SelfProfilerRef,
@@ -1354,6 +1369,7 @@ impl DepNodeColorMap {
}
}
+ #[inline]
fn insert(&self, index: SerializedDepNodeIndex, color: DepNodeColor) {
self.values[index].store(
match color {
@@ -1364,3 +1380,26 @@ impl DepNodeColorMap {
)
}
}
+
+fn backtrace_printer<'a, K: DepKind>(
+ sess: &'a rustc_session::Session,
+ graph: &'a DepGraphData<K>,
+ node: SerializedDepNodeIndex,
+) -> OnDrop<impl Fn() + 'a> {
+ OnDrop(
+ #[inline(never)]
+ #[cold]
+ move || {
+ let node = graph.previous.index_to_node(node);
+ // Do not try to rely on DepNode's Debug implementation, since it may panic.
+ let diag = rustc_errors::Diagnostic::new(
+ rustc_errors::Level::FailureNote,
+ &format!(
+ "encountered while trying to mark dependency green: {:?}({})",
+ node.kind, node.hash
+ ),
+ );
+ sess.diagnostic().force_print_diagnostic(diag);
+ },
+ )
+}
diff --git a/compiler/rustc_query_system/src/dep_graph/mod.rs b/compiler/rustc_query_system/src/dep_graph/mod.rs
index e370c6990..6969f2dbe 100644
--- a/compiler/rustc_query_system/src/dep_graph/mod.rs
+++ b/compiler/rustc_query_system/src/dep_graph/mod.rs
@@ -23,7 +23,7 @@ pub trait DepContext: Copy {
type DepKind: self::DepKind;
/// Create a hashing context for hashing new results.
- fn with_stable_hashing_context<R>(&self, f: impl FnOnce(StableHashingContext<'_>) -> R) -> R;
+ fn with_stable_hashing_context<R>(self, f: impl FnOnce(StableHashingContext<'_>) -> R) -> R;
/// Access the DepGraph.
fn dep_graph(&self) -> &DepGraph<Self::DepKind>;
@@ -37,7 +37,7 @@ pub trait DepContext: Copy {
fn dep_kind_info(&self, dep_node: Self::DepKind) -> &DepKindStruct<Self>;
#[inline(always)]
- fn fingerprint_style(&self, kind: Self::DepKind) -> FingerprintStyle {
+ fn fingerprint_style(self, kind: Self::DepKind) -> FingerprintStyle {
let data = self.dep_kind_info(kind);
if data.is_anon {
return FingerprintStyle::Opaque;
@@ -47,7 +47,7 @@ pub trait DepContext: Copy {
#[inline(always)]
/// Return whether this kind always require evaluation.
- fn is_eval_always(&self, kind: Self::DepKind) -> bool {
+ fn is_eval_always(self, kind: Self::DepKind) -> bool {
self.dep_kind_info(kind).is_eval_always
}
diff --git a/compiler/rustc_query_system/src/dep_graph/serialized.rs b/compiler/rustc_query_system/src/dep_graph/serialized.rs
index a81595b24..29513df46 100644
--- a/compiler/rustc_query_system/src/dep_graph/serialized.rs
+++ b/compiler/rustc_query_system/src/dep_graph/serialized.rs
@@ -242,8 +242,7 @@ impl<K: DepKind + Encodable<FileEncoder>> GraphEncoder<K> {
record_graph: bool,
record_stats: bool,
) -> Self {
- let record_graph =
- if record_graph { Some(Lock::new(DepGraphQuery::new(prev_node_count))) } else { None };
+ let record_graph = record_graph.then(|| Lock::new(DepGraphQuery::new(prev_node_count)));
let status = Lock::new(EncoderState::new(encoder, record_stats));
GraphEncoder { status, record_graph }
}
diff --git a/compiler/rustc_query_system/src/ich/hcx.rs b/compiler/rustc_query_system/src/ich/hcx.rs
index 163da59ed..5593a1541 100644
--- a/compiler/rustc_query_system/src/ich/hcx.rs
+++ b/compiler/rustc_query_system/src/ich/hcx.rs
@@ -90,7 +90,7 @@ impl<'a> StableHashingContext<'a> {
if let Some(def_id) = def_id.as_local() {
self.local_def_path_hash(def_id)
} else {
- self.untracked.cstore.def_path_hash(def_id)
+ self.untracked.cstore.read().def_path_hash(def_id)
}
}
@@ -146,7 +146,7 @@ impl<'a> rustc_span::HashStableContext for StableHashingContext<'a> {
#[inline]
fn def_span(&self, def_id: LocalDefId) -> Span {
- *self.untracked.source_span.get(def_id).unwrap_or(&DUMMY_SP)
+ self.untracked.source_span.get(def_id).unwrap_or(DUMMY_SP)
}
#[inline]
diff --git a/compiler/rustc_query_system/src/lib.rs b/compiler/rustc_query_system/src/lib.rs
index f47760e9a..6cc4c9a7e 100644
--- a/compiler/rustc_query_system/src/lib.rs
+++ b/compiler/rustc_query_system/src/lib.rs
@@ -3,6 +3,7 @@
#![feature(hash_raw_entry)]
#![feature(min_specialization)]
#![feature(extern_types)]
+#![feature(let_chains)]
#![allow(rustc::potential_query_instability)]
#![deny(rustc::untranslatable_diagnostic)]
#![deny(rustc::diagnostic_outside_of_impl)]
@@ -14,6 +15,9 @@ extern crate rustc_data_structures;
#[macro_use]
extern crate rustc_macros;
+use rustc_errors::{DiagnosticMessage, SubdiagnosticMessage};
+use rustc_macros::fluent_messages;
+
pub mod cache;
pub mod dep_graph;
mod error;
@@ -25,3 +29,5 @@ pub use error::HandleCycleError;
pub use error::LayoutOfDepth;
pub use error::QueryOverflow;
pub use values::Value;
+
+fluent_messages! { "../locales/en-US.ftl" }
diff --git a/compiler/rustc_query_system/src/query/caches.rs b/compiler/rustc_query_system/src/query/caches.rs
index 77d0d0314..4b3cd16c2 100644
--- a/compiler/rustc_query_system/src/query/caches.rs
+++ b/compiler/rustc_query_system/src/query/caches.rs
@@ -1,13 +1,10 @@
use crate::dep_graph::DepNodeIndex;
-use rustc_arena::TypedArena;
use rustc_data_structures::fx::FxHashMap;
use rustc_data_structures::sharded;
#[cfg(parallel_compiler)]
use rustc_data_structures::sharded::Sharded;
-#[cfg(not(parallel_compiler))]
use rustc_data_structures::sync::Lock;
-use rustc_data_structures::sync::WorkerLocal;
use rustc_index::vec::{Idx, IndexVec};
use std::fmt::Debug;
use std::hash::Hash;
@@ -16,36 +13,23 @@ use std::marker::PhantomData;
pub trait CacheSelector<'tcx, V> {
type Cache
where
- V: Clone;
- type ArenaCache;
+ V: Copy;
}
pub trait QueryStorage {
- type Value: Debug;
- type Stored: Clone;
-
- /// Store a value without putting it in the cache.
- /// This is meant to be used with cycle errors.
- fn store_nocache(&self, value: Self::Value) -> Self::Stored;
+ type Value: Copy;
}
pub trait QueryCache: QueryStorage + Sized {
- type Key: Hash + Eq + Clone + Debug;
+ type Key: Hash + Eq + Copy + Debug;
/// 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
/// to compute it.
- fn lookup<R, OnHit>(
- &self,
- key: &Self::Key,
- // `on_hit` can be called while holding a lock to the query state shard.
- on_hit: OnHit,
- ) -> Result<R, ()>
- where
- OnHit: FnOnce(&Self::Stored, DepNodeIndex) -> R;
+ fn lookup(&self, key: &Self::Key) -> Option<(Self::Value, DepNodeIndex)>;
- fn complete(&self, key: Self::Key, value: Self::Value, index: DepNodeIndex) -> Self::Stored;
+ fn complete(&self, key: Self::Key, value: Self::Value, index: DepNodeIndex);
fn iter(&self, f: &mut dyn FnMut(&Self::Key, &Self::Value, DepNodeIndex));
}
@@ -55,8 +39,7 @@ pub struct DefaultCacheSelector<K>(PhantomData<K>);
impl<'tcx, K: Eq + Hash, V: 'tcx> CacheSelector<'tcx, V> for DefaultCacheSelector<K> {
type Cache = DefaultCache<K, V>
where
- V: Clone;
- type ArenaCache = ArenaCache<'tcx, K, V>;
+ V: Copy;
}
pub struct DefaultCache<K, V> {
@@ -72,29 +55,19 @@ impl<K, V> Default for DefaultCache<K, V> {
}
}
-impl<K: Eq + Hash, V: Clone + Debug> QueryStorage for DefaultCache<K, V> {
+impl<K: Eq + Hash, V: Copy + Debug> QueryStorage for DefaultCache<K, V> {
type Value = V;
- type Stored = V;
-
- #[inline]
- fn store_nocache(&self, value: Self::Value) -> Self::Stored {
- // We have no dedicated storage
- value
- }
}
impl<K, V> QueryCache for DefaultCache<K, V>
where
- K: Eq + Hash + Clone + Debug,
- V: Clone + Debug,
+ K: Eq + Hash + Copy + Debug,
+ V: Copy + Debug,
{
type Key = K;
#[inline(always)]
- fn lookup<R, OnHit>(&self, key: &K, on_hit: OnHit) -> Result<R, ()>
- where
- OnHit: FnOnce(&V, DepNodeIndex) -> R,
- {
+ fn lookup(&self, key: &K) -> Option<(V, DepNodeIndex)> {
let key_hash = sharded::make_hash(key);
#[cfg(parallel_compiler)]
let lock = self.cache.get_shard_by_hash(key_hash).lock();
@@ -102,24 +75,18 @@ where
let lock = self.cache.lock();
let result = lock.raw_entry().from_key_hashed_nocheck(key_hash, key);
- if let Some((_, value)) = result {
- let hit_result = on_hit(&value.0, value.1);
- Ok(hit_result)
- } else {
- Err(())
- }
+ if let Some((_, value)) = result { Some(*value) } else { None }
}
#[inline]
- fn complete(&self, key: K, value: V, index: DepNodeIndex) -> Self::Stored {
+ fn complete(&self, key: K, value: V, index: DepNodeIndex) {
#[cfg(parallel_compiler)]
let mut lock = self.cache.get_shard_by_value(&key).lock();
#[cfg(not(parallel_compiler))]
let mut lock = self.cache.lock();
// We may be overwriting another value. This is all right, since the dep-graph
// will check that the fingerprint matches.
- lock.insert(key, (value.clone(), index));
- value
+ lock.insert(key, (value, index));
}
fn iter(&self, f: &mut dyn FnMut(&Self::Key, &Self::Value, DepNodeIndex)) {
@@ -142,90 +109,46 @@ where
}
}
-pub struct ArenaCache<'tcx, K, V> {
- arena: WorkerLocal<TypedArena<(V, DepNodeIndex)>>,
- #[cfg(parallel_compiler)]
- cache: Sharded<FxHashMap<K, &'tcx (V, DepNodeIndex)>>,
- #[cfg(not(parallel_compiler))]
- cache: Lock<FxHashMap<K, &'tcx (V, DepNodeIndex)>>,
+pub struct SingleCacheSelector;
+
+impl<'tcx, V: 'tcx> CacheSelector<'tcx, V> for SingleCacheSelector {
+ type Cache = SingleCache<V>
+ where
+ V: Copy;
}
-impl<'tcx, K, V> Default for ArenaCache<'tcx, K, V> {
+pub struct SingleCache<V> {
+ cache: Lock<Option<(V, DepNodeIndex)>>,
+}
+
+impl<V> Default for SingleCache<V> {
fn default() -> Self {
- ArenaCache { arena: WorkerLocal::new(|_| TypedArena::default()), cache: Default::default() }
+ SingleCache { cache: Lock::new(None) }
}
}
-impl<'tcx, K: Eq + Hash, V: Debug + 'tcx> QueryStorage for ArenaCache<'tcx, K, V> {
+impl<V: Copy + Debug> QueryStorage for SingleCache<V> {
type Value = V;
- type Stored = &'tcx V;
-
- #[inline]
- fn store_nocache(&self, value: Self::Value) -> Self::Stored {
- let value = self.arena.alloc((value, DepNodeIndex::INVALID));
- let value = unsafe { &*(&value.0 as *const _) };
- &value
- }
}
-impl<'tcx, K, V: 'tcx> QueryCache for ArenaCache<'tcx, K, V>
+impl<V> QueryCache for SingleCache<V>
where
- K: Eq + Hash + Clone + Debug,
- V: Debug,
+ V: Copy + Debug,
{
- type Key = K;
+ type Key = ();
#[inline(always)]
- fn lookup<R, OnHit>(&self, key: &K, on_hit: OnHit) -> Result<R, ()>
- where
- OnHit: FnOnce(&&'tcx V, DepNodeIndex) -> R,
- {
- let key_hash = sharded::make_hash(key);
- #[cfg(parallel_compiler)]
- let lock = self.cache.get_shard_by_hash(key_hash).lock();
- #[cfg(not(parallel_compiler))]
- let lock = self.cache.lock();
- let result = lock.raw_entry().from_key_hashed_nocheck(key_hash, key);
-
- if let Some((_, value)) = result {
- let hit_result = on_hit(&&value.0, value.1);
- Ok(hit_result)
- } else {
- Err(())
- }
+ fn lookup(&self, _key: &()) -> Option<(V, DepNodeIndex)> {
+ *self.cache.lock()
}
#[inline]
- fn complete(&self, key: K, value: V, index: DepNodeIndex) -> Self::Stored {
- let value = self.arena.alloc((value, index));
- let value = unsafe { &*(value as *const _) };
- #[cfg(parallel_compiler)]
- let mut lock = self.cache.get_shard_by_value(&key).lock();
- #[cfg(not(parallel_compiler))]
- let mut lock = self.cache.lock();
- // We may be overwriting another value. This is all right, since the dep-graph
- // will check that the fingerprint matches.
- lock.insert(key, value);
- &value.0
+ fn complete(&self, _key: (), value: V, index: DepNodeIndex) {
+ *self.cache.lock() = Some((value, index));
}
fn iter(&self, f: &mut dyn FnMut(&Self::Key, &Self::Value, DepNodeIndex)) {
- #[cfg(parallel_compiler)]
- {
- let shards = self.cache.lock_shards();
- for shard in shards.iter() {
- for (k, v) in shard.iter() {
- f(k, &v.0, v.1);
- }
- }
- }
- #[cfg(not(parallel_compiler))]
- {
- let map = self.cache.lock();
- for (k, v) in map.iter() {
- f(k, &v.0, v.1);
- }
- }
+ self.cache.lock().as_ref().map(|value| f(&(), &value.0, value.1));
}
}
@@ -234,8 +157,7 @@ pub struct VecCacheSelector<K>(PhantomData<K>);
impl<'tcx, K: Idx, V: 'tcx> CacheSelector<'tcx, V> for VecCacheSelector<K> {
type Cache = VecCache<K, V>
where
- V: Clone;
- type ArenaCache = VecArenaCache<'tcx, K, V>;
+ V: Copy;
}
pub struct VecCache<K: Idx, V> {
@@ -251,138 +173,33 @@ impl<K: Idx, V> Default for VecCache<K, V> {
}
}
-impl<K: Eq + Idx, V: Clone + Debug> QueryStorage for VecCache<K, V> {
+impl<K: Eq + Idx, V: Copy + Debug> QueryStorage for VecCache<K, V> {
type Value = V;
- type Stored = V;
-
- #[inline]
- fn store_nocache(&self, value: Self::Value) -> Self::Stored {
- // We have no dedicated storage
- value
- }
}
impl<K, V> QueryCache for VecCache<K, V>
where
- K: Eq + Idx + Clone + Debug,
- V: Clone + Debug,
-{
- type Key = K;
-
- #[inline(always)]
- fn lookup<R, OnHit>(&self, key: &K, on_hit: OnHit) -> Result<R, ()>
- where
- OnHit: FnOnce(&V, DepNodeIndex) -> R,
- {
- #[cfg(parallel_compiler)]
- let lock = self.cache.get_shard_by_hash(key.index() as u64).lock();
- #[cfg(not(parallel_compiler))]
- let lock = self.cache.lock();
- if let Some(Some(value)) = lock.get(*key) {
- let hit_result = on_hit(&value.0, value.1);
- Ok(hit_result)
- } else {
- Err(())
- }
- }
-
- #[inline]
- fn complete(&self, key: K, value: V, index: DepNodeIndex) -> Self::Stored {
- #[cfg(parallel_compiler)]
- let mut lock = self.cache.get_shard_by_hash(key.index() as u64).lock();
- #[cfg(not(parallel_compiler))]
- let mut lock = self.cache.lock();
- lock.insert(key, (value.clone(), index));
- value
- }
-
- fn iter(&self, f: &mut dyn FnMut(&Self::Key, &Self::Value, DepNodeIndex)) {
- #[cfg(parallel_compiler)]
- {
- let shards = self.cache.lock_shards();
- for shard in shards.iter() {
- for (k, v) in shard.iter_enumerated() {
- if let Some(v) = v {
- f(&k, &v.0, v.1);
- }
- }
- }
- }
- #[cfg(not(parallel_compiler))]
- {
- let map = self.cache.lock();
- for (k, v) in map.iter_enumerated() {
- if let Some(v) = v {
- f(&k, &v.0, v.1);
- }
- }
- }
- }
-}
-
-pub struct VecArenaCache<'tcx, K: Idx, V> {
- arena: WorkerLocal<TypedArena<(V, DepNodeIndex)>>,
- #[cfg(parallel_compiler)]
- cache: Sharded<IndexVec<K, Option<&'tcx (V, DepNodeIndex)>>>,
- #[cfg(not(parallel_compiler))]
- cache: Lock<IndexVec<K, Option<&'tcx (V, DepNodeIndex)>>>,
-}
-
-impl<'tcx, K: Idx, V> Default for VecArenaCache<'tcx, K, V> {
- fn default() -> Self {
- VecArenaCache {
- arena: WorkerLocal::new(|_| TypedArena::default()),
- cache: Default::default(),
- }
- }
-}
-
-impl<'tcx, K: Eq + Idx, V: Debug + 'tcx> QueryStorage for VecArenaCache<'tcx, K, V> {
- type Value = V;
- type Stored = &'tcx V;
-
- #[inline]
- fn store_nocache(&self, value: Self::Value) -> Self::Stored {
- let value = self.arena.alloc((value, DepNodeIndex::INVALID));
- let value = unsafe { &*(&value.0 as *const _) };
- &value
- }
-}
-
-impl<'tcx, K, V: 'tcx> QueryCache for VecArenaCache<'tcx, K, V>
-where
- K: Eq + Idx + Clone + Debug,
- V: Debug,
+ K: Eq + Idx + Copy + Debug,
+ V: Copy + Debug,
{
type Key = K;
#[inline(always)]
- fn lookup<R, OnHit>(&self, key: &K, on_hit: OnHit) -> Result<R, ()>
- where
- OnHit: FnOnce(&&'tcx V, DepNodeIndex) -> R,
- {
+ fn lookup(&self, key: &K) -> Option<(V, DepNodeIndex)> {
#[cfg(parallel_compiler)]
let lock = self.cache.get_shard_by_hash(key.index() as u64).lock();
#[cfg(not(parallel_compiler))]
let lock = self.cache.lock();
- if let Some(Some(value)) = lock.get(*key) {
- let hit_result = on_hit(&&value.0, value.1);
- Ok(hit_result)
- } else {
- Err(())
- }
+ if let Some(Some(value)) = lock.get(*key) { Some(*value) } else { None }
}
#[inline]
- fn complete(&self, key: K, value: V, index: DepNodeIndex) -> Self::Stored {
- let value = self.arena.alloc((value, index));
- let value = unsafe { &*(value as *const _) };
+ fn complete(&self, key: K, value: V, index: DepNodeIndex) {
#[cfg(parallel_compiler)]
let mut lock = self.cache.get_shard_by_hash(key.index() as u64).lock();
#[cfg(not(parallel_compiler))]
let mut lock = self.cache.lock();
- lock.insert(key, value);
- &value.0
+ lock.insert(key, (value, index));
}
fn iter(&self, f: &mut dyn FnMut(&Self::Key, &Self::Value, DepNodeIndex)) {
diff --git a/compiler/rustc_query_system/src/query/config.rs b/compiler/rustc_query_system/src/query/config.rs
index 8c0330e43..d56373873 100644
--- a/compiler/rustc_query_system/src/query/config.rs
+++ b/compiler/rustc_query_system/src/query/config.rs
@@ -19,11 +19,12 @@ pub type TryLoadFromDisk<Qcx, Q> =
pub trait QueryConfig<Qcx: QueryContext> {
const NAME: &'static str;
- type Key: DepNodeParams<Qcx::DepContext> + Eq + Hash + Clone + Debug;
- type Value: Debug;
- type Stored: Debug + Clone + std::borrow::Borrow<Self::Value>;
+ // `Key` and `Value` are `Copy` instead of `Clone` to ensure copying them stays cheap,
+ // but it isn't necessary.
+ type Key: DepNodeParams<Qcx::DepContext> + Eq + Hash + Copy + Debug;
+ type Value: Debug + Copy;
- type Cache: QueryCache<Key = Self::Key, Stored = Self::Stored, Value = Self::Value>;
+ type Cache: QueryCache<Key = Self::Key, Value = Self::Value>;
// Don't use this method to access query results, instead use the methods on TyCtxt
fn query_state<'a>(tcx: Qcx) -> &'a QueryState<Self::Key, Qcx::DepKind>
@@ -38,9 +39,9 @@ pub trait QueryConfig<Qcx: QueryContext> {
fn cache_on_disk(tcx: Qcx::DepContext, key: &Self::Key) -> bool;
// Don't use this method to compute query results, instead use the methods on TyCtxt
- fn execute_query(tcx: Qcx::DepContext, k: Self::Key) -> Self::Stored;
+ fn execute_query(tcx: Qcx::DepContext, k: Self::Key) -> Self::Value;
- fn compute(tcx: Qcx, key: &Self::Key) -> fn(Qcx::DepContext, Self::Key) -> Self::Value;
+ fn compute(tcx: Qcx, key: Self::Key) -> Self::Value;
fn try_load_from_disk(qcx: Qcx, idx: &Self::Key) -> TryLoadFromDisk<Qcx, Self>;
diff --git a/compiler/rustc_query_system/src/query/mod.rs b/compiler/rustc_query_system/src/query/mod.rs
index d308af192..383c63cd2 100644
--- a/compiler/rustc_query_system/src/query/mod.rs
+++ b/compiler/rustc_query_system/src/query/mod.rs
@@ -8,7 +8,8 @@ pub use self::job::{print_query_stack, QueryInfo, QueryJob, QueryJobId, QueryJob
mod caches;
pub use self::caches::{
- CacheSelector, DefaultCacheSelector, QueryCache, QueryStorage, VecCacheSelector,
+ CacheSelector, DefaultCacheSelector, QueryCache, QueryStorage, SingleCacheSelector,
+ VecCacheSelector,
};
mod config;
@@ -100,22 +101,22 @@ impl QuerySideEffects {
}
pub trait QueryContext: HasDepContext {
- fn next_job_id(&self) -> QueryJobId;
+ fn next_job_id(self) -> QueryJobId;
/// Get the query information from the TLS context.
- fn current_query_job(&self) -> Option<QueryJobId>;
+ fn current_query_job(self) -> Option<QueryJobId>;
- fn try_collect_active_jobs(&self) -> Option<QueryMap<Self::DepKind>>;
+ fn try_collect_active_jobs(self) -> Option<QueryMap<Self::DepKind>>;
/// Load side effects associated to the node in the previous session.
- fn load_side_effects(&self, prev_dep_node_index: SerializedDepNodeIndex) -> QuerySideEffects;
+ fn load_side_effects(self, prev_dep_node_index: SerializedDepNodeIndex) -> QuerySideEffects;
/// Register diagnostics for the given node, for use in next session.
- fn store_side_effects(&self, dep_node_index: DepNodeIndex, side_effects: QuerySideEffects);
+ fn store_side_effects(self, dep_node_index: DepNodeIndex, side_effects: QuerySideEffects);
/// Register diagnostics for the given node, for use in next session.
fn store_side_effects_for_anon_node(
- &self,
+ self,
dep_node_index: DepNodeIndex,
side_effects: QuerySideEffects,
);
@@ -124,12 +125,12 @@ pub trait QueryContext: HasDepContext {
/// new query job while it executes. It returns the diagnostics
/// captured during execution and the actual result.
fn start_query<R>(
- &self,
+ self,
token: QueryJobId,
depth_limit: bool,
diagnostics: Option<&Lock<ThinVec<Diagnostic>>>,
compute: impl FnOnce() -> R,
) -> R;
- fn depth_limit_error(&self, job: QueryJobId);
+ fn depth_limit_error(self, job: QueryJobId);
}
diff --git a/compiler/rustc_query_system/src/query/plumbing.rs b/compiler/rustc_query_system/src/query/plumbing.rs
index b3b939eae..5f003fa70 100644
--- a/compiler/rustc_query_system/src/query/plumbing.rs
+++ b/compiler/rustc_query_system/src/query/plumbing.rs
@@ -2,7 +2,7 @@
//! generate the actual methods on tcx which find and execute the provider,
//! manage the caches, and so forth.
-use crate::dep_graph::{DepContext, DepKind, DepNode, DepNodeIndex};
+use crate::dep_graph::{DepContext, DepKind, DepNode, DepNodeIndex, DepNodeParams};
use crate::ich::StableHashingContext;
use crate::query::caches::QueryCache;
use crate::query::job::{report_cycle, QueryInfo, QueryJob, QueryJobId, QueryJobInfo};
@@ -15,17 +15,16 @@ use rustc_data_structures::fx::FxHashMap;
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;
use rustc_errors::{DiagnosticBuilder, ErrorGuaranteed, FatalError};
use rustc_session::Session;
use rustc_span::{Span, DUMMY_SP};
-use std::borrow::Borrow;
use std::cell::Cell;
use std::collections::hash_map::Entry;
use std::fmt::Debug;
use std::hash::Hash;
use std::mem;
-use std::ptr;
use thin_vec::ThinVec;
use super::QueryConfig;
@@ -49,7 +48,7 @@ enum QueryResult<D: DepKind> {
impl<K, D> QueryState<K, D>
where
- K: Eq + Hash + Clone + Debug,
+ K: Eq + Hash + Copy + Debug,
D: DepKind,
{
pub fn all_inactive(&self) -> bool {
@@ -78,7 +77,7 @@ where
for shard in shards.iter() {
for (k, v) in shard.iter() {
if let QueryResult::Started(ref job) = *v {
- let query = make_query(qcx, k.clone());
+ let query = make_query(qcx, *k);
jobs.insert(job.id, QueryJobInfo { query, job: job.clone() });
}
}
@@ -92,7 +91,7 @@ where
// really hurt much.)
for (k, v) in self.active.try_lock()?.iter() {
if let QueryResult::Started(ref job) = *v {
- let query = make_query(qcx, k.clone());
+ let query = make_query(qcx, *k);
jobs.insert(job.id, QueryJobInfo { query, job: job.clone() });
}
}
@@ -112,7 +111,7 @@ impl<K, D: DepKind> Default for QueryState<K, D> {
/// This will poison the relevant query if dropped.
struct JobOwner<'tcx, K, D: DepKind>
where
- K: Eq + Hash + Clone,
+ K: Eq + Hash + Copy,
{
state: &'tcx QueryState<K, D>,
key: K,
@@ -121,20 +120,17 @@ where
#[cold]
#[inline(never)]
-fn mk_cycle<Qcx, V, R, D: DepKind>(
+fn mk_cycle<Qcx, R, D: DepKind>(
qcx: Qcx,
cycle_error: CycleError<D>,
handler: HandleCycleError,
- cache: &dyn crate::query::QueryStorage<Value = V, Stored = R>,
) -> R
where
Qcx: QueryContext + crate::query::HasDepContext<DepKind = D>,
- V: std::fmt::Debug + Value<Qcx::DepContext, Qcx::DepKind>,
- R: Clone,
+ R: std::fmt::Debug + Value<Qcx::DepContext, Qcx::DepKind>,
{
let error = report_cycle(qcx.dep_context().sess(), &cycle_error);
- let value = handle_cycle_error(*qcx.dep_context(), &cycle_error, error, handler);
- cache.store_nocache(value)
+ handle_cycle_error(*qcx.dep_context(), &cycle_error, error, handler)
}
fn handle_cycle_error<Tcx, V>(
@@ -167,7 +163,7 @@ where
impl<'tcx, K, D: DepKind> JobOwner<'tcx, K, D>
where
- K: Eq + Hash + Clone,
+ 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.
@@ -192,14 +188,14 @@ where
#[cfg(not(parallel_compiler))]
let mut state_lock = state.active.lock();
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 = qcx.current_query_job();
- let job = QueryJob::new(id, span, job);
+ let job = QueryJob::new(id, span, current_job_id);
- let key = entry.key().clone();
+ let key = *entry.key();
entry.insert(QueryResult::Started(job));
let owner = JobOwner { state, id, key };
@@ -216,7 +212,7 @@ where
// so we just return the error.
return TryGetJob::Cycle(id.find_cycle_in_stack(
qcx.try_collect_active_jobs().unwrap(),
- &qcx.current_query_job(),
+ &current_job_id,
span,
));
}
@@ -234,7 +230,7 @@ where
// With parallel queries we might just have to wait on some other
// thread.
- let result = latch.wait_on(qcx.current_query_job(), span);
+ let result = latch.wait_on(current_job_id, span);
match result {
Ok(()) => TryGetJob::JobCompleted(query_blocked_prof_timer),
@@ -249,40 +245,38 @@ where
/// 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) -> C::Stored
+ fn complete<C>(self, cache: &C, result: C::Value, dep_node_index: DepNodeIndex)
where
C: QueryCache<Key = K>,
{
- // We can move out of `self` here because we `mem::forget` it below
- let key = unsafe { ptr::read(&self.key) };
+ let key = self.key;
let state = self.state;
// Forget ourself so our destructor won't poison the query
mem::forget(self);
- let (job, result) = {
- let job = {
- #[cfg(parallel_compiler)]
- let mut lock = state.active.get_shard_by_value(&key).lock();
- #[cfg(not(parallel_compiler))]
- let mut lock = state.active.lock();
- match lock.remove(&key).unwrap() {
- QueryResult::Started(job) => job,
- QueryResult::Poisoned => panic!(),
- }
- };
- let result = cache.complete(key, result, dep_node_index);
- (job, result)
+ // Mark as complete before we remove the job from the active state
+ // so no other thread can re-execute this query.
+ cache.complete(key, result, dep_node_index);
+
+ let job = {
+ #[cfg(parallel_compiler)]
+ let mut lock = state.active.get_shard_by_value(&key).lock();
+ #[cfg(not(parallel_compiler))]
+ let mut lock = state.active.lock();
+ match lock.remove(&key).unwrap() {
+ QueryResult::Started(job) => job,
+ QueryResult::Poisoned => panic!(),
+ }
};
job.signal_complete();
- result
}
}
impl<'tcx, K, D> Drop for JobOwner<'tcx, K, D>
where
- K: Eq + Hash + Clone,
+ K: Eq + Hash + Copy,
D: DepKind,
{
#[inline(never)]
@@ -299,7 +293,7 @@ where
QueryResult::Started(job) => job,
QueryResult::Poisoned => panic!(),
};
- shard.insert(self.key.clone(), QueryResult::Poisoned);
+ shard.insert(self.key, QueryResult::Poisoned);
job
};
// Also signal the completion of the job, so waiters
@@ -318,7 +312,7 @@ pub(crate) struct CycleError<D: DepKind> {
/// The result of `try_start`.
enum TryGetJob<'tcx, K, D>
where
- K: Eq + Hash + Clone,
+ K: Eq + Hash + Copy,
D: DepKind,
{
/// The query is not yet started. Contains a guard to the cache eventually used to start it.
@@ -339,74 +333,62 @@ where
/// which will be used if the query is not in the cache and we need
/// to compute it.
#[inline]
-pub fn try_get_cached<Tcx, C, R, OnHit>(
- tcx: Tcx,
- cache: &C,
- key: &C::Key,
- // `on_hit` can be called while holding a lock to the query cache
- on_hit: OnHit,
-) -> Result<R, ()>
+pub fn try_get_cached<Tcx, C>(tcx: Tcx, cache: &C, key: &C::Key) -> Option<C::Value>
where
C: QueryCache,
Tcx: DepContext,
- OnHit: FnOnce(&C::Stored) -> R,
{
- cache.lookup(&key, |value, index| {
- if std::intrinsics::unlikely(tcx.profiler().enabled()) {
+ match cache.lookup(&key) {
+ Some((value, index)) => {
tcx.profiler().query_cache_hit(index.into());
+ tcx.dep_graph().read_index(index);
+ Some(value)
}
- tcx.dep_graph().read_index(index);
- on_hit(value)
- })
+ None => None,
+ }
}
+#[inline(never)]
fn try_execute_query<Q, Qcx>(
qcx: Qcx,
- state: &QueryState<Q::Key, Qcx::DepKind>,
- cache: &Q::Cache,
span: Span,
key: Q::Key,
dep_node: Option<DepNode<Qcx::DepKind>>,
-) -> (Q::Stored, Option<DepNodeIndex>)
+) -> (Q::Value, Option<DepNodeIndex>)
where
Q: QueryConfig<Qcx>,
Qcx: QueryContext,
{
- match JobOwner::<'_, Q::Key, Qcx::DepKind>::try_start(&qcx, state, span, key.clone()) {
+ let state = Q::query_state(qcx);
+ match JobOwner::<'_, Q::Key, Qcx::DepKind>::try_start(&qcx, state, span, key) {
TryGetJob::NotYetStarted(job) => {
- let (result, dep_node_index) =
- execute_job::<Q, Qcx>(qcx, key.clone(), dep_node, job.id);
+ let (result, dep_node_index) = execute_job::<Q, Qcx>(qcx, key, dep_node, job.id);
+ let cache = Q::query_cache(qcx);
if Q::FEEDABLE {
- // We may have put a value inside the cache from inside the execution.
- // Verify that it has the same hash as what we have now, to ensure consistency.
- let _ = cache.lookup(&key, |cached_result, _| {
- let hasher = Q::HASH_RESULT.expect("feedable forbids no_hash");
-
- let old_hash = qcx.dep_context().with_stable_hashing_context(|mut hcx| hasher(&mut hcx, cached_result.borrow()));
- let new_hash = qcx.dep_context().with_stable_hashing_context(|mut hcx| hasher(&mut hcx, &result));
- debug_assert_eq!(
- old_hash, new_hash,
- "Computed query value for {:?}({:?}) is inconsistent with fed value,\ncomputed={:#?}\nfed={:#?}",
- Q::DEP_KIND, key, result, cached_result,
+ // 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:?}"
);
- });
+ }
}
- let result = job.complete(cache, result, dep_node_index);
+ job.complete(cache, result, dep_node_index);
(result, Some(dep_node_index))
}
TryGetJob::Cycle(error) => {
- let result = mk_cycle(qcx, error, Q::HANDLE_CYCLE_ERROR, cache);
+ let result = mk_cycle(qcx, error, Q::HANDLE_CYCLE_ERROR);
(result, None)
}
#[cfg(parallel_compiler)]
TryGetJob::JobCompleted(query_blocked_prof_timer) => {
- let (v, index) = cache
- .lookup(&key, |value, index| (value.clone(), index))
- .unwrap_or_else(|_| panic!("value must be in cache after waiting"));
+ let Some((v, index)) = Q::query_cache(qcx).lookup(&key) else {
+ panic!("value must be in cache after waiting")
+ };
- if std::intrinsics::unlikely(qcx.dep_context().profiler().enabled()) {
- qcx.dep_context().profiler().query_cache_hit(index.into());
- }
+ qcx.dep_context().profiler().query_cache_hit(index.into());
query_blocked_prof_timer.finish_with_query_invocation_id(index.into());
(v, Some(index))
@@ -414,6 +396,7 @@ where
}
}
+#[inline(always)]
fn execute_job<Q, Qcx>(
qcx: Qcx,
key: Q::Key,
@@ -428,12 +411,27 @@ where
// Fast path for when incr. comp. is off.
if !dep_graph.is_fully_enabled() {
+ // Fingerprint the key, just to assert that it doesn't
+ // have anything we don't consider hashable
+ if cfg!(debug_assertions) {
+ let _ = key.to_fingerprint(*qcx.dep_context());
+ }
+
let prof_timer = qcx.dep_context().profiler().query_provider();
- let result = qcx.start_query(job_id, Q::DEPTH_LIMIT, None, || {
- Q::compute(qcx, &key)(*qcx.dep_context(), key)
- });
+ let result = qcx.start_query(job_id, Q::DEPTH_LIMIT, None, || Q::compute(qcx, key));
let dep_node_index = dep_graph.next_virtual_depnode_index();
prof_timer.finish_with_query_invocation_id(dep_node_index.into());
+
+ // Similarly, fingerprint the result to assert that
+ // it doesn't have anything not considered hashable.
+ if cfg!(debug_assertions)
+ && let Some(hash_result) = Q::HASH_RESULT
+ {
+ qcx.dep_context().with_stable_hashing_context(|mut hcx| {
+ hash_result(&mut hcx, &result);
+ });
+ }
+
return (result, dep_node_index);
}
@@ -457,17 +455,15 @@ where
let (result, dep_node_index) =
qcx.start_query(job_id, Q::DEPTH_LIMIT, Some(&diagnostics), || {
if Q::ANON {
- return dep_graph.with_anon_task(*qcx.dep_context(), Q::DEP_KIND, || {
- Q::compute(qcx, &key)(*qcx.dep_context(), key)
- });
+ return dep_graph
+ .with_anon_task(*qcx.dep_context(), Q::DEP_KIND, || Q::compute(qcx, key));
}
// `to_dep_node` is expensive for some `DepKind`s.
let dep_node =
dep_node_opt.unwrap_or_else(|| Q::construct_dep_node(*qcx.dep_context(), &key));
- let task = Q::compute(qcx, &key);
- dep_graph.with_task(dep_node, *qcx.dep_context(), key, task, Q::HASH_RESULT)
+ dep_graph.with_task(dep_node, qcx, key, Q::compute, Q::HASH_RESULT)
});
prof_timer.finish_with_query_invocation_id(dep_node_index.into());
@@ -486,6 +482,7 @@ where
(result, dep_node_index)
}
+#[inline(always)]
fn try_load_from_disk_and_cache_in_memory<Q, Qcx>(
qcx: Qcx,
key: &Q::Key,
@@ -558,7 +555,7 @@ where
let prof_timer = qcx.dep_context().profiler().query_provider();
// The dep-graph for this computation is already in-place.
- let result = dep_graph.with_ignore(|| Q::compute(qcx, key)(*qcx.dep_context(), key.clone()));
+ let result = dep_graph.with_ignore(|| Q::compute(qcx, *key));
prof_timer.finish_with_query_invocation_id(dep_node_index.into());
@@ -576,6 +573,7 @@ where
Some((result, dep_node_index))
}
+#[inline]
#[instrument(skip(tcx, result, hash_result), level = "debug")]
pub(crate) fn incremental_verify_ich<Tcx, V: Debug>(
tcx: Tcx,
@@ -730,7 +728,8 @@ pub enum QueryMode {
Ensure,
}
-pub fn get_query<Q, Qcx, D>(qcx: Qcx, span: Span, key: Q::Key, mode: QueryMode) -> Option<Q::Stored>
+#[inline(always)]
+pub fn get_query<Q, Qcx, D>(qcx: Qcx, span: Span, key: Q::Key, mode: QueryMode) -> Option<Q::Value>
where
D: DepKind,
Q: QueryConfig<Qcx>,
@@ -747,14 +746,8 @@ where
None
};
- let (result, dep_node_index) = try_execute_query::<Q, Qcx>(
- qcx,
- Q::query_state(qcx),
- Q::query_cache(qcx),
- span,
- key,
- dep_node,
- );
+ let (result, dep_node_index) =
+ ensure_sufficient_stack(|| try_execute_query::<Q, Qcx>(qcx, span, key, dep_node));
if let Some(dep_node_index) = dep_node_index {
qcx.dep_context().dep_graph().read_index(dep_node_index)
}
@@ -770,20 +763,12 @@ where
{
// We may be concurrently trying both execute and force a query.
// Ensure that only one of them runs the query.
- let cache = Q::query_cache(qcx);
- let cached = cache.lookup(&key, |_, index| {
- if std::intrinsics::unlikely(qcx.dep_context().profiler().enabled()) {
- qcx.dep_context().profiler().query_cache_hit(index.into());
- }
- });
-
- match cached {
- Ok(()) => return,
- Err(()) => {}
+ if let Some((_, index)) = Q::query_cache(qcx).lookup(&key) {
+ qcx.dep_context().profiler().query_cache_hit(index.into());
+ return;
}
- let state = Q::query_state(qcx);
debug_assert!(!Q::ANON);
- try_execute_query::<Q, _>(qcx, state, cache, DUMMY_SP, key, Some(dep_node));
+ ensure_sufficient_stack(|| try_execute_query::<Q, _>(qcx, DUMMY_SP, key, Some(dep_node)));
}