TileLink refactor; TileLinkPorts now available. L2Banks no longer have unique ids (suitable for hierarhical P&R).
This commit is contained in:
@ -23,12 +23,14 @@ case object TLDataBits extends Field[Int]
|
||||
case object TLDataBeats extends Field[Int]
|
||||
case object TLNetworkIsOrderedP2P extends Field[Boolean]
|
||||
|
||||
abstract trait TileLinkParameters extends UsesParameters {
|
||||
trait TileLinkParameters extends UsesParameters {
|
||||
val tlCoh = params(TLCoherencePolicy)
|
||||
val tlNManagers = params(TLNManagers)
|
||||
val tlNClients = params(TLNClients)
|
||||
val tlNCoherentClients = params(TLNCoherentClients)
|
||||
val tlNIncoherentClients = params(TLNIncoherentClients)
|
||||
val tlClientIdBits = log2Up(tlNClients)
|
||||
val tlManagerIdBits = log2Up(tlNManagers)
|
||||
val tlMaxClientXacts = params(TLMaxClientXacts)
|
||||
val tlMaxClientPorts = params(TLMaxClientPorts)
|
||||
val tlMaxManagerXacts = params(TLMaxManagerXacts)
|
||||
@ -98,6 +100,10 @@ trait HasTileLinkData extends HasTileLinkBeatId {
|
||||
def hasMultibeatData(dummy: Int = 0): Bool
|
||||
}
|
||||
|
||||
trait HasClientId extends TLBundle {
|
||||
val client_id = UInt(width = tlClientIdBits)
|
||||
}
|
||||
|
||||
// Actual TileLink channel bundle definitions
|
||||
|
||||
class Acquire extends ClientToManagerChannel
|
||||
@ -350,12 +356,19 @@ class Probe extends ManagerToClientChannel
|
||||
}
|
||||
|
||||
object Probe {
|
||||
def apply(p_type: UInt, addr_block: UInt) = {
|
||||
def apply(p_type: UInt, addr_block: UInt): Probe = {
|
||||
val prb = new Probe
|
||||
prb.p_type := p_type
|
||||
prb.addr_block := addr_block
|
||||
prb
|
||||
}
|
||||
def apply(dst: UInt, p_type: UInt, addr_block: UInt): ProbeToDst = {
|
||||
val prb = new ProbeToDst
|
||||
prb.client_id := dst
|
||||
prb.p_type := p_type
|
||||
prb.addr_block := addr_block
|
||||
prb
|
||||
}
|
||||
}
|
||||
|
||||
class Release extends ClientToManagerChannel
|
||||
@ -435,8 +448,8 @@ object Grant {
|
||||
g_type: UInt,
|
||||
client_xact_id: UInt,
|
||||
manager_xact_id: UInt,
|
||||
addr_beat: UInt = UInt(0),
|
||||
data: UInt = UInt(0)): Grant = {
|
||||
addr_beat: UInt,
|
||||
data: UInt): Grant = {
|
||||
val gnt = new Grant
|
||||
gnt.is_builtin_type := is_builtin_type
|
||||
gnt.g_type := g_type
|
||||
@ -446,6 +459,25 @@ object Grant {
|
||||
gnt.data := data
|
||||
gnt
|
||||
}
|
||||
|
||||
def apply(
|
||||
dst: UInt,
|
||||
is_builtin_type: Bool,
|
||||
g_type: UInt,
|
||||
client_xact_id: UInt,
|
||||
manager_xact_id: UInt,
|
||||
addr_beat: UInt = UInt(0),
|
||||
data: UInt = UInt(0)): GrantToDst = {
|
||||
val gnt = new GrantToDst
|
||||
gnt.client_id := dst
|
||||
gnt.is_builtin_type := is_builtin_type
|
||||
gnt.g_type := g_type
|
||||
gnt.client_xact_id := client_xact_id
|
||||
gnt.manager_xact_id := manager_xact_id
|
||||
gnt.addr_beat := addr_beat
|
||||
gnt.data := data
|
||||
gnt
|
||||
}
|
||||
}
|
||||
|
||||
class Finish extends ClientToManagerChannel with HasManagerTransactionId {
|
||||
@ -453,7 +485,14 @@ class Finish extends ClientToManagerChannel with HasManagerTransactionId {
|
||||
def hasMultibeatData(dummy: Int = 0) = Bool(false)
|
||||
}
|
||||
|
||||
// Complete IO definitions for two types of TileLink clients
|
||||
// These subtypes include a field for the source or destination ClientId
|
||||
class AcquireFromSrc extends Acquire with HasClientId
|
||||
class ProbeToDst extends Probe with HasClientId
|
||||
class ReleaseFromSrc extends Release with HasClientId
|
||||
class GrantToDst extends Grant with HasClientId
|
||||
|
||||
// Complete IO definitions for two types of TileLink clients, including
|
||||
// networking headers
|
||||
class UncachedTileLinkIO extends TLBundle {
|
||||
val acquire = new DecoupledIO(new LogicalNetworkIO(new Acquire))
|
||||
val grant = new DecoupledIO(new LogicalNetworkIO(new Grant)).flip
|
||||
@ -479,28 +518,36 @@ class TileLinkIOWrapper extends TLModule {
|
||||
io.out.release.valid := Bool(false)
|
||||
}
|
||||
|
||||
// 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.
|
||||
// This version of TileLinkIO does not contain network headers. The headers
|
||||
// are provided in the top-level that instantiates the clients and network,
|
||||
// probably using a TileLinkClientPort module.
|
||||
// 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 {
|
||||
class ClientUncachedTileLinkIO extends TLBundle {
|
||||
val acquire = new DecoupledIO(new Acquire)
|
||||
val grant = new DecoupledIO(new Grant).flip
|
||||
}
|
||||
|
||||
class HeaderlessTileLinkIO extends HeaderlessUncachedTileLinkIO {
|
||||
class ClientTileLinkIO extends ClientUncachedTileLinkIO {
|
||||
val probe = new DecoupledIO(new Probe).flip
|
||||
val release = new DecoupledIO(new Release)
|
||||
}
|
||||
|
||||
class HeaderlessTileLinkIOWrapper extends TLModule {
|
||||
class ManagerTileLinkIO extends TLBundle {
|
||||
val acquire = new DecoupledIO(new AcquireFromSrc).flip
|
||||
val grant = new DecoupledIO(new GrantToDst)
|
||||
val finish = new DecoupledIO(new Finish).flip
|
||||
val probe = new DecoupledIO(new ProbeToDst)
|
||||
val release = new DecoupledIO(new ReleaseFromSrc).flip
|
||||
}
|
||||
|
||||
class ClientTileLinkIOWrapper extends TLModule {
|
||||
val io = new Bundle {
|
||||
val in = new HeaderlessUncachedTileLinkIO().flip
|
||||
val out = new HeaderlessTileLinkIO
|
||||
val in = new ClientUncachedTileLinkIO().flip
|
||||
val out = new ClientTileLinkIO
|
||||
}
|
||||
io.out.acquire <> io.in.acquire
|
||||
io.out.grant <> io.in.grant
|
||||
@ -509,17 +556,17 @@ class HeaderlessTileLinkIOWrapper extends TLModule {
|
||||
}
|
||||
|
||||
object TileLinkIOWrapper {
|
||||
def apply(utl: HeaderlessUncachedTileLinkIO, p: Parameters): HeaderlessTileLinkIO = {
|
||||
val conv = Module(new HeaderlessTileLinkIOWrapper)(p)
|
||||
def apply(utl: ClientUncachedTileLinkIO, p: Parameters): ClientTileLinkIO = {
|
||||
val conv = Module(new ClientTileLinkIOWrapper)(p)
|
||||
conv.io.in <> utl
|
||||
conv.io.out
|
||||
}
|
||||
def apply(utl: HeaderlessUncachedTileLinkIO): HeaderlessTileLinkIO = {
|
||||
val conv = Module(new HeaderlessTileLinkIOWrapper)
|
||||
def apply(utl: ClientUncachedTileLinkIO): ClientTileLinkIO = {
|
||||
val conv = Module(new ClientTileLinkIOWrapper)
|
||||
conv.io.in <> utl
|
||||
conv.io.out
|
||||
}
|
||||
def apply(tl: HeaderlessTileLinkIO): HeaderlessTileLinkIO = tl
|
||||
def apply(tl: ClientTileLinkIO): ClientTileLinkIO = tl
|
||||
def apply(utl: UncachedTileLinkIO, p: Parameters): TileLinkIO = {
|
||||
val conv = Module(new TileLinkIOWrapper)(p)
|
||||
conv.io.in <> utl
|
||||
@ -533,89 +580,6 @@ object TileLinkIOWrapper {
|
||||
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,
|
||||
getId: LogicalNetworkIO[T] => UInt): Vec[Bool] = {
|
||||
Vec((0 until entries).map { i =>
|
||||
connectDataBeatCounter(in.fire() && getId(in.bits) === 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)
|
||||
//Module.assert(!(do_dec && cnt === UInt(0)), "Decrementing 2way beat counter before ever incrementing")
|
||||
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)))
|
||||
@ -623,8 +587,7 @@ class FinishQueueEntry extends TLBundle {
|
||||
|
||||
class FinishQueue(entries: Int) extends Queue(new FinishQueueEntry, entries)
|
||||
|
||||
class FinishUnit(srcId: Int = 0, outstanding: Int = 2) extends TLModule
|
||||
with HasDataBeatCounters {
|
||||
class FinishUnit(srcId: Int = 0, outstanding: Int = 2) extends TLModule with HasDataBeatCounters {
|
||||
val io = new Bundle {
|
||||
val grant = Decoupled(new LogicalNetworkIO(new Grant)).flip
|
||||
val refill = Decoupled(new Grant)
|
||||
@ -645,12 +608,12 @@ class FinishUnit(srcId: Int = 0, outstanding: Int = 2) extends TLModule
|
||||
// a multibeat Grant. But Grants from multiple managers or transactions may
|
||||
// get interleaved, so we could need a counter for each.
|
||||
val done = if(tlNetworkDoesNotInterleaveBeats) {
|
||||
connectIncomingDataBeatCounter(io.grant)
|
||||
connectIncomingDataBeatCounterWithHeader(io.grant)
|
||||
} else {
|
||||
val entries = 1 << tlClientXactIdBits
|
||||
def getId(g: LogicalNetworkIO[Grant]) = g.payload.client_xact_id
|
||||
assert(getId(io.grant.bits) <= UInt(entries), "Not enough grant beat counters, only " + entries + " entries.")
|
||||
connectIncomingDataBeatCounters(io.grant, entries, getId).reduce(_||_)
|
||||
connectIncomingDataBeatCountersWithHeader(io.grant, entries, getId).reduce(_||_)
|
||||
}
|
||||
val q = Module(new FinishQueue(outstanding))
|
||||
q.io.enq.valid := io.grant.fire() && g.requiresAck() && (!g.hasMultibeatData() || done)
|
||||
@ -670,30 +633,7 @@ class FinishUnit(srcId: Int = 0, outstanding: Int = 2) extends TLModule
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
object ClientTileLinkHeaderCreator {
|
||||
def apply[T <: ClientToManagerChannel with HasCacheBlockAddress : ClassTag](
|
||||
in: DecoupledIO[T],
|
||||
clientId: Int,
|
||||
@ -708,9 +648,24 @@ object TileLinkHeaderOverwriter {
|
||||
}
|
||||
}
|
||||
|
||||
class TileLinkNetworkPort(clientId: Int, addrConvert: UInt => UInt) extends TLModule {
|
||||
object ManagerTileLinkHeaderCreator {
|
||||
def apply[T <: ManagerToClientChannel with HasClientId : ClassTag](
|
||||
in: DecoupledIO[T],
|
||||
managerId: Int,
|
||||
idConvert: 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(managerId)
|
||||
out.bits.header.dst := idConvert(in.bits.client_id)
|
||||
out.valid := in.valid
|
||||
in.ready := out.ready
|
||||
out
|
||||
}
|
||||
}
|
||||
|
||||
class ClientTileLinkNetworkPort(clientId: Int, addrConvert: UInt => UInt) extends TLModule {
|
||||
val io = new Bundle {
|
||||
val client = new HeaderlessTileLinkIO().flip
|
||||
val client = new ClientTileLinkIO().flip
|
||||
val network = new TileLinkIO
|
||||
}
|
||||
|
||||
@ -718,8 +673,8 @@ class TileLinkNetworkPort(clientId: Int, addrConvert: UInt => UInt) extends TLMo
|
||||
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 acq_with_header = ClientTileLinkHeaderCreator(io.client.acquire, clientId, addrConvert)
|
||||
val rel_with_header = ClientTileLinkHeaderCreator(io.client.release, clientId, addrConvert)
|
||||
val prb_without_header = DecoupledLogicalNetworkIOUnwrapper(io.network.probe)
|
||||
val gnt_without_header = finisher.io.refill
|
||||
|
||||
@ -731,58 +686,63 @@ class TileLinkNetworkPort(clientId: Int, addrConvert: UInt => UInt) extends TLMo
|
||||
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 ManagerTileLinkNetworkPort(managerId: Int, idConvert: UInt => UInt) extends TLModule {
|
||||
val io = new Bundle {
|
||||
val manager = new ManagerTileLinkIO().flip
|
||||
val network = new TileLinkIO().flip
|
||||
}
|
||||
io.network.grant <> ManagerTileLinkHeaderCreator(io.manager.grant, managerId, (u: UInt) => u)
|
||||
io.network.probe <> ManagerTileLinkHeaderCreator(io.manager.probe, managerId, idConvert)
|
||||
io.manager.acquire.bits.client_id := io.network.acquire.bits.header.src
|
||||
io.manager.acquire <> DecoupledLogicalNetworkIOUnwrapper(io.network.acquire)
|
||||
io.manager.release.bits.client_id := io.network.release.bits.header.src
|
||||
io.manager.release <> DecoupledLogicalNetworkIOUnwrapper(io.network.release)
|
||||
io.manager.finish <> DecoupledLogicalNetworkIOUnwrapper(io.network.finish)
|
||||
}
|
||||
|
||||
class TileLinkEnqueuer(depths: (Int, Int, Int, Int, Int)) extends Module {
|
||||
case class TileLinkDepths(acq: Int, prb: Int, rel: Int, gnt: Int, fin: Int)
|
||||
|
||||
class TileLinkEnqueuer(depths: TileLinkDepths) extends Module {
|
||||
val io = new Bundle {
|
||||
val client = new TileLinkIO().flip
|
||||
val manager = new TileLinkIO
|
||||
}
|
||||
io.manager.acquire <> (if(depths._1 > 0) Queue(io.client.acquire, depths._1) else io.client.acquire)
|
||||
io.client.probe <> (if(depths._2 > 0) Queue(io.manager.probe, depths._2) else io.manager.probe)
|
||||
io.manager.release <> (if(depths._3 > 0) Queue(io.client.release, depths._3) else io.client.release)
|
||||
io.client.grant <> (if(depths._4 > 0) Queue(io.manager.grant, depths._4) else io.manager.grant)
|
||||
io.manager.finish <> (if(depths._5 > 0) Queue(io.client.finish, depths._5) else io.client.finish)
|
||||
io.manager.acquire <> (if(depths.acq > 0) Queue(io.client.acquire, depths.acq) else io.client.acquire)
|
||||
io.client.probe <> (if(depths.prb > 0) Queue(io.manager.probe, depths.prb) else io.manager.probe)
|
||||
io.manager.release <> (if(depths.rel > 0) Queue(io.client.release, depths.rel) else io.client.release)
|
||||
io.client.grant <> (if(depths.gnt > 0) Queue(io.manager.grant, depths.gnt) else io.manager.grant)
|
||||
io.manager.finish <> (if(depths.fin > 0) Queue(io.client.finish, depths.fin) else io.client.finish)
|
||||
}
|
||||
|
||||
object TileLinkEnqueuer {
|
||||
def apply(in: TileLinkIO, depths: (Int, Int, Int, Int, Int))(p: Parameters): TileLinkIO = {
|
||||
def apply(in: TileLinkIO, depths: TileLinkDepths)(p: Parameters): TileLinkIO = {
|
||||
val t = Module(new TileLinkEnqueuer(depths))(p)
|
||||
t.io.client <> in
|
||||
t.io.manager
|
||||
}
|
||||
def apply(in: TileLinkIO, depth: Int)(p: Parameters): TileLinkIO = {
|
||||
apply(in, (depth, depth, depth, depth, depth))(p)
|
||||
apply(in, TileLinkDepths(depth, depth, depth, depth, depth))(p)
|
||||
}
|
||||
}
|
||||
|
||||
abstract trait HasArbiterTypes {
|
||||
/** Utility functions for constructing TileLinkIO arbiters */
|
||||
trait TileLinkArbiterLike extends TileLinkParameters {
|
||||
// Some shorthand type variables
|
||||
type ManagerSourcedWithId = ManagerToClientChannel with HasClientTransactionId
|
||||
type ClientSourcedWithId = ClientToManagerChannel with HasClientTransactionId
|
||||
type ClientSourcedWithIdAndData = ClientToManagerChannel with
|
||||
HasClientTransactionId with
|
||||
HasTileLinkData
|
||||
}
|
||||
type ClientSourcedWithIdAndData = ClientToManagerChannel with HasClientTransactionId with HasTileLinkData
|
||||
|
||||
// Utility functions for constructing TileLinkIO arbiters
|
||||
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
|
||||
val arbN: Int // The number of ports on the client side
|
||||
|
||||
// These abstract funcs are filled in depending on whether the arbiter mucks with the
|
||||
// outgoing client ids to track sourcing 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](
|
||||
// The following functions are all wiring helpers for each of the different types of TileLink channels
|
||||
|
||||
def hookupClientSource[M <: ClientSourcedWithIdAndData](
|
||||
clts: Seq[DecoupledIO[LogicalNetworkIO[M]]],
|
||||
mngr: DecoupledIO[LogicalNetworkIO[M]]) {
|
||||
def hasData(m: LogicalNetworkIO[M]) = m.payload.hasMultibeatData()
|
||||
@ -796,7 +756,7 @@ trait TileLinkArbiterLike extends HasArbiterTypes with TileLinkParameters{
|
||||
arb.io.out <> mngr
|
||||
}
|
||||
|
||||
def hookupClientSourceHeaderless[M <: ClientSourcedWithIdAndData : ClassTag](
|
||||
def hookupClientSourceHeaderless[M <: ClientSourcedWithIdAndData](
|
||||
clts: Seq[DecoupledIO[M]],
|
||||
mngr: DecoupledIO[M]) {
|
||||
def hasData(m: M) = m.hasMultibeatData()
|
||||
@ -854,23 +814,20 @@ trait TileLinkArbiterLike extends HasArbiterTypes with TileLinkParameters{
|
||||
}
|
||||
}
|
||||
|
||||
def hookupManagerSourceBroadcast[M <: Data](
|
||||
clts: Seq[DecoupledIO[M]],
|
||||
mngr: DecoupledIO[M]) {
|
||||
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]) {
|
||||
def hookupFinish[M <: LogicalNetworkIO[Finish]]( 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.in <> clts
|
||||
arb.io.out <> mngr
|
||||
}
|
||||
}
|
||||
|
||||
/** Abstract base case for any Arbiters that have UncachedTileLinkIOs */
|
||||
abstract class UncachedTileLinkIOArbiter(val arbN: Int) extends Module with TileLinkArbiterLike {
|
||||
val io = new Bundle {
|
||||
val in = Vec.fill(arbN){new UncachedTileLinkIO}.flip
|
||||
@ -881,6 +838,7 @@ abstract class UncachedTileLinkIOArbiter(val arbN: Int) extends Module with Tile
|
||||
hookupManagerSourceWithId(io.in.map(_.grant), io.out.grant)
|
||||
}
|
||||
|
||||
/** Abstract base case for any Arbiters that have cached TileLinkIOs */
|
||||
abstract class TileLinkIOArbiter(val arbN: Int) extends Module with TileLinkArbiterLike {
|
||||
val io = new Bundle {
|
||||
val in = Vec.fill(arbN){new TileLinkIO}.flip
|
||||
@ -893,31 +851,7 @@ abstract class TileLinkIOArbiter(val arbN: Int) extends Module with TileLinkArbi
|
||||
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
|
||||
/** Appends the port index of the arbiter to the client_xact_id */
|
||||
trait AppendsArbiterId extends TileLinkArbiterLike {
|
||||
def clientSourcedClientXactId(in: ClientSourcedWithId, id: Int) =
|
||||
Cat(in.client_xact_id, UInt(id, log2Up(arbN)))
|
||||
@ -926,24 +860,125 @@ trait AppendsArbiterId extends TileLinkArbiterLike {
|
||||
def arbIdx(in: ManagerSourcedWithId) = in.client_xact_id(log2Up(arbN)-1,0).toUInt
|
||||
}
|
||||
|
||||
// Uses the client_xact_id as is (assumes it has been set to port index)
|
||||
/** Uses the client_xact_id as is (assumes it has been set to port index) */
|
||||
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
|
||||
/** Overwrites some default client_xact_id with the port idx */
|
||||
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
|
||||
}
|
||||
|
||||
// Mix-in id generation traits to make concrete arbiter classes
|
||||
// Now we can mix-in thevarious id-generation traits to make concrete arbiter classes
|
||||
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
|
||||
|
||||
/** Concrete uncached client-side arbiter that appends the arbiter's port id to client_xact_id */
|
||||
class ClientUncachedTileLinkIOArbiter(val arbN: Int) extends Module with TileLinkArbiterLike with AppendsArbiterId {
|
||||
val io = new Bundle {
|
||||
val in = Vec.fill(arbN){new ClientUncachedTileLinkIO}.flip
|
||||
val out = new ClientUncachedTileLinkIO
|
||||
}
|
||||
hookupClientSourceHeaderless(io.in.map(_.acquire), io.out.acquire)
|
||||
hookupManagerSourceHeaderlessWithId(io.in.map(_.grant), io.out.grant)
|
||||
}
|
||||
|
||||
/** Concrete client-side arbiter that appends the arbiter's port id to client_xact_id */
|
||||
class ClientTileLinkIOArbiter(val arbN: Int) extends Module with TileLinkArbiterLike with AppendsArbiterId {
|
||||
val io = new Bundle {
|
||||
val in = Vec.fill(arbN){new ClientTileLinkIO}.flip
|
||||
val out = new ClientTileLinkIO
|
||||
}
|
||||
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)
|
||||
}
|
||||
|
||||
/** Utility trait containing wiring functions to keep track of how many data beats have
|
||||
* been sent or recieved over a particular TileLinkChannel or pair of channels.
|
||||
*
|
||||
* Won't count message types that don't have data.
|
||||
* Used in XactTrackers and FinishUnit.
|
||||
*/
|
||||
trait HasDataBeatCounters {
|
||||
type HasBeat = TileLinkChannel with HasTileLinkBeatId
|
||||
|
||||
/** Returns the current count on this channel and when a message is done
|
||||
* @param inc increment the counter (usually .valid or .fire())
|
||||
* @param data the actual channel data
|
||||
* @param beat count to return for single-beat messages
|
||||
*/
|
||||
def connectDataBeatCounter[S <: TileLinkChannel](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)
|
||||
}
|
||||
|
||||
/** Counter for beats on outgoing DecoupledIOs */
|
||||
def connectOutgoingDataBeatCounter[T <: TileLinkChannel](in: DecoupledIO[T], beat: UInt = UInt(0)): (UInt, Bool) =
|
||||
connectDataBeatCounter(in.fire(), in.bits, beat)
|
||||
|
||||
/** Returns done but not cnt. Use the addr_beat subbundle instead of cnt for beats on
|
||||
* incoming channels in case of network reordering.
|
||||
*/
|
||||
def connectIncomingDataBeatCounter[T <: TileLinkChannel](in: DecoupledIO[T]): Bool =
|
||||
connectDataBeatCounter(in.fire(), in.bits, UInt(0))._2
|
||||
|
||||
/** Counter for beats on incoming DecoupledIO[LogicalNetworkIO[]]s returns done */
|
||||
def connectIncomingDataBeatCounterWithHeader[T <: TileLinkChannel](in: DecoupledIO[LogicalNetworkIO[T]]): Bool =
|
||||
connectDataBeatCounter(in.fire(), in.bits.payload, UInt(0))._2
|
||||
|
||||
/** If the network might interleave beats from different messages, we need a Vec of counters,
|
||||
* one for every outstanding message id that might be interleaved.
|
||||
*
|
||||
* @param getId mapping from Message to counter id
|
||||
*/
|
||||
def connectIncomingDataBeatCountersWithHeader[T <: TileLinkChannel with HasClientTransactionId](
|
||||
in: DecoupledIO[LogicalNetworkIO[T]],
|
||||
entries: Int,
|
||||
getId: LogicalNetworkIO[T] => UInt): Vec[Bool] = {
|
||||
Vec((0 until entries).map { i =>
|
||||
connectDataBeatCounter(in.fire() && getId(in.bits) === UInt(i), in.bits.payload, UInt(0))._2
|
||||
})
|
||||
}
|
||||
|
||||
/** Provides counters on two channels, as well a meta-counter that tracks how many
|
||||
* messages have been sent over the up channel but not yet responded to over the down channel
|
||||
*
|
||||
* @param max max number of outstanding ups with no down
|
||||
* @param up outgoing channel
|
||||
* @param down incoming channel
|
||||
* @param beat overrides cnts on single-beat messages
|
||||
* @param track whether up's message should be tracked
|
||||
* @return a tuple containing whether their are outstanding messages, up's count,
|
||||
* up's done, down's count, down's done
|
||||
*/
|
||||
def connectTwoWayBeatCounter[T <: TileLinkChannel, S <: TileLinkChannel](
|
||||
max: Int,
|
||||
up: DecoupledIO[T],
|
||||
down: DecoupledIO[S],
|
||||
beat: UInt = UInt(0),
|
||||
track: T => Bool = (t: T) => 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, beat)
|
||||
val (down_idx, down_done) = connectDataBeatCounter(down.fire(), down.bits, beat)
|
||||
val do_inc = up_done && track(up.bits)
|
||||
val do_dec = down_done
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user