Separate the query cache from the query state.
This commit is contained in:
parent
9f46259a75
commit
15b0bc6b83
9 changed files with 174 additions and 136 deletions
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
pub struct QueryCacheStore<C: QueryCache> {
|
||||
cache: C,
|
||||
shards: Sharded<C::Sharded>,
|
||||
#[cfg(debug_assertions)]
|
||||
pub cache_hits: AtomicUsize,
|
||||
}
|
||||
|
||||
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),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// 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>,
|
||||
}
|
||||
|
||||
// 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, 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<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, C: QueryCache> {
|
||||
cache: C,
|
||||
shards: Sharded<QueryStateShard<D, Q, C::Key, C::Sharded>>,
|
||||
#[cfg(debug_assertions)]
|
||||
pub cache_hits: AtomicUsize,
|
||||
}
|
||||
|
||||
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);
|
||||
let lock = self.shards.get_shard_by_index(shard).lock();
|
||||
QueryLookup { key_hash, shard, lock }
|
||||
}
|
||||
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)
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue