Bus: Split the bus into a non-coherent and coherent bus

This patch introduces a class hierarchy of buses, a non-coherent one,
and a coherent one, splitting the existing bus functionality. By doing
so it also enables further specialisation of the two types of buses.

A non-coherent bus connects a number of non-snooping masters and
slaves, and routes the request and response packets based on the
address. The request packets issued by the master connected to a
non-coherent bus could still snoop in caches attached to a coherent
bus, as is the case with the I/O bus and memory bus in most system
configurations. No snoops will, however, reach any master on the
non-coherent bus itself. The non-coherent bus can be used as a
template for modelling PCI, PCIe, and non-coherent AMBA and OCP buses,
and is typically used for the I/O buses.

A coherent bus connects a number of (potentially) snooping masters and
slaves, and routes the request and response packets based on the
address, and also forwards all requests to the snoopers and deals with
the snoop responses. The coherent bus can be used as a template for
modelling QPI, HyperTransport, ACE and coherent OCP buses, and is
typically used for the L1-to-L2 buses and as the main system
interconnect.

The configuration scripts are updated to use a NoncoherentBus for all
peripheral and I/O buses.

A bit of minor tidying up has also been done.

--HG--
rename : src/mem/bus.cc => src/mem/coherent_bus.cc
rename : src/mem/bus.hh => src/mem/coherent_bus.hh
rename : src/mem/bus.cc => src/mem/noncoherent_bus.cc
rename : src/mem/bus.hh => src/mem/noncoherent_bus.hh
This commit is contained in:
Andreas Hansson 2012-05-31 13:30:04 -04:00
parent fb9bfb9cfc
commit 0d32940711
45 changed files with 1353 additions and 755 deletions

View file

@ -43,7 +43,7 @@ def config_cache(options, system):
system.l2 = L2Cache(size = options.l2_size, assoc = options.l2_assoc,
block_size=options.cacheline_size)
system.tol2bus = Bus()
system.tol2bus = CoherentBus()
system.l2.cpu_side = system.tol2bus.master
system.l2.mem_side = system.membus.slave

View file

@ -50,7 +50,7 @@ class CowIdeDisk(IdeDisk):
def childImage(self, ci):
self.image.child.image_file = ci
class MemBus(Bus):
class MemBus(CoherentBus):
badaddr_responder = BadAddr()
default = Self.badaddr_responder.pio
@ -67,8 +67,8 @@ def makeLinuxAlphaSystem(mem_mode, mdesc = None):
# generic system
mdesc = SysConfig()
self.readfile = mdesc.script()
self.iobus = Bus(bus_id=0)
self.membus = MemBus(bus_id=1)
self.iobus = NoncoherentBus()
self.membus = MemBus()
# By default the bridge responds to all addresses above the I/O
# base address (including the PCI config space)
self.bridge = Bridge(delay='50ns', nack_delay='4ns',
@ -117,7 +117,7 @@ def makeLinuxAlphaRubySystem(mem_mode, mdesc = None):
self.readfile = mdesc.script()
# Create pio bus to connect all device pio ports to rubymem's pio port
self.piobus = Bus(bus_id=0)
self.piobus = NoncoherentBus()
#
# Pio functional accesses from devices need direct access to memory
@ -172,8 +172,8 @@ def makeSparcSystem(mem_mode, mdesc = None):
# generic system
mdesc = SysConfig()
self.readfile = mdesc.script()
self.iobus = Bus(bus_id=0)
self.membus = MemBus(bus_id=1)
self.iobus = NoncoherentBus()
self.membus = MemBus()
self.bridge = Bridge(delay='50ns', nack_delay='4ns')
self.t1000 = T1000()
self.t1000.attachOnChipIO(self.membus)
@ -237,8 +237,8 @@ def makeArmSystem(mem_mode, machine_type, mdesc = None, bare_metal=False):
mdesc = SysConfig()
self.readfile = mdesc.script()
self.iobus = Bus(bus_id=0)
self.membus = MemBus(bus_id=1)
self.iobus = NoncoherentBus()
self.membus = MemBus()
self.membus.badaddr_responder.warn_access = "warn"
self.bridge = Bridge(delay='50ns', nack_delay='4ns')
self.bridge.master = self.iobus.slave
@ -320,8 +320,8 @@ def makeLinuxMipsSystem(mem_mode, mdesc = None):
# generic system
mdesc = SysConfig()
self.readfile = mdesc.script()
self.iobus = Bus(bus_id=0)
self.membus = MemBus(bus_id=1)
self.iobus = NoncoherentBus()
self.membus = MemBus()
self.bridge = Bridge(delay='50ns', nack_delay='4ns')
self.physmem = SimpleMemory(range = AddrRange('1GB'))
self.bridge.master = self.iobus.slave
@ -363,11 +363,11 @@ def connectX86ClassicSystem(x86_sys, numCPUs):
interrupts_address_space_base = 0xa000000000000000
APIC_range_size = 1 << 12;
x86_sys.membus = MemBus(bus_id=1)
x86_sys.membus = MemBus()
x86_sys.physmem.port = x86_sys.membus.master
# North Bridge
x86_sys.iobus = Bus(bus_id=0)
x86_sys.iobus = NoncoherentBus()
x86_sys.bridge = Bridge(delay='50ns', nack_delay='4ns')
x86_sys.bridge.master = x86_sys.iobus.slave
x86_sys.bridge.slave = x86_sys.membus.master
@ -402,7 +402,7 @@ def connectX86ClassicSystem(x86_sys, numCPUs):
def connectX86RubySystem(x86_sys):
# North Bridge
x86_sys.piobus = Bus(bus_id=0)
x86_sys.piobus = NoncoherentBus()
#
# Pio functional accesses from devices need direct access to memory

View file

@ -148,7 +148,7 @@ def make_level(spec, prototypes, attach_obj, attach_port):
parent = attach_obj # use attach obj as config parent too
if len(spec) > 1 and (fanout > 1 or options.force_bus):
port = getattr(attach_obj, attach_port)
new_bus = Bus(clock="500MHz", width=16)
new_bus = CoherentBus(clock="500MHz", width=16)
if (port.role == 'MASTER'):
new_bus.slave = port
attach_port = "master"

View file

@ -153,7 +153,7 @@ np = options.num_cpus
system = System(cpu = [CPUClass(cpu_id=i) for i in xrange(np)],
physmem = SimpleMemory(range=AddrRange("512MB")),
membus = Bus(), mem_mode = test_mem_mode)
membus = CoherentBus(), mem_mode = test_mem_mode)
# Sanity check
if options.fastmem and (options.caches or options.l2cache):

View file

