Simpler / possibly better splitting: split based on conflict position, always, and use a reg hint to put the before-conflict part in the place where we determined it fit before.

This commit is contained in:
Chris Fallin
2021-05-21 01:34:52 -07:00
parent ec7fdeb8ed
commit 466ea2cd9a

View File

@@ -14,15 +14,6 @@
/* /*
* TODO: * TODO:
* *
* - tune heuristics:
* - splits:
* - safepoints?
* - split just before uses with fixed regs and/or just after defs
* with fixed regs?
* - measure average liverange length / number of splits / ...
*
* - reused-input reg: don't allocate register for input that is reused.
*
* - "Fixed-stack location": negative spillslot numbers? * - "Fixed-stack location": negative spillslot numbers?
* *
* - Rematerialization * - Rematerialization
@@ -159,6 +150,12 @@ impl LiveRange {
self.uses_spill_weight_and_flags &= !((flag as u32) << 29); self.uses_spill_weight_and_flags &= !((flag as u32) << 29);
} }
#[inline(always)] #[inline(always)]
pub fn assign_flag(&mut self, flag: LiveRangeFlag, val: bool) {
let bit = if val { (flag as u32) << 29 } else { 0 };
self.uses_spill_weight_and_flags &= 0xe000_0000;
self.uses_spill_weight_and_flags |= bit;
}
#[inline(always)]
pub fn has_flag(&self, flag: LiveRangeFlag) -> bool { pub fn has_flag(&self, flag: LiveRangeFlag) -> bool {
self.uses_spill_weight_and_flags & ((flag as u32) << 29) != 0 self.uses_spill_weight_and_flags & ((flag as u32) << 29) != 0
} }
@@ -393,6 +390,7 @@ struct PrioQueue {
struct PrioQueueEntry { struct PrioQueueEntry {
prio: u32, prio: u32,
bundle: LiveBundleIndex, bundle: LiveBundleIndex,
reg_hint: PReg,
} }
#[derive(Clone, Debug)] #[derive(Clone, Debug)]
@@ -459,10 +457,11 @@ impl PrioQueue {
} }
} }
fn insert(&mut self, bundle: LiveBundleIndex, prio: usize) { fn insert(&mut self, bundle: LiveBundleIndex, prio: usize, reg_hint: PReg) {
self.heap.push(PrioQueueEntry { self.heap.push(PrioQueueEntry {
prio: prio as u32, prio: prio as u32,
bundle, bundle,
reg_hint,
}); });
} }
@@ -470,8 +469,8 @@ impl PrioQueue {
self.heap.is_empty() self.heap.is_empty()
} }
fn pop(&mut self) -> Option<LiveBundleIndex> { fn pop(&mut self) -> Option<(LiveBundleIndex, PReg)> {
self.heap.pop().map(|entry| entry.bundle) self.heap.pop().map(|entry| (entry.bundle, entry.reg_hint))
} }
} }
@@ -2260,16 +2259,17 @@ impl<'a, F: Function> Env<'a, F> {
log::debug!(" -> prio {}", prio); log::debug!(" -> prio {}", prio);
self.bundles[bundle.index()].prio = prio; self.bundles[bundle.index()].prio = prio;
self.recompute_bundle_properties(bundle); self.recompute_bundle_properties(bundle);
self.allocation_queue.insert(bundle, prio as usize); self.allocation_queue
.insert(bundle, prio as usize, PReg::invalid());
} }
self.stats.merged_bundle_count = self.allocation_queue.heap.len(); self.stats.merged_bundle_count = self.allocation_queue.heap.len();
} }
fn process_bundles(&mut self) -> Result<(), RegAllocError> { fn process_bundles(&mut self) -> Result<(), RegAllocError> {
let mut count = 0; let mut count = 0;
while let Some(bundle) = self.allocation_queue.pop() { while let Some((bundle, reg_hint)) = self.allocation_queue.pop() {
self.stats.process_bundle_count += 1; self.stats.process_bundle_count += 1;
self.process_bundle(bundle)?; self.process_bundle(bundle, reg_hint)?;
count += 1; count += 1;
if count > self.func.insts() * 50 { if count > self.func.insts() * 50 {
self.dump_state(); self.dump_state();
@@ -2492,7 +2492,8 @@ impl<'a, F: Function> Env<'a, F> {
} }
let prio = self.bundles[bundle.index()].prio; let prio = self.bundles[bundle.index()].prio;
log::debug!(" -> prio {}; back into queue", prio); log::debug!(" -> prio {}; back into queue", prio);
self.allocation_queue.insert(bundle, prio as usize); self.allocation_queue
.insert(bundle, prio as usize, PReg::invalid());
} }
fn bundle_spill_weight(&self, bundle: LiveBundleIndex) -> u32 { fn bundle_spill_weight(&self, bundle: LiveBundleIndex) -> u32 {
@@ -2587,476 +2588,170 @@ impl<'a, F: Function> Env<'a, F> {
self.bundles[bundle.index()].cached_minimal() self.bundles[bundle.index()].cached_minimal()
} }
fn find_split_points( fn recompute_range_properties(&mut self, range: LiveRangeIndex) {
&mut self, let mut rangedata = &mut self.ranges[range.index()];
bundle: LiveBundleIndex, let w = rangedata.uses.iter().map(|u| u.weight as u32).sum();
conflicting: LiveBundleIndex, rangedata.uses_spill_weight_and_flags = w;
) -> SmallVec<[ProgPoint; 4]> { if rangedata.uses.len() > 0 && rangedata.uses[0].operand.kind() == OperandKind::Def {
// Scan the bundle's ranges once. We want to record: rangedata.set_flag(LiveRangeFlag::StartsAtDef);
// - Does the bundle contain any ranges in "hot" code and/or "cold" code?
// If so, record the transition points that are fully included in
// `bundle`: the first ProgPoint in a hot range if the prior cold
// point is also in the bundle; and the first ProgPoint in a cold
// range if the prior hot point is also in the bundle.
// - Does the bundle cross any clobbering insts?
// If so, record the ProgPoint before each such instruction.
// - Is there a register use before the conflicting bundle?
// If so, record the ProgPoint just after the last one.
// - Is there a register use after the conflicting bundle?
// If so, record the ProgPoint just before the last one.
//
// Then choose one of the above kinds of splits, in priority order.
let mut def_splits: SmallVec<[ProgPoint; 4]> = smallvec![];
let mut seen_defs = 0;
let mut cold_hot_splits: SmallVec<[ProgPoint; 4]> = smallvec![];
let mut clobber_splits: SmallVec<[ProgPoint; 4]> = smallvec![];
let mut last_before_conflict: Option<ProgPoint> = None;
let mut first_after_conflict: Option<ProgPoint> = None;
log::debug!(
"find_split_points: bundle {:?} conflicting {:?}",
bundle,
conflicting
);
// We simultaneously scan the sorted list of LiveRanges in our bundle
// and the sorted list of call instruction locations. We also take the
// total range (start of first range to end of last range) of the
// conflicting bundle, if any, so we can find the last use before it and
// first use after it. Each loop iteration handles one range in our
// bundle. Calls are scanned up until they advance past the current
// range.
let our_ranges = &self.bundles[bundle.index()].ranges[..];
let (conflict_from, conflict_to) = if conflicting.is_valid() {
(
Some(
self.bundles[conflicting.index()]
.ranges
.first()
.unwrap()
.range
.from,
),
Some(
self.bundles[conflicting.index()]
.ranges
.last()
.unwrap()
.range
.to,
),
)
} else {
(None, None)
};
let bundle_start = if our_ranges.is_empty() {
ProgPoint::before(Inst::new(0))
} else {
our_ranges.first().unwrap().range.from
};
let bundle_end = if our_ranges.is_empty() {
ProgPoint::before(Inst::new(self.func.insts()))
} else {
our_ranges.last().unwrap().range.to
};
log::debug!(" -> conflict from {:?} to {:?}", conflict_from, conflict_to);
let mut clobberidx = 0;
for entry in our_ranges {
// Probe the hot-code tree.
log::debug!(" -> range {:?}", entry.range);
if let Some(hot_range_idx) = self
.hot_code
.btree
.get(&LiveRangeKey::from_range(&entry.range))
{
// `hot_range_idx` is a range that *overlaps* with our range.
// There may be cold code in our range on either side of the hot
// range. Record the transition points if so.
let hot_range = self.ranges[hot_range_idx.index()].range;
log::debug!(" -> overlaps with hot-code range {:?}", hot_range);
let start_cold = entry.range.from < hot_range.from;
let end_cold = entry.range.to > hot_range.to;
if start_cold {
log::debug!(
" -> our start is cold; potential split at cold->hot transition {:?}",
hot_range.from,
);
// First ProgPoint in hot range.
cold_hot_splits.push(hot_range.from);
} }
if end_cold {
log::debug!(
" -> our end is cold; potential split at hot->cold transition {:?}",
hot_range.to,
);
// First ProgPoint in cold range (after hot range).
cold_hot_splits.push(hot_range.to);
}
}
// Scan through clobber-insts from last left-off position until the first
// clobbering inst past this range. Record all clobber sites as potential
// splits.
while clobberidx < self.clobbers.len() {
let cur_clobber = self.clobbers[clobberidx];
let pos = ProgPoint::before(cur_clobber);
if pos >= entry.range.to {
break;
}
clobberidx += 1;
if pos < entry.range.from {
continue;
}
if pos > bundle_start {
log::debug!(" -> potential clobber split at {:?}", pos);
clobber_splits.push(pos);
}
}
// Update last-before-conflict and first-before-conflict positions.
let mut update_with_pos = |pos: ProgPoint| {
let before_inst = ProgPoint::before(pos.inst());
let before_next_inst = before_inst.next().next();
if before_inst > bundle_start
&& (conflict_from.is_none() || before_inst < conflict_from.unwrap())
&& (last_before_conflict.is_none()
|| before_inst > last_before_conflict.unwrap())
{
last_before_conflict = Some(before_inst);
}
if before_next_inst < bundle_end
&& (conflict_to.is_none() || pos >= conflict_to.unwrap())
&& (first_after_conflict.is_none() || pos > first_after_conflict.unwrap())
{
first_after_conflict = Some(ProgPoint::before(pos.inst().next()));
}
};
for u in &self.ranges[entry.index.index()].uses {
log::debug!(" -> range has use at {:?}", u.pos);
update_with_pos(u.pos);
if u.operand.kind() == OperandKind::Def {
if seen_defs > 0 {
def_splits.push(u.pos);
}
seen_defs += 1;
}
}
}
log::debug!(
" -> first use/def after conflict range: {:?}",
first_after_conflict,
);
log::debug!(
" -> last use/def before conflict range: {:?}",
last_before_conflict,
);
// Based on the above, we can determine which split strategy we are taking at this
// iteration:
// - If we span both hot and cold code, split into separate "hot" and "cold" bundles.
// - Otherwise, if we span any calls, split just before every call instruction.
// - Otherwise, if there is a register use after the conflicting bundle,
// split at that use-point ("split before first use").
// - Otherwise, if there is a register use before the conflicting
// bundle, split at that use-point ("split after last use").
// - Otherwise, split at every use, to form minimal bundles.
if cold_hot_splits.len() > 0 {
log::debug!(" going with cold/hot splits: {:?}", cold_hot_splits);
self.stats.splits_hot += 1;
cold_hot_splits
} else if clobber_splits.len() > 0 {
log::debug!(" going with clobber splits: {:?}", clobber_splits);
self.stats.splits_clobbers += 1;
clobber_splits
} else if first_after_conflict.is_some() {
self.stats.splits_conflicts += 1;
log::debug!(" going with first after conflict");
smallvec![first_after_conflict.unwrap()]
} else if last_before_conflict.is_some() {
self.stats.splits_conflicts += 1;
log::debug!(" going with last before conflict");
smallvec![last_before_conflict.unwrap()]
} else if def_splits.len() > 0 && def_splits[0] > bundle_start {
log::debug!(" going with non-first def splits: {:?}", def_splits);
self.stats.splits_defs += 1;
def_splits
} else {
self.stats.splits_all += 1;
log::debug!(" splitting at all uses");
self.find_all_use_split_points(bundle)
}
}
fn find_all_use_split_points(&self, bundle: LiveBundleIndex) -> SmallVec<[ProgPoint; 4]> {
let mut splits = smallvec![];
let ranges = &self.bundles[bundle.index()].ranges[..];
log::debug!("finding all use/def splits for {:?}", bundle);
let bundle_start = if ranges.is_empty() {
ProgPoint::before(Inst::new(0))
} else {
self.ranges[ranges[0].index.index()].range.from
};
// N.B.: a minimal bundle must include only ProgPoints in a
// single instruction, but can include both (can include two
// ProgPoints). We split here, taking care to never split *in
// the middle* of an instruction, because we would not be able
// to insert moves to reify such an assignment.
for entry in ranges {
log::debug!(" -> range {:?}: {:?}", entry.index, entry.range);
for u in &self.ranges[entry.index.index()].uses {
log::debug!(" -> use: {:?}", u);
let before_use_inst = if u.operand.kind() == OperandKind::Def {
// For a def, split *at* the def -- this may be an
// After point, but the value cannot be live into
// the def so we don't need to insert a move.
u.pos
} else {
// For an use or mod, split before the instruction
// -- this allows us to insert a move if
// necessary.
ProgPoint::before(u.pos.inst())
};
let after_use_inst = ProgPoint::before(u.pos.inst().next());
log::debug!(
" -> splitting before and after use: {:?} and {:?}",
before_use_inst,
after_use_inst,
);
if before_use_inst > bundle_start {
splits.push(before_use_inst);
}
splits.push(after_use_inst);
}
}
splits.sort_unstable();
log::debug!(" -> final splits: {:?}", splits);
splits
} }
fn split_and_requeue_bundle( fn split_and_requeue_bundle(
&mut self, &mut self,
bundle: LiveBundleIndex, bundle: LiveBundleIndex,
first_conflicting_bundle: LiveBundleIndex, mut split_at: ProgPoint,
reg_hint: PReg,
) { ) {
self.stats.splits += 1; self.stats.splits += 1;
// Try splitting: (i) across hot code; (ii) across all calls,
// if we had a fixed-reg conflict; (iii) before first reg use;
// (iv) after reg use; (v) around all register uses. After
// each type of split, check for conflict with conflicting
// bundle(s); stop when no conflicts. In all cases, re-queue
// the split bundles on the allocation queue.
//
// The critical property here is that we must eventually split
// down to minimal bundles, which consist just of live ranges
// around each individual def/use (this is step (v)
// above). This ensures termination eventually.
let split_points = self.find_split_points(bundle, first_conflicting_bundle);
log::debug!( log::debug!(
"split bundle {:?} (conflict {:?}): split points {:?}", "split bundle {:?} at {:?} and requeue with reg hint (for first part) {:?}",
bundle, bundle,
first_conflicting_bundle, split_at,
split_points reg_hint,
); );
// Split `bundle` at every ProgPoint in `split_points`, // Split `bundle` at `split_at`, creating new LiveRanges and
// creating new LiveRanges and bundles (and updating vregs' // bundles (and updating vregs' linked lists appropriately),
// linked lists appropriately), and enqueue the new bundles. // and enqueue the new bundles.
//
// We uphold several basic invariants here:
// - The LiveRanges in every vreg, and in every bundle, are disjoint
// - Every bundle for a given vreg is disjoint
//
// To do so, we make one scan in program order: all ranges in
// the bundle, and the def/all uses in each range. We track
// the currently active bundle. For each range, we distribute
// its uses among one or more ranges, depending on whether it
// crosses any split points. If we had to split a range, then
// we need to insert the new subparts in its vreg as
// well. N.B.: to avoid the need to *remove* ranges from vregs
// (which we could not do without a lookup, since we use
// singly-linked lists and the bundle may contain multiple
// vregs so we cannot simply scan a single vreg simultaneously
// to the main scan), we instead *trim* the existing range
// into its first subpart, and then create the new
// subparts. Note that shrinking a LiveRange is always legal
// (as long as one replaces the shrunk space with new
// LiveRanges).
//
// Note that the original IonMonkey splitting code is quite a
// bit more complex and has some subtle invariants. We stick
// to the above invariants to keep this code maintainable.
let spillset = self.bundles[bundle.index()].spillset; let spillset = self.bundles[bundle.index()].spillset;
let mut split_idx = 0;
// Fast-forward past any splits that occur before or exactly assert!(!self.bundles[bundle.index()].ranges.is_empty());
// at the start of the first range in the bundle. // Split point *at* start is OK; this means we peel off
let bundle_start = if self.bundles[bundle.index()].ranges.is_empty() { // exactly one use to create a minimal bundle.
ProgPoint::before(Inst::new(0)) let bundle_start = self.bundles[bundle.index()]
.ranges
.first()
.unwrap()
.range
.from;
assert!(split_at >= bundle_start);
let bundle_end = self.bundles[bundle.index()].ranges.last().unwrap().range.to;
assert!(split_at < bundle_end);
// Is the split point *at* the start? If so, peel off the
// first use: set the split point just after it, or just
// before it if it comes after the start of the bundle.
if split_at == bundle_start {
// Find any uses; if none, just chop off one instruction.
let mut first_use = None;
'outer: for entry in &self.bundles[bundle.index()].ranges {
for u in &self.ranges[entry.index.index()].uses {
first_use = Some(u.pos);
break 'outer;
}
}
log::debug!(" -> first use loc is {:?}", first_use);
split_at = match first_use {
Some(pos) => {
if pos.inst() == bundle_start.inst() {
ProgPoint::before(pos.inst().next())
} else { } else {
self.bundles[bundle.index()].ranges[0].range.from ProgPoint::before(pos.inst())
};
while split_idx < split_points.len() && split_points[split_idx] <= bundle_start {
split_idx += 1;
} }
let mut new_bundles: LiveBundleVec = smallvec![];
let mut cur_bundle = bundle;
let ranges = std::mem::replace(&mut self.bundles[bundle.index()].ranges, smallvec![]);
// - Invariant: current LR `cur_lr` is being built; it has not
// yet been added to `cur_bundle`.
// - Invariant: uses in `cur_uses` have not yet been added to
// `cur_lr`.
for entry in &ranges {
log::debug!(" -> has range {:?} (LR {:?})", entry.range, entry.index);
// Until we reach a split point, copy or create the current range.
let mut cur_range = entry.range;
let mut cur_lr = entry.index;
let mut cur_uses =
std::mem::replace(&mut self.ranges[cur_lr.index()].uses, smallvec![]);
let mut cur_uses = cur_uses.drain(..).peekable();
self.ranges[cur_lr.index()].uses_spill_weight_and_flags = 0;
let update_lr_stats = |lr: &mut LiveRange, u: &Use| {
if lr.uses.is_empty() && u.operand.kind() == OperandKind::Def {
lr.set_flag(LiveRangeFlag::StartsAtDef);
} }
lr.uses_spill_weight_and_flags += u.weight as u32; None => ProgPoint::before(
}; self.bundles[bundle.index()]
while cur_range.to > cur_range.from {
if (split_idx >= split_points.len()) || (split_points[split_idx] >= cur_range.to) {
log::debug!(
" -> no more split points; placing all remaining uses into cur range{}",
cur_lr.index()
);
// No more split points left, or next split point
// is beyond the range: just copy the current
// range into the current bundle, and drop all the
// remaining uses into it.
for u in cur_uses {
update_lr_stats(&mut self.ranges[cur_lr.index()], &u);
log::debug!(" -> use at {:?}", u.pos);
self.ranges[cur_lr.index()].uses.push(u);
}
self.ranges[cur_lr.index()].bundle = cur_bundle;
self.bundles[cur_bundle.index()]
.ranges .ranges
.push(LiveRangeListEntry { .first()
range: cur_range, .unwrap()
index: cur_lr, .range
}); .from
break; .inst()
} .next(),
// If there is a split point prior to or exactly at
// the start of this LR, then create a new bundle but
// keep the existing LR, and go around again. Skip all
// such split-points (lump them into one), while we're
// at it.
if split_points[split_idx] <= cur_range.from {
log::debug!(
" -> split point at {:?} before start of range (range {:?} LR {:?})",
split_points[split_idx],
cur_range,
cur_lr,
);
cur_bundle = self.create_bundle();
log::debug!(" -> new bundle {:?}", cur_bundle);
self.ranges[cur_lr.index()].bundle = cur_bundle;
new_bundles.push(cur_bundle);
self.bundles[cur_bundle.index()].spillset = spillset;
while split_idx < split_points.len()
&& split_points[split_idx] <= cur_range.from
{
split_idx += 1;
}
continue;
}
// If we reach here, there is at least one split point
// that lands in the current range, so we need to
// actually split. Let's create a new LR and bundle
// for the rest (post-split-point), drop uses up to
// the split point into current LR and drop current LR
// into current bundle, then advance current LR and
// bundle to new LR and bundle.
let split = split_points[split_idx];
while split_idx < split_points.len() && split_points[split_idx] == split {
// Skip past all duplicate split-points.
split_idx += 1;
}
log::debug!(" -> split at {:?}", split);
let existing_range = CodeRange {
from: cur_range.from,
to: split,
};
let new_range = CodeRange {
from: split,
to: cur_range.to,
};
let new_lr = self.create_liverange(new_range);
let new_bundle = self.create_bundle();
log::debug!(" -> new LR {:?}, new bundle {:?}", new_lr, new_bundle);
new_bundles.push(new_bundle);
self.bundles[new_bundle.index()].spillset = spillset;
self.ranges[cur_lr.index()].range = existing_range;
self.ranges[new_lr.index()].vreg = self.ranges[cur_lr.index()].vreg;
self.ranges[new_lr.index()].bundle = new_bundle;
while let Some(u) = cur_uses.peek() {
if u.pos >= split {
break;
}
update_lr_stats(&mut self.ranges[cur_lr.index()], &u);
log::debug!(" -> use at {:?} in current LR {:?}", u.pos, cur_lr);
self.ranges[cur_lr.index()].uses.push(*u);
cur_uses.next();
}
log::debug!(
" -> adding current LR {:?} to current bundle {:?}",
cur_lr,
cur_bundle
);
self.ranges[cur_lr.index()].bundle = cur_bundle;
self.bundles[cur_bundle.index()]
.ranges
.push(LiveRangeListEntry {
range: existing_range,
index: cur_lr,
});
if self.annotations_enabled && log::log_enabled!(log::Level::Debug) {
self.annotate(
existing_range.to,
format!(
" SPLIT range{} v{} bundle{} to range{} bundle{}",
cur_lr.index(),
self.ranges[cur_lr.index()].vreg.index(),
cur_bundle.index(),
new_lr.index(),
new_bundle.index(),
), ),
};
assert!(split_at < bundle_end);
log::debug!(
"split point is at bundle start; advancing to {:?}",
split_at
); );
} else {
// Don't split in the middle of an instruction -- this could
// create impossible moves (we cannot insert a move between an
// instruction's uses and defs).
if split_at.pos() == InstPosition::After {
split_at = split_at.next();
}
if split_at >= bundle_end {
split_at = split_at.prev().prev();
}
} }
cur_range = new_range; assert!(split_at > bundle_start && split_at < bundle_end);
cur_bundle = new_bundle;
cur_lr = new_lr; // We need to find which LRs fall on each side of the split,
// which LR we need to split down the middle, then update the
// current bundle, create a new one, and (re)-queue both.
log::debug!(" -> LRs: {:?}", self.bundles[bundle.index()].ranges);
let mut last_lr_in_old_bundle_idx = 0; // last LR-list index in old bundle
let mut first_lr_in_new_bundle_idx = 0; // first LR-list index in new bundle
for (i, entry) in self.bundles[bundle.index()].ranges.iter().enumerate() {
if split_at > entry.range.from {
last_lr_in_old_bundle_idx = i;
first_lr_in_new_bundle_idx = i;
}
if split_at < entry.range.to {
first_lr_in_new_bundle_idx = i;
break;
}
}
log::debug!(
" -> last LR in old bundle: LR {:?}",
self.bundles[bundle.index()].ranges[last_lr_in_old_bundle_idx]
);
log::debug!(
" -> first LR in new bundle: LR {:?}",
self.bundles[bundle.index()].ranges[first_lr_in_new_bundle_idx]
);
// Take the sublist of LRs that will go in the new bundle.
let mut new_lr_list: LiveRangeList = self.bundles[bundle.index()]
.ranges
.iter()
.cloned()
.skip(first_lr_in_new_bundle_idx)
.collect();
self.bundles[bundle.index()]
.ranges
.truncate(last_lr_in_old_bundle_idx + 1);
// If the first entry in `new_lr_list` is a LR that is split
// down the middle, replace it with a new LR and chop off the
// end of the same LR in the original list.
if split_at > new_lr_list[0].range.from {
assert_eq!(last_lr_in_old_bundle_idx, first_lr_in_new_bundle_idx);
let orig_lr = new_lr_list[0].index;
let new_lr = self.create_liverange(CodeRange {
from: split_at,
to: new_lr_list[0].range.to,
});
self.ranges[new_lr.index()].vreg = self.ranges[orig_lr.index()].vreg;
log::debug!(" -> splitting LR {:?} into {:?}", orig_lr, new_lr);
let first_use = self.ranges[orig_lr.index()]
.uses
.iter()
.position(|u| u.pos >= split_at)
.unwrap_or(self.ranges[orig_lr.index()].uses.len());
let rest_uses: UseList = self.ranges[orig_lr.index()]
.uses
.iter()
.cloned()
.skip(first_use)
.collect();
self.ranges[new_lr.index()].uses = rest_uses;
self.ranges[orig_lr.index()].uses.truncate(first_use);
self.recompute_range_properties(orig_lr);
self.recompute_range_properties(new_lr);
new_lr_list[0].index = new_lr;
new_lr_list[0].range = self.ranges[new_lr.index()].range;
self.ranges[orig_lr.index()].range.to = split_at;
self.bundles[bundle.index()].ranges[last_lr_in_old_bundle_idx].range =
self.ranges[orig_lr.index()].range;
// Perform a lazy split in the VReg data. We just // Perform a lazy split in the VReg data. We just
// append the new LR and its range; we will sort by // append the new LR and its range; we will sort by
@@ -3067,32 +2762,45 @@ impl<'a, F: Function> Env<'a, F> {
self.vregs[self.ranges[new_lr.index()].vreg.index()] self.vregs[self.ranges[new_lr.index()].vreg.index()]
.ranges .ranges
.push(LiveRangeListEntry { .push(LiveRangeListEntry {
range: new_range, range: self.ranges[new_lr.index()].range,
index: new_lr, index: new_lr,
}); });
} }
}
// Recompute weights and priorities of all bundles, and let new_bundle = self.create_bundle();
// enqueue all split-bundles on the allocation queue. log::debug!(" -> creating new bundle {:?}", new_bundle);
let prio = self.compute_bundle_prio(bundle); self.bundles[new_bundle.index()].spillset = spillset;
self.bundles[bundle.index()].prio = prio; for entry in &new_lr_list {
self.ranges[entry.index.index()].bundle = new_bundle;
}
self.bundles[new_bundle.index()].ranges = new_lr_list;
self.recompute_bundle_properties(bundle); self.recompute_bundle_properties(bundle);
self.allocation_queue.insert(bundle, prio as usize); self.recompute_bundle_properties(new_bundle);
for &b in &new_bundles { let prio = self.compute_bundle_prio(bundle);
let prio = self.compute_bundle_prio(b); let new_prio = self.compute_bundle_prio(new_bundle);
self.bundles[b.index()].prio = prio; self.bundles[bundle.index()].prio = prio;
self.recompute_bundle_properties(b); self.bundles[new_bundle.index()].prio = new_prio;
self.allocation_queue.insert(b, prio as usize); self.allocation_queue
} .insert(bundle, prio as usize, reg_hint);
self.allocation_queue
.insert(new_bundle, new_prio as usize, PReg::invalid());
} }
fn process_bundle(&mut self, bundle: LiveBundleIndex) -> Result<(), RegAllocError> { fn process_bundle(
&mut self,
bundle: LiveBundleIndex,
reg_hint: PReg,
) -> Result<(), RegAllocError> {
// Find any requirements: for every LR, for every def/use, gather // Find any requirements: for every LR, for every def/use, gather
// requirements (fixed-reg, any-reg, any) and merge them. // requirements (fixed-reg, any-reg, any) and merge them.
let req = self.compute_requirement(bundle); let req = self.compute_requirement(bundle);
// Grab a hint from our spillset, if any. // Grab a hint from either the queue or our spillset, if any.
let hint_reg = self.spillsets[self.bundles[bundle.index()].spillset.index()].reg_hint; let hint_reg = if reg_hint != PReg::invalid() {
reg_hint
} else {
self.spillsets[self.bundles[bundle.index()].spillset.index()].reg_hint
};
log::debug!( log::debug!(
"process_bundle: bundle {:?} requirement {:?} hint {:?}", "process_bundle: bundle {:?} requirement {:?} hint {:?}",
bundle, bundle,
@@ -3102,12 +2810,12 @@ impl<'a, F: Function> Env<'a, F> {
// Try to allocate! // Try to allocate!
let mut attempts = 0; let mut attempts = 0;
let mut first_conflicting_bundle; let mut split_at_point = self.bundles[bundle.index()].ranges[0].range.from;
let mut requeue_with_reg = PReg::invalid();
loop { loop {
attempts += 1; attempts += 1;
log::debug!("attempt {}, req {:?}", attempts, req); log::debug!("attempt {}, req {:?}", attempts, req);
debug_assert!(attempts < 100 * self.func.insts()); debug_assert!(attempts < 100 * self.func.insts());
first_conflicting_bundle = None;
let req = match req { let req = match req {
Some(r) => r, Some(r) => r,
// `None` means conflicting requirements, hence impossible to // `None` means conflicting requirements, hence impossible to
@@ -3115,7 +2823,8 @@ impl<'a, F: Function> Env<'a, F> {
None => break, None => break,
}; };
let conflicting_bundles = match req { let (conflicting_bundles, latest_first_conflict_point, latest_first_conflict_reg) =
match req {
Requirement::Fixed(preg) => { Requirement::Fixed(preg) => {
let preg_idx = PRegIndex::new(preg.index()); let preg_idx = PRegIndex::new(preg.index());
self.stats.process_bundle_reg_probes_fixed += 1; self.stats.process_bundle_reg_probes_fixed += 1;
@@ -3130,19 +2839,30 @@ impl<'a, F: Function> Env<'a, F> {
} }
AllocRegResult::Conflict(bundles) => { AllocRegResult::Conflict(bundles) => {
log::debug!(" -> conflict with bundles {:?}", bundles); log::debug!(" -> conflict with bundles {:?}", bundles);
bundles let first_bundle = bundles[0];
(
bundles,
self.bundles[first_bundle.index()].ranges[0].range.from,
preg,
)
} }
AllocRegResult::ConflictWithFixed => { AllocRegResult::ConflictWithFixed => {
log::debug!(" -> conflict with fixed alloc"); log::debug!(" -> conflict with fixed alloc");
// Empty conflicts set: there's nothing we can // Empty conflicts set: there's nothing we can
// evict, because fixed conflicts cannot be moved. // evict, because fixed conflicts cannot be moved.
smallvec![] (
smallvec![],
ProgPoint::before(Inst::new(0)),
PReg::invalid(),
)
} }
} }
} }
Requirement::Register(class) => { Requirement::Register(class) => {
// Scan all pregs and attempt to allocate. // Scan all pregs and attempt to allocate.
let mut lowest_cost_conflict_set: Option<LiveBundleVec> = None; let mut lowest_cost_conflict_set: Option<LiveBundleVec> = None;
let mut latest_first_conflict_point = ProgPoint::before(Inst::new(0));
let mut latest_first_conflict_reg = PReg::invalid();
// Heuristic: start the scan for an available // Heuristic: start the scan for an available
// register at an offset influenced both by our // register at an offset influenced both by our
@@ -3178,6 +2898,14 @@ impl<'a, F: Function> Env<'a, F> {
} }
AllocRegResult::Conflict(bundles) => { AllocRegResult::Conflict(bundles) => {
log::debug!(" -> conflict with bundles {:?}", bundles); log::debug!(" -> conflict with bundles {:?}", bundles);
let first_conflict_point =
self.bundles[bundles[0].index()].ranges[0].range.from;
if first_conflict_point > latest_first_conflict_point {
latest_first_conflict_point = first_conflict_point;
latest_first_conflict_reg = preg;
}
if lowest_cost_conflict_set.is_none() { if lowest_cost_conflict_set.is_none() {
lowest_cost_conflict_set = Some(bundles); lowest_cost_conflict_set = Some(bundles);
} else if self.maximum_spill_weight_in_bundle_set(&bundles) } else if self.maximum_spill_weight_in_bundle_set(&bundles)
@@ -3201,7 +2929,11 @@ impl<'a, F: Function> Env<'a, F> {
// the conflicting bundles to evict and retry. Empty list // the conflicting bundles to evict and retry. Empty list
// means nothing to try (due to fixed conflict) so we must // means nothing to try (due to fixed conflict) so we must
// split instead. // split instead.
lowest_cost_conflict_set.unwrap_or(smallvec![]) (
lowest_cost_conflict_set.unwrap_or(smallvec![]),
latest_first_conflict_point,
latest_first_conflict_reg,
)
} }
Requirement::Stack(_) => { Requirement::Stack(_) => {
@@ -3223,6 +2955,11 @@ impl<'a, F: Function> Env<'a, F> {
}; };
log::debug!(" -> conflict set {:?}", conflicting_bundles); log::debug!(" -> conflict set {:?}", conflicting_bundles);
log::debug!(
" -> latest first conflict {:?} with reg {:?}",
latest_first_conflict_point,
latest_first_conflict_reg
);
// If we have already tried evictions once before and are // If we have already tried evictions once before and are
// still unsuccessful, give up and move on to splitting as // still unsuccessful, give up and move on to splitting as
@@ -3236,7 +2973,9 @@ impl<'a, F: Function> Env<'a, F> {
break; break;
} }
first_conflicting_bundle = Some(conflicting_bundles[0]); let bundle_start = self.bundles[bundle.index()].ranges[0].range.from;
split_at_point = std::cmp::max(latest_first_conflict_point, bundle_start);
requeue_with_reg = latest_first_conflict_reg;
// If the maximum spill weight in the conflicting-bundles set is >= this bundle's spill // If the maximum spill weight in the conflicting-bundles set is >= this bundle's spill
// weight, then don't evict. // weight, then don't evict.
@@ -3297,10 +3036,7 @@ impl<'a, F: Function> Env<'a, F> {
} }
assert!(!self.minimal_bundle(bundle)); assert!(!self.minimal_bundle(bundle));
self.split_and_requeue_bundle( self.split_and_requeue_bundle(bundle, split_at_point, requeue_with_reg);
bundle,
first_conflicting_bundle.unwrap_or(LiveBundleIndex::invalid()),
);
Ok(()) Ok(())
} }