Remove support for program moves (#118)

This commit is contained in:
Amanieu d'Antras
2023-03-05 01:38:05 +01:00
committed by GitHub
parent 54f074e507
commit 7354cfedde
9 changed files with 4 additions and 415 deletions

View File

@@ -3,10 +3,7 @@
This is a register allocator that started life as, and is about 50% This is a register allocator that started life as, and is about 50%
still, a port of IonMonkey's backtracking register allocator to still, a port of IonMonkey's backtracking register allocator to
Rust. In many regards, it has been generalized, optimized, and Rust. In many regards, it has been generalized, optimized, and
improved since the initial port, and now supports both SSA and non-SSA improved since the initial port.
use-cases. (However, non-SSA should be considered deprecated; we want to
move to SSA-only in the future, to enable some performance improvements.
See #4.)
In addition, it contains substantial amounts of testing infrastructure In addition, it contains substantial amounts of testing infrastructure
(fuzzing harnesses and checkers) that does not exist in the original (fuzzing harnesses and checkers) that does not exist in the original

View File

@@ -269,10 +269,6 @@ fn visit_all_vregs<F: Function, V: FnMut(VReg)>(f: &F, mut v: V) {
for op in f.inst_operands(inst) { for op in f.inst_operands(inst) {
v(op.vreg()); v(op.vreg());
} }
if let Some((src, dst)) = f.is_move(inst) {
v(src.vreg());
v(dst.vreg());
}
if f.is_branch(inst) { if f.is_branch(inst) {
for succ_idx in 0..f.block_succs(block).len() { for succ_idx in 0..f.block_succs(block).len() {
for &param in f.branch_blockparams(block, inst, succ_idx) { for &param in f.branch_blockparams(block, inst, succ_idx) {
@@ -565,25 +561,6 @@ impl CheckerState {
// according to the move semantics in the step // according to the move semantics in the step
// function below. // function below.
} }
&CheckerInst::ProgramMove { inst, src, dst: _ } => {
// Validate that the fixed-reg constraint, if any, on
// `src` is satisfied.
if let OperandConstraint::FixedReg(preg) = src.constraint() {
let alloc = Allocation::reg(preg);
let val = self.get_value(&alloc).unwrap_or(&default_val);
trace!(
"checker: checkinst {:?}: cheker value in {:?} is {:?}",
checkinst,
alloc,
val
);
self.check_val(inst, src, alloc, val, &[alloc], checker)?;
}
// Note that we don't do anything with `dst`
// here. That is implicitly checked whenever `dst` is
// used; the `update()` step below adds the symbolic
// vreg for `dst` into wherever `src` may be stored.
}
} }
Ok(()) Ok(())
} }
@@ -686,15 +663,6 @@ impl CheckerState {
} }
} }
} }
&CheckerInst::ProgramMove { inst: _, src, dst } => {
// Remove all earlier instances of `dst`: this vreg is
// now stale (it is being overwritten).
self.remove_vreg(dst.vreg());
// Define `dst` wherever `src` occurs.
for (_, value) in self.get_mappings_mut() {
value.copy_vreg(src.vreg(), dst.vreg());
}
}
} }
} }
@@ -786,23 +754,6 @@ pub(crate) enum CheckerInst {
/// A safepoint, with the given Allocations specified as containing /// A safepoint, with the given Allocations specified as containing
/// reftyped values. All other reftyped values become invalid. /// reftyped values. All other reftyped values become invalid.
Safepoint { inst: Inst, allocs: Vec<Allocation> }, Safepoint { inst: Inst, allocs: Vec<Allocation> },
/// An op with one source operand, and one dest operand, that
/// copies any symbolic values from the source to the dest, in
/// addition to adding the symbolic value of the dest vreg to the
/// set. This "program move" is distinguished from the above
/// `Move` by being semantically relevant in the original
/// (pre-regalloc) program.
///
/// We transform checker values as follows: for any vreg-set that
/// contains `dst`'s vreg, we first delete that vreg (because it
/// is being redefined). Then, for any vreg-set with `src`
/// present, we add `dst`.
ProgramMove {
inst: Inst,
src: Operand,
dst: Operand,
},
} }
#[derive(Debug)] #[derive(Debug)]
@@ -903,35 +854,10 @@ impl<'a, F: Function> Checker<'a, F> {
self.bb_insts.get_mut(&block).unwrap().push(checkinst); self.bb_insts.get_mut(&block).unwrap().push(checkinst);
} }
// If this is a move, handle specially. Note that the
// regalloc2-inserted moves are not semantically present in
// the original program and so do not modify the sets of
// symbolic values at all, but rather just move them around;
// but "program moves" *are* present, and have the following
// semantics: they define the destination vreg, but also
// retain any symbolic values in the source.
//
// regalloc2 reifies all moves into edits in its unified
// move/edit framework, so we don't get allocs for these moves
// in the post-regalloc output, and the embedder is not
// supposed to emit the moves. But we *do* want to check the
// semantic implications, namely definition of new vregs. So
// we emit `ProgramMove` ops that do just this.
if let Some((src, dst)) = self.f.is_move(inst) {
let src_op = Operand::any_use(src.vreg());
let dst_op = Operand::any_def(dst.vreg());
let checkinst = CheckerInst::ProgramMove {
inst,
src: src_op,
dst: dst_op,
};
trace!("checker: adding inst {:?}", checkinst);
self.bb_insts.get_mut(&block).unwrap().push(checkinst);
}
// Skip normal checks if this is a branch: the blockparams do // Skip normal checks if this is a branch: the blockparams do
// not exist in post-regalloc code, and the edge-moves have to // not exist in post-regalloc code, and the edge-moves have to
// be inserted before the branch rather than after. // be inserted before the branch rather than after.
else if !self.f.is_branch(inst) { if !self.f.is_branch(inst) {
let operands: Vec<_> = self.f.inst_operands(inst).iter().cloned().collect(); let operands: Vec<_> = self.f.inst_operands(inst).iter().cloned().collect();
let allocs: Vec<_> = out.inst_allocs(inst).iter().cloned().collect(); let allocs: Vec<_> = out.inst_allocs(inst).iter().cloned().collect();
let clobbers: Vec<_> = self.f.inst_clobbers(inst).into_iter().collect(); let clobbers: Vec<_> = self.f.inst_clobbers(inst).into_iter().collect();
@@ -1128,9 +1054,6 @@ impl<'a, F: Function> Checker<'a, F> {
} }
trace!(" safepoint: {}", slotargs.join(", ")); trace!(" safepoint: {}", slotargs.join(", "));
} }
&CheckerInst::ProgramMove { inst, src, dst } => {
trace!(" inst{}: prog_move {} -> {}", inst.index(), src, dst);
}
&CheckerInst::ParallelMove { .. } => { &CheckerInst::ParallelMove { .. } => {
panic!("unexpected parallel_move in body (non-edge)") panic!("unexpected parallel_move in body (non-edge)")
} }

