1
Fork 0

Lazy DefPath decoding for incremental compilation

This commit is contained in:
Aaron Hill 2020-07-29 12:26:15 -04:00
parent db79d2f637
commit e935d3832c
No known key found for this signature in database
GPG key ID: B4087E510E98B164
11 changed files with 275 additions and 43 deletions

View file

@ -252,8 +252,8 @@ macro_rules! define_dep_nodes {
/// has been removed.
fn extract_def_id(&self, tcx: TyCtxt<'tcx>) -> Option<DefId> {
if self.kind.can_reconstruct_query_key() {
let def_path_hash = DefPathHash(self.hash.into());
tcx.def_path_hash_to_def_id.as_ref()?.get(&def_path_hash).cloned()
let def_path_hash = DefPathHash(self.hash);
tcx.queries.on_disk_cache.as_ref()?.def_path_hash_to_def_id(tcx, def_path_hash)
} else {
None
}
@ -320,7 +320,15 @@ impl<'tcx> DepNodeParams<TyCtxt<'tcx>> for DefId {
}
fn to_fingerprint(&self, tcx: TyCtxt<'tcx>) -> Fingerprint {
tcx.def_path_hash(*self).0
let hash = tcx.def_path_hash(*self);
// If this is a foreign `DefId`, store its current value
// in the incremental cache. When we decode the cache,
// we will use the old DefIndex as an initial guess for
// a lookup into the crate metadata.
if !self.is_local() {
tcx.queries.on_disk_cache.store_foreign_def_id_hash(*self, hash);
}
hash.0
}
fn to_debug_str(&self, tcx: TyCtxt<'tcx>) -> String {
@ -359,7 +367,7 @@ impl<'tcx> DepNodeParams<TyCtxt<'tcx>> for CrateNum {
fn to_fingerprint(&self, tcx: TyCtxt<'tcx>) -> Fingerprint {
let def_id = DefId { krate: *self, index: CRATE_DEF_INDEX };
tcx.def_path_hash(def_id).0
def_id.to_fingerprint(tcx)
}
fn to_debug_str(&self, tcx: TyCtxt<'tcx>) -> String {

View file

@ -5,7 +5,7 @@ use rustc_data_structures::profiling::SelfProfilerRef;
use rustc_data_structures::sync::Lock;
use rustc_data_structures::thin_vec::ThinVec;
use rustc_errors::Diagnostic;
use rustc_hir::def_id::LocalDefId;
use rustc_hir::def_id::{DefPathHash, LocalDefId};
mod dep_node;
@ -91,6 +91,10 @@ impl<'tcx> DepContext for TyCtxt<'tcx> {
type DepKind = DepKind;
type StableHashingContext = StableHashingContext<'tcx>;
fn register_reused_dep_path_hash(&self, hash: DefPathHash) {
self.queries.on_disk_cache.register_reused_dep_path_hash(hash)
}
fn create_stable_hashing_context(&self) -> Self::StableHashingContext {
TyCtxt::create_stable_hashing_context(*self)
}

View file

@ -191,6 +191,12 @@ pub trait CrateStore {
fn def_path_hash(&self, def: DefId) -> DefPathHash;
fn all_def_path_hashes_and_def_ids(&self, cnum: CrateNum) -> Vec<(DefPathHash, DefId)>;
fn num_def_ids(&self, cnum: CrateNum) -> usize;
fn def_path_hash_to_def_id(
&self,
cnum: CrateNum,
index_guess: u32,
hash: DefPathHash,
) -> Option<DefId>;
// "queries" used in resolve that aren't tracked for incremental compilation
fn crate_name_untracked(&self, cnum: CrateNum) -> Symbol;

View file

@ -1,7 +1,7 @@
//! Type context book-keeping.
use crate::arena::Arena;
use crate::dep_graph::{self, DepConstructor, DepGraph};
use crate::dep_graph::{self, DepGraph, DepKind, DepNode, DepNodeExt};
use crate::hir::exports::ExportMap;
use crate::ich::{NodeIdHashingMode, StableHashingContext};
use crate::infer::canonical::{Canonical, CanonicalVarInfo, CanonicalVarInfos};
@ -34,12 +34,12 @@ use rustc_data_structures::stable_hasher::{
};
use rustc_data_structures::steal::Steal;
use rustc_data_structures::sync::{self, Lock, Lrc, WorkerLocal};
use rustc_data_structures::unhash::UnhashMap;
use rustc_errors::ErrorReported;
use rustc_hir as hir;
use rustc_hir::def::{DefKind, Res};
use rustc_hir::def_id::{CrateNum, DefId, DefIdMap, LocalDefId, LOCAL_CRATE};
use rustc_hir::definitions::{DefPathHash, Definitions};
use rustc_hir::def_id::{CrateNum, DefId, DefIdMap, LocalDefId};
use rustc_hir::def_id::{CRATE_DEF_INDEX, LOCAL_CRATE};
use rustc_hir::definitions::Definitions;
use rustc_hir::intravisit::Visitor;
use rustc_hir::lang_items::LangItem;
use rustc_hir::{
@ -945,10 +945,6 @@ pub struct GlobalCtxt<'tcx> {
pub(crate) untracked_crate: &'tcx hir::Crate<'tcx>,
pub(crate) definitions: &'tcx Definitions,
/// A map from `DefPathHash` -> `DefId`. Includes `DefId`s from the local crate
/// as well as all upstream crates. Only populated in incremental mode.
pub def_path_hash_to_def_id: Option<UnhashMap<DefPathHash, DefId>>,
pub queries: query::Queries<'tcx>,
maybe_unused_trait_imports: FxHashSet<LocalDefId>,
@ -1113,21 +1109,6 @@ impl<'tcx> TyCtxt<'tcx> {
let mut providers = IndexVec::from_elem_n(extern_providers, max_cnum + 1);
providers[LOCAL_CRATE] = local_providers;
let def_path_hash_to_def_id = if s.opts.build_dep_graph() {
let capacity = definitions.def_path_table().num_def_ids()
+ crates.iter().map(|cnum| cstore.num_def_ids(*cnum)).sum::<usize>();
let mut map = UnhashMap::with_capacity_and_hasher(capacity, Default::default());
map.extend(definitions.def_path_table().all_def_path_hashes_and_def_ids(LOCAL_CRATE));
for cnum in &crates {
map.extend(cstore.all_def_path_hashes_and_def_ids(*cnum).into_iter());
}
Some(map)
} else {
None
};
let mut trait_map: FxHashMap<_, FxHashMap<_, _>> = FxHashMap::default();
for (hir_id, v) in krate.trait_map.iter() {
let map = trait_map.entry(hir_id.owner).or_default();
@ -1155,7 +1136,6 @@ impl<'tcx> TyCtxt<'tcx> {
extern_prelude: resolutions.extern_prelude,
untracked_crate: krate,
definitions,
def_path_hash_to_def_id,
queries: query::Queries::new(providers, extern_providers, on_disk_query_result_cache),
ty_rcache: Default::default(),
pred_rcache: Default::default(),
@ -1329,7 +1309,8 @@ impl<'tcx> TyCtxt<'tcx> {
// We cannot use the query versions of crates() and crate_hash(), since
// those would need the DepNodes that we are allocating here.
for cnum in self.cstore.crates_untracked() {
let dep_node = DepConstructor::CrateMetadata(self, cnum);
let def_path_hash = self.def_path_hash(DefId { krate: cnum, index: CRATE_DEF_INDEX });
let dep_node = DepNode::from_def_path_hash(def_path_hash, DepKind::CrateMetadata);
let crate_hash = self.cstore.crate_hash_untracked(cnum);
self.dep_graph.with_task(
dep_node,

View file

@ -11,6 +11,7 @@ use rustc_data_structures::thin_vec::ThinVec;
use rustc_errors::Diagnostic;
use rustc_hir::def_id::{CrateNum, DefId, DefIndex, LocalDefId, LOCAL_CRATE};
use rustc_hir::definitions::DefPathHash;
use rustc_hir::definitions::Definitions;
use rustc_index::vec::{Idx, IndexVec};
use rustc_serialize::{opaque, Decodable, Decoder, Encodable, Encoder};
use rustc_session::{CrateDisambiguator, Session};
@ -21,6 +22,8 @@ use rustc_span::hygiene::{
use rustc_span::source_map::{SourceMap, StableSourceFileId};
use rustc_span::CachingSourceMapView;
use rustc_span::{BytePos, ExpnData, SourceFile, Span, DUMMY_SP};
use std::collections::hash_map::Entry;
use std::iter::FromIterator;
use std::mem;
const TAG_FILE_FOOTER: u128 = 0xC0FFEE_C0FFEE_C0FFEE_C0FFEE_C0FFEE;
@ -80,6 +83,28 @@ pub struct OnDiskCache<'sess> {
expn_data: FxHashMap<u32, AbsoluteBytePos>,
// Additional information used when decoding hygiene data.
hygiene_context: HygieneDecodeContext,
// Maps `DefPathHash`es to their `RawDefId`s from the *previous*
// compilation session. This is used as an initial 'guess' when
// we try to map a `DefPathHash` to its `DefId` in the current compilation
// session.
foreign_def_path_hashes: FxHashMap<DefPathHash, RawDefId>,
// The *next* compilation sessison's `foreign_def_path_hashes` - at
// the end of our current compilation session, this will get written
// out to the `foreign_def_path_hashes` field of the `Footer`, which
// will become `foreign_def_path_hashes` of the next compilation session.
// This stores any `DefPathHash` that we may need to map to a `DefId`
// during the next compilation session.
latest_foreign_def_path_hashes: Lock<FxHashMap<DefPathHash, RawDefId>>,
// Maps `DefPathHashes` to their corresponding `LocalDefId`s for all
// local items in the current compilation session.
local_def_path_hash_to_def_id: FxHashMap<DefPathHash, LocalDefId>,
// Caches all lookups of `DefPathHashes`, both for local and foreign
// definitions. A definition from the previous compilation session
// may no longer exist in the current compilation session, so
// we use `Option<DefId>` so that we can cache a lookup failure.
def_path_hash_to_def_id_cache: Lock<FxHashMap<DefPathHash, Option<DefId>>>,
}
// This type is used only for serialization and deserialization.
@ -95,6 +120,7 @@ struct Footer {
syntax_contexts: FxHashMap<u32, AbsoluteBytePos>,
// See `OnDiskCache.expn_data`
expn_data: FxHashMap<u32, AbsoluteBytePos>,
foreign_def_path_hashes: FxHashMap<DefPathHash, RawDefId>,
}
type EncodedQueryResultIndex = Vec<(SerializedDepNodeIndex, AbsoluteBytePos)>;
@ -118,9 +144,36 @@ impl AbsoluteBytePos {
}
}
/// Represents a potentially invalid `DefId`. This is used during incremental
/// compilation to represent a `DefId` from the *previous* compilation session,
/// which may no longer be valid. This is used to help map a `DefPathHash`
/// to a `DefId` in the current compilation session.
#[derive(Encodable, Decodable, Copy, Clone, Debug)]
crate struct RawDefId {
// We deliberately do not use `CrateNum` and `DefIndex`
// here, since a crate/index from the previous compilation
// session may no longer exist.
pub krate: u32,
pub index: u32,
}
fn make_local_def_path_hash_map(definitions: &Definitions) -> FxHashMap<DefPathHash, LocalDefId> {
FxHashMap::from_iter(
definitions
.def_path_table()
.all_def_path_hashes_and_def_ids(LOCAL_CRATE)
.map(|(hash, def_id)| (hash, def_id.as_local().unwrap())),
)
}
impl<'sess> OnDiskCache<'sess> {
/// Creates a new `OnDiskCache` instance from the serialized data in `data`.
pub fn new(sess: &'sess Session, data: Vec<u8>, start_pos: usize) -> Self {
pub fn new(
sess: &'sess Session,
data: Vec<u8>,
start_pos: usize,
definitions: &Definitions,
) -> Self {
debug_assert!(sess.opts.incremental.is_some());
// Wrap in a scope so we can borrow `data`.
@ -155,10 +208,14 @@ impl<'sess> OnDiskCache<'sess> {
syntax_contexts: footer.syntax_contexts,
expn_data: footer.expn_data,
hygiene_context: Default::default(),
foreign_def_path_hashes: footer.foreign_def_path_hashes,
latest_foreign_def_path_hashes: Default::default(),
local_def_path_hash_to_def_id: make_local_def_path_hash_map(definitions),
def_path_hash_to_def_id_cache: Default::default(),
}
}
pub fn new_empty(source_map: &'sess SourceMap) -> Self {
pub fn new_empty(source_map: &'sess SourceMap, definitions: &Definitions) -> Self {
Self {
serialized_data: Vec::new(),
file_index_to_stable_id: Default::default(),
@ -173,6 +230,10 @@ impl<'sess> OnDiskCache<'sess> {
syntax_contexts: FxHashMap::default(),
expn_data: FxHashMap::default(),
hygiene_context: Default::default(),
foreign_def_path_hashes: Default::default(),
latest_foreign_def_path_hashes: Default::default(),
local_def_path_hash_to_def_id: make_local_def_path_hash_map(definitions),
def_path_hash_to_def_id_cache: Default::default(),
}
}
@ -200,6 +261,15 @@ impl<'sess> OnDiskCache<'sess> {
(file_to_file_index, file_index_to_stable_id)
};
// Load everything into memory so we can write it out to the on-disk
// cache. The vast majority of cacheable query results should already
// be in memory, so this should be a cheap operation.
// Do this *before* we clone 'latest_foreign_def_path_hashes', since
// loading existing queries may cause us to create new DepNodes, which
// may in turn end up invoking `store_foreign_def_id_hash`
tcx.dep_graph.exec_cache_promotions(tcx);
let latest_foreign_def_path_hashes = self.latest_foreign_def_path_hashes.lock().clone();
let hygiene_encode_context = HygieneEncodeContext::default();
let mut encoder = CacheEncoder {
@ -211,13 +281,9 @@ impl<'sess> OnDiskCache<'sess> {
source_map: CachingSourceMapView::new(tcx.sess.source_map()),
file_to_file_index,
hygiene_context: &hygiene_encode_context,
latest_foreign_def_path_hashes,
};
// Load everything into memory so we can write it out to the on-disk
// cache. The vast majority of cacheable query results should already
// be in memory, so this should be a cheap operation.
tcx.dep_graph.exec_cache_promotions(tcx);
// Encode query results.
let mut query_result_index = EncodedQueryResultIndex::new();
@ -312,6 +378,9 @@ impl<'sess> OnDiskCache<'sess> {
},
)?;
let foreign_def_path_hashes =
std::mem::take(&mut encoder.latest_foreign_def_path_hashes);
// `Encode the file footer.
let footer_pos = encoder.position() as u64;
encoder.encode_tagged(
@ -324,6 +393,7 @@ impl<'sess> OnDiskCache<'sess> {
interpret_alloc_index,
syntax_contexts,
expn_data: expn_ids,
foreign_def_path_hashes,
},
)?;
@ -374,6 +444,31 @@ impl<'sess> OnDiskCache<'sess> {
debug_assert!(prev.is_none());
}
fn get_raw_def_id(&self, hash: &DefPathHash) -> Option<RawDefId> {
self.foreign_def_path_hashes.get(hash).copied()
}
fn try_remap_cnum(&self, tcx: TyCtxt<'_>, cnum: u32) -> Option<CrateNum> {
let cnum_map =
self.cnum_map.get_or_init(|| Self::compute_cnum_map(tcx, &self.prev_cnums[..]));
cnum_map[CrateNum::from_u32(cnum)]
}
pub(crate) fn store_foreign_def_id_hash(&self, def_id: DefId, hash: DefPathHash) {
// We may overwrite an existing entry, but it will have the same value,
// so it's fine
self.latest_foreign_def_path_hashes
.lock()
.insert(hash, RawDefId { krate: def_id.krate.as_u32(), index: def_id.index.as_u32() });
}
pub fn register_reused_dep_path_hash(&self, hash: DefPathHash) {
if let Some(old_id) = self.foreign_def_path_hashes.get(&hash) {
self.latest_foreign_def_path_hashes.lock().insert(hash, *old_id);
}
}
/// Returns the cached query result if there is something in the cache for
/// the given `SerializedDepNodeIndex`; otherwise returns `None`.
crate fn try_load_query_result<'tcx, T>(
@ -481,6 +576,53 @@ impl<'sess> OnDiskCache<'sess> {
map
})
}
/// Converts a `DefPathHash` to its corresponding `DefId` in the current compilation
/// session, if it still exists. This is used during incremental compilation to
/// turn a deserialized `DefPathHash` into its current `DefId`.
pub(crate) fn def_path_hash_to_def_id(
&self,
tcx: TyCtxt<'tcx>,
hash: DefPathHash,
) -> Option<DefId> {
let mut cache = self.def_path_hash_to_def_id_cache.lock();
match cache.entry(hash) {
Entry::Occupied(e) => *e.get(),
Entry::Vacant(e) => {
// Check if the `DefPathHash` corresponds to a definition in the current
// crate
if let Some(def_id) = self.local_def_path_hash_to_def_id.get(&hash).cloned() {
let def_id = def_id.to_def_id();
e.insert(Some(def_id));
return Some(def_id);
}
// This `raw_def_id` represents the `DefId` of this `DefPathHash` in
// the *previous* compliation session. The `DefPathHash` includes the
// owning crate, so if the corresponding definition still exists in the
// current compilation session, the crate is guaranteed to be the same
// (otherwise, we would compute a different `DefPathHash`).
let raw_def_id = self.get_raw_def_id(&hash)?;
// If the owning crate no longer exists, the corresponding definition definitely
// no longer exists.
let krate = self.try_remap_cnum(tcx, raw_def_id.krate)?;
// If our `DefPathHash` corresponded to a definition in the local crate,
// we should have either found it in `local_def_path_hash_to_def_id`, or
// never attempted to load it in the first place. Any query result or `DepNode`
// that references a local `DefId` should depend on some HIR-related `DepNode`.
// If a local definition is removed/modified such that its old `DefPathHash`
// no longer has a corresponding definition, that HIR-related `DepNode` should
// end up red. This should prevent us from ever calling
// `tcx.def_path_hash_to_def_id`, since we'll end up recomputing any
// queries involved.
debug_assert_ne!(krate, LOCAL_CRATE);
// Try to find a definition in the current session, using the previous `DefIndex`
// as an initial guess.
let opt_def_id = tcx.cstore.def_path_hash_to_def_id(krate, raw_def_id.index, hash);
e.insert(opt_def_id);
opt_def_id
}
}
}
}
//- DECODING -------------------------------------------------------------------
@ -713,7 +855,11 @@ impl<'a, 'tcx> Decodable<CacheDecoder<'a, 'tcx>> for DefId {
let def_path_hash = DefPathHash::decode(d)?;
// Using the `DefPathHash`, we can lookup the new `DefId`.
Ok(d.tcx().def_path_hash_to_def_id.as_ref().unwrap()[&def_path_hash])
// Subtle: We only encode a `DefId` as part of a query result.
// If we get to this point, then all of the query inputs were green,
// which means that the definition with this hash is guaranteed to
// still exist in the current compilation session.
Ok(d.tcx().queries.on_disk_cache.def_path_hash_to_def_id(d.tcx(), def_path_hash).unwrap())
}
}
@ -773,6 +919,7 @@ struct CacheEncoder<'a, 'tcx, E: OpaqueEncoder> {
source_map: CachingSourceMapView<'tcx>,
file_to_file_index: FxHashMap<*const SourceFile, SourceFileIndex>,
hygiene_context: &'a HygieneEncodeContext,
latest_foreign_def_path_hashes: FxHashMap<DefPathHash, RawDefId>,
}
impl<'a, 'tcx, E> CacheEncoder<'a, 'tcx, E>
@ -895,6 +1042,15 @@ where
{
fn encode(&self, s: &mut CacheEncoder<'a, 'tcx, E>) -> Result<(), E::Error> {
let def_path_hash = s.tcx.def_path_hash(*self);
// Store additional information when we encode a foreign `DefId`,
// so that we can map its `DefPathHash` back to a `DefId` in the next
// compilation session.
if !self.is_local() {
s.latest_foreign_def_path_hashes.insert(
def_path_hash,
RawDefId { krate: self.krate.as_u32(), index: self.index.as_u32() },
);
}
def_path_hash.encode(s)
}
}