Files
regalloc2/src/ion/fast_alloc.rs
2023-04-16 14:43:18 +02:00

1221 lines
45 KiB
Rust

use alloc::collections::VecDeque;
use alloc::format;
use alloc::vec::Vec;
use alloc::{string::String, vec};
use smallvec::{smallvec, SmallVec};
use std::{convert::TryFrom, println};
use crate::indexset::IndexSet;
use crate::{
cfg::CFGInfo, Allocation, Block, Edit, Function, Inst, MachineEnv, Operand, OperandConstraint,
OperandKind, OperandPos, Output, PReg, PRegSet, ProgPoint, RegAllocError, RegClass, SpillSlot,
VReg,
};
use crate::{domtree, postorder, InstPosition};
use super::data_structures::u64_key;
use super::Stats;
#[derive(Default, Clone)]
struct VRegData {
pub preg: Option<PReg>,
pub slot_idx: Option<u32>,
pub def_block: Option<Block>,
pub reftype: bool,
// use information
pub cur_use_idx: u32,
pub uses: SmallVec<[u32; 8]>,
}
#[derive(Default, Clone, Copy)]
struct PRegData {
pub vreg: Option<u32>,
pub stack_pseudo: bool,
}
#[derive(Default, Clone, Copy)]
struct BlockData {
pub reg_allocated: bool,
pub params_allocated: bool,
}
struct ReadOnlyData {
pub postorder: Vec<Block>,
pub idom: Vec<Block>,
pub reg_order_int: Vec<PReg>,
pub reg_order_float: Vec<PReg>,
}
impl ReadOnlyData {
pub fn init<F: Function>(func: &F, mach_env: &MachineEnv) -> Self {
let reg_order_int = {
let class = RegClass::Int as usize;
let amount = mach_env.preferred_regs_by_class[class].len()
+ mach_env.non_preferred_regs_by_class[class].len();
let mut reg_order = Vec::with_capacity(amount);
reg_order.extend_from_slice(&mach_env.preferred_regs_by_class[class]);
reg_order.extend_from_slice(&mach_env.non_preferred_regs_by_class[class]);
reg_order
};
let reg_order_float = {
let class = RegClass::Float as usize;
let amount = mach_env.preferred_regs_by_class[class].len()
+ mach_env.non_preferred_regs_by_class[class].len();
let mut reg_order = Vec::with_capacity(amount);
reg_order.extend_from_slice(&mach_env.preferred_regs_by_class[class]);
reg_order.extend_from_slice(&mach_env.non_preferred_regs_by_class[class]);
reg_order
};
let postorder = postorder::calculate(func.num_blocks(), func.entry_block(), |b| {
func.block_succs(b)
});
let idom = domtree::calculate(
func.num_blocks(),
|b| func.block_preds(b),
&postorder,
func.entry_block(),
);
Self {
reg_order_int,
reg_order_float,
postorder,
idom,
}
}
pub fn reg_order(&self, class: RegClass) -> &[PReg] {
match class {
RegClass::Int => &self.reg_order_int,
RegClass::Float => &self.reg_order_float,
}
}
fn calc_preorder<F: Function>(func: &F) -> Vec<Block> {
let entry = func.entry_block();
let mut ret = vec![entry];
struct State<'a> {
block: Block,
succs: &'a [Block],
next_succ: usize,
}
let mut stack: SmallVec<[State; 64]> = smallvec![];
stack.push(State {
block: entry,
succs: func.block_succs(entry),
next_succ: 0,
});
while let Some(ref mut state) = stack.last_mut() {
if state.next_succ >= state.succs.len() {
stack.pop();
continue;
}
let block = state.succs[state.next_succ];
let succs = func.block_succs(block);
ret.push(block);
state.next_succ += 1;
if state.next_succ >= state.succs.len() {
stack.pop();
}
if !succs.is_empty() {
stack.push(State {
block,
succs: func.block_succs(block),
next_succ: 0,
});
}
}
ret
}
}
struct FastAllocState<'a, F: Function> {
pub vregs: Vec<VRegData>,
pub pregs: Vec<PRegData>,
pub blocks: Vec<BlockData>,
pub liveins: Vec<IndexSet>,
pub liveouts: Vec<IndexSet>,
pub cur_stack_slot_idx: u32,
pub reftype_vregs_in_pregs_count: u32,
pub stack_slot_count_int: u8,
pub stack_slot_count_float: u8,
pub allocs: Vec<Allocation>,
pub inst_alloc_offsets: Vec<u32>,
pub edits: Vec<(ProgPoint, Edit)>,
pub safepoint_slots: Vec<(ProgPoint, Allocation)>,
pub reftype_vregs: &'a [VReg],
pub func: &'a F,
pub mach_env: &'a MachineEnv,
}
impl<'a, F: Function> FastAllocState<'a, F> {
pub fn init(func: &'a F, mach_env: &'a MachineEnv) -> Self {
let reftype_vregs = func.reftype_vregs();
let vregs = {
let mut vregs = Vec::with_capacity(func.num_vregs());
vregs.resize(func.num_vregs(), VRegData::default());
for vreg in reftype_vregs {
vregs[vreg.vreg()].reftype = true;
}
vregs
};
let pregs = {
let mut pregs = Vec::with_capacity(PReg::NUM_INDEX);
pregs.resize(PReg::NUM_INDEX, PRegData::default());
for preg in &mach_env.fixed_stack_slots {
trace!("{} is a stack pseudo", preg);
pregs[preg.index()].stack_pseudo = true;
}
pregs
};
let blocks = {
let mut blocks = Vec::with_capacity(func.num_blocks());
blocks.resize(func.num_blocks(), BlockData::default());
blocks
};
let mut inst_alloc_offsets = Vec::with_capacity(func.num_insts());
inst_alloc_offsets.resize(func.num_insts(), 0);
// we need to create the alloc array beforehand because it needs to be sorted by inst index
// which we cannot guarantee when iterating through the blocks in reverse post-order
// TODO: this can be folded into any of the other iterations of the blocks by counting the operand counts for each instruction
// globally and writing the op count for each inst into inst_alloc_offsets[idx]
// then just iterate inst_alloc_offsets and adjust the indices
let allocs = {
let block_count = func.num_blocks();
let mut cur_idx = 0;
for i in 0..block_count {
for inst in func.block_insns(Block::new(i)).iter() {
inst_alloc_offsets[inst.index()] = cur_idx as u32;
cur_idx += func.inst_operands(inst).len();
}
}
let mut allocs = Vec::with_capacity(cur_idx);
allocs.resize(cur_idx, Allocation::none());
allocs
};
Self {
vregs,
pregs,
blocks,
liveins: Vec::new(),
liveouts: Vec::new(),
cur_stack_slot_idx: 0,
reftype_vregs_in_pregs_count: 0,
stack_slot_count_int: u8::try_from(func.spillslot_size(RegClass::Int))
.expect("that's a big integer"),
stack_slot_count_float: u8::try_from(func.spillslot_size(RegClass::Float))
.expect("that's a big float"),
allocs,
inst_alloc_offsets,
edits: Vec::new(),
safepoint_slots: Vec::new(),
reftype_vregs,
func,
mach_env,
}
}
pub fn get_or_alloc_stack_slot(&mut self, vreg: VReg) -> u32 {
if let Some(idx) = self.vregs[vreg.vreg()].slot_idx {
return idx;
}
self.alloc_stack_slot(vreg)
}
pub fn alloc_stack_slot(&mut self, vreg: VReg) -> u32 {
let data = &mut self.vregs[vreg.vreg()];
if data.slot_idx.is_some() {
panic!("Trying to allocate already allocated stack slot");
}
let size = if vreg.class() == RegClass::Int {
self.stack_slot_count_int
} else {
self.stack_slot_count_float
};
let idx = self.cur_stack_slot_idx;
trace!("Allocated slot {} for {}", idx, vreg);
self.cur_stack_slot_idx += size as u32;
data.slot_idx = Some(idx);
idx
}
pub fn create_stack_slot(&mut self, class: RegClass) -> u32 {
let size = if class == RegClass::Int {
self.stack_slot_count_int
} else {
self.stack_slot_count_float
};
let idx = self.cur_stack_slot_idx;
self.cur_stack_slot_idx += size as u32;
idx
}
pub fn move_to_preg(&mut self, vreg: VReg, preg: PReg, pos: ProgPoint) {
if let Some(vreg) = &self.pregs[preg.index()].vreg {
let vdata = &mut self.vregs[*vreg as usize];
debug_assert!(vdata.preg.is_some());
debug_assert_eq!(vdata.preg.unwrap(), preg);
vdata.preg = None;
}
if let Some(preg) = &self.vregs[vreg.vreg()].preg {
// TODO: allow multiple pregs for a single vreg?
let pdata = &mut self.pregs[preg.index()];
debug_assert!(pdata.vreg.is_some());
debug_assert_eq!(pdata.vreg.unwrap(), vreg.vreg() as u32);
pdata.vreg = None;
}
let vdata = &mut self.vregs[vreg.vreg()];
let pdata = &mut self.pregs[preg.index()];
if vdata.slot_idx.is_none() {
panic!("Trying to move from vreg that has no stack slot to preg");
}
self.edits.push((
pos,
Edit::Move {
from: Allocation::stack(SpillSlot::new(vdata.slot_idx.unwrap() as usize)),
to: Allocation::reg(preg),
},
));
vdata.preg = Some(preg);
pdata.vreg = Some(vreg.vreg() as u32);
if vdata.reftype {
self.reftype_vregs_in_pregs_count += 1;
}
}
pub fn move_to_stack(&mut self, preg: PReg, vreg: VReg, pos: ProgPoint) {
let vdata = &mut self.vregs[vreg.vreg()];
let pdata = &mut self.pregs[preg.index()];
if pdata.vreg.is_none() || vdata.preg.is_none() {
panic!("Trying to move from unallocated preg/vreg to stack");
}
debug_assert_eq!(vdata.preg.unwrap(), preg);
debug_assert_eq!(pdata.vreg.unwrap(), vreg.vreg() as u32);
if vdata.slot_idx.is_none() {
panic!("Trying to move to vreg without stack slot");
}
self.edits.push((
pos,
Edit::Move {
from: Allocation::reg(preg),
to: Allocation::stack(SpillSlot::new(vdata.slot_idx.unwrap() as usize)),
},
));
}
pub fn assign_preg(&mut self, preg: PReg, vreg: VReg) {
// TODO: somewhere assign_preg is called without making sure the vreg is clear (or inspite of it)
// need to make sure this is intended behavior
self.clear_preg(preg);
self.pregs[preg.index()].vreg = Some(vreg.vreg() as u32);
self.vregs[vreg.vreg()].preg = Some(preg);
if self.vregs[vreg.vreg()].reftype {
self.reftype_vregs_in_pregs_count += 1;
}
}
pub fn clear_preg(&mut self, preg: PReg) {
self.clear_preg_idx(preg.index());
}
fn clear_preg_idx(&mut self, preg: usize) {
let pdata = &mut self.pregs[preg];
if let Some(vreg) = pdata.vreg {
let vdata = &mut self.vregs[vreg as usize];
debug_assert_eq!(vdata.preg.unwrap().index(), preg);
vdata.preg = None;
pdata.vreg = None;
if vdata.reftype {
self.reftype_vregs_in_pregs_count -= 1;
}
}
}
pub fn clear_vreg_from_reg(&mut self, vreg: VReg) {
let vdata = &mut self.vregs[vreg.vreg()];
if let Some(preg) = vdata.preg {
debug_assert_eq!(self.pregs[preg.index()].vreg.unwrap(), vreg.vreg() as u32);
self.pregs[preg.index()].vreg = None;
vdata.preg = None;
if vdata.reftype {
self.reftype_vregs_in_pregs_count -= 1;
}
}
}
pub fn clear_reftype_vregs(&mut self) {
if self.reftype_vregs_in_pregs_count == 0 {
return;
}
for i in 0..self.pregs.len() {
if let Some(vreg) = self.pregs[i].vreg.clone() {
let vreg = vreg as usize;
if self.vregs[vreg].reftype {
self.clear_preg_idx(i);
}
}
}
}
}
pub fn run<F: Function>(func: &F, mach_env: &MachineEnv) -> Result<Output, RegAllocError> {
if func.multi_spillslot_named_by_last_slot() {
panic!("MultiSpillslotIndexPos not supported");
}
let mut state = FastAllocState::init(func, mach_env);
let const_state = ReadOnlyData::init(func, mach_env);
calc_use_positions_and_live_bitmaps(&mut state, &const_state)?;
state.blocks[func.entry_block().index()].params_allocated = true;
let len = const_state.postorder.len();
for i in 0..len {
let block = const_state.postorder[len - 1 - i];
if state.blocks[block.index()].reg_allocated {
trace!("Block {} already allocated. Skipping", block.index());
continue;
}
state.blocks[block.index()].reg_allocated = true;
trace!("Allocating block {}", block.index());
allocate_block_insts(&mut state, &const_state, block)?;
handle_out_block_params(&mut state, &const_state, block)?;
}
// we do not iterate the blocks in their index order so the order of edits might not be sorted by progpoint
// however it should be nearly sorted
state.edits.sort_by_key(|entry| entry.0);
// these might also not be sorted
//state.safepoint_slots.sort_by_key(|entry| entry.0);
state
.safepoint_slots
.sort_unstable_by_key(|(progpoint, slot)| u64_key(progpoint.to_index(), slot.bits()));
if trace_enabled!() {
trace!("Edits:");
for edit in &state.edits {
match edit.1 {
Edit::Move { from, to } => {
trace!("At {:?} from {} to {}", edit.0, from, to);
}
}
}
trace!("StackMaps:");
for entry in &state.safepoint_slots {
trace!("At {:?} at {}", entry.0, entry.1);
}
}
Ok(Output {
num_spillslots: state.cur_stack_slot_idx as usize,
edits: state.edits,
allocs: state.allocs,
inst_alloc_offsets: state.inst_alloc_offsets,
safepoint_slots: state.safepoint_slots,
debug_locations: Vec::new(),
stats: Stats::default(),
})
}
fn allocate_block_insts<'a, F: Function>(
state: &mut FastAllocState<'a, F>,
const_state: &ReadOnlyData,
block: Block,
) -> Result<(), RegAllocError> {
for inst in state.func.block_insns(block).iter() {
let edit_start_idx = state.edits.len();
let clobbers = state.func.inst_clobbers(inst);
let operands = state.func.inst_operands(inst);
let req_refs_on_stack = state.func.requires_refs_on_stack(inst);
let alloc_idx = state.inst_alloc_offsets[inst.index()] as usize;
trace!(
"Allocating Inst {} (refs_on_stack: {}, is_ret: {}, is_branch: {}, alloc_idx: {})",
inst.index(),
req_refs_on_stack,
state.func.is_ret(inst),
state.func.is_branch(inst),
alloc_idx
);
if trace_enabled!() {
let mut str = String::new();
for preg in clobbers {
if str.is_empty() {
str.push_str(&format!("{}", preg));
} else {
str.push_str(&format!(", {}", preg));
}
}
trace!("Clobbers: {}", str);
}
// keep track of which pregs where allocated so we can clear them later on
// TODO: wouldnt need this if we look up the inst a vreg was allocated at
let mut regs_allocated = PRegSet::empty();
// keep track of which pregs hold late uses/early writes and so are unelligible
// as destinations for late writes
let mut late_write_disallow_regs = PRegSet::empty();
// we need to keep track of late defs allocated during the fixed reg stage
// as they may not overlap with late uses and there is no order guarantee for inst_operands
let mut late_write_regs = PRegSet::empty();
if req_refs_on_stack {
state.clear_reftype_vregs();
let pos = ProgPoint::before(inst);
trace!("Calculating Stackmap for {:?}", pos);
for vreg in state.reftype_vregs {
let data = &state.vregs[vreg.vreg()];
if let Some(slot) = data.slot_idx {
if domtree::dominates(&const_state.idom, data.def_block.unwrap(), block) {
trace!("Marking vreg {} as saved on stack at {}", vreg, slot);
state
.safepoint_slots
.push((pos, Allocation::stack(SpillSlot::new(slot as usize))));
} else {
trace!("Skipping {} as it does not dominate", vreg);
}
}
}
}
// we allocate fixed defs/uses and stack allocations first
trace!("First alloc pass");
for (i, op) in operands.iter().enumerate() {
let vreg = op.vreg();
trace!("Operand {}: {}", i, op);
if vreg == VReg::invalid() {
// it seems cranelift emits fixed reg uses with invalid vregs, handle them here
// TODO: treat them like normal vregs by just using last_vreg_index+1 for them?
match op.constraint() {
OperandConstraint::FixedReg(reg) => {
state.clear_preg(reg);
regs_allocated.add(reg);
state.allocs[alloc_idx + i] = Allocation::reg(reg);
trace!("Chose {} for operand {}", reg, i);
late_write_disallow_regs.add(reg);
}
_ => {
panic!(
"Invalid op constraint {:?} for invalid vreg",
op.constraint()
);
}
}
continue;
}
match op.constraint() {
OperandConstraint::FixedReg(reg) => {
match op.kind() {
OperandKind::Use => {
if req_refs_on_stack && state.vregs[vreg.vreg()].reftype {
panic!("reftype has fixed use when its required to be on stack");
return Err(RegAllocError::TooManyLiveRegs);
}
if state.pregs[reg.index()].vreg.is_some() {
// if the reg was allocated by another early use/write or late use
// OR it is allocated and we have a late use we cannot do a correct allocation
if op.pos() == OperandPos::Late || !late_write_regs.contains(reg) {
panic!("fixed reg late use would overwrite another fixed reg use/early write");
return Err(RegAllocError::TooManyLiveRegs);
}
}
state.move_to_preg(vreg, reg, ProgPoint::before(inst));
state.allocs[alloc_idx + i] = Allocation::reg(reg);
if op.pos() == OperandPos::Late {
if clobbers.contains(reg) {
panic!("fixed late use would be clobbered");
return Err(RegAllocError::TooManyLiveRegs);
}
trace!("Operand {}'s allocation may not be used by a late def", i);
// late uses cannot share a register with late defs
late_write_disallow_regs.add(reg);
}
regs_allocated.add(reg);
trace!("Chose {} for operand {}", reg, i);
}
OperandKind::Def => {
if op.pos() == OperandPos::Late {
if late_write_disallow_regs.contains(reg) {
panic!("fixed late def would overwrite late use/early def");
return Err(RegAllocError::TooManyLiveRegs);
}
late_write_regs.add(reg);
} else {
if state.pregs[reg.index()].vreg.is_some() || clobbers.contains(reg)
{
// early defs cannot share a register with anything and cannot be clobbered
panic!("early def shares reg or is clobbered");
return Err(RegAllocError::TooManyLiveRegs);
}
trace!("Operand {}'s allocation may not be used by a late def", i);
// early defs cannot share a register with late defs
late_write_disallow_regs.add(reg);
}
state.vregs[vreg.vreg()].def_block = Some(block);
state.allocs[alloc_idx + i] = Allocation::reg(reg);
state.assign_preg(reg, vreg);
// some pseudoops use the pseudo stack pregs as defs
if state.pregs[reg.index()].stack_pseudo {
// find preg to use as a tmp register
let mut pregs = PRegSet::empty();
for reg in const_state.reg_order(vreg.class()) {
if state.pregs[reg.index()].vreg.is_some() {
continue;
}
pregs.add(*reg);
}
for op in operands {
match op.constraint() {
OperandConstraint::FixedReg(reg) => {
if op.kind() == OperandKind::Use
&& op.pos() == OperandPos::Early
{
continue;
}
pregs.remove(reg);
}
_ => {}
}
}
if pregs == PRegSet::empty() {
panic!("No way to solve pseudo-stack preg");
}
// Move from pseudoreg to tmp_reg and then to stack
let tmp_reg = pregs.into_iter().next().unwrap();
if state.pregs[tmp_reg.index()].vreg.is_some() {
state.clear_preg(tmp_reg);
}
state.edits.push((
ProgPoint::after(inst),
Edit::Move {
from: Allocation::reg(reg),
to: Allocation::reg(tmp_reg),
},
));
if state.pregs[reg.index()].vreg.is_some() {
state.clear_preg(reg);
}
state.assign_preg(tmp_reg, vreg);
state.move_to_stack(tmp_reg, vreg, ProgPoint::after(inst));
regs_allocated.add(tmp_reg);
} else {
state.alloc_stack_slot(vreg);
state.move_to_stack(reg, vreg, ProgPoint::after(inst));
regs_allocated.add(reg);
}
trace!("Chose {} for operand {}", reg, i);
}
}
}
OperandConstraint::Stack | OperandConstraint::Any => {
// we allocate Any on the stack for now
match op.kind() {
OperandKind::Use => {
if let Some(slot) = &state.vregs[vreg.vreg()].slot_idx {
state.allocs[alloc_idx + i] =
Allocation::stack(SpillSlot::new(*slot as usize));
trace!("Chose slot {} for operand {}", slot, i);
} else {
return Err(RegAllocError::SSA(vreg, inst));
}
}
OperandKind::Def => {
state.vregs[vreg.vreg()].def_block = Some(block);
let slot = state.alloc_stack_slot(vreg);
state.allocs[alloc_idx + i] =
Allocation::stack(SpillSlot::new(slot as usize));
trace!("Chose slot {} for operand {}", slot, i);
}
}
}
_ => continue,
}
}
// alloc non-fixed uses and early defs in registers
trace!("Second alloc pass");
for (i, op) in operands.iter().enumerate() {
if op.kind() == OperandKind::Def && op.pos() == OperandPos::Late {
continue;
}
trace!("Operand {}: {}", i, op);
let vreg = op.vreg();
if vreg == VReg::invalid() {
continue;
}
match op.constraint() {
OperandConstraint::Reg => {
// find first non-allocated register
let reg_order = const_state.reg_order(op.class());
let mut allocated = false;
for &reg in reg_order {
if regs_allocated.contains(reg) {
continue;
}
// reg should not contain anything
debug_assert!(state.pregs[reg.index()].vreg.is_none());
state.allocs[alloc_idx + i] = Allocation::reg(reg);
regs_allocated.add(reg);
if op.kind() == OperandKind::Use {
if req_refs_on_stack && state.vregs[vreg.vreg()].reftype {
panic!("reftype required to be in reg at safepoint");
return Err(RegAllocError::TooManyLiveRegs);
}
// need to move from stack to reg
state.move_to_preg(vreg, reg, ProgPoint::before(inst));
} else {
// early def
state.vregs[vreg.vreg()].def_block = Some(block);
state.assign_preg(reg, vreg);
state.alloc_stack_slot(vreg);
state.move_to_stack(reg, vreg, ProgPoint::after(inst));
}
trace!("Chose {} for operand {}", reg, i);
allocated = true;
break;
}
if allocated {
continue;
}
trace!("Ran out of registers for operand {}", i);
// No register available
// TODO: try to evict vreg that does not need to be in a preg
panic!("Out of registers: {:?}", regs_allocated);
return Err(RegAllocError::TooManyLiveRegs);
}
OperandConstraint::Reuse(_) => {
panic!("Illegal register constraint reuse for early def or use");
}
_ => {}
}
}
// alloc non-fixed late defs and reuse
trace!("Third alloc pass");
for (i, op) in operands.iter().enumerate() {
if op.kind() != OperandKind::Def || op.pos() != OperandPos::Late {
continue;
}
trace!("Operand {}: {}", i, op);
let vreg = op.vreg();
if vreg == VReg::invalid() {
continue;
}
state.vregs[vreg.vreg()].def_block = Some(block);
match op.constraint() {
OperandConstraint::Reg => {
// find first non-allocated register
let reg_order = const_state.reg_order(op.class());
let mut allocated = false;
for &reg in reg_order {
if regs_allocated.contains(reg) || late_write_disallow_regs.contains(reg) {
continue;
}
// reg should not contain anything
regs_allocated.add(reg);
state.allocs[alloc_idx + i] = Allocation::reg(reg);
state.clear_preg(reg);
state.assign_preg(reg, vreg);
state.alloc_stack_slot(vreg);
state.move_to_stack(reg, vreg, ProgPoint::after(inst));
trace!("Chose {} for operand {}", reg, i);
allocated = true;
break;
}
if allocated {
continue;
}
// No register available
// TODO: try to evict vreg that does not need to be in a preg
panic!("out of registers");
return Err(RegAllocError::TooManyLiveRegs);
}
OperandConstraint::Reuse(idx) => {
debug_assert!(state.allocs[alloc_idx + idx].is_reg());
let preg = state.allocs[alloc_idx + idx].as_reg().unwrap();
debug_assert!(regs_allocated.contains(preg));
state.allocs[alloc_idx + i] = Allocation::reg(preg);
state.clear_preg(preg);
state.assign_preg(preg, vreg);
state.alloc_stack_slot(vreg);
state.move_to_stack(preg, vreg, ProgPoint::after(inst));
}
_ => {
debug_assert!(!state.allocs[alloc_idx + i].is_none());
}
}
}
// clear out all allocated regs
for reg in regs_allocated {
trace!("Clearing {}", reg);
state.clear_preg(reg);
}
// fixup edit order
let mut first_post_pos = None;
for i in edit_start_idx..state.edits.len() {
debug_assert!(state.edits[i].0.inst() == inst);
match first_post_pos {
None => {
if state.edits[i].0.pos() == InstPosition::After {
first_post_pos = Some(i);
}
}
Some(pos) => {
if state.edits[i].0.pos() == InstPosition::Before {
state.edits.swap(pos, i);
first_post_pos = Some(pos + 1);
}
}
}
}
}
Ok(())
}
fn handle_out_block_params<'a, F: Function>(
state: &mut FastAllocState<'a, F>,
const_state: &ReadOnlyData,
block: Block,
) -> Result<(), RegAllocError> {
trace!("Allocating outgoing blockparams for {}", block.index());
let last_inst = state.func.block_insns(block).last();
if !state.func.is_branch(last_inst) {
trace!("Last inst {} is not a branch", last_inst.index());
return Ok(());
}
let mut pregs_used_by_br = PRegSet::empty();
{
let alloc_start = state.inst_alloc_offsets[last_inst.index()] as usize;
let alloc_end = if last_inst.index() + 1 == state.inst_alloc_offsets.len() {
state.inst_alloc_offsets.len()
} else {
state.inst_alloc_offsets[last_inst.index() + 1] as usize
};
for i in alloc_start..alloc_end {
if let Some(reg) = state.allocs[i].clone().as_reg() {
pregs_used_by_br.add(reg);
}
}
}
// wouldn't need this if the edits for this were made before the moves for the branch inst but that has its own share of problems i think
let tmp_reg_int = 'block: {
for reg in const_state.reg_order(RegClass::Int) {
if !pregs_used_by_br.contains(*reg) {
break 'block *reg;
}
}
panic!("No usable tmp_reg for block param handling");
};
let tmp_reg_float = 'block: {
for reg in const_state.reg_order(RegClass::Float) {
if !pregs_used_by_br.contains(*reg) {
break 'block *reg;
}
}
panic!("No usable tmp_reg for block param handling");
};
let succs = state.func.block_succs(block);
if succs.len() == 1 && state.blocks[succs[0].index()].params_allocated {
trace!("Only one allocated successor, moving allocations");
let succ = succs[0];
// move values to the already allocated places
let in_params = state.func.block_params(succ);
let out_params = state.func.branch_blockparams(block, last_inst, 0);
debug_assert_eq!(in_params.len(), out_params.len());
for i in 0..in_params.len() {
let in_vreg = in_params[i];
let out_vreg = out_params[i];
debug_assert!(state.vregs[in_vreg.vreg()].slot_idx.is_some());
debug_assert!(state.vregs[out_vreg.vreg()].slot_idx.is_some());
let tmp_reg = if out_vreg.class() == RegClass::Int {
tmp_reg_int
} else {
tmp_reg_float
};
let out_slot = state.vregs[out_vreg.vreg()].slot_idx.unwrap();
let in_slot = state.vregs[in_vreg.vreg()].slot_idx.unwrap();
trace!(
"Move {} from slot {} to slot {} for {}",
out_vreg,
out_slot,
in_slot,
in_vreg
);
state.edits.push((
ProgPoint::before(last_inst),
Edit::Move {
from: Allocation::stack(SpillSlot::new(out_slot as usize)),
to: Allocation::reg(tmp_reg),
},
));
state.edits.push((
ProgPoint::before(last_inst),
Edit::Move {
from: Allocation::reg(tmp_reg),
to: Allocation::stack(SpillSlot::new(in_slot as usize)),
},
));
}
} else {
trace!("Successors not allocated. Creating allocation");
let mut allocs = SmallVec::<[(VReg, u32); 4]>::new();
// set incoming block params of successor to the current stack slot
for (i, &succ) in state.func.block_succs(block).iter().enumerate() {
trace!("Creating block {}", succ.index());
if state.blocks[succ.index()].params_allocated {
return Err(RegAllocError::CritEdge(block, succ));
}
// we allocate the params here
// TODO: can there be a problem if the same successor occurs multiple times?
state.blocks[succ.index()].params_allocated = true;
let in_params = state.func.block_params(succ);
let out_params = state.func.branch_blockparams(block, last_inst, i);
debug_assert_eq!(in_params.len(), out_params.len());
let mut vregs_passed = SmallVec::<[VReg; 4]>::new();
for i in 0..in_params.len() {
let out_vreg = out_params[i];
let in_vreg = in_params[i];
debug_assert!(state.vregs[out_vreg.vreg()].slot_idx.is_some());
let out_slot_idx = state.vregs[out_vreg.vreg()].slot_idx.unwrap();
state.vregs[in_vreg.vreg()].def_block = Some(succ);
// TODO: if out_vreg dies at this edge, we could reuse its stack slot
// TODO: we should also be able to reuse the slot if the successor only has one predecessor (us); check with AE
let mut no_alias = false;
if !vregs_passed.contains(&out_vreg) {
let mut alloced = false;
for alloc in &allocs {
if alloc.0 != out_vreg {
continue;
}
// we can use the already moved into stack slot
state.vregs[in_vreg.vreg()].slot_idx = Some(alloc.1);
vregs_passed.push(out_vreg);
alloced = true;
break;
}
if alloced {
continue;
}
vregs_passed.push(out_vreg);
no_alias = true;
}
// need to duplicate to avoid aliasing or create a new stack slot
// TODO: this creates multiple duplications for multiple blocks, can be avoided
let tmp_reg = if out_vreg.class() == RegClass::Int {
tmp_reg_int
} else {
tmp_reg_float
};
let slot = state.create_stack_slot(out_vreg.class());
trace!(
"Moving {} from slot {} to slot {} for {}",
out_vreg,
out_slot_idx,
slot,
in_vreg
);
state.edits.push((
ProgPoint::before(last_inst),
Edit::Move {
from: Allocation::stack(SpillSlot::new(out_slot_idx as usize)),
to: Allocation::reg(tmp_reg),
},
));
state.edits.push((
ProgPoint::before(last_inst),
Edit::Move {
from: Allocation::reg(tmp_reg),
to: Allocation::stack(SpillSlot::new(slot as usize)),
},
));
state.vregs[in_vreg.vreg()].slot_idx = Some(slot);
if no_alias {
allocs.push((out_vreg, slot));
}
}
}
}
Ok(())
}
// don't inline for better perf stats
#[inline(never)]
fn calc_use_positions_and_live_bitmaps<'a, F: Function>(
state: &mut FastAllocState<'a, F>,
const_state: &ReadOnlyData,
) -> Result<(), RegAllocError> {
// TODO: this could be folded into the bitmap calculation by making a
// reverse postorder idx -> (block idx, inst_count for all)
// mapping
//
// we use a pseudo-counter to have a uniform position for instructions
// this takes 0.5-0.8% on average but has maxes of up to 2% of compile time
// so if it does not substantially increase compilation performance it should be killed
let mut cur_pos = 0u32;
let len = const_state.postorder.len();
for i in 0..len {
let block = const_state.postorder[len - 1 - i];
trace!("Calculating uses for block {}", block.index());
let insts = state.func.block_insns(block);
for inst in insts.clone().iter() {
let operands = state.func.inst_operands(inst);
for op in operands {
if op.kind() != OperandKind::Use {
continue;
}
if op.vreg() == VReg::invalid() {
continue;
}
state.vregs[op.vreg().vreg()].uses.push(cur_pos);
}
cur_pos += 1;
}
let last_inst = insts.last();
if !state.func.is_branch(last_inst) {
continue;
}
for i in 0..state.func.block_succs(block).len() {
for vreg in state.func.branch_blockparams(block, last_inst, i) {
state.vregs[vreg.vreg()].uses.push(cur_pos);
}
}
cur_pos += 1;
}
calc_live_bitmaps(state, const_state)
}
struct BlockBitmap {
storage: SmallVec<[u64; 2]>,
}
impl BlockBitmap {
fn init(block_count: usize) -> Self {
let u64_count = (block_count + 63) / 64;
let mut storage = SmallVec::<[u64; 2]>::with_capacity(u64_count);
storage.resize(u64_count, 0);
Self { storage }
}
fn set(&mut self, idx: usize) {
let idx = idx / 64;
let bit = 1u64 << (idx % 64);
self.storage[idx] |= bit;
}
fn un_set(&mut self, idx: usize) {
let idx = idx / 64;
let bit = 1u64 << (idx % 64);
self.storage[idx] &= !bit;
}
fn is_set(&mut self, idx: usize) -> bool {
let idx = idx / 64;
let bit = 1u64 << (idx % 64);
(self.storage[idx] & bit) != 0
}
}
// currently, copy from liveranges.rs
// don't inline for better perf stats
#[inline(never)]
fn calc_live_bitmaps<'a, F: Function>(
state: &mut FastAllocState<'a, F>,
const_state: &ReadOnlyData,
) -> Result<(), RegAllocError> {
state.liveins.resize(state.blocks.len(), IndexSet::new());
state.liveouts.resize(state.blocks.len(), IndexSet::new());
// Run a worklist algorithm to precisely compute liveins and
// liveouts.
let mut workqueue = VecDeque::new();
let mut workqueue_set = BlockBitmap::init(state.liveins.len());
// Initialize workqueue with postorder traversal.
for &block in &const_state.postorder[..] {
workqueue.push_back(block);
workqueue_set.set(block.index());
}
while let Some(block) = workqueue.pop_front() {
workqueue_set.un_set(block.index());
let insns = state.func.block_insns(block);
trace!("computing liveins for block{}", block.index());
//state.stats.livein_iterations += 1;
let mut live = state.liveouts[block.index()].clone();
trace!(" -> initial liveout set: {:?}", live);
// Include outgoing blockparams in the initial live set.
if state.func.is_branch(insns.last()) {
for i in 0..state.func.block_succs(block).len() {
for &param in state.func.branch_blockparams(block, insns.last(), i) {
live.set(param.vreg(), true);
}
}
}
// TODO: evaluate if this generates better code than insns.rev().iter()
let last_idx = insns.last().index();
let len = last_idx - insns.first().index() + 1;
for inst_rev_idx in 0..len {
let inst = Inst::new(last_idx - inst_rev_idx);
// TODO: this differs from the algo in liveranges.rs by not iterating through the positions
// as in SSA it should make no difference as there can be no vreg that is both a use and def at
// a single instruction
for op in state.func.inst_operands(inst) {
if op.as_fixed_nonallocatable().is_some() {
continue;
}
let was_live = live.get(op.vreg().vreg());
trace!("op {:?} was_live = {}", op, was_live);
match op.kind() {
OperandKind::Use => {
live.set(op.vreg().vreg(), true);
}
OperandKind::Def => {
live.set(op.vreg().vreg(), false);
}
}
}
}
// TODO: i dont get why blockparams are not live when going in?
for &blockparam in state.func.block_params(block) {
live.set(blockparam.vreg(), false);
}
for &pred in state.func.block_preds(block) {
if state.liveouts[pred.index()].union_with(&live) {
if !workqueue_set.is_set(pred.index()) {
workqueue_set.set(pred.index());
workqueue.push_back(pred);
}
}
}
trace!("computed liveins at block{}: {:?}", block.index(), live);
state.liveins[block.index()] = live;
}
// Check that there are no liveins to the entry block.
if !state.liveins[state.func.entry_block().index()].is_empty() {
trace!(
"non-empty liveins to entry block: {:?}",
state.liveins[state.func.entry_block().index()]
);
return Err(RegAllocError::EntryLivein);
}
Ok(())
}