1
Fork 0

Cache expansion hash.

This commit is contained in:
Camille GILLOT 2021-06-22 19:20:25 +02:00
parent cff0ea5f88
commit 616ce3c5c0
12 changed files with 263 additions and 217 deletions

View file

@ -27,19 +27,18 @@
use crate::edition::Edition;
use crate::symbol::{kw, sym, Symbol};
use crate::with_session_globals;
use crate::{BytePos, CachingSourceMapView, HashStableContext, SourceFile, Span, DUMMY_SP};
use crate::{HashStableContext, Span, DUMMY_SP};
use crate::def_id::{CrateNum, DefId, DefPathHash, CRATE_DEF_INDEX, LOCAL_CRATE};
use crate::def_id::{CrateNum, DefId, CRATE_DEF_INDEX, LOCAL_CRATE};
use rustc_data_structures::fingerprint::Fingerprint;
use rustc_data_structures::fx::{FxHashMap, FxHashSet};
use rustc_data_structures::stable_hasher::{HashStable, StableHasher};
use rustc_data_structures::sync::{Lock, Lrc};
use rustc_data_structures::unhash::UnhashMap;
use rustc_macros::HashStable_Generic;
use rustc_serialize::{Decodable, Decoder, Encodable, Encoder};
use std::cell::RefCell;
use std::fmt;
use std::hash::Hash;
use std::thread::LocalKey;
use tracing::*;
/// A `SyntaxContext` represents a chain of pairs `(ExpnId, Transparency)` named "marks".
@ -63,6 +62,10 @@ pub struct SyntaxContextData {
#[derive(Clone, Copy, PartialEq, Eq, Hash, Debug)]
pub struct ExpnId(u32);
/// A unique hash value associated to an expansion.
#[derive(Clone, Copy, PartialEq, Eq, Hash, Debug, Encodable, Decodable, HashStable_Generic)]
pub struct ExpnHash(Fingerprint);
/// A property of a macro expansion that determines how identifiers
/// produced by that expansion are resolved.
#[derive(Copy, Clone, PartialEq, Eq, PartialOrd, Hash, Debug, Encodable, Decodable)]
@ -84,12 +87,13 @@ pub enum Transparency {
}
impl ExpnId {
pub fn fresh(expn_data: Option<ExpnData>) -> Self {
let has_data = expn_data.is_some();
let expn_id = HygieneData::with(|data| data.fresh_expn(expn_data));
if has_data {
update_disambiguator(expn_id);
}
pub fn fresh_empty() -> Self {
HygieneData::with(|data| data.fresh_expn(None))
}
pub fn fresh(expn_data: ExpnData, ctx: impl HashStableContext) -> Self {
let expn_id = HygieneData::with(|data| data.fresh_expn(Some(expn_data)));
update_disambiguator(expn_id, ctx);
expn_id
}
@ -109,13 +113,23 @@ impl ExpnId {
ExpnId(raw)
}
#[inline]
pub fn expn_hash(self) -> ExpnHash {
HygieneData::with(|data| data.expn_hash(self))
}
#[inline]
pub fn from_hash(hash: ExpnHash) -> Option<ExpnId> {
HygieneData::with(|data| data.expn_hash_to_expn_id.get(&hash).copied())
}
#[inline]
pub fn expn_data(self) -> ExpnData {
HygieneData::with(|data| data.expn_data(self).clone())
}
#[inline]
pub fn set_expn_data(self, mut expn_data: ExpnData) {
pub fn set_expn_data(self, mut expn_data: ExpnData, ctx: impl HashStableContext) {
HygieneData::with(|data| {
let old_expn_data = &mut data.expn_data[self.0 as usize];
assert!(old_expn_data.is_none(), "expansion data is reset for an expansion ID");
@ -123,7 +137,7 @@ impl ExpnId {
expn_data.orig_id = Some(self.as_u32());
*old_expn_data = Some(expn_data);
});
update_disambiguator(self)
update_disambiguator(self, ctx)
}
pub fn is_descendant_of(self, ancestor: ExpnId) -> bool {
@ -162,6 +176,8 @@ pub struct HygieneData {
/// between creation of an expansion ID and obtaining its data (e.g. macros are collected
/// first and then resolved later), so we use an `Option` here.
expn_data: Vec<Option<ExpnData>>,
expn_hashes: Vec<ExpnHash>,
expn_hash_to_expn_id: UnhashMap<ExpnHash, ExpnId>,
syntax_context_data: Vec<SyntaxContextData>,
syntax_context_map: FxHashMap<(SyntaxContext, ExpnId, Transparency), SyntaxContext>,
/// Maps the `Fingerprint` of an `ExpnData` to the next disambiguator value.
@ -185,6 +201,9 @@ impl HygieneData {
HygieneData {
expn_data: vec![Some(root_data)],
expn_hashes: vec![ExpnHash(Fingerprint::ZERO)],
expn_hash_to_expn_id: std::iter::once((ExpnHash(Fingerprint::ZERO), ExpnId(0)))
.collect(),
syntax_context_data: vec![SyntaxContextData {
outer_expn: ExpnId::root(),
outer_transparency: Transparency::Opaque,
@ -209,9 +228,15 @@ impl HygieneData {
data.orig_id = Some(raw_id);
}
self.expn_data.push(expn_data);
self.expn_hashes.push(ExpnHash(Fingerprint::ZERO));
ExpnId(raw_id)
}
#[inline]
fn expn_hash(&self, expn_id: ExpnId) -> ExpnHash {
self.expn_hashes[expn_id.0 as usize]
}
fn expn_data(&self, expn_id: ExpnId) -> &ExpnData {
self.expn_data[expn_id.0 as usize].as_ref().expect("no expansion data for an expansion ID")
}
@ -661,16 +686,17 @@ impl Span {
/// other compiler-generated code to set per-span properties like allowed unstable features.
/// The returned span belongs to the created expansion and has the new properties,
/// but its location is inherited from the current span.
pub fn fresh_expansion(self, expn_data: ExpnData) -> Span {
self.fresh_expansion_with_transparency(expn_data, Transparency::Transparent)
pub fn fresh_expansion(self, expn_data: ExpnData, ctx: impl HashStableContext) -> Span {
self.fresh_expansion_with_transparency(expn_data, Transparency::Transparent, ctx)
}
pub fn fresh_expansion_with_transparency(
self,
expn_data: ExpnData,
transparency: Transparency,
ctx: impl HashStableContext,
) -> Span {
let expn_id = ExpnId::fresh(Some(expn_data));
let expn_id = ExpnId::fresh(expn_data, ctx);
HygieneData::with(|data| {
self.with_ctxt(data.apply_mark(SyntaxContext::root(), expn_id, transparency))
})
@ -683,11 +709,13 @@ impl Span {
allow_internal_unstable: Option<Lrc<[Symbol]>>,
reason: DesugaringKind,
edition: Edition,
ctx: impl HashStableContext,
) -> Span {
self.fresh_expansion(ExpnData {
let expn_data = ExpnData {
allow_internal_unstable,
..ExpnData::default(ExpnKind::Desugaring(reason), self, edition, None, None)
})
};
self.fresh_expansion(expn_data, ctx)
}
}
@ -839,6 +867,13 @@ impl ExpnData {
pub fn is_root(&self) -> bool {
matches!(self.kind, ExpnKind::Root)
}
#[inline]
fn hash_expn(&self, ctx: &mut impl HashStableContext) -> Fingerprint {
let mut hasher = StableHasher::new();
self.hash_stable(ctx, &mut hasher);
hasher.finish()
}
}
/// Expansion kind.
@ -985,16 +1020,11 @@ pub struct HygieneEncodeContext {
}
impl HygieneEncodeContext {
pub fn encode<
T,
R,
F: FnMut(&mut T, u32, &SyntaxContextData) -> Result<(), R>,
G: FnMut(&mut T, u32, &ExpnData) -> Result<(), R>,
>(
pub fn encode<T, R>(
&self,
encoder: &mut T,
mut encode_ctxt: F,
mut encode_expn: G,
mut encode_ctxt: impl FnMut(&mut T, u32, &SyntaxContextData) -> Result<(), R>,
mut encode_expn: impl FnMut(&mut T, u32, ExpnData, ExpnHash) -> Result<(), R>,
) -> Result<(), R> {
// When we serialize a `SyntaxContextData`, we may end up serializing
// a `SyntaxContext` that we haven't seen before
@ -1012,7 +1042,7 @@ impl HygieneEncodeContext {
// It's fine to iterate over a HashMap, because the serialization
// of the table that we insert data into doesn't depend on insertion
// order
for_all_ctxts_in(latest_ctxts.into_iter(), |(index, ctxt, data)| {
for_all_ctxts_in(latest_ctxts.into_iter(), |index, ctxt, data| {
if self.serialized_ctxts.lock().insert(ctxt) {
encode_ctxt(encoder, index, data)?;
}
@ -1021,9 +1051,9 @@ impl HygieneEncodeContext {
let latest_expns = { std::mem::take(&mut *self.latest_expns.lock()) };
for_all_expns_in(latest_expns.into_iter(), |index, expn, data| {
for_all_expns_in(latest_expns.into_iter(), |index, expn, data, hash| {
if self.serialized_expns.lock().insert(expn) {
encode_expn(encoder, index, data)?;
encode_expn(encoder, index, data, hash)?;
}
Ok(())
})?;
@ -1046,16 +1076,14 @@ pub struct HygieneDecodeContext {
remapped_expns: Lock<Vec<Option<ExpnId>>>,
}
pub fn decode_expn_id<
'a,
D: Decoder,
F: FnOnce(&mut D, u32) -> Result<ExpnData, D::Error>,
G: FnOnce(CrateNum) -> &'a HygieneDecodeContext,
>(
pub fn decode_expn_id<'a, D: Decoder, G>(
d: &mut D,
mode: ExpnDataDecodeMode<'a, G>,
decode_data: F,
) -> Result<ExpnId, D::Error> {
decode_data: impl FnOnce(&mut D, u32) -> Result<(ExpnData, ExpnHash), D::Error>,
) -> Result<ExpnId, D::Error>
where
G: FnOnce(CrateNum) -> &'a HygieneDecodeContext,
{
let index = u32::decode(d)?;
let context = match mode {
ExpnDataDecodeMode::IncrComp(context) => context,
@ -1083,9 +1111,13 @@ pub fn decode_expn_id<
// Don't decode the data inside `HygieneData::with`, since we need to recursively decode
// other ExpnIds
let mut expn_data = decode_data(d, index)?;
let (mut expn_data, hash) = decode_data(d, index)?;
let expn_id = HygieneData::with(|hygiene_data| {
if let Some(&expn_id) = hygiene_data.expn_hash_to_expn_id.get(&hash) {
return expn_id;
}
let expn_id = ExpnId(hygiene_data.expn_data.len() as u32);
// If we just deserialized an `ExpnData` owned by
@ -1098,6 +1130,9 @@ pub fn decode_expn_id<
}
hygiene_data.expn_data.push(Some(expn_data));
hygiene_data.expn_hashes.push(hash);
let _old_id = hygiene_data.expn_hash_to_expn_id.insert(hash, expn_id);
debug_assert!(_old_id.is_none());
let mut expns = outer_expns.lock();
let new_len = index as usize + 1;
@ -1184,7 +1219,7 @@ pub fn decode_syntax_context<
Ok(new_ctxt)
}
fn for_all_ctxts_in<E, F: FnMut((u32, SyntaxContext, &SyntaxContextData)) -> Result<(), E>>(
fn for_all_ctxts_in<E, F: FnMut(u32, SyntaxContext, &SyntaxContextData) -> Result<(), E>>(
ctxts: impl Iterator<Item = SyntaxContext>,
mut f: F,
) -> Result<(), E> {
@ -1192,20 +1227,26 @@ fn for_all_ctxts_in<E, F: FnMut((u32, SyntaxContext, &SyntaxContextData)) -> Res
ctxts.map(|ctxt| (ctxt, data.syntax_context_data[ctxt.0 as usize].clone())).collect()
});
for (ctxt, data) in all_data.into_iter() {
f((ctxt.0, ctxt, &data))?;
f(ctxt.0, ctxt, &data)?;
}
Ok(())
}
fn for_all_expns_in<E, F: FnMut(u32, ExpnId, &ExpnData) -> Result<(), E>>(
fn for_all_expns_in<E>(
expns: impl Iterator<Item = ExpnId>,
mut f: F,
mut f: impl FnMut(u32, ExpnId, ExpnData, ExpnHash) -> Result<(), E>,
) -> Result<(), E> {
let all_data: Vec<_> = HygieneData::with(|data| {
expns.map(|expn| (expn, data.expn_data[expn.0 as usize].clone())).collect()
expns
.map(|expn| {
let idx = expn.0 as usize;
(expn, data.expn_data[idx].clone(), data.expn_hashes[idx].clone())
})
.collect()
});
for (expn, data) in all_data.into_iter() {
f(expn.0, expn, &data.unwrap_or_else(|| panic!("Missing data for {:?}", expn)))?;
for (expn, data, hash) in all_data.into_iter() {
let data = data.unwrap_or_else(|| panic!("Missing data for {:?}", expn));
f(expn.0, expn, data, hash)?;
}
Ok(())
}
@ -1306,107 +1347,51 @@ impl<D: Decoder> Decodable<D> for SyntaxContext {
/// `set_expn_data`). It is *not* called for foreign `ExpnId`s deserialized
/// from another crate's metadata - since `ExpnData` includes a `krate` field,
/// collisions are only possible between `ExpnId`s within the same crate.
fn update_disambiguator(expn_id: ExpnId) {
/// A `HashStableContext` which hashes the raw id values for `DefId`
/// and `CrateNum`, rather than using their computed stable hash.
///
/// This allows us to use the `HashStable` implementation on `ExpnId`
/// early on in compilation, before we've constructed a `TyCtxt`.
/// The `Fingerprint`s created by this context are not 'stable', since
/// the raw `CrateNum` and `DefId` values for an item may change between
/// sessions due to unrelated changes (e.g. adding/removing an different item).
///
/// However, this is fine for our purposes - we only need to detect
/// when two `ExpnData`s have the same `Fingerprint`. Since the hashes produced
/// by this context still obey the properties of `HashStable`, we have
/// that
/// `hash_stable(expn1, DummyHashStableContext) == hash_stable(expn2, DummyHashStableContext)`
/// iff `hash_stable(expn1, StableHashingContext) == hash_stable(expn2, StableHasingContext)`.
///
/// This is sufficient for determining when we need to update the disambiguator.
struct DummyHashStableContext<'a> {
caching_source_map: CachingSourceMapView<'a>,
}
impl<'a> crate::HashStableContext for DummyHashStableContext<'a> {
#[inline]
fn def_path_hash(&self, def_id: DefId) -> DefPathHash {
DefPathHash(Fingerprint::new(
def_id.krate.as_u32().into(),
def_id.index.as_u32().into(),
))
}
fn expn_id_cache() -> &'static LocalKey<ExpnIdCache> {
// This cache is only used by `DummyHashStableContext`,
// so we won't pollute the cache values of the normal `StableHashingContext`
thread_local! {
static CACHE: ExpnIdCache = const { ExpnIdCache::new(Vec::new()) };
}
&CACHE
}
fn hash_spans(&self) -> bool {
true
}
fn span_data_to_lines_and_cols(
&mut self,
span: &crate::SpanData,
) -> Option<(Lrc<SourceFile>, usize, BytePos, usize, BytePos)> {
self.caching_source_map.span_data_to_lines_and_cols(span)
}
}
let source_map = with_session_globals(|session_globals| {
session_globals.source_map.borrow().as_ref().unwrap().clone()
});
let mut ctx =
DummyHashStableContext { caching_source_map: CachingSourceMapView::new(&source_map) };
let mut hasher = StableHasher::new();
let expn_data = expn_id.expn_data();
fn update_disambiguator(expn_id: ExpnId, mut ctx: impl HashStableContext) {
let mut expn_data = expn_id.expn_data();
// This disambiguator should not have been set yet.
assert_eq!(
expn_data.disambiguator, 0,
"Already set disambiguator for ExpnData: {:?}",
expn_data
);
expn_data.hash_stable(&mut ctx, &mut hasher);
let first_hash = hasher.finish();
let mut expn_hash = expn_data.hash_expn(&mut ctx);
let modified = HygieneData::with(|data| {
let disambiguator = HygieneData::with(|data| {
// If this is the first ExpnData with a given hash, then keep our
// disambiguator at 0 (the default u32 value)
let disambig = data.expn_data_disambiguators.entry(first_hash).or_default();
data.expn_data[expn_id.0 as usize].as_mut().unwrap().disambiguator = *disambig;
let disambig = data.expn_data_disambiguators.entry(expn_hash).or_default();
let disambiguator = *disambig;
*disambig += 1;
*disambig != 1
disambiguator
});
if modified {
debug!("Set disambiguator for {:?} (hash {:?})", expn_id, first_hash);
debug!("expn_data = {:?}", expn_id.expn_data());
if disambiguator != 0 {
debug!("Set disambiguator for {:?} (hash {:?})", expn_id, expn_hash);
debug!("expn_data = {:?}", expn_data);
expn_data.disambiguator = disambiguator;
expn_hash = expn_data.hash_expn(&mut ctx);
// Verify that the new disambiguator makes the hash unique
#[cfg(debug_assertions)]
{
hasher = StableHasher::new();
expn_id.expn_data().hash_stable(&mut ctx, &mut hasher);
let new_hash: Fingerprint = hasher.finish();
HygieneData::with(|data| {
assert_eq!(
data.expn_data_disambiguators.get(&new_hash),
None,
"Hash collision after disambiguator update!",
);
});
};
HygieneData::with(|data| {
assert_eq!(
data.expn_data_disambiguators.get(&expn_hash),
None,
"Hash collision after disambiguator update!",
);
});
}
let expn_hash = ExpnHash(expn_hash);
HygieneData::with(|data| {
data.expn_data[expn_id.0 as usize].as_mut().unwrap().disambiguator = disambiguator;
debug_assert_eq!(data.expn_hashes[expn_id.0 as usize].0, Fingerprint::ZERO);
data.expn_hashes[expn_id.0 as usize] = expn_hash;
let _old_id = data.expn_hash_to_expn_id.insert(expn_hash, expn_id);
debug_assert!(_old_id.is_none());
});
}
impl<CTX: HashStableContext> HashStable<CTX> for SyntaxContext {
@ -1425,43 +1410,15 @@ impl<CTX: HashStableContext> HashStable<CTX> for SyntaxContext {
}
}
pub type ExpnIdCache = RefCell<Vec<Option<Fingerprint>>>;
impl<CTX: HashStableContext> HashStable<CTX> for ExpnId {
fn hash_stable(&self, ctx: &mut CTX, hasher: &mut StableHasher) {
const TAG_ROOT: u8 = 0;
const TAG_NOT_ROOT: u8 = 1;
if *self == ExpnId::root() {
TAG_ROOT.hash_stable(ctx, hasher);
return;
}
// Since the same expansion context is usually referenced many
// times, we cache a stable hash of it and hash that instead of
// recursing every time.
let index = self.as_u32() as usize;
let res = CTX::expn_id_cache().with(|cache| cache.borrow().get(index).copied().flatten());
if let Some(res) = res {
res.hash_stable(ctx, hasher);
let hash = if *self == ExpnId::root() {
// Avoid fetching TLS storage for a trivial often-used value.
Fingerprint::ZERO
} else {
let new_len = index + 1;
self.expn_hash().0
};
let mut sub_hasher = StableHasher::new();
TAG_NOT_ROOT.hash_stable(ctx, &mut sub_hasher);
self.expn_data().hash_stable(ctx, &mut sub_hasher);
let sub_hash: Fingerprint = sub_hasher.finish();
CTX::expn_id_cache().with(|cache| {
let mut cache = cache.borrow_mut();
if cache.len() < new_len {
cache.resize(new_len, None);
}
let prev = cache[index].replace(sub_hash);
assert_eq!(prev, None, "Cache slot was filled");
});
sub_hash.hash_stable(ctx, hasher);
}
hash.hash_stable(ctx, hasher);
}
}