diff options
Diffstat (limited to 'compiler/rustc_query_system/src/query')
| -rw-r--r-- | compiler/rustc_query_system/src/query/caches.rs | 24 | ||||
| -rw-r--r-- | compiler/rustc_query_system/src/query/config.rs | 9 | ||||
| -rw-r--r-- | compiler/rustc_query_system/src/query/plumbing.rs | 197 |
3 files changed, 130 insertions, 100 deletions
diff --git a/compiler/rustc_query_system/src/query/caches.rs b/compiler/rustc_query_system/src/query/caches.rs index 1ec32939d9f..d589c90fa7b 100644 --- a/compiler/rustc_query_system/src/query/caches.rs +++ b/compiler/rustc_query_system/src/query/caches.rs @@ -1,5 +1,5 @@ use crate::dep_graph::DepNodeIndex; -use crate::query::plumbing::{QueryLookup, QueryState}; +use crate::query::plumbing::{QueryCacheStore, QueryLookup}; use rustc_arena::TypedArena; use rustc_data_structures::fx::FxHashMap; @@ -31,13 +31,13 @@ pub trait QueryCache: QueryStorage { /// 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<'s, D, Q, R, OnHit>( + fn lookup<'s, R, OnHit>( &self, - state: &'s QueryState<D, Q, Self>, + state: &'s QueryCacheStore<Self>, key: &Self::Key, // `on_hit` can be called while holding a lock to the query state shard. on_hit: OnHit, - ) -> Result<R, QueryLookup<'s, D, Q, Self::Key, Self::Sharded>> + ) -> Result<R, QueryLookup<'s, Self::Sharded>> where OnHit: FnOnce(&Self::Stored, DepNodeIndex) -> R; @@ -93,17 +93,17 @@ where type Sharded = FxHashMap<K, (V, DepNodeIndex)>; #[inline(always)] - fn lookup<'s, D, Q, R, OnHit>( + fn lookup<'s, R, OnHit>( &self, - state: &'s QueryState<D, Q, Self>, + state: &'s QueryCacheStore<Self>, key: &K, on_hit: OnHit, - ) -> Result<R, QueryLookup<'s, D, Q, K, Self::Sharded>> + ) -> Result<R, QueryLookup<'s, Self::Sharded>> where OnHit: FnOnce(&V, DepNodeIndex) -> R, { let lookup = state.get_lookup(key); - let result = lookup.lock.cache.raw_entry().from_key_hashed_nocheck(lookup.key_hash, key); + let result = lookup.lock.raw_entry().from_key_hashed_nocheck(lookup.key_hash, key); if let Some((_, value)) = result { let hit_result = on_hit(&value.0, value.1); @@ -176,17 +176,17 @@ where type Sharded = FxHashMap<K, &'tcx (V, DepNodeIndex)>; #[inline(always)] - fn lookup<'s, D, Q, R, OnHit>( + fn lookup<'s, R, OnHit>( &self, - state: &'s QueryState<D, Q, Self>, + state: &'s QueryCacheStore<Self>, key: &K, on_hit: OnHit, - ) -> Result<R, QueryLookup<'s, D, Q, K, Self::Sharded>> + ) -> Result<R, QueryLookup<'s, Self::Sharded>> where OnHit: FnOnce(&&'tcx V, DepNodeIndex) -> R, { let lookup = state.get_lookup(key); - let result = lookup.lock.cache.raw_entry().from_key_hashed_nocheck(lookup.key_hash, key); + let result = lookup.lock.raw_entry().from_key_hashed_nocheck(lookup.key_hash, key); if let Some((_, value)) = result { let hit_result = on_hit(&&value.0, value.1); diff --git a/compiler/rustc_query_system/src/query/config.rs b/compiler/rustc_query_system/src/query/config.rs index 94e906fc433..fecd75049fb 100644 --- a/compiler/rustc_query_system/src/query/config.rs +++ b/compiler/rustc_query_system/src/query/config.rs @@ -4,7 +4,7 @@ use crate::dep_graph::DepNode; use crate::dep_graph::SerializedDepNodeIndex; use crate::query::caches::QueryCache; use crate::query::plumbing::CycleError; -use crate::query::{QueryContext, QueryState}; +use crate::query::{QueryCacheStore, QueryContext, QueryState}; use rustc_data_structures::fingerprint::Fingerprint; use std::fmt::Debug; @@ -73,7 +73,12 @@ pub trait QueryAccessors<CTX: QueryContext>: QueryConfig { type Cache: QueryCache<Key = Self::Key, Stored = Self::Stored, Value = Self::Value>; // Don't use this method to access query results, instead use the methods on TyCtxt - fn query_state<'a>(tcx: CTX) -> &'a QueryState<CTX::DepKind, CTX::Query, Self::Cache>; + fn query_state<'a>(tcx: CTX) -> &'a QueryState<CTX::DepKind, CTX::Query, Self::Key>; + + // Don't use this method to access query results, instead use the methods on TyCtxt + fn query_cache<'a>(tcx: CTX) -> &'a QueryCacheStore<Self::Cache> + where + CTX: 'a; fn to_dep_node(tcx: CTX, key: &Self::Key) -> DepNode<CTX::DepKind> where diff --git a/compiler/rustc_query_system/src/query/plumbing.rs b/compiler/rustc_query_system/src/query/plumbing.rs index 4f93017200f..51a72594b5e 100644 --- a/compiler/rustc_query_system/src/query/plumbing.rs +++ b/compiler/rustc_query_system/src/query/plumbing.rs @@ -13,7 +13,7 @@ use crate::query::{QueryContext, QueryMap}; use rustc_data_structures::cold_path; use rustc_data_structures::fingerprint::Fingerprint; use rustc_data_structures::fx::{FxHashMap, FxHasher}; -use rustc_data_structures::sharded::Sharded; +use rustc_data_structures::sharded::{get_shard_index_by_hash, Sharded}; use rustc_data_structures::sync::{Lock, LockGuard}; use rustc_data_structures::thin_vec::ThinVec; use rustc_errors::{Diagnostic, FatalError}; @@ -27,43 +27,73 @@ use std::ptr; #[cfg(debug_assertions)] use std::sync::atomic::{AtomicUsize, Ordering}; -pub(super) struct QueryStateShard<D, Q, K, C> { - pub(super) cache: C, - active: FxHashMap<K, QueryResult<D, Q>>, - - /// Used to generate unique ids for active jobs. - jobs: u32, +pub struct QueryCacheStore<C: QueryCache> { + cache: C, + shards: Sharded<C::Sharded>, + #[cfg(debug_assertions)] + pub cache_hits: AtomicUsize, } -impl<D, Q, K, C: Default> Default for QueryStateShard<D, Q, K, C> { - fn default() -> QueryStateShard<D, Q, K, C> { - QueryStateShard { cache: Default::default(), active: Default::default(), jobs: 0 } +impl<C: QueryCache> Default for QueryCacheStore<C> { + fn default() -> Self { + Self { + cache: C::default(), + shards: Default::default(), + #[cfg(debug_assertions)] + cache_hits: AtomicUsize::new(0), + } } } -pub struct QueryState<D, Q, C: QueryCache> { - cache: C, - shards: Sharded<QueryStateShard<D, Q, C::Key, C::Sharded>>, - #[cfg(debug_assertions)] - pub cache_hits: AtomicUsize, +/// Values used when checking a query cache which can be reused on a cache-miss to execute the query. +pub struct QueryLookup<'tcx, C> { + pub(super) key_hash: u64, + shard: usize, + pub(super) lock: LockGuard<'tcx, C>, } -impl<D, Q, C: QueryCache> QueryState<D, Q, C> { - pub(super) fn get_lookup<'tcx>( - &'tcx self, - key: &C::Key, - ) -> QueryLookup<'tcx, D, Q, C::Key, C::Sharded> { - // We compute the key's hash once and then use it for both the - // shard lookup and the hashmap lookup. This relies on the fact - // that both of them use `FxHasher`. - let mut hasher = FxHasher::default(); - key.hash(&mut hasher); - let key_hash = hasher.finish(); - - let shard = self.shards.get_shard_index_by_hash(key_hash); +// We compute the key's hash once and then use it for both the +// shard lookup and the hashmap lookup. This relies on the fact +// that both of them use `FxHasher`. +fn hash_for_shard<K: Hash>(key: &K) -> u64 { + let mut hasher = FxHasher::default(); + key.hash(&mut hasher); + hasher.finish() +} + +impl<C: QueryCache> QueryCacheStore<C> { + pub(super) fn get_lookup<'tcx>(&'tcx self, key: &C::Key) -> QueryLookup<'tcx, C::Sharded> { + let key_hash = hash_for_shard(key); + let shard = get_shard_index_by_hash(key_hash); let lock = self.shards.get_shard_by_index(shard).lock(); QueryLookup { key_hash, shard, lock } } + + pub fn iter_results<R>( + &self, + f: impl for<'a> FnOnce( + Box<dyn Iterator<Item = (&'a C::Key, &'a C::Value, DepNodeIndex)> + 'a>, + ) -> R, + ) -> R { + self.cache.iter(&self.shards, |shard| &mut *shard, f) + } +} + +struct QueryStateShard<D, Q, K> { + active: FxHashMap<K, QueryResult<D, Q>>, + + /// Used to generate unique ids for active jobs. + jobs: u32, +} + +impl<D, Q, K> Default for QueryStateShard<D, Q, K> { + fn default() -> QueryStateShard<D, Q, K> { + QueryStateShard { active: Default::default(), jobs: 0 } + } +} + +pub struct QueryState<D, Q, K> { + shards: Sharded<QueryStateShard<D, Q, K>>, } /// Indicates the state of a query for a given key in a query map. @@ -76,21 +106,12 @@ enum QueryResult<D, Q> { Poisoned, } -impl<D, Q, C> QueryState<D, Q, C> +impl<D, Q, K> QueryState<D, Q, K> where D: Copy + Clone + Eq + Hash, Q: Clone, - C: QueryCache, + K: Eq + Hash + Clone + Debug, { - pub fn iter_results<R>( - &self, - f: impl for<'a> FnOnce( - Box<dyn Iterator<Item = (&'a C::Key, &'a C::Value, DepNodeIndex)> + 'a>, - ) -> R, - ) -> R { - self.cache.iter(&self.shards, |shard| &mut shard.cache, f) - } - pub fn all_inactive(&self) -> bool { let shards = self.shards.lock_shards(); shards.iter().all(|shard| shard.active.is_empty()) @@ -99,7 +120,7 @@ where pub fn try_collect_active_jobs( &self, kind: D, - make_query: fn(C::Key) -> Q, + make_query: fn(K) -> Q, jobs: &mut QueryMap<D, Q>, ) -> Option<()> { // We use try_lock_shards here since we are called from the @@ -122,24 +143,12 @@ where } } -impl<D, Q, C: QueryCache> Default for QueryState<D, Q, C> { - fn default() -> QueryState<D, Q, C> { - QueryState { - cache: C::default(), - shards: Default::default(), - #[cfg(debug_assertions)] - cache_hits: AtomicUsize::new(0), - } +impl<D, Q, K> Default for QueryState<D, Q, K> { + fn default() -> QueryState<D, Q, K> { + QueryState { shards: Default::default() } } } -/// Values used when checking a query cache which can be reused on a cache-miss to execute the query. -pub struct QueryLookup<'tcx, D, Q, K, C> { - pub(super) key_hash: u64, - shard: usize, - pub(super) lock: LockGuard<'tcx, QueryStateShard<D, Q, K, C>>, -} - /// A type representing the responsibility to execute the job in the `job` field. /// This will poison the relevant query if dropped. struct JobOwner<'tcx, D, Q, C> @@ -148,7 +157,8 @@ where Q: Clone, C: QueryCache, { - state: &'tcx QueryState<D, Q, C>, + state: &'tcx QueryState<D, Q, C::Key>, + cache: &'tcx QueryCacheStore<C>, key: C::Key, id: QueryJobId<D>, } @@ -170,16 +180,20 @@ where #[inline(always)] fn try_start<'a, 'b, CTX>( tcx: CTX, - state: &'b QueryState<CTX::DepKind, CTX::Query, C>, + state: &'b QueryState<CTX::DepKind, CTX::Query, C::Key>, + cache: &'b QueryCacheStore<C>, span: Span, key: &C::Key, - mut lookup: QueryLookup<'a, CTX::DepKind, CTX::Query, C::Key, C::Sharded>, + lookup: QueryLookup<'a, C::Sharded>, query: &QueryVtable<CTX, C::Key, C::Value>, ) -> TryGetJob<'b, CTX::DepKind, CTX::Query, C> where CTX: QueryContext, { - let lock = &mut *lookup.lock; + mem::drop(lookup.lock); + let shard = lookup.shard; + let mut state_lock = state.shards.get_shard_by_index(shard).lock(); + let lock = &mut *state_lock; let (latch, mut _query_blocked_prof_timer) = match lock.active.entry((*key).clone()) { Entry::Occupied(mut entry) => { @@ -195,7 +209,7 @@ where }; // Create the id of the job we're waiting for - let id = QueryJobId::new(job.id, lookup.shard, query.dep_kind); + let id = QueryJobId::new(job.id, shard, query.dep_kind); (job.latch(id), _query_blocked_prof_timer) } @@ -210,18 +224,18 @@ where lock.jobs = id; let id = QueryShardJobId(NonZeroU32::new(id).unwrap()); - let global_id = QueryJobId::new(id, lookup.shard, query.dep_kind); + let global_id = QueryJobId::new(id, shard, query.dep_kind); let job = tcx.current_query_job(); let job = QueryJob::new(id, span, job); entry.insert(QueryResult::Started(job)); - let owner = JobOwner { state, id: global_id, key: (*key).clone() }; + let owner = JobOwner { state, cache, id: global_id, key: (*key).clone() }; return TryGetJob::NotYetStarted(owner); } }; - mem::drop(lookup.lock); + mem::drop(state_lock); // If we are single-threaded we know that we have cycle error, // so we just return the error. @@ -233,7 +247,7 @@ where span, ); let value = query.handle_cycle_error(tcx, error); - state.cache.store_nocache(value) + cache.cache.store_nocache(value) })); // With parallel queries we might just have to wait on some other @@ -244,11 +258,11 @@ where if let Err(cycle) = result { let value = query.handle_cycle_error(tcx, cycle); - let value = state.cache.store_nocache(value); + let value = cache.cache.store_nocache(value); return TryGetJob::Cycle(value); } - let cached = try_get_cached(tcx, state, key, |value, index| (value.clone(), index)) + let cached = try_get_cached(tcx, cache, key, |value, index| (value.clone(), index)) .unwrap_or_else(|_| panic!("value must be in cache after waiting")); if let Some(prof_timer) = _query_blocked_prof_timer.take() { @@ -265,17 +279,25 @@ where // We can move out of `self` here because we `mem::forget` it below let key = unsafe { ptr::read(&self.key) }; let state = self.state; + let cache = self.cache; // Forget ourself so our destructor won't poison the query mem::forget(self); let (job, result) = { - let mut lock = state.shards.get_shard_by_value(&key).lock(); - let job = match lock.active.remove(&key).unwrap() { - QueryResult::Started(job) => job, - QueryResult::Poisoned => panic!(), + let key_hash = hash_for_shard(&key); + let shard = get_shard_index_by_hash(key_hash); + let job = { + let mut lock = state.shards.get_shard_by_index(shard).lock(); + match lock.active.remove(&key).unwrap() { + QueryResult::Started(job) => job, + QueryResult::Poisoned => panic!(), + } + }; + let result = { + let mut lock = cache.shards.get_shard_by_index(shard).lock(); + cache.cache.complete(&mut lock, key, result, dep_node_index) }; - let result = state.cache.complete(&mut lock.cache, key, result, dep_node_index); (job, result) }; @@ -353,23 +375,23 @@ where /// to compute it. fn try_get_cached<'a, CTX, C, R, OnHit>( tcx: CTX, - state: &'a QueryState<CTX::DepKind, CTX::Query, C>, + cache: &'a QueryCacheStore<C>, key: &C::Key, // `on_hit` can be called while holding a lock to the query cache on_hit: OnHit, -) -> Result<R, QueryLookup<'a, CTX::DepKind, CTX::Query, C::Key, C::Sharded>> +) -> Result<R, QueryLookup<'a, C::Sharded>> where C: QueryCache, CTX: QueryContext, OnHit: FnOnce(&C::Stored, DepNodeIndex) -> R, { - state.cache.lookup(state, &key, |value, index| { + cache.cache.lookup(cache, &key, |value, index| { if unlikely!(tcx.profiler().enabled()) { tcx.profiler().query_cache_hit(index.into()); } #[cfg(debug_assertions)] { - state.cache_hits.fetch_add(1, Ordering::Relaxed); + cache.cache_hits.fetch_add(1, Ordering::Relaxed); } on_hit(value, index) }) @@ -377,10 +399,11 @@ where fn try_execute_query<CTX, C>( tcx: CTX, - state: &QueryState<CTX::DepKind, CTX::Query, C>, + state: &QueryState<CTX::DepKind, CTX::Query, C::Key>, + cache: &QueryCacheStore<C>, span: Span, key: C::Key, - lookup: QueryLookup<'_, CTX::DepKind, CTX::Query, C::Key, C::Sharded>, + lookup: QueryLookup<'_, C::Sharded>, query: &QueryVtable<CTX, C::Key, C::Value>, ) -> C::Stored where @@ -389,7 +412,7 @@ where CTX: QueryContext, { let job = match JobOwner::<'_, CTX::DepKind, CTX::Query, C>::try_start( - tcx, state, span, &key, lookup, query, + tcx, state, cache, span, &key, lookup, query, ) { TryGetJob::NotYetStarted(job) => job, TryGetJob::Cycle(result) => return result, @@ -604,7 +627,8 @@ where #[inline(never)] fn get_query_impl<CTX, C>( tcx: CTX, - state: &QueryState<CTX::DepKind, CTX::Query, C>, + state: &QueryState<CTX::DepKind, CTX::Query, C::Key>, + cache: &QueryCacheStore<C>, span: Span, key: C::Key, query: &QueryVtable<CTX, C::Key, C::Value>, @@ -614,13 +638,13 @@ where C: QueryCache, C::Key: crate::dep_graph::DepNodeParams<CTX>, { - let cached = try_get_cached(tcx, state, &key, |value, index| { + let cached = try_get_cached(tcx, cache, &key, |value, index| { tcx.dep_graph().read_index(index); value.clone() }); match cached { Ok(value) => value, - Err(lookup) => try_execute_query(tcx, state, span, key, lookup, query), + Err(lookup) => try_execute_query(tcx, state, cache, span, key, lookup, query), } } @@ -667,7 +691,8 @@ where #[inline(never)] fn force_query_impl<CTX, C>( tcx: CTX, - state: &QueryState<CTX::DepKind, CTX::Query, C>, + state: &QueryState<CTX::DepKind, CTX::Query, C::Key>, + cache: &QueryCacheStore<C>, key: C::Key, span: Span, dep_node: DepNode<CTX::DepKind>, @@ -680,7 +705,7 @@ fn force_query_impl<CTX, C>( // We may be concurrently trying both execute and force a query. // Ensure that only one of them runs the query. - let cached = try_get_cached(tcx, state, &key, |_, _| { + let cached = try_get_cached(tcx, cache, &key, |_, _| { // Cache hit, do nothing }); @@ -690,7 +715,7 @@ fn force_query_impl<CTX, C>( }; let job = match JobOwner::<'_, CTX::DepKind, CTX::Query, C>::try_start( - tcx, state, span, &key, lookup, query, + tcx, state, cache, span, &key, lookup, query, ) { TryGetJob::NotYetStarted(job) => job, TryGetJob::Cycle(_) => return, @@ -719,7 +744,7 @@ where } debug!("ty::query::get_query<{}>(key={:?}, span={:?})", Q::NAME, key, span); - let value = get_query_impl(tcx, Q::query_state(tcx), span, key, query); + let value = get_query_impl(tcx, Q::query_state(tcx), Q::query_cache(tcx), span, key, query); Some(value) } @@ -729,5 +754,5 @@ where Q::Key: crate::dep_graph::DepNodeParams<CTX>, CTX: QueryContext, { - force_query_impl(tcx, Q::query_state(tcx), key, span, dep_node, &Q::VTABLE) + force_query_impl(tcx, Q::query_state(tcx), Q::query_cache(tcx), key, span, dep_node, &Q::VTABLE) } |
