Auto merge of #75813 - petrochenkov:feature/incr-def-path-table, r=Aaron1011

Lazy decoding of DefPathTable from crate metadata (non-incremental case)

The is the half of https://github.com/rust-lang/rust/pull/74967 that doesn't touch incremental-related structures.
We are still decoding def path hashes eagerly if we are in incremental mode.

The incremental part of https://github.com/rust-lang/rust/pull/74967 feels hacky, but I'm not qualified enough to suggest improvements. I'll reassign it so someone else once this PR lands.
@Aaron1011, I wasn't asking you to do this split because I wasn't sure that it's feasible (or simple to do).

r? @Aaron1011
This commit is contained in:
bors 2020-08-23 05:56:47 +00:00
commit d5abc8d3b2
10 changed files with 130 additions and 99 deletions

View File

@ -23,7 +23,7 @@ use tracing::debug;
/// Internally the `DefPathTable` holds a tree of `DefKey`s, where each `DefKey` /// Internally the `DefPathTable` holds a tree of `DefKey`s, where each `DefKey`
/// stores the `DefIndex` of its parent. /// stores the `DefIndex` of its parent.
/// There is one `DefPathTable` for each crate. /// There is one `DefPathTable` for each crate.
#[derive(Clone, Default, Decodable, Encodable)] #[derive(Clone, Default)]
pub struct DefPathTable { pub struct DefPathTable {
index_to_key: IndexVec<DefIndex, DefKey>, index_to_key: IndexVec<DefIndex, DefKey>,
def_path_hashes: IndexVec<DefIndex, DefPathHash>, def_path_hashes: IndexVec<DefIndex, DefPathHash>,
@ -42,10 +42,6 @@ impl DefPathTable {
index index
} }
pub fn next_id(&self) -> DefIndex {
DefIndex::from(self.index_to_key.len())
}
#[inline(always)] #[inline(always)]
pub fn def_key(&self, index: DefIndex) -> DefKey { pub fn def_key(&self, index: DefIndex) -> DefKey {
self.index_to_key[index] self.index_to_key[index]
@ -58,15 +54,25 @@ impl DefPathTable {
hash hash
} }
pub fn add_def_path_hashes_to(&self, cnum: CrateNum, out: &mut FxHashMap<DefPathHash, DefId>) { pub fn num_def_ids(&self) -> usize {
out.extend(self.def_path_hashes.iter().enumerate().map(|(index, &hash)| { self.index_to_key.len()
let def_id = DefId { krate: cnum, index: DefIndex::from(index) };
(hash, def_id)
}));
} }
pub fn size(&self) -> usize { pub fn enumerated_keys_and_path_hashes(
self.index_to_key.len() &self,
) -> impl Iterator<Item = (DefIndex, &DefKey, &DefPathHash)> + '_ {
self.index_to_key
.iter_enumerated()
.map(move |(index, key)| (index, key, &self.def_path_hashes[index]))
}
pub fn all_def_path_hashes_and_def_ids(
&self,
krate: CrateNum,
) -> impl Iterator<Item = (DefPathHash, DefId)> + '_ {
self.def_path_hashes
.iter_enumerated()
.map(move |(index, hash)| (*hash, DefId { krate, index }))
} }
} }

View File

