summaryrefslogtreecommitdiff
path: root/src/mem/dram_ctrl.cc
diff options
context:
space:
mode:
Diffstat (limited to 'src/mem/dram_ctrl.cc')
-rw-r--r--src/mem/dram_ctrl.cc607
1 files changed, 521 insertions, 86 deletions
diff --git a/src/mem/dram_ctrl.cc b/src/mem/dram_ctrl.cc
index 2866925c3..5d4852d38 100644
--- a/src/mem/dram_ctrl.cc
+++ b/src/mem/dram_ctrl.cc
@@ -41,6 +41,7 @@
* Ani Udipi
* Neha Agarwal
* Omar Naji
+ * Wendy Elsasser
*/
#include "base/bitfield.hh"
@@ -60,6 +61,7 @@ DRAMCtrl::DRAMCtrl(const DRAMCtrlParams* p) :
port(name() + ".port", *this), isTimingMode(false),
retryRdReq(false), retryWrReq(false),
busState(READ),
+ busStateNext(READ),
nextReqEvent(this), respondEvent(this),
deviceSize(p->device_size),
deviceBusWidth(p->device_bus_width), burstLength(p->burst_length),
@@ -481,6 +483,9 @@ DRAMCtrl::addToReadQueue(PacketPtr pkt, unsigned int pktCount)
readQueue.push_back(dram_pkt);
+ // increment read entries of the rank
+ ++dram_pkt->rankRef.readEntries;
+
// Update stats
avgRdQLen = readQueue.size() + respQueue.size();
}
@@ -544,6 +549,9 @@ DRAMCtrl::addToWriteQueue(PacketPtr pkt, unsigned int pktCount)
// Update stats
avgWrQLen = writeQueue.size();
+
+ // increment write entries of the rank
+ ++dram_pkt->rankRef.writeEntries;
} else {
DPRINTF(DRAM, "Merging write burst with existing queue entry\n");
@@ -656,6 +664,47 @@ DRAMCtrl::processRespondEvent()
DRAMPacket* dram_pkt = respQueue.front();
+ // if a read has reached its ready-time, decrement the number of reads
+ // At this point the packet has been handled and there is a possibility
+ // to switch to low-power mode if no other packet is available
+ --dram_pkt->rankRef.readEntries;
+ DPRINTF(DRAM, "number of read entries for rank %d is %d\n",
+ dram_pkt->rank, dram_pkt->rankRef.readEntries);
+
+ // counter should at least indicate one outstanding request
+ // for this read
+ assert(dram_pkt->rankRef.outstandingEvents > 0);
+ // read response received, decrement count
+ --dram_pkt->rankRef.outstandingEvents;
+
+ // at this moment should be either ACT or IDLE depending on
+ // if PRE has occurred to close all banks
+ assert((dram_pkt->rankRef.pwrState == PWR_ACT) ||
+ (dram_pkt->rankRef.pwrState == PWR_IDLE));
+
+ // track if this is the last packet before idling
+ // and that there are no outstanding commands to this rank
+ if (dram_pkt->rankRef.lowPowerEntryReady()) {
+ // verify that there are no events scheduled
+ assert(!dram_pkt->rankRef.activateEvent.scheduled());
+ assert(!dram_pkt->rankRef.prechargeEvent.scheduled());
+ assert(dram_pkt->rankRef.refreshState == REF_IDLE);
+
+ // if coming from active state, schedule power event to
+ // active power-down else go to precharge power-down
+ DPRINTF(DRAMState, "Rank %d sleep at tick %d; current power state is "
+ "%d\n", dram_pkt->rank, curTick(), dram_pkt->rankRef.pwrState);
+
+ // default to ACT power-down unless already in IDLE state
+ // could be in IDLE if PRE issued before data returned
+ PowerState next_pwr_state = PWR_ACT_PDN;
+ if (dram_pkt->rankRef.pwrState == PWR_IDLE) {
+ next_pwr_state = PWR_PRE_PDN;
+ }
+
+ dram_pkt->rankRef.powerDownSleep(next_pwr_state, curTick());
+ }
+
if (dram_pkt->burstHelper) {
// it is a split packet
dram_pkt->burstHelper->burstsServiced++;
@@ -1012,10 +1061,13 @@ DRAMCtrl::prechargeBank(Rank& rank_ref, Bank& bank, Tick pre_at, bool trace)
// would have reached the idle state, so schedule an event and
// rather check once we actually make it to the point in time when
// the (last) precharge takes place
- if (!rank_ref.prechargeEvent.scheduled())
+ if (!rank_ref.prechargeEvent.scheduled()) {
schedule(rank_ref.prechargeEvent, pre_done_at);
- else if (rank_ref.prechargeEvent.when() < pre_done_at)
+ // New event, increment count
+ ++rank_ref.outstandingEvents;
+ } else if (rank_ref.prechargeEvent.when() < pre_done_at) {
reschedule(rank_ref.prechargeEvent, pre_done_at);
+ }
}
void
@@ -1027,6 +1079,14 @@ DRAMCtrl::doDRAMAccess(DRAMPacket* dram_pkt)
// get the rank
Rank& rank = dram_pkt->rankRef;
+ // are we in or transitioning to a low-power state and have not scheduled
+ // a power-up event?
+ // if so, wake up from power down to issue RD/WR burst
+ if (rank.inLowPowerState) {
+ assert(rank.pwrState != PWR_SREF);
+ rank.scheduleWakeUpEvent(tXP);
+ }
+
// get the bank
Bank& bank = dram_pkt->bankRef;
@@ -1229,12 +1289,33 @@ DRAMCtrl::processNextReqEvent()
int busyRanks = 0;
for (auto r : ranks) {
if (!r->isAvailable()) {
- // rank is busy refreshing
- busyRanks++;
+ if (r->pwrState != PWR_SREF) {
+ // rank is busy refreshing
+ DPRINTF(DRAMState, "Rank %d is not available\n", r->rank);
+ busyRanks++;
+
+ // let the rank know that if it was waiting to drain, it
+ // is now done and ready to proceed
+ r->checkDrainDone();
+ }
- // let the rank know that if it was waiting to drain, it
- // is now done and ready to proceed
- r->checkDrainDone();
+ // check if we were in self-refresh and haven't started
+ // to transition out
+ if ((r->pwrState == PWR_SREF) && r->inLowPowerState) {
+ DPRINTF(DRAMState, "Rank %d is in self-refresh\n", r->rank);
+ // if we have commands queued to this rank and we don't have
+ // a minimum number of active commands enqueued,
+ // exit self-refresh
+ if (r->forceSelfRefreshExit()) {
+ DPRINTF(DRAMState, "rank %d was in self refresh and"
+ " should wake up\n", r->rank);
+ //wake up from self-refresh
+ r->scheduleWakeUpEvent(tXS);
+ // things are brought back into action once a refresh is
+ // performed after self-refresh
+ // continue with selection for other ranks
+ }
+ }
}
}
@@ -1245,30 +1326,32 @@ DRAMCtrl::processNextReqEvent()
return;
}
- // pre-emptively set to false. Overwrite if in READ_TO_WRITE
- // or WRITE_TO_READ state
+ // pre-emptively set to false. Overwrite if in transitioning to
+ // a new state
bool switched_cmd_type = false;
- if (busState == READ_TO_WRITE) {
- DPRINTF(DRAM, "Switching to writes after %d reads with %d reads "
- "waiting\n", readsThisTime, readQueue.size());
-
- // sample and reset the read-related stats as we are now
- // transitioning to writes, and all reads are done
- rdPerTurnAround.sample(readsThisTime);
- readsThisTime = 0;
-
- // now proceed to do the actual writes
- busState = WRITE;
- switched_cmd_type = true;
- } else if (busState == WRITE_TO_READ) {
- DPRINTF(DRAM, "Switching to reads after %d writes with %d writes "
- "waiting\n", writesThisTime, writeQueue.size());
+ if (busState != busStateNext) {
+ if (busState == READ) {
+ DPRINTF(DRAM, "Switching to writes after %d reads with %d reads "
+ "waiting\n", readsThisTime, readQueue.size());
+
+ // sample and reset the read-related stats as we are now
+ // transitioning to writes, and all reads are done
+ rdPerTurnAround.sample(readsThisTime);
+ readsThisTime = 0;
+
+ // now proceed to do the actual writes
+ switched_cmd_type = true;
+ } else {
+ DPRINTF(DRAM, "Switching to reads after %d writes with %d writes "
+ "waiting\n", writesThisTime, writeQueue.size());
- wrPerTurnAround.sample(writesThisTime);
- writesThisTime = 0;
+ wrPerTurnAround.sample(writesThisTime);
+ writesThisTime = 0;
- busState = READ;
- switched_cmd_type = true;
+ switched_cmd_type = true;
+ }
+ // update busState to match next state until next transition
+ busState = busStateNext;
}
// when we get here it is either a read or a write
@@ -1323,6 +1406,7 @@ DRAMCtrl::processNextReqEvent()
DRAMPacket* dram_pkt = readQueue.front();
assert(dram_pkt->rankRef.isAvailable());
+
// here we get a bit creative and shift the bus busy time not
// just the tWTR, but also a CAS latency to capture the fact
// that we are allowed to prepare a new bank, but not issue a
@@ -1337,6 +1421,9 @@ DRAMCtrl::processNextReqEvent()
// At this point we're done dealing with the request
readQueue.pop_front();
+ // Every respQueue which will generate an event, increment count
+ ++dram_pkt->rankRef.outstandingEvents;
+
// sanity check
assert(dram_pkt->size <= burstSize);
assert(dram_pkt->readyTime >= curTick());
@@ -1364,7 +1451,7 @@ DRAMCtrl::processNextReqEvent()
// draining), or because the writes hit the hight threshold
if (switch_to_writes) {
// transition to writing
- busState = READ_TO_WRITE;
+ busStateNext = WRITE;
}
} else {
// bool to check if write to free rank is found
@@ -1398,6 +1485,26 @@ DRAMCtrl::processNextReqEvent()
doDRAMAccess(dram_pkt);
writeQueue.pop_front();
+
+ // removed write from queue, decrement count
+ --dram_pkt->rankRef.writeEntries;
+
+ // Schedule write done event to decrement event count
+ // after the readyTime has been reached
+ // Only schedule latest write event to minimize events
+ // required; only need to ensure that final event scheduled covers
+ // the time that writes are outstanding and bus is active
+ // to holdoff power-down entry events
+ if (!dram_pkt->rankRef.writeDoneEvent.scheduled()) {
+ schedule(dram_pkt->rankRef.writeDoneEvent, dram_pkt->readyTime);
+ // New event, increment count
+ ++dram_pkt->rankRef.outstandingEvents;
+
+ } else if (dram_pkt->rankRef.writeDoneEvent.when() <
+ dram_pkt-> readyTime) {
+ reschedule(dram_pkt->rankRef.writeDoneEvent, dram_pkt->readyTime);
+ }
+
isInWriteQueue.erase(burstAlign(dram_pkt->addr));
delete dram_pkt;
@@ -1410,7 +1517,7 @@ DRAMCtrl::processNextReqEvent()
drainState() != DrainState::Draining) ||
(!readQueue.empty() && writesThisTime >= minWritesPerSwitch)) {
// turn the bus back around for reads again
- busState = WRITE_TO_READ;
+ busStateNext = READ;
// note that the we switch back to reads also in the idle
// case, which eventually will check for any draining and
@@ -1518,11 +1625,13 @@ DRAMCtrl::minBankPrep(const deque<DRAMPacket*>& queue,
DRAMCtrl::Rank::Rank(DRAMCtrl& _memory, const DRAMCtrlParams* _p)
: EventManager(&_memory), memory(_memory),
- pwrStateTrans(PWR_IDLE), pwrState(PWR_IDLE), pwrStateTick(0),
- refreshState(REF_IDLE), refreshDueAt(0),
- power(_p, false), numBanksActive(0),
- activateEvent(*this), prechargeEvent(*this),
- refreshEvent(*this), powerEvent(*this)
+ pwrStateTrans(PWR_IDLE), pwrStatePostRefresh(PWR_IDLE),
+ pwrStateTick(0), refreshDueAt(0), pwrState(PWR_IDLE),
+ refreshState(REF_IDLE), inLowPowerState(false), rank(0),
+ readEntries(0), writeEntries(0), outstandingEvents(0),
+ wakeUpAllowedAt(0), power(_p, false), numBanksActive(0),
+ writeDoneEvent(*this), activateEvent(*this), prechargeEvent(*this),
+ refreshEvent(*this), powerEvent(*this), wakeUpEvent(*this)
{ }
void
@@ -1544,6 +1653,27 @@ DRAMCtrl::Rank::suspend()
// Update the stats
updatePowerStats();
+
+ // don't automatically transition back to LP state after next REF
+ pwrStatePostRefresh = PWR_IDLE;
+}
+
+bool
+DRAMCtrl::Rank::lowPowerEntryReady() const
+{
+ bool no_queued_cmds = ((memory.busStateNext == READ) && (readEntries == 0))
+ || ((memory.busStateNext == WRITE) &&
+ (writeEntries == 0));
+
+ if (refreshState == REF_RUN) {
+ // have not decremented outstandingEvents for refresh command
+ // still check if there are no commands queued to force PD
+ // entry after refresh completes
+ return no_queued_cmds;
+ } else {
+ // ensure no commands in Q and no commands scheduled
+ return (no_queued_cmds && (outstandingEvents == 0));
+ }
}
void
@@ -1554,7 +1684,7 @@ DRAMCtrl::Rank::checkDrainDone()
if (refreshState == REF_DRAIN) {
DPRINTF(DRAM, "Refresh drain done, now precharging\n");
- refreshState = REF_PRE;
+ refreshState = REF_PD_EXIT;
// hand control back to the refresh event loop
schedule(refreshEvent, curTick());
@@ -1602,26 +1732,59 @@ DRAMCtrl::Rank::processActivateEvent()
void
DRAMCtrl::Rank::processPrechargeEvent()
{
+ // counter should at least indicate one outstanding request
+ // for this precharge
+ assert(outstandingEvents > 0);
+ // precharge complete, decrement count
+ --outstandingEvents;
+
// if we reached zero, then special conditions apply as we track
// if all banks are precharged for the power models
if (numBanksActive == 0) {
- // we should transition to the idle state when the last bank
- // is precharged
- schedulePowerEvent(PWR_IDLE, curTick());
+ // no reads to this rank in the Q and no pending
+ // RD/WR or refresh commands
+ if (lowPowerEntryReady()) {
+ // should still be in ACT state since bank still open
+ assert(pwrState == PWR_ACT);
+
+ // All banks closed - switch to precharge power down state.
+ DPRINTF(DRAMState, "Rank %d sleep at tick %d\n",
+ rank, curTick());
+ powerDownSleep(PWR_PRE_PDN, curTick());
+ } else {
+ // we should transition to the idle state when the last bank
+ // is precharged
+ schedulePowerEvent(PWR_IDLE, curTick());
+ }
}
}
void
+DRAMCtrl::Rank::processWriteDoneEvent()
+{
+ // counter should at least indicate one outstanding request
+ // for this write
+ assert(outstandingEvents > 0);
+ // Write transfer on bus has completed
+ // decrement per rank counter
+ --outstandingEvents;
+}
+
+void
DRAMCtrl::Rank::processRefreshEvent()
{
// when first preparing the refresh, remember when it was due
- if (refreshState == REF_IDLE) {
+ if ((refreshState == REF_IDLE) || (refreshState == REF_SREF_EXIT)) {
// remember when the refresh is due
refreshDueAt = curTick();
// proceed to drain
refreshState = REF_DRAIN;
+ // make nonzero while refresh is pending to ensure
+ // power down and self-refresh are not entered
+ ++outstandingEvents;
+
DPRINTF(DRAM, "Refresh due\n");
}
@@ -1639,15 +1802,28 @@ DRAMCtrl::Rank::processRefreshEvent()
return;
} else {
+ refreshState = REF_PD_EXIT;
+ }
+ }
+
+ // at this point, ensure that rank is not in a power-down state
+ if (refreshState == REF_PD_EXIT) {
+ // if rank was sleeping and we have't started exit process,
+ // wake-up for refresh
+ if (inLowPowerState) {
+ DPRINTF(DRAM, "Wake Up for refresh\n");
+ // save state and return after refresh completes
+ scheduleWakeUpEvent(memory.tXP);
+ return;
+ } else {
refreshState = REF_PRE;
}
}
// at this point, ensure that all banks are precharged
if (refreshState == REF_PRE) {
- // precharge any active bank if we are not already in the idle
- // state
- if (pwrState != PWR_IDLE) {
+ // precharge any active bank
+ if (numBanksActive != 0) {
// at the moment, we use a precharge all even if there is
// only a single bank open
DPRINTF(DRAM, "Precharging all\n");
@@ -1681,15 +1857,22 @@ DRAMCtrl::Rank::processRefreshEvent()
DPRINTF(DRAMPower, "%llu,PREA,0,%d\n",
divCeil(pre_at, memory.tCK) -
memory.timeStampOffset, rank);
- } else {
+ } else if ((pwrState == PWR_IDLE) && (outstandingEvents == 1)) {
+ // Banks are closed, have transitioned to IDLE state, and
+ // no outstanding ACT,RD/WR,Auto-PRE sequence scheduled
DPRINTF(DRAM, "All banks already precharged, starting refresh\n");
- // go ahead and kick the power state machine into gear if
+ // go ahead and kick the power state machine into gear since
// we are already idle
schedulePowerEvent(PWR_REF, curTick());
+ } else {
+ // banks state is closed but haven't transitioned pwrState to IDLE
+ // or have outstanding ACT,RD/WR,Auto-PRE sequence scheduled
+ // should have outstanding precharge event in this case
+ assert(prechargeEvent.scheduled());
+ // will start refresh when pwrState transitions to IDLE
}
- refreshState = REF_RUN;
assert(numBanksActive == 0);
// wait for all banks to be precharged, at which point the
@@ -1700,7 +1883,7 @@ DRAMCtrl::Rank::processRefreshEvent()
}
// last but not least we perform the actual refresh
- if (refreshState == REF_RUN) {
+ if (refreshState == REF_START) {
// should never get here with any banks active
assert(numBanksActive == 0);
assert(pwrState == PWR_REF);
@@ -1720,25 +1903,70 @@ DRAMCtrl::Rank::processRefreshEvent()
DPRINTF(DRAMPower, "%llu,REF,0,%d\n", divCeil(curTick(), memory.tCK) -
memory.timeStampOffset, rank);
+ // Update for next refresh
+ refreshDueAt += memory.tREFI;
+
// make sure we did not wait so long that we cannot make up
// for it
- if (refreshDueAt + memory.tREFI < ref_done_at) {
+ if (refreshDueAt < ref_done_at) {
fatal("Refresh was delayed so long we cannot catch up\n");
}
- // compensate for the delay in actually performing the refresh
- // when scheduling the next one
- schedule(refreshEvent, refreshDueAt + memory.tREFI - memory.tRP);
+ // Run the refresh and schedule event to transition power states
+ // when refresh completes
+ refreshState = REF_RUN;
+ schedule(refreshEvent, ref_done_at);
+ return;
+ }
+
+ if (refreshState == REF_RUN) {
+ // should never get here with any banks active
+ assert(numBanksActive == 0);
+ assert(pwrState == PWR_REF);
assert(!powerEvent.scheduled());
- // move to the idle power state once the refresh is done, this
- // will also move the refresh state machine to the refresh
- // idle state
- schedulePowerEvent(PWR_IDLE, ref_done_at);
+ if ((memory.drainState() == DrainState::Draining) ||
+ (memory.drainState() == DrainState::Drained)) {
+ // if draining, do not re-enter low-power mode.
+ // simply go to IDLE and wait
+ schedulePowerEvent(PWR_IDLE, curTick());
+ } else {
+ // At the moment, we sleep when the refresh ends and wait to be
+ // woken up again if previously in a low-power state.
+ if (pwrStatePostRefresh != PWR_IDLE) {
+ // power State should be power Refresh
+ assert(pwrState == PWR_REF);
+ DPRINTF(DRAMState, "Rank %d sleeping after refresh and was in "
+ "power state %d before refreshing\n", rank,
+ pwrStatePostRefresh);
+ powerDownSleep(pwrState, curTick());
+
+ // Force PRE power-down if there are no outstanding commands
+ // in Q after refresh.
+ } else if (lowPowerEntryReady()) {
+ DPRINTF(DRAMState, "Rank %d sleeping after refresh but was NOT"
+ " in a low power state before refreshing\n", rank);
+ powerDownSleep(PWR_PRE_PDN, curTick());
+
+ } else {
+ // move to the idle power state once the refresh is done, this
+ // will also move the refresh state machine to the refresh
+ // idle state
+ schedulePowerEvent(PWR_IDLE, curTick());
+ }
+ }
+
+ // if transitioning to self refresh do not schedule a new refresh;
+ // when waking from self refresh, a refresh is scheduled again.
+ if (pwrStateTrans != PWR_SREF) {
+ // compensate for the delay in actually performing the refresh
+ // when scheduling the next one
+ schedule(refreshEvent, refreshDueAt - memory.tRP);
- DPRINTF(DRAMState, "Refresh done at %llu and next refresh at %llu\n",
- ref_done_at, refreshDueAt + memory.tREFI);
+ DPRINTF(DRAMState, "Refresh done at %llu and next refresh"
+ " at %llu\n", curTick(), refreshDueAt);
+ }
}
}
@@ -1764,8 +1992,132 @@ DRAMCtrl::Rank::schedulePowerEvent(PowerState pwr_state, Tick tick)
}
void
+DRAMCtrl::Rank::powerDownSleep(PowerState pwr_state, Tick tick)
+{
+ // if low power state is active low, schedule to active low power state.
+ // in reality tCKE is needed to enter active low power. This is neglected
+ // here and could be added in the future.
+ if (pwr_state == PWR_ACT_PDN) {
+ schedulePowerEvent(pwr_state, tick);
+ // push command to DRAMPower
+ cmdList.push_back(Command(MemCommand::PDN_F_ACT, 0, tick));
+ DPRINTF(DRAMPower, "%llu,PDN_F_ACT,0,%d\n", divCeil(tick,
+ memory.tCK) - memory.timeStampOffset, rank);
+ } else if (pwr_state == PWR_PRE_PDN) {
+ // if low power state is precharge low, schedule to precharge low
+ // power state. In reality tCKE is needed to enter active low power.
+ // This is neglected here.
+ schedulePowerEvent(pwr_state, tick);
+ //push Command to DRAMPower
+ cmdList.push_back(Command(MemCommand::PDN_F_PRE, 0, tick));
+ DPRINTF(DRAMPower, "%llu,PDN_F_PRE,0,%d\n", divCeil(tick,
+ memory.tCK) - memory.timeStampOffset, rank);
+ } else if (pwr_state == PWR_REF) {
+ // if a refresh just occured
+ // transition to PRE_PDN now that all banks are closed
+ // do not transition to SREF if commands are in Q; stay in PRE_PDN
+ if (pwrStatePostRefresh == PWR_ACT_PDN || !lowPowerEntryReady()) {
+ // prechage power down requires tCKE to enter. For simplicity
+ // this is not considered.
+ schedulePowerEvent(PWR_PRE_PDN, tick);
+ //push Command to DRAMPower
+ cmdList.push_back(Command(MemCommand::PDN_F_PRE, 0, tick));
+ DPRINTF(DRAMPower, "%llu,PDN_F_PRE,0,%d\n", divCeil(tick,
+ memory.tCK) - memory.timeStampOffset, rank);
+ } else {
+ // last low power State was power precharge
+ assert(pwrStatePostRefresh == PWR_PRE_PDN);
+ // self refresh requires time tCKESR to enter. For simplicity,
+ // this is not considered.
+ schedulePowerEvent(PWR_SREF, tick);
+ // push Command to DRAMPower
+ cmdList.push_back(Command(MemCommand::SREN, 0, tick));
+ DPRINTF(DRAMPower, "%llu,SREN,0,%d\n", divCeil(tick,
+ memory.tCK) - memory.timeStampOffset, rank);
+ }
+ }
+ // Ensure that we don't power-down and back up in same tick
+ // Once we commit to PD entry, do it and wait for at least 1tCK
+ // This could be replaced with tCKE if/when that is added to the model
+ wakeUpAllowedAt = tick + memory.tCK;
+
+ // Transitioning to a low power state, set flag
+ inLowPowerState = true;
+}
+
+void
+DRAMCtrl::Rank::scheduleWakeUpEvent(Tick exit_delay)
+{
+ Tick wake_up_tick = std::max(curTick(), wakeUpAllowedAt);
+
+ DPRINTF(DRAMState, "Scheduling wake-up for rank %d at tick %d\n",
+ rank, wake_up_tick);
+
+ // if waking for refresh, hold previous state
+ // else reset state back to IDLE
+ if (refreshState == REF_PD_EXIT) {
+ pwrStatePostRefresh = pwrState;
+ } else {
+ // don't automatically transition back to LP state after next REF
+ pwrStatePostRefresh = PWR_IDLE;
+ }
+
+ // schedule wake-up with event to ensure entry has completed before
+ // we try to wake-up
+ schedule(wakeUpEvent, wake_up_tick);
+
+ for (auto &b : banks) {
+ // respect both causality and any existing bank
+ // constraints, some banks could already have a
+ // (auto) precharge scheduled
+ b.colAllowedAt = std::max(wake_up_tick + exit_delay, b.colAllowedAt);
+ b.preAllowedAt = std::max(wake_up_tick + exit_delay, b.preAllowedAt);
+ b.actAllowedAt = std::max(wake_up_tick + exit_delay, b.actAllowedAt);
+ }
+ // Transitioning out of low power state, clear flag
+ inLowPowerState = false;
+
+ // push to DRAMPower
+ // use pwrStateTrans for cases where we have a power event scheduled
+ // to enter low power that has not yet been processed
+ if (pwrStateTrans == PWR_ACT_PDN) {
+ cmdList.push_back(Command(MemCommand::PUP_ACT, 0, wake_up_tick));
+ DPRINTF(DRAMPower, "%llu,PUP_ACT,0,%d\n", divCeil(wake_up_tick,
+ memory.tCK) - memory.timeStampOffset, rank);
+
+ } else if (pwrStateTrans == PWR_PRE_PDN) {
+ cmdList.push_back(Command(MemCommand::PUP_PRE, 0, wake_up_tick));
+ DPRINTF(DRAMPower, "%llu,PUP_PRE,0,%d\n", divCeil(wake_up_tick,
+ memory.tCK) - memory.timeStampOffset, rank);
+ } else if (pwrStateTrans == PWR_SREF) {
+ cmdList.push_back(Command(MemCommand::SREX, 0, wake_up_tick));
+ DPRINTF(DRAMPower, "%llu,SREX,0,%d\n", divCeil(wake_up_tick,
+ memory.tCK) - memory.timeStampOffset, rank);
+ }
+}
+
+void
+DRAMCtrl::Rank::processWakeUpEvent()
+{
+ // Should be in a power-down or self-refresh state
+ assert((pwrState == PWR_ACT_PDN) || (pwrState == PWR_PRE_PDN) ||
+ (pwrState == PWR_SREF));
+
+ // Check current state to determine transition state
+ if (pwrState == PWR_ACT_PDN) {
+ // banks still open, transition to PWR_ACT
+ schedulePowerEvent(PWR_ACT, curTick());
+ } else {
+ // transitioning from a precharge power-down or self-refresh state
+ // banks are closed - transition to PWR_IDLE
+ schedulePowerEvent(PWR_IDLE, curTick());
+ }
+}
+
+void
DRAMCtrl::Rank::processPowerEvent()
{
+ assert(curTick() >= pwrStateTick);
// remember where we were, and for how long
Tick duration = curTick() - pwrStateTick;
PowerState prev_state = pwrState;
@@ -1773,36 +2125,80 @@ DRAMCtrl::Rank::processPowerEvent()
// update the accounting
pwrStateTime[prev_state] += duration;
+ // track to total idle time
+ if ((prev_state == PWR_PRE_PDN) || (prev_state == PWR_ACT_PDN) ||
+ (prev_state == PWR_SREF)) {
+ totalIdleTime += duration;
+ }
+
pwrState = pwrStateTrans;
pwrStateTick = curTick();
- if (pwrState == PWR_IDLE) {
- DPRINTF(DRAMState, "All banks precharged\n");
-
- // if we were refreshing, make sure we start scheduling requests again
- if (prev_state == PWR_REF) {
- DPRINTF(DRAMState, "Was refreshing for %llu ticks\n", duration);
- assert(pwrState == PWR_IDLE);
+ // if rank was refreshing, make sure to start scheduling requests again
+ if (prev_state == PWR_REF) {
+ // bus IDLED prior to REF
+ // counter should be one for refresh command only
+ assert(outstandingEvents == 1);
+ // REF complete, decrement count
+ --outstandingEvents;
- // kick things into action again
+ DPRINTF(DRAMState, "Was refreshing for %llu ticks\n", duration);
+ // if sleeping after refresh
+ if (pwrState != PWR_IDLE) {
+ assert((pwrState == PWR_PRE_PDN) || (pwrState == PWR_SREF));
+ DPRINTF(DRAMState, "Switching to power down state after refreshing"
+ " rank %d at %llu tick\n", rank, curTick());
+ }
+ if (pwrState != PWR_SREF) {
+ // rank is not available in SREF
+ // don't transition to IDLE in this case
refreshState = REF_IDLE;
- // a request event could be already scheduled by the state
- // machine of the other rank
- if (!memory.nextReqEvent.scheduled())
- schedule(memory.nextReqEvent, curTick());
- } else {
- assert(prev_state == PWR_ACT);
+ }
+ // a request event could be already scheduled by the state
+ // machine of the other rank
+ if (!memory.nextReqEvent.scheduled()) {
+ DPRINTF(DRAM, "Scheduling next request after refreshing rank %d\n",
+ rank);
+ schedule(memory.nextReqEvent, curTick());
+ }
+ } else if (pwrState == PWR_ACT) {
+ if (refreshState == REF_PD_EXIT) {
+ // kick the refresh event loop into action again
+ assert(prev_state == PWR_ACT_PDN);
+ // go back to REF event and close banks
+ refreshState = REF_PRE;
+ schedule(refreshEvent, curTick());
+ }
+ } else if (pwrState == PWR_IDLE) {
+ DPRINTF(DRAMState, "All banks precharged\n");
+ if (prev_state == PWR_SREF) {
+ // set refresh state to REF_SREF_EXIT, ensuring isAvailable
+ // continues to return false during tXS after SREF exit
+ // Schedule a refresh which kicks things back into action
+ // when it finishes
+ refreshState = REF_SREF_EXIT;
+ schedule(refreshEvent, curTick() + memory.tXS);
+ } else {
// if we have a pending refresh, and are now moving to
- // the idle state, direclty transition to a refresh
- if (refreshState == REF_RUN) {
- // there should be nothing waiting at this point
- assert(!powerEvent.scheduled());
-
- // update the state in zero time and proceed below
- pwrState = PWR_REF;
+ // the idle state, directly transition to a refresh
+ if ((refreshState == REF_PRE) || (refreshState == REF_PD_EXIT)) {
+ // ensure refresh is restarted only after final PRE command.
+ // do not restart refresh if controller is in an intermediate
+ // state, after PRE_PDN exit, when banks are IDLE but an
+ // ACT is scheduled.
+ if (!activateEvent.scheduled()) {
+ // there should be nothing waiting at this point
+ assert(!powerEvent.scheduled());
+ // update the state in zero time and proceed below
+ pwrState = PWR_REF;
+ } else {
+ // must have PRE scheduled to transition back to IDLE
+ // and re-kick off refresh
+ assert(prechargeEvent.scheduled());
+ }
}
- }
+ }
}
// we transition to the refresh state, let the refresh state
@@ -1810,12 +2206,20 @@ DRAMCtrl::Rank::processPowerEvent()
// scheduling of the next power state transition as well as the
// following refresh
if (pwrState == PWR_REF) {
+ assert(refreshState == REF_PRE || refreshState == REF_PD_EXIT);
DPRINTF(DRAMState, "Refreshing\n");
+
// kick the refresh event loop into action again, and that
// in turn will schedule a transition to the idle power
// state once the refresh is done
- assert(refreshState == REF_RUN);
- processRefreshEvent();
+ if (refreshState == REF_PD_EXIT) {
+ // Wait for PD exit timing to complete before issuing REF
+ schedule(refreshEvent, curTick() + memory.tXP);
+ } else {
+ schedule(refreshEvent, curTick());
+ }
+ // Banks transitioned to IDLE, start REF
+ refreshState = REF_START;
}
}
@@ -1852,6 +2256,9 @@ DRAMCtrl::Rank::updatePowerStats()
refreshEnergy = energy.ref_energy * memory.devicesPerRank;
actBackEnergy = energy.act_stdby_energy * memory.devicesPerRank;
preBackEnergy = energy.pre_stdby_energy * memory.devicesPerRank;
+ actPowerDownEnergy = energy.f_act_pd_energy * memory.devicesPerRank;
+ prePowerDownEnergy = energy.f_pre_pd_energy * memory.devicesPerRank;
+ selfRefreshEnergy = energy.sref_energy * memory.devicesPerRank;
totalEnergy = energy.total_energy * memory.devicesPerRank;
averagePower = rank_power.average_power * memory.devicesPerRank;
}
@@ -1880,14 +2287,15 @@ DRAMCtrl::Rank::regStats()
using namespace Stats;
pwrStateTime
- .init(5)
+ .init(6)
.name(name() + ".memoryStateTime")
.desc("Time in different power states");
pwrStateTime.subname(0, "IDLE");
pwrStateTime.subname(1, "REF");
- pwrStateTime.subname(2, "PRE_PDN");
- pwrStateTime.subname(3, "ACT");
- pwrStateTime.subname(4, "ACT_PDN");
+ pwrStateTime.subname(2, "SREF");
+ pwrStateTime.subname(3, "PRE_PDN");
+ pwrStateTime.subname(4, "ACT");
+ pwrStateTime.subname(5, "ACT_PDN");
actEnergy
.name(name() + ".actEnergy")
@@ -1917,6 +2325,18 @@ DRAMCtrl::Rank::regStats()
.name(name() + ".preBackEnergy")
.desc("Energy for precharge background per rank (pJ)");
+ actPowerDownEnergy
+ .name(name() + ".actPowerDownEnergy")
+ .desc("Energy for active power-down per rank (pJ)");
+
+ prePowerDownEnergy
+ .name(name() + ".prePowerDownEnergy")
+ .desc("Energy for precharge power-down per rank (pJ)");
+
+ selfRefreshEnergy
+ .name(name() + ".selfRefreshEnergy")
+ .desc("Energy for self refresh per rank (pJ)");
+
totalEnergy
.name(name() + ".totalEnergy")
.desc("Total energy per rank (pJ)");
@@ -1925,6 +2345,10 @@ DRAMCtrl::Rank::regStats()
.name(name() + ".averagePower")
.desc("Core power per rank (mW)");
+ totalIdleTime
+ .name(name() + ".totalIdleTime")
+ .desc("Total Idle time Per DRAM Rank");
+
registerDumpCallback(new RankDumpCallback(this));
}
void
@@ -2215,11 +2639,22 @@ DRAMCtrl::drain()
" resp: %d\n", writeQueue.size(), readQueue.size(),
respQueue.size());
- // the only part that is not drained automatically over time
+ // the only queue that is not drained automatically over time
// is the write queue, thus kick things into action if needed
if (!writeQueue.empty() && !nextReqEvent.scheduled()) {
schedule(nextReqEvent, curTick());
}
+
+ // also need to kick off events to exit self-refresh
+ for (auto r : ranks) {
+ // force self-refresh exit, which in turn will issue auto-refresh
+ if (r->pwrState == PWR_SREF) {
+ DPRINTF(DRAM,"Rank%d: Forcing self-refresh wakeup in drain\n",
+ r->rank);
+ r->scheduleWakeUpEvent(tXS);
+ }
+ }
+
return DrainState::Draining;
} else {
return DrainState::Drained;