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%
still, a port of IonMonkey's backtracking register allocator to
Rust. In many regards, it has been generalized, optimized, and
improved since the initial port, and now supports both SSA and non-SSA
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.)
improved since the initial port.
In addition, it contains substantial amounts of testing infrastructure
(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) {
v(op.vreg());
}
if let Some((src, dst)) = f.is_move(inst) {
v(src.vreg());
v(dst.vreg());
}
if f.is_branch(inst) {
for succ_idx in 0..f.block_succs(block).len() {
for &param in f.branch_blockparams(block, inst, succ_idx) {
@@ -565,25 +561,6 @@ impl CheckerState {
// according to the move semantics in the step
// 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(())
}
@@ -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
/// reftyped values. All other reftyped values become invalid.
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)]
@@ -903,35 +854,10 @@ impl<'a, F: Function> Checker<'a, F> {
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
// not exist in post-regalloc code, and the edge-moves have to
// 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 allocs: Vec<_> = out.inst_allocs(inst).iter().cloned().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(", "));
}
&CheckerInst::ProgramMove { inst, src, dst } => {
trace!(" inst{}: prog_move {} -> {}", inst.index(), src, dst);
}
&CheckerInst::ParallelMove { .. } => {
panic!("unexpected parallel_move in body (non-edge)")
}

View File

@@ -124,10 +124,6 @@ impl Function for Func {
&self.debug_value_labels[..]
}
fn is_move(&self, _: Inst) -> Option<(Operand, Operand)> {
None
}
fn inst_operands(&self, insn: Inst) -> &[Operand] {
&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 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
// single VReg at a single program point (this can happen for,
// 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)]
pub enum InsertMovePrio {
InEdgeMoves,
BlockParam,
Regular,
PostRegular,
MultiFixedRegInitial,
MultiFixedRegSecondary,
ReusedInput,
@@ -660,10 +643,6 @@ pub struct Stats {
pub livein_iterations: usize,
pub initial_liverange_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_reg_probes_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() {
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 op in self.func.inst_operands(inst) {
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
// are any fixed-reg-constrained defs at Late 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_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.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);
}
// 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");
}

View File

@@ -71,10 +71,6 @@ impl<'a, F: Function> Env<'a, F> {
extra_spillslots_by_class: [smallvec![], smallvec![]],
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![],
inserted_moves: vec![],
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_out_idx = 0;
let mut prog_move_src_idx = 0;
let mut prog_move_dst_idx = 0;
for vreg in 0..self.vregs.len() {
let vreg = VRegIndex::new(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
// half-moves. We also scan over `blockparam_ins` and
// `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();
for range_idx in 0..self.vregs[vreg.index()].ranges.len() {
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;
}
}
@@ -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
// invariant to the client.
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
/// class.
///
/// A virtual register ("vreg") corresponds to an SSA value for SSA
/// input, or just a register when we allow for non-SSA input. All
/// A virtual register ("vreg") corresponds to an SSA value. All
/// dataflow in the input program is specified via flow through a
/// virtual register; even uses of specially-constrained locations,
/// such as fixed physical registers, are done by using vregs, because
@@ -1029,10 +1028,6 @@ pub trait Function {
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
// --------------------------