1
0

Restructure L2 state machine and utilize HeaderlessTileLinkIO

This commit is contained in:
Henry Cook
2015-03-24 02:06:53 -07:00
parent ced627f00a
commit 9708d25dff
9 changed files with 780 additions and 565 deletions

View File

@ -3,7 +3,8 @@
package uncore
import Chisel._
import scala.math.max
import scala.reflect.ClassTag
import scala.reflect._
import scala.reflect.runtime.universe._
// Parameters exposed to the top-level design, set based on
// external requirements or design space exploration
@ -11,17 +12,22 @@ import scala.reflect.ClassTag
case object TLId extends Field[String] // Unique name per network
case object TLCoherencePolicy extends Field[CoherencePolicy]
case object TLBlockAddrBits extends Field[Int]
case object TLManagerXactIdBits extends Field[Int]
case object TLClientXactIdBits extends Field[Int]
case object TLMaxClientXacts extends Field[Int]
case object TLMaxClientPorts extends Field[Int]
case object TLMaxManagerXacts extends Field[Int]
case object TLDataBits extends Field[Int]
case object TLDataBeats extends Field[Int]
case object TLNetworkIsOrderedP2P extends Field[Boolean]
abstract trait TileLinkParameters extends UsesParameters {
val tlBlockAddrBits = params(TLBlockAddrBits)
val tlClientXactIdBits = params(TLClientXactIdBits)
val tlManagerXactIdBits = params(TLManagerXactIdBits)
val tlMaxClientXacts = params(TLMaxClientXacts)
val tlMaxClientPorts = params(TLMaxClientPorts)
val tlMaxManagerXacts = params(TLMaxManagerXacts)
val tlClientXactIdBits = log2Up(tlMaxClientXacts*tlMaxClientPorts)
val tlManagerXactIdBits = log2Up(tlMaxManagerXacts)
val tlDataBits = params(TLDataBits)
val tlDataBytes = tlDataBits/8
val tlDataBeats = params(TLDataBeats)
val tlCoh = params(TLCoherencePolicy)
val tlWriteMaskBits = if(tlDataBits/8 < 1) 1 else tlDataBits/8
@ -41,12 +47,16 @@ abstract trait TileLinkParameters extends UsesParameters {
val amoAluOperandBits = params(AmoAluOperandBits)
}
abstract class TLBundle extends Bundle with TileLinkParameters
abstract class TLBundle extends Bundle with TileLinkParameters {
}
abstract class TLModule extends Module with TileLinkParameters
// Directionality of message channel
// Used to hook up logical network ports to physical network ports
trait TileLinkChannel extends TLBundle
trait TileLinkChannel extends TLBundle {
def hasData(dummy: Int = 0): Bool
def hasMultibeatData(dummy: Int = 0): Bool
}
trait ClientToManagerChannel extends TileLinkChannel
trait ManagerToClientChannel extends TileLinkChannel
trait ClientToClientChannel extends TileLinkChannel // Unused for now
@ -97,8 +107,7 @@ class Acquire extends ClientToManagerChannel
val addrByteOff = tlMemoryOperandSizeBits + opSizeOff
val addrByteMSB = tlByteAddrBits + addrByteOff
def allocate(dummy: Int = 0) = union(0)
def op_code(dummy: Int = 0) = Mux(isBuiltInType() &&
(a_type === Acquire.putType || a_type === Acquire.putBlockType),
def op_code(dummy: Int = 0) = Mux(isBuiltInType(Acquire.putType) || isBuiltInType(Acquire.putBlockType),
M_XWR, union(opSizeOff-1, opCodeOff))
def op_size(dummy: Int = 0) = union(addrByteOff-1, opSizeOff)
def addr_byte(dummy: Int = 0) = union(addrByteMSB-1, addrByteOff)
@ -328,6 +337,8 @@ class Probe extends ManagerToClientChannel
val p_type = UInt(width = tlCoh.probeTypeWidth)
def is(t: UInt) = p_type === t
def hasData(dummy: Int = 0) = Bool(false)
def hasMultibeatData(dummy: Int = 0) = Bool(false)
}
object Probe {
@ -383,6 +394,7 @@ class Grant extends ManagerToClientChannel
// Helper funcs
def isBuiltInType(dummy: Int = 0): Bool = is_builtin_type
def isBuiltInType(t: UInt): Bool = is_builtin_type && g_type === t
def is(t: UInt):Bool = g_type === t
def hasData(dummy: Int = 0): Bool = Mux(isBuiltInType(),
Grant.typesWithData.contains(g_type),
@ -394,7 +406,7 @@ class Grant extends ManagerToClientChannel
def isVoluntary(dummy: Int = 0): Bool = isBuiltInType() && (g_type === Grant.voluntaryAckType)
def requiresAck(dummy: Int = 0): Bool = !Bool(tlNetworkPreservesPointToPointOrdering) && !isVoluntary()
def makeFinish(dummy: Int = 0): Finish = {
val f = Bundle(new Finish, { case TLManagerXactIdBits => tlManagerXactIdBits })
val f = Bundle(new Finish, { case TLMaxManagerXacts => tlMaxManagerXacts })
f.manager_xact_id := this.manager_xact_id
f
}
@ -428,7 +440,10 @@ object Grant {
}
}
class Finish extends ClientToManagerChannel with HasManagerTransactionId
class Finish extends ClientToManagerChannel with HasManagerTransactionId {
def hasData(dummy: Int = 0) = Bool(false)
def hasMultibeatData(dummy: Int = 0) = Bool(false)
}
// Complete IO definitions for two types of TileLink clients
class UncachedTileLinkIO extends TLBundle {
@ -455,33 +470,22 @@ class TileLinkIOWrapper extends TLModule {
io.out.probe.ready := Bool(true)
io.out.release.valid := Bool(false)
}
object TileLinkIOWrapper {
def apply(utl: UncachedTileLinkIO, p: Parameters): TileLinkIO = {
val conv = Module(new TileLinkIOWrapper)(p)
conv.io.in <> utl
conv.io.out
}
def apply(utl: UncachedTileLinkIO): TileLinkIO = {
val conv = Module(new TileLinkIOWrapper)
conv.io.in <> utl
conv.io.out
}
def apply(tl: TileLinkIO) = tl
}
// This version of TileLinkIO does not contain network headers for packets
// that originate in the Clients (i.e. Acquire and Release). These headers
// are provided in the top-level that instantiates the clients and network.
// By eliding the header subbundles within the clients we can enable
// hierarchical P&R while minimizing unconnected port errors in GDS.
// Secondly, this version of the interface elides Finish messages, with the
// assumption that a FinishUnit has been coupled to the TileLinkIO port
// to deal with acking received Grants.
class HeaderlessUncachedTileLinkIO extends TLBundle {
val acquire = new DecoupledIO(new Acquire)
val grant = new DecoupledIO(new LogicalNetworkIO(new Grant)).flip
val finish = new DecoupledIO(new LogicalNetworkIO(new Finish))
val grant = new DecoupledIO(new Grant).flip
}
class HeaderlessTileLinkIO extends HeaderlessUncachedTileLinkIO {
val probe = new DecoupledIO(new LogicalNetworkIO(new Probe)).flip
val probe = new DecoupledIO(new Probe).flip
val release = new DecoupledIO(new Release)
}
@ -492,17 +496,231 @@ class HeaderlessTileLinkIOWrapper extends TLModule {
}
io.out.acquire <> io.in.acquire
io.out.grant <> io.in.grant
io.out.finish <> io.in.finish
io.out.probe.ready := Bool(true)
io.out.release.valid := Bool(false)
}
object HeaderlessTileLinkIOWrapper {
object TileLinkIOWrapper {
def apply(utl: HeaderlessUncachedTileLinkIO, p: Parameters): HeaderlessTileLinkIO = {
val conv = Module(new HeaderlessTileLinkIOWrapper)(p)
conv.io.in <> utl
conv.io.out
}
def apply(utl: HeaderlessUncachedTileLinkIO): HeaderlessTileLinkIO = {
val conv = Module(new HeaderlessTileLinkIOWrapper)
conv.io.in <> utl
conv.io.out
}
def apply(tl: HeaderlessTileLinkIO): HeaderlessTileLinkIO = tl
def apply(utl: UncachedTileLinkIO, p: Parameters): TileLinkIO = {
val conv = Module(new TileLinkIOWrapper)(p)
conv.io.in <> utl
conv.io.out
}
def apply(utl: UncachedTileLinkIO): TileLinkIO = {
val conv = Module(new TileLinkIOWrapper)
conv.io.in <> utl
conv.io.out
}
def apply(tl: TileLinkIO): TileLinkIO = tl
}
trait HasDataBeatCounters {
type HasBeat = TileLinkChannel with HasTileLinkBeatId
type HasClientId = TileLinkChannel with HasClientTransactionId
type LNAcquire = LogicalNetworkIO[Acquire]
type LNRelease = LogicalNetworkIO[Release]
type LNGrant = LogicalNetworkIO[Grant]
def connectDataBeatCounter[S <: TileLinkChannel : ClassTag](inc: Bool, data: S, beat: UInt) = {
val multi = data.hasMultibeatData()
val (multi_cnt, multi_done) = Counter(inc && multi, data.tlDataBeats)
val cnt = Mux(multi, multi_cnt, beat)
val done = Mux(multi, multi_done, inc)
(cnt, done)
}
def connectOutgoingDataBeatCounter[T <: Data : TypeTag](
in: DecoupledIO[T],
beat: UInt = UInt(0)): (UInt, Bool) = {
in.bits match {
case p: TileLinkChannel if typeTag[T].tpe <:< typeTag[TileLinkChannel].tpe =>
connectDataBeatCounter(in.fire(), p, beat)
case ln: LNGrant if typeTag[T].tpe <:< typeTag[LNGrant].tpe =>
connectDataBeatCounter(in.fire(), ln.payload, beat)
case _ => { require(false, "Don't know how to connect a beat counter to " + typeTag[T].tpe); (UInt(0), Bool(false))}
}
}
def connectIncomingDataBeatCounters[T <: HasClientId : ClassTag](
in: DecoupledIO[LogicalNetworkIO[T]],
entries: Int): Vec[Bool] = {
val id = in.bits.payload.client_xact_id
Vec((0 until entries).map { i =>
connectDataBeatCounter(in.fire() && id === UInt(i), in.bits.payload, UInt(0))._2
})
}
def connectIncomingDataBeatCounter[T <: Data : TypeTag](in: DecoupledIO[T]): Bool = {
in.bits match {
case p: TileLinkChannel if typeTag[T].tpe <:< typeTag[TileLinkChannel].tpe =>
connectDataBeatCounter(in.fire(), p, UInt(0))._2
case ln: LNAcquire if typeTag[T].tpe =:= typeTag[LNAcquire].tpe =>
connectDataBeatCounter(in.fire(), ln.payload, UInt(0))._2
case ln: LNRelease if typeTag[T].tpe =:= typeTag[LNRelease].tpe =>
connectDataBeatCounter(in.fire(), ln.payload, UInt(0))._2
case ln: LNGrant if typeTag[T].tpe =:= typeTag[LNGrant].tpe =>
connectDataBeatCounter(in.fire(), ln.payload, UInt(0))._2
case _ => { require(false, "Don't know how to connect a beat counter to " + typeTag[T].tpe); Bool(false)}
}
}
def connectHeaderlessTwoWayBeatCounter[ T <: TileLinkChannel : ClassTag, S <: TileLinkChannel : ClassTag](
max: Int,
up: DecoupledIO[T],
down: DecoupledIO[S],
beat: UInt): (Bool, UInt, Bool, UInt, Bool) = {
val cnt = Reg(init = UInt(0, width = log2Up(max+1)))
val (up_idx, do_inc) = connectDataBeatCounter(up.fire(), up.bits, beat)
val (down_idx, do_dec) = connectDataBeatCounter(down.fire(), down.bits, beat)
cnt := Mux(do_dec,
Mux(do_inc, cnt, cnt - UInt(1)),
Mux(do_inc, cnt + UInt(1), cnt))
(cnt > UInt(0), up_idx, do_inc, down_idx, do_dec)
}
def connectTwoWayBeatCounter[ T <: TileLinkChannel : ClassTag, S <: TileLinkChannel : ClassTag](
max: Int,
up: DecoupledIO[LogicalNetworkIO[T]],
down: DecoupledIO[LogicalNetworkIO[S]],
inc: T => Bool = (t: T) => Bool(true),
dec: S => Bool = (s: S) => Bool(true)): (Bool, UInt, Bool, UInt, Bool) = {
val cnt = Reg(init = UInt(0, width = log2Up(max+1)))
val (up_idx, up_done) = connectDataBeatCounter(up.fire(), up.bits.payload, UInt(0))
val (down_idx, down_done) = connectDataBeatCounter(down.fire(), down.bits.payload, UInt(0))
val do_inc = up_done && inc(up.bits.payload)
val do_dec = down_done && dec(down.bits.payload)
cnt := Mux(do_dec,
Mux(do_inc, cnt, cnt - UInt(1)),
Mux(do_inc, cnt + UInt(1), cnt))
(cnt > UInt(0), up_idx, up_done, down_idx, down_done)
}
}
class FinishQueueEntry extends TLBundle {
val fin = new Finish
val dst = UInt(width = log2Up(params(LNEndpoints)))
}
class FinishQueue(entries: Int) extends Queue(new FinishQueueEntry, entries)
class FinishUnit(srcId: Int = 0) extends TLModule
with HasDataBeatCounters {
val io = new Bundle {
val grant = Decoupled(new LogicalNetworkIO(new Grant)).flip
val refill = Decoupled(new Grant)
val finish = Decoupled(new LogicalNetworkIO(new Finish))
val ready = Bool(OUTPUT)
val grant_done = Bool(OUTPUT)
val pending_finish = Bool(OUTPUT)
}
val entries = 1 << tlClientXactIdBits
val g = io.grant.bits.payload
assert(g.client_xact_id <= UInt(entries), "No grant beat counter provisioned, only " + entries)
val done = connectIncomingDataBeatCounters(io.grant, entries).reduce(_||_)
val q = Module(new FinishQueue(entries))
q.io.enq.valid := io.grant.valid && g.requiresAck() && (!g.hasMultibeatData() || done)
q.io.enq.bits.fin := g.makeFinish()
q.io.enq.bits.dst := io.grant.bits.header.src
io.finish.bits.header.src := UInt(srcId)
io.finish.bits.header.dst := q.io.deq.bits.dst
io.finish.bits.payload := q.io.deq.bits.fin
io.finish.valid := q.io.deq.valid
q.io.deq.ready := io.finish.ready
io.refill.valid := io.grant.valid
io.refill.bits := io.grant.bits.payload
io.grant.ready := (q.io.enq.ready || !g.requiresAck()) && (io.refill.ready || !g.hasData())
io.ready := q.io.enq.ready
io.grant_done := done
io.pending_finish := q.io.deq.valid
}
object TileLinkHeaderOverwriter {
def apply[T <: ClientToManagerChannel](
in: DecoupledIO[LogicalNetworkIO[T]],
clientId: Int,
passThrough: Boolean): DecoupledIO[LogicalNetworkIO[T]] = {
val out = in.clone.asDirectionless
out.bits.payload := in.bits.payload
out.bits.header.src := UInt(clientId)
out.bits.header.dst := (if(passThrough) in.bits.header.dst else UInt(0))
out.valid := in.valid
in.ready := out.ready
out
}
def apply[T <: ClientToManagerChannel with HasCacheBlockAddress](
in: DecoupledIO[LogicalNetworkIO[T]],
clientId: Int,
nBanks: Int,
addrConvert: UInt => UInt): DecoupledIO[LogicalNetworkIO[T]] = {
val out: DecoupledIO[LogicalNetworkIO[T]] = apply(in, clientId, false)
out.bits.header.dst := addrConvert(in.bits.payload.addr_block)
out
}
def apply[T <: ClientToManagerChannel with HasCacheBlockAddress : ClassTag](
in: DecoupledIO[T],
clientId: Int,
addrConvert: UInt => UInt): DecoupledIO[LogicalNetworkIO[T]] = {
val out = new DecoupledIO(new LogicalNetworkIO(in.bits.clone)).asDirectionless
out.bits.payload := in.bits
out.bits.header.src := UInt(clientId)
out.bits.header.dst := addrConvert(in.bits.addr_block)
out.valid := in.valid
in.ready := out.ready
out
}
}
class TileLinkNetworkPort(clientId: Int, addrConvert: UInt => UInt) extends TLModule {
val io = new Bundle {
val client = new HeaderlessTileLinkIO().flip
val network = new TileLinkIO
}
val finisher = Module(new FinishUnit(clientId))
finisher.io.grant <> io.network.grant
io.network.finish <> finisher.io.finish
val acq_with_header = TileLinkHeaderOverwriter(io.client.acquire, clientId, addrConvert)
val rel_with_header = TileLinkHeaderOverwriter(io.client.release, clientId, addrConvert)
val prb_without_header = DecoupledLogicalNetworkIOUnwrapper(io.network.probe)
val gnt_without_header = finisher.io.refill
io.network.acquire.bits := acq_with_header.bits
io.network.acquire.valid := acq_with_header.valid && finisher.io.ready
acq_with_header.ready := io.network.acquire.ready && finisher.io.ready
io.network.release <> rel_with_header
io.client.probe <> prb_without_header
io.client.grant <> gnt_without_header
}
object TileLinkNetworkPort {
def apply[T <: Data](
client: HeaderlessTileLinkIO,
clientId: Int = 0,
addrConvert: UInt => UInt = u => UInt(0))(implicit p: Parameters): TileLinkIO = {
val port = Module(new TileLinkNetworkPort(clientId, addrConvert))(p)
port.io.client <> client
port.io.network
}
}
class TileLinkEnqueuer(depths: (Int, Int, Int, Int, Int)) extends Module {
@ -529,7 +747,6 @@ object TileLinkEnqueuer {
}
abstract trait HasArbiterTypes {
val arbN: Int
type ManagerSourcedWithId = ManagerToClientChannel with HasClientTransactionId
type ClientSourcedWithId = ClientToManagerChannel with HasClientTransactionId
type ClientSourcedWithIdAndData = ClientToManagerChannel with
@ -538,20 +755,19 @@ abstract trait HasArbiterTypes {
}
// Utility functions for constructing TileLinkIO arbiters
abstract class TileLinkArbiterLike(val arbN: Int) extends TLModule
with HasArbiterTypes {
trait TileLinkArbiterLike extends HasArbiterTypes with TileLinkParameters{
val arbN: Int
// These are filled in depending on whether the arbiter mucks with the
// client ids and then needs to revert them on the way back
def clientSourcedClientXactId(in: ClientSourcedWithId, id: Int): Bits
def managerSourcedClientXactId(in: ManagerSourcedWithId): Bits
def arbIdx(in: ManagerSourcedWithId): UInt
def hookupClientSource[M <: ClientSourcedWithIdAndData : ClassTag]
(clts: Seq[DecoupledIO[LogicalNetworkIO[M]]],
mngr: DecoupledIO[LogicalNetworkIO[M]]) {
def hookupClientSource[M <: ClientSourcedWithIdAndData : ClassTag](
clts: Seq[DecoupledIO[LogicalNetworkIO[M]]],
mngr: DecoupledIO[LogicalNetworkIO[M]]) {
def hasData(m: LogicalNetworkIO[M]) = m.payload.hasMultibeatData()
val arb = Module(new LockingRRArbiter(mngr.bits.clone, arbN, params(TLDataBeats), Some(hasData _)))
val arb = Module(new LockingRRArbiter(mngr.bits.clone, arbN, tlDataBeats, Some(hasData _)))
clts.zipWithIndex.zip(arb.io.in).map{ case ((req, id), arb) => {
arb.valid := req.valid
arb.bits := req.bits
@ -561,11 +777,11 @@ abstract class TileLinkArbiterLike(val arbN: Int) extends TLModule
arb.io.out <> mngr
}
def hookupClientSourceHeaderless[M <: ClientSourcedWithIdAndData : ClassTag]
(clts: Seq[DecoupledIO[M]],
mngr: DecoupledIO[M]) {
def hookupClientSourceHeaderless[M <: ClientSourcedWithIdAndData : ClassTag](
clts: Seq[DecoupledIO[M]],
mngr: DecoupledIO[M]) {
def hasData(m: M) = m.hasMultibeatData()
val arb = Module(new LockingRRArbiter(mngr.bits.clone, arbN, params(TLDataBeats), Some(hasData _)))
val arb = Module(new LockingRRArbiter(mngr.bits.clone, arbN, tlDataBeats, Some(hasData _)))
clts.zipWithIndex.zip(arb.io.in).map{ case ((req, id), arb) => {
arb.valid := req.valid
arb.bits := req.bits
@ -575,17 +791,23 @@ abstract class TileLinkArbiterLike(val arbN: Int) extends TLModule
arb.io.out <> mngr
}
def hookupFinish[M <: LogicalNetworkIO[Finish] : ClassTag]
(clts: Seq[DecoupledIO[M]],
mngr: DecoupledIO[M]) {
val arb = Module(new RRArbiter(mngr.bits.clone, arbN))
arb.io.in zip clts map { case (arb, req) => arb <> req }
arb.io.out <> mngr
def hookupManagerSourceWithHeader[M <: ManagerToClientChannel](
clts: Seq[DecoupledIO[LogicalNetworkIO[M]]],
mngr: DecoupledIO[LogicalNetworkIO[M]]) {
mngr.ready := Bool(false)
for (i <- 0 until arbN) {
clts(i).valid := Bool(false)
when (mngr.bits.header.dst === UInt(i)) {
clts(i).valid := mngr.valid
mngr.ready := clts(i).ready
}
clts(i).bits := mngr.bits
}
}
def hookupManagerSourceWithId[M <: ManagerSourcedWithId]
(clts: Seq[DecoupledIO[LogicalNetworkIO[M]]],
mngr: DecoupledIO[LogicalNetworkIO[M]]) {
def hookupManagerSourceWithId[M <: ManagerSourcedWithId](
clts: Seq[DecoupledIO[LogicalNetworkIO[M]]],
mngr: DecoupledIO[LogicalNetworkIO[M]]) {
mngr.ready := Bool(false)
for (i <- 0 until arbN) {
clts(i).valid := Bool(false)
@ -594,24 +816,45 @@ abstract class TileLinkArbiterLike(val arbN: Int) extends TLModule
mngr.ready := clts(i).ready
}
clts(i).bits := mngr.bits
clts(i).bits.payload.client_xact_id :=
managerSourcedClientXactId(mngr.bits.payload)
clts(i).bits.payload.client_xact_id := managerSourcedClientXactId(mngr.bits.payload)
}
}
def hookupManagerSourceBroadcast[M <: ManagerToClientChannel]
(clts: Seq[DecoupledIO[LogicalNetworkIO[M]]],
mngr: DecoupledIO[LogicalNetworkIO[M]]) {
def hookupManagerSourceHeaderlessWithId[M <: ManagerSourcedWithId](
clts: Seq[DecoupledIO[M]],
mngr: DecoupledIO[M]) {
mngr.ready := Bool(false)
for (i <- 0 until arbN) {
clts(i).valid := Bool(false)
when (arbIdx(mngr.bits) === UInt(i)) {
clts(i).valid := mngr.valid
mngr.ready := clts(i).ready
}
clts(i).bits := mngr.bits
clts(i).bits.client_xact_id := managerSourcedClientXactId(mngr.bits)
}
}
def hookupManagerSourceBroadcast[M <: Data](
clts: Seq[DecoupledIO[M]],
mngr: DecoupledIO[M]) {
clts.map{ _.valid := mngr.valid }
clts.map{ _.bits := mngr.bits }
mngr.ready := clts.map(_.ready).reduce(_&&_)
}
def hookupFinish[M <: LogicalNetworkIO[Finish] : ClassTag](
clts: Seq[DecoupledIO[M]],
mngr: DecoupledIO[M]) {
val arb = Module(new RRArbiter(mngr.bits.clone, arbN))
arb.io.in zip clts map { case (arb, req) => arb <> req }
arb.io.out <> mngr
}
}
abstract class UncachedTileLinkIOArbiter(n: Int)
extends TileLinkArbiterLike(n) {
abstract class UncachedTileLinkIOArbiter(val arbN: Int) extends Module with TileLinkArbiterLike {
val io = new Bundle {
val in = Vec.fill(n){new UncachedTileLinkIO}.flip
val in = Vec.fill(arbN){new UncachedTileLinkIO}.flip
val out = new UncachedTileLinkIO
}
hookupClientSource(io.in.map(_.acquire), io.out.acquire)
@ -619,9 +862,9 @@ abstract class UncachedTileLinkIOArbiter(n: Int)
hookupManagerSourceWithId(io.in.map(_.grant), io.out.grant)
}
abstract class TileLinkIOArbiter(n: Int) extends TileLinkArbiterLike(n) {
abstract class TileLinkIOArbiter(val arbN: Int) extends Module with TileLinkArbiterLike {
val io = new Bundle {
val in = Vec.fill(n){new TileLinkIO}.flip
val in = Vec.fill(arbN){new TileLinkIO}.flip
val out = new TileLinkIO
}
hookupClientSource(io.in.map(_.acquire), io.out.acquire)
@ -631,8 +874,32 @@ abstract class TileLinkIOArbiter(n: Int) extends TileLinkArbiterLike(n) {
hookupManagerSourceWithId(io.in.map(_.grant), io.out.grant)
}
class HeaderlessUncachedTileLinkIOArbiter(val arbN: Int) extends Module
with TileLinkArbiterLike
with AppendsArbiterId {
val io = new Bundle {
val in = Vec.fill(arbN){new HeaderlessUncachedTileLinkIO}.flip
val out = new HeaderlessUncachedTileLinkIO
}
hookupClientSourceHeaderless(io.in.map(_.acquire), io.out.acquire)
hookupManagerSourceHeaderlessWithId(io.in.map(_.grant), io.out.grant)
}
class HeaderlessTileLinkIOArbiter(val arbN: Int) extends Module
with TileLinkArbiterLike
with AppendsArbiterId {
val io = new Bundle {
val in = Vec.fill(arbN){new HeaderlessTileLinkIO}.flip
val out = new HeaderlessTileLinkIO
}
hookupClientSourceHeaderless(io.in.map(_.acquire), io.out.acquire)
hookupClientSourceHeaderless(io.in.map(_.release), io.out.release)
hookupManagerSourceBroadcast(io.in.map(_.probe), io.out.probe)
hookupManagerSourceHeaderlessWithId(io.in.map(_.grant), io.out.grant)
}
// Appends the port index of the arbiter to the client_xact_id
abstract trait AppendsArbiterId extends HasArbiterTypes {
trait AppendsArbiterId extends TileLinkArbiterLike {
def clientSourcedClientXactId(in: ClientSourcedWithId, id: Int) =
Cat(in.client_xact_id, UInt(id, log2Up(arbN)))
def managerSourcedClientXactId(in: ManagerSourcedWithId) =
@ -641,14 +908,14 @@ abstract trait AppendsArbiterId extends HasArbiterTypes {
}
// Uses the client_xact_id as is (assumes it has been set to port index)
abstract trait PassesId extends HasArbiterTypes {
trait PassesId extends TileLinkArbiterLike {
def clientSourcedClientXactId(in: ClientSourcedWithId, id: Int) = in.client_xact_id
def managerSourcedClientXactId(in: ManagerSourcedWithId) = in.client_xact_id
def arbIdx(in: ManagerSourcedWithId) = in.client_xact_id
}
// Overwrites some default client_xact_id with the port idx
abstract trait UsesNewId extends HasArbiterTypes {
trait UsesNewId extends TileLinkArbiterLike {
def clientSourcedClientXactId(in: ClientSourcedWithId, id: Int) = UInt(id, log2Up(arbN))
def managerSourcedClientXactId(in: ManagerSourcedWithId) = UInt(0)
def arbIdx(in: ManagerSourcedWithId) = in.client_xact_id