Newer
Older
Evan Cheng
committed
/// isReMaterializable - Returns true if every definition of MI of every
/// val# of the specified interval is re-materializable.
bool
LiveIntervals::isReMaterializable(const LiveInterval &li,
const SmallVectorImpl<LiveInterval*> &SpillIs,
bool &isLoad) {
Evan Cheng
committed
isLoad = false;
for (LiveInterval::const_vni_iterator i = li.vni_begin(), e = li.vni_end();
i != e; ++i) {
const VNInfo *VNI = *i;
Evan Cheng
committed
continue; // Dead val#.
// Is the def for the val# rematerializable?
MachineInstr *ReMatDefMI = getInstructionFromIndex(VNI->def);
if (!ReMatDefMI)
return false;
Evan Cheng
committed
bool DefIsLoad = false;
Evan Cheng
committed
if (!ReMatDefMI ||
!isReMaterializable(li, VNI, ReMatDefMI, SpillIs, DefIsLoad))
Evan Cheng
committed
isLoad |= DefIsLoad;
/// FilterFoldedOps - Filter out two-address use operands. Return
/// true if it finds any issue with the operands that ought to prevent
/// folding.
static bool FilterFoldedOps(MachineInstr *MI,
SmallVector<unsigned, 2> &Ops,
unsigned &MRInfo,
SmallVector<unsigned, 2> &FoldOps) {
MRInfo = 0;
for (unsigned i = 0, e = Ops.size(); i != e; ++i) {
unsigned OpIdx = Ops[i];
Evan Cheng
committed
MachineOperand &MO = MI->getOperand(OpIdx);
// FIXME: fold subreg use.
Evan Cheng
committed
if (MO.getSubReg())
return true;
Evan Cheng
committed
if (MO.isDef())
MRInfo |= (unsigned)VirtRegMap::isMod;
else {
// Filter out two-address use operand(s).
if (MI->isRegTiedToDefOperand(OpIdx)) {
MRInfo = VirtRegMap::isModRef;
continue;
}
MRInfo |= (unsigned)VirtRegMap::isRef;
}
FoldOps.push_back(OpIdx);
Evan Cheng
committed
}
return false;
}
/// tryFoldMemoryOperand - Attempts to fold either a spill / restore from
/// slot / to reg or any rematerialized load into ith operand of specified
/// MI. If it is successul, MI is updated with the newly created MI and
/// returns true.
bool LiveIntervals::tryFoldMemoryOperand(MachineInstr* &MI,
VirtRegMap &vrm, MachineInstr *DefMI,
SmallVector<unsigned, 2> &Ops,
bool isSS, int Slot, unsigned Reg) {
// If it is an implicit def instruction, just delete it.
if (MI->isImplicitDef()) {
RemoveMachineInstrFromMaps(MI);
vrm.RemoveMachineInstrFromMaps(MI);
MI->eraseFromParent();
++numFolds;
return true;
}
// Filter the list of operand indexes that are to be folded. Abort if
// any operand will prevent folding.
unsigned MRInfo = 0;
SmallVector<unsigned, 2> FoldOps;
if (FilterFoldedOps(MI, Ops, MRInfo, FoldOps))
return false;
Evan Cheng
committed
Evan Cheng
committed
// The only time it's safe to fold into a two address instruction is when
// it's folding reload and spill from / into a spill stack slot.
if (DefMI && (MRInfo & VirtRegMap::isMod))
Evan Cheng
committed
return false;
Jakob Stoklund Olesen
committed
MachineInstr *fmi = isSS ? tii_->foldMemoryOperand(MI, FoldOps, Slot)
: tii_->foldMemoryOperand(MI, FoldOps, DefMI);
// Remember this instruction uses the spill slot.
if (isSS) vrm.addSpillSlotUse(Slot, fmi);
// Attempt to fold the memory reference into the instruction. If
// we can do this, we don't need to insert spill code.
if (isSS && !mf_->getFrameInfo()->isImmutableObjectIndex(Slot))
vrm.virtFolded(Reg, MI, fmi, (VirtRegMap::ModRef)MRInfo);
vrm.transferRestorePts(MI, fmi);
vrm.transferEmergencySpills(MI, fmi);
Jakob Stoklund Olesen
committed
MI->eraseFromParent();
MI = fmi;
++numFolds;
Evan Cheng
committed
/// canFoldMemoryOperand - Returns true if the specified load / store
/// folding is possible.
bool LiveIntervals::canFoldMemoryOperand(MachineInstr *MI,
SmallVector<unsigned, 2> &Ops,
// Filter the list of operand indexes that are to be folded. Abort if
// any operand will prevent folding.
unsigned MRInfo = 0;
Evan Cheng
committed
SmallVector<unsigned, 2> FoldOps;
if (FilterFoldedOps(MI, Ops, MRInfo, FoldOps))
return false;
Evan Cheng
committed
// It's only legal to remat for a use, not a def.
if (ReMat && (MRInfo & VirtRegMap::isMod))
return false;
Evan Cheng
committed
Evan Cheng
committed
return tii_->canFoldMemoryOperand(MI, FoldOps);
}
bool LiveIntervals::intervalIsInOneMBB(const LiveInterval &li) const {
LiveInterval::Ranges::const_iterator itr = li.ranges.begin();
MachineBasicBlock *mbb = indexes_->getMBBCoveringRange(itr->start, itr->end);
if (mbb == 0)
return false;
for (++itr; itr != li.ranges.end(); ++itr) {
MachineBasicBlock *mbb2 =
indexes_->getMBBCoveringRange(itr->start, itr->end);
if (mbb2 != mbb)
Evan Cheng
committed
/// rewriteImplicitOps - Rewrite implicit use operands of MI (i.e. uses of
/// interval on to-be re-materialized operands of MI) with new register.
void LiveIntervals::rewriteImplicitOps(const LiveInterval &li,
MachineInstr *MI, unsigned NewVReg,
VirtRegMap &vrm) {
// There is an implicit use. That means one of the other operand is
// being remat'ed and the remat'ed instruction has li.reg as an
// use operand. Make sure we rewrite that as well.
for (unsigned i = 0, e = MI->getNumOperands(); i != e; ++i) {
MachineOperand &MO = MI->getOperand(i);
if (!MO.isReg())
Evan Cheng
committed
continue;
unsigned Reg = MO.getReg();
Jakob Stoklund Olesen
committed
if (!TargetRegisterInfo::isVirtualRegister(Reg))
Evan Cheng
committed
continue;
if (!vrm.isReMaterialized(Reg))
continue;
MachineInstr *ReMatMI = vrm.getReMaterializedMI(Reg);
MachineOperand *UseMO = ReMatMI->findRegisterUseOperand(li.reg);
if (UseMO)
UseMO->setReg(NewVReg);
Evan Cheng
committed
}
}
/// rewriteInstructionForSpills, rewriteInstructionsForSpills - Helper functions
/// for addIntervalsForSpills to rewrite uses / defs for the given live range.
Evan Cheng
committed
bool LiveIntervals::
Evan Cheng
committed
rewriteInstructionForSpills(const LiveInterval &li, const VNInfo *VNI,
bool TrySplit, SlotIndex index, SlotIndex end,
MachineInstr *MI,
MachineInstr *ReMatOrigDefMI, MachineInstr *ReMatDefMI,
unsigned Slot, int LdSlot,
bool isLoad, bool isLoadSS, bool DefIsReMat, bool CanDelete,
Evan Cheng
committed
VirtRegMap &vrm,
const TargetRegisterClass* rc,
SmallVector<int, 4> &ReMatIds,
unsigned &NewVReg, unsigned ImpUse, bool &HasDef, bool &HasUse,
DenseMap<unsigned,unsigned> &MBBVRegsMap,
Evan Cheng
committed
std::vector<LiveInterval*> &NewLIs) {
Evan Cheng
committed
bool CanFold = false;
RestartInstruction:
for (unsigned i = 0; i != MI->getNumOperands(); ++i) {
MachineOperand& mop = MI->getOperand(i);
if (!mop.isReg())
Jakob Stoklund Olesen
committed
if (!TargetRegisterInfo::isVirtualRegister(Reg))
continue;
if (Reg != li.reg)
continue;
bool TryFold = !DefIsReMat;
bool FoldSS = true; // Default behavior unless it's a remat.
int FoldSlot = Slot;
if (DefIsReMat) {
// If this is the rematerializable definition MI itself and
// all of its uses are rematerialized, simply delete it.
DEBUG(dbgs() << "\t\t\t\tErasing re-materializable def: "
<< *MI << '\n');
MI->eraseFromParent();
break;
}
// If def for this use can't be rematerialized, then try folding.
// If def is rematerializable and it's a load, also try folding.
TryFold = !ReMatDefMI || (ReMatDefMI && (MI == ReMatOrigDefMI || isLoad));
if (isLoad) {
// Try fold loads (from stack slot, constant pool, etc.) into uses.
FoldSS = isLoadSS;
FoldSlot = LdSlot;
}
}
// Scan all of the operands of this instruction rewriting operands
// to use NewVReg instead of li.reg as appropriate. We do this for
// two reasons:
//
// 1. If the instr reads the same spilled vreg multiple times, we
// want to reuse the NewVReg.
// 2. If the instr is a two-addr instruction, we are required to
// keep the src/dst regs pinned.
//
// Keep track of whether we replace a use and/or def so that we can
// create the spill interval with the appropriate range.
SmallVector<unsigned, 2> Ops;
Jakob Stoklund Olesen
committed
tie(HasUse, HasDef) = MI->readsWritesVirtualRegister(Reg, &Ops);
// Create a new virtual register for the spill interval.
// Create the new register now so we can map the fold instruction
// to the new register so when it is unfolded we get the correct
// answer.
bool CreatedNewVReg = false;
if (NewVReg == 0) {
NewVReg = mri_->createVirtualRegister(rc);
vrm.grow();
CreatedNewVReg = true;
Jakob Stoklund Olesen
committed
// The new virtual register should get the same allocation hints as the
// old one.
std::pair<unsigned, unsigned> Hint = mri_->getRegAllocationHint(Reg);
if (Hint.first || Hint.second)
mri_->setRegAllocationHint(NewVReg, Hint.first, Hint.second);
if (!TryFold)
CanFold = false;
else {
Evan Cheng
committed
// Do not fold load / store here if we are splitting. We'll find an
// optimal point to insert a load / store later.
if (!TrySplit) {
if (tryFoldMemoryOperand(MI, vrm, ReMatDefMI, index,
Evan Cheng
committed
// Folding the load/store can completely change the instruction in
// unpredictable ways, rescan it from the beginning.
if (FoldSS) {
// We need to give the new vreg the same stack slot as the
// spilled interval.
vrm.assignVirt2StackSlot(NewVReg, FoldSlot);
}
Evan Cheng
committed
HasUse = false;
HasDef = false;
CanFold = false;
Evan Cheng
committed
if (isNotInMIMap(MI))
Evan Cheng
committed
break;
Evan Cheng
committed
goto RestartInstruction;
}
} else {
// We'll try to fold it later if it's profitable.
CanFold = canFoldMemoryOperand(MI, Ops, DefIsReMat);
Evan Cheng
committed
}
Evan Cheng
committed
mop.setReg(NewVReg);
Evan Cheng
committed
if (mop.isImplicit())
rewriteImplicitOps(li, MI, NewVReg, vrm);
Evan Cheng
committed
// Reuse NewVReg for other reads.
bool HasEarlyClobber = false;
Evan Cheng
committed
for (unsigned j = 0, e = Ops.size(); j != e; ++j) {
MachineOperand &mopj = MI->getOperand(Ops[j]);
mopj.setReg(NewVReg);
if (mopj.isImplicit())
rewriteImplicitOps(li, MI, NewVReg, vrm);
if (mopj.isEarlyClobber())
HasEarlyClobber = true;
Evan Cheng
committed
}
Evan Cheng
committed
vrm.setVirtIsReMaterialized(NewVReg, ReMatDefMI);
Evan Cheng
committed
if (ReMatIds[VNI->id] == VirtRegMap::MAX_STACK_SLOT) {
Evan Cheng
committed
ReMatIds[VNI->id] = vrm.assignVirtReMatId(NewVReg);
Evan Cheng
committed
vrm.assignVirtReMatId(NewVReg, ReMatIds[VNI->id]);
}
if (!CanDelete || (HasUse && HasDef)) {
// If this is a two-addr instruction then its use operands are
// rematerializable but its def is not. It should be assigned a
// stack slot.
vrm.assignVirt2StackSlot(NewVReg, Slot);
}
} else {
vrm.assignVirt2StackSlot(NewVReg, Slot);
}
} else if (HasUse && HasDef &&
vrm.getStackSlot(NewVReg) == VirtRegMap::NO_STACK_SLOT) {
// If this interval hasn't been assigned a stack slot (because earlier
// def is a deleted remat def), do it now.
assert(Slot != VirtRegMap::NO_STACK_SLOT);
vrm.assignVirt2StackSlot(NewVReg, Slot);
// Re-matting an instruction with virtual register use. Add the
// register as an implicit use on the use MI.
if (DefIsReMat && ImpUse)
MI->addOperand(MachineOperand::CreateReg(ImpUse, false, true));
// Create a new register interval for this spill / remat.
if (CreatedNewVReg) {
NewLIs.push_back(&nI);
MBBVRegsMap.insert(std::make_pair(MI->getParent()->getNumber(), NewVReg));
if (TrySplit)
vrm.setIsSplitFromReg(NewVReg, li.reg);
}
LiveRange LR(index.getLoadIndex(), index.getDefIndex(),
nI.getNextValue(SlotIndex(), 0, VNInfoAllocator));
nI.addRange(LR);
} else {
// Extend the split live interval to this def / use.
LiveRange LR(nI.ranges[nI.ranges.size()-1].end, End,
nI.getValNumInfo(nI.getNumValNums()-1));
// An early clobber starts at the use slot, except for an early clobber
// tied to a use operand (yes, that is a thing).
LiveRange LR(HasEarlyClobber && !HasUse ?
index.getUseIndex() : index.getDefIndex(),
index.getStoreIndex(),
nI.getNextValue(SlotIndex(), 0, VNInfoAllocator));
dbgs() << "\t\t\t\tAdded new interval: ";
nI.print(dbgs(), tri_);
dbgs() << '\n';
Evan Cheng
committed
return CanFold;
bool LiveIntervals::anyKillInMBBAfterIdx(const LiveInterval &li,
const VNInfo *VNI,
MachineBasicBlock *MBB,
return li.killedInRange(Idx.getNextSlot(), getMBBEndIdx(MBB));
/// RewriteInfo - Keep track of machine instrs that will be rewritten
/// during spilling.
namespace {
struct RewriteInfo {
MachineInstr *MI;
Jakob Stoklund Olesen
committed
RewriteInfo(SlotIndex i, MachineInstr *mi) : Index(i), MI(mi) {}
};
struct RewriteInfoCompare {
bool operator()(const RewriteInfo &LHS, const RewriteInfo &RHS) const {
return LHS.Index < RHS.Index;
}
};
}
rewriteInstructionsForSpills(const LiveInterval &li, bool TrySplit,
MachineInstr *ReMatOrigDefMI, MachineInstr *ReMatDefMI,
unsigned Slot, int LdSlot,
bool isLoad, bool isLoadSS, bool DefIsReMat, bool CanDelete,
Evan Cheng
committed
VirtRegMap &vrm,
const TargetRegisterClass* rc,
SmallVector<int, 4> &ReMatIds,
DenseMap<unsigned, std::vector<SRInfo> > &SpillIdxes,
BitVector &RestoreMBBs,
DenseMap<unsigned, std::vector<SRInfo> > &RestoreIdxes,
DenseMap<unsigned,unsigned> &MBBVRegsMap,
Evan Cheng
committed
std::vector<LiveInterval*> &NewLIs) {
Evan Cheng
committed
bool AllCanFold = true;
SlotIndex start = I->start.getBaseIndex();
SlotIndex end = I->end.getPrevSlot().getBaseIndex().getNextIndex();
// First collect all the def / use in this live range that will be rewritten.
Evan Cheng
committed
// Make sure they are sorted according to instruction index.
std::vector<RewriteInfo> RewriteMIs;
Evan Cheng
committed
for (MachineRegisterInfo::reg_iterator ri = mri_->reg_begin(li.reg),
re = mri_->reg_end(); ri != re; ) {
MachineInstr *MI = &*ri;
MachineOperand &O = ri.getOperand();
++ri;
if (MI->isDebugValue()) {
Evan Cheng
committed
// Modify DBG_VALUE now that the value is in a spill slot.
if (Slot != VirtRegMap::MAX_STACK_SLOT || isLoadSS) {
uint64_t Offset = MI->getOperand(1).getImm();
const MDNode *MDPtr = MI->getOperand(2).getMetadata();
DebugLoc DL = MI->getDebugLoc();
int FI = isLoadSS ? LdSlot : (int)Slot;
if (MachineInstr *NewDV = tii_->emitFrameIndexDebugValue(*mf_, FI,
Offset, MDPtr, DL)) {
DEBUG(dbgs() << "Modifying debug info due to spill:" << "\t" << *MI);
ReplaceMachineInstrInMaps(MI, NewDV);
MachineBasicBlock *MBB = MI->getParent();
MBB->insert(MBB->erase(MI), NewDV);
continue;
}
Evan Cheng
committed
}
DEBUG(dbgs() << "Removing debug info due to spill:" << "\t" << *MI);
RemoveMachineInstrFromMaps(MI);
vrm.RemoveMachineInstrFromMaps(MI);
MI->eraseFromParent();
Jakob Stoklund Olesen
committed
assert(!(O.isImplicit() && O.isUse()) &&
"Spilling register that's used as implicit use?");
if (index < start || index >= end)
continue;
Evan Cheng
committed
if (O.isUndef())
Evan Cheng
committed
// Must be defined by an implicit def. It should not be spilled. Note,
// this is for correctness reason. e.g.
// 8 %reg1024<def> = IMPLICIT_DEF
// 12 %reg1024<def> = INSERT_SUBREG %reg1024<kill>, %reg1025, 2
// The live range [12, 14) are not part of the r1024 live interval since
// it's defined by an implicit def. It will not conflicts with live
// interval of r1025. Now suppose both registers are spilled, you can
Evan Cheng
committed
// the INSERT_SUBREG and both target registers that would overlap.
continue;
Jakob Stoklund Olesen
committed
RewriteMIs.push_back(RewriteInfo(index, MI));
}
std::sort(RewriteMIs.begin(), RewriteMIs.end(), RewriteInfoCompare());
unsigned ImpUse = DefIsReMat ? getReMatImplicitUse(li, ReMatDefMI) : 0;
// Now rewrite the defs and uses.
for (unsigned i = 0, e = RewriteMIs.size(); i != e; ) {
RewriteInfo &rwi = RewriteMIs[i];
++i;
MachineInstr *MI = rwi.MI;
// If MI def and/or use the same register multiple times, then there
// are multiple entries.
while (i != e && RewriteMIs[i].MI == MI) {
assert(RewriteMIs[i].Index == index);
++i;
}
if (ImpUse && MI != ReMatDefMI) {
// Re-matting an instruction with virtual register use. Prevent interval
// from being spilled.
getInterval(ImpUse).markNotSpillable();
unsigned MBBId = MBB->getNumber();
Evan Cheng
committed
unsigned ThisVReg = 0;
DenseMap<unsigned,unsigned>::iterator NVI = MBBVRegsMap.find(MBBId);
if (NVI != MBBVRegsMap.end()) {
Evan Cheng
committed
ThisVReg = NVI->second;
// One common case:
// x = use
// ...
// ...
// def = ...
// = use
// It's better to start a new interval to avoid artifically
// extend the new interval.
Jakob Stoklund Olesen
committed
if (MI->readsWritesVirtualRegister(li.reg) ==
std::make_pair(false,true)) {
MBBVRegsMap.erase(MBB->getNumber());
Evan Cheng
committed
ThisVReg = 0;
}
}
Evan Cheng
committed
bool IsNew = ThisVReg == 0;
if (IsNew) {
// This ends the previous live interval. If all of its def / use
// can be folded, give it a low spill weight.
if (NewVReg && TrySplit && AllCanFold) {
LiveInterval &nI = getOrCreateInterval(NewVReg);
nI.weight /= 10.0F;
}
AllCanFold = true;
}
NewVReg = ThisVReg;
Evan Cheng
committed
bool CanFold = rewriteInstructionForSpills(li, I->valno, TrySplit,
index, end, MI, ReMatOrigDefMI, ReMatDefMI,
Slot, LdSlot, isLoad, isLoadSS, DefIsReMat,
CanDelete, vrm, rc, ReMatIds, loopInfo, NewVReg,
Evan Cheng
committed
ImpUse, HasDef, HasUse, MBBVRegsMap, NewLIs);
Evan Cheng
committed
AllCanFold &= CanFold;
// Update weight of spill interval.
LiveInterval &nI = getOrCreateInterval(NewVReg);
// The spill weight is now infinity as it cannot be spilled again.
nI.markNotSpillable();
continue;
}
// Keep track of the last def and first use in each MBB.
if (HasDef) {
if (MI != ReMatOrigDefMI || !CanDelete) {
bool HasKill = false;
if (!HasUse)
HasKill = anyKillInMBBAfterIdx(li, I->valno, MBB, index.getDefIndex());
// If this is a two-address code, then this index starts a new VNInfo.
const VNInfo *VNI = li.findDefinedVNInfoForRegInt(index.getDefIndex());
if (VNI)
HasKill = anyKillInMBBAfterIdx(li, VNI, MBB, index.getDefIndex());
DenseMap<unsigned, std::vector<SRInfo> >::iterator SII =
SpillIdxes.find(MBBId);
if (!HasKill) {
if (SII == SpillIdxes.end()) {
std::vector<SRInfo> S;
S.push_back(SRInfo(index, NewVReg, true));
SpillIdxes.insert(std::make_pair(MBBId, S));
} else if (SII->second.back().vreg != NewVReg) {
SII->second.push_back(SRInfo(index, NewVReg, true));
} else if (index > SII->second.back().index) {
// If there is an earlier def and this is a two-address
// instruction, then it's not possible to fold the store (which
// would also fold the load).
SRInfo &Info = SII->second.back();
Info.index = index;
Info.canFold = !HasUse;
SpillMBBs.set(MBBId);
} else if (SII != SpillIdxes.end() &&
SII->second.back().vreg == NewVReg &&
index > SII->second.back().index) {
// There is an earlier def that's not killed (must be two-address).
// The spill is no longer needed.
SII->second.pop_back();
if (SII->second.empty()) {
SpillIdxes.erase(MBBId);
SpillMBBs.reset(MBBId);
}
if (HasUse) {
DenseMap<unsigned, std::vector<SRInfo> >::iterator SII =
SpillIdxes.find(MBBId);
if (SII != SpillIdxes.end() &&
SII->second.back().vreg == NewVReg &&
index > SII->second.back().index)
// Use(s) following the last def, it's not safe to fold the spill.
SII->second.back().canFold = false;
DenseMap<unsigned, std::vector<SRInfo> >::iterator RII =
RestoreIdxes.find(MBBId);
if (RII != RestoreIdxes.end() && RII->second.back().vreg == NewVReg)
// If we are splitting live intervals, only fold if it's the first
// use and there isn't another use later in the MBB.
RII->second.back().canFold = false;
else if (IsNew) {
// Only need a reload if there isn't an earlier def / use.
if (RII == RestoreIdxes.end()) {
std::vector<SRInfo> Infos;
Infos.push_back(SRInfo(index, NewVReg, true));
RestoreIdxes.insert(std::make_pair(MBBId, Infos));
} else {
RII->second.push_back(SRInfo(index, NewVReg, true));
}
RestoreMBBs.set(MBBId);
}
// Update spill weight.
unsigned loopDepth = loopInfo->getLoopDepth(MBB);
nI.weight += getSpillWeight(HasDef, HasUse, loopDepth);
Evan Cheng
committed
if (NewVReg && TrySplit && AllCanFold) {
// If all of its def / use can be folded, give it a low spill weight.
LiveInterval &nI = getOrCreateInterval(NewVReg);
nI.weight /= 10.0F;
}
bool LiveIntervals::alsoFoldARestore(int Id, SlotIndex index,
unsigned vr, BitVector &RestoreMBBs,
DenseMap<unsigned,std::vector<SRInfo> > &RestoreIdxes) {
if (!RestoreMBBs[Id])
return false;
std::vector<SRInfo> &Restores = RestoreIdxes[Id];
for (unsigned i = 0, e = Restores.size(); i != e; ++i)
if (Restores[i].index == index &&
Restores[i].vreg == vr &&
Restores[i].canFold)
return true;
return false;
}
void LiveIntervals::eraseRestoreInfo(int Id, SlotIndex index,
unsigned vr, BitVector &RestoreMBBs,
DenseMap<unsigned,std::vector<SRInfo> > &RestoreIdxes) {
if (!RestoreMBBs[Id])
return;
std::vector<SRInfo> &Restores = RestoreIdxes[Id];
for (unsigned i = 0, e = Restores.size(); i != e; ++i)
if (Restores[i].index == index && Restores[i].vreg)
}
Evan Cheng
committed
/// handleSpilledImpDefs - Remove IMPLICIT_DEF instructions which are being
/// spilled and create empty intervals for their uses.
void
LiveIntervals::handleSpilledImpDefs(const LiveInterval &li, VirtRegMap &vrm,
const TargetRegisterClass* rc,
std::vector<LiveInterval*> &NewLIs) {
for (MachineRegisterInfo::reg_iterator ri = mri_->reg_begin(li.reg),
re = mri_->reg_end(); ri != re; ) {
Evan Cheng
committed
MachineOperand &O = ri.getOperand();
MachineInstr *MI = &*ri;
++ri;
if (MI->isDebugValue()) {
// Remove debug info for now.
O.setReg(0U);
DEBUG(dbgs() << "Removing debug info due to spill:" << "\t" << *MI);
continue;
}
Evan Cheng
committed
if (O.isDef()) {
assert(MI->isImplicitDef() &&
Evan Cheng
committed
"Register def was not rewritten?");
RemoveMachineInstrFromMaps(MI);
vrm.RemoveMachineInstrFromMaps(MI);
MI->eraseFromParent();
} else {
// This must be an use of an implicit_def so it's not part of the live
// interval. Create a new empty live interval for it.
// FIXME: Can we simply erase some of the instructions? e.g. Stores?
unsigned NewVReg = mri_->createVirtualRegister(rc);
vrm.grow();
vrm.setIsImplicitlyDefined(NewVReg);
NewLIs.push_back(&getOrCreateInterval(NewVReg));
for (unsigned i = 0, e = MI->getNumOperands(); i != e; ++i) {
MachineOperand &MO = MI->getOperand(i);
if (MO.isReg() && MO.getReg() == li.reg) {
Evan Cheng
committed
MO.setReg(NewVReg);
MO.setIsUndef();
}
Evan Cheng
committed
}
}
}
}
float
LiveIntervals::getSpillWeight(bool isDef, bool isUse, unsigned loopDepth) {
// Limit the loop depth ridiculousness.
if (loopDepth > 200)
loopDepth = 200;
// The loop depth is used to roughly estimate the number of times the
// instruction is executed. Something like 10^d is simple, but will quickly
// overflow a float. This expression behaves like 10^d for small d, but is
// more tempered for large d. At d=200 we get 6.7e33 which leaves a bit of
// headroom before overflow.
float lc = std::pow(1 + (100.0f / (loopDepth+10)), (float)loopDepth);
return (isDef + isUse) * lc;
}
Jakob Stoklund Olesen
committed
static void normalizeSpillWeights(std::vector<LiveInterval*> &NewLIs) {
Jakob Stoklund Olesen
committed
for (unsigned i = 0, e = NewLIs.size(); i != e; ++i)
Jakob Stoklund Olesen
committed
NewLIs[i]->weight =
normalizeSpillWeight(NewLIs[i]->weight, NewLIs[i]->getSize());
Jakob Stoklund Olesen
committed
}
const SmallVectorImpl<LiveInterval*> &SpillIs,
Evan Cheng
committed
const MachineLoopInfo *loopInfo, VirtRegMap &vrm) {
assert(li.isSpillable() && "attempt to spill already spilled interval!");
dbgs() << "\t\t\t\tadding intervals for spills for interval: ";
li.print(dbgs(), tri_);
dbgs() << '\n';
DenseMap<unsigned, std::vector<SRInfo> > SpillIdxes;
BitVector RestoreMBBs(mf_->getNumBlockIDs());
DenseMap<unsigned, std::vector<SRInfo> > RestoreIdxes;
DenseMap<unsigned,unsigned> MBBVRegsMap;
Evan Cheng
committed
const TargetRegisterClass* rc = mri_->getRegClass(li.reg);
unsigned NumValNums = li.getNumValNums();
SmallVector<MachineInstr*, 4> ReMatDefs;
ReMatDefs.resize(NumValNums, NULL);
SmallVector<MachineInstr*, 4> ReMatOrigDefs;
ReMatOrigDefs.resize(NumValNums, NULL);
SmallVector<int, 4> ReMatIds;
ReMatIds.resize(NumValNums, VirtRegMap::MAX_STACK_SLOT);
BitVector ReMatDelete(NumValNums);
unsigned Slot = VirtRegMap::MAX_STACK_SLOT;
// Spilling a split live interval. It cannot be split any further. Also,
// it's also guaranteed to be a single val# / range interval.
if (vrm.getPreSplitReg(li.reg)) {
vrm.setIsSplitFromReg(li.reg, 0);
// Unset the split kill marker on the last use.
SlotIndex KillIdx = vrm.getKillPoint(li.reg);
if (KillIdx != SlotIndex()) {
MachineInstr *KillMI = getInstructionFromIndex(KillIdx);
assert(KillMI && "Last use disappeared?");
int KillOp = KillMI->findRegisterUseOperandIdx(li.reg, true);
assert(KillOp != -1 && "Last use disappeared?");
KillMI->getOperand(KillOp).setIsKill(false);
Evan Cheng
committed
vrm.removeKillPoint(li.reg);
bool DefIsReMat = vrm.isReMaterialized(li.reg);
Slot = vrm.getStackSlot(li.reg);
assert(Slot != VirtRegMap::MAX_STACK_SLOT);
MachineInstr *ReMatDefMI = DefIsReMat ?
vrm.getReMaterializedMI(li.reg) : NULL;
int LdSlot = 0;
bool isLoadSS = DefIsReMat && tii_->isLoadFromStackSlot(ReMatDefMI, LdSlot);
bool isLoad = isLoadSS ||
(DefIsReMat && (ReMatDefMI->getDesc().canFoldAsLoad()));
bool IsFirstRange = true;
for (LiveInterval::Ranges::const_iterator
I = li.ranges.begin(), E = li.ranges.end(); I != E; ++I) {
// If this is a split live interval with multiple ranges, it means there
// are two-address instructions that re-defined the value. Only the
// first def can be rematerialized!
if (IsFirstRange) {
// Note ReMatOrigDefMI has already been deleted.
rewriteInstructionsForSpills(li, false, I, NULL, ReMatDefMI,
Slot, LdSlot, isLoad, isLoadSS, DefIsReMat,
Evan Cheng
committed
false, vrm, rc, ReMatIds, loopInfo,
SpillMBBs, SpillIdxes, RestoreMBBs, RestoreIdxes,
Evan Cheng
committed
MBBVRegsMap, NewLIs);
} else {
rewriteInstructionsForSpills(li, false, I, NULL, 0,
Slot, 0, false, false, false,
Evan Cheng
committed
false, vrm, rc, ReMatIds, loopInfo,
SpillMBBs, SpillIdxes, RestoreMBBs, RestoreIdxes,
Evan Cheng
committed
MBBVRegsMap, NewLIs);
Evan Cheng
committed
handleSpilledImpDefs(li, vrm, rc, NewLIs);
Jakob Stoklund Olesen
committed
normalizeSpillWeights(NewLIs);
Evan Cheng
committed
bool TrySplit = !intervalIsInOneMBB(li);
if (TrySplit)
++numSplits;
bool NeedStackSlot = false;
for (LiveInterval::const_vni_iterator i = li.vni_begin(), e = li.vni_end();
i != e; ++i) {
const VNInfo *VNI = *i;
unsigned VN = VNI->id;
continue; // Dead val#.
// Is the def for the val# rematerializable?
MachineInstr *ReMatDefMI = getInstructionFromIndex(VNI->def);
Evan Cheng
committed
bool dummy;
if (ReMatDefMI && isReMaterializable(li, VNI, ReMatDefMI, SpillIs, dummy)) {
// Original def may be modified so we have to make a copy here.
MachineInstr *Clone = mf_->CloneMachineInstr(ReMatDefMI);
Evan Cheng
committed
CloneMIs.push_back(Clone);
// A kill is a phi node, not all of its uses can be rematerialized.
CanDelete = false;
// Need a stack slot if there is any live range where uses cannot be
// rematerialized.
NeedStackSlot = true;
}
if (CanDelete)
ReMatDelete.set(VN);
} else {
// Need a stack slot if there is any live range where uses cannot be
// rematerialized.
NeedStackSlot = true;
}
}
// One stack slot per live interval.
Owen Anderson
committed
if (NeedStackSlot && vrm.getPreSplitReg(li.reg) == 0) {
if (vrm.getStackSlot(li.reg) == VirtRegMap::NO_STACK_SLOT)
Slot = vrm.assignVirt2StackSlot(li.reg);
Owen Anderson
committed
// This case only occurs when the prealloc splitter has already assigned
// a stack slot to this vreg.
else
Slot = vrm.getStackSlot(li.reg);
}
// Create new intervals and rewrite defs and uses.
for (LiveInterval::Ranges::const_iterator
I = li.ranges.begin(), E = li.ranges.end(); I != E; ++I) {
MachineInstr *ReMatDefMI = ReMatDefs[I->valno->id];
MachineInstr *ReMatOrigDefMI = ReMatOrigDefs[I->valno->id];
bool DefIsReMat = ReMatDefMI != NULL;
bool CanDelete = ReMatDelete[I->valno->id];
int LdSlot = 0;
bool isLoadSS = DefIsReMat && tii_->isLoadFromStackSlot(ReMatDefMI, LdSlot);
(DefIsReMat && ReMatDefMI->getDesc().canFoldAsLoad());
rewriteInstructionsForSpills(li, TrySplit, I, ReMatOrigDefMI, ReMatDefMI,
Slot, LdSlot, isLoad, isLoadSS, DefIsReMat,
Evan Cheng
committed
CanDelete, vrm, rc, ReMatIds, loopInfo,
SpillMBBs, SpillIdxes, RestoreMBBs, RestoreIdxes,
Evan Cheng
committed
MBBVRegsMap, NewLIs);
// Insert spills / restores if we are splitting.
if (!TrySplit) {
Evan Cheng
committed
handleSpilledImpDefs(li, vrm, rc, NewLIs);
Jakob Stoklund Olesen
committed
normalizeSpillWeights(NewLIs);
return NewLIs;
SmallPtrSet<LiveInterval*, 4> AddedKill;
SmallVector<unsigned, 2> Ops;
if (NeedStackSlot) {
int Id = SpillMBBs.find_first();
while (Id != -1) {
std::vector<SRInfo> &spills = SpillIdxes[Id];
for (unsigned i = 0, e = spills.size(); i != e; ++i) {
unsigned VReg = spills[i].vreg;
LiveInterval &nI = getOrCreateInterval(VReg);
bool isReMat = vrm.isReMaterialized(VReg);
MachineInstr *MI = getInstructionFromIndex(index);
bool CanFold = false;
bool FoundUse = false;
Ops.clear();
Evan Cheng
committed
if (spills[i].canFold) {
CanFold = true;
for (unsigned j = 0, ee = MI->getNumOperands(); j != ee; ++j) {
MachineOperand &MO = MI->getOperand(j);
if (!MO.isReg() || MO.getReg() != VReg)
continue;
Ops.push_back(j);
if (MO.isDef())
Evan Cheng
committed
continue;
(!FoundUse && !alsoFoldARestore(Id, index, VReg,
RestoreMBBs, RestoreIdxes))) {
// MI has two-address uses of the same register. If the use
// isn't the first and only use in the BB, then we can't fold
// it. FIXME: Move this to rewriteInstructionsForSpills.
CanFold = false;
Evan Cheng
committed
break;
}
FoundUse = true;
}
}
// Fold the store into the def if possible.
Evan Cheng
committed
bool Folded = false;
if (CanFold && !Ops.empty()) {
if (tryFoldMemoryOperand(MI, vrm, NULL, index, Ops, true, Slot,VReg)){
Evan Cheng
committed
Folded = true;
// Also folded uses, do not issue a load.
eraseRestoreInfo(Id, index, VReg, RestoreMBBs, RestoreIdxes);
nI.removeRange(index.getLoadIndex(), index.getDefIndex());
nI.removeRange(index.getDefIndex(), index.getStoreIndex());
Evan Cheng
committed
}
Evan Cheng
committed
// Otherwise tell the spiller to issue a spill.
if (!Folded) {
LiveRange *LR = &nI.ranges[nI.ranges.size()-1];
if (!MI->registerDefIsDead(nI.reg))
// No need to spill a dead def.
vrm.addSpillPoint(VReg, isKill, MI);
if (isKill)
AddedKill.insert(&nI);
}
Id = SpillMBBs.find_next(Id);
}
int Id = RestoreMBBs.find_first();
while (Id != -1) {
std::vector<SRInfo> &restores = RestoreIdxes[Id];
for (unsigned i = 0, e = restores.size(); i != e; ++i) {
SlotIndex index = restores[i].index;
if (index == SlotIndex())
continue;
unsigned VReg = restores[i].vreg;
LiveInterval &nI = getOrCreateInterval(VReg);
bool isReMat = vrm.isReMaterialized(VReg);
MachineInstr *MI = getInstructionFromIndex(index);
bool CanFold = false;
Ops.clear();
Evan Cheng
committed
if (restores[i].canFold) {
CanFold = true;
for (unsigned j = 0, ee = MI->getNumOperands(); j != ee; ++j) {
MachineOperand &MO = MI->getOperand(j);
if (!MO.isReg() || MO.getReg() != VReg)
if (MO.isDef()) {
// If this restore were to be folded, it would have been folded
// already.
CanFold = false;
Ops.push_back(j);
// Fold the load into the use if possible.
Evan Cheng
committed
bool Folded = false;
if (CanFold && !Ops.empty()) {
Folded = tryFoldMemoryOperand(MI, vrm, NULL,index,Ops,true,Slot,VReg);
else {
MachineInstr *ReMatDefMI = vrm.getReMaterializedMI(VReg);
int LdSlot = 0;
bool isLoadSS = tii_->isLoadFromStackSlot(ReMatDefMI, LdSlot);
// If the rematerializable def is a load, also try to fold it.
if (isLoadSS || ReMatDefMI->getDesc().canFoldAsLoad())
Folded = tryFoldMemoryOperand(MI, vrm, ReMatDefMI, index,
Ops, isLoadSS, LdSlot, VReg);
if (!Folded) {
unsigned ImpUse = getReMatImplicitUse(li, ReMatDefMI);
if (ImpUse) {
// Re-matting an instruction with virtual register use. Add the
// register as an implicit use on the use MI and mark the register
// interval as unspillable.
LiveInterval &ImpLi = getInterval(ImpUse);
ImpLi.markNotSpillable();
MI->addOperand(MachineOperand::CreateReg(ImpUse, false, true));
}
Evan Cheng
committed
}
}
// If folding is not possible / failed, then tell the spiller to issue a
// load / rematerialization for us.
nI.removeRange(index.getLoadIndex(), index.getDefIndex());
vrm.addRestorePoint(VReg, MI);