1
0

add multiclock support to Coreplex

This commit is contained in:
Yunsup Lee 2016-09-21 16:54:35 -07:00
parent 8e63f4a1a5
commit 7afd630d3e
12 changed files with 442 additions and 204 deletions

View 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") }
}
}

View File

@ -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

View File

@ -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(

View File

@ -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(_&&_)
} }

View File

@ -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)
} }

View File

@ -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)
}
}

View File

@ -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)

View File

@ -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
} }

View File

@ -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

View File

@ -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
}
}

View File

@ -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
} }

View 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)
}
}