tile: BaseTile refactor, pt 1
* Make dts generation reusable across tile subclasses * First attempt to standardize tile IO nodes and connect methods * hartid => hartId when talking about scala Ints
This commit is contained in:
parent
ba6dd160a3
commit
1cd018546c
@ -45,7 +45,7 @@ class WithNBigCores(n: Int) extends Config((site, here, up) => {
|
||||
icache = Some(ICacheParams(
|
||||
rowBits = site(SystemBusKey).beatBits,
|
||||
blockBytes = site(CacheBlockBytes))))
|
||||
List.tabulate(n)(i => big.copy(hartid = i))
|
||||
List.tabulate(n)(i => big.copy(hartId = i))
|
||||
}
|
||||
})
|
||||
|
||||
@ -67,7 +67,7 @@ class WithNSmallCores(n: Int) extends Config((site, here, up) => {
|
||||
nWays = 1,
|
||||
nTLBEntries = 4,
|
||||
blockBytes = site(CacheBlockBytes))))
|
||||
List.tabulate(n)(i => small.copy(hartid = i))
|
||||
List.tabulate(n)(i => small.copy(hartId = i))
|
||||
}
|
||||
})
|
||||
|
||||
|
@ -17,7 +17,7 @@ trait HasTiles extends HasSystemBus {
|
||||
val tiles: Seq[BaseTile]
|
||||
protected def tileParams: Seq[TileParams] = tiles.map(_.tileParams)
|
||||
def nTiles: Int = tileParams.size
|
||||
def hartIdList: Seq[Int] = tileParams.map(_.hartid)
|
||||
def hartIdList: Seq[Int] = tileParams.map(_.hartId)
|
||||
def localIntCounts: Seq[Int] = tileParams.map(_.core.nLocalInterrupts)
|
||||
}
|
||||
|
||||
|
@ -110,22 +110,20 @@ trait HasRocketTiles extends HasTiles
|
||||
// are decoded from rocket.intNode inside the tile.
|
||||
|
||||
// 1. always async crossing for debug
|
||||
wrapper.intXbar.intnode := wrapper { IntSyncCrossingSink(3) } := debug.intnode
|
||||
wrapper.intInwardNode := wrapper { IntSyncCrossingSink(3) } := debug.intnode
|
||||
|
||||
// 2. clint+plic conditionally crossing
|
||||
val periphIntNode = wrapper.intXbar.intnode :=* wrapper.crossIntIn
|
||||
val periphIntNode = wrapper.intInwardNode :=* wrapper.crossIntIn
|
||||
periphIntNode := clint.intnode // msip+mtip
|
||||
periphIntNode := plic.intnode // meip
|
||||
if (tp.core.useVM) periphIntNode := plic.intnode // seip
|
||||
|
||||
// 3. local interrupts never cross
|
||||
// this.localIntNode is wired up externally // lip
|
||||
// this.intInwardNode is wired up externally // lip
|
||||
|
||||
// 4. conditional crossing from core to PLIC
|
||||
wrapper.rocket.intOutputNode.foreach { i =>
|
||||
FlipRendering { implicit p =>
|
||||
plic.intnode :=* wrapper.crossIntOut :=* i
|
||||
}
|
||||
FlipRendering { implicit p =>
|
||||
plic.intnode :=* wrapper.crossIntOut :=* wrapper.intOutwardNode
|
||||
}
|
||||
|
||||
wrapper
|
||||
|
@ -31,6 +31,25 @@ package object diplomacy
|
||||
}
|
||||
}
|
||||
|
||||
type PropertyOption = Option[(String, Seq[ResourceValue])]
|
||||
type PropertyMap = Iterable[(String, Seq[ResourceValue])]
|
||||
|
||||
implicit class IntToProperty(x: Int) {
|
||||
def asProperty: Seq[ResourceValue] = Seq(ResourceInt(BigInt(x)))
|
||||
}
|
||||
|
||||
implicit class BigIntToProperty(x: BigInt) {
|
||||
def asProperty: Seq[ResourceValue] = Seq(ResourceInt(x))
|
||||
}
|
||||
|
||||
implicit class StringToProperty(x: String) {
|
||||
def asProperty: Seq[ResourceValue] = Seq(ResourceString(x))
|
||||
}
|
||||
|
||||
implicit class DeviceToPeroperty(x: Device) {
|
||||
def asProperty: Seq[ResourceValue] = Seq(ResourceReference(x.label))
|
||||
}
|
||||
|
||||
def EnableMonitors[T](body: Parameters => T)(implicit p: Parameters) = body(p.alterPartial {
|
||||
case MonitorsEnabled => true
|
||||
})
|
||||
|
@ -7,8 +7,10 @@ import Chisel._
|
||||
import freechips.rocketchip.config._
|
||||
import freechips.rocketchip.diplomacy._
|
||||
import freechips.rocketchip.coreplex._
|
||||
import freechips.rocketchip.interrupts._
|
||||
import freechips.rocketchip.rocket.{DCache, RocketCoreParams}
|
||||
import freechips.rocketchip.tile._
|
||||
import freechips.rocketchip.tilelink._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
trait GroundTestTileParams extends TileParams {
|
||||
@ -29,8 +31,10 @@ trait GroundTestTileParams extends TileParams {
|
||||
case object GroundTestTilesKey extends Field[Seq[GroundTestTileParams]]
|
||||
|
||||
abstract class GroundTestTile(params: GroundTestTileParams)(implicit p: Parameters) extends BaseTile(params)(p) {
|
||||
val slave = None
|
||||
val localIntNode = None
|
||||
val intInwardNode: IntInwardNode = IntIdentityNode()
|
||||
val intOutwardNode: IntOutwardNode = IntIdentityNode()
|
||||
val slaveNode: TLInwardNode = TLIdentityNode()
|
||||
|
||||
val dcacheOpt = params.dcache.map { dc => LazyModule(new DCache(0)) }
|
||||
|
||||
override lazy val module = new GroundTestTileModule(this, () => new GroundTestTileBundle(this))
|
||||
|
@ -23,6 +23,7 @@ import Chisel._
|
||||
import freechips.rocketchip.config.{Field, Parameters}
|
||||
import freechips.rocketchip.rocket._
|
||||
import freechips.rocketchip.tile._
|
||||
import freechips.rocketchip.tilelink._
|
||||
import freechips.rocketchip.util._
|
||||
import scala.util.Random
|
||||
|
||||
@ -65,7 +66,7 @@ case class TraceGenParams(
|
||||
memStart: BigInt, //p(ExtMem).base
|
||||
numGens: Int) extends GroundTestTileParams {
|
||||
def build(i: Int, p: Parameters): GroundTestTile = new TraceGenTile(i, this)(p)
|
||||
val hartid = 0
|
||||
val hartId = 0
|
||||
val trace = false
|
||||
val blockerCtrlAddr = None
|
||||
}
|
||||
@ -578,6 +579,7 @@ class TraceGenerator(val params: TraceGenParams)(implicit val p: Parameters) ext
|
||||
// =======================
|
||||
|
||||
class TraceGenTile(val id: Int, val params: TraceGenParams)(implicit p: Parameters) extends GroundTestTile(params) {
|
||||
val masterNode: TLOutwardNode = dcacheOpt.map(_.node).getOrElse(TLIdentityNode())
|
||||
override lazy val module = new TraceGenTileModule(this)
|
||||
}
|
||||
|
||||
|
@ -307,19 +307,19 @@ class FrontendModule(outer: Frontend) extends LazyModuleImp(outer)
|
||||
}
|
||||
|
||||
/** Mix-ins for constructing tiles that have an ICache-based pipeline frontend */
|
||||
trait HasICacheFrontend extends CanHavePTW with HasTileLinkMasterPort {
|
||||
trait HasICacheFrontend extends CanHavePTW { this: BaseTile =>
|
||||
val module: HasICacheFrontendModule
|
||||
val frontend = LazyModule(new Frontend(tileParams.icache.get, hartid: Int))
|
||||
val hartid: Int
|
||||
tileBus.node := frontend.masterNode
|
||||
val frontend = LazyModule(new Frontend(tileParams.icache.get, hartId))
|
||||
tlMasterXbar.node := frontend.masterNode
|
||||
connectTLSlave(frontend.slaveNode, tileParams.core.fetchBytes)
|
||||
nPTWPorts += 1
|
||||
}
|
||||
|
||||
trait HasICacheFrontendBundle extends HasTileLinkMasterPortBundle {
|
||||
trait HasICacheFrontendBundle {
|
||||
val outer: HasICacheFrontend
|
||||
}
|
||||
|
||||
trait HasICacheFrontendModule extends CanHavePTWModule with HasTileLinkMasterPortModule {
|
||||
trait HasICacheFrontendModule extends CanHavePTWModule {
|
||||
val outer: HasICacheFrontend
|
||||
ptwPorts += outer.frontend.module.io.ptw
|
||||
}
|
||||
|
@ -194,25 +194,24 @@ class HellaCacheModule(outer: HellaCache) extends LazyModuleImp(outer)
|
||||
|
||||
/** Mix-ins for constructing tiles that have a HellaCache */
|
||||
|
||||
trait HasHellaCache extends HasTileLinkMasterPort with HasTileParameters {
|
||||
trait HasHellaCache extends HasTileParameters { this: BaseTile =>
|
||||
val module: HasHellaCacheModule
|
||||
implicit val p: Parameters
|
||||
def findScratchpadFromICache: Option[AddressSet]
|
||||
val hartid: Int
|
||||
var nDCachePorts = 0
|
||||
val dcache: HellaCache = LazyModule(
|
||||
if(tileParams.dcache.get.nMSHRs == 0) {
|
||||
new DCache(hartid, findScratchpadFromICache _, p(RocketCrossingKey).head.knownRatio)
|
||||
} else { new NonBlockingDCache(hartid) })
|
||||
new DCache(hartId, findScratchpadFromICache _, p(RocketCrossingKey).head.knownRatio)
|
||||
} else { new NonBlockingDCache(hartId) })
|
||||
|
||||
tileBus.node := dcache.node
|
||||
tlMasterXbar.node := dcache.node
|
||||
}
|
||||
|
||||
trait HasHellaCacheBundle extends HasTileLinkMasterPortBundle {
|
||||
trait HasHellaCacheBundle {
|
||||
val outer: HasHellaCache
|
||||
}
|
||||
|
||||
trait HasHellaCacheModule extends HasTileLinkMasterPortModule {
|
||||
trait HasHellaCacheModule {
|
||||
val outer: HasHellaCache
|
||||
//val io: HasHellaCacheBundle
|
||||
val dcachePorts = ListBuffer[HellaCacheIO]()
|
||||
|
@ -45,11 +45,11 @@ class ICacheErrors(implicit p: Parameters) extends CoreBundle()(p)
|
||||
val uncorrectable = (cacheParams.itimAddr.nonEmpty && cacheParams.dataECC.canDetect).option(Valid(UInt(width = paddrBits)))
|
||||
}
|
||||
|
||||
class ICache(val icacheParams: ICacheParams, val hartid: Int)(implicit p: Parameters) extends LazyModule {
|
||||
class ICache(val icacheParams: ICacheParams, val hartId: Int)(implicit p: Parameters) extends LazyModule {
|
||||
lazy val module = new ICacheModule(this)
|
||||
val masterNode = TLClientNode(Seq(TLClientPortParameters(Seq(TLClientParameters(
|
||||
sourceId = IdRange(0, 1 + icacheParams.prefetch.toInt), // 0=refill, 1=hint
|
||||
name = s"Core ${hartid} ICache")))))
|
||||
name = s"Core ${hartId} ICache")))))
|
||||
|
||||
val size = icacheParams.nSets * icacheParams.nWays * icacheParams.blockBytes
|
||||
val device = new SimpleDevice("itim", Seq("sifive,itim0"))
|
||||
@ -100,7 +100,7 @@ class ICacheBundle(outer: ICache) extends CoreBundle()(outer.p) {
|
||||
// get a tile-specific property without breaking deduplication
|
||||
object GetPropertyByHartId {
|
||||
def apply[T <: Data](tiles: Seq[RocketTileParams], f: RocketTileParams => Option[T], hartId: UInt): T = {
|
||||
PriorityMux(tiles.collect { case t if f(t).isDefined => (t.hartid === hartId) -> f(t).get })
|
||||
PriorityMux(tiles.collect { case t if f(t).isDefined => (t.hartId === hartId) -> f(t).get })
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -287,7 +287,7 @@ class PTW(n: Int)(implicit edge: TLEdgeOut, p: Parameters) extends CoreModule()(
|
||||
}
|
||||
|
||||
/** Mix-ins for constructing tiles that might have a PTW */
|
||||
trait CanHavePTW extends HasHellaCache {
|
||||
trait CanHavePTW extends HasHellaCache { this: BaseTile =>
|
||||
implicit val p: Parameters
|
||||
val module: CanHavePTWModule
|
||||
var nPTWPorts = 1
|
||||
|
@ -6,7 +6,6 @@ import Chisel._
|
||||
import Chisel.ImplicitConversions._
|
||||
|
||||
import freechips.rocketchip.config.Parameters
|
||||
import freechips.rocketchip.coreplex.{CacheBlockBytes, SystemBusKey}
|
||||
import freechips.rocketchip.devices.tilelink._
|
||||
import freechips.rocketchip.diplomacy._
|
||||
import freechips.rocketchip.tile._
|
||||
@ -96,19 +95,22 @@ class ScratchpadSlavePort(address: AddressSet, coreDataBytes: Int, usingAtomics:
|
||||
}
|
||||
|
||||
/** Mix-ins for constructing tiles that have optional scratchpads */
|
||||
trait CanHaveScratchpad extends HasHellaCache with HasICacheFrontend {
|
||||
trait CanHaveScratchpad extends HasHellaCache with HasICacheFrontend { this: BaseTile =>
|
||||
val module: CanHaveScratchpadModule
|
||||
val cacheBlockBytes = p(CacheBlockBytes)
|
||||
val masterPortBeatBytes = p(SystemBusKey).beatBytes
|
||||
|
||||
val intOutwardNode = IntIdentityNode()
|
||||
val slaveNode = TLIdentityNode()
|
||||
val masterNode = TLIdentityNode()
|
||||
|
||||
val scratch = tileParams.dcache.flatMap { d => d.scratch.map(s =>
|
||||
LazyModule(new ScratchpadSlavePort(AddressSet(s, d.dataScratchpadBytes-1), xBytes, tileParams.core.useAtomics && !tileParams.core.useAtomicsOnlyForIO)))
|
||||
}
|
||||
scratch.foreach(lm => connectTLSlave(lm.node, xBytes))
|
||||
|
||||
val intOutputNode = tileParams.core.tileControlAddr.map(dummy => IntIdentityNode())
|
||||
val busErrorUnit = tileParams.core.tileControlAddr map { a =>
|
||||
val beu = LazyModule(new BusErrorUnit(new L1BusErrors, BusErrorUnitParams(a)))
|
||||
intOutputNode.get := beu.intNode
|
||||
intOutwardNode := beu.intNode
|
||||
connectTLSlave(beu.node, xBytes)
|
||||
beu
|
||||
}
|
||||
|
||||
@ -117,27 +119,13 @@ trait CanHaveScratchpad extends HasHellaCache with HasICacheFrontend {
|
||||
.map(BasicBusBlockerParams(_, xBytes, masterPortBeatBytes, deadlock = true))
|
||||
.map(bp => LazyModule(new BasicBusBlocker(bp)))
|
||||
|
||||
masterNode := tile_master_blocker.map { _.node := tileBus.node } getOrElse { tileBus.node }
|
||||
tile_master_blocker.foreach(lm => connectTLSlave(lm.controlNode, xBytes))
|
||||
|
||||
// connect any combination of ITIM, DTIM, and BusErrorUnit
|
||||
val slaveNode = TLIdentityNode()
|
||||
DisableMonitors { implicit p =>
|
||||
val xbarPorts =
|
||||
scratch.map(lm => (lm.node, xBytes)) ++
|
||||
busErrorUnit.map(lm => (lm.node, xBytes)) ++
|
||||
tileParams.icache.flatMap(icache => icache.itimAddr.map(a => (frontend.slaveNode, tileParams.core.fetchBytes))) ++
|
||||
tile_master_blocker.map( lm => (lm.controlNode, xBytes))
|
||||
// TODO: this doesn't block other masters, e.g. RoCCs
|
||||
tlOtherMastersNode := tile_master_blocker.map { _.node := tlMasterXbar.node } getOrElse { tlMasterXbar.node }
|
||||
masterNode :=* tlOtherMastersNode
|
||||
tlSlaveXbar.node :*= slaveNode
|
||||
|
||||
if (xbarPorts.nonEmpty) {
|
||||
val xbar = LazyModule(new TLXbar)
|
||||
xbar.node := slaveNode
|
||||
xbarPorts.foreach { case (port, bytes) =>
|
||||
(Seq(port, TLFragmenter(bytes, cacheBlockBytes, earlyAck=EarlyAck.PutFulls))
|
||||
++ (xBytes != bytes).option(TLWidthWidget(xBytes)))
|
||||
.foldRight(xbar.node:TLOutwardNode)(_ := _)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def findScratchpadFromICache: Option[AddressSet] = scratch.map { s =>
|
||||
val finalNode = frontend.masterNode.edges.out.head.manager.managers.find(_.nodePath.last == s.node)
|
||||
@ -155,7 +143,6 @@ trait CanHaveScratchpadBundle extends HasHellaCacheBundle with HasICacheFrontend
|
||||
|
||||
trait CanHaveScratchpadModule extends HasHellaCacheModule with HasICacheFrontendModule {
|
||||
val outer: CanHaveScratchpad
|
||||
val io: CanHaveScratchpadBundle
|
||||
|
||||
outer.scratch.foreach { lm => dcachePorts += lm.module.io.dmem }
|
||||
outer.busErrorUnit.foreach { lm =>
|
||||
|
@ -5,6 +5,7 @@ package freechips.rocketchip.tile
|
||||
import Chisel._
|
||||
|
||||
import freechips.rocketchip.config._
|
||||
import freechips.rocketchip.coreplex.{CacheBlockBytes, SystemBusKey}
|
||||
import freechips.rocketchip.diplomacy._
|
||||
import freechips.rocketchip.interrupts._
|
||||
import freechips.rocketchip.rocket._
|
||||
@ -23,7 +24,7 @@ trait TileParams {
|
||||
val rocc: Seq[RoCCParams]
|
||||
val btb: Option[BTBParams]
|
||||
val trace: Boolean
|
||||
val hartid: Int
|
||||
val hartId: Int
|
||||
val blockerCtrlAddr: Option[BigInt]
|
||||
}
|
||||
|
||||
@ -63,10 +64,61 @@ trait HasTileParameters {
|
||||
def vaddrBitsExtended: Int = vpnBitsExtended + pgIdxBits
|
||||
def maxPAddrBits: Int = xLen match { case 32 => 34; case 64 => 56 }
|
||||
|
||||
def hartId: Int = tileParams.hartId
|
||||
def hartIdLen: Int = p(MaxHartIdBits)
|
||||
def resetVectorLen: Int = paddrBits
|
||||
|
||||
def cacheBlockBytes = p(CacheBlockBytes)
|
||||
def lgCacheBlockBytes = log2Up(cacheBlockBytes)
|
||||
def masterPortBeatBytes = p(SystemBusKey).beatBytes
|
||||
|
||||
def dcacheArbPorts = 1 + usingVM.toInt + usingDataScratchpad.toInt + tileParams.rocc.size
|
||||
|
||||
// TODO merge with isaString in CSR.scala
|
||||
def isaDTS: String = {
|
||||
val m = if (tileParams.core.mulDiv.nonEmpty) "m" else ""
|
||||
val a = if (tileParams.core.useAtomics) "a" else ""
|
||||
val f = if (tileParams.core.fpu.nonEmpty) "f" else ""
|
||||
val d = if (tileParams.core.fpu.nonEmpty && p(XLen) > 32) "d" else ""
|
||||
val c = if (tileParams.core.useCompressed) "c" else ""
|
||||
s"rv${p(XLen)}i$m$a$f$d$c"
|
||||
}
|
||||
|
||||
def tileProperties: PropertyMap = {
|
||||
val dcache = tileParams.dcache.filter(!_.scratch.isDefined).map(d => Map(
|
||||
"d-cache-block-size" -> cacheBlockBytes.asProperty,
|
||||
"d-cache-sets" -> d.nSets.asProperty,
|
||||
"d-cache-size" -> (d.nSets * d.nWays * cacheBlockBytes).asProperty)
|
||||
).getOrElse(Nil)
|
||||
|
||||
val incoherent = if (!tileParams.core.useAtomicsOnlyForIO) Nil else Map(
|
||||
"sifive,d-cache-incoherent" -> Nil)
|
||||
|
||||
val icache = tileParams.icache.map(i => Map(
|
||||
"i-cache-block-size" -> cacheBlockBytes.asProperty,
|
||||
"i-cache-sets" -> i.nSets.asProperty,
|
||||
"i-cache-size" -> (i.nSets * i.nWays * cacheBlockBytes).asProperty)
|
||||
).getOrElse(Nil)
|
||||
|
||||
val dtlb = tileParams.dcache.filter(_ => tileParams.core.useVM).map(d => Map(
|
||||
"d-tlb-size" -> d.nTLBEntries.asProperty,
|
||||
"d-tlb-sets" -> 1.asProperty)).getOrElse(Nil)
|
||||
|
||||
val itlb = tileParams.icache.filter(_ => tileParams.core.useVM).map(i => Map(
|
||||
"i-tlb-size" -> i.nTLBEntries.asProperty,
|
||||
"i-tlb-sets" -> 1.asProperty)).getOrElse(Nil)
|
||||
|
||||
val mmu = if (!tileParams.core.useVM) Nil else Map(
|
||||
"tlb-split" -> Nil,
|
||||
"mmu-type" -> (p(PgLevels) match {
|
||||
case 2 => "riscv,sv32"
|
||||
case 3 => "riscv,sv39"
|
||||
case 4 => "riscv,sv48"
|
||||
}).asProperty)
|
||||
|
||||
dcache ++ icache ++ dtlb ++ itlb ++ mmu ++ incoherent
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
abstract class BareTile(implicit p: Parameters) extends LazyModule
|
||||
@ -81,23 +133,6 @@ abstract class BareTileModule[+L <: BareTile, +B <: BareTileBundle[L]](_outer: L
|
||||
val io = IO(_io ())
|
||||
}
|
||||
|
||||
/** Uses TileLink master port to connect caches and accelerators to the coreplex */
|
||||
trait HasTileLinkMasterPort {
|
||||
implicit val p: Parameters
|
||||
val module: HasTileLinkMasterPortModule
|
||||
val masterNode = TLIdentityNode()
|
||||
val tileBus = LazyModule(new TLXbar) // TileBus xbar for cache backends to connect to
|
||||
}
|
||||
|
||||
trait HasTileLinkMasterPortBundle {
|
||||
val outer: HasTileLinkMasterPort
|
||||
}
|
||||
|
||||
trait HasTileLinkMasterPortModule {
|
||||
val outer: HasTileLinkMasterPort
|
||||
val io: HasTileLinkMasterPortBundle
|
||||
}
|
||||
|
||||
/** Some other standard inputs */
|
||||
trait HasExternallyDrivenTileConstants extends Bundle with HasTileParameters {
|
||||
val hartid = UInt(INPUT, hartIdLen)
|
||||
@ -112,7 +147,48 @@ trait CanHaveInstructionTracePort extends Bundle with HasTileParameters {
|
||||
abstract class BaseTile(tileParams: TileParams)(implicit p: Parameters) extends BareTile
|
||||
with HasTileParameters {
|
||||
def module: BaseTileModule[BaseTile, BaseTileBundle[BaseTile]]
|
||||
val localIntNode: Option[IntInwardNode]
|
||||
def masterNode: TLOutwardNode
|
||||
def slaveNode: TLInwardNode
|
||||
def intInwardNode: IntInwardNode
|
||||
def intOutwardNode: IntOutwardNode
|
||||
|
||||
protected val tlOtherMastersNode = TLIdentityNode()
|
||||
protected val tlMasterXbar = LazyModule(new TLXbar)
|
||||
protected val tlSlaveXbar = LazyModule(new TLXbar)
|
||||
protected val intXbar = LazyModule(new IntXbar)
|
||||
|
||||
def connectTLSlave(node: TLNode, bytes: Int) {
|
||||
DisableMonitors { implicit p =>
|
||||
(Seq(node, TLFragmenter(bytes, cacheBlockBytes, earlyAck=EarlyAck.PutFulls))
|
||||
++ (xBytes != bytes).option(TLWidthWidget(xBytes)))
|
||||
.foldRight(tlSlaveXbar.node:TLOutwardNode)(_ :*= _)
|
||||
}
|
||||
}
|
||||
|
||||
// Find resource labels for all the outward caches
|
||||
def nextLevelCacheProperty: PropertyOption = {
|
||||
val outer = tlMasterXbar.node.edges.out
|
||||
.flatMap(_.manager.managers)
|
||||
.filter(_.supportsAcquireB)
|
||||
.flatMap(_.resources.headOption)
|
||||
.map(_.owner.label)
|
||||
.distinct
|
||||
if (outer.isEmpty) None
|
||||
else Some("next-level-cache" -> outer.map(l => ResourceReference(l)).toList)
|
||||
}
|
||||
|
||||
def toDescription(resources: ResourceBindings)(compat: String, extraProperties: PropertyMap = Nil): Description = {
|
||||
val cpuProperties: PropertyMap = Map(
|
||||
"reg" -> resources("reg").map(_.value),
|
||||
"device_type" -> "cpu".asProperty,
|
||||
"compatible" -> Seq(ResourceString(compat), ResourceString("riscv")),
|
||||
"status" -> "okay".asProperty,
|
||||
"clock-frequency" -> tileParams.core.bootFreqHz.asProperty,
|
||||
"riscv,isa" -> isaDTS.asProperty,
|
||||
"timebase-frequency" -> p(DTSTimebase).asProperty)
|
||||
|
||||
Description(s"cpus/cpu@${hartId}", (cpuProperties ++ nextLevelCacheProperty ++ tileProperties ++ extraProperties).toMap)
|
||||
}
|
||||
}
|
||||
|
||||
abstract class BaseTileBundle[+L <: BaseTile](_outer: L) extends BareTileBundle(_outer)
|
||||
@ -126,4 +202,5 @@ class BaseTileModule[+L <: BaseTile, +B <: BaseTileBundle[L]](_outer: L, _io: ()
|
||||
require(paddrBits <= maxPAddrBits)
|
||||
require(resetVectorLen <= xLen)
|
||||
require(resetVectorLen <= vaddrBitsExtended)
|
||||
require (log2Up(hartId + 1) <= hartIdLen, s"p(MaxHartIdBits) of $hartIdLen is not enough for hartid $hartId")
|
||||
}
|
||||
|
@ -13,6 +13,7 @@ case object XLen extends Field[Int]
|
||||
|
||||
// These parameters can be varied per-core
|
||||
trait CoreParams {
|
||||
val bootFreqHz: BigInt
|
||||
val useVM: Boolean
|
||||
val useUser: Boolean
|
||||
val useDebug: Boolean
|
||||
|
@ -5,6 +5,7 @@ package freechips.rocketchip.tile
|
||||
import Chisel._
|
||||
|
||||
import freechips.rocketchip.config.Parameters
|
||||
import freechips.rocketchip.diplomacy._
|
||||
import freechips.rocketchip.interrupts._
|
||||
import freechips.rocketchip.util._
|
||||
|
||||
@ -22,8 +23,30 @@ trait HasExternalInterrupts extends HasTileParameters {
|
||||
implicit val p: Parameters
|
||||
val module: HasExternalInterruptsModule
|
||||
|
||||
val intNode = IntSinkNode(IntSinkPortSimple())
|
||||
val localIntNode: Option[IntInwardNode] = None
|
||||
val intInwardNode = IntSinkNode(IntSinkPortSimple())
|
||||
|
||||
val intcDevice = new Device {
|
||||
def describe(resources: ResourceBindings): Description = {
|
||||
Description(s"cpus/cpu@$hartId/interrupt-controller", Map(
|
||||
"compatible" -> "riscv,cpu-intc".asProperty,
|
||||
"interrupt-controller" -> Nil,
|
||||
"#interrupt-cells" -> 1.asProperty))
|
||||
}
|
||||
}
|
||||
|
||||
ResourceBinding {
|
||||
Resource(intcDevice, "reg").bind(ResourceInt(BigInt(hartId)))
|
||||
|
||||
intInwardNode.edges.in.flatMap(_.source.sources).map { case s =>
|
||||
for (i <- s.range.start until s.range.end) {
|
||||
csrIntMap.lift(i).foreach { j =>
|
||||
s.resources.foreach { r =>
|
||||
r.bind(intcDevice, ResourceInt(j))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: the order of the following two functions must match, and
|
||||
// also match the order which things are connected to the
|
||||
@ -35,6 +58,7 @@ trait HasExternalInterrupts extends HasTileParameters {
|
||||
val seip = if (usingVM) Seq(9) else Nil
|
||||
List(65535, 3, 7, 11) ++ seip ++ List.tabulate(nlips)(_ + 16)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
trait HasExternalInterruptsBundle {
|
||||
@ -43,7 +67,6 @@ trait HasExternalInterruptsBundle {
|
||||
|
||||
trait HasExternalInterruptsModule {
|
||||
val outer: HasExternalInterrupts
|
||||
val io: HasExternalInterruptsBundle
|
||||
|
||||
// go from flat diplomatic Interrupts to bundled TileInterrupts
|
||||
def decodeCoreInterrupts(core: TileInterrupts) {
|
||||
@ -57,7 +80,7 @@ trait HasExternalInterruptsModule {
|
||||
|
||||
val core_ips = core.lip
|
||||
|
||||
val (interrupts, _) = outer.intNode.in(0)
|
||||
val (interrupts, _) = outer.intInwardNode.in(0)
|
||||
(async_ips ++ periph_ips ++ seip ++ core_ips).zip(interrupts).foreach { case(c, i) => c := i }
|
||||
}
|
||||
}
|
||||
|
@ -14,16 +14,13 @@ trait L1CacheParams {
|
||||
def nWays: Int
|
||||
def rowBits: Int
|
||||
def nTLBEntries: Int
|
||||
def blockBytes: Int
|
||||
def blockBytes: Int // TODO this is ignored in favor of p(CacheBlockBytes) in BaseTile
|
||||
}
|
||||
|
||||
trait HasL1CacheParameters {
|
||||
implicit val p: Parameters
|
||||
trait HasL1CacheParameters extends HasTileParameters {
|
||||
val cacheParams: L1CacheParams
|
||||
private val bundleParams = p(SharedMemoryTLEdge).bundle
|
||||
|
||||
def cacheBlockBytes = cacheParams.blockBytes
|
||||
def lgCacheBlockBytes = log2Up(cacheBlockBytes)
|
||||
def nSets = cacheParams.nSets
|
||||
def blockOffBits = lgCacheBlockBytes
|
||||
def idxBits = log2Up(cacheParams.nSets)
|
||||
|
@ -77,7 +77,7 @@ class LazyRoCCModule(outer: LazyRoCC) extends LazyModuleImp(outer) {
|
||||
|
||||
/** Mixins for including RoCC **/
|
||||
|
||||
trait HasLazyRoCC extends CanHaveSharedFPU with CanHavePTW with HasTileLinkMasterPort {
|
||||
trait HasLazyRoCC extends CanHaveSharedFPU with CanHavePTW { this: BaseTile =>
|
||||
implicit val p: Parameters
|
||||
val module: HasLazyRoCCModule
|
||||
|
||||
@ -86,8 +86,8 @@ trait HasLazyRoCC extends CanHaveSharedFPU with CanHavePTW with HasTileLinkMaste
|
||||
case RoccNPTWPorts => accelParams.nPTWPorts
|
||||
}))}
|
||||
|
||||
roccs.map(_.atlNode).foreach { atl => tileBus.node :=* atl }
|
||||
roccs.map(_.tlNode).foreach { tl => masterNode :=* tl }
|
||||
roccs.map(_.atlNode).foreach { atl => tlMasterXbar.node :=* atl }
|
||||
roccs.map(_.tlNode).foreach { tl => tlOtherMastersNode :=* tl }
|
||||
|
||||
nPTWPorts += p(BuildRoCC).map(_.nPTWPorts).foldLeft(0)(_ + _)
|
||||
nDCachePorts += roccs.size
|
||||
@ -95,8 +95,7 @@ trait HasLazyRoCC extends CanHaveSharedFPU with CanHavePTW with HasTileLinkMaste
|
||||
|
||||
trait HasLazyRoCCModule extends CanHaveSharedFPUModule
|
||||
with CanHavePTWModule
|
||||
with HasCoreParameters
|
||||
with HasTileLinkMasterPortModule {
|
||||
with HasCoreParameters {
|
||||
val outer: HasLazyRoCC
|
||||
val roccCore = Wire(new RoCCCoreIO()(outer.p))
|
||||
|
||||
|
@ -22,7 +22,7 @@ case class RocketTileParams(
|
||||
trace: Boolean = false,
|
||||
hcfOnUncorrectable: Boolean = false,
|
||||
name: Option[String] = Some("tile"),
|
||||
hartid: Int = 0,
|
||||
hartId: Int = 0,
|
||||
blockerCtrlAddr: Option[BigInt] = None,
|
||||
boundaryBuffers: Boolean = false // if synthesized with hierarchical PnR, cut feed-throughs?
|
||||
) extends TileParams {
|
||||
@ -30,117 +30,32 @@ case class RocketTileParams(
|
||||
require(dcache.isDefined)
|
||||
}
|
||||
|
||||
abstract class HartedTile(tileParams: TileParams, val hartid: Int)(implicit p: Parameters) extends BaseTile(tileParams)(p) {
|
||||
require (log2Up(hartid + 1) <= p(MaxHartIdBits), s"p(MaxHartIdBits) of ${p(MaxHartIdBits)} is not enough for hartid ${hartid}")
|
||||
}
|
||||
|
||||
class RocketTile(val rocketParams: RocketTileParams)(implicit p: Parameters) extends HartedTile(rocketParams, rocketParams.hartid)(p)
|
||||
class RocketTile(val rocketParams: RocketTileParams)(implicit p: Parameters) extends BaseTile(rocketParams)(p)
|
||||
with HasExternalInterrupts
|
||||
with HasLazyRoCC // implies CanHaveSharedFPU with CanHavePTW with HasHellaCache
|
||||
with CanHaveScratchpad { // implies CanHavePTW with HasHellaCache with HasICacheFrontend
|
||||
|
||||
nDCachePorts += 1 // core TODO dcachePorts += () => module.core.io.dmem ??
|
||||
|
||||
private def ofInt(x: Int) = Seq(ResourceInt(BigInt(x)))
|
||||
private def ofStr(x: String) = Seq(ResourceString(x))
|
||||
private def ofRef(x: Device) = Seq(ResourceReference(x.label))
|
||||
val dtimProperty = scratch.map(d => Map(
|
||||
"sifive,dtim" -> d.device.asProperty)).getOrElse(Nil)
|
||||
|
||||
val itimProperty = tileParams.icache.flatMap(_.itimAddr.map(i => Map(
|
||||
"sifive,itim" -> frontend.icache.device.asProperty))).getOrElse(Nil)
|
||||
|
||||
val cpuDevice = new Device {
|
||||
def describe(resources: ResourceBindings): Description = {
|
||||
val block = p(CacheBlockBytes)
|
||||
val m = if (rocketParams.core.mulDiv.nonEmpty) "m" else ""
|
||||
val a = if (rocketParams.core.useAtomics) "a" else ""
|
||||
val f = if (rocketParams.core.fpu.nonEmpty) "f" else ""
|
||||
val d = if (rocketParams.core.fpu.nonEmpty && p(XLen) > 32) "d" else ""
|
||||
val c = if (rocketParams.core.useCompressed) "c" else ""
|
||||
val isa = s"rv${p(XLen)}i$m$a$f$d$c"
|
||||
|
||||
val dcache = rocketParams.dcache.filter(!_.scratch.isDefined).map(d => Map(
|
||||
"d-cache-block-size" -> ofInt(block),
|
||||
"d-cache-sets" -> ofInt(d.nSets),
|
||||
"d-cache-size" -> ofInt(d.nSets * d.nWays * block))).getOrElse(Map())
|
||||
|
||||
val dtim = scratch.map(d => Map(
|
||||
"sifive,dtim" -> ofRef(d.device))).getOrElse(Map())
|
||||
|
||||
val itim = if (frontend.icache.slaveNode.edges.in.isEmpty) Map() else Map(
|
||||
"sifive,itim" -> ofRef(frontend.icache.device))
|
||||
|
||||
val incoherent = if (!rocketParams.core.useAtomicsOnlyForIO) Map() else Map(
|
||||
"sifive,d-cache-incoherent" -> Nil)
|
||||
|
||||
val icache = rocketParams.icache.map(i => Map(
|
||||
"i-cache-block-size" -> ofInt(block),
|
||||
"i-cache-sets" -> ofInt(i.nSets),
|
||||
"i-cache-size" -> ofInt(i.nSets * i.nWays * block))).getOrElse(Map())
|
||||
|
||||
val dtlb = rocketParams.dcache.filter(_ => rocketParams.core.useVM).map(d => Map(
|
||||
"d-tlb-size" -> ofInt(d.nTLBEntries),
|
||||
"d-tlb-sets" -> ofInt(1))).getOrElse(Map())
|
||||
|
||||
val itlb = rocketParams.icache.filter(_ => rocketParams.core.useVM).map(i => Map(
|
||||
"i-tlb-size" -> ofInt(i.nTLBEntries),
|
||||
"i-tlb-sets" -> ofInt(1))).getOrElse(Map())
|
||||
|
||||
val mmu = if (!rocketParams.core.useVM) Map() else Map(
|
||||
"tlb-split" -> Nil,
|
||||
"mmu-type" -> ofStr(p(PgLevels) match {
|
||||
case 2 => "riscv,sv32"
|
||||
case 3 => "riscv,sv39"
|
||||
case 4 => "riscv,sv48"
|
||||
}))
|
||||
|
||||
// Find all the caches
|
||||
val outer = masterNode.edges.out
|
||||
.flatMap(_.manager.managers)
|
||||
.filter(_.supportsAcquireB)
|
||||
.flatMap(_.resources.headOption)
|
||||
.map(_.owner.label)
|
||||
.distinct
|
||||
val nextlevel: Option[(String, Seq[ResourceValue])] =
|
||||
if (outer.isEmpty) None else
|
||||
Some("next-level-cache" -> outer.map(l => ResourceReference(l)).toList)
|
||||
|
||||
Description(s"cpus/cpu@${hartid}", Map(
|
||||
"reg" -> resources("reg").map(_.value),
|
||||
"device_type" -> ofStr("cpu"),
|
||||
"compatible" -> Seq(ResourceString("sifive,rocket0"), ResourceString("riscv")),
|
||||
"status" -> ofStr("okay"),
|
||||
"clock-frequency" -> Seq(ResourceInt(rocketParams.core.bootFreqHz)),
|
||||
"riscv,isa" -> ofStr(isa),
|
||||
"timebase-frequency" -> Seq(ResourceInt(p(DTSTimebase)))) ++ dcache ++ icache ++ nextlevel ++ mmu ++ itlb ++ dtlb ++ dtim ++ itim ++ incoherent)
|
||||
}
|
||||
}
|
||||
val intcDevice = new Device {
|
||||
def describe(resources: ResourceBindings): Description = {
|
||||
Description(s"cpus/cpu@${hartid}/interrupt-controller", Map(
|
||||
"compatible" -> ofStr("riscv,cpu-intc"),
|
||||
"interrupt-controller" -> Nil,
|
||||
"#interrupt-cells" -> ofInt(1)))
|
||||
}
|
||||
def describe(resources: ResourceBindings): Description =
|
||||
toDescription(resources)("sifive,rocket0", dtimProperty ++ itimProperty)
|
||||
}
|
||||
|
||||
ResourceBinding {
|
||||
Resource(cpuDevice, "reg").bind(ResourceInt(BigInt(hartid)))
|
||||
Resource(intcDevice, "reg").bind(ResourceInt(BigInt(hartid)))
|
||||
|
||||
intNode.edges.in.flatMap(_.source.sources).map { case s =>
|
||||
for (i <- s.range.start until s.range.end) {
|
||||
csrIntMap.lift(i).foreach { j =>
|
||||
s.resources.foreach { r =>
|
||||
r.bind(intcDevice, ResourceInt(j))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
Resource(cpuDevice, "reg").bind(ResourceInt(BigInt(hartId)))
|
||||
}
|
||||
|
||||
override lazy val module = new RocketTileModule(this)
|
||||
}
|
||||
|
||||
class RocketTileBundle(outer: RocketTile) extends BaseTileBundle(outer)
|
||||
with HasExternalInterruptsBundle
|
||||
with CanHaveScratchpadBundle
|
||||
with CanHaltAndCatchFire {
|
||||
val halt_and_catch_fire = outer.rocketParams.hcfOnUncorrectable.option(Bool(OUTPUT))
|
||||
}
|
||||
@ -203,7 +118,7 @@ class RocketTileWrapper(
|
||||
|
||||
// The buffers needed to cut feed-through paths are microarchitecture specific, so belong here
|
||||
val masterBuffer = LazyModule(new TLBuffer(BufferParams.none, BufferParams.flow, BufferParams.none, BufferParams.flow, BufferParams(1)))
|
||||
val masterNode: TLOutwardNode = crossing match {
|
||||
val masterNode = crossing match {
|
||||
case _: AsynchronousCrossing => rocket.masterNode
|
||||
case SynchronousCrossing(b) =>
|
||||
require (!params.boundaryBuffers || (b.depth >= 1 && !b.flow && !b.pipe), "Buffer misconfiguration creates feed-through paths")
|
||||
@ -229,9 +144,9 @@ class RocketTileWrapper(
|
||||
}
|
||||
}
|
||||
|
||||
val intXbar = LazyModule(new IntXbar)
|
||||
val localIntNode = Some(intXbar.intnode)
|
||||
rocket.intNode := intXbar.intnode
|
||||
rocket.intInwardNode := intXbar.intnode
|
||||
val intInwardNode = intXbar.intnode
|
||||
val intOutwardNode = rocket.intOutwardNode
|
||||
|
||||
override lazy val module = new BaseTileModule(this, () => new RocketTileWrapperBundle(this)) {
|
||||
// signals that do not change based on crossing type:
|
||||
|
@ -20,7 +20,7 @@ class TLRAM(
|
||||
val node = TLManagerNode(Seq(TLManagerPortParameters(
|
||||
Seq(TLManagerParameters(
|
||||
address = List(address) ++ errors,
|
||||
resources = resources,
|
||||
resources = device.reg("mem"),
|
||||
regionType = if (cacheable) RegionType.UNCACHED else RegionType.UNCACHEABLE,
|
||||
executable = executable,
|
||||
supportsGet = TransferSizes(1, beatBytes),
|
||||
|
Loading…
Reference in New Issue
Block a user