View File

@@ -124,10 +124,6 @@ impl Function for Func {
&self.debug_value_labels[..] &self.debug_value_labels[..]
} }
fn is_move(&self, _: Inst) -> Option<(Operand, Operand)> {
None
}
fn inst_operands(&self, insn: Inst) -> &[Operand] { fn inst_operands(&self, insn: Inst) -> &[Operand] {
&self.insts[insn.index()].operands[..] &self.insts[insn.index()].operands[..]
} }

View File

@@ -405,21 +405,6 @@ pub struct Env<'a, F: Function> {
pub extra_spillslots_by_class: [SmallVec<[Allocation; 2]>; 2], pub extra_spillslots_by_class: [SmallVec<[Allocation; 2]>; 2],
pub preferred_victim_by_class: [PReg; 2], pub preferred_victim_by_class: [PReg; 2],
// Program moves: these are moves in the provided program that we
// handle with our internal machinery, in order to avoid the
// overhead of ordinary operand processing. We expect the client
// to not generate any code for instructions that return
// `Some(..)` for `.is_move()`, and instead use the edits that we
// provide to implement those moves (or some simplified version of
// them) post-regalloc.
//
// (from-vreg, inst, from-alloc), sorted by (from-vreg, inst)
pub prog_move_srcs: Vec<((VRegIndex, Inst), Allocation)>,
// (to-vreg, inst, to-alloc), sorted by (to-vreg, inst)
pub prog_move_dsts: Vec<((VRegIndex, Inst), Allocation)>,
// (from-vreg, to-vreg) for bundle-merging.
pub prog_move_merges: Vec<(LiveRangeIndex, LiveRangeIndex)>,
// When multiple fixed-register constraints are present on a // When multiple fixed-register constraints are present on a
// single VReg at a single program point (this can happen for, // single VReg at a single program point (this can happen for,
// e.g., call args that use the same value multiple times), we // e.g., call args that use the same value multiple times), we
@@ -628,9 +613,7 @@ pub struct InsertedMove {
#[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord)] #[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord)]
pub enum InsertMovePrio { pub enum InsertMovePrio {
InEdgeMoves, InEdgeMoves,
BlockParam,
Regular, Regular,
PostRegular,
MultiFixedRegInitial, MultiFixedRegInitial,
MultiFixedRegSecondary, MultiFixedRegSecondary,
ReusedInput, ReusedInput,
@@ -660,10 +643,6 @@ pub struct Stats {
pub livein_iterations: usize, pub livein_iterations: usize,
pub initial_liverange_count: usize, pub initial_liverange_count: usize,
pub merged_bundle_count: usize, pub merged_bundle_count: usize,
pub prog_moves: usize,
pub prog_moves_dead_src: usize,
pub prog_move_merge_attempt: usize,
pub prog_move_merge_success: usize,
pub process_bundle_count: usize, pub process_bundle_count: usize,
pub process_bundle_reg_probes_fixed: usize, pub process_bundle_reg_probes_fixed: usize,
pub process_bundle_reg_success_fixed: usize, pub process_bundle_reg_success_fixed: usize,

View File

@@ -356,13 +356,6 @@ impl<'a, F: Function> Env<'a, F> {
} }
for inst in insns.rev().iter() { for inst in insns.rev().iter() {
if let Some((src, dst)) = self.func.is_move(inst) {
live.set(dst.vreg().vreg(), false);
live.set(src.vreg().vreg(), true);
self.observe_vreg_class(src.vreg());
self.observe_vreg_class(dst.vreg());
}
for pos in &[OperandPos::Late, OperandPos::Early] { for pos in &[OperandPos::Late, OperandPos::Early] {
for op in self.func.inst_operands(inst) { for op in self.func.inst_operands(inst) {
if op.as_fixed_nonallocatable().is_some() { if op.as_fixed_nonallocatable().is_some() {
@@ -519,148 +512,6 @@ impl<'a, F: Function> Env<'a, F> {
} }
} }
// If this is a move, handle specially.
if let Some((src, dst)) = self.func.is_move(inst) {
assert!(
src.vreg() != dst.vreg(),
"Invalid move: overwriting an SSA value"
);
trace!(" -> move inst{}: src {} -> dst {}", inst.index(), src, dst);
debug_assert_eq!(src.class(), dst.class());
debug_assert_eq!(src.kind(), OperandKind::Use);
debug_assert_eq!(src.pos(), OperandPos::Early);
debug_assert_eq!(dst.kind(), OperandKind::Def);
debug_assert_eq!(dst.pos(), OperandPos::Late);
// Redefine src and dst operands to have
// positions of After and Before respectively
// (see note below), and to have Any
// constraints if they were originally Reg.
let src_constraint = match src.constraint() {
OperandConstraint::Reg => OperandConstraint::Any,
x => x,
};
let dst_constraint = match dst.constraint() {
OperandConstraint::Reg => OperandConstraint::Any,
x => x,
};
let src = Operand::new(
src.vreg(),
src_constraint,
OperandKind::Use,
OperandPos::Late,
);
let dst = Operand::new(
dst.vreg(),
dst_constraint,
OperandKind::Def,
OperandPos::Early,
);
if self.annotations_enabled {
self.annotate(
ProgPoint::after(inst),
format!(
" prog-move v{} ({:?}) -> v{} ({:?})",
src.vreg().vreg(),
src_constraint,
dst.vreg().vreg(),
dst_constraint,
),
);
}
// N.B.: in order to integrate with the move
// resolution that joins LRs in general, we
// conceptually treat the move as happening
// between the move inst's After and the next
// inst's Before. Thus the src LR goes up to
// (exclusive) next-inst-pre, and the dst LR
// starts at next-inst-pre. We have to take
// care in our move insertion to handle this
// like other inter-inst moves, i.e., at
// `Regular` priority, so it properly happens
// in parallel with other inter-LR moves.
//
// Why the progpoint between move and next
// inst, and not the progpoint between prev
// inst and move? Because a move can be the
// first inst in a block, but cannot be the
// last; so the following progpoint is always
// within the same block, while the previous
// one may be an inter-block point (and the
// After of the prev inst in a different
// block).
// Handle the def w.r.t. liveranges: trim the
// start of the range and mark it dead at this
// point in our backward scan.
let pos = ProgPoint::before(inst.next());
let mut dst_lr = vreg_ranges[dst.vreg().vreg()];
if !live.get(dst.vreg().vreg()) {
let from = pos;
let to = pos.next();
dst_lr = self.add_liverange_to_vreg(
VRegIndex::new(dst.vreg().vreg()),
CodeRange { from, to },
);
trace!(" -> invalid LR for def; created {:?}", dst_lr);
}
trace!(" -> has existing LR {:?}", dst_lr);
// Trim the LR to start here.
if self.ranges[dst_lr.index()].range.from
== self.cfginfo.block_entry[block.index()]
{
trace!(" -> started at block start; trimming to {:?}", pos);
self.ranges[dst_lr.index()].range.from = pos;
}
self.ranges[dst_lr.index()].set_flag(LiveRangeFlag::StartsAtDef);
live.set(dst.vreg().vreg(), false);
vreg_ranges[dst.vreg().vreg()] = LiveRangeIndex::invalid();
// Handle the use w.r.t. liveranges: make it live
// and create an initial LR back to the start of
// the block.
let pos = ProgPoint::after(inst);
let src_lr = if !live.get(src.vreg().vreg()) {
let range = CodeRange {
from: self.cfginfo.block_entry[block.index()],
to: pos.next(),
};
let src_lr =
self.add_liverange_to_vreg(VRegIndex::new(src.vreg().vreg()), range);
vreg_ranges[src.vreg().vreg()] = src_lr;
src_lr
} else {
vreg_ranges[src.vreg().vreg()]
};
trace!(" -> src LR {:?}", src_lr);
// Add to live-set.
let src_is_dead_after_move = !live.get(src.vreg().vreg());
live.set(src.vreg().vreg(), true);
// Add to program-moves lists.
self.prog_move_srcs.push((
(VRegIndex::new(src.vreg().vreg()), inst),
Allocation::none(),
));
self.prog_move_dsts.push((
(VRegIndex::new(dst.vreg().vreg()), inst.next()),
Allocation::none(),
));
self.stats.prog_moves += 1;
if src_is_dead_after_move {
self.stats.prog_moves_dead_src += 1;
self.prog_move_merges.push((src_lr, dst_lr));
}
continue;
}
// Preprocess defs and uses. Specifically, if there // Preprocess defs and uses. Specifically, if there
// are any fixed-reg-constrained defs at Late position // are any fixed-reg-constrained defs at Late position
// and fixed-reg-constrained uses at Early position // and fixed-reg-constrained uses at Early position
@@ -1016,11 +867,6 @@ impl<'a, F: Function> Env<'a, F> {
self.blockparam_ins.sort_unstable_by_key(|x| x.key()); self.blockparam_ins.sort_unstable_by_key(|x| x.key());
self.blockparam_outs.sort_unstable_by_key(|x| x.key()); self.blockparam_outs.sort_unstable_by_key(|x| x.key());
self.prog_move_srcs.sort_unstable_by_key(|(pos, _)| *pos);
self.prog_move_dsts.sort_unstable_by_key(|(pos, _)| *pos);
trace!("prog_move_srcs = {:?}", self.prog_move_srcs);
trace!("prog_move_dsts = {:?}", self.prog_move_dsts);
self.stats.initial_liverange_count = self.ranges.len(); self.stats.initial_liverange_count = self.ranges.len();
self.stats.blockparam_ins_count = self.blockparam_ins.len(); self.stats.blockparam_ins_count = self.blockparam_ins.len();

View File

@@ -351,28 +351,6 @@ impl<'a, F: Function> Env<'a, F> {
self.merge_bundles(from_bundle, to_bundle); self.merge_bundles(from_bundle, to_bundle);
} }
// Attempt to merge move srcs/dsts.
for i in 0..self.prog_move_merges.len() {
let (src, dst) = self.prog_move_merges[i];
trace!("trying to merge move src LR {:?} to dst LR {:?}", src, dst);
let src = self.resolve_merged_lr(src);
let dst = self.resolve_merged_lr(dst);
trace!(
"resolved LR-construction merging chains: move-merge is now src LR {:?} to dst LR {:?}",
src,
dst
);
let src_bundle = self.ranges[src.index()].bundle;
debug_assert!(src_bundle.is_valid());
let dest_bundle = self.ranges[dst.index()].bundle;
debug_assert!(dest_bundle.is_valid());
self.stats.prog_move_merge_attempt += 1;
if self.merge_bundles(/* from */ dest_bundle, /* to */ src_bundle) {
self.stats.prog_move_merge_success += 1;
}
}
trace!("done merging bundles"); trace!("done merging bundles");
} }

View File

@@ -71,10 +71,6 @@ impl<'a, F: Function> Env<'a, F> {
extra_spillslots_by_class: [smallvec![], smallvec![]], extra_spillslots_by_class: [smallvec![], smallvec![]],
preferred_victim_by_class: [PReg::invalid(), PReg::invalid()], preferred_victim_by_class: [PReg::invalid(), PReg::invalid()],
prog_move_srcs: Vec::with_capacity(n / 2),
prog_move_dsts: Vec::with_capacity(n / 2),
prog_move_merges: Vec::with_capacity(n / 2),
multi_fixed_reg_fixups: vec![], multi_fixed_reg_fixups: vec![],
inserted_moves: vec![], inserted_moves: vec![],
edits: Vec::with_capacity(n), edits: Vec::with_capacity(n),

View File

@@ -179,8 +179,6 @@ impl<'a, F: Function> Env<'a, F> {
let mut blockparam_in_idx = 0; let mut blockparam_in_idx = 0;
let mut blockparam_out_idx = 0; let mut blockparam_out_idx = 0;
let mut prog_move_src_idx = 0;
let mut prog_move_dst_idx = 0;
for vreg in 0..self.vregs.len() { for vreg in 0..self.vregs.len() {
let vreg = VRegIndex::new(vreg); let vreg = VRegIndex::new(vreg);
if !self.is_vreg_used(vreg) { if !self.is_vreg_used(vreg) {
@@ -190,7 +188,7 @@ impl<'a, F: Function> Env<'a, F> {
// For each range in each vreg, insert moves or // For each range in each vreg, insert moves or
// half-moves. We also scan over `blockparam_ins` and // half-moves. We also scan over `blockparam_ins` and
// `blockparam_outs`, which are sorted by (block, vreg), // `blockparam_outs`, which are sorted by (block, vreg),
// and over program-move srcs/dsts to fill in allocations. // to fill in allocations.
let mut prev = LiveRangeIndex::invalid(); let mut prev = LiveRangeIndex::invalid();
for range_idx in 0..self.vregs[vreg.index()].ranges.len() { for range_idx in 0..self.vregs[vreg.index()].ranges.len() {
let entry = self.vregs[vreg.index()].ranges[range_idx]; let entry = self.vregs[vreg.index()].ranges[range_idx];
@@ -524,89 +522,6 @@ impl<'a, F: Function> Env<'a, F> {
} }
} }
// Scan over program move srcs/dsts to fill in allocations.
// Move srcs happen at `After` of a given
// inst. Compute [from, to) semi-inclusive range of
// inst indices for which we should fill in the source
// with this LR's allocation.
//
// range from inst-Before or inst-After covers cur
// inst's After; so includes move srcs from inst.
let move_src_start = (vreg, range.from.inst());
// range to (exclusive) inst-Before or inst-After
// covers only prev inst's After; so includes move
// srcs to (exclusive) inst.
let move_src_end = (vreg, range.to.inst());
trace!(
"vreg {:?} range {:?}: looking for program-move sources from {:?} to {:?}",
vreg,
range,
move_src_start,
move_src_end
);
while prog_move_src_idx < self.prog_move_srcs.len()
&& self.prog_move_srcs[prog_move_src_idx].0 < move_src_start
{
trace!(" -> skipping idx {}", prog_move_src_idx);
prog_move_src_idx += 1;
}
while prog_move_src_idx < self.prog_move_srcs.len()
&& self.prog_move_srcs[prog_move_src_idx].0 < move_src_end
{
trace!(
" -> setting idx {} ({:?}) to alloc {:?}",
prog_move_src_idx,
self.prog_move_srcs[prog_move_src_idx].0,
alloc
);
self.prog_move_srcs[prog_move_src_idx].1 = alloc;
prog_move_src_idx += 1;
}
// move dsts happen at Before point.
//
// Range from inst-Before includes cur inst, while inst-After includes only next inst.
let move_dst_start = if range.from.pos() == InstPosition::Before {
(vreg, range.from.inst())
} else {
(vreg, range.from.inst().next())
};
// Range to (exclusive) inst-Before includes prev
// inst, so to (exclusive) cur inst; range to
// (exclusive) inst-After includes cur inst, so to
// (exclusive) next inst.
let move_dst_end = if range.to.pos() == InstPosition::Before {
(vreg, range.to.inst())
} else {
(vreg, range.to.inst().next())
};
trace!(
"vreg {:?} range {:?}: looking for program-move dests from {:?} to {:?}",
vreg,
range,
move_dst_start,
move_dst_end
);
while prog_move_dst_idx < self.prog_move_dsts.len()
&& self.prog_move_dsts[prog_move_dst_idx].0 < move_dst_start
{
trace!(" -> skipping idx {}", prog_move_dst_idx);
prog_move_dst_idx += 1;
}
while prog_move_dst_idx < self.prog_move_dsts.len()
&& self.prog_move_dsts[prog_move_dst_idx].0 < move_dst_end
{
trace!(
" -> setting idx {} ({:?}) to alloc {:?}",
prog_move_dst_idx,
self.prog_move_dsts[prog_move_dst_idx].0,
alloc
);
self.prog_move_dsts[prog_move_dst_idx].1 = alloc;
prog_move_dst_idx += 1;
}
prev = entry.index; prev = entry.index;
} }
} }
@@ -820,42 +735,6 @@ impl<'a, F: Function> Env<'a, F> {
} }
} }
// Sort the prog-moves lists and insert moves to reify the
// input program's move operations.
self.prog_move_srcs
.sort_unstable_by_key(|((_, inst), _)| *inst);
self.prog_move_dsts
.sort_unstable_by_key(|((_, inst), _)| inst.prev());
let prog_move_srcs = std::mem::replace(&mut self.prog_move_srcs, vec![]);
let prog_move_dsts = std::mem::replace(&mut self.prog_move_dsts, vec![]);
debug_assert_eq!(prog_move_srcs.len(), prog_move_dsts.len());
for (&((_, from_inst), from_alloc), &((to_vreg, to_inst), to_alloc)) in
prog_move_srcs.iter().zip(prog_move_dsts.iter())
{
trace!(
"program move at inst {:?}: alloc {:?} -> {:?} (v{})",
from_inst,
from_alloc,
to_alloc,
to_vreg.index(),
);
debug_assert!(from_alloc.is_some());
debug_assert!(to_alloc.is_some());
debug_assert_eq!(from_inst, to_inst.prev());
// N.B.: these moves happen with the *same* priority as
// LR-to-LR moves, because they work just like them: they
// connect a use at one progpoint (move-After) with a def
// at an adjacent progpoint (move+1-Before), so they must
// happen in parallel with all other LR-to-LR moves.
self.insert_move(
ProgPoint::before(to_inst),
InsertMovePrio::Regular,
from_alloc,
to_alloc,
self.vreg(to_vreg),
);
}
// Sort the debug-locations vector; we provide this // Sort the debug-locations vector; we provide this
// invariant to the client. // invariant to the client.
self.debug_locations.sort_unstable(); self.debug_locations.sort_unstable();

View File

@@ -266,8 +266,7 @@ impl From<&MachineEnv> for PRegSet {
/// A virtual register. Contains a virtual register number and a /// A virtual register. Contains a virtual register number and a
/// class. /// class.
/// ///
/// A virtual register ("vreg") corresponds to an SSA value for SSA /// A virtual register ("vreg") corresponds to an SSA value. All
/// input, or just a register when we allow for non-SSA input. All
/// dataflow in the input program is specified via flow through a /// dataflow in the input program is specified via flow through a
/// virtual register; even uses of specially-constrained locations, /// virtual register; even uses of specially-constrained locations,
/// such as fixed physical registers, are done by using vregs, because /// such as fixed physical registers, are done by using vregs, because
@@ -1029,10 +1028,6 @@ pub trait Function {
false false
} }
/// Determine whether an instruction is a move; if so, return the
/// Operands for (src, dst).
fn is_move(&self, insn: Inst) -> Option<(Operand, Operand)>;
// -------------------------- // --------------------------
// Instruction register slots // Instruction register slots
// -------------------------- // --------------------------