Merge pull request #321 from ucb-bar/add-multiclock-coreplex
add multiclock support to Coreplex
This commit is contained in:
commit
2522bdd7b8
167
src/main/scala/coreplex/BaseCoreplex.scala
Normal file
167
src/main/scala/coreplex/BaseCoreplex.scala
Normal file
@ -0,0 +1,167 @@
|
|||||||
|
package coreplex
|
||||||
|
|
||||||
|
import Chisel._
|
||||||
|
import cde.{Parameters, Field}
|
||||||
|
import junctions._
|
||||||
|
import uncore.tilelink._
|
||||||
|
import uncore.tilelink2._
|
||||||
|
import uncore.coherence._
|
||||||
|
import uncore.agents._
|
||||||
|
import uncore.devices._
|
||||||
|
import uncore.util._
|
||||||
|
import uncore.converters._
|
||||||
|
import rocket._
|
||||||
|
import rocket.Util._
|
||||||
|
|
||||||
|
/** Number of memory channels */
|
||||||
|
case object NMemoryChannels extends Field[Int]
|
||||||
|
/** Number of banks per memory channel */
|
||||||
|
case object NBanksPerMemoryChannel extends Field[Int]
|
||||||
|
/** Least significant bit of address used for bank partitioning */
|
||||||
|
case object BankIdLSB extends Field[Int]
|
||||||
|
/** Function for building some kind of coherence manager agent */
|
||||||
|
case object BuildL2CoherenceManager extends Field[(Int, Parameters) => CoherenceAgent]
|
||||||
|
/** Function for building some kind of tile connected to a reset signal */
|
||||||
|
case object BuildTiles extends Field[Seq[(Bool, Parameters) => Tile]]
|
||||||
|
/** The file to read the BootROM contents from */
|
||||||
|
case object BootROMFile extends Field[String]
|
||||||
|
|
||||||
|
trait HasCoreplexParameters {
|
||||||
|
implicit val p: Parameters
|
||||||
|
lazy val nBanksPerMemChannel = p(NBanksPerMemoryChannel)
|
||||||
|
lazy val lsb = p(BankIdLSB)
|
||||||
|
lazy val innerParams = p.alterPartial({ case TLId => "L1toL2" })
|
||||||
|
lazy val outermostParams = p.alterPartial({ case TLId => "Outermost" })
|
||||||
|
lazy val outermostMMIOParams = p.alterPartial({ case TLId => "MMIO_Outermost" })
|
||||||
|
lazy val globalAddrMap = p(rocketchip.GlobalAddrMap)
|
||||||
|
}
|
||||||
|
|
||||||
|
case class CoreplexConfig(
|
||||||
|
nTiles: Int,
|
||||||
|
nExtInterrupts: Int,
|
||||||
|
nSlaves: Int,
|
||||||
|
nMemChannels: Int,
|
||||||
|
hasSupervisor: Boolean,
|
||||||
|
hasExtMMIOPort: Boolean)
|
||||||
|
{
|
||||||
|
val plicKey = PLICConfig(nTiles, hasSupervisor, nExtInterrupts, 0)
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract class BaseCoreplex(c: CoreplexConfig)(implicit p: Parameters) extends LazyModule
|
||||||
|
|
||||||
|
abstract class BaseCoreplexBundle(val c: CoreplexConfig)(implicit val p: Parameters) extends Bundle with HasCoreplexParameters {
|
||||||
|
val master = new Bundle {
|
||||||
|
val mem = Vec(c.nMemChannels, new ClientUncachedTileLinkIO()(outermostParams))
|
||||||
|
val mmio = c.hasExtMMIOPort.option(new ClientUncachedTileLinkIO()(outermostMMIOParams))
|
||||||
|
}
|
||||||
|
val slave = Vec(c.nSlaves, new ClientUncachedTileLinkIO()(innerParams)).flip
|
||||||
|
val interrupts = Vec(c.nExtInterrupts, Bool()).asInput
|
||||||
|
val debug = new DebugBusIO()(p).flip
|
||||||
|
val clint = Vec(c.nTiles, new CoreplexLocalInterrupts).asInput
|
||||||
|
val success = Bool(OUTPUT)
|
||||||
|
val resetVector = UInt(INPUT, p(XLen))
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract class BaseCoreplexModule[+L <: BaseCoreplex, +B <: BaseCoreplexBundle](
|
||||||
|
c: CoreplexConfig, l: L, b: => B)(implicit val p: Parameters) extends LazyModuleImp(l) with HasCoreplexParameters {
|
||||||
|
val outer: L = l
|
||||||
|
val io: B = b
|
||||||
|
|
||||||
|
// Coreplex doesn't know when to stop running
|
||||||
|
io.success := Bool(false)
|
||||||
|
|
||||||
|
// Build a set of Tiles
|
||||||
|
val tiles = p(BuildTiles) map { _(reset, p) }
|
||||||
|
val uncoreTileIOs = (tiles zipWithIndex) map { case (tile, i) => Wire(tile.io) }
|
||||||
|
|
||||||
|
val nCachedPorts = tiles.map(tile => tile.io.cached.size).reduce(_ + _)
|
||||||
|
val nUncachedPorts = tiles.map(tile => tile.io.uncached.size).reduce(_ + _)
|
||||||
|
val nBanks = c.nMemChannels * nBanksPerMemChannel
|
||||||
|
|
||||||
|
// Build an uncore backing the Tiles
|
||||||
|
buildUncore(p.alterPartial({
|
||||||
|
case HastiId => "TL"
|
||||||
|
case TLId => "L1toL2"
|
||||||
|
case NCachedTileLinkPorts => nCachedPorts
|
||||||
|
case NUncachedTileLinkPorts => nUncachedPorts
|
||||||
|
}))
|
||||||
|
|
||||||
|
def buildUncore(implicit p: Parameters) = {
|
||||||
|
// Create a simple L1toL2 NoC between the tiles and the banks of outer memory
|
||||||
|
// Cached ports are first in client list, making sharerToClientId just an indentity function
|
||||||
|
// addrToBank is sed to hash physical addresses (of cache blocks) to banks (and thereby memory channels)
|
||||||
|
def sharerToClientId(sharerId: UInt) = sharerId
|
||||||
|
def addrToBank(addr: UInt): UInt = if (nBanks == 0) UInt(0) else {
|
||||||
|
val isMemory = globalAddrMap.isInRegion("mem", addr << log2Up(p(CacheBlockBytes)))
|
||||||
|
Mux(isMemory, addr.extract(lsb + log2Ceil(nBanks) - 1, lsb), UInt(nBanks))
|
||||||
|
}
|
||||||
|
val l1tol2net = Module(new PortedTileLinkCrossbar(addrToBank, sharerToClientId))
|
||||||
|
|
||||||
|
// Create point(s) of coherence serialization
|
||||||
|
val managerEndpoints = List.tabulate(nBanks){id => p(BuildL2CoherenceManager)(id, p)}
|
||||||
|
managerEndpoints.flatMap(_.incoherent).foreach(_ := Bool(false))
|
||||||
|
|
||||||
|
val mmioManager = Module(new MMIOTileLinkManager()(p.alterPartial({
|
||||||
|
case TLId => "L1toL2"
|
||||||
|
case InnerTLId => "L1toL2"
|
||||||
|
case OuterTLId => "L2toMMIO"
|
||||||
|
})))
|
||||||
|
|
||||||
|
// Wire the tiles to the TileLink client ports of the L1toL2 network,
|
||||||
|
// and coherence manager(s) to the other side
|
||||||
|
l1tol2net.io.clients_cached <> uncoreTileIOs.map(_.cached).flatten
|
||||||
|
l1tol2net.io.clients_uncached <> uncoreTileIOs.map(_.uncached).flatten ++ io.slave
|
||||||
|
l1tol2net.io.managers <> managerEndpoints.map(_.innerTL) :+ mmioManager.io.inner
|
||||||
|
|
||||||
|
// Create a converter between TileLinkIO and MemIO for each channel
|
||||||
|
val mem_ic = Module(new TileLinkMemoryInterconnect(nBanksPerMemChannel, c.nMemChannels)(outermostParams))
|
||||||
|
|
||||||
|
val outerTLParams = p.alterPartial({ case TLId => "L2toMC" })
|
||||||
|
val backendBuffering = TileLinkDepths(0,0,0,0,0)
|
||||||
|
for ((bank, icPort) <- managerEndpoints zip mem_ic.io.in) {
|
||||||
|
val unwrap = Module(new ClientTileLinkIOUnwrapper()(outerTLParams))
|
||||||
|
unwrap.io.in <> TileLinkEnqueuer(bank.outerTL, backendBuffering)(outerTLParams)
|
||||||
|
TileLinkWidthAdapter(icPort, unwrap.io.out)
|
||||||
|
}
|
||||||
|
|
||||||
|
io.master.mem <> mem_ic.io.out
|
||||||
|
|
||||||
|
buildMMIONetwork(TileLinkEnqueuer(mmioManager.io.outer, 1))(
|
||||||
|
p.alterPartial({case TLId => "L2toMMIO"}))
|
||||||
|
}
|
||||||
|
|
||||||
|
def buildMMIONetwork(mmio: ClientUncachedTileLinkIO)(implicit p: Parameters) = {
|
||||||
|
val ioAddrMap = globalAddrMap.subMap("io")
|
||||||
|
|
||||||
|
val mmioNetwork = Module(new TileLinkRecursiveInterconnect(1, ioAddrMap))
|
||||||
|
mmioNetwork.io.in.head <> mmio
|
||||||
|
|
||||||
|
val plic = Module(new PLIC(c.plicKey))
|
||||||
|
plic.io.tl <> mmioNetwork.port("int:plic")
|
||||||
|
for (i <- 0 until io.interrupts.size) {
|
||||||
|
val gateway = Module(new LevelGateway)
|
||||||
|
gateway.io.interrupt := io.interrupts(i)
|
||||||
|
plic.io.devices(i) <> gateway.io.plic
|
||||||
|
}
|
||||||
|
|
||||||
|
val debugModule = Module(new DebugModule)
|
||||||
|
debugModule.io.tl <> mmioNetwork.port("int:debug")
|
||||||
|
debugModule.io.db <> io.debug
|
||||||
|
|
||||||
|
// connect coreplex-internal interrupts to tiles
|
||||||
|
for ((tile, i) <- (uncoreTileIOs zipWithIndex)) {
|
||||||
|
tile.interrupts := io.clint(i)
|
||||||
|
tile.interrupts.meip := plic.io.harts(plic.cfg.context(i, 'M'))
|
||||||
|
tile.interrupts.seip.foreach(_ := plic.io.harts(plic.cfg.context(i, 'S')))
|
||||||
|
tile.interrupts.debug := debugModule.io.debugInterrupts(i)
|
||||||
|
tile.hartid := i
|
||||||
|
tile.resetVector := io.resetVector
|
||||||
|
}
|
||||||
|
|
||||||
|
val tileSlavePorts = (0 until c.nTiles) map (i => s"int:dmem$i") filter (ioAddrMap contains _)
|
||||||
|
for ((t, m) <- (uncoreTileIOs.map(_.slave).flatten) zip (tileSlavePorts map (mmioNetwork port _)))
|
||||||
|
t <> m
|
||||||
|
|
||||||
|
io.master.mmio.foreach { _ <> mmioNetwork.port("ext") }
|
||||||
|
}
|
||||||
|
}
|
@ -4,165 +4,81 @@ import Chisel._
|
|||||||
import cde.{Parameters, Field}
|
import cde.{Parameters, Field}
|
||||||
import junctions._
|
import junctions._
|
||||||
import uncore.tilelink._
|
import uncore.tilelink._
|
||||||
import uncore.coherence._
|
|
||||||
import uncore.agents._
|
|
||||||
import uncore.devices._
|
|
||||||
import uncore.util._
|
import uncore.util._
|
||||||
import uncore.converters._
|
|
||||||
import rocket._
|
import rocket._
|
||||||
import rocket.Util._
|
|
||||||
|
|
||||||
/** Number of memory channels */
|
trait DirectConnection {
|
||||||
case object NMemoryChannels extends Field[Int]
|
val tiles: Seq[Tile]
|
||||||
/** Number of banks per memory channel */
|
val uncoreTileIOs: Seq[TileIO]
|
||||||
case object NBanksPerMemoryChannel extends Field[Int]
|
|
||||||
/** Least significant bit of address used for bank partitioning */
|
|
||||||
case object BankIdLSB extends Field[Int]
|
|
||||||
/** Function for building some kind of coherence manager agent */
|
|
||||||
case object BuildL2CoherenceManager extends Field[(Int, Parameters) => CoherenceAgent]
|
|
||||||
/** Function for building some kind of tile connected to a reset signal */
|
|
||||||
case object BuildTiles extends Field[Seq[(Bool, Parameters) => Tile]]
|
|
||||||
/** The file to read the BootROM contents from */
|
|
||||||
case object BootROMFile extends Field[String]
|
|
||||||
|
|
||||||
trait HasCoreplexParameters {
|
val tlBuffering = TileLinkDepths(1,1,2,2,0)
|
||||||
implicit val p: Parameters
|
val ultBuffering = UncachedTileLinkDepths(1,2)
|
||||||
lazy val nBanksPerMemChannel = p(NBanksPerMemoryChannel)
|
|
||||||
lazy val lsb = p(BankIdLSB)
|
|
||||||
lazy val innerParams = p.alterPartial({ case TLId => "L1toL2" })
|
|
||||||
lazy val outermostParams = p.alterPartial({ case TLId => "Outermost" })
|
|
||||||
lazy val outermostMMIOParams = p.alterPartial({ case TLId => "MMIO_Outermost" })
|
|
||||||
lazy val globalAddrMap = p(rocketchip.GlobalAddrMap)
|
|
||||||
}
|
|
||||||
|
|
||||||
case class CoreplexConfig(
|
(tiles zip uncoreTileIOs) foreach { case (tile, uncore) =>
|
||||||
nTiles: Int,
|
(uncore.cached zip tile.io.cached) foreach { case (u, t) => u <> TileLinkEnqueuer(t, tlBuffering)(t.p) }
|
||||||
nExtInterrupts: Int,
|
(uncore.uncached zip tile.io.uncached) foreach { case (u, t) => u <> TileLinkEnqueuer(t, ultBuffering)(t.p) }
|
||||||
nSlaves: Int,
|
tile.io.slave.foreach { _ <> TileLinkEnqueuer(uncore.slave.get, 1)(uncore.slave.get.p) }
|
||||||
nMemChannels: Int,
|
|
||||||
hasSupervisor: Boolean,
|
|
||||||
hasExtMMIOPort: Boolean)
|
|
||||||
{
|
|
||||||
val plicKey = PLICConfig(nTiles, hasSupervisor, nExtInterrupts, 0)
|
|
||||||
}
|
|
||||||
|
|
||||||
abstract class Coreplex(implicit val p: Parameters, implicit val c: CoreplexConfig) extends Module
|
tile.io.interrupts <> uncore.interrupts
|
||||||
with HasCoreplexParameters {
|
|
||||||
class CoreplexIO(implicit val p: Parameters, implicit val c: CoreplexConfig) extends Bundle {
|
|
||||||
val master = new Bundle {
|
|
||||||
val mem = Vec(c.nMemChannels, new ClientUncachedTileLinkIO()(outermostParams))
|
|
||||||
val mmio = c.hasExtMMIOPort.option(new ClientUncachedTileLinkIO()(outermostMMIOParams))
|
|
||||||
}
|
|
||||||
val slave = Vec(c.nSlaves, new ClientUncachedTileLinkIO()(innerParams)).flip
|
|
||||||
val interrupts = Vec(c.nExtInterrupts, Bool()).asInput
|
|
||||||
val debug = new DebugBusIO()(p).flip
|
|
||||||
val clint = Vec(c.nTiles, new CoreplexLocalInterrupts).asInput
|
|
||||||
val success = Bool(OUTPUT)
|
|
||||||
val resetVector = UInt(INPUT, p(XLen))
|
|
||||||
}
|
|
||||||
|
|
||||||
val io = new CoreplexIO
|
tile.io.hartid := uncore.hartid
|
||||||
}
|
tile.io.resetVector := uncore.resetVector
|
||||||
|
|
||||||
class DefaultCoreplex(tp: Parameters, tc: CoreplexConfig) extends Coreplex()(tp, tc) {
|
|
||||||
// Coreplex doesn't know when to stop running
|
|
||||||
io.success := Bool(false)
|
|
||||||
|
|
||||||
// Build a set of Tiles
|
|
||||||
val tileResets = Wire(Vec(tc.nTiles, Bool()))
|
|
||||||
val tileList = p(BuildTiles).zip(tileResets).map {
|
|
||||||
case (tile, rst) => tile(rst, p)
|
|
||||||
}
|
|
||||||
val nCachedPorts = tileList.map(tile => tile.io.cached.size).reduce(_ + _)
|
|
||||||
val nUncachedPorts = tileList.map(tile => tile.io.uncached.size).reduce(_ + _)
|
|
||||||
val nBanks = tc.nMemChannels * nBanksPerMemChannel
|
|
||||||
|
|
||||||
// Build an uncore backing the Tiles
|
|
||||||
buildUncore(p.alterPartial({
|
|
||||||
case HastiId => "TL"
|
|
||||||
case TLId => "L1toL2"
|
|
||||||
case NCachedTileLinkPorts => nCachedPorts
|
|
||||||
case NUncachedTileLinkPorts => nUncachedPorts
|
|
||||||
}))
|
|
||||||
|
|
||||||
def buildUncore(implicit p: Parameters) = {
|
|
||||||
// Create a simple L1toL2 NoC between the tiles and the banks of outer memory
|
|
||||||
// Cached ports are first in client list, making sharerToClientId just an indentity function
|
|
||||||
// addrToBank is sed to hash physical addresses (of cache blocks) to banks (and thereby memory channels)
|
|
||||||
def sharerToClientId(sharerId: UInt) = sharerId
|
|
||||||
def addrToBank(addr: UInt): UInt = if (nBanks == 0) UInt(0) else {
|
|
||||||
val isMemory = globalAddrMap.isInRegion("mem", addr << log2Up(p(CacheBlockBytes)))
|
|
||||||
Mux(isMemory, addr.extract(lsb + log2Ceil(nBanks) - 1, lsb), UInt(nBanks))
|
|
||||||
}
|
|
||||||
val preBuffering = TileLinkDepths(1,1,2,2,0)
|
|
||||||
val l1tol2net = Module(new PortedTileLinkCrossbar(addrToBank, sharerToClientId, preBuffering))
|
|
||||||
|
|
||||||
// Create point(s) of coherence serialization
|
|
||||||
val managerEndpoints = List.tabulate(nBanks){id => p(BuildL2CoherenceManager)(id, p)}
|
|
||||||
managerEndpoints.flatMap(_.incoherent).foreach(_ := Bool(false))
|
|
||||||
|
|
||||||
val mmioManager = Module(new MMIOTileLinkManager()(p.alterPartial({
|
|
||||||
case TLId => "L1toL2"
|
|
||||||
case InnerTLId => "L1toL2"
|
|
||||||
case OuterTLId => "L2toMMIO"
|
|
||||||
})))
|
|
||||||
|
|
||||||
// Wire the tiles to the TileLink client ports of the L1toL2 network,
|
|
||||||
// and coherence manager(s) to the other side
|
|
||||||
l1tol2net.io.clients_cached <> tileList.map(_.io.cached).flatten
|
|
||||||
l1tol2net.io.clients_uncached <> tileList.map(_.io.uncached).flatten ++ io.slave
|
|
||||||
l1tol2net.io.managers <> managerEndpoints.map(_.innerTL) :+ mmioManager.io.inner
|
|
||||||
|
|
||||||
// Create a converter between TileLinkIO and MemIO for each channel
|
|
||||||
val mem_ic = Module(new TileLinkMemoryInterconnect(nBanksPerMemChannel, tc.nMemChannels)(outermostParams))
|
|
||||||
|
|
||||||
val outerTLParams = p.alterPartial({ case TLId => "L2toMC" })
|
|
||||||
val backendBuffering = TileLinkDepths(0,0,0,0,0)
|
|
||||||
for ((bank, icPort) <- managerEndpoints zip mem_ic.io.in) {
|
|
||||||
val unwrap = Module(new ClientTileLinkIOUnwrapper()(outerTLParams))
|
|
||||||
unwrap.io.in <> TileLinkEnqueuer(bank.outerTL, backendBuffering)(outerTLParams)
|
|
||||||
TileLinkWidthAdapter(icPort, unwrap.io.out)
|
|
||||||
}
|
|
||||||
|
|
||||||
io.master.mem <> mem_ic.io.out
|
|
||||||
|
|
||||||
buildMMIONetwork(TileLinkEnqueuer(mmioManager.io.outer, 1))(
|
|
||||||
p.alterPartial({case TLId => "L2toMMIO"}))
|
|
||||||
}
|
|
||||||
|
|
||||||
def buildMMIONetwork(mmio: ClientUncachedTileLinkIO)(implicit p: Parameters) = {
|
|
||||||
val ioAddrMap = globalAddrMap.subMap("io")
|
|
||||||
|
|
||||||
val mmioNetwork = Module(new TileLinkRecursiveInterconnect(1, ioAddrMap))
|
|
||||||
mmioNetwork.io.in.head <> mmio
|
|
||||||
|
|
||||||
val plic = Module(new PLIC(c.plicKey))
|
|
||||||
plic.io.tl <> mmioNetwork.port("int:plic")
|
|
||||||
for (i <- 0 until io.interrupts.size) {
|
|
||||||
val gateway = Module(new LevelGateway)
|
|
||||||
gateway.io.interrupt := io.interrupts(i)
|
|
||||||
plic.io.devices(i) <> gateway.io.plic
|
|
||||||
}
|
|
||||||
|
|
||||||
val debugModule = Module(new DebugModule)
|
|
||||||
debugModule.io.tl <> mmioNetwork.port("int:debug")
|
|
||||||
debugModule.io.db <> io.debug
|
|
||||||
|
|
||||||
// connect coreplex-internal interrupts to tiles
|
|
||||||
for (((tile, tileReset), i) <- (tileList zip tileResets) zipWithIndex) {
|
|
||||||
tileReset := reset // TODO should tiles be reset separately from coreplex?
|
|
||||||
tile.io.interrupts := io.clint(i)
|
|
||||||
tile.io.interrupts.meip := plic.io.harts(plic.cfg.context(i, 'M'))
|
|
||||||
tile.io.interrupts.seip.foreach(_ := plic.io.harts(plic.cfg.context(i, 'S')))
|
|
||||||
tile.io.interrupts.debug := debugModule.io.debugInterrupts(i)
|
|
||||||
tile.io.hartid := i
|
|
||||||
tile.io.resetVector := io.resetVector
|
|
||||||
}
|
|
||||||
|
|
||||||
val tileSlavePorts = (0 until tc.nTiles) map (i => s"int:dmem$i") filter (ioAddrMap contains _)
|
|
||||||
for ((t, m) <- (tileList.map(_.io.slave).flatten) zip (tileSlavePorts map (mmioNetwork port _)))
|
|
||||||
t <> TileLinkEnqueuer(m, 1)
|
|
||||||
|
|
||||||
io.master.mmio.foreach { _ <> mmioNetwork.port("ext") }
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class DefaultCoreplex(c: CoreplexConfig)(implicit p: Parameters) extends BaseCoreplex(c)(p) {
|
||||||
|
override lazy val module = Module(new DefaultCoreplexModule(c, this, new DefaultCoreplexBundle(c)(p))(p))
|
||||||
|
}
|
||||||
|
|
||||||
|
class DefaultCoreplexBundle(c: CoreplexConfig)(implicit p: Parameters) extends BaseCoreplexBundle(c)(p)
|
||||||
|
|
||||||
|
class DefaultCoreplexModule[+L <: DefaultCoreplex, +B <: DefaultCoreplexBundle](
|
||||||
|
c: CoreplexConfig, l: L, b: => B)(implicit p: Parameters) extends BaseCoreplexModule(c, l, b)(p)
|
||||||
|
with DirectConnection
|
||||||
|
|
||||||
|
/////
|
||||||
|
|
||||||
|
trait TileClockResetBundle {
|
||||||
|
val c: CoreplexConfig
|
||||||
|
val trcs = Vec(c.nTiles, new Bundle {
|
||||||
|
val clock = Clock(INPUT)
|
||||||
|
val reset = Bool(INPUT)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
trait AsyncConnection {
|
||||||
|
val io: TileClockResetBundle
|
||||||
|
val tiles: Seq[Tile]
|
||||||
|
val uncoreTileIOs: Seq[TileIO]
|
||||||
|
|
||||||
|
(tiles, uncoreTileIOs, io.trcs).zipped foreach { case (tile, uncore, trc) =>
|
||||||
|
tile.clock := trc.clock
|
||||||
|
tile.reset := trc.reset
|
||||||
|
|
||||||
|
(uncore.cached zip tile.io.cached) foreach { case (u, t) => u <> AsyncTileLinkFrom(trc.clock, trc.reset, t) }
|
||||||
|
(uncore.uncached zip tile.io.uncached) foreach { case (u, t) => u <> AsyncUTileLinkFrom(trc.clock, trc.reset, t) }
|
||||||
|
tile.io.slave.foreach { _ <> AsyncUTileLinkTo(trc.clock, trc.reset, uncore.slave.get)}
|
||||||
|
|
||||||
|
val ti = tile.io.interrupts
|
||||||
|
val ui = uncore.interrupts
|
||||||
|
ti.debug := LevelSyncTo(trc.clock, ui.debug)
|
||||||
|
ti.mtip := LevelSyncTo(trc.clock, ui.mtip)
|
||||||
|
ti.msip := LevelSyncTo(trc.clock, ui.msip)
|
||||||
|
ti.meip := LevelSyncTo(trc.clock, ui.meip)
|
||||||
|
ti.seip.foreach { _ := LevelSyncTo(trc.clock, ui.seip.get) }
|
||||||
|
|
||||||
|
tile.io.hartid := uncore.hartid
|
||||||
|
tile.io.resetVector := uncore.resetVector
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class MultiClockCoreplex(c: CoreplexConfig)(implicit p: Parameters) extends BaseCoreplex(c)(p) {
|
||||||
|
override lazy val module = Module(new MultiClockCoreplexModule(c, this, new MultiClockCoreplexBundle(c)(p))(p))
|
||||||
|
}
|
||||||
|
|
||||||
|
class MultiClockCoreplexBundle(c: CoreplexConfig)(implicit p: Parameters) extends BaseCoreplexBundle(c)(p)
|
||||||
|
with TileClockResetBundle
|
||||||
|
|
||||||
|
class MultiClockCoreplexModule[+L <: MultiClockCoreplex, +B <: MultiClockCoreplexBundle](
|
||||||
|
c: CoreplexConfig, l: L, b: => B)(implicit p: Parameters) extends BaseCoreplexModule(c, l, b)(p)
|
||||||
|
with AsyncConnection
|
||||||
|
@ -83,7 +83,7 @@ class PCIeMockupTestConfig extends Config(
|
|||||||
class WithGroundTest extends Config(
|
class WithGroundTest extends Config(
|
||||||
(pname, site, here) => pname match {
|
(pname, site, here) => pname match {
|
||||||
case BuildCoreplex =>
|
case BuildCoreplex =>
|
||||||
(p: Parameters, c: CoreplexConfig) => Module(new GroundTestCoreplex(p, c))
|
(c: CoreplexConfig, p: Parameters) => uncore.tilelink2.LazyModule(new GroundTestCoreplex(c)(p)).module
|
||||||
case TLKey("L1toL2") => {
|
case TLKey("L1toL2") => {
|
||||||
val useMEI = site(NTiles) <= 1 && site(NCachedTileLinkPorts) <= 1
|
val useMEI = site(NTiles) <= 1 && site(NCachedTileLinkPorts) <= 1
|
||||||
TileLinkParameters(
|
TileLinkParameters(
|
||||||
|
@ -2,8 +2,15 @@ package groundtest
|
|||||||
|
|
||||||
import Chisel._
|
import Chisel._
|
||||||
import cde.{Parameters}
|
import cde.{Parameters}
|
||||||
import coreplex.{CoreplexConfig, DefaultCoreplex}
|
import coreplex._
|
||||||
|
|
||||||
class GroundTestCoreplex(tp: Parameters, tc: CoreplexConfig) extends DefaultCoreplex(tp, tc) {
|
class GroundTestCoreplex(c: CoreplexConfig)(implicit p: Parameters) extends BaseCoreplex(c)(p) {
|
||||||
io.success := tileList.flatMap(_.io.elements get "success").map(_.asInstanceOf[Bool]).reduce(_&&_)
|
override lazy val module = Module(new GroundTestCoreplexModule(c, this, new GroundTestCoreplexBundle(c)(p))(p))
|
||||||
|
}
|
||||||
|
|
||||||
|
class GroundTestCoreplexBundle(c: CoreplexConfig)(implicit p: Parameters) extends BaseCoreplexBundle(c)(p)
|
||||||
|
|
||||||
|
class GroundTestCoreplexModule[+L <: GroundTestCoreplex, +B <: GroundTestCoreplexBundle](
|
||||||
|
c: CoreplexConfig, l: L, b: => B)(implicit p: Parameters) extends BaseCoreplexModule(c, l, b)(p) with DirectConnection {
|
||||||
|
io.success := tiles.flatMap(_.io.elements get "success").map(_.asInstanceOf[Bool]).reduce(_&&_)
|
||||||
}
|
}
|
||||||
|
@ -101,7 +101,7 @@ class GroundTestTile(resetSignal: Bool)
|
|||||||
extends Tile(resetSignal = resetSignal)(p)
|
extends Tile(resetSignal = resetSignal)(p)
|
||||||
with HasGroundTestParameters {
|
with HasGroundTestParameters {
|
||||||
|
|
||||||
override val io = new TileIO {
|
override val io = new TileIO(bc) {
|
||||||
val success = Bool(OUTPUT)
|
val success = Bool(OUTPUT)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -85,3 +85,50 @@ object AsyncIrrevocableFrom
|
|||||||
PostQueueIrrevocablize(AsyncDecoupledFrom(from_clock, from_reset, from_source, depth, sync))
|
PostQueueIrrevocablize(AsyncDecoupledFrom(from_clock, from_reset, from_source, depth, sync))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This helper object synchronizes a level-sensitive signal from one
|
||||||
|
* clock domain to another.
|
||||||
|
*/
|
||||||
|
object LevelSyncCrossing {
|
||||||
|
class SynchronizerBackend(sync: Int, _clock: Clock) extends Module(Some(_clock)) {
|
||||||
|
val io = new Bundle {
|
||||||
|
val in = Bool(INPUT)
|
||||||
|
val out = Bool(OUTPUT)
|
||||||
|
}
|
||||||
|
|
||||||
|
io.out := ShiftRegister(io.in, sync)
|
||||||
|
}
|
||||||
|
|
||||||
|
class SynchronizerFrontend(_clock: Clock) extends Module(Some(_clock)) {
|
||||||
|
val io = new Bundle {
|
||||||
|
val in = Bool(INPUT)
|
||||||
|
val out = Bool(OUTPUT)
|
||||||
|
}
|
||||||
|
|
||||||
|
io.out := RegNext(io.in)
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply(from_clock: Clock, to_clock: Clock, in: Bool, sync: Int = 2): Bool = {
|
||||||
|
val front = Module(new SynchronizerFrontend(from_clock))
|
||||||
|
val back = Module(new SynchronizerBackend(sync, to_clock))
|
||||||
|
|
||||||
|
front.io.in := in
|
||||||
|
back.io.in := front.io.out
|
||||||
|
back.io.out
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object LevelSyncTo {
|
||||||
|
def apply(to_clock: Clock, in: Bool, sync: Int = 2): Bool = {
|
||||||
|
val scope = AsyncScope()
|
||||||
|
LevelSyncCrossing(scope.clock, to_clock, in, sync)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object LevelSyncFrom {
|
||||||
|
def apply(from_clock: Clock, in: Bool, sync: Int = 2): Bool = {
|
||||||
|
val scope = AsyncScope()
|
||||||
|
LevelSyncCrossing(from_clock, scope.clock, in, sync)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -22,22 +22,35 @@ case class RoccParameters(
|
|||||||
nPTWPorts : Int = 0,
|
nPTWPorts : Int = 0,
|
||||||
useFPU: Boolean = false)
|
useFPU: Boolean = false)
|
||||||
|
|
||||||
|
case class TileBundleConfig(
|
||||||
|
nCachedTileLinkPorts: Int,
|
||||||
|
nUncachedTileLinkPorts: Int,
|
||||||
|
xLen: Int,
|
||||||
|
hasSlavePort: Boolean)
|
||||||
|
|
||||||
|
class TileIO(c: TileBundleConfig)(implicit p: Parameters) extends Bundle {
|
||||||
|
val cached = Vec(c.nCachedTileLinkPorts, new ClientTileLinkIO)
|
||||||
|
val uncached = Vec(c.nUncachedTileLinkPorts, new ClientUncachedTileLinkIO)
|
||||||
|
val hartid = UInt(INPUT, c.xLen)
|
||||||
|
val interrupts = new TileInterrupts().asInput
|
||||||
|
val slave = c.hasSlavePort.option(new ClientUncachedTileLinkIO().flip)
|
||||||
|
val resetVector = UInt(INPUT, c.xLen)
|
||||||
|
|
||||||
|
override def cloneType = new TileIO(c).asInstanceOf[this.type]
|
||||||
|
}
|
||||||
|
|
||||||
abstract class Tile(clockSignal: Clock = null, resetSignal: Bool = null)
|
abstract class Tile(clockSignal: Clock = null, resetSignal: Bool = null)
|
||||||
(implicit p: Parameters) extends Module(Option(clockSignal), Option(resetSignal)) {
|
(implicit p: Parameters) extends Module(Option(clockSignal), Option(resetSignal)) {
|
||||||
val nCachedTileLinkPorts = p(NCachedTileLinkPorts)
|
val nCachedTileLinkPorts = p(NCachedTileLinkPorts)
|
||||||
val nUncachedTileLinkPorts = p(NUncachedTileLinkPorts)
|
val nUncachedTileLinkPorts = p(NUncachedTileLinkPorts)
|
||||||
val dcacheParams = p.alterPartial({ case CacheName => "L1D" })
|
val dcacheParams = p.alterPartial({ case CacheName => "L1D" })
|
||||||
|
val bc = TileBundleConfig(
|
||||||
|
nCachedTileLinkPorts = nCachedTileLinkPorts,
|
||||||
|
nUncachedTileLinkPorts = nUncachedTileLinkPorts,
|
||||||
|
xLen = p(XLen),
|
||||||
|
hasSlavePort = p(DataScratchpadSize) > 0)
|
||||||
|
|
||||||
class TileIO extends Bundle {
|
val io = new TileIO(bc)
|
||||||
val cached = Vec(nCachedTileLinkPorts, new ClientTileLinkIO)
|
|
||||||
val uncached = Vec(nUncachedTileLinkPorts, new ClientUncachedTileLinkIO)
|
|
||||||
val hartid = UInt(INPUT, p(XLen))
|
|
||||||
val interrupts = new TileInterrupts().asInput
|
|
||||||
val slave = (p(DataScratchpadSize) > 0).option(new ClientUncachedTileLinkIO().flip)
|
|
||||||
val resetVector = UInt(INPUT, p(XLen))
|
|
||||||
}
|
|
||||||
|
|
||||||
val io = new TileIO
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class RocketTile(clockSignal: Clock = null, resetSignal: Bool = null)
|
class RocketTile(clockSignal: Clock = null, resetSignal: Bool = null)
|
||||||
|
@ -18,7 +18,7 @@ case object GlobalAddrMap extends Field[AddrMap]
|
|||||||
case object ConfigString extends Field[String]
|
case object ConfigString extends Field[String]
|
||||||
case object NCoreplexExtClients extends Field[Int]
|
case object NCoreplexExtClients extends Field[Int]
|
||||||
/** Function for building Coreplex */
|
/** Function for building Coreplex */
|
||||||
case object BuildCoreplex extends Field[(Parameters, CoreplexConfig) => Coreplex]
|
case object BuildCoreplex extends Field[(CoreplexConfig, Parameters) => BaseCoreplexModule[BaseCoreplex, BaseCoreplexBundle]]
|
||||||
|
|
||||||
/** Base Top with no Periphery */
|
/** Base Top with no Periphery */
|
||||||
abstract class BaseTop(q: Parameters) extends LazyModule {
|
abstract class BaseTop(q: Parameters) extends LazyModule {
|
||||||
@ -53,20 +53,22 @@ abstract class BaseTop(q: Parameters) extends LazyModule {
|
|||||||
peripheryBus.node := TLBuffer(TLWidthWidget(TLHintHandler(legacy.node), legacy.tlDataBytes))
|
peripheryBus.node := TLBuffer(TLWidthWidget(TLHintHandler(legacy.node), legacy.tlDataBytes))
|
||||||
}
|
}
|
||||||
|
|
||||||
class BaseTopBundle(val p: Parameters, val c: Coreplex) extends ParameterizedBundle()(p) {
|
abstract class BaseTopBundle(val p: Parameters) extends Bundle {
|
||||||
val success = Bool(OUTPUT)
|
val success = Bool(OUTPUT)
|
||||||
}
|
}
|
||||||
|
|
||||||
abstract class BaseTopModule[+L <: BaseTop, +B <: BaseTopBundle](val p: Parameters, l: L, b: Coreplex => B) extends LazyModuleImp(l) {
|
abstract class BaseTopModule[+L <: BaseTop, +B <: BaseTopBundle](
|
||||||
|
val p: Parameters, l: L, b: => B) extends LazyModuleImp(l) {
|
||||||
val outer: L = l
|
val outer: L = l
|
||||||
|
val io: B = b
|
||||||
|
|
||||||
val coreplex = p(BuildCoreplex)(p, outer.c)
|
val coreplex = p(BuildCoreplex)(outer.c, p)
|
||||||
val io: B = b(coreplex)
|
val coreplexIO = coreplex.io
|
||||||
|
|
||||||
val mmioNetwork =
|
val mmioNetwork =
|
||||||
Module(new TileLinkRecursiveInterconnect(1, p(GlobalAddrMap).subMap("io:ext"))(
|
Module(new TileLinkRecursiveInterconnect(1, p(GlobalAddrMap).subMap("io:ext"))(
|
||||||
p.alterPartial({ case TLId => "L2toMMIO" })))
|
p.alterPartial({ case TLId => "L2toMMIO" })))
|
||||||
mmioNetwork.io.in.head <> coreplex.io.master.mmio.get
|
mmioNetwork.io.in.head <> coreplexIO.master.mmio.get
|
||||||
outer.legacy.module.io.legacy <> mmioNetwork.port("TL2")
|
outer.legacy.module.io.legacy <> mmioNetwork.port("TL2")
|
||||||
|
|
||||||
println("Generated Address Map")
|
println("Generated Address Map")
|
||||||
@ -86,5 +88,5 @@ abstract class BaseTopModule[+L <: BaseTop, +B <: BaseTopBundle](val p: Paramete
|
|||||||
println(p(ConfigString))
|
println(p(ConfigString))
|
||||||
ConfigStringOutput.contents = Some(p(ConfigString))
|
ConfigStringOutput.contents = Some(p(ConfigString))
|
||||||
|
|
||||||
io.success := coreplex.io.success
|
io.success := coreplexIO.success
|
||||||
}
|
}
|
||||||
|
@ -40,7 +40,7 @@ class BasePlatformConfig extends Config(
|
|||||||
idBits = Dump("MEM_ID_BITS", site(MIFTagBits)))
|
idBits = Dump("MEM_ID_BITS", site(MIFTagBits)))
|
||||||
}
|
}
|
||||||
case BuildCoreplex =>
|
case BuildCoreplex =>
|
||||||
(p: Parameters, c: CoreplexConfig) => Module(new DefaultCoreplex(p, c))
|
(c: CoreplexConfig, p: Parameters) => uncore.tilelink2.LazyModule(new DefaultCoreplex(c)(p)).module
|
||||||
case NExtTopInterrupts => 2
|
case NExtTopInterrupts => 2
|
||||||
// Note that PLIC asserts that this is > 0.
|
// Note that PLIC asserts that this is > 0.
|
||||||
case AsyncDebugBus => false
|
case AsyncDebugBus => false
|
||||||
|
@ -4,33 +4,66 @@ package rocketchip
|
|||||||
|
|
||||||
import Chisel._
|
import Chisel._
|
||||||
import cde.{Parameters, Field}
|
import cde.{Parameters, Field}
|
||||||
import coreplex.Coreplex
|
import junctions._
|
||||||
|
import coreplex._
|
||||||
import rocketchip._
|
import rocketchip._
|
||||||
|
|
||||||
/** Example Top with Periphery */
|
/** Example Top with Periphery */
|
||||||
class ExampleTop(q: Parameters) extends BaseTop(q)
|
class ExampleTop(q: Parameters) extends BaseTop(q)
|
||||||
with PeripheryBootROM with PeripheryDebug with PeripheryExtInterrupts with PeripheryCoreplexLocalInterrupter
|
with PeripheryBootROM
|
||||||
with PeripheryMasterMem with PeripheryMasterMMIO with PeripherySlave {
|
with PeripheryDebug
|
||||||
override lazy val module = Module(new ExampleTopModule(p, this, new ExampleTopBundle(p, _)))
|
with PeripheryExtInterrupts
|
||||||
|
with PeripheryCoreplexLocalInterrupter
|
||||||
|
with PeripheryMasterMem
|
||||||
|
with PeripheryMasterMMIO
|
||||||
|
with PeripherySlave {
|
||||||
|
override lazy val module = Module(new ExampleTopModule(p, this, new ExampleTopBundle(p)))
|
||||||
}
|
}
|
||||||
|
|
||||||
class ExampleTopBundle(p: Parameters, c: Coreplex) extends BaseTopBundle(p, c)
|
class ExampleTopBundle(p: Parameters) extends BaseTopBundle(p)
|
||||||
with PeripheryBootROMBundle with PeripheryDebugBundle with PeripheryExtInterruptsBundle with PeripheryCoreplexLocalInterrupterBundle
|
with PeripheryBootROMBundle
|
||||||
with PeripheryMasterMemBundle with PeripheryMasterMMIOBundle with PeripherySlaveBundle
|
with PeripheryDebugBundle
|
||||||
|
with PeripheryExtInterruptsBundle
|
||||||
|
with PeripheryCoreplexLocalInterrupterBundle
|
||||||
|
with PeripheryMasterMemBundle
|
||||||
|
with PeripheryMasterMMIOBundle
|
||||||
|
with PeripherySlaveBundle
|
||||||
|
|
||||||
class ExampleTopModule[+L <: ExampleTop, +B <: ExampleTopBundle](p: Parameters, l: L, b: Coreplex => B) extends BaseTopModule(p, l, b)
|
class ExampleTopModule[+L <: ExampleTop, +B <: ExampleTopBundle](p: Parameters, l: L, b: => B) extends BaseTopModule(p, l, b)
|
||||||
with PeripheryBootROMModule with PeripheryDebugModule with PeripheryExtInterruptsModule with PeripheryCoreplexLocalInterrupterModule
|
with PeripheryBootROMModule
|
||||||
with PeripheryMasterMemModule with PeripheryMasterMMIOModule with PeripherySlaveModule
|
with PeripheryDebugModule
|
||||||
|
with PeripheryExtInterruptsModule
|
||||||
|
with PeripheryCoreplexLocalInterrupterModule
|
||||||
|
with PeripheryMasterMemModule
|
||||||
|
with PeripheryMasterMMIOModule
|
||||||
|
with PeripherySlaveModule
|
||||||
with HardwiredResetVector
|
with HardwiredResetVector
|
||||||
|
|
||||||
/** Example Top with TestRAM */
|
/** Example Top with TestRAM */
|
||||||
class ExampleTopWithTestRAM(q: Parameters) extends ExampleTop(q)
|
class ExampleTopWithTestRAM(q: Parameters) extends ExampleTop(q)
|
||||||
with PeripheryTestRAM {
|
with PeripheryTestRAM {
|
||||||
override lazy val module = Module(new ExampleTopWithTestRAMModule(p, this, new ExampleTopWithTestRAMBundle(p, _)))
|
override lazy val module = Module(new ExampleTopWithTestRAMModule(p, this, new ExampleTopWithTestRAMBundle(p)))
|
||||||
}
|
}
|
||||||
|
|
||||||
class ExampleTopWithTestRAMBundle(p: Parameters, c: Coreplex) extends ExampleTopBundle(p, c)
|
class ExampleTopWithTestRAMBundle(p: Parameters) extends ExampleTopBundle(p)
|
||||||
with PeripheryTestRAMBundle
|
with PeripheryTestRAMBundle
|
||||||
|
|
||||||
class ExampleTopWithTestRAMModule[+L <: ExampleTopWithTestRAM, +B <: ExampleTopWithTestRAMBundle](p: Parameters, l: L, b: Coreplex => B) extends ExampleTopModule(p, l, b)
|
class ExampleTopWithTestRAMModule[+L <: ExampleTopWithTestRAM, +B <: ExampleTopWithTestRAMBundle](p: Parameters, l: L, b: => B) extends ExampleTopModule(p, l, b)
|
||||||
with PeripheryTestRAMModule
|
with PeripheryTestRAMModule
|
||||||
|
|
||||||
|
/** Example Top with Multi Clock */
|
||||||
|
class ExampleMultiClockTop(q: Parameters) extends ExampleTop(q)
|
||||||
|
with PeripheryTestRAM {
|
||||||
|
override lazy val module = Module(new ExampleMultiClockTopModule(p, this, new ExampleMultiClockTopBundle(p)))
|
||||||
|
}
|
||||||
|
|
||||||
|
class ExampleMultiClockTopBundle(p: Parameters) extends ExampleTopBundle(p)
|
||||||
|
|
||||||
|
class ExampleMultiClockTopModule[+L <: ExampleMultiClockTop, +B <: ExampleMultiClockTopBundle](p: Parameters, l: L, b: => B) extends ExampleTopModule(p, l, b) {
|
||||||
|
val multiClockCoreplexIO = coreplexIO.asInstanceOf[MultiClockCoreplexBundle]
|
||||||
|
|
||||||
|
multiClockCoreplexIO.trcs foreach { trc =>
|
||||||
|
trc.clock := clock
|
||||||
|
trc.reset := reset
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -101,16 +101,16 @@ trait PeripheryDebugModule {
|
|||||||
implicit val p: Parameters
|
implicit val p: Parameters
|
||||||
val outer: PeripheryDebug
|
val outer: PeripheryDebug
|
||||||
val io: PeripheryDebugBundle
|
val io: PeripheryDebugBundle
|
||||||
val coreplex: Coreplex
|
val coreplexIO: BaseCoreplexBundle
|
||||||
|
|
||||||
if (p(IncludeJtagDTM)) {
|
if (p(IncludeJtagDTM)) {
|
||||||
// JtagDTMWithSync is a wrapper which
|
// JtagDTMWithSync is a wrapper which
|
||||||
// handles the synchronization as well.
|
// handles the synchronization as well.
|
||||||
val dtm = Module (new JtagDTMWithSync()(p))
|
val dtm = Module (new JtagDTMWithSync()(p))
|
||||||
dtm.io.jtag <> io.jtag.get
|
dtm.io.jtag <> io.jtag.get
|
||||||
coreplex.io.debug <> dtm.io.debug
|
coreplexIO.debug <> dtm.io.debug
|
||||||
} else {
|
} else {
|
||||||
coreplex.io.debug <>
|
coreplexIO.debug <>
|
||||||
(if (p(AsyncDebugBus)) AsyncDebugBusFrom(io.debug_clk.get, io.debug_rst.get, io.debug.get)
|
(if (p(AsyncDebugBus)) AsyncDebugBusFrom(io.debug_clk.get, io.debug_rst.get, io.debug.get)
|
||||||
else io.debug.get)
|
else io.debug.get)
|
||||||
}
|
}
|
||||||
@ -134,12 +134,12 @@ trait PeripheryExtInterruptsModule {
|
|||||||
implicit val p: Parameters
|
implicit val p: Parameters
|
||||||
val outer: PeripheryExtInterrupts
|
val outer: PeripheryExtInterrupts
|
||||||
val io: PeripheryExtInterruptsBundle
|
val io: PeripheryExtInterruptsBundle
|
||||||
val coreplex: Coreplex
|
val coreplexIO: BaseCoreplexBundle
|
||||||
|
|
||||||
{
|
{
|
||||||
val r = outer.pInterrupts.range("ext")
|
val r = outer.pInterrupts.range("ext")
|
||||||
((r._1 until r._2) zipWithIndex) foreach { case (c, i) =>
|
((r._1 until r._2) zipWithIndex) foreach { case (c, i) =>
|
||||||
coreplex.io.interrupts(c) := io.interrupts(i)
|
coreplexIO.interrupts(c) := io.interrupts(i)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -163,10 +163,10 @@ trait PeripheryMasterMemModule extends HasPeripheryParameters {
|
|||||||
implicit val p: Parameters
|
implicit val p: Parameters
|
||||||
val outer: PeripheryMasterMem
|
val outer: PeripheryMasterMem
|
||||||
val io: PeripheryMasterMemBundle
|
val io: PeripheryMasterMemBundle
|
||||||
val coreplex: Coreplex
|
val coreplexIO: BaseCoreplexBundle
|
||||||
|
|
||||||
// Abuse the fact that zip takes the shorter of the two lists
|
// Abuse the fact that zip takes the shorter of the two lists
|
||||||
((io.mem_axi zip coreplex.io.master.mem) zipWithIndex) foreach { case ((axi, mem), idx) =>
|
((io.mem_axi zip coreplexIO.master.mem) zipWithIndex) foreach { case ((axi, mem), idx) =>
|
||||||
val axi_sync = PeripheryUtils.convertTLtoAXI(mem)(outermostParams)
|
val axi_sync = PeripheryUtils.convertTLtoAXI(mem)(outermostParams)
|
||||||
axi_sync.ar.bits.cache := CACHE_NORMAL_NOCACHE_BUF
|
axi_sync.ar.bits.cache := CACHE_NORMAL_NOCACHE_BUF
|
||||||
axi_sync.aw.bits.cache := CACHE_NORMAL_NOCACHE_BUF
|
axi_sync.aw.bits.cache := CACHE_NORMAL_NOCACHE_BUF
|
||||||
@ -176,11 +176,11 @@ trait PeripheryMasterMemModule extends HasPeripheryParameters {
|
|||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
(io.mem_ahb zip coreplex.io.master.mem) foreach { case (ahb, mem) =>
|
(io.mem_ahb zip coreplexIO.master.mem) foreach { case (ahb, mem) =>
|
||||||
ahb <> PeripheryUtils.convertTLtoAHB(mem, atomics = false)(outermostParams)
|
ahb <> PeripheryUtils.convertTLtoAHB(mem, atomics = false)(outermostParams)
|
||||||
}
|
}
|
||||||
|
|
||||||
(io.mem_tl zip coreplex.io.master.mem) foreach { case (tl, mem) =>
|
(io.mem_tl zip coreplexIO.master.mem) foreach { case (tl, mem) =>
|
||||||
tl <> TileLinkEnqueuer(mem, 2)(outermostParams)
|
tl <> TileLinkEnqueuer(mem, 2)(outermostParams)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -258,7 +258,7 @@ trait PeripherySlaveModule extends HasPeripheryParameters {
|
|||||||
implicit val p: Parameters
|
implicit val p: Parameters
|
||||||
val outer: PeripherySlave
|
val outer: PeripherySlave
|
||||||
val io: PeripherySlaveBundle
|
val io: PeripherySlaveBundle
|
||||||
val coreplex: Coreplex
|
val coreplexIO: BaseCoreplexBundle
|
||||||
|
|
||||||
if (p(NExtBusAXIChannels) > 0) {
|
if (p(NExtBusAXIChannels) > 0) {
|
||||||
val arb = Module(new NastiArbiter(p(NExtBusAXIChannels)))
|
val arb = Module(new NastiArbiter(p(NExtBusAXIChannels)))
|
||||||
@ -273,7 +273,7 @@ trait PeripherySlaveModule extends HasPeripheryParameters {
|
|||||||
|
|
||||||
val r = outer.pBusMasters.range("ext")
|
val r = outer.pBusMasters.range("ext")
|
||||||
require(r._2 - r._1 == 1, "RangeManager should return 1 slot")
|
require(r._2 - r._1 == 1, "RangeManager should return 1 slot")
|
||||||
coreplex.io.slave(r._1) <> conv.io.tl
|
coreplexIO.slave(r._1) <> conv.io.tl
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -299,10 +299,10 @@ trait PeripheryCoreplexLocalInterrupterModule extends HasPeripheryParameters {
|
|||||||
implicit val p: Parameters
|
implicit val p: Parameters
|
||||||
val outer: PeripheryCoreplexLocalInterrupter
|
val outer: PeripheryCoreplexLocalInterrupter
|
||||||
val io: PeripheryCoreplexLocalInterrupterBundle
|
val io: PeripheryCoreplexLocalInterrupterBundle
|
||||||
val coreplex: Coreplex
|
val coreplexIO: BaseCoreplexBundle
|
||||||
|
|
||||||
outer.clint.module.io.rtcTick := Counter(p(RTCPeriod)).inc()
|
outer.clint.module.io.rtcTick := Counter(p(RTCPeriod)).inc()
|
||||||
coreplex.io.clint <> outer.clint.module.io.tiles
|
coreplexIO.clint <> outer.clint.module.io.tiles
|
||||||
}
|
}
|
||||||
|
|
||||||
/////
|
/////
|
||||||
@ -371,6 +371,6 @@ trait PeripheryTestBusMasterModule {
|
|||||||
/////
|
/////
|
||||||
|
|
||||||
trait HardwiredResetVector {
|
trait HardwiredResetVector {
|
||||||
val coreplex: Coreplex
|
val coreplexIO: BaseCoreplexBundle
|
||||||
coreplex.io.resetVector := UInt(0x1000) // boot ROM
|
coreplexIO.resetVector := UInt(0x1000) // boot ROM
|
||||||
}
|
}
|
||||||
|
53
src/main/scala/uncore/tilelink/Crossing.scala
Normal file
53
src/main/scala/uncore/tilelink/Crossing.scala
Normal file
@ -0,0 +1,53 @@
|
|||||||
|
package uncore.tilelink
|
||||||
|
|
||||||
|
import Chisel._
|
||||||
|
import junctions._
|
||||||
|
|
||||||
|
object AsyncClientUncachedTileLinkCrossing {
|
||||||
|
def apply(from_clock: Clock, from_reset: Bool, from_source: ClientUncachedTileLinkIO, to_clock: Clock, to_reset: Bool, depth: Int = 8, sync: Int = 3): ClientUncachedTileLinkIO = {
|
||||||
|
val to_sink = Wire(new ClientUncachedTileLinkIO()(from_source.p))
|
||||||
|
to_sink.acquire <> AsyncDecoupledCrossing(from_clock, from_reset, from_source.acquire, to_clock, to_reset, depth, sync)
|
||||||
|
from_source.grant <> AsyncDecoupledCrossing(to_clock, to_reset, to_sink.grant, from_clock, from_reset, depth, sync)
|
||||||
|
to_sink
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object AsyncClientTileLinkCrossing {
|
||||||
|
def apply(from_clock: Clock, from_reset: Bool, from_source: ClientTileLinkIO, to_clock: Clock, to_reset: Bool, depth: Int = 8, sync: Int = 3): ClientTileLinkIO = {
|
||||||
|
val to_sink = Wire(new ClientTileLinkIO()(from_source.p))
|
||||||
|
to_sink.acquire <> AsyncDecoupledCrossing(from_clock, from_reset, from_source.acquire, to_clock, to_reset, depth, sync)
|
||||||
|
to_sink.release <> AsyncDecoupledCrossing(from_clock, from_reset, from_source.release, to_clock, to_reset, depth, sync)
|
||||||
|
to_sink.finish <> AsyncDecoupledCrossing(from_clock, from_reset, from_source.finish, to_clock, to_reset, depth, sync)
|
||||||
|
from_source.grant <> AsyncDecoupledCrossing(to_clock, to_reset, to_sink.grant, from_clock, from_reset, depth, sync)
|
||||||
|
from_source.probe <> AsyncDecoupledCrossing(to_clock, to_reset, to_sink.probe, from_clock, from_reset, depth, sync)
|
||||||
|
to_sink
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object AsyncUTileLinkTo {
|
||||||
|
def apply(to_clock: Clock, to_reset: Bool, source: ClientUncachedTileLinkIO, depth: Int = 8, sync: Int = 3): ClientUncachedTileLinkIO = {
|
||||||
|
val scope = AsyncScope()
|
||||||
|
AsyncClientUncachedTileLinkCrossing(scope.clock, scope.reset, source, to_clock, to_reset, depth, sync)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object AsyncUTileLinkFrom {
|
||||||
|
def apply(from_clock: Clock, from_reset: Bool, from_source: ClientUncachedTileLinkIO, depth: Int = 8, sync: Int = 3): ClientUncachedTileLinkIO = {
|
||||||
|
val scope = AsyncScope()
|
||||||
|
AsyncClientUncachedTileLinkCrossing(from_clock, from_reset, from_source, scope.clock, scope.reset, depth, sync)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object AsyncTileLinkTo {
|
||||||
|
def apply(to_clock: Clock, to_reset: Bool, source: ClientTileLinkIO, depth: Int = 8, sync: Int = 3): ClientTileLinkIO = {
|
||||||
|
val scope = AsyncScope()
|
||||||
|
AsyncClientTileLinkCrossing(scope.clock, scope.reset, source, to_clock, to_reset, depth, sync)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object AsyncTileLinkFrom {
|
||||||
|
def apply(from_clock: Clock, from_reset: Bool, from_source: ClientTileLinkIO, depth: Int = 8, sync: Int = 3): ClientTileLinkIO = {
|
||||||
|
val scope = AsyncScope()
|
||||||
|
AsyncClientTileLinkCrossing(from_clock, from_reset, from_source, scope.clock, scope.reset, depth, sync)
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue
Block a user