cargo/core/compiler/job_queue/
mod.rs

1//! Management of the interaction between the main `cargo` and all spawned jobs.
2//!
3//! ## Overview
4//!
5//! This module implements a job queue. A job here represents a unit of work,
6//! which is roughly a rustc invocation, a build script run, or just a no-op.
7//! The job queue primarily handles the following things:
8//!
9//! * Spawns concurrent jobs. Depending on its [`Freshness`], a job could be
10//!     either executed on a spawned thread or ran on the same thread to avoid
11//!     the threading overhead.
12//! * Controls the number of concurrency. It allocates and manages [`jobserver`]
13//!     tokens to each spawned off rustc and build scripts.
14//! * Manages the communication between the main `cargo` process and its
15//!     spawned jobs. Those [`Message`]s are sent over a [`Queue`] shared
16//!     across threads.
17//! * Schedules the execution order of each [`Job`]. Priorities are determined
18//!     when calling [`JobQueue::enqueue`] to enqueue a job. The scheduling is
19//!     relatively rudimentary and could likely be improved.
20//!
21//! A rough outline of building a queue and executing jobs is:
22//!
23//! 1. [`JobQueue::new`] to simply create one queue.
24//! 2. [`JobQueue::enqueue`] to add new jobs onto the queue.
25//! 3. Consumes the queue and executes all jobs via [`JobQueue::execute`].
26//!
27//! The primary loop happens insides [`JobQueue::execute`], which is effectively
28//! [`DrainState::drain_the_queue`]. [`DrainState`] is, as its name tells,
29//! the running state of the job queue getting drained.
30//!
31//! ## Jobserver
32//!
33//! As of Feb. 2023, Cargo and rustc have a relatively simple jobserver
34//! relationship with each other. They share a single jobserver amongst what
35//! is potentially hundreds of threads of work on many-cored systems.
36//! The jobserver could come from either the environment (e.g., from a `make`
37//! invocation), or from Cargo creating its own jobserver server if there is no
38//! jobserver to inherit from.
39//!
40//! Cargo wants to complete the build as quickly as possible, fully saturating
41//! all cores (as constrained by the `-j=N`) parameter. Cargo also must not spawn
42//! more than N threads of work: the total amount of tokens we have floating
43//! around must always be limited to N.
44//!
45//! It is not really possible to optimally choose which crate should build
46//! first or last; nor is it possible to decide whether to give an additional
47//! token to rustc first or rather spawn a new crate of work. The algorithm in
48//! Cargo prioritizes spawning as many crates (i.e., rustc processes) as
49//! possible. In short, the jobserver relationship among Cargo and rustc
50//! processes is **1 `cargo` to N `rustc`**. Cargo knows nothing beyond rustc
51//! processes in terms of parallelism[^parallel-rustc].
52//!
53//! We integrate with the [jobserver] crate, originating from GNU make
54//! [POSIX jobserver], to make sure that build scripts which use make to
55//! build C code can cooperate with us on the number of used tokens and
56//! avoid overfilling the system we're on.
57//!
58//! ## Scheduling
59//!
60//! The current scheduling algorithm is not really polished. It is simply based
61//! on a dependency graph [`DependencyQueue`]. We continue adding nodes onto
62//! the graph until we finalize it. When the graph gets finalized, it finds the
63//! sum of the cost of each dependencies of each node, including transitively.
64//! The sum of dependency cost turns out to be the cost of each given node.
65//!
66//! At the time being, the cost is just passed as a fixed placeholder in
67//! [`JobQueue::enqueue`]. In the future, we could explore more possibilities
68//! around it. For instance, we start persisting timing information for each
69//! build somewhere. For a subsequent build, we can look into the historical
70//! data and perform a PGO-like optimization to prioritize jobs, making a build
71//! fully pipelined.
72//!
73//! ## Message queue
74//!
75//! Each spawned thread running a process uses the message queue [`Queue`] to
76//! send messages back to the main thread (the one running `cargo`).
77//! The main thread coordinates everything, and handles printing output.
78//!
79//! It is important to be careful which messages use [`push`] vs [`push_bounded`].
80//! `push` is for priority messages (like tokens, or "finished") where the
81//! sender shouldn't block. We want to handle those so real work can proceed
82//! ASAP.
83//!
84//! `push_bounded` is only for messages being printed to stdout/stderr. Being
85//! bounded prevents a flood of messages causing a large amount of memory
86//! being used.
87//!
88//! `push` also avoids blocking which helps avoid deadlocks. For example, when
89//! the diagnostic server thread is dropped, it waits for the thread to exit.
90//! But if the thread is blocked on a full queue, and there is a critical
91//! error, the drop will deadlock. This should be fixed at some point in the
92//! future. The jobserver thread has a similar problem, though it will time
93//! out after 1 second.
94//!
95//! To access the message queue, each running `Job` is given its own [`JobState`],
96//! containing everything it needs to communicate with the main thread.
97//!
98//! See [`Message`] for all available message kinds.
99//!
100//! [^parallel-rustc]: In fact, `jobserver` that Cargo uses also manages the
101//!     allocation of tokens to rustc beyond the implicit token each rustc owns
102//!     (i.e., the ones used for parallel LLVM work and parallel rustc threads).
103//!     See also ["Rust Compiler Development Guide: Parallel Compilation"]
104//!     and [this comment][rustc-codegen] in rust-lang/rust.
105//!
106//! ["Rust Compiler Development Guide: Parallel Compilation"]: https://rustc-dev-guide.rust-lang.org/parallel-rustc.html
107//! [rustc-codegen]: https://github.com/rust-lang/rust/blob/5423745db8b434fcde54888b35f518f00cce00e4/compiler/rustc_codegen_ssa/src/back/write.rs#L1204-L1217
108//! [jobserver]: https://docs.rs/jobserver
109//! [POSIX jobserver]: https://www.gnu.org/software/make/manual/html_node/POSIX-Jobserver.html
110//! [`push`]: Queue::push
111//! [`push_bounded`]: Queue::push_bounded
112
113mod job;
114mod job_state;
115
116use std::cell::RefCell;
117use std::collections::{HashMap, HashSet};
118use std::fmt::Write as _;
119use std::io;
120use std::path::{Path, PathBuf};
121use std::sync::Arc;
122use std::thread::{self, Scope};
123use std::time::Duration;
124
125use anyhow::{Context as _, format_err};
126use cargo_util::ProcessBuilder;
127use jobserver::{Acquired, HelperThread};
128use semver::Version;
129use tracing::{debug, trace};
130
131pub use self::job::Freshness::{self, Dirty, Fresh};
132pub use self::job::{Job, Work};
133pub use self::job_state::JobState;
134use super::build_runner::OutputFile;
135use super::custom_build::Severity;
136use super::timings::Timings;
137use super::{BuildContext, BuildPlan, BuildRunner, CompileMode, Unit};
138use crate::core::compiler::descriptive_pkg_name;
139use crate::core::compiler::future_incompat::{
140    self, FutureBreakageItem, FutureIncompatReportPackage,
141};
142use crate::core::resolver::ResolveBehavior;
143use crate::core::{PackageId, Shell, TargetKind};
144use crate::util::CargoResult;
145use crate::util::context::WarningHandling;
146use crate::util::diagnostic_server::{self, DiagnosticPrinter};
147use crate::util::errors::AlreadyPrintedError;
148use crate::util::machine_message::{self, Message as _};
149use crate::util::{self, internal};
150use crate::util::{DependencyQueue, GlobalContext, Progress, ProgressStyle, Queue};
151
152/// This structure is backed by the `DependencyQueue` type and manages the
153/// queueing of compilation steps for each package. Packages enqueue units of
154/// work and then later on the entire graph is converted to `DrainState` and
155/// executed.
156pub struct JobQueue<'gctx> {
157    queue: DependencyQueue<Unit, Artifact, Job>,
158    counts: HashMap<PackageId, usize>,
159    timings: Timings<'gctx>,
160}
161
162/// This structure is backed by the `DependencyQueue` type and manages the
163/// actual compilation step of each package. Packages enqueue units of work and
164/// then later on the entire graph is processed and compiled.
165///
166/// It is created from `JobQueue` when we have fully assembled the crate graph
167/// (i.e., all package dependencies are known).
168struct DrainState<'gctx> {
169    // This is the length of the DependencyQueue when starting out
170    total_units: usize,
171
172    queue: DependencyQueue<Unit, Artifact, Job>,
173    messages: Arc<Queue<Message>>,
174    /// Diagnostic deduplication support.
175    diag_dedupe: DiagDedupe<'gctx>,
176    /// Count of warnings, used to print a summary after the job succeeds
177    warning_count: HashMap<JobId, WarningCount>,
178    active: HashMap<JobId, Unit>,
179    compiled: HashSet<PackageId>,
180    documented: HashSet<PackageId>,
181    scraped: HashSet<PackageId>,
182    counts: HashMap<PackageId, usize>,
183    progress: Progress<'gctx>,
184    next_id: u32,
185    timings: Timings<'gctx>,
186
187    /// Tokens that are currently owned by this Cargo, and may be "associated"
188    /// with a rustc process. They may also be unused, though if so will be
189    /// dropped on the next loop iteration.
190    ///
191    /// Note that the length of this may be zero, but we will still spawn work,
192    /// as we share the implicit token given to this Cargo process with a
193    /// single rustc process.
194    tokens: Vec<Acquired>,
195
196    /// The list of jobs that we have not yet started executing, but have
197    /// retrieved from the `queue`. We eagerly pull jobs off the main queue to
198    /// allow us to request jobserver tokens pretty early.
199    pending_queue: Vec<(Unit, Job, usize)>,
200    print: DiagnosticPrinter<'gctx>,
201
202    /// How many jobs we've finished
203    finished: usize,
204    per_package_future_incompat_reports: Vec<FutureIncompatReportPackage>,
205}
206
207/// Count of warnings, used to print a summary after the job succeeds
208#[derive(Default)]
209pub struct WarningCount {
210    /// total number of warnings
211    pub total: usize,
212    /// number of warnings that were suppressed because they
213    /// were duplicates of a previous warning
214    pub duplicates: usize,
215    /// number of fixable warnings set to `NotAllowed`
216    /// if any errors have been seen ofr the current
217    /// target
218    pub fixable: FixableWarnings,
219}
220
221impl WarningCount {
222    /// If an error is seen this should be called
223    /// to set `fixable` to `NotAllowed`
224    fn disallow_fixable(&mut self) {
225        self.fixable = FixableWarnings::NotAllowed;
226    }
227
228    /// Checks fixable if warnings are allowed
229    /// fixable warnings are allowed if no
230    /// errors have been seen for the current
231    /// target. If an error was seen `fixable`
232    /// will be `NotAllowed`.
233    fn fixable_allowed(&self) -> bool {
234        match &self.fixable {
235            FixableWarnings::NotAllowed => false,
236            _ => true,
237        }
238    }
239}
240
241/// Used to keep track of how many fixable warnings there are
242/// and if fixable warnings are allowed
243#[derive(Default)]
244pub enum FixableWarnings {
245    NotAllowed,
246    #[default]
247    Zero,
248    Positive(usize),
249}
250
251pub struct ErrorsDuringDrain {
252    pub count: usize,
253}
254
255struct ErrorToHandle {
256    error: anyhow::Error,
257
258    /// This field is true for "interesting" errors and false for "mundane"
259    /// errors. If false, we print the above error only if it's the first one
260    /// encountered so far while draining the job queue.
261    ///
262    /// At most places that an error is propagated, we set this to false to
263    /// avoid scenarios where Cargo might end up spewing tons of redundant error
264    /// messages. For example if an i/o stream got closed somewhere, we don't
265    /// care about individually reporting every thread that it broke; just the
266    /// first is enough.
267    ///
268    /// The exception where `print_always` is true is that we do report every
269    /// instance of a rustc invocation that failed with diagnostics. This
270    /// corresponds to errors from `Message::Finish`.
271    print_always: bool,
272}
273
274impl<E> From<E> for ErrorToHandle
275where
276    anyhow::Error: From<E>,
277{
278    fn from(error: E) -> Self {
279        ErrorToHandle {
280            error: anyhow::Error::from(error),
281            print_always: false,
282        }
283    }
284}
285
286#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)]
287pub struct JobId(pub u32);
288
289impl std::fmt::Display for JobId {
290    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
291        write!(f, "{}", self.0)
292    }
293}
294
295/// Handler for deduplicating diagnostics.
296struct DiagDedupe<'gctx> {
297    seen: RefCell<HashSet<u64>>,
298    gctx: &'gctx GlobalContext,
299}
300
301impl<'gctx> DiagDedupe<'gctx> {
302    fn new(gctx: &'gctx GlobalContext) -> Self {
303        DiagDedupe {
304            seen: RefCell::new(HashSet::new()),
305            gctx,
306        }
307    }
308
309    /// Emits a diagnostic message.
310    ///
311    /// Returns `true` if the message was emitted, or `false` if it was
312    /// suppressed for being a duplicate.
313    fn emit_diag(&self, diag: &str) -> CargoResult<bool> {
314        let h = util::hash_u64(diag);
315        if !self.seen.borrow_mut().insert(h) {
316            return Ok(false);
317        }
318        let mut shell = self.gctx.shell();
319        shell.print_ansi_stderr(diag.as_bytes())?;
320        shell.err().write_all(b"\n")?;
321        Ok(true)
322    }
323}
324
325/// Possible artifacts that can be produced by compilations, used as edge values
326/// in the dependency graph.
327///
328/// As edge values we can have multiple kinds of edges depending on one node,
329/// for example some units may only depend on the metadata for an rlib while
330/// others depend on the full rlib. This `Artifact` enum is used to distinguish
331/// this case and track the progress of compilations as they proceed.
332#[derive(Copy, Clone, Eq, PartialEq, Hash, Debug)]
333enum Artifact {
334    /// A generic placeholder for "depends on everything run by a step" and
335    /// means that we can't start the next compilation until the previous has
336    /// finished entirely.
337    All,
338
339    /// A node indicating that we only depend on the metadata of a compilation,
340    /// but the compilation is typically also producing an rlib. We can start
341    /// our step, however, before the full rlib is available.
342    Metadata,
343}
344
345enum Message {
346    Run(JobId, String),
347    BuildPlanMsg(String, ProcessBuilder, Arc<Vec<OutputFile>>),
348    Stdout(String),
349    Stderr(String),
350
351    // This is for general stderr output from subprocesses
352    Diagnostic {
353        id: JobId,
354        level: String,
355        diag: String,
356        fixable: bool,
357    },
358    // This handles duplicate output that is suppressed, for showing
359    // only a count of duplicate messages instead
360    WarningCount {
361        id: JobId,
362        emitted: bool,
363        fixable: bool,
364    },
365    // This is for warnings generated by Cargo's interpretation of the
366    // subprocess output, e.g. scrape-examples prints a warning if a
367    // unit fails to be scraped
368    Warning {
369        id: JobId,
370        warning: String,
371    },
372
373    FixDiagnostic(diagnostic_server::Message),
374    Token(io::Result<Acquired>),
375    Finish(JobId, Artifact, CargoResult<()>),
376    FutureIncompatReport(JobId, Vec<FutureBreakageItem>),
377}
378
379impl<'gctx> JobQueue<'gctx> {
380    pub fn new(bcx: &BuildContext<'_, 'gctx>) -> JobQueue<'gctx> {
381        JobQueue {
382            queue: DependencyQueue::new(),
383            counts: HashMap::new(),
384            timings: Timings::new(bcx, &bcx.roots),
385        }
386    }
387
388    pub fn enqueue(
389        &mut self,
390        build_runner: &BuildRunner<'_, 'gctx>,
391        unit: &Unit,
392        job: Job,
393    ) -> CargoResult<()> {
394        let dependencies = build_runner.unit_deps(unit);
395        let mut queue_deps = dependencies
396            .iter()
397            .filter(|dep| {
398                // Binaries aren't actually needed to *compile* tests, just to run
399                // them, so we don't include this dependency edge in the job graph.
400                // But we shouldn't filter out dependencies being scraped for Rustdoc.
401                (!dep.unit.target.is_test() && !dep.unit.target.is_bin())
402                    || dep.unit.artifact.is_true()
403                    || dep.unit.mode.is_doc_scrape()
404            })
405            .map(|dep| {
406                // Handle the case here where our `unit -> dep` dependency may
407                // only require the metadata, not the full compilation to
408                // finish. Use the tables in `build_runner` to figure out what
409                // kind of artifact is associated with this dependency.
410                let artifact = if build_runner.only_requires_rmeta(unit, &dep.unit) {
411                    Artifact::Metadata
412                } else {
413                    Artifact::All
414                };
415                (dep.unit.clone(), artifact)
416            })
417            .collect::<HashMap<_, _>>();
418
419        // This is somewhat tricky, but we may need to synthesize some
420        // dependencies for this target if it requires full upstream
421        // compilations to have completed. Because of pipelining, some
422        // dependency edges may be `Metadata` due to the above clause (as
423        // opposed to everything being `All`). For example consider:
424        //
425        //    a (binary)
426        //    └ b (lib)
427        //        └ c (lib)
428        //
429        // Here the dependency edge from B to C will be `Metadata`, and the
430        // dependency edge from A to B will be `All`. For A to be compiled,
431        // however, it currently actually needs the full rlib of C. This means
432        // that we need to synthesize a dependency edge for the dependency graph
433        // from A to C. That's done here.
434        //
435        // This will walk all dependencies of the current target, and if any of
436        // *their* dependencies are `Metadata` then we depend on the `All` of
437        // the target as well. This should ensure that edges changed to
438        // `Metadata` propagate upwards `All` dependencies to anything that
439        // transitively contains the `Metadata` edge.
440        if unit.requires_upstream_objects() {
441            for dep in dependencies {
442                depend_on_deps_of_deps(build_runner, &mut queue_deps, dep.unit.clone());
443            }
444
445            fn depend_on_deps_of_deps(
446                build_runner: &BuildRunner<'_, '_>,
447                deps: &mut HashMap<Unit, Artifact>,
448                unit: Unit,
449            ) {
450                for dep in build_runner.unit_deps(&unit) {
451                    if deps.insert(dep.unit.clone(), Artifact::All).is_none() {
452                        depend_on_deps_of_deps(build_runner, deps, dep.unit.clone());
453                    }
454                }
455            }
456        }
457
458        // For now we use a fixed placeholder value for the cost of each unit, but
459        // in the future this could be used to allow users to provide hints about
460        // relative expected costs of units, or this could be automatically set in
461        // a smarter way using timing data from a previous compilation.
462        self.queue.queue(unit.clone(), job, queue_deps, 100);
463        *self.counts.entry(unit.pkg.package_id()).or_insert(0) += 1;
464        Ok(())
465    }
466
467    /// Executes all jobs necessary to build the dependency graph.
468    ///
469    /// This function will spawn off `config.jobs()` workers to build all of the
470    /// necessary dependencies, in order. Freshness is propagated as far as
471    /// possible along each dependency chain.
472    #[tracing::instrument(skip_all)]
473    pub fn execute(
474        mut self,
475        build_runner: &mut BuildRunner<'_, '_>,
476        plan: &mut BuildPlan,
477    ) -> CargoResult<()> {
478        self.queue.queue_finished();
479
480        let progress =
481            Progress::with_style("Building", ProgressStyle::Ratio, build_runner.bcx.gctx);
482        let state = DrainState {
483            total_units: self.queue.len(),
484            queue: self.queue,
485            // 100 here is somewhat arbitrary. It is a few screenfulls of
486            // output, and hopefully at most a few megabytes of memory for
487            // typical messages. If you change this, please update the test
488            // caching_large_output, too.
489            messages: Arc::new(Queue::new(100)),
490            diag_dedupe: DiagDedupe::new(build_runner.bcx.gctx),
491            warning_count: HashMap::new(),
492            active: HashMap::new(),
493            compiled: HashSet::new(),
494            documented: HashSet::new(),
495            scraped: HashSet::new(),
496            counts: self.counts,
497            progress,
498            next_id: 0,
499            timings: self.timings,
500            tokens: Vec::new(),
501            pending_queue: Vec::new(),
502            print: DiagnosticPrinter::new(
503                build_runner.bcx.gctx,
504                &build_runner.bcx.rustc().workspace_wrapper,
505            ),
506            finished: 0,
507            per_package_future_incompat_reports: Vec::new(),
508        };
509
510        // Create a helper thread for acquiring jobserver tokens
511        let messages = state.messages.clone();
512        let helper = build_runner
513            .jobserver
514            .clone()
515            .into_helper_thread(move |token| {
516                messages.push(Message::Token(token));
517            })
518            .context("failed to create helper thread for jobserver management")?;
519
520        // Create a helper thread to manage the diagnostics for rustfix if
521        // necessary.
522        let messages = state.messages.clone();
523        // It is important that this uses `push` instead of `push_bounded` for
524        // now. If someone wants to fix this to be bounded, the `drop`
525        // implementation needs to be changed to avoid possible deadlocks.
526        let _diagnostic_server = build_runner
527            .bcx
528            .build_config
529            .rustfix_diagnostic_server
530            .borrow_mut()
531            .take()
532            .map(move |srv| srv.start(move |msg| messages.push(Message::FixDiagnostic(msg))));
533
534        thread::scope(move |scope| {
535            match state.drain_the_queue(build_runner, plan, scope, &helper) {
536                Some(err) => Err(err),
537                None => Ok(()),
538            }
539        })
540    }
541}
542
543impl<'gctx> DrainState<'gctx> {
544    fn spawn_work_if_possible<'s>(
545        &mut self,
546        build_runner: &mut BuildRunner<'_, '_>,
547        jobserver_helper: &HelperThread,
548        scope: &'s Scope<'s, '_>,
549    ) -> CargoResult<()> {
550        // Dequeue as much work as we can, learning about everything
551        // possible that can run. Note that this is also the point where we
552        // start requesting job tokens. Each job after the first needs to
553        // request a token.
554        while let Some((unit, job, priority)) = self.queue.dequeue() {
555            // We want to keep the pieces of work in the `pending_queue` sorted
556            // by their priorities, and insert the current job at its correctly
557            // sorted position: following the lower priority jobs, and the ones
558            // with the same priority (since they were dequeued before the
559            // current one, we also keep that relation).
560            let idx = self
561                .pending_queue
562                .partition_point(|&(_, _, p)| p <= priority);
563            self.pending_queue.insert(idx, (unit, job, priority));
564            if self.active.len() + self.pending_queue.len() > 1 {
565                jobserver_helper.request_token();
566            }
567        }
568
569        // Now that we've learned of all possible work that we can execute
570        // try to spawn it so long as we've got a jobserver token which says
571        // we're able to perform some parallel work.
572        // The `pending_queue` is sorted in ascending priority order, and we
573        // remove items from its end to schedule the highest priority items
574        // sooner.
575        while self.has_extra_tokens() && !self.pending_queue.is_empty() {
576            let (unit, job, _) = self.pending_queue.pop().unwrap();
577            *self.counts.get_mut(&unit.pkg.package_id()).unwrap() -= 1;
578            if !build_runner.bcx.build_config.build_plan {
579                // Print out some nice progress information.
580                // NOTE: An error here will drop the job without starting it.
581                // That should be OK, since we want to exit as soon as
582                // possible during an error.
583                self.note_working_on(
584                    build_runner.bcx.gctx,
585                    build_runner.bcx.ws.root(),
586                    &unit,
587                    job.freshness(),
588                )?;
589            }
590            self.run(&unit, job, build_runner, scope);
591        }
592
593        Ok(())
594    }
595
596    fn has_extra_tokens(&self) -> bool {
597        self.active.len() < self.tokens.len() + 1
598    }
599
600    fn handle_event(
601        &mut self,
602        build_runner: &mut BuildRunner<'_, '_>,
603        plan: &mut BuildPlan,
604        event: Message,
605    ) -> Result<(), ErrorToHandle> {
606        let warning_handling = build_runner.bcx.gctx.warning_handling()?;
607        match event {
608            Message::Run(id, cmd) => {
609                build_runner
610                    .bcx
611                    .gctx
612                    .shell()
613                    .verbose(|c| c.status("Running", &cmd))?;
614                self.timings.unit_start(id, self.active[&id].clone());
615            }
616            Message::BuildPlanMsg(module_name, cmd, filenames) => {
617                plan.update(&module_name, &cmd, &filenames)?;
618            }
619            Message::Stdout(out) => {
620                writeln!(build_runner.bcx.gctx.shell().out(), "{}", out)?;
621            }
622            Message::Stderr(err) => {
623                let mut shell = build_runner.bcx.gctx.shell();
624                shell.print_ansi_stderr(err.as_bytes())?;
625                shell.err().write_all(b"\n")?;
626            }
627            Message::Diagnostic {
628                id,
629                level,
630                diag,
631                fixable,
632            } => {
633                let emitted = self.diag_dedupe.emit_diag(&diag)?;
634                if level == "warning" {
635                    self.bump_warning_count(id, emitted, fixable);
636                }
637                if level == "error" {
638                    let cnts = self.warning_count.entry(id).or_default();
639                    // If there is an error, the `cargo fix` message should not show
640                    cnts.disallow_fixable();
641                }
642            }
643            Message::Warning { id, warning } => {
644                if warning_handling != WarningHandling::Allow {
645                    build_runner.bcx.gctx.shell().warn(warning)?;
646                }
647                self.bump_warning_count(id, true, false);
648            }
649            Message::WarningCount {
650                id,
651                emitted,
652                fixable,
653            } => {
654                self.bump_warning_count(id, emitted, fixable);
655            }
656            Message::FixDiagnostic(msg) => {
657                self.print.print(&msg)?;
658            }
659            Message::Finish(id, artifact, result) => {
660                let unit = match artifact {
661                    // If `id` has completely finished we remove it
662                    // from the `active` map ...
663                    Artifact::All => {
664                        trace!("end: {:?}", id);
665                        self.finished += 1;
666                        self.report_warning_count(
667                            build_runner,
668                            id,
669                            &build_runner.bcx.rustc().workspace_wrapper,
670                        );
671                        self.active.remove(&id).unwrap()
672                    }
673                    // ... otherwise if it hasn't finished we leave it
674                    // in there as we'll get another `Finish` later on.
675                    Artifact::Metadata => {
676                        trace!("end (meta): {:?}", id);
677                        self.active[&id].clone()
678                    }
679                };
680                debug!("end ({:?}): {:?}", unit, result);
681                match result {
682                    Ok(()) => self.finish(id, &unit, artifact, build_runner)?,
683                    Err(_) if build_runner.bcx.unit_can_fail_for_docscraping(&unit) => {
684                        build_runner
685                            .failed_scrape_units
686                            .lock()
687                            .unwrap()
688                            .insert(build_runner.files().metadata(&unit).unit_id());
689                        self.queue.finish(&unit, &artifact);
690                    }
691                    Err(error) => {
692                        let show_warnings = true;
693                        self.emit_log_messages(&unit, build_runner, show_warnings)?;
694                        self.back_compat_notice(build_runner, &unit)?;
695                        return Err(ErrorToHandle {
696                            error,
697                            print_always: true,
698                        });
699                    }
700                }
701            }
702            Message::FutureIncompatReport(id, items) => {
703                let unit = &self.active[&id];
704                let package_id = unit.pkg.package_id();
705                let is_local = unit.is_local();
706                self.per_package_future_incompat_reports
707                    .push(FutureIncompatReportPackage {
708                        package_id,
709                        is_local,
710                        items,
711                    });
712            }
713            Message::Token(acquired_token) => {
714                let token = acquired_token.context("failed to acquire jobserver token")?;
715                self.tokens.push(token);
716            }
717        }
718
719        Ok(())
720    }
721
722    // This will also tick the progress bar as appropriate
723    fn wait_for_events(&mut self) -> Vec<Message> {
724        // Drain all events at once to avoid displaying the progress bar
725        // unnecessarily. If there's no events we actually block waiting for
726        // an event, but we keep a "heartbeat" going to allow `record_cpu`
727        // to run above to calculate CPU usage over time. To do this we
728        // listen for a message with a timeout, and on timeout we run the
729        // previous parts of the loop again.
730        let mut events = self.messages.try_pop_all();
731        if events.is_empty() {
732            loop {
733                self.tick_progress();
734                self.tokens.truncate(self.active.len() - 1);
735                match self.messages.pop(Duration::from_millis(500)) {
736                    Some(message) => {
737                        events.push(message);
738                        break;
739                    }
740                    None => continue,
741                }
742            }
743        }
744        events
745    }
746
747    /// This is the "main" loop, where Cargo does all work to run the
748    /// compiler.
749    ///
750    /// This returns an Option to prevent the use of `?` on `Result` types
751    /// because it is important for the loop to carefully handle errors.
752    fn drain_the_queue<'s>(
753        mut self,
754        build_runner: &mut BuildRunner<'_, '_>,
755        plan: &mut BuildPlan,
756        scope: &'s Scope<'s, '_>,
757        jobserver_helper: &HelperThread,
758    ) -> Option<anyhow::Error> {
759        trace!("queue: {:#?}", self.queue);
760
761        // Iteratively execute the entire dependency graph. Each turn of the
762        // loop starts out by scheduling as much work as possible (up to the
763        // maximum number of parallel jobs we have tokens for). A local queue
764        // is maintained separately from the main dependency queue as one
765        // dequeue may actually dequeue quite a bit of work (e.g., 10 binaries
766        // in one package).
767        //
768        // After a job has finished we update our internal state if it was
769        // successful and otherwise wait for pending work to finish if it failed
770        // and then immediately return (or keep going, if requested by the build
771        // config).
772        let mut errors = ErrorsDuringDrain { count: 0 };
773        // CAUTION! Do not use `?` or break out of the loop early. Every error
774        // must be handled in such a way that the loop is still allowed to
775        // drain event messages.
776        loop {
777            if errors.count == 0 || build_runner.bcx.build_config.keep_going {
778                if let Err(e) = self.spawn_work_if_possible(build_runner, jobserver_helper, scope) {
779                    self.handle_error(&mut build_runner.bcx.gctx.shell(), &mut errors, e);
780                }
781            }
782
783            // If after all that we're not actually running anything then we're
784            // done!
785            if self.active.is_empty() {
786                break;
787            }
788
789            // And finally, before we block waiting for the next event, drop any
790            // excess tokens we may have accidentally acquired. Due to how our
791            // jobserver interface is architected we may acquire a token that we
792            // don't actually use, and if this happens just relinquish it back
793            // to the jobserver itself.
794            for event in self.wait_for_events() {
795                if let Err(event_err) = self.handle_event(build_runner, plan, event) {
796                    self.handle_error(&mut build_runner.bcx.gctx.shell(), &mut errors, event_err);
797                }
798            }
799        }
800        self.progress.clear();
801
802        let profile_name = build_runner.bcx.build_config.requested_profile;
803        // NOTE: this may be a bit inaccurate, since this may not display the
804        // profile for what was actually built. Profile overrides can change
805        // these settings, and in some cases different targets are built with
806        // different profiles. To be accurate, it would need to collect a
807        // list of Units built, and maybe display a list of the different
808        // profiles used. However, to keep it simple and compatible with old
809        // behavior, we just display what the base profile is.
810        let profile = build_runner.bcx.profiles.base_profile();
811        let mut opt_type = String::from(if profile.opt_level.as_str() == "0" {
812            "unoptimized"
813        } else {
814            "optimized"
815        });
816        if profile.debuginfo.is_turned_on() {
817            opt_type += " + debuginfo";
818        }
819
820        let time_elapsed = util::elapsed(build_runner.bcx.gctx.creation_time().elapsed());
821        if let Err(e) = self.timings.finished(build_runner, &errors.to_error()) {
822            self.handle_error(&mut build_runner.bcx.gctx.shell(), &mut errors, e);
823        }
824        if build_runner.bcx.build_config.emit_json() {
825            let mut shell = build_runner.bcx.gctx.shell();
826            let msg = machine_message::BuildFinished {
827                success: errors.count == 0,
828            }
829            .to_json_string();
830            if let Err(e) = writeln!(shell.out(), "{}", msg) {
831                self.handle_error(&mut shell, &mut errors, e);
832            }
833        }
834
835        if let Some(error) = errors.to_error() {
836            // Any errors up to this point have already been printed via the
837            // `display_error` inside `handle_error`.
838            Some(anyhow::Error::new(AlreadyPrintedError::new(error)))
839        } else if self.queue.is_empty() && self.pending_queue.is_empty() {
840            let profile_link = build_runner.bcx.gctx.shell().err_hyperlink(
841                "https://doc.rust-lang.org/cargo/reference/profiles.html#default-profiles",
842            );
843            let message = format!(
844                "{profile_link}`{profile_name}` profile [{opt_type}]{profile_link:#} target(s) in {time_elapsed}",
845            );
846            if !build_runner.bcx.build_config.build_plan {
847                // It doesn't really matter if this fails.
848                let _ = build_runner.bcx.gctx.shell().status("Finished", message);
849                future_incompat::save_and_display_report(
850                    build_runner.bcx,
851                    &self.per_package_future_incompat_reports,
852                );
853            }
854
855            None
856        } else {
857            debug!("queue: {:#?}", self.queue);
858            Some(internal("finished with jobs still left in the queue"))
859        }
860    }
861
862    fn handle_error(
863        &mut self,
864        shell: &mut Shell,
865        err_state: &mut ErrorsDuringDrain,
866        new_err: impl Into<ErrorToHandle>,
867    ) {
868        let new_err = new_err.into();
869        if new_err.print_always || err_state.count == 0 {
870            crate::display_error(&new_err.error, shell);
871            if err_state.count == 0 && !self.active.is_empty() {
872                self.progress.indicate_error();
873                let _ = shell.warn("build failed, waiting for other jobs to finish...");
874            }
875            err_state.count += 1;
876        } else {
877            tracing::warn!("{:?}", new_err.error);
878        }
879    }
880
881    // This also records CPU usage and marks concurrency; we roughly want to do
882    // this as often as we spin on the events receiver (at least every 500ms or
883    // so).
884    fn tick_progress(&mut self) {
885        // Record some timing information if `--timings` is enabled, and
886        // this'll end up being a noop if we're not recording this
887        // information.
888        self.timings.mark_concurrency(
889            self.active.len(),
890            self.pending_queue.len(),
891            self.queue.len(),
892        );
893        self.timings.record_cpu();
894
895        let active_names = self
896            .active
897            .values()
898            .map(|u| self.name_for_progress(u))
899            .collect::<Vec<_>>();
900        let _ = self.progress.tick_now(
901            self.finished,
902            self.total_units,
903            &format!(": {}", active_names.join(", ")),
904        );
905    }
906
907    fn name_for_progress(&self, unit: &Unit) -> String {
908        let pkg_name = unit.pkg.name();
909        let target_name = unit.target.name();
910        match unit.mode {
911            CompileMode::Doc { .. } => format!("{}(doc)", pkg_name),
912            CompileMode::RunCustomBuild => format!("{}(build)", pkg_name),
913            CompileMode::Test | CompileMode::Check { test: true } => match unit.target.kind() {
914                TargetKind::Lib(_) => format!("{}(test)", target_name),
915                TargetKind::CustomBuild => panic!("cannot test build script"),
916                TargetKind::Bin => format!("{}(bin test)", target_name),
917                TargetKind::Test => format!("{}(test)", target_name),
918                TargetKind::Bench => format!("{}(bench)", target_name),
919                TargetKind::ExampleBin | TargetKind::ExampleLib(_) => {
920                    format!("{}(example test)", target_name)
921                }
922            },
923            _ => match unit.target.kind() {
924                TargetKind::Lib(_) => pkg_name.to_string(),
925                TargetKind::CustomBuild => format!("{}(build.rs)", pkg_name),
926                TargetKind::Bin => format!("{}(bin)", target_name),
927                TargetKind::Test => format!("{}(test)", target_name),
928                TargetKind::Bench => format!("{}(bench)", target_name),
929                TargetKind::ExampleBin | TargetKind::ExampleLib(_) => {
930                    format!("{}(example)", target_name)
931                }
932            },
933        }
934    }
935
936    /// Executes a job.
937    ///
938    /// Fresh jobs block until finished (which should be very fast!), Dirty
939    /// jobs will spawn a thread in the background and return immediately.
940    fn run<'s>(
941        &mut self,
942        unit: &Unit,
943        job: Job,
944        build_runner: &BuildRunner<'_, '_>,
945        scope: &'s Scope<'s, '_>,
946    ) {
947        let id = JobId(self.next_id);
948        self.next_id = self.next_id.checked_add(1).unwrap();
949
950        debug!("start {}: {:?}", id, unit);
951
952        assert!(self.active.insert(id, unit.clone()).is_none());
953
954        let messages = self.messages.clone();
955        let is_fresh = job.freshness().is_fresh();
956        let rmeta_required = build_runner.rmeta_required(unit);
957
958        let doit = move |diag_dedupe| {
959            let state = JobState::new(id, messages, diag_dedupe, rmeta_required);
960            state.run_to_finish(job);
961        };
962
963        match is_fresh {
964            true => {
965                self.timings.add_fresh();
966                // Running a fresh job on the same thread is often much faster than spawning a new
967                // thread to run the job.
968                doit(Some(&self.diag_dedupe));
969            }
970            false => {
971                self.timings.add_dirty();
972                scope.spawn(move || doit(None));
973            }
974        }
975    }
976
977    fn emit_log_messages(
978        &self,
979        unit: &Unit,
980        build_runner: &mut BuildRunner<'_, '_>,
981        show_warnings: bool,
982    ) -> CargoResult<()> {
983        let outputs = build_runner.build_script_outputs.lock().unwrap();
984        let Some(metadata_vec) = build_runner.find_build_script_metadatas(unit) else {
985            return Ok(());
986        };
987        let bcx = &mut build_runner.bcx;
988        for metadata in metadata_vec {
989            if let Some(output) = outputs.get(metadata) {
990                if !output.log_messages.is_empty()
991                    && (show_warnings
992                        || output
993                            .log_messages
994                            .iter()
995                            .any(|(severity, _)| *severity == Severity::Error))
996                {
997                    let msg_with_package =
998                        |msg: &str| format!("{}@{}: {}", unit.pkg.name(), unit.pkg.version(), msg);
999
1000                    for (severity, message) in output.log_messages.iter() {
1001                        match severity {
1002                            Severity::Error => {
1003                                bcx.gctx.shell().error(msg_with_package(message))?;
1004                            }
1005                            Severity::Warning => {
1006                                bcx.gctx.shell().warn(msg_with_package(message))?;
1007                            }
1008                        }
1009                    }
1010                }
1011            }
1012        }
1013
1014        Ok(())
1015    }
1016
1017    fn bump_warning_count(&mut self, id: JobId, emitted: bool, fixable: bool) {
1018        let cnts = self.warning_count.entry(id).or_default();
1019        cnts.total += 1;
1020        if !emitted {
1021            cnts.duplicates += 1;
1022        // Don't add to fixable if it's already been emitted
1023        } else if fixable {
1024            // Do not add anything to the fixable warning count if
1025            // is `NotAllowed` since that indicates there was an
1026            // error while building this `Unit`
1027            if cnts.fixable_allowed() {
1028                cnts.fixable = match cnts.fixable {
1029                    FixableWarnings::NotAllowed => FixableWarnings::NotAllowed,
1030                    FixableWarnings::Zero => FixableWarnings::Positive(1),
1031                    FixableWarnings::Positive(fixable) => FixableWarnings::Positive(fixable + 1),
1032                };
1033            }
1034        }
1035    }
1036
1037    /// Displays a final report of the warnings emitted by a particular job.
1038    fn report_warning_count(
1039        &mut self,
1040        runner: &mut BuildRunner<'_, '_>,
1041        id: JobId,
1042        rustc_workspace_wrapper: &Option<PathBuf>,
1043    ) {
1044        let gctx = runner.bcx.gctx;
1045        let count = match self.warning_count.get(&id) {
1046            // An error could add an entry for a `Unit`
1047            // with 0 warnings but having fixable
1048            // warnings be disallowed
1049            Some(count) if count.total > 0 => count,
1050            None | Some(_) => return,
1051        };
1052        runner.compilation.warning_count += count.total;
1053        let unit = &self.active[&id];
1054        let mut message = descriptive_pkg_name(&unit.pkg.name(), &unit.target, &unit.mode);
1055        message.push_str(" generated ");
1056        match count.total {
1057            1 => message.push_str("1 warning"),
1058            n => {
1059                let _ = write!(message, "{} warnings", n);
1060            }
1061        };
1062        match count.duplicates {
1063            0 => {}
1064            1 => message.push_str(" (1 duplicate)"),
1065            n => {
1066                let _ = write!(message, " ({} duplicates)", n);
1067            }
1068        }
1069        // Only show the `cargo fix` message if its a local `Unit`
1070        if unit.is_local() {
1071            // Do not show this if there are any errors or no fixable warnings
1072            if let FixableWarnings::Positive(fixable) = count.fixable {
1073                // `cargo fix` doesn't have an option for custom builds
1074                if !unit.target.is_custom_build() {
1075                    // To make sure the correct command is shown for `clippy` we
1076                    // check if `RUSTC_WORKSPACE_WRAPPER` is set and pointing towards
1077                    // `clippy-driver`.
1078                    let clippy = std::ffi::OsStr::new("clippy-driver");
1079                    let command = match rustc_workspace_wrapper.as_ref().and_then(|x| x.file_stem())
1080                    {
1081                        Some(wrapper) if wrapper == clippy => "cargo clippy --fix",
1082                        _ => "cargo fix",
1083                    };
1084                    let mut args = {
1085                        let named = unit.target.description_named();
1086                        // if its a lib we need to add the package to fix
1087                        if unit.target.is_lib() {
1088                            format!("{} -p {}", named, unit.pkg.name())
1089                        } else {
1090                            named
1091                        }
1092                    };
1093                    if unit.mode.is_rustc_test()
1094                        && !(unit.target.is_test() || unit.target.is_bench())
1095                    {
1096                        args.push_str(" --tests");
1097                    }
1098                    let mut suggestions = format!("{} suggestion", fixable);
1099                    if fixable > 1 {
1100                        suggestions.push_str("s")
1101                    }
1102                    let _ = write!(
1103                        message,
1104                        " (run `{command} --{args}` to apply {suggestions})"
1105                    );
1106                }
1107            }
1108        }
1109        // Errors are ignored here because it is tricky to handle them
1110        // correctly, and they aren't important.
1111        let _ = gctx.shell().warn(message);
1112    }
1113
1114    fn finish(
1115        &mut self,
1116        id: JobId,
1117        unit: &Unit,
1118        artifact: Artifact,
1119        build_runner: &mut BuildRunner<'_, '_>,
1120    ) -> CargoResult<()> {
1121        if unit.mode.is_run_custom_build() {
1122            self.emit_log_messages(
1123                unit,
1124                build_runner,
1125                unit.show_warnings(build_runner.bcx.gctx),
1126            )?;
1127        }
1128        let unlocked = self.queue.finish(unit, &artifact);
1129        match artifact {
1130            Artifact::All => self.timings.unit_finished(id, unlocked),
1131            Artifact::Metadata => self.timings.unit_rmeta_finished(id, unlocked),
1132        }
1133        Ok(())
1134    }
1135
1136    // This isn't super trivial because we don't want to print loads and
1137    // loads of information to the console, but we also want to produce a
1138    // faithful representation of what's happening. This is somewhat nuanced
1139    // as a package can start compiling *very* early on because of custom
1140    // build commands and such.
1141    //
1142    // In general, we try to print "Compiling" for the first nontrivial task
1143    // run for a package, regardless of when that is. We then don't print
1144    // out any more information for a package after we've printed it once.
1145    fn note_working_on(
1146        &mut self,
1147        gctx: &GlobalContext,
1148        ws_root: &Path,
1149        unit: &Unit,
1150        fresh: &Freshness,
1151    ) -> CargoResult<()> {
1152        if (self.compiled.contains(&unit.pkg.package_id())
1153            && !unit.mode.is_doc()
1154            && !unit.mode.is_doc_scrape())
1155            || (self.documented.contains(&unit.pkg.package_id()) && unit.mode.is_doc())
1156            || (self.scraped.contains(&unit.pkg.package_id()) && unit.mode.is_doc_scrape())
1157        {
1158            return Ok(());
1159        }
1160
1161        match fresh {
1162            // Any dirty stage which runs at least one command gets printed as
1163            // being a compiled package.
1164            Dirty(dirty_reason) => {
1165                if !dirty_reason.is_fresh_build() {
1166                    gctx.shell()
1167                        .verbose(|shell| dirty_reason.present_to(shell, unit, ws_root))?;
1168                }
1169
1170                if unit.mode.is_doc() {
1171                    self.documented.insert(unit.pkg.package_id());
1172                    gctx.shell().status("Documenting", &unit.pkg)?;
1173                } else if unit.mode.is_doc_test() {
1174                    // Skip doc test.
1175                } else if unit.mode.is_doc_scrape() {
1176                    self.scraped.insert(unit.pkg.package_id());
1177                    gctx.shell().status("Scraping", &unit.pkg)?;
1178                } else {
1179                    self.compiled.insert(unit.pkg.package_id());
1180                    if unit.mode.is_check() {
1181                        gctx.shell().status("Checking", &unit.pkg)?;
1182                    } else {
1183                        gctx.shell().status("Compiling", &unit.pkg)?;
1184                    }
1185                }
1186            }
1187            Fresh => {
1188                // If doc test are last, only print "Fresh" if nothing has been printed.
1189                if self.counts[&unit.pkg.package_id()] == 0
1190                    && !(unit.mode.is_doc_test() && self.compiled.contains(&unit.pkg.package_id()))
1191                {
1192                    self.compiled.insert(unit.pkg.package_id());
1193                    gctx.shell().verbose(|c| c.status("Fresh", &unit.pkg))?;
1194                }
1195            }
1196        }
1197        Ok(())
1198    }
1199
1200    fn back_compat_notice(
1201        &self,
1202        build_runner: &BuildRunner<'_, '_>,
1203        unit: &Unit,
1204    ) -> CargoResult<()> {
1205        if unit.pkg.name() != "diesel"
1206            || unit.pkg.version() >= &Version::new(1, 4, 8)
1207            || build_runner.bcx.ws.resolve_behavior() == ResolveBehavior::V1
1208            || !unit.pkg.package_id().source_id().is_registry()
1209            || !unit.features.is_empty()
1210        {
1211            return Ok(());
1212        }
1213        if !build_runner
1214            .bcx
1215            .unit_graph
1216            .keys()
1217            .any(|unit| unit.pkg.name() == "diesel" && !unit.features.is_empty())
1218        {
1219            return Ok(());
1220        }
1221        build_runner.bcx.gctx.shell().note(
1222            "\
1223This error may be due to an interaction between diesel and Cargo's new
1224feature resolver. Try updating to diesel 1.4.8 to fix this error.
1225",
1226        )?;
1227        Ok(())
1228    }
1229}
1230
1231impl ErrorsDuringDrain {
1232    fn to_error(&self) -> Option<anyhow::Error> {
1233        match self.count {
1234            0 => None,
1235            1 => Some(format_err!("1 job failed")),
1236            n => Some(format_err!("{} jobs failed", n)),
1237        }
1238    }
1239}