From 35cf19d441ed15d054d00674ec67ab5bc769f6d7 Mon Sep 17 00:00:00 2001 From: Steve Reinhardt Date: Sun, 17 Jun 2007 17:27:53 -0700 Subject: More major reorg of cache. Seems to work for atomic mode now, timing mode still broken. configs/example/memtest.py: Revamp options. src/cpu/memtest/memtest.cc: No need for memory initialization. No need to make atomic response... memory system should do that now. src/cpu/memtest/memtest.hh: MemTest really doesn't want to snoop. src/mem/bridge.cc: checkFunctional() cleanup. src/mem/bus.cc: src/mem/bus.hh: src/mem/cache/base_cache.cc: src/mem/cache/base_cache.hh: src/mem/cache/cache.cc: src/mem/cache/cache.hh: src/mem/cache/cache_blk.hh: src/mem/cache/cache_builder.cc: src/mem/cache/cache_impl.hh: src/mem/cache/coherence/coherence_protocol.cc: src/mem/cache/coherence/coherence_protocol.hh: src/mem/cache/coherence/simple_coherence.hh: src/mem/cache/miss/SConscript: src/mem/cache/miss/mshr.cc: src/mem/cache/miss/mshr.hh: src/mem/cache/miss/mshr_queue.cc: src/mem/cache/miss/mshr_queue.hh: src/mem/cache/prefetch/base_prefetcher.cc: src/mem/cache/tags/fa_lru.cc: src/mem/cache/tags/fa_lru.hh: src/mem/cache/tags/iic.cc: src/mem/cache/tags/iic.hh: src/mem/cache/tags/lru.cc: src/mem/cache/tags/lru.hh: src/mem/cache/tags/split.cc: src/mem/cache/tags/split.hh: src/mem/cache/tags/split_lifo.cc: src/mem/cache/tags/split_lifo.hh: src/mem/cache/tags/split_lru.cc: src/mem/cache/tags/split_lru.hh: src/mem/packet.cc: src/mem/packet.hh: src/mem/physical.cc: src/mem/physical.hh: src/mem/tport.cc: More major reorg. Seems to work for atomic mode now, timing mode still broken. --HG-- extra : convert_revision : 7e70dfc4a752393b911880ff028271433855ae87 --- src/mem/cache/cache_impl.hh | 1976 ++++++++++++++++++++----------------------- 1 file changed, 904 insertions(+), 1072 deletions(-) (limited to 'src/mem/cache/cache_impl.hh') diff --git a/src/mem/cache/cache_impl.hh b/src/mem/cache/cache_impl.hh index a7f96603e..0f66e613c 100644 --- a/src/mem/cache/cache_impl.hh +++ b/src/mem/cache/cache_impl.hh @@ -37,17 +37,8 @@ * Cache definitions. */ -#include -#include - -#include -#include -#include -#include - #include "sim/host.hh" #include "base/misc.hh" -#include "cpu/smt.hh" #include "mem/cache/cache.hh" #include "mem/cache/cache_blk.hh" @@ -56,25 +47,16 @@ #include "sim/sim_exit.hh" // for SimExitEvent -bool SIGNAL_NACK_HACK; template Cache::Cache(const std::string &_name, Cache::Params ¶ms) : BaseCache(_name, params.baseParams), prefetchAccess(params.prefetchAccess), - tags(params.tags), missQueue(params.missQueue), + tags(params.tags), coherence(params.coherence), prefetcher(params.prefetcher), - hitLatency(params.hitLatency), - compressionAlg(params.compressionAlg), - blkSize(params.blkSize), doFastWrites(params.doFastWrites), - prefetchMiss(params.prefetchMiss), - storeCompressed(params.storeCompressed), - compressOnWriteback(params.compressOnWriteback), - compLatency(params.compLatency), - adaptiveCompression(params.adaptiveCompression), - writebackCompressed(params.writebackCompressed) + prefetchMiss(params.prefetchMiss) { cpuSidePort = new CpuSidePort(_name + "-cpu_side_port", this); memSidePort = new MemSidePort(_name + "-mem_side_port", this); @@ -82,12 +64,8 @@ Cache::Cache(const std::string &_name, memSidePort->setOtherPort(cpuSidePort); tags->setCache(this); - missQueue->setCache(this); - missQueue->setPrefetcher(prefetcher); coherence->setCache(this); prefetcher->setCache(this); - invalidateReq = new Request((Addr) NULL, blkSize, 0); - invalidatePkt = new Packet(invalidateReq, MemCmd::InvalidateReq, 0); } template @@ -96,51 +74,221 @@ Cache::regStats() { BaseCache::regStats(); tags->regStats(name()); - missQueue->regStats(name()); coherence->regStats(name()); prefetcher->regStats(name()); } template -typename Cache::BlkType* -Cache::handleAccess(PacketPtr &pkt, int & lat, - PacketList & writebacks, bool update) +Port * +Cache::getPort(const std::string &if_name, int idx) +{ + if (if_name == "" || if_name == "cpu_side") { + return cpuSidePort; + } else if (if_name == "mem_side") { + return memSidePort; + } else if (if_name == "functional") { + return new CpuSidePort(name() + "-cpu_side_funcport", this); + } else { + panic("Port name %s unrecognized\n", if_name); + } +} + +template +void +Cache::deletePortRefs(Port *p) +{ + if (cpuSidePort == p || memSidePort == p) + panic("Can only delete functional ports\n"); + + delete p; +} + + +template +void +Cache::cmpAndSwap(BlkType *blk, PacketPtr pkt) { - // Set the block offset here + uint64_t overwrite_val; + bool overwrite_mem; + uint64_t condition_val64; + uint32_t condition_val32; + int offset = tags->extractBlkOffset(pkt->getAddr()); + uint8_t *blk_data = blk->data + offset; + + assert(sizeof(uint64_t) >= pkt->getSize()); + + overwrite_mem = true; + // keep a copy of our possible write value, and copy what is at the + // memory address into the packet + pkt->writeData((uint8_t *)&overwrite_val); + pkt->setData(blk_data); + + if (pkt->req->isCondSwap()) { + if (pkt->getSize() == sizeof(uint64_t)) { + condition_val64 = pkt->req->getExtraData(); + overwrite_mem = !std::memcmp(&condition_val64, blk_data, + sizeof(uint64_t)); + } else if (pkt->getSize() == sizeof(uint32_t)) { + condition_val32 = (uint32_t)pkt->req->getExtraData(); + overwrite_mem = !std::memcmp(&condition_val32, blk_data, + sizeof(uint32_t)); + } else + panic("Invalid size for conditional read/write\n"); + } + + if (overwrite_mem) + std::memcpy(blk_data, &overwrite_val, pkt->getSize()); +} + + +///////////////////////////////////////////////////// +// +// MSHR helper functions +// +///////////////////////////////////////////////////// + + +template +MSHR * +Cache::allocateBuffer(PacketPtr pkt, Tick time, + bool isFill, bool requestBus) +{ + int size = isFill ? blkSize : pkt->getSize(); + Addr addr = isFill ? tags->blkAlign(pkt->getAddr()) : pkt->getAddr(); + + MSHR *mshr = NULL; + + if (pkt->isWrite()) { + /** + * @todo Add write merging here. + */ + mshr = writeBuffer.allocate(addr, size, pkt, isFill); + mshr->order = order++; - BlkType *blk = NULL; - if (update) { - blk = tags->findBlock(pkt->getAddr(), lat); + if (writeBuffer.isFull()) { + setBlocked(Blocked_NoWBBuffers); + } + + if (requestBus) { + requestMemSideBus(Request_WB, time); + } } else { - blk = tags->findBlock(pkt->getAddr()); - lat = 0; + mshr = mshrQueue.allocate(addr, size, pkt, isFill); + mshr->order = order++; + if (mshrQueue.isFull()) { + setBlocked(Blocked_NoMSHRs); + } + if (requestBus) { + requestMemSideBus(Request_MSHR, time); + } } - if (blk != NULL) { - if (!update) { + assert(mshr != NULL); + return mshr; +} - if (pkt->isWrite()){ - assert(offset < blkSize); - assert(pkt->getSize() <= blkSize); - assert(offset+pkt->getSize() <= blkSize); - std::memcpy(blk->data + offset, pkt->getPtr(), - pkt->getSize()); - } else if (pkt->isReadWrite()) { - cmpAndSwap(blk, pkt); - } else if (!(pkt->flags & SATISFIED)) { - pkt->flags |= SATISFIED; - pkt->result = Packet::Success; - assert(offset < blkSize); - assert(pkt->getSize() <= blkSize); - assert(offset + pkt->getSize() <=blkSize); - std::memcpy(pkt->getPtr(), blk->data + offset, - pkt->getSize()); + +template +void +Cache::markInService(MSHR *mshr) +{ + bool unblock = false; + BlockedCause cause = NUM_BLOCKED_CAUSES; + + /** + * @todo Should include MSHRQueue pointer in MSHR to select the correct + * one. + */ + if (mshr->queue == &writeBuffer) { + // Forwarding a write/ writeback, don't need to change + // the command + unblock = writeBuffer.isFull(); + writeBuffer.markInService(mshr); + if (!writeBuffer.havePending()){ + deassertMemSideBusRequest(Request_WB); + } + if (unblock) { + // Do we really unblock? + unblock = !writeBuffer.isFull(); + cause = Blocked_NoWBBuffers; + } + } else { + assert(mshr->queue == &mshrQueue); + unblock = mshrQueue.isFull(); + mshrQueue.markInService(mshr); + if (!mshrQueue.havePending()){ + deassertMemSideBusRequest(Request_MSHR); + } +#if 0 + if (mshr->originalCmd == MemCmd::HardPFReq) { + DPRINTF(HWPrefetch, "%s:Marking a HW_PF in service\n", + name()); + //Also clear pending if need be + if (!prefetcher->havePending()) + { + deassertMemSideBusRequest(Request_PF); } - return blk; } +#endif + if (unblock) { + unblock = !mshrQueue.isFull(); + cause = Blocked_NoMSHRs; + } + } + if (unblock) { + clearBlocked(cause); + } +} + + +template +void +Cache::squash(int threadNum) +{ + bool unblock = false; + BlockedCause cause = NUM_BLOCKED_CAUSES; - // Hit + if (noTargetMSHR && noTargetMSHR->threadNum == threadNum) { + noTargetMSHR = NULL; + unblock = true; + cause = Blocked_NoTargets; + } + if (mshrQueue.isFull()) { + unblock = true; + cause = Blocked_NoMSHRs; + } + mshrQueue.squash(threadNum); + if (!mshrQueue.havePending()) { + deassertMemSideBusRequest(Request_MSHR); + } + if (unblock && !mshrQueue.isFull()) { + clearBlocked(cause); + } +} + +///////////////////////////////////////////////////// +// +// Access path: requests coming in from the CPU side +// +///////////////////////////////////////////////////// + +template +bool +Cache::access(PacketPtr pkt, BlkType *blk, int &lat) +{ + bool satisfied = false; // assume the worst + + if (prefetchAccess) { + //We are determining prefetches on access stream, call prefetcher + prefetcher->handleMiss(pkt, curTick); + } + + DPRINTF(Cache, "%s %x %s\n", pkt->cmdString(), pkt->getAddr(), + (blk) ? "hit" : "miss"); + + if (blk != NULL) { + // HIT if (blk->isPrefetch()) { //Signal that this was a hit under prefetch (no need for //use prefetch (only can get here if true) @@ -154,639 +302,620 @@ Cache::handleAccess(PacketPtr &pkt, int & lat, } } - if ((pkt->isReadWrite() && blk->isWritable()) || - (pkt->isWrite() && blk->isWritable()) || - (pkt->isRead() && blk->isValid())) { - - // We are satisfying the request - pkt->flags |= SATISFIED; - - if (blk->isCompressed()) { - // If the data is compressed, need to increase the latency - lat += (compLatency/4); - } - - bool write_data = false; - - assert(verifyData(blk)); - - assert(offset < blkSize); - assert(pkt->getSize() <= blkSize); - assert(offset+pkt->getSize() <= blkSize); + if (pkt->needsExclusive() ? blk->isWritable() : blk->isValid()) { + // OK to satisfy access + hits[pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/]++; + satisfied = true; - if (pkt->isWrite()) { - if (blk->checkWrite(pkt->req)) { - write_data = true; + if (pkt->cmd == MemCmd::SwapReq) { + cmpAndSwap(blk, pkt); + } else if (pkt->isWrite()) { + if (blk->checkWrite(pkt)) { blk->status |= BlkDirty; - std::memcpy(blk->data + offset, pkt->getPtr(), - pkt->getSize()); + pkt->writeDataToBlock(blk->data, blkSize); } - } else if (pkt->isReadWrite()) { - cmpAndSwap(blk, pkt); } else { assert(pkt->isRead()); - if (pkt->req->isLocked()) { - blk->trackLoadLocked(pkt->req); + if (pkt->isLocked()) { + blk->trackLoadLocked(pkt); } - std::memcpy(pkt->getPtr(), blk->data + offset, - pkt->getSize()); - } - - if (write_data || - (adaptiveCompression && blk->isCompressed())) - { - // If we wrote data, need to update the internal block - // data. - updateData(blk, writebacks, - !(adaptiveCompression && - blk->isReferenced())); + pkt->setDataFromBlock(blk->data, blkSize); } } else { - // permission violation, treat it as a miss - blk = NULL; + // permission violation... nothing to do here, leave unsatisfied + // for statistics purposes this counts like a complete miss + incMissCount(pkt); } } else { // complete miss (no matching block) - if (pkt->req->isLocked() && pkt->isWrite()) { + incMissCount(pkt); + + if (pkt->isLocked() && pkt->isWrite()) { // miss on store conditional... just give up now pkt->req->setExtraData(0); - pkt->flags |= SATISFIED; + satisfied = true; } } - return blk; + return satisfied; } -template -void -Cache::cmpAndSwap(BlkType *blk, PacketPtr &pkt){ - uint64_t overwrite_val; - bool overwrite_mem; - uint64_t condition_val64; - uint32_t condition_val32; - - int offset = tags->extractBlkOffset(pkt->getAddr()); - - assert(sizeof(uint64_t) >= pkt->getSize()); - - overwrite_mem = true; - // keep a copy of our possible write value, and copy what is at the - // memory address into the packet - std::memcpy(&overwrite_val, pkt->getPtr(), pkt->getSize()); - std::memcpy(pkt->getPtr(), blk->data + offset, - pkt->getSize()); - - if (pkt->req->isCondSwap()) { - if (pkt->getSize() == sizeof(uint64_t)) { - condition_val64 = pkt->req->getExtraData(); - overwrite_mem = !std::memcmp(&condition_val64, blk->data + offset, - sizeof(uint64_t)); - } else if (pkt->getSize() == sizeof(uint32_t)) { - condition_val32 = (uint32_t)pkt->req->getExtraData(); - overwrite_mem = !std::memcmp(&condition_val32, blk->data + offset, - sizeof(uint32_t)); - } else - panic("Invalid size for conditional read/write\n"); - } - - if (overwrite_mem) - std::memcpy(blk->data + offset, - &overwrite_val, pkt->getSize()); - -} template -typename Cache::BlkType* -Cache::handleFill(BlkType *blk, PacketPtr &pkt, - CacheBlk::State new_state, - PacketList & writebacks, - PacketPtr target) +bool +Cache::timingAccess(PacketPtr pkt) { -#ifndef NDEBUG - BlkType *tmp_blk = tags->findBlock(pkt->getAddr()); - assert(tmp_blk == blk); -#endif - blk = doReplacement(blk, pkt, new_state, writebacks); +//@todo Add back in MemDebug Calls +// MemDebug::cacheAccess(pkt); + // we charge hitLatency for doing just about anything here + Tick time = curTick + hitLatency; - if (pkt->isRead()) { - std::memcpy(blk->data, pkt->getPtr(), blkSize); + if (pkt->req->isUncacheable()) { + allocateBuffer(pkt, time, false, true); + assert(pkt->needsResponse()); // else we should delete it here?? + return true; } - blk->whenReady = pkt->finishTime; + PacketList writebacks; + int lat = hitLatency; + BlkType *blk = tags->findBlock(pkt->getAddr(), lat); + bool satisfied = false; + + Addr blk_addr = pkt->getAddr() & ~(Addr(blkSize-1)); - // Respond to target, if any - if (target) { + MSHR *mshr = mshrQueue.findMatch(blk_addr); - target->flags |= SATISFIED; + if (!mshr) { + // no outstanding access to this block, look up in cache + // (otherwise if we allow reads while there's an outstanding + // write miss, the read could return stale data out of the + // cache block... a more aggressive system could detect the + // overlap (if any) and forward data out of the MSHRs, but we + // don't do that yet) + satisfied = access(pkt, blk, lat); + } - if (target->cmd == MemCmd::InvalidateReq) { - tags->invalidateBlk(blk); - blk = NULL; +#if 0 + // If this is a block size write/hint (WH64) allocate the block here + // if the coherence protocol allows it. + /** @todo make the fast write alloc (wh64) work with coherence. */ + /** @todo Do we want to do fast writes for writebacks as well? */ + if (!blk && pkt->getSize() >= blkSize && coherence->allowFastWrites() && + (pkt->cmd == MemCmd::WriteReq + || pkt->cmd == MemCmd::WriteInvalidateReq) ) { + // not outstanding misses, can do this + MSHR *outstanding_miss = mshrQueue.findMatch(pkt->getAddr()); + if (pkt->cmd == MemCmd::WriteInvalidateReq || !outstanding_miss) { + if (outstanding_miss) { + warn("WriteInv doing a fastallocate" + "with an outstanding miss to the same address\n"); + } + blk = handleFill(NULL, pkt, BlkValid | BlkWritable, + writebacks); + ++fastWrites; } + } +#endif - if (blk && ((target->isWrite() || target->isReadWrite()) ? - blk->isWritable() : blk->isValid())) { - assert(target->isWrite() || target->isReadWrite() || target->isRead()); - assert(target->getOffset(blkSize) + target->getSize() <= blkSize); - if (target->isWrite()) { - if (blk->checkWrite(pkt->req)) { - blk->status |= BlkDirty; - std::memcpy(blk->data + target->getOffset(blkSize), - target->getPtr(), target->getSize()); - } - } else if (target->isReadWrite()) { - cmpAndSwap(blk, target); - } else { - if (pkt->req->isLocked()) { - blk->trackLoadLocked(pkt->req); - } - std::memcpy(target->getPtr(), - blk->data + target->getOffset(blkSize), - target->getSize()); + // copy writebacks to write buffer + while (!writebacks.empty()) { + PacketPtr wbPkt = writebacks.front(); + allocateBuffer(wbPkt, time, false, true); + writebacks.pop_front(); + } + + bool needsResponse = pkt->needsResponse(); + + if (satisfied) { + assert(needsResponse); + pkt->makeTimingResponse(); + cpuSidePort->respond(pkt, curTick+lat); + } else { + // miss + if (prefetchMiss) + prefetcher->handleMiss(pkt, time); + + if (mshr) { + // MSHR hit + //@todo remove hw_pf here + mshr_hits[pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/]++; + if (mshr->threadNum != 0/*pkt->req->getThreadNum()*/) { + mshr->threadNum = -1; } + mshr->allocateTarget(pkt, true); + if (mshr->getNumTargets() == numTarget) { + noTargetMSHR = mshr; + setBlocked(Blocked_NoTargets); + mshrQueue.moveToFront(mshr); + } + } else { + // no MSHR + mshr_misses[pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/]++; + // always mark as cache fill for now... if we implement + // no-write-allocate or bypass accesses this will have to + // be changed. + allocateBuffer(pkt, time, true, true); } } - if (blk) { - // Need to write the data into the block - updateData(blk, writebacks, !adaptiveCompression || true); + if (!needsResponse) { + // Need to clean up the packet on a writeback miss, but leave + // the request for the next level. + delete pkt; } - return blk; + + return true; } + template -typename Cache::BlkType* -Cache::handleFill(BlkType *blk, MSHR * mshr, - CacheBlk::State new_state, - PacketList & writebacks, PacketPtr pkt) +Tick +Cache::atomicAccess(PacketPtr pkt) { -/* -#ifndef NDEBUG - BlkType *tmp_blk = findBlock(mshr->pkt->getAddr()); - assert(tmp_blk == blk); -#endif - PacketPtr pkt = mshr->pkt;*/ - blk = doReplacement(blk, pkt, new_state, writebacks); + // should assert here that there are no outstanding MSHRs or + // writebacks... that would mean that someone used an atomic + // access in timing mode - if (pkt->isRead()) { - std::memcpy(blk->data, pkt->getPtr(), blkSize); + if (pkt->req->isUncacheable()) { + // Uncacheables just go through + return memSidePort->sendAtomic(pkt); } - blk->whenReady = pkt->finishTime; + PacketList writebacks; + int lat = hitLatency; + BlkType *blk = tags->findBlock(pkt->getAddr(), lat); + bool satisfied = access(pkt, blk, lat); + if (!satisfied) { + // MISS + CacheBlk::State old_state = (blk) ? blk->status : 0; + MemCmd cmd = coherence->getBusCmd(pkt->cmd, old_state); + Packet busPkt = Packet(pkt->req, cmd, Packet::Broadcast, blkSize); + busPkt.allocate(); - // respond to MSHR targets, if any + DPRINTF(Cache, "Sending a atomic %s for %x\n", + busPkt.cmdString(), busPkt.getAddr()); - // First offset for critical word first calculations - int initial_offset = 0; + lat += memSidePort->sendAtomic(&busPkt); - if (mshr->hasTargets()) { - initial_offset = mshr->getTarget()->getOffset(blkSize); + DPRINTF(Cache, "Receive response: %s for addr %x in state %i\n", + busPkt.cmdString(), busPkt.getAddr(), old_state); + + blk = handleFill(&busPkt, blk, writebacks); + bool status = satisfyCpuSideRequest(pkt, blk); + assert(status); } - while (mshr->hasTargets()) { - PacketPtr target = mshr->getTarget(); + // We now have the block one way or another (hit or completed miss) - target->flags |= SATISFIED; + // Handle writebacks if needed + while (!writebacks.empty()){ + PacketPtr wbPkt = writebacks.front(); + memSidePort->sendAtomic(wbPkt); + writebacks.pop_front(); + delete wbPkt; + } - // How many bytes pass the first request is this one - int transfer_offset = target->getOffset(blkSize) - initial_offset; - if (transfer_offset < 0) { - transfer_offset += blkSize; - } + if (pkt->needsResponse()) { + pkt->makeAtomicResponse(); + pkt->result = Packet::Success; + } - // If critical word (no offset) return first word time - Tick completion_time = tags->getHitLatency() + - transfer_offset ? pkt->finishTime : pkt->firstWordTime; + return lat; +} - if (target->cmd == MemCmd::InvalidateReq) { - //Mark the blk as invalid now, if it hasn't been already - if (blk) { - tags->invalidateBlk(blk); - blk = NULL; - } - //Also get rid of the invalidate - mshr->popTarget(); +template +void +Cache::functionalAccess(PacketPtr pkt, + CachePort *otherSidePort) +{ + Addr blk_addr = pkt->getAddr() & ~(blkSize - 1); + BlkType *blk = tags->findBlock(pkt->getAddr()); - DPRINTF(Cache, "Popping off a Invalidate for addr %x\n", - pkt->getAddr()); + if (blk && pkt->checkFunctional(blk_addr, blkSize, blk->data)) { + // request satisfied from block + return; + } - continue; - } + // Need to check for outstanding misses and writes - if (blk && ((target->isWrite() || target->isReadWrite()) ? - blk->isWritable() : blk->isValid())) { - assert(target->isWrite() || target->isRead() || target->isReadWrite() ); - assert(target->getOffset(blkSize) + target->getSize() <= blkSize); - if (target->isWrite()) { - if (blk->checkWrite(pkt->req)) { - blk->status |= BlkDirty; - std::memcpy(blk->data + target->getOffset(blkSize), - target->getPtr(), target->getSize()); - } - } else if (target->isReadWrite()) { - cmpAndSwap(blk, target); - } else { - if (target->req->isLocked()) { - blk->trackLoadLocked(target->req); - } - std::memcpy(target->getPtr(), - blk->data + target->getOffset(blkSize), - target->getSize()); - } - } else { - // Invalid access, need to do another request - // can occur if block is invalidated, or not correct - // permissions -// mshr->pkt = pkt; - break; - } - if (!target->req->isUncacheable()) { - missLatency[target->cmdToIndex()][0/*pkt->req->getThreadNum()*/] += - completion_time - target->time; + // There can only be one matching outstanding miss. + MSHR *mshr = mshrQueue.findMatch(blk_addr); + if (mshr) { + MSHR::TargetList *targets = mshr->getTargetList(); + MSHR::TargetList::iterator i = targets->begin(); + MSHR::TargetList::iterator end = targets->end(); + for (; i != end; ++i) { + PacketPtr targetPkt = i->pkt; + if (pkt->checkFunctional(targetPkt)) + return; } - respond(target, completion_time); - mshr->popTarget(); } - if (blk) { - // Need to write the data into the block - updateData(blk, writebacks, !adaptiveCompression || true); + // There can be many matching outstanding writes. + std::vector writes; + writeBuffer.findMatches(blk_addr, writes); + for (int i = 0; i < writes.size(); ++i) { + MSHR *mshr = writes[i]; + if (pkt->checkFunctional(mshr->addr, mshr->size, mshr->writeData)) + return; } - return blk; + otherSidePort->checkAndSendFunctional(pkt); } +///////////////////////////////////////////////////// +// +// Response handling: responses from the memory side +// +///////////////////////////////////////////////////// + template void -Cache::handleSnoop(BlkType *blk, - CacheBlk::State new_state, - PacketPtr &pkt) +Cache::handleResponse(PacketPtr pkt, Tick time) { - //Must have the block to supply - assert(blk); - // Can only supply data, and if it hasn't already been supllied - assert(pkt->isRead()); - assert(!(pkt->flags & SATISFIED)); - pkt->flags |= SATISFIED; - Addr offset = pkt->getOffset(blkSize); - assert(offset < blkSize); - assert(pkt->getSize() <= blkSize); - assert(offset + pkt->getSize() <=blkSize); - std::memcpy(pkt->getPtr(), blk->data + offset, pkt->getSize()); - - handleSnoop(blk, new_state); + MSHR *mshr = dynamic_cast(pkt->senderState); +#ifndef NDEBUG + int num_targets = mshr->getNumTargets(); +#endif + + bool unblock = false; + bool unblock_target = false; + BlockedCause cause = NUM_BLOCKED_CAUSES; + + if (mshr->isCacheFill) { +#if 0 + mshr_miss_latency[mshr->originalCmd.toInt()][0/*pkt->req->getThreadNum()*/] += + curTick - pkt->time; +#endif + // targets were handled in the cache tags + if (mshr == noTargetMSHR) { + // we always clear at least one target + unblock_target = true; + cause = Blocked_NoTargets; + noTargetMSHR = NULL; + } + + if (mshr->hasTargets()) { + // Didn't satisfy all the targets, need to resend + mshrQueue.markPending(mshr); + mshr->order = order++; + requestMemSideBus(Request_MSHR, time); + } + else { + unblock = mshrQueue.isFull(); + mshrQueue.deallocate(mshr); + if (unblock) { + unblock = !mshrQueue.isFull(); + cause = Blocked_NoMSHRs; + } + } + } else { + if (pkt->req->isUncacheable()) { + mshr_uncacheable_lat[pkt->cmd.toInt()][0/*pkt->req->getThreadNum()*/] += + curTick - pkt->time; + } + if (mshr->hasTargets() && pkt->req->isUncacheable()) { + // Should only have 1 target if we had any + assert(num_targets == 1); + MSHR::Target *target = mshr->getTarget(); + assert(target->cpuSide); + mshr->popTarget(); + if (pkt->isRead()) { + target->pkt->setData(pkt->getPtr()); + } + cpuSidePort->respond(target->pkt, time); + assert(!mshr->hasTargets()); + } + else if (mshr->hasTargets()) { + //Must be a no_allocate with possibly more than one target + assert(!mshr->isCacheFill); + while (mshr->hasTargets()) { + MSHR::Target *target = mshr->getTarget(); + assert(target->isCpuSide()); + mshr->popTarget(); + if (pkt->isRead()) { + target->pkt->setData(pkt->getPtr()); + } + cpuSidePort->respond(target->pkt, time); + } + } + + if (pkt->isWrite()) { + // If the wrtie buffer is full, we might unblock now + unblock = writeBuffer.isFull(); + writeBuffer.deallocate(mshr); + if (unblock) { + // Did we really unblock? + unblock = !writeBuffer.isFull(); + cause = Blocked_NoWBBuffers; + } + } else { + unblock = mshrQueue.isFull(); + mshrQueue.deallocate(mshr); + if (unblock) { + unblock = !mshrQueue.isFull(); + cause = Blocked_NoMSHRs; + } + } + } + if (unblock || unblock_target) { + clearBlocked(cause); + } } + template void -Cache::handleSnoop(BlkType *blk, - CacheBlk::State new_state) +Cache::handleResponse(PacketPtr pkt) { - if (blk && blk->status != new_state) { - if ((new_state && BlkValid) == 0) { - tags->invalidateBlk(blk); - } else { - assert(new_state >= 0 && new_state < 128); - blk->status = new_state; + Tick time = curTick + hitLatency; + MSHR *mshr = dynamic_cast(pkt->senderState); + assert(mshr); + if (pkt->result == Packet::Nacked) { + //pkt->reinitFromRequest(); + warn("NACKs from devices not connected to the same bus " + "not implemented\n"); + return; + } + assert(pkt->result != Packet::BadAddress); + assert(pkt->result == Packet::Success); + DPRINTF(Cache, "Handling reponse to %x\n", pkt->getAddr()); + + if (mshr->isCacheFill) { + DPRINTF(Cache, "Block for addr %x being updated in Cache\n", + pkt->getAddr()); + BlkType *blk = tags->findBlock(pkt->getAddr()); + PacketList writebacks; + blk = handleFill(pkt, blk, writebacks); + satisfyMSHR(mshr, pkt, blk); + // copy writebacks to write buffer + while (!writebacks.empty()) { + PacketPtr wbPkt = writebacks.front(); + allocateBuffer(wbPkt, time, false, true); + writebacks.pop_front(); } } + handleResponse(pkt, time); } + + + template PacketPtr Cache::writebackBlk(BlkType *blk) { - assert(blk && blk->isValid() && blk->isModified()); - int data_size = blkSize; - data_size = blk->size; - if (compressOnWriteback) { - // not already compressed - // need to compress to ship it - assert(data_size == blkSize); - uint8_t *tmp_data = new uint8_t[blkSize]; - data_size = compressionAlg->compress(tmp_data,blk->data, - data_size); - delete [] tmp_data; - } + assert(blk && blk->isValid() && blk->isDirty()); -/* PacketPtr writeback = - buildWritebackReq(tags->regenerateBlkAddr(blk->tag, blk->set), - blk->asid, blkSize, - blk->data, data_size); -*/ + writebacks[0/*pkt->req->getThreadNum()*/]++; Request *writebackReq = new Request(tags->regenerateBlkAddr(blk->tag, blk->set), blkSize, 0); PacketPtr writeback = new Packet(writebackReq, MemCmd::Writeback, -1); writeback->allocate(); - std::memcpy(writeback->getPtr(),blk->data,blkSize); + std::memcpy(writeback->getPtr(), blk->data, blkSize); blk->status &= ~BlkDirty; return writeback; } -template -bool -Cache::verifyData(BlkType *blk) -{ - bool retval; - // The data stored in the blk - uint8_t *blk_data = new uint8_t[blkSize]; - tags->readData(blk, blk_data); - // Pointer for uncompressed data, assumed uncompressed - uint8_t *tmp_data = blk_data; - // The size of the data being stored, assumed uncompressed - int data_size = blkSize; - - // If the block is compressed need to uncompress to access - if (blk->isCompressed()){ - // Allocate new storage for the data - tmp_data = new uint8_t[blkSize]; - data_size = compressionAlg->uncompress(tmp_data,blk_data, blk->size); - assert(data_size == blkSize); - // Don't need to keep blk_data around - delete [] blk_data; - } else { - assert(blkSize == blk->size); - } - - retval = std::memcmp(tmp_data, blk->data, blkSize) == 0; - delete [] tmp_data; - return retval; -} - -template -void -Cache::updateData(BlkType *blk, PacketList &writebacks, - bool compress_block) -{ - if (storeCompressed && compress_block) { - uint8_t *comp_data = new uint8_t[blkSize]; - int new_size = compressionAlg->compress(comp_data, blk->data, blkSize); - if (new_size > (blkSize - tags->getSubBlockSize())){ - // no benefit to storing it compressed - blk->status &= ~BlkCompressed; - tags->writeData(blk, blk->data, blkSize, - writebacks); - } else { - // Store the data compressed - blk->status |= BlkCompressed; - tags->writeData(blk, comp_data, new_size, - writebacks); - } - delete [] comp_data; - } else { - blk->status &= ~BlkCompressed; - tags->writeData(blk, blk->data, blkSize, writebacks); - } -} - +// Note that the reason we return a list of writebacks rather than +// inserting them directly in the write buffer is that this function +// is called by both atomic and timing-mode accesses, and in atomic +// mode we don't mess with the write buffer (we just perform the +// writebacks atomically once the original request is complete). template typename Cache::BlkType* -Cache::doReplacement(BlkType *blk, PacketPtr &pkt, - CacheBlk::State new_state, - PacketList &writebacks) +Cache::handleFill(PacketPtr pkt, BlkType *blk, + PacketList &writebacks) { + Addr addr = pkt->getAddr(); + if (blk == NULL) { + // need to do a replacement - BlkList compress_list; - blk = tags->findReplacement(pkt, writebacks, compress_list); - while (adaptiveCompression && !compress_list.empty()) { - updateData(compress_list.front(), writebacks, true); - compress_list.pop_front(); - } + blk = tags->findReplacement(addr, writebacks); if (blk->isValid()) { DPRINTF(Cache, "replacement: replacing %x with %x: %s\n", - tags->regenerateBlkAddr(blk->tag,blk->set), pkt->getAddr(), - (blk->isModified()) ? "writeback" : "clean"); + tags->regenerateBlkAddr(blk->tag, blk->set), addr, + blk->isDirty() ? "writeback" : "clean"); - if (blk->isModified()) { - // Need to write the data back + if (blk->isDirty()) { + // Save writeback packet for handling by caller writebacks.push_back(writebackBlk(blk)); } } - blk->tag = tags->extractTag(pkt->getAddr(), blk); + + blk->tag = tags->extractTag(addr); + blk->status = coherence->getNewState(pkt); + assert(pkt->isRead()); } else { - // must be a status change - // assert(blk->status != new_state); - if (blk->status == new_state) warn("Changing state to same value\n"); + // existing block... probably an upgrade + assert(blk->tag == tags->extractTag(addr)); + // either we're getting new data or the block should already be valid + assert(pkt->isRead() || blk->isValid()); + CacheBlk::State old_state = blk->status; + blk->status = coherence->getNewState(pkt, old_state); + if (blk->status != old_state) + DPRINTF(Cache, "Block addr %x moving from state %i to %i\n", + addr, old_state, blk->status); + else + warn("Changing state to same value\n"); } - blk->status = new_state; + // if we got new data, copy it in + if (pkt->isRead()) { + std::memcpy(blk->data, pkt->getPtr(), blkSize); + } + + blk->whenReady = pkt->finishTime; + return blk; } template bool -Cache::access(PacketPtr &pkt) +Cache::satisfyCpuSideRequest(PacketPtr pkt, BlkType *blk) { -//@todo Add back in MemDebug Calls -// MemDebug::cacheAccess(pkt); - BlkType *blk = NULL; - PacketList writebacks; - int size = blkSize; - int lat = hitLatency; - if (prefetchAccess) { - //We are determining prefetches on access stream, call prefetcher - prefetcher->handleMiss(pkt, curTick); - } - - Addr blk_addr = pkt->getAddr() & ~(Addr(blkSize-1)); - - if (!pkt->req->isUncacheable()) { - if (!missQueue->findMSHR(blk_addr)) { - blk = handleAccess(pkt, lat, writebacks); - } - } else { - size = pkt->getSize(); - } - // If this is a block size write/hint (WH64) allocate the block here - // if the coherence protocol allows it. - /** @todo make the fast write alloc (wh64) work with coherence. */ - /** @todo Do we want to do fast writes for writebacks as well? */ - if (!blk && pkt->getSize() >= blkSize && coherence->allowFastWrites() && - (pkt->cmd == MemCmd::WriteReq - || pkt->cmd == MemCmd::WriteInvalidateReq) ) { - // not outstanding misses, can do this - MSHR* outstanding_miss = missQueue->findMSHR(pkt->getAddr()); - if (pkt->cmd == MemCmd::WriteInvalidateReq || !outstanding_miss) { - if (outstanding_miss) { - warn("WriteInv doing a fastallocate" - "with an outstanding miss to the same address\n"); + if (blk && (pkt->needsExclusive() ? blk->isWritable() : blk->isValid())) { + assert(pkt->isWrite() || pkt->isReadWrite() || pkt->isRead()); + assert(pkt->getOffset(blkSize) + pkt->getSize() <= blkSize); + + if (pkt->isWrite()) { + if (blk->checkWrite(pkt)) { + blk->status |= BlkDirty; + pkt->writeDataToBlock(blk->data, blkSize); } - blk = handleFill(NULL, pkt, BlkValid | BlkWritable, - writebacks); - ++fastWrites; + } else if (pkt->isReadWrite()) { + cmpAndSwap(blk, pkt); + } else { + if (pkt->isLocked()) { + blk->trackLoadLocked(pkt); + } + pkt->setDataFromBlock(blk->data, blkSize); } + + return true; + } else { + return false; } - while (!writebacks.empty()) { - PacketPtr wbPkt = writebacks.front(); - missQueue->doWriteback(wbPkt); - writebacks.pop_front(); - delete wbPkt; - } +} + + +template +bool +Cache::satisfyTarget(MSHR::Target *target, BlkType *blk) +{ + assert(target != NULL); + assert(target->isCpuSide()); + return satisfyCpuSideRequest(target->pkt, blk); +} + +template +void +Cache::satisfyMSHR(MSHR *mshr, PacketPtr pkt, + BlkType *blk) +{ + // respond to MSHR targets, if any - DPRINTF(Cache, "%s %x %s\n", pkt->cmdString(), pkt->getAddr(), - (blk) ? "hit" : "miss"); + // First offset for critical word first calculations + int initial_offset = 0; - if (blk) { - // Hit - hits[pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/]++; - // clear dirty bit if write through - respond(pkt, curTick+lat); - return true; + if (mshr->hasTargets()) { + initial_offset = mshr->getTarget()->pkt->getOffset(blkSize); } - // Miss - if (!pkt->req->isUncacheable()) { - misses[pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/]++; - /** @todo Move miss count code into BaseCache */ - if (missCount) { - --missCount; - if (missCount == 0) - exitSimLoop("A cache reached the maximum miss count"); - } - } + while (mshr->hasTargets()) { + MSHR::Target *target = mshr->getTarget(); - if (pkt->flags & SATISFIED) { - // happens when a store conditional fails because it missed - // the cache completely - respond(pkt, curTick+lat); - } else { - missQueue->handleMiss(pkt, size, curTick + hitLatency); - } + if (!satisfyTarget(target, blk)) { + // Invalid access, need to do another request + // can occur if block is invalidated, or not correct + // permissions + break; + } - if (!pkt->needsResponse()) { - //Need to clean up the packet on a writeback miss, but leave the request - //for the next level. - delete pkt; - } - return true; -} + // How many bytes pass the first request is this one + int transfer_offset = target->pkt->getOffset(blkSize) - initial_offset; + if (transfer_offset < 0) { + transfer_offset += blkSize; + } + // If critical word (no offset) return first word time + Tick completion_time = tags->getHitLatency() + + transfer_offset ? pkt->finishTime : pkt->firstWordTime; -template -PacketPtr -Cache::getPacket() -{ - assert(missQueue->havePending()); - PacketPtr pkt = missQueue->getPacket(); - if (pkt) { - if (!pkt->req->isUncacheable()) { - if (pkt->cmd == MemCmd::HardPFReq) - misses[MemCmd::HardPFReq][0/*pkt->req->getThreadNum()*/]++; - BlkType *blk = tags->findBlock(pkt->getAddr()); - MemCmd cmd = - coherence->getBusCmd(pkt->cmd, (blk) ? blk->status : 0); - missQueue->setBusCmd(pkt, cmd); + if (!target->pkt->req->isUncacheable()) { + missLatency[target->pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/] += + completion_time - target->time; } + target->pkt->makeTimingResponse(); + cpuSidePort->respond(target->pkt, completion_time); + mshr->popTarget(); } - - assert(!isMemSideBusRequested() || missQueue->havePending()); - assert(!pkt || pkt->time <= curTick); - SIGNAL_NACK_HACK = false; - return pkt; } + +///////////////////////////////////////////////////// +// +// Snoop path: requests coming in from the memory side +// +///////////////////////////////////////////////////// + template void -Cache::sendResult(PacketPtr &pkt, MSHR* mshr, - bool success) +Cache::doTimingSupplyResponse(PacketPtr req_pkt, + uint8_t *blk_data) { - if (success && !(SIGNAL_NACK_HACK)) { - //Remember if it was an upgrade because writeback MSHR's are removed - //in Mark in Service - bool upgrade = (mshr->pkt && mshr->pkt->cmd == MemCmd::UpgradeReq); - - missQueue->markInService(mshr->pkt, mshr); - - //Temp Hack for UPGRADES - if (upgrade) { - assert(pkt); //Upgrades need to be fixed - pkt->flags &= ~CACHE_LINE_FILL; - BlkType *blk = tags->findBlock(pkt->getAddr()); - CacheBlk::State old_state = (blk) ? blk->status : 0; - CacheBlk::State new_state = coherence->getNewState(pkt,old_state); - if (old_state != new_state) - DPRINTF(Cache, "Block for blk addr %x moving from state " - "%i to %i\n", pkt->getAddr(), old_state, new_state); - //Set the state on the upgrade - std::memcpy(pkt->getPtr(), blk->data, blkSize); - PacketList writebacks; - handleFill(blk, mshr, new_state, writebacks, pkt); - assert(writebacks.empty()); - missQueue->handleResponse(pkt, curTick + hitLatency); - } - } else if (pkt && !pkt->req->isUncacheable()) { - pkt->flags &= ~NACKED_LINE; - SIGNAL_NACK_HACK = false; - pkt->flags &= ~SATISFIED; - -//Rmove copy from mshr - delete mshr->pkt; - mshr->pkt = pkt; - - missQueue->restoreOrigCmd(pkt); - } + // timing-mode snoop responses require a new packet + PacketPtr pkt = new Packet(req_pkt); + pkt->allocate(); + pkt->makeTimingResponse(); + pkt->setDataFromBlock(blk_data, blkSize); + memSidePort->respond(pkt, curTick + hitLatency); } template void -Cache::handleResponse(PacketPtr &pkt) +Cache::handleSnoop(PacketPtr pkt, BlkType *blk, + bool is_timing) { - BlkType *blk = NULL; - if (pkt->senderState) { - //Delete temp copy in MSHR, restore it. - delete ((MSHR*)pkt->senderState)->pkt; - ((MSHR*)pkt->senderState)->pkt = pkt; - if (pkt->result == Packet::Nacked) { - //pkt->reinitFromRequest(); - warn("NACKs from devices not connected to the same bus " - "not implemented\n"); - return; - } - if (pkt->result == Packet::BadAddress) { - //Make the response a Bad address and send it + if (!blk || !blk->isValid()) { + return; + } + + // we may end up modifying both the block state and the packet (if + // we respond in atomic mode), so just figure out what to do now + // and then do it later + bool supply = blk->isDirty() && pkt->isRead(); + bool invalidate = pkt->isInvalidate(); + + if (pkt->isRead() && !pkt->isInvalidate()) { + assert(!pkt->needsExclusive()); + pkt->assertShared(); + int bits_to_clear = BlkWritable; + const bool haveOwnershipState = true; // for now + if (!haveOwnershipState) { + // if we don't support pure ownership (dirty && !writable), + // have to clear dirty bit here, assume memory snarfs data + // on cache-to-cache xfer + bits_to_clear |= BlkDirty; } -// MemDebug::cacheResponse(pkt); - DPRINTF(Cache, "Handling reponse to %x\n", pkt->getAddr()); - - if (pkt->isCacheFill() && !pkt->isNoAllocate()) { - DPRINTF(Cache, "Block for addr %x being updated in Cache\n", - pkt->getAddr()); - blk = tags->findBlock(pkt->getAddr()); - CacheBlk::State old_state = (blk) ? blk->status : 0; - PacketList writebacks; - CacheBlk::State new_state = coherence->getNewState(pkt,old_state); - if (old_state != new_state) - DPRINTF(Cache, "Block for blk addr %x moving from " - "state %i to %i\n", - pkt->getAddr(), - old_state, new_state); - blk = handleFill(blk, (MSHR*)pkt->senderState, - new_state, writebacks, pkt); - while (!writebacks.empty()) { - PacketPtr wbPkt = writebacks.front(); - missQueue->doWriteback(wbPkt); - writebacks.pop_front(); - delete wbPkt; - } + blk->status &= ~bits_to_clear; + } + + if (supply) { + pkt->assertMemInhibit(); + if (is_timing) { + doTimingSupplyResponse(pkt, blk->data); + } else { + pkt->makeAtomicResponse(); + pkt->setDataFromBlock(blk->data, blkSize); } - missQueue->handleResponse(pkt, curTick + hitLatency); } + + // Do this last in case it deallocates block data or something + // like that + if (invalidate) { + tags->invalidateBlk(blk); + } + + DPRINTF(Cache, "snooped a %s request for addr %x, %snew state is %i\n", + pkt->cmdString(), blockAlign(pkt->getAddr()), + supply ? "supplying data, " : "", blk->status); } template void -Cache::snoop(PacketPtr &pkt) +Cache::snoopTiming(PacketPtr pkt) { if (pkt->req->isUncacheable()) { //Can't get a hit on an uncacheable address @@ -794,351 +923,190 @@ Cache::snoop(PacketPtr &pkt) return; } - ///// PROPAGATE SNOOP UPWARD HERE + BlkType *blk = tags->findBlock(pkt->getAddr()); Addr blk_addr = pkt->getAddr() & ~(Addr(blkSize-1)); - BlkType *blk = tags->findBlock(pkt->getAddr()); - MSHR *mshr = missQueue->findMSHR(blk_addr); - if (coherence->hasProtocol() || pkt->isInvalidate()) { - //@todo Move this into handle bus req - //If we find an mshr, and it is in service, we need to NACK or - //invalidate - if (mshr) { - if (mshr->inService) { - if ((mshr->pkt->isInvalidate() || !mshr->pkt->isCacheFill()) - && (pkt->cmd != MemCmd::InvalidateReq - && pkt->cmd != MemCmd::WriteInvalidateReq)) { - //If the outstanding request was an invalidate - //(upgrade,readex,..) Then we need to ACK the request - //until we get the data Also NACK if the outstanding - //request is not a cachefill (writeback) - assert(!(pkt->flags & SATISFIED)); - pkt->flags |= SATISFIED; - pkt->flags |= NACKED_LINE; - SIGNAL_NACK_HACK = true; - ///@todo NACK's from other levels - //warn("NACKs from devices not connected to the same bus " - //"not implemented\n"); - //respondToSnoop(pkt, curTick + hitLatency); - return; - } - else { - //The supplier will be someone else, because we are - //waiting for the data. This should cause this cache to - //be forced to go to the shared state, not the exclusive - //even though the shared line won't be asserted. But for - //now we will just invlidate ourselves and allow the other - //cache to go into the exclusive state. @todo Make it so - //a read to a pending read doesn't invalidate. @todo Make - //it so that a read to a pending read can't be exclusive - //now. - - //Set the address so find match works - //panic("Don't have invalidates yet\n"); - invalidatePkt->addrOverride(pkt->getAddr()); - - //Append the invalidate on - missQueue->addTarget(mshr,invalidatePkt); - DPRINTF(Cache, "Appending Invalidate to addr: %x\n", - pkt->getAddr()); - return; + MSHR *mshr = mshrQueue.findMatch(blk_addr); + // better not be snooping a request that conflicts with something + // we have outstanding... + assert(!mshr || !mshr->inService); + + //We also need to check the writeback buffers and handle those + std::vector writebacks; + if (writeBuffer.findMatches(blk_addr, writebacks)) { + DPRINTF(Cache, "Snoop hit in writeback to addr: %x\n", + pkt->getAddr()); + + //Look through writebacks for any non-uncachable writes, use that + for (int i=0; iisUncacheable()); + + if (pkt->isRead()) { + pkt->assertMemInhibit(); + if (!pkt->needsExclusive()) { + pkt->assertShared(); + } else { + // if we're not asserting the shared line, we need to + // invalidate our copy. we'll do that below as long as + // the packet's invalidate flag is set... + assert(pkt->isInvalidate()); } + doTimingSupplyResponse(pkt, mshr->writeData); } - } - //We also need to check the writeback buffers and handle those - std::vector writebacks; - if (missQueue->findWrites(blk_addr, writebacks)) { - DPRINTF(Cache, "Snoop hit in writeback to addr: %x\n", - pkt->getAddr()); - - //Look through writebacks for any non-uncachable writes, use that - for (int i=0; ipkt->req->isUncacheable()) { - if (pkt->isRead()) { - //Only Upgrades don't get here - //Supply the data - assert(!(pkt->flags & SATISFIED)); - pkt->flags |= SATISFIED; - - //If we are in an exclusive protocol, make it ask again - //to get write permissions (upgrade), signal shared - pkt->flags |= SHARED_LINE; - - assert(pkt->isRead()); - Addr offset = pkt->getAddr() & (blkSize - 1); - assert(offset < blkSize); - assert(pkt->getSize() <= blkSize); - assert(offset + pkt->getSize() <=blkSize); - std::memcpy(pkt->getPtr(), mshr->pkt->getPtr() + offset, pkt->getSize()); - - respondToSnoop(pkt, curTick + hitLatency); - } - - if (pkt->isInvalidate()) { - //This must be an upgrade or other cache will take - //ownership - missQueue->markInService(mshr->pkt, mshr); - } - return; - } + + if (pkt->isInvalidate()) { + // Invalidation trumps our writeback... discard here + assert(0); + markInService(mshr); } + return; } } - CacheBlk::State new_state; - bool satisfy = coherence->handleBusRequest(pkt,blk,mshr, new_state); - - if (blk && mshr && !mshr->inService && new_state == 0) { - //There was a outstanding write to a shared block, not need ReadEx - //not update, so change No Allocate param in MSHR - mshr->pkt->flags &= ~NO_ALLOCATE; - } - - if (satisfy) { - DPRINTF(Cache, "Cache snooped a %s request for addr %x and " - "now supplying data, new state is %i\n", - pkt->cmdString(), blk_addr, new_state); - - handleSnoop(blk, new_state, pkt); - respondToSnoop(pkt, curTick + hitLatency); - return; - } - if (blk) - DPRINTF(Cache, "Cache snooped a %s request for addr %x, " - "new state is %i\n", pkt->cmdString(), blk_addr, new_state); - handleSnoop(blk, new_state); + handleSnoop(pkt, blk, true); } + template -void -Cache::snoopResponse(PacketPtr &pkt) +Tick +Cache::snoopAtomic(PacketPtr pkt) { - //Need to handle the response, if NACKED - if (pkt->flags & NACKED_LINE) { - //Need to mark it as not in service, and retry for bus - assert(0); //Yeah, we saw a NACK come through - - //For now this should never get called, we return false when we see a - //NACK instead, by doing this we allow the bus_blocked mechanism to - //handle the retry For now it retrys in just 2 cycles, need to figure - //out how to change that Eventually we will want to also have success - //come in as a parameter Need to make sure that we handle the - //functionality that happens on successufl return of the sendAddr - //function + if (pkt->req->isUncacheable()) { + // Can't get a hit on an uncacheable address + // Revisit this for multi level coherence + return hitLatency; } + + BlkType *blk = tags->findBlock(pkt->getAddr()); + handleSnoop(pkt, blk, false); + return hitLatency; } -/** - * @todo Fix to not assume write allocate - */ template -Tick -Cache::probe(PacketPtr &pkt, bool update, - CachePort* otherSidePort) +MSHR * +Cache::getNextMSHR() { -// MemDebug::cacheProbe(pkt); - if (!pkt->req->isUncacheable()) { - if (pkt->isInvalidate() && !pkt->isRead() && !pkt->isWrite()) { - //Upgrade or Invalidate, satisfy it, don't forward - DPRINTF(Cache, "%s %x ?\n", pkt->cmdString(), pkt->getAddr()); - pkt->flags |= SATISFIED; - return 0; - } - } + // Check both MSHR queue and write buffer for potential requests + MSHR *miss_mshr = mshrQueue.getNextMSHR(); + MSHR *write_mshr = writeBuffer.getNextMSHR(); - if (!update && (otherSidePort == cpuSidePort)) { - // Still need to change data in all locations. - otherSidePort->checkAndSendFunctional(pkt); - if (pkt->isRead() && pkt->result == Packet::Success) - return 0; + // Now figure out which one to send... some cases are easy + if (miss_mshr && !write_mshr) { + return miss_mshr; + } + if (write_mshr && !miss_mshr) { + return write_mshr; } - PacketList writebacks; - int lat; - - BlkType *blk = handleAccess(pkt, lat, writebacks, update); - - DPRINTF(Cache, "%s %x %s\n", pkt->cmdString(), - pkt->getAddr(), (blk) ? "hit" : "miss"); - - - // Need to check for outstanding misses and writes - Addr blk_addr = pkt->getAddr() & ~(blkSize - 1); - - // There can only be one matching outstanding miss. - MSHR* mshr = missQueue->findMSHR(blk_addr); - - // There can be many matching outstanding writes. - std::vector writes; - missQueue->findWrites(blk_addr, writes); - - if (!update) { - bool notDone = !(pkt->flags & SATISFIED); //Hit in cache (was a block) - // Check for data in MSHR and writebuffer. - if (mshr) { - MSHR::TargetList *targets = mshr->getTargetList(); - MSHR::TargetList::iterator i = targets->begin(); - MSHR::TargetList::iterator end = targets->end(); - for (; i != end && notDone; ++i) { - PacketPtr target = *i; - // If the target contains data, and it overlaps the - // probed request, need to update data - if (target->intersect(pkt)) { - DPRINTF(Cache, "Functional %s access to blk_addr %x intersects a MSHR\n", - pkt->cmdString(), blk_addr); - notDone = fixPacket(pkt, target); - } - } - } - for (int i = 0; i < writes.size() && notDone; ++i) { - PacketPtr write = writes[i]->pkt; - if (write->intersect(pkt)) { - DPRINTF(Cache, "Functional %s access to blk_addr %x intersects a writeback\n", - pkt->cmdString(), blk_addr); - notDone = fixPacket(pkt, write); - } - } - if (notDone && otherSidePort == memSidePort) { - otherSidePort->checkAndSendFunctional(pkt); - assert(pkt->result == Packet::Success); - } - return 0; - } else if (!blk && !(pkt->flags & SATISFIED)) { - // update the cache state and statistics - if (mshr || !writes.empty()){ - // Can't handle it, return request unsatisfied. - panic("Atomic access ran into outstanding MSHR's or WB's!"); - } - if (!pkt->req->isUncacheable() /*Uncacheables just go through*/ - && (pkt->cmd != MemCmd::Writeback)/*Writebacks on miss fall through*/) { - // Fetch the cache block to fill - BlkType *blk = tags->findBlock(pkt->getAddr()); - MemCmd temp_cmd = - coherence->getBusCmd(pkt->cmd, (blk) ? blk->status : 0); - - PacketPtr busPkt = new Packet(pkt->req,temp_cmd, -1, blkSize); - - busPkt->allocate(); - - busPkt->time = curTick; - - DPRINTF(Cache, "Sending a atomic %s for %x\n", - busPkt->cmdString(), busPkt->getAddr()); - - lat = memSidePort->sendAtomic(busPkt); - - //Be sure to flip the response to a request for coherence - if (busPkt->needsResponse()) { - busPkt->makeAtomicResponse(); + if (miss_mshr && write_mshr) { + // We have one of each... normally we favor the miss request + // unless the write buffer is full + if (writeBuffer.isFull() && writeBuffer.inServiceEntries == 0) { + // Write buffer is full, so we'd like to issue a write; + // need to search MSHR queue for conflicting earlier miss. + MSHR *conflict_mshr = + mshrQueue.findPending(write_mshr->addr, write_mshr->size); + + if (conflict_mshr && conflict_mshr->order < write_mshr->order) { + // Service misses in order until conflict is cleared. + return conflict_mshr; } -/* if (!(busPkt->flags & SATISFIED)) { -// blocked at a higher level, just return -return 0; -} - -*/ misses[pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/]++; - - CacheBlk::State old_state = (blk) ? blk->status : 0; - CacheBlk::State new_state = - coherence->getNewState(busPkt, old_state); - DPRINTF(Cache, "Receive response: %s for addr %x in state %i\n", - busPkt->cmdString(), busPkt->getAddr(), old_state); - if (old_state != new_state) - DPRINTF(Cache, "Block for blk addr %x moving from state " - "%i to %i\n", busPkt->getAddr(), old_state, new_state); - - handleFill(blk, busPkt, new_state, writebacks, pkt); - //Free the packet - delete busPkt; - - // Handle writebacks if needed - while (!writebacks.empty()){ - PacketPtr wbPkt = writebacks.front(); - memSidePort->sendAtomic(wbPkt); - writebacks.pop_front(); - delete wbPkt; - } - return lat + hitLatency; - } else { - return memSidePort->sendAtomic(pkt); + // No conflicts; issue write + return write_mshr; } - } else { - if (blk) { - // There was a cache hit. - // Handle writebacks if needed - while (!writebacks.empty()){ - PacketPtr wbPkt = writebacks.front(); - memSidePort->sendAtomic(wbPkt); - writebacks.pop_front(); - delete wbPkt; - } - hits[pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/]++; + // Write buffer isn't full, but need to check it for + // conflicting earlier writeback + MSHR *conflict_mshr = + writeBuffer.findPending(miss_mshr->addr, miss_mshr->size); + if (conflict_mshr) { + // not sure why we don't check order here... it was in the + // original code but commented out. + + // The only way this happens is if we are + // doing a write and we didn't have permissions + // then subsequently saw a writeback (owned got evicted) + // We need to make sure to perform the writeback first + // To preserve the dirty data, then we can issue the write + + // should we return write_mshr here instead? I.e. do we + // have to flush writes in order? I don't think so... not + // for Alpha anyway. Maybe for x86? + return conflict_mshr; } - return hitLatency; + // No conclifts; issue read + return miss_mshr; + } + + // fall through... no pending requests. Try a prefetch. + assert(!miss_mshr && !write_mshr); + if (!mshrQueue.isFull()) { + // If we have a miss queue slot, we can try a prefetch + PacketPtr pkt = prefetcher->getPacket(); + if (pkt) { + // Update statistic on number of prefetches issued + // (hwpf_mshr_misses) + mshr_misses[pkt->cmdToIndex()][0/*pkt->req->getThreadNum()*/]++; + // Don't request bus, since we already have it + return allocateBuffer(pkt, curTick, true, false); + } } - return 0; + return NULL; } + template -Tick -Cache::snoopProbe(PacketPtr &pkt) +PacketPtr +Cache::getPacket() { - ///// PROPAGATE SNOOP UPWARD HERE + MSHR *mshr = getNextMSHR(); - Addr blk_addr = pkt->getAddr() & ~(Addr(blkSize-1)); - BlkType *blk = tags->findBlock(pkt->getAddr()); - MSHR *mshr = missQueue->findMSHR(blk_addr); - CacheBlk::State new_state = 0; - bool satisfy = coherence->handleBusRequest(pkt,blk,mshr, new_state); - if (satisfy) { - DPRINTF(Cache, "Cache snooped a %s request for addr %x and " - "now supplying data, new state is %i\n", - pkt->cmdString(), blk_addr, new_state); - - handleSnoop(blk, new_state, pkt); - return hitLatency; + if (mshr == NULL) { + return NULL; } - if (blk) - DPRINTF(Cache, "Cache snooped a %s request for addr %x, " - "new state is %i\n", - pkt->cmdString(), blk_addr, new_state); - handleSnoop(blk, new_state); - return 0; -} -template -Port * -Cache::getPort(const std::string &if_name, int idx) -{ - if (if_name == "" || if_name == "cpu_side") { - return cpuSidePort; - } else if (if_name == "mem_side") { - return memSidePort; - } else if (if_name == "functional") { - return new CpuSidePort(name() + "-cpu_side_funcport", this); + BlkType *blk = tags->findBlock(mshr->addr); + + // use request from 1st target + MSHR::Target *tgt1 = mshr->getTarget(); + PacketPtr tgt1_pkt = tgt1->pkt; + PacketPtr pkt; + + if (mshr->isCacheFill) { + MemCmd cmd; + if (blk && blk->isValid()) { + // only reason to be here is that blk is shared + // (read-only) and we need exclusive + assert(mshr->needsExclusive && !blk->isWritable()); + cmd = MemCmd::UpgradeReq; + } else { + // block is invalid + cmd = mshr->needsExclusive ? MemCmd::ReadExReq : MemCmd::ReadReq; + } + pkt = new Packet(tgt1_pkt->req, cmd, Packet::Broadcast); } else { - panic("Port name %s unrecognized\n", if_name); + assert(blk == NULL); + assert(mshr->getNumTargets() == 1); + pkt = new Packet(tgt1_pkt->req, tgt1_pkt->cmd, Packet::Broadcast); } -} - -template -void -Cache::deletePortRefs(Port *p) -{ - if (cpuSidePort == p || memSidePort == p) - panic("Can only delete functional ports\n"); - delete p; + pkt->senderState = mshr; + pkt->allocate(); + return pkt; } +/////////////// +// +// CpuSidePort +// +/////////////// + template void Cache::CpuSidePort:: @@ -1155,131 +1123,57 @@ template bool Cache::CpuSidePort::recvTiming(PacketPtr pkt) { - assert(pkt->result != Packet::Nacked); - - if (!pkt->req->isUncacheable() - && pkt->isInvalidate() - && !pkt->isRead() && !pkt->isWrite()) { - //Upgrade or Invalidate - //Look into what happens if two slave caches on bus - DPRINTF(Cache, "%s %x ?\n", pkt->cmdString(), pkt->getAddr()); - - assert(!(pkt->flags & SATISFIED)); - pkt->flags |= SATISFIED; - //Invalidates/Upgrades need no response if they get the bus - return true; - } - - if (pkt->isRequest() && blocked) - { + if (pkt->isRequest() && blocked) { DPRINTF(Cache,"Scheduling a retry while blocked\n"); mustSendRetry = true; return false; } - if (pkt->isWrite() && (pkt->req->isLocked())) { - pkt->req->setExtraData(1); - } - myCache()->access(pkt); + myCache()->timingAccess(pkt); return true; } template -void -Cache::CpuSidePort::recvRetry() -{ - recvRetryCommon(); -} - - -template -void -Cache::CpuSidePort::processRequestEvent() +Tick +Cache::CpuSidePort::recvAtomic(PacketPtr pkt) { - if (waitingOnRetry) - return; - //We have some responses to drain first - if (!drainList.empty()) { - if (!drainResponse()) { - // more responses to drain... re-request bus - scheduleRequestEvent(curTick + 1); - } - } + return myCache()->atomicAccess(pkt); } template void -Cache::CpuSidePort::processResponseEvent() +Cache::CpuSidePort::recvFunctional(PacketPtr pkt) { - assert(transmitList.size()); - assert(transmitList.front().first <= curTick); - PacketPtr pkt = transmitList.front().second; - transmitList.pop_front(); - if (!transmitList.empty()) { - Tick time = transmitList.front().first; - responseEvent->schedule(time <= curTick ? curTick+1 : time); - } - - if (pkt->flags & NACKED_LINE) - pkt->result = Packet::Nacked; - else - pkt->result = Packet::Success; - pkt->makeTimingResponse(); - DPRINTF(CachePort, "%s attempting to send a response\n", name()); - if (!drainList.empty() || waitingOnRetry) { - //Already have a list, just append - drainList.push_back(pkt); - DPRINTF(CachePort, "%s appending response onto drain list\n", name()); - } - else if (!sendTiming(pkt)) { - //It failed, save it to list of drain events - DPRINTF(CachePort, "%s now waiting for a retry\n", name()); - drainList.push_back(pkt); - waitingOnRetry = true; - } - - // Check if we're done draining once this list is empty - if (drainList.empty() && transmitList.empty()) - myCache()->checkDrain(); + checkFunctional(pkt); + if (pkt->result != Packet::Success) + myCache()->functionalAccess(pkt, cache->memSidePort); } template -Tick -Cache::CpuSidePort::recvAtomic(PacketPtr pkt) -{ - myCache()->probe(pkt, true, NULL); - //TEMP ALWAYS SUCCES FOR NOW - pkt->result = Packet::Success; - //Fix this timing info - return myCache()->hitLatency; -} - -template -void -Cache::CpuSidePort::recvFunctional(PacketPtr pkt) +Cache:: +CpuSidePort::CpuSidePort(const std::string &_name, + Cache *_cache) + : BaseCache::CachePort(_name, _cache) { - if (checkFunctional(pkt)) { - //TEMP USE CPU?THREAD 0 0 - pkt->req->setThreadContext(0,0); - - myCache()->probe(pkt, false, cache->memSidePort); - //TEMP ALWAYS SUCCESFUL FOR NOW - pkt->result = Packet::Success; - } } +/////////////// +// +// MemSidePort +// +/////////////// template void Cache::MemSidePort:: getDeviceAddressRanges(AddrRangeList &resp, bool &snoop) { - // Memory-side port always snoops. - bool dummy; - otherPort->getPeerAddressRanges(resp, dummy); + otherPort->getPeerAddressRanges(resp, snoop); + // Memory-side port always snoops, so unconditionally set flag for + // caller. snoop = true; } @@ -1303,177 +1197,115 @@ Cache::MemSidePort::recvTiming(PacketPtr pkt) if (pkt->isResponse()) { myCache()->handleResponse(pkt); } else { - myCache()->snoop(pkt); + myCache()->snoopTiming(pkt); } return true; } + template -void -Cache::MemSidePort::recvRetry() +Tick +Cache::MemSidePort::recvAtomic(PacketPtr pkt) { - if (recvRetryCommon()) { - return; - } - - DPRINTF(CachePort, "%s attempting to send a retry for MSHR\n", name()); - if (!cache->isMemSideBusRequested()) { - //This can happen if I am the owner of a block and see an upgrade - //while the block was in my WB Buffers. I just remove the - //wb and de-assert the masterRequest - waitingOnRetry = false; - return; - } - PacketPtr pkt = myCache()->getPacket(); - MSHR* mshr = (MSHR*) pkt->senderState; - //Copy the packet, it may be modified/destroyed elsewhere - PacketPtr copyPkt = new Packet(*pkt); - copyPkt->dataStatic(pkt->getPtr()); - mshr->pkt = copyPkt; - - bool success = sendTiming(pkt); - DPRINTF(Cache, "Address %x was %s in sending the timing request\n", - pkt->getAddr(), success ? "succesful" : "unsuccesful"); - - waitingOnRetry = !success; - if (waitingOnRetry) { - DPRINTF(CachePort, "%s now waiting on a retry\n", name()); - } + // in atomic mode, responses go back to the sender via the + // function return from sendAtomic(), not via a separate + // sendAtomic() from the responder. Thus we should never see a + // response packet in recvAtomic() (anywhere, not just here). + assert(!pkt->isResponse()); + return myCache()->snoopAtomic(pkt); +} - myCache()->sendResult(pkt, mshr, success); - if (success && cache->isMemSideBusRequested()) - { - DPRINTF(CachePort, "%s has more requests\n", name()); - //Still more to issue, rerequest in 1 cycle - new RequestEvent(this, curTick + 1); - } +template +void +Cache::MemSidePort::recvFunctional(PacketPtr pkt) +{ + checkFunctional(pkt); + if (pkt->result != Packet::Success) + myCache()->functionalAccess(pkt, cache->cpuSidePort); } + template void -Cache::MemSidePort::processRequestEvent() +Cache::MemSidePort::sendPacket() { - if (waitingOnRetry) - return; - //We have some responses to drain first - if (!drainList.empty()) { - if (!drainResponse()) { - // more responses to drain... re-request bus - scheduleRequestEvent(curTick + 1); - } - return; - } + // if we have responses that are ready, they take precedence + if (deferredPacketReady()) { + bool success = sendTiming(transmitList.front().pkt); - DPRINTF(CachePort, "%s trying to send a MSHR request\n", name()); - if (!isBusRequested()) { - //This can happen if I am the owner of a block and see an upgrade - //while the block was in my WB Buffers. I just remove the - //wb and de-assert the masterRequest - return; - } + if (success) { + //send successful, remove packet + transmitList.pop_front(); + } - PacketPtr pkt = myCache()->getPacket(); - MSHR* mshr = (MSHR*) pkt->senderState; - //Copy the packet, it may be modified/destroyed elsewhere - PacketPtr copyPkt = new Packet(*pkt); - copyPkt->dataStatic(pkt->getPtr()); - mshr->pkt = copyPkt; + waitingOnRetry = !success; + } else { + // check for non-response packets (requests & writebacks) + PacketPtr pkt = myCache()->getPacket(); + MSHR *mshr = dynamic_cast(pkt->senderState); - bool success = sendTiming(pkt); - DPRINTF(Cache, "Address %x was %s in sending the timing request\n", - pkt->getAddr(), success ? "succesful" : "unsuccesful"); + bool success = sendTiming(pkt); + DPRINTF(Cache, "Address %x was %s in sending the timing request\n", + pkt->getAddr(), success ? "successful" : "unsuccessful"); - waitingOnRetry = !success; - if (waitingOnRetry) { - DPRINTF(CachePort, "%s now waiting on a retry\n", name()); + waitingOnRetry = !success; + if (waitingOnRetry) { + DPRINTF(CachePort, "%s now waiting on a retry\n", name()); + } else { + myCache()->markInService(mshr); + } } - myCache()->sendResult(pkt, mshr, success); - if (success && isBusRequested()) - { - DPRINTF(CachePort, "%s still more MSHR requests to send\n", name()); - //Still more to issue, rerequest in 1 cycle - scheduleRequestEvent(curTick+1); + + // tried to send packet... if it was successful (no retry), see if + // we need to rerequest bus or not + if (!waitingOnRetry) { + if (isBusRequested()) { + // more requests/writebacks: rerequest ASAP + DPRINTF(CachePort, "%s still more MSHR requests to send\n", + name()); + sendEvent->schedule(curTick+1); + } else if (!transmitList.empty()) { + // deferred packets: rerequest bus, but possibly not until later + Tick time = transmitList.front().tick; + sendEvent->schedule(time <= curTick ? curTick+1 : time); + } else { + // no more to send right now: if we're draining, we may be done + if (drainEvent) { + drainEvent->process(); + drainEvent = NULL; + } + } } } - template void -Cache::MemSidePort::processResponseEvent() +Cache::MemSidePort::recvRetry() { - assert(transmitList.size()); - assert(transmitList.front().first <= curTick); - PacketPtr pkt = transmitList.front().second; - transmitList.pop_front(); - if (!transmitList.empty()) { - Tick time = transmitList.front().first; - responseEvent->schedule(time <= curTick ? curTick+1 : time); - } - - if (pkt->flags & NACKED_LINE) - pkt->result = Packet::Nacked; - else - pkt->result = Packet::Success; - pkt->makeTimingResponse(); - DPRINTF(CachePort, "%s attempting to send a response\n", name()); - if (!drainList.empty() || waitingOnRetry) { - //Already have a list, just append - drainList.push_back(pkt); - DPRINTF(CachePort, "%s appending response onto drain list\n", name()); - } - else if (!sendTiming(pkt)) { - //It failed, save it to list of drain events - DPRINTF(CachePort, "%s now waiting for a retry\n", name()); - drainList.push_back(pkt); - waitingOnRetry = true; - } - - // Check if we're done draining once this list is empty - if (drainList.empty() && transmitList.empty()) - myCache()->checkDrain(); + assert(waitingOnRetry); + sendPacket(); } -template -Tick -Cache::MemSidePort::recvAtomic(PacketPtr pkt) -{ - if (pkt->isResponse()) - myCache()->handleResponse(pkt); - else - return myCache()->snoopProbe(pkt); - //Fix this timing info - return myCache()->hitLatency; -} - template void -Cache::MemSidePort::recvFunctional(PacketPtr pkt) +Cache::MemSidePort::processSendEvent() { - myCache()->probe(pkt, false, cache->cpuSidePort); - if (pkt->result != Packet::Success) - checkFunctional(pkt); + assert(!waitingOnRetry); + sendPacket(); } -template -Cache:: -CpuSidePort::CpuSidePort(const std::string &_name, - Cache *_cache) - : BaseCache::CachePort(_name, _cache) -{ - responseEvent = new ResponseEvent(this); -} - template Cache:: MemSidePort::MemSidePort(const std::string &_name, Cache *_cache) : BaseCache::CachePort(_name, _cache) { - responseEvent = new ResponseEvent(this); + // override default send event from SimpleTimingPort + delete sendEvent; + sendEvent = new SendEvent(this); } - -- cgit v1.2.3