提交 6e2a3447 编写于 作者: B bors

Auto merge of #83114 - cjgillot:hop, r=eddyb

Move HIR parenting information out of hir_owner

Split out of #82681.

The parent of a HIR node and its content are currently bundled together, but are rarely used together.
This PR separates both information in two distinct queries for HIR owners.
This reduces incremental invalidation for HIR items that appear within a function body when this body (and the local ids) changes.
......@@ -338,7 +338,7 @@ fn lower_legacy_const_generics(
let mut generic_args = vec![];
for (idx, arg) in args.into_iter().enumerate() {
if legacy_args_idx.contains(&idx) {
let parent_def_id = self.current_hir_id_owner.last().unwrap().0;
let parent_def_id = self.current_hir_id_owner.0;
let node_id = self.resolver.next_node_id();
// Add a definition for the in-band const def.
......
......@@ -165,7 +165,7 @@ struct LoweringContext<'a, 'hir: 'a> {
type_def_lifetime_params: DefIdMap<usize>,
current_hir_id_owner: Vec<(LocalDefId, u32)>,
current_hir_id_owner: (LocalDefId, u32),
item_local_id_counters: NodeMap<u32>,
node_id_to_hir_id: IndexVec<NodeId, Option<hir::HirId>>,
......@@ -321,7 +321,7 @@ pub fn lower_crate<'a, 'hir>(
anonymous_lifetime_mode: AnonymousLifetimeMode::PassThrough,
type_def_lifetime_params: Default::default(),
current_module: CRATE_DEF_ID,
current_hir_id_owner: vec![(CRATE_DEF_ID, 0)],
current_hir_id_owner: (CRATE_DEF_ID, 0),
item_local_id_counters: Default::default(),
node_id_to_hir_id: IndexVec::new(),
generator_kind: None,
......@@ -594,9 +594,10 @@ fn with_hir_id_owner<T>(&mut self, owner: NodeId, f: impl FnOnce(&mut Self) -> T
.insert(owner, HIR_ID_COUNTER_LOCKED)
.unwrap_or_else(|| panic!("no `item_local_id_counters` entry for {:?}", owner));
let def_id = self.resolver.local_def_id(owner);
self.current_hir_id_owner.push((def_id, counter));
let old_owner = std::mem::replace(&mut self.current_hir_id_owner, (def_id, counter));
let ret = f(self);
let (new_def_id, new_counter) = self.current_hir_id_owner.pop().unwrap();
let (new_def_id, new_counter) =
std::mem::replace(&mut self.current_hir_id_owner, old_owner);
debug_assert!(def_id == new_def_id);
debug_assert!(new_counter >= counter);
......@@ -614,8 +615,7 @@ fn with_hir_id_owner<T>(&mut self, owner: NodeId, f: impl FnOnce(&mut Self) -> T
/// properly. Calling the method twice with the same `NodeId` is fine though.
fn lower_node_id(&mut self, ast_node_id: NodeId) -> hir::HirId {
self.lower_node_id_generic(ast_node_id, |this| {
let &mut (owner, ref mut local_id_counter) =
this.current_hir_id_owner.last_mut().unwrap();
let &mut (owner, ref mut local_id_counter) = &mut this.current_hir_id_owner;
let local_id = *local_id_counter;
*local_id_counter += 1;
hir::HirId { owner, local_id: hir::ItemLocalId::from_u32(local_id) }
......@@ -868,10 +868,7 @@ fn add_in_band_defs<T>(
// wouldn't have been added yet.
let generics = this.lower_generics_mut(
generics,
ImplTraitContext::Universal(
&mut params,
this.current_hir_id_owner.last().unwrap().0,
),
ImplTraitContext::Universal(&mut params, this.current_hir_id_owner.0),
);
let res = f(this, &mut params);
(params, (generics, res))
......@@ -1077,7 +1074,7 @@ fn lower_assoc_ty_constraint(
}
AssocTyConstraintKind::Bound { ref bounds } => {
let mut capturable_lifetimes;
let mut parent_def_id = self.current_hir_id_owner.last().unwrap().0;
let mut parent_def_id = self.current_hir_id_owner.0;
// Piggy-back on the `impl Trait` context to figure out the correct behavior.
let (desugar_to_impl_trait, itctx) = match itctx {
// We are in the return position:
......@@ -1198,7 +1195,7 @@ fn lower_generic_arg(
// Construct a AnonConst where the expr is the "ty"'s path.
let parent_def_id = self.current_hir_id_owner.last().unwrap().0;
let parent_def_id = self.current_hir_id_owner.0;
let node_id = self.resolver.next_node_id();
// Add a definition for the in-band const def.
......@@ -1814,10 +1811,7 @@ fn lower_fn_decl(
if let Some((_, ibty)) = &mut in_band_ty_params {
this.lower_ty_direct(
&param.ty,
ImplTraitContext::Universal(
ibty,
this.current_hir_id_owner.last().unwrap().0,
),
ImplTraitContext::Universal(ibty, this.current_hir_id_owner.0),
)
} else {
this.lower_ty_direct(&param.ty, ImplTraitContext::disallowed())
......
......@@ -91,7 +91,7 @@
[] predicates: rustc_middle::ty::PredicateInner<$tcx>,
// HIR query types
[few] indexed_hir: rustc_middle::hir::map::IndexedHir<$tcx>,
[few] indexed_hir: rustc_middle::hir::IndexedHir<$tcx>,
[few] hir_definitions: rustc_hir::definitions::Definitions,
[] hir_owner: rustc_middle::hir::Owner<$tcx>,
[] hir_owner_nodes: rustc_middle::hir::OwnerNodes<$tcx>,
......
use crate::arena::Arena;
use crate::hir::map::{Entry, HirOwnerData, Map};
use crate::hir::{Owner, OwnerNodes, ParentedNode};
use crate::hir::map::{HirOwnerData, Map};
use crate::hir::{IndexedHir, Owner, OwnerNodes, ParentedNode};
use crate::ich::StableHashingContext;
use crate::middle::cstore::CrateStore;
use rustc_data_structures::fingerprint::Fingerprint;
use rustc_data_structures::fx::FxHashMap;
use rustc_data_structures::stable_hasher::{HashStable, StableHasher};
use rustc_data_structures::svh::Svh;
use rustc_hir as hir;
use rustc_hir::def_id::LocalDefId;
use rustc_hir::def_id::CRATE_DEF_INDEX;
use rustc_hir::def_id::{LocalDefId, LOCAL_CRATE};
use rustc_hir::definitions::{self, DefPathHash};
use rustc_hir::definitions;
use rustc_hir::intravisit::{self, NestedVisitorMap, Visitor};
use rustc_hir::*;
use rustc_index::vec::{Idx, IndexVec};
use rustc_session::{CrateDisambiguator, Session};
use rustc_session::Session;
use rustc_span::source_map::SourceMap;
use rustc_span::{Span, Symbol, DUMMY_SP};
use rustc_span::{Span, DUMMY_SP};
use std::iter::repeat;
......@@ -31,6 +29,7 @@ pub(super) struct NodeCollector<'a, 'hir> {
source_map: &'a SourceMap,
map: IndexVec<LocalDefId, HirOwnerData<'hir>>,
parenting: FxHashMap<LocalDefId, HirId>,
/// The parent of this node
parent_node: hir::HirId,
......@@ -40,10 +39,6 @@ pub(super) struct NodeCollector<'a, 'hir> {
definitions: &'a definitions::Definitions,
hcx: StableHashingContext<'a>,
// We are collecting HIR hashes here so we can compute the
// crate hash from them later on.
hir_body_nodes: Vec<(DefPathHash, Fingerprint)>,
}
fn insert_vec_map<K: Idx, V: Clone>(map: &mut IndexVec<K, Option<V>>, k: K, v: V) {
......@@ -58,34 +53,20 @@ fn insert_vec_map<K: Idx, V: Clone>(map: &mut IndexVec<K, Option<V>>, k: K, v: V
fn hash_body(
hcx: &mut StableHashingContext<'_>,
def_path_hash: DefPathHash,
item_like: impl for<'a> HashStable<StableHashingContext<'a>>,
hir_body_nodes: &mut Vec<(DefPathHash, Fingerprint)>,
) -> Fingerprint {
let hash = {
let mut stable_hasher = StableHasher::new();
hcx.while_hashing_hir_bodies(true, |hcx| {
item_like.hash_stable(hcx, &mut stable_hasher);
});
stable_hasher.finish()
};
hir_body_nodes.push((def_path_hash, hash));
hash
let mut stable_hasher = StableHasher::new();
hcx.while_hashing_hir_bodies(true, |hcx| {
item_like.hash_stable(hcx, &mut stable_hasher);
});
stable_hasher.finish()
}
fn upstream_crates(cstore: &dyn CrateStore) -> Vec<(Symbol, Fingerprint, Svh)> {
let mut upstream_crates: Vec<_> = cstore
.crates_untracked()
.iter()
.map(|&cnum| {
let name = cstore.crate_name_untracked(cnum);
let disambiguator = cstore.crate_disambiguator_untracked(cnum).to_fingerprint();
let hash = cstore.crate_hash_untracked(cnum);
(name, disambiguator, hash)
})
.collect();
upstream_crates.sort_unstable_by_key(|&(name, dis, _)| (name.as_str(), dis));
upstream_crates
/// Represents an entry and its parent `HirId`.
#[derive(Copy, Clone, Debug)]
pub struct Entry<'hir> {
parent: HirId,
node: Node<'hir>,
}
impl<'a, 'hir> NodeCollector<'a, 'hir> {
......@@ -96,11 +77,6 @@ pub(super) fn root(
definitions: &'a definitions::Definitions,
mut hcx: StableHashingContext<'a>,
) -> NodeCollector<'a, 'hir> {
let root_mod_def_path_hash =
definitions.def_path_hash(LocalDefId { local_def_index: CRATE_DEF_INDEX });
let mut hir_body_nodes = Vec::new();
let hash = {
let Crate {
ref item,
......@@ -120,7 +96,7 @@ pub(super) fn root(
attrs: _,
} = *krate;
hash_body(&mut hcx, root_mod_def_path_hash, item, &mut hir_body_nodes)
hash_body(&mut hcx, item)
};
let mut collector = NodeCollector {
......@@ -131,10 +107,10 @@ pub(super) fn root(
current_dep_node_owner: LocalDefId { local_def_index: CRATE_DEF_INDEX },
definitions,
hcx,
hir_body_nodes,
map: (0..definitions.def_index_count())
.map(|_| HirOwnerData { signature: None, with_bodies: None })
.collect(),
parenting: FxHashMap::default(),
};
collector.insert_entry(
hir::CRATE_HIR_ID,
......@@ -145,55 +121,13 @@ pub(super) fn root(
collector
}
pub(super) fn finalize_and_compute_crate_hash(
mut self,
crate_disambiguator: CrateDisambiguator,
cstore: &dyn CrateStore,
commandline_args_hash: u64,
) -> (IndexVec<LocalDefId, HirOwnerData<'hir>>, Svh) {
pub(super) fn finalize_and_compute_crate_hash(mut self) -> IndexedHir<'hir> {
// Insert bodies into the map
for (id, body) in self.krate.bodies.iter() {
let bodies = &mut self.map[id.hir_id.owner].with_bodies.as_mut().unwrap().bodies;
assert!(bodies.insert(id.hir_id.local_id, body).is_none());
}
self.hir_body_nodes.sort_unstable_by_key(|bn| bn.0);
let node_hashes = self.hir_body_nodes.iter().fold(
Fingerprint::ZERO,
|combined_fingerprint, &(def_path_hash, fingerprint)| {
combined_fingerprint.combine(def_path_hash.0.combine(fingerprint))
},
);
let upstream_crates = upstream_crates(cstore);
// We hash the final, remapped names of all local source files so we
// don't have to include the path prefix remapping commandline args.
// If we included the full mapping in the SVH, we could only have
// reproducible builds by compiling from the same directory. So we just
// hash the result of the mapping instead of the mapping itself.
let mut source_file_names: Vec<_> = self
.source_map
.files()
.iter()
.filter(|source_file| source_file.cnum == LOCAL_CRATE)
.map(|source_file| source_file.name_hash)
.collect();
source_file_names.sort_unstable();
let crate_hash_input = (
((node_hashes, upstream_crates), source_file_names),
(commandline_args_hash, crate_disambiguator.to_fingerprint()),
);
let mut stable_hasher = StableHasher::new();
crate_hash_input.hash_stable(&mut self.hcx, &mut stable_hasher);
let crate_hash: Fingerprint = stable_hasher.finish();
let svh = Svh::new(crate_hash.to_smaller_hash());
(self.map, svh)
IndexedHir { map: self.map, parenting: self.parenting }
}
fn insert_entry(&mut self, id: HirId, entry: Entry<'hir>, hash: Fingerprint) {
......@@ -218,8 +152,7 @@ fn insert_entry(&mut self, id: HirId, entry: Entry<'hir>, hash: Fingerprint) {
nodes.hash = hash;
debug_assert!(data.signature.is_none());
data.signature =
Some(self.arena.alloc(Owner { parent: entry.parent, node: entry.node }));
data.signature = Some(self.arena.alloc(Owner { node: entry.node }));
let dk_parent = self.definitions.def_key(id.owner).parent;
if let Some(dk_parent) = dk_parent {
......@@ -231,6 +164,8 @@ fn insert_entry(&mut self, id: HirId, entry: Entry<'hir>, hash: Fingerprint) {
id.owner, dk_parent, entry.parent,
)
}
debug_assert_eq!(self.parenting.get(&id.owner), Some(&entry.parent));
}
} else {
assert_eq!(entry.parent.owner, id.owner);
......@@ -294,15 +229,28 @@ fn with_dep_node_owner<
f: F,
) {
let prev_owner = self.current_dep_node_owner;
let def_path_hash = self.definitions.def_path_hash(dep_node_owner);
let hash = hash_body(&mut self.hcx, def_path_hash, item_like, &mut self.hir_body_nodes);
let hash = hash_body(&mut self.hcx, item_like);
self.current_dep_node_owner = dep_node_owner;
f(self, hash);
self.current_dep_node_owner = prev_owner;
}
fn insert_nested(&mut self, item: LocalDefId) {
#[cfg(debug_assertions)]
{
let dk_parent = self.definitions.def_key(item).parent.unwrap();
let dk_parent = LocalDefId { local_def_index: dk_parent };
let dk_parent = self.definitions.local_def_id_to_hir_id(dk_parent);
debug_assert_eq!(
dk_parent.owner, self.parent_node.owner,
"Different parents for {:?}",
item
)
}
assert_eq!(self.parenting.insert(item, self.parent_node), None);
}
}
impl<'a, 'hir> Visitor<'hir> for NodeCollector<'a, 'hir> {
......@@ -318,18 +266,22 @@ fn nested_visit_map(&mut self) -> NestedVisitorMap<Self::Map> {
fn visit_nested_item(&mut self, item: ItemId) {
debug!("visit_nested_item: {:?}", item);
self.insert_nested(item.def_id);
self.visit_item(self.krate.item(item));
}
fn visit_nested_trait_item(&mut self, item_id: TraitItemId) {
self.insert_nested(item_id.def_id);
self.visit_trait_item(self.krate.trait_item(item_id));
}
fn visit_nested_impl_item(&mut self, item_id: ImplItemId) {
self.insert_nested(item_id.def_id);
self.visit_impl_item(self.krate.impl_item(item_id));
}
fn visit_nested_foreign_item(&mut self, foreign_id: ForeignItemId) {
self.insert_nested(foreign_id.def_id);
self.visit_foreign_item(self.krate.foreign_item(foreign_id));
}
......@@ -517,6 +469,7 @@ fn visit_macro_def(&mut self, macro_def: &'hir MacroDef<'hir>) {
self.definitions.local_def_id_to_hir_id(LocalDefId { local_def_index })
});
self.with_parent(parent, |this| {
this.insert_nested(macro_def.def_id);
this.with_dep_node_owner(macro_def.def_id, macro_def, |this, hash| {
this.insert_with_hash(
macro_def.span,
......
use self::collector::NodeCollector;
use crate::hir::{Owner, OwnerNodes};
use crate::hir::{HirOwnerData, IndexedHir};
use crate::middle::cstore::CrateStore;
use crate::ty::TyCtxt;
use rustc_ast as ast;
use rustc_data_structures::fingerprint::Fingerprint;
use rustc_data_structures::stable_hasher::{HashStable, StableHasher};
use rustc_data_structures::svh::Svh;
use rustc_hir::def::{DefKind, Res};
use rustc_hir::def_id::{CrateNum, DefId, LocalDefId, CRATE_DEF_INDEX, LOCAL_CRATE};
......@@ -11,7 +14,7 @@
use rustc_hir::intravisit::Visitor;
use rustc_hir::itemlikevisit::ItemLikeVisitor;
use rustc_hir::*;
use rustc_index::vec::IndexVec;
use rustc_index::vec::Idx;
use rustc_span::hygiene::MacroKind;
use rustc_span::source_map::Spanned;
use rustc_span::symbol::{kw, Ident, Symbol};
......@@ -21,22 +24,6 @@
pub mod blocks;
mod collector;
/// Represents an entry and its parent `HirId`.
#[derive(Copy, Clone, Debug)]
pub struct Entry<'hir> {
parent: HirId,
node: Node<'hir>,
}
impl<'hir> Entry<'hir> {
fn parent_node(self) -> Option<HirId> {
match self.node {
Node::Crate(_) => None,
_ => Some(self.parent),
}
}
}
fn fn_decl<'hir>(node: Node<'hir>) -> Option<&'hir FnDecl<'hir>> {
match node {
Node::Item(Item { kind: ItemKind::Fn(sig, _, _), .. })
......@@ -86,20 +73,6 @@ fn is_body_owner<'hir>(node: Node<'hir>, hir_id: HirId) -> bool {
}
}
#[derive(Debug)]
pub(super) struct HirOwnerData<'hir> {
pub(super) signature: Option<&'hir Owner<'hir>>,
pub(super) with_bodies: Option<&'hir mut OwnerNodes<'hir>>,
}
#[derive(Debug)]
pub struct IndexedHir<'hir> {
/// The SVH of the local crate.
pub crate_hash: Svh,
pub(super) map: IndexVec<LocalDefId, HirOwnerData<'hir>>,
}
#[derive(Copy, Clone)]
pub struct Map<'hir> {
pub(super) tcx: TyCtxt<'hir>,
......@@ -129,10 +102,48 @@ fn next(&mut self) -> Option<Self::Item> {
}
self.current_id = parent_id;
if let Some(entry) = self.map.find_entry(parent_id) {
return Some((parent_id, entry.node));
if let Some(node) = self.map.find(parent_id) {
return Some((parent_id, node));
}
// If this `HirId` doesn't have an entry, skip it and look for its `parent_id`.
}
}
}
/// An iterator that walks up the ancestor tree of a given `HirId`.
/// Constructed using `tcx.hir().parent_owner_iter(hir_id)`.
pub struct ParentOwnerIterator<'map, 'hir> {
current_id: HirId,
map: &'map Map<'hir>,
}
impl<'hir> Iterator for ParentOwnerIterator<'_, 'hir> {
type Item = (HirId, Node<'hir>);
fn next(&mut self) -> Option<Self::Item> {
if self.current_id.local_id.index() != 0 {
self.current_id.local_id = ItemLocalId::new(0);
if let Some(node) = self.map.find(self.current_id) {
return Some((self.current_id, node));
}
}
if self.current_id == CRATE_HIR_ID {
return None;
}
loop {
// There are nodes that do not have entries, so we need to skip them.
let parent_id = self.map.def_key(self.current_id.owner).parent;
let parent_id = parent_id.map_or(CRATE_HIR_ID.owner, |local_def_index| {
let def_id = LocalDefId { local_def_index };
self.map.local_def_id_to_hir_id(def_id).owner
});
self.current_id = HirId::make_owner(parent_id);
// If this `HirId` doesn't have an entry, skip it and look for its `parent_id`.
if let Some(node) = self.map.find(self.current_id) {
return Some((self.current_id, node));
}
// If this `HirId` doesn't have an `Entry`, skip it and look for its `parent_id`.
}
}
}
......@@ -165,7 +176,7 @@ pub fn local_def_id(&self, hir_id: HirId) -> LocalDefId {
bug!(
"local_def_id: no entry for `{:?}`, which has a map of `{:?}`",
hir_id,
self.find_entry(hir_id)
self.find(hir_id)
)
})
}
......@@ -272,27 +283,60 @@ pub fn def_kind(&self, local_def_id: LocalDefId) -> DefKind {
.unwrap_or_else(|| bug!("def_kind: unsupported node: {:?}", local_def_id))
}
fn find_entry(&self, id: HirId) -> Option<Entry<'hir>> {
pub fn find_parent_node(&self, id: HirId) -> Option<HirId> {
if id.local_id == ItemLocalId::from_u32(0) {
let owner = self.tcx.hir_owner(id.owner);
owner.map(|owner| Entry { parent: owner.parent, node: owner.node })
Some(self.tcx.hir_owner_parent(id.owner))
} else {
let owner = self.tcx.hir_owner_nodes(id.owner);
owner.and_then(|owner| {
let node = owner.nodes[id.local_id].as_ref();
// FIXME(eddyb) use a single generic type instead of having both
// `Entry` and `ParentedNode`, which are effectively the same.
// Alternatively, rewrite code using `Entry` to use `ParentedNode`.
node.map(|node| Entry {
parent: HirId { owner: id.owner, local_id: node.parent },
node: node.node,
})
})
let owner = self.tcx.hir_owner_nodes(id.owner)?;
let node = owner.nodes[id.local_id].as_ref()?;
let hir_id = HirId { owner: id.owner, local_id: node.parent };
Some(hir_id)
}
}
pub fn get_parent_node(&self, hir_id: HirId) -> HirId {
self.find_parent_node(hir_id).unwrap_or(CRATE_HIR_ID)
}
/// Retrieves the `Node` corresponding to `id`, returning `None` if cannot be found.
pub fn find(&self, id: HirId) -> Option<Node<'hir>> {
if id.local_id == ItemLocalId::from_u32(0) {
let owner = self.tcx.hir_owner(id.owner)?;
Some(owner.node)
} else {
let owner = self.tcx.hir_owner_nodes(id.owner)?;
let node = owner.nodes[id.local_id].as_ref()?;
Some(node.node)
}
}
fn get_entry(&self, id: HirId) -> Entry<'hir> {
self.find_entry(id).unwrap()
/// Retrieves the `Node` corresponding to `id`, panicking if it cannot be found.
pub fn get(&self, id: HirId) -> Node<'hir> {
self.find(id).unwrap_or_else(|| bug!("couldn't find hir id {} in the HIR map", id))
}
pub fn get_if_local(&self, id: DefId) -> Option<Node<'hir>> {
id.as_local().and_then(|id| self.find(self.local_def_id_to_hir_id(id)))
}
pub fn get_generics(&self, id: DefId) -> Option<&'hir Generics<'hir>> {
self.get_if_local(id).and_then(|node| match &node {
Node::ImplItem(impl_item) => Some(&impl_item.generics),
Node::TraitItem(trait_item) => Some(&trait_item.generics),
Node::Item(Item {
kind:
ItemKind::Fn(_, generics, _)
| ItemKind::TyAlias(_, generics)
| ItemKind::Enum(_, generics)
| ItemKind::Struct(_, generics)
| ItemKind::Union(_, generics)
| ItemKind::Trait(_, _, generics, ..)
| ItemKind::TraitAlias(generics, _)
| ItemKind::Impl(Impl { generics, .. }),
..
}) => Some(generics),
_ => None,
})
}
pub fn item(&self, id: ItemId) -> &'hir Item<'hir> {
......@@ -457,7 +501,7 @@ pub fn krate_attrs(&self) -> &'hir [ast::Attribute] {
pub fn get_module(&self, module: LocalDefId) -> (&'hir Mod<'hir>, Span, HirId) {
let hir_id = self.local_def_id_to_hir_id(module);
match self.get_entry(hir_id).node {
match self.get(hir_id) {
Node::Item(&Item { span, kind: ItemKind::Mod(ref m), .. }) => (m, span, hir_id),
Node::Crate(item) => (&item, item.inner, hir_id),
node => panic!("not a module: {:?}", node),
......@@ -496,60 +540,18 @@ pub fn visit_exported_macros_in_krate<V>(&self, visitor: &mut V)
}
}
/// Retrieves the `Node` corresponding to `id`, panicking if it cannot be found.
pub fn get(&self, id: HirId) -> Node<'hir> {
self.find(id).unwrap_or_else(|| bug!("couldn't find hir id {} in the HIR map", id))
}
pub fn get_if_local(&self, id: DefId) -> Option<Node<'hir>> {
id.as_local().and_then(|id| self.find(self.local_def_id_to_hir_id(id)))
}
pub fn get_generics(&self, id: DefId) -> Option<&'hir Generics<'hir>> {
self.get_if_local(id).and_then(|node| match &node {
Node::ImplItem(impl_item) => Some(&impl_item.generics),
Node::TraitItem(trait_item) => Some(&trait_item.generics),
Node::Item(Item {
kind:
ItemKind::Fn(_, generics, _)
| ItemKind::TyAlias(_, generics)
| ItemKind::Enum(_, generics)
| ItemKind::Struct(_, generics)
| ItemKind::Union(_, generics)
| ItemKind::Trait(_, _, generics, ..)
| ItemKind::TraitAlias(generics, _)
| ItemKind::Impl(Impl { generics, .. }),
..
}) => Some(generics),
_ => None,
})
}
/// Retrieves the `Node` corresponding to `id`, returning `None` if cannot be found.
pub fn find(&self, hir_id: HirId) -> Option<Node<'hir>> {
self.find_entry(hir_id).map(|entry| entry.node)
}
/// Similar to `get_parent`; returns the parent HIR Id, or just `hir_id` if there
/// is no parent. Note that the parent may be `CRATE_HIR_ID`, which is not itself
/// present in the map, so passing the return value of `get_parent_node` to
/// `get` may in fact panic.
/// This function returns the immediate parent in the HIR, whereas `get_parent`
/// returns the enclosing item. Note that this might not be the actual parent
/// node in the HIR -- some kinds of nodes are not in the map and these will
/// never appear as the parent node. Thus, you can always walk the parent nodes
/// from a node to the root of the HIR (unless you get back the same ID here,
/// which can happen if the ID is not in the map itself or is just weird).
pub fn get_parent_node(&self, hir_id: HirId) -> HirId {
self.get_entry(hir_id).parent_node().unwrap_or(hir_id)
}
/// Returns an iterator for the nodes in the ancestor tree of the `current_id`
/// until the crate root is reached. Prefer this over your own loop using `get_parent_node`.
pub fn parent_iter(&self, current_id: HirId) -> ParentHirIterator<'_, 'hir> {
ParentHirIterator { current_id, map: self }
}
/// Returns an iterator for the nodes in the ancestor tree of the `current_id`
/// until the crate root is reached. Prefer this over your own loop using `get_parent_node`.
pub fn parent_owner_iter(&self, current_id: HirId) -> ParentOwnerIterator<'_, 'hir> {
ParentOwnerIterator { current_id, map: self }
}
/// Checks if the node is left-hand side of an assignment.
pub fn is_lhs(&self, id: HirId) -> bool {
match self.find(self.get_parent_node(id)) {
......@@ -570,7 +572,7 @@ pub fn is_inside_const_context(&self, hir_id: HirId) -> bool {
/// Whether `hir_id` corresponds to a `mod` or a crate.
pub fn is_hir_id_module(&self, hir_id: HirId) -> bool {
matches!(
self.get_entry(hir_id).node,
self.get(hir_id),
Node::Item(Item { kind: ItemKind::Mod(_), .. }) | Node::Crate(..)
)
}
......@@ -600,8 +602,8 @@ pub fn is_hir_id_module(&self, hir_id: HirId) -> bool {
pub fn get_return_block(&self, id: HirId) -> Option<HirId> {
let mut iter = self.parent_iter(id).peekable();
let mut ignore_tail = false;
if let Some(entry) = self.find_entry(id) {
if let Node::Expr(Expr { kind: ExprKind::Ret(_), .. }) = entry.node {
if let Some(node) = self.find(id) {
if let Node::Expr(Expr { kind: ExprKind::Ret(_), .. }) = node {
// When dealing with `return` statements, we don't care about climbing only tail
// expressions.
ignore_tail = true;
......@@ -638,23 +640,23 @@ pub fn get_return_block(&self, id: HirId) -> Option<HirId> {
/// in the HIR which is recorded by the map and is an item, either an item
/// in a module, trait, or impl.
pub fn get_parent_item(&self, hir_id: HirId) -> HirId {
for (hir_id, node) in self.parent_iter(hir_id) {
match node {
Node::Crate(_)
| Node::Item(_)
| Node::ForeignItem(_)
| Node::TraitItem(_)
| Node::ImplItem(_) => return hir_id,
_ => {}
for (hir_id, node) in self.parent_owner_iter(hir_id) {
if let Node::Crate(_)
| Node::Item(_)
| Node::ForeignItem(_)
| Node::TraitItem(_)
| Node::ImplItem(_) = node
{
return hir_id;
}
}
hir_id
CRATE_HIR_ID
}
/// Returns the `HirId` of `id`'s nearest module parent, or `id` itself if no
/// module parent is in this map.
pub(super) fn get_module_parent_node(&self, hir_id: HirId) -> HirId {
for (hir_id, node) in self.parent_iter(hir_id) {
for (hir_id, node) in self.parent_owner_iter(hir_id) {
if let Node::Item(&Item { kind: ItemKind::Mod(_), .. }) = node {
return hir_id;
}
......@@ -728,12 +730,8 @@ pub fn get_parent_did(&self, id: HirId) -> LocalDefId {
pub fn get_foreign_abi(&self, hir_id: HirId) -> Abi {
let parent = self.get_parent_item(hir_id);
if let Some(entry) = self.find_entry(parent) {
if let Entry {
node: Node::Item(Item { kind: ItemKind::ForeignMod { abi, .. }, .. }),
..
} = entry
{
if let Some(node) = self.find(parent) {
if let Node::Item(Item { kind: ItemKind::ForeignMod { abi, .. }, .. }) = node {
return *abi;
}
}
......@@ -827,7 +825,7 @@ pub fn span(&self, hir_id: HirId) -> Span {
}
pub fn opt_span(&self, hir_id: HirId) -> Option<Span> {
let span = match self.find_entry(hir_id)?.node {
let span = match self.find(hir_id)? {
Node::Param(param) => param.span,
Node::Item(item) => match &item.kind {
ItemKind::Fn(sig, _, _) => sig.span,
......@@ -876,7 +874,7 @@ pub fn opt_span(&self, hir_id: HirId) -> Option<Span> {
/// Like `hir.span()`, but includes the body of function items
/// (instead of just the function header)
pub fn span_with_body(&self, hir_id: HirId) -> Span {
match self.find_entry(hir_id).map(|entry| entry.node) {
match self.find(hir_id) {
Some(Node::TraitItem(item)) => item.span,
Some(Node::ImplItem(impl_item)) => impl_item.span,
Some(Node::Item(item)) => item.span,
......@@ -935,19 +933,78 @@ pub(super) fn index_hir<'tcx>(tcx: TyCtxt<'tcx>, cnum: CrateNum) -> &'tcx Indexe
let _prof_timer = tcx.sess.prof.generic_activity("build_hir_map");
let (map, crate_hash) = {
let hcx = tcx.create_stable_hashing_context();
let hcx = tcx.create_stable_hashing_context();
let mut collector =
NodeCollector::root(tcx.sess, &**tcx.arena, tcx.untracked_crate, &tcx.definitions, hcx);
intravisit::walk_crate(&mut collector, tcx.untracked_crate);
let mut collector =
NodeCollector::root(tcx.sess, &**tcx.arena, tcx.untracked_crate, &tcx.definitions, hcx);
intravisit::walk_crate(&mut collector, tcx.untracked_crate);
let map = collector.finalize_and_compute_crate_hash();
tcx.arena.alloc(map)
}
let crate_disambiguator = tcx.sess.local_crate_disambiguator();
let cmdline_args = tcx.sess.opts.dep_tracking_hash(true);
collector.finalize_and_compute_crate_hash(crate_disambiguator, &*tcx.cstore, cmdline_args)
};
pub(super) fn crate_hash(tcx: TyCtxt<'_>, crate_num: CrateNum) -> Svh {
let mut hir_body_nodes: Vec<_> = tcx
.index_hir(crate_num)
.map
.iter_enumerated()
.filter_map(|(def_id, hod)| {
let def_path_hash = tcx.definitions.def_path_hash(def_id);
let hash = hod.with_bodies.as_ref()?.hash;
Some((def_path_hash, hash))
})
.collect();
hir_body_nodes.sort_unstable_by_key(|bn| bn.0);
tcx.arena.alloc(IndexedHir { crate_hash, map })
let node_hashes = hir_body_nodes.iter().fold(
Fingerprint::ZERO,
|combined_fingerprint, &(def_path_hash, fingerprint)| {
combined_fingerprint.combine(def_path_hash.0.combine(fingerprint))
},
);
let upstream_crates = upstream_crates(&*tcx.cstore);
// We hash the final, remapped names of all local source files so we
// don't have to include the path prefix remapping commandline args.
// If we included the full mapping in the SVH, we could only have
// reproducible builds by compiling from the same directory. So we just
// hash the result of the mapping instead of the mapping itself.
let mut source_file_names: Vec<_> = tcx
.sess
.source_map()
.files()
.iter()
.filter(|source_file| source_file.cnum == LOCAL_CRATE)
.map(|source_file| source_file.name_hash)
.collect();
source_file_names.sort_unstable();
let mut hcx = tcx.create_stable_hashing_context();
let mut stable_hasher = StableHasher::new();
node_hashes.hash_stable(&mut hcx, &mut stable_hasher);
upstream_crates.hash_stable(&mut hcx, &mut stable_hasher);
source_file_names.hash_stable(&mut hcx, &mut stable_hasher);
tcx.sess.opts.dep_tracking_hash(true).hash_stable(&mut hcx, &mut stable_hasher);
tcx.sess.local_crate_disambiguator().to_fingerprint().hash_stable(&mut hcx, &mut stable_hasher);
let crate_hash: Fingerprint = stable_hasher.finish();
Svh::new(crate_hash.to_smaller_hash())
}
fn upstream_crates(cstore: &dyn CrateStore) -> Vec<(Symbol, Fingerprint, Svh)> {
let mut upstream_crates: Vec<_> = cstore
.crates_untracked()
.iter()
.map(|&cnum| {
let name = cstore.crate_name_untracked(cnum);
let disambiguator = cstore.crate_disambiguator_untracked(cnum).to_fingerprint();
let hash = cstore.crate_hash_untracked(cnum);
(name, disambiguator, hash)
})
.collect();
upstream_crates.sort_unstable_by_key(|&(name, dis, _)| (name.as_str(), dis));
upstream_crates
}
fn hir_id_to_string(map: &Map<'_>, id: HirId) -> String {
......
......@@ -19,19 +19,27 @@
use rustc_span::DUMMY_SP;
use std::collections::BTreeMap;
#[derive(Debug)]
struct HirOwnerData<'hir> {
signature: Option<&'hir Owner<'hir>>,
with_bodies: Option<&'hir mut OwnerNodes<'hir>>,
}
#[derive(Debug)]
pub struct IndexedHir<'hir> {
map: IndexVec<LocalDefId, HirOwnerData<'hir>>,
parenting: FxHashMap<LocalDefId, HirId>,
}
#[derive(Debug)]
pub struct Owner<'tcx> {
parent: HirId,
node: Node<'tcx>,
}
impl<'a, 'tcx> HashStable<StableHashingContext<'a>> for Owner<'tcx> {
fn hash_stable(&self, hcx: &mut StableHashingContext<'a>, hasher: &mut StableHasher) {
let Owner { parent, node } = self;
hcx.while_hashing_hir_bodies(false, |hcx| {
parent.hash_stable(hcx, hasher);
node.hash_stable(hcx, hasher);
});
let Owner { node } = self;
hcx.while_hashing_hir_bodies(false, |hcx| node.hash_stable(hcx, hasher));
}
}
......@@ -117,9 +125,14 @@ pub fn provide(providers: &mut Providers) {
};
providers.hir_crate = |tcx, _| tcx.untracked_crate;
providers.index_hir = map::index_hir;
providers.crate_hash = map::crate_hash;
providers.hir_module_items = |tcx, id| &tcx.untracked_crate.modules[&id];
providers.hir_owner = |tcx, id| tcx.index_hir(LOCAL_CRATE).map[id].signature;
providers.hir_owner_nodes = |tcx, id| tcx.index_hir(LOCAL_CRATE).map[id].with_bodies.as_deref();
providers.hir_owner_parent = |tcx, id| {
let index = tcx.index_hir(LOCAL_CRATE);
index.parenting.get(&id).copied().unwrap_or(CRATE_HIR_ID)
};
providers.hir_attrs = |tcx, id| AttributeMap { map: &tcx.untracked_crate.attrs, prefix: id };
providers.def_span = |tcx, def_id| tcx.hir().span_if_local(def_id).unwrap_or(DUMMY_SP);
providers.fn_arg_names = |tcx, id| {
......
......@@ -28,7 +28,7 @@
/// The indexed HIR. This can be conveniently accessed by `tcx.hir()`.
/// Avoid calling this query directly.
query index_hir(_: CrateNum) -> &'tcx map::IndexedHir<'tcx> {
query index_hir(_: CrateNum) -> &'tcx crate::hir::IndexedHir<'tcx> {
eval_always
no_hash
desc { "index HIR" }
......@@ -52,6 +52,15 @@
desc { |tcx| "HIR owner of `{}`", tcx.def_path_str(key.to_def_id()) }
}
/// Gives access to the HIR node's parent for the HIR owner `key`.
///
/// This can be conveniently accessed by methods on `tcx.hir()`.
/// Avoid calling this query directly.
query hir_owner_parent(key: LocalDefId) -> hir::HirId {
eval_always
desc { |tcx| "HIR parent of `{}`", tcx.def_path_str(key.to_def_id()) }
}
/// Gives access to the HIR nodes and bodies inside the HIR owner `key`.
///
/// This can be conveniently accessed by methods on `tcx.hir()`.
......
use crate::dep_graph;
use crate::hir::exports::Export;
use crate::hir::map;
use crate::infer::canonical::{self, Canonical};
use crate::lint::LintLevelMap;
use crate::middle::codegen_fn_attrs::CodegenFnAttrs;
......
use rustc_data_structures::fx::FxIndexSet;
use rustc_data_structures::svh::Svh;
use rustc_hir as hir;
use rustc_hir::def_id::{CrateNum, DefId, LocalDefId, LOCAL_CRATE};
use rustc_middle::hir::map as hir_map;
......@@ -400,10 +399,6 @@ fn original_crate_name(tcx: TyCtxt<'_>, crate_num: CrateNum) -> Symbol {
tcx.crate_name
}
fn crate_hash(tcx: TyCtxt<'_>, crate_num: CrateNum) -> Svh {
tcx.index_hir(crate_num).crate_hash
}
fn instance_def_size_estimate<'tcx>(
tcx: TyCtxt<'tcx>,
instance_def: ty::InstanceDef<'tcx>,
......@@ -551,7 +546,6 @@ pub fn provide(providers: &mut ty::query::Providers) {
trait_of_item,
crate_disambiguator,
original_crate_name,
crate_hash,
instance_def_size_estimate,
issue33140_self_ty,
impl_defaultness,
......
......@@ -23,7 +23,7 @@ pub fn method_name() { }
}
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes,associated_item_def_ids")]
#[rustc_clean(cfg="cfail2", except="hir_owner,associated_item_def_ids")]
#[rustc_clean(cfg="cfail3")]
impl Foo {
#[rustc_clean(cfg="cfail3")]
......@@ -85,7 +85,7 @@ pub fn method_privacy() { }
}
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
impl Foo {
#[rustc_clean(cfg="cfail2", except="associated_item,hir_owner,hir_owner_nodes")]
......@@ -100,7 +100,7 @@ pub fn method_selfness() { }
}
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
impl Foo {
#[rustc_dirty(cfg="cfail2", except="type_of,predicates_of,promoted_mir")]
......@@ -135,7 +135,7 @@ pub fn add_method_to_impl1(&self) { }
}
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes,associated_item_def_ids")]
#[rustc_clean(cfg="cfail2", except="hir_owner,associated_item_def_ids")]
#[rustc_clean(cfg="cfail3")]
impl Foo {
#[rustc_clean(cfg="cfail2")]
......@@ -468,7 +468,7 @@ pub fn change_impl_self_type(&self) { }
}
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
impl Bar<u64> {
#[rustc_clean(cfg="cfail2", except="fn_sig,optimized_mir,typeck")]
......
......@@ -24,7 +24,7 @@
type ChangePrimitiveType = i32;
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
type ChangePrimitiveType = i64;
......@@ -35,7 +35,7 @@
type ChangeMutability = &'static i32;
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
type ChangeMutability = &'static mut i32;
......@@ -60,7 +60,7 @@
type ChangeTypeStruct = Struct1;
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
type ChangeTypeStruct = Struct2;
......@@ -71,7 +71,7 @@
type ChangeTypeTuple = (u32, u64);
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
type ChangeTypeTuple = (u32, i64);
......@@ -91,7 +91,7 @@ enum Enum2 {
type ChangeTypeEnum = Enum1;
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
type ChangeTypeEnum = Enum2;
......@@ -102,7 +102,7 @@ enum Enum2 {
type AddTupleField = (i32, i64);
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
type AddTupleField = (i32, i64, i16);
......@@ -113,7 +113,7 @@ enum Enum2 {
type ChangeNestedTupleField = (i32, (i64, i16));
#[cfg(not(cfail1))]
#[rustc_clean(cfg="cfail2", except="hir_owner,hir_owner_nodes")]
#[rustc_clean(cfg="cfail2", except="hir_owner")]
#[rustc_clean(cfg="cfail3")]
type ChangeNestedTupleField = (i32, (i64, i8));
......
......@@ -14,10 +14,7 @@ pub fn foo() {
#[cfg(cfail1)]
pub fn baz() {} // order is different...
// FIXME: Make "hir_owner" use `rustc_clean` here. Currently "hir_owner" includes a reference to
// the parent node, which is the statement holding this item. Changing the position of
// `bar` in `foo` will update that reference and make `hir_owner(bar)` dirty.
#[rustc_dirty(label = "hir_owner", cfg = "cfail2")]
#[rustc_clean(label = "hir_owner", cfg = "cfail2")]
#[rustc_clean(label = "hir_owner_nodes", cfg = "cfail2")]
pub fn bar() {} // but that doesn't matter.
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册