Rollup merge of #136053 - Zalathar:defer-counters, r=saethlin

coverage: Defer part of counter-creation until codegen

Follow-up to #135481 and #135873.

One of the pleasant properties of the new counter-assignment algorithm is that we can stop partway through the process, store the intermediate state in MIR, and then resume the rest of the algorithm during codegen. This lets it take into account which parts of the control-flow graph were eliminated by MIR opts, resulting in fewer physical counters and simpler counter expressions.

Those improvements end up completely obsoleting much larger chunks of code that were previously responsible for cleaning up the coverage metadata after MIR opts, while also doing a more thorough cleanup job.

(That change also unlocks some further simplifications that I've kept out of this PR to limit its scope.)
This commit is contained in:
Jubilee 2025-02-10 00:51:49 -08:00 committed by GitHub
commit 7f8108afc8
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
28 changed files with 356 additions and 665 deletions

View file

@ -11,7 +11,8 @@ use rustc_codegen_ssa::traits::{
BaseTypeCodegenMethods, ConstCodegenMethods, StaticCodegenMethods,
};
use rustc_middle::mir::coverage::{
CovTerm, CoverageIdsInfo, Expression, FunctionCoverageInfo, Mapping, MappingKind, Op,
BasicCoverageBlock, CovTerm, CoverageIdsInfo, Expression, FunctionCoverageInfo, Mapping,
MappingKind, Op,
};
use rustc_middle::ty::{Instance, TyCtxt};
use rustc_span::Span;
@ -53,7 +54,7 @@ pub(crate) fn prepare_covfun_record<'tcx>(
let fn_cov_info = tcx.instance_mir(instance.def).function_coverage_info.as_deref()?;
let ids_info = tcx.coverage_ids_info(instance.def)?;
let expressions = prepare_expressions(fn_cov_info, ids_info, is_used);
let expressions = prepare_expressions(ids_info);
let mut covfun = CovfunRecord {
mangled_function_name: tcx.symbol_name(instance).name,
@ -75,26 +76,14 @@ pub(crate) fn prepare_covfun_record<'tcx>(
}
/// Convert the function's coverage-counter expressions into a form suitable for FFI.
fn prepare_expressions(
fn_cov_info: &FunctionCoverageInfo,
ids_info: &CoverageIdsInfo,
is_used: bool,
) -> Vec<ffi::CounterExpression> {
// If any counters or expressions were removed by MIR opts, replace their
// terms with zero.
let counter_for_term = |term| {
if !is_used || ids_info.is_zero_term(term) {
ffi::Counter::ZERO
} else {
ffi::Counter::from_term(term)
}
};
fn prepare_expressions(ids_info: &CoverageIdsInfo) -> Vec<ffi::CounterExpression> {
let counter_for_term = ffi::Counter::from_term;
// We know that LLVM will optimize out any unused expressions before
// producing the final coverage map, so there's no need to do the same
// thing on the Rust side unless we're confident we can do much better.
// (See `CounterExpressionsMinimizer` in `CoverageMappingWriter.cpp`.)
fn_cov_info
ids_info
.expressions
.iter()
.map(move |&Expression { lhs, op, rhs }| ffi::CounterExpression {
@ -136,11 +125,16 @@ fn fill_region_tables<'tcx>(
// For each counter/region pair in this function+file, convert it to a
// form suitable for FFI.
let is_zero_term = |term| !covfun.is_used || ids_info.is_zero_term(term);
for &Mapping { ref kind, span } in &fn_cov_info.mappings {
// If the mapping refers to counters/expressions that were removed by
// MIR opts, replace those occurrences with zero.
let kind = kind.map_terms(|term| if is_zero_term(term) { CovTerm::Zero } else { term });
// If this function is unused, replace all counters with zero.
let counter_for_bcb = |bcb: BasicCoverageBlock| -> ffi::Counter {
let term = if covfun.is_used {
ids_info.term_for_bcb[bcb].expect("every BCB in a mapping was given a term")
} else {
CovTerm::Zero
};
ffi::Counter::from_term(term)
};
// Convert the `Span` into coordinates that we can pass to LLVM, or
// discard the span if conversion fails. In rare, cases _all_ of a
@ -154,23 +148,22 @@ fn fill_region_tables<'tcx>(
continue;
}
match kind {
MappingKind::Code(term) => {
code_regions
.push(ffi::CodeRegion { cov_span, counter: ffi::Counter::from_term(term) });
match *kind {
MappingKind::Code { bcb } => {
code_regions.push(ffi::CodeRegion { cov_span, counter: counter_for_bcb(bcb) });
}
MappingKind::Branch { true_term, false_term } => {
MappingKind::Branch { true_bcb, false_bcb } => {
branch_regions.push(ffi::BranchRegion {
cov_span,
true_counter: ffi::Counter::from_term(true_term),
false_counter: ffi::Counter::from_term(false_term),
true_counter: counter_for_bcb(true_bcb),
false_counter: counter_for_bcb(false_bcb),
});
}
MappingKind::MCDCBranch { true_term, false_term, mcdc_params } => {
MappingKind::MCDCBranch { true_bcb, false_bcb, mcdc_params } => {
mcdc_branch_regions.push(ffi::MCDCBranchRegion {
cov_span,
true_counter: ffi::Counter::from_term(true_term),
false_counter: ffi::Counter::from_term(false_term),
true_counter: counter_for_bcb(true_bcb),
false_counter: counter_for_bcb(false_bcb),
mcdc_branch_params: ffi::mcdc::BranchParameters::from(mcdc_params),
});
}

View file

@ -160,21 +160,12 @@ impl<'tcx> CoverageInfoBuilderMethods<'tcx> for Builder<'_, '_, 'tcx> {
CoverageKind::SpanMarker | CoverageKind::BlockMarker { .. } => unreachable!(
"marker statement {kind:?} should have been removed by CleanupPostBorrowck"
),
CoverageKind::CounterIncrement { id } => {
// The number of counters passed to `llvm.instrprof.increment` might
// be smaller than the number originally inserted by the instrumentor,
// if some high-numbered counters were removed by MIR optimizations.
// If so, LLVM's profiler runtime will use fewer physical counters.
let num_counters = ids_info.num_counters_after_mir_opts();
assert!(
num_counters as usize <= function_coverage_info.num_counters,
"num_counters disagreement: query says {num_counters} but function info only has {}",
function_coverage_info.num_counters
);
CoverageKind::VirtualCounter { bcb }
if let Some(&id) = ids_info.phys_counter_for_node.get(&bcb) =>
{
let fn_name = bx.get_pgo_func_name_var(instance);
let hash = bx.const_u64(function_coverage_info.function_source_hash);
let num_counters = bx.const_u32(num_counters);
let num_counters = bx.const_u32(ids_info.num_counters);
let index = bx.const_u32(id.as_u32());
debug!(
"codegen intrinsic instrprof.increment(fn_name={:?}, hash={:?}, num_counters={:?}, index={:?})",
@ -182,10 +173,8 @@ impl<'tcx> CoverageInfoBuilderMethods<'tcx> for Builder<'_, '_, 'tcx> {
);
bx.instrprof_increment(fn_name, hash, num_counters, index);
}
CoverageKind::ExpressionUsed { id: _ } => {
// Expression-used statements are markers that are handled by
// `coverage_ids_info`, so there's nothing to codegen here.
}
// If a BCB doesn't have an associated physical counter, there's nothing to codegen.
CoverageKind::VirtualCounter { .. } => {}
CoverageKind::CondBitmapUpdate { index, decision_depth } => {
let cond_bitmap = coverage_cx
.try_get_mcdc_condition_bitmap(&instance, decision_depth)

View file

@ -13,6 +13,7 @@
#![feature(extern_types)]
#![feature(file_buffered)]
#![feature(hash_raw_entry)]
#![feature(if_let_guard)]
#![feature(impl_trait_in_assoc_type)]
#![feature(iter_intersperse)]
#![feature(let_chains)]

View file

@ -2,8 +2,8 @@
use std::fmt::{self, Debug, Formatter};
use rustc_index::IndexVec;
use rustc_index::bit_set::DenseBitSet;
use rustc_data_structures::fx::FxIndexMap;
use rustc_index::{Idx, IndexVec};
use rustc_macros::{HashStable, TyDecodable, TyEncodable};
use rustc_span::Span;
@ -103,23 +103,12 @@ pub enum CoverageKind {
/// Should be erased before codegen (at some point after `InstrumentCoverage`).
BlockMarker { id: BlockMarkerId },
/// Marks the point in MIR control flow represented by a coverage counter.
/// Marks its enclosing basic block with the ID of the coverage graph node
/// that it was part of during the `InstrumentCoverage` MIR pass.
///
/// This is eventually lowered to `llvm.instrprof.increment` in LLVM IR.
///
/// If this statement does not survive MIR optimizations, any mappings that
/// refer to this counter can have those references simplified to zero.
CounterIncrement { id: CounterId },
/// Marks the point in MIR control-flow represented by a coverage expression.
///
/// If this statement does not survive MIR optimizations, any mappings that
/// refer to this expression can have those references simplified to zero.
///
/// (This is only inserted for expression IDs that are directly used by
/// mappings. Intermediate expressions with no direct mappings are
/// retained/zeroed based on whether they are transitively used.)
ExpressionUsed { id: ExpressionId },
/// During codegen, this might be lowered to `llvm.instrprof.increment` or
/// to a no-op, depending on the outcome of counter-creation.
VirtualCounter { bcb: BasicCoverageBlock },
/// Marks the point in MIR control flow represented by a evaluated condition.
///
@ -138,8 +127,7 @@ impl Debug for CoverageKind {
match self {
SpanMarker => write!(fmt, "SpanMarker"),
BlockMarker { id } => write!(fmt, "BlockMarker({:?})", id.index()),
CounterIncrement { id } => write!(fmt, "CounterIncrement({:?})", id.index()),
ExpressionUsed { id } => write!(fmt, "ExpressionUsed({:?})", id.index()),
VirtualCounter { bcb } => write!(fmt, "VirtualCounter({bcb:?})"),
CondBitmapUpdate { index, decision_depth } => {
write!(fmt, "CondBitmapUpdate(index={:?}, depth={:?})", index, decision_depth)
}
@ -179,34 +167,19 @@ pub struct Expression {
#[derive(TyEncodable, TyDecodable, Hash, HashStable)]
pub enum MappingKind {
/// Associates a normal region of code with a counter/expression/zero.
Code(CovTerm),
Code { bcb: BasicCoverageBlock },
/// Associates a branch region with separate counters for true and false.
Branch { true_term: CovTerm, false_term: CovTerm },
Branch { true_bcb: BasicCoverageBlock, false_bcb: BasicCoverageBlock },
/// Associates a branch region with separate counters for true and false.
MCDCBranch { true_term: CovTerm, false_term: CovTerm, mcdc_params: ConditionInfo },
MCDCBranch {
true_bcb: BasicCoverageBlock,
false_bcb: BasicCoverageBlock,
mcdc_params: ConditionInfo,
},
/// Associates a decision region with a bitmap and number of conditions.
MCDCDecision(DecisionInfo),
}
impl MappingKind {
/// Returns a copy of this mapping kind, in which all coverage terms have
/// been replaced with ones returned by the given function.
pub fn map_terms(&self, map_fn: impl Fn(CovTerm) -> CovTerm) -> Self {
match *self {
Self::Code(term) => Self::Code(map_fn(term)),
Self::Branch { true_term, false_term } => {
Self::Branch { true_term: map_fn(true_term), false_term: map_fn(false_term) }
}
Self::MCDCBranch { true_term, false_term, mcdc_params } => Self::MCDCBranch {
true_term: map_fn(true_term),
false_term: map_fn(false_term),
mcdc_params,
},
Self::MCDCDecision(param) => Self::MCDCDecision(param),
}
}
}
#[derive(Clone, Debug)]
#[derive(TyEncodable, TyDecodable, Hash, HashStable)]
pub struct Mapping {
@ -222,10 +195,15 @@ pub struct Mapping {
pub struct FunctionCoverageInfo {
pub function_source_hash: u64,
pub body_span: Span,
pub num_counters: usize,
pub mcdc_bitmap_bits: usize,
pub expressions: IndexVec<ExpressionId, Expression>,
/// Used in conjunction with `priority_list` to create physical counters
/// and counter expressions, after MIR optimizations.
pub node_flow_data: NodeFlowData<BasicCoverageBlock>,
pub priority_list: Vec<BasicCoverageBlock>,
pub mappings: Vec<Mapping>,
pub mcdc_bitmap_bits: usize,
/// The depth of the deepest decision is used to know how many
/// temp condbitmaps should be allocated for the function.
pub mcdc_num_condition_bitmaps: usize,
@ -292,40 +270,55 @@ pub struct MCDCDecisionSpan {
pub num_conditions: usize,
}
/// Summarizes coverage IDs inserted by the `InstrumentCoverage` MIR pass
/// (for compiler option `-Cinstrument-coverage`), after MIR optimizations
/// have had a chance to potentially remove some of them.
/// Contains information needed during codegen, obtained by inspecting the
/// function's MIR after MIR optimizations.
///
/// Used by the `coverage_ids_info` query.
/// Returned by the `coverage_ids_info` query.
#[derive(Clone, TyEncodable, TyDecodable, Debug, HashStable)]
pub struct CoverageIdsInfo {
pub counters_seen: DenseBitSet<CounterId>,
pub zero_expressions: DenseBitSet<ExpressionId>,
pub num_counters: u32,
pub phys_counter_for_node: FxIndexMap<BasicCoverageBlock, CounterId>,
pub term_for_bcb: IndexVec<BasicCoverageBlock, Option<CovTerm>>,
pub expressions: IndexVec<ExpressionId, Expression>,
}
impl CoverageIdsInfo {
/// Coverage codegen needs to know how many coverage counters are ever
/// incremented within a function, so that it can set the `num-counters`
/// argument of the `llvm.instrprof.increment` intrinsic.
rustc_index::newtype_index! {
/// During the `InstrumentCoverage` MIR pass, a BCB is a node in the
/// "coverage graph", which is a refinement of the MIR control-flow graph
/// that merges or omits some blocks that aren't relevant to coverage.
///
/// This may be less than the highest counter ID emitted by the
/// InstrumentCoverage MIR pass, if the highest-numbered counter increments
/// were removed by MIR optimizations.
pub fn num_counters_after_mir_opts(&self) -> u32 {
// FIXME(Zalathar): Currently this treats an unused counter as "used"
// if its ID is less than that of the highest counter that really is
// used. Fixing this would require adding a renumbering step somewhere.
self.counters_seen.last_set_in(..).map_or(0, |max| max.as_u32() + 1)
}
/// Returns `true` if the given term is known to have a value of zero, taking
/// into account knowledge of which counters are unused and which expressions
/// are always zero.
pub fn is_zero_term(&self, term: CovTerm) -> bool {
match term {
CovTerm::Zero => true,
CovTerm::Counter(id) => !self.counters_seen.contains(id),
CovTerm::Expression(id) => self.zero_expressions.contains(id),
}
/// After that pass is complete, the coverage graph no longer exists, so a
/// BCB is effectively an opaque ID.
#[derive(HashStable)]
#[encodable]
#[orderable]
#[debug_format = "bcb{}"]
pub struct BasicCoverageBlock {
const START_BCB = 0;
}
}
/// Data representing a view of some underlying graph, in which each node's
/// successors have been merged into a single "supernode".
///
/// The resulting supernodes have no obvious meaning on their own.
/// However, merging successor nodes means that a node's out-edges can all
/// be combined into a single out-edge, whose flow is the same as the flow
/// (execution count) of its corresponding node in the original graph.
///
/// With all node flows now in the original graph now represented as edge flows
/// in the merged graph, it becomes possible to analyze the original node flows
/// using techniques for analyzing edge flows.
#[derive(Clone, Debug)]
#[derive(TyEncodable, TyDecodable, Hash, HashStable)]
pub struct NodeFlowData<Node: Idx> {
/// Maps each node to the supernode that contains it, indicated by some
/// arbitrary "root" node that is part of that supernode.
pub supernodes: IndexVec<Node, Node>,
/// For each node, stores the single supernode that all of its successors
/// have been merged into.
///
/// (Note that each node in a supernode can potentially have a _different_
/// successor supernode from its peers.)
pub succ_supernodes: IndexVec<Node, Node>,
}

View file

@ -619,13 +619,9 @@ fn write_function_coverage_info(
function_coverage_info: &coverage::FunctionCoverageInfo,
w: &mut dyn io::Write,
) -> io::Result<()> {
let coverage::FunctionCoverageInfo { body_span, expressions, mappings, .. } =
function_coverage_info;
let coverage::FunctionCoverageInfo { body_span, mappings, .. } = function_coverage_info;
writeln!(w, "{INDENT}coverage body span: {body_span:?}")?;
for (id, expression) in expressions.iter_enumerated() {
writeln!(w, "{INDENT}coverage {id:?} => {expression:?};")?;
}
for coverage::Mapping { kind, span } in mappings {
writeln!(w, "{INDENT}coverage {kind:?} => {span:?};")?;
}

View file

@ -614,9 +614,16 @@ rustc_queries! {
feedable
}
/// Summarizes coverage IDs inserted by the `InstrumentCoverage` MIR pass
/// (for compiler option `-Cinstrument-coverage`), after MIR optimizations
/// have had a chance to potentially remove some of them.
/// Scans through a function's MIR after MIR optimizations, to prepare the
/// information needed by codegen when `-Cinstrument-coverage` is active.
///
/// This includes the details of where to insert `llvm.instrprof.increment`
/// intrinsics, and the expression tables to be embedded in the function's
/// coverage metadata.
///
/// FIXME(Zalathar): This query's purpose has drifted a bit and should
/// probably be renamed, but that can wait until after the potential
/// follow-ups to #136053 have settled down.
///
/// Returns `None` for functions that were not instrumented.
query coverage_ids_info(key: ty::InstanceKind<'tcx>) -> Option<&'tcx mir::coverage::CoverageIdsInfo> {

View file

@ -2,7 +2,6 @@ use std::cmp::Ordering;
use either::Either;
use itertools::Itertools;
use rustc_data_structures::captures::Captures;
use rustc_data_structures::fx::{FxHashMap, FxIndexMap};
use rustc_data_structures::graph::DirectedGraph;
use rustc_index::IndexVec;
@ -11,31 +10,35 @@ use rustc_middle::mir::coverage::{CounterId, CovTerm, Expression, ExpressionId,
use crate::coverage::counters::balanced_flow::BalancedFlowGraph;
use crate::coverage::counters::node_flow::{
CounterTerm, NodeCounters, make_node_counters, node_flow_data_for_balanced_graph,
CounterTerm, NodeCounters, NodeFlowData, node_flow_data_for_balanced_graph,
};
use crate::coverage::graph::{BasicCoverageBlock, CoverageGraph};
mod balanced_flow;
mod node_flow;
pub(crate) mod node_flow;
mod union_find;
/// Ensures that each BCB node needing a counter has one, by creating physical
/// counters or counter expressions for nodes as required.
pub(super) fn make_bcb_counters(
graph: &CoverageGraph,
bcb_needs_counter: &DenseBitSet<BasicCoverageBlock>,
) -> CoverageCounters {
/// Struct containing the results of [`prepare_bcb_counters_data`].
pub(crate) struct BcbCountersData {
pub(crate) node_flow_data: NodeFlowData<BasicCoverageBlock>,
pub(crate) priority_list: Vec<BasicCoverageBlock>,
}
/// Analyzes the coverage graph to create intermediate data structures that
/// will later be used (during codegen) to create physical counters or counter
/// expressions for each BCB node that needs one.
pub(crate) fn prepare_bcb_counters_data(graph: &CoverageGraph) -> BcbCountersData {
// Create the derived graphs that are necessary for subsequent steps.
let balanced_graph = BalancedFlowGraph::for_graph(graph, |n| !graph[n].is_out_summable);
let node_flow_data = node_flow_data_for_balanced_graph(&balanced_graph);
// Use those graphs to determine which nodes get physical counters, and how
// to compute the execution counts of other nodes from those counters.
// Also create a "priority list" of coverage graph nodes, to help determine
// which ones get physical counters or counter expressions. This needs to
// be done now, because the later parts of the counter-creation process
// won't have access to the original coverage graph.
let priority_list = make_node_flow_priority_list(graph, balanced_graph);
let node_counters = make_node_counters(&node_flow_data, &priority_list);
// Convert the counters into a form suitable for embedding into MIR.
transcribe_counters(&node_counters, bcb_needs_counter)
BcbCountersData { node_flow_data, priority_list }
}
/// Arranges the nodes in `balanced_graph` into a list, such that earlier nodes
@ -74,31 +77,33 @@ fn make_node_flow_priority_list(
}
// Converts node counters into a form suitable for embedding into MIR.
fn transcribe_counters(
pub(crate) fn transcribe_counters(
old: &NodeCounters<BasicCoverageBlock>,
bcb_needs_counter: &DenseBitSet<BasicCoverageBlock>,
bcbs_seen: &DenseBitSet<BasicCoverageBlock>,
) -> CoverageCounters {
let mut new = CoverageCounters::with_num_bcbs(bcb_needs_counter.domain_size());
for bcb in bcb_needs_counter.iter() {
if !bcbs_seen.contains(bcb) {
// This BCB's code was removed by MIR opts, so its counter is always zero.
new.set_node_counter(bcb, CovTerm::Zero);
continue;
}
// Our counter-creation algorithm doesn't guarantee that a node's list
// of terms starts or ends with a positive term, so partition the
// counters into "positive" and "negative" lists for easier handling.
let (mut pos, mut neg): (Vec<_>, Vec<_>) =
old.counter_terms[bcb].iter().partition_map(|&CounterTerm { node, op }| match op {
let (mut pos, mut neg): (Vec<_>, Vec<_>) = old.counter_terms[bcb]
.iter()
// Filter out any BCBs that were removed by MIR opts;
// this treats them as having an execution count of 0.
.filter(|term| bcbs_seen.contains(term.node))
.partition_map(|&CounterTerm { node, op }| match op {
Op::Add => Either::Left(node),
Op::Subtract => Either::Right(node),
});
if pos.is_empty() {
// If we somehow end up with no positive terms, fall back to
// creating a physical counter. There's no known way for this
// to happen, but we can avoid an ICE if it does.
debug_assert!(false, "{bcb:?} has no positive counter terms");
pos = vec![bcb];
neg = vec![];
}
// These intermediate sorts are not strictly necessary, but were helpful
// in reducing churn when switching to the current counter-creation scheme.
// They also help to slightly decrease the overall size of the expression
@ -116,7 +121,7 @@ fn transcribe_counters(
pos.sort();
neg.sort();
let pos_counter = new.make_sum(&pos).expect("`pos` should not be empty");
let pos_counter = new.make_sum(&pos).unwrap_or(CovTerm::Zero);
let new_counter = new.make_subtracted_sum(pos_counter, &neg);
new.set_node_counter(bcb, new_counter);
}
@ -129,15 +134,15 @@ fn transcribe_counters(
pub(super) struct CoverageCounters {
/// List of places where a counter-increment statement should be injected
/// into MIR, each with its corresponding counter ID.
phys_counter_for_node: FxIndexMap<BasicCoverageBlock, CounterId>,
next_counter_id: CounterId,
pub(crate) phys_counter_for_node: FxIndexMap<BasicCoverageBlock, CounterId>,
pub(crate) next_counter_id: CounterId,
/// Coverage counters/expressions that are associated with individual BCBs.
node_counters: IndexVec<BasicCoverageBlock, Option<CovTerm>>,
pub(crate) node_counters: IndexVec<BasicCoverageBlock, Option<CovTerm>>,
/// Table of expression data, associating each expression ID with its
/// corresponding operator (+ or -) and its LHS/RHS operands.
expressions: IndexVec<ExpressionId, Expression>,
pub(crate) expressions: IndexVec<ExpressionId, Expression>,
/// Remember expressions that have already been created (or simplified),
/// so that we don't create unnecessary duplicates.
expressions_memo: FxHashMap<Expression, CovTerm>,
@ -188,12 +193,6 @@ impl CoverageCounters {
self.make_expression(lhs, Op::Subtract, rhs_sum)
}
pub(super) fn num_counters(&self) -> usize {
let num_counters = self.phys_counter_for_node.len();
assert_eq!(num_counters, self.next_counter_id.as_usize());
num_counters
}
fn set_node_counter(&mut self, bcb: BasicCoverageBlock, counter: CovTerm) -> CovTerm {
let existing = self.node_counters[bcb].replace(counter);
assert!(
@ -202,34 +201,4 @@ impl CoverageCounters {
);
counter
}
pub(super) fn term_for_bcb(&self, bcb: BasicCoverageBlock) -> Option<CovTerm> {
self.node_counters[bcb]
}
/// Returns an iterator over all the nodes in the coverage graph that
/// should have a counter-increment statement injected into MIR, along with
/// each site's corresponding counter ID.
pub(super) fn counter_increment_sites(
&self,
) -> impl Iterator<Item = (CounterId, BasicCoverageBlock)> + Captures<'_> {
self.phys_counter_for_node.iter().map(|(&site, &id)| (id, site))
}
/// Returns an iterator over the subset of BCB nodes that have been associated
/// with a counter *expression*, along with the ID of that expression.
pub(super) fn bcb_nodes_with_coverage_expressions(
&self,
) -> impl Iterator<Item = (BasicCoverageBlock, ExpressionId)> + Captures<'_> {
self.node_counters.iter_enumerated().filter_map(|(bcb, &counter)| match counter {
// Yield the BCB along with its associated expression ID.
Some(CovTerm::Expression(id)) => Some((bcb, id)),
// This BCB is associated with a counter or nothing, so skip it.
Some(CovTerm::Counter { .. } | CovTerm::Zero) | None => None,
})
}
pub(super) fn into_expressions(self) -> IndexVec<ExpressionId, Expression> {
self.expressions
}
}

View file

@ -9,6 +9,7 @@
use rustc_data_structures::graph;
use rustc_index::bit_set::DenseBitSet;
use rustc_index::{Idx, IndexSlice, IndexVec};
pub(crate) use rustc_middle::mir::coverage::NodeFlowData;
use rustc_middle::mir::coverage::Op;
use crate::coverage::counters::union_find::UnionFind;
@ -16,30 +17,6 @@ use crate::coverage::counters::union_find::UnionFind;
#[cfg(test)]
mod tests;
/// Data representing a view of some underlying graph, in which each node's
/// successors have been merged into a single "supernode".
///
/// The resulting supernodes have no obvious meaning on their own.
/// However, merging successor nodes means that a node's out-edges can all
/// be combined into a single out-edge, whose flow is the same as the flow
/// (execution count) of its corresponding node in the original graph.
///
/// With all node flows now in the original graph now represented as edge flows
/// in the merged graph, it becomes possible to analyze the original node flows
/// using techniques for analyzing edge flows.
#[derive(Debug)]
pub(crate) struct NodeFlowData<Node: Idx> {
/// Maps each node to the supernode that contains it, indicated by some
/// arbitrary "root" node that is part of that supernode.
supernodes: IndexVec<Node, Node>,
/// For each node, stores the single supernode that all of its successors
/// have been merged into.
///
/// (Note that each node in a supernode can potentially have a _different_
/// successor supernode from its peers.)
succ_supernodes: IndexVec<Node, Node>,
}
/// Creates a "merged" view of an underlying graph.
///
/// The given graph is assumed to have [“balanced flow”](balanced-flow),

View file

@ -8,6 +8,7 @@ use rustc_data_structures::graph::dominators::Dominators;
use rustc_data_structures::graph::{self, DirectedGraph, StartNode};
use rustc_index::IndexVec;
use rustc_index::bit_set::DenseBitSet;
pub(crate) use rustc_middle::mir::coverage::{BasicCoverageBlock, START_BCB};
use rustc_middle::mir::{self, BasicBlock, Terminator, TerminatorKind};
use tracing::debug;
@ -269,15 +270,6 @@ impl graph::Predecessors for CoverageGraph {
}
}
rustc_index::newtype_index! {
/// A node in the control-flow graph of CoverageGraph.
#[orderable]
#[debug_format = "bcb{}"]
pub(crate) struct BasicCoverageBlock {
const START_BCB = 0;
}
}
/// `BasicCoverageBlockData` holds the data indexed by a `BasicCoverageBlock`.
///
/// A `BasicCoverageBlock` (BCB) represents the maximal-length sequence of MIR `BasicBlock`s without

View file

@ -1,9 +1,7 @@
use std::collections::BTreeSet;
use rustc_data_structures::fx::FxIndexMap;
use rustc_data_structures::graph::DirectedGraph;
use rustc_index::IndexVec;
use rustc_index::bit_set::DenseBitSet;
use rustc_middle::mir::coverage::{
BlockMarkerId, BranchSpan, ConditionId, ConditionInfo, CoverageInfoHi, CoverageKind,
};
@ -63,10 +61,6 @@ const MCDC_MAX_BITMAP_SIZE: usize = i32::MAX as usize;
#[derive(Default)]
pub(super) struct ExtractedMappings {
/// Store our own copy of [`CoverageGraph::num_nodes`], so that we don't
/// need access to the whole graph when allocating per-BCB data. This is
/// only public so that other code can still use exhaustive destructuring.
pub(super) num_bcbs: usize,
pub(super) code_mappings: Vec<CodeMapping>,
pub(super) branch_pairs: Vec<BranchPair>,
pub(super) mcdc_bitmap_bits: usize,
@ -118,7 +112,6 @@ pub(super) fn extract_all_mapping_info_from_mir<'tcx>(
);
ExtractedMappings {
num_bcbs: graph.num_nodes(),
code_mappings,
branch_pairs,
mcdc_bitmap_bits,
@ -127,60 +120,6 @@ pub(super) fn extract_all_mapping_info_from_mir<'tcx>(
}
}
impl ExtractedMappings {
pub(super) fn all_bcbs_with_counter_mappings(&self) -> DenseBitSet<BasicCoverageBlock> {
// Fully destructure self to make sure we don't miss any fields that have mappings.
let Self {
num_bcbs,
code_mappings,
branch_pairs,
mcdc_bitmap_bits: _,
mcdc_degraded_branches,
mcdc_mappings,
} = self;
// Identify which BCBs have one or more mappings.
let mut bcbs_with_counter_mappings = DenseBitSet::new_empty(*num_bcbs);
let mut insert = |bcb| {
bcbs_with_counter_mappings.insert(bcb);
};
for &CodeMapping { span: _, bcb } in code_mappings {
insert(bcb);
}
for &BranchPair { true_bcb, false_bcb, .. } in branch_pairs {
insert(true_bcb);
insert(false_bcb);
}
for &MCDCBranch { true_bcb, false_bcb, .. } in mcdc_degraded_branches
.iter()
.chain(mcdc_mappings.iter().map(|(_, branches)| branches.into_iter()).flatten())
{
insert(true_bcb);
insert(false_bcb);
}
// MC/DC decisions refer to BCBs, but don't require those BCBs to have counters.
if bcbs_with_counter_mappings.is_empty() {
debug_assert!(
mcdc_mappings.is_empty(),
"A function with no counter mappings shouldn't have any decisions: {mcdc_mappings:?}",
);
}
bcbs_with_counter_mappings
}
/// Returns the set of BCBs that have one or more `Code` mappings.
pub(super) fn bcbs_with_ordinary_code_mappings(&self) -> DenseBitSet<BasicCoverageBlock> {
let mut bcbs = DenseBitSet::new_empty(self.num_bcbs);
for &CodeMapping { span: _, bcb } in &self.code_mappings {
bcbs.insert(bcb);
}
bcbs
}
}
fn resolve_block_markers(
coverage_info_hi: &CoverageInfoHi,
mir_body: &mir::Body<'_>,

View file

@ -21,7 +21,7 @@ use rustc_span::Span;
use rustc_span::def_id::LocalDefId;
use tracing::{debug, debug_span, trace};
use crate::coverage::counters::CoverageCounters;
use crate::coverage::counters::BcbCountersData;
use crate::coverage::graph::CoverageGraph;
use crate::coverage::mappings::ExtractedMappings;
@ -82,28 +82,21 @@ fn instrument_function_for_coverage<'tcx>(tcx: TyCtxt<'tcx>, mir_body: &mut mir:
let extracted_mappings =
mappings::extract_all_mapping_info_from_mir(tcx, mir_body, &hir_info, &graph);
////////////////////////////////////////////////////
// Create an optimized mix of `Counter`s and `Expression`s for the `CoverageGraph`. Ensure
// every coverage span has a `Counter` or `Expression` assigned to its `BasicCoverageBlock`
// and all `Expression` dependencies (operands) are also generated, for any other
// `BasicCoverageBlock`s not already associated with a coverage span.
let bcbs_with_counter_mappings = extracted_mappings.all_bcbs_with_counter_mappings();
if bcbs_with_counter_mappings.is_empty() {
// No relevant spans were found in MIR, so skip instrumenting this function.
return;
}
let coverage_counters = counters::make_bcb_counters(&graph, &bcbs_with_counter_mappings);
let mappings = create_mappings(&extracted_mappings, &coverage_counters);
let mappings = create_mappings(&extracted_mappings);
if mappings.is_empty() {
// No spans could be converted into valid mappings, so skip this function.
debug!("no spans could be converted into valid mappings; skipping");
return;
}
inject_coverage_statements(mir_body, &graph, &extracted_mappings, &coverage_counters);
// Use the coverage graph to prepare intermediate data that will eventually
// be used to assign physical counters and counter expressions to points in
// the control-flow graph
let BcbCountersData { node_flow_data, priority_list } =
counters::prepare_bcb_counters_data(&graph);
// Inject coverage statements into MIR.
inject_coverage_statements(mir_body, &graph);
inject_mcdc_statements(mir_body, &graph, &extracted_mappings);
let mcdc_num_condition_bitmaps = extracted_mappings
@ -116,29 +109,25 @@ fn instrument_function_for_coverage<'tcx>(tcx: TyCtxt<'tcx>, mir_body: &mut mir:
mir_body.function_coverage_info = Some(Box::new(FunctionCoverageInfo {
function_source_hash: hir_info.function_source_hash,
body_span: hir_info.body_span,
num_counters: coverage_counters.num_counters(),
mcdc_bitmap_bits: extracted_mappings.mcdc_bitmap_bits,
expressions: coverage_counters.into_expressions(),
node_flow_data,
priority_list,
mappings,
mcdc_bitmap_bits: extracted_mappings.mcdc_bitmap_bits,
mcdc_num_condition_bitmaps,
}));
}
/// For each coverage span extracted from MIR, create a corresponding
/// mapping.
/// For each coverage span extracted from MIR, create a corresponding mapping.
///
/// Precondition: All BCBs corresponding to those spans have been given
/// coverage counters.
fn create_mappings(
extracted_mappings: &ExtractedMappings,
coverage_counters: &CoverageCounters,
) -> Vec<Mapping> {
let term_for_bcb =
|bcb| coverage_counters.term_for_bcb(bcb).expect("all BCBs with spans were given counters");
/// FIXME(Zalathar): This used to be where BCBs in the extracted mappings were
/// resolved to a `CovTerm`. But that is now handled elsewhere, so this
/// function can potentially be simplified even further.
fn create_mappings(extracted_mappings: &ExtractedMappings) -> Vec<Mapping> {
// Fully destructure the mappings struct to make sure we don't miss any kinds.
let ExtractedMappings {
num_bcbs: _,
code_mappings,
branch_pairs,
mcdc_bitmap_bits: _,
@ -150,23 +139,18 @@ fn create_mappings(
mappings.extend(code_mappings.iter().map(
// Ordinary code mappings are the simplest kind.
|&mappings::CodeMapping { span, bcb }| {
let kind = MappingKind::Code(term_for_bcb(bcb));
let kind = MappingKind::Code { bcb };
Mapping { kind, span }
},
));
mappings.extend(branch_pairs.iter().map(
|&mappings::BranchPair { span, true_bcb, false_bcb }| {
let true_term = term_for_bcb(true_bcb);
let false_term = term_for_bcb(false_bcb);
let kind = MappingKind::Branch { true_term, false_term };
let kind = MappingKind::Branch { true_bcb, false_bcb };
Mapping { kind, span }
},
));
let term_for_bcb =
|bcb| coverage_counters.term_for_bcb(bcb).expect("all BCBs with spans were given counters");
// MCDC branch mappings are appended with their decisions in case decisions were ignored.
mappings.extend(mcdc_degraded_branches.iter().map(
|&mappings::MCDCBranch {
@ -176,11 +160,7 @@ fn create_mappings(
condition_info: _,
true_index: _,
false_index: _,
}| {
let true_term = term_for_bcb(true_bcb);
let false_term = term_for_bcb(false_bcb);
Mapping { kind: MappingKind::Branch { true_term, false_term }, span }
},
}| { Mapping { kind: MappingKind::Branch { true_bcb, false_bcb }, span } },
));
for (decision, branches) in mcdc_mappings {
@ -201,12 +181,10 @@ fn create_mappings(
true_index: _,
false_index: _,
}| {
let true_term = term_for_bcb(true_bcb);
let false_term = term_for_bcb(false_bcb);
Mapping {
kind: MappingKind::MCDCBranch {
true_term,
false_term,
true_bcb,
false_bcb,
mcdc_params: condition_info,
},
span,
@ -227,41 +205,11 @@ fn create_mappings(
mappings
}
/// For each BCB node or BCB edge that has an associated coverage counter,
/// inject any necessary coverage statements into MIR.
fn inject_coverage_statements<'tcx>(
mir_body: &mut mir::Body<'tcx>,
graph: &CoverageGraph,
extracted_mappings: &ExtractedMappings,
coverage_counters: &CoverageCounters,
) {
// Inject counter-increment statements into MIR.
for (id, bcb) in coverage_counters.counter_increment_sites() {
let target_bb = graph[bcb].leader_bb();
inject_statement(mir_body, CoverageKind::CounterIncrement { id }, target_bb);
}
// For each counter expression that is directly associated with at least one
// span, we inject an "expression-used" statement, so that coverage codegen
// can check whether the injected statement survived MIR optimization.
// (BCB edges can't have spans, so we only need to process BCB nodes here.)
//
// We only do this for ordinary `Code` mappings, because branch and MC/DC
// mappings might have expressions that don't correspond to any single
// point in the control-flow graph.
//
// See the code in `rustc_codegen_llvm::coverageinfo::map_data` that deals
// with "expressions seen" and "zero terms".
let eligible_bcbs = extracted_mappings.bcbs_with_ordinary_code_mappings();
for (bcb, expression_id) in coverage_counters
.bcb_nodes_with_coverage_expressions()
.filter(|&(bcb, _)| eligible_bcbs.contains(bcb))
{
inject_statement(
mir_body,
CoverageKind::ExpressionUsed { id: expression_id },
graph[bcb].leader_bb(),
);
/// Inject any necessary coverage statements into MIR, so that they influence codegen.
fn inject_coverage_statements<'tcx>(mir_body: &mut mir::Body<'tcx>, graph: &CoverageGraph) {
for (bcb, data) in graph.iter_enumerated() {
let target_bb = data.leader_bb();
inject_statement(mir_body, CoverageKind::VirtualCounter { bcb }, target_bb);
}
}

View file

@ -1,10 +1,7 @@
use rustc_data_structures::captures::Captures;
use rustc_index::bit_set::DenseBitSet;
use rustc_middle::middle::codegen_fn_attrs::CodegenFnAttrFlags;
use rustc_middle::mir::coverage::{
CounterId, CovTerm, CoverageIdsInfo, CoverageKind, Expression, ExpressionId,
FunctionCoverageInfo, MappingKind, Op,
};
use rustc_middle::mir::coverage::{BasicCoverageBlock, CoverageIdsInfo, CoverageKind, MappingKind};
use rustc_middle::mir::{Body, Statement, StatementKind};
use rustc_middle::ty::{self, TyCtxt};
use rustc_middle::util::Providers;
@ -12,6 +9,9 @@ use rustc_span::def_id::LocalDefId;
use rustc_span::sym;
use tracing::trace;
use crate::coverage::counters::node_flow::make_node_counters;
use crate::coverage::counters::{CoverageCounters, transcribe_counters};
/// Registers query/hook implementations related to coverage.
pub(crate) fn provide(providers: &mut Providers) {
providers.hooks.is_eligible_for_coverage = is_eligible_for_coverage;
@ -89,39 +89,57 @@ fn coverage_ids_info<'tcx>(
let mir_body = tcx.instance_mir(instance_def);
let fn_cov_info = mir_body.function_coverage_info.as_deref()?;
let mut counters_seen = DenseBitSet::new_empty(fn_cov_info.num_counters);
let mut expressions_seen = DenseBitSet::new_filled(fn_cov_info.expressions.len());
// For each expression ID that is directly used by one or more mappings,
// mark it as not-yet-seen. This indicates that we expect to see a
// corresponding `ExpressionUsed` statement during MIR traversal.
for mapping in fn_cov_info.mappings.iter() {
// Currently we only worry about ordinary code mappings.
// For branch and MC/DC mappings, expressions might not correspond
// to any particular point in the control-flow graph.
// (Keep this in sync with the injection of `ExpressionUsed`
// statements in the `InstrumentCoverage` MIR pass.)
if let MappingKind::Code(CovTerm::Expression(id)) = mapping.kind {
expressions_seen.remove(id);
}
}
// Scan through the final MIR to see which BCBs survived MIR opts.
// Any BCB not in this set was optimized away.
let mut bcbs_seen = DenseBitSet::new_empty(fn_cov_info.priority_list.len());
for kind in all_coverage_in_mir_body(mir_body) {
match *kind {
CoverageKind::CounterIncrement { id } => {
counters_seen.insert(id);
}
CoverageKind::ExpressionUsed { id } => {
expressions_seen.insert(id);
CoverageKind::VirtualCounter { bcb } => {
bcbs_seen.insert(bcb);
}
_ => {}
}
}
let zero_expressions =
identify_zero_expressions(fn_cov_info, &counters_seen, &expressions_seen);
// Determine the set of BCBs that are referred to by mappings, and therefore
// need a counter. Any node not in this set will only get a counter if it
// is part of the counter expression for a node that is in the set.
let mut bcb_needs_counter =
DenseBitSet::<BasicCoverageBlock>::new_empty(fn_cov_info.priority_list.len());
for mapping in &fn_cov_info.mappings {
match mapping.kind {
MappingKind::Code { bcb } => {
bcb_needs_counter.insert(bcb);
}
MappingKind::Branch { true_bcb, false_bcb } => {
bcb_needs_counter.insert(true_bcb);
bcb_needs_counter.insert(false_bcb);
}
MappingKind::MCDCBranch { true_bcb, false_bcb, mcdc_params: _ } => {
bcb_needs_counter.insert(true_bcb);
bcb_needs_counter.insert(false_bcb);
}
MappingKind::MCDCDecision(_) => {}
}
}
Some(CoverageIdsInfo { counters_seen, zero_expressions })
// FIXME(Zalathar): It should be possible to sort `priority_list[1..]` by
// `!bcbs_seen.contains(bcb)` to simplify the mappings even further, at the
// expense of some churn in the tests. When doing so, also consider removing
// the sorts in `transcribe_counters`.
let node_counters = make_node_counters(&fn_cov_info.node_flow_data, &fn_cov_info.priority_list);
let coverage_counters = transcribe_counters(&node_counters, &bcb_needs_counter, &bcbs_seen);
let CoverageCounters {
phys_counter_for_node, next_counter_id, node_counters, expressions, ..
} = coverage_counters;
Some(CoverageIdsInfo {
num_counters: next_counter_id.as_u32(),
phys_counter_for_node,
term_for_bcb: node_counters,
expressions,
})
}
fn all_coverage_in_mir_body<'a, 'tcx>(
@ -139,94 +157,3 @@ fn is_inlined(body: &Body<'_>, statement: &Statement<'_>) -> bool {
let scope_data = &body.source_scopes[statement.source_info.scope];
scope_data.inlined.is_some() || scope_data.inlined_parent_scope.is_some()
}
/// Identify expressions that will always have a value of zero, and note their
/// IDs in a `DenseBitSet`. Mappings that refer to a zero expression can instead
/// become mappings to a constant zero value.
///
/// This function mainly exists to preserve the simplifications that were
/// already being performed by the Rust-side expression renumbering, so that
/// the resulting coverage mappings don't get worse.
fn identify_zero_expressions(
fn_cov_info: &FunctionCoverageInfo,
counters_seen: &DenseBitSet<CounterId>,
expressions_seen: &DenseBitSet<ExpressionId>,
) -> DenseBitSet<ExpressionId> {
// The set of expressions that either were optimized out entirely, or
// have zero as both of their operands, and will therefore always have
// a value of zero. Other expressions that refer to these as operands
// can have those operands replaced with `CovTerm::Zero`.
let mut zero_expressions = DenseBitSet::new_empty(fn_cov_info.expressions.len());
// Simplify a copy of each expression based on lower-numbered expressions,
// and then update the set of always-zero expressions if necessary.
// (By construction, expressions can only refer to other expressions
// that have lower IDs, so one pass is sufficient.)
for (id, expression) in fn_cov_info.expressions.iter_enumerated() {
if !expressions_seen.contains(id) {
// If an expression was not seen, it must have been optimized away,
// so any operand that refers to it can be replaced with zero.
zero_expressions.insert(id);
continue;
}
// We don't need to simplify the actual expression data in the
// expressions list; we can just simplify a temporary copy and then
// use that to update the set of always-zero expressions.
let Expression { mut lhs, op, mut rhs } = *expression;
// If an expression has an operand that is also an expression, the
// operand's ID must be strictly lower. This is what lets us find
// all zero expressions in one pass.
let assert_operand_expression_is_lower = |operand_id: ExpressionId| {
assert!(
operand_id < id,
"Operand {operand_id:?} should be less than {id:?} in {expression:?}",
)
};
// If an operand refers to a counter or expression that is always
// zero, then that operand can be replaced with `CovTerm::Zero`.
let maybe_set_operand_to_zero = |operand: &mut CovTerm| {
if let CovTerm::Expression(id) = *operand {
assert_operand_expression_is_lower(id);
}
if is_zero_term(&counters_seen, &zero_expressions, *operand) {
*operand = CovTerm::Zero;
}
};
maybe_set_operand_to_zero(&mut lhs);
maybe_set_operand_to_zero(&mut rhs);
// Coverage counter values cannot be negative, so if an expression
// involves subtraction from zero, assume that its RHS must also be zero.
// (Do this after simplifications that could set the LHS to zero.)
if lhs == CovTerm::Zero && op == Op::Subtract {
rhs = CovTerm::Zero;
}
// After the above simplifications, if both operands are zero, then
// we know that this expression is always zero too.
if lhs == CovTerm::Zero && rhs == CovTerm::Zero {
zero_expressions.insert(id);
}
}
zero_expressions
}
/// Returns `true` if the given term is known to have a value of zero, taking
/// into account knowledge of which counters are unused and which expressions
/// are always zero.
fn is_zero_term(
counters_seen: &DenseBitSet<CounterId>,
zero_expressions: &DenseBitSet<ExpressionId>,
term: CovTerm,
) -> bool {
match term {
CovTerm::Zero => true,
CovTerm::Counter(id) => !counters_seen.contains(id),
CovTerm::Expression(id) => zero_expressions.contains(id),
}
}

View file

@ -137,8 +137,7 @@ fn filtered_statement_span(statement: &Statement<'_>) -> Option<Span> {
// These coverage statements should not exist prior to coverage instrumentation.
StatementKind::Coverage(
CoverageKind::CounterIncrement { .. }
| CoverageKind::ExpressionUsed { .. }
CoverageKind::VirtualCounter { .. }
| CoverageKind::CondBitmapUpdate { .. }
| CoverageKind::TestVectorBitmapUpdate { .. },
) => bug!(

View file

@ -1,15 +1,13 @@
Function name: assert_not::main
Raw bytes (31): 0x[01, 01, 01, 0d, 00, 05, 01, 06, 01, 01, 12, 05, 02, 05, 00, 14, 09, 01, 05, 00, 14, 0d, 01, 05, 00, 16, 02, 01, 01, 00, 02]
Raw bytes (29): 0x[01, 01, 00, 05, 01, 06, 01, 01, 12, 05, 02, 05, 00, 14, 09, 01, 05, 00, 14, 0d, 01, 05, 00, 16, 0d, 01, 01, 00, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(3), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 5
- Code(Counter(0)) at (prev + 6, 1) to (start + 1, 18)
- Code(Counter(1)) at (prev + 2, 5) to (start + 0, 20)
- Code(Counter(2)) at (prev + 1, 5) to (start + 0, 20)
- Code(Counter(3)) at (prev + 1, 5) to (start + 0, 22)
- Code(Expression(0, Sub)) at (prev + 1, 1) to (start + 0, 2)
= (c3 - Zero)
- Code(Counter(3)) at (prev + 1, 1) to (start + 0, 2)
Highest counter ID seen: c3

View file

@ -9,15 +9,13 @@ Number of file 0 mappings: 2
Highest counter ID seen: c0
Function name: bad_counter_ids::eq_bad_message
Raw bytes (21): 0x[01, 01, 01, 01, 00, 03, 01, 29, 01, 02, 0f, 02, 02, 20, 00, 2b, 00, 01, 01, 00, 02]
Raw bytes (19): 0x[01, 01, 00, 03, 01, 29, 01, 02, 0f, 01, 02, 20, 00, 2b, 00, 01, 01, 00, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 3
- Code(Counter(0)) at (prev + 41, 1) to (start + 2, 15)
- Code(Expression(0, Sub)) at (prev + 2, 32) to (start + 0, 43)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 2, 32) to (start + 0, 43)
- Code(Zero) at (prev + 1, 1) to (start + 0, 2)
Highest counter ID seen: c0
@ -64,27 +62,23 @@ Number of file 0 mappings: 3
Highest counter ID seen: c1
Function name: bad_counter_ids::ne_good
Raw bytes (16): 0x[01, 01, 01, 01, 00, 02, 01, 1a, 01, 02, 1f, 02, 03, 01, 00, 02]
Raw bytes (14): 0x[01, 01, 00, 02, 01, 1a, 01, 02, 1f, 01, 03, 01, 00, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 2
- Code(Counter(0)) at (prev + 26, 1) to (start + 2, 31)
- Code(Expression(0, Sub)) at (prev + 3, 1) to (start + 0, 2)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 3, 1) to (start + 0, 2)
Highest counter ID seen: c0
Function name: bad_counter_ids::ne_good_message
Raw bytes (21): 0x[01, 01, 01, 01, 00, 03, 01, 1f, 01, 02, 0f, 00, 02, 20, 00, 2b, 02, 01, 01, 00, 02]
Raw bytes (19): 0x[01, 01, 00, 03, 01, 1f, 01, 02, 0f, 00, 02, 20, 00, 2b, 01, 01, 01, 00, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 3
- Code(Counter(0)) at (prev + 31, 1) to (start + 2, 15)
- Code(Zero) at (prev + 2, 32) to (start + 0, 43)
- Code(Expression(0, Sub)) at (prev + 1, 1) to (start + 0, 2)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 1, 1) to (start + 0, 2)
Highest counter ID seen: c0

View file

@ -8,44 +8,38 @@ Number of file 0 mappings: 1
Highest counter ID seen: c0
Function name: fn_sig_into_try::b
Raw bytes (26): 0x[01, 01, 01, 01, 00, 04, 01, 11, 01, 03, 0f, 00, 03, 0f, 00, 10, 02, 01, 05, 00, 0c, 01, 01, 01, 00, 02]
Raw bytes (24): 0x[01, 01, 00, 04, 01, 11, 01, 03, 0f, 00, 03, 0f, 00, 10, 01, 01, 05, 00, 0c, 01, 01, 01, 00, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 4
- Code(Counter(0)) at (prev + 17, 1) to (start + 3, 15)
- Code(Zero) at (prev + 3, 15) to (start + 0, 16)
- Code(Expression(0, Sub)) at (prev + 1, 5) to (start + 0, 12)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 1, 5) to (start + 0, 12)
- Code(Counter(0)) at (prev + 1, 1) to (start + 0, 2)
Highest counter ID seen: c0
Function name: fn_sig_into_try::c
Raw bytes (26): 0x[01, 01, 01, 01, 00, 04, 01, 18, 01, 03, 17, 00, 03, 17, 00, 18, 02, 01, 05, 00, 0c, 01, 01, 01, 00, 02]
Raw bytes (24): 0x[01, 01, 00, 04, 01, 18, 01, 03, 17, 00, 03, 17, 00, 18, 01, 01, 05, 00, 0c, 01, 01, 01, 00, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 4
- Code(Counter(0)) at (prev + 24, 1) to (start + 3, 23)
- Code(Zero) at (prev + 3, 23) to (start + 0, 24)
- Code(Expression(0, Sub)) at (prev + 1, 5) to (start + 0, 12)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 1, 5) to (start + 0, 12)
- Code(Counter(0)) at (prev + 1, 1) to (start + 0, 2)
Highest counter ID seen: c0
Function name: fn_sig_into_try::d
Raw bytes (26): 0x[01, 01, 01, 01, 00, 04, 01, 1f, 01, 04, 0f, 00, 04, 0f, 00, 10, 02, 01, 05, 00, 0c, 01, 01, 01, 00, 02]
Raw bytes (24): 0x[01, 01, 00, 04, 01, 1f, 01, 04, 0f, 00, 04, 0f, 00, 10, 01, 01, 05, 00, 0c, 01, 01, 01, 00, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 4
- Code(Counter(0)) at (prev + 31, 1) to (start + 4, 15)
- Code(Zero) at (prev + 4, 15) to (start + 0, 16)
- Code(Expression(0, Sub)) at (prev + 1, 5) to (start + 0, 12)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 1, 5) to (start + 0, 12)
- Code(Counter(0)) at (prev + 1, 1) to (start + 0, 2)
Highest counter ID seen: c0

View file

@ -32,16 +32,13 @@ Number of file 0 mappings: 2
Highest counter ID seen: c0
Function name: inline_dead::main::{closure#0}
Raw bytes (23): 0x[01, 01, 02, 07, 00, 01, 00, 03, 01, 07, 17, 01, 16, 00, 01, 17, 00, 18, 02, 01, 05, 00, 06]
Raw bytes (19): 0x[01, 01, 00, 03, 01, 07, 17, 01, 16, 00, 01, 17, 00, 18, 01, 01, 05, 00, 06]
Number of files: 1
- file 0 => global file 1
Number of expressions: 2
- expression 0 operands: lhs = Expression(1, Add), rhs = Zero
- expression 1 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 3
- Code(Counter(0)) at (prev + 7, 23) to (start + 1, 22)
- Code(Zero) at (prev + 1, 23) to (start + 0, 24)
- Code(Expression(0, Sub)) at (prev + 1, 5) to (start + 0, 6)
= ((c0 + Zero) - Zero)
- Code(Counter(0)) at (prev + 1, 5) to (start + 0, 6)
Highest counter ID seen: c0

View file

@ -59,10 +59,10 @@ Number of file 0 mappings: 1
Highest counter ID seen: c0
Function name: issue_84561::test3
Raw bytes (317): 0x[01, 01, 1c, 1d, 21, 25, 29, 21, 25, 2d, 31, 21, 17, 25, 2d, 41, 45, 49, 4d, 51, 55, 33, 51, 49, 4d, 33, 37, 49, 4d, 51, 59, 55, 59, 55, 59, 47, 5d, 55, 59, 61, 65, 71, 75, 69, 6d, 69, 6d, 69, 5f, 6d, 00, 67, 79, 71, 75, 79, 7d, 7d, 81, 01, 33, 01, 08, 01, 03, 1c, 05, 04, 09, 01, 1c, 09, 02, 05, 04, 1f, 0d, 05, 05, 00, 1f, 11, 01, 05, 00, 1f, 15, 01, 09, 01, 1c, 19, 02, 05, 00, 1f, 1d, 01, 05, 00, 0f, 02, 00, 20, 00, 30, 21, 01, 05, 03, 0f, 25, 03, 20, 00, 30, 29, 00, 33, 00, 41, 06, 00, 4b, 00, 5a, 0a, 01, 05, 00, 0f, 2d, 05, 09, 03, 10, 31, 05, 0d, 00, 1b, 0e, 02, 0d, 00, 1c, 12, 04, 09, 05, 06, 35, 06, 05, 03, 06, 39, 04, 05, 03, 06, 3d, 04, 09, 04, 06, 41, 05, 08, 00, 0f, 45, 01, 09, 03, 0a, 1a, 05, 09, 03, 0a, 33, 05, 08, 00, 0f, 51, 01, 09, 00, 13, 22, 03, 0d, 00, 1d, 26, 03, 09, 00, 13, 2e, 03, 0d, 00, 1d, 47, 03, 05, 00, 0f, 47, 01, 0c, 00, 13, 5d, 01, 0d, 00, 13, 42, 02, 0d, 00, 13, 61, 04, 05, 02, 13, 65, 03, 0d, 00, 13, 4a, 02, 0d, 00, 13, 67, 03, 05, 00, 0f, 69, 01, 0c, 00, 13, 6d, 01, 0d, 03, 0e, 71, 04, 0d, 00, 13, 56, 02, 0d, 00, 17, 56, 01, 14, 00, 1b, 00, 01, 15, 00, 1b, 5a, 02, 15, 00, 1b, 75, 04, 0d, 00, 13, 62, 03, 09, 00, 19, 79, 02, 05, 00, 0f, 6a, 03, 09, 00, 22, 7d, 02, 05, 00, 0f, 6e, 03, 09, 00, 2c, 81, 01, 02, 01, 00, 02]
Raw bytes (315): 0x[01, 01, 1b, 1d, 21, 25, 29, 21, 25, 2d, 31, 21, 17, 25, 2d, 41, 45, 49, 4d, 51, 55, 33, 51, 49, 4d, 33, 37, 49, 4d, 51, 59, 55, 59, 55, 59, 47, 5d, 55, 59, 61, 65, 71, 75, 69, 6d, 69, 6d, 69, 6d, 63, 79, 71, 75, 79, 7d, 7d, 81, 01, 33, 01, 08, 01, 03, 1c, 05, 04, 09, 01, 1c, 09, 02, 05, 04, 1f, 0d, 05, 05, 00, 1f, 11, 01, 05, 00, 1f, 15, 01, 09, 01, 1c, 19, 02, 05, 00, 1f, 1d, 01, 05, 00, 0f, 02, 00, 20, 00, 30, 21, 01, 05, 03, 0f, 25, 03, 20, 00, 30, 29, 00, 33, 00, 41, 06, 00, 4b, 00, 5a, 0a, 01, 05, 00, 0f, 2d, 05, 09, 03, 10, 31, 05, 0d, 00, 1b, 0e, 02, 0d, 00, 1c, 12, 04, 09, 05, 06, 35, 06, 05, 03, 06, 39, 04, 05, 03, 06, 3d, 04, 09, 04, 06, 41, 05, 08, 00, 0f, 45, 01, 09, 03, 0a, 1a, 05, 09, 03, 0a, 33, 05, 08, 00, 0f, 51, 01, 09, 00, 13, 22, 03, 0d, 00, 1d, 26, 03, 09, 00, 13, 2e, 03, 0d, 00, 1d, 47, 03, 05, 00, 0f, 47, 01, 0c, 00, 13, 5d, 01, 0d, 00, 13, 42, 02, 0d, 00, 13, 61, 04, 05, 02, 13, 65, 03, 0d, 00, 13, 4a, 02, 0d, 00, 13, 63, 03, 05, 00, 0f, 69, 01, 0c, 00, 13, 6d, 01, 0d, 03, 0e, 71, 04, 0d, 00, 13, 5a, 02, 0d, 00, 17, 5a, 01, 14, 00, 1b, 00, 01, 15, 00, 1b, 5a, 02, 15, 00, 1b, 75, 04, 0d, 00, 13, 5e, 03, 09, 00, 19, 79, 02, 05, 00, 0f, 66, 03, 09, 00, 22, 7d, 02, 05, 00, 0f, 6a, 03, 09, 00, 2c, 81, 01, 02, 01, 00, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 28
Number of expressions: 27
- expression 0 operands: lhs = Counter(7), rhs = Counter(8)
- expression 1 operands: lhs = Counter(9), rhs = Counter(10)
- expression 2 operands: lhs = Counter(8), rhs = Counter(9)
@ -85,12 +85,11 @@ Number of expressions: 28
- expression 19 operands: lhs = Counter(28), rhs = Counter(29)
- expression 20 operands: lhs = Counter(26), rhs = Counter(27)
- expression 21 operands: lhs = Counter(26), rhs = Counter(27)
- expression 22 operands: lhs = Counter(26), rhs = Expression(23, Add)
- expression 23 operands: lhs = Counter(27), rhs = Zero
- expression 24 operands: lhs = Expression(25, Add), rhs = Counter(30)
- expression 25 operands: lhs = Counter(28), rhs = Counter(29)
- expression 26 operands: lhs = Counter(30), rhs = Counter(31)
- expression 27 operands: lhs = Counter(31), rhs = Counter(32)
- expression 22 operands: lhs = Counter(26), rhs = Counter(27)
- expression 23 operands: lhs = Expression(24, Add), rhs = Counter(30)
- expression 24 operands: lhs = Counter(28), rhs = Counter(29)
- expression 25 operands: lhs = Counter(30), rhs = Counter(31)
- expression 26 operands: lhs = Counter(31), rhs = Counter(32)
Number of file 0 mappings: 51
- Code(Counter(0)) at (prev + 8, 1) to (start + 3, 28)
- Code(Counter(1)) at (prev + 4, 9) to (start + 1, 28)
@ -142,26 +141,26 @@ Number of file 0 mappings: 51
- Code(Counter(25)) at (prev + 3, 13) to (start + 0, 19)
- Code(Expression(18, Sub)) at (prev + 2, 13) to (start + 0, 19)
= (c24 - c25)
- Code(Expression(25, Add)) at (prev + 3, 5) to (start + 0, 15)
- Code(Expression(24, Add)) at (prev + 3, 5) to (start + 0, 15)
= (c28 + c29)
- Code(Counter(26)) at (prev + 1, 12) to (start + 0, 19)
- Code(Counter(27)) at (prev + 1, 13) to (start + 3, 14)
- Code(Counter(28)) at (prev + 4, 13) to (start + 0, 19)
- Code(Expression(21, Sub)) at (prev + 2, 13) to (start + 0, 23)
- Code(Expression(22, Sub)) at (prev + 2, 13) to (start + 0, 23)
= (c26 - c27)
- Code(Expression(21, Sub)) at (prev + 1, 20) to (start + 0, 27)
- Code(Expression(22, Sub)) at (prev + 1, 20) to (start + 0, 27)
= (c26 - c27)
- Code(Zero) at (prev + 1, 21) to (start + 0, 27)
- Code(Expression(22, Sub)) at (prev + 2, 21) to (start + 0, 27)
= (c26 - (c27 + Zero))
= (c26 - c27)
- Code(Counter(29)) at (prev + 4, 13) to (start + 0, 19)
- Code(Expression(24, Sub)) at (prev + 3, 9) to (start + 0, 25)
- Code(Expression(23, Sub)) at (prev + 3, 9) to (start + 0, 25)
= ((c28 + c29) - c30)
- Code(Counter(30)) at (prev + 2, 5) to (start + 0, 15)
- Code(Expression(26, Sub)) at (prev + 3, 9) to (start + 0, 34)
- Code(Expression(25, Sub)) at (prev + 3, 9) to (start + 0, 34)
= (c30 - c31)
- Code(Counter(31)) at (prev + 2, 5) to (start + 0, 15)
- Code(Expression(27, Sub)) at (prev + 3, 9) to (start + 0, 44)
- Code(Expression(26, Sub)) at (prev + 3, 9) to (start + 0, 44)
= (c31 - c32)
- Code(Counter(32)) at (prev + 2, 1) to (start + 0, 2)
Highest counter ID seen: c32

View file

@ -1,85 +1,70 @@
Function name: <loops_branches::DebugTest as core::fmt::Debug>::fmt
Raw bytes (116): 0x[01, 01, 06, 05, 00, 1d, 00, 0f, 13, 01, 19, 11, 15, 15, 19, 14, 01, 09, 05, 01, 10, 05, 02, 10, 00, 15, 00, 01, 17, 00, 1b, 00, 00, 1c, 00, 1e, 02, 01, 0d, 00, 0e, 05, 01, 0d, 00, 1e, 11, 00, 1e, 00, 1f, 00, 01, 10, 01, 0a, 19, 03, 0d, 00, 0e, 15, 00, 12, 00, 17, 19, 01, 10, 00, 14, 1d, 01, 14, 00, 19, 00, 01, 1b, 00, 1f, 00, 00, 20, 00, 22, 06, 01, 11, 00, 12, 1d, 01, 11, 00, 22, 0a, 00, 22, 00, 23, 00, 01, 14, 01, 0e, 16, 03, 09, 00, 0f, 01, 01, 05, 00, 06]
Raw bytes (112): 0x[01, 01, 04, 07, 0b, 01, 11, 09, 0d, 0d, 11, 14, 01, 09, 05, 01, 10, 05, 02, 10, 00, 15, 00, 01, 17, 00, 1b, 00, 00, 1c, 00, 1e, 05, 01, 0d, 00, 0e, 05, 01, 0d, 00, 1e, 09, 00, 1e, 00, 1f, 00, 01, 10, 01, 0a, 11, 03, 0d, 00, 0e, 0d, 00, 12, 00, 17, 11, 01, 10, 00, 14, 15, 01, 14, 00, 19, 00, 01, 1b, 00, 1f, 00, 00, 20, 00, 22, 15, 01, 11, 00, 12, 15, 01, 11, 00, 22, 02, 00, 22, 00, 23, 00, 01, 14, 01, 0e, 0e, 03, 09, 00, 0f, 01, 01, 05, 00, 06]
Number of files: 1
- file 0 => global file 1
Number of expressions: 6
- expression 0 operands: lhs = Counter(1), rhs = Zero
- expression 1 operands: lhs = Counter(7), rhs = Zero
- expression 2 operands: lhs = Expression(3, Add), rhs = Expression(4, Add)
- expression 3 operands: lhs = Counter(0), rhs = Counter(6)
- expression 4 operands: lhs = Counter(4), rhs = Counter(5)
- expression 5 operands: lhs = Counter(5), rhs = Counter(6)
Number of expressions: 4
- expression 0 operands: lhs = Expression(1, Add), rhs = Expression(2, Add)
- expression 1 operands: lhs = Counter(0), rhs = Counter(4)
- expression 2 operands: lhs = Counter(2), rhs = Counter(3)
- expression 3 operands: lhs = Counter(3), rhs = Counter(4)
Number of file 0 mappings: 20
- Code(Counter(0)) at (prev + 9, 5) to (start + 1, 16)
- Code(Counter(1)) at (prev + 2, 16) to (start + 0, 21)
- Code(Zero) at (prev + 1, 23) to (start + 0, 27)
- Code(Zero) at (prev + 0, 28) to (start + 0, 30)
- Code(Expression(0, Sub)) at (prev + 1, 13) to (start + 0, 14)
= (c1 - Zero)
- Code(Counter(1)) at (prev + 1, 13) to (start + 0, 14)
- Code(Counter(1)) at (prev + 1, 13) to (start + 0, 30)
- Code(Counter(4)) at (prev + 0, 30) to (start + 0, 31)
- Code(Counter(2)) at (prev + 0, 30) to (start + 0, 31)
- Code(Zero) at (prev + 1, 16) to (start + 1, 10)
- Code(Counter(6)) at (prev + 3, 13) to (start + 0, 14)
- Code(Counter(5)) at (prev + 0, 18) to (start + 0, 23)
- Code(Counter(6)) at (prev + 1, 16) to (start + 0, 20)
- Code(Counter(7)) at (prev + 1, 20) to (start + 0, 25)
- Code(Counter(4)) at (prev + 3, 13) to (start + 0, 14)
- Code(Counter(3)) at (prev + 0, 18) to (start + 0, 23)
- Code(Counter(4)) at (prev + 1, 16) to (start + 0, 20)
- Code(Counter(5)) at (prev + 1, 20) to (start + 0, 25)
- Code(Zero) at (prev + 1, 27) to (start + 0, 31)
- Code(Zero) at (prev + 0, 32) to (start + 0, 34)
- Code(Expression(1, Sub)) at (prev + 1, 17) to (start + 0, 18)
= (c7 - Zero)
- Code(Counter(7)) at (prev + 1, 17) to (start + 0, 34)
- Code(Expression(2, Sub)) at (prev + 0, 34) to (start + 0, 35)
= ((c0 + c6) - (c4 + c5))
- Code(Counter(5)) at (prev + 1, 17) to (start + 0, 18)
- Code(Counter(5)) at (prev + 1, 17) to (start + 0, 34)
- Code(Expression(0, Sub)) at (prev + 0, 34) to (start + 0, 35)
= ((c0 + c4) - (c2 + c3))
- Code(Zero) at (prev + 1, 20) to (start + 1, 14)
- Code(Expression(5, Sub)) at (prev + 3, 9) to (start + 0, 15)
= (c5 - c6)
- Code(Expression(3, Sub)) at (prev + 3, 9) to (start + 0, 15)
= (c3 - c4)
- Code(Counter(0)) at (prev + 1, 5) to (start + 0, 6)
Highest counter ID seen: c7
Highest counter ID seen: c5
Function name: <loops_branches::DisplayTest as core::fmt::Display>::fmt
Raw bytes (122): 0x[01, 01, 09, 01, 00, 01, 00, 0d, 00, 0d, 00, 0d, 00, 1b, 1f, 01, 0d, 09, 1d, 09, 0d, 14, 01, 22, 05, 01, 11, 00, 01, 12, 01, 0a, 02, 02, 10, 00, 15, 00, 01, 17, 00, 1b, 00, 00, 1c, 00, 1e, 06, 01, 0d, 00, 0e, 02, 01, 0d, 00, 1e, 1d, 00, 1e, 00, 1f, 0d, 02, 0d, 00, 0e, 09, 00, 12, 00, 17, 0d, 01, 10, 00, 15, 00, 00, 16, 01, 0e, 12, 02, 14, 00, 19, 00, 01, 1b, 00, 1f, 00, 00, 20, 00, 22, 0e, 01, 11, 00, 12, 12, 01, 11, 00, 22, 16, 00, 22, 00, 23, 22, 03, 09, 00, 0f, 01, 01, 05, 00, 06]
Raw bytes (112): 0x[01, 01, 04, 07, 0b, 01, 09, 05, 0d, 05, 09, 14, 01, 22, 05, 01, 11, 00, 01, 12, 01, 0a, 01, 02, 10, 00, 15, 00, 01, 17, 00, 1b, 00, 00, 1c, 00, 1e, 01, 01, 0d, 00, 0e, 01, 01, 0d, 00, 1e, 0d, 00, 1e, 00, 1f, 09, 02, 0d, 00, 0e, 05, 00, 12, 00, 17, 09, 01, 10, 00, 15, 00, 00, 16, 01, 0e, 09, 02, 14, 00, 19, 00, 01, 1b, 00, 1f, 00, 00, 20, 00, 22, 09, 01, 11, 00, 12, 09, 01, 11, 00, 22, 02, 00, 22, 00, 23, 0e, 03, 09, 00, 0f, 01, 01, 05, 00, 06]
Number of files: 1
- file 0 => global file 1
Number of expressions: 9
- expression 0 operands: lhs = Counter(0), rhs = Zero
- expression 1 operands: lhs = Counter(0), rhs = Zero
- expression 2 operands: lhs = Counter(3), rhs = Zero
- expression 3 operands: lhs = Counter(3), rhs = Zero
- expression 4 operands: lhs = Counter(3), rhs = Zero
- expression 5 operands: lhs = Expression(6, Add), rhs = Expression(7, Add)
- expression 6 operands: lhs = Counter(0), rhs = Counter(3)
- expression 7 operands: lhs = Counter(2), rhs = Counter(7)
- expression 8 operands: lhs = Counter(2), rhs = Counter(3)
Number of expressions: 4
- expression 0 operands: lhs = Expression(1, Add), rhs = Expression(2, Add)
- expression 1 operands: lhs = Counter(0), rhs = Counter(2)
- expression 2 operands: lhs = Counter(1), rhs = Counter(3)
- expression 3 operands: lhs = Counter(1), rhs = Counter(2)
Number of file 0 mappings: 20
- Code(Counter(0)) at (prev + 34, 5) to (start + 1, 17)
- Code(Zero) at (prev + 1, 18) to (start + 1, 10)
- Code(Expression(0, Sub)) at (prev + 2, 16) to (start + 0, 21)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 2, 16) to (start + 0, 21)
- Code(Zero) at (prev + 1, 23) to (start + 0, 27)
- Code(Zero) at (prev + 0, 28) to (start + 0, 30)
- Code(Expression(1, Sub)) at (prev + 1, 13) to (start + 0, 14)
= (c0 - Zero)
- Code(Expression(0, Sub)) at (prev + 1, 13) to (start + 0, 30)
= (c0 - Zero)
- Code(Counter(7)) at (prev + 0, 30) to (start + 0, 31)
- Code(Counter(3)) at (prev + 2, 13) to (start + 0, 14)
- Code(Counter(2)) at (prev + 0, 18) to (start + 0, 23)
- Code(Counter(3)) at (prev + 1, 16) to (start + 0, 21)
- Code(Counter(0)) at (prev + 1, 13) to (start + 0, 14)
- Code(Counter(0)) at (prev + 1, 13) to (start + 0, 30)
- Code(Counter(3)) at (prev + 0, 30) to (start + 0, 31)
- Code(Counter(2)) at (prev + 2, 13) to (start + 0, 14)
- Code(Counter(1)) at (prev + 0, 18) to (start + 0, 23)
- Code(Counter(2)) at (prev + 1, 16) to (start + 0, 21)
- Code(Zero) at (prev + 0, 22) to (start + 1, 14)
- Code(Expression(4, Sub)) at (prev + 2, 20) to (start + 0, 25)
= (c3 - Zero)
- Code(Counter(2)) at (prev + 2, 20) to (start + 0, 25)
- Code(Zero) at (prev + 1, 27) to (start + 0, 31)
- Code(Zero) at (prev + 0, 32) to (start + 0, 34)
- Code(Expression(3, Sub)) at (prev + 1, 17) to (start + 0, 18)
= (c3 - Zero)
- Code(Expression(4, Sub)) at (prev + 1, 17) to (start + 0, 34)
= (c3 - Zero)
- Code(Expression(5, Sub)) at (prev + 0, 34) to (start + 0, 35)
= ((c0 + c3) - (c2 + c7))
- Code(Expression(8, Sub)) at (prev + 3, 9) to (start + 0, 15)
= (c2 - c3)
- Code(Counter(2)) at (prev + 1, 17) to (start + 0, 18)
- Code(Counter(2)) at (prev + 1, 17) to (start + 0, 34)
- Code(Expression(0, Sub)) at (prev + 0, 34) to (start + 0, 35)
= ((c0 + c2) - (c1 + c3))
- Code(Expression(3, Sub)) at (prev + 3, 9) to (start + 0, 15)
= (c1 - c2)
- Code(Counter(0)) at (prev + 1, 5) to (start + 0, 6)
Highest counter ID seen: c7
Highest counter ID seen: c3
Function name: loops_branches::main
Raw bytes (9): 0x[01, 01, 00, 01, 01, 37, 01, 05, 02]

View file

@ -1,13 +1,11 @@
Function name: no_spans_if_not::affected_function
Raw bytes (21): 0x[01, 01, 01, 01, 00, 03, 01, 16, 1c, 01, 12, 02, 02, 0d, 00, 0f, 00, 02, 0d, 00, 0f]
Raw bytes (19): 0x[01, 01, 00, 03, 01, 16, 1c, 01, 12, 01, 02, 0d, 00, 0f, 00, 02, 0d, 00, 0f]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 3
- Code(Counter(0)) at (prev + 22, 28) to (start + 1, 18)
- Code(Expression(0, Sub)) at (prev + 2, 13) to (start + 0, 15)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 2, 13) to (start + 0, 15)
- Code(Zero) at (prev + 2, 13) to (start + 0, 15)
Highest counter ID seen: c0

View file

@ -1,13 +1,11 @@
Function name: tight_inf_loop::main
Raw bytes (21): 0x[01, 01, 01, 01, 00, 03, 01, 01, 01, 01, 0d, 00, 02, 09, 00, 10, 02, 01, 06, 01, 02]
Raw bytes (19): 0x[01, 01, 00, 03, 01, 01, 01, 01, 0d, 00, 02, 09, 00, 10, 01, 01, 06, 01, 02]
Number of files: 1
- file 0 => global file 1
Number of expressions: 1
- expression 0 operands: lhs = Counter(0), rhs = Zero
Number of expressions: 0
Number of file 0 mappings: 3
- Code(Counter(0)) at (prev + 1, 1) to (start + 1, 13)
- Code(Zero) at (prev + 2, 9) to (start + 0, 16)
- Code(Expression(0, Sub)) at (prev + 1, 6) to (start + 1, 2)
= (c0 - Zero)
- Code(Counter(0)) at (prev + 1, 6) to (start + 1, 2)
Highest counter ID seen: c0

View file

@ -27,18 +27,15 @@
}
+ coverage body span: $DIR/branch_match_arms.rs:14:11: 21:2 (#0)
+ coverage ExpressionId(0) => Expression { lhs: Counter(1), op: Add, rhs: Counter(2) };
+ coverage ExpressionId(1) => Expression { lhs: Expression(0), op: Add, rhs: Counter(3) };
+ coverage ExpressionId(2) => Expression { lhs: Counter(0), op: Subtract, rhs: Expression(1) };
+ coverage Code(Counter(0)) => $DIR/branch_match_arms.rs:14:1: 15:21 (#0);
+ coverage Code(Counter(1)) => $DIR/branch_match_arms.rs:16:17: 16:33 (#0);
+ coverage Code(Counter(2)) => $DIR/branch_match_arms.rs:17:17: 17:33 (#0);
+ coverage Code(Counter(3)) => $DIR/branch_match_arms.rs:18:17: 18:33 (#0);
+ coverage Code(Expression(2)) => $DIR/branch_match_arms.rs:19:17: 19:33 (#0);
+ coverage Code(Counter(0)) => $DIR/branch_match_arms.rs:21:2: 21:2 (#0);
+ coverage Code { bcb: bcb0 } => $DIR/branch_match_arms.rs:14:1: 15:21 (#0);
+ coverage Code { bcb: bcb1 } => $DIR/branch_match_arms.rs:16:17: 16:33 (#0);
+ coverage Code { bcb: bcb3 } => $DIR/branch_match_arms.rs:17:17: 17:33 (#0);
+ coverage Code { bcb: bcb4 } => $DIR/branch_match_arms.rs:18:17: 18:33 (#0);
+ coverage Code { bcb: bcb5 } => $DIR/branch_match_arms.rs:19:17: 19:33 (#0);
+ coverage Code { bcb: bcb2 } => $DIR/branch_match_arms.rs:21:2: 21:2 (#0);
+
bb0: {
+ Coverage::CounterIncrement(0);
+ Coverage::VirtualCounter(bcb0);
StorageLive(_1);
_1 = Enum::A(const 0_u32);
PlaceMention(_1);
@ -52,22 +49,22 @@
}
bb2: {
+ Coverage::CounterIncrement(1);
+ Coverage::VirtualCounter(bcb1);
falseEdge -> [real: bb8, imaginary: bb3];
}
bb3: {
+ Coverage::CounterIncrement(2);
+ Coverage::VirtualCounter(bcb3);
falseEdge -> [real: bb7, imaginary: bb4];
}
bb4: {
+ Coverage::CounterIncrement(3);
+ Coverage::VirtualCounter(bcb4);
falseEdge -> [real: bb6, imaginary: bb5];
}
bb5: {
+ Coverage::ExpressionUsed(2);
+ Coverage::VirtualCounter(bcb5);
StorageLive(_9);
_9 = copy ((_1 as A).0: u32);
StorageLive(_10);
@ -124,6 +121,7 @@
}
bb13: {
+ Coverage::VirtualCounter(bcb2);
StorageDead(_1);
return;
}

View file

@ -5,10 +5,10 @@
let mut _0: bool;
+ coverage body span: $DIR/instrument_coverage.rs:29:18: 31:2 (#0)
+ coverage Code(Counter(0)) => $DIR/instrument_coverage.rs:29:1: 31:2 (#0);
+ coverage Code { bcb: bcb0 } => $DIR/instrument_coverage.rs:29:1: 31:2 (#0);
+
bb0: {
+ Coverage::CounterIncrement(0);
+ Coverage::VirtualCounter(bcb0);
_0 = const true;
return;
}

View file

@ -8,20 +8,19 @@
let mut _3: !;
+ coverage body span: $DIR/instrument_coverage.rs:14:11: 20:2 (#0)
+ coverage ExpressionId(0) => Expression { lhs: Counter(1), op: Subtract, rhs: Counter(0) };
+ coverage Code(Counter(0)) => $DIR/instrument_coverage.rs:14:1: 14:11 (#0);
+ coverage Code(Counter(1)) => $DIR/instrument_coverage.rs:16:12: 16:17 (#0);
+ coverage Code(Counter(0)) => $DIR/instrument_coverage.rs:17:13: 17:18 (#0);
+ coverage Code(Expression(0)) => $DIR/instrument_coverage.rs:18:10: 18:10 (#0);
+ coverage Code(Counter(0)) => $DIR/instrument_coverage.rs:20:2: 20:2 (#0);
+ coverage Code { bcb: bcb0 } => $DIR/instrument_coverage.rs:14:1: 14:11 (#0);
+ coverage Code { bcb: bcb1 } => $DIR/instrument_coverage.rs:16:12: 16:17 (#0);
+ coverage Code { bcb: bcb2 } => $DIR/instrument_coverage.rs:17:13: 17:18 (#0);
+ coverage Code { bcb: bcb3 } => $DIR/instrument_coverage.rs:18:10: 18:10 (#0);
+ coverage Code { bcb: bcb2 } => $DIR/instrument_coverage.rs:20:2: 20:2 (#0);
+
bb0: {
+ Coverage::CounterIncrement(0);
+ Coverage::VirtualCounter(bcb0);
goto -> bb1;
}
bb1: {
+ Coverage::CounterIncrement(1);
+ Coverage::VirtualCounter(bcb1);
falseUnwind -> [real: bb2, unwind: bb6];
}
@ -35,13 +34,14 @@
}
bb4: {
+ Coverage::VirtualCounter(bcb2);
_0 = const ();
StorageDead(_2);
return;
}
bb5: {
+ Coverage::ExpressionUsed(0);
+ Coverage::VirtualCounter(bcb3);
_1 = const ();
StorageDead(_2);
goto -> bb1;

View file

@ -8,9 +8,9 @@
// EMIT_MIR instrument_coverage.main.InstrumentCoverage.diff
// CHECK-LABEL: fn main()
// CHECK: coverage body span:
// CHECK: coverage Code(Counter({{[0-9]+}})) =>
// CHECK: coverage Code { bcb: bcb{{[0-9]+}} } =>
// CHECK: bb0:
// CHECK: Coverage::CounterIncrement
// CHECK: Coverage::VirtualCounter
fn main() {
loop {
if bar() {
@ -22,9 +22,9 @@ fn main() {
// EMIT_MIR instrument_coverage.bar.InstrumentCoverage.diff
// CHECK-LABEL: fn bar()
// CHECK: coverage body span:
// CHECK: coverage Code(Counter({{[0-9]+}})) =>
// CHECK: coverage Code { bcb: bcb{{[0-9]+}} } =>
// CHECK: bb0:
// CHECK: Coverage::CounterIncrement
// CHECK: Coverage::VirtualCounter
#[inline(never)]
fn bar() -> bool {
true

View file

@ -8,15 +8,14 @@
coverage branch { true: BlockMarkerId(0), false: BlockMarkerId(1) } => $DIR/instrument_coverage_cleanup.rs:14:8: 14:36 (#0)
coverage body span: $DIR/instrument_coverage_cleanup.rs:13:11: 15:2 (#0)
coverage ExpressionId(0) => Expression { lhs: Counter(0), op: Subtract, rhs: Counter(1) };
coverage Code(Counter(0)) => $DIR/instrument_coverage_cleanup.rs:13:1: 14:36 (#0);
coverage Code(Expression(0)) => $DIR/instrument_coverage_cleanup.rs:14:37: 14:39 (#0);
coverage Code(Counter(1)) => $DIR/instrument_coverage_cleanup.rs:14:39: 14:39 (#0);
coverage Code(Counter(0)) => $DIR/instrument_coverage_cleanup.rs:15:2: 15:2 (#0);
coverage Branch { true_term: Expression(0), false_term: Counter(1) } => $DIR/instrument_coverage_cleanup.rs:14:8: 14:36 (#0);
coverage Code { bcb: bcb0 } => $DIR/instrument_coverage_cleanup.rs:13:1: 14:36 (#0);
coverage Code { bcb: bcb3 } => $DIR/instrument_coverage_cleanup.rs:14:37: 14:39 (#0);
coverage Code { bcb: bcb1 } => $DIR/instrument_coverage_cleanup.rs:14:39: 14:39 (#0);
coverage Code { bcb: bcb2 } => $DIR/instrument_coverage_cleanup.rs:15:2: 15:2 (#0);
coverage Branch { true_bcb: bcb3, false_bcb: bcb1 } => $DIR/instrument_coverage_cleanup.rs:14:8: 14:36 (#0);
bb0: {
Coverage::CounterIncrement(0);
Coverage::VirtualCounter(bcb0);
- Coverage::SpanMarker;
+ nop;
StorageLive(_1);
@ -28,7 +27,7 @@
}
bb2: {
Coverage::CounterIncrement(1);
Coverage::VirtualCounter(bcb1);
- Coverage::BlockMarker(1);
+ nop;
_0 = const ();
@ -36,7 +35,7 @@
}
bb3: {
Coverage::ExpressionUsed(0);
Coverage::VirtualCounter(bcb3);
- Coverage::BlockMarker(0);
+ nop;
_0 = const ();
@ -44,6 +43,7 @@
}
bb4: {
Coverage::VirtualCounter(bcb2);
StorageDead(_1);
return;
}

View file

@ -8,15 +8,14 @@
coverage branch { true: BlockMarkerId(0), false: BlockMarkerId(1) } => $DIR/instrument_coverage_cleanup.rs:14:8: 14:36 (#0)
+ coverage body span: $DIR/instrument_coverage_cleanup.rs:13:11: 15:2 (#0)
+ coverage ExpressionId(0) => Expression { lhs: Counter(0), op: Subtract, rhs: Counter(1) };
+ coverage Code(Counter(0)) => $DIR/instrument_coverage_cleanup.rs:13:1: 14:36 (#0);
+ coverage Code(Expression(0)) => $DIR/instrument_coverage_cleanup.rs:14:37: 14:39 (#0);
+ coverage Code(Counter(1)) => $DIR/instrument_coverage_cleanup.rs:14:39: 14:39 (#0);
+ coverage Code(Counter(0)) => $DIR/instrument_coverage_cleanup.rs:15:2: 15:2 (#0);
+ coverage Branch { true_term: Expression(0), false_term: Counter(1) } => $DIR/instrument_coverage_cleanup.rs:14:8: 14:36 (#0);
+ coverage Code { bcb: bcb0 } => $DIR/instrument_coverage_cleanup.rs:13:1: 14:36 (#0);
+ coverage Code { bcb: bcb3 } => $DIR/instrument_coverage_cleanup.rs:14:37: 14:39 (#0);
+ coverage Code { bcb: bcb1 } => $DIR/instrument_coverage_cleanup.rs:14:39: 14:39 (#0);
+ coverage Code { bcb: bcb2 } => $DIR/instrument_coverage_cleanup.rs:15:2: 15:2 (#0);
+ coverage Branch { true_bcb: bcb3, false_bcb: bcb1 } => $DIR/instrument_coverage_cleanup.rs:14:8: 14:36 (#0);
+
bb0: {
+ Coverage::CounterIncrement(0);
+ Coverage::VirtualCounter(bcb0);
Coverage::SpanMarker;
StorageLive(_1);
_1 = std::hint::black_box::<bool>(const true) -> [return: bb1, unwind: bb5];
@ -27,20 +26,21 @@
}
bb2: {
+ Coverage::CounterIncrement(1);
+ Coverage::VirtualCounter(bcb1);
Coverage::BlockMarker(1);
_0 = const ();
goto -> bb4;
}
bb3: {
+ Coverage::ExpressionUsed(0);
+ Coverage::VirtualCounter(bcb3);
Coverage::BlockMarker(0);
_0 = const ();
goto -> bb4;
}
bb4: {
+ Coverage::VirtualCounter(bcb2);
StorageDead(_1);
return;
}

View file

@ -3,7 +3,7 @@
// but leaves the statements that were added by InstrumentCoverage.
//
// Removed statement kinds: BlockMarker, SpanMarker
// Retained statement kinds: CounterIncrement, ExpressionUsed
// Retained statement kinds: VirtualCounter
//@ test-mir-pass: InstrumentCoverage
//@ compile-flags: -Cinstrument-coverage -Zcoverage-options=branch -Zno-profiler-runtime
@ -16,6 +16,6 @@ fn main() {
// CHECK-NOT: Coverage::BlockMarker
// CHECK-NOT: Coverage::SpanMarker
// CHECK: Coverage::CounterIncrement
// CHECK: Coverage::VirtualCounter
// CHECK-NOT: Coverage::BlockMarker
// CHECK-NOT: Coverage::SpanMarker