1
0

Full conversion to params. Compiles but does not elaborate.

This commit is contained in:
Henry Cook 2014-08-08 12:21:57 -07:00
parent 3c329df7e7
commit f411fdcce3
7 changed files with 301 additions and 370 deletions

View File

@ -1,75 +1,18 @@
package uncore
import Chisel._
trait CacheConfig {
def sets: Int
def ways: Int
def tl: TileLinkConfiguration
def as: AddressSpaceConfiguration
def dm: Boolean
def states: Int
def lines: Int
def tagbits: Int
def idxbits: Int
def offbits: Int
def untagbits: Int
def rowbits: Int
}
case class L2CacheConfig(
val sets: Int, val ways: Int,
val nrpq: Int, val nsdq: Int,
val nReleaseTransactions: Int,
val nAcquireTransactions: Int,
val tl: TileLinkConfiguration,
val as: AddressSpaceConfiguration)
extends CoherenceAgentConfiguration
with CacheConfig
{
def states = tl.co.nMasterStates
def lines = sets*ways
def dm = ways == 1
def offbits = 0
def lineaddrbits = tl.addrBits
def idxbits = log2Up(sets)
def waybits = log2Up(ways)
def untagbits = offbits + idxbits
def tagbits = lineaddrbits - idxbits
def wordbits = 64
def wordbytes = wordbits/8
def wordoffbits = log2Up(wordbytes)
def rowbits = tl.dataBits
def rowbytes = rowbits/8
def rowoffbits = log2Up(rowbytes)
def refillcycles = tl.dataBits/(rowbits)
def statebits = log2Up(states)
def nTSHRs = nReleaseTransactions + nAcquireTransactions
require(states > 0)
require(isPow2(sets))
require(isPow2(ways)) // TODO: relax this
require(refillcycles == 1) //TODO: relax this?
}
abstract trait CacheBundle extends Bundle {
implicit val cacheconf: CacheConfig
override def clone = this.getClass.getConstructors.head.newInstance(cacheconf).asInstanceOf[this.type]
}
abstract trait L2CacheBundle extends Bundle {
implicit val l2cacheconf: L2CacheConfig
override def clone = this.getClass.getConstructors.head.newInstance(l2cacheconf).asInstanceOf[this.type]
}
trait HasL2Id extends L2CacheBundle {
val id = UInt(width = log2Up(l2cacheconf.nTSHRs))
}
trait HasL2InternalRequestState extends L2CacheBundle {
val tag_match = Bool()
val old_meta = new L2Metadata
val way_en = Bits(width = l2cacheconf.ways)
}
case object NSets extends Field[Int]
case object NWays extends Field[Int]
case object IsDM extends Field[Boolean]
case object TagBits extends Field[Int]
case object IdxBits extends Field[Int]
case object OffBits extends Field[Int]
case object UntagBits extends Field[Int]
case object WayBits extends Field[Int]
case object RowBits extends Field[Int]
case object WordBits extends Field[Int]
case object RefillCycles extends Field[Int]
case object Replacer extends Field[() => ReplacementPolicy]
abstract class ReplacementPolicy {
def way: UInt
@ -77,54 +20,53 @@ abstract class ReplacementPolicy {
def hit: Unit
}
class RandomReplacement(implicit val cacheconf: CacheConfig) extends ReplacementPolicy {
class RandomReplacement(ways: Int) extends ReplacementPolicy {
private val replace = Bool()
replace := Bool(false)
val lfsr = LFSR16(replace)
def way = if(cacheconf.dm) UInt(0) else lfsr(log2Up(cacheconf.ways)-1,0)
def way = if(ways == 1) UInt(0) else lfsr(log2Up(ways)-1,0)
def miss = replace := Bool(true)
def hit = {}
}
abstract class Metadata(implicit val cacheconf: CacheConfig) extends CacheBundle {
val tag = Bits(width = cacheconf.tagbits)
abstract class Metadata extends Bundle {
val tag = Bits(width = params(TagBits))
val coh: CoherenceMetadata
}
class MetaReadReq(implicit val cacheconf: CacheConfig) extends CacheBundle {
val idx = Bits(width = cacheconf.idxbits)
class MetaReadReq extends Bundle {
val idx = Bits(width = params(IdxBits))
}
class MetaWriteReq[T <: Metadata](gen: T)(implicit conf: CacheConfig) extends MetaReadReq {
val way_en = Bits(width = conf.ways)
class MetaWriteReq[T <: Metadata](gen: T) extends MetaReadReq {
val way_en = Bits(width = params(WayBits))
val data = gen.clone
override def clone = new MetaWriteReq(gen)(conf).asInstanceOf[this.type]
override def clone = new MetaWriteReq(gen).asInstanceOf[this.type]
}
class MetadataArray[T <: Metadata](makeRstVal: () => T)(implicit conf: CacheConfig) extends Module {
implicit val tl = conf.tl
class MetadataArray[T <: Metadata](makeRstVal: () => T) extends Module {
val rstVal = makeRstVal()
val io = new Bundle {
val read = Decoupled(new MetaReadReq).flip
val write = Decoupled(new MetaWriteReq(rstVal.clone)).flip
val resp = Vec.fill(conf.ways){rstVal.clone.asOutput}
val resp = Vec.fill(params(NWays)){rstVal.clone.asOutput}
}
val metabits = rstVal.getWidth
val rst_cnt = Reg(init=UInt(0, log2Up(conf.sets+1)))
val rst = rst_cnt < UInt(conf.sets)
val rst_cnt = Reg(init=UInt(0, log2Up(params(NSets)+1)))
val rst = rst_cnt < UInt(params(NSets))
val waddr = Mux(rst, rst_cnt, io.write.bits.idx)
val wdata = Mux(rst, rstVal, io.write.bits.data).toBits
val wmask = Mux(rst, SInt(-1), io.write.bits.way_en)
when (rst) { rst_cnt := rst_cnt+UInt(1) }
val tag_arr = Mem(UInt(width = metabits*conf.ways), conf.sets, seqRead = true)
val tag_arr = Mem(UInt(width = metabits*params(NWays)), params(NSets), seqRead = true)
when (rst || io.write.valid) {
tag_arr.write(waddr, Fill(conf.ways, wdata), FillInterleaved(metabits, wmask))
tag_arr.write(waddr, Fill(params(NWays), wdata), FillInterleaved(metabits, wmask))
}
val tags = tag_arr(RegEnable(io.read.bits.idx, io.read.valid))
for (w <- 0 until conf.ways) {
for (w <- 0 until params(NWays)) {
val m = tags(metabits*(w+1)-1, metabits*w)
io.resp(w) := rstVal.clone.fromBits(m)
}
@ -133,62 +75,63 @@ class MetadataArray[T <: Metadata](makeRstVal: () => T)(implicit conf: CacheConf
io.write.ready := !rst
}
trait HasL2Id extends Bundle {
val id = UInt(width = log2Up(params(NTransactors)))
}
trait HasL2InternalRequestState extends Bundle {
val tag_match = Bool()
val old_meta = new L2Metadata
val way_en = Bits(width = params(NWays))
}
object L2Metadata {
def apply(tag: Bits, coh: MasterMetadata)(implicit conf: L2CacheConfig) = {
def apply(tag: Bits, coh: MasterMetadata) = {
val meta = new L2Metadata
meta.tag := tag
meta.coh := coh
meta
}
}
class L2Metadata(implicit val l2cacheconf: L2CacheConfig) extends Metadata
with L2CacheBundle {
val coh = l2cacheconf.tl.co.masterMetadataOnFlush.clone
class L2Metadata extends Metadata {
val coh = params(TLCoherence).masterMetadataOnFlush.clone
}
/*
class L3Metadata(implicit conf: L3CacheConfig) extends Metadata
with L3CacheBundle {
val coh = MixedMetaData(conf.innerTL.co, conf.outerTL.co) ?
}
*/
class L2MetaReadReq(implicit val l2cacheconf: L2CacheConfig) extends MetaReadReq
with HasL2Id {
val tag = Bits(width = l2cacheconf.tagbits)
class L2MetaReadReq extends MetaReadReq with HasL2Id {
val tag = Bits(width = params(TagBits))
}
class L2MetaWriteReq(implicit val l2cacheconf: L2CacheConfig) extends MetaWriteReq[L2Metadata](new L2Metadata)
class L2MetaWriteReq extends MetaWriteReq[L2Metadata](new L2Metadata)
with HasL2Id
class L2MetaResp(implicit val l2cacheconf: L2CacheConfig) extends L2CacheBundle
class L2MetaResp extends Bundle
with HasL2Id
with HasL2InternalRequestState
class L2MetadataArray(implicit conf: L2CacheConfig) extends Module {
implicit val tl = conf.tl
class L2MetadataArray extends Module {
val (co, ways) = (params(TLCoherence), params(NWays))
val io = new Bundle {
val read = Decoupled(new L2MetaReadReq).flip
val write = Decoupled(new L2MetaWriteReq).flip
val resp = Valid(new L2MetaResp)
}
val meta = Module(new MetadataArray(() => L2Metadata(UInt(0), tl.co.masterMetadataOnFlush)))
val meta = Module(new MetadataArray(() => L2Metadata(UInt(0), co.masterMetadataOnFlush)))
meta.io.read <> io.read
meta.io.write <> io.write
val s1_clk_en = Reg(next = io.read.fire())
val s1_tag = RegEnable(io.read.bits.tag, io.read.valid)
val s1_id = RegEnable(io.read.bits.id, io.read.valid)
def wayMap[T <: Data](f: Int => T) = Vec((0 until conf.ways).map(f))
def wayMap[T <: Data](f: Int => T) = Vec((0 until ways).map(f))
val s1_tag_eq_way = wayMap((w: Int) => meta.io.resp(w).tag === s1_tag)
val s1_tag_match_way = wayMap((w: Int) => s1_tag_eq_way(w) && tl.co.isValid(meta.io.resp(w).coh)).toBits
val s1_tag_match_way = wayMap((w: Int) => s1_tag_eq_way(w) && co.isValid(meta.io.resp(w).coh)).toBits
val s2_tag_match_way = RegEnable(s1_tag_match_way, s1_clk_en)
val s2_tag_match = s2_tag_match_way.orR
val s2_hit_coh = Mux1H(s2_tag_match_way, wayMap((w: Int) => RegEnable(meta.io.resp(w).coh, s1_clk_en)))
//val s2_hit = s2_tag_match && tl.co.isHit(s2_req.cmd, s2_hit_state) && s2_hit_state === tl.co.newStateOnHit(s2_req.cmd, s2_hit_state)
val replacer = new RandomReplacement
val replacer = params(Replacer)()
val s1_replaced_way_en = UIntToOH(replacer.way)
val s2_replaced_way_en = UIntToOH(RegEnable(replacer.way, s1_clk_en))
val s2_repl_meta = Mux1H(s2_replaced_way_en, wayMap((w: Int) =>
@ -203,23 +146,21 @@ class L2MetadataArray(implicit conf: L2CacheConfig) extends Module {
io.resp.bits.way_en := Mux(s2_tag_match, s2_tag_match_way, s2_replaced_way_en)
}
class L2DataReadReq(implicit val l2cacheconf: L2CacheConfig) extends L2CacheBundle
with HasL2Id {
val way_en = Bits(width = l2cacheconf.ways)
val addr = Bits(width = l2cacheconf.tl.addrBits)
class L2DataReadReq extends Bundle with HasL2Id {
val way_en = Bits(width = params(NWays))
val addr = Bits(width = params(TLAddrBits))
}
class L2DataWriteReq(implicit l2cacheconf: L2CacheConfig) extends L2DataReadReq {
val wmask = Bits(width = l2cacheconf.tl.writeMaskBits)
val data = Bits(width = l2cacheconf.tl.dataBits)
class L2DataWriteReq extends L2DataReadReq {
val wmask = Bits(width = params(TLWriteMaskBits))
val data = Bits(width = params(TLDataBits))
}
class L2DataResp(implicit val l2cacheconf: L2CacheConfig) extends L2CacheBundle
with HasL2Id {
val data = Bits(width = l2cacheconf.tl.dataBits)
class L2DataResp extends Bundle with HasL2Id {
val data = Bits(width = params(TLDataBits))
}
class L2DataArray(implicit conf: L2CacheConfig) extends Module {
class L2DataArray extends Module {
val io = new Bundle {
val read = Decoupled(new L2DataReadReq).flip
val write = Decoupled(new L2DataWriteReq).flip
@ -228,9 +169,9 @@ class L2DataArray(implicit conf: L2CacheConfig) extends Module {
val waddr = io.write.bits.addr
val raddr = io.read.bits.addr
val wmask = FillInterleaved(conf.wordbits, io.write.bits.wmask)
val resp = (0 until conf.ways).map { w =>
val array = Mem(Bits(width=conf.rowbits), conf.sets*conf.refillcycles, seqRead = true)
val wmask = FillInterleaved(params(WordBits), io.write.bits.wmask)
val resp = (0 until params(NWays)).map { w =>
val array = Mem(Bits(width=params(RowBits)), params(NSets)*params(RefillCycles), seqRead = true)
when (io.write.bits.way_en(w) && io.write.valid) {
array.write(waddr, io.write.bits.data, wmask)
}
@ -244,23 +185,11 @@ class L2DataArray(implicit conf: L2CacheConfig) extends Module {
io.write.ready := Bool(true)
}
class L2InternalAcquire(implicit val l2cacheconf: L2CacheConfig) extends Acquire()(l2cacheconf.tl)
with HasL2InternalRequestState
class L2HellaCache(bankId: Int) extends CoherenceAgent {
class L2InternalRelease(implicit val l2cacheconf: L2CacheConfig) extends Release()(l2cacheconf.tl)
with HasL2InternalRequestState
class InternalTileLinkIO(implicit val l2cacheconf: L2CacheConfig) extends L2CacheBundle {
implicit val (tl, ln) = (l2cacheconf.tl, l2cacheconf.tl.ln)
val acquire = new DecoupledIO(new LogicalNetworkIO(new L2InternalAcquire))
val probe = new DecoupledIO(new LogicalNetworkIO(new Probe)).flip
val release = new DecoupledIO(new LogicalNetworkIO(new L2InternalRelease))
val grant = new DecoupledIO(new LogicalNetworkIO(new Grant)).flip
val finish = new DecoupledIO(new LogicalNetworkIO(new Finish))
}
class L2HellaCache(bankId: Int)(implicit conf: L2CacheConfig) extends CoherenceAgent {
implicit val (tl, ln, co) = (conf.tl, conf.tl.ln, conf.tl.co)
require(isPow2(params(NSets)))
require(isPow2(params(NWays)))
require(params(RefillCycles) == 1)
val tshrfile = Module(new TSHRFile(bankId))
val meta = Module(new L2MetadataArray)
@ -278,12 +207,12 @@ class L2HellaCache(bankId: Int)(implicit conf: L2CacheConfig) extends CoherenceA
}
class TSHRFile(bankId: Int)(implicit conf: L2CacheConfig) extends Module {
implicit val (tl, ln, co) = (conf.tl, conf.tl.ln, conf.tl.co)
class TSHRFile(bankId: Int) extends Module {
val (co, nClients) = (params(TLCoherence), params(NClients))
val io = new Bundle {
val inner = (new TileLinkIO).flip
val outer = new UncachedTileLinkIO
val incoherent = Vec.fill(ln.nClients){Bool()}.asInput
val incoherent = Vec.fill(nClients){Bool()}.asInput
val meta_read = Decoupled(new L2MetaReadReq)
val meta_write = Decoupled(new L2MetaWriteReq)
val meta_resp = Valid(new L2MetaResp).flip
@ -305,10 +234,9 @@ class TSHRFile(bankId: Int)(implicit conf: L2CacheConfig) extends Module {
}
// Create TSHRs for outstanding transactions
val nTrackers = conf.nReleaseTransactions + conf.nAcquireTransactions
val trackerList = (0 until conf.nReleaseTransactions).map { id =>
val trackerList = (0 until params(NReleaseTransactors)).map { id =>
Module(new L2VoluntaryReleaseTracker(id, bankId))
} ++ (conf.nReleaseTransactions until nTrackers).map { id =>
} ++ (params(NReleaseTransactors) until params(NTransactors)).map { id =>
Module(new L2AcquireTracker(id, bankId))
}
@ -373,12 +301,12 @@ class TSHRFile(bankId: Int)(implicit conf: L2CacheConfig) extends Module {
}
abstract class L2XactTracker()(implicit conf: L2CacheConfig) extends Module {
implicit val (tl, ln, co) = (conf.tl, conf.tl.ln, conf.tl.co)
abstract class L2XactTracker extends Module {
val (co, nClients) = (params(TLCoherence),params(NClients))
val io = new Bundle {
val inner = (new TileLinkIO).flip
val outer = new UncachedTileLinkIO
val tile_incoherent = Bits(INPUT, ln.nClients)
val tile_incoherent = Bits(INPUT, nClients)
val has_acquire_conflict = Bool(OUTPUT)
val has_release_conflict = Bool(OUTPUT)
val meta_read = Decoupled(new L2MetaReadReq)
@ -397,11 +325,11 @@ abstract class L2XactTracker()(implicit conf: L2CacheConfig) extends Module {
}
class L2VoluntaryReleaseTracker(trackerId: Int, bankId: Int)(implicit conf: L2CacheConfig) extends L2XactTracker()(conf) {
class L2VoluntaryReleaseTracker(trackerId: Int, bankId: Int) extends L2XactTracker {
val s_idle :: s_mem :: s_ack :: s_busy :: Nil = Enum(UInt(), 4)
val state = Reg(init=s_idle)
val xact = Reg{ new Release }
val init_client_id = Reg(init=UInt(0, width = log2Up(ln.nClients)))
val init_client_id = Reg(init=UInt(0, width = log2Up(nClients)))
val new_meta = Reg(new L2Metadata)
val incoming_rel = io.inner.release.bits
@ -460,16 +388,16 @@ Mux(co.messageHasData(xact), s_mem, s_ack)
}
}
class L2AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: L2CacheConfig) extends L2XactTracker()(conf) {
class L2AcquireTracker(trackerId: Int, bankId: Int) extends L2XactTracker {
val s_idle :: s_probe :: s_mem_read :: s_mem_write :: s_make_grant :: s_busy :: Nil = Enum(UInt(), 6)
val state = Reg(init=s_idle)
val xact = Reg{ new Acquire }
val init_client_id = Reg(init=UInt(0, width = log2Up(ln.nClients)))
val init_client_id = Reg(init=UInt(0, width = log2Up(nClients)))
//TODO: Will need id reg for merged release xacts
val init_sharer_cnt = Reg(init=UInt(0, width = log2Up(ln.nClients)))
val release_count = if (ln.nClients == 1) UInt(0) else Reg(init=UInt(0, width = log2Up(ln.nClients)))
val probe_flags = Reg(init=Bits(0, width = ln.nClients))
val init_sharer_cnt = Reg(init=UInt(0, width = log2Up(nClients)))
val release_count = if(nClients == 1) UInt(0) else Reg(init=UInt(0, width = log2Up(nClients)))
val probe_flags = Reg(init=Bits(0, width = nClients))
val curr_p_id = PriorityEncoder(probe_flags)
val pending_outer_write = co.messageHasData(xact)
@ -480,12 +408,12 @@ class L2AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: L2CacheConfig
xact.addr, UInt(trackerId), c_rel.payload.data)
val outer_read = Acquire(co.getUncachedReadAcquireType, xact.addr, UInt(trackerId))
val probe_initial_flags = Bits(width = ln.nClients)
val probe_initial_flags = Bits(width = nClients)
probe_initial_flags := Bits(0)
if (ln.nClients > 1) {
if (nClients > 1) {
// issue self-probes for uncached read xacts to facilitate I$ coherence
val probe_self = Bool(true) //co.needsSelfProbe(io.inner.acquire.bits.payload)
val myflag = Mux(probe_self, Bits(0), UIntToOH(c_acq.header.src(log2Up(ln.nClients)-1,0)))
val myflag = Mux(probe_self, Bits(0), UIntToOH(c_acq.header.src(log2Up(nClients)-1,0)))
probe_initial_flags := ~(io.tile_incoherent | myflag)
}
@ -501,7 +429,7 @@ class L2AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: L2CacheConfig
io.inner.probe.valid := Bool(false)
io.inner.probe.bits.header.src := UInt(bankId)
io.inner.probe.bits.header.dst := curr_p_id
io.inner.probe.bits.payload := Probe(co.getProbeType(xact.a_type, conf.tl.co.masterMetadataOnFlush),
io.inner.probe.bits.payload := Probe(co.getProbeType(xact.a_type, co.masterMetadataOnFlush),
xact.addr,
UInt(trackerId))
@ -525,9 +453,9 @@ class L2AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: L2CacheConfig
when( io.inner.acquire.valid ) {
xact := c_acq.payload
init_client_id := c_acq.header.src
init_sharer_cnt := UInt(ln.nClients) // TODO: Broadcast only
init_sharer_cnt := UInt(nClients) // TODO: Broadcast only
probe_flags := probe_initial_flags
if(ln.nClients > 1) {
if(nClients > 1) {
release_count := PopCount(probe_initial_flags)
state := Mux(probe_initial_flags.orR, s_probe,
Mux(needs_outer_write, s_mem_write,
@ -550,7 +478,7 @@ class L2AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: L2CacheConfig
io.outer.acquire.bits.payload := outer_write_rel
when(io.outer.acquire.ready) {
io.inner.release.ready := Bool(true)
if(ln.nClients > 1) release_count := release_count - UInt(1)
if(nClients > 1) release_count := release_count - UInt(1)
when(release_count === UInt(1)) {
state := Mux(pending_outer_write, s_mem_write,
Mux(pending_outer_read, s_mem_read, s_make_grant))
@ -558,7 +486,7 @@ class L2AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: L2CacheConfig
}
} .otherwise {
io.inner.release.ready := Bool(true)
if(ln.nClients > 1) release_count := release_count - UInt(1)
if(nClients > 1) release_count := release_count - UInt(1)
when(release_count === UInt(1)) {
state := Mux(pending_outer_write, s_mem_write,
Mux(pending_outer_read, s_mem_read, s_make_grant))

View File

@ -4,6 +4,11 @@ import Chisel._
import Node._
import uncore._
case object HTIFWidth extends Field[Int]
case object HTIFNSCR extends Field[Int]
case object HTIFOffsetBits extends Field[Int]
case object HTIFNCores extends Field[Int]
class HostIO(val w: Int) extends Bundle
{
val clk = Bool(OUTPUT)
@ -20,43 +25,48 @@ class PCRReq extends Bundle
val data = Bits(width = 64)
}
class HTIFIO(ntiles: Int) extends Bundle
class HTIFIO extends Bundle
{
val reset = Bool(INPUT)
val id = UInt(INPUT, log2Up(ntiles))
val id = UInt(INPUT, log2Up(params(HTIFNCores)))
val pcr_req = Decoupled(new PCRReq).flip
val pcr_rep = Decoupled(Bits(width = 64))
val ipi_req = Decoupled(Bits(width = log2Up(ntiles)))
val ipi_req = Decoupled(Bits(width = log2Up(params(HTIFNCores))))
val ipi_rep = Decoupled(Bool()).flip
val debug_stats_pcr = Bool(OUTPUT)
// wired directly to stats register
// expected to be used to quickly indicate to testbench to do logging b/c in 'interesting' work
}
class SCRIO(n: Int) extends Bundle
class SCRIO extends Bundle
{
val rdata = Vec.fill(n){Bits(INPUT, 64)}
val rdata = Vec.fill(params(HTIFNSCR)){Bits(INPUT, 64)}
val wen = Bool(OUTPUT)
val waddr = UInt(OUTPUT, log2Up(n))
val waddr = UInt(OUTPUT, log2Up(params(HTIFNSCR)))
val wdata = Bits(OUTPUT, 64)
}
class HTIF(w: Int, pcr_RESET: Int, nSCR: Int, offsetBits: Int)(implicit conf: TileLinkConfiguration) extends Module
class HTIF(pcr_RESET: Int) extends Module
{
implicit val (ln, co) = (conf.ln, conf.co)
val nTiles = ln.nClients-1 // This HTIF is itself a TileLink client
val dataBits = params(TLDataBits)
val co = params(TLCoherence)
val w = params(HTIFWidth)
val nSCR = params(HTIFNSCR)
val offsetBits = params(HTIFOffsetBits)
val nCores = params(HTIFNCores)
val io = new Bundle {
val host = new HostIO(w)
val cpu = Vec.fill(nTiles){new HTIFIO(nTiles).flip}
val cpu = Vec.fill(nCores){new HTIFIO().flip}
val mem = new TileLinkIO
val scr = new SCRIO(nSCR)
val scr = new SCRIO
}
io.host.debug_stats_pcr := io.cpu.map(_.debug_stats_pcr).reduce(_||_)
// system is 'interesting' if any tile is 'interesting'
val short_request_bits = 64
val long_request_bits = short_request_bits + conf.dataBits
val long_request_bits = short_request_bits + dataBits
require(short_request_bits % w == 0)
val rx_count_w = 13 + log2Up(64) - log2Up(w) // data size field is 12 bits
@ -92,7 +102,7 @@ class HTIF(w: Int, pcr_RESET: Int, nSCR: Int, offsetBits: Int)(implicit conf: Ti
val cmd_readmem :: cmd_writemem :: cmd_readcr :: cmd_writecr :: cmd_ack :: cmd_nack :: Nil = Enum(UInt(), 6)
val pcr_addr = addr(io.cpu(0).pcr_req.bits.addr.width-1, 0)
val pcr_coreid = addr(log2Up(nTiles)-1+20+1,20)
val pcr_coreid = addr(log2Up(nCores)-1+20+1,20)
val pcr_wdata = packet_ram(0)
val bad_mem_packet = size(offsetBits-1-3,0).orR || addr(offsetBits-1-3,0).orR
@ -114,13 +124,13 @@ class HTIF(w: Int, pcr_RESET: Int, nSCR: Int, offsetBits: Int)(implicit conf: Ti
val mem_acked = Reg(init=Bool(false))
val mem_gxid = Reg(Bits())
val mem_gsrc = Reg(UInt(width = conf.ln.idBits))
val mem_gsrc = Reg(UInt())
val mem_needs_ack = Reg(Bool())
when (io.mem.grant.valid) {
mem_acked := Bool(true)
mem_gxid := io.mem.grant.bits.payload.master_xact_id
mem_gsrc := io.mem.grant.bits.header.src
mem_needs_ack := conf.co.requiresAckForGrant(io.mem.grant.bits.payload.g_type)
mem_needs_ack := co.requiresAckForGrant(io.mem.grant.bits.payload.g_type)
}
io.mem.grant.ready := Bool(true)
@ -168,8 +178,8 @@ class HTIF(w: Int, pcr_RESET: Int, nSCR: Int, offsetBits: Int)(implicit conf: Ti
}
var mem_req_data: Bits = null
for (i <- 0 until conf.dataBits/short_request_bits) {
val idx = UInt(i, log2Up(conf.dataBits/short_request_bits))
for (i <- 0 until dataBits/short_request_bits) {
val idx = UInt(i, log2Up(dataBits/short_request_bits))
when (state === state_mem_rresp && io.mem.grant.valid) {
packet_ram(idx) := io.mem.grant.bits.payload.data((i+1)*short_request_bits-1, i*short_request_bits)
}
@ -184,7 +194,7 @@ class HTIF(w: Int, pcr_RESET: Int, nSCR: Int, offsetBits: Int)(implicit conf: Ti
acq_q.io.deq.ready := io.mem.acquire.ready
io.mem.acquire.bits.payload := acq_q.io.deq.bits
io.mem.acquire.bits.payload.data := mem_req_data
io.mem.acquire.bits.header.src := UInt(conf.ln.nClients) // By convention HTIF is the client with the largest id
io.mem.acquire.bits.header.src := UInt(params(LNClients)) // By convention HTIF is the client with the largest id
io.mem.acquire.bits.header.dst := UInt(0) // DNC; Overwritten outside module
io.mem.finish.valid := (state === state_mem_finish) && mem_needs_ack
io.mem.finish.bits.payload.master_xact_id := mem_gxid
@ -194,7 +204,7 @@ class HTIF(w: Int, pcr_RESET: Int, nSCR: Int, offsetBits: Int)(implicit conf: Ti
val pcr_reset = UInt(pcr_RESET)(pcr_addr.getWidth-1,0)
val pcrReadData = Reg(Bits(width = io.cpu(0).pcr_rep.bits.getWidth))
for (i <- 0 until nTiles) {
for (i <- 0 until nCores) {
val my_reset = Reg(init=Bool(true))
val my_ipi = Reg(init=Bool(false))
@ -211,7 +221,7 @@ class HTIF(w: Int, pcr_RESET: Int, nSCR: Int, offsetBits: Int)(implicit conf: Ti
}
cpu.ipi_rep.valid := my_ipi
cpu.ipi_req.ready := Bool(true)
for (j <- 0 until nTiles) {
for (j <- 0 until nCores) {
when (io.cpu(j).ipi_req.valid && io.cpu(j).ipi_req.bits === UInt(i)) {
my_ipi := Bool(true)
}
@ -239,8 +249,8 @@ class HTIF(w: Int, pcr_RESET: Int, nSCR: Int, offsetBits: Int)(implicit conf: Ti
val scr_rdata = Vec.fill(io.scr.rdata.size){Bits(width = 64)}
for (i <- 0 until scr_rdata.size)
scr_rdata(i) := io.scr.rdata(i)
scr_rdata(0) := UInt(nTiles)
scr_rdata(1) := UInt((BigInt(conf.dataBits/8) << acq_q.io.enq.bits.addr.getWidth) >> 20)
scr_rdata(0) := UInt(nCores)
scr_rdata(1) := UInt((BigInt(dataBits/8) << acq_q.io.enq.bits.addr.getWidth) >> 20)
io.scr.wen := Bool(false)
io.scr.wdata := pcr_wdata

View File

@ -72,26 +72,26 @@ class BigMem[T <: Data](n: Int, preLatency: Int, postLatency: Int, leaf: Mem[UIn
io.rdata := Mux1H(rdataSel, rdataDeep)
}
class LLCDataReq(ways: Int)(implicit conf: MemoryIFConfiguration) extends MemReqCmd
class LLCDataReq(ways: Int) extends MemReqCmd
{
val way = UInt(width = log2Up(ways))
val isWriteback = Bool()
override def clone = new LLCDataReq(ways)(conf).asInstanceOf[this.type]
override def clone = new LLCDataReq(ways).asInstanceOf[this.type]
}
class LLCTagReq(ways: Int)(implicit val conf: MemoryIFConfiguration) extends HasMemAddr
class LLCTagReq(ways: Int) extends HasMemAddr
{
val way = UInt(width = log2Up(ways))
override def clone = new LLCTagReq(ways)(conf).asInstanceOf[this.type]
override def clone = new LLCTagReq(ways).asInstanceOf[this.type]
}
class LLCMSHRFile(sets: Int, ways: Int, outstanding: Int, refill_cycles: Int)(implicit conf: MemoryIFConfiguration) extends Module
class LLCMSHRFile(sets: Int, ways: Int, outstanding: Int, refill_cycles: Int) extends Module
{
val io = new Bundle {
val cpu = Decoupled(new MemReqCmd).flip
val repl_way = UInt(INPUT, log2Up(ways))
val repl_dirty = Bool(INPUT)
val repl_tag = UInt(INPUT, conf.addrBits - log2Up(sets))
val repl_tag = UInt(INPUT, params(MIFAddrBits) - log2Up(sets))
val data = Decoupled(new LLCDataReq(ways))
val tag = Decoupled(new LLCTagReq(ways))
val mem = new MemPipeIO
@ -100,14 +100,14 @@ class LLCMSHRFile(sets: Int, ways: Int, outstanding: Int, refill_cycles: Int)(im
}
class MSHR extends Bundle {
val addr = UInt(width = conf.addrBits)
val addr = UInt(width = params(MIFAddrBits))
val way = UInt(width = log2Up(ways))
val tag = io.cpu.bits.tag.clone
val refilled = Bool()
val refillCount = UInt(width = log2Up(refill_cycles))
val requested = Bool()
val old_dirty = Bool()
val old_tag = UInt(width = conf.addrBits - log2Up(sets))
val old_tag = UInt(width = params(MIFAddrBits) - log2Up(sets))
val wb_busy = Bool()
override def clone = new MSHR().asInstanceOf[this.type]
@ -183,10 +183,10 @@ class LLCMSHRFile(sets: Int, ways: Int, outstanding: Int, refill_cycles: Int)(im
io.mem_resp_way := mshr(refillId).way
}
class LLCWriteback(requestors: Int, refill_cycles: Int)(implicit conf: MemoryIFConfiguration) extends Module
class LLCWriteback(requestors: Int, refill_cycles: Int) extends Module
{
val io = new Bundle {
val req = Vec.fill(requestors){Decoupled(UInt(width = conf.addrBits)).flip }
val req = Vec.fill(requestors){Decoupled(UInt(width = params(MIFAddrBits))).flip }
val data = Vec.fill(requestors){Decoupled(new MemData).flip }
val mem = new MemPipeIO
}
@ -232,12 +232,12 @@ class LLCWriteback(requestors: Int, refill_cycles: Int)(implicit conf: MemoryIFC
io.mem.req_data.bits := io.data(who).bits
}
class LLCData(latency: Int, sets: Int, ways: Int, refill_cycles: Int, leaf: Mem[UInt])(implicit conf: MemoryIFConfiguration) extends Module
class LLCData(latency: Int, sets: Int, ways: Int, refill_cycles: Int, leaf: Mem[UInt]) extends Module
{
val io = new Bundle {
val req = Decoupled(new LLCDataReq(ways)).flip
val req_data = Decoupled(new MemData).flip
val writeback = Decoupled(UInt(width = conf.addrBits))
val writeback = Decoupled(UInt(width = params(MIFAddrBits)))
val writeback_data = Decoupled(new MemData)
val resp = Decoupled(new MemResp)
val mem_resp = Valid(new MemResp).flip
@ -245,7 +245,7 @@ class LLCData(latency: Int, sets: Int, ways: Int, refill_cycles: Int, leaf: Mem[
val mem_resp_way = UInt(INPUT, log2Up(ways))
}
val data = Module(new BigMem(sets*ways*refill_cycles, 1, latency-1, leaf)(Bits(width = conf.dataBits)))
val data = Module(new BigMem(sets*ways*refill_cycles, 1, latency-1, leaf)(Bits(width = params(MIFDataBits))))
class QEntry extends MemResp {
val isWriteback = Bool()
override def clone = new QEntry().asInstanceOf[this.type]
@ -300,7 +300,7 @@ class LLCData(latency: Int, sets: Int, ways: Int, refill_cycles: Int, leaf: Mem[
io.writeback_data.bits := q.io.deq.bits
}
class MemReqArb(n: Int, refill_cycles: Int)(implicit conf: MemoryIFConfiguration) extends Module
class MemReqArb(n: Int, refill_cycles: Int) extends Module
{
val io = new Bundle {
val cpu = Vec.fill(n){new MemIO().flip}
@ -343,16 +343,16 @@ class MemReqArb(n: Int, refill_cycles: Int)(implicit conf: MemoryIFConfiguration
when (unlock) { lock := Bool(false) }
}
abstract class DRAMSideLLCLike(implicit conf: MemoryIFConfiguration) extends Module {
abstract class DRAMSideLLCLike extends Module {
val io = new Bundle {
val cpu = new MemIO().flip
val mem = new MemPipeIO
}
}
class DRAMSideLLC(sets: Int, ways: Int, outstanding: Int, refill_cycles: Int, tagLeaf: Mem[UInt], dataLeaf: Mem[UInt])(implicit conf: MemoryIFConfiguration) extends DRAMSideLLCLike
class DRAMSideLLC(sets: Int, ways: Int, outstanding: Int, refill_cycles: Int, tagLeaf: Mem[UInt], dataLeaf: Mem[UInt]) extends DRAMSideLLCLike
{
val tagWidth = conf.addrBits - log2Up(sets)
val tagWidth = params(MIFAddrBits) - log2Up(sets)
val metaWidth = tagWidth + 2 // valid + dirty
val memCmdArb = Module(new Arbiter(new MemReqCmd, 2))
@ -499,8 +499,7 @@ object HellaQueue
}
}
class DRAMSideLLCNull(numRequests: Int, refillCycles: Int)(implicit conf: MemoryIFConfiguration)
extends DRAMSideLLCLike {
class DRAMSideLLCNull(numRequests: Int, refillCycles: Int) extends DRAMSideLLCLike {
val numEntries = numRequests * refillCycles
val size = log2Down(numEntries) + 1

View File

@ -2,57 +2,58 @@ package uncore
import Chisel._
import scala.math._
case class AddressSpaceConfiguration(paddrBits: Int, vaddrBits: Int, pgIdxBits: Int, asidBits: Int, permBits:Int) {
val ppnBits = paddrBits - pgIdxBits
val vpnBits = vaddrBits - pgIdxBits
case object PAddrBits extends Field[Int]
case object VAddrBits extends Field[Int]
case object PgIdxBits extends Field[Int]
case object ASIdBits extends Field[Int]
case object PermBits extends Field[Int]
case object PPNBits extends Field[Int]
case object VPNBits extends Field[Int]
case object MIFAddrBits extends Field[Int]
case object MIFDataBits extends Field[Int]
case object MIFTagBits extends Field[Int]
case object MIFDataBeats extends Field[Int]
trait HasMemData extends Bundle {
val data = Bits(width = params(MIFDataBits))
}
case class MemoryIFConfiguration(addrBits: Int, dataBits: Int, tagBits: Int, dataBeats: Int)
abstract trait MemoryIFSubBundle extends Bundle {
implicit val conf: MemoryIFConfiguration
override def clone = this.getClass.getConstructors.head.newInstance(conf).asInstanceOf[this.type]
trait HasMemAddr extends Bundle {
val addr = UInt(width = params(MIFAddrBits))
}
trait HasMemData extends MemoryIFSubBundle {
val data = Bits(width = conf.dataBits)
trait HasMemTag extends Bundle {
val tag = UInt(width = params(MIFTagBits))
}
trait HasMemAddr extends MemoryIFSubBundle {
val addr = UInt(width = conf.addrBits)
}
trait HasMemTag extends MemoryIFSubBundle {
val tag = UInt(width = conf.tagBits)
}
class MemReqCmd(implicit val conf: MemoryIFConfiguration) extends HasMemAddr with HasMemTag {
class MemReqCmd extends HasMemAddr with HasMemTag {
val rw = Bool()
}
class MemResp(implicit val conf: MemoryIFConfiguration) extends HasMemData with HasMemTag
class MemResp extends HasMemData with HasMemTag
class MemData(implicit val conf: MemoryIFConfiguration) extends HasMemData
class MemData extends HasMemData
class MemIO(implicit val conf: MemoryIFConfiguration) extends Bundle {
class MemIO extends Bundle {
val req_cmd = Decoupled(new MemReqCmd)
val req_data = Decoupled(new MemData)
val resp = Decoupled(new MemResp).flip
}
class MemPipeIO(implicit val conf: MemoryIFConfiguration) extends Bundle {
class MemPipeIO extends Bundle {
val req_cmd = Decoupled(new MemReqCmd)
val req_data = Decoupled(new MemData)
val resp = Valid(new MemResp).flip
}
class MemSerializedIO(w: Int)(implicit val conf: MemoryIFConfiguration) extends Bundle
class MemSerializedIO(w: Int) extends Bundle
{
val req = Decoupled(Bits(width = w))
val resp = Valid(Bits(width = w)).flip
}
class MemSerdes(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module
class MemSerdes(w: Int) extends Module
{
val io = new Bundle {
val wide = new MemIO().flip
@ -61,6 +62,7 @@ class MemSerdes(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module
val abits = io.wide.req_cmd.bits.toBits.getWidth
val dbits = io.wide.req_data.bits.toBits.getWidth
val rbits = io.wide.resp.bits.getWidth
val dbeats = params(MIFDataBeats)
val out_buf = Reg(Bits())
val in_buf = Reg(Bits())
@ -68,7 +70,7 @@ class MemSerdes(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module
val s_idle :: s_read_addr :: s_write_addr :: s_write_idle :: s_write_data :: Nil = Enum(UInt(), 5)
val state = Reg(init=s_idle)
val send_cnt = Reg(init=UInt(0, log2Up((max(abits, dbits)+w-1)/w)))
val data_send_cnt = Reg(init=UInt(0, log2Up(conf.dataBeats)))
val data_send_cnt = Reg(init=UInt(0, log2Up(dbeats)))
val adone = io.narrow.req.ready && send_cnt === UInt((abits-1)/w)
val ddone = io.narrow.req.ready && send_cnt === UInt((dbits-1)/w)
@ -104,12 +106,12 @@ class MemSerdes(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module
}
when (state === s_write_data && ddone) {
data_send_cnt := data_send_cnt + UInt(1)
state := Mux(data_send_cnt === UInt(conf.dataBeats-1), s_idle, s_write_idle)
state := Mux(data_send_cnt === UInt(dbeats-1), s_idle, s_write_idle)
send_cnt := UInt(0)
}
val recv_cnt = Reg(init=UInt(0, log2Up((rbits+w-1)/w)))
val data_recv_cnt = Reg(init=UInt(0, log2Up(conf.dataBeats)))
val data_recv_cnt = Reg(init=UInt(0, log2Up(dbeats)))
val resp_val = Reg(init=Bool(false))
resp_val := Bool(false)
@ -127,21 +129,22 @@ class MemSerdes(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module
io.wide.resp.bits := io.wide.resp.bits.fromBits(in_buf)
}
class MemDesserIO(w: Int)(implicit val conf: MemoryIFConfiguration) extends Bundle {
class MemDesserIO(w: Int) extends Bundle {
val narrow = new MemSerializedIO(w).flip
val wide = new MemIO
}
class MemDesser(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module // test rig side
class MemDesser(w: Int) extends Module // test rig side
{
val io = new MemDesserIO(w)
val abits = io.wide.req_cmd.bits.toBits.getWidth
val dbits = io.wide.req_data.bits.toBits.getWidth
val rbits = io.wide.resp.bits.getWidth
val dbeats = params(MIFDataBeats)
require(dbits >= abits && rbits >= dbits)
val recv_cnt = Reg(init=UInt(0, log2Up((rbits+w-1)/w)))
val data_recv_cnt = Reg(init=UInt(0, log2Up(conf.dataBeats)))
val data_recv_cnt = Reg(init=UInt(0, log2Up(dbeats)))
val adone = io.narrow.req.valid && recv_cnt === UInt((abits-1)/w)
val ddone = io.narrow.req.valid && recv_cnt === UInt((dbits-1)/w)
val rdone = io.narrow.resp.valid && recv_cnt === UInt((rbits-1)/w)
@ -169,13 +172,13 @@ class MemDesser(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module
}
when (state === s_data && io.wide.req_data.ready) {
state := s_data_recv
when (data_recv_cnt === UInt(conf.dataBeats-1)) {
when (data_recv_cnt === UInt(dbeats-1)) {
state := s_cmd_recv
}
data_recv_cnt := data_recv_cnt + UInt(1)
}
when (rdone) { // state === s_reply
when (data_recv_cnt === UInt(conf.dataBeats-1)) {
when (data_recv_cnt === UInt(dbeats-1)) {
state := s_cmd_recv
}
recv_cnt := UInt(0)
@ -189,7 +192,7 @@ class MemDesser(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module
io.wide.req_data.valid := state === s_data
io.wide.req_data.bits.data := in_buf >> UInt(((rbits+w-1)/w - (dbits+w-1)/w)*w)
val dataq = Module(new Queue(new MemResp, conf.dataBeats))
val dataq = Module(new Queue(new MemResp, dbeats))
dataq.io.enq <> io.wide.resp
dataq.io.deq.ready := recv_cnt === UInt((rbits-1)/w)
@ -198,18 +201,21 @@ class MemDesser(w: Int)(implicit val conf: MemoryIFConfiguration) extends Module
}
//Adapter betweewn an UncachedTileLinkIO and a mem controller MemIO
class MemIOUncachedTileLinkIOConverter(qDepth: Int)(implicit tlconf: TileLinkConfiguration, mifconf: MemoryIFConfiguration) extends Module {
class MemIOUncachedTileLinkIOConverter(qDepth: Int) extends Module {
val io = new Bundle {
val uncached = new UncachedTileLinkIO().flip
val mem = new MemIO
}
require(tlconf.dataBits == mifconf.dataBits*mifconf.dataBeats)
//require(tlconf.clientXactIdBits <= mifconf.tagBits)
val co = params(TLCoherence)
val tbits = params(MIFTagBits)
val dbits = params(MIFDataBits)
val dbeats = params(MIFDataBits)
require(params(TLDataBits) == dbits*dbeats)
//require(params(TLClientXactIdBits) <= params(MIFTagBits))
val mem_cmd_q = Module(new Queue(new MemReqCmd, qDepth))
val mem_data_q = Module(new Queue(new MemData, qDepth))
val cnt_max = mifconf.dataBeats
val cnt_max = dbeats
val cnt_out = Reg(UInt(width = log2Up(cnt_max+1)))
val active_out = Reg(init=Bool(false))
@ -222,7 +228,7 @@ class MemIOUncachedTileLinkIOConverter(qDepth: Int)(implicit tlconf: TileLinkCon
val cnt_in = Reg(UInt(width = log2Up(cnt_max+1)))
val active_in = Reg(init=Bool(false))
val buf_in = Reg(Bits())
val tag_in = Reg(UInt(width = mifconf.tagBits))
val tag_in = Reg(UInt(width = tbits))
// Decompose outgoing TL Acquires into MemIO cmd and data
when(!active_out && io.uncached.acquire.valid) {
@ -232,7 +238,7 @@ class MemIOUncachedTileLinkIOConverter(qDepth: Int)(implicit tlconf: TileLinkCon
buf_out := io.uncached.acquire.bits.payload.data
tag_out := io.uncached.acquire.bits.payload.client_xact_id
addr_out := io.uncached.acquire.bits.payload.addr
has_data := tlconf.co.messageHasData(io.uncached.acquire.bits.payload)
has_data := co.messageHasData(io.uncached.acquire.bits.payload)
}
when(active_out) {
when(mem_cmd_q.io.enq.fire()) {
@ -240,7 +246,7 @@ class MemIOUncachedTileLinkIOConverter(qDepth: Int)(implicit tlconf: TileLinkCon
}
when(mem_data_q.io.enq.fire()) {
cnt_out := cnt_out + UInt(1)
buf_out := buf_out >> UInt(mifconf.dataBits)
buf_out := buf_out >> UInt(dbits)
}
when(cmd_sent_out && (!has_data || cnt_out === UInt(cnt_max))) {
active_out := Bool(false)
@ -264,7 +270,7 @@ class MemIOUncachedTileLinkIOConverter(qDepth: Int)(implicit tlconf: TileLinkCon
when(!active_in && io.mem.resp.valid) {
active_in := Bool(true)
cnt_in := UInt(1)
buf_in := io.mem.resp.bits.data << UInt(mifconf.dataBits*(cnt_max-1))
buf_in := io.mem.resp.bits.data << UInt(dbits*(cnt_max-1))
tag_in := io.mem.resp.bits.tag
}
when(active_in) {
@ -272,7 +278,7 @@ class MemIOUncachedTileLinkIOConverter(qDepth: Int)(implicit tlconf: TileLinkCon
active_in := Bool(false)
}
when(io.mem.resp.fire()) {
buf_in := Cat(io.mem.resp.bits.data, buf_in(cnt_max*mifconf.dataBits-1,mifconf.dataBits))
buf_in := Cat(io.mem.resp.bits.data, buf_in(cnt_max*dbits-1,dbits))
cnt_in := cnt_in + UInt(1)
}
}

View File

@ -1,32 +1,35 @@
package uncore
import Chisel._
case class PhysicalNetworkConfiguration(nEndpoints: Int, idBits: Int)
case object LNMasters extends Field[Int]
case object LNClients extends Field[Int]
case object LNEndpoints extends Field[Int]
class PhysicalHeader(implicit conf: PhysicalNetworkConfiguration) extends Bundle {
val src = UInt(width = conf.idBits)
val dst = UInt(width = conf.idBits)
class PhysicalHeader(n: Int) extends Bundle {
val src = UInt(width = log2Up(n))
val dst = UInt(width = log2Up(n))
}
class PhysicalNetworkIO[T <: Data](dType: T)(implicit conf: PhysicalNetworkConfiguration) extends Bundle {
val header = new PhysicalHeader
class PhysicalNetworkIO[T <: Data](n: Int, dType: T) extends Bundle {
val header = new PhysicalHeader(n)
val payload = dType.clone
override def clone = { new PhysicalNetworkIO(dType).asInstanceOf[this.type] }
override def clone = new PhysicalNetworkIO(n,dType).asInstanceOf[this.type]
}
abstract class PhysicalNetwork(conf: PhysicalNetworkConfiguration) extends Module
class BasicCrossbarIO[T <: Data](dType: T)(implicit conf: PhysicalNetworkConfiguration) extends Bundle {
val in = Vec.fill(conf.nEndpoints){Decoupled(new PhysicalNetworkIO(dType))}.flip
val out = Vec.fill(conf.nEndpoints){Decoupled(new PhysicalNetworkIO(dType))}
class BasicCrossbarIO[T <: Data](n: Int, dType: T) extends Bundle {
val in = Vec.fill(n){Decoupled(new PhysicalNetworkIO(n,dType))}.flip
val out = Vec.fill(n){Decoupled(new PhysicalNetworkIO(n,dType))}
}
class BasicCrossbar[T <: Data](dType: T, count: Int = 1)(implicit conf: PhysicalNetworkConfiguration) extends PhysicalNetwork(conf) {
val io = new BasicCrossbarIO(dType)
val rdyVecs = List.fill(conf.nEndpoints)(Vec.fill(conf.nEndpoints)(Bool()))
abstract class PhysicalNetwork extends Module
class BasicCrossbar[T <: Data](n: Int, dType: T, count: Int = 1) extends PhysicalNetwork {
val io = new BasicCrossbarIO(n, dType)
val rdyVecs = List.fill(n){Vec.fill(n)(Bool())}
io.out.zip(rdyVecs).zipWithIndex.map{ case ((out, rdys), i) => {
val rrarb = Module(new LockingRRArbiter(io.in(0).bits, conf.nEndpoints, count))
val rrarb = Module(new LockingRRArbiter(io.in(0).bits, n, count))
(rrarb.io.in, io.in, rdys).zipped.map{ case (arb, in, rdy) => {
arb.valid := in.valid && (in.bits.header.dst === UInt(i))
arb.bits := in.bits
@ -34,30 +37,26 @@ class BasicCrossbar[T <: Data](dType: T, count: Int = 1)(implicit conf: Physical
}}
out <> rrarb.io.out
}}
for(i <- 0 until conf.nEndpoints) {
for(i <- 0 until n) {
io.in(i).ready := rdyVecs.map(r => r(i)).reduceLeft(_||_)
}
}
case class LogicalNetworkConfiguration(idBits: Int, nMasters: Int, nClients: Int) {
val nEndpoints = nMasters + nClients
abstract class LogicalNetwork extends Module
class LogicalHeader extends Bundle {
val src = UInt(width = log2Up(params(LNEndpoints)))
val dst = UInt(width = log2Up(params(LNEndpoints)))
}
abstract class LogicalNetwork[TileLinkType <: Bundle](implicit conf: LogicalNetworkConfiguration) extends Module
class LogicalHeader(implicit conf: LogicalNetworkConfiguration) extends Bundle {
val src = UInt(width = conf.idBits)
val dst = UInt(width = conf.idBits)
}
class LogicalNetworkIO[T <: Data](dType: T)(implicit conf: LogicalNetworkConfiguration) extends Bundle {
class LogicalNetworkIO[T <: Data](dType: T) extends Bundle {
val header = new LogicalHeader
val payload = dType.clone
override def clone = { new LogicalNetworkIO(dType).asInstanceOf[this.type] }
}
object DecoupledLogicalNetworkIOWrapper {
def apply[T <: Data](in: DecoupledIO[T], src: UInt = UInt(0), dst: UInt = UInt(0))(implicit conf: LogicalNetworkConfiguration) = {
def apply[T <: Data](in: DecoupledIO[T], src: UInt = UInt(0), dst: UInt = UInt(0)) = {
val out = Decoupled(new LogicalNetworkIO(in.bits.clone)).asDirectionless
out.valid := in.valid
out.bits.payload := in.bits
@ -69,7 +68,7 @@ object DecoupledLogicalNetworkIOWrapper {
}
object DecoupledLogicalNetworkIOUnwrapper {
def apply[T <: Data](in: DecoupledIO[LogicalNetworkIO[T]])(implicit conf: LogicalNetworkConfiguration) = {
def apply[T <: Data](in: DecoupledIO[LogicalNetworkIO[T]]) = {
val out = Decoupled(in.bits.payload.clone).asDirectionless
out.valid := in.valid
out.bits := in.bits.payload

View File

@ -1,27 +1,29 @@
package uncore
import Chisel._
case class TileLinkConfiguration(co: CoherencePolicyWithUncached, ln: LogicalNetworkConfiguration, addrBits: Int, masterXactIdBits: Int, clientXactIdBits: Int, dataBits: Int, writeMaskBits: Int, wordAddrBits: Int, atomicOpBits: Int)
case object TLCoherence extends Field[CoherencePolicyWithUncached]
case object TLAddrBits extends Field[Int]
case object TLMasterXactIdBits extends Field[Int]
case object TLClientXactIdBits extends Field[Int]
case object TLDataBits extends Field[Int]
case object TLWriteMaskBits extends Field[Int]
case object TLWordAddrBits extends Field[Int]
case object TLAtomicOpBits extends Field[Int]
abstract trait TileLinkSubBundle extends Bundle {
implicit val tlconf: TileLinkConfiguration
override def clone = this.getClass.getConstructors.head.newInstance(tlconf).asInstanceOf[this.type]
trait HasPhysicalAddress extends Bundle {
val addr = UInt(width = params(TLAddrBits))
}
trait HasPhysicalAddress extends TileLinkSubBundle {
val addr = UInt(width = tlconf.addrBits)
trait HasClientTransactionId extends Bundle {
val client_xact_id = Bits(width = params(TLClientXactIdBits))
}
trait HasClientTransactionId extends TileLinkSubBundle {
val client_xact_id = Bits(width = tlconf.clientXactIdBits)
trait HasMasterTransactionId extends Bundle {
val master_xact_id = Bits(width = params(TLMasterXactIdBits))
}
trait HasMasterTransactionId extends TileLinkSubBundle {
val master_xact_id = Bits(width = tlconf.masterXactIdBits)
}
trait HasTileLinkData extends TileLinkSubBundle {
val data = Bits(width = tlconf.dataBits)
trait HasTileLinkData extends Bundle {
val data = Bits(width = params(TLDataBits))
}
trait SourcedMessage extends Bundle
@ -30,7 +32,7 @@ trait MasterSourcedMessage extends SourcedMessage
object Acquire
{
def apply(a_type: Bits, addr: UInt, client_xact_id: UInt)(implicit conf: TileLinkConfiguration): Acquire = {
def apply(a_type: Bits, addr: UInt, client_xact_id: UInt): Acquire = {
val acq = new Acquire
acq.a_type := a_type
acq.addr := addr
@ -41,42 +43,42 @@ object Acquire
acq.atomic_opcode := Bits(0)
acq
}
def apply(a_type: Bits, addr: UInt, client_xact_id: UInt, data: UInt)(implicit conf: TileLinkConfiguration): Acquire = {
def apply(a_type: Bits, addr: UInt, client_xact_id: UInt, data: UInt): Acquire = {
val acq = apply(a_type, addr, client_xact_id)
acq.data := data
acq
}
def apply(a_type: UInt, addr: UInt, client_xact_id: UInt, write_mask: Bits, data: UInt)(implicit conf: TileLinkConfiguration): Acquire = {
def apply(a_type: UInt, addr: UInt, client_xact_id: UInt, write_mask: Bits, data: UInt): Acquire = {
val acq = apply(a_type, addr, client_xact_id, data)
acq.write_mask := write_mask
acq
}
def apply(a_type: UInt, addr: UInt, client_xact_id: UInt, subword_addr: UInt, atomic_opcode: UInt, data: UInt)(implicit conf: TileLinkConfiguration): Acquire = {
def apply(a_type: UInt, addr: UInt, client_xact_id: UInt, subword_addr: UInt, atomic_opcode: UInt, data: UInt): Acquire = {
val acq = apply(a_type, addr, client_xact_id, data)
acq.subword_addr := subword_addr
acq.atomic_opcode := atomic_opcode
acq
}
def apply(a: Acquire)(implicit conf: TileLinkConfiguration): Acquire = {
def apply(a: Acquire): Acquire = {
val acq = new Acquire
acq := a
acq
}
}
class Acquire(implicit val tlconf: TileLinkConfiguration) extends ClientSourcedMessage
class Acquire extends ClientSourcedMessage
with HasPhysicalAddress
with HasClientTransactionId
with HasTileLinkData {
val a_type = UInt(width = tlconf.co.acquireTypeWidth)
val write_mask = Bits(width = tlconf.writeMaskBits)
val subword_addr = Bits(width = tlconf.wordAddrBits)
val atomic_opcode = Bits(width = tlconf.atomicOpBits)
val a_type = UInt(width = params(TLCoherence).acquireTypeWidth)
val write_mask = Bits(width = params(TLWriteMaskBits))
val subword_addr = Bits(width = params(TLWordAddrBits))
val atomic_opcode = Bits(width = params(TLAtomicOpBits))
}
object Probe
{
def apply(p_type: UInt, addr: UInt, master_xact_id: UInt)(implicit conf: TileLinkConfiguration) = {
def apply(p_type: UInt, addr: UInt, master_xact_id: UInt) = {
val prb = new Probe
prb.p_type := p_type
prb.addr := addr
@ -85,22 +87,22 @@ object Probe
}
}
class Probe(implicit val tlconf: TileLinkConfiguration) extends MasterSourcedMessage
class Probe extends MasterSourcedMessage
with HasPhysicalAddress
with HasMasterTransactionId {
val p_type = UInt(width = tlconf.co.probeTypeWidth)
val p_type = UInt(width = params(TLCoherence).probeTypeWidth)
}
object Release
{
def apply(r_type: UInt, addr: UInt, data: UInt)(implicit conf: TileLinkConfiguration) = {
def apply(r_type: UInt, addr: UInt, data: UInt): Release = {
val rel = new Release
rel.r_type := r_type
rel.addr := addr
rel.data := data
rel
}
def apply(r_type: UInt, addr: UInt, client_xact_id: UInt, master_xact_id: UInt)(implicit conf: TileLinkConfiguration) = {
def apply(r_type: UInt, addr: UInt, client_xact_id: UInt, master_xact_id: UInt): Release = {
val rel = new Release
rel.r_type := r_type
rel.addr := addr
@ -109,24 +111,24 @@ object Release
rel.data := UInt(0)
rel
}
def apply(r_type: UInt, addr: UInt, client_xact_id: UInt, master_xact_id: UInt, data: UInt)(implicit conf: TileLinkConfiguration): Release = {
def apply(r_type: UInt, addr: UInt, client_xact_id: UInt, master_xact_id: UInt, data: UInt): Release = {
val rel = apply(r_type, addr, client_xact_id, master_xact_id)
rel.data := data
rel
}
}
class Release(implicit val tlconf: TileLinkConfiguration) extends ClientSourcedMessage
class Release extends ClientSourcedMessage
with HasPhysicalAddress
with HasClientTransactionId
with HasMasterTransactionId
with HasTileLinkData {
val r_type = UInt(width = tlconf.co.releaseTypeWidth)
val r_type = UInt(width = params(TLCoherence).releaseTypeWidth)
}
object Grant
{
def apply(g_type: UInt, client_xact_id: UInt, master_xact_id: UInt)(implicit conf: TileLinkConfiguration) = {
def apply(g_type: UInt, client_xact_id: UInt, master_xact_id: UInt): Grant = {
val gnt = new Grant
gnt.g_type := g_type
gnt.client_xact_id := client_xact_id
@ -134,40 +136,35 @@ object Grant
gnt.data := UInt(0)
gnt
}
def apply(g_type: UInt, client_xact_id: UInt, master_xact_id: UInt, data: UInt)(implicit conf: TileLinkConfiguration): Grant = {
def apply(g_type: UInt, client_xact_id: UInt, master_xact_id: UInt, data: UInt): Grant = {
val gnt = apply(g_type, client_xact_id, master_xact_id)
gnt.data := data
gnt
}
}
class Grant(implicit val tlconf: TileLinkConfiguration) extends MasterSourcedMessage
class Grant extends MasterSourcedMessage
with HasTileLinkData
with HasClientTransactionId
with HasMasterTransactionId {
val g_type = UInt(width = tlconf.co.grantTypeWidth)
val g_type = UInt(width = params(TLCoherence).grantTypeWidth)
}
class Finish(implicit val tlconf: TileLinkConfiguration) extends ClientSourcedMessage with HasMasterTransactionId
class Finish extends ClientSourcedMessage with HasMasterTransactionId
class UncachedTileLinkIO(implicit conf: TileLinkConfiguration) extends Bundle {
implicit val ln = conf.ln
class UncachedTileLinkIO extends Bundle {
val acquire = new DecoupledIO(new LogicalNetworkIO(new Acquire))
val grant = new DecoupledIO(new LogicalNetworkIO(new Grant)).flip
val finish = new DecoupledIO(new LogicalNetworkIO(new Finish))
override def clone = { new UncachedTileLinkIO().asInstanceOf[this.type] }
}
class TileLinkIO(implicit conf: TileLinkConfiguration) extends UncachedTileLinkIO()(conf) {
class TileLinkIO extends UncachedTileLinkIO {
val probe = new DecoupledIO(new LogicalNetworkIO(new Probe)).flip
val release = new DecoupledIO(new LogicalNetworkIO(new Release))
override def clone = { new TileLinkIO().asInstanceOf[this.type] }
}
abstract class TileLinkArbiterLike(val arbN: Int)(implicit conf: TileLinkConfiguration) extends Module {
implicit val (ln, co) = (conf.ln, conf.co)
abstract class TileLinkArbiterLike(val arbN: Int) extends Module {
type MasterSourcedWithId = MasterSourcedMessage with HasClientTransactionId
type ClientSourcedWithId = ClientSourcedMessage with HasClientTransactionId
@ -178,7 +175,7 @@ abstract class TileLinkArbiterLike(val arbN: Int)(implicit conf: TileLinkConfigu
def hookupClientSource[M <: ClientSourcedWithId]
(ins: Seq[DecoupledIO[LogicalNetworkIO[M]]],
out: DecoupledIO[LogicalNetworkIO[M]]) {
def hasData(m: LogicalNetworkIO[M]) = co.messageHasData(m.payload)
def hasData(m: LogicalNetworkIO[M]) = params(TLCoherence).messageHasData(m.payload)
val arb = Module(new RRArbiter(out.bits.clone, arbN))
out <> arb.io.out
ins.zipWithIndex.zip(arb.io.in).map{ case ((req,id), arb) => {
@ -205,7 +202,8 @@ abstract class TileLinkArbiterLike(val arbN: Int)(implicit conf: TileLinkConfigu
}
}
abstract class UncachedTileLinkIOArbiter(n: Int)(implicit conf: TileLinkConfiguration) extends TileLinkArbiterLike(n)(conf) {
abstract class UncachedTileLinkIOArbiter(n: Int)
extends TileLinkArbiterLike(n) {
val io = new Bundle {
val in = Vec.fill(n){new UncachedTileLinkIO}.flip
val out = new UncachedTileLinkIO
@ -219,7 +217,7 @@ abstract class UncachedTileLinkIOArbiter(n: Int)(implicit conf: TileLinkConfigur
finish_arb.io.in zip io.in map { case (arb, req) => arb <> req.finish }
}
abstract class TileLinkIOArbiter(n: Int)(implicit conf: TileLinkConfiguration) extends TileLinkArbiterLike(n)(conf) {
abstract class TileLinkIOArbiter(n: Int) extends TileLinkArbiterLike(n) {
val io = new Bundle {
val in = Vec.fill(n){new TileLinkIO}.flip
val out = new TileLinkIO
@ -267,9 +265,9 @@ abstract trait UsesNewId {
in.client_xact_id
}
class UncachedTileLinkIOArbiterThatAppendsArbiterId(val n: Int)(implicit conf: TileLinkConfiguration) extends UncachedTileLinkIOArbiter(n)(conf) with AppendsArbiterId
class UncachedTileLinkIOArbiterThatPassesId(val n: Int)(implicit conf: TileLinkConfiguration) extends UncachedTileLinkIOArbiter(n)(conf) with PassesId
class UncachedTileLinkIOArbiterThatUsesNewId(val n: Int)(implicit conf: TileLinkConfiguration) extends UncachedTileLinkIOArbiter(n)(conf) with UsesNewId
class TileLinkIOArbiterThatAppendsArbiterId(val n: Int)(implicit conf: TileLinkConfiguration) extends TileLinkIOArbiter(n)(conf) with AppendsArbiterId
class TileLinkIOArbiterThatPassesId(val n: Int)(implicit conf: TileLinkConfiguration) extends TileLinkIOArbiter(n)(conf) with PassesId
class TileLinkIOArbiterThatUsesNewId(val n: Int)(implicit conf: TileLinkConfiguration) extends TileLinkIOArbiter(n)(conf) with UsesNewId
class UncachedTileLinkIOArbiterThatAppendsArbiterId(val n: Int) extends UncachedTileLinkIOArbiter(n) with AppendsArbiterId
class UncachedTileLinkIOArbiterThatPassesId(val n: Int) extends UncachedTileLinkIOArbiter(n) with PassesId
class UncachedTileLinkIOArbiterThatUsesNewId(val n: Int) extends UncachedTileLinkIOArbiter(n) with UsesNewId
class TileLinkIOArbiterThatAppendsArbiterId(val n: Int) extends TileLinkIOArbiter(n) with AppendsArbiterId
class TileLinkIOArbiterThatPassesId(val n: Int) extends TileLinkIOArbiter(n) with PassesId
class TileLinkIOArbiterThatUsesNewId(val n: Int) extends TileLinkIOArbiter(n) with UsesNewId

View File

@ -1,33 +1,27 @@
package uncore
import Chisel._
trait CoherenceAgentConfiguration {
def tl: TileLinkConfiguration
def nReleaseTransactions: Int
def nAcquireTransactions: Int
}
case object NReleaseTransactors extends Field[Int]
case object NAcquireTransactors extends Field[Int]
case object NTransactors extends Field[Int]
case object NClients extends Field[Int]
abstract class CoherenceAgent(implicit conf: CoherenceAgentConfiguration) extends Module {
abstract class CoherenceAgent extends Module {
val co = params(TLCoherence)
val io = new Bundle {
val inner = (new TileLinkIO()(conf.tl)).flip
val outer = new UncachedTileLinkIO()(conf.tl)
val incoherent = Vec.fill(conf.tl.ln.nClients){Bool()}.asInput
val inner = (new TileLinkIO).flip
val outer = new UncachedTileLinkIO
val incoherent = Vec.fill(params(NClients)){Bool()}.asInput
}
}
case class L2CoherenceAgentConfiguration(
val tl: TileLinkConfiguration,
val nReleaseTransactions: Int,
val nAcquireTransactions: Int) extends CoherenceAgentConfiguration
class L2CoherenceAgent(bankId: Int)(implicit conf: CoherenceAgentConfiguration)
extends CoherenceAgent {
implicit val (tl, ln, co) = (conf.tl, conf.tl.ln, conf.tl.co)
class L2CoherenceAgent(bankId: Int) extends CoherenceAgent {
// Create SHRs for outstanding transactions
val nTrackers = conf.nReleaseTransactions + conf.nAcquireTransactions
val trackerList = (0 until conf.nReleaseTransactions).map(id => Module(new VoluntaryReleaseTracker(id, bankId))) ++
(conf.nReleaseTransactions until nTrackers).map(id => Module(new AcquireTracker(id, bankId)))
val trackerList = (0 until params(NReleaseTransactors)).map(id =>
Module(new VoluntaryReleaseTracker(id, bankId))) ++
(params(NReleaseTransactors) until params(NTransactors)).map(id =>
Module(new AcquireTracker(id, bankId)))
// Propagate incoherence flags
trackerList.map(_.io.tile_incoherent := io.incoherent.toBits)
@ -85,12 +79,12 @@ class L2CoherenceAgent(bankId: Int)(implicit conf: CoherenceAgentConfiguration)
}
abstract class XactTracker()(implicit conf: CoherenceAgentConfiguration) extends Module {
implicit val (tl, ln, co) = (conf.tl, conf.tl.ln, conf.tl.co)
abstract class XactTracker extends Module {
val (co, nClients) = (params(TLCoherence),params(NClients))
val io = new Bundle {
val inner = (new TileLinkIO).flip
val outer = new UncachedTileLinkIO
val tile_incoherent = Bits(INPUT, ln.nClients)
val tile_incoherent = Bits(INPUT, params(NClients))
val has_acquire_conflict = Bool(OUTPUT)
val has_release_conflict = Bool(OUTPUT)
}
@ -100,15 +94,13 @@ abstract class XactTracker()(implicit conf: CoherenceAgentConfiguration) extends
val c_gnt = io.inner.grant.bits
val c_ack = io.inner.finish.bits
val m_gnt = io.outer.grant.bits
}
class VoluntaryReleaseTracker(trackerId: Int, bankId: Int)(implicit conf: CoherenceAgentConfiguration)
extends XactTracker()(conf) {
class VoluntaryReleaseTracker(trackerId: Int, bankId: Int) extends XactTracker {
val s_idle :: s_mem :: s_ack :: s_busy :: Nil = Enum(UInt(), 4)
val state = Reg(init=s_idle)
val xact = Reg{ new Release }
val init_client_id = Reg(init=UInt(0, width = log2Up(ln.nClients)))
val init_client_id = Reg(init=UInt(0, width = log2Up(nClients)))
val incoming_rel = io.inner.release.bits
io.has_acquire_conflict := Bool(false)
@ -153,17 +145,16 @@ class VoluntaryReleaseTracker(trackerId: Int, bankId: Int)(implicit conf: Cohere
}
}
class AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: CoherenceAgentConfiguration)
extends XactTracker()(conf) {
class AcquireTracker(trackerId: Int, bankId: Int) extends XactTracker {
val s_idle :: s_probe :: s_mem_read :: s_mem_write :: s_make_grant :: s_busy :: Nil = Enum(UInt(), 6)
val state = Reg(init=s_idle)
val xact = Reg{ new Acquire }
val init_client_id = Reg(init=UInt(0, width = log2Up(ln.nClients)))
val init_client_id = Reg(init=UInt(0, width = log2Up(nClients)))
//TODO: Will need id reg for merged release xacts
val init_sharer_cnt = Reg(init=UInt(0, width = log2Up(ln.nClients)))
val release_count = if (ln.nClients == 1) UInt(0) else Reg(init=UInt(0, width = log2Up(ln.nClients)))
val probe_flags = Reg(init=Bits(0, width = ln.nClients))
val init_sharer_cnt = Reg(init=UInt(0, width = log2Up(nClients)))
val release_count = if (nClients == 1) UInt(0) else Reg(init=UInt(0, width = log2Up(nClients)))
val probe_flags = Reg(init=Bits(0, width = nClients))
val curr_p_id = PriorityEncoder(probe_flags)
val pending_outer_write = co.messageHasData(xact)
@ -174,12 +165,12 @@ class AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: CoherenceAgentC
xact.addr, UInt(trackerId), c_rel.payload.data)
val outer_read = Acquire(co.getUncachedReadAcquireType, xact.addr, UInt(trackerId))
val probe_initial_flags = Bits(width = ln.nClients)
val probe_initial_flags = Bits(width = nClients)
probe_initial_flags := Bits(0)
if (ln.nClients > 1) {
if (nClients > 1) {
// issue self-probes for uncached read xacts to facilitate I$ coherence
val probe_self = Bool(true) //co.needsSelfProbe(io.inner.acquire.bits.payload)
val myflag = Mux(probe_self, Bits(0), UIntToOH(c_acq.header.src(log2Up(ln.nClients)-1,0)))
val myflag = Mux(probe_self, Bits(0), UIntToOH(c_acq.header.src(log2Up(nClients)-1,0)))
probe_initial_flags := ~(io.tile_incoherent | myflag)
}
@ -219,9 +210,9 @@ class AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: CoherenceAgentC
when( io.inner.acquire.valid ) {
xact := c_acq.payload
init_client_id := c_acq.header.src
init_sharer_cnt := UInt(ln.nClients) // TODO: Broadcast only
init_sharer_cnt := UInt(nClients) // TODO: Broadcast only
probe_flags := probe_initial_flags
if(ln.nClients > 1) {
if(nClients > 1) {
release_count := PopCount(probe_initial_flags)
state := Mux(probe_initial_flags.orR, s_probe,
Mux(needs_outer_write, s_mem_write,
@ -244,7 +235,7 @@ class AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: CoherenceAgentC
io.outer.acquire.bits.payload := outer_write_rel
when(io.outer.acquire.ready) {
io.inner.release.ready := Bool(true)
if(ln.nClients > 1) release_count := release_count - UInt(1)
if(nClients > 1) release_count := release_count - UInt(1)
when(release_count === UInt(1)) {
state := Mux(pending_outer_write, s_mem_write,
Mux(pending_outer_read, s_mem_read, s_make_grant))
@ -252,7 +243,7 @@ class AcquireTracker(trackerId: Int, bankId: Int)(implicit conf: CoherenceAgentC
}
} .otherwise {
io.inner.release.ready := Bool(true)
if(ln.nClients > 1) release_count := release_count - UInt(1)
if(nClients > 1) release_count := release_count - UInt(1)
when(release_count === UInt(1)) {
state := Mux(pending_outer_write, s_mem_write,
Mux(pending_outer_read, s_mem_read, s_make_grant))