about summary refs log tree commit diff
path: root/compiler/rustc_query_system/src/dep_graph/serialized.rs
diff options
context:
space:
mode:
Diffstat (limited to 'compiler/rustc_query_system/src/dep_graph/serialized.rs')
-rw-r--r--compiler/rustc_query_system/src/dep_graph/serialized.rs284
1 files changed, 256 insertions, 28 deletions
diff --git a/compiler/rustc_query_system/src/dep_graph/serialized.rs b/compiler/rustc_query_system/src/dep_graph/serialized.rs
index edddfda6242..91460fa0399 100644
--- a/compiler/rustc_query_system/src/dep_graph/serialized.rs
+++ b/compiler/rustc_query_system/src/dep_graph/serialized.rs
@@ -1,6 +1,6 @@
 //! The data that we will serialize and deserialize.
 //!
-//! The dep-graph is serialized as a sequence of NodeInfo, with the dependencies
+//! Notionally, the dep-graph is a sequence of NodeInfo with the dependencies
 //! specified inline. The total number of nodes and edges are stored as the last
 //! 16 bytes of the file, so we can find them easily at decoding time.
 //!
@@ -14,14 +14,16 @@
 
 use super::query::DepGraphQuery;
 use super::{DepKind, DepNode, DepNodeIndex};
+use crate::dep_graph::EdgesVec;
 use rustc_data_structures::fingerprint::Fingerprint;
+use rustc_data_structures::fingerprint::PackedFingerprint;
 use rustc_data_structures::fx::FxHashMap;
 use rustc_data_structures::profiling::SelfProfilerRef;
 use rustc_data_structures::sync::Lock;
 use rustc_index::{Idx, IndexVec};
 use rustc_serialize::opaque::{FileEncodeResult, FileEncoder, IntEncodedWithFixedSize, MemDecoder};
-use rustc_serialize::{Decodable, Decoder, Encodable};
-use smallvec::SmallVec;
+use rustc_serialize::{Decodable, Decoder, Encodable, Encoder};
+use std::marker::PhantomData;
 
 // The maximum value of `SerializedDepNodeIndex` leaves the upper two bits
 // unused so that we can store multiple index types in `CompressedHybridIndex`,
@@ -31,6 +33,16 @@ rustc_index::newtype_index! {
     pub struct SerializedDepNodeIndex {}
 }
 
