summaryrefslogtreecommitdiff
path: root/src/mem/cache/cache_impl.hh
diff options
context:
space:
mode:
authorRon Dreslinski <rdreslin@umich.edu>2006-06-28 11:02:14 -0400
committerRon Dreslinski <rdreslin@umich.edu>2006-06-28 11:02:14 -0400
commited8564a6b9f0702a40995d95cc4da54de3d35462 (patch)
tree156901f9e5a2e92ddfa44eea664103de5d210aa7 /src/mem/cache/cache_impl.hh
parentecab4b426c949dad797df0bde1c0c120b4b5fb00 (diff)
downloadgem5-ed8564a6b9f0702a40995d95cc4da54de3d35462.tar.xz
Was having difficulty with merging the cache, reverted to an early version and will add back in the patches to make it work soon.
src/mem/cache/prefetch/tagged_prefetcher_impl.hh: Trying to merge 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/coherence/uni_coherence.cc: src/mem/cache/coherence/uni_coherence.hh: src/mem/cache/miss/blocking_buffer.cc: src/mem/cache/miss/blocking_buffer.hh: src/mem/cache/miss/miss_queue.cc: src/mem/cache/miss/miss_queue.hh: 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/prefetch/base_prefetcher.hh: src/mem/cache/prefetch/ghb_prefetcher.cc: src/mem/cache/prefetch/ghb_prefetcher.hh: src/mem/cache/prefetch/stride_prefetcher.cc: src/mem/cache/prefetch/stride_prefetcher.hh: src/mem/cache/prefetch/tagged_prefetcher.hh: src/mem/cache/tags/base_tags.cc: src/mem/cache/tags/base_tags.hh: 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/repl/gen.cc: src/mem/cache/tags/repl/gen.hh: src/mem/cache/tags/repl/repl.cc: src/mem/cache/tags/repl/repl.hh: src/mem/cache/tags/split.cc: src/mem/cache/tags/split.hh: src/mem/cache/tags/split_blk.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: Pulling an early version of the cache into the tree due to merging issues. Will apply patches and push. --HG-- extra : convert_revision : 3276e5fb9a6272681a1690babf2b586dd0e1f380
Diffstat (limited to 'src/mem/cache/cache_impl.hh')
-rw-r--r--src/mem/cache/cache_impl.hh661
1 files changed, 661 insertions, 0 deletions
diff --git a/src/mem/cache/cache_impl.hh b/src/mem/cache/cache_impl.hh
new file mode 100644
index 000000000..3dd8d74cd
--- /dev/null
+++ b/src/mem/cache/cache_impl.hh
@@ -0,0 +1,661 @@
+/*
+ * Copyright (c) 2002-2005 The Regents of The University of Michigan
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met: redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer;
+ * redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution;
+ * neither the name of the copyright holders nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ * Authors: Erik Hallnor
+ * Dave Greene
+ * Nathan Binkert
+ */
+
+/**
+ * @file
+ * Cache definitions.
+ */
+
+#include <assert.h>
+#include <math.h>
+
+#include <cassert>
+#include <iostream>
+#include <string>
+
+#include "sim/host.hh"
+#include "base/misc.hh"
+#include "cpu/smt.hh"
+
+#include "mem/cache/cache.hh"
+#include "mem/cache/cache_blk.hh"
+#include "mem/cache/miss/mshr.hh"
+#include "mem/cache/prefetch/prefetcher.hh"
+
+#include "mem/bus/bus.hh"
+
+#include "mem/bus/slave_interface.hh"
+#include "mem/memory_interface.hh"
+#include "mem/bus/master_interface.hh"
+
+#include "mem/mem_debug.hh"
+
+#include "sim/sim_events.hh" // for SimExitEvent
+
+using namespace std;
+
+template<class TagStore, class Buffering, class Coherence>
+bool
+Cache<TagStore,Buffering,Coherence>::
+doTimingAccess(Packet *pkt, MemoryPort *memoryPort, bool isCpuSide)
+{
+ if (isCpuSide)
+ {
+ access(pkt);
+ }
+ else
+ {
+ if (pkt->isRespnse())
+ handleResponse(pkt);
+ else
+ snoop(pkt);
+ }
+}
+
+template<class TagStore, class Buffering, class Coherence>
+Tick
+Cache<TagStore,Buffering,Coherence>::
+doAtomicAccess(Packet *pkt, MemoryPort *memoryPort, bool isCpuSide)
+{
+ if (isCpuSide)
+ {
+ probe(pkt, true);
+ }
+ else
+ {
+ if (pkt->isRespnse())
+ handleResponse(pkt);
+ else
+ snoopProbe(pkt, true);
+ }
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::
+doFunctionalAccess(Packet *pkt, MemoryPort *memoryPort, bool isCpuSide)
+{
+ if (isCpuSide)
+ {
+ probe(pkt, false);
+ }
+ else
+ {
+ if (pkt->isRespnse())
+ handleResponse(pkt);
+ else
+ snoopProbe(pkt, false);
+ }
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::
+recvStatusChange(Port::Status status, bool isCpuSide)
+{
+
+}
+
+
+template<class TagStore, class Buffering, class Coherence>
+Cache<TagStore,Buffering,Coherence>::
+Cache(const std::string &_name, HierParams *hier_params,
+ Cache<TagStore,Buffering,Coherence>::Params &params)
+ : BaseCache(_name, hier_params, params.baseParams),
+ prefetchAccess(params.prefetchAccess),
+ tags(params.tags), missQueue(params.missQueue),
+ coherence(params.coherence), prefetcher(params.prefetcher),
+ doCopy(params.doCopy), blockOnCopy(params.blockOnCopy)
+{
+ if (params.in == NULL) {
+ topLevelCache = true;
+ }
+ tags->setCache(this, params.out->width, params.out->clockRate);
+ tags->setPrefetcher(prefetcher);
+ missQueue->setCache(this);
+ missQueue->setPrefetcher(prefetcher);
+ coherence->setCache(this);
+ prefetcher->setCache(this);
+ prefetcher->setTags(tags);
+ prefetcher->setBuffer(missQueue);
+ invalidatePkt = new Packet;
+ invalidatePkt->cmd = Invalidate;
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::regStats()
+{
+ BaseCache::regStats();
+ tags->regStats(name());
+ missQueue->regStats(name());
+ coherence->regStats(name());
+ prefetcher->regStats(name());
+}
+
+template<class TagStore, class Buffering, class Coherence>
+MemAccessResult
+Cache<TagStore,Buffering,Coherence>::access(Packet &pkt)
+{
+ 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);
+ }
+ if (!pkt->isUncacheable()) {
+ if (pkt->cmd.isInvalidate() && !pkt->cmd.isRead()
+ && !pkt->cmd.isWrite()) {
+ //Upgrade or Invalidate
+ //Look into what happens if two slave caches on bus
+ DPRINTF(Cache, "%s %d %x ? blk_addr: %x\n", pkt->cmd.toString(),
+ pkt->req->asid, pkt->paddr & (((ULL(1))<<48)-1),
+ pkt->paddr & ~((Addr)blkSize - 1));
+
+ //@todo Should this return latency have the hit latency in it?
+// respond(pkt,curTick+lat);
+ pkt->flags |= SATISFIED;
+ return MA_HIT;
+ }
+ blk = tags->handleAccess(pkt, lat, writebacks);
+ } else {
+ size = pkt->size;
+ }
+ // 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->size >= blkSize && coherence->allowFastWrites() &&
+ (pkt->cmd == Write || pkt->cmd == WriteInvalidate) ) {
+ // not outstanding misses, can do this
+ MSHR* outstanding_miss = missQueue->findMSHR(pkt->paddr, pkt->req->asid);
+ if (pkt->cmd ==WriteInvalidate || !outstanding_miss) {
+ if (outstanding_miss) {
+ warn("WriteInv doing a fastallocate"
+ "with an outstanding miss to the same address\n");
+ }
+ blk = tags->handleFill(NULL, pkt, BlkValid | BlkWritable,
+ writebacks);
+ ++fastWrites;
+ }
+ }
+ while (!writebacks.empty()) {
+ missQueue->doWriteback(writebacks.front());
+ writebacks.pop_front();
+ }
+ DPRINTF(Cache, "%s %d %x %s blk_addr: %x pc %x\n", pkt->cmd.toString(),
+ pkt->req->asid, pkt->paddr & (((ULL(1))<<48)-1), (blk) ? "hit" : "miss",
+ pkt->paddr & ~((Addr)blkSize - 1), pkt->pc);
+ if (blk) {
+ // Hit
+ hits[pkt->cmd.toIndex()][pkt->thread_num]++;
+ // clear dirty bit if write through
+ if (!pkt->cmd.isNoResponse())
+ respond(pkt, curTick+lat);
+ return MA_HIT;
+ }
+
+ // Miss
+ if (!pkt->isUncacheable()) {
+ misses[pkt->cmd.toIndex()][pkt->thread_num]++;
+ /** @todo Move miss count code into BaseCache */
+ if (missCount) {
+ --missCount;
+ if (missCount == 0)
+ new SimExitEvent("A cache reached the maximum miss count");
+ }
+ }
+ missQueue->handleMiss(pkt, size, curTick + hitLatency);
+ return MA_CACHE_MISS;
+}
+
+
+template<class TagStore, class Buffering, class Coherence>
+Packet *
+Cache<TagStore,Buffering,Coherence>::getPacket()
+{
+ Packet * pkt = missQueue->getPacket();
+ if (pkt) {
+ if (!pkt->isUncacheable()) {
+ if (pkt->cmd == Hard_Prefetch) misses[Hard_Prefetch][pkt->thread_num]++;
+ BlkType *blk = tags->findBlock(pkt);
+ Packet::Command cmd = coherence->getBusCmd(pkt->cmd,
+ (blk)? blk->status : 0);
+ missQueue->setBusCmd(pkt, cmd);
+ }
+ }
+
+ assert(!doMasterPktuest() || missQueue->havePending());
+ assert(!pkt || pkt->time <= curTick);
+ return pkt;
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::sendResult(MemPktPtr &pkt, bool success)
+{
+ if (success) {
+ missQueue->markInService(pkt);
+ //Temp Hack for UPGRADES
+ if (pkt->cmd == Upgrade) {
+ handleResponse(pkt);
+ }
+ } else if (pkt && !pkt->isUncacheable()) {
+ missQueue->restoreOrigCmd(pkt);
+ }
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::handleResponse(Packet * &pkt)
+{
+ BlkType *blk = NULL;
+ if (pkt->senderState) {
+ MemDebug::cacheResponse(pkt);
+ DPRINTF(Cache, "Handling reponse to %x, blk addr: %x\n",pkt->paddr,
+ pkt->paddr & (((ULL(1))<<48)-1));
+
+ if (pkt->isCacheFill() && !pkt->isNoAllocate()) {
+ blk = tags->findBlock(pkt);
+ CacheBlk::State old_state = (blk) ? blk->status : 0;
+ MemPktList writebacks;
+ blk = tags->handleFill(blk, pkt->senderState,
+ coherence->getNewState(pkt,old_state),
+ writebacks);
+ while (!writebacks.empty()) {
+ missQueue->doWriteback(writebacks.front());
+ }
+ }
+ missQueue->handleResponse(pkt, curTick + hitLatency);
+ }
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::pseudoFill(Addr addr, int asid)
+{
+ // Need to temporarily move this blk into MSHRs
+ MSHR *mshr = missQueue->allocateTargetList(addr, asid);
+ int lat;
+ PacketList* dummy;
+ // Read the data into the mshr
+ BlkType *blk = tags->handleAccess(mshr->pkt, lat, dummy, false);
+ assert(dummy.empty());
+ assert(mshr->pkt->isSatisfied());
+ // can overload order since it isn't used on non pending blocks
+ mshr->order = blk->status;
+ // temporarily remove the block from the cache.
+ tags->invalidateBlk(addr, asid);
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::pseudoFill(MSHR *mshr)
+{
+ // Need to temporarily move this blk into MSHRs
+ assert(mshr->pkt->cmd == Read);
+ int lat;
+ PacketList* dummy;
+ // Read the data into the mshr
+ BlkType *blk = tags->handleAccess(mshr->pkt, lat, dummy, false);
+ assert(dummy.empty());
+ assert(mshr->pkt->isSatisfied());
+ // can overload order since it isn't used on non pending blocks
+ mshr->order = blk->status;
+ // temporarily remove the block from the cache.
+ tags->invalidateBlk(mshr->pkt->paddr, mshr->pkt->req->asid);
+}
+
+
+template<class TagStore, class Buffering, class Coherence>
+Packet *
+Cache<TagStore,Buffering,Coherence>::getCoherenceReq()
+{
+ return coherence->getPacket();
+}
+
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::snoop(Packet * &pkt)
+{
+ Addr blk_addr = pkt->paddr & ~(Addr(blkSize-1));
+ BlkType *blk = tags->findBlock(pkt);
+ MSHR *mshr = missQueue->findMSHR(blk_addr, pkt->req->asid);
+ if (isTopLevel() && coherence->hasProtocol()) { //@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->cmd.isInvalidate() || !mshr->pkt->isCacheFill())
+ && (pkt->cmd != Invalidate && pkt->cmd != WriteInvalidate)) {
+ //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)
+ pkt->flags |= NACKED_LINE;
+ 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
+ invalidatePkt->paddr = pkt->paddr;
+
+ //Append the invalidate on
+ missQueue->addTarget(mshr,invalidatePkt);
+ DPRINTF(Cache, "Appending Invalidate to blk_addr: %x\n", pkt->paddr & (((ULL(1))<<48)-1));
+ return;
+ }
+ }
+ }
+ //We also need to check the writeback buffers and handle those
+ std::vector<MSHR *> writebacks;
+ if (missQueue->findWrites(blk_addr, pkt->req->asid, writebacks)) {
+ DPRINTF(Cache, "Snoop hit in writeback to blk_addr: %x\n", pkt->paddr & (((ULL(1))<<48)-1));
+
+ //Look through writebacks for any non-uncachable writes, use that
+ for (int i=0; i<writebacks.size(); i++) {
+ mshr = writebacks[i];
+
+ if (!mshr->pkt->isUncacheable()) {
+ if (pkt->cmd.isRead()) {
+ //Only Upgrades don't get here
+ //Supply the data
+ 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;
+
+ if (doData()) {
+ assert(pkt->cmd.isRead());
+
+ assert(pkt->offset < blkSize);
+ assert(pkt->size <= blkSize);
+ assert(pkt->offset + pkt->size <=blkSize);
+ memcpy(pkt->data, mshr->pkt->data + pkt->offset, pkt->size);
+ }
+ respondToSnoop(pkt);
+ }
+
+ if (pkt->cmd.isInvalidate()) {
+ //This must be an upgrade or other cache will take ownership
+ missQueue->markInService(mshr->pkt);
+ }
+ return;
+ }
+ }
+ }
+ }
+ CacheBlk::State new_state;
+ bool satisfy = coherence->handleBusRequest(pkt,blk,mshr, new_state);
+ if (satisfy) {
+ tags->handleSnoop(blk, new_state, pkt);
+ respondToSnoop(pkt);
+ return;
+ }
+ tags->handleSnoop(blk, new_state);
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::snoopResponse(Packet * &pkt)
+{
+ //Need to handle the response, if NACKED
+ if (pkt->isNacked()) {
+ //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
+ }
+}
+
+template<class TagStore, class Buffering, class Coherence>
+void
+Cache<TagStore,Buffering,Coherence>::invalidateBlk(Addr addr, int asid)
+{
+ tags->invalidateBlk(addr,asid);
+}
+
+
+/**
+ * @todo Fix to not assume write allocate
+ */
+template<class TagStore, class Buffering, class Coherence>
+Tick
+Cache<TagStore,Buffering,Coherence>::probe(Packet * &pkt, bool update)
+{
+ MemDebug::cacheProbe(pkt);
+
+ if (!pkt->isUncacheable()) {
+ if (pkt->cmd.isInvalidate() && !pkt->cmd.isRead()
+ && !pkt->cmd.isWrite()) {
+ //Upgrade or Invalidate, satisfy it, don't forward
+ DPRINTF(Cache, "%s %d %x ? blk_addr: %x\n", pkt->cmd.toString(),
+ pkt->req->asid, pkt->paddr & (((ULL(1))<<48)-1),
+ pkt->paddr & ~((Addr)blkSize - 1));
+ pkt->flags |= SATISFIED;
+ return 0;
+ }
+ }
+
+ if (!update && !doData()) {
+ // Nothing to do here
+ return mi->sendProbe(pkt,update);
+ }
+
+ PacketList* writebacks;
+ int lat;
+ BlkType *blk = tags->handleAccess(pkt, lat, writebacks, update);
+
+ if (!blk) {
+ // Need to check for outstanding misses and writes
+ Addr blk_addr = pkt->paddr & ~(blkSize - 1);
+
+ // There can only be one matching outstanding miss.
+ MSHR* mshr = missQueue->findMSHR(blk_addr, pkt->req->asid);
+
+ // There can be many matching outstanding writes.
+ vector<MSHR*> writes;
+ missQueue->findWrites(blk_addr, pkt->req->asid, writes);
+
+ if (!update) {
+ mi->sendProbe(pkt, update);
+ // Check for data in MSHR and writebuffer.
+ if (mshr) {
+ warn("Found outstanding miss on an non-update probe");
+ MSHR::TargetList *targets = mshr->getTargetList();
+ MSHR::TargetList::iterator i = targets->begin();
+ MSHR::TargetList::iterator end = targets->end();
+ for (; i != end; ++i) {
+ Packet * target = *i;
+ // If the target contains data, and it overlaps the
+ // probed request, need to update data
+ if (target->cmd.isWrite() && target->overlaps(pkt)) {
+ uint8_t* pkt_data;
+ uint8_t* write_data;
+ int data_size;
+ if (target->paddr < pkt->paddr) {
+ int offset = pkt->paddr - target->paddr;
+ pkt_data = pkt->data;
+ write_data = target->data + offset;
+ data_size = target->size - offset;
+ assert(data_size > 0);
+ if (data_size > pkt->size)
+ data_size = pkt->size;
+ } else {
+ int offset = target->paddr - pkt->paddr;
+ pkt_data = pkt->data + offset;
+ write_data = target->data;
+ data_size = pkt->size - offset;
+ assert(data_size > pkt->size);
+ if (data_size > target->size)
+ data_size = target->size;
+ }
+
+ if (pkt->cmd.isWrite()) {
+ memcpy(pkt_data, write_data, data_size);
+ } else {
+ memcpy(write_data, pkt_data, data_size);
+ }
+ }
+ }
+ }
+ for (int i = 0; i < writes.size(); ++i) {
+ Packet * write = writes[i]->pkt;
+ if (write->overlaps(pkt)) {
+ warn("Found outstanding write on an non-update probe");
+ uint8_t* pkt_data;
+ uint8_t* write_data;
+ int data_size;
+ if (write->paddr < pkt->paddr) {
+ int offset = pkt->paddr - write->paddr;
+ pkt_data = pkt->data;
+ write_data = write->data + offset;
+ data_size = write->size - offset;
+ assert(data_size > 0);
+ if (data_size > pkt->size)
+ data_size = pkt->size;
+ } else {
+ int offset = write->paddr - pkt->paddr;
+ pkt_data = pkt->data + offset;
+ write_data = write->data;
+ data_size = pkt->size - offset;
+ assert(data_size > pkt->size);
+ if (data_size > write->size)
+ data_size = write->size;
+ }
+
+ if (pkt->cmd.isWrite()) {
+ memcpy(pkt_data, write_data, data_size);
+ } else {
+ memcpy(write_data, pkt_data, data_size);
+ }
+
+ }
+ }
+ return 0;
+ } else {
+ // update the cache state and statistics
+ if (mshr || !writes.empty()){
+ // Can't handle it, return pktuest unsatisfied.
+ return 0;
+ }
+ if (!pkt->isUncacheable()) {
+ // Fetch the cache block to fill
+ Packet * busPkt = new MemPkt();
+ busPkt->paddr = blk_addr;
+ busPkt->size = blkSize;
+ busPkt->data = new uint8_t[blkSize];
+
+ BlkType *blk = tags->findBlock(pkt);
+ busPkt->cmd = coherence->getBusCmd(pkt->cmd,
+ (blk)? blk->status : 0);
+
+ busPkt->req->asid = pkt->req->asid;
+ busPkt->xc = pkt->xc;
+ busPkt->thread_num = pkt->thread_num;
+ busPkt->time = curTick;
+
+ lat = mi->sendProbe(busPkt, update);
+
+ if (!busPkt->isSatisfied()) {
+ // blocked at a higher level, just return
+ return 0;
+ }
+
+ misses[pkt->cmd.toIndex()][pkt->thread_num]++;
+
+ CacheBlk::State old_state = (blk) ? blk->status : 0;
+ tags->handleFill(blk, busPkt,
+ coherence->getNewState(busPkt, old_state),
+ writebacks, pkt);
+ // Handle writebacks if needed
+ while (!writebacks.empty()){
+ mi->sendProbe(writebacks.front(), update);
+ writebacks.pop_front();
+ }
+ return lat + hitLatency;
+ } else {
+ return mi->sendProbe(pkt,update);
+ }
+ }
+ } else {
+ // There was a cache hit.
+ // Handle writebacks if needed
+ while (!writebacks.empty()){
+ mi->sendProbe(writebacks.front(), update);
+ writebacks.pop_front();
+ }
+
+ if (update) {
+ hits[pkt->cmd.toIndex()][pkt->thread_num]++;
+ } else if (pkt->cmd.isWrite()) {
+ // Still need to change data in all locations.
+ return mi->sendProbe(pkt, update);
+ }
+ return curTick + lat;
+ }
+ fatal("Probe not handled.\n");
+ return 0;
+}
+
+template<class TagStore, class Buffering, class Coherence>
+Tick
+Cache<TagStore,Buffering,Coherence>::snoopProbe(MemPktPtr &pkt, bool update)
+{
+ Addr blk_addr = pkt->paddr & ~(Addr(blkSize-1));
+ BlkType *blk = tags->findBlock(pkt);
+ MSHR *mshr = missQueue->findMSHR(blk_addr, pkt->req->asid);
+ CacheBlk::State new_state = 0;
+ bool satisfy = coherence->handleBusPktuest(pkt,blk,mshr, new_state);
+ if (satisfy) {
+ tags->handleSnoop(blk, new_state, pkt);
+ return hitLatency;
+ }
+ tags->handleSnoop(blk, new_state);
+ return 0;
+}
+