Path: blob/main/cranelift/codegen/src/machinst/vcode.rs
1693 views
//! This implements the VCode container: a CFG of Insts that have been lowered.1//!2//! VCode is virtual-register code. An instruction in VCode is almost a machine3//! instruction; however, its register slots can refer to virtual registers in4//! addition to real machine registers.5//!6//! VCode is structured with traditional basic blocks, and7//! each block must be terminated by an unconditional branch (one target), a8//! conditional branch (two targets), or a return (no targets). Note that this9//! slightly differs from the machine code of most ISAs: in most ISAs, a10//! conditional branch has one target (and the not-taken case falls through).11//! However, we expect that machine backends will elide branches to the following12//! block (i.e., zero-offset jumps), and will be able to codegen a branch-cond /13//! branch-uncond pair if *both* targets are not fallthrough. This allows us to14//! play with layout prior to final binary emission, as well, if we want.15//!16//! See the main module comment in `mod.rs` for more details on the VCode-based17//! backend pipeline.1819use crate::CodegenError;20use crate::ir::pcc::*;21use crate::ir::{self, Constant, ConstantData, ValueLabel, types};22use crate::ranges::Ranges;23use crate::timing;24use crate::trace;25use crate::{LabelValueLoc, ValueLocRange};26use crate::{machinst::*, trace_log_enabled};27use regalloc2::{28Edit, Function as RegallocFunction, InstOrEdit, InstPosition, InstRange, Operand,29OperandConstraint, OperandKind, PRegSet, ProgPoint, RegClass,30};31use rustc_hash::FxHashMap;3233use core::cmp::Ordering;34use core::fmt::{self, Write};35use core::mem::take;36use cranelift_entity::{Keys, entity_impl};37use std::collections::HashMap;38use std::collections::hash_map::Entry;3940/// Index referring to an instruction in VCode.41pub type InsnIndex = regalloc2::Inst;4243/// Extension trait for `InsnIndex` to allow conversion to a44/// `BackwardsInsnIndex`.45trait ToBackwardsInsnIndex {46fn to_backwards_insn_index(&self, num_insts: usize) -> BackwardsInsnIndex;47}4849impl ToBackwardsInsnIndex for InsnIndex {50fn to_backwards_insn_index(&self, num_insts: usize) -> BackwardsInsnIndex {51BackwardsInsnIndex::new(num_insts - self.index() - 1)52}53}5455/// An index referring to an instruction in the VCode when it is backwards,56/// during VCode construction.57#[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)]58#[cfg_attr(59feature = "enable-serde",60derive(::serde::Serialize, ::serde::Deserialize)61)]62pub struct BackwardsInsnIndex(InsnIndex);6364impl BackwardsInsnIndex {65pub fn new(i: usize) -> Self {66BackwardsInsnIndex(InsnIndex::new(i))67}68}6970/// Index referring to a basic block in VCode.71pub type BlockIndex = regalloc2::Block;7273/// VCodeInst wraps all requirements for a MachInst to be in VCode: it must be74/// a `MachInst` and it must be able to emit itself at least to a `SizeCodeSink`.75pub trait VCodeInst: MachInst + MachInstEmit {}76impl<I: MachInst + MachInstEmit> VCodeInst for I {}7778/// A function in "VCode" (virtualized-register code) form, after79/// lowering. This is essentially a standard CFG of basic blocks,80/// where each basic block consists of lowered instructions produced81/// by the machine-specific backend.82///83/// Note that the VCode is immutable once produced, and is not84/// modified by register allocation in particular. Rather, register85/// allocation on the `VCode` produces a separate `regalloc2::Output`86/// struct, and this can be passed to `emit`. `emit` in turn does not87/// modify the vcode, but produces an `EmitResult`, which contains the88/// machine code itself, and the associated disassembly and/or89/// metadata as requested.90pub struct VCode<I: VCodeInst> {91/// VReg IR-level types.92vreg_types: Vec<Type>,9394/// Lowered machine instructions in order corresponding to the original IR.95insts: Vec<I>,9697/// A map from backwards instruction index to the user stack map for that98/// instruction.99///100/// This is a sparse side table that only has entries for instructions that101/// are safepoints, and only for a subset of those that have an associated102/// user stack map.103user_stack_maps: FxHashMap<BackwardsInsnIndex, ir::UserStackMap>,104105/// Operands: pre-regalloc references to virtual registers with106/// constraints, in one flattened array. This allows the regalloc107/// to efficiently access all operands without requiring expensive108/// matches or method invocations on insts.109operands: Vec<Operand>,110111/// Operand index ranges: for each instruction in `insts`, there112/// is a tuple here providing the range in `operands` for that113/// instruction's operands.114operand_ranges: Ranges,115116/// Clobbers: a sparse map from instruction indices to clobber masks.117clobbers: FxHashMap<InsnIndex, PRegSet>,118119/// Source locations for each instruction. (`SourceLoc` is a `u32`, so it is120/// reasonable to keep one of these per instruction.)121srclocs: Vec<RelSourceLoc>,122123/// Entry block.124entry: BlockIndex,125126/// Block instruction indices.127block_ranges: Ranges,128129/// Block successors: index range in the `block_succs` list.130block_succ_range: Ranges,131132/// Block successor lists, concatenated into one vec. The133/// `block_succ_range` list of tuples above gives (start, end)134/// ranges within this list that correspond to each basic block's135/// successors.136block_succs: Vec<regalloc2::Block>,137138/// Block predecessors: index range in the `block_preds` list.139block_pred_range: Ranges,140141/// Block predecessor lists, concatenated into one vec. The142/// `block_pred_range` list of tuples above gives (start, end)143/// ranges within this list that correspond to each basic block's144/// predecessors.145block_preds: Vec<regalloc2::Block>,146147/// Block parameters: index range in `block_params` below.148block_params_range: Ranges,149150/// Block parameter lists, concatenated into one vec. The151/// `block_params_range` list of tuples above gives (start, end)152/// ranges within this list that correspond to each basic block's153/// blockparam vregs.154block_params: Vec<regalloc2::VReg>,155156/// Outgoing block arguments on branch instructions, concatenated157/// into one list.158///159/// Note that this is conceptually a 3D array: we have a VReg list160/// per block, per successor. We flatten those three dimensions161/// into this 1D vec, then store index ranges in two levels of162/// indirection.163///164/// Indexed by the indices in `branch_block_arg_succ_range`.165branch_block_args: Vec<regalloc2::VReg>,166167/// Array of sequences of (start, end) tuples in168/// `branch_block_args`, one for each successor; these sequences169/// for each block are concatenated.170///171/// Indexed by the indices in `branch_block_arg_succ_range`.172branch_block_arg_range: Ranges,173174/// For a given block, indices in `branch_block_arg_range`175/// corresponding to all of its successors.176branch_block_arg_succ_range: Ranges,177178/// Block-order information.179block_order: BlockLoweringOrder,180181/// ABI object.182pub(crate) abi: Callee<I::ABIMachineSpec>,183184/// Constant information used during code emission. This should be185/// immutable across function compilations within the same module.186emit_info: I::Info,187188/// Constants.189pub(crate) constants: VCodeConstants,190191/// Value labels for debuginfo attached to vregs.192debug_value_labels: Vec<(VReg, InsnIndex, InsnIndex, u32)>,193194pub(crate) sigs: SigSet,195196/// Facts on VRegs, for proof-carrying code verification.197facts: Vec<Option<Fact>>,198199log2_min_function_alignment: u8,200}201202/// The result of `VCode::emit`. Contains all information computed203/// during emission: actual machine code, optionally a disassembly,204/// and optionally metadata about the code layout.205pub struct EmitResult {206/// The MachBuffer containing the machine code.207pub buffer: MachBufferFinalized<Stencil>,208209/// Offset of each basic block, recorded during emission. Computed210/// only if `machine_code_cfg_info` is enabled.211pub bb_offsets: Vec<CodeOffset>,212213/// Final basic-block edges, in terms of code offsets of214/// bb-starts. Computed only if `machine_code_cfg_info` is enabled.215pub bb_edges: Vec<(CodeOffset, CodeOffset)>,216217/// The pretty-printed disassembly, if any. This uses the same218/// pretty-printing for MachInsts as the pre-regalloc VCode Debug219/// implementation, but additionally includes the prologue and220/// epilogue(s), and makes use of the regalloc results.221pub disasm: Option<String>,222223/// Offsets of sized stackslots.224pub sized_stackslot_offsets: PrimaryMap<StackSlot, u32>,225226/// Offsets of dynamic stackslots.227pub dynamic_stackslot_offsets: PrimaryMap<DynamicStackSlot, u32>,228229/// Value-labels information (debug metadata).230pub value_labels_ranges: ValueLabelsRanges,231232/// Stack frame size.233pub frame_size: u32,234}235236/// A builder for a VCode function body.237///238/// This builder has the ability to accept instructions in either239/// forward or reverse order, depending on the pass direction that240/// produces the VCode. The lowering from CLIF to VCode<MachInst>241/// ordinarily occurs in reverse order (in order to allow instructions242/// to be lowered only if used, and not merged) so a reversal will243/// occur at the end of lowering to ensure the VCode is in machine244/// order.245///246/// If built in reverse, block and instruction indices used once the247/// VCode is built are relative to the final (reversed) order, not the248/// order of construction. Note that this means we do not know the249/// final block or instruction indices when building, so we do not250/// hand them out. (The user is assumed to know them when appending251/// terminator instructions with successor blocks.)252pub struct VCodeBuilder<I: VCodeInst> {253/// In-progress VCode.254pub(crate) vcode: VCode<I>,255256/// In what direction is the build occurring?257direction: VCodeBuildDirection,258259/// Debug-value label in-progress map, keyed by label. For each260/// label, we keep disjoint ranges mapping to vregs. We'll flatten261/// this into (vreg, range, label) tuples when done.262debug_info: FxHashMap<ValueLabel, Vec<(InsnIndex, InsnIndex, VReg)>>,263}264265/// Direction in which a VCodeBuilder builds VCode.266#[derive(Clone, Copy, Debug, PartialEq, Eq)]267pub enum VCodeBuildDirection {268// TODO: add `Forward` once we need it and can test it adequately.269/// Backward-build pass: we expect the producer to call `emit()`270/// with instructions in reverse program order within each block.271Backward,272}273274impl<I: VCodeInst> VCodeBuilder<I> {275/// Create a new VCodeBuilder.276pub fn new(277sigs: SigSet,278abi: Callee<I::ABIMachineSpec>,279emit_info: I::Info,280block_order: BlockLoweringOrder,281constants: VCodeConstants,282direction: VCodeBuildDirection,283log2_min_function_alignment: u8,284) -> Self {285let vcode = VCode::new(286sigs,287abi,288emit_info,289block_order,290constants,291log2_min_function_alignment,292);293294VCodeBuilder {295vcode,296direction,297debug_info: FxHashMap::default(),298}299}300301pub fn init_retval_area(&mut self, vregs: &mut VRegAllocator<I>) -> CodegenResult<()> {302self.vcode.abi.init_retval_area(&self.vcode.sigs, vregs)303}304305/// Access the ABI object.306pub fn abi(&self) -> &Callee<I::ABIMachineSpec> {307&self.vcode.abi308}309310/// Access the ABI object.311pub fn abi_mut(&mut self) -> &mut Callee<I::ABIMachineSpec> {312&mut self.vcode.abi313}314315pub fn sigs(&self) -> &SigSet {316&self.vcode.sigs317}318319pub fn sigs_mut(&mut self) -> &mut SigSet {320&mut self.vcode.sigs321}322323/// Access to the BlockLoweringOrder object.324pub fn block_order(&self) -> &BlockLoweringOrder {325&self.vcode.block_order326}327328/// Set the current block as the entry block.329pub fn set_entry(&mut self, block: BlockIndex) {330self.vcode.entry = block;331}332333/// End the current basic block. Must be called after emitting vcode insts334/// for IR insts and prior to ending the function (building the VCode).335pub fn end_bb(&mut self) {336let end_idx = self.vcode.insts.len();337// Add the instruction index range to the list of blocks.338self.vcode.block_ranges.push_end(end_idx);339// End the successors list.340let succ_end = self.vcode.block_succs.len();341self.vcode.block_succ_range.push_end(succ_end);342// End the blockparams list.343let block_params_end = self.vcode.block_params.len();344self.vcode.block_params_range.push_end(block_params_end);345// End the branch blockparam args list.346let branch_block_arg_succ_end = self.vcode.branch_block_arg_range.len();347self.vcode348.branch_block_arg_succ_range349.push_end(branch_block_arg_succ_end);350}351352pub fn add_block_param(&mut self, param: VirtualReg) {353self.vcode.block_params.push(param.into());354}355356fn add_branch_args_for_succ(&mut self, args: &[Reg]) {357self.vcode358.branch_block_args359.extend(args.iter().map(|&arg| VReg::from(arg)));360let end = self.vcode.branch_block_args.len();361self.vcode.branch_block_arg_range.push_end(end);362}363364/// Push an instruction for the current BB and current IR inst365/// within the BB.366pub fn push(&mut self, insn: I, loc: RelSourceLoc) {367assert!(!insn.is_low_level_branch()); // These are not meant to be in VCode.368self.vcode.insts.push(insn);369self.vcode.srclocs.push(loc);370}371372/// Add a successor block with branch args.373pub fn add_succ(&mut self, block: BlockIndex, args: &[Reg]) {374self.vcode.block_succs.push(block);375self.add_branch_args_for_succ(args);376}377378/// Add a debug value label to a register.379pub fn add_value_label(&mut self, reg: Reg, label: ValueLabel) {380// 1) In the reversed order, we consider the instructions381// that define ranges in the "debug_info" array to refer382// to the IP **after** them (when reversed):383// IP[2]__| Inst 3 |384// IP[1]__| Inst 2 |385// IP[0]__| Inst 1 |386// | Inst 0 |387// This is so that we can represent IP[<function start>],388// done at the cost of not being to represent IP[<function end>],389// which is OK since no values will be live at that point.390// 2) The live range for "reg" begins at the current IP391// and continues until the next, in execution order,392// VReg that defines "label". Since the ranges are open393// at the end, the subtraction of 1 cancels out:394// [last..current IP] <=>395// [last..last emitted inst index] <=>396// [last..next_inst_index - 1] <=>397// [last..next_inst_index)398//399let next_inst_index = self.vcode.insts.len();400if next_inst_index == 0 {401// This would produce a defective [0..0) range.402return;403}404let next_inst = InsnIndex::new(next_inst_index);405let labels = self.debug_info.entry(label).or_insert_with(|| vec![]);406let last = labels407.last()408.map(|(_start, end, _vreg)| *end)409.unwrap_or(InsnIndex::new(0));410labels.push((last, next_inst, reg.into()));411}412413/// Access the constants.414pub fn constants(&mut self) -> &mut VCodeConstants {415&mut self.vcode.constants416}417418fn compute_preds_from_succs(&mut self) {419// Do a linear-time counting sort: first determine how many420// times each block appears as a successor.421let mut starts = vec![0u32; self.vcode.num_blocks()];422for succ in &self.vcode.block_succs {423starts[succ.index()] += 1;424}425426// Determine for each block the starting index where that427// block's predecessors should go. This is equivalent to the428// ranges we need to store in block_pred_range.429self.vcode.block_pred_range.reserve(starts.len());430let mut end = 0;431for count in starts.iter_mut() {432let start = end;433end += *count;434*count = start;435self.vcode.block_pred_range.push_end(end as usize);436}437let end = end as usize;438debug_assert_eq!(end, self.vcode.block_succs.len());439440// Walk over the successors again, this time grouped by441// predecessor, and push the predecessor at the current442// starting position of each of its successors. We build443// each group of predecessors in whatever order Ranges::iter444// returns them; regalloc2 doesn't care.445self.vcode.block_preds.resize(end, BlockIndex::invalid());446for (pred, range) in self.vcode.block_succ_range.iter() {447let pred = BlockIndex::new(pred);448for succ in &self.vcode.block_succs[range] {449let pos = &mut starts[succ.index()];450self.vcode.block_preds[*pos as usize] = pred;451*pos += 1;452}453}454debug_assert!(self.vcode.block_preds.iter().all(|pred| pred.is_valid()));455}456457/// Called once, when a build in Backward order is complete, to458/// perform the overall reversal (into final forward order) and459/// finalize metadata accordingly.460fn reverse_and_finalize(&mut self, vregs: &VRegAllocator<I>) {461let n_insts = self.vcode.insts.len();462if n_insts == 0 {463return;464}465466// Reverse the per-block and per-inst sequences.467self.vcode.block_ranges.reverse_index();468self.vcode.block_ranges.reverse_target(n_insts);469// block_params_range is indexed by block (and blocks were470// traversed in reverse) so we reverse it; but block-param471// sequences in the concatenated vec can remain in reverse472// order (it is effectively an arena of arbitrarily-placed473// referenced sequences).474self.vcode.block_params_range.reverse_index();475// Likewise, we reverse block_succ_range, but the block_succ476// concatenated array can remain as-is.477self.vcode.block_succ_range.reverse_index();478self.vcode.insts.reverse();479self.vcode.srclocs.reverse();480// Likewise, branch_block_arg_succ_range is indexed by block481// so must be reversed.482self.vcode.branch_block_arg_succ_range.reverse_index();483484// To translate an instruction index *endpoint* in reversed485// order to forward order, compute `n_insts - i`.486//487// Why not `n_insts - 1 - i`? That would be correct to488// translate an individual instruction index (for ten insts 0489// to 9 inclusive, inst 0 becomes 9, and inst 9 becomes490// 0). But for the usual inclusive-start, exclusive-end range491// idiom, inclusive starts become exclusive ends and492// vice-versa, so e.g. an (inclusive) start of 0 becomes an493// (exclusive) end of 10.494let translate = |inst: InsnIndex| InsnIndex::new(n_insts - inst.index());495496// Generate debug-value labels based on per-label maps.497for (label, tuples) in &self.debug_info {498for &(start, end, vreg) in tuples {499let vreg = vregs.resolve_vreg_alias(vreg);500let fwd_start = translate(end);501let fwd_end = translate(start);502self.vcode503.debug_value_labels504.push((vreg, fwd_start, fwd_end, label.as_u32()));505}506}507508// Now sort debug value labels by VReg, as required509// by regalloc2.510self.vcode511.debug_value_labels512.sort_unstable_by_key(|(vreg, _, _, _)| *vreg);513}514515fn collect_operands(&mut self, vregs: &VRegAllocator<I>) {516let allocatable = PRegSet::from(self.vcode.abi.machine_env());517for (i, insn) in self.vcode.insts.iter_mut().enumerate() {518// Push operands from the instruction onto the operand list.519//520// We rename through the vreg alias table as we collect521// the operands. This is better than a separate post-pass522// over operands, because it has more cache locality:523// operands only need to pass through L1 once. This is524// also better than renaming instructions'525// operands/registers while lowering, because here we only526// need to do the `match` over the instruction to visit527// its register fields (which is slow, branchy code) once.528529let mut op_collector =530OperandCollector::new(&mut self.vcode.operands, allocatable, |vreg| {531vregs.resolve_vreg_alias(vreg)532});533insn.get_operands(&mut op_collector);534let (ops, clobbers) = op_collector.finish();535self.vcode.operand_ranges.push_end(ops);536537if clobbers != PRegSet::default() {538self.vcode.clobbers.insert(InsnIndex::new(i), clobbers);539}540541if let Some((dst, src)) = insn.is_move() {542// We should never see non-virtual registers present in move543// instructions.544assert!(545src.is_virtual(),546"the real register {src:?} was used as the source of a move instruction"547);548assert!(549dst.to_reg().is_virtual(),550"the real register {:?} was used as the destination of a move instruction",551dst.to_reg()552);553}554}555556// Translate blockparam args via the vreg aliases table as well.557for arg in &mut self.vcode.branch_block_args {558let new_arg = vregs.resolve_vreg_alias(*arg);559trace!("operandcollector: block arg {:?} -> {:?}", arg, new_arg);560*arg = new_arg;561}562}563564/// Build the final VCode.565pub fn build(mut self, mut vregs: VRegAllocator<I>) -> VCode<I> {566self.vcode.vreg_types = take(&mut vregs.vreg_types);567self.vcode.facts = take(&mut vregs.facts);568569if self.direction == VCodeBuildDirection::Backward {570self.reverse_and_finalize(&vregs);571}572self.collect_operands(&vregs);573574self.compute_preds_from_succs();575self.vcode.debug_value_labels.sort_unstable();576577// At this point, nothing in the vcode should mention any578// VReg which has been aliased. All the appropriate rewriting579// should have happened above. Just to be sure, let's580// double-check each field which has vregs.581// Note: can't easily check vcode.insts, resolved in collect_operands.582// Operands are resolved in collect_operands.583vregs.debug_assert_no_vreg_aliases(self.vcode.operands.iter().map(|op| op.vreg()));584// Currently block params are never aliased to another vreg.585vregs.debug_assert_no_vreg_aliases(self.vcode.block_params.iter().copied());586// Branch block args are resolved in collect_operands.587vregs.debug_assert_no_vreg_aliases(self.vcode.branch_block_args.iter().copied());588// Debug value labels are resolved in reverse_and_finalize.589vregs.debug_assert_no_vreg_aliases(590self.vcode.debug_value_labels.iter().map(|&(vreg, ..)| vreg),591);592// Facts are resolved eagerly during set_vreg_alias.593vregs.debug_assert_no_vreg_aliases(594self.vcode595.facts596.iter()597.zip(&vregs.vreg_types)598.enumerate()599.filter(|(_, (fact, _))| fact.is_some())600.map(|(vreg, (_, &ty))| {601let (regclasses, _) = I::rc_for_type(ty).unwrap();602VReg::new(vreg, regclasses[0])603}),604);605606self.vcode607}608609/// Add a user stack map for the associated instruction.610pub fn add_user_stack_map(611&mut self,612inst: BackwardsInsnIndex,613entries: &[ir::UserStackMapEntry],614) {615let stack_map = ir::UserStackMap::new(entries, self.vcode.abi.sized_stackslot_offsets());616let old_entry = self.vcode.user_stack_maps.insert(inst, stack_map);617debug_assert!(old_entry.is_none());618}619}620621const NO_INST_OFFSET: CodeOffset = u32::MAX;622623impl<I: VCodeInst> VCode<I> {624/// New empty VCode.625fn new(626sigs: SigSet,627abi: Callee<I::ABIMachineSpec>,628emit_info: I::Info,629block_order: BlockLoweringOrder,630constants: VCodeConstants,631log2_min_function_alignment: u8,632) -> Self {633let n_blocks = block_order.lowered_order().len();634VCode {635sigs,636vreg_types: vec![],637insts: Vec::with_capacity(10 * n_blocks),638user_stack_maps: FxHashMap::default(),639operands: Vec::with_capacity(30 * n_blocks),640operand_ranges: Ranges::with_capacity(10 * n_blocks),641clobbers: FxHashMap::default(),642srclocs: Vec::with_capacity(10 * n_blocks),643entry: BlockIndex::new(0),644block_ranges: Ranges::with_capacity(n_blocks),645block_succ_range: Ranges::with_capacity(n_blocks),646block_succs: Vec::with_capacity(n_blocks),647block_pred_range: Ranges::default(),648block_preds: Vec::new(),649block_params_range: Ranges::with_capacity(n_blocks),650block_params: Vec::with_capacity(5 * n_blocks),651branch_block_args: Vec::with_capacity(10 * n_blocks),652branch_block_arg_range: Ranges::with_capacity(2 * n_blocks),653branch_block_arg_succ_range: Ranges::with_capacity(n_blocks),654block_order,655abi,656emit_info,657constants,658debug_value_labels: vec![],659facts: vec![],660log2_min_function_alignment,661}662}663664/// Get the number of blocks. Block indices will be in the range `0 ..665/// (self.num_blocks() - 1)`.666pub fn num_blocks(&self) -> usize {667self.block_ranges.len()668}669670/// The number of lowered instructions.671pub fn num_insts(&self) -> usize {672self.insts.len()673}674675fn compute_clobbers_and_function_calls(676&self,677regalloc: ®alloc2::Output,678) -> (Vec<Writable<RealReg>>, FunctionCalls) {679let mut clobbered = PRegSet::default();680let mut function_calls = FunctionCalls::None;681682// All moves are included in clobbers.683for (_, Edit::Move { to, .. }) in ®alloc.edits {684if let Some(preg) = to.as_reg() {685clobbered.add(preg);686}687}688689for (i, range) in self.operand_ranges.iter() {690let operands = &self.operands[range.clone()];691let allocs = ®alloc.allocs[range];692for (operand, alloc) in operands.iter().zip(allocs.iter()) {693if operand.kind() == OperandKind::Def {694if let Some(preg) = alloc.as_reg() {695clobbered.add(preg);696}697}698}699700function_calls.update(self.insts[i].call_type());701702// Also add explicitly-clobbered registers.703//704// Skip merging this instruction's clobber list if not705// "included in clobbers" as per the MachInst. (Some706// backends use this to implement ABI specifics; e.g.,707// excluding calls of the same ABI as the current function708// from clobbers, because by definition everything709// clobbered by the call can be clobbered by this function710// without saving as well.711//712// This is important for a particular optimization: when713// some registers are "half-clobbered", e.g. vector/float714// registers on aarch64, we want them to be seen as715// clobbered by regalloc so it avoids carrying values716// across calls in these registers but not seen as717// clobbered by prologue generation here (because the718// actual half-clobber implied by the clobber list fits719// within the clobbers that we allow without720// clobber-saves).721if self.insts[i].is_included_in_clobbers() {722if let Some(&inst_clobbered) = self.clobbers.get(&InsnIndex::new(i)) {723clobbered.union_from(inst_clobbered);724}725}726}727728let clobbered_regs = clobbered729.into_iter()730.map(|preg| Writable::from_reg(RealReg::from(preg)))731.collect();732733(clobbered_regs, function_calls)734}735736/// Emit the instructions to a `MachBuffer`, containing fixed-up737/// code and external reloc/trap/etc. records ready for use. Takes738/// the regalloc results as well.739///740/// Returns the machine code itself, and optionally metadata741/// and/or a disassembly, as an `EmitResult`. The `VCode` itself742/// is consumed by the emission process.743pub fn emit(744mut self,745regalloc: ®alloc2::Output,746want_disasm: bool,747flags: &settings::Flags,748ctrl_plane: &mut ControlPlane,749) -> EmitResult750where751I: VCodeInst,752{753let _tt = timing::vcode_emit();754let mut buffer = MachBuffer::new();755buffer.set_log2_min_function_alignment(self.log2_min_function_alignment);756let mut bb_starts: Vec<Option<CodeOffset>> = vec![];757758// The first M MachLabels are reserved for block indices.759buffer.reserve_labels_for_blocks(self.num_blocks());760761// Register all allocated constants with the `MachBuffer` to ensure that762// any references to the constants during instructions can be handled763// correctly.764buffer.register_constants(&self.constants);765766// Construct the final order we emit code in: cold blocks at the end.767let mut final_order: SmallVec<[BlockIndex; 16]> = smallvec![];768let mut cold_blocks: SmallVec<[BlockIndex; 16]> = smallvec![];769for block in 0..self.num_blocks() {770let block = BlockIndex::new(block);771if self.block_order.is_cold(block) {772cold_blocks.push(block);773} else {774final_order.push(block);775}776}777final_order.extend(cold_blocks.clone());778779// Compute/save info we need for the prologue: clobbers and780// number of spillslots.781//782// We clone `abi` here because we will mutate it as we783// generate the prologue and set other info, but we can't784// mutate `VCode`. The info it usually carries prior to785// setting clobbers is fairly minimal so this should be786// relatively cheap.787let (clobbers, function_calls) = self.compute_clobbers_and_function_calls(regalloc);788self.abi.compute_frame_layout(789&self.sigs,790regalloc.num_spillslots,791clobbers,792function_calls,793);794795// Emit blocks.796let mut cur_srcloc = None;797let mut last_offset = None;798let mut inst_offsets = vec![];799let mut state = I::State::new(&self.abi, std::mem::take(ctrl_plane));800801let mut disasm = String::new();802803if !self.debug_value_labels.is_empty() {804inst_offsets.resize(self.insts.len(), NO_INST_OFFSET);805}806807// Count edits per block ahead of time; this is needed for808// lookahead island emission. (We could derive it per-block809// with binary search in the edit list, but it's more810// efficient to do it in one pass here.)811let mut ra_edits_per_block: SmallVec<[u32; 64]> = smallvec![];812let mut edit_idx = 0;813for block in 0..self.num_blocks() {814let end_inst = InsnIndex::new(self.block_ranges.get(block).end);815let start_edit_idx = edit_idx;816while edit_idx < regalloc.edits.len() && regalloc.edits[edit_idx].0.inst() < end_inst {817edit_idx += 1;818}819let end_edit_idx = edit_idx;820ra_edits_per_block.push((end_edit_idx - start_edit_idx) as u32);821}822823let is_forward_edge_cfi_enabled = self.abi.is_forward_edge_cfi_enabled();824let mut bb_padding = match flags.bb_padding_log2_minus_one() {8250 => Vec::new(),826n => vec![0; 1 << (n - 1)],827};828let mut total_bb_padding = 0;829830for (block_order_idx, &block) in final_order.iter().enumerate() {831trace!("emitting block {:?}", block);832833// Call the new block hook for state834state.on_new_block();835836// Emit NOPs to align the block.837let new_offset = I::align_basic_block(buffer.cur_offset());838while new_offset > buffer.cur_offset() {839// Pad with NOPs up to the aligned block offset.840let nop = I::gen_nop((new_offset - buffer.cur_offset()) as usize);841nop.emit(&mut buffer, &self.emit_info, &mut Default::default());842}843assert_eq!(buffer.cur_offset(), new_offset);844845let do_emit = |inst: &I,846disasm: &mut String,847buffer: &mut MachBuffer<I>,848state: &mut I::State| {849if want_disasm && !inst.is_args() {850let mut s = state.clone();851writeln!(disasm, " {}", inst.pretty_print_inst(&mut s)).unwrap();852}853inst.emit(buffer, &self.emit_info, state);854};855856// Is this the first block? Emit the prologue directly if so.857if block == self.entry {858trace!(" -> entry block");859buffer.start_srcloc(Default::default());860for inst in &self.abi.gen_prologue() {861do_emit(&inst, &mut disasm, &mut buffer, &mut state);862}863buffer.end_srcloc();864}865866// Now emit the regular block body.867868buffer.bind_label(MachLabel::from_block(block), state.ctrl_plane_mut());869870if want_disasm {871writeln!(&mut disasm, "block{}:", block.index()).unwrap();872}873874if flags.machine_code_cfg_info() {875// Track BB starts. If we have backed up due to MachBuffer876// branch opts, note that the removed blocks were removed.877let cur_offset = buffer.cur_offset();878if last_offset.is_some() && cur_offset <= last_offset.unwrap() {879for i in (0..bb_starts.len()).rev() {880if bb_starts[i].is_some() && cur_offset > bb_starts[i].unwrap() {881break;882}883bb_starts[i] = None;884}885}886bb_starts.push(Some(cur_offset));887last_offset = Some(cur_offset);888}889890if let Some(block_start) = I::gen_block_start(891self.block_order.is_indirect_branch_target(block),892is_forward_edge_cfi_enabled,893) {894do_emit(&block_start, &mut disasm, &mut buffer, &mut state);895}896897for inst_or_edit in regalloc.block_insts_and_edits(&self, block) {898match inst_or_edit {899InstOrEdit::Inst(iix) => {900if !self.debug_value_labels.is_empty() {901// If we need to produce debug info,902// record the offset of each instruction903// so that we can translate value-label904// ranges to machine-code offsets.905906// Cold blocks violate monotonicity907// assumptions elsewhere (that908// instructions in inst-index order are in909// order in machine code), so we omit910// their offsets here. Value-label range911// generation below will skip empty ranges912// and ranges with to-offsets of zero.913if !self.block_order.is_cold(block) {914inst_offsets[iix.index()] = buffer.cur_offset();915}916}917918// Update the srcloc at this point in the buffer.919let srcloc = self.srclocs[iix.index()];920if cur_srcloc != Some(srcloc) {921if cur_srcloc.is_some() {922buffer.end_srcloc();923}924buffer.start_srcloc(srcloc);925cur_srcloc = Some(srcloc);926}927928// If this is a safepoint, compute a stack map929// and pass it to the emit state.930let stack_map_disasm = if self.insts[iix.index()].is_safepoint() {931let (user_stack_map, user_stack_map_disasm) = {932// The `user_stack_maps` is keyed by reverse933// instruction index, so we must flip the934// index. We can't put this into a helper method935// due to borrowck issues because parts of936// `self` are borrowed mutably elsewhere in this937// function.938let index = iix.to_backwards_insn_index(self.num_insts());939let user_stack_map = self.user_stack_maps.remove(&index);940let user_stack_map_disasm =941user_stack_map.as_ref().map(|m| format!(" ; {m:?}"));942(user_stack_map, user_stack_map_disasm)943};944945state.pre_safepoint(user_stack_map);946947user_stack_map_disasm948} else {949None950};951952// If the instruction we are about to emit is953// a return, place an epilogue at this point954// (and don't emit the return; the actual955// epilogue will contain it).956if self.insts[iix.index()].is_term() == MachTerminator::Ret {957for inst in self.abi.gen_epilogue() {958do_emit(&inst, &mut disasm, &mut buffer, &mut state);959}960} else {961// Update the operands for this inst using the962// allocations from the regalloc result.963let mut allocs = regalloc.inst_allocs(iix).iter();964self.insts[iix.index()].get_operands(965&mut |reg: &mut Reg, constraint, _kind, _pos| {966let alloc =967allocs.next().expect("enough allocations for all operands");968969if let Some(alloc) = alloc.as_reg() {970let alloc: Reg = alloc.into();971if let OperandConstraint::FixedReg(rreg) = constraint {972debug_assert_eq!(Reg::from(rreg), alloc);973}974*reg = alloc;975} else if let Some(alloc) = alloc.as_stack() {976let alloc: Reg = alloc.into();977*reg = alloc;978}979},980);981debug_assert!(allocs.next().is_none());982983// Emit the instruction!984do_emit(985&self.insts[iix.index()],986&mut disasm,987&mut buffer,988&mut state,989);990if let Some(stack_map_disasm) = stack_map_disasm {991disasm.push_str(&stack_map_disasm);992disasm.push('\n');993}994}995}996997InstOrEdit::Edit(Edit::Move { from, to }) => {998// Create a move/spill/reload instruction and999// immediately emit it.1000match (from.as_reg(), to.as_reg()) {1001(Some(from), Some(to)) => {1002// Reg-to-reg move.1003let from_rreg = Reg::from(from);1004let to_rreg = Writable::from_reg(Reg::from(to));1005debug_assert_eq!(from.class(), to.class());1006let ty = I::canonical_type_for_rc(from.class());1007let mv = I::gen_move(to_rreg, from_rreg, ty);1008do_emit(&mv, &mut disasm, &mut buffer, &mut state);1009}1010(Some(from), None) => {1011// Spill from register to spillslot.1012let to = to.as_stack().unwrap();1013let from_rreg = RealReg::from(from);1014let spill = self.abi.gen_spill(to, from_rreg);1015do_emit(&spill, &mut disasm, &mut buffer, &mut state);1016}1017(None, Some(to)) => {1018// Load from spillslot to register.1019let from = from.as_stack().unwrap();1020let to_rreg = Writable::from_reg(RealReg::from(to));1021let reload = self.abi.gen_reload(to_rreg, from);1022do_emit(&reload, &mut disasm, &mut buffer, &mut state);1023}1024(None, None) => {1025panic!("regalloc2 should have eliminated stack-to-stack moves!");1026}1027}1028}1029}1030}10311032if cur_srcloc.is_some() {1033buffer.end_srcloc();1034cur_srcloc = None;1035}10361037// Do we need an island? Get the worst-case size of the next BB, add1038// it to the optional padding behind the block, and pass this to the1039// `MachBuffer` to determine if an island is necessary.1040let worst_case_next_bb = if block_order_idx < final_order.len() - 1 {1041let next_block = final_order[block_order_idx + 1];1042let next_block_range = self.block_ranges.get(next_block.index());1043let next_block_size = next_block_range.len() as u32;1044let next_block_ra_insertions = ra_edits_per_block[next_block.index()];1045I::worst_case_size() * (next_block_size + next_block_ra_insertions)1046} else {104701048};1049let padding = if bb_padding.is_empty() {105001051} else {1052bb_padding.len() as u32 + I::LabelUse::ALIGN - 11053};1054if buffer.island_needed(padding + worst_case_next_bb) {1055buffer.emit_island(padding + worst_case_next_bb, ctrl_plane);1056}10571058// Insert padding, if configured, to stress the `MachBuffer`'s1059// relocation and island calculations.1060//1061// Padding can get quite large during fuzzing though so place a1062// total cap on it where when a per-function threshold is exceeded1063// the padding is turned back down to zero. This avoids a small-ish1064// test case generating a GB+ memory footprint in Cranelift for1065// example.1066if !bb_padding.is_empty() {1067buffer.put_data(&bb_padding);1068buffer.align_to(I::LabelUse::ALIGN);1069total_bb_padding += bb_padding.len();1070if total_bb_padding > (150 << 20) {1071bb_padding = Vec::new();1072}1073}1074}10751076debug_assert!(1077self.user_stack_maps.is_empty(),1078"any stack maps should have been consumed by instruction emission, still have: {:#?}",1079self.user_stack_maps,1080);10811082// Do any optimizations on branches at tail of buffer, as if we had1083// bound one last label.1084buffer.optimize_branches(ctrl_plane);10851086// emission state is not needed anymore, move control plane back out1087*ctrl_plane = state.take_ctrl_plane();10881089let func_body_len = buffer.cur_offset();10901091// Create `bb_edges` and final (filtered) `bb_starts`.1092let mut bb_edges = vec![];1093let mut bb_offsets = vec![];1094if flags.machine_code_cfg_info() {1095for block in 0..self.num_blocks() {1096if bb_starts[block].is_none() {1097// Block was deleted by MachBuffer; skip.1098continue;1099}1100let from = bb_starts[block].unwrap();11011102bb_offsets.push(from);1103// Resolve each `succ` label and add edges.1104let succs = self.block_succs(BlockIndex::new(block));1105for &succ in succs.iter() {1106let to = buffer.resolve_label_offset(MachLabel::from_block(succ));1107bb_edges.push((from, to));1108}1109}1110}11111112self.monotonize_inst_offsets(&mut inst_offsets[..], func_body_len);1113let value_labels_ranges =1114self.compute_value_labels_ranges(regalloc, &inst_offsets[..], func_body_len);1115let frame_size = self.abi.frame_size();11161117EmitResult {1118buffer: buffer.finish(&self.constants, ctrl_plane),1119bb_offsets,1120bb_edges,1121disasm: if want_disasm { Some(disasm) } else { None },1122sized_stackslot_offsets: self.abi.sized_stackslot_offsets().clone(),1123dynamic_stackslot_offsets: self.abi.dynamic_stackslot_offsets().clone(),1124value_labels_ranges,1125frame_size,1126}1127}11281129fn monotonize_inst_offsets(&self, inst_offsets: &mut [CodeOffset], func_body_len: u32) {1130if self.debug_value_labels.is_empty() {1131return;1132}11331134// During emission, branch removal can make offsets of instructions incorrect.1135// Consider the following sequence: [insi][jmp0][jmp1][jmp2][insj]1136// It will be recorded as (say): [30] [34] [38] [42] [<would be 46>]1137// When the jumps get removed we are left with (in "inst_offsets"):1138// [insi][jmp0][jmp1][jmp2][insj][...]1139// [30] [34] [38] [42] [34]1140// Which violates the monotonicity invariant. This method sets offsets of these1141// removed instructions such as to make them appear zero-sized:1142// [insi][jmp0][jmp1][jmp2][insj][...]1143// [30] [34] [34] [34] [34]1144//1145let mut next_offset = func_body_len;1146for inst_index in (0..(inst_offsets.len() - 1)).rev() {1147let inst_offset = inst_offsets[inst_index];11481149// Not all instructions get their offsets recorded.1150if inst_offset == NO_INST_OFFSET {1151continue;1152}11531154if inst_offset > next_offset {1155trace!(1156"Fixing code offset of the removed Inst {}: {} -> {}",1157inst_index, inst_offset, next_offset1158);1159inst_offsets[inst_index] = next_offset;1160continue;1161}11621163next_offset = inst_offset;1164}1165}11661167fn compute_value_labels_ranges(1168&self,1169regalloc: ®alloc2::Output,1170inst_offsets: &[CodeOffset],1171func_body_len: u32,1172) -> ValueLabelsRanges {1173if self.debug_value_labels.is_empty() {1174return ValueLabelsRanges::default();1175}11761177if trace_log_enabled!() {1178self.log_value_labels_ranges(regalloc, inst_offsets);1179}11801181let mut value_labels_ranges: ValueLabelsRanges = HashMap::new();1182for &(label, from, to, alloc) in ®alloc.debug_locations {1183let label = ValueLabel::from_u32(label);1184let ranges = value_labels_ranges.entry(label).or_insert_with(|| vec![]);1185let prog_point_to_inst = |prog_point: ProgPoint| {1186let mut inst = prog_point.inst();1187if prog_point.pos() == InstPosition::After {1188inst = inst.next();1189}1190inst.index()1191};1192let from_inst_index = prog_point_to_inst(from);1193let to_inst_index = prog_point_to_inst(to);1194let from_offset = inst_offsets[from_inst_index];1195let to_offset = if to_inst_index == inst_offsets.len() {1196func_body_len1197} else {1198inst_offsets[to_inst_index]1199};12001201// Empty ranges or unavailable offsets can happen1202// due to cold blocks and branch removal (see above).1203if from_offset == NO_INST_OFFSET1204|| to_offset == NO_INST_OFFSET1205|| from_offset == to_offset1206{1207continue;1208}12091210let loc = if let Some(preg) = alloc.as_reg() {1211LabelValueLoc::Reg(Reg::from(preg))1212} else {1213let slot = alloc.as_stack().unwrap();1214let slot_offset = self.abi.get_spillslot_offset(slot);1215let slot_base_to_caller_sp_offset = self.abi.slot_base_to_caller_sp_offset();1216let caller_sp_to_cfa_offset =1217crate::isa::unwind::systemv::caller_sp_to_cfa_offset();1218// NOTE: this is a negative offset because it's relative to the caller's SP1219let cfa_to_sp_offset =1220-((slot_base_to_caller_sp_offset + caller_sp_to_cfa_offset) as i64);1221LabelValueLoc::CFAOffset(cfa_to_sp_offset + slot_offset)1222};12231224// Coalesce adjacent ranges that for the same location1225// to minimize output size here and for the consumers.1226if let Some(last_loc_range) = ranges.last_mut() {1227if last_loc_range.loc == loc && last_loc_range.end == from_offset {1228trace!(1229"Extending debug range for {:?} in {:?} to Inst {} ({})",1230label, loc, to_inst_index, to_offset1231);1232last_loc_range.end = to_offset;1233continue;1234}1235}12361237trace!(1238"Recording debug range for {:?} in {:?}: [Inst {}..Inst {}) [{}..{})",1239label, loc, from_inst_index, to_inst_index, from_offset, to_offset1240);12411242ranges.push(ValueLocRange {1243loc,1244start: from_offset,1245end: to_offset,1246});1247}12481249value_labels_ranges1250}12511252fn log_value_labels_ranges(&self, regalloc: ®alloc2::Output, inst_offsets: &[CodeOffset]) {1253debug_assert!(trace_log_enabled!());12541255// What debug labels do we have? Note we'll skip those that have not been1256// allocated any location at all. They will show up as numeric gaps in the table.1257let mut labels = vec![];1258for &(label, _, _, _) in ®alloc.debug_locations {1259if Some(&label) == labels.last() {1260continue;1261}1262labels.push(label);1263}12641265// Reformat the data on what VRegs were the VLs assigned to by lowering, since1266// the array we have is sorted by VReg, and we want it sorted by VL for easy1267// access in the loop below.1268let mut vregs = vec![];1269for &(vreg, start, end, label) in &self.debug_value_labels {1270if matches!(labels.binary_search(&label), Ok(_)) {1271vregs.push((label, start, end, vreg));1272}1273}1274vregs.sort_unstable_by(1275|(l_label, l_start, _, _), (r_label, r_start, _, _)| match l_label.cmp(r_label) {1276Ordering::Equal => l_start.cmp(r_start),1277cmp => cmp,1278},1279);12801281#[derive(PartialEq)]1282enum Mode {1283Measure,1284Emit,1285}1286#[derive(PartialEq)]1287enum Row {1288Head,1289Line,1290Inst(usize, usize),1291}12921293let mut widths = vec![0; 3 + 2 * labels.len()];1294let mut row = String::new();1295let mut output_row = |row_kind: Row, mode: Mode| {1296let mut column_index = 0;1297row.clear();12981299macro_rules! output_cell_impl {1300($fill:literal, $span:literal, $($cell_fmt:tt)*) => {1301let column_start = row.len();1302{1303row.push('|');1304write!(row, $($cell_fmt)*).unwrap();1305}13061307let next_column_index = column_index + $span;1308let expected_width: usize = widths[column_index..next_column_index].iter().sum();1309if mode == Mode::Measure {1310let actual_width = row.len() - column_start;1311if actual_width > expected_width {1312widths[next_column_index - 1] += actual_width - expected_width;1313}1314} else {1315let column_end = column_start + expected_width;1316while row.len() != column_end {1317row.push($fill);1318}1319}1320column_index = next_column_index;1321};1322}1323macro_rules! output_cell {1324($($cell_fmt:tt)*) => {1325output_cell_impl!(' ', 1, $($cell_fmt)*);1326};1327}13281329match row_kind {1330Row::Head => {1331output_cell!("BB");1332output_cell!("Inst");1333output_cell!("IP");1334for label in &labels {1335output_cell_impl!(' ', 2, "{:?}", ValueLabel::from_u32(*label));1336}1337}1338Row::Line => {1339debug_assert!(mode == Mode::Emit);1340for _ in 0..3 {1341output_cell_impl!('-', 1, "");1342}1343for _ in &labels {1344output_cell_impl!('-', 2, "");1345}1346}1347Row::Inst(block_index, inst_index) => {1348debug_assert!(inst_index < self.num_insts());1349if self.block_ranges.get(block_index).start == inst_index {1350output_cell!("B{}", block_index);1351} else {1352output_cell!("");1353}1354output_cell!("Inst {inst_index} ");1355output_cell!("{} ", inst_offsets[inst_index]);13561357for label in &labels {1358// First, the VReg.1359use regalloc2::Inst;1360let vreg_cmp = |inst: usize,1361vreg_label: &u32,1362range_start: &Inst,1363range_end: &Inst| {1364match vreg_label.cmp(&label) {1365Ordering::Equal => {1366if range_end.index() <= inst {1367Ordering::Less1368} else if range_start.index() > inst {1369Ordering::Greater1370} else {1371Ordering::Equal1372}1373}1374cmp => cmp,1375}1376};1377let vreg_index =1378vregs.binary_search_by(|(l, s, e, _)| vreg_cmp(inst_index, l, s, e));1379if let Ok(vreg_index) = vreg_index {1380let mut prev_vreg = None;1381if inst_index > 0 {1382let prev_vreg_index = vregs.binary_search_by(|(l, s, e, _)| {1383vreg_cmp(inst_index - 1, l, s, e)1384});1385if let Ok(prev_vreg_index) = prev_vreg_index {1386prev_vreg = Some(vregs[prev_vreg_index].3);1387}1388}13891390let vreg = vregs[vreg_index].3;1391if Some(vreg) == prev_vreg {1392output_cell!("*");1393} else {1394output_cell!("{}", vreg);1395}1396} else {1397output_cell!("");1398}13991400// Second, the allocated location.1401let inst_prog_point = ProgPoint::before(Inst::new(inst_index));1402let range_index = regalloc.debug_locations.binary_search_by(1403|(range_label, range_start, range_end, _)| match range_label.cmp(label)1404{1405Ordering::Equal => {1406if *range_end <= inst_prog_point {1407Ordering::Less1408} else if *range_start > inst_prog_point {1409Ordering::Greater1410} else {1411Ordering::Equal1412}1413}1414cmp => cmp,1415},1416);1417if let Ok(range_index) = range_index {1418// Live at this instruction, print the location.1419if let Some(reg) = regalloc.debug_locations[range_index].3.as_reg() {1420output_cell!("{:?}", Reg::from(reg));1421} else {1422output_cell!("Stk");1423}1424} else {1425// Not live at this instruction.1426output_cell!("");1427}1428}1429}1430}1431row.push('|');14321433if mode == Mode::Emit {1434trace!("{}", row.as_str());1435}1436};14371438for block_index in 0..self.num_blocks() {1439for inst_index in self.block_ranges.get(block_index) {1440output_row(Row::Inst(block_index, inst_index), Mode::Measure);1441}1442}1443output_row(Row::Head, Mode::Measure);14441445output_row(Row::Head, Mode::Emit);1446output_row(Row::Line, Mode::Emit);1447for block_index in 0..self.num_blocks() {1448for inst_index in self.block_ranges.get(block_index) {1449output_row(Row::Inst(block_index, inst_index), Mode::Emit);1450}1451}1452}14531454/// Get the IR block for a BlockIndex, if one exists.1455pub fn bindex_to_bb(&self, block: BlockIndex) -> Option<ir::Block> {1456self.block_order.lowered_order()[block.index()].orig_block()1457}14581459/// Get the type of a VReg.1460pub fn vreg_type(&self, vreg: VReg) -> Type {1461self.vreg_types[vreg.vreg()]1462}14631464/// Get the fact, if any, for a given VReg.1465pub fn vreg_fact(&self, vreg: VReg) -> Option<&Fact> {1466self.facts[vreg.vreg()].as_ref()1467}14681469/// Set the fact for a given VReg.1470pub fn set_vreg_fact(&mut self, vreg: VReg, fact: Fact) {1471trace!("set fact on {}: {:?}", vreg, fact);1472self.facts[vreg.vreg()] = Some(fact);1473}14741475/// Does a given instruction define any facts?1476pub fn inst_defines_facts(&self, inst: InsnIndex) -> bool {1477self.inst_operands(inst)1478.iter()1479.filter(|o| o.kind() == OperandKind::Def)1480.map(|o| o.vreg())1481.any(|vreg| self.facts[vreg.vreg()].is_some())1482}14831484/// Get the user stack map associated with the given forward instruction index.1485pub fn get_user_stack_map(&self, inst: InsnIndex) -> Option<&ir::UserStackMap> {1486let index = inst.to_backwards_insn_index(self.num_insts());1487self.user_stack_maps.get(&index)1488}1489}14901491impl<I: VCodeInst> std::ops::Index<InsnIndex> for VCode<I> {1492type Output = I;1493fn index(&self, idx: InsnIndex) -> &Self::Output {1494&self.insts[idx.index()]1495}1496}14971498impl<I: VCodeInst> RegallocFunction for VCode<I> {1499fn num_insts(&self) -> usize {1500self.insts.len()1501}15021503fn num_blocks(&self) -> usize {1504self.block_ranges.len()1505}15061507fn entry_block(&self) -> BlockIndex {1508self.entry1509}15101511fn block_insns(&self, block: BlockIndex) -> InstRange {1512let range = self.block_ranges.get(block.index());1513InstRange::new(InsnIndex::new(range.start), InsnIndex::new(range.end))1514}15151516fn block_succs(&self, block: BlockIndex) -> &[BlockIndex] {1517let range = self.block_succ_range.get(block.index());1518&self.block_succs[range]1519}15201521fn block_preds(&self, block: BlockIndex) -> &[BlockIndex] {1522let range = self.block_pred_range.get(block.index());1523&self.block_preds[range]1524}15251526fn block_params(&self, block: BlockIndex) -> &[VReg] {1527// As a special case we don't return block params for the entry block, as all the arguments1528// will be defined by the `Inst::Args` instruction.1529if block == self.entry {1530return &[];1531}15321533let range = self.block_params_range.get(block.index());1534&self.block_params[range]1535}15361537fn branch_blockparams(&self, block: BlockIndex, _insn: InsnIndex, succ_idx: usize) -> &[VReg] {1538let succ_range = self.branch_block_arg_succ_range.get(block.index());1539debug_assert!(succ_idx < succ_range.len());1540let branch_block_args = self.branch_block_arg_range.get(succ_range.start + succ_idx);1541&self.branch_block_args[branch_block_args]1542}15431544fn is_ret(&self, insn: InsnIndex) -> bool {1545match self.insts[insn.index()].is_term() {1546// We treat blocks terminated by an unconditional trap like a return for regalloc.1547MachTerminator::None => self.insts[insn.index()].is_trap(),1548MachTerminator::Ret | MachTerminator::RetCall => true,1549MachTerminator::Branch => false,1550}1551}15521553fn is_branch(&self, insn: InsnIndex) -> bool {1554match self.insts[insn.index()].is_term() {1555MachTerminator::Branch => true,1556_ => false,1557}1558}15591560fn inst_operands(&self, insn: InsnIndex) -> &[Operand] {1561let range = self.operand_ranges.get(insn.index());1562&self.operands[range]1563}15641565fn inst_clobbers(&self, insn: InsnIndex) -> PRegSet {1566self.clobbers.get(&insn).cloned().unwrap_or_default()1567}15681569fn num_vregs(&self) -> usize {1570self.vreg_types.len()1571}15721573fn debug_value_labels(&self) -> &[(VReg, InsnIndex, InsnIndex, u32)] {1574&self.debug_value_labels1575}15761577fn spillslot_size(&self, regclass: RegClass) -> usize {1578self.abi.get_spillslot_size(regclass) as usize1579}15801581fn allow_multiple_vreg_defs(&self) -> bool {1582// At least the s390x backend requires this, because the1583// `Loop` pseudo-instruction aggregates all Operands so pinned1584// vregs (RealRegs) may occur more than once.1585true1586}1587}15881589impl<I: VCodeInst> Debug for VRegAllocator<I> {1590fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {1591writeln!(f, "VRegAllocator {{")?;15921593let mut alias_keys = self.vreg_aliases.keys().cloned().collect::<Vec<_>>();1594alias_keys.sort_unstable();1595for key in alias_keys {1596let dest = self.vreg_aliases.get(&key).unwrap();1597writeln!(f, " {:?} := {:?}", Reg::from(key), Reg::from(*dest))?;1598}15991600for (vreg, fact) in self.facts.iter().enumerate() {1601if let Some(fact) = fact {1602writeln!(f, " v{vreg} ! {fact}")?;1603}1604}16051606writeln!(f, "}}")1607}1608}16091610impl<I: VCodeInst> fmt::Debug for VCode<I> {1611fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {1612writeln!(f, "VCode {{")?;1613writeln!(f, " Entry block: {}", self.entry.index())?;16141615let mut state = Default::default();16161617for block in 0..self.num_blocks() {1618let block = BlockIndex::new(block);1619writeln!(1620f,1621"Block {}({:?}):",1622block.index(),1623self.block_params(block)1624)?;1625if let Some(bb) = self.bindex_to_bb(block) {1626writeln!(f, " (original IR block: {bb})")?;1627}1628for (succ_idx, succ) in self.block_succs(block).iter().enumerate() {1629writeln!(1630f,1631" (successor: Block {}({:?}))",1632succ.index(),1633self.branch_blockparams(block, InsnIndex::new(0) /* dummy */, succ_idx)1634)?;1635}1636for inst in self.block_ranges.get(block.index()) {1637writeln!(1638f,1639" Inst {}: {}",1640inst,1641self.insts[inst].pretty_print_inst(&mut state)1642)?;1643if !self.operands.is_empty() {1644for operand in self.inst_operands(InsnIndex::new(inst)) {1645if operand.kind() == OperandKind::Def {1646if let Some(fact) = &self.facts[operand.vreg().vreg()] {1647writeln!(f, " v{} ! {}", operand.vreg().vreg(), fact)?;1648}1649}1650}1651}1652if let Some(user_stack_map) = self.get_user_stack_map(InsnIndex::new(inst)) {1653writeln!(f, " {user_stack_map:?}")?;1654}1655}1656}16571658writeln!(f, "}}")?;1659Ok(())1660}1661}16621663/// This structure manages VReg allocation during the lifetime of the VCodeBuilder.1664pub struct VRegAllocator<I> {1665/// VReg IR-level types.1666vreg_types: Vec<Type>,16671668/// VReg aliases. When the final VCode is built we rewrite all1669/// uses of the keys in this table to their replacement values.1670///1671/// We use these aliases to rename an instruction's expected1672/// result vregs to the returned vregs from lowering, which are1673/// usually freshly-allocated temps.1674vreg_aliases: FxHashMap<regalloc2::VReg, regalloc2::VReg>,16751676/// A deferred error, to be bubbled up to the top level of the1677/// lowering algorithm. We take this approach because we cannot1678/// currently propagate a `Result` upward through ISLE code (the1679/// lowering rules) or some ABI code.1680deferred_error: Option<CodegenError>,16811682/// Facts on VRegs, for proof-carrying code.1683facts: Vec<Option<Fact>>,16841685/// The type of instruction that this allocator makes registers for.1686_inst: core::marker::PhantomData<I>,1687}16881689impl<I: VCodeInst> VRegAllocator<I> {1690/// Make a new VRegAllocator.1691pub fn with_capacity(capacity: usize) -> Self {1692let capacity = first_user_vreg_index() + capacity;1693let mut vreg_types = Vec::with_capacity(capacity);1694vreg_types.resize(first_user_vreg_index(), types::INVALID);1695Self {1696vreg_types,1697vreg_aliases: FxHashMap::with_capacity_and_hasher(capacity, Default::default()),1698deferred_error: None,1699facts: Vec::with_capacity(capacity),1700_inst: core::marker::PhantomData::default(),1701}1702}17031704/// Allocate a fresh ValueRegs.1705pub fn alloc(&mut self, ty: Type) -> CodegenResult<ValueRegs<Reg>> {1706if self.deferred_error.is_some() {1707return Err(CodegenError::CodeTooLarge);1708}1709let v = self.vreg_types.len();1710let (regclasses, tys) = I::rc_for_type(ty)?;1711if v + regclasses.len() >= VReg::MAX {1712return Err(CodegenError::CodeTooLarge);1713}17141715let regs: ValueRegs<Reg> = match regclasses {1716&[rc0] => ValueRegs::one(VReg::new(v, rc0).into()),1717&[rc0, rc1] => ValueRegs::two(VReg::new(v, rc0).into(), VReg::new(v + 1, rc1).into()),1718// We can extend this if/when we support 32-bit targets; e.g.,1719// an i128 on a 32-bit machine will need up to four machine regs1720// for a `Value`.1721_ => panic!("Value must reside in 1 or 2 registers"),1722};1723for (®_ty, ®) in tys.iter().zip(regs.regs().iter()) {1724let vreg = reg.to_virtual_reg().unwrap();1725debug_assert_eq!(self.vreg_types.len(), vreg.index());1726self.vreg_types.push(reg_ty);1727}17281729// Create empty facts for each allocated vreg.1730self.facts.resize(self.vreg_types.len(), None);17311732Ok(regs)1733}17341735/// Allocate a fresh ValueRegs, deferring any out-of-vregs1736/// errors. This is useful in places where we cannot bubble a1737/// `CodegenResult` upward easily, and which are known to be1738/// invoked from within the lowering loop that checks the deferred1739/// error status below.1740pub fn alloc_with_deferred_error(&mut self, ty: Type) -> ValueRegs<Reg> {1741match self.alloc(ty) {1742Ok(x) => x,1743Err(e) => {1744self.deferred_error = Some(e);1745self.bogus_for_deferred_error(ty)1746}1747}1748}17491750/// Take any deferred error that was accumulated by `alloc_with_deferred_error`.1751pub fn take_deferred_error(&mut self) -> Option<CodegenError> {1752self.deferred_error.take()1753}17541755/// Produce an bogus VReg placeholder with the proper number of1756/// registers for the given type. This is meant to be used with1757/// deferred allocation errors (see `Lower::alloc_tmp()`).1758fn bogus_for_deferred_error(&self, ty: Type) -> ValueRegs<Reg> {1759let (regclasses, _tys) = I::rc_for_type(ty).expect("must have valid type");1760match regclasses {1761&[rc0] => ValueRegs::one(VReg::new(0, rc0).into()),1762&[rc0, rc1] => ValueRegs::two(VReg::new(0, rc0).into(), VReg::new(1, rc1).into()),1763_ => panic!("Value must reside in 1 or 2 registers"),1764}1765}17661767/// Rewrite any mention of `from` into `to`.1768pub fn set_vreg_alias(&mut self, from: Reg, to: Reg) {1769let from = from.into();1770let resolved_to = self.resolve_vreg_alias(to.into());1771// Disallow cycles (see below).1772assert_ne!(resolved_to, from);17731774// Maintain the invariant that PCC facts only exist on vregs1775// which aren't aliases. We want to preserve whatever was1776// stated about the vreg before its producer was lowered.1777if let Some(fact) = self.facts[from.vreg()].take() {1778self.set_fact(resolved_to, fact);1779}17801781let old_alias = self.vreg_aliases.insert(from, resolved_to);1782debug_assert_eq!(old_alias, None);1783}17841785fn resolve_vreg_alias(&self, mut vreg: regalloc2::VReg) -> regalloc2::VReg {1786// We prevent cycles from existing by resolving targets of1787// aliases eagerly before setting them. If the target resolves1788// to the origin of the alias, then a cycle would be created1789// and the alias is disallowed. Because of the structure of1790// SSA code (one instruction can refer to another's defs but1791// not vice-versa, except indirectly through1792// phis/blockparams), cycles should not occur as we use1793// aliases to redirect vregs to the temps that actually define1794// them.1795while let Some(to) = self.vreg_aliases.get(&vreg) {1796vreg = *to;1797}1798vreg1799}18001801#[inline]1802fn debug_assert_no_vreg_aliases(&self, mut list: impl Iterator<Item = VReg>) {1803debug_assert!(list.all(|vreg| !self.vreg_aliases.contains_key(&vreg)));1804}18051806/// Set the proof-carrying code fact on a given virtual register.1807///1808/// Returns the old fact, if any (only one fact can be stored).1809fn set_fact(&mut self, vreg: regalloc2::VReg, fact: Fact) -> Option<Fact> {1810trace!("vreg {:?} has fact: {:?}", vreg, fact);1811debug_assert!(!self.vreg_aliases.contains_key(&vreg));1812self.facts[vreg.vreg()].replace(fact)1813}18141815/// Set a fact only if one doesn't already exist.1816pub fn set_fact_if_missing(&mut self, vreg: VirtualReg, fact: Fact) {1817let vreg = self.resolve_vreg_alias(vreg.into());1818if self.facts[vreg.vreg()].is_none() {1819self.set_fact(vreg, fact);1820}1821}18221823/// Allocate a fresh ValueRegs, with a given fact to apply if1824/// the value fits in one VReg.1825pub fn alloc_with_maybe_fact(1826&mut self,1827ty: Type,1828fact: Option<Fact>,1829) -> CodegenResult<ValueRegs<Reg>> {1830let result = self.alloc(ty)?;18311832// Ensure that we don't lose a fact on a value that splits1833// into multiple VRegs.1834assert!(result.len() == 1 || fact.is_none());1835if let Some(fact) = fact {1836self.set_fact(result.regs()[0].into(), fact);1837}18381839Ok(result)1840}1841}18421843/// This structure tracks the large constants used in VCode that will be emitted separately by the1844/// [MachBuffer].1845///1846/// First, during the lowering phase, constants are inserted using1847/// [VCodeConstants.insert]; an intermediate handle, `VCodeConstant`, tracks what constants are1848/// used in this phase. Some deduplication is performed, when possible, as constant1849/// values are inserted.1850///1851/// Secondly, during the emission phase, the [MachBuffer] assigns [MachLabel]s for each of the1852/// constants so that instructions can refer to the value's memory location. The [MachBuffer]1853/// then writes the constant values to the buffer.1854#[derive(Default)]1855pub struct VCodeConstants {1856constants: PrimaryMap<VCodeConstant, VCodeConstantData>,1857pool_uses: HashMap<Constant, VCodeConstant>,1858well_known_uses: HashMap<*const [u8], VCodeConstant>,1859u64s: HashMap<[u8; 8], VCodeConstant>,1860}1861impl VCodeConstants {1862/// Initialize the structure with the expected number of constants.1863pub fn with_capacity(expected_num_constants: usize) -> Self {1864Self {1865constants: PrimaryMap::with_capacity(expected_num_constants),1866pool_uses: HashMap::with_capacity(expected_num_constants),1867well_known_uses: HashMap::new(),1868u64s: HashMap::new(),1869}1870}18711872/// Insert a constant; using this method indicates that a constant value will be used and thus1873/// will be emitted to the `MachBuffer`. The current implementation can deduplicate constants1874/// that are [VCodeConstantData::Pool] or [VCodeConstantData::WellKnown] but not1875/// [VCodeConstantData::Generated].1876pub fn insert(&mut self, data: VCodeConstantData) -> VCodeConstant {1877match data {1878VCodeConstantData::Generated(_) => self.constants.push(data),1879VCodeConstantData::Pool(constant, _) => match self.pool_uses.get(&constant) {1880None => {1881let vcode_constant = self.constants.push(data);1882self.pool_uses.insert(constant, vcode_constant);1883vcode_constant1884}1885Some(&vcode_constant) => vcode_constant,1886},1887VCodeConstantData::WellKnown(data_ref) => {1888match self.well_known_uses.entry(data_ref as *const [u8]) {1889Entry::Vacant(v) => {1890let vcode_constant = self.constants.push(data);1891v.insert(vcode_constant);1892vcode_constant1893}1894Entry::Occupied(o) => *o.get(),1895}1896}1897VCodeConstantData::U64(value) => match self.u64s.entry(value) {1898Entry::Vacant(v) => {1899let vcode_constant = self.constants.push(data);1900v.insert(vcode_constant);1901vcode_constant1902}1903Entry::Occupied(o) => *o.get(),1904},1905}1906}19071908/// Return the number of constants inserted.1909pub fn len(&self) -> usize {1910self.constants.len()1911}19121913/// Iterate over the `VCodeConstant` keys inserted in this structure.1914pub fn keys(&self) -> Keys<VCodeConstant> {1915self.constants.keys()1916}19171918/// Iterate over the `VCodeConstant` keys and the data (as a byte slice) inserted in this1919/// structure.1920pub fn iter(&self) -> impl Iterator<Item = (VCodeConstant, &VCodeConstantData)> {1921self.constants.iter()1922}19231924/// Returns the data associated with the specified constant.1925pub fn get(&self, c: VCodeConstant) -> &VCodeConstantData {1926&self.constants[c]1927}19281929/// Checks if the given [VCodeConstantData] is registered as1930/// used by the pool.1931pub fn pool_uses(&self, constant: &VCodeConstantData) -> bool {1932match constant {1933VCodeConstantData::Pool(c, _) => self.pool_uses.contains_key(c),1934_ => false,1935}1936}1937}19381939/// A use of a constant by one or more VCode instructions; see [VCodeConstants].1940#[derive(Clone, Copy, Debug, PartialEq, Eq)]1941pub struct VCodeConstant(u32);1942entity_impl!(VCodeConstant);19431944/// Identify the different types of constant that can be inserted into [VCodeConstants]. Tracking1945/// these separately instead of as raw byte buffers allows us to avoid some duplication.1946pub enum VCodeConstantData {1947/// A constant already present in the Cranelift IR1948/// [ConstantPool](crate::ir::constant::ConstantPool).1949Pool(Constant, ConstantData),1950/// A reference to a well-known constant value that is statically encoded within the compiler.1951WellKnown(&'static [u8]),1952/// A constant value generated during lowering; the value may depend on the instruction context1953/// which makes it difficult to de-duplicate--if possible, use other variants.1954Generated(ConstantData),1955/// A constant of at most 64 bits. These are deduplicated as1956/// well. Stored as a fixed-size array of `u8` so that we do not1957/// encounter endianness problems when cross-compiling.1958U64([u8; 8]),1959}1960impl VCodeConstantData {1961/// Retrieve the constant data as a byte slice.1962pub fn as_slice(&self) -> &[u8] {1963match self {1964VCodeConstantData::Pool(_, d) | VCodeConstantData::Generated(d) => d.as_slice(),1965VCodeConstantData::WellKnown(d) => d,1966VCodeConstantData::U64(value) => &value[..],1967}1968}19691970/// Calculate the alignment of the constant data.1971pub fn alignment(&self) -> u32 {1972if self.as_slice().len() <= 8 { 8 } else { 16 }1973}1974}19751976#[cfg(test)]1977mod test {1978use super::*;1979use std::mem::size_of;19801981#[test]1982fn size_of_constant_structs() {1983assert_eq!(size_of::<Constant>(), 4);1984assert_eq!(size_of::<VCodeConstant>(), 4);1985assert_eq!(size_of::<ConstantData>(), 3 * size_of::<usize>());1986assert_eq!(size_of::<VCodeConstantData>(), 4 * size_of::<usize>());1987assert_eq!(1988size_of::<PrimaryMap<VCodeConstant, VCodeConstantData>>(),19893 * size_of::<usize>()1990);1991// TODO The VCodeConstants structure's memory size could be further optimized.1992// With certain versions of Rust, each `HashMap` in `VCodeConstants` occupied at1993// least 48 bytes, making an empty `VCodeConstants` cost 120 bytes.1994}1995}199619971998