From 79768eb0d9496ba2156a839c3a1c0758e51fd5a8 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Tue, 29 Jul 2014 10:20:42 -0700 Subject: [PATCH] Prepare for per-target fingerprints This commit refactors all related infrastructure for the cargo_rustc module to move the granularity of fingerprints from packages to targets. This involved heavily modifying the `JobQueue` structure to understand a finer-grained target for dirtiness propagation and dependency management, and then dealing with fallout in the main module. The fingerprint module has been refactored to support a per-target fingerprint. A fallout of this change is that each build step has its own fingerprint, including the custom build command step. This will be implemented in a future commit. As fallout of this reorganization, we are now exploiting the maximal parallelism within packages themselves whereas before we were only partially parallelizing. No new features were added as part of this commit, so I just ensured that all the tests ran. --- src/cargo/core/manifest.rs | 4 +- src/cargo/ops/cargo_rustc/context.rs | 21 +- src/cargo/ops/cargo_rustc/fingerprint.rs | 243 +++++++++++++++-------- src/cargo/ops/cargo_rustc/job.rs | 59 ++---- src/cargo/ops/cargo_rustc/job_queue.rs | 200 +++++++++++++------ src/cargo/ops/cargo_rustc/layout.rs | 19 ++ src/cargo/ops/cargo_rustc/mod.rs | 180 +++++++++-------- src/cargo/util/dependency_queue.rs | 35 ++-- src/cargo/util/toml.rs | 6 + tests/test_cargo_compile.rs | 2 +- tests/test_cargo_test.rs | 3 +- 11 files changed, 463 insertions(+), 309 deletions(-) diff --git a/src/cargo/core/manifest.rs b/src/cargo/core/manifest.rs index afe616848ef..3751f6958aa 100644 --- a/src/cargo/core/manifest.rs +++ b/src/cargo/core/manifest.rs @@ -344,13 +344,13 @@ impl Target { pub fn lib_target(name: &str, crate_targets: Vec, src_path: &Path, profile: &Profile, - metadata: &Metadata) -> Target { + metadata: Metadata) -> Target { Target { kind: LibTarget(crate_targets), name: name.to_string(), src_path: src_path.clone(), profile: profile.clone(), - metadata: Some(metadata.clone()) + metadata: Some(metadata) } } diff --git a/src/cargo/ops/cargo_rustc/context.rs b/src/cargo/ops/cargo_rustc/context.rs index 6e65574ac07..472ca348495 100644 --- a/src/cargo/ops/cargo_rustc/context.rs +++ b/src/cargo/ops/cargo_rustc/context.rs @@ -5,6 +5,7 @@ use core::{SourceMap, Package, PackageId, PackageSet, Resolve, Target}; use util; use util::{CargoResult, ChainError, internal, Config, profile}; +use super::{Kind, KindPlugin, KindTarget}; use super::layout::{Layout, LayoutProxy}; #[deriving(Show)] @@ -155,12 +156,12 @@ impl<'a, 'b> Context<'a, 'b> { } /// Returns the appropriate directory layout for either a plugin or not. - pub fn layout(&self, plugin: bool) -> LayoutProxy { - if plugin { - LayoutProxy::new(&self.host, self.primary) - } else { - LayoutProxy::new(self.target.as_ref().unwrap_or(&self.host), - self.primary) + pub fn layout(&self, kind: Kind) -> LayoutProxy { + match kind { + KindPlugin => LayoutProxy::new(&self.host, self.primary), + KindTarget => LayoutProxy::new(self.target.as_ref() + .unwrap_or(&self.host), + self.primary) } } @@ -168,8 +169,8 @@ impl<'a, 'b> Context<'a, 'b> { /// /// If `plugin` is true, the pair corresponds to the host platform, /// otherwise it corresponds to the target platform. - fn dylib(&self, plugin: bool) -> (&str, &str) { - let pair = if plugin {&self.host_dylib} else {&self.target_dylib}; + fn dylib(&self, kind: Kind) -> (&str, &str) { + let pair = if kind == KindPlugin {&self.host_dylib} else {&self.target_dylib}; (pair.ref0().as_slice(), pair.ref1().as_slice()) } @@ -182,7 +183,9 @@ impl<'a, 'b> Context<'a, 'b> { ret.push(format!("{}{}", stem, self.target_exe)); } else { if target.is_dylib() { - let (prefix, suffix) = self.dylib(target.get_profile().is_plugin()); + let plugin = target.get_profile().is_plugin(); + let kind = if plugin {KindPlugin} else {KindTarget}; + let (prefix, suffix) = self.dylib(kind); ret.push(format!("{}{}{}", prefix, stem, suffix)); } if target.is_rlib() { diff --git a/src/cargo/ops/cargo_rustc/fingerprint.rs b/src/cargo/ops/cargo_rustc/fingerprint.rs index 3b4fd04dd8d..e41ca39872a 100644 --- a/src/cargo/ops/cargo_rustc/fingerprint.rs +++ b/src/cargo/ops/cargo_rustc/fingerprint.rs @@ -1,100 +1,164 @@ -use std::hash::Hasher; +use std::hash::{Hash, Hasher}; use std::hash::sip::SipHasher; -use std::io::{fs, File}; +use std::io::{fs, File, UserRWX}; use core::{Package, Target}; use util; use util::hex::short_hash; use util::{CargoResult, Fresh, Dirty, Freshness, internal, Require, profile}; -use super::job::Job; +use super::{Kind, KindTarget}; +use super::job::Work; use super::context::Context; -/// Calculates the fingerprint of a package's targets and prepares to write a -/// new fingerprint. +/// A tuple result of the `prepare_foo` functions in this module. /// -/// This function will first calculate the freshness of the package and return -/// it as the first part of the return tuple. It will then prepare a job to -/// update the fingerprint if this package is actually rebuilt as part of -/// compilation, returning the job as the second part of the tuple. +/// The first element of the triple is whether the target in question is +/// currently fresh or not, and the second two elements are work to perform when +/// the target is dirty or fresh, respectively. /// -/// The third part of the tuple is a job to run when a package is discovered to -/// be fresh to ensure that all of its artifacts are moved to the correct -/// location. -pub fn prepare<'a, 'b>(cx: &mut Context<'a, 'b>, pkg: &'a Package, - targets: &[&'a Target]) - -> CargoResult<(Freshness, Job, Job)> { - let _p = profile::start(format!("fingerprint: {}", pkg)); - let filename = format!(".{}.{}.fingerprint", pkg.get_name(), - short_hash(pkg.get_package_id())); - let filename = filename.as_slice(); - let (old_fingerprint_loc, new_fingerprint_loc) = { - let layout = cx.layout(false); - (layout.old_root().join(filename), layout.root().join(filename)) +/// Both units of work are always generated because a fresh package may still be +/// rebuilt if some upstream dependency changes. +pub type Preparation = (Freshness, Work, Work); + +/// Prepare the necessary work for the fingerprint for a specific target. +/// +/// When dealing with fingerprints, cargo gets to choose what granularity +/// "freshness" is considered at. One option is considering freshness at the +/// package level. This means that if anything in a package changes, the entire +/// package is rebuilt, unconditionally. This simplicity comes at a cost, +/// however, in that test-only changes will cause libraries to be rebuilt, which +/// is quite unfortunate! +/// +/// The cost was deemed high enough that fingerprints are now calculated at the +/// layer of a target rather than a package. Each target can then be kept track +/// of separately and only rebuilt as necessary. This requires cargo to +/// understand what the inputs are to a target, so we drive rustc with the +/// --dep-info flag to learn about all input files to a unit of compilation. +/// +/// This function will calculate the fingerprint for a target and prepare the +/// work necessary to either write the fingerprint or copy over all fresh files +/// from the old directories to their new locations. +pub fn prepare_target(cx: &mut Context, pkg: &Package, target: &Target, + kind: Kind) -> CargoResult { + let _p = profile::start(format!("fingerprint: {} / {}", + pkg.get_package_id(), target)); + let (old, new) = dirs(cx, pkg, kind); + let filename = if target.is_lib() { + format!("lib-{}", target.get_name()) + } else if target.get_profile().is_doc() { + format!("doc-{}", target.get_name()) + } else { + format!("bin-{}", target.get_name()) }; + let old_loc = old.join(filename.as_slice()); + let new_loc = new.join(filename.as_slice()); - // First, figure out if the old location exists, and if it does whether it's - // still fresh or not. - let (is_fresh, fingerprint) = try!(is_fresh(pkg, &old_fingerprint_loc, - cx, targets)); - - // Prepare a job to update the location of the new fingerprint. - let new_fingerprint_loc2 = new_fingerprint_loc.clone(); - let write_fingerprint = Job::new(proc() { - let mut f = try!(File::create(&new_fingerprint_loc2)); - try!(f.write_str(fingerprint.as_slice())); - Ok(Vec::new()) - }); - - // Prepare a job to copy over all old artifacts into their new destination. - let mut pairs = Vec::new(); - pairs.push((old_fingerprint_loc, new_fingerprint_loc)); - - // TODO: this shouldn't explicitly pass false, for more info see - // cargo_rustc::compile_custom - if pkg.get_manifest().get_build().len() > 0 { - let layout = cx.layout(false); - pairs.push((layout.old_native(pkg), layout.native(pkg))); - } + let new_fingerprint = try!(calculate_target_fingerprint(cx, pkg, target)); + let new_fingerprint = mk_fingerprint(cx, &new_fingerprint); - for &target in targets.iter() { - if target.get_profile().is_doc() { continue } - let target_layout = cx.layout(false); - let plugin_layout = cx.layout(true); - let req = cx.get_requirement(pkg, target); + let is_fresh = try!(is_fresh(&old_loc, new_fingerprint.as_slice())); + let layout = cx.layout(kind); + let mut pairs = vec![(old_loc, new_loc.clone())]; - for filename in cx.target_filenames(target).iter() { + if !target.get_profile().is_doc() { + pairs.extend(cx.target_filenames(target).iter().map(|filename| { let filename = filename.as_slice(); - if req.is_target() { - pairs.push((target_layout.old_root().join(filename), - target_layout.root().join(filename))); - } - if req.is_plugin() && plugin_layout.root() != target_layout.root() { - pairs.push((plugin_layout.old_root().join(filename), - plugin_layout.root().join(filename))); - } - } + ((layout.old_root().join(filename), layout.root().join(filename))) + })); } - let move_old = Job::new(proc() { - for &(ref src, ref dst) in pairs.iter() { + + Ok(prepare(is_fresh, new_loc, new_fingerprint, pairs)) +} + +/// Prepare the necessary work for the fingerprint of a build command. +/// +/// Build commands are located on packages, not on targets. Additionally, we +/// don't have --dep-info to drive calculation of the fingerprint of a build +/// command. This brings up an interesting predicament which gives us a few +/// options to figure out whether a build command is dirty or not: +/// +/// 1. A build command is dirty if *any* file in a package changes. In theory +/// all files are candidate for being used by the build command. +/// 2. A build command is dirty if any file in a *specific directory* changes. +/// This may lose information as it may require files outside of the specific +/// directory. +/// 3. A build command must itself provide a dep-info-like file stating how it +/// should be considered dirty or not. +/// +/// The currently implemented solution is option (1), although it is planned to +/// migrate to option (2) in the near future. +pub fn prepare_build_cmd(cx: &mut Context, pkg: &Package) + -> CargoResult { + let _p = profile::start(format!("fingerprint build cmd: {}", + pkg.get_package_id())); + + // TODO: this should not explicitly pass KindTarget + let kind = KindTarget; + + if pkg.get_manifest().get_build().len() == 0 { + return Ok((Fresh, proc() Ok(()), proc() Ok(()))) + } + let (old, new) = dirs(cx, pkg, kind); + let old_loc = old.join("build"); + let new_loc = new.join("build"); + + let new_fingerprint = try!(calculate_build_cmd_fingerprint(cx, pkg)); + let new_fingerprint = mk_fingerprint(cx, &new_fingerprint); + + let is_fresh = try!(is_fresh(&old_loc, new_fingerprint.as_slice())); + let layout = cx.layout(kind); + let pairs = vec![(old_loc, new_loc.clone()), + (layout.old_native(pkg), layout.native(pkg))]; + + Ok(prepare(is_fresh, new_loc, new_fingerprint, pairs)) +} + +/// Prepare work for when a package starts to build +pub fn prepare_init(cx: &mut Context, pkg: &Package, kind: Kind) + -> (Work, Work) { + let (_, new1) = dirs(cx, pkg, kind); + let new2 = new1.clone(); + + let work1 = proc() { try!(fs::mkdir(&new1, UserRWX)); Ok(()) }; + let work2 = proc() { try!(fs::mkdir(&new2, UserRWX)); Ok(()) }; + + (work1, work2) +} + +/// Given the data to build and write a fingerprint, generate some Work +/// instances to actually perform the necessary work. +fn prepare(is_fresh: bool, loc: Path, fingerprint: String, + to_copy: Vec<(Path, Path)>) -> Preparation { + let write_fingerprint = proc() { + try!(File::create(&loc).write_str(fingerprint.as_slice())); + Ok(()) + }; + + let move_old = proc() { + for &(ref src, ref dst) in to_copy.iter() { try!(fs::rename(src, dst)); } - Ok(Vec::new()) - }); + Ok(()) + }; - Ok((if is_fresh {Fresh} else {Dirty}, write_fingerprint, move_old)) + (if is_fresh {Fresh} else {Dirty}, write_fingerprint, move_old) } -fn is_fresh(dep: &Package, loc: &Path, cx: &mut Context, targets: &[&Target]) - -> CargoResult<(bool, String)> { - let dep_fingerprint = try!(get_fingerprint(dep, cx)); - let new_pkg_fingerprint = format!("{}{}", cx.rustc_version, dep_fingerprint); - - let new_fingerprint = fingerprint(new_pkg_fingerprint, hash_targets(targets)); +/// Return the (old, new) location for fingerprints for a package +pub fn dirs(cx: &mut Context, pkg: &Package, kind: Kind) -> (Path, Path) { + let dirname = format!("{}-{}", pkg.get_name(), + short_hash(pkg.get_package_id())); + let dirname = dirname.as_slice(); + let layout = cx.layout(kind); + let layout = layout.proxy(); + (layout.old_fingerprint().join(dirname), layout.fingerprint().join(dirname)) +} +fn is_fresh(loc: &Path, new_fingerprint: &str) -> CargoResult { let mut file = match File::open(loc) { Ok(file) => file, - Err(..) => return Ok((false, new_fingerprint)), + Err(..) => return Ok(false), }; let old_fingerprint = try!(file.read_to_string()); @@ -102,24 +166,31 @@ fn is_fresh(dep: &Package, loc: &Path, cx: &mut Context, targets: &[&Target]) log!(5, "old fingerprint: {}", old_fingerprint); log!(5, "new fingerprint: {}", new_fingerprint); - Ok((old_fingerprint == new_fingerprint, new_fingerprint)) + Ok(old_fingerprint.as_slice() == new_fingerprint) } -fn get_fingerprint(pkg: &Package, cx: &Context) -> CargoResult { - let id = pkg.get_package_id().get_source_id(); - let source = try!(cx.sources.get(id).require(|| { - internal(format!("Missing package source for: {}", id)) - })); - source.fingerprint(pkg) +/// Frob in the necessary data from the context to generate the real +/// fingerprint. +fn mk_fingerprint(cx: &Context, data: &T) -> String { + let hasher = SipHasher::new_with_keys(0,0); + util::to_hex(hasher.hash(&(&cx.rustc_version, data))) } -fn hash_targets(targets: &[&Target]) -> u64 { - let hasher = SipHasher::new_with_keys(0,0); - let targets = targets.iter().map(|t| (*t).clone()).collect::>(); - hasher.hash(&targets) +fn calculate_target_fingerprint(cx: &Context, pkg: &Package, target: &Target) + -> CargoResult { + let source = cx.sources + .get(pkg.get_package_id().get_source_id()) + .expect("BUG: Missing package source"); + + let pkg_fingerprint = try!(source.fingerprint(pkg)); + Ok(pkg_fingerprint + short_hash(target)) } -fn fingerprint(package: String, profiles: u64) -> String { - let hasher = SipHasher::new_with_keys(0,0); - util::to_hex(hasher.hash(&(package, profiles))) +fn calculate_build_cmd_fingerprint(cx: &Context, pkg: &Package) + -> CargoResult { + let source = cx.sources + .get(pkg.get_package_id().get_source_id()) + .expect("BUG: Missing package source"); + + source.fingerprint(pkg) } diff --git a/src/cargo/ops/cargo_rustc/job.rs b/src/cargo/ops/cargo_rustc/job.rs index 613ad0dd1e4..fa11bb0954f 100644 --- a/src/cargo/ops/cargo_rustc/job.rs +++ b/src/cargo/ops/cargo_rustc/job.rs @@ -1,57 +1,22 @@ -use util::CargoResult; -use std::sync::{Arc, Mutex}; +use util::{CargoResult, Fresh, Dirty, Freshness}; -pub struct Job { - work: proc():Send -> CargoResult>, -} +pub struct Job { dirty: Work, fresh: Work } + +pub type Work = proc():Send -> CargoResult<()>; impl Job { /// Create a new job representing a unit of work. - pub fn new(work: proc():Send -> CargoResult>) -> Job { - Job { work: work } - } - - /// Creates a new job which will execute all of `jobs` and then return the - /// work `after` if they all succeed sequentially. - pub fn all(jobs: Vec, after: Vec) -> Job { - Job::new(proc() { - for job in jobs.move_iter() { - try!(job.run()); - } - Ok(after) - }) - } - - /// Maps a list of jobs to a new list of jobs which will run `after` once - /// all the jobs have completed. - pub fn after(jobs: Vec, after: Job) -> Vec { - if jobs.len() == 0 { return vec![after] } - - struct State { job: Option, remaining: uint } - - let lock = Arc::new(Mutex::new(State { - job: Some(after), - remaining: jobs.len(), - })); - - jobs.move_iter().map(|job| { - let my_lock = lock.clone(); - Job::new(proc() { - try!(job.run()); - let mut state = my_lock.lock(); - state.remaining -= 1; - Ok(if state.remaining == 0 { - vec![state.job.take().unwrap()] - } else { - Vec::new() - }) - }) - }).collect() + pub fn new(dirty: proc():Send -> CargoResult<()>, + fresh: proc():Send -> CargoResult<()>) -> Job { + Job { dirty: dirty, fresh: fresh } } /// Consumes this job by running it, returning the result of the /// computation. - pub fn run(self) -> CargoResult> { - (self.work)() + pub fn run(self, fresh: Freshness) -> CargoResult<()> { + match fresh { + Fresh => (self.fresh)(), + Dirty => (self.dirty)(), + } } } diff --git a/src/cargo/ops/cargo_rustc/job_queue.rs b/src/cargo/ops/cargo_rustc/job_queue.rs index 1c09b4dcb14..270277214cb 100644 --- a/src/cargo/ops/cargo_rustc/job_queue.rs +++ b/src/cargo/ops/cargo_rustc/job_queue.rs @@ -1,5 +1,4 @@ use std::collections::HashMap; -use std::iter::AdditiveIterator; use term::color::YELLOW; use core::{Package, PackageId, Resolve}; @@ -8,47 +7,87 @@ use util::{CargoResult, Dependency, profile}; use super::job::Job; +/// A management structure of the entire dependency graph to compile. +/// +/// This structure is backed by the `DependencyQueue` type and manages the +/// actual compilation step of each package. Packages enqueue units of work and +/// then later on the entire graph is processed and compiled. pub struct JobQueue<'a, 'b> { pool: TaskPool, - queue: DependencyQueue<&'a PackageId, (&'a Package, (Job, Job))>, + queue: DependencyQueue<(&'a PackageId, TargetStage), + (&'a Package, Vec<(Job, Freshness)>)>, tx: Sender, rx: Receiver, - active: HashMap<&'a PackageId, uint>, - config: &'b mut Config<'b>, + resolve: &'a Resolve, + active: uint, + pending: HashMap<(&'a PackageId, TargetStage), PendingBuild>, + state: HashMap<&'a PackageId, Freshness>, } -type Message = (PackageId, CargoResult>); +/// A helper structure for metadata about the state of a building package. +struct PendingBuild { + /// Number of jobs currently active + amt: uint, + /// Current freshness state of this package. Any dirty target within a + /// package will cause the entire package to become dirty. + fresh: Freshness, +} + +/// Current stage of compilation for an individual package. +/// +/// This is the second layer of keys on the dependency queue to track the state +/// of where a particular package is in the compilation pipeline. Each of these +/// stages has a network of dependencies among them, outlined by the +/// `Dependency` implementation found below. +/// +/// Each build step for a package is registered with one of these stages, and +/// each stage has a vector of work to perform in parallel. +#[deriving(Hash, PartialEq, Eq, Clone, PartialOrd, Ord, Show)] +pub enum TargetStage { + StageStart, + StageCustomBuild, + StageLibraries, + StageBinaries, + StageEnd, +} + +type Message = (PackageId, TargetStage, Freshness, CargoResult<()>); impl<'a, 'b> JobQueue<'a, 'b> { - pub fn new(config: &'b mut Config<'b>, - resolve: &'a Resolve, - jobs: Vec<(&'a Package, Freshness, (Job, Job))>) - -> JobQueue<'a, 'b> { + pub fn new(resolve: &'a Resolve, config: &mut Config) -> JobQueue<'a, 'b> { let (tx, rx) = channel(); - let mut queue = DependencyQueue::new(); - for &(pkg, _, _) in jobs.iter() { - queue.register(pkg.get_package_id()); - } - for (pkg, fresh, job) in jobs.move_iter() { - queue.enqueue(&resolve, fresh, pkg.get_package_id(), (pkg, job)); - } - JobQueue { pool: TaskPool::new(config.jobs()), - queue: queue, + queue: DependencyQueue::new(), tx: tx, rx: rx, - active: HashMap::new(), - config: config, + resolve: resolve, + active: 0, + pending: HashMap::new(), + state: HashMap::new(), } } + pub fn enqueue(&mut self, pkg: &'a Package, stage: TargetStage, + jobs: Vec<(Job, Freshness)>) { + // Record the freshness state of this package as dirty if any job is + // dirty or fresh otherwise + let fresh = jobs.iter().fold(Fresh, |f1, &(_, f2)| f1.combine(f2)); + let prev = self.state.find_or_insert(pkg.get_package_id(), fresh); + *prev = prev.combine(fresh); + + // Add the package to the dependency graph + self.queue.enqueue(&self.resolve, Fresh, + (pkg.get_package_id(), stage), + (pkg, jobs)); + } + /// Execute all jobs necessary to build the dependency graph. /// /// This function will spawn off `config.jobs()` workers to build all of the /// necessary dependencies, in order. Freshness is propagated as far as /// possible along each dependency chain. - pub fn execute(&mut self) -> CargoResult<()> { + pub fn execute(&mut self, config: &mut Config) -> CargoResult<()> { let _p = profile::start("executing the job graph"); // Iteratively execute the dependency graph. Each turn of this loop will @@ -57,18 +96,8 @@ impl<'a, 'b> JobQueue<'a, 'b> { while self.queue.len() > 0 { loop { match self.queue.dequeue() { - Some((Fresh, id, (pkg, (_, fresh)))) => { - assert!(self.active.insert(id, 1u)); - try!(self.config.shell().status("Fresh", pkg)); - self.tx.send((id.clone(), Ok(Vec::new()))); - try!(fresh.run()); - } - Some((Dirty, id, (pkg, (dirty, _)))) => { - assert!(self.active.insert(id, 1)); - try!(self.config.shell().status("Compiling", pkg)); - let my_tx = self.tx.clone(); - let id = id.clone(); - self.pool.execute(proc() my_tx.send((id, dirty.run()))); + Some((fresh, (_, stage), (pkg, jobs))) => { + try!(self.run(pkg, stage, fresh, jobs, config)); } None => break, } @@ -77,35 +106,24 @@ impl<'a, 'b> JobQueue<'a, 'b> { // Now that all possible work has been scheduled, wait for a piece // of work to finish. If any package fails to build then we stop // scheduling work as quickly as possibly. - let (id, result) = self.rx.recv(); - let id = self.active.iter().map(|(&k, _)| k).find(|&k| k == &id) - .unwrap(); - *self.active.get_mut(&id) -= 1; + let (id, stage, fresh, result) = self.rx.recv(); + let id = *self.state.keys().find(|&k| *k == &id).unwrap(); + self.active -= 1; match result { - Ok(v) => { - for job in v.move_iter() { - *self.active.get_mut(&id) += 1; - let my_tx = self.tx.clone(); - let my_id = id.clone(); - self.pool.execute(proc() { - my_tx.send((my_id, job.run())); - }); - } - if *self.active.get(&id) == 0 { - self.active.remove(&id); - self.queue.finish(&id); + Ok(()) => { + let state = self.pending.get_mut(&(id, stage)); + state.amt -= 1; + state.fresh = state.fresh.combine(fresh); + if state.amt == 0 { + self.queue.finish(&(id, stage), state.fresh); } } Err(e) => { - if *self.active.get(&id) == 0 { - self.active.remove(&id); - } - if self.active.len() > 0 && self.config.jobs() > 1 { - try!(self.config.shell().say( + if self.active > 0 { + try!(config.shell().say( "Build failed, waiting for other \ jobs to finish...", YELLOW)); - let amt = self.active.iter().map(|(_, v)| *v).sum(); - for _ in self.rx.iter().take(amt) {} + for _ in self.rx.iter().take(self.active) {} } return Err(e) } @@ -116,10 +134,74 @@ impl<'a, 'b> JobQueue<'a, 'b> { Ok(()) } + + /// Execute a stage of compilation for a package. + /// + /// The input freshness is from `dequeue()` and indicates the combined + /// freshness of all upstream dependencies. This function will schedule all + /// work in `jobs` to be executed. + fn run(&mut self, pkg: &'a Package, stage: TargetStage, fresh: Freshness, + jobs: Vec<(Job, Freshness)>, config: &mut Config) -> CargoResult<()> { + let njobs = jobs.len(); + let amt = if njobs == 0 {1} else {njobs}; + let id = pkg.get_package_id().clone(); + + if stage == StageStart { + let fresh = fresh.combine(*self.state.get(&pkg.get_package_id())); + let msg = match fresh { Fresh => "Fresh", Dirty => "Compiling" }; + try!(config.shell().status(msg, pkg)); + } + + // While the jobs are all running, we maintain some metadata about how + // many are running, the current state of freshness (of all the combined + // jobs), and the stage to pass to finish() later on. + self.active += amt; + self.pending.insert((pkg.get_package_id(), stage), PendingBuild { + amt: amt, + fresh: fresh, + }); + + for (job, job_freshness) in jobs.move_iter() { + let fresh = job_freshness.combine(fresh); + let my_tx = self.tx.clone(); + let id = id.clone(); + self.pool.execute(proc() { + my_tx.send((id, stage, fresh, job.run(fresh))); + }); + } + + // If no work was scheduled, make sure that a message is actually send + // on this channel. + if njobs == 0 { + self.tx.send((id, stage, fresh, Ok(()))); + } + Ok(()) + } } -impl<'a> Dependency<&'a PackageId, &'a Resolve> for &'a PackageId { - fn dependencies(&self, resolve: &&'a Resolve) -> Vec<&'a PackageId> { - resolve.deps(*self).move_iter().flat_map(|a| a).collect() +impl<'a> Dependency<&'a Resolve> for (&'a PackageId, TargetStage) { + fn dependencies(&self, resolve: &&'a Resolve) + -> Vec<(&'a PackageId, TargetStage)> { + // This implementation of `Dependency` is the driver for the structure + // of the dependency graph of packages to be built. The "key" here is + // a pair of the package being built and the stage that it's at. + // + // Each stage here lists dependencies on the previous stages except for + // the start state which depends on the ending state of all dependent + // packages (as determined by the resolve context). + let (id, stage) = *self; + match stage { + StageStart => { + resolve.deps(id).move_iter().flat_map(|a| a).filter(|dep| { + *dep != id + }).map(|dep| { + (dep, StageEnd) + }).collect() + } + StageCustomBuild => vec![(id, StageStart)], + StageLibraries => vec![(id, StageCustomBuild)], + StageBinaries => vec![(id, StageLibraries)], + StageEnd => vec![(id, StageBinaries), (id, StageLibraries)], + } } } diff --git a/src/cargo/ops/cargo_rustc/layout.rs b/src/cargo/ops/cargo_rustc/layout.rs index 8c06656ba6d..b02fb0744a9 100644 --- a/src/cargo/ops/cargo_rustc/layout.rs +++ b/src/cargo/ops/cargo_rustc/layout.rs @@ -22,6 +22,10 @@ //! $pkg2/ //! $pkg3/ //! +//! # Hidden directory that holds all of the fingerprint files for all +//! # packages +//! .fingerprint/ +//! //! # This is a temporary directory as part of the build process. When a //! # build starts, it initially moves the old `deps` directory to this //! # location. This is done to ensure that there are no stale artifacts @@ -38,6 +42,7 @@ //! //! # Same as the two above old directories //! old-native/ +//! old-fingerprint/ use std::io; use std::io::{fs, IoResult}; @@ -49,10 +54,12 @@ pub struct Layout { root: Path, deps: Path, native: Path, + fingerprint: Path, old_deps: Path, old_root: Path, old_native: Path, + old_fingerprint: Path, } pub struct LayoutProxy<'a> { @@ -65,9 +72,11 @@ impl Layout { Layout { deps: root.join("deps"), native: root.join("native"), + fingerprint: root.join(".fingerprint"), old_deps: root.join("old-deps"), old_root: root.join("old-root"), old_native: root.join("old-native"), + old_fingerprint: root.join("old-fingerprint"), root: root, } } @@ -86,15 +95,22 @@ impl Layout { if self.old_native.exists() { try!(fs::rmdir_recursive(&self.old_native)); } + if self.old_fingerprint.exists() { + try!(fs::rmdir_recursive(&self.old_fingerprint)); + } if self.deps.exists() { try!(fs::rename(&self.deps, &self.old_deps)); } if self.native.exists() { try!(fs::rename(&self.native, &self.old_native)); } + if self.fingerprint.exists() { + try!(fs::rename(&self.fingerprint, &self.old_fingerprint)); + } try!(fs::mkdir(&self.deps, io::UserRWX)); try!(fs::mkdir(&self.native, io::UserRWX)); + try!(fs::mkdir(&self.fingerprint, io::UserRWX)); try!(fs::mkdir(&self.old_root, io::UserRWX)); for file in try!(fs::readdir(&self.root)).iter() { @@ -111,12 +127,14 @@ impl Layout { pub fn native(&self, package: &Package) -> Path { self.native.join(self.native_name(package)) } + pub fn fingerprint(&self) -> &Path { &self.fingerprint } pub fn old_dest<'a>(&'a self) -> &'a Path { &self.old_root } pub fn old_deps<'a>(&'a self) -> &'a Path { &self.old_deps } pub fn old_native(&self, package: &Package) -> Path { self.old_native.join(self.native_name(package)) } + pub fn old_fingerprint(&self) -> &Path { &self.old_fingerprint } fn native_name(&self, pkg: &Package) -> String { format!("{}-{}", pkg.get_name(), short_hash(pkg.get_package_id())) @@ -128,6 +146,7 @@ impl Drop for Layout { let _ = fs::rmdir_recursive(&self.old_deps); let _ = fs::rmdir_recursive(&self.old_root); let _ = fs::rmdir_recursive(&self.old_native); + let _ = fs::rmdir_recursive(&self.old_fingerprint); } } diff --git a/src/cargo/ops/cargo_rustc/mod.rs b/src/cargo/ops/cargo_rustc/mod.rs index 1f5d6890974..b535de7dd0a 100644 --- a/src/cargo/ops/cargo_rustc/mod.rs +++ b/src/cargo/ops/cargo_rustc/mod.rs @@ -7,10 +7,11 @@ use semver::Version; use core::{SourceMap, Package, PackageId, PackageSet, Target, Resolve}; use util; use util::{CargoResult, ProcessBuilder, CargoError, human, caused_human}; -use util::{Config, Freshness, internal, ChainError, profile}; +use util::{Config, internal, ChainError, Fresh, profile}; -use self::job::Job; -use self::job_queue::JobQueue; +use self::job::{Job, Work}; +use self::job_queue::{JobQueue, StageStart, StageCustomBuild, StageLibraries}; +use self::job_queue::{StageBinaries, StageEnd}; use self::context::{Context, PlatformRequirement, Target, Plugin, PluginAndTarget}; mod context; @@ -19,6 +20,9 @@ mod job; mod job_queue; mod layout; +#[deriving(PartialEq, Eq)] +enum Kind { KindPlugin, KindTarget } + // This is a temporary assert that ensures the consistency of the arguments // given the current limitations of Cargo. The long term fix is to have each // Target know the absolute path to the build location. @@ -55,6 +59,7 @@ pub fn compile_targets<'a>(env: &str, targets: &[&'a Target], pkg: &'a Package, let mut cx = try!(Context::new(env, resolve, sources, deps, config, host_layout, target_layout)); + let mut queue = JobQueue::new(cx.resolve, cx.config); // First ensure that the destination directory exists try!(cx.prepare(pkg)); @@ -63,29 +68,27 @@ pub fn compile_targets<'a>(env: &str, targets: &[&'a Target], pkg: &'a Package, // particular package. No actual work is executed as part of this, that's // all done later as part of the `execute` function which will run // everything in order with proper parallelism. - let mut jobs = Vec::new(); for dep in deps.iter() { - if dep == pkg { continue; } + if dep == pkg { continue } // Only compile lib targets for dependencies let targets = dep.get_targets().iter().filter(|target| { cx.is_relevant_target(*target) }).collect::>(); - try!(compile(targets.as_slice(), dep, &mut cx, &mut jobs)); + try!(compile(targets.as_slice(), dep, &mut cx, &mut queue)); } cx.primary(); - try!(compile(targets, pkg, &mut cx, &mut jobs)); + try!(compile(targets, pkg, &mut cx, &mut queue)); // Now that we've figured out everything that we're going to do, do it! - JobQueue::new(cx.config, cx.resolve, jobs).execute() + queue.execute(cx.config) } fn compile<'a, 'b>(targets: &[&'a Target], pkg: &'a Package, cx: &mut Context<'a, 'b>, - jobs: &mut Vec<(&'a Package, Freshness, (Job, Job))>) - -> CargoResult<()> { + jobs: &mut JobQueue<'a, 'b>) -> CargoResult<()> { debug!("compile_pkg; pkg={}; targets={}", pkg, targets); let _p = profile::start(format!("preparing: {}", pkg)); @@ -93,64 +96,66 @@ fn compile<'a, 'b>(targets: &[&'a Target], pkg: &'a Package, return Ok(()) } + // Prepare the fingerprint directory as the first step of building a package + let (target1, target2) = fingerprint::prepare_init(cx, pkg, KindTarget); + let mut init = vec![(Job::new(target1, target2), Fresh)]; + if cx.config.target().is_some() { + let (plugin1, plugin2) = fingerprint::prepare_init(cx, pkg, KindPlugin); + init.push((Job::new(plugin1, plugin2), Fresh)); + } + jobs.enqueue(pkg, StageStart, init); + // First part of the build step of a target is to execute all of the custom // build commands. - // - // TODO: Should this be on the target or the package? let mut build_cmds = Vec::new(); for (i, build_cmd) in pkg.get_manifest().get_build().iter().enumerate() { - build_cmds.push(try!(compile_custom(pkg, build_cmd.as_slice(), - cx, i == 0))); + let work = try!(compile_custom(pkg, build_cmd.as_slice(), cx, i == 0)); + build_cmds.push(work); } + let (freshness, dirty, fresh) = + try!(fingerprint::prepare_build_cmd(cx, pkg)); + let dirty = proc() { + for cmd in build_cmds.move_iter() { try!(cmd()) } + dirty() + }; + jobs.enqueue(pkg, StageCustomBuild, vec![(Job::new(dirty, fresh), freshness)]); // After the custom command has run, execute rustc for all targets of our // package. // - // Note that bins can all be built in parallel because they all depend on - // one another, but libs must be built sequentially because they may have - // interdependencies. + // Each target has its own concept of freshness to ensure incremental + // rebuilds on the *target* granularity, not the *package* granularity. let (mut libs, mut bins) = (Vec::new(), Vec::new()); for &target in targets.iter() { - let jobs = if target.get_profile().is_doc() { - vec![rustdoc(pkg, target, cx)] + let work = if target.get_profile().is_doc() { + vec![(rustdoc(pkg, target, cx), KindTarget)] } else { let req = cx.get_requirement(pkg, target); rustc(pkg, target, cx, req) }; - if target.is_lib() { - libs.push_all_move(jobs); - } else { - bins.push_all_move(jobs); + + let dst = if target.is_lib() {&mut libs} else {&mut bins}; + for (work, kind) in work.move_iter() { + let (freshness, dirty, fresh) = + try!(fingerprint::prepare_target(cx, pkg, target, kind)); + + let dirty = proc() { try!(work()); dirty() }; + dst.push((Job::new(dirty, fresh), freshness)); } } - - // Only after all the binaries have been built can we actually write the - // fingerprint. Currently fingerprints are transactionally done per package, - // not per-target. - // - // TODO: Can a fingerprint be per-target instead of per-package? Doing so - // would likely involve altering the granularity of key for the - // dependency queue that is later used to run jobs. - let (freshness, write_fingerprint, copy_old) = - try!(fingerprint::prepare(cx, pkg, targets)); - - // Note that we build the job backwards because each job will produce more - // work. - let bins = Job::after(bins, write_fingerprint); - let build_libs = Job::all(libs, bins); - let job = Job::all(build_cmds, vec![build_libs]); - - jobs.push((pkg, freshness, (job, copy_old))); + jobs.enqueue(pkg, StageLibraries, libs); + jobs.enqueue(pkg, StageBinaries, bins); + jobs.enqueue(pkg, StageEnd, Vec::new()); Ok(()) } fn compile_custom(pkg: &Package, cmd: &str, - cx: &Context, first: bool) -> CargoResult { + cx: &Context, first: bool) -> CargoResult { // TODO: this needs to be smarter about splitting let mut cmd = cmd.split(' '); - // TODO: this shouldn't explicitly pass `false` for dest/deps_dir, we may - // be building a C lib for a plugin - let layout = cx.layout(false); + // TODO: this shouldn't explicitly pass `KindTarget` for dest/deps_dir, we + // may be building a C lib for a plugin + let layout = cx.layout(KindTarget); let output = layout.native(pkg); let mut p = process(cmd.next().unwrap(), pkg, cx) .env("OUT_DIR", Some(&output)) @@ -159,19 +164,19 @@ fn compile_custom(pkg: &Package, cmd: &str, for arg in cmd { p = p.arg(arg); } - Ok(Job::new(proc() { + Ok(proc() { if first { try!(fs::mkdir(&output, UserRWX).chain_error(|| { internal("failed to create output directory for build command") })); } try!(p.exec_with_output().map(|_| ()).map_err(|e| e.mark_human())); - Ok(Vec::new()) - })) + Ok(()) + }) } fn rustc(package: &Package, target: &Target, - cx: &mut Context, req: PlatformRequirement) -> Vec { + cx: &mut Context, req: PlatformRequirement) -> Vec<(Work, Kind)> { let crate_types = target.rustc_crate_types(); let root = package.get_root(); @@ -181,22 +186,22 @@ fn rustc(package: &Package, target: &Target, let primary = cx.primary; let rustcs = prepare_rustc(package, target, crate_types, cx, req); - log!(5, "commands={}", rustcs); - let _ = cx.config.shell().verbose(|shell| { - for rustc in rustcs.iter() { + for &(ref rustc, _) in rustcs.iter() { try!(shell.status("Running", rustc.to_string())); } Ok(()) }); - rustcs.move_iter().map(|rustc| { + rustcs.move_iter().map(|(rustc, kind)| { let name = package.get_name().to_string(); - Job::new(proc() { + (proc() { if primary { log!(5, "executing primary"); - try!(rustc.exec().chain_error(|| human(format!("Could not compile `{}`.", name)))) + try!(rustc.exec().chain_error(|| { + human(format!("Could not compile `{}`.", name)) + })) } else { log!(5, "executing deps"); try!(rustc.exec_with_output().and(Ok(())).map_err(|err| { @@ -204,44 +209,48 @@ fn rustc(package: &Package, target: &Target, name, err.output().unwrap()), err) })) } - Ok(Vec::new()) - }) + Ok(()) + }, kind) }).collect() } fn prepare_rustc(package: &Package, target: &Target, crate_types: Vec<&str>, - cx: &Context, req: PlatformRequirement) -> Vec { + cx: &Context, req: PlatformRequirement) + -> Vec<(ProcessBuilder, Kind)> { let base = process("rustc", package, cx); let base = build_base_args(base, target, crate_types.as_slice()); - let target_cmd = build_plugin_args(base.clone(), cx, false); - let plugin_cmd = build_plugin_args(base, cx, true); - let target_cmd = build_deps_args(target_cmd, target, package, cx, false); - let plugin_cmd = build_deps_args(plugin_cmd, target, package, cx, true); + let target_cmd = build_plugin_args(base.clone(), cx, KindTarget); + let plugin_cmd = build_plugin_args(base, cx, KindPlugin); + let target_cmd = build_deps_args(target_cmd, target, package, cx, KindTarget); + let plugin_cmd = build_deps_args(plugin_cmd, target, package, cx, KindPlugin); match req { - Target => vec![target_cmd], - Plugin => vec![plugin_cmd], - PluginAndTarget if cx.config.target().is_none() => vec![target_cmd], - PluginAndTarget => vec![target_cmd, plugin_cmd], + Target => vec![(target_cmd, KindTarget)], + Plugin => vec![(plugin_cmd, KindPlugin)], + PluginAndTarget if cx.config.target().is_none() => + vec![(target_cmd, KindTarget)], + PluginAndTarget => vec![(target_cmd, KindTarget), + (plugin_cmd, KindPlugin)], } } -fn rustdoc(package: &Package, target: &Target, cx: &mut Context) -> Job { +fn rustdoc(package: &Package, target: &Target, cx: &mut Context) -> Work { // Can't document binaries, but they have a doc target listed so we can // build documentation of dependencies even when `cargo doc` is run. if target.is_bin() { - return Job::new(proc() Ok(Vec::new())) + return proc() Ok(()) } + let kind = KindTarget; let pkg_root = package.get_root(); - let cx_root = cx.layout(false).proxy().dest().dir_path().join("doc"); + let cx_root = cx.layout(kind).proxy().dest().dir_path().join("doc"); let rustdoc = util::process("rustdoc").cwd(pkg_root.clone()); let rustdoc = rustdoc.arg(target.get_src_path()) .arg("-o").arg(cx_root) .arg("--crate-name").arg(target.get_name()); - let rustdoc = build_deps_args(rustdoc, target, package, cx, false); + let rustdoc = build_deps_args(rustdoc, target, package, cx, kind); log!(5, "commands={}", rustdoc); @@ -251,7 +260,7 @@ fn rustdoc(package: &Package, target: &Target, cx: &mut Context) -> Job { let primary = cx.primary; let name = package.get_name().to_string(); - Job::new(proc() { + proc() { if primary { try!(rustdoc.exec().chain_error(|| { human(format!("Could not document `{}`.", name)) @@ -262,9 +271,10 @@ fn rustdoc(package: &Package, target: &Target, cx: &mut Context) -> Job { name, err.output().unwrap()), err) })) } - Ok(Vec::new()) - }) + Ok(()) + } } + fn build_base_args(mut cmd: ProcessBuilder, target: &Target, crate_types: &[&str]) -> ProcessBuilder { @@ -310,11 +320,11 @@ fn build_base_args(mut cmd: ProcessBuilder, fn build_plugin_args(mut cmd: ProcessBuilder, cx: &Context, - plugin: bool) -> ProcessBuilder { + kind: Kind) -> ProcessBuilder { cmd = cmd.arg("--out-dir"); - cmd = cmd.arg(cx.layout(plugin).root()); + cmd = cmd.arg(cx.layout(kind).root()); - if !plugin { + if kind == KindTarget { fn opt(cmd: ProcessBuilder, key: &str, prefix: &str, val: Option<&str>) -> ProcessBuilder { match val { @@ -335,10 +345,10 @@ fn build_plugin_args(mut cmd: ProcessBuilder, cx: &Context, } fn build_deps_args(mut cmd: ProcessBuilder, target: &Target, package: &Package, - cx: &Context, plugin: bool) -> ProcessBuilder { + cx: &Context, kind: Kind) -> ProcessBuilder { enum LinkReason { Dependency, LocalLib } - let layout = cx.layout(plugin); + let layout = cx.layout(kind); cmd = cmd.arg("-L").arg(layout.root()); cmd = cmd.arg("-L").arg(layout.deps()); @@ -347,7 +357,7 @@ fn build_deps_args(mut cmd: ProcessBuilder, target: &Target, package: &Package, cmd = push_native_dirs(cmd, &layout, package, cx, &mut HashSet::new()); for &(_, target) in cx.dep_targets(package).iter() { - cmd = link_to(cmd, target, cx, plugin, Dependency); + cmd = link_to(cmd, target, cx, kind, Dependency); } let mut targets = package.get_targets().iter().filter(|target| { @@ -356,18 +366,22 @@ fn build_deps_args(mut cmd: ProcessBuilder, target: &Target, package: &Package, if target.is_bin() { for target in targets { - cmd = link_to(cmd, target, cx, plugin, LocalLib); + cmd = link_to(cmd, target, cx, kind, LocalLib); } } return cmd; fn link_to(mut cmd: ProcessBuilder, target: &Target, - cx: &Context, plugin: bool, reason: LinkReason) -> ProcessBuilder { + cx: &Context, kind: Kind, reason: LinkReason) -> ProcessBuilder { // If this target is itself a plugin *or* if it's being linked to a // plugin, then we want the plugin directory. Otherwise we want the // target directory (hence the || here). - let layout = cx.layout(plugin || target.get_profile().is_plugin()); + let layout = cx.layout(match kind { + KindPlugin => KindPlugin, + KindTarget if target.get_profile().is_plugin() => KindPlugin, + KindTarget => KindTarget, + }); for filename in cx.target_filenames(target).iter() { let mut v = Vec::new(); @@ -409,7 +423,7 @@ pub fn process(cmd: T, pkg: &Package, cx: &Context) -> ProcessBuilder // When invoking a tool, we need the *host* deps directory in the dynamic // library search path for plugins and such which have dynamic dependencies. let mut search_path = DynamicLibrary::search_path(); - search_path.push(cx.layout(true).deps().clone()); + search_path.push(cx.layout(KindPlugin).deps().clone()); let search_path = os::join_paths(search_path.as_slice()).unwrap(); util::process(cmd) diff --git a/src/cargo/util/dependency_queue.rs b/src/cargo/util/dependency_queue.rs index 450cd7e6190..d8b51abc92a 100644 --- a/src/cargo/util/dependency_queue.rs +++ b/src/cargo/util/dependency_queue.rs @@ -29,7 +29,7 @@ pub struct DependencyQueue { /// The packages which are currently being built, waiting for a call to /// `finish`. - pending: HashMap, + pending: HashSet, } /// Indication of the freshness of a package. @@ -43,28 +43,27 @@ pub enum Freshness { } /// A trait for discovering the dependencies of a piece of data. -pub trait Dependency: Hash + Eq + Clone { - fn dependencies(&self, cx: &C) -> Vec; +pub trait Dependency: Hash + Eq + Clone { + fn dependencies(&self, cx: &C) -> Vec; } -impl, V> DependencyQueue { +impl Freshness { + pub fn combine(&self, other: Freshness) -> Freshness { + match *self { Fresh => other, Dirty => Dirty } + } +} + +impl, V> DependencyQueue { /// Creates a new dependency queue with 0 packages. pub fn new() -> DependencyQueue { DependencyQueue { dep_map: HashMap::new(), reverse_dep_map: HashMap::new(), dirty: HashSet::new(), - pending: HashMap::new(), + pending: HashSet::new(), } } - /// Registers a package with this queue. - /// - /// Only registered packages will be returned from dequeue(). - pub fn register(&mut self, step: K) { - self.reverse_dep_map.insert(step, HashSet::new()); - } - /// Adds a new package to this dependency queue. /// /// It is assumed that any dependencies of this package will eventually also @@ -79,12 +78,6 @@ impl, V> DependencyQueue { let mut my_dependencies = HashSet::new(); for dep in key.dependencies(cx).move_iter() { - if dep == key { continue } - // skip deps which were filtered out as part of resolve - if !self.reverse_dep_map.find(&dep).is_some() { - continue - } - assert!(my_dependencies.insert(dep.clone())); let rev = self.reverse_dep_map.find_or_insert(dep, HashSet::new()); assert!(rev.insert(key.clone())); @@ -105,7 +98,7 @@ impl, V> DependencyQueue { }; let (_, data) = self.dep_map.pop(&key).unwrap(); let fresh = if self.dirty.contains(&key) {Dirty} else {Fresh}; - self.pending.insert(key.clone(), fresh); + self.pending.insert(key.clone()); Some((fresh, key, data)) } @@ -118,8 +111,8 @@ impl, V> DependencyQueue { /// /// This function will update the dependency queue with this information, /// possibly allowing the next invocation of `dequeue` to return a package. - pub fn finish(&mut self, key: &K) { - let fresh = self.pending.pop(key).unwrap(); + pub fn finish(&mut self, key: &K, fresh: Freshness) { + assert!(self.pending.remove(key)); let reverse_deps = match self.reverse_dep_map.find(key) { Some(deps) => deps, None => return, diff --git a/src/cargo/util/toml.rs b/src/cargo/util/toml.rs index f8f9cecf042..22196e6701b 100644 --- a/src/cargo/util/toml.rs +++ b/src/cargo/util/toml.rs @@ -537,6 +537,12 @@ fn normalize(libs: &[TomlLibTarget], }); for profile in target_profiles(l, dep).iter() { + let mut metadata = metadata.clone(); + // Libs and their tests are built in parallel, so we need to make + // sure that their metadata is different. + if profile.is_test() { + metadata.mix(&"test"); + } dst.push(Target::lib_target(l.name.as_slice(), crate_types.clone(), &path.to_path(), profile, metadata)); diff --git a/tests/test_cargo_compile.rs b/tests/test_cargo_compile.rs index 49e45b96f00..32e57fb4612 100644 --- a/tests/test_cargo_compile.rs +++ b/tests/test_cargo_compile.rs @@ -1269,7 +1269,7 @@ test!(implicit_examples { fn main() { println!("{}, {}!", world::get_goodbye(), world::get_world()); } "#); - assert_that(p.cargo_process("cargo-test"), execs()); + assert_that(p.cargo_process("cargo-test"), execs().with_status(0)); assert_that(process(p.bin("test/hello")), execs().with_stdout("Hello, World!\n")); assert_that(process(p.bin("test/goodbye")), execs().with_stdout("Goodbye, World!\n")); }) diff --git a/tests/test_cargo_test.rs b/tests/test_cargo_test.rs index 1f992e6e406..e6509e95816 100644 --- a/tests/test_cargo_test.rs +++ b/tests/test_cargo_test.rs @@ -418,7 +418,8 @@ test result: ok. 1 passed; 0 failed; 0 ignored; 0 measured"; compiling = COMPILING, dir = p.root().display()); assert!(out == format!("{}\n\n{}\n\n\n{}\n\n", head, bin, lib).as_slice() || - out == format!("{}\n\n{}\n\n\n{}\n\n", head, lib, bin).as_slice()); + out == format!("{}\n\n{}\n\n\n{}\n\n", head, lib, bin).as_slice(), + "bad output: {}", out); }) test!(lib_with_standard_name {