rewrite the predecessors code to create a reduced graph

The old code created a flat listing of "HIR -> WorkProduct" edges.
While perfectly general, this could lead to a lot of repetition if the
same HIR nodes affect many work-products. This is set to be a problem
when we start to skip typeck, since we will be adding a lot more
"work-product"-like nodes.

The newer code uses an alternative strategy: it "reduces" the graph
instead. Basically we walk the dep-graph and convert it to a DAG, where
we only keep intermediate nodes if they are used by multiple
work-products.

This DAG does not contain the same set of nodes as the original graph,
but it is guaranteed that (a) every output node is included in the graph
and (b) the set of input nodes that can reach each output node is
unchanged.

(Input nodes are basically HIR nodes and foreign metadata; output nodes
are nodes that have assocaited state which we will persist to disk in
some way. These are assumed to be disjoint sets.)
This commit is contained in:
Niko Matsakis 2017-01-31 04:35:17 -05:00
parent 55f9712d14
commit d848f1d782
14 changed files with 1093 additions and 465 deletions

View File

@ -23,6 +23,9 @@
#![feature(staged_api)]
#![feature(rand)]
#![feature(core_intrinsics)]
#![feature(conservative_impl_trait)]
#![feature(field_init_shorthand)]
#![feature(pub_restricted)]
extern crate graphviz;
#[macro_use] extern crate rustc;

View File

