/// This has to be enabled in combination with shouldTrackPressure().
virtual bool shouldTrackLaneMasks() const { return false; }
+ // If this method returns true, handling of the scheduling regions
+ // themselves (in case of a scheduling boundary in MBB) will be done
+ // beginning with the topmost region of MBB.
+ virtual bool doMBBSchedRegionsTopDown() const { return false; }
+
/// Initialize the strategy after building the DAG for a new region.
virtual void initialize(ScheduleDAGMI *DAG) = 0;
+ /// Tell the strategy that MBB is about to be processed.
+ virtual void enterMBB(MachineBasicBlock *MBB) {};
+
+ /// Tell the strategy that current MBB is done.
+ virtual void leaveMBB() {};
+
/// Notify this strategy that all roots have been released (including those
/// that depend on EntrySU or ExitSU).
virtual void registerRoots() {}
// Provide a vtable anchor
~ScheduleDAGMI() override;
+ /// If this method returns true, handling of the scheduling regions
+ /// themselves (in case of a scheduling boundary in MBB) will be done
+ /// beginning with the topmost region of MBB.
+ bool doMBBSchedRegionsTopDown() const override {
+ return SchedImpl->doMBBSchedRegionsTopDown();
+ }
+
// Returns LiveIntervals instance for use in DAG mutators and such.
LiveIntervals *getLIS() const { return LIS; }
/// reorderable instructions.
void schedule() override;
+ void startBlock(MachineBasicBlock *bb) override;
+ void finishBlock() override;
+
/// Change the position of an instruction within the basic block and update
/// live ranges and region boundary iterators.
void moveInstruction(MachineInstr *MI, MachineBasicBlock::iterator InsertPos);
/// Returns an existing SUnit for this MI, or nullptr.
SUnit *getSUnit(MachineInstr *MI) const;
+ /// If this method returns true, handling of the scheduling regions
+ /// themselves (in case of a scheduling boundary in MBB) will be done
+ /// beginning with the topmost region of MBB.
+ virtual bool doMBBSchedRegionsTopDown() const { return false; }
+
/// Prepares to perform scheduling in the given block.
virtual void startBlock(MachineBasicBlock *BB);
// Initialize the context of the pass.
MF = &mf;
+ MLI = &getAnalysis<MachineLoopInfo>();
PassConfig = &getAnalysis<TargetPassConfig>();
if (VerifyScheduling)
return MI->isCall() || TII->isSchedulingBoundary(*MI, MBB, *MF);
}
+/// A region of an MBB for scheduling.
+struct SchedRegion {
+ /// RegionBegin is the first instruction in the scheduling region, and
+ /// RegionEnd is either MBB->end() or the scheduling boundary after the
+ /// last instruction in the scheduling region. These iterators cannot refer
+ /// to instructions outside of the identified scheduling region because
+ /// those may be reordered before scheduling this region.
+ MachineBasicBlock::iterator RegionBegin;
+ MachineBasicBlock::iterator RegionEnd;
+ unsigned NumRegionInstrs;
+ SchedRegion(MachineBasicBlock::iterator B, MachineBasicBlock::iterator E,
+ unsigned N) :
+ RegionBegin(B), RegionEnd(E), NumRegionInstrs(N) {}
+};
+
+typedef SmallVector<SchedRegion, 16> MBBRegionsVector;
+static void
+getSchedRegions(MachineBasicBlock *MBB,
+ MBBRegionsVector &Regions,
+ bool RegionsTopDown) {
+ MachineFunction *MF = MBB->getParent();
+ const TargetInstrInfo *TII = MF->getSubtarget().getInstrInfo();
+
+ MachineBasicBlock::iterator I = nullptr;
+ for(MachineBasicBlock::iterator RegionEnd = MBB->end();
+ RegionEnd != MBB->begin(); RegionEnd = I) {
+
+ // Avoid decrementing RegionEnd for blocks with no terminator.
+ if (RegionEnd != MBB->end() ||
+ isSchedBoundary(&*std::prev(RegionEnd), &*MBB, MF, TII)) {
+ --RegionEnd;
+ }
+
+ // The next region starts above the previous region. Look backward in the
+ // instruction stream until we find the nearest boundary.
+ unsigned NumRegionInstrs = 0;
+ I = RegionEnd;
+ for (;I != MBB->begin(); --I) {
+ MachineInstr &MI = *std::prev(I);
+ if (isSchedBoundary(&MI, &*MBB, MF, TII))
+ break;
+ if (!MI.isDebugValue())
+ // MBB::size() uses instr_iterator to count. Here we need a bundle to
+ // count as a single instruction.
+ ++NumRegionInstrs;
+ }
+
+ Regions.push_back(SchedRegion(I, RegionEnd, NumRegionInstrs));
+ }
+
+ if (RegionsTopDown)
+ std::reverse(Regions.begin(), Regions.end());
+}
+
/// Main driver for both MachineScheduler and PostMachineScheduler.
void MachineSchedulerBase::scheduleRegions(ScheduleDAGInstrs &Scheduler,
bool FixKillFlags) {
- const TargetInstrInfo *TII = MF->getSubtarget().getInstrInfo();
-
// Visit all machine basic blocks.
//
// TODO: Visit blocks in global postorder or postorder within the bottom-up
continue;
#endif
- // Break the block into scheduling regions [I, RegionEnd), and schedule each
- // region as soon as it is discovered. RegionEnd points the scheduling
- // boundary at the bottom of the region. The DAG does not include RegionEnd,
- // but the region does (i.e. the next RegionEnd is above the previous
- // RegionBegin). If the current block has no terminator then RegionEnd ==
- // MBB->end() for the bottom region.
+ // Break the block into scheduling regions [I, RegionEnd). RegionEnd
+ // points to the scheduling boundary at the bottom of the region. The DAG
+ // does not include RegionEnd, but the region does (i.e. the next
+ // RegionEnd is above the previous RegionBegin). If the current block has
+ // no terminator then RegionEnd == MBB->end() for the bottom region.
+ //
+ // All the regions of MBB are first found and stored in MBBRegions, which
+ // will be processed (MBB) top-down if initialized with true.
//
// The Scheduler may insert instructions during either schedule() or
// exitRegion(), even for empty regions. So the local iterators 'I' and
- // 'RegionEnd' are invalid across these calls.
- //
- // MBB::size() uses instr_iterator to count. Here we need a bundle to count
- // as a single instruction.
- for(MachineBasicBlock::iterator RegionEnd = MBB->end();
- RegionEnd != MBB->begin(); RegionEnd = Scheduler.begin()) {
-
- // Avoid decrementing RegionEnd for blocks with no terminator.
- if (RegionEnd != MBB->end() ||
- isSchedBoundary(&*std::prev(RegionEnd), &*MBB, MF, TII)) {
- --RegionEnd;
- }
+ // 'RegionEnd' are invalid across these calls. Instructions must not be
+ // added to other regions than the current one without updating MBBRegions.
+
+ MBBRegionsVector MBBRegions;
+ getSchedRegions(&*MBB, MBBRegions, Scheduler.doMBBSchedRegionsTopDown());
+ for (MBBRegionsVector::iterator R = MBBRegions.begin();
+ R != MBBRegions.end(); ++R) {
+ MachineBasicBlock::iterator I = R->RegionBegin;
+ MachineBasicBlock::iterator RegionEnd = R->RegionEnd;
+ unsigned NumRegionInstrs = R->NumRegionInstrs;
- // The next region starts above the previous region. Look backward in the
- // instruction stream until we find the nearest boundary.
- unsigned NumRegionInstrs = 0;
- MachineBasicBlock::iterator I = RegionEnd;
- for (; I != MBB->begin(); --I) {
- MachineInstr &MI = *std::prev(I);
- if (isSchedBoundary(&MI, &*MBB, MF, TII))
- break;
- if (!MI.isDebugValue())
- ++NumRegionInstrs;
- }
// Notify the scheduler of the region, even if we may skip scheduling
// it. Perhaps it still needs to be bundled.
Scheduler.enterRegion(&*MBB, I, RegionEnd, NumRegionInstrs);
}
// Schedule a region: possibly reorder instructions.
- // This invalidates 'RegionEnd' and 'I'.
+ // This invalidates the original region iterators.
Scheduler.schedule();
// Close the current region.
Scheduler.exitRegion();
-
- // Scheduling has invalidated the current iterator 'I'. Ask the
- // scheduler for the top of it's scheduled region.
- RegionEnd = Scheduler.begin();
}
Scheduler.finishBlock();
// FIXME: Ideally, no further passes should rely on kill flags. However,
releasePred(SU, &Pred);
}
+void ScheduleDAGMI::startBlock(MachineBasicBlock *bb) {
+ ScheduleDAGInstrs::startBlock(bb);
+ SchedImpl->enterMBB(bb);
+}
+
+void ScheduleDAGMI::finishBlock() {
+ SchedImpl->leaveMBB();
+ ScheduleDAGInstrs::finishBlock();
+}
+
/// enterRegion - Called back from MachineScheduler::runOnMachineFunction after
/// crossing a scheduling boundary. [begin, end) includes all instructions in
/// the region, including the boundary itself and single-instruction regions
// * Processor resources usage. It is beneficial to balance the use of
// resources.
//
+// A goal is to consider all instructions, also those outside of any
+// scheduling region. Such instructions are "advanced" past and include
+// single instructions before a scheduling region, branches etc.
+//
+// A block that has only one predecessor continues scheduling with the state
+// of it (which may be updated by emitting branches).
+//
// ===---------------------------------------------------------------------===//
#include "SystemZHazardRecognizer.h"
"resources during scheduling."),
cl::init(8));
-SystemZHazardRecognizer::
-SystemZHazardRecognizer(const MachineSchedContext *C) : DAG(nullptr),
- SchedModel(nullptr) {}
-
unsigned SystemZHazardRecognizer::
getNumDecoderSlots(SUnit *SU) const {
- const MCSchedClassDesc *SC = DAG->getSchedClass(SU);
+ const MCSchedClassDesc *SC = getSchedClass(SU);
if (!SC->isValid())
return 0; // IMPLICIT_DEF / KILL -- will not make impact in output.
clearProcResCounters();
GrpCount = 0;
LastFPdOpCycleIdx = UINT_MAX;
+ LastEmittedMI = nullptr;
DEBUG(CurGroupDbg = "";);
}
bool
SystemZHazardRecognizer::fitsIntoCurrentGroup(SUnit *SU) const {
- const MCSchedClassDesc *SC = DAG->getSchedClass(SU);
+ const MCSchedClassDesc *SC = getSchedClass(SU);
if (!SC->isValid())
return true;
#ifndef NDEBUG // Debug output
void SystemZHazardRecognizer::dumpSU(SUnit *SU, raw_ostream &OS) const {
OS << "SU(" << SU->NodeNum << "):";
- OS << SchedModel->getInstrInfo()->getName(SU->getInstr()->getOpcode());
+ OS << TII->getName(SU->getInstr()->getOpcode());
- const MCSchedClassDesc *SC = DAG->getSchedClass(SU);
+ const MCSchedClassDesc *SC = getSchedClass(SU);
if (!SC->isValid())
return;
CriticalResourceIdx = UINT_MAX;
}
+static inline bool isBranchRetTrap(MachineInstr *MI) {
+ return (MI->isBranch() || MI->isReturn() ||
+ MI->getOpcode() == SystemZ::CondTrap);
+}
+
// Update state with SU as the next scheduled unit.
void SystemZHazardRecognizer::
EmitInstruction(SUnit *SU) {
- const MCSchedClassDesc *SC = DAG->getSchedClass(SU);
+ const MCSchedClassDesc *SC = getSchedClass(SU);
DEBUG( dumpCurrGroup("Decode group before emission"););
// If scheduling an SU that must begin a new decoder group, move on
cgd << ", ";
dumpSU(SU, cgd););
+ LastEmittedMI = SU->getInstr();
+
// After returning from a call, we don't know much about the state.
- if (SU->getInstr()->isCall()) {
+ if (SU->isCall) {
DEBUG (dbgs() << "+++ Clearing state after call.\n";);
clearProcResCounters();
LastFPdOpCycleIdx = UINT_MAX;
<< LastFPdOpCycleIdx << "\n";);
}
+ bool GroupEndingBranch =
+ (CurrGroupSize >= 1 && isBranchRetTrap(SU->getInstr()));
+
// Insert SU into current group by increasing number of slots used
// in current group.
CurrGroupSize += getNumDecoderSlots(SU);
// Check if current group is now full/ended. If so, move on to next
// group to be ready to evaluate more candidates.
- if (CurrGroupSize == 3 || SC->EndGroup)
+ if (CurrGroupSize == 3 || SC->EndGroup || GroupEndingBranch)
nextGroup();
}
int SystemZHazardRecognizer::groupingCost(SUnit *SU) const {
- const MCSchedClassDesc *SC = DAG->getSchedClass(SU);
+ const MCSchedClassDesc *SC = getSchedClass(SU);
if (!SC->isValid())
return 0;
resourcesCost(SUnit *SU) {
int Cost = 0;
- const MCSchedClassDesc *SC = DAG->getSchedClass(SU);
+ const MCSchedClassDesc *SC = getSchedClass(SU);
if (!SC->isValid())
return 0;
return Cost;
}
+void SystemZHazardRecognizer::emitInstruction(MachineInstr *MI,
+ bool TakenBranch) {
+ // Make a temporary SUnit.
+ SUnit SU(MI, 0);
+
+ // Set interesting flags.
+ SU.isCall = MI->isCall();
+
+ const MCSchedClassDesc *SC = SchedModel->resolveSchedClass(MI);
+ for (const MCWriteProcResEntry &PRE :
+ make_range(SchedModel->getWriteProcResBegin(SC),
+ SchedModel->getWriteProcResEnd(SC))) {
+ switch (SchedModel->getProcResource(PRE.ProcResourceIdx)->BufferSize) {
+ case 0:
+ SU.hasReservedResource = true;
+ break;
+ case 1:
+ SU.isUnbuffered = true;
+ break;
+ default:
+ break;
+ }
+ }
+
+ EmitInstruction(&SU);
+
+ if (TakenBranch && CurrGroupSize > 0)
+ nextGroup(false /*DbgOutput*/);
+
+ assert ((!MI->isTerminator() || isBranchRetTrap(MI)) &&
+ "Scheduler: unhandled terminator!");
+}
+
+void SystemZHazardRecognizer::
+copyState(SystemZHazardRecognizer *Incoming) {
+ // Current decoder group
+ CurrGroupSize = Incoming->CurrGroupSize;
+ DEBUG (CurGroupDbg = Incoming->CurGroupDbg;);
+
+ // Processor resources
+ ProcResourceCounters = Incoming->ProcResourceCounters;
+ CriticalResourceIdx = Incoming->CriticalResourceIdx;
+
+ // FPd
+ LastFPdOpCycleIdx = Incoming->LastFPdOpCycleIdx;
+ GrpCount = Incoming->GrpCount;
+}
// * Processor resources usage. It is beneficial to balance the use of
// resources.
//
+// A goal is to consider all instructions, also those outside of any
+// scheduling region. Such instructions are "advanced" past and include
+// single instructions before a scheduling region, branches etc.
+//
+// A block that has only one predecessor continues scheduling with the state
+// of it (which may be updated by emitting branches).
+//
// ===---------------------------------------------------------------------===//
#ifndef LLVM_LIB_TARGET_SYSTEMZ_SYSTEMZHAZARDRECOGNIZER_H
namespace llvm {
-/// SystemZHazardRecognizer maintains the state during scheduling.
+/// SystemZHazardRecognizer maintains the state for one MBB during scheduling.
class SystemZHazardRecognizer : public ScheduleHazardRecognizer {
- ScheduleDAGMI *DAG;
+ const SystemZInstrInfo *TII;
const TargetSchedModel *SchedModel;
/// Keep track of the number of decoder slots used in the current
/// ops, return true if it seems good to schedule an FPd op next.
bool isFPdOpPreferred_distance(const SUnit *SU);
+ /// Last emitted instruction or nullptr.
+ MachineInstr *LastEmittedMI;
+
public:
- SystemZHazardRecognizer(const MachineSchedContext *C);
+ SystemZHazardRecognizer(const SystemZInstrInfo *tii,
+ const TargetSchedModel *SM)
+ : TII(tii), SchedModel(SM) { Reset(); }
- void setDAG(ScheduleDAGMI *dag) {
- DAG = dag;
- SchedModel = dag->getSchedModel();
- }
-
HazardType getHazardType(SUnit *m, int Stalls = 0) override;
void Reset() override;
void EmitInstruction(SUnit *SU) override;
+ /// Resolves and cache a resolved scheduling class for an SUnit.
+ const MCSchedClassDesc *getSchedClass(SUnit *SU) const {
+ if (!SU->SchedClass && SchedModel->hasInstrSchedModel())
+ SU->SchedClass = SchedModel->resolveSchedClass(SU->getInstr());
+ return SU->SchedClass;
+ }
+
+ /// Wrap a non-scheduled instruction in an SU and emit it.
+ void emitInstruction(MachineInstr *MI, bool TakenBranch = false);
+
// Cost functions used by SystemZPostRASchedStrategy while
// evaluating candidates.
void dumpCurrGroup(std::string Msg = "") const;
void dumpProcResourceCounters() const;
#endif
+
+ MachineBasicBlock::iterator getLastEmittedMI() { return LastEmittedMI; }
+
+ /// Copy counters from end of single predecessor.
+ void copyState(SystemZHazardRecognizer *Incoming);
};
} // namespace llvm
// SystemZPostRASchedStrategy is a scheduling strategy which is plugged into
// the MachineScheduler. It has a sorted Available set of SUs and a pickNode()
// implementation that looks to optimize decoder grouping and balance the
-// usage of processor resources.
+// usage of processor resources. Scheduler states are saved for the end
+// region of each MBB, so that a successor block can learn from it.
//===----------------------------------------------------------------------===//
#include "SystemZMachineScheduler.h"
}
#endif
+// Try to find a single predecessor that would be interesting for the
+// scheduler in the top-most region of MBB.
+static MachineBasicBlock *getSingleSchedPred(MachineBasicBlock *MBB,
+ const MachineLoop *Loop) {
+ MachineBasicBlock *PredMBB = nullptr;
+ if (MBB->pred_size() == 1)
+ PredMBB = *MBB->pred_begin();
+
+ // The loop header has two predecessors, return the latch, but not for a
+ // single block loop.
+ if (MBB->pred_size() == 2 && Loop != nullptr && Loop->getHeader() == MBB) {
+ for (auto I = MBB->pred_begin(); I != MBB->pred_end(); ++I)
+ if (Loop->contains(*I))
+ PredMBB = (*I == MBB ? nullptr : *I);
+ }
+
+ assert ((PredMBB == nullptr || !Loop || Loop->contains(PredMBB))
+ && "Loop MBB should not consider predecessor outside of loop.");
+
+ return PredMBB;
+}
+
+void SystemZPostRASchedStrategy::
+advanceTo(MachineBasicBlock::iterator NextBegin) {
+ MachineBasicBlock::iterator LastEmittedMI = HazardRec->getLastEmittedMI();
+ MachineBasicBlock::iterator I =
+ ((LastEmittedMI != nullptr && LastEmittedMI->getParent() == MBB) ?
+ std::next(LastEmittedMI) : MBB->begin());
+
+ for (; I != NextBegin; ++I) {
+ if (I->isPosition() || I->isDebugValue())
+ continue;
+ HazardRec->emitInstruction(&*I);
+ }
+}
+
+void SystemZPostRASchedStrategy::enterMBB(MachineBasicBlock *NextMBB) {
+ assert ((SchedStates.find(NextMBB) == SchedStates.end()) &&
+ "Entering MBB twice?");
+ DEBUG (dbgs() << "+++ Entering MBB#" << NextMBB->getNumber());
+
+ MBB = NextMBB;
+ /// Create a HazardRec for MBB, save it in SchedStates and set HazardRec to
+ /// point to it.
+ HazardRec = SchedStates[MBB] = new SystemZHazardRecognizer(TII, &SchedModel);
+ DEBUG (const MachineLoop *Loop = MLI->getLoopFor(MBB);
+ if(Loop && Loop->getHeader() == MBB)
+ dbgs() << " (Loop header)";
+ dbgs() << ":\n";);
+
+ // Try to take over the state from a single predecessor, if it has been
+ // scheduled. If this is not possible, we are done.
+ MachineBasicBlock *SinglePredMBB =
+ getSingleSchedPred(MBB, MLI->getLoopFor(MBB));
+ if (SinglePredMBB == nullptr ||
+ SchedStates.find(SinglePredMBB) == SchedStates.end())
+ return;
+
+ DEBUG (dbgs() << "+++ Continued scheduling from MBB#"
+ << SinglePredMBB->getNumber() << "\n";);
+
+ HazardRec->copyState(SchedStates[SinglePredMBB]);
+
+ // Emit incoming terminator(s). Be optimistic and assume that branch
+ // prediction will generally do "the right thing".
+ for (MachineBasicBlock::iterator I = SinglePredMBB->getFirstTerminator();
+ I != SinglePredMBB->end(); I++) {
+ DEBUG (dbgs() << "+++ Emitting incoming branch: "; I->dump(););
+ bool TakenBranch = (I->isBranch() &&
+ (TII->getBranchInfo(*I).Target->isReg() || // Relative branch
+ TII->getBranchInfo(*I).Target->getMBB() == MBB));
+ HazardRec->emitInstruction(&*I, TakenBranch);
+ if (TakenBranch)
+ break;
+ }
+}
+
+void SystemZPostRASchedStrategy::leaveMBB() {
+ DEBUG (dbgs() << "+++ Leaving MBB#" << MBB->getNumber() << "\n";);
+
+ // Advance to first terminator. The successor block will handle terminators
+ // dependent on CFG layout (T/NT branch etc).
+ advanceTo(MBB->getFirstTerminator());
+}
+
SystemZPostRASchedStrategy::
SystemZPostRASchedStrategy(const MachineSchedContext *C)
- : DAG(nullptr), HazardRec(C) {}
+ : MLI(C->MLI),
+ TII(static_cast<const SystemZInstrInfo *>
+ (C->MF->getSubtarget().getInstrInfo())),
+ MBB(nullptr), HazardRec(nullptr) {
+ const TargetSubtargetInfo *ST = &C->MF->getSubtarget();
+ SchedModel.init(ST->getSchedModel(), ST, TII);
+}
+
+SystemZPostRASchedStrategy::~SystemZPostRASchedStrategy() {
+ // Delete hazard recognizers kept around for each MBB.
+ for (auto I : SchedStates) {
+ SystemZHazardRecognizer *hazrec = I.second;
+ delete hazrec;
+ }
+}
+
+void SystemZPostRASchedStrategy::initPolicy(MachineBasicBlock::iterator Begin,
+ MachineBasicBlock::iterator End,
+ unsigned NumRegionInstrs) {
+ // Don't emit the terminators.
+ if (Begin->isTerminator())
+ return;
-void SystemZPostRASchedStrategy::initialize(ScheduleDAGMI *dag) {
- DAG = dag;
- HazardRec.setDAG(dag);
- HazardRec.Reset();
+ // Emit any instructions before start of region.
+ advanceTo(Begin);
}
// Pick the next node to schedule.
// If only one choice, return it.
if (Available.size() == 1) {
DEBUG (dbgs() << "+++ Only one: ";
- HazardRec.dumpSU(*Available.begin(), dbgs()); dbgs() << "\n";);
+ HazardRec->dumpSU(*Available.begin(), dbgs()); dbgs() << "\n";);
return *Available.begin();
}
// All nodes that are possible to schedule are stored by in the
// Available set.
- DEBUG(dbgs() << "+++ Available: "; Available.dump(HazardRec););
+ DEBUG(dbgs() << "+++ Available: "; Available.dump(*HazardRec););
Candidate Best;
for (auto *SU : Available) {
// SU is the next candidate to be compared against current Best.
- Candidate c(SU, HazardRec);
+ Candidate c(SU, *HazardRec);
// Remeber which SU is the best candidate.
if (Best.SU == nullptr || c < Best) {
Best = c;
DEBUG(dbgs() << "+++ Best sofar: ";
- HazardRec.dumpSU(Best.SU, dbgs());
+ HazardRec->dumpSU(Best.SU, dbgs());
if (Best.GroupingCost != 0)
dbgs() << "\tGrouping cost:" << Best.GroupingCost;
if (Best.ResourcesCost != 0)
// Remove SU from Available set and update HazardRec.
Available.erase(SU);
- HazardRec.EmitInstruction(SU);
+ HazardRec->EmitInstruction(SU);
}
void SystemZPostRASchedStrategy::releaseTopNode(SUnit *SU) {
// Set isScheduleHigh flag on all SUs that we want to consider first in
// pickNode().
- const MCSchedClassDesc *SC = DAG->getSchedClass(SU);
+ const MCSchedClassDesc *SC = HazardRec->getSchedClass(SU);
bool AffectsGrouping = (SC->isValid() && (SC->BeginGroup || SC->EndGroup));
SU->isScheduleHigh = (AffectsGrouping || SU->isUnbuffered);
// SystemZPostRASchedStrategy is a scheduling strategy which is plugged into
// the MachineScheduler. It has a sorted Available set of SUs and a pickNode()
// implementation that looks to optimize decoder grouping and balance the
-// usage of processor resources.
+// usage of processor resources. Scheduler states are saved for the end
+// region of each MBB, so that a successor block can learn from it.
//===----------------------------------------------------------------------===//
#include "SystemZHazardRecognizer.h"
/// A MachineSchedStrategy implementation for SystemZ post RA scheduling.
class SystemZPostRASchedStrategy : public MachineSchedStrategy {
- ScheduleDAGMI *DAG;
+
+ const MachineLoopInfo *MLI;
+ const SystemZInstrInfo *TII;
+
+ // A SchedModel is needed before any DAG is built while advancing past
+ // non-scheduled instructions, so it would not always be possible to call
+ // DAG->getSchedClass(SU).
+ TargetSchedModel SchedModel;
/// A candidate during instruction evaluation.
struct Candidate {
/// The set of available SUs to schedule next.
SUSet Available;
- // HazardRecognizer that tracks the scheduler state for the current
- // region.
- SystemZHazardRecognizer HazardRec;
-
+ /// Current MBB
+ MachineBasicBlock *MBB;
+
+ /// Maintain hazard recognizers for all blocks, so that the scheduler state
+ /// can be maintained past BB boundaries when appropariate.
+ typedef std::map<MachineBasicBlock*, SystemZHazardRecognizer*> MBB2HazRec;
+ MBB2HazRec SchedStates;
+
+ /// Pointer to the HazardRecognizer that tracks the scheduler state for
+ /// the current region.
+ SystemZHazardRecognizer *HazardRec;
+
+ /// Update the scheduler state by emitting (non-scheduled) instructions
+ /// up to, but not including, NextBegin.
+ void advanceTo(MachineBasicBlock::iterator NextBegin);
+
public:
SystemZPostRASchedStrategy(const MachineSchedContext *C);
+ virtual ~SystemZPostRASchedStrategy();
+
+ /// Called for a region before scheduling.
+ void initPolicy(MachineBasicBlock::iterator Begin,
+ MachineBasicBlock::iterator End,
+ unsigned NumRegionInstrs) override;
/// PostRA scheduling does not track pressure.
bool shouldTrackPressure() const override { return false; }
- /// Initialize the strategy after building the DAG for a new region.
- void initialize(ScheduleDAGMI *dag) override;
+ // Process scheduling regions top-down so that scheduler states can be
+ // transferrred over scheduling boundaries.
+ bool doMBBSchedRegionsTopDown() const override { return true; }
+
+ void initialize(ScheduleDAGMI *dag) override {}
+
+ /// Tell the strategy that MBB is about to be processed.
+ void enterMBB(MachineBasicBlock *NextMBB) override;
+
+ /// Tell the strategy that current MBB is done.
+ void leaveMBB() override;
/// Pick the next node to schedule, or return NULL.
SUnit *pickNode(bool &IsTopNode) override;
define void @f2(i8 *%src) {
; CHECK-LABEL: f2:
; CHECK: llc [[REG:%r[0-5]]], 0(%r2)
-; CHECK: tmll [[REG]], 1
-; CHECK: mvi 0(%r2), 0
+; CHECK-DAG: mvi 0(%r2), 0
+; CHECK-DAG: tmll [[REG]], 1
; CHECK: ber %r14
; CHECK: br %r14
entry: