use std::collections::hash_map::Entry; use std::mem; use std::sync::Arc; use rustc_data_structures::fx::{FxHashMap, FxIndexSet}; use rustc_data_structures::memmap::Mmap; use rustc_data_structures::sync::{HashMapExt, Lock, RwLock}; use rustc_data_structures::unhash::UnhashMap; use rustc_data_structures::unord::{UnordMap, UnordSet}; use rustc_hir::def_id::{CrateNum, DefId, DefIndex, LOCAL_CRATE, LocalDefId, StableCrateId}; use rustc_hir::definitions::DefPathHash; use rustc_index::{Idx, IndexVec}; use rustc_macros::{Decodable, Encodable}; use rustc_query_system::query::QuerySideEffects; use rustc_serialize::opaque::{FileEncodeResult, FileEncoder, IntEncodedWithFixedSize, MemDecoder}; use rustc_serialize::{Decodable, Decoder, Encodable, Encoder}; use rustc_session::Session; use rustc_span::hygiene::{ ExpnId, HygieneDecodeContext, HygieneEncodeContext, SyntaxContext, SyntaxContextData, }; use rustc_span::source_map::Spanned; use rustc_span::{ BytePos, CachingSourceMapView, ExpnData, ExpnHash, Pos, RelativeBytePos, SourceFile, Span, SpanDecoder, SpanEncoder, StableSourceFileId, Symbol, }; use crate::dep_graph::{DepNodeIndex, SerializedDepNodeIndex}; use crate::mir::interpret::{AllocDecodingSession, AllocDecodingState}; use crate::mir::mono::MonoItem; use crate::mir::{self, interpret}; use crate::ty::codec::{RefDecodable, TyDecoder, TyEncoder}; use crate::ty::{self, Ty, TyCtxt}; const TAG_FILE_FOOTER: u128 = 0xC0FFEE_C0FFEE_C0FFEE_C0FFEE_C0FFEE; // A normal span encoded with both location information and a `SyntaxContext` const TAG_FULL_SPAN: u8 = 0; // A partial span with no location information, encoded only with a `SyntaxContext` const TAG_PARTIAL_SPAN: u8 = 1; const TAG_RELATIVE_SPAN: u8 = 2; const TAG_SYNTAX_CONTEXT: u8 = 0; const TAG_EXPN_DATA: u8 = 1; // Tags for encoding Symbol's const SYMBOL_STR: u8 = 0; const SYMBOL_OFFSET: u8 = 1; const SYMBOL_PREINTERNED: u8 = 2; /// Provides an interface to incremental compilation data cached from the /// previous compilation session. This data will eventually include the results /// of a few selected queries (like `typeck` and `mir_optimized`) and /// any side effects that have been emitted during a query. pub struct OnDiskCache { // The complete cache data in serialized form. serialized_data: RwLock>, // Collects all `QuerySideEffects` created during the current compilation // session. current_side_effects: Lock>, file_index_to_stable_id: FxHashMap, // Caches that are populated lazily during decoding. file_index_to_file: Lock>>, // A map from dep-node to the position of the cached query result in // `serialized_data`. query_result_index: FxHashMap, // A map from dep-node to the position of any associated `QuerySideEffects` in // `serialized_data`. prev_side_effects_index: FxHashMap, alloc_decoding_state: AllocDecodingState, // A map from syntax context ids to the position of their associated // `SyntaxContextData`. We use a `u32` instead of a `SyntaxContext` // to represent the fact that we are storing *encoded* ids. When we decode // a `SyntaxContext`, a new id will be allocated from the global `HygieneData`, // which will almost certainly be different than the serialized id. syntax_contexts: FxHashMap, // A map from the `DefPathHash` of an `ExpnId` to the position // of their associated `ExpnData`. Ideally, we would store a `DefId`, // but we need to decode this before we've constructed a `TyCtxt` (which // makes it difficult to decode a `DefId`). // Note that these `DefPathHashes` correspond to both local and foreign // `ExpnData` (e.g `ExpnData.krate` may not be `LOCAL_CRATE`). Alternatively, // we could look up the `ExpnData` from the metadata of foreign crates, // but it seemed easier to have `OnDiskCache` be independent of the `CStore`. expn_data: UnhashMap, // Additional information used when decoding hygiene data. hygiene_context: HygieneDecodeContext, // Maps `ExpnHash`es to their raw value from the *previous* // compilation session. This is used as an initial 'guess' when // we try to map an `ExpnHash` to its value in the current // compilation session. foreign_expn_data: UnhashMap, } // This type is used only for serialization and deserialization. #[derive(Encodable, Decodable)] struct Footer { file_index_to_stable_id: FxHashMap, query_result_index: EncodedDepNodeIndex, side_effects_index: EncodedDepNodeIndex, // The location of all allocations. // Most uses only need values up to u32::MAX, but benchmarking indicates that we can use a u64 // without measurable overhead. This permits larger const allocations without ICEing. interpret_alloc_index: Vec, // See `OnDiskCache.syntax_contexts` syntax_contexts: FxHashMap, // See `OnDiskCache.expn_data` expn_data: UnhashMap, foreign_expn_data: UnhashMap, } pub type EncodedDepNodeIndex = Vec<(SerializedDepNodeIndex, AbsoluteBytePos)>; #[derive(Copy, Clone, PartialEq, Eq, Hash, Debug, Encodable, Decodable)] struct SourceFileIndex(u32); #[derive(Copy, Clone, Debug, Hash, Eq, PartialEq, Encodable, Decodable)] pub struct AbsoluteBytePos(u64); impl AbsoluteBytePos { #[inline] pub fn new(pos: usize) -> AbsoluteBytePos { AbsoluteBytePos(pos.try_into().expect("Incremental cache file size overflowed u64.")) } #[inline] fn to_usize(self) -> usize { self.0 as usize } } #[derive(Encodable, Decodable, Clone, Debug)] struct EncodedSourceFileId { stable_source_file_id: StableSourceFileId, stable_crate_id: StableCrateId, } impl EncodedSourceFileId { #[inline] fn new(tcx: TyCtxt<'_>, file: &SourceFile) -> EncodedSourceFileId { EncodedSourceFileId { stable_source_file_id: file.stable_id, stable_crate_id: tcx.stable_crate_id(file.cnum), } } } impl OnDiskCache { /// Creates a new `OnDiskCache` instance from the serialized data in `data`. /// /// The serialized cache has some basic integrity checks, if those checks indicate that the /// on-disk data is corrupt, an error is returned. pub fn new(sess: &Session, data: Mmap, start_pos: usize) -> Result { assert!(sess.opts.incremental.is_some()); let mut decoder = MemDecoder::new(&data, start_pos)?; // Decode the *position* of the footer, which can be found in the // last 8 bytes of the file. let footer_pos = decoder .with_position(decoder.len() - IntEncodedWithFixedSize::ENCODED_SIZE, |decoder| { IntEncodedWithFixedSize::decode(decoder).0 as usize }); // Decode the file footer, which contains all the lookup tables, etc. let footer: Footer = decoder.with_position(footer_pos, |decoder| decode_tagged(decoder, TAG_FILE_FOOTER)); Ok(Self { serialized_data: RwLock::new(Some(data)), file_index_to_stable_id: footer.file_index_to_stable_id, file_index_to_file: Default::default(), current_side_effects: Default::default(), query_result_index: footer.query_result_index.into_iter().collect(), prev_side_effects_index: footer.side_effects_index.into_iter().collect(), alloc_decoding_state: AllocDecodingState::new(footer.interpret_alloc_index), syntax_contexts: footer.syntax_contexts, expn_data: footer.expn_data, foreign_expn_data: footer.foreign_expn_data, hygiene_context: Default::default(), }) } pub fn new_empty() -> Self { Self { serialized_data: RwLock::new(None), file_index_to_stable_id: Default::default(), file_index_to_file: Default::default(), current_side_effects: Default::default(), query_result_index: Default::default(), prev_side_effects_index: Default::default(), alloc_decoding_state: AllocDecodingState::new(Vec::new()), syntax_contexts: FxHashMap::default(), expn_data: UnhashMap::default(), foreign_expn_data: UnhashMap::default(), hygiene_context: Default::default(), } } /// Execute all cache promotions and release the serialized backing Mmap. /// /// Cache promotions require invoking queries, which needs to read the serialized data. /// In order to serialize the new on-disk cache, the former on-disk cache file needs to be /// deleted, hence we won't be able to refer to its memmapped data. pub fn drop_serialized_data(&self, tcx: TyCtxt<'_>) { // 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); *self.serialized_data.write() = None; } pub fn serialize(&self, tcx: TyCtxt<'_>, encoder: FileEncoder) -> FileEncodeResult { // Serializing the `DepGraph` should not modify it. tcx.dep_graph.with_ignore(|| { // Allocate `SourceFileIndex`es. let (file_to_file_index, file_index_to_stable_id) = { let files = tcx.sess.source_map().files(); let mut file_to_file_index = FxHashMap::with_capacity_and_hasher(files.len(), Default::default()); let mut file_index_to_stable_id = FxHashMap::with_capacity_and_hasher(files.len(), Default::default()); for (index, file) in files.iter().enumerate() { let index = SourceFileIndex(index as u32); let file_ptr: *const SourceFile = &raw const **file; file_to_file_index.insert(file_ptr, index); let source_file_id = EncodedSourceFileId::new(tcx, file); file_index_to_stable_id.insert(index, source_file_id); } (file_to_file_index, file_index_to_stable_id) }; let hygiene_encode_context = HygieneEncodeContext::default(); let mut encoder = CacheEncoder { tcx, encoder, type_shorthands: Default::default(), predicate_shorthands: Default::default(), interpret_allocs: Default::default(), source_map: CachingSourceMapView::new(tcx.sess.source_map()), file_to_file_index, hygiene_context: &hygiene_encode_context, symbol_table: Default::default(), }; // Encode query results. let mut query_result_index = EncodedDepNodeIndex::new(); tcx.sess.time("encode_query_results", || { let enc = &mut encoder; let qri = &mut query_result_index; (tcx.query_system.fns.encode_query_results)(tcx, enc, qri); }); // Encode side effects. let side_effects_index: EncodedDepNodeIndex = self .current_side_effects .borrow() .iter() .map(|(dep_node_index, side_effects)| { let pos = AbsoluteBytePos::new(encoder.position()); let dep_node_index = SerializedDepNodeIndex::new(dep_node_index.index()); encoder.encode_tagged(dep_node_index, side_effects); (dep_node_index, pos) }) .collect(); let interpret_alloc_index = { let mut interpret_alloc_index = Vec::new(); let mut n = 0; loop { let new_n = encoder.interpret_allocs.len(); // If we have found new IDs, serialize those too. if n == new_n { // Otherwise, abort. break; } interpret_alloc_index.reserve(new_n - n); for idx in n..new_n { let id = encoder.interpret_allocs[idx]; let pos: u64 = encoder.position().try_into().unwrap(); interpret_alloc_index.push(pos); interpret::specialized_encode_alloc_id(&mut encoder, tcx, id); } n = new_n; } interpret_alloc_index }; let mut syntax_contexts = FxHashMap::default(); let mut expn_data = UnhashMap::default(); let mut foreign_expn_data = UnhashMap::default(); // Encode all hygiene data (`SyntaxContextData` and `ExpnData`) from the current // session. hygiene_encode_context.encode( &mut encoder, |encoder, index, ctxt_data| { let pos = AbsoluteBytePos::new(encoder.position()); encoder.encode_tagged(TAG_SYNTAX_CONTEXT, ctxt_data); syntax_contexts.insert(index, pos); }, |encoder, expn_id, data, hash| { if expn_id.krate == LOCAL_CRATE { let pos = AbsoluteBytePos::new(encoder.position()); encoder.encode_tagged(TAG_EXPN_DATA, data); expn_data.insert(hash, pos); } else { foreign_expn_data.insert(hash, expn_id.local_id.as_u32()); } }, ); // Encode the file footer. let footer_pos = encoder.position() as u64; encoder.encode_tagged( TAG_FILE_FOOTER, &Footer { file_index_to_stable_id, query_result_index, side_effects_index, interpret_alloc_index, syntax_contexts, expn_data, foreign_expn_data, }, ); // Encode the position of the footer as the last 8 bytes of the // file so we know where to look for it. IntEncodedWithFixedSize(footer_pos).encode(&mut encoder.encoder); // DO NOT WRITE ANYTHING TO THE ENCODER AFTER THIS POINT! The address // of the footer must be the last thing in the data stream. encoder.finish() }) } /// Loads a `QuerySideEffects` created during the previous compilation session. pub fn load_side_effects( &self, tcx: TyCtxt<'_>, dep_node_index: SerializedDepNodeIndex, ) -> QuerySideEffects { let side_effects: Option = self.load_indexed(tcx, dep_node_index, &self.prev_side_effects_index); side_effects.unwrap_or_default() } /// Stores a `QuerySideEffects` emitted during the current compilation session. /// Anything stored like this will be available via `load_side_effects` in /// the next compilation session. pub fn store_side_effects(&self, dep_node_index: DepNodeIndex, side_effects: QuerySideEffects) { let mut current_side_effects = self.current_side_effects.borrow_mut(); let prev = current_side_effects.insert(dep_node_index, side_effects); debug_assert!(prev.is_none()); } /// Return whether the cached query result can be decoded. #[inline] pub fn loadable_from_disk(&self, dep_node_index: SerializedDepNodeIndex) -> bool { self.query_result_index.contains_key(&dep_node_index) // with_decoder is infallible, so we can stop here } /// Returns the cached query result if there is something in the cache for /// the given `SerializedDepNodeIndex`; otherwise returns `None`. pub fn try_load_query_result<'tcx, T>( &self, tcx: TyCtxt<'tcx>, dep_node_index: SerializedDepNodeIndex, ) -> Option where T: for<'a> Decodable>, { let opt_value = self.load_indexed(tcx, dep_node_index, &self.query_result_index); debug_assert_eq!(opt_value.is_some(), self.loadable_from_disk(dep_node_index)); opt_value } /// Stores side effect emitted during computation of an anonymous query. /// Since many anonymous queries can share the same `DepNode`, we aggregate /// them -- as opposed to regular queries where we assume that there is a /// 1:1 relationship between query-key and `DepNode`. pub fn store_side_effects_for_anon_node( &self, dep_node_index: DepNodeIndex, side_effects: QuerySideEffects, ) { let mut current_side_effects = self.current_side_effects.borrow_mut(); let x = current_side_effects.entry(dep_node_index).or_default(); x.append(side_effects); } fn load_indexed<'tcx, T>( &self, tcx: TyCtxt<'tcx>, dep_node_index: SerializedDepNodeIndex, index: &FxHashMap, ) -> Option where T: for<'a> Decodable>, { let pos = index.get(&dep_node_index).cloned()?; let value = self.with_decoder(tcx, pos, |decoder| decode_tagged(decoder, dep_node_index)); Some(value) } fn with_decoder<'a, 'tcx, T, F: for<'s> FnOnce(&mut CacheDecoder<'s, 'tcx>) -> T>( &self, tcx: TyCtxt<'tcx>, pos: AbsoluteBytePos, f: F, ) -> T where T: Decodable>, { let serialized_data = self.serialized_data.read(); let mut decoder = CacheDecoder { tcx, opaque: MemDecoder::new(serialized_data.as_deref().unwrap_or(&[]), pos.to_usize()) .unwrap(), file_index_to_file: &self.file_index_to_file, file_index_to_stable_id: &self.file_index_to_stable_id, alloc_decoding_session: self.alloc_decoding_state.new_decoding_session(), syntax_contexts: &self.syntax_contexts, expn_data: &self.expn_data, foreign_expn_data: &self.foreign_expn_data, hygiene_context: &self.hygiene_context, }; f(&mut decoder) } } //- DECODING ------------------------------------------------------------------- /// A decoder that can read from the incremental compilation cache. It is similar to the one /// we use for crate metadata decoding in that it can rebase spans and eventually /// will also handle things that contain `Ty` instances. pub struct CacheDecoder<'a, 'tcx> { tcx: TyCtxt<'tcx>, opaque: MemDecoder<'a>, file_index_to_file: &'a Lock>>, file_index_to_stable_id: &'a FxHashMap, alloc_decoding_session: AllocDecodingSession<'a>, syntax_contexts: &'a FxHashMap, expn_data: &'a UnhashMap, foreign_expn_data: &'a UnhashMap, hygiene_context: &'a HygieneDecodeContext, } impl<'a, 'tcx> CacheDecoder<'a, 'tcx> { #[inline] fn file_index_to_file(&self, index: SourceFileIndex) -> Arc { let CacheDecoder { tcx, file_index_to_file, file_index_to_stable_id, .. } = *self; Arc::clone(file_index_to_file.borrow_mut().entry(index).or_insert_with(|| { let source_file_id = &file_index_to_stable_id[&index]; let source_file_cnum = tcx.stable_crate_id_to_crate_num(source_file_id.stable_crate_id); // If this `SourceFile` is from a foreign crate, then make sure // that we've imported all of the source files from that crate. // This has usually already been done during macro invocation. // However, when encoding query results like `TypeckResults`, // we might encode an `AdtDef` for a foreign type (because it // was referenced in the body of the function). There is no guarantee // that we will load the source files from that crate during macro // expansion, so we use `import_source_files` to ensure that the foreign // source files are actually imported before we call `source_file_by_stable_id`. if source_file_cnum != LOCAL_CRATE { self.tcx.import_source_files(source_file_cnum); } tcx.sess .source_map() .source_file_by_stable_id(source_file_id.stable_source_file_id) .expect("failed to lookup `SourceFile` in new context") })) } } // Decodes something that was encoded with `encode_tagged()` and verify that the // tag matches and the correct amount of bytes was read. fn decode_tagged(decoder: &mut D, expected_tag: T) -> V where T: Decodable + Eq + std::fmt::Debug, V: Decodable, D: Decoder, { let start_pos = decoder.position(); let actual_tag = T::decode(decoder); assert_eq!(actual_tag, expected_tag); let value = V::decode(decoder); let end_pos = decoder.position(); let expected_len: u64 = Decodable::decode(decoder); assert_eq!((end_pos - start_pos) as u64, expected_len); value } impl<'a, 'tcx> TyDecoder<'tcx> for CacheDecoder<'a, 'tcx> { const CLEAR_CROSS_CRATE: bool = false; #[inline] fn interner(&self) -> TyCtxt<'tcx> { self.tcx } fn cached_ty_for_shorthand(&mut self, shorthand: usize, or_insert_with: F) -> Ty<'tcx> where F: FnOnce(&mut Self) -> Ty<'tcx>, { let tcx = self.tcx; let cache_key = ty::CReaderCacheKey { cnum: None, pos: shorthand }; if let Some(&ty) = tcx.ty_rcache.borrow().get(&cache_key) { return ty; } let ty = or_insert_with(self); // This may overwrite the entry, but it should overwrite with the same value. tcx.ty_rcache.borrow_mut().insert_same(cache_key, ty); ty } fn with_position(&mut self, pos: usize, f: F) -> R where F: FnOnce(&mut Self) -> R, { debug_assert!(pos < self.opaque.len()); let new_opaque = self.opaque.split_at(pos); let old_opaque = mem::replace(&mut self.opaque, new_opaque); let r = f(self); self.opaque = old_opaque; r } fn decode_alloc_id(&mut self) -> interpret::AllocId { let alloc_decoding_session = self.alloc_decoding_session; alloc_decoding_session.decode_alloc_id(self) } } crate::implement_ty_decoder!(CacheDecoder<'a, 'tcx>); // This ensures that the `Decodable::decode` specialization for `Vec` is used // when a `CacheDecoder` is passed to `Decodable::decode`. Unfortunately, we have to manually opt // into specializations this way, given how `CacheDecoder` and the decoding traits currently work. impl<'a, 'tcx> Decodable> for Vec { fn decode(d: &mut CacheDecoder<'a, 'tcx>) -> Self { Decodable::decode(&mut d.opaque) } } impl<'a, 'tcx> SpanDecoder for CacheDecoder<'a, 'tcx> { fn decode_syntax_context(&mut self) -> SyntaxContext { let syntax_contexts = self.syntax_contexts; rustc_span::hygiene::decode_syntax_context(self, self.hygiene_context, |this, id| { // This closure is invoked if we haven't already decoded the data for the `SyntaxContext` we are deserializing. // We look up the position of the associated `SyntaxData` and decode it. let pos = syntax_contexts.get(&id).unwrap(); this.with_position(pos.to_usize(), |decoder| { let data: SyntaxContextData = decode_tagged(decoder, TAG_SYNTAX_CONTEXT); data }) }) } fn decode_expn_id(&mut self) -> ExpnId { let hash = ExpnHash::decode(self); if hash.is_root() { return ExpnId::root(); } if let Some(expn_id) = ExpnId::from_hash(hash) { return expn_id; } let krate = self.tcx.stable_crate_id_to_crate_num(hash.stable_crate_id()); let expn_id = if krate == LOCAL_CRATE { // We look up the position of the associated `ExpnData` and decode it. let pos = self .expn_data .get(&hash) .unwrap_or_else(|| panic!("Bad hash {:?} (map {:?})", hash, self.expn_data)); let data: ExpnData = self.with_position(pos.to_usize(), |decoder| decode_tagged(decoder, TAG_EXPN_DATA)); let expn_id = rustc_span::hygiene::register_local_expn_id(data, hash); #[cfg(debug_assertions)] { use rustc_data_structures::stable_hasher::{HashStable, StableHasher}; let local_hash = self.tcx.with_stable_hashing_context(|mut hcx| { let mut hasher = StableHasher::new(); expn_id.expn_data().hash_stable(&mut hcx, &mut hasher); hasher.finish() }); debug_assert_eq!(hash.local_hash(), local_hash); } expn_id } else { let index_guess = self.foreign_expn_data[&hash]; self.tcx.expn_hash_to_expn_id(krate, index_guess, hash) }; debug_assert_eq!(expn_id.krate, krate); expn_id } fn decode_span(&mut self) -> Span { let ctxt = SyntaxContext::decode(self); let parent = Option::::decode(self); let tag: u8 = Decodable::decode(self); if tag == TAG_PARTIAL_SPAN { return Span::new(BytePos(0), BytePos(0), ctxt, parent); } else if tag == TAG_RELATIVE_SPAN { let dlo = u32::decode(self); let dto = u32::decode(self); let enclosing = self.tcx.source_span_untracked(parent.unwrap()).data_untracked(); let span = Span::new( enclosing.lo + BytePos::from_u32(dlo), enclosing.lo + BytePos::from_u32(dto), ctxt, parent, ); return span; } else { debug_assert_eq!(tag, TAG_FULL_SPAN); } let file_lo_index = SourceFileIndex::decode(self); let line_lo = usize::decode(self); let col_lo = RelativeBytePos::decode(self); let len = BytePos::decode(self); let file_lo = self.file_index_to_file(file_lo_index); let lo = file_lo.lines()[line_lo - 1] + col_lo; let lo = file_lo.absolute_position(lo); let hi = lo + len; Span::new(lo, hi, ctxt, parent) } // copy&paste impl from rustc_metadata #[inline] fn decode_symbol(&mut self) -> Symbol { let tag = self.read_u8(); match tag { SYMBOL_STR => { let s = self.read_str(); Symbol::intern(s) } SYMBOL_OFFSET => { // read str offset let pos = self.read_usize(); // move to str offset and read self.opaque.with_position(pos, |d| { let s = d.read_str(); Symbol::intern(s) }) } SYMBOL_PREINTERNED => { let symbol_index = self.read_u32(); Symbol::new_from_decoded(symbol_index) } _ => unreachable!(), } } fn decode_crate_num(&mut self) -> CrateNum { let stable_id = StableCrateId::decode(self); let cnum = self.tcx.stable_crate_id_to_crate_num(stable_id); cnum } // This impl makes sure that we get a runtime error when we try decode a // `DefIndex` that is not contained in a `DefId`. Such a case would be problematic // because we would not know how to transform the `DefIndex` to the current // context. fn decode_def_index(&mut self) -> DefIndex { panic!("trying to decode `DefIndex` outside the context of a `DefId`") } // Both the `CrateNum` and the `DefIndex` of a `DefId` can change in between two // compilation sessions. We use the `DefPathHash`, which is stable across // sessions, to map the old `DefId` to the new one. fn decode_def_id(&mut self) -> DefId { // Load the `DefPathHash` which is was we encoded the `DefId` as. let def_path_hash = DefPathHash::decode(self); // Using the `DefPathHash`, we can lookup the new `DefId`. // 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. match self.tcx.def_path_hash_to_def_id(def_path_hash) { Some(r) => r, None => panic!("Failed to convert DefPathHash {def_path_hash:?}"), } } fn decode_attr_id(&mut self) -> rustc_span::AttrId { panic!("cannot decode `AttrId` with `CacheDecoder`"); } } impl<'a, 'tcx> Decodable> for &'tcx UnordSet { #[inline] fn decode(d: &mut CacheDecoder<'a, 'tcx>) -> Self { RefDecodable::decode(d) } } impl<'a, 'tcx> Decodable> for &'tcx UnordMap>> { #[inline] fn decode(d: &mut CacheDecoder<'a, 'tcx>) -> Self { RefDecodable::decode(d) } } impl<'a, 'tcx> Decodable> for &'tcx IndexVec> { #[inline] fn decode(d: &mut CacheDecoder<'a, 'tcx>) -> Self { RefDecodable::decode(d) } } impl<'a, 'tcx> Decodable> for &'tcx [(ty::Clause<'tcx>, Span)] { #[inline] fn decode(d: &mut CacheDecoder<'a, 'tcx>) -> Self { RefDecodable::decode(d) } } impl<'a, 'tcx> Decodable> for &'tcx [rustc_ast::InlineAsmTemplatePiece] { #[inline] fn decode(d: &mut CacheDecoder<'a, 'tcx>) -> Self { RefDecodable::decode(d) } } impl<'a, 'tcx> Decodable> for &'tcx [Spanned>] { #[inline] fn decode(d: &mut CacheDecoder<'a, 'tcx>) -> Self { RefDecodable::decode(d) } } impl<'a, 'tcx> Decodable> for &'tcx crate::traits::specialization_graph::Graph { #[inline] fn decode(d: &mut CacheDecoder<'a, 'tcx>) -> Self { RefDecodable::decode(d) } } macro_rules! impl_ref_decoder { (<$tcx:tt> $($ty:ty,)*) => { $(impl<'a, $tcx> Decodable> for &$tcx [$ty] { #[inline] fn decode(d: &mut CacheDecoder<'a, $tcx>) -> Self { RefDecodable::decode(d) } })* }; } impl_ref_decoder! {<'tcx> Span, rustc_hir::Attribute, rustc_span::Ident, ty::Variance, rustc_span::def_id::DefId, rustc_span::def_id::LocalDefId, (rustc_middle::middle::exported_symbols::ExportedSymbol<'tcx>, rustc_middle::middle::exported_symbols::SymbolExportInfo), ty::DeducedParamAttrs, } //- ENCODING ------------------------------------------------------------------- /// An encoder that can write to the incremental compilation cache. pub struct CacheEncoder<'a, 'tcx> { tcx: TyCtxt<'tcx>, encoder: FileEncoder, type_shorthands: FxHashMap, usize>, predicate_shorthands: FxHashMap, usize>, interpret_allocs: FxIndexSet, source_map: CachingSourceMapView<'tcx>, file_to_file_index: FxHashMap<*const SourceFile, SourceFileIndex>, hygiene_context: &'a HygieneEncodeContext, symbol_table: FxHashMap, } impl<'a, 'tcx> CacheEncoder<'a, 'tcx> { #[inline] fn source_file_index(&mut self, source_file: Arc) -> SourceFileIndex { self.file_to_file_index[&(&raw const *source_file)] } /// Encode something with additional information that allows to do some /// sanity checks when decoding the data again. This method will first /// encode the specified tag, then the given value, then the number of /// bytes taken up by tag and value. On decoding, we can then verify that /// we get the expected tag and read the expected number of bytes. pub fn encode_tagged, V: Encodable>(&mut self, tag: T, value: &V) { let start_pos = self.position(); tag.encode(self); value.encode(self); let end_pos = self.position(); ((end_pos - start_pos) as u64).encode(self); } #[inline] fn finish(mut self) -> FileEncodeResult { self.encoder.finish() } } impl<'a, 'tcx> SpanEncoder for CacheEncoder<'a, 'tcx> { fn encode_syntax_context(&mut self, syntax_context: SyntaxContext) { rustc_span::hygiene::raw_encode_syntax_context(syntax_context, self.hygiene_context, self); } fn encode_expn_id(&mut self, expn_id: ExpnId) { self.hygiene_context.schedule_expn_data_for_encoding(expn_id); expn_id.expn_hash().encode(self); } fn encode_span(&mut self, span: Span) { let span_data = span.data_untracked(); span_data.ctxt.encode(self); span_data.parent.encode(self); if span_data.is_dummy() { return TAG_PARTIAL_SPAN.encode(self); } if let Some(parent) = span_data.parent { let enclosing = self.tcx.source_span_untracked(parent).data_untracked(); if enclosing.contains(span_data) { TAG_RELATIVE_SPAN.encode(self); (span_data.lo - enclosing.lo).to_u32().encode(self); (span_data.hi - enclosing.lo).to_u32().encode(self); return; } } let pos = self.source_map.byte_pos_to_line_and_col(span_data.lo); let partial_span = match &pos { Some((file_lo, _, _)) => !file_lo.contains(span_data.hi), None => true, }; if partial_span { return TAG_PARTIAL_SPAN.encode(self); } let (file_lo, line_lo, col_lo) = pos.unwrap(); let len = span_data.hi - span_data.lo; let source_file_index = self.source_file_index(file_lo); TAG_FULL_SPAN.encode(self); source_file_index.encode(self); line_lo.encode(self); col_lo.encode(self); len.encode(self); } // copy&paste impl from rustc_metadata fn encode_symbol(&mut self, symbol: Symbol) { // if symbol preinterned, emit tag and symbol index if symbol.is_preinterned() { self.encoder.emit_u8(SYMBOL_PREINTERNED); self.encoder.emit_u32(symbol.as_u32()); } else { // otherwise write it as string or as offset to it match self.symbol_table.entry(symbol) { Entry::Vacant(o) => { self.encoder.emit_u8(SYMBOL_STR); let pos = self.encoder.position(); o.insert(pos); self.emit_str(symbol.as_str()); } Entry::Occupied(o) => { let x = *o.get(); self.emit_u8(SYMBOL_OFFSET); self.emit_usize(x); } } } } fn encode_crate_num(&mut self, crate_num: CrateNum) { self.tcx.stable_crate_id(crate_num).encode(self); } fn encode_def_id(&mut self, def_id: DefId) { self.tcx.def_path_hash(def_id).encode(self); } fn encode_def_index(&mut self, _def_index: DefIndex) { bug!("encoding `DefIndex` without context"); } } impl<'a, 'tcx> TyEncoder<'tcx> for CacheEncoder<'a, 'tcx> { const CLEAR_CROSS_CRATE: bool = false; #[inline] fn position(&self) -> usize { self.encoder.position() } #[inline] fn type_shorthands(&mut self) -> &mut FxHashMap, usize> { &mut self.type_shorthands } #[inline] fn predicate_shorthands(&mut self) -> &mut FxHashMap, usize> { &mut self.predicate_shorthands } #[inline] fn encode_alloc_id(&mut self, alloc_id: &interpret::AllocId) { let (index, _) = self.interpret_allocs.insert_full(*alloc_id); index.encode(self); } } macro_rules! encoder_methods { ($($name:ident($ty:ty);)*) => { #[inline] $(fn $name(&mut self, value: $ty) { self.encoder.$name(value) })* } } impl<'a, 'tcx> Encoder for CacheEncoder<'a, 'tcx> { encoder_methods! { emit_usize(usize); emit_u128(u128); emit_u64(u64); emit_u32(u32); emit_u16(u16); emit_u8(u8); emit_isize(isize); emit_i128(i128); emit_i64(i64); emit_i32(i32); emit_i16(i16); emit_raw_bytes(&[u8]); } } // This ensures that the `Encodable::encode` specialization for byte slices // is used when a `CacheEncoder` having an `opaque::FileEncoder` is passed to `Encodable::encode`. // Unfortunately, we have to manually opt into specializations this way, given how `CacheEncoder` // and the encoding traits currently work. impl<'a, 'tcx> Encodable> for [u8] { fn encode(&self, e: &mut CacheEncoder<'a, 'tcx>) { self.encode(&mut e.encoder); } }