)
}
+ fn arg_build_plan(self) -> Self {
+ self._arg(opt("build-plan", "Output the build plan in JSON"))
+ }
+
fn arg_new_opts(self) -> Self {
self._arg(
opt(
let mut build_config = BuildConfig::new(config, self.jobs()?, &self.target(), mode)?;
build_config.message_format = message_format;
build_config.release = self._is_present("release");
+ build_config.build_plan = self._is_present("build-plan");
+ if build_config.build_plan && !config.cli_unstable().unstable_options {
+ Err(format_err!(
+ "`--build-plan` flag is unstable, pass `-Z unstable-options` to enable it"
+ ))?;
+ };
let opts = CompileOptions {
config,
.arg(opt("out-dir", "Copy final artifacts to this directory").value_name("PATH"))
.arg_manifest_path()
.arg_message_format()
+ .arg_build_plan()
.after_help(
"\
If the --package argument is given, then SPEC is a package id specification
pub mode: CompileMode,
/// Whether to print std output in json format (for machine reading)
pub message_format: MessageFormat,
+ /// Output a build plan to stdout instead of actually compiling.
+ pub build_plan: bool,
}
impl BuildConfig {
release: false,
mode,
message_format: MessageFormat::Human,
+ build_plan: false,
})
}
}
None
}
+
+ /// Return the list of filenames read by cargo to generate the BuildContext
+ /// (all Cargo.toml, etc).
+ pub fn inputs(&self) -> CargoResult<Vec<PathBuf>> {
+ let mut inputs = Vec::new();
+ for id in self.packages.package_ids() {
+ let pkg = self.get_package(id)?;
+ inputs.push(pkg.manifest_path().to_path_buf());
+ }
+ inputs.sort();
+ Ok(inputs)
+ }
}
/// Information required to build for a target
--- /dev/null
+//! A graph-like structure used to represent the rustc commands to build the project and the
+//! interdependencies between them.
+//!
+//! The BuildPlan structure is used to store the dependency graph of a dry run so that it can be
+//! shared with an external build system. Each Invocation in the BuildPlan comprises a single
+//! subprocess and defines the build environment, the outputs produced by the subprocess, and the
+//! dependencies on other Invocations.
+
+use std::collections::BTreeMap;
+
+use core::TargetKind;
+use super::{Context, Kind, Unit};
+use super::context::OutputFile;
+use util::{internal, CargoResult, ProcessBuilder};
+use std::sync::Arc;
+use std::path::PathBuf;
+use serde_json;
+use semver;
+
+#[derive(Debug, Serialize)]
+struct Invocation {
+ package_name: String,
+ package_version: semver::Version,
+ target_kind: TargetKind,
+ kind: Kind,
+ deps: Vec<usize>,
+ outputs: Vec<PathBuf>,
+ links: BTreeMap<PathBuf, PathBuf>,
+ program: String,
+ args: Vec<String>,
+ env: BTreeMap<String, String>,
+ cwd: Option<PathBuf>,
+}
+
+#[derive(Debug)]
+pub struct BuildPlan {
+ invocation_map: BTreeMap<String, usize>,
+ plan: SerializedBuildPlan,
+}
+
+#[derive(Debug, Serialize)]
+struct SerializedBuildPlan {
+ invocations: Vec<Invocation>,
+ inputs: Vec<PathBuf>,
+}
+
+impl Invocation {
+ pub fn new(unit: &Unit, deps: Vec<usize>) -> Invocation {
+ let id = unit.pkg.package_id();
+ Invocation {
+ package_name: id.name().to_string(),
+ package_version: id.version().clone(),
+ kind: unit.kind,
+ target_kind: unit.target.kind().clone(),
+ deps: deps,
+ outputs: Vec::new(),
+ links: BTreeMap::new(),
+ program: String::new(),
+ args: Vec::new(),
+ env: BTreeMap::new(),
+ cwd: None,
+ }
+ }
+
+ pub fn add_output(&mut self, path: &PathBuf, link: &Option<PathBuf>) {
+ self.outputs.push(path.clone());
+ if let Some(ref link) = *link {
+ self.links.insert(link.clone(), path.clone());
+ }
+ }
+
+ pub fn update_cmd(&mut self, cmd: ProcessBuilder) -> CargoResult<()> {
+ self.program = cmd.get_program()
+ .to_str()
+ .ok_or_else(|| format_err!("unicode program string required"))?
+ .to_string()
+ .clone();
+ self.cwd = Some(cmd.get_cwd().unwrap().to_path_buf());
+ for arg in cmd.get_args().iter() {
+ self.args.push(
+ arg.to_str()
+ .ok_or_else(|| format_err!("unicode argument string required"))?
+ .to_string()
+ .clone(),
+ );
+ }
+ for var in cmd.get_envs().keys() {
+ let value = cmd.get_env(var).unwrap_or_default();
+ self.env.insert(
+ var.clone(),
+ value
+ .to_str()
+ .ok_or_else(|| format_err!("unicode environment value required"))?
+ .to_string(),
+ );
+ }
+ Ok(())
+ }
+}
+
+impl BuildPlan {
+ pub fn new() -> BuildPlan {
+ BuildPlan {
+ invocation_map: BTreeMap::new(),
+ plan: SerializedBuildPlan::new(),
+ }
+ }
+
+ pub fn add(&mut self, cx: &Context, unit: &Unit) -> CargoResult<()> {
+ let id = self.plan.invocations.len();
+ self.invocation_map.insert(unit.buildkey(), id);
+ let deps = cx.dep_targets(&unit)
+ .iter()
+ .map(|dep| self.invocation_map[&dep.buildkey()])
+ .collect();
+ let invocation = Invocation::new(unit, deps);
+ self.plan.invocations.push(invocation);
+ Ok(())
+ }
+
+ pub fn update(
+ &mut self,
+ invocation_name: String,
+ cmd: ProcessBuilder,
+ outputs: Arc<Vec<OutputFile>>,
+ ) -> CargoResult<()> {
+ let id = self.invocation_map[&invocation_name];
+ let invocation = self.plan
+ .invocations
+ .get_mut(id)
+ .ok_or_else(|| internal(format!("couldn't find invocation for {}", invocation_name)))?;
+
+ invocation.update_cmd(cmd)?;
+ for output in outputs.iter() {
+ invocation.add_output(&output.path, &output.hardlink);
+ }
+
+ Ok(())
+ }
+
+ pub fn set_inputs(&mut self, inputs: Vec<PathBuf>) {
+ self.plan.inputs = inputs;
+ }
+
+ pub fn output_plan(self) {
+ let encoded = serde_json::to_string(&self.plan).unwrap();
+ println!("{}", encoded);
+ }
+}
+
+impl SerializedBuildPlan {
+ pub fn new() -> SerializedBuildPlan {
+ SerializedBuildPlan {
+ invocations: Vec::new(),
+ inputs: Vec::new(),
+ }
+ }
+}
use std::fmt::Write;
use std::path::PathBuf;
use std::sync::Arc;
+use std::cmp::Ordering;
use jobserver::Client;
use core::{Package, PackageId, Resolve, Target};
use core::profiles::Profile;
use util::errors::{CargoResult, CargoResultExt};
-use util::{internal, profile, Config};
+use util::{internal, profile, Config, short_hash};
use super::custom_build::{self, BuildDeps, BuildScripts, BuildState};
use super::fingerprint::Fingerprint;
use super::job_queue::JobQueue;
use super::layout::Layout;
use super::{BuildContext, Compilation, CompileMode, Executor, FileFlavor, Kind};
+use super::build_plan::BuildPlan;
mod unit_dependencies;
use self::unit_dependencies::build_unit_dependencies;
mod compilation_files;
-pub use self::compilation_files::Metadata;
-use self::compilation_files::{CompilationFiles, OutputFile};
+pub use self::compilation_files::{Metadata, OutputFile};
+use self::compilation_files::CompilationFiles;
/// All information needed to define a Unit.
///
pub mode: CompileMode,
}
+impl<'a> Unit<'a> {
+ pub fn buildkey(&self) -> String {
+ format!("{}-{}", self.pkg.name(), short_hash(self))
+ }
+}
+
+impl<'a> Ord for Unit<'a> {
+ fn cmp(&self, other: &Unit) -> Ordering {
+ self.buildkey().cmp(&other.buildkey())
+ }
+}
+
+impl<'a> PartialOrd for Unit<'a> {
+ fn partial_cmp(&self, other: &Unit) -> Option<Ordering> {
+ Some(self.cmp(other))
+ }
+}
+
pub struct Context<'a, 'cfg: 'a> {
pub bcx: &'a BuildContext<'a, 'cfg>,
pub compilation: Compilation<'cfg>,
exec: &Arc<Executor>,
) -> CargoResult<Compilation<'cfg>> {
let mut queue = JobQueue::new(self.bcx);
+ let mut plan = BuildPlan::new();
+ let build_plan = self.bcx.build_config.build_plan;
self.prepare_units(export_dir, units)?;
self.prepare()?;
custom_build::build_map(&mut self, units)?;
// part of this, that's all done next as part of the `execute`
// function which will run everything in order with proper
// parallelism.
- super::compile(&mut self, &mut queue, unit, exec)?;
+ super::compile(&mut self, &mut queue, &mut plan, unit, exec)?;
}
// Now that we've figured out everything that we're going to do, do it!
- queue.execute(&mut self)?;
+ queue.execute(&mut self, &mut plan)?;
+
+ if build_plan {
+ plan.set_inputs(self.bcx.inputs()?);
+ plan.output_plan();
+ }
for unit in units.iter() {
for output in self.outputs(unit)?.iter() {
return Vec::new();
}
}
- self.unit_dependencies[unit].clone()
+ let mut deps = self.unit_dependencies[unit].clone();
+ deps.sort();
+ deps
}
pub fn incremental_args(&self, unit: &Unit) -> CargoResult<Vec<String>> {
build_work(cx, unit)?
};
- // Now that we've prep'd our work, build the work needed to manage the
- // fingerprint and then start returning that upwards.
- let (freshness, dirty, fresh) = fingerprint::prepare_build_cmd(cx, unit)?;
+ if cx.bcx.build_config.build_plan {
+ Ok((work_dirty, work_fresh, Freshness::Dirty))
+ } else {
+ // Now that we've prep'd our work, build the work needed to manage the
+ // fingerprint and then start returning that upwards.
+ let (freshness, dirty, fresh) = fingerprint::prepare_build_cmd(cx, unit)?;
- Ok((work_dirty.then(dirty), work_fresh.then(fresh), freshness))
+ Ok((work_dirty.then(dirty), work_fresh.then(fresh), freshness))
+ }
}
fn build_work<'a, 'cfg>(cx: &mut Context<'a, 'cfg>, unit: &Unit<'a>) -> CargoResult<(Work, Work)> {
.expect("running a script not depending on an actual script");
let script_output = cx.files().build_script_dir(build_script_unit);
let build_output = cx.files().build_script_out_dir(unit);
+ let build_plan = bcx.build_config.build_plan;
+ let invocation_name = unit.buildkey();
// Building the command to execute
let to_exec = script_output.join(unit.target.name());
// along to this custom build command. We're also careful to augment our
// dynamic library search path in case the build script depended on any
// native dynamic libraries.
- {
+ if !build_plan {
let build_state = build_state.outputs.lock().unwrap();
for (name, id) in lib_deps {
let key = (id.clone(), kind);
}
// And now finally, run the build command itself!
- state.running(&cmd);
- let output = cmd.exec_with_streaming(
- &mut |out_line| {
- state.stdout(out_line);
- Ok(())
- },
- &mut |err_line| {
- state.stderr(err_line);
- Ok(())
- },
- true,
- ).map_err(|e| {
- format_err!(
- "failed to run custom build command for `{}`\n{}",
- pkg_name,
- e
- )
- })?;
-
- // After the build command has finished running, we need to be sure to
- // remember all of its output so we can later discover precisely what it
- // was, even if we don't run the build command again (due to freshness).
- //
- // This is also the location where we provide feedback into the build
- // state informing what variables were discovered via our script as
- // well.
- paths::write(&output_file, &output.stdout)?;
- paths::write(&err_file, &output.stderr)?;
- paths::write(&root_output_file, util::path2bytes(&root_output)?)?;
- let parsed_output =
- BuildOutput::parse(&output.stdout, &pkg_name, &root_output, &root_output)?;
-
- if json_messages {
- let library_paths = parsed_output
- .library_paths
- .iter()
- .map(|l| l.display().to_string())
- .collect::<Vec<_>>();
- machine_message::emit(&machine_message::BuildScript {
- package_id: &id,
- linked_libs: &parsed_output.library_links,
- linked_paths: &library_paths,
- cfgs: &parsed_output.cfgs,
- env: &parsed_output.env,
- });
- }
+ if build_plan {
+ state.build_plan(invocation_name, cmd.clone(), Arc::new(Vec::new()));
+ } else {
+ state.running(&cmd);
+ let output = cmd.exec_with_streaming(
+ &mut |out_line| {
+ state.stdout(out_line);
+ Ok(())
+ },
+ &mut |err_line| {
+ state.stderr(err_line);
+ Ok(())
+ },
+ true,
+ ).map_err(|e| {
+ format_err!(
+ "failed to run custom build command for `{}`\n{}",
+ pkg_name,
+ e
+ )
+ })?;
- build_state.insert(id, kind, parsed_output);
+ // After the build command has finished running, we need to be sure to
+ // remember all of its output so we can later discover precisely what it
+ // was, even if we don't run the build command again (due to freshness).
+ //
+ // This is also the location where we provide feedback into the build
+ // state informing what variables were discovered via our script as
+ // well.
+ paths::write(&output_file, &output.stdout)?;
+ paths::write(&err_file, &output.stderr)?;
+ paths::write(&root_output_file, util::path2bytes(&root_output)?)?;
+ let parsed_output =
+ BuildOutput::parse(&output.stdout, &pkg_name, &root_output, &root_output)?;
+
+ if json_messages {
+ let library_paths = parsed_output
+ .library_paths
+ .iter()
+ .map(|l| l.display().to_string())
+ .collect::<Vec<_>>();
+ machine_message::emit(&machine_message::BuildScript {
+ package_id: &id,
+ linked_libs: &parsed_output.library_links,
+ linked_paths: &library_paths,
+ cfgs: &parsed_output.cfgs,
+ env: &parsed_output.env,
+ });
+ }
+ build_state.insert(id, kind, parsed_output);
+ }
Ok(())
});
use std::io;
use std::mem;
use std::sync::mpsc::{channel, Receiver, Sender};
+use std::sync::Arc;
use crossbeam::{self, Scope};
use jobserver::{Acquired, HelperThread};
use util::{Config, DependencyQueue, Dirty, Fresh, Freshness};
use super::job::Job;
-use super::{BuildContext, CompileMode, Context, Kind, Unit};
+use super::{BuildContext, BuildPlan, CompileMode, Context, Kind, Unit};
+use super::context::OutputFile;
/// A management structure of the entire dependency graph to compile.
///
enum Message<'a> {
Run(String),
+ BuildPlanMsg(String, ProcessBuilder, Arc<Vec<OutputFile>>),
Stdout(String),
Stderr(String),
Token(io::Result<Acquired>),
let _ = self.tx.send(Message::Run(cmd.to_string()));
}
+ pub fn build_plan(
+ &self,
+ module_name: String,
+ cmd: ProcessBuilder,
+ filenames: Arc<Vec<OutputFile>>,
+ ) {
+ let _ = self.tx
+ .send(Message::BuildPlanMsg(module_name, cmd, filenames));
+ }
+
pub fn stdout(&self, out: &str) {
let _ = self.tx.send(Message::Stdout(out.to_string()));
}
/// 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, cx: &mut Context) -> CargoResult<()> {
+ pub fn execute(&mut self, cx: &mut Context, plan: &mut BuildPlan) -> CargoResult<()> {
let _p = profile::start("executing the job graph");
self.queue.queue_finished();
})
.chain_err(|| "failed to create helper thread for jobserver management")?;
- crossbeam::scope(|scope| self.drain_the_queue(cx, scope, &helper))
+ crossbeam::scope(|scope| self.drain_the_queue(cx, plan, scope, &helper))
}
fn drain_the_queue(
&mut self,
cx: &mut Context,
+ plan: &mut BuildPlan,
scope: &Scope<'a>,
jobserver_helper: &HelperThread,
) -> CargoResult<()> {
let mut tokens = Vec::new();
let mut queue = Vec::new();
+ let build_plan = cx.bcx.build_config.build_plan;
trace!("queue: {:#?}", self.queue);
// Iteratively execute the entire dependency graph. Each turn of the
// we're able to perform some parallel work.
while error.is_none() && self.active < tokens.len() + 1 && !queue.is_empty() {
let (key, job, fresh) = queue.remove(0);
- self.run(key, fresh, job, cx.bcx.config, scope)?;
+ self.run(key, fresh, job, cx.bcx.config, scope, build_plan)?;
}
// If after all that we're not actually running anything then we're
.shell()
.verbose(|c| c.status("Running", &cmd))?;
}
+ Message::BuildPlanMsg(module_name, cmd, filenames) => {
+ plan.update(module_name, cmd, filenames)?;
+ }
Message::Stdout(out) => {
if cx.bcx.config.extra_verbose() {
println!("{}", out);
"{} [{}] target(s) in {}",
build_type, opt_type, time_elapsed
);
- cx.bcx.config.shell().status("Finished", message)?;
+ if !build_plan {
+ cx.bcx.config.shell().status("Finished", message)?;
+ }
Ok(())
} else if let Some(e) = error {
Err(e)
job: Job,
config: &Config,
scope: &Scope<'a>,
+ build_plan: bool,
) -> CargoResult<()> {
info!("start: {:?}", key);
}
}
- // Print out some nice progress information
- self.note_working_on(config, &key, fresh)?;
+ if !build_plan {
+ // Print out some nice progress information
+ self.note_working_on(config, &key, fresh)?;
+ }
Ok(())
}
use util::{self, machine_message, Freshness, ProcessBuilder};
use util::{internal, join_paths, profile};
+use self::build_plan::BuildPlan;
use self::job::{Job, Work};
use self::job_queue::JobQueue;
mod build_config;
mod build_context;
+mod build_plan;
mod compilation;
mod context;
mod custom_build;
/// Whether an object is for the host arch, or the target arch.
///
/// These will be the same unless cross-compiling.
-#[derive(PartialEq, Eq, Hash, Debug, Clone, Copy, PartialOrd, Ord)]
+#[derive(PartialEq, Eq, Hash, Debug, Clone, Copy, PartialOrd, Ord, Serialize)]
pub enum Kind {
Host,
Target,
fn compile<'a, 'cfg: 'a>(
cx: &mut Context<'a, 'cfg>,
jobs: &mut JobQueue<'a>,
+ plan: &mut BuildPlan,
unit: &Unit<'a>,
exec: &Arc<Executor>,
) -> CargoResult<()> {
let bcx = cx.bcx;
+ let build_plan = bcx.build_config.build_plan;
if !cx.compiled.insert(*unit) {
return Ok(());
}
} else if unit.mode == CompileMode::Doctest {
// we run these targets later, so this is just a noop for now
(Work::noop(), Work::noop(), Freshness::Fresh)
+ } else if build_plan {
+ (
+ rustc(cx, unit, &exec.clone())?,
+ Work::noop(),
+ Freshness::Dirty,
+ )
} else {
let (mut freshness, dirty, fresh) = fingerprint::prepare_target(cx, unit)?;
let work = if unit.mode.is_doc() {
// Be sure to compile all dependencies of this target as well.
for unit in cx.dep_targets(unit).iter() {
- compile(cx, jobs, unit, exec)?;
+ compile(cx, jobs, plan, unit, exec)?;
+ }
+ if build_plan {
+ plan.add(cx, unit)?;
}
Ok(())
exec: &Arc<Executor>,
) -> CargoResult<Work> {
let mut rustc = prepare_rustc(cx, &unit.target.rustc_crate_types(), unit)?;
+ let build_plan = cx.bcx.build_config.build_plan;
let name = unit.pkg.name().to_string();
+ let buildkey = unit.buildkey();
// If this is an upstream dep we don't want warnings from, turn off all
// lints.
// previous build scripts, we include them in the rustc invocation.
if let Some(build_deps) = build_deps {
let build_state = build_state.outputs.lock().unwrap();
- add_native_deps(
- &mut rustc,
- &build_state,
- &build_deps,
- pass_l_flag,
- ¤t_id,
- )?;
- add_plugin_deps(&mut rustc, &build_state, &build_deps, &root_output)?;
+ if !build_plan {
+ add_native_deps(
+ &mut rustc,
+ &build_state,
+ &build_deps,
+ pass_l_flag,
+ ¤t_id,
+ )?;
+ add_plugin_deps(&mut rustc, &build_state, &build_deps, &root_output)?;
+ }
add_custom_env(&mut rustc, &build_state, ¤t_id, kind)?;
}
Ok(())
},
).chain_err(|| format!("Could not compile `{}`.", name))?;
+ } else if build_plan {
+ state.build_plan(buildkey, rustc.clone(), outputs.clone());
} else {
exec.exec(rustc, &package_id, &target)
.map_err(Internal::new)
--- /dev/null
+use cargotest::ChannelChanger;
+use cargotest::support::{basic_bin_manifest, execs, main_file, project};
+use hamcrest::{assert_that, existing_file, is_not};
+
+#[test]
+fn cargo_build_plan_simple() {
+ let p = project("foo")
+ .file("Cargo.toml", &basic_bin_manifest("foo"))
+ .file("src/foo.rs", &main_file(r#""i am foo""#, &[]))
+ .build();
+
+ assert_that(
+ p.cargo("build")
+ .masquerade_as_nightly_cargo()
+ .arg("--build-plan")
+ .arg("-Zunstable-options"),
+ execs().with_status(0).with_json(
+ r#"
+ {
+ "inputs": [
+ "[..][/]foo[/]Cargo.toml"
+ ],
+ "invocations": [
+ {
+ "args": "{...}",
+ "cwd": "[..][/]target[/]cit[/][..][/]foo",
+ "deps": [],
+ "env": "{...}",
+ "kind": "Host",
+ "links": "{...}",
+ "outputs": "{...}",
+ "package_name": "foo",
+ "package_version": "0.5.0",
+ "program": "rustc",
+ "target_kind": ["bin"]
+ }
+ ]
+ }
+ "#,
+ ),
+ );
+ assert_that(&p.bin("foo"), is_not(existing_file()));
+}
+
+#[test]
+fn cargo_build_plan_single_dep() {
+ let p = project("foo")
+ .file(
+ "Cargo.toml",
+ r#"
+ [package]
+ name = "foo"
+ authors = []
+ version = "0.5.0"
+
+ [dependencies]
+ bar = { path = "bar" }
+ "#,
+ )
+ .file(
+ "src/lib.rs",
+ r#"
+ extern crate bar;
+ pub fn foo() { bar::bar(); }
+
+ #[test]
+ fn test() { foo(); }
+ "#,
+ )
+ .file(
+ "bar/Cargo.toml",
+ r#"
+ [package]
+ name = "bar"
+ version = "0.0.1"
+ authors = []
+ "#,
+ )
+ .file("bar/src/lib.rs", "pub fn bar() {}")
+ .build();
+ assert_that(
+ p.cargo("build")
+ .masquerade_as_nightly_cargo()
+ .arg("--build-plan")
+ .arg("-Zunstable-options"),
+ execs().with_status(0).with_json(
+ r#"
+ {
+ "inputs": [
+ "[..][/]foo[/]Cargo.toml",
+ "[..][/]foo[/]bar[/]Cargo.toml"
+ ],
+ "invocations": [
+ {
+ "args": "{...}",
+ "cwd": "[..][/]target[/]cit[/][..][/]foo",
+ "deps": [],
+ "env": "{...}",
+ "kind": "Host",
+ "links": "{...}",
+ "outputs": [
+ "[..][/]foo[/]target[/]debug[/]deps[/]libbar-[..].rlib"
+ ],
+ "package_name": "bar",
+ "package_version": "0.0.1",
+ "program": "rustc",
+ "target_kind": ["lib"]
+ },
+ {
+ "args": "{...}",
+ "cwd": "[..][/]target[/]cit[/][..][/]foo",
+ "deps": [0],
+ "env": "{...}",
+ "kind": "Host",
+ "links": "{...}",
+ "outputs": [
+ "[..][/]foo[/]target[/]debug[/]deps[/]libfoo-[..].rlib"
+ ],
+ "package_name": "foo",
+ "package_version": "0.5.0",
+ "program": "rustc",
+ "target_kind": ["lib"]
+ }
+ ]
+ }
+ "#,
+ ),
+ );
+}
+
+#[test]
+fn cargo_build_plan_build_script() {
+ let p = project("foo")
+ .file(
+ "Cargo.toml",
+ r#"
+ [project]
+
+ name = "foo"
+ version = "0.5.0"
+ authors = ["wycats@example.com"]
+ build = "build.rs"
+ "#,
+ )
+ .file("src/main.rs", r#"fn main() {}"#)
+ .file("build.rs", r#"fn main() {}"#)
+ .build();
+
+ assert_that(
+ p.cargo("build")
+ .masquerade_as_nightly_cargo()
+ .arg("--build-plan")
+ .arg("-Zunstable-options"),
+ execs().with_status(0).with_json(
+ r#"
+ {
+ "inputs": [
+ "[..][/]foo[/]Cargo.toml"
+ ],
+ "invocations": [
+ {
+ "args": "{...}",
+ "cwd": "[..][/]target[/]cit[/][..][/]foo",
+ "deps": [],
+ "env": "{...}",
+ "kind": "Host",
+ "links": "{...}",
+ "outputs": [
+ "[..][/]foo[/]target[/]debug[/]build[/][..][/]build_script_build-[..]"
+ ],
+ "package_name": "foo",
+ "package_version": "0.5.0",
+ "program": "rustc",
+ "target_kind": ["custom-build"]
+ },
+ {
+ "args": "{...}",
+ "cwd": "[..][/]target[/]cit[/][..][/]foo",
+ "deps": [0],
+ "env": "{...}",
+ "kind": "Host",
+ "links": "{...}",
+ "outputs": [],
+ "package_name": "foo",
+ "package_version": "0.5.0",
+ "program": "[..][/]build-script-build",
+ "target_kind": ["custom-build"]
+ },
+ {
+ "args": "{...}",
+ "cwd": "[..][/]target[/]cit[/][..][/]foo",
+ "deps": [1],
+ "env": "{...}",
+ "kind": "Host",
+ "links": "{...}",
+ "outputs": "{...}",
+ "package_name": "foo",
+ "package_version": "0.5.0",
+ "program": "rustc",
+ "target_kind": ["bin"]
+ }
+ ]
+ }
+ "#,
+ ),
+ );
+}
mod build_auth;
mod build_lib;
mod build;
+mod build_plan;
mod build_script_env;
mod build_script;
mod cargo_alias_config;