incr.comp.: Manage dependency graph on main thread.

This commit is contained in:
Michael Woerister 2017-07-04 15:06:57 +02:00
parent 13e87d1dbb
commit 089860b603
7 changed files with 174 additions and 477 deletions

View File

@ -9,13 +9,16 @@
// except according to those terms.
use rustc_data_structures::fx::{FxHashMap, FxHashSet};
use std::env;
use super::{DepGraphQuery, DepNode};
use super::debug::EdgeFilter;
pub struct DepGraphEdges {
nodes: Vec<DepNode>,
indices: FxHashMap<DepNode, IdIndex>,
edges: FxHashSet<(IdIndex, IdIndex)>,
open_nodes: Vec<OpenNode>,
task_stack: Vec<OpenTask>,
forbidden_edge: Option<EdgeFilter>,
}
#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)]
@ -35,67 +38,80 @@ impl IdIndex {
}
#[derive(Clone, Debug, PartialEq)]
enum OpenNode {
Node(IdIndex),
enum OpenTask {
Regular {
node: DepNode,
reads: Vec<DepNode>,
read_set: FxHashSet<DepNode>,
},
Ignore,
}
impl DepGraphEdges {
pub fn new() -> DepGraphEdges {
let forbidden_edge = if cfg!(debug_assertions) {
match env::var("RUST_FORBID_DEP_GRAPH_EDGE") {
Ok(s) => {
match EdgeFilter::new(&s) {
Ok(f) => Some(f),
Err(err) => bug!("RUST_FORBID_DEP_GRAPH_EDGE invalid: {}", err),
}
}
Err(_) => None,
}
} else {
None
};
DepGraphEdges {
nodes: vec![],
indices: FxHashMap(),
edges: FxHashSet(),
open_nodes: Vec::new()
task_stack: Vec::new(),
forbidden_edge,
}
}
fn id(&self, index: IdIndex) -> DepNode {
self.nodes[index.index()].clone()
}
/// Creates a node for `id` in the graph.
fn make_node(&mut self, id: DepNode) -> IdIndex {
if let Some(&i) = self.indices.get(&id) {
return i;
}
let index = IdIndex::new(self.nodes.len());
self.nodes.push(id.clone());
self.indices.insert(id, index);
index
}
/// Top of the stack of open nodes.
fn current_node(&self) -> Option<OpenNode> {
self.open_nodes.last().cloned()
self.nodes[index.index()]
}
pub fn push_ignore(&mut self) {
self.open_nodes.push(OpenNode::Ignore);
self.task_stack.push(OpenTask::Ignore);
}
pub fn pop_ignore(&mut self) {
let popped_node = self.open_nodes.pop().unwrap();
assert_eq!(popped_node, OpenNode::Ignore);
let popped_node = self.task_stack.pop().unwrap();
debug_assert_eq!(popped_node, OpenTask::Ignore);
}
pub fn push_task(&mut self, key: DepNode) {
let top_node = self.current_node();
let new_node = self.make_node(key);
self.open_nodes.push(OpenNode::Node(new_node));
// if we are in the midst of doing task T, then this new task
// N is a subtask of T, so add an edge N -> T.
if let Some(top_node) = top_node {
self.add_edge_from_open_node(top_node, |t| (new_node, t));
}
self.task_stack.push(OpenTask::Regular {
node: key,
reads: Vec::new(),
read_set: FxHashSet(),
});
}
pub fn pop_task(&mut self, key: DepNode) {
let popped_node = self.open_nodes.pop().unwrap();
assert_eq!(OpenNode::Node(self.indices[&key]), popped_node);
let popped_node = self.task_stack.pop().unwrap();
if let OpenTask::Regular {
node,
read_set: _,
reads
} = popped_node {
debug_assert_eq!(node, key);
let target_id = self.get_or_create_node(node);
for read in reads.into_iter() {
let source_id = self.get_or_create_node(read);
self.edges.insert((source_id, target_id));
}
} else {
bug!("pop_task() - Expected regular task to be popped")
}
}
/// Indicates that the current task `C` reads `v` by adding an
@ -103,57 +119,28 @@ impl DepGraphEdges {
/// effect. Note that *reading* from tracked state is harmless if
/// you are not in a task; what is bad is *writing* to tracked
/// state (and leaking data that you read into a tracked task).
pub fn read(&mut self, v: DepNode) {
if self.current_node().is_some() {
let source = self.make_node(v);
self.add_edge_from_current_node(|current| (source, current))
}
}
pub fn read(&mut self, source: DepNode) {
match self.task_stack.last_mut() {
Some(&mut OpenTask::Regular {
node: target,
ref mut reads,
ref mut read_set,
}) => {
if read_set.insert(source) {
reads.push(source);
/// Indicates that the current task `C` writes `v` by adding an
/// edge from `C` to `v`. If there is no current task, panics. If
/// you want to suppress this edge, use `ignore`.
pub fn write(&mut self, v: DepNode) {
let target = self.make_node(v);
self.add_edge_from_current_node(|current| (current, target))
}
/// Invoke `add_edge_from_open_node` with the top of the stack, or
/// panic if stack is empty.
fn add_edge_from_current_node<OP>(&mut self,
op: OP)
where OP: FnOnce(IdIndex) -> (IdIndex, IdIndex)
{
match self.current_node() {
Some(open_node) => self.add_edge_from_open_node(open_node, op),
None => bug!("no current node, cannot add edge into dependency graph")
}
}
/// Adds an edge to or from the `open_node`, assuming `open_node`
/// is not `Ignore`. The direction of the edge is determined by
/// the closure `op` --- we pass as argument the open node `n`,
/// and the closure returns a (source, target) tuple, which should
/// include `n` in one spot or another.
fn add_edge_from_open_node<OP>(&mut self,
open_node: OpenNode,
op: OP)
where OP: FnOnce(IdIndex) -> (IdIndex, IdIndex)
{
let (source, target) = match open_node {
OpenNode::Node(n) => op(n),
OpenNode::Ignore => { return; }
};
// ignore trivial self edges, which are not very interesting
if source == target {
return;
}
if self.edges.insert((source, target)) {
debug!("adding edge from {:?} to {:?}",
self.id(source),
self.id(target));
if cfg!(debug_assertions) {
if let Some(ref forbidden_edge) = self.forbidden_edge {
if forbidden_edge.test(&source, &target) {
bug!("forbidden edge {:?} -> {:?} created", source, target)
}
}
}
}
}
Some(&mut OpenTask::Ignore) | None => {
// ignore
}
}
}
@ -163,4 +150,30 @@ impl DepGraphEdges {
.collect();
DepGraphQuery::new(&self.nodes, &edges)
}
#[inline]
pub fn add_edge(&mut self, source: DepNode, target: DepNode) {
let source = self.get_or_create_node(source);
let target = self.get_or_create_node(target);
self.edges.insert((source, target));
}
pub fn add_node(&mut self, node: DepNode) {
self.get_or_create_node(node);
}
#[inline]
fn get_or_create_node(&mut self, dep_node: DepNode) -> IdIndex {
let DepGraphEdges {
ref mut indices,
ref mut nodes,
..
} = *self;
*indices.entry(dep_node).or_insert_with(|| {
let next_id = nodes.len();
nodes.push(dep_node);
IdIndex::new(next_id)
})
}
}

View File

@ -17,17 +17,16 @@ use super::dep_node::{DepNode, WorkProductId};
use super::query::DepGraphQuery;
use super::raii;
use super::safe::DepGraphSafe;
use super::thread::{DepGraphThreadData, DepMessage};
use super::edges::DepGraphEdges;
#[derive(Clone)]
pub struct DepGraph {
data: Rc<DepGraphData>
data: Option<Rc<DepGraphData>>
}
struct DepGraphData {
/// We send messages to the thread to let it build up the dep-graph
/// from the current run.
thread: DepGraphThreadData,
/// The actual graph data.
edges: RefCell<DepGraphEdges>,
/// When we load, there may be `.o` files, cached mir, or other such
/// things available to us. If we find that they are not dirty, we
@ -44,31 +43,35 @@ struct DepGraphData {
impl DepGraph {
pub fn new(enabled: bool) -> DepGraph {
DepGraph {
data: Rc::new(DepGraphData {
thread: DepGraphThreadData::new(enabled),
previous_work_products: RefCell::new(FxHashMap()),
work_products: RefCell::new(FxHashMap()),
dep_node_debug: RefCell::new(FxHashMap()),
})
data: if enabled {
Some(Rc::new(DepGraphData {
previous_work_products: RefCell::new(FxHashMap()),
work_products: RefCell::new(FxHashMap()),
edges: RefCell::new(DepGraphEdges::new()),
dep_node_debug: RefCell::new(FxHashMap()),
}))
} else {
None
}
}
}
/// True if we are actually building the full dep-graph.
#[inline]
pub fn is_fully_enabled(&self) -> bool {
self.data.thread.is_fully_enabled()
self.data.is_some()
}
pub fn query(&self) -> DepGraphQuery {
self.data.thread.query()
self.data.as_ref().unwrap().edges.borrow().query()
}
pub fn in_ignore<'graph>(&'graph self) -> Option<raii::IgnoreTask<'graph>> {
raii::IgnoreTask::new(&self.data.thread)
self.data.as_ref().map(|data| raii::IgnoreTask::new(&data.edges))
}
pub fn in_task<'graph>(&'graph self, key: DepNode) -> Option<raii::DepTask<'graph>> {
raii::DepTask::new(&self.data.thread, key)
self.data.as_ref().map(|data| raii::DepTask::new(&data.edges, key))
}
pub fn with_ignore<OP,R>(&self, op: OP) -> R
@ -112,19 +115,35 @@ impl DepGraph {
task(cx, arg)
}
#[inline]
pub fn read(&self, v: DepNode) {
if self.data.thread.is_enqueue_enabled() {
self.data.thread.enqueue(DepMessage::Read(v));
if let Some(ref data) = self.data {
data.edges.borrow_mut().read(v);
}
}
/// Only to be used during graph loading
#[inline]
pub fn add_edge_directly(&self, source: DepNode, target: DepNode) {
self.data.as_ref().unwrap().edges.borrow_mut().add_edge(source, target);
}
/// Only to be used during graph loading
pub fn add_node_directly(&self, node: DepNode) {
self.data.as_ref().unwrap().edges.borrow_mut().add_node(node);
}
/// Indicates that a previous work product exists for `v`. This is
/// invoked during initial start-up based on what nodes are clean
/// (and what files exist in the incr. directory).
pub fn insert_previous_work_product(&self, v: &WorkProductId, data: WorkProduct) {
debug!("insert_previous_work_product({:?}, {:?})", v, data);
self.data.previous_work_products.borrow_mut()
.insert(v.clone(), data);
self.data
.as_ref()
.unwrap()
.previous_work_products
.borrow_mut()
.insert(v.clone(), data);
}
/// Indicates that we created the given work-product in this run
@ -132,28 +151,34 @@ impl DepGraph {
/// run.
pub fn insert_work_product(&self, v: &WorkProductId, data: WorkProduct) {
debug!("insert_work_product({:?}, {:?})", v, data);
self.data.work_products.borrow_mut()
.insert(v.clone(), data);
self.data
.as_ref()
.unwrap()
.work_products
.borrow_mut()
.insert(v.clone(), data);
}
/// Check whether a previous work product exists for `v` and, if
/// so, return the path that leads to it. Used to skip doing work.
pub fn previous_work_product(&self, v: &WorkProductId) -> Option<WorkProduct> {
self.data.previous_work_products.borrow()
.get(v)
.cloned()
self.data
.as_ref()
.and_then(|data| {
data.previous_work_products.borrow().get(v).cloned()
})
}
/// Access the map of work-products created during this run. Only
/// used during saving of the dep-graph.
pub fn work_products(&self) -> Ref<FxHashMap<WorkProductId, WorkProduct>> {
self.data.work_products.borrow()
self.data.as_ref().unwrap().work_products.borrow()
}
/// Access the map of work-products created during the cached run. Only
/// used during saving of the dep-graph.
pub fn previous_work_products(&self) -> Ref<FxHashMap<WorkProductId, WorkProduct>> {
self.data.previous_work_products.borrow()
self.data.as_ref().unwrap().previous_work_products.borrow()
}
#[inline(always)]
@ -162,14 +187,14 @@ impl DepGraph {
debug_str_gen: F)
where F: FnOnce() -> String
{
let mut dep_node_debug = self.data.dep_node_debug.borrow_mut();
let mut dep_node_debug = self.data.as_ref().unwrap().dep_node_debug.borrow_mut();
dep_node_debug.entry(dep_node)
.or_insert_with(debug_str_gen);
}
pub(super) fn dep_node_debug_str(&self, dep_node: DepNode) -> Option<String> {
self.data.dep_node_debug.borrow().get(&dep_node).cloned()
self.data.as_ref().unwrap().dep_node_debug.borrow().get(&dep_node).cloned()
}
}

View File

@ -16,8 +16,6 @@ mod graph;
mod query;
mod raii;
mod safe;
mod shadow;
mod thread;
pub use self::dep_tracking_map::{DepTrackingMap, DepTrackingMapConfig};
pub use self::dep_node::DepNode;

View File

@ -9,53 +9,49 @@
// except according to those terms.
use super::DepNode;
use super::thread::{DepGraphThreadData, DepMessage};
use super::edges::DepGraphEdges;
use std::cell::RefCell;
pub struct DepTask<'graph> {
data: &'graph DepGraphThreadData,
key: Option<DepNode>,
graph: &'graph RefCell<DepGraphEdges>,
key: DepNode,
}
impl<'graph> DepTask<'graph> {
pub fn new(data: &'graph DepGraphThreadData, key: DepNode)
-> Option<DepTask<'graph>> {
if data.is_enqueue_enabled() {
data.enqueue(DepMessage::PushTask(key.clone()));
Some(DepTask { data: data, key: Some(key) })
} else {
None
pub fn new(graph: &'graph RefCell<DepGraphEdges>,
key: DepNode)
-> DepTask<'graph> {
graph.borrow_mut().push_task(key);
DepTask {
graph,
key,
}
}
}
impl<'graph> Drop for DepTask<'graph> {
fn drop(&mut self) {
if self.data.is_enqueue_enabled() {
self.data.enqueue(DepMessage::PopTask(self.key.take().unwrap()));
}
self.graph.borrow_mut().pop_task(self.key);
}
}
pub struct IgnoreTask<'graph> {
data: &'graph DepGraphThreadData
graph: &'graph RefCell<DepGraphEdges>,
}
impl<'graph> IgnoreTask<'graph> {
pub fn new(data: &'graph DepGraphThreadData) -> Option<IgnoreTask<'graph>> {
if data.is_enqueue_enabled() {
data.enqueue(DepMessage::PushIgnore);
Some(IgnoreTask { data: data })
} else {
None
pub fn new(graph: &'graph RefCell<DepGraphEdges>) -> IgnoreTask<'graph> {
graph.borrow_mut().push_ignore();
IgnoreTask {
graph
}
}
}
impl<'graph> Drop for IgnoreTask<'graph> {
fn drop(&mut self) {
if self.data.is_enqueue_enabled() {
self.data.enqueue(DepMessage::PopIgnore);
}
self.graph.borrow_mut().pop_ignore();
}
}

View File

@ -1,151 +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.
//! The "Shadow Graph" is maintained on the main thread and which
//! tracks each message relating to the dep-graph and applies some
//! sanity checks as they go by. If an error results, it means you get
//! a nice stack-trace telling you precisely what caused the error.
//!
//! NOTE: This is a debugging facility which can potentially have non-trivial
//! runtime impact. Therefore, it is largely compiled out if
//! debug-assertions are not enabled.
//!
//! The basic sanity check, enabled if you have debug assertions
//! enabled, is that there is always a task (or ignore) on the stack
//! when you do read/write, and that the tasks are pushed/popped
//! according to a proper stack discipline.
//!
//! Optionally, if you specify RUST_FORBID_DEP_GRAPH_EDGE, you can
//! specify an edge filter to be applied to each edge as it is
//! created. See `./README.md` for details.
use std::cell::RefCell;
use std::env;
use super::DepNode;
use super::thread::DepMessage;
use super::debug::EdgeFilter;
pub struct ShadowGraph {
// if you push None onto the stack, that corresponds to an Ignore
stack: RefCell<Vec<Option<DepNode>>>,
forbidden_edge: Option<EdgeFilter>,
}
const ENABLED: bool = cfg!(debug_assertions);
impl ShadowGraph {
pub fn new() -> Self {
let forbidden_edge = if !ENABLED {
None
} else {
match env::var("RUST_FORBID_DEP_GRAPH_EDGE") {
Ok(s) => {
match EdgeFilter::new(&s) {
Ok(f) => Some(f),
Err(err) => bug!("RUST_FORBID_DEP_GRAPH_EDGE invalid: {}", err),
}
}
Err(_) => None,
}
};
ShadowGraph {
stack: RefCell::new(vec![]),
forbidden_edge,
}
}
#[inline]
pub fn enabled(&self) -> bool {
ENABLED
}
pub fn enqueue(&self, message: &DepMessage) {
if ENABLED {
if self.stack.try_borrow().is_err() {
// When we apply edge filters, that invokes the Debug trait on
// DefIds, which in turn reads from various bits of state and
// creates reads! Ignore those recursive reads.
return;
}
let mut stack = self.stack.borrow_mut();
match *message {
// It is ok to READ shared state outside of a
// task. That can't do any harm (at least, the only
// way it can do harm is by leaking that data into a
// query or task, which would be a problem
// anyway). What would be bad is WRITING to that
// state.
DepMessage::Read(_) => { }
DepMessage::Write(ref n) => self.check_edge(top(&stack), Some(Some(n))),
DepMessage::PushTask(ref n) => stack.push(Some(n.clone())),
DepMessage::PushIgnore => stack.push(None),
DepMessage::PopTask(ref n) => {
match stack.pop() {
Some(Some(m)) => {
if *n != m {
bug!("stack mismatch: found {:?} expected {:?}", m, n)
}
}
Some(None) => bug!("stack mismatch: found Ignore expected {:?}", n),
None => bug!("stack mismatch: found empty stack, expected {:?}", n),
}
}
DepMessage::PopIgnore => {
match stack.pop() {
Some(Some(m)) => bug!("stack mismatch: found {:?} expected ignore", m),
Some(None) => (),
None => bug!("stack mismatch: found empty stack, expected ignore"),
}
}
DepMessage::Query => (),
}
}
}
fn check_edge(&self,
source: Option<Option<&DepNode>>,
target: Option<Option<&DepNode>>) {
assert!(ENABLED);
match (source, target) {
// cannot happen, one side is always Some(Some(_))
(None, None) => unreachable!(),
// nothing on top of the stack
(None, Some(n)) | (Some(n), None) => bug!("write of {:?} but no current task", n),
// this corresponds to an Ignore being top of the stack
(Some(None), _) | (_, Some(None)) => (),
// a task is on top of the stack
(Some(Some(source)), Some(Some(target))) => {
if let Some(ref forbidden_edge) = self.forbidden_edge {
if forbidden_edge.test(source, target) {
bug!("forbidden edge {:?} -> {:?} created", source, target)
}
}
}
}
}
}
// Do a little juggling: we get back a reference to an option at the
// top of the stack, convert it to an optional reference.
fn top<'s>(stack: &'s Vec<Option<DepNode>>) -> Option<Option<&'s DepNode>> {
stack.last()
.map(|n: &'s Option<DepNode>| -> Option<&'s DepNode> {
// (*)
// (*) type annotation just there to clarify what would
// otherwise be some *really* obscure code
n.as_ref()
})
}

View File

@ -1,178 +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.
//! Manages the communication between the compiler's main thread and
//! the thread that constructs the dependency graph. The basic idea is
//! to use double buffering to lower the cost of producing a message.
//! In the compiler thread, we accumulate messages in a vector until
//! the vector is full, or until we want to query the graph, and then
//! we send that vector over to the depgraph thread. At the same time,
//! we receive an empty vector from the depgraph thread that we can use
//! to accumulate more messages. This way we only ever have two vectors
//! allocated (and both have a fairly large capacity).
use rustc_data_structures::veccell::VecCell;
use std::sync::mpsc::{self, Sender, Receiver};
use std::thread;
use super::DepGraphQuery;
use super::DepNode;
use super::edges::DepGraphEdges;
use super::shadow::ShadowGraph;
#[derive(Debug)]
pub enum DepMessage {
Read(DepNode),
Write(DepNode),
PushTask(DepNode),
PopTask(DepNode),
PushIgnore,
PopIgnore,
Query,
}
pub struct DepGraphThreadData {
enabled: bool,
// The "shadow graph" is a debugging aid. We give it each message
// in real time as it arrives and it checks for various errors
// (for example, a read/write when there is no current task; it
// can also apply user-defined filters; see `shadow` module for
// details). This only occurs if debug-assertions are enabled.
//
// Note that in some cases the same errors will occur when the
// data is processed off the main thread, but that's annoying
// because it lacks precision about the source of the error.
shadow_graph: ShadowGraph,
// current buffer, where we accumulate messages
messages: VecCell<DepMessage>,
// where to receive new buffer when full
swap_in: Receiver<Vec<DepMessage>>,
// where to send buffer when full
swap_out: Sender<Vec<DepMessage>>,
// where to receive query results
query_in: Receiver<DepGraphQuery>,
}
const INITIAL_CAPACITY: usize = 2048;
impl DepGraphThreadData {
pub fn new(enabled: bool) -> DepGraphThreadData {
let (tx1, rx1) = mpsc::channel();
let (tx2, rx2) = mpsc::channel();
let (txq, rxq) = mpsc::channel();
if enabled {
thread::spawn(move || main(rx1, tx2, txq));
}
DepGraphThreadData {
enabled,
shadow_graph: ShadowGraph::new(),
messages: VecCell::with_capacity(INITIAL_CAPACITY),
swap_in: rx2,
swap_out: tx1,
query_in: rxq,
}
}
/// True if we are actually building the full dep-graph.
#[inline]
pub fn is_fully_enabled(&self) -> bool {
self.enabled
}
/// True if (a) we are actually building the full dep-graph, or (b) we are
/// only enqueuing messages in order to sanity-check them (which happens
/// when debug assertions are enabled).
#[inline]
pub fn is_enqueue_enabled(&self) -> bool {
self.is_fully_enabled() || self.shadow_graph.enabled()
}
/// Sends the current batch of messages to the thread. Installs a
/// new vector of messages.
fn swap(&self) {
assert!(self.is_fully_enabled(), "should never swap if not fully enabled");
// should be a buffer waiting for us (though of course we may
// have to wait for depgraph thread to finish processing the
// old messages)
let new_messages = self.swap_in.recv().unwrap();
assert!(new_messages.is_empty());
// swap in the empty buffer and extract the full one
let old_messages = self.messages.swap(new_messages);
// send full buffer to depgraph thread to be processed
self.swap_out.send(old_messages).unwrap();
}
pub fn query(&self) -> DepGraphQuery {
assert!(self.is_fully_enabled(), "should never query if not fully enabled");
self.enqueue(DepMessage::Query);
self.swap();
self.query_in.recv().unwrap()
}
/// Enqueue a message to be sent when things are next swapped. (If
/// the buffer is full, this may swap.)
#[inline]
pub fn enqueue(&self, message: DepMessage) {
assert!(self.is_enqueue_enabled(), "should never enqueue if not enqueue-enabled");
self.shadow_graph.enqueue(&message);
if self.is_fully_enabled() {
self.enqueue_enabled(message);
}
}
// Outline this fn since I expect it may want to be inlined
// separately.
fn enqueue_enabled(&self, message: DepMessage) {
let len = self.messages.push(message);
if len == INITIAL_CAPACITY {
self.swap();
}
}
}
/// Definition of the depgraph thread.
pub fn main(swap_in: Receiver<Vec<DepMessage>>,
swap_out: Sender<Vec<DepMessage>>,
query_out: Sender<DepGraphQuery>) {
let mut edges = DepGraphEdges::new();
// the compiler thread always expects a fresh buffer to be
// waiting, so queue one up
swap_out.send(Vec::with_capacity(INITIAL_CAPACITY)).unwrap();
// process the buffers from compiler thread as we receive them
for mut messages in swap_in {
for msg in messages.drain(..) {
match msg {
DepMessage::Read(node) => edges.read(node),
DepMessage::Write(node) => edges.write(node),
DepMessage::PushTask(node) => edges.push_task(node),
DepMessage::PopTask(node) => edges.pop_task(node),
DepMessage::PushIgnore => edges.push_ignore(),
DepMessage::PopIgnore => edges.pop_ignore(),
DepMessage::Query => query_out.send(edges.query()).unwrap(),
}
}
if let Err(_) = swap_out.send(messages) {
// the receiver must have been dropped already
break;
}
}
}

View File

@ -202,11 +202,7 @@ pub fn decode_dep_graph<'a, 'tcx>(tcx: TyCtxt<'a, 'tcx, 'tcx>,
clean_work_products.insert(wp_id);
}
tcx.dep_graph.with_task(*bootstrap_output, (), (), create_node);
fn create_node((): (), (): ()) {
// just create the node with no inputs
}
tcx.dep_graph.add_node_directly(*bootstrap_output);
}
// Add in work-products that are still clean, and delete those that are
@ -453,8 +449,7 @@ fn process_edge<'a, 'tcx, 'edges>(
if !dirty_raw_nodes.contains_key(&target) {
let target = nodes[target];
let source = nodes[source];
let _task = tcx.dep_graph.in_task(target);
tcx.dep_graph.read(source);
tcx.dep_graph.add_edge_directly(source, target);
if let DepKind::WorkProduct = target.kind {
let wp_id = WorkProductId::from_fingerprint(target.hash);
@ -462,4 +457,3 @@ fn process_edge<'a, 'tcx, 'edges>(
}
}
}