@ -67,9 +67,9 @@ pub fn check_dirty_clean_annotations<'a, 'tcx>(tcx: TyCtxt<'a, 'tcx, 'tcx>,
let _ignore = tcx.dep_graph.in_ignore();
let dirty_inputs: FxHashSet<DepNode<DefId>> =
dirty_inputs.iter()
.filter_map(|d| retraced.map(d))
.collect();
dirty_inputs.keys()
.filter_map(|d| retraced.map(d))
.collect();
let query = tcx.dep_graph.query();
debug!("query-nodes: {:?}", query.nodes());
let krate = tcx.hir.krate();

View File

@ -10,7 +10,7 @@
//! Code to save/load the dep-graph from files.
use rustc::dep_graph::DepNode;
use rustc::dep_graph::{DepNode, WorkProductId};
use rustc::hir::def_id::DefId;
use rustc::hir::svh::Svh;
use rustc::session::Session;
@ -19,6 +19,7 @@ use rustc_data_structures::fx::{FxHashSet, FxHashMap};
use rustc_serialize::Decodable as RustcDecodable;
use rustc_serialize::opaque::Decoder;
use std::path::{Path};
use std::sync::Arc;
use IncrementalHashesMap;
use ich::Fingerprint;
@ -30,7 +31,9 @@ use super::fs::*;
use super::file_format;
use super::work_product;
pub type DirtyNodes = FxHashSet<DepNode<DefPathIndex>>;
// The key is a dirty node. The value is **some** base-input that we
// can blame it on.
pub type DirtyNodes = FxHashMap<DepNode<DefPathIndex>, DepNode<DefPathIndex>>;
/// If we are in incremental mode, and a previous dep-graph exists,
/// then load up those nodes/edges that are still valid into the
@ -152,83 +155,65 @@ pub fn decode_dep_graph<'a, 'tcx>(tcx: TyCtxt<'a, 'tcx, 'tcx>,
// Retrace the paths in the directory to find their current location (if any).
let retraced = directory.retrace(tcx);
// Compute the set of Hir nodes whose data has changed or which
// have been removed. These are "raw" source nodes, which means
// that they still use the original `DefPathIndex` values from the
// encoding, rather than having been retraced to a `DefId`. The
// reason for this is that this way we can include nodes that have
// been removed (which no longer have a `DefId` in the current
// compilation).
let dirty_raw_source_nodes = dirty_nodes(tcx,
incremental_hashes_map,
&serialized_dep_graph.hashes,
&retraced);
// Create a list of (raw-source-node ->
// retracted-target-node) edges. In the process of retracing the
// target nodes, we may discover some of them def-paths no longer exist,
// in which case there is no need to mark the corresopnding nodes as dirty
// (they are just not present). So this list may be smaller than the original.
//
// Note though that in the common case the target nodes are
// `DepNode::WorkProduct` instances, and those don't have a
// def-id, so they will never be considered to not exist. Instead,
// we do a secondary hashing step (later, in trans) when we know
// the set of symbols that go into a work-product: if any symbols
// have been removed (or added) the hash will be different and
// we'll ignore the work-product then.
let retraced_edges: Vec<_> =
serialized_dep_graph.edges.iter()
.filter_map(|&(ref raw_source_node, ref raw_target_node)| {
retraced.map(raw_target_node)
.map(|target_node| (raw_source_node, target_node))
})
.collect();
// Compute which work-products have an input that has changed or
// been removed. Put the dirty ones into a set.
let mut dirty_target_nodes = FxHashSet();
for &(raw_source_node, ref target_node) in &retraced_edges {
if dirty_raw_source_nodes.contains(raw_source_node) {
if !dirty_target_nodes.contains(target_node) {
dirty_target_nodes.insert(target_node.clone());
// Compute the set of nodes from the old graph where some input
// has changed or been removed. These are "raw" source nodes,
// which means that they still use the original `DefPathIndex`
// values from the encoding, rather than having been retraced to a
// `DefId`. The reason for this is that this way we can include
// nodes that have been removed (which no longer have a `DefId` in
// the current compilation).
let dirty_raw_nodes = initial_dirty_nodes(tcx,
incremental_hashes_map,
&serialized_dep_graph.hashes,
&retraced);
let dirty_raw_nodes = transitive_dirty_nodes(&serialized_dep_graph.edges, dirty_raw_nodes);
// Recreate the edges in the graph that are still clean.
let mut clean_work_products = FxHashSet();
let mut dirty_work_products = FxHashSet(); // incomplete; just used to suppress debug output
for edge in &serialized_dep_graph.edges {
// If the target is dirty, skip the edge. If this is an edge
// that targets a work-product, we can print the blame
// information now.
if let Some(blame) = dirty_raw_nodes.get(&edge.1) {
if let DepNode::WorkProduct(ref wp) = edge.1 {
if tcx.sess.opts.debugging_opts.incremental_info {
// It'd be nice to pretty-print these paths better than just
// using the `Debug` impls, but wev.
println!("incremental: module {:?} is dirty because {:?} \
changed or was removed",
target_node,
raw_source_node.map_def(|&index| {
Some(directory.def_path_string(tcx, index))
}).unwrap());
if dirty_work_products.insert(wp.clone()) {
// It'd be nice to pretty-print these paths better than just
// using the `Debug` impls, but wev.
println!("incremental: module {:?} is dirty because {:?} \
changed or was removed",
wp,
blame.map_def(|&index| {
Some(directory.def_path_string(tcx, index))
}).unwrap());
}
}
}
}
}
// For work-products that are still clean, add their deps into the
// graph. This is needed because later we will have to save this
// back out again!
let dep_graph = tcx.dep_graph.clone();
for (raw_source_node, target_node) in retraced_edges {
if dirty_target_nodes.contains(&target_node) {
continue;
}
let source_node = retraced.map(raw_source_node).unwrap();
// If the source is dirty, the target will be dirty.
assert!(!dirty_raw_nodes.contains_key(&edge.0));
debug!("decode_dep_graph: clean edge: {:?} -> {:?}", source_node, target_node);
let _task = dep_graph.in_task(target_node);
dep_graph.read(source_node);
// Retrace the source -> target edges to def-ids and then
// create an edge in the graph. Retracing may yield none if
// some of the data happens to have been removed; this ought
// to be impossible unless it is dirty, so we can unwrap.
let source_node = retraced.map(&edge.0).unwrap();
let target_node = retraced.map(&edge.1).unwrap();
let _task = tcx.dep_graph.in_task(target_node);
tcx.dep_graph.read(source_node);
if let DepNode::WorkProduct(ref wp) = edge.1 {
clean_work_products.insert(wp.clone());
}
}
// Add in work-products that are still clean, and delete those that are
// dirty.
reconcile_work_products(tcx, work_products, &dirty_target_nodes);
reconcile_work_products(tcx, work_products, &clean_work_products);
dirty_clean::check_dirty_clean_annotations(tcx, &dirty_raw_source_nodes, &retraced);
dirty_clean::check_dirty_clean_annotations(tcx, &dirty_raw_nodes, &retraced);
load_prev_metadata_hashes(tcx,
&retraced,
@ -238,13 +223,13 @@ pub fn decode_dep_graph<'a, 'tcx>(tcx: TyCtxt<'a, 'tcx, 'tcx>,
/// Computes which of the original set of def-ids are dirty. Stored in
/// a bit vector where the index is the DefPathIndex.
fn dirty_nodes<'a, 'tcx>(tcx: TyCtxt<'a, 'tcx, 'tcx>,
incremental_hashes_map: &IncrementalHashesMap,
serialized_hashes: &[SerializedHash],
retraced: &RetracedDefIdDirectory)
-> DirtyNodes {
fn initial_dirty_nodes<'a, 'tcx>(tcx: TyCtxt<'a, 'tcx, 'tcx>,
incremental_hashes_map: &IncrementalHashesMap,
serialized_hashes: &[SerializedHash],
retraced: &RetracedDefIdDirectory)
-> DirtyNodes {
let mut hcx = HashContext::new(tcx, incremental_hashes_map);
let mut dirty_nodes = FxHashSet();
let mut dirty_nodes = FxHashMap();
for hash in serialized_hashes {
if let Some(dep_node) = retraced.map(&hash.dep_node) {
@ -277,21 +262,37 @@ fn dirty_nodes<'a, 'tcx>(tcx: TyCtxt<'a, 'tcx, 'tcx>,
hash.dep_node);
}
dirty_nodes.insert(hash.dep_node.clone());
dirty_nodes.insert(hash.dep_node.clone(), hash.dep_node.clone());
}
dirty_nodes
}
fn transitive_dirty_nodes(edges: &[SerializedEdge],
mut dirty_nodes: DirtyNodes)
-> DirtyNodes
{
let mut len = 0;
while len != dirty_nodes.len() {
len = dirty_nodes.len();
for edge in edges {
if let Some(n) = dirty_nodes.get(&edge.0).cloned() {
dirty_nodes.insert(edge.1.clone(), n);
}
}
}
dirty_nodes
}
/// Go through the list of work-products produced in the previous run.
/// Delete any whose nodes have been found to be dirty or which are
/// otherwise no longer applicable.
fn reconcile_work_products<'a, 'tcx>(tcx: TyCtxt<'a, 'tcx, 'tcx>,
work_products: Vec<SerializedWorkProduct>,
dirty_target_nodes: &FxHashSet<DepNode<DefId>>) {
clean_work_products: &FxHashSet<Arc<WorkProductId>>) {
debug!("reconcile_work_products({:?})", work_products);
for swp in work_products {
if dirty_target_nodes.contains(&DepNode::WorkProduct(swp.id.clone())) {
if !clean_work_products.contains(&swp.id) {
debug!("reconcile_work_products: dep-node for {:?} is dirty", swp);
delete_dirty_work_product(tcx, swp);
} else {

View File

@ -1,366 +0,0 @@
// Copyright 2012-2015 The Rust Project Developers. See the COPYRIGHT
// file at the top-level directory of this distribution and at
// http://rust-lang.org/COPYRIGHT.
//
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
// option. This file may not be copied, modified, or distributed
// except according to those terms.
use rustc::dep_graph::{DepGraphQuery, DepNode};
use rustc::hir::def_id::DefId;
use rustc_data_structures::fx::FxHashMap;
use rustc_data_structures::bitvec::BitVector;
use rustc_data_structures::graph::{NodeIndex, Graph};
use super::hash::*;
use ich::Fingerprint;
/// A data-structure that makes it easy to enumerate the hashable
/// predecessors of any given dep-node.
pub struct Predecessors<'query> {
// - Keys: dep-nodes that may have work-products, output meta-data
// nodes.
// - Values: transitive predecessors of the key that are hashable
// (e.g., HIR nodes, input meta-data nodes)
pub inputs: FxHashMap<&'query DepNode<DefId>, Vec<&'query DepNode<DefId>>>,
// - Keys: some hashable node
// - Values: the hash thereof
pub hashes: FxHashMap<&'query DepNode<DefId>, Fingerprint>,
}
impl<'q> Predecessors<'q> {
pub fn new(query: &'q DepGraphQuery<DefId>, hcx: &mut HashContext) -> Self {
let tcx = hcx.tcx;
let collect_for_metadata = tcx.sess.opts.debugging_opts.incremental_cc ||
tcx.sess.opts.debugging_opts.query_dep_graph;
// Find nodes for which we want to know the full set of preds
let node_count = query.graph.len_nodes();
// Set up some data structures the cache predecessor search needs:
let mut visit_counts: Vec<u32> = Vec::new();
let mut node_cache: Vec<Option<Box<[u32]>>> = Vec::new();
visit_counts.resize(node_count, 0);
node_cache.resize(node_count, None);
let mut dfs_workspace1 = DfsWorkspace::new(node_count);
let mut dfs_workspace2 = DfsWorkspace::new(node_count);
let inputs: FxHashMap<_, _> = query
.graph
.all_nodes()
.iter()
.enumerate()
.filter(|&(_, node)| match node.data {
DepNode::WorkProduct(_) => true,
DepNode::MetaData(ref def_id) => collect_for_metadata && def_id.is_local(),
// if -Z query-dep-graph is passed, save more extended data
// to enable better unit testing
DepNode::TypeckTables(_) |
DepNode::TransCrateItem(_) => tcx.sess.opts.debugging_opts.query_dep_graph,
_ => false,
})
.map(|(node_index, node)| {
find_roots(&query.graph,
node_index as u32,
&mut visit_counts,
&mut node_cache[..],
HashContext::is_hashable,
&mut dfs_workspace1,
Some(&mut dfs_workspace2));
let inputs: Vec<_> = dfs_workspace1.output.nodes.iter().map(|&i| {
query.graph.node_data(NodeIndex(i as usize))
}).collect();
(&node.data, inputs)
})
.collect();
let mut hashes = FxHashMap();
for input in inputs.values().flat_map(|v| v.iter().cloned()) {
hashes.entry(input)
.or_insert_with(|| hcx.hash(input).unwrap());
}
Predecessors {
inputs: inputs,
hashes: hashes,
}
}
}
const CACHING_THRESHOLD: u32 = 60;
// Starting at `start_node`, this function finds this node's "roots", that is,
// anything that is hashable, in the dep-graph. It uses a simple depth-first
// search to achieve that. However, since some sub-graphs are traversed over
// and over again, the function also some caching built into it: Each time it
// visits a node it increases a counter for that node. If a node has been
// visited more often than CACHING_THRESHOLD, the function will allocate a
// cache entry in the `cache` array. This cache entry contains a flat list of
// all roots reachable from the given node. The next time the node is visited,
// the search can just add the contents of this array to the output instead of
// recursing further.
//
// The function takes two `DfsWorkspace` arguments. These contains some data
// structures that would be expensive to re-allocate all the time, so they are
// allocated once up-front. There are two of them because building a cache entry
// requires a recursive invocation of this function. Two are enough though,
// since function never recurses more than once.
fn find_roots<T, F>(graph: &Graph<T, ()>,
start_node: u32,
visit_counts: &mut [u32],
cache: &mut [Option<Box<[u32]>>],
is_root: F,
workspace: &mut DfsWorkspace,
mut sub_workspace: Option<&mut DfsWorkspace>)
where F: Copy + Fn(&T) -> bool,
T: ::std::fmt::Debug,
{
workspace.visited.clear();
workspace.output.clear();
workspace.stack.clear();
workspace.stack.push(start_node);
loop {
let node = match workspace.stack.pop() {
Some(node) => node,
None => return,
};
if !workspace.visited.insert(node as usize) {
continue
}
if is_root(graph.node_data(NodeIndex(node as usize))) {
// If this is a root, just add it to the output.
workspace.output.insert(node);
} else {
if let Some(ref cached) = cache[node as usize] {
for &n in &cached[..] {
workspace.output.insert(n);
}
// No need to recurse further from this node
continue
}
visit_counts[node as usize] += 1;
// If this node has been visited often enough to be cached ...
if visit_counts[node as usize] > CACHING_THRESHOLD {
// ... we are actually allowed to cache something, do so:
if let Some(ref mut sub_workspace) = sub_workspace {
// Note that the following recursive invocation does never
// write to the cache (since we pass None as sub_workspace).
// This is intentional: The graph we are working with
// contains cycles and this prevent us from simply building
// our caches recursively on-demand.
// However, we can just do a regular, non-caching DFS to
// yield the set of roots and cache that.
find_roots(graph,
node,
visit_counts,
cache,
is_root,
sub_workspace,
None);
for &n in &sub_workspace.output.nodes {
workspace.output.insert(n);
}
cache[node as usize] = Some(sub_workspace.output
.nodes
.clone()
.into_boxed_slice());
// No need to recurse further from this node
continue
}
}
for pred in graph.predecessor_nodes(NodeIndex(node as usize)) {
workspace.stack.push(pred.node_id() as u32);
}
}
}
}
struct DfsWorkspace {
stack: Vec<u32>,
visited: BitVector,
output: NodeIndexSet,
}
impl DfsWorkspace {
fn new(total_node_count: usize) -> DfsWorkspace {
DfsWorkspace {
stack: Vec::new(),
visited: BitVector::new(total_node_count),
output: NodeIndexSet::new(total_node_count),
}
}
}
struct NodeIndexSet {
bitset: BitVector,
nodes: Vec<u32>,
}
impl NodeIndexSet {
fn new(total_node_count: usize) -> NodeIndexSet {
NodeIndexSet {
bitset: BitVector::new(total_node_count),
nodes: Vec::new(),
}
}
#[inline]
fn clear(&mut self) {
self.bitset.clear();
self.nodes.clear();
}
#[inline]
fn insert(&mut self, node: u32) {
if self.bitset.insert(node as usize) {
self.nodes.push(node)
}
}
}
#[test]
fn test_cached_dfs_acyclic() {
// 0 1 2
// | \ /
// 3---+ |
// | | |
// | | |
// 4 5 6
// \ / \ / \
// | | |
// 7 8 9
let mut g: Graph<bool, ()> = Graph::new();
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(true);
g.add_node(true);
g.add_node(true);
g.add_edge(NodeIndex(3), NodeIndex(0), ());
g.add_edge(NodeIndex(4), NodeIndex(3), ());
g.add_edge(NodeIndex(7), NodeIndex(4), ());
g.add_edge(NodeIndex(5), NodeIndex(3), ());
g.add_edge(NodeIndex(7), NodeIndex(5), ());
g.add_edge(NodeIndex(8), NodeIndex(5), ());
g.add_edge(NodeIndex(8), NodeIndex(6), ());
g.add_edge(NodeIndex(9), NodeIndex(6), ());
g.add_edge(NodeIndex(6), NodeIndex(1), ());
g.add_edge(NodeIndex(6), NodeIndex(2), ());
let mut ws1 = DfsWorkspace::new(g.len_nodes());
let mut ws2 = DfsWorkspace::new(g.len_nodes());
let mut visit_counts: Vec<_> = g.all_nodes().iter().map(|_| 0u32).collect();
let mut cache: Vec<Option<Box<[u32]>>> = g.all_nodes().iter().map(|_| None).collect();
fn is_root(x: &bool) -> bool { *x }
for _ in 0 .. CACHING_THRESHOLD + 1 {
find_roots(&g, 5, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![7, 8]);
find_roots(&g, 6, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![8, 9]);
find_roots(&g, 0, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![7, 8]);
find_roots(&g, 1, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![8, 9]);
find_roots(&g, 2, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![8, 9]);
find_roots(&g, 3, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![7, 8]);
find_roots(&g, 4, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![7]);
}
}
#[test]
fn test_cached_dfs_cyclic() {
// 0 1 <---- 2 3
// ^ | ^ ^
// | v | |
// 4 ----> 5 ----> 6 ----> 7
// ^ ^ ^ ^
// | | | |
// 8 9 10 11
let mut g: Graph<bool, ()> = Graph::new();
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(false);
g.add_node(true);
g.add_node(true);
g.add_node(true);
g.add_node(true);
g.add_edge(NodeIndex( 4), NodeIndex(0), ());
g.add_edge(NodeIndex( 8), NodeIndex(4), ());
g.add_edge(NodeIndex( 4), NodeIndex(5), ());
g.add_edge(NodeIndex( 1), NodeIndex(5), ());
g.add_edge(NodeIndex( 9), NodeIndex(5), ());
g.add_edge(NodeIndex( 5), NodeIndex(6), ());
g.add_edge(NodeIndex( 6), NodeIndex(2), ());
g.add_edge(NodeIndex( 2), NodeIndex(1), ());
g.add_edge(NodeIndex(10), NodeIndex(6), ());
g.add_edge(NodeIndex( 6), NodeIndex(7), ());
g.add_edge(NodeIndex(11), NodeIndex(7), ());
g.add_edge(NodeIndex( 7), NodeIndex(3), ());
let mut ws1 = DfsWorkspace::new(g.len_nodes());
let mut ws2 = DfsWorkspace::new(g.len_nodes());
let mut visit_counts: Vec<_> = g.all_nodes().iter().map(|_| 0u32).collect();
let mut cache: Vec<Option<Box<[u32]>>> = g.all_nodes().iter().map(|_| None).collect();
fn is_root(x: &bool) -> bool { *x }
for _ in 0 .. CACHING_THRESHOLD + 1 {
find_roots(&g, 2, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![8, 9, 10]);
find_roots(&g, 3, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
ws1.output.nodes.sort();
assert_eq!(ws1.output.nodes, vec![8, 9, 10, 11]);
}
}

View File

@ -0,0 +1,48 @@
Graph compression
The graph compression algorithm is intended to remove and minimize the
size of the dependency graph so it can be saved, while preserving
everything we care about. In particular, given a set of input/output
nodes in the graph (which must be disjoint), we ensure that the set of
input nodes that can reach a given output node does not change,
although the intermediate nodes may change in various ways. In short,
the output nodes are intended to be the ones whose existence we care
about when we start up, because they have some associated data that we
will try to re-use (and hence if they are dirty, we have to throw that
data away). The other intermediate nodes don't really matter so much.
### Overview
The algorithm works as follows:
1. Do a single walk of the graph to construct a DAG
- in this walk, we identify and unify all cycles, electing a representative "head" node
- this is done using the union-find implementation
- this code is found in the `classify` module
2. The result from this walk is a `Dag`:
- the set of SCCs, represented by the union-find table
- a set of edges in the new DAG, represented by:
- a vector of parent nodes for each child node
- a vector of cross-edges
- once these are canonicalized, some of these edges may turn out to be cyclic edges
(i.e., an edge A -> A where A is the head of some SCC)
3. We pass this `Dag` into the construct code, which then creates a
new graph. This graph has a smaller set of indices which includes
*at least* the inputs/outputs from the original graph, but may have
other nodes as well, if keeping them reduces the overall size of
the graph.
- This code is found in the `construct` module.
### Some notes
The input graph is assumed to have *read-by* edges. i.e., `A -> B`
means that the task B reads data from A. But the DAG defined by
classify is expressed in terms of *reads-from* edges, which are the
inverse. So `A -> B` is the same as `B -rf-> A`. *reads-from* edges
are more natural since we want to walk from the outputs to the inputs,
effectively. When we construct the final graph, we reverse these edges
back into the *read-by* edges common elsewhere.

View File

@ -0,0 +1,139 @@
//! First phase. Detect cycles and cross-edges.
use super::*;
#[cfg(test)]
mod test;
pub struct Classify<'a, 'g: 'a, N: 'g, I: 'a, O: 'a>
where N: Debug + Clone + 'g,
I: Fn(&N) -> bool,
O: Fn(&N) -> bool,
{
r: &'a mut GraphReduce<'g, N, I, O>,
stack: Vec<NodeIndex>,
colors: Vec<Color>,
dag: Dag,
}
#[derive(Copy, Clone, Debug, PartialEq)]
enum Color {
// not yet visited
White,
// visiting; usize is index on stack
Grey(usize),
// finished visiting
Black,
}
impl<'a, 'g, N, I, O> Classify<'a, 'g, N, I, O>
where N: Debug + Clone + 'g,
I: Fn(&N) -> bool,
O: Fn(&N) -> bool,
{
pub(super) fn new(r: &'a mut GraphReduce<'g, N, I, O>) -> Self {
Classify {
r: r,
colors: vec![Color::White; r.in_graph.len_nodes()],
stack: vec![],
dag: Dag {
parents: (0..r.in_graph.len_nodes()).map(|i| NodeIndex(i)).collect(),
cross_edges: vec![],
input_nodes: vec![],
output_nodes: vec![],
},
}
}
pub(super) fn walk(mut self) -> Dag {
for (index, node) in self.r.in_graph.all_nodes().iter().enumerate() {
if (self.r.is_output)(&node.data) {
let index = NodeIndex(index);
self.dag.output_nodes.push(index);
match self.colors[index.0] {
Color::White => self.open(index),
Color::Grey(_) => panic!("grey node but have not yet started a walk"),
Color::Black => (), // already visited, skip
}
}
}
// At this point we've identifed all the cycles, and we've
// constructed a spanning tree over the original graph
// (encoded in `self.parents`) as well as a list of
// cross-edges that reflect additional edges from the DAG.
//
// If we converted each node to its `cycle-head` (a
// representative choice from each SCC, basically) and then
// take the union of `self.parents` and `self.cross_edges`
// (after canonicalization), that is basically our DAG.
//
// Note that both of those may well contain trivial `X -rf-> X`
// cycle edges after canonicalization, though. e.g., if you
// have a graph `{A -rf-> B, B -rf-> A}`, we will have unioned A and
// B, but A will also be B's parent (or vice versa), and hence
// when we canonicalize the parent edge it would become `A -rf->
// A` (or `B -rf-> B`).
self.dag
}
fn open(&mut self, node: NodeIndex) {
let index = self.stack.len();
self.stack.push(node);
self.colors[node.0] = Color::Grey(index);
for child in self.r.inputs(node) {
self.walk_edge(node, child);
}
self.stack.pop().unwrap();
self.colors[node.0] = Color::Black;
if (self.r.is_input)(&self.r.in_graph.node_data(node)) {
// base inputs should have no inputs
assert!(self.r.inputs(node).next().is_none());
debug!("input: `{:?}`", self.r.in_graph.node_data(node));
self.dag.input_nodes.push(node);
}
}
fn walk_edge(&mut self, parent: NodeIndex, child: NodeIndex) {
debug!("walk_edge: {:?} -rf-> {:?}, {:?}",
self.r.in_graph.node_data(parent),
self.r.in_graph.node_data(child),
self.colors[child.0]);
// Ignore self-edges, just in case they exist.
if child == parent {
return;
}
match self.colors[child.0] {
Color::White => {
// Not yet visited this node; start walking it.
assert_eq!(self.dag.parents[child.0], child);
self.dag.parents[child.0] = parent;
self.open(child);
}
Color::Grey(stack_index) => {
// Back-edge; unify everything on stack between here and `stack_index`
// since we are all participating in a cycle
assert!(self.stack[stack_index] == child);
for &n in &self.stack[stack_index..] {
debug!("cycle `{:?}` and `{:?}`", self.r.in_graph.node_data(n), self.r.in_graph.node_data(parent));
self.r.mark_cycle(n, parent);
}
}
Color::Black => {
// Cross-edge, record and ignore
self.dag.cross_edges.push((parent, child));
debug!("cross-edge `{:?} -rf-> {:?}`",
self.r.in_graph.node_data(parent),
self.r.in_graph.node_data(child));
}
}
}
}

View File

@ -0,0 +1,79 @@
use super::*;
#[test]
fn detect_cycles() {
let (graph, nodes) = graph! {
A -> C0,
A -> C1,
B -> C1,
C0 -> C1,
C1 -> C0,
C0 -> D,
C1 -> E,
};
let inputs = ["A", "B"];
let outputs = ["D", "E"];
let mut reduce = GraphReduce::new(&graph, |n| inputs.contains(n), |n| outputs.contains(n));
Classify::new(&mut reduce).walk();
assert!(!reduce.in_cycle(nodes("A"), nodes("C0")));
assert!(!reduce.in_cycle(nodes("B"), nodes("C0")));
assert!(reduce.in_cycle(nodes("C0"), nodes("C1")));
assert!(!reduce.in_cycle(nodes("D"), nodes("C0")));
assert!(!reduce.in_cycle(nodes("E"), nodes("C0")));
assert!(!reduce.in_cycle(nodes("E"), nodes("A")));
}
/// Regr test for a bug where we forgot to pop nodes off of the stack
/// as we were walking. In this case, because edges are pushed to the front
/// of the list, we would visit OUT, then A, then IN, and then close IN (but forget
/// to POP. Then visit B, C, and then A, which would mark everything from A to C as
/// cycle. But since we failed to pop IN, the stack was `OUT, A, IN, B, C` so that
/// marked C and IN as being in a cycle.
#[test]
fn edge_order1() {
let (graph, nodes) = graph! {
A -> C,
C -> B,
B -> A,
IN -> B,
IN -> A,
A -> OUT,
};
let inputs = ["IN"];
let outputs = ["OUT"];
let mut reduce = GraphReduce::new(&graph, |n| inputs.contains(n), |n| outputs.contains(n));
Classify::new(&mut reduce).walk();
assert!(reduce.in_cycle(nodes("B"), nodes("C")));
assert!(!reduce.in_cycle(nodes("IN"), nodes("A")));
assert!(!reduce.in_cycle(nodes("IN"), nodes("B")));
assert!(!reduce.in_cycle(nodes("IN"), nodes("C")));
assert!(!reduce.in_cycle(nodes("IN"), nodes("OUT")));
}
/// Same as `edge_order1` but in reverse order so as to detect a failure
/// if we were to enqueue edges onto end of list instead.
#[test]
fn edge_order2() {
let (graph, nodes) = graph! {
A -> OUT,
IN -> A,
IN -> B,
B -> A,
C -> B,
A -> C,
};
let inputs = ["IN"];
let outputs = ["OUT"];
let mut reduce = GraphReduce::new(&graph, |n| inputs.contains(n), |n| outputs.contains(n));
Classify::new(&mut reduce).walk();
assert!(reduce.in_cycle(nodes("B"), nodes("C")));
assert!(!reduce.in_cycle(nodes("IN"), nodes("A")));
assert!(!reduce.in_cycle(nodes("IN"), nodes("B")));
assert!(!reduce.in_cycle(nodes("IN"), nodes("C")));
assert!(!reduce.in_cycle(nodes("IN"), nodes("OUT")));
}

View File

@ -0,0 +1,201 @@
//! Second phase. Construct new graph. The previous phase has
//! converted the input graph into a DAG by detecting and unifying
//! cycles. It provides us with the following (which is a
//! representation of the DAG):
//!
//! - SCCs, in the form of a union-find repr that can convert each node to
//! its *cycle head* (an arbitrarly chosen representative from the cycle)
//! - a vector of *leaf nodes*, just a convenience
//! - a vector of *parents* for each node (in some cases, nodes have no parents,
//! or their parent is another member of same cycle; in that case, the vector
//! will be stored `v[i] == i`, after canonicalization)
//! - a vector of *cross edges*, meaning add'l edges between graphs nodes beyond
//! the parents.
use rustc_data_structures::fx::FxHashMap;
use super::*;
pub(super) fn construct_graph<'g, N, I, O>(r: &mut GraphReduce<'g, N, I, O>, dag: Dag)
-> Reduction<'g, N>
where N: Debug + Clone, I: Fn(&N) -> bool, O: Fn(&N) -> bool,
{
let Dag { parents: old_parents, input_nodes, output_nodes, cross_edges } = dag;
let in_graph = r.in_graph;
debug!("construct_graph");
// Create a canonical list of edges; this includes both parent and
// cross-edges. We store this in `(target -> Vec<source>)` form.
// We call the first edge to any given target its "parent".
let mut edges = FxHashMap();
let old_parent_edges = old_parents.iter().cloned().zip((0..).map(NodeIndex));
for (source, target) in old_parent_edges.chain(cross_edges) {
debug!("original edge `{:?} -rf-> {:?}`",
in_graph.node_data(source),
in_graph.node_data(target));
let source = r.cycle_head(source);
let target = r.cycle_head(target);
if source != target {
let v = edges.entry(target).or_insert(vec![]);
if !v.contains(&source) {
debug!("edge `{:?} -rf-> {:?}` is edge #{} with that target",
in_graph.node_data(source),
in_graph.node_data(target),
v.len());
v.push(source);
}
}
}
let parent = |ni: NodeIndex| -> NodeIndex {
edges[&ni][0]
};
// `retain_map`: a map of those nodes that we will want to
// *retain* in the ultimate graph; the key is the node index in
// the old graph, the value is the node index in the new
// graph. These are nodes in the following categories:
//
// - inputs
// - work-products
// - targets of a cross-edge
//
// The first two categories hopefully make sense. We want the
// inputs so we can compare hashes later. We want the
// work-products so we can tell precisely when a given
// work-product is invalidated. But the last one isn't strictly
// needed; we keep cross-target edges so as to minimize the total
// graph size.
//
// Consider a graph like:
//
// WP0 -rf-> Y
// WP1 -rf-> Y
// Y -rf-> INPUT0
// Y -rf-> INPUT1
// Y -rf-> INPUT2
// Y -rf-> INPUT3
//
// Now if we were to remove Y, we would have a total of 8 edges: both WP0 and WP1
// depend on INPUT0...INPUT3. As it is, we have 6 edges.
let mut retain_map = FxHashMap();
let mut new_graph = Graph::new();
{
// Start by adding start-nodes and inputs.
let retained_nodes = output_nodes.iter().chain(&input_nodes).map(|&n| r.cycle_head(n));
// Next add in targets of cross-edges. Due to the canonicalization,
// some of these may be self-edges or may may duplicate the parent
// edges, so ignore those.
let retained_nodes = retained_nodes.chain(
edges.iter()
.filter(|&(_, ref sources)| sources.len() > 1)
.map(|(&target, _)| target));
// Now create the new graph, adding in the entries from the map.
for n in retained_nodes {
retain_map.entry(n)
.or_insert_with(|| {
let data = in_graph.node_data(n);
debug!("retaining node `{:?}`", data);
new_graph.add_node(data)
});
}
}
// Given a cycle-head `ni`, converts it to the closest parent that has
// been retained in the output graph.
let retained_parent = |mut ni: NodeIndex| -> NodeIndex {
loop {
debug!("retained_parent({:?})", in_graph.node_data(ni));
match retain_map.get(&ni) {
Some(&v) => return v,
None => ni = parent(ni),
}
}
};
// Now add in the edges into the graph.
for (&target, sources) in &edges {
if let Some(&r_target) = retain_map.get(&target) {
debug!("adding edges that target `{:?}`", in_graph.node_data(target));
for &source in sources {
debug!("new edge `{:?} -rf-> {:?}`",
in_graph.node_data(source),
in_graph.node_data(target));
let r_source = retained_parent(source);
// NB. In the input graph, we have `a -> b` if b
// **reads from** a. But in the terminology of this
// code, we would describe that edge as `b -> a`,
// because we have edges *from* outputs *to* inputs.
// Therefore, when we create our new graph, we have to
// reverse the edge.
new_graph.add_edge(r_target, r_source, ());
}
} else {
assert_eq!(sources.len(), 1);
}
}
// One complication. In some cases, output nodes *may* participate in
// cycles. An example:
//
// [HIR0] [HIR1]
// | |
// v v
// TypeckClosureBody(X) -> ItemSignature(X::SomeClosureInX)
// | ^ | |
// | +-------------------------+ |
// | |
// v v
// Foo Bar
//
// In these cases, the output node may not wind up as the head
// of the cycle, in which case it would be absent from the
// final graph. We don't wish this to happen, therefore we go
// over the list of output nodes again and check for any that
// are not their own cycle-head. If we find such a node, we
// add it to the graph now with an edge from the cycle head.
// So the graph above could get transformed into this:
//
// [HIR0, HIR1]
// |
// v
// TypeckClosureBody(X) ItemSignature(X::SomeClosureInX)
// ^ | |
// +-------------------------+ |
// v
// [Foo, Bar]
//
// (Note that all the edges here are "read-by" edges, not
// "reads-from" edges.)
for &output_node in &output_nodes {
let head = r.cycle_head(output_node);
if output_node == head {
assert!(retain_map.contains_key(&output_node));
} else {
assert!(!retain_map.contains_key(&output_node));
let output_data = in_graph.node_data(output_node);
let new_node = new_graph.add_node(output_data);
let new_head_node = retain_map[&head];
new_graph.add_edge(new_head_node, new_node, ());
}
}
// Finally, prepare a list of the input node indices as found in
// the new graph. Note that since all input nodes are leaves in
// the graph, they should never participate in a cycle.
let input_nodes =
input_nodes.iter()
.map(|&n| {
assert_eq!(r.cycle_head(n), n, "input node participating in a cycle");
retain_map[&n]
})
.collect();
Reduction { graph: new_graph, input_nodes: input_nodes }
}

View File

@ -0,0 +1,33 @@
use rustc_data_structures::graph::NodeIndex;
use rustc_data_structures::unify::UnifyKey;
#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)]
pub struct DagId {
index: u32,
}
impl DagId {
pub fn from_in_index(n: NodeIndex) -> Self {
DagId { index: n.0 as u32 }
}
pub fn as_in_index(&self) -> NodeIndex {
NodeIndex(self.index as usize)
}
}
impl UnifyKey for DagId {
type Value = ();
fn index(&self) -> u32 {
self.index
}
fn from_index(u: u32) -> Self {
DagId { index: u }
}
fn tag(_: Option<Self>) -> &'static str {
"DagId"
}
}

View File

@ -0,0 +1,125 @@
// Copyright 2012-2013 The Rust Project Developers. See the COPYRIGHT
// file at the top-level directory of this distribution and at
// http://rust-lang.org/COPYRIGHT.
//
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
// option. This file may not be copied, modified, or distributed
// except according to those terms.
//! Graph compression. See `README.md`.
use rustc_data_structures::graph::{Graph, NodeIndex};
use rustc_data_structures::unify::UnificationTable;
use std::fmt::Debug;
#[cfg(test)]
#[macro_use]
mod test_macro;
mod construct;
mod classify;
use self::classify::Classify;
mod dag_id;
use self::dag_id::DagId;
#[cfg(test)]
mod test;
pub fn reduce_graph<N, I, O>(graph: &Graph<N, ()>,
is_input: I,
is_output: O) -> Reduction<N>
where N: Debug + Clone,
I: Fn(&N) -> bool,
O: Fn(&N) -> bool,
{
GraphReduce::new(graph, is_input, is_output).compute()
}
pub struct Reduction<'q, N> where N: 'q + Debug + Clone {
pub graph: Graph<&'q N, ()>,
pub input_nodes: Vec<NodeIndex>,
}
struct GraphReduce<'q, N, I, O>
where N: 'q + Debug + Clone,
I: Fn(&N) -> bool,
O: Fn(&N) -> bool,
{
in_graph: &'q Graph<N, ()>,
unify: UnificationTable<DagId>,
is_input: I,
is_output: O,
}
struct Dag {
// The "parent" of a node is the node which reached it during the
// initial DFS. To encode the case of "no parent" (i.e., for the
// roots of the walk), we make `parents[i] == i` to start, which
// turns out be convenient.
parents: Vec<NodeIndex>,
// Additional edges beyond the parents.
cross_edges: Vec<(NodeIndex, NodeIndex)>,
// Nodes which we found that are considered "outputs"
output_nodes: Vec<NodeIndex>,
// Nodes which we found that are considered "inputs"
input_nodes: Vec<NodeIndex>,
}
#[derive(Copy, Clone, PartialEq, Eq, Hash)]
struct DagNode {
in_index: NodeIndex
}
impl<'q, N, I, O> GraphReduce<'q, N, I, O>
where N: Debug + Clone,
I: Fn(&N) -> bool,
O: Fn(&N) -> bool,
{
fn new(in_graph: &'q Graph<N, ()>, is_input: I, is_output: O) -> Self {
let mut unify = UnificationTable::new();
// create a set of unification keys whose indices
// correspond to the indices from the input graph
for i in 0..in_graph.len_nodes() {
let k = unify.new_key(());
assert!(k == DagId::from_in_index(NodeIndex(i)));
}
GraphReduce { in_graph, unify, is_input, is_output }
}
fn compute(mut self) -> Reduction<'q, N> {
let dag = Classify::new(&mut self).walk();
construct::construct_graph(&mut self, dag)
}
fn inputs(&self, in_node: NodeIndex) -> impl Iterator<Item = NodeIndex> + 'q {
self.in_graph.predecessor_nodes(in_node)
}
fn mark_cycle(&mut self, in_node1: NodeIndex, in_node2: NodeIndex) {
let dag_id1 = DagId::from_in_index(in_node1);
let dag_id2 = DagId::from_in_index(in_node2);
self.unify.union(dag_id1, dag_id2);
}
/// Convert a dag-id into its cycle head representative. This will
/// be a no-op unless `in_node` participates in a cycle, in which
/// case a distinct node *may* be returned.
fn cycle_head(&mut self, in_node: NodeIndex) -> NodeIndex {
let i = DagId::from_in_index(in_node);
self.unify.find(i).as_in_index()
}
#[cfg(test)]
fn in_cycle(&mut self, ni1: NodeIndex, ni2: NodeIndex) -> bool {
self.cycle_head(ni1) == self.cycle_head(ni2)
}
}

View File

@ -0,0 +1,243 @@
use super::*;
fn reduce(graph: &Graph<&'static str, ()>,
inputs: &[&'static str],
outputs: &[&'static str],
expected: &[&'static str])
{
let reduce = GraphReduce::new(&graph,
|n| inputs.contains(n),
|n| outputs.contains(n));
let result = reduce.compute();
let mut edges: Vec<String> =
result.graph
.all_edges()
.iter()
.map(|edge| format!("{} -> {}",
result.graph.node_data(edge.source()),
result.graph.node_data(edge.target())))
.collect();
edges.sort();
println!("{:#?}", edges);
assert_eq!(edges.len(), expected.len());
for (expected, actual) in expected.iter().zip(&edges) {
assert_eq!(expected, actual);
}
}
#[test]
fn test1() {
// +---------------+
// | |
// | +--------|------+
// | | v v
// [A] -> [C0] -> [C1] [D]
// [ ] <- [ ] -> [E]
// ^
// [B] -------------+
let (graph, _nodes) = graph! {
A -> C0,
A -> C1,
B -> C1,
C0 -> C1,
C1 -> C0,
C0 -> D,
C1 -> E,
};
// [A] -> [C1] -> [D]
// [B] -> [ ] -> [E]
reduce(&graph, &["A", "B"], &["D", "E"], &[
"A -> C1",
"B -> C1",
"C1 -> D",
"C1 -> E",
]);
}
#[test]
fn test2() {
// +---------------+
// | |
// | +--------|------+
// | | v v
// [A] -> [C0] -> [C1] [D] -> [E]
// [ ] <- [ ]
// ^
// [B] -------------+
let (graph, _nodes) = graph! {
A -> C0,
A -> C1,
B -> C1,
C0 -> C1,
C1 -> C0,
C0 -> D,
D -> E,
};
// [A] -> [D] -> [E]
// [B] -> [ ]
reduce(&graph, &["A", "B"], &["D", "E"], &[
"A -> D",
"B -> D",
"D -> E",
]);
}
#[test]
fn test2b() {
// Variant on test2 in which [B] is not
// considered an input.
let (graph, _nodes) = graph! {
A -> C0,
A -> C1,
B -> C1,
C0 -> C1,
C1 -> C0,
C0 -> D,
D -> E,
};
// [A] -> [D] -> [E]
reduce(&graph, &["A"], &["D", "E"], &[
"A -> D",
"D -> E",
]);
}
#[test]
fn test3() {
// Edges going *downwards*, so 0, 1 and 2 are inputs,
// while 7, 8, and 9 are outputs.
//
// 0 1 2
// | \ /
// 3---+ |
// | | |
// | | |
// 4 5 6
// \ / \ / \
// | | |
// 7 8 9
//
// Here the end result removes node 4, instead encoding an edge
// from n3 -> n7, but keeps nodes 5 and 6, as they are common
// inputs to nodes 8/9.
let (graph, _nodes) = graph! {
n0 -> n3,
n3 -> n4,
n3 -> n5,
n4 -> n7,
n5 -> n7,
n5 -> n8,
n1 -> n6,
n2 -> n6,
n6 -> n8,
n6 -> n9,
};
reduce(&graph, &["n0", "n1", "n2"], &["n7", "n8", "n9"], &[
"n0 -> n3",
"n1 -> n6",
"n2 -> n6",
"n3 -> n5",
"n3 -> n7",
"n5 -> n7",
"n5 -> n8",
"n6 -> n8",
"n6 -> n9"
]);
}
//#[test]
//fn test_cached_dfs_cyclic() {
//
// // 0 1 <---- 2 3
// // ^ | ^ ^
// // | v | |
// // 4 ----> 5 ----> 6 ----> 7
// // ^ ^ ^ ^
// // | | | |
// // 8 9 10 11
//
//
// let mut g: Graph<bool, ()> = Graph::new();
// g.add_node(false);
// g.add_node(false);
// g.add_node(false);
// g.add_node(false);
// g.add_node(false);
// g.add_node(false);
// g.add_node(false);
// g.add_node(false);
// g.add_node(true);
// g.add_node(true);
// g.add_node(true);
// g.add_node(true);
//
// g.add_edge(NodeIndex( 4), NodeIndex(0), ());
// g.add_edge(NodeIndex( 8), NodeIndex(4), ());
// g.add_edge(NodeIndex( 4), NodeIndex(5), ());
// g.add_edge(NodeIndex( 1), NodeIndex(5), ());
// g.add_edge(NodeIndex( 9), NodeIndex(5), ());
// g.add_edge(NodeIndex( 5), NodeIndex(6), ());
// g.add_edge(NodeIndex( 6), NodeIndex(2), ());
// g.add_edge(NodeIndex( 2), NodeIndex(1), ());
// g.add_edge(NodeIndex(10), NodeIndex(6), ());
// g.add_edge(NodeIndex( 6), NodeIndex(7), ());
// g.add_edge(NodeIndex(11), NodeIndex(7), ());
// g.add_edge(NodeIndex( 7), NodeIndex(3), ());
//
// let mut ws1 = DfsWorkspace::new(g.len_nodes());
// let mut ws2 = DfsWorkspace::new(g.len_nodes());
// let mut visit_counts: Vec<_> = g.all_nodes().iter().map(|_| 0u32).collect();
// let mut cache: Vec<Option<Box<[u32]>>> = g.all_nodes().iter().map(|_| None).collect();
//
// fn is_root(x: &bool) -> bool { *x }
//
// for _ in 0 .. CACHING_THRESHOLD + 1 {
// find_roots(&g, 2, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
// ws1.output.nodes.sort();
// assert_eq!(ws1.output.nodes, vec![8, 9, 10]);
//
// find_roots(&g, 3, &mut visit_counts, &mut cache[..], is_root, &mut ws1, Some(&mut ws2));
// ws1.output.nodes.sort();
// assert_eq!(ws1.output.nodes, vec![8, 9, 10, 11]);
// }
//}
#[test]
fn test_cycle_output() {
// +---------------+
// | |
// | +--------|------+
// | | v v
// [A] -> [C0] <-> [C1] <- [D]
// +----> [E]
// ^
// [B] ----------------- ---+
let (graph, _nodes) = graph! {
A -> C0,
A -> C1,
B -> E,
C0 -> C1,
C1 -> C0,
C0 -> D,
C1 -> E,
D -> C1,
};
// [A] -> [C0] <-> [D]
// +----> [E]
// ^
// [B] -------------+
reduce(&graph, &["A", "B"], &["D", "E"], &[
"A -> C0",
"B -> E",
"C0 -> D",
"C0 -> E",
"D -> C0"
]);
}

View File

@ -0,0 +1,40 @@
macro_rules! graph {
($( $source:ident -> $target:ident, )*) => {
{
use $crate::rustc_data_structures::graph::{Graph, NodeIndex};
use $crate::rustc_data_structures::fx::FxHashMap;
let mut graph = Graph::new();
let mut nodes: FxHashMap<&'static str, NodeIndex> = FxHashMap();
for &name in &[ $(stringify!($source), stringify!($target)),* ] {
let name: &'static str = name;
nodes.entry(name)
.or_insert_with(|| graph.add_node(name));
}
$(
{
let source = nodes[&stringify!($source)];
let target = nodes[&stringify!($target)];
graph.add_edge(source, target, ());
}
)*
let f = move |name: &'static str| -> NodeIndex { nodes[&name] };
(graph, f)
}
}
}
macro_rules! set {
($( $value:expr ),*) => {
{
use $crate::rustc_data_structures::fx::FxHashSet;
let mut set = FxHashSet();
$(set.insert($value);)*
set
}
}
}

View File

@ -0,0 +1,75 @@
// Copyright 2012-2015 The Rust Project Developers. See the COPYRIGHT
// file at the top-level directory of this distribution and at
// http://rust-lang.org/COPYRIGHT.
//
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
// option. This file may not be copied, modified, or distributed
// except according to those terms.
use rustc::dep_graph::{DepGraphQuery, DepNode};
use rustc::hir::def_id::DefId;
use rustc_data_structures::fx::FxHashMap;
use rustc_data_structures::graph::Graph;
use super::hash::*;
use ich::Fingerprint;
mod compress;
/// A data-structure that makes it easy to enumerate the hashable
/// predecessors of any given dep-node.
pub struct Predecessors<'query> {
// A reduced version of the input graph that contains fewer nodes.
// This is intended to keep all of the base inputs (i.e., HIR
// nodes) and all of the "work-products" we may care about
// later. Other nodes may be retained if it keeps the overall size
// of the graph down.
pub reduced_graph: Graph<&'query DepNode<DefId>, ()>,
// For the inputs (hir/foreign-metadata), we include hashes.
pub hashes: FxHashMap<&'query DepNode<DefId>, Fingerprint>,
}
impl<'q> Predecessors<'q> {
pub fn new(query: &'q DepGraphQuery<DefId>, hcx: &mut HashContext) -> Self {
let tcx = hcx.tcx;
let collect_for_metadata = tcx.sess.opts.debugging_opts.incremental_cc ||
tcx.sess.opts.debugging_opts.query_dep_graph;
// Find the set of "start nodes". These are nodes that we will
// possibly query later.
let is_output = |node: &DepNode<DefId>| -> bool {
match *node {
DepNode::WorkProduct(_) => true,
DepNode::MetaData(ref def_id) => collect_for_metadata && def_id.is_local(),
// if -Z query-dep-graph is passed, save more extended data
// to enable better unit testing
DepNode::TypeckTables(_) |
DepNode::TransCrateItem(_) => tcx.sess.opts.debugging_opts.query_dep_graph,
_ => false,
}
};
// Reduce the graph to the most important nodes.
let compress::Reduction { graph, input_nodes } =
compress::reduce_graph(&query.graph, HashContext::is_hashable, is_output);
let mut hashes = FxHashMap();
for input_index in input_nodes {
let input = *graph.node_data(input_index);
debug!("computing hash for input node `{:?}`", input);
hashes.entry(input)
.or_insert_with(|| hcx.hash(input).unwrap());
}
Predecessors {
reduced_graph: graph,
hashes: hashes,
}
}
}

View File

@ -14,6 +14,7 @@ use rustc::hir::svh::Svh;
use rustc::session::Session;
use rustc::ty::TyCtxt;
use rustc_data_structures::fx::FxHashMap;
use rustc_data_structures::graph::{NodeIndex, INCOMING};
use rustc_serialize::Encodable as RustcEncodable;
use rustc_serialize::opaque::Encoder;
use std::hash::Hash;
@ -178,7 +179,9 @@ pub fn encode_dep_graph(preds: &Predecessors,
// Create a flat list of (Input, WorkProduct) edges for
// serialization.
let mut edges = vec![];
for (&target, sources) in &preds.inputs {
for edge in preds.reduced_graph.all_edges() {
let source = *preds.reduced_graph.node_data(edge.source());
let target = *preds.reduced_graph.node_data(edge.target());
match *target {
DepNode::MetaData(ref def_id) => {
// Metadata *targets* are always local metadata nodes. We have
@ -188,11 +191,10 @@ pub fn encode_dep_graph(preds: &Predecessors,
}
_ => (),
}
debug!("serialize edge: {:?} -> {:?}", source, target);
let source = builder.map(source);
let target = builder.map(target);
for &source in sources {
let source = builder.map(source);
edges.push((source, target.clone()));
}
edges.push((source, target));
}
if tcx.sess.opts.debugging_opts.incremental_dump_hash {
@ -250,12 +252,10 @@ pub fn encode_metadata_hashes(tcx: TyCtxt,
let mut def_id_hashes = FxHashMap();
for (&target, sources) in &preds.inputs {
let def_id = match *target {
DepNode::MetaData(def_id) => {
assert!(def_id.is_local());
def_id
}
for (index, target) in preds.reduced_graph.all_nodes().iter().enumerate() {
let index = NodeIndex(index);
let def_id = match *target.data {
DepNode::MetaData(def_id) if def_id.is_local() => def_id,
_ => continue,
};
@ -281,13 +281,17 @@ pub fn encode_metadata_hashes(tcx: TyCtxt,
// is the det. hash of the def-path. This is convenient
// because we can sort this to get a stable ordering across
// compilations, even if the def-ids themselves have changed.
let mut hashes: Vec<(DepNode<u64>, Fingerprint)> = sources.iter()
.map(|dep_node| {
let hash_dep_node = dep_node.map_def(|&def_id| Some(def_id_hash(def_id))).unwrap();
let hash = preds.hashes[dep_node];
(hash_dep_node, hash)
})
.collect();
let mut hashes: Vec<(DepNode<u64>, Fingerprint)> =
preds.reduced_graph
.depth_traverse(index, INCOMING)
.map(|index| preds.reduced_graph.node_data(index))
.filter(|dep_node| HashContext::is_hashable(dep_node))
.map(|dep_node| {
let hash_dep_node = dep_node.map_def(|&def_id| Some(def_id_hash(def_id))).unwrap();
let hash = preds.hashes[dep_node];
(hash_dep_node, hash)
})
.collect();
hashes.sort();
let mut state = IchHasher::new();
@ -298,9 +302,12 @@ pub fn encode_metadata_hashes(tcx: TyCtxt,
if tcx.sess.opts.debugging_opts.incremental_dump_hash {
println!("metadata hash for {:?} is {}", def_id, hash);
for dep_node in sources {
println!("metadata hash for {:?} depends on {:?} with hash {}",
def_id, dep_node, preds.hashes[dep_node]);
for pred_index in preds.reduced_graph.depth_traverse(index, INCOMING) {
let dep_node = preds.reduced_graph.node_data(pred_index);
if HashContext::is_hashable(&dep_node) {
println!("metadata hash for {:?} depends on {:?} with hash {}",
def_id, dep_node, preds.hashes[dep_node]);
}
}
}