@ -16,7 +16,6 @@ use rustc_expand::proc_macro::{AttrProcMacro, BangProcMacro, ProcMacroDerive};
use rustc_hir as hir; use rustc_hir as hir;
use rustc_hir::def::{CtorKind, CtorOf, DefKind, Res}; use rustc_hir::def::{CtorKind, CtorOf, DefKind, Res};
use rustc_hir::def_id::{CrateNum, DefId, DefIndex, CRATE_DEF_INDEX, LOCAL_CRATE}; use rustc_hir::def_id::{CrateNum, DefId, DefIndex, CRATE_DEF_INDEX, LOCAL_CRATE};
use rustc_hir::definitions::DefPathTable;
use rustc_hir::definitions::{DefKey, DefPath, DefPathData, DefPathHash}; use rustc_hir::definitions::{DefKey, DefPath, DefPathData, DefPathHash};
use rustc_hir::lang_items; use rustc_hir::lang_items;
use rustc_index::vec::{Idx, IndexVec}; use rustc_index::vec::{Idx, IndexVec};
@ -29,7 +28,6 @@ use rustc_middle::mir::interpret::{AllocDecodingSession, AllocDecodingState};
use rustc_middle::mir::{self, Body, Promoted}; use rustc_middle::mir::{self, Body, Promoted};
use rustc_middle::ty::codec::TyDecoder; use rustc_middle::ty::codec::TyDecoder;
use rustc_middle::ty::{self, Ty, TyCtxt}; use rustc_middle::ty::{self, Ty, TyCtxt};
use rustc_middle::util::common::record_time;
use rustc_serialize::{opaque, Decodable, Decoder}; use rustc_serialize::{opaque, Decodable, Decoder};
use rustc_session::Session; use rustc_session::Session;
use rustc_span::hygiene::ExpnDataDecodeMode; use rustc_span::hygiene::ExpnDataDecodeMode;
@ -69,12 +67,6 @@ crate struct CrateMetadata {
/// universal (`for<'tcx>`), that is paired up with whichever `TyCtxt` /// universal (`for<'tcx>`), that is paired up with whichever `TyCtxt`
/// is being used to decode those values. /// is being used to decode those values.
root: CrateRoot<'static>, root: CrateRoot<'static>,
/// For each definition in this crate, we encode a key. When the
/// crate is loaded, we read all the keys and put them in this
/// hashmap, which gives the reverse mapping. This allows us to
/// quickly retrace a `DefPath`, which is needed for incremental
/// compilation support.
def_path_table: DefPathTable,
/// Trait impl data. /// Trait impl data.
/// FIXME: Used only from queries and can use query cache, /// FIXME: Used only from queries and can use query cache,
/// so pre-decoding can probably be avoided. /// so pre-decoding can probably be avoided.
@ -91,6 +83,10 @@ crate struct CrateMetadata {
/// Do not access the value directly, as it might not have been initialized yet. /// Do not access the value directly, as it might not have been initialized yet.
/// The field must always be initialized to `DepNodeIndex::INVALID`. /// The field must always be initialized to `DepNodeIndex::INVALID`.
dep_node_index: AtomicCell<DepNodeIndex>, dep_node_index: AtomicCell<DepNodeIndex>,
/// Caches decoded `DefKey`s.
def_key_cache: Lock<FxHashMap<DefIndex, DefKey>>,
/// Caches decoded `DefPathHash`es.
def_path_hash_cache: Lock<FxHashMap<DefIndex, DefPathHash>>,
// --- Other significant crate properties --- // --- Other significant crate properties ---
/// ID of this crate, from the current compilation session's point of view. /// ID of this crate, from the current compilation session's point of view.
@ -807,7 +803,7 @@ impl<'a, 'tcx> CrateMetadataRef<'a> {
data.has_auto_impl, data.has_auto_impl,
data.is_marker, data.is_marker,
data.specialization_kind, data.specialization_kind,
self.def_path_table.def_path_hash(item_id), self.def_path_hash(item_id),
) )
} }
EntryKind::TraitAlias => ty::TraitDef::new( EntryKind::TraitAlias => ty::TraitDef::new(
@ -817,7 +813,7 @@ impl<'a, 'tcx> CrateMetadataRef<'a> {
false, false,
false, false,
ty::trait_def::TraitSpecializationKind::None, ty::trait_def::TraitSpecializationKind::None,
self.def_path_table.def_path_hash(item_id), self.def_path_hash(item_id),
), ),
_ => bug!("def-index does not refer to trait or trait alias"), _ => bug!("def-index does not refer to trait or trait alias"),
} }
@ -1509,12 +1505,14 @@ impl<'a, 'tcx> CrateMetadataRef<'a> {
#[inline] #[inline]
fn def_key(&self, index: DefIndex) -> DefKey { fn def_key(&self, index: DefIndex) -> DefKey {
let mut key = self.def_path_table.def_key(index); *self.def_key_cache.lock().entry(index).or_insert_with(|| {
if self.is_proc_macro(index) { let mut key = self.root.tables.def_keys.get(self, index).unwrap().decode(self);
let name = self.raw_proc_macro(index).name(); if self.is_proc_macro(index) {
key.disambiguated_data.data = DefPathData::MacroNs(Symbol::intern(name)); let name = self.raw_proc_macro(index).name();
} key.disambiguated_data.data = DefPathData::MacroNs(Symbol::intern(name));
key }
key
})
} }
// Returns the path leading to the thing with this `id`. // Returns the path leading to the thing with this `id`.
@ -1523,6 +1521,57 @@ impl<'a, 'tcx> CrateMetadataRef<'a> {
DefPath::make(self.cnum, id, |parent| self.def_key(parent)) DefPath::make(self.cnum, id, |parent| self.def_key(parent))
} }
fn def_path_hash_unlocked(
&self,
index: DefIndex,
def_path_hashes: &mut FxHashMap<DefIndex, DefPathHash>,
) -> DefPathHash {
*def_path_hashes.entry(index).or_insert_with(|| {
self.root.tables.def_path_hashes.get(self, index).unwrap().decode(self)
})
}
#[inline]
fn def_path_hash(&self, index: DefIndex) -> DefPathHash {
let mut def_path_hashes = self.def_path_hash_cache.lock();
self.def_path_hash_unlocked(index, &mut def_path_hashes)
}
fn all_def_path_hashes_and_def_ids(&self) -> Vec<(DefPathHash, DefId)> {
let mut def_path_hashes = self.def_path_hash_cache.lock();
(0..self.num_def_ids())
.map(|index| {
let index = DefIndex::from_usize(index);
(self.def_path_hash_unlocked(index, &mut def_path_hashes), self.local_def_id(index))
})
.collect()
}
/// Get the `DepNodeIndex` corresponding this crate. The result of this
/// method is cached in the `dep_node_index` field.
fn get_crate_dep_node_index(&self, tcx: TyCtxt<'tcx>) -> DepNodeIndex {
let mut dep_node_index = self.dep_node_index.load();
if unlikely!(dep_node_index == DepNodeIndex::INVALID) {
// We have not cached the DepNodeIndex for this upstream crate yet,
// so use the dep-graph to find it out and cache it.
// Note that multiple threads can enter this block concurrently.
// That is fine because the DepNodeIndex remains constant
// throughout the whole compilation session, and multiple stores
// would always write the same value.
let def_path_hash = self.def_path_hash(CRATE_DEF_INDEX);
let dep_node =
DepNode::from_def_path_hash(def_path_hash, dep_graph::DepKind::CrateMetadata);
dep_node_index = tcx.dep_graph.dep_node_index_of(&dep_node);
assert!(dep_node_index != DepNodeIndex::INVALID);
self.dep_node_index.store(dep_node_index);
}
dep_node_index
}
/// Imports the source_map from an external crate into the source_map of the crate /// Imports the source_map from an external crate into the source_map of the crate
/// currently being compiled (the "local crate"). /// currently being compiled (the "local crate").
/// ///
@ -1723,9 +1772,6 @@ impl CrateMetadata {
private_dep: bool, private_dep: bool,
host_hash: Option<Svh>, host_hash: Option<Svh>,
) -> CrateMetadata { ) -> CrateMetadata {
let def_path_table = record_time(&sess.perf_stats.decode_def_path_tables_time, || {
root.def_path_table.decode((&blob, sess))
});
let trait_impls = root let trait_impls = root
.impls .impls
.decode((&blob, sess)) .decode((&blob, sess))
@ -1737,7 +1783,6 @@ impl CrateMetadata {
CrateMetadata { CrateMetadata {
blob, blob,
root, root,
def_path_table,
trait_impls, trait_impls,
raw_proc_macros, raw_proc_macros,
source_map_import_info: OnceCell::new(), source_map_import_info: OnceCell::new(),
@ -1752,6 +1797,8 @@ impl CrateMetadata {
host_hash, host_hash,
extern_crate: Lock::new(None), extern_crate: Lock::new(None),
hygiene_context: Default::default(), hygiene_context: Default::default(),
def_key_cache: Default::default(),
def_path_hash_cache: Default::default(),
} }
} }
@ -1828,6 +1875,10 @@ impl CrateMetadata {
self.root.hash self.root.hash
} }
fn num_def_ids(&self) -> usize {
self.root.tables.def_keys.size()
}
fn local_def_id(&self, index: DefIndex) -> DefId { fn local_def_id(&self, index: DefIndex) -> DefId {
DefId { krate: self.cnum, index } DefId { krate: self.cnum, index }
} }
@ -1843,36 +1894,6 @@ impl CrateMetadata {
None None
} }
#[inline]
fn def_path_hash(&self, index: DefIndex) -> DefPathHash {
self.def_path_table.def_path_hash(index)
}
/// Get the `DepNodeIndex` corresponding this crate. The result of this
/// method is cached in the `dep_node_index` field.
fn get_crate_dep_node_index(&self, tcx: TyCtxt<'tcx>) -> DepNodeIndex {
let mut dep_node_index = self.dep_node_index.load();
if unlikely!(dep_node_index == DepNodeIndex::INVALID) {
// We have not cached the DepNodeIndex for this upstream crate yet,
// so use the dep-graph to find it out and cache it.
// Note that multiple threads can enter this block concurrently.
// That is fine because the DepNodeIndex remains constant
// throughout the whole compilation session, and multiple stores
// would always write the same value.
let def_path_hash = self.def_path_hash(CRATE_DEF_INDEX);
let dep_node =
DepNode::from_def_path_hash(def_path_hash, dep_graph::DepKind::CrateMetadata);
dep_node_index = tcx.dep_graph.dep_node_index_of(&dep_node);
assert!(dep_node_index != DepNodeIndex::INVALID);
self.dep_node_index.store(dep_node_index);
}
dep_node_index
}
} }
// Cannot be implemented on 'ProcMacro', as libproc_macro // Cannot be implemented on 'ProcMacro', as libproc_macro