@ -173,7 +173,7 @@ if options.timing:
for j in xrange(options.numclusters):
clusters[j].id = j
for cluster in clusters:
cluster.clusterbus = Bus(clock=busFrequency)
cluster.clusterbus = CoherentBus(clock=busFrequency)
all_l1buses += [cluster.clusterbus]
cluster.cpus = [TimingSimpleCPU(cpu_id = i + cluster.id,
clock=options.frequency)
@ -186,7 +186,7 @@ elif options.detailed:
for j in xrange(options.numclusters):
clusters[j].id = j
for cluster in clusters:
cluster.clusterbus = Bus(clock=busFrequency)
cluster.clusterbus = CoherentBus(clock=busFrequency)
all_l1buses += [cluster.clusterbus]
cluster.cpus = [DerivO3CPU(cpu_id = i + cluster.id,
clock=options.frequency)
@ -199,7 +199,7 @@ else:
for j in xrange(options.numclusters):
clusters[j].id = j
for cluster in clusters:
cluster.clusterbus = Bus(clock=busFrequency)
cluster.clusterbus = CoherentBus(clock=busFrequency)
all_l1buses += [cluster.clusterbus]
cluster.cpus = [AtomicSimpleCPU(cpu_id = i + cluster.id,
clock=options.frequency)
@ -212,9 +212,10 @@ else:
# Create a system, and add system wide objects
# ----------------------
system = System(cpu = all_cpus, l1_ = all_l1s, l1bus_ = all_l1buses,
physmem = SimpleMemory(), membus = Bus(clock = busFrequency))
physmem = SimpleMemory(),
membus = CoherentBus(clock = busFrequency))
system.toL2bus = Bus(clock = busFrequency)
system.toL2bus = CoherentBus(clock = busFrequency)
system.l2 = L2(size = options.l2size, assoc = 8)
# ----------------------

View file

@ -198,9 +198,9 @@ else:
# Create a system, and add system wide objects
# ----------------------
system = System(cpu = cpus, physmem = SimpleMemory(),
membus = Bus(clock = busFrequency))
membus = CoherentBus(clock = busFrequency))
system.toL2bus = Bus(clock = busFrequency)
system.toL2bus = CoherentBus(clock = busFrequency)
system.l2 = L2(size = options.l2size, assoc = 8)
# ----------------------

View file

@ -47,7 +47,7 @@ from m5.defines import buildEnv
from m5.params import *
from m5.proxy import *
from Bus import Bus
from Bus import CoherentBus
from InstTracer import InstTracer
from ExeTracer import ExeTracer
from MemObject import MemObject
@ -223,7 +223,7 @@ class BaseCPU(MemObject):
def addTwoLevelCacheHierarchy(self, ic, dc, l2c, iwc = None, dwc = None):
self.addPrivateSplitL1Caches(ic, dc, iwc, dwc)
self.toL2Bus = Bus()
self.toL2Bus = CoherentBus()
self.connectCachedPorts(self.toL2Bus)
self.l2cache = l2c
self.toL2Bus.master = self.l2cache.cpu_side

View file

@ -1,3 +1,15 @@
# Copyright (c) 2012 ARM Limited
# All rights reserved.
#
# The license below extends only to copyright in the software and shall
# not be construed as granting a license to any other intellectual
# property including but not limited to intellectual property relating
# to a hardware implementation of the functionality of the software
# licensed hereunder. You may use the software subject to the license
# terms below provided that you ensure that this notice is replicated
# unmodified and in its entirety in all distributions of the software,
# modified or unmodified, in source code or in binary form.
#
# Copyright (c) 2005-2008 The Regents of The University of Michigan
# All rights reserved.
#
@ -25,22 +37,36 @@
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
#
# Authors: Nathan Binkert
# Andreas Hansson
from m5.defines import buildEnv
from m5.params import *
from m5.proxy import *
from MemObject import MemObject
from m5.params import *
class Bus(MemObject):
type = 'Bus'
class BaseBus(MemObject):
type = 'BaseBus'
abstract = True
slave = VectorSlavePort("vector port for connecting masters")
master = VectorMasterPort("vector port for connecting slaves")
bus_id = Param.Int(0, "blah")
clock = Param.Clock("1GHz", "bus clock speed")
header_cycles = Param.Int(1, "cycles of overhead per transaction")
width = Param.Int(64, "bus width (bytes)")
block_size = Param.Int(64, "The default block size if one isn't set by a device attached to the bus.")
default = MasterPort("Default port for requests that aren't handled " \
"by a device.")
use_default_range = \
Param.Bool(False, "Query default port device for legal range.")
block_size = Param.Int(64, "The default block size if not set by " \
"any connected module")
# The default port can be left unconnected, or be used to connect
# a default slave port
default = MasterPort("Port for connecting an optional default slave")
# The default port can be used unconditionally, or based on
# address range, in which case it may overlap with other
# ports. The default range is always checked first, thus creating
# a two-level hierarchical lookup. This is useful e.g. for the PCI
# bus configuration.
use_default_range = Param.Bool(False, "Perform address mapping for " \
"the default port")
class NoncoherentBus(BaseBus):
type = 'NoncoherentBus'
class CoherentBus(BaseBus):
type = 'CoherentBus'

View file

@ -37,9 +37,11 @@ SimObject('MemObject.py')
Source('bridge.cc')
Source('bus.cc')
Source('coherent_bus.cc')
Source('comm_monitor.cc')
Source('mem_object.cc')
Source('mport.cc')
Source('noncoherent_bus.cc')
Source('packet.cc')
Source('port.cc')
Source('packet_queue.cc')
@ -56,8 +58,13 @@ if env['TARGET_ISA'] != 'no':
Source('page_table.cc')
Source('physical.cc')
DebugFlag('Bus')
DebugFlag('BaseBus')
DebugFlag('BusAddrRanges')
DebugFlag('CoherentBus')
DebugFlag('NoncoherentBus')
CompoundFlag('Bus', ['BaseBus', 'BusAddrRanges', 'CoherentBus',
'NoncoherentBus'])
DebugFlag('BusBridge')
DebugFlag('CommMonitor')
DebugFlag('LLSC')

View file

@ -53,7 +53,7 @@
#include "debug/BusAddrRanges.hh"
#include "mem/bus.hh"
Bus::Bus(const BusParams *p)
BaseBus::BaseBus(const BaseBusParams *p)
: MemObject(p), clock(p->clock),
headerCycles(p->header_cycles), width(p->width), tickNextIdle(0),
drainEvent(NULL), busIdleEvent(this), inRetry(false),
@ -69,37 +69,23 @@ Bus::Bus(const BusParams *p)
fatal("Bus clock period must be positive\n");
if (headerCycles <= 0)
fatal("Number of header cycles must be positive\n");
}
// create the ports based on the size of the master and slave
// vector ports, and the presence of the default port, the ports
// are enumerated starting from zero
for (int i = 0; i < p->port_master_connection_count; ++i) {
std::string portName = csprintf("%s-p%d", name(), i);
MasterPort* bp = new BusMasterPort(portName, this, i);
masterPorts.push_back(bp);
BaseBus::~BaseBus()
{
for (MasterPortIter m = masterPorts.begin(); m != masterPorts.end();
++m) {
delete *m;
}
// see if we have a default slave device connected and if so add
// our corresponding master port
if (p->port_default_connection_count) {
defaultPortID = masterPorts.size();
std::string portName = csprintf("%s-default", name());
MasterPort* bp = new BusMasterPort(portName, this, defaultPortID);
masterPorts.push_back(bp);
for (SlavePortIter s = slavePorts.begin(); s != slavePorts.end();
++s) {
delete *s;
}
// create the slave ports, once again starting at zero
for (int i = 0; i < p->port_slave_connection_count; ++i) {
std::string portName = csprintf("%s-p%d", name(), i);
SlavePort* bp = new BusSlavePort(portName, this, i);
slavePorts.push_back(bp);
}
clearPortCache();
}
MasterPort &
Bus::getMasterPort(const std::string &if_name, int idx)
BaseBus::getMasterPort(const std::string &if_name, int idx)
{
if (if_name == "master" && idx < masterPorts.size()) {
// the master port index translates directly to the vector position
@ -112,7 +98,7 @@ Bus::getMasterPort(const std::string &if_name, int idx)
}
SlavePort &
Bus::getSlavePort(const std::string &if_name, int idx)
BaseBus::getSlavePort(const std::string &if_name, int idx)
{
if (if_name == "slave" && idx < slavePorts.size()) {
// the slave port index translates directly to the vector position
@ -122,23 +108,8 @@ Bus::getSlavePort(const std::string &if_name, int idx)
}
}
void
Bus::init()
{
// iterate over our slave ports and determine which of our
// neighbouring master ports are snooping and add them as snoopers
for (SlavePortConstIter p = slavePorts.begin(); p != slavePorts.end();
++p) {
if ((*p)->getMasterPort().isSnooping()) {
DPRINTF(BusAddrRanges, "Adding snooping neighbour %s\n",
(*p)->getMasterPort().name());
snoopPorts.push_back(*p);
}
}
}
Tick
Bus::calcPacketTiming(PacketPtr pkt)
BaseBus::calcPacketTiming(PacketPtr pkt)
{
// determine the current time rounded to the closest following
// clock edge
@ -169,7 +140,7 @@ Bus::calcPacketTiming(PacketPtr pkt)
return headerTime;
}
void Bus::occupyBus(Tick until)
void BaseBus::occupyBus(Tick until)
{
if (until == 0) {
// shortcut for express snoop packets
@ -179,12 +150,12 @@ void Bus::occupyBus(Tick until)
tickNextIdle = until;
reschedule(busIdleEvent, tickNextIdle, true);
DPRINTF(Bus, "The bus is now occupied from tick %d to %d\n",
DPRINTF(BaseBus, "The bus is now occupied from tick %d to %d\n",
curTick(), tickNextIdle);
}
bool
Bus::isOccupied(Port* port)
BaseBus::isOccupied(Port* port)
{
// first we see if the next idle tick is in the future, next the
// bus is considered occupied if there are ports on the retry list
@ -197,220 +168,15 @@ Bus::isOccupied(Port* port)
return false;
}
bool
Bus::recvTimingReq(PacketPtr pkt, PortID slave_port_id)
{
// determine the source port based on the id
SlavePort *src_port = slavePorts[slave_port_id];
// test if the bus should be considered occupied for the current
// port, and exclude express snoops from the check
if (!pkt->isExpressSnoop() && isOccupied(src_port)) {
DPRINTF(Bus, "recvTimingReq: src %s %s 0x%x BUSY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
return false;
}
DPRINTF(Bus, "recvTimingReq: src %s %s 0x%x\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
// set the source port for routing of the response
pkt->setSrc(slave_port_id);
Tick headerFinishTime = pkt->isExpressSnoop() ? 0 : calcPacketTiming(pkt);
Tick packetFinishTime = pkt->isExpressSnoop() ? 0 : pkt->finishTime;
// uncacheable requests need never be snooped
if (!pkt->req->isUncacheable()) {
// the packet is a memory-mapped request and should be
// broadcasted to our snoopers but the source
forwardTiming(pkt, slave_port_id);
}
// remember if we add an outstanding req so we can undo it if
// necessary, if the packet needs a response, we should add it
// as outstanding and express snoops never fail so there is
// not need to worry about them
bool add_outstanding = !pkt->isExpressSnoop() && pkt->needsResponse();
// keep track that we have an outstanding request packet
// matching this request, this is used by the coherency
// mechanism in determining what to do with snoop responses
// (in recvTimingSnoop)
if (add_outstanding) {
// we should never have an exsiting request outstanding
assert(outstandingReq.find(pkt->req) == outstandingReq.end());
outstandingReq.insert(pkt->req);
}
// since it is a normal request, determine the destination
// based on the address and attempt to send the packet
bool success = masterPorts[findPort(pkt->getAddr())]->sendTimingReq(pkt);
if (!success) {
// inhibited packets should never be forced to retry
assert(!pkt->memInhibitAsserted());
// if it was added as outstanding and the send failed, then
// erase it again
if (add_outstanding)
outstandingReq.erase(pkt->req);
DPRINTF(Bus, "recvTimingReq: src %s %s 0x%x RETRY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
addToRetryList(src_port);
occupyBus(headerFinishTime);
return false;
}
succeededTiming(packetFinishTime);
return true;
}
bool
Bus::recvTimingResp(PacketPtr pkt, PortID master_port_id)
{
// determine the source port based on the id
MasterPort *src_port = masterPorts[master_port_id];
// test if the bus should be considered occupied for the current
// port
if (isOccupied(src_port)) {
DPRINTF(Bus, "recvTimingResp: src %s %s 0x%x BUSY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
return false;
}
DPRINTF(Bus, "recvTimingResp: src %s %s 0x%x\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
calcPacketTiming(pkt);
Tick packetFinishTime = pkt->finishTime;
// the packet is a normal response to a request that we should
// have seen passing through the bus
assert(outstandingReq.find(pkt->req) != outstandingReq.end());
// remove it as outstanding
outstandingReq.erase(pkt->req);
// send the packet to the destination through one of our slave
// ports, as determined by the destination field
bool success M5_VAR_USED = slavePorts[pkt->getDest()]->sendTimingResp(pkt);
// currently it is illegal to block responses... can lead to
// deadlock
assert(success);
succeededTiming(packetFinishTime);
return true;
}
void
Bus::recvTimingSnoopReq(PacketPtr pkt, PortID master_port_id)
{
DPRINTF(Bus, "recvTimingSnoopReq: src %s %s 0x%x\n",
masterPorts[master_port_id]->name(), pkt->cmdString(),
pkt->getAddr());
// we should only see express snoops from caches
assert(pkt->isExpressSnoop());
// set the source port for routing of the response
pkt->setSrc(master_port_id);
// forward to all snoopers
forwardTiming(pkt, InvalidPortID);
// a snoop request came from a connected slave device (one of
// our master ports), and if it is not coming from the slave
// device responsible for the address range something is
// wrong, hence there is nothing further to do as the packet
// would be going back to where it came from
assert(master_port_id == findPort(pkt->getAddr()));
// this is an express snoop and is never forced to retry
assert(!inRetry);
}
bool
Bus::recvTimingSnoopResp(PacketPtr pkt, PortID slave_port_id)
{
// determine the source port based on the id
SlavePort* src_port = slavePorts[slave_port_id];
// test if the bus should be considered occupied for the current
// port
if (isOccupied(src_port)) {
DPRINTF(Bus, "recvTimingSnoopResp: src %s %s 0x%x BUSY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
return false;
}
DPRINTF(Bus, "recvTimingSnoop: src %s %s 0x%x\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
// get the destination from the packet
PortID dest = pkt->getDest();
// responses are never express snoops
assert(!pkt->isExpressSnoop());
calcPacketTiming(pkt);
Tick packetFinishTime = pkt->finishTime;
// determine if the response is from a snoop request we
// created as the result of a normal request (in which case it
// should be in the outstandingReq), or if we merely forwarded
// someone else's snoop request
if (outstandingReq.find(pkt->req) == outstandingReq.end()) {
// this is a snoop response to a snoop request we
// forwarded, e.g. coming from the L1 and going to the L2
// this should be forwarded as a snoop response
bool success M5_VAR_USED = masterPorts[dest]->sendTimingSnoopResp(pkt);
assert(success);
} else {
// we got a snoop response on one of our slave ports,
// i.e. from a coherent master connected to the bus, and
// since we created the snoop request as part of
// recvTiming, this should now be a normal response again
outstandingReq.erase(pkt->req);
// this is a snoop response from a coherent master, with a
// destination field set on its way through the bus as
// request, hence it should never go back to where the
// snoop response came from, but instead to where the
// original request came from
assert(slave_port_id != dest);
// as a normal response, it should go back to a master
// through one of our slave ports
bool success M5_VAR_USED = slavePorts[dest]->sendTimingResp(pkt);
// currently it is illegal to block responses... can lead
// to deadlock
assert(success);
}
succeededTiming(packetFinishTime);
return true;
}
void
Bus::succeededTiming(Tick busy_time)
BaseBus::succeededTiming(Tick busy_time)
{
// occupy the bus accordingly
occupyBus(busy_time);
// if a retrying port succeeded, also take it off the retry list
if (inRetry) {
DPRINTF(Bus, "Remove retry from list %s\n",
DPRINTF(BaseBus, "Remove retry from list %s\n",
retryList.front()->name());
retryList.pop_front();
inRetry = false;
@ -418,24 +184,7 @@ Bus::succeededTiming(Tick busy_time)
}
void
Bus::forwardTiming(PacketPtr pkt, PortID exclude_slave_port_id)
{
for (SlavePortIter s = snoopPorts.begin(); s != snoopPorts.end(); ++s) {
SlavePort *p = *s;
// we could have gotten this request from a snooping master
// (corresponding to our own slave port that is also in
// snoopPorts) and should not send it back to where it came
// from
if (exclude_slave_port_id == InvalidPortID ||
p->getId() != exclude_slave_port_id) {
// cache is not allowed to refuse snoop
p->sendTimingSnoopReq(pkt);
}
}
}
void
Bus::releaseBus()
BaseBus::releaseBus()
{
// releasing the bus means we should now be idle
assert(curTick() >= tickNextIdle);
@ -458,7 +207,7 @@ Bus::releaseBus()
}
void
Bus::retryWaiting()
BaseBus::retryWaiting()
{
// this should never be called with an empty retry list
assert(!retryList.empty());
@ -489,7 +238,7 @@ Bus::retryWaiting()
}
void
Bus::recvRetry()
BaseBus::recvRetry()
{
// we got a retry from a peer that we tried to send something to
// and failed, but we sent it on the account of someone else, and
@ -509,7 +258,7 @@ Bus::recvRetry()
}
PortID
Bus::findPort(Addr addr)
BaseBus::findPort(Addr addr)
{
/* An interval tree would be a better way to do this. --ali. */
PortID dest_id = checkPortCache(addr);
@ -529,12 +278,13 @@ Bus::findPort(Addr addr)
AddrRangeIter a_end = defaultRange.end();
for (AddrRangeIter i = defaultRange.begin(); i != a_end; i++) {
if (*i == addr) {
DPRINTF(Bus, " found addr %#llx on default\n", addr);
DPRINTF(BusAddrRanges, " found addr %#llx on default\n",
addr);
return defaultPortID;
}
}
} else if (defaultPortID != InvalidPortID) {
DPRINTF(Bus, "Unable to find destination for addr %#llx, "
DPRINTF(BusAddrRanges, "Unable to find destination for addr %#llx, "
"will use default port\n", addr);
return defaultPortID;
}
@ -545,170 +295,9 @@ Bus::findPort(Addr addr)
name());
}
Tick
Bus::recvAtomic(PacketPtr pkt, PortID slave_port_id)
{
DPRINTF(Bus, "recvAtomic: packet src %s addr 0x%x cmd %s\n",
slavePorts[slave_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
MemCmd snoop_response_cmd = MemCmd::InvalidCmd;
Tick snoop_response_latency = 0;
// uncacheable requests need never be snooped
if (!pkt->req->isUncacheable()) {
// forward to all snoopers but the source
std::pair<MemCmd, Tick> snoop_result =
forwardAtomic(pkt, slave_port_id);
snoop_response_cmd = snoop_result.first;
snoop_response_latency = snoop_result.second;
}
// even if we had a snoop response, we must continue and also
// perform the actual request at the destination
PortID dest_id = findPort(pkt->getAddr());
// forward the request to the appropriate destination
Tick response_latency = masterPorts[dest_id]->sendAtomic(pkt);
// if we got a response from a snooper, restore it here
if (snoop_response_cmd != MemCmd::InvalidCmd) {
// no one else should have responded
assert(!pkt->isResponse());
pkt->cmd = snoop_response_cmd;
response_latency = snoop_response_latency;
}
pkt->finishTime = curTick() + response_latency;
return response_latency;
}
Tick
Bus::recvAtomicSnoop(PacketPtr pkt, PortID master_port_id)
{
DPRINTF(Bus, "recvAtomicSnoop: packet src %s addr 0x%x cmd %s\n",
masterPorts[master_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
// forward to all snoopers
std::pair<MemCmd, Tick> snoop_result =
forwardAtomic(pkt, InvalidPortID);
MemCmd snoop_response_cmd = snoop_result.first;
Tick snoop_response_latency = snoop_result.second;
if (snoop_response_cmd != MemCmd::InvalidCmd)
pkt->cmd = snoop_response_cmd;
pkt->finishTime = curTick() + snoop_response_latency;
return snoop_response_latency;
}
std::pair<MemCmd, Tick>
Bus::forwardAtomic(PacketPtr pkt, PortID exclude_slave_port_id)
{
// the packet may be changed on snoops, record the original
// command to enable us to restore it between snoops so that
// additional snoops can take place properly
MemCmd orig_cmd = pkt->cmd;
MemCmd snoop_response_cmd = MemCmd::InvalidCmd;
Tick snoop_response_latency = 0;
for (SlavePortIter s = snoopPorts.begin(); s != snoopPorts.end(); ++s) {
SlavePort *p = *s;
// we could have gotten this request from a snooping master
// (corresponding to our own slave port that is also in
// snoopPorts) and should not send it back to where it came
// from
if (exclude_slave_port_id == InvalidPortID ||
p->getId() != exclude_slave_port_id) {
Tick latency = p->sendAtomicSnoop(pkt);
// in contrast to a functional access, we have to keep on
// going as all snoopers must be updated even if we get a
// response
if (pkt->isResponse()) {
// response from snoop agent
assert(pkt->cmd != orig_cmd);
assert(pkt->memInhibitAsserted());
// should only happen once
assert(snoop_response_cmd == MemCmd::InvalidCmd);
// save response state
snoop_response_cmd = pkt->cmd;
snoop_response_latency = latency;
// restore original packet state for remaining snoopers
pkt->cmd = orig_cmd;
}
}
}
// the packet is restored as part of the loop and any potential
// snoop response is part of the returned pair
return std::make_pair(snoop_response_cmd, snoop_response_latency);
}
void
Bus::recvFunctional(PacketPtr pkt, PortID slave_port_id)
{
if (!pkt->isPrint()) {
// don't do DPRINTFs on PrintReq as it clutters up the output
DPRINTF(Bus,
"recvFunctional: packet src %s addr 0x%x cmd %s\n",
slavePorts[slave_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
}
// uncacheable requests need never be snooped
if (!pkt->req->isUncacheable()) {
// forward to all snoopers but the source
forwardFunctional(pkt, slave_port_id);
}
// there is no need to continue if the snooping has found what we
// were looking for and the packet is already a response
if (!pkt->isResponse()) {
PortID dest_id = findPort(pkt->getAddr());
masterPorts[dest_id]->sendFunctional(pkt);
}
}
void
Bus::recvFunctionalSnoop(PacketPtr pkt, PortID master_port_id)
{
if (!pkt->isPrint()) {
// don't do DPRINTFs on PrintReq as it clutters up the output
DPRINTF(Bus,
"recvFunctionalSnoop: packet src %s addr 0x%x cmd %s\n",
masterPorts[master_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
}
// forward to all snoopers
forwardFunctional(pkt, InvalidPortID);
}
void
Bus::forwardFunctional(PacketPtr pkt, PortID exclude_slave_port_id)
{
for (SlavePortIter s = snoopPorts.begin(); s != snoopPorts.end(); ++s) {
SlavePort *p = *s;
// we could have gotten this request from a snooping master
// (corresponding to our own slave port that is also in
// snoopPorts) and should not send it back to where it came
// from
if (exclude_slave_port_id == InvalidPortID ||
p->getId() != exclude_slave_port_id)
p->sendFunctionalSnoop(pkt);
// if we get a response we are done
if (pkt->isResponse()) {
break;
}
}
}
/** Function called by the port when the bus is receiving a range change.*/
void
Bus::recvRangeChange(PortID master_port_id)
BaseBus::recvRangeChange(PortID master_port_id)
{
AddrRangeList ranges;
AddrRangeIter iter;
@ -773,7 +362,7 @@ Bus::recvRangeChange(PortID master_port_id)
}
AddrRangeList
Bus::getAddrRanges()
BaseBus::getAddrRanges()
{
AddrRangeList ranges;
@ -813,29 +402,22 @@ Bus::getAddrRanges()
return ranges;
}
bool
Bus::isSnooping() const
{
// in essence, answer the question if there are snooping ports
return !snoopPorts.empty();
}
unsigned
Bus::findBlockSize()
BaseBus::findBlockSize()
{
if (cachedBlockSizeValid)
return cachedBlockSize;
unsigned max_bs = 0;
PortIter p_end = portMap.end();
for (PortIter p_iter = portMap.begin(); p_iter != p_end; p_iter++) {
unsigned tmp_bs = masterPorts[p_iter->second]->peerBlockSize();
for (MasterPortConstIter m = masterPorts.begin(); m != masterPorts.end();
++m) {
unsigned tmp_bs = (*m)->peerBlockSize();
if (tmp_bs > max_bs)
max_bs = tmp_bs;
}
for (SlavePortConstIter s = snoopPorts.begin(); s != snoopPorts.end();
for (SlavePortConstIter s = slavePorts.begin(); s != slavePorts.end();
++s) {
unsigned tmp_bs = (*s)->peerBlockSize();
if (tmp_bs > max_bs)
@ -853,7 +435,7 @@ Bus::findBlockSize()
unsigned int
Bus::drain(Event * de)
BaseBus::drain(Event * de)
{
//We should check that we're not "doing" anything, and that noone is
//waiting. We might be idle but have someone waiting if the device we
@ -867,14 +449,8 @@ Bus::drain(Event * de)
}
void
Bus::startup()
BaseBus::startup()
{
if (tickNextIdle < curTick())
tickNextIdle = (curTick() / clock) * clock + clock;
}
Bus *
BusParams::create()
{
return new Bus(this);
}

View file

@ -45,7 +45,7 @@
/**
* @file
* Declaration of a bus object.
* Declaration of an abstract bus base class.
*/
#ifndef __MEM_BUS_HH__
@ -53,155 +53,26 @@
#include <list>
#include <set>
#include <string>
#include "base/range.hh"
#include "base/range_map.hh"
#include "base/types.hh"
#include "mem/mem_object.hh"
#include "mem/packet.hh"
#include "mem/port.hh"
#include "params/Bus.hh"
#include "sim/eventq.hh"
#include "params/BaseBus.hh"
class Bus : public MemObject
/**
* The base bus contains the common elements of the non-coherent and
* coherent bus. It is an abstract class that does not have any of the
* functionality relating to the actual reception and transmission of
* packets, as this is left for the subclasses.
*
* The BaseBus is responsible for the basic flow control (busy or
* not), the administration of retries, and the address decoding.
*/
class BaseBus : public MemObject
{
/**
* Declaration of the bus slave port type, one will be
* instantiated for each of the master interfaces connecting to
* the bus.
*/
class BusSlavePort : public SlavePort
{
private:
/** A pointer to the bus to which this port belongs. */
Bus *bus;
public:
/** Constructor for the BusSlavePort.*/
BusSlavePort(const std::string &_name, Bus *_bus, PortID _id)
: SlavePort(_name, _bus, _id), bus(_bus)
{ }
protected:
/**
* When receiving a timing request, pass it to the bus.
*/
virtual bool recvTimingReq(PacketPtr pkt)
{ return bus->recvTimingReq(pkt, id); }
/**
* When receiving a timing snoop response, pass it to the bus.
*/
virtual bool recvTimingSnoopResp(PacketPtr pkt)
{ return bus->recvTimingSnoopResp(pkt, id); }
/**
* When receiving an atomic request, pass it to the bus.
*/
virtual Tick recvAtomic(PacketPtr pkt)
{ return bus->recvAtomic(pkt, id); }
/**
* When receiving a functional request, pass it to the bus.
*/
virtual void recvFunctional(PacketPtr pkt)
{ bus->recvFunctional(pkt, id); }
/**
* When receiving a retry, pass it to the bus.
*/
virtual void recvRetry()
{ panic("Bus slave ports always succeed and should never retry.\n"); }
// This should return all the 'owned' addresses that are
// downstream from this bus, yes? That is, the union of all
// the 'owned' address ranges of all the other interfaces on
// this bus...
virtual AddrRangeList getAddrRanges()
{ return bus->getAddrRanges(); }
// Ask the bus to ask everyone on the bus what their block size is and
// take the max of it. This might need to be changed a bit if we ever
// support multiple block sizes.
virtual unsigned deviceBlockSize() const
{ return bus->findBlockSize(); }
};
/**
* Declaration of the bus master port type, one will be
* instantiated for each of the slave interfaces connecting to the
* bus.
*/
class BusMasterPort : public MasterPort
{
private:
/** A pointer to the bus to which this port belongs. */
Bus *bus;
public:
/** Constructor for the BusMasterPort.*/
BusMasterPort(const std::string &_name, Bus *_bus, PortID _id)
: MasterPort(_name, _bus, _id), bus(_bus)
{ }
/**
* Determine if this port should be considered a snooper. This
* is determined by the bus.
*
* @return a boolean that is true if this port is snooping
*/
virtual bool isSnooping() const
{ return bus->isSnooping(); }
protected:
/**
* When receiving a timing response, pass it to the bus.
*/
virtual bool recvTimingResp(PacketPtr pkt)
{ return bus->recvTimingResp(pkt, id); }
/**
* When receiving a timing snoop request, pass it to the bus.
*/
virtual void recvTimingSnoopReq(PacketPtr pkt)
{ return bus->recvTimingSnoopReq(pkt, id); }
/**
* When receiving an atomic snoop request, pass it to the bus.
*/
virtual Tick recvAtomicSnoop(PacketPtr pkt)
{ return bus->recvAtomicSnoop(pkt, id); }
/**
* When receiving a functional snoop request, pass it to the bus.
*/
virtual void recvFunctionalSnoop(PacketPtr pkt)
{ bus->recvFunctionalSnoop(pkt, id); }
/** When reciving a range change from the peer port (at id),
pass it to the bus. */
virtual void recvRangeChange()
{ bus->recvRangeChange(id); }
/** When reciving a retry from the peer port (at id),
pass it to the bus. */
virtual void recvRetry()
{ bus->recvRetry(); }
// Ask the bus to ask everyone on the bus what their block size is and
// take the max of it. This might need to be changed a bit if we ever
// support multiple block sizes.
virtual unsigned deviceBlockSize() const
{ return bus->findBlockSize(); }
};
protected:
/** the clock speed for the bus */
int clock;
@ -219,41 +90,6 @@ class Bus : public MemObject
AddrRangeList defaultRange;
std::vector<SlavePort*> snoopPorts;
/**
* Store the outstanding requests so we can determine which ones
* we generated and which ones were merely forwarded. This is used
* in the coherent bus when coherency responses come back.
*/
std::set<RequestPtr> outstandingReq;
/** Function called by the port when the bus is recieving a Timing
request packet.*/
bool recvTimingReq(PacketPtr pkt, PortID slave_port_id);
/** Function called by the port when the bus is recieving a Timing
response packet.*/
bool recvTimingResp(PacketPtr pkt, PortID master_port_id);
/** Function called by the port when the bus is recieving a timing
snoop request.*/
void recvTimingSnoopReq(PacketPtr pkt, PortID master_port_id);
/** Function called by the port when the bus is recieving a timing
snoop response.*/
bool recvTimingSnoopResp(PacketPtr pkt, PortID slave_port_id);
/**
* Forward a timing packet to our snoopers, potentially excluding
* one of the connected coherent masters to avoid sending a packet
* back to where it came from.
*
* @param pkt Packet to forward
* @param exclude_slave_port_id Id of slave port to exclude
*/
void forwardTiming(PacketPtr pkt, PortID exclude_slave_port_id);
/**
* Determine if the bus is to be considered occupied when being
* presented with a packet from a specific port. If so, the port
@ -274,45 +110,6 @@ class Bus : public MemObject
*/
void succeededTiming(Tick busy_time);
/** Function called by the port when the bus is recieving a Atomic
transaction.*/
Tick recvAtomic(PacketPtr pkt, PortID slave_port_id);
/** Function called by the port when the bus is recieving an
atomic snoop transaction.*/
Tick recvAtomicSnoop(PacketPtr pkt, PortID master_port_id);
/**
* Forward an atomic packet to our snoopers, potentially excluding
* one of the connected coherent masters to avoid sending a packet
* back to where it came from.
*
* @param pkt Packet to forward
* @param exclude_slave_port_id Id of slave port to exclude
*
* @return a pair containing the snoop response and snoop latency
*/
std::pair<MemCmd, Tick> forwardAtomic(PacketPtr pkt,
PortID exclude_slave_port_id);
/** Function called by the port when the bus is recieving a Functional
transaction.*/
void recvFunctional(PacketPtr pkt, PortID slave_port_id);
/** Function called by the port when the bus is recieving a functional
snoop transaction.*/
void recvFunctionalSnoop(PacketPtr pkt, PortID master_port_id);
/**
* Forward a functional packet to our snoopers, potentially
* excluding one of the connected coherent masters to avoid
* sending a packet back to where it came from.
*
* @param pkt Packet to forward
* @param exclude_slave_port_id Id of slave port to exclude
*/
void forwardFunctional(PacketPtr pkt, PortID exclude_slave_port_id);
/** Timing function called by port when it is once again able to process
* requests. */
void recvRetry();
@ -392,13 +189,6 @@ class Bus : public MemObject
*/
AddrRangeList getAddrRanges();
/**
* Determine if the bus port is snooping or not.
*
* @return a boolean indicating if this port is snooping or not
*/
bool isSnooping() const;
/** Calculate the timing parameters for the packet. Updates the
* firstWordTime and finishTime fields of the packet object.
* Returns the tick at which the packet header is completed (which
@ -430,7 +220,7 @@ class Bus : public MemObject
unsigned findBlockSize();
// event used to schedule a release of the bus
EventWrapper<Bus, &Bus::releaseBus> busIdleEvent;
EventWrapper<BaseBus, &BaseBus::releaseBus> busIdleEvent;
bool inRetry;
std::set<PortID> inRecvRangeChange;
@ -439,8 +229,11 @@ class Bus : public MemObject
std::vector<SlavePort*> slavePorts;
std::vector<MasterPort*> masterPorts;
/** Convenience typedefs. */
typedef std::vector<SlavePort*>::iterator SlavePortIter;
typedef std::vector<MasterPort*>::iterator MasterPortIter;
typedef std::vector<SlavePort*>::const_iterator SlavePortConstIter;
typedef std::vector<MasterPort*>::const_iterator MasterPortConstIter;
/** An array of pointers to ports that retry should be called on because the
* original send failed for whatever reason.*/
@ -478,18 +271,20 @@ class Bus : public MemObject
unsigned cachedBlockSize;
bool cachedBlockSizeValid;
BaseBus(const BaseBusParams *p);
virtual ~BaseBus();
public:
/** A function used to return the port associated with this bus object. */
virtual MasterPort& getMasterPort(const std::string& if_name, int idx = -1);
virtual SlavePort& getSlavePort(const std::string& if_name, int idx = -1);
virtual void init();
virtual void startup();
unsigned int drain(Event *de);
Bus(const BusParams *p);
};
#endif //__MEM_BUS_HH__

493
src/mem/coherent_bus.cc Normal file
View file

@ -0,0 +1,493 @@
/*
* Copyright (c) 2011-2012 ARM Limited
* All rights reserved
*
* The license below extends only to copyright in the software and shall
* not be construed as granting a license to any other intellectual
* property including but not limited to intellectual property relating
* to a hardware implementation of the functionality of the software
* licensed hereunder. You may use the software subject to the license
* terms below provided that you ensure that this notice is replicated
* unmodified and in its entirety in all distributions of the software,
* modified or unmodified, in source code or in binary form.
*
* Copyright (c) 2006 The Regents of The University of Michigan
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are
* met: redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer;
* redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution;
* neither the name of the copyright holders nor the names of its
* contributors may be used to endorse or promote products derived from
* this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*
* Authors: Ali Saidi
* Andreas Hansson
* William Wang
*/
/**
* @file
* Definition of a bus object.
*/
#include "base/misc.hh"
#include "base/trace.hh"
#include "debug/BusAddrRanges.hh"
#include "debug/CoherentBus.hh"
#include "mem/coherent_bus.hh"
CoherentBus::CoherentBus(const CoherentBusParams *p)
: BaseBus(p)
{
// create the ports based on the size of the master and slave
// vector ports, and the presence of the default port, the ports
// are enumerated starting from zero
for (int i = 0; i < p->port_master_connection_count; ++i) {
std::string portName = csprintf("%s-p%d", name(), i);
MasterPort* bp = new CoherentBusMasterPort(portName, *this, i);
masterPorts.push_back(bp);
}
// see if we have a default slave device connected and if so add
// our corresponding master port
if (p->port_default_connection_count) {
defaultPortID = masterPorts.size();
std::string portName = csprintf("%s-default", name());
MasterPort* bp = new CoherentBusMasterPort(portName, *this,
defaultPortID);
masterPorts.push_back(bp);
}
// create the slave ports, once again starting at zero
for (int i = 0; i < p->port_slave_connection_count; ++i) {
std::string portName = csprintf("%s-p%d", name(), i);
SlavePort* bp = new CoherentBusSlavePort(portName, *this, i);
slavePorts.push_back(bp);
}
clearPortCache();
}
void
CoherentBus::init()
{
// iterate over our slave ports and determine which of our
// neighbouring master ports are snooping and add them as snoopers
for (SlavePortConstIter p = slavePorts.begin(); p != slavePorts.end();
++p) {
if ((*p)->getMasterPort().isSnooping()) {
DPRINTF(BusAddrRanges, "Adding snooping master %s\n",
(*p)->getMasterPort().name());
snoopPorts.push_back(*p);
}
}
if (snoopPorts.empty())
warn("CoherentBus %s has no snooping ports attached!\n", name());
}
bool
CoherentBus::recvTimingReq(PacketPtr pkt, PortID slave_port_id)
{
// determine the source port based on the id
SlavePort *src_port = slavePorts[slave_port_id];
// test if the bus should be considered occupied for the current
// port, and exclude express snoops from the check
if (!pkt->isExpressSnoop() && isOccupied(src_port)) {
DPRINTF(CoherentBus, "recvTimingReq: src %s %s 0x%x BUSY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
return false;
}
DPRINTF(CoherentBus, "recvTimingReq: src %s %s 0x%x\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
// set the source port for routing of the response
pkt->setSrc(slave_port_id);
Tick headerFinishTime = pkt->isExpressSnoop() ? 0 : calcPacketTiming(pkt);
Tick packetFinishTime = pkt->isExpressSnoop() ? 0 : pkt->finishTime;
// uncacheable requests need never be snooped
if (!pkt->req->isUncacheable()) {
// the packet is a memory-mapped request and should be
// broadcasted to our snoopers but the source
forwardTiming(pkt, slave_port_id);
}
// remember if we add an outstanding req so we can undo it if
// necessary, if the packet needs a response, we should add it
// as outstanding and express snoops never fail so there is
// not need to worry about them
bool add_outstanding = !pkt->isExpressSnoop() && pkt->needsResponse();
// keep track that we have an outstanding request packet
// matching this request, this is used by the coherency
// mechanism in determining what to do with snoop responses
// (in recvTimingSnoop)
if (add_outstanding) {
// we should never have an exsiting request outstanding
assert(outstandingReq.find(pkt->req) == outstandingReq.end());
outstandingReq.insert(pkt->req);
}
// since it is a normal request, determine the destination
// based on the address and attempt to send the packet
bool success = masterPorts[findPort(pkt->getAddr())]->sendTimingReq(pkt);
if (!success) {
// inhibited packets should never be forced to retry
assert(!pkt->memInhibitAsserted());
// if it was added as outstanding and the send failed, then
// erase it again
if (add_outstanding)
outstandingReq.erase(pkt->req);
DPRINTF(CoherentBus, "recvTimingReq: src %s %s 0x%x RETRY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
addToRetryList(src_port);
occupyBus(headerFinishTime);
return false;
}
succeededTiming(packetFinishTime);
return true;
}
bool
CoherentBus::recvTimingResp(PacketPtr pkt, PortID master_port_id)
{
// determine the source port based on the id
MasterPort *src_port = masterPorts[master_port_id];
// test if the bus should be considered occupied for the current
// port
if (isOccupied(src_port)) {
DPRINTF(CoherentBus, "recvTimingResp: src %s %s 0x%x BUSY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
return false;
}
DPRINTF(CoherentBus, "recvTimingResp: src %s %s 0x%x\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
calcPacketTiming(pkt);
Tick packetFinishTime = pkt->finishTime;
// the packet is a normal response to a request that we should
// have seen passing through the bus
assert(outstandingReq.find(pkt->req) != outstandingReq.end());
// remove it as outstanding
outstandingReq.erase(pkt->req);
// send the packet to the destination through one of our slave
// ports, as determined by the destination field
bool success M5_VAR_USED = slavePorts[pkt->getDest()]->sendTimingResp(pkt);
// currently it is illegal to block responses... can lead to
// deadlock
assert(success);
succeededTiming(packetFinishTime);
return true;
}
void
CoherentBus::recvTimingSnoopReq(PacketPtr pkt, PortID master_port_id)
{
DPRINTF(CoherentBus, "recvTimingSnoopReq: src %s %s 0x%x\n",
masterPorts[master_port_id]->name(), pkt->cmdString(),
pkt->getAddr());
// we should only see express snoops from caches
assert(pkt->isExpressSnoop());
// set the source port for routing of the response
pkt->setSrc(master_port_id);
// forward to all snoopers
forwardTiming(pkt, InvalidPortID);
// a snoop request came from a connected slave device (one of
// our master ports), and if it is not coming from the slave
// device responsible for the address range something is
// wrong, hence there is nothing further to do as the packet
// would be going back to where it came from
assert(master_port_id == findPort(pkt->getAddr()));
// this is an express snoop and is never forced to retry
assert(!inRetry);
}
bool
CoherentBus::recvTimingSnoopResp(PacketPtr pkt, PortID slave_port_id)
{
// determine the source port based on the id
SlavePort* src_port = slavePorts[slave_port_id];
// test if the bus should be considered occupied for the current
// port
if (isOccupied(src_port)) {
DPRINTF(CoherentBus, "recvTimingSnoopResp: src %s %s 0x%x BUSY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
return false;
}
DPRINTF(CoherentBus, "recvTimingSnoop: src %s %s 0x%x\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
// get the destination from the packet
PortID dest = pkt->getDest();
// responses are never express snoops
assert(!pkt->isExpressSnoop());
calcPacketTiming(pkt);
Tick packetFinishTime = pkt->finishTime;
// determine if the response is from a snoop request we
// created as the result of a normal request (in which case it
// should be in the outstandingReq), or if we merely forwarded
// someone else's snoop request
if (outstandingReq.find(pkt->req) == outstandingReq.end()) {
// this is a snoop response to a snoop request we
// forwarded, e.g. coming from the L1 and going to the L2
// this should be forwarded as a snoop response
bool success M5_VAR_USED = masterPorts[dest]->sendTimingSnoopResp(pkt);
assert(success);
} else {
// we got a snoop response on one of our slave ports,
// i.e. from a coherent master connected to the bus, and
// since we created the snoop request as part of
// recvTiming, this should now be a normal response again
outstandingReq.erase(pkt->req);
// this is a snoop response from a coherent master, with a
// destination field set on its way through the bus as
// request, hence it should never go back to where the
// snoop response came from, but instead to where the
// original request came from
assert(slave_port_id != dest);
// as a normal response, it should go back to a master
// through one of our slave ports
bool success M5_VAR_USED = slavePorts[dest]->sendTimingResp(pkt);
// currently it is illegal to block responses... can lead
// to deadlock
assert(success);
}
succeededTiming(packetFinishTime);
return true;
}
void
CoherentBus::forwardTiming(PacketPtr pkt, PortID exclude_slave_port_id)
{
for (SlavePortIter s = snoopPorts.begin(); s != snoopPorts.end(); ++s) {
SlavePort *p = *s;
// we could have gotten this request from a snooping master
// (corresponding to our own slave port that is also in
// snoopPorts) and should not send it back to where it came
// from
if (exclude_slave_port_id == InvalidPortID ||
p->getId() != exclude_slave_port_id) {
// cache is not allowed to refuse snoop
p->sendTimingSnoopReq(pkt);
}
}
}
Tick
CoherentBus::recvAtomic(PacketPtr pkt, PortID slave_port_id)
{
DPRINTF(CoherentBus, "recvAtomic: packet src %s addr 0x%x cmd %s\n",
slavePorts[slave_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
MemCmd snoop_response_cmd = MemCmd::InvalidCmd;
Tick snoop_response_latency = 0;
// uncacheable requests need never be snooped
if (!pkt->req->isUncacheable()) {
// forward to all snoopers but the source
std::pair<MemCmd, Tick> snoop_result =
forwardAtomic(pkt, slave_port_id);
snoop_response_cmd = snoop_result.first;
snoop_response_latency = snoop_result.second;
}
// even if we had a snoop response, we must continue and also
// perform the actual request at the destination
PortID dest_id = findPort(pkt->getAddr());
// forward the request to the appropriate destination
Tick response_latency = masterPorts[dest_id]->sendAtomic(pkt);
// if we got a response from a snooper, restore it here
if (snoop_response_cmd != MemCmd::InvalidCmd) {
// no one else should have responded
assert(!pkt->isResponse());
pkt->cmd = snoop_response_cmd;
response_latency = snoop_response_latency;
}
pkt->finishTime = curTick() + response_latency;
return response_latency;
}
Tick
CoherentBus::recvAtomicSnoop(PacketPtr pkt, PortID master_port_id)
{
DPRINTF(CoherentBus, "recvAtomicSnoop: packet src %s addr 0x%x cmd %s\n",
masterPorts[master_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
// forward to all snoopers
std::pair<MemCmd, Tick> snoop_result =
forwardAtomic(pkt, InvalidPortID);
MemCmd snoop_response_cmd = snoop_result.first;
Tick snoop_response_latency = snoop_result.second;
if (snoop_response_cmd != MemCmd::InvalidCmd)
pkt->cmd = snoop_response_cmd;
pkt->finishTime = curTick() + snoop_response_latency;
return snoop_response_latency;
}
std::pair<MemCmd, Tick>
CoherentBus::forwardAtomic(PacketPtr pkt, PortID exclude_slave_port_id)
{
// the packet may be changed on snoops, record the original
// command to enable us to restore it between snoops so that
// additional snoops can take place properly
MemCmd orig_cmd = pkt->cmd;
MemCmd snoop_response_cmd = MemCmd::InvalidCmd;
Tick snoop_response_latency = 0;
for (SlavePortIter s = snoopPorts.begin(); s != snoopPorts.end(); ++s) {
SlavePort *p = *s;
// we could have gotten this request from a snooping master
// (corresponding to our own slave port that is also in
// snoopPorts) and should not send it back to where it came
// from
if (exclude_slave_port_id == InvalidPortID ||
p->getId() != exclude_slave_port_id) {
Tick latency = p->sendAtomicSnoop(pkt);
// in contrast to a functional access, we have to keep on
// going as all snoopers must be updated even if we get a
// response
if (pkt->isResponse()) {
// response from snoop agent
assert(pkt->cmd != orig_cmd);
assert(pkt->memInhibitAsserted());
// should only happen once
assert(snoop_response_cmd == MemCmd::InvalidCmd);
// save response state
snoop_response_cmd = pkt->cmd;
snoop_response_latency = latency;
// restore original packet state for remaining snoopers
pkt->cmd = orig_cmd;
}
}
}
// the packet is restored as part of the loop and any potential
// snoop response is part of the returned pair
return std::make_pair(snoop_response_cmd, snoop_response_latency);
}
void
CoherentBus::recvFunctional(PacketPtr pkt, PortID slave_port_id)
{
if (!pkt->isPrint()) {
// don't do DPRINTFs on PrintReq as it clutters up the output
DPRINTF(CoherentBus,
"recvFunctional: packet src %s addr 0x%x cmd %s\n",
slavePorts[slave_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
}
// uncacheable requests need never be snooped
if (!pkt->req->isUncacheable()) {
// forward to all snoopers but the source
forwardFunctional(pkt, slave_port_id);
}
// there is no need to continue if the snooping has found what we
// were looking for and the packet is already a response
if (!pkt->isResponse()) {
PortID dest_id = findPort(pkt->getAddr());
masterPorts[dest_id]->sendFunctional(pkt);
}
}
void
CoherentBus::recvFunctionalSnoop(PacketPtr pkt, PortID master_port_id)
{
if (!pkt->isPrint()) {
// don't do DPRINTFs on PrintReq as it clutters up the output
DPRINTF(CoherentBus,
"recvFunctionalSnoop: packet src %s addr 0x%x cmd %s\n",
masterPorts[master_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
}
// forward to all snoopers
forwardFunctional(pkt, InvalidPortID);
}
void
CoherentBus::forwardFunctional(PacketPtr pkt, PortID exclude_slave_port_id)
{
for (SlavePortIter s = snoopPorts.begin(); s != snoopPorts.end(); ++s) {
SlavePort *p = *s;
// we could have gotten this request from a snooping master
// (corresponding to our own slave port that is also in
// snoopPorts) and should not send it back to where it came
// from
if (exclude_slave_port_id == InvalidPortID ||
p->getId() != exclude_slave_port_id)
p->sendFunctionalSnoop(pkt);
// if we get a response we are done
if (pkt->isResponse()) {
break;
}
}
}
CoherentBus *
CoherentBusParams::create()
{
return new CoherentBus(this);
}

290
src/mem/coherent_bus.hh Normal file
View file

@ -0,0 +1,290 @@
/*
* Copyright (c) 2011-2012 ARM Limited
* All rights reserved
*
* The license below extends only to copyright in the software and shall
* not be construed as granting a license to any other intellectual
* property including but not limited to intellectual property relating
* to a hardware implementation of the functionality of the software
* licensed hereunder. You may use the software subject to the license
* terms below provided that you ensure that this notice is replicated
* unmodified and in its entirety in all distributions of the software,
* modified or unmodified, in source code or in binary form.
*
* Copyright (c) 2002-2005 The Regents of The University of Michigan
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are
* met: redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer;
* redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution;
* neither the name of the copyright holders nor the names of its
* contributors may be used to endorse or promote products derived from
* this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*
* Authors: Ron Dreslinski
* Ali Saidi
* Andreas Hansson
* William Wang
*/
/**
* @file
* Declaration of a coherent bus.
*/
#ifndef __MEM_COHERENT_BUS_HH__
#define __MEM_COHERENT_BUS_HH__
#include "mem/bus.hh"
#include "params/CoherentBus.hh"
/**
* A coherent bus connects a number of (potentially) snooping masters
* and slaves, and routes the request and response packets based on
* the address, and also forwards all requests to the snoopers and
* deals with the snoop responses.
*
* The coherent bus can be used as a template for modelling QPI,
* HyperTransport, ACE and coherent OCP buses, and is typically used
* for the L1-to-L2 buses and as the main system interconnect.
*/
class CoherentBus : public BaseBus
{
protected:
/**
* Declaration of the coherent bus slave port type, one will be
* instantiated for each of the master ports connecting to the
* bus.
*/
class CoherentBusSlavePort : public SlavePort
{
private:
/** A reference to the bus to which this port belongs. */
CoherentBus &bus;
public:
CoherentBusSlavePort(const std::string &_name,
CoherentBus &_bus, PortID _id)
: SlavePort(_name, &_bus, _id), bus(_bus)
{ }
protected:
/**
* When receiving a timing request, pass it to the bus.
*/
virtual bool recvTimingReq(PacketPtr pkt)
{ return bus.recvTimingReq(pkt, id); }
/**
* When receiving a timing snoop response, pass it to the bus.
*/
virtual bool recvTimingSnoopResp(PacketPtr pkt)
{ return bus.recvTimingSnoopResp(pkt, id); }
/**
* When receiving an atomic request, pass it to the bus.
*/
virtual Tick recvAtomic(PacketPtr pkt)
{ return bus.recvAtomic(pkt, id); }
/**
* When receiving a functional request, pass it to the bus.
*/
virtual void recvFunctional(PacketPtr pkt)
{ bus.recvFunctional(pkt, id); }
/**
* When receiving a retry, pass it to the bus.
*/
virtual void recvRetry()
{ panic("Bus slave ports always succeed and should never retry.\n"); }
/**
* Return the union of all adress ranges seen by this bus.
*/
virtual AddrRangeList getAddrRanges()
{ return bus.getAddrRanges(); }
/**
* Get the maximum block size as seen by the bus.
*/
virtual unsigned deviceBlockSize() const
{ return bus.findBlockSize(); }
};
/**
* Declaration of the coherent bus master port type, one will be
* instantiated for each of the slave interfaces connecting to the
* bus.
*/
class CoherentBusMasterPort : public MasterPort
{
private:
/** A reference to the bus to which this port belongs. */
CoherentBus &bus;
public:
CoherentBusMasterPort(const std::string &_name,
CoherentBus &_bus, PortID _id)
: MasterPort(_name, &_bus, _id), bus(_bus)
{ }
protected:
/**
* Determine if this port should be considered a snooper. For
* a coherent bus master port this is always true.
*
* @return a boolean that is true if this port is snooping
*/
virtual bool isSnooping() const
{ return true; }
/**
* When receiving a timing response, pass it to the bus.
*/
virtual bool recvTimingResp(PacketPtr pkt)
{ return bus.recvTimingResp(pkt, id); }
/**
* When receiving a timing snoop request, pass it to the bus.
*/
virtual void recvTimingSnoopReq(PacketPtr pkt)
{ return bus.recvTimingSnoopReq(pkt, id); }
/**
* When receiving an atomic snoop request, pass it to the bus.
*/
virtual Tick recvAtomicSnoop(PacketPtr pkt)
{ return bus.recvAtomicSnoop(pkt, id); }
/**
* When receiving a functional snoop request, pass it to the bus.
*/
virtual void recvFunctionalSnoop(PacketPtr pkt)
{ bus.recvFunctionalSnoop(pkt, id); }
/** When reciving a range change from the peer port (at id),
pass it to the bus. */
virtual void recvRangeChange()
{ bus.recvRangeChange(id); }
/** When reciving a retry from the peer port (at id),
pass it to the bus. */
virtual void recvRetry()
{ bus.recvRetry(); }
// Ask the bus to ask everyone on the bus what their block size is and
// take the max of it. This might need to be changed a bit if we ever
// support multiple block sizes.
virtual unsigned deviceBlockSize() const
{ return bus.findBlockSize(); }
};
std::vector<SlavePort*> snoopPorts;
/**
* Store the outstanding requests so we can determine which ones
* we generated and which ones were merely forwarded. This is used
* in the coherent bus when coherency responses come back.
*/
std::set<RequestPtr> outstandingReq;
/** Function called by the port when the bus is recieving a Timing
request packet.*/
bool recvTimingReq(PacketPtr pkt, PortID slave_port_id);
/** Function called by the port when the bus is recieving a Timing
response packet.*/
bool recvTimingResp(PacketPtr pkt, PortID master_port_id);
/** Function called by the port when the bus is recieving a timing
snoop request.*/
void recvTimingSnoopReq(PacketPtr pkt, PortID master_port_id);
/** Function called by the port when the bus is recieving a timing
snoop response.*/
bool recvTimingSnoopResp(PacketPtr pkt, PortID slave_port_id);
/**
* Forward a timing packet to our snoopers, potentially excluding
* one of the connected coherent masters to avoid sending a packet
* back to where it came from.
*
* @param pkt Packet to forward
* @param exclude_slave_port_id Id of slave port to exclude
*/
void forwardTiming(PacketPtr pkt, PortID exclude_slave_port_id);
/** Function called by the port when the bus is recieving a Atomic
transaction.*/
Tick recvAtomic(PacketPtr pkt, PortID slave_port_id);
/** Function called by the port when the bus is recieving an
atomic snoop transaction.*/
Tick recvAtomicSnoop(PacketPtr pkt, PortID master_port_id);
/**
* Forward an atomic packet to our snoopers, potentially excluding
* one of the connected coherent masters to avoid sending a packet
* back to where it came from.
*
* @param pkt Packet to forward
* @param exclude_slave_port_id Id of slave port to exclude
*
* @return a pair containing the snoop response and snoop latency
*/
std::pair<MemCmd, Tick> forwardAtomic(PacketPtr pkt,
PortID exclude_slave_port_id);
/** Function called by the port when the bus is recieving a Functional
transaction.*/
void recvFunctional(PacketPtr pkt, PortID slave_port_id);
/** Function called by the port when the bus is recieving a functional
snoop transaction.*/
void recvFunctionalSnoop(PacketPtr pkt, PortID master_port_id);
/**
* Forward a functional packet to our snoopers, potentially
* excluding one of the connected coherent masters to avoid
* sending a packet back to where it came from.
*
* @param pkt Packet to forward
* @param exclude_slave_port_id Id of slave port to exclude
*/
void forwardFunctional(PacketPtr pkt, PortID exclude_slave_port_id);
public:
virtual void init();
CoherentBus(const CoherentBusParams *p);
};
#endif //__MEM_COHERENT_BUS_HH__

209
src/mem/noncoherent_bus.cc Normal file
View file

@ -0,0 +1,209 @@
/*
* Copyright (c) 2011-2012 ARM Limited
* All rights reserved
*
* The license below extends only to copyright in the software and shall
* not be construed as granting a license to any other intellectual
* property including but not limited to intellectual property relating
* to a hardware implementation of the functionality of the software
* licensed hereunder. You may use the software subject to the license
* terms below provided that you ensure that this notice is replicated
* unmodified and in its entirety in all distributions of the software,
* modified or unmodified, in source code or in binary form.
*
* Copyright (c) 2006 The Regents of The University of Michigan
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are
* met: redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer;
* redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution;
* neither the name of the copyright holders nor the names of its
* contributors may be used to endorse or promote products derived from
* this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*
* Authors: Ali Saidi
* Andreas Hansson
* William Wang
*/
/**
* @file
* Definition of a bus object.
*/
#include "base/misc.hh"
#include "base/trace.hh"
#include "debug/Bus.hh"
#include "debug/BusAddrRanges.hh"
#include "debug/NoncoherentBus.hh"
#include "mem/noncoherent_bus.hh"
NoncoherentBus::NoncoherentBus(const NoncoherentBusParams *p)
: BaseBus(p)
{
// create the ports based on the size of the master and slave
// vector ports, and the presence of the default port, the ports
// are enumerated starting from zero
for (int i = 0; i < p->port_master_connection_count; ++i) {
std::string portName = csprintf("%s-p%d", name(), i);
MasterPort* bp = new NoncoherentBusMasterPort(portName, *this, i);
masterPorts.push_back(bp);
}
// see if we have a default slave device connected and if so add
// our corresponding master port
if (p->port_default_connection_count) {
defaultPortID = masterPorts.size();
std::string portName = csprintf("%s-default", name());
MasterPort* bp = new NoncoherentBusMasterPort(portName, *this,
defaultPortID);
masterPorts.push_back(bp);
}
// create the slave ports, once again starting at zero
for (int i = 0; i < p->port_slave_connection_count; ++i) {
std::string portName = csprintf("%s-p%d", name(), i);
SlavePort* bp = new NoncoherentBusSlavePort(portName, *this, i);
slavePorts.push_back(bp);
}
clearPortCache();
}
bool
NoncoherentBus::recvTimingReq(PacketPtr pkt, PortID slave_port_id)
{
// determine the source port based on the id
SlavePort *src_port = slavePorts[slave_port_id];
// we should never see express snoops on a non-coherent bus
assert(!pkt->isExpressSnoop());
// test if the bus should be considered occupied for the current
// port
if (isOccupied(src_port)) {
DPRINTF(NoncoherentBus, "recvTimingReq: src %s %s 0x%x BUSY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
return false;
}
DPRINTF(NoncoherentBus, "recvTimingReq: src %s %s 0x%x\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
// set the source port for routing of the response
pkt->setSrc(slave_port_id);
Tick headerFinishTime = calcPacketTiming(pkt);
Tick packetFinishTime = pkt->finishTime;
// since it is a normal request, determine the destination
// based on the address and attempt to send the packet
bool success = masterPorts[findPort(pkt->getAddr())]->sendTimingReq(pkt);
if (!success) {
// inhibited packets should never be forced to retry
assert(!pkt->memInhibitAsserted());
DPRINTF(NoncoherentBus, "recvTimingReq: src %s %s 0x%x RETRY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
addToRetryList(src_port);
occupyBus(headerFinishTime);
return false;
}
succeededTiming(packetFinishTime);
return true;
}
bool
NoncoherentBus::recvTimingResp(PacketPtr pkt, PortID master_port_id)
{
// determine the source port based on the id
MasterPort *src_port = masterPorts[master_port_id];
// test if the bus should be considered occupied for the current
// port
if (isOccupied(src_port)) {
DPRINTF(NoncoherentBus, "recvTimingResp: src %s %s 0x%x BUSY\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
return false;
}
DPRINTF(NoncoherentBus, "recvTimingResp: src %s %s 0x%x\n",
src_port->name(), pkt->cmdString(), pkt->getAddr());
calcPacketTiming(pkt);
Tick packetFinishTime = pkt->finishTime;
// send the packet to the destination through one of our slave
// ports, as determined by the destination field
bool success M5_VAR_USED = slavePorts[pkt->getDest()]->sendTimingResp(pkt);
// currently it is illegal to block responses... can lead to
// deadlock
assert(success);
succeededTiming(packetFinishTime);
return true;
}
Tick
NoncoherentBus::recvAtomic(PacketPtr pkt, PortID slave_port_id)
{
DPRINTF(NoncoherentBus, "recvAtomic: packet src %s addr 0x%x cmd %s\n",
slavePorts[slave_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
// determine the destination port
PortID dest_id = findPort(pkt->getAddr());
// forward the request to the appropriate destination
Tick response_latency = masterPorts[dest_id]->sendAtomic(pkt);
pkt->finishTime = curTick() + response_latency;
return response_latency;
}
void
NoncoherentBus::recvFunctional(PacketPtr pkt, PortID slave_port_id)
{
if (!pkt->isPrint()) {
// don't do DPRINTFs on PrintReq as it clutters up the output
DPRINTF(NoncoherentBus,
"recvFunctional: packet src %s addr 0x%x cmd %s\n",
slavePorts[slave_port_id]->name(), pkt->getAddr(),
pkt->cmdString());
}
// determine the destination port
PortID dest_id = findPort(pkt->getAddr());
// forward the request to the appropriate destination
masterPorts[dest_id]->sendFunctional(pkt);
}
NoncoherentBus*
NoncoherentBusParams::create()
{
return new NoncoherentBus(this);
}

201
src/mem/noncoherent_bus.hh Normal file
View file

@ -0,0 +1,201 @@
/*
* Copyright (c) 2011-2012 ARM Limited
* All rights reserved
*
* The license below extends only to copyright in the software and shall
* not be construed as granting a license to any other intellectual
* property including but not limited to intellectual property relating
* to a hardware implementation of the functionality of the software
* licensed hereunder. You may use the software subject to the license
* terms below provided that you ensure that this notice is replicated
* unmodified and in its entirety in all distributions of the software,
* modified or unmodified, in source code or in binary form.
*
* Copyright (c) 2002-2005 The Regents of The University of Michigan
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are
* met: redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer;
* redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution;
* neither the name of the copyright holders nor the names of its
* contributors may be used to endorse or promote products derived from
* this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*
* Authors: Ron Dreslinski
* Ali Saidi
* Andreas Hansson
* William Wang
*/
/**
* @file
* Declaration of a non-coherent bus.
*/
#ifndef __MEM_NONCOHERENT_BUS_HH__
#define __MEM_NONCOHERENT_BUS_HH__
#include "mem/bus.hh"
#include "params/NoncoherentBus.hh"
/**
* A non-coherent bus connects a number of non-snooping masters and
* slaves, and routes the request and response packets based on the
* address. The request packets issued by the master connected to a
* non-coherent bus could still snoop in caches attached to a coherent
* bus, as is the case with the I/O bus and memory bus in most system
* configurations. No snoops will, however, reach any master on the
* non-coherent bus itself.
*
* The non-coherent bus can be used as a template for modelling PCI,
* PCIe, and non-coherent AMBA and OCP buses, and is typically used
* for the I/O buses.
*/
class NoncoherentBus : public BaseBus
{
protected:
/**
* Declaration of the non-coherent bus slave port type, one will
* be instantiated for each of the master ports connecting to the
* bus.
*/
class NoncoherentBusSlavePort : public SlavePort
{
private:
/** A reference to the bus to which this port belongs. */
NoncoherentBus &bus;
public:
NoncoherentBusSlavePort(const std::string &_name,
NoncoherentBus &_bus, PortID _id)
: SlavePort(_name, &_bus, _id), bus(_bus)
{ }
protected:
/**
* When receiving a timing request, pass it to the bus.
*/
virtual bool recvTimingReq(PacketPtr pkt)
{ return bus.recvTimingReq(pkt, id); }
/**
* When receiving an atomic request, pass it to the bus.
*/
virtual Tick recvAtomic(PacketPtr pkt)
{ return bus.recvAtomic(pkt, id); }
/**
* When receiving a functional request, pass it to the bus.
*/
virtual void recvFunctional(PacketPtr pkt)
{ bus.recvFunctional(pkt, id); }
/**
* When receiving a retry, pass it to the bus.
*/
virtual void recvRetry()
{ panic("Bus slave ports always succeed and should never retry.\n"); }
/**
* Return the union of all adress ranges seen by this bus.
*/
virtual AddrRangeList getAddrRanges()
{ return bus.getAddrRanges(); }
/**
* Get the maximum block size as seen by the bus.
*/
virtual unsigned deviceBlockSize() const
{ return bus.findBlockSize(); }
};
/**
* Declaration of the bus master port type, one will be
* instantiated for each of the slave ports connecting to the
* bus.
*/
class NoncoherentBusMasterPort : public MasterPort
{
private:
/** A reference to the bus to which this port belongs. */
NoncoherentBus &bus;
public:
NoncoherentBusMasterPort(const std::string &_name,
NoncoherentBus &_bus, PortID _id)
: MasterPort(_name, &_bus, _id), bus(_bus)
{ }
protected:
/**
* When receiving a timing response, pass it to the bus.
*/
virtual bool recvTimingResp(PacketPtr pkt)
{ return bus.recvTimingResp(pkt, id); }
/** When reciving a range change from the peer port (at id),
pass it to the bus. */
virtual void recvRangeChange()
{ bus.recvRangeChange(id); }
/** When reciving a retry from the peer port (at id),
pass it to the bus. */
virtual void recvRetry()
{ bus.recvRetry(); }
/**
* Get the maximum block size as seen by the bus.
*/
virtual unsigned deviceBlockSize() const
{ return bus.findBlockSize(); }
};
/** Function called by the port when the bus is recieving a Timing
request packet.*/
bool recvTimingReq(PacketPtr pkt, PortID slave_port_id);
/** Function called by the port when the bus is recieving a Timing
response packet.*/
bool recvTimingResp(PacketPtr pkt, PortID master_port_id);
/** Function called by the port when the bus is recieving a Atomic
transaction.*/
Tick recvAtomic(PacketPtr pkt, PortID slave_port_id);
/** Function called by the port when the bus is recieving a Functional
transaction.*/
void recvFunctional(PacketPtr pkt, PortID slave_port_id);
public:
NoncoherentBus(const NoncoherentBusParams *p);
};
#endif //__MEM_NONCOHERENT_BUS_HH__

View file

@ -49,7 +49,7 @@ cpu.clock = '2GHz'
system = System(cpu = cpu,
physmem = SimpleMemory(),
membus = Bus())
membus = CoherentBus())
system.system_port = system.membus.slave
system.physmem.port = system.membus.master
# create the interrupt controller

View file

@ -58,10 +58,10 @@ cpus = [ MemTest() for i in xrange(nb_cores) ]
# system simulated
system = System(cpu = cpus, funcmem = SimpleMemory(in_addr_map = False),
physmem = SimpleMemory(),
membus = Bus(clock="500GHz", width=16))
membus = CoherentBus(clock="500GHz", width=16))
# l2cache & bus
system.toL2Bus = Bus(clock="500GHz", width=16)
system.toL2Bus = CoherentBus(clock="500GHz", width=16)
system.l2c = L2(size='64kB', assoc=8)
system.l2c.cpu_side = system.toL2Bus.master

View file

@ -60,7 +60,7 @@ cpu.clock = '2GHz'
system = System(cpu = cpu,
physmem = SimpleMemory(),
membus = Bus())
membus = CoherentBus())
system.system_port = system.membus.slave
system.physmem.port = system.membus.master
cpu.connectAllPorts(system.membus)

View file

@ -37,7 +37,7 @@ import ruby_config
ruby_memory = ruby_config.generate("TwoLevel_SplitL1UnifiedL2.rb", nb_cores)
# system simulated
system = System(cpu = cpus, physmem = ruby_memory, membus = Bus())
system = System(cpu = cpus, physmem = ruby_memory, membus = CoherentBus())
for cpu in cpus:
# create the interrupt controller

View file

@ -56,10 +56,10 @@ nb_cores = 4
cpus = [ DerivO3CPU(cpu_id=i) for i in xrange(nb_cores) ]
# system simulated
system = System(cpu = cpus, physmem = SimpleMemory(), membus = Bus())
system = System(cpu = cpus, physmem = SimpleMemory(), membus = CoherentBus())
# l2cache & bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.l2c = L2(size='4MB', assoc=8)
system.l2c.cpu_side = system.toL2Bus.master

View file

@ -39,7 +39,7 @@ cpu.clock = '2GHz'
system = System(cpu = cpu,
physmem = ruby_memory,
membus = Bus())
membus = CoherentBus())
system.physmem.port = system.membus.master
# create the interrupt controller
cpu.createInterruptController()

View file

@ -49,7 +49,7 @@ cpu.clock = '2GHz'
system = System(cpu = cpu,
physmem = SimpleMemory(),
membus = Bus())
membus = CoherentBus())
system.system_port = system.membus.slave
system.physmem.port = system.membus.master
# create the interrupt controller

View file

@ -92,7 +92,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -94,7 +94,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -90,7 +90,7 @@ system.kernel = FSConfig.binary('x86_64-vmlinux-2.6.22.9')
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.iocache = IOCache()
system.iocache.cpu_side = system.iobus.master
system.iocache.mem_side = system.membus.slave

View file

@ -83,7 +83,7 @@ system = FSConfig.makeArmSystem('timing', "RealView_PBX", None, False)
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.iocache = IOCache()
system.iocache.cpu_side = system.iobus.master
system.iocache.mem_side = system.membus.slave

View file

@ -77,7 +77,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpus
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -74,7 +74,7 @@ system = FSConfig.makeArmSystem('timing', "RealView_PBX", None, False)
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.iocache = IOCache()
system.iocache.cpu_side = system.iobus.master
system.iocache.mem_side = system.membus.slave

View file

@ -77,7 +77,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpus
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -76,7 +76,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -77,7 +77,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpus
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -74,7 +74,7 @@ system = FSConfig.makeArmSystem('timing', "RealView_PBX", None, False)
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.iocache = IOCache()
system.iocache.cpu_side = system.iobus.master
system.iocache.mem_side = system.membus.slave

View file

@ -40,7 +40,7 @@ from m5.objects import *
system = System(cpu = AtomicSimpleCPU(cpu_id=0),
physmem = SimpleMemory(),
membus = Bus())
membus = CoherentBus())
system.system_port = system.membus.slave
system.physmem.port = system.membus.master
system.cpu.addCheckerCpu()

View file

@ -37,7 +37,7 @@ import ruby_config
ruby_memory = ruby_config.generate("TwoLevel_SplitL1UnifiedL2.rb", nb_cores)
# system simulated
system = System(cpu = cpus, physmem = ruby_memory, membus = Bus())
system = System(cpu = cpus, physmem = ruby_memory, membus = CoherentBus())
# add L1 caches
for cpu in cpus:

View file

@ -57,10 +57,10 @@ cpus = [ AtomicSimpleCPU(cpu_id=i) for i in xrange(nb_cores) ]
# system simulated
system = System(cpu = cpus,
physmem = SimpleMemory(range = AddrRange('1024MB')),
membus = Bus())
membus = CoherentBus())
# l2cache & bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.l2c = L2(size='4MB', assoc=8)
system.l2c.cpu_side = system.toL2Bus.master

View file

@ -31,7 +31,7 @@ from m5.objects import *
system = System(cpu = AtomicSimpleCPU(cpu_id=0),
physmem = SimpleMemory(),
membus = Bus())
membus = CoherentBus())
system.system_port = system.membus.slave
system.physmem.port = system.membus.master
# create the interrupt controller

View file

@ -55,10 +55,10 @@ nb_cores = 4
cpus = [ TimingSimpleCPU(cpu_id=i) for i in xrange(nb_cores) ]
# system simulated
system = System(cpu = cpus, physmem = SimpleMemory(), membus = Bus())
system = System(cpu = cpus, physmem = SimpleMemory(), membus = CoherentBus())
# l2cache & bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.l2c = L2(size='4MB', assoc=8)
system.l2c.cpu_side = system.toL2Bus.master

View file

@ -45,7 +45,7 @@ cpu.addTwoLevelCacheHierarchy(MyL1Cache(size = '128kB'),
MyCache(size = '2MB', latency='10ns'))
system = System(cpu = cpu,
physmem = SimpleMemory(),
membus = Bus())
membus = CoherentBus())
system.system_port = system.membus.slave
system.physmem.port = system.membus.master
# create the interrupt controller

View file

@ -78,7 +78,7 @@ system = FSConfig.makeLinuxAlphaSystem('timing')
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.iocache = IOCache()
system.iocache.cpu_side = system.iobus.master
system.iocache.mem_side = system.membus.slave

View file

@ -75,7 +75,7 @@ system = FSConfig.makeLinuxAlphaSystem('timing')
system.cpu = cpus
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.iocache = IOCache()
system.iocache.cpu_side = system.iobus.master
system.iocache.mem_side = system.membus.slave

View file

@ -75,7 +75,7 @@ system = FSConfig.makeLinuxAlphaSystem('timing')
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.iocache = IOCache()
system.iocache.cpu_side = system.iobus.master
system.iocache.mem_side = system.membus.slave

View file

@ -77,7 +77,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpus
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -77,7 +77,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -77,7 +77,7 @@ system.iocache.mem_side = system.membus.slave
system.cpu = cpus
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
#connect up the l2 cache
system.l2c = L2(size='4MB', assoc=8)

View file

@ -75,7 +75,7 @@ system = FSConfig.makeLinuxAlphaSystem('timing')
system.cpu = cpu
#create the l1/l2 bus
system.toL2Bus = Bus()
system.toL2Bus = CoherentBus()
system.iocache = IOCache()
system.iocache.cpu_side = system.iobus.master
system.iocache.mem_side = system.membus.slave