+const DEP_NODE_SIZE: usize = std::mem::size_of::<SerializedDepNodeIndex>();
+/// Amount of padding we need to add to the edge list data so that we can retrieve every
+/// SerializedDepNodeIndex with a fixed-size read then mask.
+const DEP_NODE_PAD: usize = DEP_NODE_SIZE - 1;
+/// Amount of bits we need to store the number of used bytes in a SerializedDepNodeIndex.
+/// Note that wherever we encode byte widths like this we actually store the number of bytes used
+/// minus 1; for a 4-byte value we technically would have 5 widths to store, but using one byte to
+/// store zeroes (which are relatively rare) is a decent tradeoff to save a bit in our bitfields.
+const DEP_NODE_WIDTH_BITS: usize = DEP_NODE_SIZE / 2;
+
 /// Data for use when recompiling the **current crate**.
 #[derive(Debug)]
 pub struct SerializedDepGraph<K: DepKind> {
@@ -42,10 +54,10 @@ pub struct SerializedDepGraph<K: DepKind> {
     /// For each DepNode, stores the list of edges originating from that
     /// DepNode. Encoded as a [start, end) pair indexing into edge_list_data,
     /// which holds the actual DepNodeIndices of the target nodes.
-    edge_list_indices: IndexVec<SerializedDepNodeIndex, (u32, u32)>,
-    /// A flattened list of all edge targets in the graph. Edge sources are
-    /// implicit in edge_list_indices.
-    edge_list_data: Vec<SerializedDepNodeIndex>,
+    edge_list_indices: IndexVec<SerializedDepNodeIndex, EdgeHeader>,
+    /// A flattened list of all edge targets in the graph, stored in the same
+    /// varint encoding that we use on disk. Edge sources are implicit in edge_list_indices.
+    edge_list_data: Vec<u8>,
     /// Reciprocal map to `nodes`.
     index: FxHashMap<DepNode<K>, SerializedDepNodeIndex>,
 }
@@ -64,9 +76,35 @@ impl<K: DepKind> Default for SerializedDepGraph<K> {
 
 impl<K: DepKind> SerializedDepGraph<K> {
     #[inline]
-    pub fn edge_targets_from(&self, source: SerializedDepNodeIndex) -> &[SerializedDepNodeIndex] {
-        let targets = self.edge_list_indices[source];
-        &self.edge_list_data[targets.0 as usize..targets.1 as usize]
+    pub fn edge_targets_from(
+        &self,
+        source: SerializedDepNodeIndex,
+    ) -> impl Iterator<Item = SerializedDepNodeIndex> + '_ {
+        let header = self.edge_list_indices[source];
+        let mut raw = &self.edge_list_data[header.start()..];
+        // Figure out where the edge list for `source` ends by getting the start index of the next
+        // edge list, or the end of the array if this is the last edge.
+        let end = self
+            .edge_list_indices
+            .get(source + 1)
+            .map(|h| h.start())
+            .unwrap_or_else(|| self.edge_list_data.len() - DEP_NODE_PAD);
+
+        // The number of edges for this node is implicitly stored in the combination of the byte
+        // width and the length.
+        let bytes_per_index = header.bytes_per_index();
+        let len = (end - header.start()) / bytes_per_index;
+
+        // LLVM doesn't hoist EdgeHeader::mask so we do it ourselves.
+        let mask = header.mask();
+        (0..len).map(move |_| {
+            // Doing this slicing in this order ensures that the first bounds check suffices for
+            // all the others.
+            let index = &raw[..DEP_NODE_SIZE];
+            raw = &raw[bytes_per_index..];
+            let index = u32::from_le_bytes(index.try_into().unwrap()) & mask;
+            SerializedDepNodeIndex::from_u32(index)
+        })
     }
 
     #[inline]
@@ -84,11 +122,42 @@ impl<K: DepKind> SerializedDepGraph<K> {
         self.fingerprints[dep_node_index]
     }
 
+    #[inline]
     pub fn node_count(&self) -> usize {
         self.index.len()
     }
 }
 
+/// A packed representation of an edge's start index and byte width.
+///
+/// This is packed by stealing 2 bits from the start index, which means we only accomodate edge
+/// data arrays up to a quarter of our address space. Which seems fine.
+#[derive(Debug, Clone, Copy)]
+struct EdgeHeader {
+    repr: usize,
+}
+
+impl EdgeHeader {
+    #[inline]
+    fn start(self) -> usize {
+        self.repr >> DEP_NODE_WIDTH_BITS
+    }
+
+    #[inline]
+    fn bytes_per_index(self) -> usize {
+        (self.repr & mask(DEP_NODE_WIDTH_BITS)) + 1
+    }
+
+    #[inline]
+    fn mask(self) -> u32 {
+        mask(self.bytes_per_index() * 8) as u32
+    }
+}
+
+fn mask(bits: usize) -> usize {
+    usize::MAX >> ((std::mem::size_of::<usize>() * 8) - bits)
+}
+
 impl<'a, K: DepKind + Decodable<MemDecoder<'a>>> Decodable<MemDecoder<'a>>
     for SerializedDepGraph<K>
 {
@@ -107,32 +176,51 @@ impl<'a, K: DepKind + Decodable<MemDecoder<'a>>> Decodable<MemDecoder<'a>>
 
         debug!(?node_count, ?edge_count);
 
+        let graph_bytes = d.len() - (2 * IntEncodedWithFixedSize::ENCODED_SIZE) - d.position();
+
         let mut nodes = IndexVec::with_capacity(node_count);
         let mut fingerprints = IndexVec::with_capacity(node_count);
         let mut edge_list_indices = IndexVec::with_capacity(node_count);
-        let mut edge_list_data = Vec::with_capacity(edge_count);
+        // This slightly over-estimates the amount of bytes used for all the edge data but never by
+        // more than ~6%, because over-estimation only occurs for large nodes.
+        let mut edge_list_data = Vec::with_capacity(
+            graph_bytes - node_count * std::mem::size_of::<SerializedNodeHeader<K>>(),
+        );
 
         for _index in 0..node_count {
-            let dep_node: DepNode<K> = Decodable::decode(d);
-            let _i: SerializedDepNodeIndex = nodes.push(dep_node);
+            // Decode the header for this edge; the header packs together as many of the fixed-size
+            // fields as possible to limit the number of times we update decoder state.
+            let node_header = SerializedNodeHeader { bytes: d.read_array(), _marker: PhantomData };
+
+            let _i: SerializedDepNodeIndex = nodes.push(node_header.node());
             debug_assert_eq!(_i.index(), _index);
 
-            let fingerprint: Fingerprint = Decodable::decode(d);
-            let _i: SerializedDepNodeIndex = fingerprints.push(fingerprint);
+            let _i: SerializedDepNodeIndex = fingerprints.push(node_header.fingerprint());
             debug_assert_eq!(_i.index(), _index);
 
-            // Deserialize edges -- sequence of DepNodeIndex
-            let len = d.read_usize();
-            let start = edge_list_data.len().try_into().unwrap();
-            for _ in 0..len {
-                let edge = Decodable::decode(d);
-                edge_list_data.push(edge);
-            }
-            let end = edge_list_data.len().try_into().unwrap();
-            let _i: SerializedDepNodeIndex = edge_list_indices.push((start, end));
+            // If the length of this node's edge list is small, the length is stored in the header.
+            // If it is not, we fall back to another decoder call.
+            let num_edges = node_header.len().unwrap_or_else(|| d.read_usize());
+
+            // The edges index list uses the same varint strategy as rmeta tables; we select the
+            // number of byte elements per-array not per-element. This lets us read the whole edge
+            // list for a node with one decoder call and also use the on-disk format in memory.
+            let edges_len_bytes = node_header.bytes_per_index() * num_edges;
+            // The in-memory structure for the edges list stores the byte width of the edges on
+            // this node with the offset into the global edge data array.
+            let edges_header = node_header.edges_header(&edge_list_data);
+
+            edge_list_data.extend(d.read_raw_bytes(edges_len_bytes));
+
+            let _i: SerializedDepNodeIndex = edge_list_indices.push(edges_header);
             debug_assert_eq!(_i.index(), _index);
         }
 
+        // When we access the edge list data, we do a fixed-size read from the edge list data then
+        // mask off the bytes that aren't for that edge index, so the last read may dangle off the
+        // end of the array. This padding ensure it doesn't.
+        edge_list_data.extend(&[0u8; DEP_NODE_PAD]);
+
         let index: FxHashMap<_, _> =
             nodes.iter_enumerated().map(|(idx, &dep_node)| (dep_node, idx)).collect();
 
@@ -140,11 +228,151 @@ impl<'a, K: DepKind + Decodable<MemDecoder<'a>>> Decodable<MemDecoder<'a>>
     }
 }
 
-#[derive(Debug, Encodable, Decodable)]
-pub struct NodeInfo<K: DepKind> {
+/// A packed representation of all the fixed-size fields in a `NodeInfo`.
+///
+/// This stores in one byte array:
+/// * The `Fingerprint` in the `NodeInfo`
+/// * The `Fingerprint` in `DepNode` that is in this `NodeInfo`
+/// * The `DepKind`'s discriminant (a u16, but not all bits are used...)
+/// * The byte width of the encoded edges for this node
+/// * In whatever bits remain, the length of the edge list for this node, if it fits
+struct SerializedNodeHeader<K> {
+    // 2 bytes for the DepNode
+    // 16 for Fingerprint in DepNode
+    // 16 for Fingerprint in NodeInfo
+    bytes: [u8; 34],
+    _marker: PhantomData<K>,
+}
+
+// The fields of a `SerializedNodeHeader`, this struct is an implementation detail and exists only
+// to make the implementation of `SerializedNodeHeader` simpler.
+struct Unpacked<K> {
+    len: Option<usize>,
+    bytes_per_index: usize,
+    kind: K,
+    hash: PackedFingerprint,
+    fingerprint: Fingerprint,
+}
+
+// Bit fields are
+// 0..?    length of the edge
+// ?..?+2  bytes per index
+// ?+2..16 kind
+impl<K: DepKind> SerializedNodeHeader<K> {
+    const TOTAL_BITS: usize = std::mem::size_of::<K>() * 8;
+    const LEN_BITS: usize = Self::TOTAL_BITS - Self::KIND_BITS - Self::WIDTH_BITS;
+    const WIDTH_BITS: usize = DEP_NODE_WIDTH_BITS;
+    const KIND_BITS: usize = Self::TOTAL_BITS - K::MAX.leading_zeros() as usize;
+    const MAX_INLINE_LEN: usize = (u16::MAX as usize >> (Self::TOTAL_BITS - Self::LEN_BITS)) - 1;
+
+    #[inline]
+    fn new(node_info: &NodeInfo<K>) -> Self {
+        debug_assert_eq!(Self::TOTAL_BITS, Self::LEN_BITS + Self::WIDTH_BITS + Self::KIND_BITS);
+
+        let NodeInfo { node, fingerprint, edges } = node_info;
+
+        let mut head = node.kind.to_u16();
+
+        let free_bytes = edges.max_index().leading_zeros() as usize / 8;
+        let bytes_per_index = (DEP_NODE_SIZE - free_bytes).saturating_sub(1);
+        head |= (bytes_per_index as u16) << Self::KIND_BITS;
+
+        // Encode number of edges + 1 so that we can reserve 0 to indicate that the len doesn't fit
+        // in this bitfield.
+        if edges.len() <= Self::MAX_INLINE_LEN {
+            head |= (edges.len() as u16 + 1) << (Self::KIND_BITS + Self::WIDTH_BITS);
+        }
+
+        let hash: Fingerprint = node.hash.into();
+
+        // Using half-open ranges ensures an unconditional panic if we get the magic numbers wrong.
+        let mut bytes = [0u8; 34];
+        bytes[..2].copy_from_slice(&head.to_le_bytes());
+        bytes[2..18].copy_from_slice(&hash.to_le_bytes());
+        bytes[18..].copy_from_slice(&fingerprint.to_le_bytes());
+
+        #[cfg(debug_assertions)]
+        {
+            let res = Self { bytes, _marker: PhantomData };
+            assert_eq!(node_info.fingerprint, res.fingerprint());
+            assert_eq!(node_info.node, res.node());
+            if let Some(len) = res.len() {
+                assert_eq!(node_info.edges.len(), len);
+            }
+        }
+        Self { bytes, _marker: PhantomData }
+    }
+
+    #[inline]
+    fn unpack(&self) -> Unpacked<K> {
+        let head = u16::from_le_bytes(self.bytes[..2].try_into().unwrap());
+        let hash = self.bytes[2..18].try_into().unwrap();
+        let fingerprint = self.bytes[18..].try_into().unwrap();
+
+        let kind = head & mask(Self::KIND_BITS) as u16;
+        let bytes_per_index = (head >> Self::KIND_BITS) & mask(Self::WIDTH_BITS) as u16;
+        let len = (head as usize) >> (Self::WIDTH_BITS + Self::KIND_BITS);
+
+        Unpacked {
+            len: len.checked_sub(1),
+            bytes_per_index: bytes_per_index as usize + 1,
+            kind: DepKind::from_u16(kind),
+            hash: Fingerprint::from_le_bytes(hash).into(),
+            fingerprint: Fingerprint::from_le_bytes(fingerprint),
+        }
+    }
+
+    #[inline]
+    fn len(&self) -> Option<usize> {
+        self.unpack().len
+    }
+
+    #[inline]
+    fn bytes_per_index(&self) -> usize {
+        self.unpack().bytes_per_index
+    }
+
+    #[inline]
+    fn fingerprint(&self) -> Fingerprint {
+        self.unpack().fingerprint
+    }
+
+    #[inline]
+    fn node(&self) -> DepNode<K> {
+        let Unpacked { kind, hash, .. } = self.unpack();
+        DepNode { kind, hash }
+    }
+
+    #[inline]
+    fn edges_header(&self, edge_list_data: &[u8]) -> EdgeHeader {
+        EdgeHeader {
+            repr: (edge_list_data.len() << DEP_NODE_WIDTH_BITS) | (self.bytes_per_index() - 1),
+        }
+    }
+}
+
+#[derive(Debug)]
+struct NodeInfo<K: DepKind> {
     node: DepNode<K>,
     fingerprint: Fingerprint,
-    edges: SmallVec<[DepNodeIndex; 8]>,
+    edges: EdgesVec,
+}
+
+impl<K: DepKind> Encodable<FileEncoder> for NodeInfo<K> {
+    fn encode(&self, e: &mut FileEncoder) {
+        let header = SerializedNodeHeader::new(self);
+        e.emit_raw_bytes(&header.bytes);
+
+        if header.len().is_none() {
+            e.emit_usize(self.edges.len());
+        }
+
+        let bytes_per_index = header.bytes_per_index();
+        for node_index in self.edges.iter() {
+            let bytes = node_index.as_u32().to_le_bytes();
+            e.emit_raw_bytes(&bytes[..bytes_per_index]);
+        }
+    }
 }
 
 struct Stat<K: DepKind> {
@@ -303,7 +531,7 @@ impl<K: DepKind + Encodable<FileEncoder>> GraphEncoder<K> {
         profiler: &SelfProfilerRef,
         node: DepNode<K>,
         fingerprint: Fingerprint,
-        edges: SmallVec<[DepNodeIndex; 8]>,
+        edges: EdgesVec,
     ) -> DepNodeIndex {
         let _prof_timer = profiler.generic_activity("incr_comp_encode_dep_graph");
         let node = NodeInfo { node, fingerprint, edges };