View File

@ -9,7 +9,6 @@ use rustc_ast::expand::allocator::AllocatorKind;
use rustc_data_structures::svh::Svh; use rustc_data_structures::svh::Svh;
use rustc_hir as hir; use rustc_hir as hir;
use rustc_hir::def_id::{CrateNum, DefId, DefIdMap, CRATE_DEF_INDEX, LOCAL_CRATE}; use rustc_hir::def_id::{CrateNum, DefId, DefIdMap, CRATE_DEF_INDEX, LOCAL_CRATE};
use rustc_hir::definitions::DefPathTable;
use rustc_hir::definitions::{DefKey, DefPath, DefPathHash}; use rustc_hir::definitions::{DefKey, DefPath, DefPathHash};
use rustc_middle::hir::exports::Export; use rustc_middle::hir::exports::Export;
use rustc_middle::middle::cstore::{CrateSource, CrateStore, EncodedMetadata}; use rustc_middle::middle::cstore::{CrateSource, CrateStore, EncodedMetadata};
@ -486,8 +485,12 @@ impl CrateStore for CStore {
self.get_crate_data(def.krate).def_path_hash(def.index) self.get_crate_data(def.krate).def_path_hash(def.index)
} }
fn def_path_table(&self, cnum: CrateNum) -> &DefPathTable { fn all_def_path_hashes_and_def_ids(&self, cnum: CrateNum) -> Vec<(DefPathHash, DefId)> {
&self.get_crate_data(cnum).cdata.def_path_table self.get_crate_data(cnum).all_def_path_hashes_and_def_ids()
}
fn num_def_ids(&self, cnum: CrateNum) -> usize {
self.get_crate_data(cnum).num_def_ids()
} }
fn crates_untracked(&self) -> Vec<CrateNum> { fn crates_untracked(&self) -> Vec<CrateNum> {

View File

@ -9,7 +9,6 @@ use rustc_data_structures::sync::{join, Lrc};
use rustc_hir as hir; use rustc_hir as hir;
use rustc_hir::def::CtorKind; use rustc_hir::def::CtorKind;
use rustc_hir::def_id::{CrateNum, DefId, DefIndex, LocalDefId, CRATE_DEF_INDEX, LOCAL_CRATE}; use rustc_hir::def_id::{CrateNum, DefId, DefIndex, LocalDefId, CRATE_DEF_INDEX, LOCAL_CRATE};
use rustc_hir::definitions::DefPathTable;
use rustc_hir::intravisit::{self, NestedVisitorMap, Visitor}; use rustc_hir::intravisit::{self, NestedVisitorMap, Visitor};
use rustc_hir::itemlikevisit::{ItemLikeVisitor, ParItemLikeVisitor}; use rustc_hir::itemlikevisit::{ItemLikeVisitor, ParItemLikeVisitor};
use rustc_hir::lang_items; use rustc_hir::lang_items;
@ -418,9 +417,14 @@ impl<'a, 'tcx> EncodeContext<'a, 'tcx> {
} }
} }
fn encode_def_path_table(&mut self) -> Lazy<DefPathTable> { fn encode_def_path_table(&mut self) {
let definitions = self.tcx.hir().definitions(); let table = self.tcx.hir().definitions().def_path_table();
self.lazy(definitions.def_path_table()) for (def_index, def_key, def_path_hash) in table.enumerated_keys_and_path_hashes() {
let def_key = self.lazy(def_key);
let def_path_hash = self.lazy(def_path_hash);
self.tables.def_keys.set(def_index, def_key);
self.tables.def_path_hashes.set(def_index, def_path_hash);
}
} }
fn encode_source_map(&mut self) -> Lazy<[rustc_span::SourceFile]> { fn encode_source_map(&mut self) -> Lazy<[rustc_span::SourceFile]> {
@ -525,7 +529,7 @@ impl<'a, 'tcx> EncodeContext<'a, 'tcx> {
// Encode DefPathTable // Encode DefPathTable
i = self.position(); i = self.position();
let def_path_table = self.encode_def_path_table(); self.encode_def_path_table();
let def_path_table_bytes = self.position() - i; let def_path_table_bytes = self.position() - i;
// Encode the def IDs of impls, for coherence checking. // Encode the def IDs of impls, for coherence checking.
@ -642,7 +646,6 @@ impl<'a, 'tcx> EncodeContext<'a, 'tcx> {
native_libraries, native_libraries,
foreign_modules, foreign_modules,
source_map, source_map,
def_path_table,
impls, impls,
exported_symbols, exported_symbols,
interpret_alloc_index, interpret_alloc_index,

View File

@ -7,7 +7,8 @@ use rustc_data_structures::svh::Svh;
use rustc_data_structures::sync::MetadataRef; use rustc_data_structures::sync::MetadataRef;
use rustc_hir as hir; use rustc_hir as hir;
use rustc_hir::def::CtorKind; use rustc_hir::def::CtorKind;
use rustc_hir::def_id::{DefId, DefIndex}; use rustc_hir::def_id::{DefId, DefIndex, DefPathHash};
use rustc_hir::definitions::DefKey;
use rustc_hir::lang_items; use rustc_hir::lang_items;
use rustc_index::{bit_set::FiniteBitSet, vec::IndexVec}; use rustc_index::{bit_set::FiniteBitSet, vec::IndexVec};
use rustc_middle::hir::exports::Export; use rustc_middle::hir::exports::Export;
@ -195,7 +196,6 @@ crate struct CrateRoot<'tcx> {
diagnostic_items: Lazy<[(Symbol, DefIndex)]>, diagnostic_items: Lazy<[(Symbol, DefIndex)]>,
native_libraries: Lazy<[NativeLib]>, native_libraries: Lazy<[NativeLib]>,
foreign_modules: Lazy<[ForeignModule]>, foreign_modules: Lazy<[ForeignModule]>,
def_path_table: Lazy<rustc_hir::definitions::DefPathTable>,
impls: Lazy<[TraitImpls]>, impls: Lazy<[TraitImpls]>,
interpret_alloc_index: Lazy<[u32]>, interpret_alloc_index: Lazy<[u32]>,
@ -285,6 +285,12 @@ define_tables! {
mir: Table<DefIndex, Lazy!(mir::Body<'tcx>)>, mir: Table<DefIndex, Lazy!(mir::Body<'tcx>)>,
promoted_mir: Table<DefIndex, Lazy!(IndexVec<mir::Promoted, mir::Body<'tcx>>)>, promoted_mir: Table<DefIndex, Lazy!(IndexVec<mir::Promoted, mir::Body<'tcx>>)>,
unused_generic_params: Table<DefIndex, Lazy<FiniteBitSet<u32>>>, unused_generic_params: Table<DefIndex, Lazy<FiniteBitSet<u32>>>,
// `def_keys` and `def_path_hashes` represent a lazy version of a
// `DefPathTable`. This allows us to avoid deserializing an entire
// `DefPathTable` up front, since we may only ever use a few
// definitions from any given crate.
def_keys: Table<DefIndex, Lazy<DefKey>>,
def_path_hashes: Table<DefIndex, Lazy<DefPathHash>>
} }
#[derive(Copy, Clone, MetadataEncodable, MetadataDecodable)] #[derive(Copy, Clone, MetadataEncodable, MetadataDecodable)]

View File

@ -201,4 +201,9 @@ where
let bytes = &metadata.raw_bytes()[start..start + self.meta]; let bytes = &metadata.raw_bytes()[start..start + self.meta];
<Option<T>>::maybe_read_from_bytes_at(bytes, i.index())? <Option<T>>::maybe_read_from_bytes_at(bytes, i.index())?
} }
/// Size of the table in entries, including possible gaps.
pub(super) fn size(&self) -> usize {
self.meta / <Option<T>>::BYTE_LEN
}
} }

