gem5  v22.1.0.0
base.cc
Go to the documentation of this file.
1 /*
2  * Copyright (c) 2012-2013, 2018-2019 ARM Limited
3  * All rights reserved.
4  *
5  * The license below extends only to copyright in the software and shall
6  * not be construed as granting a license to any other intellectual
7  * property including but not limited to intellectual property relating
8  * to a hardware implementation of the functionality of the software
9  * licensed hereunder. You may use the software subject to the license
10  * terms below provided that you ensure that this notice is replicated
11  * unmodified and in its entirety in all distributions of the software,
12  * modified or unmodified, in source code or in binary form.
13  *
14  * Copyright (c) 2003-2005 The Regents of The University of Michigan
15  * All rights reserved.
16  *
17  * Redistribution and use in source and binary forms, with or without
18  * modification, are permitted provided that the following conditions are
19  * met: redistributions of source code must retain the above copyright
20  * notice, this list of conditions and the following disclaimer;
21  * redistributions in binary form must reproduce the above copyright
22  * notice, this list of conditions and the following disclaimer in the
23  * documentation and/or other materials provided with the distribution;
24  * neither the name of the copyright holders nor the names of its
25  * contributors may be used to endorse or promote products derived from
26  * this software without specific prior written permission.
27  *
28  * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
29  * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
30  * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
31  * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
32  * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
33  * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
34  * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
35  * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
36  * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
37  * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
38  * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
39  */
40 
46 #include "mem/cache/base.hh"
47 
48 #include "base/compiler.hh"
49 #include "base/logging.hh"
50 #include "debug/Cache.hh"
51 #include "debug/CacheComp.hh"
52 #include "debug/CachePort.hh"
53 #include "debug/CacheRepl.hh"
54 #include "debug/CacheVerbose.hh"
55 #include "debug/HWPrefetch.hh"
57 #include "mem/cache/mshr.hh"
59 #include "mem/cache/queue_entry.hh"
62 #include "params/BaseCache.hh"
63 #include "params/WriteAllocator.hh"
64 #include "sim/cur_tick.hh"
65 
66 namespace gem5
67 {
68 
70  BaseCache *_cache,
71  const std::string &_label)
72  : QueuedResponsePort(_name, _cache, queue),
73  queue(*_cache, *this, true, _label),
74  blocked(false), mustSendRetry(false),
75  sendRetryEvent([this]{ processSendRetry(); }, _name)
76 {
77 }
78 
79 BaseCache::BaseCache(const BaseCacheParams &p, unsigned blk_size)
80  : ClockedObject(p),
81  cpuSidePort (p.name + ".cpu_side_port", this, "CpuSidePort"),
82  memSidePort(p.name + ".mem_side_port", this, "MemSidePort"),
83  mshrQueue("MSHRs", p.mshrs, 0, p.demand_mshr_reserve, p.name),
84  writeBuffer("write buffer", p.write_buffers, p.mshrs, p.name),
85  tags(p.tags),
88  writeAllocator(p.write_allocator),
89  writebackClean(p.writeback_clean),
90  tempBlockWriteback(nullptr),
92  name(), false,
94  blkSize(blk_size),
95  lookupLatency(p.tag_latency),
96  dataLatency(p.data_latency),
97  forwardLatency(p.tag_latency),
98  fillLatency(p.data_latency),
99  responseLatency(p.response_latency),
100  sequentialAccess(p.sequential_access),
101  numTarget(p.tgts_per_mshr),
102  forwardSnoops(true),
103  clusivity(p.clusivity),
104  isReadOnly(p.is_read_only),
105  replaceExpansions(p.replace_expansions),
106  moveContractions(p.move_contractions),
107  blocked(0),
108  order(0),
109  noTargetMSHR(nullptr),
110  missCount(p.max_miss_count),
111  addrRanges(p.addr_ranges.begin(), p.addr_ranges.end()),
112  system(p.system),
113  stats(*this)
114 {
115  // the MSHR queue has no reserve entries as we check the MSHR
116  // queue on every single allocation, whereas the write queue has
117  // as many reserve entries as we have MSHRs, since every MSHR may
118  // eventually require a writeback, and we do not check the write
119  // buffer before committing to an MSHR
120 
121  // forward snoops is overridden in init() once we can query
122  // whether the connected requestor is actually snooping or not
123 
124  tempBlock = new TempCacheBlk(blkSize);
125 
126  tags->tagsInit();
127  if (prefetcher)
128  prefetcher->setCache(this);
129 
130  fatal_if(compressor && !dynamic_cast<CompressedTags*>(tags),
131  "The tags of compressed cache %s must derive from CompressedTags",
132  name());
133  warn_if(!compressor && dynamic_cast<CompressedTags*>(tags),
134  "Compressed cache %s does not have a compression algorithm", name());
135  if (compressor)
136  compressor->setCache(this);
137 }
138 
140 {
141  delete tempBlock;
142 }
143 
144 void
146 {
147  assert(!blocked);
148  DPRINTF(CachePort, "Port is blocking new requests\n");
149  blocked = true;
150  // if we already scheduled a retry in this cycle, but it has not yet
151  // happened, cancel it
152  if (sendRetryEvent.scheduled()) {
154  DPRINTF(CachePort, "Port descheduled retry\n");
155  mustSendRetry = true;
156  }
157 }
158 
159 void
161 {
162  assert(blocked);
163  DPRINTF(CachePort, "Port is accepting new requests\n");
164  blocked = false;
165  if (mustSendRetry) {
166  // @TODO: need to find a better time (next cycle?)
167  owner.schedule(sendRetryEvent, curTick() + 1);
168  }
169 }
170 
171 void
173 {
174  DPRINTF(CachePort, "Port is sending retry\n");
175 
176  // reset the flag and call retry
177  mustSendRetry = false;
178  sendRetryReq();
179 }
180 
181 Addr
183 {
184  if (blk != tempBlock) {
185  return tags->regenerateBlkAddr(blk);
186  } else {
187  return tempBlock->getAddr();
188  }
189 }
190 
191 void
193 {
195  fatal("Cache ports on %s are not connected\n", name());
198 }
199 
200 Port &
201 BaseCache::getPort(const std::string &if_name, PortID idx)
202 {
203  if (if_name == "mem_side") {
204  return memSidePort;
205  } else if (if_name == "cpu_side") {
206  return cpuSidePort;
207  } else {
208  return ClockedObject::getPort(if_name, idx);
209  }
210 }
211 
212 bool
214 {
215  for (const auto& r : addrRanges) {
216  if (r.contains(addr)) {
217  return true;
218  }
219  }
220  return false;
221 }
222 
223 void
225 {
226 
227  // handle special cases for LockedRMW transactions
228  if (pkt->isLockedRMW()) {
229  Addr blk_addr = pkt->getBlockAddr(blkSize);
230 
231  if (pkt->isRead()) {
232  // Read hit for LockedRMW. Since it requires exclusive
233  // permissions, there should be no outstanding access.
234  assert(!mshrQueue.findMatch(blk_addr, pkt->isSecure()));
235  // The keys to LockedRMW are that (1) we always have an MSHR
236  // allocated during the RMW interval to catch snoops and
237  // defer them until after the RMW completes, and (2) we
238  // clear permissions on the block to turn any upstream
239  // access other than the matching write into a miss, causing
240  // it to append to the MSHR as well.
241 
242  // Because we hit in the cache, we have to fake an MSHR to
243  // achieve part (1). If the read had missed, this MSHR
244  // would get allocated as part of normal miss processing.
245  // Basically we need to get the MSHR in the same state as if
246  // we had missed and just received the response.
247  // Request *req2 = new Request(*(pkt->req));
248  RequestPtr req2 = std::make_shared<Request>(*(pkt->req));
249  PacketPtr pkt2 = new Packet(req2, pkt->cmd);
250  MSHR *mshr = allocateMissBuffer(pkt2, curTick(), true);
251  // Mark the MSHR "in service" (even though it's not) to prevent
252  // the cache from sending out a request.
253  mshrQueue.markInService(mshr, false);
254  // Part (2): mark block inaccessible
255  assert(blk);
258  } else {
259  assert(pkt->isWrite());
260  // All LockedRMW writes come here, as they cannot miss.
261  // Need to undo the two things described above. Block
262  // permissions were already restored earlier in this
263  // function, prior to the access() call. Now we just need
264  // to clear out the MSHR.
265 
266  // Read should have already allocated MSHR.
267  MSHR *mshr = mshrQueue.findMatch(blk_addr, pkt->isSecure());
268  assert(mshr);
269  // Fake up a packet and "respond" to the still-pending
270  // LockedRMWRead, to process any pending targets and clear
271  // out the MSHR
272  PacketPtr resp_pkt =
274  resp_pkt->senderState = mshr;
275  recvTimingResp(resp_pkt);
276  }
277  }
278 
279  if (pkt->needsResponse()) {
280  // These delays should have been consumed by now
281  assert(pkt->headerDelay == 0);
282  assert(pkt->payloadDelay == 0);
283 
284  pkt->makeTimingResponse();
285 
286  // In this case we are considering request_time that takes
287  // into account the delay of the xbar, if any, and just
288  // lat, neglecting responseLatency, modelling hit latency
289  // just as the value of lat overriden by access(), which calls
290  // the calculateAccessLatency() function.
291  cpuSidePort.schedTimingResp(pkt, request_time);
292  } else {
293  DPRINTF(Cache, "%s satisfied %s, no response needed\n", __func__,
294  pkt->print());
295 
296  // queue the packet for deletion, as the sending cache is
297  // still relying on it; if the block is found in access(),
298  // CleanEvict and Writeback messages will be deleted
299  // here as well
300  pendingDelete.reset(pkt);
301  }
302 }
303 
304 void
306  Tick forward_time, Tick request_time)
307 {
308  if (writeAllocator &&
309  pkt && pkt->isWrite() && !pkt->req->isUncacheable()) {
310  writeAllocator->updateMode(pkt->getAddr(), pkt->getSize(),
311  pkt->getBlockAddr(blkSize));
312  }
313 
314  if (mshr) {
318 
319  //@todo remove hw_pf here
320 
321  // Coalesce unless it was a software prefetch (see above).
322  if (pkt) {
323  assert(!pkt->isWriteback());
324  // CleanEvicts corresponding to blocks which have
325  // outstanding requests in MSHRs are simply sunk here
326  if (pkt->cmd == MemCmd::CleanEvict) {
327  pendingDelete.reset(pkt);
328  } else if (pkt->cmd == MemCmd::WriteClean) {
329  // A WriteClean should never coalesce with any
330  // outstanding cache maintenance requests.
331 
332  // We use forward_time here because there is an
333  // uncached memory write, forwarded to WriteBuffer.
334  allocateWriteBuffer(pkt, forward_time);
335  } else {
336  DPRINTF(Cache, "%s coalescing MSHR for %s\n", __func__,
337  pkt->print());
338 
339  assert(pkt->req->requestorId() < system->maxRequestors());
340  stats.cmdStats(pkt).mshrHits[pkt->req->requestorId()]++;
341 
342  // We use forward_time here because it is the same
343  // considering new targets. We have multiple
344  // requests for the same address here. It
345  // specifies the latency to allocate an internal
346  // buffer and to schedule an event to the queued
347  // port and also takes into account the additional
348  // delay of the xbar.
349  mshr->allocateTarget(pkt, forward_time, order++,
350  allocOnFill(pkt->cmd));
351  if (mshr->getNumTargets() >= numTarget) {
352  noTargetMSHR = mshr;
354  // need to be careful with this... if this mshr isn't
355  // ready yet (i.e. time > curTick()), we don't want to
356  // move it ahead of mshrs that are ready
357  // mshrQueue.moveToFront(mshr);
358  }
359  }
360  }
361  } else {
362  // no MSHR
363  assert(pkt->req->requestorId() < system->maxRequestors());
364  stats.cmdStats(pkt).mshrMisses[pkt->req->requestorId()]++;
365  if (prefetcher && pkt->isDemand())
367 
368  if (pkt->isEviction() || pkt->cmd == MemCmd::WriteClean) {
369  // We use forward_time here because there is an
370  // writeback or writeclean, forwarded to WriteBuffer.
371  allocateWriteBuffer(pkt, forward_time);
372  } else {
373  if (blk && blk->isValid()) {
374  // If we have a write miss to a valid block, we
375  // need to mark the block non-readable. Otherwise
376  // if we allow reads while there's an outstanding
377  // write miss, the read could return stale data
378  // out of the cache block... a more aggressive
379  // system could detect the overlap (if any) and
380  // forward data out of the MSHRs, but we don't do
381  // that yet. Note that we do need to leave the
382  // block valid so that it stays in the cache, in
383  // case we get an upgrade response (and hence no
384  // new data) when the write miss completes.
385  // As long as CPUs do proper store/load forwarding
386  // internally, and have a sufficiently weak memory
387  // model, this is probably unnecessary, but at some
388  // point it must have seemed like we needed it...
389  assert((pkt->needsWritable() &&
390  !blk->isSet(CacheBlk::WritableBit)) ||
391  pkt->req->isCacheMaintenance());
393  }
394  // Here we are using forward_time, modelling the latency of
395  // a miss (outbound) just as forwardLatency, neglecting the
396  // lookupLatency component.
397  allocateMissBuffer(pkt, forward_time);
398  }
399  }
400 }
401 
402 void
404 {
405  // anything that is merely forwarded pays for the forward latency and
406  // the delay provided by the crossbar
407  Tick forward_time = clockEdge(forwardLatency) + pkt->headerDelay;
408 
409  if (pkt->cmd == MemCmd::LockedRMWWriteReq) {
410  // For LockedRMW accesses, we mark the block inaccessible after the
411  // read (see below), to make sure no one gets in before the write.
412  // Now that the write is here, mark it accessible again, so the
413  // write will succeed. LockedRMWReadReq brings the block in in
414  // exclusive mode, so we know it was previously writable.
415  CacheBlk *blk = tags->findBlock(pkt->getAddr(), pkt->isSecure());
416  assert(blk && blk->isValid());
417  assert(!blk->isSet(CacheBlk::WritableBit) &&
421  }
422 
423  Cycles lat;
424  CacheBlk *blk = nullptr;
425  bool satisfied = false;
426  {
427  PacketList writebacks;
428  // Note that lat is passed by reference here. The function
429  // access() will set the lat value.
430  satisfied = access(pkt, blk, lat, writebacks);
431 
432  // After the evicted blocks are selected, they must be forwarded
433  // to the write buffer to ensure they logically precede anything
434  // happening below
435  doWritebacks(writebacks, clockEdge(lat + forwardLatency));
436  }
437 
438  // Here we charge the headerDelay that takes into account the latencies
439  // of the bus, if the packet comes from it.
440  // The latency charged is just the value set by the access() function.
441  // In case of a hit we are neglecting response latency.
442  // In case of a miss we are neglecting forward latency.
443  Tick request_time = clockEdge(lat);
444  // Here we reset the timing of the packet.
445  pkt->headerDelay = pkt->payloadDelay = 0;
446 
447  if (satisfied) {
448  // notify before anything else as later handleTimingReqHit might turn
449  // the packet in a response
450  ppHit->notify(pkt);
451 
452  if (prefetcher && blk && blk->wasPrefetched()) {
453  DPRINTF(Cache, "Hit on prefetch for addr %#x (%s)\n",
454  pkt->getAddr(), pkt->isSecure() ? "s" : "ns");
455  blk->clearPrefetched();
456  }
457 
458  handleTimingReqHit(pkt, blk, request_time);
459  } else {
460  handleTimingReqMiss(pkt, blk, forward_time, request_time);
461 
462  ppMiss->notify(pkt);
463  }
464 
465  if (prefetcher) {
466  // track time of availability of next prefetch, if any
467  Tick next_pf_time = prefetcher->nextPrefetchReadyTime();
468  if (next_pf_time != MaxTick) {
469  schedMemSideSendEvent(next_pf_time);
470  }
471  }
472 }
473 
474 void
476 {
477  Tick completion_time = clockEdge(responseLatency) +
478  pkt->headerDelay + pkt->payloadDelay;
479 
480  // Reset the bus additional time as it is now accounted for
481  pkt->headerDelay = pkt->payloadDelay = 0;
482 
483  cpuSidePort.schedTimingResp(pkt, completion_time);
484 }
485 
486 void
488 {
489  assert(pkt->isResponse());
490 
491  // all header delay should be paid for by the crossbar, unless
492  // this is a prefetch response from above
493  panic_if(pkt->headerDelay != 0 && pkt->cmd != MemCmd::HardPFResp,
494  "%s saw a non-zero packet delay\n", name());
495 
496  const bool is_error = pkt->isError();
497 
498  if (is_error) {
499  DPRINTF(Cache, "%s: Cache received %s with error\n", __func__,
500  pkt->print());
501  }
502 
503  DPRINTF(Cache, "%s: Handling response %s\n", __func__,
504  pkt->print());
505 
506  // if this is a write, we should be looking at an uncacheable
507  // write
508  if (pkt->isWrite() && pkt->cmd != MemCmd::LockedRMWWriteResp) {
509  assert(pkt->req->isUncacheable());
511  return;
512  }
513 
514  // we have dealt with any (uncacheable) writes above, from here on
515  // we know we are dealing with an MSHR due to a miss or a prefetch
516  MSHR *mshr = dynamic_cast<MSHR*>(pkt->popSenderState());
517  assert(mshr);
518 
519  if (mshr == noTargetMSHR) {
520  // we always clear at least one target
522  noTargetMSHR = nullptr;
523  }
524 
525  // Initial target is used just for stats
526  const QueueEntry::Target *initial_tgt = mshr->getTarget();
527  const Tick miss_latency = curTick() - initial_tgt->recvTime;
528  if (pkt->req->isUncacheable()) {
529  assert(pkt->req->requestorId() < system->maxRequestors());
530  stats.cmdStats(initial_tgt->pkt)
531  .mshrUncacheableLatency[pkt->req->requestorId()] += miss_latency;
532  } else {
533  assert(pkt->req->requestorId() < system->maxRequestors());
534  stats.cmdStats(initial_tgt->pkt)
535  .mshrMissLatency[pkt->req->requestorId()] += miss_latency;
536  }
537 
538  PacketList writebacks;
539 
540  bool is_fill = !mshr->isForward &&
541  (pkt->isRead() || pkt->cmd == MemCmd::UpgradeResp ||
542  mshr->wasWholeLineWrite);
543 
544  // make sure that if the mshr was due to a whole line write then
545  // the response is an invalidation
546  assert(!mshr->wasWholeLineWrite || pkt->isInvalidate());
547 
548  CacheBlk *blk = tags->findBlock(pkt->getAddr(), pkt->isSecure());
549 
550  if (is_fill && !is_error) {
551  DPRINTF(Cache, "Block for addr %#llx being updated in Cache\n",
552  pkt->getAddr());
553 
554  const bool allocate = (writeAllocator && mshr->wasWholeLineWrite) ?
555  writeAllocator->allocate() : mshr->allocOnFill();
556  blk = handleFill(pkt, blk, writebacks, allocate);
557  assert(blk != nullptr);
558  ppFill->notify(pkt);
559  }
560 
561  // Don't want to promote the Locked RMW Read until
562  // the locked write comes in
563  if (!mshr->hasLockedRMWReadTarget()) {
564  if (blk && blk->isValid() && pkt->isClean() && !pkt->isInvalidate()) {
565  // The block was marked not readable while there was a pending
566  // cache maintenance operation, restore its flag.
568 
569  // This was a cache clean operation (without invalidate)
570  // and we have a copy of the block already. Since there
571  // is no invalidation, we can promote targets that don't
572  // require a writable copy
573  mshr->promoteReadable();
574  }
575 
576  if (blk && blk->isSet(CacheBlk::WritableBit) &&
577  !pkt->req->isCacheInvalidate()) {
578  // If at this point the referenced block is writable and the
579  // response is not a cache invalidate, we promote targets that
580  // were deferred as we couldn't guarrantee a writable copy
581  mshr->promoteWritable();
582  }
583  }
584 
585  serviceMSHRTargets(mshr, pkt, blk);
586  // We are stopping servicing targets early for the Locked RMW Read until
587  // the write comes.
588  if (!mshr->hasLockedRMWReadTarget()) {
589  if (mshr->promoteDeferredTargets()) {
590  // avoid later read getting stale data while write miss is
591  // outstanding.. see comment in timingAccess()
592  if (blk) {
594  }
595  mshrQueue.markPending(mshr);
597  } else {
598  // while we deallocate an mshr from the queue we still have to
599  // check the isFull condition before and after as we might
600  // have been using the reserved entries already
601  const bool was_full = mshrQueue.isFull();
602  mshrQueue.deallocate(mshr);
603  if (was_full && !mshrQueue.isFull()) {
605  }
606 
607  // Request the bus for a prefetch if this deallocation freed enough
608  // MSHRs for a prefetch to take place
609  if (prefetcher && mshrQueue.canPrefetch() && !isBlocked()) {
610  Tick next_pf_time = std::max(
612  if (next_pf_time != MaxTick)
613  schedMemSideSendEvent(next_pf_time);
614  }
615  }
616 
617  // if we used temp block, check to see if its valid and then clear it
618  if (blk == tempBlock && tempBlock->isValid()) {
619  evictBlock(blk, writebacks);
620  }
621  }
622 
623  const Tick forward_time = clockEdge(forwardLatency) + pkt->headerDelay;
624  // copy writebacks to write buffer
625  doWritebacks(writebacks, forward_time);
626 
627  DPRINTF(CacheVerbose, "%s: Leaving with %s\n", __func__, pkt->print());
628  delete pkt;
629 }
630 
631 
632 Tick
634 {
635  // should assert here that there are no outstanding MSHRs or
636  // writebacks... that would mean that someone used an atomic
637  // access in timing mode
638 
639  // We use lookupLatency here because it is used to specify the latency
640  // to access.
641  Cycles lat = lookupLatency;
642 
643  CacheBlk *blk = nullptr;
644  PacketList writebacks;
645  bool satisfied = access(pkt, blk, lat, writebacks);
646 
647  if (pkt->isClean() && blk && blk->isSet(CacheBlk::DirtyBit)) {
648  // A cache clean opearation is looking for a dirty
649  // block. If a dirty block is encountered a WriteClean
650  // will update any copies to the path to the memory
651  // until the point of reference.
652  DPRINTF(CacheVerbose, "%s: packet %s found block: %s\n",
653  __func__, pkt->print(), blk->print());
654  PacketPtr wb_pkt = writecleanBlk(blk, pkt->req->getDest(), pkt->id);
655  writebacks.push_back(wb_pkt);
656  pkt->setSatisfied();
657  }
658 
659  // handle writebacks resulting from the access here to ensure they
660  // logically precede anything happening below
661  doWritebacksAtomic(writebacks);
662  assert(writebacks.empty());
663 
664  if (!satisfied) {
665  lat += handleAtomicReqMiss(pkt, blk, writebacks);
666  }
667 
668  // Note that we don't invoke the prefetcher at all in atomic mode.
669  // It's not clear how to do it properly, particularly for
670  // prefetchers that aggressively generate prefetch candidates and
671  // rely on bandwidth contention to throttle them; these will tend
672  // to pollute the cache in atomic mode since there is no bandwidth
673  // contention. If we ever do want to enable prefetching in atomic
674  // mode, though, this is the place to do it... see timingAccess()
675  // for an example (though we'd want to issue the prefetch(es)
676  // immediately rather than calling requestMemSideBus() as we do
677  // there).
678 
679  // do any writebacks resulting from the response handling
680  doWritebacksAtomic(writebacks);
681 
682  // if we used temp block, check to see if its valid and if so
683  // clear it out, but only do so after the call to recvAtomic is
684  // finished so that any downstream observers (such as a snoop
685  // filter), first see the fill, and only then see the eviction
686  if (blk == tempBlock && tempBlock->isValid()) {
687  // the atomic CPU calls recvAtomic for fetch and load/store
688  // sequentuially, and we may already have a tempBlock
689  // writeback from the fetch that we have not yet sent
690  if (tempBlockWriteback) {
691  // if that is the case, write the prevoius one back, and
692  // do not schedule any new event
694  } else {
695  // the writeback/clean eviction happens after the call to
696  // recvAtomic has finished (but before any successive
697  // calls), so that the response handling from the fill is
698  // allowed to happen first
700  }
701 
703  }
704 
705  if (pkt->needsResponse()) {
706  pkt->makeAtomicResponse();
707  }
708 
709  return lat * clockPeriod();
710 }
711 
712 void
713 BaseCache::functionalAccess(PacketPtr pkt, bool from_cpu_side)
714 {
715  Addr blk_addr = pkt->getBlockAddr(blkSize);
716  bool is_secure = pkt->isSecure();
717  CacheBlk *blk = tags->findBlock(pkt->getAddr(), is_secure);
718  MSHR *mshr = mshrQueue.findMatch(blk_addr, is_secure);
719 
720  pkt->pushLabel(name());
721 
722  CacheBlkPrintWrapper cbpw(blk);
723 
724  // Note that just because an L2/L3 has valid data doesn't mean an
725  // L1 doesn't have a more up-to-date modified copy that still
726  // needs to be found. As a result we always update the request if
727  // we have it, but only declare it satisfied if we are the owner.
728 
729  // see if we have data at all (owned or otherwise)
730  bool have_data = blk && blk->isValid()
731  && pkt->trySatisfyFunctional(&cbpw, blk_addr, is_secure, blkSize,
732  blk->data);
733 
734  // data we have is dirty if marked as such or if we have an
735  // in-service MSHR that is pending a modified line
736  bool have_dirty =
737  have_data && (blk->isSet(CacheBlk::DirtyBit) ||
738  (mshr && mshr->inService && mshr->isPendingModified()));
739 
740  bool done = have_dirty ||
745 
746  DPRINTF(CacheVerbose, "%s: %s %s%s%s\n", __func__, pkt->print(),
747  (blk && blk->isValid()) ? "valid " : "",
748  have_data ? "data " : "", done ? "done " : "");
749 
750  // We're leaving the cache, so pop cache->name() label
751  pkt->popLabel();
752 
753  if (done) {
754  pkt->makeResponse();
755  } else {
756  // if it came as a request from the CPU side then make sure it
757  // continues towards the memory side
758  if (from_cpu_side) {
760  } else if (cpuSidePort.isSnooping()) {
761  // if it came from the memory side, it must be a snoop request
762  // and we should only forward it if we are forwarding snoops
764  }
765  }
766 }
767 
768 void
770  bool has_old_data)
771 {
772  DataUpdate data_update(regenerateBlkAddr(blk), blk->isSecure());
773  if (ppDataUpdate->hasListeners()) {
774  if (has_old_data) {
775  data_update.oldData = std::vector<uint64_t>(blk->data,
776  blk->data + (blkSize / sizeof(uint64_t)));
777  }
778  }
779 
780  // Actually perform the data update
781  if (cpkt) {
782  cpkt->writeDataToBlock(blk->data, blkSize);
783  }
784 
785  if (ppDataUpdate->hasListeners()) {
786  if (cpkt) {
787  data_update.newData = std::vector<uint64_t>(blk->data,
788  blk->data + (blkSize / sizeof(uint64_t)));
789  }
790  ppDataUpdate->notify(data_update);
791  }
792 }
793 
794 void
796 {
797  assert(pkt->isRequest());
798 
799  uint64_t overwrite_val;
800  bool overwrite_mem;
801  uint64_t condition_val64;
802  uint32_t condition_val32;
803 
804  int offset = pkt->getOffset(blkSize);
805  uint8_t *blk_data = blk->data + offset;
806 
807  assert(sizeof(uint64_t) >= pkt->getSize());
808 
809  // Get a copy of the old block's contents for the probe before the update
810  DataUpdate data_update(regenerateBlkAddr(blk), blk->isSecure());
811  if (ppDataUpdate->hasListeners()) {
812  data_update.oldData = std::vector<uint64_t>(blk->data,
813  blk->data + (blkSize / sizeof(uint64_t)));
814  }
815 
816  overwrite_mem = true;
817  // keep a copy of our possible write value, and copy what is at the
818  // memory address into the packet
819  pkt->writeData((uint8_t *)&overwrite_val);
820  pkt->setData(blk_data);
821 
822  if (pkt->req->isCondSwap()) {
823  if (pkt->getSize() == sizeof(uint64_t)) {
824  condition_val64 = pkt->req->getExtraData();
825  overwrite_mem = !std::memcmp(&condition_val64, blk_data,
826  sizeof(uint64_t));
827  } else if (pkt->getSize() == sizeof(uint32_t)) {
828  condition_val32 = (uint32_t)pkt->req->getExtraData();
829  overwrite_mem = !std::memcmp(&condition_val32, blk_data,
830  sizeof(uint32_t));
831  } else
832  panic("Invalid size for conditional read/write\n");
833  }
834 
835  if (overwrite_mem) {
836  std::memcpy(blk_data, &overwrite_val, pkt->getSize());
838 
839  if (ppDataUpdate->hasListeners()) {
840  data_update.newData = std::vector<uint64_t>(blk->data,
841  blk->data + (blkSize / sizeof(uint64_t)));
842  ppDataUpdate->notify(data_update);
843  }
844  }
845 }
846 
847 QueueEntry*
849 {
850  // Check both MSHR queue and write buffer for potential requests,
851  // note that null does not mean there is no request, it could
852  // simply be that it is not ready
853  MSHR *miss_mshr = mshrQueue.getNext();
854  WriteQueueEntry *wq_entry = writeBuffer.getNext();
855 
856  // If we got a write buffer request ready, first priority is a
857  // full write buffer, otherwise we favour the miss requests
858  if (wq_entry && (writeBuffer.isFull() || !miss_mshr)) {
859  // need to search MSHR queue for conflicting earlier miss.
860  MSHR *conflict_mshr = mshrQueue.findPending(wq_entry);
861 
862  if (conflict_mshr && conflict_mshr->order < wq_entry->order) {
863  // Service misses in order until conflict is cleared.
864  return conflict_mshr;
865 
866  // @todo Note that we ignore the ready time of the conflict here
867  }
868 
869  // No conflicts; issue write
870  return wq_entry;
871  } else if (miss_mshr) {
872  // need to check for conflicting earlier writeback
873  WriteQueueEntry *conflict_mshr = writeBuffer.findPending(miss_mshr);
874  if (conflict_mshr) {
875  // not sure why we don't check order here... it was in the
876  // original code but commented out.
877 
878  // The only way this happens is if we are
879  // doing a write and we didn't have permissions
880  // then subsequently saw a writeback (owned got evicted)
881  // We need to make sure to perform the writeback first
882  // To preserve the dirty data, then we can issue the write
883 
884  // should we return wq_entry here instead? I.e. do we
885  // have to flush writes in order? I don't think so... not
886  // for Alpha anyway. Maybe for x86?
887  return conflict_mshr;
888 
889  // @todo Note that we ignore the ready time of the conflict here
890  }
891 
892  // No conflicts; issue read
893  return miss_mshr;
894  }
895 
896  // fall through... no pending requests. Try a prefetch.
897  assert(!miss_mshr && !wq_entry);
898  if (prefetcher && mshrQueue.canPrefetch() && !isBlocked()) {
899  // If we have a miss queue slot, we can try a prefetch
900  PacketPtr pkt = prefetcher->getPacket();
901  if (pkt) {
902  Addr pf_addr = pkt->getBlockAddr(blkSize);
903  if (tags->findBlock(pf_addr, pkt->isSecure())) {
904  DPRINTF(HWPrefetch, "Prefetch %#x has hit in cache, "
905  "dropped.\n", pf_addr);
907  // free the request and packet
908  delete pkt;
909  } else if (mshrQueue.findMatch(pf_addr, pkt->isSecure())) {
910  DPRINTF(HWPrefetch, "Prefetch %#x has hit in a MSHR, "
911  "dropped.\n", pf_addr);
913  // free the request and packet
914  delete pkt;
915  } else if (writeBuffer.findMatch(pf_addr, pkt->isSecure())) {
916  DPRINTF(HWPrefetch, "Prefetch %#x has hit in the "
917  "Write Buffer, dropped.\n", pf_addr);
919  // free the request and packet
920  delete pkt;
921  } else {
922  // Update statistic on number of prefetches issued
923  // (hwpf_mshr_misses)
924  assert(pkt->req->requestorId() < system->maxRequestors());
925  stats.cmdStats(pkt).mshrMisses[pkt->req->requestorId()]++;
926 
927  // allocate an MSHR and return it, note
928  // that we send the packet straight away, so do not
929  // schedule the send
930  return allocateMissBuffer(pkt, curTick(), false);
931  }
932  }
933  }
934 
935  return nullptr;
936 }
937 
938 bool
940  PacketList &writebacks)
941 {
942  bool replacement = false;
943  for (const auto& blk : evict_blks) {
944  if (blk->isValid()) {
945  replacement = true;
946 
947  const MSHR* mshr =
948  mshrQueue.findMatch(regenerateBlkAddr(blk), blk->isSecure());
949  if (mshr) {
950  // Must be an outstanding upgrade or clean request on a block
951  // we're about to replace
952  assert((!blk->isSet(CacheBlk::WritableBit) &&
953  mshr->needsWritable()) || mshr->isCleaning());
954  return false;
955  }
956  }
957  }
958 
959  // The victim will be replaced by a new entry, so increase the replacement
960  // counter if a valid block is being replaced
961  if (replacement) {
963 
964  // Evict valid blocks associated to this victim block
965  for (auto& blk : evict_blks) {
966  if (blk->isValid()) {
967  evictBlock(blk, writebacks);
968  }
969  }
970  }
971 
972  return true;
973 }
974 
975 bool
977  PacketList &writebacks)
978 {
979  // tempBlock does not exist in the tags, so don't do anything for it.
980  if (blk == tempBlock) {
981  return true;
982  }
983 
984  // The compressor is called to compress the updated data, so that its
985  // metadata can be updated.
986  Cycles compression_lat = Cycles(0);
987  Cycles decompression_lat = Cycles(0);
988  const auto comp_data =
989  compressor->compress(data, compression_lat, decompression_lat);
990  std::size_t compression_size = comp_data->getSizeBits();
991 
992  // Get previous compressed size
993  CompressionBlk* compression_blk = static_cast<CompressionBlk*>(blk);
994  [[maybe_unused]] const std::size_t prev_size =
995  compression_blk->getSizeBits();
996 
997  // If compressed size didn't change enough to modify its co-allocatability
998  // there is nothing to do. Otherwise we may be facing a data expansion
999  // (block passing from more compressed to less compressed state), or a
1000  // data contraction (less to more).
1001  bool is_data_expansion = false;
1002  bool is_data_contraction = false;
1003  const CompressionBlk::OverwriteType overwrite_type =
1004  compression_blk->checkExpansionContraction(compression_size);
1005  std::string op_name = "";
1006  if (overwrite_type == CompressionBlk::DATA_EXPANSION) {
1007  op_name = "expansion";
1008  is_data_expansion = true;
1009  } else if ((overwrite_type == CompressionBlk::DATA_CONTRACTION) &&
1010  moveContractions) {
1011  op_name = "contraction";
1012  is_data_contraction = true;
1013  }
1014 
1015  // If block changed compression state, it was possibly co-allocated with
1016  // other blocks and cannot be co-allocated anymore, so one or more blocks
1017  // must be evicted to make room for the expanded/contracted block
1018  std::vector<CacheBlk*> evict_blks;
1019  if (is_data_expansion || is_data_contraction) {
1020  std::vector<CacheBlk*> evict_blks;
1021  bool victim_itself = false;
1022  CacheBlk *victim = nullptr;
1023  if (replaceExpansions || is_data_contraction) {
1024  victim = tags->findVictim(regenerateBlkAddr(blk),
1025  blk->isSecure(), compression_size, evict_blks);
1026 
1027  // It is valid to return nullptr if there is no victim
1028  if (!victim) {
1029  return false;
1030  }
1031 
1032  // If the victim block is itself the block won't need to be moved,
1033  // and the victim should not be evicted
1034  if (blk == victim) {
1035  victim_itself = true;
1036  auto it = std::find_if(evict_blks.begin(), evict_blks.end(),
1037  [&blk](CacheBlk* evict_blk){ return evict_blk == blk; });
1038  evict_blks.erase(it);
1039  }
1040 
1041  // Print victim block's information
1042  DPRINTF(CacheRepl, "Data %s replacement victim: %s\n",
1043  op_name, victim->print());
1044  } else {
1045  // If we do not move the expanded block, we must make room for
1046  // the expansion to happen, so evict every co-allocated block
1047  const SuperBlk* superblock = static_cast<const SuperBlk*>(
1048  compression_blk->getSectorBlock());
1049  for (auto& sub_blk : superblock->blks) {
1050  if (sub_blk->isValid() && (blk != sub_blk)) {
1051  evict_blks.push_back(sub_blk);
1052  }
1053  }
1054  }
1055 
1056  // Try to evict blocks; if it fails, give up on update
1057  if (!handleEvictions(evict_blks, writebacks)) {
1058  return false;
1059  }
1060 
1061  DPRINTF(CacheComp, "Data %s: [%s] from %d to %d bits\n",
1062  op_name, blk->print(), prev_size, compression_size);
1063 
1064  if (!victim_itself && (replaceExpansions || is_data_contraction)) {
1065  // Move the block's contents to the invalid block so that it now
1066  // co-allocates with the other existing superblock entry
1067  tags->moveBlock(blk, victim);
1068  blk = victim;
1069  compression_blk = static_cast<CompressionBlk*>(blk);
1070  }
1071  }
1072 
1073  // Update the number of data expansions/contractions
1074  if (is_data_expansion) {
1076  } else if (is_data_contraction) {
1078  }
1079 
1080  compression_blk->setSizeBits(compression_size);
1081  compression_blk->setDecompressionLatency(decompression_lat);
1082 
1083  return true;
1084 }
1085 
1086 void
1088 {
1089  assert(pkt->isRequest());
1090 
1091  assert(blk && blk->isValid());
1092  // Occasionally this is not true... if we are a lower-level cache
1093  // satisfying a string of Read and ReadEx requests from
1094  // upper-level caches, a Read will mark the block as shared but we
1095  // can satisfy a following ReadEx anyway since we can rely on the
1096  // Read requestor(s) to have buffered the ReadEx snoop and to
1097  // invalidate their blocks after receiving them.
1098  // assert(!pkt->needsWritable() || blk->isSet(CacheBlk::WritableBit));
1099  assert(pkt->getOffset(blkSize) + pkt->getSize() <= blkSize);
1100 
1101  // Check RMW operations first since both isRead() and
1102  // isWrite() will be true for them
1103  if (pkt->cmd == MemCmd::SwapReq) {
1104  if (pkt->isAtomicOp()) {
1105  // Get a copy of the old block's contents for the probe before
1106  // the update
1107  DataUpdate data_update(regenerateBlkAddr(blk), blk->isSecure());
1108  if (ppDataUpdate->hasListeners()) {
1109  data_update.oldData = std::vector<uint64_t>(blk->data,
1110  blk->data + (blkSize / sizeof(uint64_t)));
1111  }
1112 
1113  // extract data from cache and save it into the data field in
1114  // the packet as a return value from this atomic op
1115  int offset = tags->extractBlkOffset(pkt->getAddr());
1116  uint8_t *blk_data = blk->data + offset;
1117  pkt->setData(blk_data);
1118 
1119  // execute AMO operation
1120  (*(pkt->getAtomicOp()))(blk_data);
1121 
1122  // Inform of this block's data contents update
1123  if (ppDataUpdate->hasListeners()) {
1124  data_update.newData = std::vector<uint64_t>(blk->data,
1125  blk->data + (blkSize / sizeof(uint64_t)));
1126  ppDataUpdate->notify(data_update);
1127  }
1128 
1129  // set block status to dirty
1131  } else {
1132  cmpAndSwap(blk, pkt);
1133  }
1134  } else if (pkt->isWrite()) {
1135  // we have the block in a writable state and can go ahead,
1136  // note that the line may be also be considered writable in
1137  // downstream caches along the path to memory, but always
1138  // Exclusive, and never Modified
1139  assert(blk->isSet(CacheBlk::WritableBit));
1140  // Write or WriteLine at the first cache with block in writable state
1141  if (blk->checkWrite(pkt)) {
1142  updateBlockData(blk, pkt, true);
1143  }
1144  // Always mark the line as dirty (and thus transition to the
1145  // Modified state) even if we are a failed StoreCond so we
1146  // supply data to any snoops that have appended themselves to
1147  // this cache before knowing the store will fail.
1149  DPRINTF(CacheVerbose, "%s for %s (write)\n", __func__, pkt->print());
1150  } else if (pkt->isRead()) {
1151  if (pkt->isLLSC()) {
1152  blk->trackLoadLocked(pkt);
1153  }
1154 
1155  // all read responses have a data payload
1156  assert(pkt->hasRespData());
1157  pkt->setDataFromBlock(blk->data, blkSize);
1158  } else if (pkt->isUpgrade()) {
1159  // sanity check
1160  assert(!pkt->hasSharers());
1161 
1162  if (blk->isSet(CacheBlk::DirtyBit)) {
1163  // we were in the Owned state, and a cache above us that
1164  // has the line in Shared state needs to be made aware
1165  // that the data it already has is in fact dirty
1166  pkt->setCacheResponding();
1168  }
1169  } else if (pkt->isClean()) {
1171  } else {
1172  assert(pkt->isInvalidate());
1173  invalidateBlock(blk);
1174  DPRINTF(CacheVerbose, "%s for %s (invalidation)\n", __func__,
1175  pkt->print());
1176  }
1177 }
1178 
1180 //
1181 // Access path: requests coming in from the CPU side
1182 //
1184 Cycles
1186  const Cycles lookup_lat) const
1187 {
1188  // A tag-only access has to wait for the packet to arrive in order to
1189  // perform the tag lookup.
1190  return ticksToCycles(delay) + lookup_lat;
1191 }
1192 
1193 Cycles
1194 BaseCache::calculateAccessLatency(const CacheBlk* blk, const uint32_t delay,
1195  const Cycles lookup_lat) const
1196 {
1197  Cycles lat(0);
1198 
1199  if (blk != nullptr) {
1200  // As soon as the access arrives, for sequential accesses first access
1201  // tags, then the data entry. In the case of parallel accesses the
1202  // latency is dictated by the slowest of tag and data latencies.
1203  if (sequentialAccess) {
1204  lat = ticksToCycles(delay) + lookup_lat + dataLatency;
1205  } else {
1206  lat = ticksToCycles(delay) + std::max(lookup_lat, dataLatency);
1207  }
1208 
1209  // Check if the block to be accessed is available. If not, apply the
1210  // access latency on top of when the block is ready to be accessed.
1211  const Tick tick = curTick() + delay;
1212  const Tick when_ready = blk->getWhenReady();
1213  if (when_ready > tick &&
1214  ticksToCycles(when_ready - tick) > lat) {
1215  lat += ticksToCycles(when_ready - tick);
1216  }
1217  } else {
1218  // In case of a miss, we neglect the data access in a parallel
1219  // configuration (i.e., the data access will be stopped as soon as
1220  // we find out it is a miss), and use the tag-only latency.
1221  lat = calculateTagOnlyLatency(delay, lookup_lat);
1222  }
1223 
1224  return lat;
1225 }
1226 
1227 bool
1229  PacketList &writebacks)
1230 {
1231  // sanity check
1232  assert(pkt->isRequest());
1233 
1234  gem5_assert(!(isReadOnly && pkt->isWrite()),
1235  "Should never see a write in a read-only cache %s\n",
1236  name());
1237 
1238  // Access block in the tags
1239  Cycles tag_latency(0);
1240  blk = tags->accessBlock(pkt, tag_latency);
1241 
1242  DPRINTF(Cache, "%s for %s %s\n", __func__, pkt->print(),
1243  blk ? "hit " + blk->print() : "miss");
1244 
1245  if (pkt->req->isCacheMaintenance()) {
1246  // A cache maintenance operation is always forwarded to the
1247  // memory below even if the block is found in dirty state.
1248 
1249  // We defer any changes to the state of the block until we
1250  // create and mark as in service the mshr for the downstream
1251  // packet.
1252 
1253  // Calculate access latency on top of when the packet arrives. This
1254  // takes into account the bus delay.
1255  lat = calculateTagOnlyLatency(pkt->headerDelay, tag_latency);
1256 
1257  return false;
1258  }
1259 
1260  if (pkt->isEviction()) {
1261  // We check for presence of block in above caches before issuing
1262  // Writeback or CleanEvict to write buffer. Therefore the only
1263  // possible cases can be of a CleanEvict packet coming from above
1264  // encountering a Writeback generated in this cache peer cache and
1265  // waiting in the write buffer. Cases of upper level peer caches
1266  // generating CleanEvict and Writeback or simply CleanEvict and
1267  // CleanEvict almost simultaneously will be caught by snoops sent out
1268  // by crossbar.
1269  WriteQueueEntry *wb_entry = writeBuffer.findMatch(pkt->getAddr(),
1270  pkt->isSecure());
1271  if (wb_entry) {
1272  assert(wb_entry->getNumTargets() == 1);
1273  PacketPtr wbPkt = wb_entry->getTarget()->pkt;
1274  assert(wbPkt->isWriteback());
1275 
1276  if (pkt->isCleanEviction()) {
1277  // The CleanEvict and WritebackClean snoops into other
1278  // peer caches of the same level while traversing the
1279  // crossbar. If a copy of the block is found, the
1280  // packet is deleted in the crossbar. Hence, none of
1281  // the other upper level caches connected to this
1282  // cache have the block, so we can clear the
1283  // BLOCK_CACHED flag in the Writeback if set and
1284  // discard the CleanEvict by returning true.
1285  wbPkt->clearBlockCached();
1286 
1287  // A clean evict does not need to access the data array
1288  lat = calculateTagOnlyLatency(pkt->headerDelay, tag_latency);
1289 
1290  return true;
1291  } else {
1292  assert(pkt->cmd == MemCmd::WritebackDirty);
1293  // Dirty writeback from above trumps our clean
1294  // writeback... discard here
1295  // Note: markInService will remove entry from writeback buffer.
1296  markInService(wb_entry);
1297  delete wbPkt;
1298  }
1299  }
1300  }
1301 
1302  // The critical latency part of a write depends only on the tag access
1303  if (pkt->isWrite()) {
1304  lat = calculateTagOnlyLatency(pkt->headerDelay, tag_latency);
1305  }
1306 
1307  // Writeback handling is special case. We can write the block into
1308  // the cache without having a writeable copy (or any copy at all).
1309  if (pkt->isWriteback()) {
1310  assert(blkSize == pkt->getSize());
1311 
1312  // we could get a clean writeback while we are having
1313  // outstanding accesses to a block, do the simple thing for
1314  // now and drop the clean writeback so that we do not upset
1315  // any ordering/decisions about ownership already taken
1316  if (pkt->cmd == MemCmd::WritebackClean &&
1317  mshrQueue.findMatch(pkt->getAddr(), pkt->isSecure())) {
1318  DPRINTF(Cache, "Clean writeback %#llx to block with MSHR, "
1319  "dropping\n", pkt->getAddr());
1320 
1321  // A writeback searches for the block, then writes the data.
1322  // As the writeback is being dropped, the data is not touched,
1323  // and we just had to wait for the time to find a match in the
1324  // MSHR. As of now assume a mshr queue search takes as long as
1325  // a tag lookup for simplicity.
1326  return true;
1327  }
1328 
1329  const bool has_old_data = blk && blk->isValid();
1330  if (!blk) {
1331  // need to do a replacement
1332  blk = allocateBlock(pkt, writebacks);
1333  if (!blk) {
1334  // no replaceable block available: give up, fwd to next level.
1335  incMissCount(pkt);
1336  return false;
1337  }
1338 
1340  } else if (compressor) {
1341  // This is an overwrite to an existing block, therefore we need
1342  // to check for data expansion (i.e., block was compressed with
1343  // a smaller size, and now it doesn't fit the entry anymore).
1344  // If that is the case we might need to evict blocks.
1345  if (!updateCompressionData(blk, pkt->getConstPtr<uint64_t>(),
1346  writebacks)) {
1347  invalidateBlock(blk);
1348  return false;
1349  }
1350  }
1351 
1352  // only mark the block dirty if we got a writeback command,
1353  // and leave it as is for a clean writeback
1354  if (pkt->cmd == MemCmd::WritebackDirty) {
1355  // TODO: the coherent cache can assert that the dirty bit is set
1357  }
1358  // if the packet does not have sharers, it is passing
1359  // writable, and we got the writeback in Modified or Exclusive
1360  // state, if not we are in the Owned or Shared state
1361  if (!pkt->hasSharers()) {
1363  }
1364  // nothing else to do; writeback doesn't expect response
1365  assert(!pkt->needsResponse());
1366 
1367  updateBlockData(blk, pkt, has_old_data);
1368  DPRINTF(Cache, "%s new state is %s\n", __func__, blk->print());
1369  incHitCount(pkt);
1370 
1371  // When the packet metadata arrives, the tag lookup will be done while
1372  // the payload is arriving. Then the block will be ready to access as
1373  // soon as the fill is done
1375  std::max(cyclesToTicks(tag_latency), (uint64_t)pkt->payloadDelay));
1376 
1377  return true;
1378  } else if (pkt->cmd == MemCmd::CleanEvict) {
1379  // A CleanEvict does not need to access the data array
1380  lat = calculateTagOnlyLatency(pkt->headerDelay, tag_latency);
1381 
1382  if (blk) {
1383  // Found the block in the tags, need to stop CleanEvict from
1384  // propagating further down the hierarchy. Returning true will
1385  // treat the CleanEvict like a satisfied write request and delete
1386  // it.
1387  return true;
1388  }
1389  // We didn't find the block here, propagate the CleanEvict further
1390  // down the memory hierarchy. Returning false will treat the CleanEvict
1391  // like a Writeback which could not find a replaceable block so has to
1392  // go to next level.
1393  return false;
1394  } else if (pkt->cmd == MemCmd::WriteClean) {
1395  // WriteClean handling is a special case. We can allocate a
1396  // block directly if it doesn't exist and we can update the
1397  // block immediately. The WriteClean transfers the ownership
1398  // of the block as well.
1399  assert(blkSize == pkt->getSize());
1400 
1401  const bool has_old_data = blk && blk->isValid();
1402  if (!blk) {
1403  if (pkt->writeThrough()) {
1404  // if this is a write through packet, we don't try to
1405  // allocate if the block is not present
1406  return false;
1407  } else {
1408  // a writeback that misses needs to allocate a new block
1409  blk = allocateBlock(pkt, writebacks);
1410  if (!blk) {
1411  // no replaceable block available: give up, fwd to
1412  // next level.
1413  incMissCount(pkt);
1414  return false;
1415  }
1416 
1418  }
1419  } else if (compressor) {
1420  // This is an overwrite to an existing block, therefore we need
1421  // to check for data expansion (i.e., block was compressed with
1422  // a smaller size, and now it doesn't fit the entry anymore).
1423  // If that is the case we might need to evict blocks.
1424  if (!updateCompressionData(blk, pkt->getConstPtr<uint64_t>(),
1425  writebacks)) {
1426  invalidateBlock(blk);
1427  return false;
1428  }
1429  }
1430 
1431  // at this point either this is a writeback or a write-through
1432  // write clean operation and the block is already in this
1433  // cache, we need to update the data and the block flags
1434  assert(blk);
1435  // TODO: the coherent cache can assert that the dirty bit is set
1436  if (!pkt->writeThrough()) {
1438  }
1439  // nothing else to do; writeback doesn't expect response
1440  assert(!pkt->needsResponse());
1441 
1442  updateBlockData(blk, pkt, has_old_data);
1443  DPRINTF(Cache, "%s new state is %s\n", __func__, blk->print());
1444 
1445  incHitCount(pkt);
1446 
1447  // When the packet metadata arrives, the tag lookup will be done while
1448  // the payload is arriving. Then the block will be ready to access as
1449  // soon as the fill is done
1451  std::max(cyclesToTicks(tag_latency), (uint64_t)pkt->payloadDelay));
1452 
1453  // If this a write-through packet it will be sent to cache below
1454  return !pkt->writeThrough();
1455  } else if (blk && (pkt->needsWritable() ?
1457  blk->isSet(CacheBlk::ReadableBit))) {
1458  // OK to satisfy access
1459  incHitCount(pkt);
1460 
1461  // Calculate access latency based on the need to access the data array
1462  if (pkt->isRead()) {
1463  lat = calculateAccessLatency(blk, pkt->headerDelay, tag_latency);
1464 
1465  // When a block is compressed, it must first be decompressed
1466  // before being read. This adds to the access latency.
1467  if (compressor) {
1468  lat += compressor->getDecompressionLatency(blk);
1469  }
1470  } else {
1471  lat = calculateTagOnlyLatency(pkt->headerDelay, tag_latency);
1472  }
1473 
1474  satisfyRequest(pkt, blk);
1475  maintainClusivity(pkt->fromCache(), blk);
1476 
1477  return true;
1478  }
1479 
1480  // Can't satisfy access normally... either no block (blk == nullptr)
1481  // or have block but need writable
1482 
1483  incMissCount(pkt);
1484 
1485  lat = calculateAccessLatency(blk, pkt->headerDelay, tag_latency);
1486 
1487  if (!blk && pkt->isLLSC() && pkt->isWrite()) {
1488  // complete miss on store conditional... just give up now
1489  pkt->req->setExtraData(0);
1490  return true;
1491  }
1492 
1493  return false;
1494 }
1495 
1496 void
1498 {
1499  if (from_cache && blk && blk->isValid() &&
1500  !blk->isSet(CacheBlk::DirtyBit) && clusivity == enums::mostly_excl) {
1501  // if we have responded to a cache, and our block is still
1502  // valid, but not dirty, and this cache is mostly exclusive
1503  // with respect to the cache above, drop the block
1504  invalidateBlock(blk);
1505  }
1506 }
1507 
1508 CacheBlk*
1510  bool allocate)
1511 {
1512  assert(pkt->isResponse());
1513  Addr addr = pkt->getAddr();
1514  bool is_secure = pkt->isSecure();
1515  const bool has_old_data = blk && blk->isValid();
1516  const std::string old_state = (debug::Cache && blk) ? blk->print() : "";
1517 
1518  // When handling a fill, we should have no writes to this line.
1519  assert(addr == pkt->getBlockAddr(blkSize));
1520  assert(!writeBuffer.findMatch(addr, is_secure));
1521 
1522  if (!blk) {
1523  // better have read new data...
1524  assert(pkt->hasData() || pkt->cmd == MemCmd::InvalidateResp);
1525 
1526  // need to do a replacement if allocating, otherwise we stick
1527  // with the temporary storage
1528  blk = allocate ? allocateBlock(pkt, writebacks) : nullptr;
1529 
1530  if (!blk) {
1531  // No replaceable block or a mostly exclusive
1532  // cache... just use temporary storage to complete the
1533  // current request and then get rid of it
1534  blk = tempBlock;
1535  tempBlock->insert(addr, is_secure);
1536  DPRINTF(Cache, "using temp block for %#llx (%s)\n", addr,
1537  is_secure ? "s" : "ns");
1538  }
1539  } else {
1540  // existing block... probably an upgrade
1541  // don't clear block status... if block is already dirty we
1542  // don't want to lose that
1543  }
1544 
1545  // Block is guaranteed to be valid at this point
1546  assert(blk->isValid());
1547  assert(blk->isSecure() == is_secure);
1548  assert(regenerateBlkAddr(blk) == addr);
1549 
1551 
1552  // sanity check for whole-line writes, which should always be
1553  // marked as writable as part of the fill, and then later marked
1554  // dirty as part of satisfyRequest
1555  if (pkt->cmd == MemCmd::InvalidateResp) {
1556  assert(!pkt->hasSharers());
1557  }
1558 
1559  // here we deal with setting the appropriate state of the line,
1560  // and we start by looking at the hasSharers flag, and ignore the
1561  // cacheResponding flag (normally signalling dirty data) if the
1562  // packet has sharers, thus the line is never allocated as Owned
1563  // (dirty but not writable), and always ends up being either
1564  // Shared, Exclusive or Modified, see Packet::setCacheResponding
1565  // for more details
1566  if (!pkt->hasSharers()) {
1567  // we could get a writable line from memory (rather than a
1568  // cache) even in a read-only cache, note that we set this bit
1569  // even for a read-only cache, possibly revisit this decision
1571 
1572  // check if we got this via cache-to-cache transfer (i.e., from a
1573  // cache that had the block in Modified or Owned state)
1574  if (pkt->cacheResponding()) {
1575  // we got the block in Modified state, and invalidated the
1576  // owners copy
1578 
1579  gem5_assert(!isReadOnly, "Should never see dirty snoop response "
1580  "in read-only cache %s\n", name());
1581 
1582  }
1583  }
1584 
1585  DPRINTF(Cache, "Block addr %#llx (%s) moving from %s to %s\n",
1586  addr, is_secure ? "s" : "ns", old_state, blk->print());
1587 
1588  // if we got new data, copy it in (checking for a read response
1589  // and a response that has data is the same in the end)
1590  if (pkt->isRead()) {
1591  // sanity checks
1592  assert(pkt->hasData());
1593  assert(pkt->getSize() == blkSize);
1594 
1595  updateBlockData(blk, pkt, has_old_data);
1596  }
1597  // The block will be ready when the payload arrives and the fill is done
1599  pkt->payloadDelay);
1600 
1601  return blk;
1602 }
1603 
1604 CacheBlk*
1606 {
1607  // Get address
1608  const Addr addr = pkt->getAddr();
1609 
1610  // Get secure bit
1611  const bool is_secure = pkt->isSecure();
1612 
1613  // Block size and compression related access latency. Only relevant if
1614  // using a compressor, otherwise there is no extra delay, and the block
1615  // is fully sized
1616  std::size_t blk_size_bits = blkSize*8;
1617  Cycles compression_lat = Cycles(0);
1618  Cycles decompression_lat = Cycles(0);
1619 
1620  // If a compressor is being used, it is called to compress data before
1621  // insertion. Although in Gem5 the data is stored uncompressed, even if a
1622  // compressor is used, the compression/decompression methods are called to
1623  // calculate the amount of extra cycles needed to read or write compressed
1624  // blocks.
1625  if (compressor && pkt->hasData()) {
1626  const auto comp_data = compressor->compress(
1627  pkt->getConstPtr<uint64_t>(), compression_lat, decompression_lat);
1628  blk_size_bits = comp_data->getSizeBits();
1629  }
1630 
1631  // Find replacement victim
1632  std::vector<CacheBlk*> evict_blks;
1633  CacheBlk *victim = tags->findVictim(addr, is_secure, blk_size_bits,
1634  evict_blks);
1635 
1636  // It is valid to return nullptr if there is no victim
1637  if (!victim)
1638  return nullptr;
1639 
1640  // Print victim block's information
1641  DPRINTF(CacheRepl, "Replacement victim: %s\n", victim->print());
1642 
1643  // Try to evict blocks; if it fails, give up on allocation
1644  if (!handleEvictions(evict_blks, writebacks)) {
1645  return nullptr;
1646  }
1647 
1648  // Insert new block at victimized entry
1649  tags->insertBlock(pkt, victim);
1650 
1651  // If using a compressor, set compression data. This must be done after
1652  // insertion, as the compression bit may be set.
1653  if (compressor) {
1654  compressor->setSizeBits(victim, blk_size_bits);
1655  compressor->setDecompressionLatency(victim, decompression_lat);
1656  }
1657 
1658  return victim;
1659 }
1660 
1661 void
1663 {
1664  // If block is still marked as prefetched, then it hasn't been used
1665  if (blk->wasPrefetched()) {
1667  }
1668 
1669  // Notify that the data contents for this address are no longer present
1670  updateBlockData(blk, nullptr, blk->isValid());
1671 
1672  // If handling a block present in the Tags, let it do its invalidation
1673  // process, which will update stats and invalidate the block itself
1674  if (blk != tempBlock) {
1675  tags->invalidate(blk);
1676  } else {
1677  tempBlock->invalidate();
1678  }
1679 }
1680 
1681 void
1683 {
1684  PacketPtr pkt = evictBlock(blk);
1685  if (pkt) {
1686  writebacks.push_back(pkt);
1687  }
1688 }
1689 
1690 PacketPtr
1692 {
1694  "Writeback from read-only cache");
1695  assert(blk && blk->isValid() &&
1697 
1699 
1700  RequestPtr req = std::make_shared<Request>(
1702 
1703  if (blk->isSecure())
1704  req->setFlags(Request::SECURE);
1705 
1706  req->taskId(blk->getTaskId());
1707 
1708  PacketPtr pkt =
1709  new Packet(req, blk->isSet(CacheBlk::DirtyBit) ?
1711 
1712  DPRINTF(Cache, "Create Writeback %s writable: %d, dirty: %d\n",
1713  pkt->print(), blk->isSet(CacheBlk::WritableBit),
1714  blk->isSet(CacheBlk::DirtyBit));
1715 
1716  if (blk->isSet(CacheBlk::WritableBit)) {
1717  // not asserting shared means we pass the block in modified
1718  // state, mark our own block non-writeable
1720  } else {
1721  // we are in the Owned state, tell the receiver
1722  pkt->setHasSharers();
1723  }
1724 
1725  // make sure the block is not marked dirty
1727 
1728  pkt->allocate();
1729  pkt->setDataFromBlock(blk->data, blkSize);
1730 
1731  // When a block is compressed, it must first be decompressed before being
1732  // sent for writeback.
1733  if (compressor) {
1735  }
1736 
1737  return pkt;
1738 }
1739 
1740 PacketPtr
1742 {
1743  RequestPtr req = std::make_shared<Request>(
1745 
1746  if (blk->isSecure()) {
1747  req->setFlags(Request::SECURE);
1748  }
1749  req->taskId(blk->getTaskId());
1750 
1751  PacketPtr pkt = new Packet(req, MemCmd::WriteClean, blkSize, id);
1752 
1753  if (dest) {
1754  req->setFlags(dest);
1755  pkt->setWriteThrough();
1756  }
1757 
1758  DPRINTF(Cache, "Create %s writable: %d, dirty: %d\n", pkt->print(),
1760 
1761  if (blk->isSet(CacheBlk::WritableBit)) {
1762  // not asserting shared means we pass the block in modified
1763  // state, mark our own block non-writeable
1765  } else {
1766  // we are in the Owned state, tell the receiver
1767  pkt->setHasSharers();
1768  }
1769 
1770  // make sure the block is not marked dirty
1772 
1773  pkt->allocate();
1774  pkt->setDataFromBlock(blk->data, blkSize);
1775 
1776  // When a block is compressed, it must first be decompressed before being
1777  // sent for writeback.
1778  if (compressor) {
1780  }
1781 
1782  return pkt;
1783 }
1784 
1785 
1786 void
1788 {
1789  tags->forEachBlk([this](CacheBlk &blk) { writebackVisitor(blk); });
1790 }
1791 
1792 void
1794 {
1795  tags->forEachBlk([this](CacheBlk &blk) { invalidateVisitor(blk); });
1796 }
1797 
1798 bool
1800 {
1801  return tags->anyBlk([](CacheBlk &blk) {
1802  return blk.isSet(CacheBlk::DirtyBit); });
1803 }
1804 
1805 bool
1807 {
1808  return writeAllocator && writeAllocator->coalesce();
1809 }
1810 
1811 void
1813 {
1814  if (blk.isSet(CacheBlk::DirtyBit)) {
1815  assert(blk.isValid());
1816 
1817  RequestPtr request = std::make_shared<Request>(
1819 
1820  request->taskId(blk.getTaskId());
1821  if (blk.isSecure()) {
1822  request->setFlags(Request::SECURE);
1823  }
1824 
1825  Packet packet(request, MemCmd::WriteReq);
1826  packet.dataStatic(blk.data);
1827 
1828  memSidePort.sendFunctional(&packet);
1829 
1831  }
1832 }
1833 
1834 void
1836 {
1837  if (blk.isSet(CacheBlk::DirtyBit))
1838  warn_once("Invalidating dirty cache lines. " \
1839  "Expect things to break.\n");
1840 
1841  if (blk.isValid()) {
1842  assert(!blk.isSet(CacheBlk::DirtyBit));
1843  invalidateBlock(&blk);
1844  }
1845 }
1846 
1847 Tick
1849 {
1850  Tick nextReady = std::min(mshrQueue.nextReadyTime(),
1852 
1853  // Don't signal prefetch ready time if no MSHRs available
1854  // Will signal once enoguh MSHRs are deallocated
1855  if (prefetcher && mshrQueue.canPrefetch() && !isBlocked()) {
1856  nextReady = std::min(nextReady,
1858  }
1859 
1860  return nextReady;
1861 }
1862 
1863 
1864 bool
1866 {
1867  assert(mshr);
1868 
1869  // use request from 1st target
1870  PacketPtr tgt_pkt = mshr->getTarget()->pkt;
1871 
1872  DPRINTF(Cache, "%s: MSHR %s\n", __func__, tgt_pkt->print());
1873 
1874  // if the cache is in write coalescing mode or (additionally) in
1875  // no allocation mode, and we have a write packet with an MSHR
1876  // that is not a whole-line write (due to incompatible flags etc),
1877  // then reset the write mode
1878  if (writeAllocator && writeAllocator->coalesce() && tgt_pkt->isWrite()) {
1879  if (!mshr->isWholeLineWrite()) {
1880  // if we are currently write coalescing, hold on the
1881  // MSHR as many cycles extra as we need to completely
1882  // write a cache line
1883  if (writeAllocator->delay(mshr->blkAddr)) {
1884  Tick delay = blkSize / tgt_pkt->getSize() * clockPeriod();
1885  DPRINTF(CacheVerbose, "Delaying pkt %s %llu ticks to allow "
1886  "for write coalescing\n", tgt_pkt->print(), delay);
1887  mshrQueue.delay(mshr, delay);
1888  return false;
1889  } else {
1890  writeAllocator->reset();
1891  }
1892  } else {
1894  }
1895  }
1896 
1897  CacheBlk *blk = tags->findBlock(mshr->blkAddr, mshr->isSecure);
1898 
1899  // either a prefetch that is not present upstream, or a normal
1900  // MSHR request, proceed to get the packet to send downstream
1901  PacketPtr pkt = createMissPacket(tgt_pkt, blk, mshr->needsWritable(),
1902  mshr->isWholeLineWrite());
1903 
1904  mshr->isForward = (pkt == nullptr);
1905 
1906  if (mshr->isForward) {
1907  // not a cache block request, but a response is expected
1908  // make copy of current packet to forward, keep current
1909  // copy for response handling
1910  pkt = new Packet(tgt_pkt, false, true);
1911  assert(!pkt->isWrite());
1912  }
1913 
1914  // play it safe and append (rather than set) the sender state,
1915  // as forwarded packets may already have existing state
1916  pkt->pushSenderState(mshr);
1917 
1918  if (pkt->isClean() && blk && blk->isSet(CacheBlk::DirtyBit)) {
1919  // A cache clean opearation is looking for a dirty block. Mark
1920  // the packet so that the destination xbar can determine that
1921  // there will be a follow-up write packet as well.
1922  pkt->setSatisfied();
1923  }
1924 
1925  if (!memSidePort.sendTimingReq(pkt)) {
1926  // we are awaiting a retry, but we
1927  // delete the packet and will be creating a new packet
1928  // when we get the opportunity
1929  delete pkt;
1930 
1931  // note that we have now masked any requestBus and
1932  // schedSendEvent (we will wait for a retry before
1933  // doing anything), and this is so even if we do not
1934  // care about this packet and might override it before
1935  // it gets retried
1936  return true;
1937  } else {
1938  // As part of the call to sendTimingReq the packet is
1939  // forwarded to all neighbouring caches (and any caches
1940  // above them) as a snoop. Thus at this point we know if
1941  // any of the neighbouring caches are responding, and if
1942  // so, we know it is dirty, and we can determine if it is
1943  // being passed as Modified, making our MSHR the ordering
1944  // point
1945  bool pending_modified_resp = !pkt->hasSharers() &&
1946  pkt->cacheResponding();
1947  markInService(mshr, pending_modified_resp);
1948 
1949  if (pkt->isClean() && blk && blk->isSet(CacheBlk::DirtyBit)) {
1950  // A cache clean opearation is looking for a dirty
1951  // block. If a dirty block is encountered a WriteClean
1952  // will update any copies to the path to the memory
1953  // until the point of reference.
1954  DPRINTF(CacheVerbose, "%s: packet %s found block: %s\n",
1955  __func__, pkt->print(), blk->print());
1956  PacketPtr wb_pkt = writecleanBlk(blk, pkt->req->getDest(),
1957  pkt->id);
1958  PacketList writebacks;
1959  writebacks.push_back(wb_pkt);
1960  doWritebacks(writebacks, 0);
1961  }
1962 
1963  return false;
1964  }
1965 }
1966 
1967 bool
1969 {
1970  assert(wq_entry);
1971 
1972  // always a single target for write queue entries
1973  PacketPtr tgt_pkt = wq_entry->getTarget()->pkt;
1974 
1975  DPRINTF(Cache, "%s: write %s\n", __func__, tgt_pkt->print());
1976 
1977  // forward as is, both for evictions and uncacheable writes
1978  if (!memSidePort.sendTimingReq(tgt_pkt)) {
1979  // note that we have now masked any requestBus and
1980  // schedSendEvent (we will wait for a retry before
1981  // doing anything), and this is so even if we do not
1982  // care about this packet and might override it before
1983  // it gets retried
1984  return true;
1985  } else {
1986  markInService(wq_entry);
1987  return false;
1988  }
1989 }
1990 
1991 void
1993 {
1994  bool dirty(isDirty());
1995 
1996  if (dirty) {
1997  warn("*** The cache still contains dirty data. ***\n");
1998  warn(" Make sure to drain the system using the correct flags.\n");
1999  warn(" This checkpoint will not restore correctly " \
2000  "and dirty data in the cache will be lost!\n");
2001  }
2002 
2003  // Since we don't checkpoint the data in the cache, any dirty data
2004  // will be lost when restoring from a checkpoint of a system that
2005  // wasn't drained properly. Flag the checkpoint as invalid if the
2006  // cache contains dirty data.
2007  bool bad_checkpoint(dirty);
2008  SERIALIZE_SCALAR(bad_checkpoint);
2009 }
2010 
2011 void
2013 {
2014  bool bad_checkpoint;
2015  UNSERIALIZE_SCALAR(bad_checkpoint);
2016  if (bad_checkpoint) {
2017  fatal("Restoring from checkpoints with dirty caches is not "
2018  "supported in the classic memory system. Please remove any "
2019  "caches or drain them properly before taking checkpoints.\n");
2020  }
2021 }
2022 
2023 
2025  const std::string &name)
2026  : statistics::Group(&c, name.c_str()), cache(c),
2027  ADD_STAT(hits, statistics::units::Count::get(),
2028  ("number of " + name + " hits").c_str()),
2029  ADD_STAT(misses, statistics::units::Count::get(),
2030  ("number of " + name + " misses").c_str()),
2031  ADD_STAT(hitLatency, statistics::units::Tick::get(),
2032  ("number of " + name + " hit ticks").c_str()),
2033  ADD_STAT(missLatency, statistics::units::Tick::get(),
2034  ("number of " + name + " miss ticks").c_str()),
2035  ADD_STAT(accesses, statistics::units::Count::get(),
2036  ("number of " + name + " accesses(hits+misses)").c_str()),
2037  ADD_STAT(missRate, statistics::units::Ratio::get(),
2038  ("miss rate for " + name + " accesses").c_str()),
2039  ADD_STAT(avgMissLatency, statistics::units::Rate<
2040  statistics::units::Tick, statistics::units::Count>::get(),
2041  ("average " + name + " miss latency").c_str()),
2042  ADD_STAT(mshrHits, statistics::units::Count::get(),
2043  ("number of " + name + " MSHR hits").c_str()),
2044  ADD_STAT(mshrMisses, statistics::units::Count::get(),
2045  ("number of " + name + " MSHR misses").c_str()),
2046  ADD_STAT(mshrUncacheable, statistics::units::Count::get(),
2047  ("number of " + name + " MSHR uncacheable").c_str()),
2048  ADD_STAT(mshrMissLatency, statistics::units::Tick::get(),
2049  ("number of " + name + " MSHR miss ticks").c_str()),
2050  ADD_STAT(mshrUncacheableLatency, statistics::units::Tick::get(),
2051  ("number of " + name + " MSHR uncacheable ticks").c_str()),
2052  ADD_STAT(mshrMissRate, statistics::units::Ratio::get(),
2053  ("mshr miss rate for " + name + " accesses").c_str()),
2054  ADD_STAT(avgMshrMissLatency, statistics::units::Rate<
2055  statistics::units::Tick, statistics::units::Count>::get(),
2056  ("average " + name + " mshr miss latency").c_str()),
2057  ADD_STAT(avgMshrUncacheableLatency, statistics::units::Rate<
2058  statistics::units::Tick, statistics::units::Count>::get(),
2059  ("average " + name + " mshr uncacheable latency").c_str())
2060 {
2061 }
2062 
2063 void
2065 {
2066  using namespace statistics;
2067 
2069  System *system = cache.system;
2070  const auto max_requestors = system->maxRequestors();
2071 
2072  hits
2073  .init(max_requestors)
2074  .flags(total | nozero | nonan)
2075  ;
2076  for (int i = 0; i < max_requestors; i++) {
2077  hits.subname(i, system->getRequestorName(i));
2078  }
2079 
2080  // Miss statistics
2081  misses
2082  .init(max_requestors)
2083  .flags(total | nozero | nonan)
2084  ;
2085  for (int i = 0; i < max_requestors; i++) {
2086  misses.subname(i, system->getRequestorName(i));
2087  }
2088 
2089  // Hit latency statistics
2090  hitLatency
2091  .init(max_requestors)
2092  .flags(total | nozero | nonan)
2093  ;
2094  for (int i = 0; i < max_requestors; i++) {
2095  hitLatency.subname(i, system->getRequestorName(i));
2096  }
2097 
2098  // Miss latency statistics
2099  missLatency
2100  .init(max_requestors)
2101  .flags(total | nozero | nonan)
2102  ;
2103  for (int i = 0; i < max_requestors; i++) {
2104  missLatency.subname(i, system->getRequestorName(i));
2105  }
2106 
2107  // access formulas
2108  accesses.flags(total | nozero | nonan);
2109  accesses = hits + misses;
2110  for (int i = 0; i < max_requestors; i++) {
2111  accesses.subname(i, system->getRequestorName(i));
2112  }
2113 
2114  // miss rate formulas
2115  missRate.flags(total | nozero | nonan);
2116  missRate = misses / accesses;
2117  for (int i = 0; i < max_requestors; i++) {
2118  missRate.subname(i, system->getRequestorName(i));
2119  }
2120 
2121  // miss latency formulas
2122  avgMissLatency.flags(total | nozero | nonan);
2123  avgMissLatency = missLatency / misses;
2124  for (int i = 0; i < max_requestors; i++) {
2125  avgMissLatency.subname(i, system->getRequestorName(i));
2126  }
2127 
2128  // MSHR statistics
2129  // MSHR hit statistics
2130  mshrHits
2131  .init(max_requestors)
2132  .flags(total | nozero | nonan)
2133  ;
2134  for (int i = 0; i < max_requestors; i++) {
2135  mshrHits.subname(i, system->getRequestorName(i));
2136  }
2137 
2138  // MSHR miss statistics
2139  mshrMisses
2140  .init(max_requestors)
2141  .flags(total | nozero | nonan)
2142  ;
2143  for (int i = 0; i < max_requestors; i++) {
2144  mshrMisses.subname(i, system->getRequestorName(i));
2145  }
2146 
2147  // MSHR miss latency statistics
2148  mshrMissLatency
2149  .init(max_requestors)
2150  .flags(total | nozero | nonan)
2151  ;
2152  for (int i = 0; i < max_requestors; i++) {
2153  mshrMissLatency.subname(i, system->getRequestorName(i));
2154  }
2155 
2156  // MSHR uncacheable statistics
2157  mshrUncacheable
2158  .init(max_requestors)
2159  .flags(total | nozero | nonan)
2160  ;
2161  for (int i = 0; i < max_requestors; i++) {
2162  mshrUncacheable.subname(i, system->getRequestorName(i));
2163  }
2164 
2165  // MSHR miss latency statistics
2166  mshrUncacheableLatency
2167  .init(max_requestors)
2168  .flags(total | nozero | nonan)
2169  ;
2170  for (int i = 0; i < max_requestors; i++) {
2171  mshrUncacheableLatency.subname(i, system->getRequestorName(i));
2172  }
2173 
2174  // MSHR miss rate formulas
2175  mshrMissRate.flags(total | nozero | nonan);
2176  mshrMissRate = mshrMisses / accesses;
2177 
2178  for (int i = 0; i < max_requestors; i++) {
2179  mshrMissRate.subname(i, system->getRequestorName(i));
2180  }
2181 
2182  // mshrMiss latency formulas
2183  avgMshrMissLatency.flags(total | nozero | nonan);
2184  avgMshrMissLatency = mshrMissLatency / mshrMisses;
2185  for (int i = 0; i < max_requestors; i++) {
2186  avgMshrMissLatency.subname(i, system->getRequestorName(i));
2187  }
2188 
2189  // mshrUncacheable latency formulas
2190  avgMshrUncacheableLatency.flags(total | nozero | nonan);
2191  avgMshrUncacheableLatency = mshrUncacheableLatency / mshrUncacheable;
2192  for (int i = 0; i < max_requestors; i++) {
2193  avgMshrUncacheableLatency.subname(i, system->getRequestorName(i));
2194  }
2195 }
2196 
2198  : statistics::Group(&c), cache(c),
2199 
2200  ADD_STAT(demandHits, statistics::units::Count::get(),
2201  "number of demand (read+write) hits"),
2202  ADD_STAT(overallHits, statistics::units::Count::get(),
2203  "number of overall hits"),
2204  ADD_STAT(demandHitLatency, statistics::units::Tick::get(),
2205  "number of demand (read+write) hit ticks"),
2206  ADD_STAT(overallHitLatency, statistics::units::Tick::get(),
2207  "number of overall hit ticks"),
2208  ADD_STAT(demandMisses, statistics::units::Count::get(),
2209  "number of demand (read+write) misses"),
2210  ADD_STAT(overallMisses, statistics::units::Count::get(),
2211  "number of overall misses"),
2212  ADD_STAT(demandMissLatency, statistics::units::Tick::get(),
2213  "number of demand (read+write) miss ticks"),
2214  ADD_STAT(overallMissLatency, statistics::units::Tick::get(),
2215  "number of overall miss ticks"),
2216  ADD_STAT(demandAccesses, statistics::units::Count::get(),
2217  "number of demand (read+write) accesses"),
2218  ADD_STAT(overallAccesses, statistics::units::Count::get(),
2219  "number of overall (read+write) accesses"),
2220  ADD_STAT(demandMissRate, statistics::units::Ratio::get(),
2221  "miss rate for demand accesses"),
2222  ADD_STAT(overallMissRate, statistics::units::Ratio::get(),
2223  "miss rate for overall accesses"),
2224  ADD_STAT(demandAvgMissLatency, statistics::units::Rate<
2225  statistics::units::Tick, statistics::units::Count>::get(),
2226  "average overall miss latency in ticks"),
2227  ADD_STAT(overallAvgMissLatency, statistics::units::Rate<
2228  statistics::units::Tick, statistics::units::Count>::get(),
2229  "average overall miss latency"),
2230  ADD_STAT(blockedCycles, statistics::units::Cycle::get(),
2231  "number of cycles access was blocked"),
2232  ADD_STAT(blockedCauses, statistics::units::Count::get(),
2233  "number of times access was blocked"),
2234  ADD_STAT(avgBlocked, statistics::units::Rate<
2235  statistics::units::Cycle, statistics::units::Count>::get(),
2236  "average number of cycles each access was blocked"),
2237  ADD_STAT(writebacks, statistics::units::Count::get(),
2238  "number of writebacks"),
2239  ADD_STAT(demandMshrHits, statistics::units::Count::get(),
2240  "number of demand (read+write) MSHR hits"),
2241  ADD_STAT(overallMshrHits, statistics::units::Count::get(),
2242  "number of overall MSHR hits"),
2243  ADD_STAT(demandMshrMisses, statistics::units::Count::get(),
2244  "number of demand (read+write) MSHR misses"),
2245  ADD_STAT(overallMshrMisses, statistics::units::Count::get(),
2246  "number of overall MSHR misses"),
2247  ADD_STAT(overallMshrUncacheable, statistics::units::Count::get(),
2248  "number of overall MSHR uncacheable misses"),
2249  ADD_STAT(demandMshrMissLatency, statistics::units::Tick::get(),
2250  "number of demand (read+write) MSHR miss ticks"),
2251  ADD_STAT(overallMshrMissLatency, statistics::units::Tick::get(),
2252  "number of overall MSHR miss ticks"),
2253  ADD_STAT(overallMshrUncacheableLatency, statistics::units::Tick::get(),
2254  "number of overall MSHR uncacheable ticks"),
2255  ADD_STAT(demandMshrMissRate, statistics::units::Ratio::get(),
2256  "mshr miss ratio for demand accesses"),
2257  ADD_STAT(overallMshrMissRate, statistics::units::Ratio::get(),
2258  "mshr miss ratio for overall accesses"),
2259  ADD_STAT(demandAvgMshrMissLatency, statistics::units::Rate<
2260  statistics::units::Tick, statistics::units::Count>::get(),
2261  "average overall mshr miss latency"),
2262  ADD_STAT(overallAvgMshrMissLatency, statistics::units::Rate<
2263  statistics::units::Tick, statistics::units::Count>::get(),
2264  "average overall mshr miss latency"),
2265  ADD_STAT(overallAvgMshrUncacheableLatency, statistics::units::Rate<
2266  statistics::units::Tick, statistics::units::Count>::get(),
2267  "average overall mshr uncacheable latency"),
2268  ADD_STAT(replacements, statistics::units::Count::get(),
2269  "number of replacements"),
2270  ADD_STAT(dataExpansions, statistics::units::Count::get(),
2271  "number of data expansions"),
2272  ADD_STAT(dataContractions, statistics::units::Count::get(),
2273  "number of data contractions"),
2274  cmd(MemCmd::NUM_MEM_CMDS)
2275 {
2276  for (int idx = 0; idx < MemCmd::NUM_MEM_CMDS; ++idx)
2277  cmd[idx].reset(new CacheCmdStats(c, MemCmd(idx).toString()));
2278 }
2279 
2280 void
2282 {
2283  using namespace statistics;
2284 
2286 
2287  System *system = cache.system;
2288  const auto max_requestors = system->maxRequestors();
2289 
2290  for (auto &cs : cmd)
2291  cs->regStatsFromParent();
2292 
2293 // These macros make it easier to sum the right subset of commands and
2294 // to change the subset of commands that are considered "demand" vs
2295 // "non-demand"
2296 #define SUM_DEMAND(s) \
2297  (cmd[MemCmd::ReadReq]->s + cmd[MemCmd::WriteReq]->s + \
2298  cmd[MemCmd::WriteLineReq]->s + cmd[MemCmd::ReadExReq]->s + \
2299  cmd[MemCmd::ReadCleanReq]->s + cmd[MemCmd::ReadSharedReq]->s)
2300 
2301 // should writebacks be included here? prior code was inconsistent...
2302 #define SUM_NON_DEMAND(s) \
2303  (cmd[MemCmd::SoftPFReq]->s + cmd[MemCmd::HardPFReq]->s + \
2304  cmd[MemCmd::SoftPFExReq]->s)
2305 
2306  demandHits.flags(total | nozero | nonan);
2307  demandHits = SUM_DEMAND(hits);
2308  for (int i = 0; i < max_requestors; i++) {
2309  demandHits.subname(i, system->getRequestorName(i));
2310  }
2311 
2312  overallHits.flags(total | nozero | nonan);
2313  overallHits = demandHits + SUM_NON_DEMAND(hits);
2314  for (int i = 0; i < max_requestors; i++) {
2315  overallHits.subname(i, system->getRequestorName(i));
2316  }
2317 
2318  demandMisses.flags(total | nozero | nonan);
2319  demandMisses = SUM_DEMAND(misses);
2320  for (int i = 0; i < max_requestors; i++) {
2321  demandMisses.subname(i, system->getRequestorName(i));
2322  }
2323 
2324  overallMisses.flags(total | nozero | nonan);
2325  overallMisses = demandMisses + SUM_NON_DEMAND(misses);
2326  for (int i = 0; i < max_requestors; i++) {
2327  overallMisses.subname(i, system->getRequestorName(i));
2328  }
2329 
2330  demandMissLatency.flags(total | nozero | nonan);
2331  demandMissLatency = SUM_DEMAND(missLatency);
2332  for (int i = 0; i < max_requestors; i++) {
2333  demandMissLatency.subname(i, system->getRequestorName(i));
2334  }
2335 
2336  overallMissLatency.flags(total | nozero | nonan);
2337  overallMissLatency = demandMissLatency + SUM_NON_DEMAND(missLatency);
2338  for (int i = 0; i < max_requestors; i++) {
2339  overallMissLatency.subname(i, system->getRequestorName(i));
2340  }
2341 
2342  demandHitLatency.flags(total | nozero | nonan);
2343  demandHitLatency = SUM_DEMAND(hitLatency);
2344  for (int i = 0; i < max_requestors; i++) {
2345  demandHitLatency.subname(i, system->getRequestorName(i));
2346  }
2347  overallHitLatency.flags(total | nozero | nonan);
2348  overallHitLatency = demandHitLatency + SUM_NON_DEMAND(hitLatency);
2349  for (int i = 0; i < max_requestors; i++) {
2350  overallHitLatency.subname(i, system->getRequestorName(i));
2351  }
2352 
2353  demandAccesses.flags(total | nozero | nonan);
2354  demandAccesses = demandHits + demandMisses;
2355  for (int i = 0; i < max_requestors; i++) {
2356  demandAccesses.subname(i, system->getRequestorName(i));
2357  }
2358 
2359  overallAccesses.flags(total | nozero | nonan);
2360  overallAccesses = overallHits + overallMisses;
2361  for (int i = 0; i < max_requestors; i++) {
2362  overallAccesses.subname(i, system->getRequestorName(i));
2363  }
2364 
2365  demandMissRate.flags(total | nozero | nonan);
2366  demandMissRate = demandMisses / demandAccesses;
2367  for (int i = 0; i < max_requestors; i++) {
2368  demandMissRate.subname(i, system->getRequestorName(i));
2369  }
2370 
2371  overallMissRate.flags(total | nozero | nonan);
2372  overallMissRate = overallMisses / overallAccesses;
2373  for (int i = 0; i < max_requestors; i++) {
2374  overallMissRate.subname(i, system->getRequestorName(i));
2375  }
2376 
2377  demandAvgMissLatency.flags(total | nozero | nonan);
2378  demandAvgMissLatency = demandMissLatency / demandMisses;
2379  for (int i = 0; i < max_requestors; i++) {
2380  demandAvgMissLatency.subname(i, system->getRequestorName(i));
2381  }
2382 
2383  overallAvgMissLatency.flags(total | nozero | nonan);
2384  overallAvgMissLatency = overallMissLatency / overallMisses;
2385  for (int i = 0; i < max_requestors; i++) {
2386  overallAvgMissLatency.subname(i, system->getRequestorName(i));
2387  }
2388 
2389  blockedCycles.init(NUM_BLOCKED_CAUSES);
2390  blockedCycles
2391  .subname(Blocked_NoMSHRs, "no_mshrs")
2392  .subname(Blocked_NoTargets, "no_targets")
2393  ;
2394 
2395 
2396  blockedCauses.init(NUM_BLOCKED_CAUSES);
2397  blockedCauses
2398  .subname(Blocked_NoMSHRs, "no_mshrs")
2399  .subname(Blocked_NoTargets, "no_targets")
2400  ;
2401 
2402  avgBlocked
2403  .subname(Blocked_NoMSHRs, "no_mshrs")
2404  .subname(Blocked_NoTargets, "no_targets")
2405  ;
2406  avgBlocked = blockedCycles / blockedCauses;
2407 
2408  writebacks
2409  .init(max_requestors)
2410  .flags(total | nozero | nonan)
2411  ;
2412  for (int i = 0; i < max_requestors; i++) {
2413  writebacks.subname(i, system->getRequestorName(i));
2414  }
2415 
2416  demandMshrHits.flags(total | nozero | nonan);
2417  demandMshrHits = SUM_DEMAND(mshrHits);
2418  for (int i = 0; i < max_requestors; i++) {
2419  demandMshrHits.subname(i, system->getRequestorName(i));
2420  }
2421 
2422  overallMshrHits.flags(total | nozero | nonan);
2423  overallMshrHits = demandMshrHits + SUM_NON_DEMAND(mshrHits);
2424  for (int i = 0; i < max_requestors; i++) {
2425  overallMshrHits.subname(i, system->getRequestorName(i));
2426  }
2427 
2428  demandMshrMisses.flags(total | nozero | nonan);
2429  demandMshrMisses = SUM_DEMAND(mshrMisses);
2430  for (int i = 0; i < max_requestors; i++) {
2431  demandMshrMisses.subname(i, system->getRequestorName(i));
2432  }
2433 
2434  overallMshrMisses.flags(total | nozero | nonan);
2435  overallMshrMisses = demandMshrMisses + SUM_NON_DEMAND(mshrMisses);
2436  for (int i = 0; i < max_requestors; i++) {
2437  overallMshrMisses.subname(i, system->getRequestorName(i));
2438  }
2439 
2440  demandMshrMissLatency.flags(total | nozero | nonan);
2441  demandMshrMissLatency = SUM_DEMAND(mshrMissLatency);
2442  for (int i = 0; i < max_requestors; i++) {
2443  demandMshrMissLatency.subname(i, system->getRequestorName(i));
2444  }
2445 
2446  overallMshrMissLatency.flags(total | nozero | nonan);
2447  overallMshrMissLatency =
2448  demandMshrMissLatency + SUM_NON_DEMAND(mshrMissLatency);
2449  for (int i = 0; i < max_requestors; i++) {
2450  overallMshrMissLatency.subname(i, system->getRequestorName(i));
2451  }
2452 
2453  overallMshrUncacheable.flags(total | nozero | nonan);
2454  overallMshrUncacheable =
2455  SUM_DEMAND(mshrUncacheable) + SUM_NON_DEMAND(mshrUncacheable);
2456  for (int i = 0; i < max_requestors; i++) {
2457  overallMshrUncacheable.subname(i, system->getRequestorName(i));
2458  }
2459 
2460 
2461  overallMshrUncacheableLatency.flags(total | nozero | nonan);
2462  overallMshrUncacheableLatency =
2463  SUM_DEMAND(mshrUncacheableLatency) +
2464  SUM_NON_DEMAND(mshrUncacheableLatency);
2465  for (int i = 0; i < max_requestors; i++) {
2466  overallMshrUncacheableLatency.subname(i, system->getRequestorName(i));
2467  }
2468 
2469  demandMshrMissRate.flags(total | nozero | nonan);
2470  demandMshrMissRate = demandMshrMisses / demandAccesses;
2471  for (int i = 0; i < max_requestors; i++) {
2472  demandMshrMissRate.subname(i, system->getRequestorName(i));
2473  }
2474 
2475  overallMshrMissRate.flags(total | nozero | nonan);
2476  overallMshrMissRate = overallMshrMisses / overallAccesses;
2477  for (int i = 0; i < max_requestors; i++) {
2478  overallMshrMissRate.subname(i, system->getRequestorName(i));
2479  }
2480 
2481  demandAvgMshrMissLatency.flags(total | nozero | nonan);
2482  demandAvgMshrMissLatency = demandMshrMissLatency / demandMshrMisses;
2483  for (int i = 0; i < max_requestors; i++) {
2484  demandAvgMshrMissLatency.subname(i, system->getRequestorName(i));
2485  }
2486 
2487  overallAvgMshrMissLatency.flags(total | nozero | nonan);
2488  overallAvgMshrMissLatency = overallMshrMissLatency / overallMshrMisses;
2489  for (int i = 0; i < max_requestors; i++) {
2490  overallAvgMshrMissLatency.subname(i, system->getRequestorName(i));
2491  }
2492 
2493  overallAvgMshrUncacheableLatency.flags(total | nozero | nonan);
2494  overallAvgMshrUncacheableLatency =
2495  overallMshrUncacheableLatency / overallMshrUncacheable;
2496  for (int i = 0; i < max_requestors; i++) {
2497  overallAvgMshrUncacheableLatency.subname(i,
2499  }
2500 
2501  dataExpansions.flags(nozero | nonan);
2502  dataContractions.flags(nozero | nonan);
2503 }
2504 
2505 void
2507 {
2508  ppHit = new ProbePointArg<PacketPtr>(this->getProbeManager(), "Hit");
2509  ppMiss = new ProbePointArg<PacketPtr>(this->getProbeManager(), "Miss");
2510  ppFill = new ProbePointArg<PacketPtr>(this->getProbeManager(), "Fill");
2511  ppDataUpdate =
2512  new ProbePointArg<DataUpdate>(this->getProbeManager(), "Data Update");
2513 }
2514 
2516 //
2517 // CpuSidePort
2518 //
2520 bool
2522 {
2523  // Snoops shouldn't happen when bypassing caches
2524  assert(!cache->system->bypassCaches());
2525 
2526  assert(pkt->isResponse());
2527 
2528  // Express snoop responses from requestor to responder, e.g., from L1 to L2
2529  cache->recvTimingSnoopResp(pkt);
2530  return true;
2531 }
2532 
2533 
2534 bool
2536 {
2537  if (cache->system->bypassCaches() || pkt->isExpressSnoop()) {
2538  // always let express snoop packets through even if blocked
2539  return true;
2540  } else if (blocked || mustSendRetry) {
2541  // either already committed to send a retry, or blocked
2542  mustSendRetry = true;
2543  return false;
2544  }
2545  mustSendRetry = false;
2546  return true;
2547 }
2548 
2549 bool
2551 {
2552  assert(pkt->isRequest());
2553 
2554  if (cache->system->bypassCaches()) {
2555  // Just forward the packet if caches are disabled.
2556  // @todo This should really enqueue the packet rather
2557  [[maybe_unused]] bool success = cache->memSidePort.sendTimingReq(pkt);
2558  assert(success);
2559  return true;
2560  } else if (tryTiming(pkt)) {
2561  cache->recvTimingReq(pkt);
2562  return true;
2563  }
2564  return false;
2565 }
2566 
2567 Tick
2569 {
2570  if (cache->system->bypassCaches()) {
2571  // Forward the request if the system is in cache bypass mode.
2572  return cache->memSidePort.sendAtomic(pkt);
2573  } else {
2574  return cache->recvAtomic(pkt);
2575  }
2576 }
2577 
2578 void
2580 {
2581  if (cache->system->bypassCaches()) {
2582  // The cache should be flushed if we are in cache bypass mode,
2583  // so we don't need to check if we need to update anything.
2584  cache->memSidePort.sendFunctional(pkt);
2585  return;
2586  }
2587 
2588  // functional request
2589  cache->functionalAccess(pkt, true);
2590 }
2591 
2594 {
2595  return cache->getAddrRanges();
2596 }
2597 
2598 
2600 CpuSidePort::CpuSidePort(const std::string &_name, BaseCache *_cache,
2601  const std::string &_label)
2602  : CacheResponsePort(_name, _cache, _label), cache(_cache)
2603 {
2604 }
2605 
2607 //
2608 // MemSidePort
2609 //
2611 bool
2613 {
2614  cache->recvTimingResp(pkt);
2615  return true;
2616 }
2617 
2618 // Express snooping requests to memside port
2619 void
2621 {
2622  // Snoops shouldn't happen when bypassing caches
2623  assert(!cache->system->bypassCaches());
2624 
2625  // handle snooping requests
2626  cache->recvTimingSnoopReq(pkt);
2627 }
2628 
2629 Tick
2631 {
2632  // Snoops shouldn't happen when bypassing caches
2633  assert(!cache->system->bypassCaches());
2634 
2635  return cache->recvAtomicSnoop(pkt);
2636 }
2637 
2638 void
2640 {
2641  // Snoops shouldn't happen when bypassing caches
2642  assert(!cache->system->bypassCaches());
2643 
2644  // functional snoop (note that in contrast to atomic we don't have
2645  // a specific functionalSnoop method, as they have the same
2646  // behaviour regardless)
2647  cache->functionalAccess(pkt, false);
2648 }
2649 
2650 void
2652 {
2653  // sanity check
2654  assert(!waitingOnRetry);
2655 
2656  // there should never be any deferred request packets in the
2657  // queue, instead we resly on the cache to provide the packets
2658  // from the MSHR queue or write queue
2659  assert(deferredPacketReadyTime() == MaxTick);
2660 
2661  // check for request packets (requests & writebacks)
2662  QueueEntry* entry = cache.getNextQueueEntry();
2663 
2664  if (!entry) {
2665  // can happen if e.g. we attempt a writeback and fail, but
2666  // before the retry, the writeback is eliminated because
2667  // we snoop another cache's ReadEx.
2668  } else {
2669  // let our snoop responses go first if there are responses to
2670  // the same addresses
2671  if (checkConflictingSnoop(entry->getTarget()->pkt)) {
2672  return;
2673  }
2674  waitingOnRetry = entry->sendPacket(cache);
2675  }
2676 
2677  // if we succeeded and are not waiting for a retry, schedule the
2678  // next send considering when the next queue is ready, note that
2679  // snoop responses have their own packet queue and thus schedule
2680  // their own events
2681  if (!waitingOnRetry) {
2682  schedSendEvent(cache.nextQueueReadyTime());
2683  }
2684 }
2685 
2687  BaseCache *_cache,
2688  const std::string &_label)
2689  : CacheRequestPort(_name, _cache, _reqQueue, _snoopRespQueue),
2690  _reqQueue(*_cache, *this, _snoopRespQueue, _label),
2691  _snoopRespQueue(*_cache, *this, true, _label), cache(_cache)
2692 {
2693 }
2694 
2695 void
2696 WriteAllocator::updateMode(Addr write_addr, unsigned write_size,
2697  Addr blk_addr)
2698 {
2699  // check if we are continuing where the last write ended
2700  if (nextAddr == write_addr) {
2701  delayCtr[blk_addr] = delayThreshold;
2702  // stop if we have already saturated
2703  if (mode != WriteMode::NO_ALLOCATE) {
2704  byteCount += write_size;
2705  // switch to streaming mode if we have passed the lower
2706  // threshold
2707  if (mode == WriteMode::ALLOCATE &&
2708  byteCount > coalesceLimit) {
2709  mode = WriteMode::COALESCE;
2710  DPRINTF(Cache, "Switched to write coalescing\n");
2711  } else if (mode == WriteMode::COALESCE &&
2712  byteCount > noAllocateLimit) {
2713  // and continue and switch to non-allocating mode if we
2714  // pass the upper threshold
2715  mode = WriteMode::NO_ALLOCATE;
2716  DPRINTF(Cache, "Switched to write-no-allocate\n");
2717  }
2718  }
2719  } else {
2720  // we did not see a write matching the previous one, start
2721  // over again
2722  byteCount = write_size;
2723  mode = WriteMode::ALLOCATE;
2724  resetDelay(blk_addr);
2725  }
2726  nextAddr = write_addr + write_size;
2727 }
2728 
2729 } // namespace gem5
#define DPRINTF(x,...)
Definition: trace.hh:186
const char data[]
virtual void sendDeferredPacket()
Override the normal sendDeferredPacket and do not only consider the transmit list (used for responses...
Definition: base.cc:2651
A cache request port is used for the memory-side port of the cache, and in addition to the basic timi...
Definition: base.hh:153
A cache response port is used for the CPU-side port of the cache, and it is basically a simple timing...
Definition: base.hh:275
void clearBlocked()
Return to normal operation and accept new requests.
Definition: base.cc:160
EventFunctionWrapper sendRetryEvent
Definition: base.hh:303
CacheResponsePort(const std::string &_name, BaseCache *_cache, const std::string &_label)
Definition: base.cc:69
void setBlocked()
Do not accept any new requests.
Definition: base.cc:145
CpuSidePort(const std::string &_name, BaseCache *_cache, const std::string &_label)
Definition: base.cc:2600
virtual bool recvTimingSnoopResp(PacketPtr pkt) override
Receive a timing snoop response from the peer.
Definition: base.cc:2521
virtual Tick recvAtomic(PacketPtr pkt) override
Receive an atomic request packet from the peer.
Definition: base.cc:2568
virtual bool recvTimingReq(PacketPtr pkt) override
Receive a timing request from the peer.
Definition: base.cc:2550
virtual bool tryTiming(PacketPtr pkt) override
Availability request from the peer.
Definition: base.cc:2535
virtual void recvFunctional(PacketPtr pkt) override
Receive a functional request packet from the peer.
Definition: base.cc:2579
virtual AddrRangeList getAddrRanges() const override
Get a list of the non-overlapping address ranges the owner is responsible for.
Definition: base.cc:2593
virtual Tick recvAtomicSnoop(PacketPtr pkt)
Receive an atomic snoop request packet from our peer.
Definition: base.cc:2630
virtual void recvFunctionalSnoop(PacketPtr pkt)
Receive a functional snoop request packet from the peer.
Definition: base.cc:2639
virtual bool recvTimingResp(PacketPtr pkt)
Receive a timing response from the peer.
Definition: base.cc:2612
virtual void recvTimingSnoopReq(PacketPtr pkt)
Receive a timing snoop request from the peer.
Definition: base.cc:2620
MemSidePort(const std::string &_name, BaseCache *_cache, const std::string &_label)
Definition: base.cc:2686
A basic cache interface.
Definition: base.hh:96
virtual void serviceMSHRTargets(MSHR *mshr, const PacketPtr pkt, CacheBlk *blk)=0
Service non-deferred MSHR targets using the received response.
virtual void functionalAccess(PacketPtr pkt, bool from_cpu_side)
Performs the access specified by the request.
Definition: base.cc:713
const bool isReadOnly
Is this cache read only, for example the instruction cache, or table-walker cache.
Definition: base.hh:943
virtual void recvTimingReq(PacketPtr pkt)
Performs the access specified by the request.
Definition: base.cc:403
virtual void doWritebacks(PacketList &writebacks, Tick forward_time)=0
Insert writebacks into the write buffer.
const Cycles fillLatency
The latency to fill a cache block.
Definition: base.hh:910
const Cycles dataLatency
The latency of data access of a cache.
Definition: base.hh:900
void invalidateVisitor(CacheBlk &blk)
Cache block visitor that invalidates all blocks in the cache.
Definition: base.cc:1835
virtual Cycles handleAtomicReqMiss(PacketPtr pkt, CacheBlk *&blk, PacketList &writebacks)=0
Handle a request in atomic mode that missed in this cache.
ProbePointArg< DataUpdate > * ppDataUpdate
To probe when the contents of a block are updated.
Definition: base.hh:372
virtual void doWritebacksAtomic(PacketList &writebacks)=0
Send writebacks down the memory hierarchy in atomic mode.
void updateBlockData(CacheBlk *blk, const PacketPtr cpkt, bool has_old_data)
Update the data contents of a block.
Definition: base.cc:769
MSHR * allocateMissBuffer(PacketPtr pkt, Tick time, bool sched_send=true)
Definition: base.hh:1173
PacketPtr tempBlockWriteback
Writebacks from the tempBlock, resulting on the response path in atomic mode, must happen after the c...
Definition: base.hh:684
CpuSidePort cpuSidePort
Definition: base.hh:338
bool isDirty() const
Determine if there are any dirty blocks in the cache.
Definition: base.cc:1799
void invalidateBlock(CacheBlk *blk)
Invalidate a cache block.
Definition: base.cc:1662
MSHR * noTargetMSHR
Pointer to the MSHR that has no targets.
Definition: base.hh:974
const bool writebackClean
Determine if clean lines should be written back or not.
Definition: base.hh:675
bool sendWriteQueuePacket(WriteQueueEntry *wq_entry)
Similar to sendMSHR, but for a write-queue entry instead.
Definition: base.cc:1968
bool inRange(Addr addr) const
Determine if an address is in the ranges covered by this cache.
Definition: base.cc:213
virtual void handleTimingReqMiss(PacketPtr pkt, CacheBlk *blk, Tick forward_time, Tick request_time)=0
bool allocOnFill(MemCmd cmd) const
Determine whether we should allocate on a fill or not.
Definition: base.hh:441
bool forwardSnoops
Do we forward snoops from mem side port through to cpu side port?
Definition: base.hh:928
uint64_t order
Increasing order number assigned to each incoming request.
Definition: base.hh:968
void incHitCount(PacketPtr pkt)
Definition: base.hh:1307
virtual void satisfyRequest(PacketPtr pkt, CacheBlk *blk, bool deferred_response=false, bool pending_downgrade=false)
Perform any necessary updates to the block and perform any data exchange between the packet and the b...
Definition: base.cc:1087
virtual void memWriteback() override
Write back dirty blocks in the cache using functional accesses.
Definition: base.cc:1787
bool updateCompressionData(CacheBlk *&blk, const uint64_t *data, PacketList &writebacks)
When a block is overwriten, its compression information must be updated, and it may need to be recomp...
Definition: base.cc:976
bool isBlocked() const
Returns true if the cache is blocked for accesses.
Definition: base.hh:1226
gem5::BaseCache::CacheStats stats
ProbePointArg< PacketPtr > * ppFill
To probe when a cache fill occurs.
Definition: base.hh:365
const Cycles lookupLatency
The latency of tag lookup of a cache.
Definition: base.hh:894
Cycles calculateAccessLatency(const CacheBlk *blk, const uint32_t delay, const Cycles lookup_lat) const
Calculate access latency in ticks given a tag lookup latency, and whether access was a hit or miss.
Definition: base.cc:1194
Tick nextQueueReadyTime() const
Find next request ready time from among possible sources.
Definition: base.cc:1848
ProbePointArg< PacketPtr > * ppHit
To probe when a cache hit occurs.
Definition: base.hh:359
void regProbePoints() override
Registers probes.
Definition: base.cc:2506
virtual void memInvalidate() override
Invalidates all blocks in the cache.
Definition: base.cc:1793
MSHRQueue mshrQueue
Miss status registers.
Definition: base.hh:344
virtual PacketPtr createMissPacket(PacketPtr cpu_pkt, CacheBlk *blk, bool needs_writable, bool is_whole_line_write) const =0
Create an appropriate downstream bus request packet.
Port & getPort(const std::string &if_name, PortID idx=InvalidPortID) override
Get a port with a given name and index.
Definition: base.cc:201
QueueEntry * getNextQueueEntry()
Return the next queue entry to service, either a pending miss from the MSHR queue,...
Definition: base.cc:848
void handleUncacheableWriteResp(PacketPtr pkt)
Handling the special case of uncacheable write responses to make recvTimingResp less cluttered.
Definition: base.cc:475
const unsigned blkSize
Block size of this cache.
Definition: base.hh:888
void writebackTempBlockAtomic()
Send the outstanding tempBlock writeback.
Definition: base.hh:691
@ NUM_BLOCKED_CAUSES
Definition: base.hh:116
@ Blocked_NoTargets
Definition: base.hh:115
@ Blocked_NoMSHRs
Definition: base.hh:113
const Cycles forwardLatency
This is the forward latency of the cache.
Definition: base.hh:907
compression::Base * compressor
Compression method being used.
Definition: base.hh:353
const Cycles responseLatency
The latency of sending reponse to its upper level cache/core on a linefill.
Definition: base.hh:917
PacketPtr writecleanBlk(CacheBlk *blk, Request::Flags dest, PacketId id)
Create a writeclean request for the given block.
Definition: base.cc:1741
void schedMemSideSendEvent(Tick time)
Schedule a send event for the memory-side port.
Definition: base.hh:1274
ProbePointArg< PacketPtr > * ppMiss
To probe when a cache miss occurs.
Definition: base.hh:362
MemSidePort memSidePort
Definition: base.hh:339
virtual void handleTimingReqHit(PacketPtr pkt, CacheBlk *blk, Tick request_time)
Definition: base.cc:224
virtual Tick recvAtomic(PacketPtr pkt)
Performs the access specified by the request.
Definition: base.cc:633
void unserialize(CheckpointIn &cp) override
Unserialize an object.
Definition: base.cc:2012
void cmpAndSwap(CacheBlk *blk, PacketPtr pkt)
Handle doing the Compare and Swap function for SPARC.
Definition: base.cc:795
virtual void recvTimingResp(PacketPtr pkt)
Handles a response (cache line fill/write ack) from the bus.
Definition: base.cc:487
virtual bool access(PacketPtr pkt, CacheBlk *&blk, Cycles &lat, PacketList &writebacks)
Does all the processing necessary to perform the provided request.
Definition: base.cc:1228
void setBlocked(BlockedCause cause)
Marks the access path of the cache as blocked for the given cause.
Definition: base.hh:1236
BaseCache(const BaseCacheParams &p, unsigned blk_size)
Definition: base.cc:79
Addr regenerateBlkAddr(CacheBlk *blk)
Regenerate block address using tags.
Definition: base.cc:182
std::unique_ptr< Packet > pendingDelete
Upstream caches need this packet until true is returned, so hold it for deletion until a subsequent c...
Definition: base.hh:402
CacheBlk * allocateBlock(const PacketPtr pkt, PacketList &writebacks)
Allocate a new block and perform any necessary writebacks.
Definition: base.cc:1605
prefetch::Base * prefetcher
Prefetcher.
Definition: base.hh:356
uint8_t blocked
Bit vector of the blocking reasons for the access path.
Definition: base.hh:965
TempCacheBlk * tempBlock
Temporary cache block for occasional transitory use.
Definition: base.hh:396
const AddrRangeList addrRanges
The address range to which the cache responds on the CPU side.
Definition: base.hh:982
const int numTarget
The number of targets for each MSHR.
Definition: base.hh:925
const bool moveContractions
Similar to data expansions, after a block improves its compression, it may need to be moved elsewhere...
Definition: base.hh:959
WriteAllocator *const writeAllocator
The writeAllocator drive optimizations for streaming writes.
Definition: base.hh:388
void markInService(MSHR *mshr, bool pending_modified_resp)
Mark a request as in service (sent downstream in the memory system), effectively making this MSHR the...
Definition: base.hh:408
void allocateWriteBuffer(PacketPtr pkt, Tick time)
Definition: base.hh:1191
Cycles calculateTagOnlyLatency(const uint32_t delay, const Cycles lookup_lat) const
Calculate latency of accesses that only touch the tag array.
Definition: base.cc:1185
CacheBlk * handleFill(PacketPtr pkt, CacheBlk *blk, PacketList &writebacks, bool allocate)
Handle a fill operation caused by a received packet.
Definition: base.cc:1509
void incMissCount(PacketPtr pkt)
Definition: base.hh:1296
WriteQueue writeBuffer
Write/writeback buffer.
Definition: base.hh:347
const bool replaceExpansions
when a data expansion of a compressed block happens it will not be able to co-allocate where it is at...
Definition: base.hh:951
void serialize(CheckpointOut &cp) const override
Serialize the state of the caches.
Definition: base.cc:1992
bool coalesce() const
Checks if the cache is coalescing writes.
Definition: base.cc:1806
const bool sequentialAccess
Whether tags and data are accessed sequentially.
Definition: base.hh:922
bool handleEvictions(std::vector< CacheBlk * > &evict_blks, PacketList &writebacks)
Try to evict the given blocks.
Definition: base.cc:939
void init() override
init() is called after all C++ SimObjects have been created and all ports are connected.
Definition: base.cc:192
PacketPtr writebackBlk(CacheBlk *blk)
Create a writeback request for the given block.
Definition: base.cc:1691
void clearBlocked(BlockedCause cause)
Marks the cache as unblocked for the given cause.
Definition: base.hh:1255
virtual PacketPtr evictBlock(CacheBlk *blk)=0
Evict a cache block.
void writebackVisitor(CacheBlk &blk)
Cache block visitor that writes back dirty cache blocks using functional writes.
Definition: base.cc:1812
EventFunctionWrapper writebackTempBlockAtomicEvent
An event to writeback the tempBlock after recvAtomic finishes.
Definition: base.hh:703
BaseTags * tags
Tag and data Storage.
Definition: base.hh:350
const enums::Clusivity clusivity
Clusivity with respect to the upstream cache, determining if we fill into both this cache and the cac...
Definition: base.hh:935
virtual bool sendMSHRQueuePacket(MSHR *mshr)
Take an MSHR, turn it into a suitable downstream packet, and send it out.
Definition: base.cc:1865
void maintainClusivity(bool from_cache, CacheBlk *blk)
Maintain the clusivity of this cache by potentially invalidating a block.
Definition: base.cc:1497
System * system
System we are currently operating in.
Definition: base.hh:986
virtual CacheBlk * accessBlock(const PacketPtr pkt, Cycles &lat)=0
Access block and update replacement data.
virtual void insertBlock(const PacketPtr pkt, CacheBlk *blk)
Insert the new block into the cache and update stats.
Definition: base.cc:102
int extractBlkOffset(Addr addr) const
Calculate the block offset of an address.
Definition: base.hh:225
virtual CacheBlk * findBlock(Addr addr, bool is_secure) const
Finds the block in the cache without touching it.
Definition: base.cc:80
virtual CacheBlk * findVictim(Addr addr, const bool is_secure, const std::size_t size, std::vector< CacheBlk * > &evict_blks)=0
Find replacement victim based on address.
virtual Addr regenerateBlkAddr(const CacheBlk *blk) const =0
Regenerate the block address.
virtual void forEachBlk(std::function< void(CacheBlk &)> visitor)=0
Visit each block in the tags and apply a visitor.
virtual bool anyBlk(std::function< bool(CacheBlk &)> visitor)=0
Find if any of the blocks satisfies a condition.
virtual void invalidate(CacheBlk *blk)
This function updates the tags when a block is invalidated.
Definition: base.hh:254
virtual void moveBlock(CacheBlk *src_blk, CacheBlk *dest_blk)
Move a block's metadata to another location decided by the replacement policy.
Definition: base.cc:130
Simple class to provide virtual print() method on cache blocks without allocating a vtable pointer fo...
Definition: cache_blk.hh:555
A Basic Cache block.
Definition: cache_blk.hh:71
void setWhenReady(const Tick tick)
Set tick at which block's data will be available for access.
Definition: cache_blk.hh:278
@ ReadableBit
Read permission.
Definition: cache_blk.hh:85
@ WritableBit
write permission
Definition: cache_blk.hh:80
@ DirtyBit
dirty (modified)
Definition: cache_blk.hh:87
Tick getWhenReady() const
Get tick at which block's data will be available for access.
Definition: cache_blk.hh:265
void clearPrefetched()
Clear the prefetching bit.
Definition: cache_blk.hh:255
std::string print() const override
Pretty-print tag, set and way, and interpret state bits to readable form including mapping to a MOESI...
Definition: cache_blk.hh:364
bool isSet(unsigned bits) const
Checks the given coherence bits are set.
Definition: cache_blk.hh:239
bool checkWrite(PacketPtr pkt)
Handle interaction of load-locked operations and stores.
Definition: cache_blk.hh:412
void clearCoherenceBits(unsigned bits)
Clear the corresponding coherence bits.
Definition: cache_blk.hh:231
uint32_t getTaskId() const
Get the task id associated to this block.
Definition: cache_blk.hh:285
void trackLoadLocked(PacketPtr pkt)
Track the fact that a local locked was issued to the block.
Definition: cache_blk.hh:327
bool wasPrefetched() const
Check if this block was the result of a hardware prefetch, yet to be touched.
Definition: cache_blk.hh:249
uint8_t * data
Contains a copy of the data in this block for easy access.
Definition: cache_blk.hh:103
void setCoherenceBits(unsigned bits)
Sets the corresponding coherence bits.
Definition: cache_blk.hh:220
A coherent cache that can be arranged in flexible topologies.
Definition: cache.hh:68
The ClockedObject class extends the SimObject with a clock and accessor functions to relate ticks to ...
Tick clockEdge(Cycles cycles=Cycles(0)) const
Determine the tick when a cycle begins, by default the current one, but the argument also enables the...
Tick cyclesToTicks(Cycles c) const
Tick clockPeriod() const
Cycles ticksToCycles(Tick t) const
A CompressedTags cache tag store.
A superblock is composed of sub-blocks, and each sub-block has information regarding its superblock a...
Definition: super_blk.hh:52
std::size_t getSizeBits() const
Definition: super_blk.cc:93
void setSizeBits(const std::size_t size)
Set size, in bits, of this compressed block's data.
Definition: super_blk.cc:99
void setDecompressionLatency(const Cycles lat)
Set number of cycles needed to decompress this block.
Definition: super_blk.cc:135
OverwriteType
When an overwrite happens, the data size may change an not fit in its current container any longer.
Definition: super_blk.hh:75
@ DATA_EXPANSION
New data contents are considered larger than previous contents.
Definition: super_blk.hh:81
@ DATA_CONTRACTION
New data contents are considered smaller than previous contents.
Definition: super_blk.hh:77
OverwriteType checkExpansionContraction(const std::size_t size) const
Determines if changing the size of the block will cause a data expansion (new size is bigger) or cont...
Definition: super_blk.cc:148
Cycles is a wrapper class for representing cycle counts, i.e.
Definition: types.hh:79
void delay(MSHR *mshr, Tick delay_ticks)
Adds a delay to the provided MSHR and moves MSHRs that will be ready earlier than this entry to the t...
Definition: mshr_queue.cc:104
void deallocate(MSHR *mshr) override
Deallocate a MSHR and its targets.
Definition: mshr_queue.cc:83
bool canPrefetch() const
Returns true if sufficient mshrs for prefetch.
Definition: mshr_queue.hh:158
void markPending(MSHR *mshr)
Mark an in service entry as pending, used to resend a request.
Definition: mshr_queue.cc:123
void markInService(MSHR *mshr, bool pending_modified_resp)
Mark the given MSHR as in service.
Definition: mshr_queue.cc:115
Miss Status and handling Register.
Definition: mshr.hh:75
bool wasWholeLineWrite
Track if we sent this as a whole line write or not.
Definition: mshr.hh:124
bool isPendingModified() const
Definition: mshr.hh:326
void promoteReadable()
Promotes deferred targets that do not require writable.
Definition: mshr.cc:658
bool isCleaning() const
Definition: mshr.hh:321
int getNumTargets() const
Returns the current number of allocated targets.
Definition: mshr.hh:446
bool needsWritable() const
The pending* and post* flags are only valid if inService is true.
Definition: mshr.hh:319
bool isForward
True if the entry is just a simple forward from an upper level.
Definition: mshr.hh:127
bool hasLockedRMWReadTarget()
Determine if there are any LockedRMWReads in the Targets list.
Definition: mshr.cc:793
bool promoteDeferredTargets()
Definition: mshr.cc:596
bool isWholeLineWrite() const
Check if this MSHR contains only compatible writes, and if they span the entire cache line.
Definition: mshr.hh:406
void allocateTarget(PacketPtr target, Tick when, Counter order, bool alloc_on_fill)
Add a request to the list of targets.
Definition: mshr.cc:376
void promoteWritable()
Promotes deferred targets that do not require writable.
Definition: mshr.cc:679
QueueEntry::Target * getTarget() override
Returns a reference to the first target.
Definition: mshr.hh:473
bool allocOnFill() const
Definition: mshr.hh:340
@ WritebackDirty
Definition: packet.hh:92
@ InvalidateResp
Definition: packet.hh:144
@ WriteClean
Definition: packet.hh:94
@ LockedRMWWriteReq
Definition: packet.hh:117
@ WritebackClean
Definition: packet.hh:93
@ LockedRMWWriteResp
Definition: packet.hh:118
@ NUM_MEM_CMDS
Definition: packet.hh:151
@ CleanEvict
Definition: packet.hh:95
const std::string _name
Definition: named.hh:41
virtual std::string name() const
Definition: named.hh:47
A Packet is used to encapsulate a transfer between two objects in the memory system (e....
Definition: packet.hh:294
bool isUpgrade() const
Definition: packet.hh:595
bool isRead() const
Definition: packet.hh:592
bool isSecure() const
Definition: packet.hh:834
const PacketId id
Definition: packet.hh:373
void writeDataToBlock(uint8_t *blk_data, int blkSize) const
Copy data from the packet to the provided block pointer, which is aligned to the given block size.
Definition: packet.hh:1333
Addr getAddr() const
Definition: packet.hh:805
AtomicOpFunctor * getAtomicOp() const
Accessor function to atomic op.
Definition: packet.hh:843
void pushLabel(const std::string &lbl)
Push label for PrintReq (safe to call unconditionally).
Definition: packet.hh:1448
bool isError() const
Definition: packet.hh:621
bool isLockedRMW() const
Definition: packet.hh:620
void setWriteThrough()
A writeback/writeclean cmd gets propagated further downstream by the receiver when the flag is set.
Definition: packet.hh:733
bool isAtomicOp() const
Definition: packet.hh:844
bool isResponse() const
Definition: packet.hh:597
void makeTimingResponse()
Definition: packet.hh:1077
bool needsWritable() const
Definition: packet.hh:598
void print(std::ostream &o, int verbosity=0, const std::string &prefix="") const
Definition: packet.cc:368
bool isDemand() const
Definition: packet.hh:594
bool isCleanEviction() const
Is this packet a clean eviction, including both actual clean evict packets, but also clean writebacks...
Definition: packet.hh:1422
bool needsResponse() const
Definition: packet.hh:607
void dataStatic(T *p)
Set the data pointer to the following value that should not be freed.
Definition: packet.hh:1162
SenderState * senderState
This packet's sender state.
Definition: packet.hh:544
uint32_t payloadDelay
The extra pipelining delay from seeing the packet until the end of payload is transmitted by the comp...
Definition: packet.hh:448
void makeResponse()
Take a request packet and modify it in place to be suitable for returning as a response to that reque...
Definition: packet.hh:1059
uint32_t headerDelay
The extra delay from seeing the packet until the header is transmitted.
Definition: packet.hh:430
Addr getOffset(unsigned int blk_size) const
Definition: packet.hh:824
void clearBlockCached()
Definition: packet.hh:759
void pushSenderState(SenderState *sender_state)
Push a new sender state to the packet and make the current sender state the predecessor of the new on...
Definition: packet.cc:334
bool hasData() const
Definition: packet.hh:613
SenderState * popSenderState()
Pop the top of the state stack and return a pointer to it.
Definition: packet.cc:342
bool hasRespData() const
Definition: packet.hh:614
bool writeThrough() const
Definition: packet.hh:740
bool fromCache() const
Definition: packet.hh:611
void setData(const uint8_t *p)
Copy data into the packet from the provided pointer.
Definition: packet.hh:1280
bool isWrite() const
Definition: packet.hh:593
void setDataFromBlock(const uint8_t *blk_data, int blkSize)
Copy data into the packet from the provided block pointer, which is aligned to the given block size.
Definition: packet.hh:1299
bool trySatisfyFunctional(PacketPtr other)
Check a functional request against a memory value stored in another packet (i.e.
Definition: packet.hh:1386
Addr getBlockAddr(unsigned int blk_size) const
Definition: packet.hh:829
RequestPtr req
A pointer to the original request.
Definition: packet.hh:376
unsigned getSize() const
Definition: packet.hh:815
void setCacheResponding()
Snoop flags.
Definition: packet.hh:651
bool isClean() const
Definition: packet.hh:610
void popLabel()
Pop label for PrintReq (safe to call unconditionally).
Definition: packet.hh:1458
bool isExpressSnoop() const
Definition: packet.hh:700
bool isWriteback() const
Definition: packet.hh:612
void setHasSharers()
On fills, the hasSharers flag is used by the caches in combination with the cacheResponding flag,...
Definition: packet.hh:683
const T * getConstPtr() const
Definition: packet.hh:1221
bool isLLSC() const
Definition: packet.hh:619
bool cacheResponding() const
Definition: packet.hh:657
void makeAtomicResponse()
Definition: packet.hh:1071
void setSatisfied()
Set when a request hits in a cache and the cache is not going to respond.
Definition: packet.hh:747
MemCmd cmd
The command field of the packet.
Definition: packet.hh:371
bool isInvalidate() const
Definition: packet.hh:608
void writeData(uint8_t *p) const
Copy data from the packet to the memory at the provided pointer.
Definition: packet.hh:1309
bool hasSharers() const
Definition: packet.hh:684
void allocate()
Allocate memory for the packet.
Definition: packet.hh:1354
bool isEviction() const
Definition: packet.hh:609
bool isRequest() const
Definition: packet.hh:596
Ports are used to interface objects to each other.
Definition: port.hh:62
bool isConnected() const
Is this port currently connected to a peer?
Definition: port.hh:133
ProbePointArg generates a point for the class of Arg.
Definition: probe.hh:264
A queue entry is holding packets that will be serviced as soon as resources are available.
Definition: queue_entry.hh:88
const Tick recvTime
Time when request was received (for stats)
Definition: queue_entry.hh:90
PacketPtr pkt
Pending request packet.
Definition: queue_entry.hh:93
A queue entry base class, to be used by both the MSHRs and write-queue entries.
Definition: queue_entry.hh:63
virtual bool sendPacket(BaseCache &cache)=0
Send this queue entry as a downstream packet, with the exact behaviour depending on the specific entr...
Addr blkAddr
Block aligned address.
Definition: queue_entry.hh:116
Counter order
Order number assigned to disambiguate writes and misses.
Definition: queue_entry.hh:113
bool inService
True if the entry has been sent downstream.
Definition: queue_entry.hh:110
bool isSecure
True if the entry targets the secure memory space.
Definition: queue_entry.hh:122
virtual Target * getTarget()=0
Returns a pointer to the first target.
bool isFull() const
Definition: queue.hh:150
Entry * findPending(const QueueEntry *entry) const
Find any pending requests that overlap the given request of a different queue.
Definition: queue.hh:207
bool trySatisfyFunctional(PacketPtr pkt)
Definition: queue.hh:186
Tick nextReadyTime() const
Definition: queue.hh:229
Entry * getNext() const
Returns the WriteQueueEntry at the head of the readyList.
Definition: queue.hh:221
Entry * findMatch(Addr blk_addr, bool is_secure, bool ignore_uncacheable=true) const
Find the first entry that matches the provided address.
Definition: queue.hh:168
bool trySatisfyFunctional(PacketPtr pkt)
Check the list of buffered packets against the supplied functional request.
Definition: qport.hh:163
A queued port is a port that has an infinite queue for outgoing packets and thus decouples the module...
Definition: qport.hh:62
bool trySatisfyFunctional(PacketPtr pkt)
Check the list of buffered packets against the supplied functional request.
Definition: qport.hh:98
void schedTimingResp(PacketPtr pkt, Tick when)
Schedule the sending of a timing response.
Definition: qport.hh:93
bool sendTimingReq(PacketPtr pkt)
Attempt to send a timing request to the responder port by calling its corresponding receive function.
Definition: port.hh:495
void sendFunctional(PacketPtr pkt) const
Send a functional request packet, where the data is instantly updated everywhere in the memory system...
Definition: port.hh:485
@ SECURE
The request targets the secure memory space.
Definition: request.hh:186
@ funcRequestorId
This requestor id is used for functional requests that don't come from a particular device.
Definition: request.hh:279
@ wbRequestorId
This requestor id is used for writeback requests by the caches.
Definition: request.hh:274
SimObject & owner
Definition: port.hh:279
void sendFunctionalSnoop(PacketPtr pkt) const
Send a functional snoop request packet, where the data is instantly updated everywhere in the memory ...
Definition: port.hh:346
bool isSnooping() const
Find out if the peer request port is snooping or not.
Definition: port.hh:291
void sendRangeChange() const
Called by the owner to send a range change.
Definition: port.hh:296
std::vector< SectorSubBlk * > blks
List of blocks associated to this sector.
Definition: sector_blk.hh:147
SectorBlk * getSectorBlock() const
Get sector block associated to this block.
Definition: sector_blk.cc:52
A basic compression superblock.
Definition: super_blk.hh:171
std::string getRequestorName(RequestorID requestor_id)
Get the name of an object for a given request id.
Definition: system.cc:526
RequestorID maxRequestors()
Get the number of requestors registered in the system.
Definition: system.hh:498
virtual bool isValid() const
Checks if the entry is valid.
Definition: tagged_entry.hh:57
bool isSecure() const
Check if this block holds data from the secure memory space.
Definition: tagged_entry.hh:64
Special instance of CacheBlk for use with tempBlk that deals with its block address regeneration.
Definition: cache_blk.hh:501
void insert(const Addr addr, const bool is_secure) override
Insert the block by assigning it a tag and marking it valid.
Definition: cache_blk.hh:531
Addr getAddr() const
Get block's address.
Definition: cache_blk.hh:542
void invalidate() override
Invalidate the block and clear all state.
Definition: cache_blk.hh:524
bool coalesce() const
Should writes be coalesced? This is true if the mode is set to NO_ALLOCATE.
Definition: base.hh:1396
bool delay(Addr blk_addr)
Access whether we need to delay the current write.
Definition: base.hh:1427
void reset()
Reset the write allocator state, meaning that it allocates for writes and has not recorded any inform...
Definition: base.hh:1415
bool allocate() const
Should writes allocate?
Definition: base.hh:1405
void updateMode(Addr write_addr, unsigned write_size, Addr blk_addr)
Update the write mode based on the current write packet.
Definition: base.cc:2696
void resetDelay(Addr blk_addr)
Clear delay counter for the input block.
Definition: base.hh:1441
Write queue entry.
int getNumTargets() const
Returns the current number of allocated targets.
Target * getTarget() override
Returns a reference to the first target.
virtual std::unique_ptr< CompressionData > compress(const std::vector< Chunk > &chunks, Cycles &comp_lat, Cycles &decomp_lat)=0
Apply the compression process to the cache line.
Cycles getDecompressionLatency(const CacheBlk *blk)
Get the decompression latency if the block is compressed.
Definition: base.cc:197
static void setSizeBits(CacheBlk *blk, const std::size_t size_bits)
Set the size of the compressed block, in bits.
Definition: base.cc:225
static void setDecompressionLatency(CacheBlk *blk, const Cycles lat)
Set the decompression latency of compressed block.
Definition: base.cc:215
void incrDemandMhsrMisses()
Definition: base.hh:397
void pfHitInWB()
Definition: base.hh:415
void pfHitInCache()
Definition: base.hh:403
void pfHitInMSHR()
Definition: base.hh:409
virtual PacketPtr getPacket()=0
void prefetchUnused()
Definition: base.hh:391
virtual Tick nextPrefetchReadyTime() const =0
Statistics container.
Definition: group.hh:94
STL list class.
Definition: stl.hh:51
Declaration of a compressed set associative tag store using superblocks.
#define ADD_STAT(n,...)
Convenience macro to add a stat to a statistics group.
Definition: group.hh:75
static const Priority Delayed_Writeback_Pri
For some reason "delayed" inter-cluster writebacks are scheduled before regular writebacks (which hav...
Definition: eventq.hh:172
void deschedule(Event &event)
Definition: eventq.hh:1028
bool scheduled() const
Determine if the current event is scheduled.
Definition: eventq.hh:465
void schedule(Event &event, Tick when)
Definition: eventq.hh:1019
#define panic(...)
This implements a cprintf based panic() function.
Definition: logging.hh:178
#define fatal_if(cond,...)
Conditional fatal macro that checks the supplied condition and only causes a fatal error if the condi...
Definition: logging.hh:226
#define gem5_assert(cond,...)
The assert macro will function like a normal assert, but will use panic instead of straight abort().
Definition: logging.hh:318
#define fatal(...)
This implements a cprintf based fatal() function.
Definition: logging.hh:190
#define panic_if(cond,...)
Conditional panic macro that checks the supplied condition and only panics if the condition is true a...
Definition: logging.hh:204
ProbeManager * getProbeManager()
Get the probe manager for this object.
Definition: sim_object.cc:120
virtual Port & getPort(const std::string &if_name, PortID idx=InvalidPortID)
Get a port with a given name and index.
Definition: sim_object.cc:126
virtual void regStats()
Callback to set stat parameters.
Definition: group.cc:69
#define warn(...)
Definition: logging.hh:246
#define warn_once(...)
Definition: logging.hh:250
#define warn_if(cond,...)
Conditional warning macro that checks the supplied condition and only prints a warning if the conditi...
Definition: logging.hh:273
#define SUM_DEMAND(s)
#define SUM_NON_DEMAND(s)
Declares a basic cache interface BaseCache.
Definition of a basic cache compressor.
Miss and writeback queue declarations.
Miss Status and Handling Register (MSHR) declaration.
Bitfield< 4, 0 > mode
Definition: misc_types.hh:74
Bitfield< 7 > i
Definition: misc_types.hh:67
Bitfield< 23, 0 > offset
Definition: types.hh:144
Bitfield< 5 > r
Definition: pagetable.hh:60
Bitfield< 2 > c
Definition: pagetable.hh:63
Bitfield< 54 > p
Definition: pagetable.hh:70
Bitfield< 15 > system
Definition: misc.hh:1004
Bitfield< 3 > addr
Definition: types.hh:84
ProbePointArg< PacketInfo > Packet
Packet probe point.
Definition: mem.hh:109
const FlagsType nonan
Don't print if this is NAN.
Definition: info.hh:70
const FlagsType nozero
Don't print if this is zero.
Definition: info.hh:68
const FlagsType total
Print the total.
Definition: info.hh:60
Reference material can be found at the JEDEC website: UFS standard http://www.jedec....
std::shared_ptr< Request > RequestPtr
Definition: request.hh:92
Tick curTick()
The universal simulation clock.
Definition: cur_tick.hh:46
std::ostream CheckpointOut
Definition: serialize.hh:66
uint64_t Addr
Address type This will probably be moved somewhere else in the near future.
Definition: types.hh:147
int16_t PortID
Port index/ID type, and a symbolic name for an invalid port id.
Definition: types.hh:245
uint64_t Tick
Tick count type.
Definition: types.hh:58
uint64_t PacketId
Definition: packet.hh:73
const Tick MaxTick
Definition: types.hh:60
Generic queue entry.
#define UNSERIALIZE_SCALAR(scalar)
Definition: serialize.hh:575
#define SERIALIZE_SCALAR(scalar)
Definition: serialize.hh:568
void regStatsFromParent()
Callback to register stats from parent CacheStats::regStats().
Definition: base.cc:2064
statistics::Vector mshrUncacheableLatency
Total tick latency of each MSHR miss, per command and thread.
Definition: base.hh:1033
statistics::Vector mshrHits
Number of misses that hit in the MSHRs per command and thread.
Definition: base.hh:1025
CacheCmdStats(BaseCache &c, const std::string &name)
Definition: base.cc:2024
statistics::Vector mshrMisses
Number of misses that miss in the MSHRs, per command and thread.
Definition: base.hh:1027
statistics::Vector mshrMissLatency
Total tick latency of each MSHR miss, per command and thread.
Definition: base.hh:1031
statistics::Scalar replacements
Number of replacements of valid blocks.
Definition: base.hh:1134
statistics::Scalar dataContractions
Number of data contractions (blocks that had their compression factor improved).
Definition: base.hh:1143
CacheStats(BaseCache &c)
Definition: base.cc:2197
void regStats() override
Callback to set stat parameters.
Definition: base.cc:2281
statistics::Scalar dataExpansions
Number of data expansions.
Definition: base.hh:1137
std::vector< std::unique_ptr< CacheCmdStats > > cmd
Per-command statistics.
Definition: base.hh:1146
statistics::Vector writebacks
Number of blocks written back per thread.
Definition: base.hh:1097
CacheCmdStats & cmdStats(const PacketPtr p)
Definition: base.hh:1048
A data contents update is composed of the updated block's address, the old contents,...
Definition: base.hh:125
std::vector< uint64_t > newData
The new data contents.
Definition: base.hh:133
std::vector< uint64_t > oldData
The stale data contents.
Definition: base.hh:131
Copyright (c) 2018 Inria All rights reserved.
const std::string & name()
Definition: trace.cc:49

Generated on Wed Dec 21 2022 10:22:30 for gem5 by doxygen 1.9.1