rustc_query_system: share previous graph data with current graph
Reduce memory consumption by taking advantage of red/green algorithm properties to share the previous dependency graph's node data with the current graph instead of storing node data redundantly. Red nodes can share the `DepNode`, and green nodes can share the `DepNode` and `Fingerprint`. Edges will be shared when possible in a later change.
This commit is contained in:
parent
bb1fbbf844
commit
f6d6b0c96d
@ -68,7 +68,7 @@ struct DepGraphData<K: DepKind> {
|
||||
/// The new encoding of the dependency graph, optimized for red/green
|
||||
/// tracking. The `current` field is the dependency graph of only the
|
||||
/// current compilation session: We don't merge the previous dep-graph into
|
||||
/// current one anymore.
|
||||
/// current one anymore, but we do reference shared data to save space.
|
||||
current: CurrentDepGraph<K>,
|
||||
|
||||
/// The dep-graph from the previous compilation session. It contains all
|
||||
@ -134,15 +134,43 @@ impl<K: DepKind> DepGraph<K> {
|
||||
}
|
||||
|
||||
pub fn query(&self) -> DepGraphQuery<K> {
|
||||
let data = self.data.as_ref().unwrap().current.data.lock();
|
||||
let nodes: Vec<_> = data.iter().map(|n| n.node).collect();
|
||||
let mut edges = Vec::new();
|
||||
for (from, edge_targets) in data.iter().map(|d| (d.node, &d.edges)) {
|
||||
for &edge_target in edge_targets.iter() {
|
||||
let to = data[edge_target].node;
|
||||
edges.push((from, to));
|
||||
let data = self.data.as_ref().unwrap();
|
||||
let previous = &data.previous;
|
||||
let data = data.current.data.lock();
|
||||
|
||||
let node_count = data.hybrid_indices.len();
|
||||
|
||||
let edge_count = data.new.edges.iter().map(|e| e.len()).sum::<usize>()
|
||||
+ data.red.edges.iter().map(|e| e.len()).sum::<usize>()
|
||||
+ data.green.edges.iter().map(|e| e.len()).sum::<usize>();
|
||||
|
||||
let mut nodes = Vec::with_capacity(node_count);
|
||||
let mut edges = Vec::with_capacity(edge_count);
|
||||
|
||||
for (index, &hybrid_index) in data.hybrid_indices.iter_enumerated() {
|
||||
let src = index.index();
|
||||
|
||||
match hybrid_index.into() {
|
||||
HybridIndex::New(new_index) => {
|
||||
let new = &data.new;
|
||||
nodes.push(new.nodes[new_index]);
|
||||
edges.extend(new.edges[new_index].iter().map(|dst| (src, dst.index())));
|
||||
}
|
||||
HybridIndex::Red(red_index) => {
|
||||
let red = &data.red;
|
||||
nodes.push(previous.index_to_node(red.node_indices[red_index]));
|
||||
edges.extend(red.edges[red_index].iter().map(|dst| (src, dst.index())));
|
||||
}
|
||||
HybridIndex::Green(green_index) => {
|
||||
let green = &data.green;
|
||||
nodes.push(previous.index_to_node(green.node_indices[green_index]));
|
||||
edges.extend(green.edges[green_index].iter().map(|dst| (src, dst.index())));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
debug_assert_eq!(nodes.len(), node_count);
|
||||
debug_assert_eq!(edges.len(), edge_count);
|
||||
|
||||
DepGraphQuery::new(&nodes[..], &edges[..])
|
||||
}
|
||||
@ -212,7 +240,6 @@ impl<K: DepKind> DepGraph<K> {
|
||||
phantom_data: PhantomData,
|
||||
})
|
||||
},
|
||||
|data, key, fingerprint, task| data.complete_task(key, task.unwrap(), fingerprint),
|
||||
hash_result,
|
||||
)
|
||||
}
|
||||
@ -225,12 +252,6 @@ impl<K: DepKind> DepGraph<K> {
|
||||
no_tcx: bool,
|
||||
task: fn(Ctxt, A) -> R,
|
||||
create_task: fn(DepNode<K>) -> Option<TaskDeps<K>>,
|
||||
finish_task_and_alloc_depnode: fn(
|
||||
&CurrentDepGraph<K>,
|
||||
DepNode<K>,
|
||||
Fingerprint,
|
||||
Option<TaskDeps<K>>,
|
||||
) -> DepNodeIndex,
|
||||
hash_result: impl FnOnce(&mut Ctxt::StableHashingContext, &R) -> Option<Fingerprint>,
|
||||
) -> (R, DepNodeIndex) {
|
||||
if let Some(ref data) = self.data {
|
||||
@ -249,39 +270,54 @@ impl<K: DepKind> DepGraph<K> {
|
||||
K::with_deps(task_deps.as_ref(), || task(cx, arg))
|
||||
};
|
||||
|
||||
let current_fingerprint = hash_result(&mut hcx, &result);
|
||||
let edges = task_deps.map_or_else(|| smallvec![], |lock| lock.into_inner().reads);
|
||||
|
||||
let dep_node_index = finish_task_and_alloc_depnode(
|
||||
&data.current,
|
||||
key,
|
||||
current_fingerprint.unwrap_or(Fingerprint::ZERO),
|
||||
task_deps.map(|lock| lock.into_inner()),
|
||||
);
|
||||
let current_fingerprint = hash_result(&mut hcx, &result);
|
||||
|
||||
let print_status = cfg!(debug_assertions) && cx.debug_dep_tasks();
|
||||
|
||||
// Determine the color of the new DepNode.
|
||||
if let Some(prev_index) = data.previous.node_to_index_opt(&key) {
|
||||
let prev_fingerprint = data.previous.fingerprint_by_index(prev_index);
|
||||
|
||||
let color = if let Some(current_fingerprint) = current_fingerprint {
|
||||
if current_fingerprint == prev_fingerprint {
|
||||
// Intern the new `DepNode`.
|
||||
let dep_node_index = if let Some(prev_index) = data.previous.node_to_index_opt(&key) {
|
||||
// Determine the color and index of the new `DepNode`.
|
||||
let (color, dep_node_index) = if let Some(current_fingerprint) = current_fingerprint
|
||||
{
|
||||
if current_fingerprint == data.previous.fingerprint_by_index(prev_index) {
|
||||
if print_status {
|
||||
eprintln!("[task::green] {:?}", key);
|
||||
}
|
||||
DepNodeColor::Green(dep_node_index)
|
||||
|
||||
let dep_node_index =
|
||||
data.current.intern_green_node(&data.previous, prev_index, edges);
|
||||
|
||||
(DepNodeColor::Green(dep_node_index), dep_node_index)
|
||||
} else {
|
||||
if print_status {
|
||||
eprintln!("[task::red] {:?}", key);
|
||||
}
|
||||
DepNodeColor::Red
|
||||
|
||||
let dep_node_index = data.current.intern_red_node(
|
||||
&data.previous,
|
||||
prev_index,
|
||||
edges,
|
||||
current_fingerprint,
|
||||
);
|
||||
|
||||
(DepNodeColor::Red, dep_node_index)
|
||||
}
|
||||
} else {
|
||||
if print_status {
|
||||
eprintln!("[task::unknown] {:?}", key);
|
||||
}
|
||||
|
||||
let dep_node_index = data.current.intern_red_node(
|
||||
&data.previous,
|
||||
prev_index,
|
||||
edges,
|
||||
Fingerprint::ZERO,
|
||||
);
|
||||
|
||||
// Mark the node as Red if we can't hash the result
|
||||
DepNodeColor::Red
|
||||
(DepNodeColor::Red, dep_node_index)
|
||||
};
|
||||
|
||||
debug_assert!(
|
||||
@ -292,10 +328,20 @@ impl<K: DepKind> DepGraph<K> {
|
||||
);
|
||||
|
||||
data.colors.insert(prev_index, color);
|
||||
} else if print_status {
|
||||
dep_node_index
|
||||
} else {
|
||||
if print_status {
|
||||
eprintln!("[task::new] {:?}", key);
|
||||
}
|
||||
|
||||
data.current.intern_node(
|
||||
&data.previous,
|
||||
key,
|
||||
edges,
|
||||
current_fingerprint.unwrap_or(Fingerprint::ZERO),
|
||||
)
|
||||
};
|
||||
|
||||
(result, dep_node_index)
|
||||
} else {
|
||||
(task(cx, arg), self.next_virtual_depnode_index())
|
||||
@ -308,13 +354,36 @@ impl<K: DepKind> DepGraph<K> {
|
||||
where
|
||||
OP: FnOnce() -> R,
|
||||
{
|
||||
debug_assert!(!dep_kind.is_eval_always());
|
||||
|
||||
if let Some(ref data) = self.data {
|
||||
let task_deps = Lock::new(TaskDeps::default());
|
||||
|
||||
let result = K::with_deps(Some(&task_deps), op);
|
||||
let task_deps = task_deps.into_inner();
|
||||
|
||||
let dep_node_index = data.current.complete_anon_task(dep_kind, task_deps);
|
||||
// The dep node indices are hashed here instead of hashing the dep nodes of the
|
||||
// dependencies. These indices may refer to different nodes per session, but this isn't
|
||||
// a problem here because we that ensure the final dep node hash is per session only by
|
||||
// combining it with the per session random number `anon_id_seed`. This hash only need
|
||||
// to map the dependencies to a single value on a per session basis.
|
||||
let mut hasher = StableHasher::new();
|
||||
task_deps.reads.hash(&mut hasher);
|
||||
|
||||
let target_dep_node = DepNode {
|
||||
kind: dep_kind,
|
||||
// Fingerprint::combine() is faster than sending Fingerprint
|
||||
// through the StableHasher (at least as long as StableHasher
|
||||
// is so slow).
|
||||
hash: data.current.anon_id_seed.combine(hasher.finish()).into(),
|
||||
};
|
||||
|
||||
let dep_node_index = data.current.intern_node(
|
||||
&data.previous,
|
||||
target_dep_node,
|
||||
task_deps.reads,
|
||||
Fingerprint::ZERO,
|
||||
);
|
||||
|
||||
(result, dep_node_index)
|
||||
} else {
|
||||
(op(), self.next_virtual_depnode_index())
|
||||
@ -331,69 +400,104 @@ impl<K: DepKind> DepGraph<K> {
|
||||
task: fn(Ctxt, A) -> R,
|
||||
hash_result: impl FnOnce(&mut Ctxt::StableHashingContext, &R) -> Option<Fingerprint>,
|
||||
) -> (R, DepNodeIndex) {
|
||||
self.with_task_impl(
|
||||
key,
|
||||
cx,
|
||||
arg,
|
||||
false,
|
||||
task,
|
||||
|_| None,
|
||||
|data, key, fingerprint, _| data.alloc_node(key, smallvec![], fingerprint),
|
||||
hash_result,
|
||||
)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn read(&self, v: DepNode<K>) {
|
||||
if let Some(ref data) = self.data {
|
||||
let map = data.current.node_to_node_index.get_shard_by_value(&v).lock();
|
||||
if let Some(dep_node_index) = map.get(&v).copied() {
|
||||
std::mem::drop(map);
|
||||
data.read_index(dep_node_index);
|
||||
} else {
|
||||
panic!("DepKind {:?} should be pre-allocated but isn't.", v.kind)
|
||||
}
|
||||
}
|
||||
self.with_task_impl(key, cx, arg, false, task, |_| None, hash_result)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn read_index(&self, dep_node_index: DepNodeIndex) {
|
||||
if let Some(ref data) = self.data {
|
||||
data.read_index(dep_node_index);
|
||||
K::read_deps(|task_deps| {
|
||||
if let Some(task_deps) = task_deps {
|
||||
let mut task_deps = task_deps.lock();
|
||||
let task_deps = &mut *task_deps;
|
||||
if cfg!(debug_assertions) {
|
||||
data.current.total_read_count.fetch_add(1, Relaxed);
|
||||
}
|
||||
|
||||
// As long as we only have a low number of reads we can avoid doing a hash
|
||||
// insert and potentially allocating/reallocating the hashmap
|
||||
let new_read = if task_deps.reads.len() < TASK_DEPS_READS_CAP {
|
||||
task_deps.reads.iter().all(|other| *other != dep_node_index)
|
||||
} else {
|
||||
task_deps.read_set.insert(dep_node_index)
|
||||
};
|
||||
if new_read {
|
||||
task_deps.reads.push(dep_node_index);
|
||||
if task_deps.reads.len() == TASK_DEPS_READS_CAP {
|
||||
// Fill `read_set` with what we have so far so we can use the hashset
|
||||
// next time
|
||||
task_deps.read_set.extend(task_deps.reads.iter().copied());
|
||||
}
|
||||
|
||||
#[cfg(debug_assertions)]
|
||||
{
|
||||
if let Some(target) = task_deps.node {
|
||||
if let Some(ref forbidden_edge) = data.current.forbidden_edge {
|
||||
let src = self.dep_node_of(dep_node_index);
|
||||
if forbidden_edge.test(&src, &target) {
|
||||
panic!("forbidden edge {:?} -> {:?} created", src, target)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if cfg!(debug_assertions) {
|
||||
data.current.total_duplicate_read_count.fetch_add(1, Relaxed);
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn dep_node_index_of(&self, dep_node: &DepNode<K>) -> DepNodeIndex {
|
||||
self.data
|
||||
.as_ref()
|
||||
.unwrap()
|
||||
.current
|
||||
.node_to_node_index
|
||||
.get_shard_by_value(dep_node)
|
||||
.lock()
|
||||
.get(dep_node)
|
||||
.cloned()
|
||||
.unwrap()
|
||||
self.dep_node_index_of_opt(dep_node).unwrap()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn dep_node_index_of_opt(&self, dep_node: &DepNode<K>) -> Option<DepNodeIndex> {
|
||||
let data = self.data.as_ref().unwrap();
|
||||
let current = &data.current;
|
||||
|
||||
if let Some(prev_index) = data.previous.node_to_index_opt(dep_node) {
|
||||
current.prev_index_to_index.lock()[prev_index]
|
||||
} else {
|
||||
current.new_node_to_index.get_shard_by_value(dep_node).lock().get(dep_node).copied()
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn dep_node_exists(&self, dep_node: &DepNode<K>) -> bool {
|
||||
if let Some(ref data) = self.data {
|
||||
data.current
|
||||
.node_to_node_index
|
||||
.get_shard_by_value(&dep_node)
|
||||
.lock()
|
||||
.contains_key(dep_node)
|
||||
} else {
|
||||
false
|
||||
self.data.is_some() && self.dep_node_index_of_opt(dep_node).is_some()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn dep_node_of(&self, dep_node_index: DepNodeIndex) -> DepNode<K> {
|
||||
let data = self.data.as_ref().unwrap();
|
||||
let previous = &data.previous;
|
||||
let data = data.current.data.lock();
|
||||
|
||||
match data.hybrid_indices[dep_node_index].into() {
|
||||
HybridIndex::New(new_index) => data.new.nodes[new_index],
|
||||
HybridIndex::Red(red_index) => previous.index_to_node(data.red.node_indices[red_index]),
|
||||
HybridIndex::Green(green_index) => {
|
||||
previous.index_to_node(data.green.node_indices[green_index])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn fingerprint_of(&self, dep_node_index: DepNodeIndex) -> Fingerprint {
|
||||
let data = self.data.as_ref().expect("dep graph enabled").current.data.lock();
|
||||
data[dep_node_index].fingerprint
|
||||
let data = self.data.as_ref().unwrap();
|
||||
let previous = &data.previous;
|
||||
let data = data.current.data.lock();
|
||||
|
||||
match data.hybrid_indices[dep_node_index].into() {
|
||||
HybridIndex::New(new_index) => data.new.fingerprints[new_index],
|
||||
HybridIndex::Red(red_index) => data.red.fingerprints[red_index],
|
||||
HybridIndex::Green(green_index) => {
|
||||
previous.fingerprint_by_index(data.green.node_indices[green_index])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn prev_fingerprint_of(&self, dep_node: &DepNode<K>) -> Option<Fingerprint> {
|
||||
@ -444,29 +548,62 @@ impl<K: DepKind> DepGraph<K> {
|
||||
}
|
||||
|
||||
pub fn serialize(&self) -> SerializedDepGraph<K> {
|
||||
let data = self.data.as_ref().unwrap().current.data.lock();
|
||||
type SDNI = SerializedDepNodeIndex;
|
||||
|
||||
let fingerprints: IndexVec<SerializedDepNodeIndex, _> =
|
||||
data.iter().map(|d| d.fingerprint).collect();
|
||||
let nodes: IndexVec<SerializedDepNodeIndex, _> = data.iter().map(|d| d.node).collect();
|
||||
let data = self.data.as_ref().unwrap();
|
||||
let previous = &data.previous;
|
||||
let data = data.current.data.lock();
|
||||
|
||||
let total_edge_count: usize = data.iter().map(|d| d.edges.len()).sum();
|
||||
let node_count = data.hybrid_indices.len();
|
||||
|
||||
let mut edge_list_indices = IndexVec::with_capacity(nodes.len());
|
||||
let mut edge_list_data = Vec::with_capacity(total_edge_count);
|
||||
let edge_count = data.new.edges.iter().map(|e| e.len()).sum::<usize>()
|
||||
+ data.red.edges.iter().map(|e| e.len()).sum::<usize>()
|
||||
+ data.green.edges.iter().map(|e| e.len()).sum::<usize>();
|
||||
|
||||
for (current_dep_node_index, edges) in data.iter_enumerated().map(|(i, d)| (i, &d.edges)) {
|
||||
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);
|
||||
|
||||
fn add_edges<'a, I: Iterator<Item = &'a DepNodeIndex>>(
|
||||
edge_list_indices: &mut IndexVec<SerializedDepNodeIndex, (u32, u32)>,
|
||||
edge_list_data: &mut Vec<SerializedDepNodeIndex>,
|
||||
iter: I,
|
||||
) {
|
||||
let start = edge_list_data.len() as u32;
|
||||
// This should really just be a memcpy :/
|
||||
edge_list_data.extend(edges.iter().map(|i| SerializedDepNodeIndex::new(i.index())));
|
||||
edge_list_data.extend(iter.map(|i| SDNI::new(i.index())));
|
||||
let end = edge_list_data.len() as u32;
|
||||
|
||||
debug_assert_eq!(current_dep_node_index.index(), edge_list_indices.len());
|
||||
edge_list_indices.push((start, end));
|
||||
};
|
||||
|
||||
for &hybrid_index in data.hybrid_indices.iter() {
|
||||
match hybrid_index.into() {
|
||||
HybridIndex::New(i) => {
|
||||
let new = &data.new;
|
||||
nodes.push(new.nodes[i]);
|
||||
fingerprints.push(new.fingerprints[i]);
|
||||
add_edges(&mut edge_list_indices, &mut edge_list_data, new.edges[i].iter());
|
||||
}
|
||||
HybridIndex::Red(i) => {
|
||||
let red = &data.red;
|
||||
nodes.push(previous.index_to_node(red.node_indices[i]));
|
||||
fingerprints.push(red.fingerprints[i]);
|
||||
add_edges(&mut edge_list_indices, &mut edge_list_data, red.edges[i].iter());
|
||||
}
|
||||
HybridIndex::Green(i) => {
|
||||
let green = &data.green;
|
||||
nodes.push(previous.index_to_node(green.node_indices[i]));
|
||||
fingerprints.push(previous.fingerprint_by_index(green.node_indices[i]));
|
||||
add_edges(&mut edge_list_indices, &mut edge_list_data, green.edges[i].iter());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
debug_assert_eq!(nodes.len(), node_count);
|
||||
debug_assert_eq!(fingerprints.len(), node_count);
|
||||
debug_assert_eq!(edge_list_indices.len(), node_count);
|
||||
debug_assert_eq!(edge_list_data.len(), edge_count);
|
||||
debug_assert!(edge_list_data.len() <= u32::MAX as usize);
|
||||
debug_assert_eq!(edge_list_data.len(), total_edge_count);
|
||||
|
||||
SerializedDepGraph { nodes, fingerprints, edge_list_indices, edge_list_data }
|
||||
}
|
||||
@ -540,14 +677,7 @@ impl<K: DepKind> DepGraph<K> {
|
||||
|
||||
#[cfg(not(parallel_compiler))]
|
||||
{
|
||||
debug_assert!(
|
||||
!data
|
||||
.current
|
||||
.node_to_node_index
|
||||
.get_shard_by_value(dep_node)
|
||||
.lock()
|
||||
.contains_key(dep_node)
|
||||
);
|
||||
debug_assert!(!self.dep_node_exists(dep_node));
|
||||
debug_assert!(data.colors.get(prev_dep_node_index).is_none());
|
||||
}
|
||||
|
||||
@ -690,13 +820,9 @@ impl<K: DepKind> DepGraph<K> {
|
||||
// There may be multiple threads trying to mark the same dep node green concurrently
|
||||
|
||||
let dep_node_index = {
|
||||
// Copy the fingerprint from the previous graph,
|
||||
// so we don't have to recompute it
|
||||
let fingerprint = data.previous.fingerprint_by_index(prev_dep_node_index);
|
||||
|
||||
// We allocating an entry for the node in the current dependency graph and
|
||||
// adding all the appropriate edges imported from the previous graph
|
||||
data.current.intern_node(*dep_node, current_deps, fingerprint)
|
||||
data.current.intern_green_node(&data.previous, prev_dep_node_index, current_deps)
|
||||
};
|
||||
|
||||
// ... emitting any stored diagnostic ...
|
||||
@ -871,31 +997,181 @@ pub struct WorkProduct {
|
||||
pub saved_file: Option<String>,
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
struct DepNodeData<K> {
|
||||
node: DepNode<K>,
|
||||
edges: EdgesVec,
|
||||
fingerprint: Fingerprint,
|
||||
// The maximum value of the follow index types leaves the upper two bits unused
|
||||
// so that we can store multiple index types in `CompressedHybridIndex`, and use
|
||||
// those bits to encode which index type it contains.
|
||||
|
||||
// Index type for `NewDepNodeData`.
|
||||
rustc_index::newtype_index! {
|
||||
struct NewDepNodeIndex {
|
||||
MAX = 0x7FFF_FFFF
|
||||
}
|
||||
}
|
||||
|
||||
/// `CurrentDepGraph` stores the dependency graph for the current session.
|
||||
/// It will be populated as we run queries or tasks.
|
||||
// Index type for `RedDepNodeData`.
|
||||
rustc_index::newtype_index! {
|
||||
struct RedDepNodeIndex {
|
||||
MAX = 0x7FFF_FFFF
|
||||
}
|
||||
}
|
||||
|
||||
// Index type for `GreenDepNodeData`.
|
||||
rustc_index::newtype_index! {
|
||||
struct GreenDepNodeIndex {
|
||||
MAX = 0x7FFF_FFFF
|
||||
}
|
||||
}
|
||||
|
||||
/// Compressed representation of `HybridIndex` enum. Bits unused by the
|
||||
/// contained index types are used to encode which index type it contains.
|
||||
#[derive(Copy, Clone)]
|
||||
struct CompressedHybridIndex(u32);
|
||||
|
||||
impl CompressedHybridIndex {
|
||||
const NEW_TAG: u32 = 0b0000_0000_0000_0000_0000_0000_0000_0000;
|
||||
const RED_TAG: u32 = 0b0100_0000_0000_0000_0000_0000_0000_0000;
|
||||
const GREEN_TAG: u32 = 0b1000_0000_0000_0000_0000_0000_0000_0000;
|
||||
|
||||
const TAG_MASK: u32 = 0b1100_0000_0000_0000_0000_0000_0000_0000;
|
||||
const INDEX_MASK: u32 = !Self::TAG_MASK;
|
||||
}
|
||||
|
||||
impl From<NewDepNodeIndex> for CompressedHybridIndex {
|
||||
#[inline]
|
||||
fn from(index: NewDepNodeIndex) -> Self {
|
||||
CompressedHybridIndex(Self::NEW_TAG | index.as_u32())
|
||||
}
|
||||
}
|
||||
|
||||
impl From<RedDepNodeIndex> for CompressedHybridIndex {
|
||||
#[inline]
|
||||
fn from(index: RedDepNodeIndex) -> Self {
|
||||
CompressedHybridIndex(Self::RED_TAG | index.as_u32())
|
||||
}
|
||||
}
|
||||
|
||||
impl From<GreenDepNodeIndex> for CompressedHybridIndex {
|
||||
#[inline]
|
||||
fn from(index: GreenDepNodeIndex) -> Self {
|
||||
CompressedHybridIndex(Self::GREEN_TAG | index.as_u32())
|
||||
}
|
||||
}
|
||||
|
||||
/// Contains an index into one of several node data collections. Elsewhere, we
|
||||
/// store `CompressedHyridIndex` instead of this to save space, but convert to
|
||||
/// this type during processing to take advantage of the enum match ergonomics.
|
||||
enum HybridIndex {
|
||||
New(NewDepNodeIndex),
|
||||
Red(RedDepNodeIndex),
|
||||
Green(GreenDepNodeIndex),
|
||||
}
|
||||
|
||||
impl From<CompressedHybridIndex> for HybridIndex {
|
||||
#[inline]
|
||||
fn from(hybrid_index: CompressedHybridIndex) -> Self {
|
||||
let index = hybrid_index.0 & CompressedHybridIndex::INDEX_MASK;
|
||||
|
||||
match hybrid_index.0 & CompressedHybridIndex::TAG_MASK {
|
||||
CompressedHybridIndex::NEW_TAG => HybridIndex::New(NewDepNodeIndex::from_u32(index)),
|
||||
CompressedHybridIndex::RED_TAG => HybridIndex::Red(RedDepNodeIndex::from_u32(index)),
|
||||
CompressedHybridIndex::GREEN_TAG => {
|
||||
HybridIndex::Green(GreenDepNodeIndex::from_u32(index))
|
||||
}
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Data for nodes in the current graph, divided into different collections
|
||||
/// based on their presence in the previous graph, and if present, their color.
|
||||
/// We divide nodes this way because different types of nodes are able to share
|
||||
/// more or less data with the previous graph.
|
||||
///
|
||||
/// The nodes in it are identified by an index (`DepNodeIndex`).
|
||||
/// The data for each node is stored in its `DepNodeData`, found in the `data` field.
|
||||
/// Node data is stored in parallel vectors to eliminate the padding between
|
||||
/// elements that would be needed to satisfy alignment requirements of the
|
||||
/// structure that would contain all of a node's data. We could group tightly
|
||||
/// packing subsets of node data together and use fewer vectors, but for
|
||||
/// consistency's sake, we use separate vectors for each piece of data.
|
||||
struct DepNodeData<K> {
|
||||
/// Data for nodes not in previous graph.
|
||||
new: NewDepNodeData<K>,
|
||||
|
||||
/// Data for nodes in previous graph that have been marked red.
|
||||
red: RedDepNodeData,
|
||||
|
||||
/// Data for nodes in previous graph that have been marked green.
|
||||
green: GreenDepNodeData,
|
||||
|
||||
/// Mapping from `DepNodeIndex` to an index into a collection above.
|
||||
/// Indicates which of the above collections contains a node's data.
|
||||
///
|
||||
/// We never remove nodes from the graph: they are only added.
|
||||
/// This collection is wasteful in time and space during incr-full builds,
|
||||
/// because for those, all nodes are new. However, the waste is relatively
|
||||
/// small, and the maintenance cost of avoiding using this for incr-full
|
||||
/// builds is somewhat high and prone to bugginess. It does not seem worth
|
||||
/// it at the time of this writing, but we may want to revisit the idea.
|
||||
hybrid_indices: IndexVec<DepNodeIndex, CompressedHybridIndex>,
|
||||
}
|
||||
|
||||
/// Data for nodes not in previous graph. Since we cannot share any data with
|
||||
/// the previous graph, so we must store all of such a node's data here.
|
||||
struct NewDepNodeData<K> {
|
||||
nodes: IndexVec<NewDepNodeIndex, DepNode<K>>,
|
||||
edges: IndexVec<NewDepNodeIndex, EdgesVec>,
|
||||
fingerprints: IndexVec<NewDepNodeIndex, Fingerprint>,
|
||||
}
|
||||
|
||||
/// Data for nodes in previous graph that have been marked red. We can share the
|
||||
/// dep node with the previous graph, but the edges may be different, and the
|
||||
/// fingerprint is known to be different, so we store the latter two directly.
|
||||
struct RedDepNodeData {
|
||||
node_indices: IndexVec<RedDepNodeIndex, SerializedDepNodeIndex>,
|
||||
edges: IndexVec<RedDepNodeIndex, EdgesVec>,
|
||||
fingerprints: IndexVec<RedDepNodeIndex, Fingerprint>,
|
||||
}
|
||||
|
||||
/// Data for nodes in previous graph that have been marked green. We can share
|
||||
/// both the dep node and the fingerprint with previous graph, but the edges may
|
||||
/// be different, so we store the latter directly.
|
||||
struct GreenDepNodeData {
|
||||
node_indices: IndexVec<GreenDepNodeIndex, SerializedDepNodeIndex>,
|
||||
edges: IndexVec<GreenDepNodeIndex, EdgesVec>,
|
||||
}
|
||||
|
||||
/// `CurrentDepGraph` stores the dependency graph for the current session. It
|
||||
/// will be populated as we run queries or tasks. We never remove nodes from the
|
||||
/// graph: they are only added.
|
||||
///
|
||||
/// This struct uses two locks internally. The `data` and `node_to_node_index` fields are
|
||||
/// locked separately. Operations that take a `DepNodeIndex` typically just access
|
||||
/// the data field.
|
||||
/// The nodes in it are identified by a `DepNodeIndex`. Internally, this maps to
|
||||
/// a `HybridIndex`, which identifies which collection in the `data` field
|
||||
/// contains a node's data. Which collection is used for a node depends on
|
||||
/// whether the node was present in the `PreviousDepGraph`, and if so, the color
|
||||
/// of the node. Each type of node can share more or less data with the previous
|
||||
/// graph. When possible, we can store just the index of the node in the
|
||||
/// previous graph, rather than duplicating its data in our own collections.
|
||||
/// This is important, because these graph structures are some of the largest in
|
||||
/// the compiler.
|
||||
///
|
||||
/// The only operation that must manipulate both locks is adding new nodes, in which case
|
||||
/// we first acquire the `node_to_node_index` lock and then, once a new node is to be inserted,
|
||||
/// acquire the lock on `data.`
|
||||
/// For the same reason, we also avoid storing `DepNode`s more than once as map
|
||||
/// keys. The `new_node_to_index` map only contains nodes not in the previous
|
||||
/// graph, and we map nodes in the previous graph to indices via a two-step
|
||||
/// mapping. `PreviousDepGraph` maps from `DepNode` to `SerializedDepNodeIndex`,
|
||||
/// and the `prev_index_to_index` vector (which is more compact and faster than
|
||||
/// using a map) maps from `SerializedDepNodeIndex` to `DepNodeIndex`.
|
||||
///
|
||||
/// This struct uses three locks internally. The `data`, `new_node_to_index`,
|
||||
/// and `prev_index_to_index` fields are locked separately. Operations that take
|
||||
/// a `DepNodeIndex` typically just access the `data` field.
|
||||
///
|
||||
/// We only need to manipulate at most two locks simultaneously:
|
||||
/// `new_node_to_index` and `data`, or `prev_index_to_index` and `data`. The
|
||||
/// only operation that must manipulate both locks is adding new nodes, in which
|
||||
/// case we first acquire the `new_node_to_index` or `prev_index_to_index` lock
|
||||
/// and then, once a new node is to be inserted, acquire the lock on `data`.
|
||||
pub(super) struct CurrentDepGraph<K> {
|
||||
data: Lock<IndexVec<DepNodeIndex, DepNodeData<K>>>,
|
||||
node_to_node_index: Sharded<FxHashMap<DepNode<K>, DepNodeIndex>>,
|
||||
data: Lock<DepNodeData<K>>,
|
||||
new_node_to_index: Sharded<FxHashMap<DepNode<K>, DepNodeIndex>>,
|
||||
prev_index_to_index: Lock<IndexVec<SerializedDepNodeIndex, Option<DepNodeIndex>>>,
|
||||
|
||||
/// Used to trap when a specific edge is added to the graph.
|
||||
/// This is used for debug purposes and is only active with `debug_assertions`.
|
||||
@ -944,18 +1220,46 @@ impl<K: DepKind> CurrentDepGraph<K> {
|
||||
|
||||
// Pre-allocate the dep node structures. We over-allocate a little so
|
||||
// that we hopefully don't have to re-allocate during this compilation
|
||||
// session. The over-allocation is 2% plus a small constant to account
|
||||
// for the fact that in very small crates 2% might not be enough.
|
||||
let new_node_count_estimate = (prev_graph_node_count * 102) / 100 + 200;
|
||||
// session. The over-allocation for new nodes is 2% plus a small
|
||||
// constant to account for the fact that in very small crates 2% might
|
||||
// not be enough. The allocation for red and green node data doesn't
|
||||
// include a constant, as we don't want to allocate anything for these
|
||||
// structures during full incremental builds, where they aren't used.
|
||||
let new_node_count_estimate = (prev_graph_node_count * 2) / 100 + 200;
|
||||
let red_node_count_estimate = (prev_graph_node_count * 3) / 100;
|
||||
let green_node_count_estimate = (prev_graph_node_count * 95) / 100;
|
||||
let total_node_count_estimate = prev_graph_node_count + new_node_count_estimate;
|
||||
|
||||
// We store a large collection of these in `prev_index_to_index` during
|
||||
// non-full incremental builds, and want to ensure that the element size
|
||||
// doesn't inadvertently increase.
|
||||
static_assert_size!(Option<DepNodeIndex>, 4);
|
||||
|
||||
CurrentDepGraph {
|
||||
data: Lock::new(IndexVec::with_capacity(new_node_count_estimate)),
|
||||
node_to_node_index: Sharded::new(|| {
|
||||
data: Lock::new(DepNodeData {
|
||||
new: NewDepNodeData {
|
||||
nodes: IndexVec::with_capacity(new_node_count_estimate),
|
||||
edges: IndexVec::with_capacity(new_node_count_estimate),
|
||||
fingerprints: IndexVec::with_capacity(new_node_count_estimate),
|
||||
},
|
||||
red: RedDepNodeData {
|
||||
node_indices: IndexVec::with_capacity(red_node_count_estimate),
|
||||
edges: IndexVec::with_capacity(red_node_count_estimate),
|
||||
fingerprints: IndexVec::with_capacity(red_node_count_estimate),
|
||||
},
|
||||
green: GreenDepNodeData {
|
||||
node_indices: IndexVec::with_capacity(green_node_count_estimate),
|
||||
edges: IndexVec::with_capacity(green_node_count_estimate),
|
||||
},
|
||||
hybrid_indices: IndexVec::with_capacity(total_node_count_estimate),
|
||||
}),
|
||||
new_node_to_index: Sharded::new(|| {
|
||||
FxHashMap::with_capacity_and_hasher(
|
||||
new_node_count_estimate / sharded::SHARDS,
|
||||
Default::default(),
|
||||
)
|
||||
}),
|
||||
prev_index_to_index: Lock::new(IndexVec::from_elem_n(None, prev_graph_node_count)),
|
||||
anon_id_seed: stable_hasher.finish(),
|
||||
forbidden_edge,
|
||||
total_read_count: AtomicU64::new(0),
|
||||
@ -963,113 +1267,92 @@ impl<K: DepKind> CurrentDepGraph<K> {
|
||||
}
|
||||
}
|
||||
|
||||
fn complete_task(
|
||||
&self,
|
||||
node: DepNode<K>,
|
||||
task_deps: TaskDeps<K>,
|
||||
fingerprint: Fingerprint,
|
||||
) -> DepNodeIndex {
|
||||
self.alloc_node(node, task_deps.reads, fingerprint)
|
||||
}
|
||||
|
||||
fn complete_anon_task(&self, kind: K, task_deps: TaskDeps<K>) -> DepNodeIndex {
|
||||
debug_assert!(!kind.is_eval_always());
|
||||
|
||||
let mut hasher = StableHasher::new();
|
||||
|
||||
// The dep node indices are hashed here instead of hashing the dep nodes of the
|
||||
// dependencies. These indices may refer to different nodes per session, but this isn't
|
||||
// a problem here because we that ensure the final dep node hash is per session only by
|
||||
// combining it with the per session random number `anon_id_seed`. This hash only need
|
||||
// to map the dependencies to a single value on a per session basis.
|
||||
task_deps.reads.hash(&mut hasher);
|
||||
|
||||
let target_dep_node = DepNode {
|
||||
kind,
|
||||
|
||||
// Fingerprint::combine() is faster than sending Fingerprint
|
||||
// through the StableHasher (at least as long as StableHasher
|
||||
// is so slow).
|
||||
hash: self.anon_id_seed.combine(hasher.finish()).into(),
|
||||
};
|
||||
|
||||
self.intern_node(target_dep_node, task_deps.reads, Fingerprint::ZERO)
|
||||
}
|
||||
|
||||
fn alloc_node(
|
||||
fn intern_node(
|
||||
&self,
|
||||
prev_graph: &PreviousDepGraph<K>,
|
||||
dep_node: DepNode<K>,
|
||||
edges: EdgesVec,
|
||||
fingerprint: Fingerprint,
|
||||
) -> DepNodeIndex {
|
||||
debug_assert!(
|
||||
!self.node_to_node_index.get_shard_by_value(&dep_node).lock().contains_key(&dep_node)
|
||||
prev_graph.node_to_index_opt(&dep_node).is_none(),
|
||||
"node in previous graph should be interned using \
|
||||
`intern_red_node` or `intern_green_node`"
|
||||
);
|
||||
self.intern_node(dep_node, edges, fingerprint)
|
||||
}
|
||||
|
||||
fn intern_node(
|
||||
&self,
|
||||
dep_node: DepNode<K>,
|
||||
edges: EdgesVec,
|
||||
fingerprint: Fingerprint,
|
||||
) -> DepNodeIndex {
|
||||
match self.node_to_node_index.get_shard_by_value(&dep_node).lock().entry(dep_node) {
|
||||
match self.new_node_to_index.get_shard_by_value(&dep_node).lock().entry(dep_node) {
|
||||
Entry::Occupied(entry) => *entry.get(),
|
||||
Entry::Vacant(entry) => {
|
||||
let mut data = self.data.lock();
|
||||
let dep_node_index = DepNodeIndex::new(data.len());
|
||||
data.push(DepNodeData { node: dep_node, edges, fingerprint });
|
||||
let new_index = data.new.nodes.push(dep_node);
|
||||
data.new.edges.push(edges);
|
||||
data.new.fingerprints.push(fingerprint);
|
||||
let dep_node_index = data.hybrid_indices.push(new_index.into());
|
||||
entry.insert(dep_node_index);
|
||||
dep_node_index
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn intern_red_node(
|
||||
&self,
|
||||
prev_graph: &PreviousDepGraph<K>,
|
||||
prev_index: SerializedDepNodeIndex,
|
||||
edges: EdgesVec,
|
||||
fingerprint: Fingerprint,
|
||||
) -> DepNodeIndex {
|
||||
self.debug_assert_not_in_new_nodes(prev_graph, prev_index);
|
||||
|
||||
let mut prev_index_to_index = self.prev_index_to_index.lock();
|
||||
|
||||
match prev_index_to_index[prev_index] {
|
||||
Some(dep_node_index) => dep_node_index,
|
||||
None => {
|
||||
let mut data = self.data.lock();
|
||||
let red_index = data.red.node_indices.push(prev_index);
|
||||
data.red.edges.push(edges);
|
||||
data.red.fingerprints.push(fingerprint);
|
||||
let dep_node_index = data.hybrid_indices.push(red_index.into());
|
||||
prev_index_to_index[prev_index] = Some(dep_node_index);
|
||||
dep_node_index
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<K: DepKind> DepGraphData<K> {
|
||||
#[inline(never)]
|
||||
fn read_index(&self, source: DepNodeIndex) {
|
||||
K::read_deps(|task_deps| {
|
||||
if let Some(task_deps) = task_deps {
|
||||
let mut task_deps = task_deps.lock();
|
||||
let task_deps = &mut *task_deps;
|
||||
if cfg!(debug_assertions) {
|
||||
self.current.total_read_count.fetch_add(1, Relaxed);
|
||||
fn intern_green_node(
|
||||
&self,
|
||||
prev_graph: &PreviousDepGraph<K>,
|
||||
prev_index: SerializedDepNodeIndex,
|
||||
edges: EdgesVec,
|
||||
) -> DepNodeIndex {
|
||||
self.debug_assert_not_in_new_nodes(prev_graph, prev_index);
|
||||
|
||||
let mut prev_index_to_index = self.prev_index_to_index.lock();
|
||||
|
||||
match prev_index_to_index[prev_index] {
|
||||
Some(dep_node_index) => dep_node_index,
|
||||
None => {
|
||||
let mut data = self.data.lock();
|
||||
let green_index = data.green.node_indices.push(prev_index);
|
||||
data.green.edges.push(edges);
|
||||
let dep_node_index = data.hybrid_indices.push(green_index.into());
|
||||
prev_index_to_index[prev_index] = Some(dep_node_index);
|
||||
dep_node_index
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// As long as we only have a low number of reads we can avoid doing a hash
|
||||
// insert and potentially allocating/reallocating the hashmap
|
||||
let new_read = if task_deps.reads.len() < TASK_DEPS_READS_CAP {
|
||||
task_deps.reads.iter().all(|other| *other != source)
|
||||
} else {
|
||||
task_deps.read_set.insert(source)
|
||||
};
|
||||
if new_read {
|
||||
task_deps.reads.push(source);
|
||||
if task_deps.reads.len() == TASK_DEPS_READS_CAP {
|
||||
// Fill `read_set` with what we have so far so we can use the hashset next
|
||||
// time
|
||||
task_deps.read_set.extend(task_deps.reads.iter().copied());
|
||||
}
|
||||
|
||||
#[cfg(debug_assertions)]
|
||||
{
|
||||
if let Some(target) = task_deps.node {
|
||||
let data = self.current.data.lock();
|
||||
if let Some(ref forbidden_edge) = self.current.forbidden_edge {
|
||||
let source = data[source].node;
|
||||
if forbidden_edge.test(&source, &target) {
|
||||
panic!("forbidden edge {:?} -> {:?} created", source, target)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if cfg!(debug_assertions) {
|
||||
self.current.total_duplicate_read_count.fetch_add(1, Relaxed);
|
||||
}
|
||||
}
|
||||
})
|
||||
#[inline]
|
||||
fn debug_assert_not_in_new_nodes(
|
||||
&self,
|
||||
prev_graph: &PreviousDepGraph<K>,
|
||||
prev_index: SerializedDepNodeIndex,
|
||||
) {
|
||||
let node = &prev_graph.index_to_node(prev_index);
|
||||
debug_assert!(
|
||||
!self.new_node_to_index.get_shard_by_value(node).lock().contains_key(node),
|
||||
"node from previous graph present in new node collection"
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -9,16 +9,16 @@ pub struct DepGraphQuery<K> {
|
||||
}
|
||||
|
||||
impl<K: DepKind> DepGraphQuery<K> {
|
||||
pub fn new(nodes: &[DepNode<K>], edges: &[(DepNode<K>, DepNode<K>)]) -> DepGraphQuery<K> {
|
||||
pub fn new(nodes: &[DepNode<K>], edges: &[(usize, usize)]) -> DepGraphQuery<K> {
|
||||
let mut graph = Graph::with_capacity(nodes.len(), edges.len());
|
||||
let mut indices = FxHashMap::default();
|
||||
for node in nodes {
|
||||
indices.insert(*node, graph.add_node(*node));
|
||||
}
|
||||
|
||||
for &(ref source, ref target) in edges {
|
||||
let source = indices[source];
|
||||
let target = indices[target];
|
||||
for &(source, target) in edges {
|
||||
let source = indices[&nodes[source]];
|
||||
let target = indices[&nodes[target]];
|
||||
graph.add_edge(source, target, ());
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user