View File

@ -9,7 +9,7 @@ use rustc_ast::expand::allocator::AllocatorKind;
use rustc_data_structures::svh::Svh; use rustc_data_structures::svh::Svh;
use rustc_data_structures::sync::{self, MetadataRef}; use rustc_data_structures::sync::{self, MetadataRef};
use rustc_hir::def_id::{CrateNum, DefId, LOCAL_CRATE}; use rustc_hir::def_id::{CrateNum, DefId, LOCAL_CRATE};
use rustc_hir::definitions::{DefKey, DefPath, DefPathHash, DefPathTable}; use rustc_hir::definitions::{DefKey, DefPath, DefPathHash};
use rustc_macros::HashStable; use rustc_macros::HashStable;
use rustc_session::search_paths::PathKind; use rustc_session::search_paths::PathKind;
use rustc_session::utils::NativeLibKind; use rustc_session::utils::NativeLibKind;
@ -187,7 +187,8 @@ pub trait CrateStore {
fn def_key(&self, def: DefId) -> DefKey; fn def_key(&self, def: DefId) -> DefKey;
fn def_path(&self, def: DefId) -> DefPath; fn def_path(&self, def: DefId) -> DefPath;
fn def_path_hash(&self, def: DefId) -> DefPathHash; fn def_path_hash(&self, def: DefId) -> DefPathHash;
fn def_path_table(&self, cnum: CrateNum) -> &DefPathTable; fn all_def_path_hashes_and_def_ids(&self, cnum: CrateNum) -> Vec<(DefPathHash, DefId)>;
fn num_def_ids(&self, cnum: CrateNum) -> usize;
// "queries" used in resolve that aren't tracked for incremental compilation // "queries" used in resolve that aren't tracked for incremental compilation
fn crate_name_untracked(&self, cnum: CrateNum) -> Symbol; fn crate_name_untracked(&self, cnum: CrateNum) -> Symbol;

View File

@ -1102,20 +1102,13 @@ impl<'tcx> TyCtxt<'tcx> {
providers[LOCAL_CRATE] = local_providers; providers[LOCAL_CRATE] = local_providers;
let def_path_hash_to_def_id = if s.opts.build_dep_graph() { let def_path_hash_to_def_id = if s.opts.build_dep_graph() {
let def_path_tables = crates let capacity = definitions.def_path_table().num_def_ids()
.iter() + crates.iter().map(|cnum| cstore.num_def_ids(*cnum)).sum::<usize>();
.map(|&cnum| (cnum, cstore.def_path_table(cnum))) let mut map = FxHashMap::with_capacity_and_hasher(capacity, Default::default());
.chain(iter::once((LOCAL_CRATE, definitions.def_path_table())));
// Precompute the capacity of the hashmap so we don't have to map.extend(definitions.def_path_table().all_def_path_hashes_and_def_ids(LOCAL_CRATE));
// re-allocate when populating it. for cnum in &crates {
let capacity = def_path_tables.clone().map(|(_, t)| t.size()).sum::<usize>(); map.extend(cstore.all_def_path_hashes_and_def_ids(*cnum).into_iter());
let mut map: FxHashMap<_, _> =
FxHashMap::with_capacity_and_hasher(capacity, ::std::default::Default::default());
for (cnum, def_path_table) in def_path_tables {
def_path_table.add_def_path_hashes_to(cnum, &mut map);
} }
Some(map) Some(map)

View File

@ -218,8 +218,6 @@ pub struct Session {
pub struct PerfStats { pub struct PerfStats {
/// The accumulated time spent on computing symbol hashes. /// The accumulated time spent on computing symbol hashes.
pub symbol_hash_time: Lock<Duration>, pub symbol_hash_time: Lock<Duration>,
/// The accumulated time spent decoding def path tables from metadata.
pub decode_def_path_tables_time: Lock<Duration>,
/// Total number of values canonicalized queries constructed. /// Total number of values canonicalized queries constructed.
pub queries_canonicalized: AtomicUsize, pub queries_canonicalized: AtomicUsize,
/// Number of times this query is invoked. /// Number of times this query is invoked.
@ -862,10 +860,6 @@ impl Session {
"Total time spent computing symbol hashes: {}", "Total time spent computing symbol hashes: {}",
duration_to_secs_str(*self.perf_stats.symbol_hash_time.lock()) duration_to_secs_str(*self.perf_stats.symbol_hash_time.lock())
); );
println!(
"Total time spent decoding DefPath tables: {}",
duration_to_secs_str(*self.perf_stats.decode_def_path_tables_time.lock())
);
println!( println!(
"Total queries canonicalized: {}", "Total queries canonicalized: {}",
self.perf_stats.queries_canonicalized.load(Ordering::Relaxed) self.perf_stats.queries_canonicalized.load(Ordering::Relaxed)
@ -1339,7 +1333,6 @@ pub fn build_session(
prof, prof,
perf_stats: PerfStats { perf_stats: PerfStats {
symbol_hash_time: Lock::new(Duration::from_secs(0)), symbol_hash_time: Lock::new(Duration::from_secs(0)),
decode_def_path_tables_time: Lock::new(Duration::from_secs(0)),
queries_canonicalized: AtomicUsize::new(0), queries_canonicalized: AtomicUsize::new(0),
normalize_generic_arg_after_erasing_regions: AtomicUsize::new(0), normalize_generic_arg_after_erasing_regions: AtomicUsize::new(0),
normalize_projection_ty: AtomicUsize::new(0), normalize_projection_ty: AtomicUsize::new(0),

View File

@ -117,13 +117,13 @@ impl<'tcx> DocContext<'tcx> {
// def ids, as we'll end up with a panic if we use the DefId Debug impl for fake DefIds // def ids, as we'll end up with a panic if we use the DefId Debug impl for fake DefIds
pub fn next_def_id(&self, crate_num: CrateNum) -> DefId { pub fn next_def_id(&self, crate_num: CrateNum) -> DefId {
let start_def_id = { let start_def_id = {
let next_id = if crate_num == LOCAL_CRATE { let num_def_ids = if crate_num == LOCAL_CRATE {
self.tcx.hir().definitions().def_path_table().next_id() self.tcx.hir().definitions().def_path_table().num_def_ids()
} else { } else {
self.enter_resolver(|r| r.cstore().def_path_table(crate_num).next_id()) self.enter_resolver(|r| r.cstore().num_def_ids(crate_num))
}; };
DefId { krate: crate_num, index: next_id } DefId { krate: crate_num, index: DefIndex::from_usize(num_def_ids) }
}; };
let mut fake_ids = self.fake_def_ids.borrow_mut(); let mut fake_ids = self.fake_def_ids.borrow_mut();