L2AcquireTracker refactor to support merging Gets and Puts into Prefetches of the correct type.
Transaction metadata for primary and seconday misses now stored in the secondary miss queue. Added BuiltInAcquireBuilder factory.
This commit is contained in:
parent
03fa06e6e7
commit
64aaf71b06
@ -414,7 +414,7 @@ abstract class L2XactTracker(implicit p: Parameters) extends XactTracker()(p)
|
||||
|
||||
def addPendingBitWhenBeatHasPartialWritemask(in: DecoupledIO[AcquireFromSrc]): UInt = {
|
||||
val a = in.bits
|
||||
val isPartial = a.wmask() != Acquire.fullWriteMask
|
||||
val isPartial = a.wmask() =/= Acquire.fullWriteMask
|
||||
addPendingBitWhenBeat(in.fire() && isPartial && Bool(ignoresWriteMask), a)
|
||||
}
|
||||
|
||||
@ -533,44 +533,62 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
val state = Reg(init=s_idle)
|
||||
|
||||
// State holding transaction metadata
|
||||
val xact = Reg(new BufferedAcquireFromSrc()(p.alterPartial({ case TLId => p(InnerTLId) })))
|
||||
val data_buffer = Reg(init=Vec.fill(innerDataBeats)(UInt(0, width = innerDataBits)))
|
||||
val wmask_buffer = Reg(init=Vec.fill(innerDataBeats)(UInt(0, width = innerDataBits/8)))
|
||||
val xact_addr_block = Reg{ io.inner.acquire.bits.addr_block }
|
||||
val xact_tag_match = Reg{ Bool() }
|
||||
val xact_way_en = Reg{ Bits(width = nWays) }
|
||||
val xact_old_meta = Reg{ new L2Metadata }
|
||||
val pending_coh = Reg{ xact_old_meta.coh }
|
||||
val xact_allocate = Reg{ Bool() }
|
||||
val xact_amo_shift_bits = Reg{ UInt() }
|
||||
val xact_op_code = Reg{ UInt() }
|
||||
val xact_addr_byte = Reg{ UInt() }
|
||||
val xact_op_size = Reg{ UInt() }
|
||||
|
||||
// Secondary miss queue
|
||||
val ignt_q = Module(new Queue(new SecondaryMissInfo()(p.alterPartial({ case TLId => p(InnerTLId) })),
|
||||
nSecondaryMisses))
|
||||
// Miss queue holds transaction metadata used to make grants
|
||||
val ignt_q = Module(new Queue(
|
||||
new SecondaryMissInfo()(p.alterPartial({ case TLId => p(InnerTLId) })),
|
||||
1 + nSecondaryMisses))
|
||||
|
||||
// State holding progress made on processing this transaction
|
||||
val iacq_data_done = connectIncomingDataBeatCounter(io.inner.acquire)
|
||||
// Some accessor wires derived from the the above state
|
||||
val xact = ignt_q.io.deq.bits
|
||||
val xact_addr_idx = xact_addr_block(idxMSB,idxLSB)
|
||||
val xact_addr_tag = xact_addr_block >> UInt(idxBits)
|
||||
|
||||
// Counters and scoreboard tracking progress made on processing this transaction
|
||||
val pending_irels = connectTwoWayBeatCounter(
|
||||
max = io.inner.tlNCachingClients,
|
||||
up = io.inner.probe,
|
||||
down = io.inner.release)._1
|
||||
|
||||
val (pending_ognt, oacq_data_idx, oacq_data_done, ognt_data_idx, ognt_data_done) =
|
||||
connectTwoWayBeatCounter(
|
||||
max = 1,
|
||||
up = io.outer.acquire,
|
||||
down = io.outer.grant,
|
||||
beat = xact.addr_beat)
|
||||
val (ignt_data_idx, ignt_data_done) = connectOutgoingDataBeatCounter(io.inner.grant, ignt_q.io.deq.bits.addr_beat)
|
||||
|
||||
val (ignt_data_idx, ignt_data_done) = connectOutgoingDataBeatCounter(
|
||||
out = io.inner.grant,
|
||||
beat = ignt_q.io.deq.bits.addr_beat)
|
||||
|
||||
val pending_ifins = connectTwoWayBeatCounter(
|
||||
max = nSecondaryMisses,
|
||||
up = io.inner.grant,
|
||||
down = io.inner.finish,
|
||||
track = (g: Grant) => g.requiresAck())._1
|
||||
|
||||
val pending_puts = Reg(init=Bits(0, width = io.inner.tlDataBeats))
|
||||
val pending_iprbs = Reg(init = Bits(0, width = io.inner.tlNCachingClients))
|
||||
val pending_reads = Reg(init=Bits(0, width = io.inner.tlDataBeats))
|
||||
val pending_writes = Reg(init=Bits(0, width = io.inner.tlDataBeats))
|
||||
val pending_resps = Reg(init=Bits(0, width = io.inner.tlDataBeats))
|
||||
val pending_ignt_data = Reg(init=Bits(0, width = io.inner.tlDataBeats))
|
||||
val pending_ignt_ack = Reg(init = Bool(false))
|
||||
val pending_meta_write = Reg(Bool())
|
||||
val ignt_data_ready = Reg(init=Bits(0, width = io.inner.tlDataBeats))
|
||||
val ignt_ack_ready = Reg(init = Bool(false))
|
||||
val pending_meta_write = Reg(init = Bool(false))
|
||||
|
||||
// Used to decide when to escape from s_busy
|
||||
val all_pending_done =
|
||||
!(pending_reads.orR ||
|
||||
pending_writes.orR ||
|
||||
@ -584,16 +602,14 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
// Provide a single ALU per tracker to merge Puts and AMOs with data being
|
||||
// refilled, written back, or extant in the cache
|
||||
val amoalu = Module(new AMOALU(rhsIsAligned = true))
|
||||
amoalu.io.addr := xact.full_addr()
|
||||
amoalu.io.cmd := xact.op_code()
|
||||
amoalu.io.typ := xact.op_size()
|
||||
amoalu.io.addr := Cat(xact_addr_block, xact.addr_beat, xact_addr_byte)
|
||||
amoalu.io.cmd := xact_op_code
|
||||
amoalu.io.typ := xact_op_size
|
||||
amoalu.io.lhs := io.data.resp.bits.data // default, overwritten by calls to mergeData
|
||||
amoalu.io.rhs := xact.data_buffer.head // default, overwritten by calls to mergeData
|
||||
amoalu.io.rhs := data_buffer.head // default, overwritten by calls to mergeData
|
||||
val amo_result = Reg(init = UInt(0, xact.tlDataBits))
|
||||
|
||||
// Utility functions for updating the data and metadata that will be kept in
|
||||
// the cache or granted to the original requestor after this transaction:
|
||||
|
||||
// Utility function for updating the metadata that will be kept in this cache
|
||||
def updatePendingCohWhen(flag: Bool, next: HierarchicalMetadata) {
|
||||
when(flag && pending_coh != next) {
|
||||
pending_meta_write := Bool(true)
|
||||
@ -601,79 +617,88 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
}
|
||||
}
|
||||
|
||||
// Defined here because of Chisel default wire demands, used in s_meta_resp
|
||||
val pending_coh_on_hit = HierarchicalMetadata(
|
||||
io.meta.resp.bits.meta.coh.inner,
|
||||
io.meta.resp.bits.meta.coh.outer.onHit(xact_op_code))
|
||||
|
||||
val pending_coh_on_miss = HierarchicalMetadata.onReset
|
||||
|
||||
// Utility function for applying any buffered stored data to the cache line
|
||||
// before storing it back into the data array
|
||||
def mergeData(dataBits: Int)(beat: UInt, incoming: UInt) {
|
||||
val old_data = incoming // Refilled, written back, or de-cached data
|
||||
val new_data = xact.data_buffer(beat) // Newly Put data is already in the buffer
|
||||
amoalu.io.lhs := old_data >> xact.amo_shift_bits()
|
||||
amoalu.io.rhs := new_data >> xact.amo_shift_bits()
|
||||
val new_data = data_buffer(beat) // Newly Put data is already in the buffer
|
||||
amoalu.io.lhs := old_data >> xact_amo_shift_bits
|
||||
amoalu.io.rhs := new_data >> xact_amo_shift_bits
|
||||
val wmask = FillInterleaved(8, wmask_buffer(beat))
|
||||
xact.data_buffer(beat) := ~wmask & old_data |
|
||||
data_buffer(beat) := ~wmask & old_data |
|
||||
wmask & Mux(xact.isBuiltInType(Acquire.putAtomicType),
|
||||
amoalu.io.out << xact.amo_shift_bits(),
|
||||
amoalu.io.out << xact_amo_shift_bits,
|
||||
new_data)
|
||||
wmask_buffer(beat) := ~UInt(0, wmask_buffer.head.getWidth)
|
||||
when(xact.is(Acquire.putAtomicType) && xact.addr_beat === beat) { amo_result := old_data }
|
||||
}
|
||||
|
||||
// TODO: Deal with the possibility that rowBits != tlDataBits
|
||||
def mergeDataInternal[T <: L2HellaCacheBundle with HasL2Data with HasL2BeatAddr](in: ValidIO[T]) {
|
||||
when(in.valid) { mergeData(rowBits)(in.bits.addr_beat, in.bits.data) }
|
||||
}
|
||||
|
||||
def mergeDataInner[T <: TLBundle with HasTileLinkData with HasTileLinkBeatId](in: DecoupledIO[T]) {
|
||||
when(in.fire() && in.bits.hasData()) {
|
||||
mergeData(innerDataBits)(in.bits.addr_beat, in.bits.data)
|
||||
}
|
||||
}
|
||||
|
||||
def mergeDataOuter[T <: TLBundle with HasTileLinkData with HasTileLinkBeatId](in: DecoupledIO[T]) {
|
||||
when(in.fire() && in.bits.hasData()) {
|
||||
mergeData(outerDataBits)(in.bits.addr_beat, in.bits.data)
|
||||
}
|
||||
}
|
||||
|
||||
def addPendingBitFromBufferedAcquire(xact: AcquireMetadata): UInt =
|
||||
Mux(xact.hasMultibeatData(),
|
||||
Fill(innerDataBeats, UInt(1, 1)),
|
||||
UIntToOH(xact.addr_beat))
|
||||
// and Puts-under-Put, and either may also merge witha preceding prefetch
|
||||
// that requested the correct permissions (via op_code)
|
||||
def acquiresAreMergeable(sec: AcquireMetadata): Bool = {
|
||||
val allowedTypes = List((Acquire.getType, Acquire.getType),
|
||||
(Acquire.putType, Acquire.putType),
|
||||
(Acquire.putBlockType, Acquire.putBlockType),
|
||||
(Acquire.prefetchType, Acquire.prefetchType),
|
||||
(Acquire.prefetchType, Acquire.getType),
|
||||
(Acquire.prefetchType, Acquire.putType),
|
||||
(Acquire.prefetchType, Acquire.putBlockType))
|
||||
allowedTypes.map { case(a, b) => xact.isBuiltInType(a) && sec.isBuiltInType(b) }.reduce(_||_) &&
|
||||
xact_op_code === sec.op_code() &&
|
||||
sec.conflicts(xact_addr_block) &&
|
||||
(xact_allocate || xact.isBuiltInType(Acquire.putBlockType))
|
||||
}
|
||||
|
||||
// Actual transaction processing logic begins here:
|
||||
//
|
||||
// First, take care of accpeting new requires or secondary misses
|
||||
// For now, the only allowed secondary miss types are Gets-under-Get
|
||||
// and Puts-under-Put from the same client
|
||||
val can_merge_iacq_get = (xact.isBuiltInType(Acquire.getType) &&
|
||||
io.iacq().isBuiltInType(Acquire.getType)) &&
|
||||
xact.client_id === io.iacq().client_id && //TODO remove
|
||||
xact.conflicts(io.iacq()) &&
|
||||
state != s_idle && state != s_meta_write &&
|
||||
!all_pending_done &&
|
||||
xact.allocate() &&
|
||||
!io.inner.release.fire() &&
|
||||
!io.outer.grant.fire() &&
|
||||
!io.data.resp.valid &&
|
||||
ignt_q.io.enq.ready
|
||||
// First, take care of accpeting new acquires or secondary misses
|
||||
val iacq_can_merge = acquiresAreMergeable(io.iacq()) &&
|
||||
state =/= s_idle && state =/= s_meta_write &&
|
||||
!all_pending_done &&
|
||||
!io.inner.release.fire() &&
|
||||
!io.outer.grant.fire() &&
|
||||
!io.data.resp.valid &&
|
||||
ignt_q.io.enq.ready && ignt_q.io.deq.valid
|
||||
|
||||
// This logic also allows the tail beats of a PutBlock to be merged in
|
||||
val can_merge_iacq_put = ((xact.isBuiltInType(Acquire.putType) &&
|
||||
io.iacq().isBuiltInType(Acquire.putType)) ||
|
||||
(xact.isBuiltInType(Acquire.putBlockType) &&
|
||||
io.iacq().isBuiltInType(Acquire.putBlockType))) &&
|
||||
xact.client_id === io.iacq().client_id && //TODO remove
|
||||
xact.conflicts(io.iacq()) &&
|
||||
state != s_idle && state != s_meta_write &&
|
||||
!all_pending_done &&
|
||||
(xact.allocate() || xact.isBuiltInType(Acquire.putBlockType)) &&
|
||||
!io.inner.release.fire() &&
|
||||
!io.outer.grant.fire() &&
|
||||
!io.data.resp.valid &&
|
||||
ignt_q.io.enq.ready
|
||||
io.inner.acquire.ready := state === s_idle || iacq_can_merge
|
||||
|
||||
io.inner.acquire.ready := state === s_idle ||
|
||||
can_merge_iacq_put ||
|
||||
can_merge_iacq_get
|
||||
// Handling of primary and secondary misses' data and write mask merging
|
||||
when(io.inner.acquire.fire() && io.iacq().hasData()) {
|
||||
val beat = io.iacq().addr_beat
|
||||
val full = FillInterleaved(8, io.iacq().wmask())
|
||||
data_buffer(beat) := (~full & data_buffer(beat)) | (full & io.iacq().data)
|
||||
wmask_buffer(beat) := io.iacq().wmask() | wmask_buffer(beat) // assumes wmask_buffer is zeroed
|
||||
}
|
||||
|
||||
// Enqueue secondary miss information
|
||||
ignt_q.io.enq.valid := iacq_data_done
|
||||
ignt_q.io.enq.bits.client_xact_id := io.iacq().client_xact_id
|
||||
ignt_q.io.enq.bits.addr_beat := io.iacq().addr_beat
|
||||
// TODO add ignt.dst <- iacq.src
|
||||
// Enqueue some metadata information that we'll use to make coherence updates with later
|
||||
ignt_q.io.enq.valid := Mux(io.iacq().hasMultibeatData(),
|
||||
io.inner.acquire.fire() && state === s_idle,
|
||||
io.inner.acquire.fire())
|
||||
ignt_q.io.enq.bits := io.iacq()
|
||||
|
||||
// Track whether any beats are missing from a PutBlock
|
||||
pending_puts := (pending_puts & dropPendingBitWhenBeatHasData(io.inner.acquire))
|
||||
@ -681,13 +706,13 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
// Begin a transaction by getting the current block metadata
|
||||
io.meta.read.valid := state === s_meta_read
|
||||
io.meta.read.bits.id := UInt(trackerId)
|
||||
io.meta.read.bits.idx := xact.addr_block(idxMSB,idxLSB)
|
||||
io.meta.read.bits.tag := xact.addr_block >> UInt(idxBits)
|
||||
io.meta.read.bits.idx := xact_addr_idx
|
||||
io.meta.read.bits.tag := xact_addr_tag
|
||||
|
||||
// Issue a request to the writeback unit
|
||||
io.wb.req.valid := state === s_wb_req
|
||||
io.wb.req.bits.id := UInt(trackerId)
|
||||
io.wb.req.bits.idx := xact.addr_block(idxMSB,idxLSB)
|
||||
io.wb.req.bits.idx := xact_addr_idx
|
||||
io.wb.req.bits.tag := xact_old_meta.tag
|
||||
io.wb.req.bits.coh := xact_old_meta.coh
|
||||
io.wb.req.bits.way_en := xact_way_en
|
||||
@ -696,7 +721,7 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
pending_iprbs := pending_iprbs & dropPendingBitAtDest(io.inner.probe)
|
||||
val curr_probe_dst = PriorityEncoder(pending_iprbs)
|
||||
io.inner.probe.valid := state === s_inner_probe && pending_iprbs.orR
|
||||
io.inner.probe.bits := pending_coh.inner.makeProbe(curr_probe_dst, xact)
|
||||
io.inner.probe.bits := pending_coh.inner.makeProbe(curr_probe_dst, xact, xact_addr_block)
|
||||
|
||||
// Handle incoming releases from clients, which may reduce sharer counts
|
||||
// and/or write back dirty data
|
||||
@ -709,61 +734,101 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
updatePendingCohWhen(io.inner.release.fire(), pending_coh_on_irel)
|
||||
mergeDataInner(io.inner.release)
|
||||
|
||||
val outerParams = p.alterPartial({ case TLId => p(OuterTLId) })
|
||||
// Handle misses or coherence permission upgrades by initiating a new transaction in the outer memory:
|
||||
//
|
||||
// If we're allocating in this cache, we can use the current metadata
|
||||
// to make an appropriate custom Acquire, otherwise we copy over the
|
||||
// built-in Acquire from the inner TL to the outer TL
|
||||
io.outer.acquire.valid := state === s_outer_acquire &&
|
||||
(xact.allocate() || !pending_puts(oacq_data_idx))
|
||||
io.outer.acquire.bits := Mux(xact.allocate(),
|
||||
(xact_allocate || !pending_puts(oacq_data_idx))
|
||||
io.outer.acquire.bits := Mux(xact_allocate,
|
||||
xact_old_meta.coh.outer.makeAcquire(
|
||||
op_code = xact_op_code,
|
||||
client_xact_id = UInt(0),
|
||||
addr_block = xact.addr_block,
|
||||
op_code = xact.op_code()),
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
addr_block = xact_addr_block),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = xact.a_type,
|
||||
client_xact_id = UInt(0),
|
||||
addr_block = xact.addr_block,
|
||||
addr_beat = xact.addr_beat,
|
||||
union = xact.union)(outerParams))
|
||||
|
||||
io.oacq().data := xact.data_buffer(oacq_data_idx)
|
||||
addr_block = xact_addr_block,
|
||||
addr_beat = oacq_data_idx,
|
||||
data = data_buffer(oacq_data_idx),
|
||||
addr_byte = xact_addr_byte,
|
||||
operand_size = xact_op_size,
|
||||
opcode = xact_op_code,
|
||||
wmask = wmask_buffer(oacq_data_idx),
|
||||
alloc = Bool(false))
|
||||
(p.alterPartial({ case TLId => p(OuterTLId)})))
|
||||
|
||||
// Handle the response from outer memory
|
||||
io.outer.grant.ready := state === s_busy
|
||||
val pending_coh_on_ognt = HierarchicalMetadata(
|
||||
ManagerMetadata.onReset,
|
||||
pending_coh.outer.onGrant(io.outer.grant.bits, xact.op_code()))
|
||||
pending_coh.outer.onGrant(io.outer.grant.bits, xact_op_code))
|
||||
updatePendingCohWhen(ognt_data_done, pending_coh_on_ognt)
|
||||
mergeDataOuter(io.outer.grant)
|
||||
|
||||
// Going back to the original inner transaction, we can issue a Grant as
|
||||
// Going back to the original inner transaction:
|
||||
// We read from the the cache at this level if data wasn't written back or refilled.
|
||||
// We may still merge further Gets, requiring further beats to be read.
|
||||
// If ECC requires a full writemask, we'll read out data on partial writes as well.
|
||||
pending_reads := (pending_reads &
|
||||
dropPendingBit(io.data.read) &
|
||||
dropPendingBitWhenBeatHasData(io.inner.release) &
|
||||
dropPendingBitWhenBeatHasData(io.outer.grant)) |
|
||||
addPendingBitWhenBeatIsGetOrAtomic(io.inner.acquire) |
|
||||
addPendingBitWhenBeatHasPartialWritemask(io.inner.acquire)
|
||||
val curr_read_beat = PriorityEncoder(pending_reads)
|
||||
io.data.read.valid := state === s_busy && pending_reads.orR && !pending_ognt
|
||||
io.data.read.bits.id := UInt(trackerId)
|
||||
io.data.read.bits.way_en := xact_way_en
|
||||
io.data.read.bits.addr_idx := xact_addr_idx
|
||||
io.data.read.bits.addr_beat := curr_read_beat
|
||||
|
||||
pending_resps := (pending_resps & dropPendingBitInternal(io.data.resp)) |
|
||||
addPendingBitInternal(io.data.read)
|
||||
mergeDataInternal(io.data.resp)
|
||||
|
||||
// We write data to the cache at this level if it was Put here with allocate flag,
|
||||
// written back dirty, or refilled from outer memory.
|
||||
pending_writes := (pending_writes & dropPendingBit(io.data.write)) |
|
||||
addPendingBitWhenBeatHasDataAndAllocs(io.inner.acquire) |
|
||||
addPendingBitWhenBeatHasData(io.inner.release) |
|
||||
addPendingBitWhenBeatHasData(io.outer.grant, xact_allocate)
|
||||
val curr_write_beat = PriorityEncoder(pending_writes)
|
||||
io.data.write.valid := state === s_busy &&
|
||||
pending_writes.orR &&
|
||||
!pending_ognt &&
|
||||
!pending_reads(curr_write_beat) &&
|
||||
!pending_resps(curr_write_beat)
|
||||
io.data.write.bits.id := UInt(trackerId)
|
||||
io.data.write.bits.way_en := xact_way_en
|
||||
io.data.write.bits.addr_idx := xact_addr_idx
|
||||
io.data.write.bits.addr_beat := curr_write_beat
|
||||
io.data.write.bits.wmask := wmask_buffer(curr_write_beat)
|
||||
io.data.write.bits.data := data_buffer(curr_write_beat)
|
||||
|
||||
// soon as the data is released, granted, put, or read from the cache
|
||||
pending_ignt_data := pending_ignt_data |
|
||||
ignt_data_ready := ignt_data_ready |
|
||||
addPendingBitWhenBeatHasData(io.inner.release) |
|
||||
addPendingBitWhenBeatHasData(io.outer.grant) |
|
||||
addPendingBitInternal(io.data.resp)
|
||||
// We can issue a grant for a pending write once the write is committed
|
||||
pending_ignt_ack := pending_ignt_ack |
|
||||
ignt_ack_ready := ignt_ack_ready |
|
||||
io.data.write.fire() |
|
||||
io.outer.grant.fire() && !io.outer.grant.bits.hasData()
|
||||
ignt_q.io.deq.ready := ignt_data_done
|
||||
io.inner.grant.valid := state === s_busy &&
|
||||
ignt_q.io.deq.valid &&
|
||||
Mux(io.ignt().hasData(),
|
||||
pending_ignt_data(ignt_data_idx),
|
||||
pending_ignt_ack)
|
||||
ignt_data_ready(ignt_data_idx),
|
||||
ignt_ack_ready)
|
||||
// Make the Grant message using the data stored in the secondary miss queue
|
||||
io.inner.grant.bits := pending_coh.inner.makeGrant(
|
||||
pri = xact,
|
||||
sec = ignt_q.io.deq.bits,
|
||||
manager_xact_id = UInt(trackerId),
|
||||
data = Mux(xact.is(Acquire.putAtomicType),
|
||||
amo_result,
|
||||
xact.data_buffer(ignt_data_idx)))
|
||||
data_buffer(ignt_data_idx)))
|
||||
io.inner.grant.bits.addr_beat := ignt_data_idx // override based on outgoing counter
|
||||
|
||||
val pending_coh_on_ignt = HierarchicalMetadata(
|
||||
@ -776,72 +841,22 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
// We must wait for as many Finishes as we sent Grants
|
||||
io.inner.finish.ready := state === s_busy
|
||||
|
||||
// We read from the the cache at this level if data wasn't written back or refilled.
|
||||
// We may merge Gets, requiring further beats to be read.
|
||||
// If ECC requires a full writemask, we'll read out data on partial writes as well.
|
||||
pending_reads := (pending_reads &
|
||||
dropPendingBit(io.data.read) &
|
||||
dropPendingBitWhenBeatHasData(io.inner.release) &
|
||||
dropPendingBitWhenBeatHasData(io.outer.grant)) |
|
||||
addPendingBitWhenBeatIsGetOrAtomic(io.inner.acquire) |
|
||||
addPendingBitWhenBeatHasPartialWritemask(io.inner.acquire)
|
||||
val curr_read_beat = PriorityEncoder(pending_reads)
|
||||
io.data.read.valid := state === s_busy && pending_reads.orR && !pending_ognt
|
||||
io.data.read.bits.id := UInt(trackerId)
|
||||
io.data.read.bits.way_en := xact_way_en
|
||||
io.data.read.bits.addr_idx := xact.addr_block(idxMSB,idxLSB)
|
||||
io.data.read.bits.addr_beat := curr_read_beat
|
||||
|
||||
pending_resps := (pending_resps & dropPendingBitInternal(io.data.resp)) |
|
||||
addPendingBitInternal(io.data.read)
|
||||
mergeDataInternal(io.data.resp)
|
||||
|
||||
// We write data to the cache at this level if it was Put here with allocate flag,
|
||||
// written back dirty, or refilled from outer memory.
|
||||
pending_writes := (pending_writes & dropPendingBit(io.data.write)) |
|
||||
addPendingBitWhenBeatHasDataAndAllocs(io.inner.acquire) |
|
||||
addPendingBitWhenBeatHasData(io.inner.release) |
|
||||
(Fill(refillCycles, xact.allocate()) &
|
||||
addPendingBitWhenBeatHasData(io.outer.grant))
|
||||
val curr_write_beat = PriorityEncoder(pending_writes)
|
||||
io.data.write.valid := state === s_busy &&
|
||||
pending_writes.orR &&
|
||||
!pending_ognt &&
|
||||
!pending_reads(curr_write_beat) &&
|
||||
!pending_resps(curr_write_beat)
|
||||
io.data.write.bits.id := UInt(trackerId)
|
||||
io.data.write.bits.way_en := xact_way_en
|
||||
io.data.write.bits.addr_idx := xact.addr_block(idxMSB,idxLSB)
|
||||
io.data.write.bits.addr_beat := curr_write_beat
|
||||
io.data.write.bits.wmask := wmask_buffer(curr_write_beat)
|
||||
io.data.write.bits.data := xact.data_buffer(curr_write_beat)
|
||||
|
||||
// End a transaction by updating the block metadata
|
||||
io.meta.write.valid := state === s_meta_write
|
||||
io.meta.write.bits.id := UInt(trackerId)
|
||||
io.meta.write.bits.idx := xact.addr_block(idxMSB,idxLSB)
|
||||
io.meta.write.bits.idx := xact_addr_idx
|
||||
io.meta.write.bits.way_en := xact_way_en
|
||||
io.meta.write.bits.data.tag := xact.addr_block >> UInt(idxBits)
|
||||
io.meta.write.bits.data.tag := xact_addr_tag
|
||||
io.meta.write.bits.data.coh := pending_coh
|
||||
|
||||
// Handling of secondary misses (Gets and Puts only for now)
|
||||
when(io.inner.acquire.fire() && io.iacq().hasData()) { // state <= s_meta_wrtie
|
||||
val beat = io.iacq().addr_beat
|
||||
val wmask = io.iacq().wmask()
|
||||
val full = FillInterleaved(8, wmask)
|
||||
xact.data_buffer(beat) := (~full & xact.data_buffer(beat)) | (full & io.iacq().data)
|
||||
wmask_buffer(beat) := wmask | Mux(state === s_idle, Bits(0), wmask_buffer(beat))
|
||||
}
|
||||
|
||||
// Defined here because of Chisel default wire demands, used in s_meta_resp
|
||||
val pending_coh_on_hit = HierarchicalMetadata(
|
||||
io.meta.resp.bits.meta.coh.inner,
|
||||
io.meta.resp.bits.meta.coh.outer.onHit(xact.op_code()))
|
||||
val pending_coh_on_miss = HierarchicalMetadata.onReset
|
||||
|
||||
// State machine updates and transaction handler metadata intialization
|
||||
when(state === s_idle && io.inner.acquire.valid) {
|
||||
xact := io.iacq()
|
||||
xact_addr_block := io.iacq().addr_block
|
||||
xact_allocate := io.iacq().allocate()
|
||||
xact_amo_shift_bits := io.iacq().amo_shift_bits()
|
||||
xact_op_code := io.iacq().op_code()
|
||||
xact_addr_byte := io.iacq().addr_byte()
|
||||
xact_op_size := io.iacq().op_size()
|
||||
amo_result := UInt(0)
|
||||
pending_puts := Mux( // Make sure to collect all data from a PutBlock
|
||||
io.iacq().isBuiltInType(Acquire.putBlockType),
|
||||
@ -854,8 +869,8 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
addPendingBitWhenBeatHasPartialWritemask(io.inner.acquire)).toSInt).toUInt
|
||||
pending_writes := addPendingBitWhenBeatHasDataAndAllocs(io.inner.acquire)
|
||||
pending_resps := UInt(0)
|
||||
pending_ignt_data := UInt(0)
|
||||
pending_ignt_ack := Bool(false)
|
||||
ignt_data_ready := UInt(0)
|
||||
ignt_ack_ready := Bool(false)
|
||||
pending_meta_write := Bool(false)
|
||||
state := s_meta_read
|
||||
}
|
||||
@ -866,15 +881,15 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
xact_way_en := io.meta.resp.bits.way_en
|
||||
val coh = io.meta.resp.bits.meta.coh
|
||||
val tag_match = io.meta.resp.bits.tag_match
|
||||
val is_hit = (if(!isLastLevelCache) tag_match && coh.outer.isHit(xact.op_code())
|
||||
val is_hit = (if(!isLastLevelCache) tag_match && coh.outer.isHit(xact_op_code)
|
||||
else xact.isBuiltInType(Acquire.putBlockType) ||
|
||||
tag_match && coh.outer.isValid())
|
||||
val needs_writeback = !tag_match &&
|
||||
xact.allocate() &&
|
||||
xact_allocate &&
|
||||
(coh.outer.requiresVoluntaryWriteback() ||
|
||||
coh.inner.requiresProbesOnVoluntaryWriteback())
|
||||
val needs_inner_probes = tag_match && coh.inner.requiresProbes(xact)
|
||||
val should_update_meta = !tag_match && xact.allocate() ||
|
||||
val should_update_meta = !tag_match && xact_allocate ||
|
||||
is_hit && pending_coh_on_hit != coh
|
||||
// Determine any changes to the coherence metadata
|
||||
when (should_update_meta) { pending_meta_write := Bool(true) }
|
||||
@ -889,14 +904,14 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
val mask_incoherent = mask_self & ~io.incoherent.toBits
|
||||
pending_iprbs := mask_incoherent
|
||||
}
|
||||
// If a prefetch is a hit, note that we have to ack it
|
||||
// If a prefetch is a hit, note that we should ack it ASAP
|
||||
when (is_hit && xact.isPrefetch()) {
|
||||
pending_ignt_ack := Bool(true)
|
||||
ignt_ack_ready := Bool(true)
|
||||
}
|
||||
// If the write is marked no-allocate but is already in the cache,
|
||||
// we do, in fact, need to write the data to the cache
|
||||
when (is_hit && !xact.allocate() && xact.hasData()) {
|
||||
pending_writes := addPendingBitFromBufferedAcquire(xact)
|
||||
// If some kind of Put is marked no-allocate but is already in the cache,
|
||||
// we need to write its data to the data array
|
||||
when (is_hit && !xact_allocate && xact.hasData()) {
|
||||
pending_writes := addPendingBitsFromAcquire(xact)
|
||||
}
|
||||
// Next: request writeback, issue probes, query outer memory, or respond
|
||||
state := Mux(needs_writeback, s_wb_req,
|
||||
@ -913,28 +928,25 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra
|
||||
when(state === s_inner_probe && !(pending_iprbs.orR || pending_irels)) {
|
||||
// Tag matches, so if this is the last level cache we can use the data without upgrading permissions
|
||||
val skip_outer_acquire =
|
||||
(if(!isLastLevelCache) xact_old_meta.coh.outer.isHit(xact.op_code())
|
||||
(if(!isLastLevelCache) xact_old_meta.coh.outer.isHit(xact_op_code)
|
||||
else xact.isBuiltInType(Acquire.putBlockType) || xact_old_meta.coh.outer.isValid())
|
||||
state := Mux(!skip_outer_acquire, s_outer_acquire, s_busy)
|
||||
}
|
||||
when(state === s_outer_acquire && oacq_data_done) { state := s_busy }
|
||||
when(state === s_busy && all_pending_done) {
|
||||
wmask_buffer.foreach { w => w := UInt(0) }
|
||||
wmask_buffer.foreach { w => w := UInt(0) } // This is the only reg that must be clear in s_idle
|
||||
state := Mux(pending_meta_write, s_meta_write, s_idle)
|
||||
}
|
||||
when(state === s_meta_write && io.meta.write.ready) { state := s_idle }
|
||||
|
||||
// These IOs are used for routing in the parent
|
||||
val in_same_set = xact.addr_block(idxMSB,idxLSB) === io.iacq().addr_block(idxMSB,idxLSB)
|
||||
io.has_release_match := xact.conflicts(io.irel()) && !io.irel().isVoluntary() && io.inner.release.ready
|
||||
io.has_acquire_match := can_merge_iacq_put || can_merge_iacq_get
|
||||
io.has_acquire_conflict := in_same_set && (state != s_idle) && !io.has_acquire_match
|
||||
val in_same_set = xact_addr_idx === io.iacq().addr_block(idxMSB,idxLSB)
|
||||
io.has_release_match := io.irel().conflicts(xact_addr_block) &&
|
||||
!io.irel().isVoluntary() &&
|
||||
io.inner.release.ready
|
||||
io.has_acquire_match := iacq_can_merge
|
||||
io.has_acquire_conflict := in_same_set && (state =/= s_idle) && !io.has_acquire_match
|
||||
//TODO: relax from in_same_set to xact.conflicts(io.iacq())?
|
||||
|
||||
// Checks for illegal behavior
|
||||
assert(!(state != s_idle && io.inner.acquire.fire() &&
|
||||
io.inner.acquire.bits.client_id != xact.client_id),
|
||||
"AcquireTracker accepted data beat from different network source than initial request.")
|
||||
}
|
||||
|
||||
class L2WritebackReq(implicit p: Parameters) extends L2Metadata()(p) with HasL2Id {
|
||||
|
@ -70,13 +70,13 @@ trait HasClientSideCoherencePolicy {
|
||||
// Determine which custom message type to use
|
||||
def getAcquireType(cmd: UInt, meta: ClientMetadata): UInt
|
||||
def getReleaseType(cmd: UInt, meta: ClientMetadata): UInt
|
||||
def getReleaseType(p: Probe, meta: ClientMetadata): UInt
|
||||
def getReleaseType(p: HasProbeType, meta: ClientMetadata): UInt
|
||||
|
||||
// Mutate ClientMetadata based on messages or cmds
|
||||
def clientMetadataOnHit(cmd: UInt, meta: ClientMetadata): ClientMetadata
|
||||
def clientMetadataOnCacheControl(cmd: UInt, meta: ClientMetadata): ClientMetadata
|
||||
def clientMetadataOnGrant(incoming: Grant, cmd: UInt, meta: ClientMetadata): ClientMetadata
|
||||
def clientMetadataOnProbe(incoming: Probe, meta: ClientMetadata): ClientMetadata
|
||||
def clientMetadataOnGrant(incoming: HasGrantType, cmd: UInt, meta: ClientMetadata): ClientMetadata
|
||||
def clientMetadataOnProbe(incoming: HasProbeType, meta: ClientMetadata): ClientMetadata
|
||||
}
|
||||
|
||||
/** This API contains all functions required for manager coherence agents.
|
||||
@ -90,18 +90,18 @@ trait HasManagerSideCoherencePolicy extends HasDirectoryRepresentation {
|
||||
def masterStateWidth = log2Ceil(nManagerStates)
|
||||
|
||||
// Transaction probing logic
|
||||
def requiresProbes(acq: AcquireMetadata, meta: ManagerMetadata): Bool
|
||||
def requiresProbes(acq: HasAcquireType, meta: ManagerMetadata): Bool
|
||||
def requiresProbes(cmd: UInt, meta: ManagerMetadata): Bool
|
||||
|
||||
// Determine which custom message type to use in response
|
||||
def getProbeType(cmd: UInt, meta: ManagerMetadata): UInt
|
||||
def getProbeType(acq: AcquireMetadata, meta: ManagerMetadata): UInt
|
||||
def getGrantType(acq: AcquireMetadata, meta: ManagerMetadata): UInt
|
||||
def getProbeType(acq: HasAcquireType, meta: ManagerMetadata): UInt
|
||||
def getGrantType(acq: HasAcquireType, meta: ManagerMetadata): UInt
|
||||
def getExclusiveGrantType(): UInt
|
||||
|
||||
// Mutate ManagerMetadata based on messages or cmds
|
||||
def managerMetadataOnRelease(incoming: ReleaseMetadata, src: UInt, meta: ManagerMetadata): ManagerMetadata
|
||||
def managerMetadataOnGrant(outgoing: GrantMetadata, dst: UInt, meta: ManagerMetadata) =
|
||||
def managerMetadataOnRelease(incoming: HasReleaseType, src: UInt, meta: ManagerMetadata): ManagerMetadata
|
||||
def managerMetadataOnGrant(outgoing: HasGrantType, dst: UInt, meta: ManagerMetadata) =
|
||||
ManagerMetadata(sharers=Mux(outgoing.isBuiltInType(), // Assumes all built-ins are uncached
|
||||
meta.sharers,
|
||||
dir.push(meta.sharers, dst)))(meta.p)
|
||||
@ -151,7 +151,7 @@ class MICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_CLEAN -> Mux(dirty, releaseCopyData, releaseCopyAck)))
|
||||
}
|
||||
|
||||
def getReleaseType(incoming: Probe, meta: ClientMetadata): UInt =
|
||||
def getReleaseType(incoming: HasProbeType, meta: ClientMetadata): UInt =
|
||||
MuxLookup(incoming.p_type, releaseInvalidateAck, Array(
|
||||
probeInvalidate -> getReleaseType(M_FLUSH, meta),
|
||||
probeCopy -> getReleaseType(M_CLEAN, meta)))
|
||||
@ -161,17 +161,17 @@ class MICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
def clientMetadataOnCacheControl(cmd: UInt, meta: ClientMetadata) =
|
||||
ClientMetadata(Mux(cmd === M_FLUSH, clientInvalid, meta.state))(meta.p)
|
||||
|
||||
def clientMetadataOnGrant(incoming: Grant, cmd: UInt, meta: ClientMetadata) =
|
||||
def clientMetadataOnGrant(incoming: HasGrantType, cmd: UInt, meta: ClientMetadata) =
|
||||
ClientMetadata(Mux(incoming.isBuiltInType(), clientInvalid, clientValid))(meta.p)
|
||||
|
||||
def clientMetadataOnProbe(incoming: Probe, meta: ClientMetadata) =
|
||||
def clientMetadataOnProbe(incoming: HasProbeType, meta: ClientMetadata) =
|
||||
ClientMetadata(Mux(incoming.p_type === probeInvalidate,
|
||||
clientInvalid, meta.state))(meta.p)
|
||||
|
||||
// Manager states and functions:
|
||||
val nManagerStates = 0 // We don't actually need any states for this protocol
|
||||
|
||||
def requiresProbes(a: AcquireMetadata, meta: ManagerMetadata) = !dir.none(meta.sharers)
|
||||
def requiresProbes(a: HasAcquireType, meta: ManagerMetadata) = !dir.none(meta.sharers)
|
||||
|
||||
def requiresProbes(cmd: UInt, meta: ManagerMetadata) = !dir.none(meta.sharers)
|
||||
|
||||
@ -179,7 +179,7 @@ class MICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
MuxLookup(cmd, probeCopy, Array(
|
||||
M_FLUSH -> probeInvalidate))
|
||||
|
||||
def getProbeType(a: AcquireMetadata, meta: ManagerMetadata): UInt =
|
||||
def getProbeType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(),
|
||||
MuxLookup(a.a_type, probeCopy, Array(
|
||||
Acquire.getBlockType -> probeCopy,
|
||||
@ -189,10 +189,11 @@ class MICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
Acquire.putAtomicType -> probeInvalidate)),
|
||||
probeInvalidate)
|
||||
|
||||
def getGrantType(a: AcquireMetadata, meta: ManagerMetadata): UInt = grantExclusive
|
||||
def getGrantType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(), Acquire.getBuiltInGrantType(a.a_type), grantExclusive)
|
||||
def getExclusiveGrantType(): UInt = grantExclusive
|
||||
|
||||
def managerMetadataOnRelease(incoming: ReleaseMetadata, src: UInt, meta: ManagerMetadata) = {
|
||||
def managerMetadataOnRelease(incoming: HasReleaseType, src: UInt, meta: ManagerMetadata) = {
|
||||
val popped = ManagerMetadata(sharers=dir.pop(meta.sharers, src))(meta.p)
|
||||
MuxBundle(meta, Array(
|
||||
incoming.is(releaseInvalidateData) -> popped,
|
||||
@ -240,7 +241,7 @@ class MEICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_CLEAN -> Mux(dirty, releaseCopyData, releaseCopyAck)))
|
||||
}
|
||||
|
||||
def getReleaseType(incoming: Probe, meta: ClientMetadata): UInt =
|
||||
def getReleaseType(incoming: HasProbeType, meta: ClientMetadata): UInt =
|
||||
MuxLookup(incoming.p_type, releaseInvalidateAck, Array(
|
||||
probeInvalidate -> getReleaseType(M_FLUSH, meta),
|
||||
probeDowngrade -> getReleaseType(M_PRODUCE, meta),
|
||||
@ -255,12 +256,12 @@ class MEICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_FLUSH -> clientInvalid,
|
||||
M_CLEAN -> Mux(meta.state === clientExclusiveDirty, clientExclusiveClean, meta.state))))(meta.p)
|
||||
|
||||
def clientMetadataOnGrant(incoming: Grant, cmd: UInt, meta: ClientMetadata) =
|
||||
def clientMetadataOnGrant(incoming: HasGrantType, cmd: UInt, meta: ClientMetadata) =
|
||||
ClientMetadata(
|
||||
Mux(incoming.isBuiltInType(), clientInvalid,
|
||||
Mux(isWrite(cmd), clientExclusiveDirty, clientExclusiveClean)))(meta.p)
|
||||
|
||||
def clientMetadataOnProbe(incoming: Probe, meta: ClientMetadata) =
|
||||
def clientMetadataOnProbe(incoming: HasProbeType, meta: ClientMetadata) =
|
||||
ClientMetadata(
|
||||
MuxLookup(incoming.p_type, meta.state, Array(
|
||||
probeInvalidate -> clientInvalid,
|
||||
@ -270,7 +271,7 @@ class MEICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
// Manager states and functions:
|
||||
val nManagerStates = 0 // We don't actually need any states for this protocol
|
||||
|
||||
def requiresProbes(a: AcquireMetadata, meta: ManagerMetadata) = !dir.none(meta.sharers)
|
||||
def requiresProbes(a: HasAcquireType, meta: ManagerMetadata) = !dir.none(meta.sharers)
|
||||
def requiresProbes(cmd: UInt, meta: ManagerMetadata) = !dir.none(meta.sharers)
|
||||
|
||||
def getProbeType(cmd: UInt, meta: ManagerMetadata): UInt =
|
||||
@ -278,7 +279,7 @@ class MEICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_FLUSH -> probeInvalidate,
|
||||
M_PRODUCE -> probeDowngrade))
|
||||
|
||||
def getProbeType(a: AcquireMetadata, meta: ManagerMetadata): UInt =
|
||||
def getProbeType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(),
|
||||
MuxLookup(a.a_type, probeCopy, Array(
|
||||
Acquire.getBlockType -> probeCopy,
|
||||
@ -288,10 +289,11 @@ class MEICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
Acquire.putAtomicType -> probeInvalidate)),
|
||||
probeInvalidate)
|
||||
|
||||
def getGrantType(a: AcquireMetadata, meta: ManagerMetadata): UInt = grantExclusive
|
||||
def getGrantType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(), Acquire.getBuiltInGrantType(a.a_type), grantExclusive)
|
||||
def getExclusiveGrantType(): UInt = grantExclusive
|
||||
|
||||
def managerMetadataOnRelease(incoming: ReleaseMetadata, src: UInt, meta: ManagerMetadata) = {
|
||||
def managerMetadataOnRelease(incoming: HasReleaseType, src: UInt, meta: ManagerMetadata) = {
|
||||
val popped = ManagerMetadata(sharers=dir.pop(meta.sharers, src))(meta.p)
|
||||
MuxBundle(meta, Array(
|
||||
incoming.is(releaseInvalidateData) -> popped,
|
||||
@ -339,7 +341,7 @@ class MSICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_CLEAN -> Mux(dirty, releaseCopyData, releaseCopyAck)))
|
||||
}
|
||||
|
||||
def getReleaseType(incoming: Probe, meta: ClientMetadata): UInt =
|
||||
def getReleaseType(incoming: HasProbeType, meta: ClientMetadata): UInt =
|
||||
MuxLookup(incoming.p_type, releaseInvalidateAck, Array(
|
||||
probeInvalidate -> getReleaseType(M_FLUSH, meta),
|
||||
probeDowngrade -> getReleaseType(M_PRODUCE, meta),
|
||||
@ -355,7 +357,7 @@ class MSICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_PRODUCE -> Mux(clientStatesWithWritePermission.contains(meta.state),
|
||||
clientShared, meta.state))))(meta.p)
|
||||
|
||||
def clientMetadataOnGrant(incoming: Grant, cmd: UInt, meta: ClientMetadata) =
|
||||
def clientMetadataOnGrant(incoming: HasGrantType, cmd: UInt, meta: ClientMetadata) =
|
||||
ClientMetadata(
|
||||
Mux(incoming.isBuiltInType(), clientInvalid,
|
||||
MuxLookup(incoming.g_type, clientInvalid, Array(
|
||||
@ -363,7 +365,7 @@ class MSICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
grantExclusive -> clientExclusiveDirty,
|
||||
grantExclusiveAck -> clientExclusiveDirty))))(meta.p)
|
||||
|
||||
def clientMetadataOnProbe(incoming: Probe, meta: ClientMetadata) =
|
||||
def clientMetadataOnProbe(incoming: HasProbeType, meta: ClientMetadata) =
|
||||
ClientMetadata(
|
||||
MuxLookup(incoming.p_type, meta.state, Array(
|
||||
probeInvalidate -> clientInvalid,
|
||||
@ -376,7 +378,7 @@ class MSICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
// notification msg to track clean drops)
|
||||
// Also could avoid probes on outer WBs.
|
||||
|
||||
def requiresProbes(a: AcquireMetadata, meta: ManagerMetadata) =
|
||||
def requiresProbes(a: HasAcquireType, meta: ManagerMetadata) =
|
||||
Mux(dir.none(meta.sharers), Bool(false),
|
||||
Mux(dir.one(meta.sharers), Bool(true), //TODO: for now we assume it's Exclusive
|
||||
Mux(a.isBuiltInType(), a.hasData(), a.a_type != acquireShared)))
|
||||
@ -388,7 +390,7 @@ class MSICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_FLUSH -> probeInvalidate,
|
||||
M_PRODUCE -> probeDowngrade))
|
||||
|
||||
def getProbeType(a: AcquireMetadata, meta: ManagerMetadata): UInt =
|
||||
def getProbeType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(),
|
||||
MuxLookup(a.a_type, probeCopy, Array(
|
||||
Acquire.getBlockType -> probeCopy,
|
||||
@ -400,13 +402,14 @@ class MSICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
acquireShared -> probeDowngrade,
|
||||
acquireExclusive -> probeInvalidate)))
|
||||
|
||||
def getGrantType(a: AcquireMetadata, meta: ManagerMetadata): UInt =
|
||||
Mux(a.a_type === acquireShared,
|
||||
Mux(!dir.none(meta.sharers), grantShared, grantExclusive),
|
||||
grantExclusive)
|
||||
def getGrantType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(), Acquire.getBuiltInGrantType(a.a_type),
|
||||
Mux(a.a_type === acquireShared,
|
||||
Mux(!dir.none(meta.sharers), grantShared, grantExclusive),
|
||||
grantExclusive))
|
||||
def getExclusiveGrantType(): UInt = grantExclusive
|
||||
|
||||
def managerMetadataOnRelease(incoming: ReleaseMetadata, src: UInt, meta: ManagerMetadata) = {
|
||||
def managerMetadataOnRelease(incoming: HasReleaseType, src: UInt, meta: ManagerMetadata) = {
|
||||
val popped = ManagerMetadata(sharers=dir.pop(meta.sharers, src))(meta.p)
|
||||
MuxBundle(meta, Array(
|
||||
incoming.is(releaseInvalidateData) -> popped,
|
||||
@ -441,7 +444,7 @@ class MESICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
def clientStatesWithWritePermission = Vec(clientExclusiveClean, clientExclusiveDirty)
|
||||
def clientStatesWithDirtyData = Vec(clientExclusiveDirty)
|
||||
|
||||
def isValid (meta: ClientMetadata): Bool = meta.state != clientInvalid
|
||||
def isValid(meta: ClientMetadata): Bool = meta.state != clientInvalid
|
||||
|
||||
def getAcquireType(cmd: UInt, meta: ClientMetadata): UInt =
|
||||
Mux(isWriteIntent(cmd), acquireExclusive, acquireShared)
|
||||
@ -454,7 +457,7 @@ class MESICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_CLEAN -> Mux(dirty, releaseCopyData, releaseCopyAck)))
|
||||
}
|
||||
|
||||
def getReleaseType(incoming: Probe, meta: ClientMetadata): UInt =
|
||||
def getReleaseType(incoming: HasProbeType, meta: ClientMetadata): UInt =
|
||||
MuxLookup(incoming.p_type, releaseInvalidateAck, Array(
|
||||
probeInvalidate -> getReleaseType(M_FLUSH, meta),
|
||||
probeDowngrade -> getReleaseType(M_PRODUCE, meta),
|
||||
@ -472,7 +475,7 @@ class MESICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_CLEAN -> Mux(meta.state === clientExclusiveDirty,
|
||||
clientExclusiveClean, meta.state))))(meta.p)
|
||||
|
||||
def clientMetadataOnGrant(incoming: Grant, cmd: UInt, meta: ClientMetadata) =
|
||||
def clientMetadataOnGrant(incoming: HasGrantType, cmd: UInt, meta: ClientMetadata) =
|
||||
ClientMetadata(
|
||||
Mux(incoming.isBuiltInType(), clientInvalid,
|
||||
MuxLookup(incoming.g_type, clientInvalid, Array(
|
||||
@ -480,7 +483,7 @@ class MESICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
grantExclusive -> Mux(isWrite(cmd), clientExclusiveDirty, clientExclusiveClean),
|
||||
grantExclusiveAck -> clientExclusiveDirty))))(meta.p)
|
||||
|
||||
def clientMetadataOnProbe(incoming: Probe, meta: ClientMetadata) =
|
||||
def clientMetadataOnProbe(incoming: HasProbeType, meta: ClientMetadata) =
|
||||
ClientMetadata(
|
||||
MuxLookup(incoming.p_type, meta.state, Array(
|
||||
probeInvalidate -> clientInvalid,
|
||||
@ -493,7 +496,7 @@ class MESICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
// notification msg to track clean drops)
|
||||
// Also could avoid probes on outer WBs.
|
||||
|
||||
def requiresProbes(a: AcquireMetadata, meta: ManagerMetadata) =
|
||||
def requiresProbes(a: HasAcquireType, meta: ManagerMetadata) =
|
||||
Mux(dir.none(meta.sharers), Bool(false),
|
||||
Mux(dir.one(meta.sharers), Bool(true), //TODO: for now we assume it's Exclusive
|
||||
Mux(a.isBuiltInType(), a.hasData(), a.a_type != acquireShared)))
|
||||
@ -505,7 +508,7 @@ class MESICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
M_FLUSH -> probeInvalidate,
|
||||
M_PRODUCE -> probeDowngrade))
|
||||
|
||||
def getProbeType(a: AcquireMetadata, meta: ManagerMetadata): UInt =
|
||||
def getProbeType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(),
|
||||
MuxLookup(a.a_type, probeCopy, Array(
|
||||
Acquire.getBlockType -> probeCopy,
|
||||
@ -517,13 +520,14 @@ class MESICoherence(dir: DirectoryRepresentation) extends CoherencePolicy(dir) {
|
||||
acquireShared -> probeDowngrade,
|
||||
acquireExclusive -> probeInvalidate)))
|
||||
|
||||
def getGrantType(a: AcquireMetadata, meta: ManagerMetadata): UInt =
|
||||
Mux(a.a_type === acquireShared,
|
||||
Mux(!dir.none(meta.sharers), grantShared, grantExclusive),
|
||||
grantExclusive)
|
||||
def getGrantType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(), Acquire.getBuiltInGrantType(a.a_type),
|
||||
Mux(a.a_type === acquireShared,
|
||||
Mux(!dir.none(meta.sharers), grantShared, grantExclusive),
|
||||
grantExclusive))
|
||||
def getExclusiveGrantType(): UInt = grantExclusive
|
||||
|
||||
def managerMetadataOnRelease(incoming: ReleaseMetadata, src: UInt, meta: ManagerMetadata) = {
|
||||
def managerMetadataOnRelease(incoming: HasReleaseType, src: UInt, meta: ManagerMetadata) = {
|
||||
val popped = ManagerMetadata(sharers=dir.pop(meta.sharers, src))(meta.p)
|
||||
MuxBundle(meta, Array(
|
||||
incoming.is(releaseInvalidateData) -> popped,
|
||||
@ -569,7 +573,7 @@ class MigratoryCoherence(dir: DirectoryRepresentation) extends CoherencePolicy(d
|
||||
M_CLEAN -> Mux(dirty, releaseCopyData, releaseCopyAck)))
|
||||
}
|
||||
|
||||
def getReleaseType(incoming: Probe, meta: ClientMetadata): UInt = {
|
||||
def getReleaseType(incoming: HasProbeType, meta: ClientMetadata): UInt = {
|
||||
val dirty = clientStatesWithDirtyData.contains(meta.state)
|
||||
val with_data = MuxLookup(incoming.p_type, releaseInvalidateData, Array(
|
||||
probeInvalidate -> Mux(Vec(clientExclusiveDirty, clientMigratoryDirty).contains(meta.state),
|
||||
@ -605,7 +609,7 @@ class MigratoryCoherence(dir: DirectoryRepresentation) extends CoherencePolicy(d
|
||||
clientExclusiveDirty -> clientExclusiveClean,
|
||||
clientMigratoryDirty -> clientMigratoryClean)))))(meta.p)
|
||||
|
||||
def clientMetadataOnGrant(incoming: Grant, cmd: UInt, meta: ClientMetadata) =
|
||||
def clientMetadataOnGrant(incoming: HasGrantType, cmd: UInt, meta: ClientMetadata) =
|
||||
ClientMetadata(
|
||||
Mux(incoming.isBuiltInType(), clientInvalid,
|
||||
MuxLookup(incoming.g_type, clientInvalid, Array(
|
||||
@ -615,7 +619,7 @@ class MigratoryCoherence(dir: DirectoryRepresentation) extends CoherencePolicy(d
|
||||
grantReadMigratory -> Mux(isWrite(cmd),
|
||||
clientMigratoryDirty, clientMigratoryClean)))))(meta.p)
|
||||
|
||||
def clientMetadataOnProbe(incoming: Probe, meta: ClientMetadata) =
|
||||
def clientMetadataOnProbe(incoming: HasProbeType, meta: ClientMetadata) =
|
||||
ClientMetadata(
|
||||
MuxLookup(incoming.p_type, meta.state, Array(
|
||||
probeInvalidate -> clientInvalid,
|
||||
@ -631,7 +635,7 @@ class MigratoryCoherence(dir: DirectoryRepresentation) extends CoherencePolicy(d
|
||||
// Manager states and functions:
|
||||
val nManagerStates = 0 // TODO: we could add some states to reduce the number of message types
|
||||
|
||||
def requiresProbes(a: AcquireMetadata, meta: ManagerMetadata) =
|
||||
def requiresProbes(a: HasAcquireType, meta: ManagerMetadata) =
|
||||
Mux(dir.none(meta.sharers), Bool(false),
|
||||
Mux(dir.one(meta.sharers), Bool(true), //TODO: for now we assume it's Exclusive
|
||||
Mux(a.isBuiltInType(), a.hasData(), a.a_type != acquireShared)))
|
||||
@ -643,7 +647,7 @@ class MigratoryCoherence(dir: DirectoryRepresentation) extends CoherencePolicy(d
|
||||
M_FLUSH -> probeInvalidate,
|
||||
M_PRODUCE -> probeDowngrade))
|
||||
|
||||
def getProbeType(a: AcquireMetadata, meta: ManagerMetadata): UInt =
|
||||
def getProbeType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(),
|
||||
MuxLookup(a.a_type, probeCopy, Array(
|
||||
Acquire.getBlockType -> probeCopy,
|
||||
@ -656,14 +660,15 @@ class MigratoryCoherence(dir: DirectoryRepresentation) extends CoherencePolicy(d
|
||||
acquireExclusive -> probeInvalidate,
|
||||
acquireInvalidateOthers -> probeInvalidateOthers)))
|
||||
|
||||
def getGrantType(a: AcquireMetadata, meta: ManagerMetadata): UInt =
|
||||
def getGrantType(a: HasAcquireType, meta: ManagerMetadata): UInt =
|
||||
Mux(a.isBuiltInType(), Acquire.getBuiltInGrantType(a.a_type),
|
||||
MuxLookup(a.a_type, grantShared, Array(
|
||||
acquireShared -> Mux(!dir.none(meta.sharers), grantShared, grantExclusive),
|
||||
acquireExclusive -> grantExclusive,
|
||||
acquireInvalidateOthers -> grantExclusiveAck)) //TODO: add this to MESI for broadcast?
|
||||
acquireInvalidateOthers -> grantExclusiveAck))) //TODO: add this to MESI for broadcast?
|
||||
def getExclusiveGrantType(): UInt = grantExclusive
|
||||
|
||||
def managerMetadataOnRelease(incoming: ReleaseMetadata, src: UInt, meta: ManagerMetadata) = {
|
||||
def managerMetadataOnRelease(incoming: HasReleaseType, src: UInt, meta: ManagerMetadata) = {
|
||||
val popped = ManagerMetadata(sharers=dir.pop(meta.sharers, src))(meta.p)
|
||||
MuxBundle(meta, Array(
|
||||
incoming.is(releaseInvalidateData) -> popped,
|
||||
|
@ -174,7 +174,7 @@ class ManagerMetadata(implicit p: Parameters) extends CoherenceMetadata()(p) {
|
||||
def full(dummy: Int = 0): UInt = co.dir.full(this.sharers)
|
||||
|
||||
/** Does this [[uncore.Acquire]] require [[uncore.Probe Probes]] to be sent */
|
||||
def requiresProbes(acq: AcquireMetadata): Bool = co.requiresProbes(acq, this)
|
||||
def requiresProbes(acq: HasAcquireType): Bool = co.requiresProbes(acq, this)
|
||||
/** Does this memory op require [[uncore.Probe Probes]] to be sent */
|
||||
def requiresProbes(op_code: UInt): Bool = co.requiresProbes(op_code, this)
|
||||
/** Does an eviction require [[uncore.Probe Probes]] to be sent */
|
||||
@ -185,8 +185,17 @@ class ManagerMetadata(implicit p: Parameters) extends CoherenceMetadata()(p) {
|
||||
*
|
||||
* @param dst Destination client id for this Probe
|
||||
* @param acq Acquire message triggering this Probe
|
||||
* @param addr_block address of the cache block being probed
|
||||
*/
|
||||
def makeProbe(dst: UInt, acq: AcquireMetadata): ProbeToDst =
|
||||
def makeProbe(dst: UInt, acq: HasAcquireType, addr_block: UInt): ProbeToDst =
|
||||
Probe(dst, co.getProbeType(acq, this), addr_block)(p)
|
||||
|
||||
/** Construct an appropriate [[uncore.ProbeToDst]] for a given [[uncore.Acquire]]
|
||||
*
|
||||
* @param dst Destination client id for this Probe
|
||||
* @param acq Acquire message triggering this Probe
|
||||
*/
|
||||
def makeProbe(dst: UInt, acq: AcquireMetadata): ProbeToDst =
|
||||
Probe(dst, co.getProbeType(acq, this), acq.addr_block)(p)
|
||||
|
||||
/** Construct an appropriate [[uncore.ProbeToDst]] for a given mem op
|
||||
@ -236,9 +245,7 @@ class ManagerMetadata(implicit p: Parameters) extends CoherenceMetadata()(p) {
|
||||
Grant(
|
||||
dst = acq.client_id,
|
||||
is_builtin_type = acq.isBuiltInType(),
|
||||
g_type = Mux(acq.isBuiltInType(),
|
||||
acq.getBuiltInGrantType(),
|
||||
co.getGrantType(acq, this)),
|
||||
g_type = co.getGrantType(acq, this),
|
||||
client_xact_id = acq.client_xact_id,
|
||||
manager_xact_id = manager_xact_id,
|
||||
addr_beat = addr_beat,
|
||||
@ -249,19 +256,22 @@ class ManagerMetadata(implicit p: Parameters) extends CoherenceMetadata()(p) {
|
||||
* Used to respond to secondary misses merged into this transaction.
|
||||
* May contain single or multiple beats of data.
|
||||
*
|
||||
* @param pri Primary miss's Acquire message, used to get g_type and dst
|
||||
* @param sec Secondary miss info, used to get beat and client_xact_id
|
||||
* @param sec Secondary miss info
|
||||
* @param manager_xact_id manager's transaction id
|
||||
* @param data data being refilled to the original requestor
|
||||
*/
|
||||
def makeGrant(
|
||||
pri: AcquireMetadata with HasClientId,
|
||||
sec: SecondaryMissInfo,
|
||||
manager_xact_id: UInt,
|
||||
manager_xact_id: UInt,
|
||||
data: UInt): GrantToDst = {
|
||||
val g = makeGrant(pri, manager_xact_id, sec.addr_beat, data)
|
||||
g.client_xact_id := sec.client_xact_id
|
||||
g
|
||||
Grant(
|
||||
dst = sec.client_id,
|
||||
is_builtin_type = sec.isBuiltInType(),
|
||||
g_type = co.getGrantType(sec, this),
|
||||
client_xact_id = sec.client_xact_id,
|
||||
manager_xact_id = manager_xact_id,
|
||||
addr_beat = sec.addr_beat,
|
||||
data = data)(p)
|
||||
}
|
||||
|
||||
/** New metadata after receiving a [[uncore.ReleaseFromSrc]]
|
||||
|
@ -143,26 +143,11 @@ trait HasClientId extends HasTileLinkParameters {
|
||||
val client_id = UInt(width = tlClientIdBits)
|
||||
}
|
||||
|
||||
/** TileLink channel bundle definitions */
|
||||
|
||||
/** The Acquire channel is used to intiate coherence protocol transactions in
|
||||
* order to gain access to a cache block's data with certain permissions
|
||||
* enabled. Messages sent over this channel may be custom types defined by
|
||||
* a [[uncore.CoherencePolicy]] for cached data accesse or may be built-in types
|
||||
* used for uncached data accesses. Acquires may contain data for Put or
|
||||
* PutAtomic built-in types. After sending an Acquire, clients must
|
||||
* wait for a manager to send them a [[uncore.Grant]] message in response.
|
||||
*/
|
||||
class AcquireMetadata(implicit p: Parameters) extends ClientToManagerChannel()(p)
|
||||
with HasCacheBlockAddress
|
||||
with HasClientTransactionId
|
||||
with HasTileLinkBeatId {
|
||||
// Actual bundle fields:
|
||||
val is_builtin_type = Bool()
|
||||
val a_type = UInt(width = tlAcquireTypeBits)
|
||||
trait HasAcquireUnion extends HasTileLinkParameters {
|
||||
val union = Bits(width = tlAcquireUnionBits)
|
||||
|
||||
// Utility funcs for accessing subblock union:
|
||||
def isBuiltInType(t: UInt): Bool
|
||||
val opCodeOff = 1
|
||||
val opSizeOff = tlMemoryOpcodeBits + opCodeOff
|
||||
val addrByteOff = tlMemoryOperandSizeBits + opSizeOff
|
||||
@ -177,22 +162,25 @@ class AcquireMetadata(implicit p: Parameters) extends ClientToManagerChannel()(p
|
||||
def op_size(dummy: Int = 0) = union(addrByteOff-1, opSizeOff)
|
||||
/** Byte address for [[uncore.PutAtomic]] operand */
|
||||
def addr_byte(dummy: Int = 0) = union(addrByteMSB-1, addrByteOff)
|
||||
private def amo_offset(dummy: Int = 0) = addr_byte()(tlByteAddrBits-1, log2Up(amoAluOperandBits/8))
|
||||
def amo_offset(dummy: Int = 0) = addr_byte()(tlByteAddrBits-1, log2Up(amoAluOperandBits/8))
|
||||
/** Bit offset of [[uncore.PutAtomic]] operand */
|
||||
def amo_shift_bits(dummy: Int = 0) = UInt(amoAluOperandBits)*amo_offset()
|
||||
/** Write mask for [[uncore.Put]], [[uncore.PutBlock]], [[uncore.PutAtomic]] */
|
||||
def wmask(dummy: Int = 0) =
|
||||
def wmask(dummy: Int = 0): UInt = {
|
||||
Mux(isBuiltInType(Acquire.putAtomicType),
|
||||
FillInterleaved(amoAluOperandBits/8, UIntToOH(amo_offset())),
|
||||
Mux(isBuiltInType(Acquire.putBlockType) || isBuiltInType(Acquire.putType),
|
||||
union(tlWriteMaskBits, 1),
|
||||
UInt(0, width = tlWriteMaskBits)))
|
||||
}
|
||||
/** Full, beat-sized writemask */
|
||||
def full_wmask(dummy: Int = 0) = FillInterleaved(8, wmask())
|
||||
/** Complete physical address for block, beat or operand */
|
||||
def full_addr(dummy: Int = 0) = Cat(this.addr_block, this.addr_beat, this.addr_byte())
|
||||
}
|
||||
|
||||
trait HasAcquireType extends HasTileLinkParameters {
|
||||
val is_builtin_type = Bool()
|
||||
val a_type = UInt(width = tlAcquireTypeBits)
|
||||
|
||||
// Other helper functions:
|
||||
/** Message type equality */
|
||||
def is(t: UInt) = a_type === t //TODO: make this more opaque; def ===?
|
||||
|
||||
@ -219,31 +207,93 @@ class AcquireMetadata(implicit p: Parameters) extends ClientToManagerChannel()(p
|
||||
*/
|
||||
def requiresSelfProbe(dummy: Int = 0) = Bool(false)
|
||||
|
||||
/** Mapping between each built-in Acquire type (defined in companion object)
|
||||
* and a built-in Grant type.
|
||||
*/
|
||||
def getBuiltInGrantType(dummy: Int = 0): UInt = {
|
||||
MuxLookup(this.a_type, Grant.putAckType, Array(
|
||||
Acquire.getType -> Grant.getDataBeatType,
|
||||
Acquire.getBlockType -> Grant.getDataBlockType,
|
||||
Acquire.putType -> Grant.putAckType,
|
||||
Acquire.putBlockType -> Grant.putAckType,
|
||||
Acquire.putAtomicType -> Grant.getDataBeatType,
|
||||
Acquire.prefetchType -> Grant.prefetchAckType))
|
||||
}
|
||||
/** Mapping between each built-in Acquire type and a built-in Grant type. */
|
||||
def getBuiltInGrantType(dummy: Int = 0): UInt = Acquire.getBuiltInGrantType(this.a_type)
|
||||
}
|
||||
|
||||
trait HasProbeType extends HasTileLinkParameters {
|
||||
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)
|
||||
}
|
||||
|
||||
trait HasReleaseType extends HasTileLinkParameters {
|
||||
val voluntary = Bool()
|
||||
val r_type = UInt(width = tlCoh.releaseTypeWidth)
|
||||
|
||||
def is(t: UInt) = r_type === t
|
||||
def hasData(dummy: Int = 0) = tlCoh.releaseTypesWithData.contains(r_type)
|
||||
def hasMultibeatData(dummy: Int = 0) = Bool(tlDataBeats > 1) &&
|
||||
tlCoh.releaseTypesWithData.contains(r_type)
|
||||
def isVoluntary(dummy: Int = 0) = voluntary
|
||||
def requiresAck(dummy: Int = 0) = !Bool(tlNetworkPreservesPointToPointOrdering)
|
||||
}
|
||||
|
||||
trait HasGrantType extends HasTileLinkParameters {
|
||||
val is_builtin_type = Bool()
|
||||
val g_type = UInt(width = tlGrantTypeBits)
|
||||
|
||||
// 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),
|
||||
tlCoh.grantTypesWithData.contains(g_type))
|
||||
def hasMultibeatData(dummy: Int = 0): Bool =
|
||||
Bool(tlDataBeats > 1) && Mux(isBuiltInType(),
|
||||
Grant.typesWithMultibeatData.contains(g_type),
|
||||
tlCoh.grantTypesWithData.contains(g_type))
|
||||
def isVoluntary(dummy: Int = 0): Bool = isBuiltInType() && (g_type === Grant.voluntaryAckType)
|
||||
def requiresAck(dummy: Int = 0): Bool = !Bool(tlNetworkPreservesPointToPointOrdering) && !isVoluntary()
|
||||
}
|
||||
|
||||
/** TileLink channel bundle definitions */
|
||||
|
||||
/** The Acquire channel is used to intiate coherence protocol transactions in
|
||||
* order to gain access to a cache block's data with certain permissions
|
||||
* enabled. Messages sent over this channel may be custom types defined by
|
||||
* a [[uncore.CoherencePolicy]] for cached data accesse or may be built-in types
|
||||
* used for uncached data accesses. Acquires may contain data for Put or
|
||||
* PutAtomic built-in types. After sending an Acquire, clients must
|
||||
* wait for a manager to send them a [[uncore.Grant]] message in response.
|
||||
*/
|
||||
class AcquireMetadata(implicit p: Parameters) extends ClientToManagerChannel
|
||||
with HasCacheBlockAddress
|
||||
with HasClientTransactionId
|
||||
with HasTileLinkBeatId
|
||||
with HasAcquireType
|
||||
with HasAcquireUnion {
|
||||
/** Complete physical address for block, beat or operand */
|
||||
def full_addr(dummy: Int = 0) = Cat(this.addr_block, this.addr_beat, this.addr_byte())
|
||||
}
|
||||
|
||||
/** [[uncore.AcquireMetadata]] with an extra field containing the data beat */
|
||||
class Acquire(implicit p: Parameters) extends AcquireMetadata()(p) with HasTileLinkData
|
||||
class Acquire(implicit p: Parameters) extends AcquireMetadata
|
||||
with HasTileLinkData
|
||||
|
||||
/** [[uncore.AcquireMetadata]] with an extra field containing the entire cache block */
|
||||
class BufferedAcquire(implicit p: Parameters) extends AcquireMetadata()(p) with HasTileLinkBlock
|
||||
class BufferedAcquire(implicit p: Parameters) extends AcquireMetadata
|
||||
with HasTileLinkBlock
|
||||
|
||||
/** [[uncore.Acquire]] with an extra field stating its source id */
|
||||
class AcquireFromSrc(implicit p: Parameters) extends Acquire()(p) with HasClientId
|
||||
class AcquireFromSrc(implicit p: Parameters) extends Acquire
|
||||
with HasClientId
|
||||
|
||||
/** [[uncore.BufferedAcquire]] with an extra field stating its source id */
|
||||
class BufferedAcquireFromSrc(implicit p: Parameters) extends BufferedAcquire()(p) with HasClientId
|
||||
class BufferedAcquireFromSrc(implicit p: Parameters) extends BufferedAcquire
|
||||
with HasClientId
|
||||
|
||||
/** Used to track metadata for transactions where multiple secondary misses have been merged
|
||||
* and handled by a single transaction tracker.
|
||||
*/
|
||||
class SecondaryMissInfo(implicit p: Parameters) extends TLBundle
|
||||
with HasClientTransactionId
|
||||
with HasTileLinkBeatId
|
||||
with HasClientId
|
||||
with HasAcquireType
|
||||
|
||||
/** Contains definitions of the the built-in Acquire types and a factory
|
||||
* for [[uncore.Acquire]]
|
||||
@ -273,6 +323,33 @@ object Acquire {
|
||||
def typesWithMultibeatData = Vec(putBlockType)
|
||||
def typesOnSubBlocks = Vec(putType, getType, putAtomicType)
|
||||
|
||||
/** Mapping between each built-in Acquire type and a built-in Grant type. */
|
||||
def getBuiltInGrantType(a_type: UInt): UInt = {
|
||||
MuxLookup(a_type, Grant.putAckType, Array(
|
||||
Acquire.getType -> Grant.getDataBeatType,
|
||||
Acquire.getBlockType -> Grant.getDataBlockType,
|
||||
Acquire.putType -> Grant.putAckType,
|
||||
Acquire.putBlockType -> Grant.putAckType,
|
||||
Acquire.putAtomicType -> Grant.getDataBeatType,
|
||||
Acquire.prefetchType -> Grant.prefetchAckType))
|
||||
}
|
||||
|
||||
def makeUnion(
|
||||
a_type: UInt,
|
||||
addr_byte: UInt,
|
||||
operand_size: UInt,
|
||||
opcode: UInt,
|
||||
wmask: UInt,
|
||||
alloc: Bool): UInt = {
|
||||
MuxLookup(a_type, UInt(0), Array(
|
||||
Acquire.getType -> Cat(addr_byte, operand_size, opcode, alloc),
|
||||
Acquire.getBlockType -> Cat(operand_size, opcode, alloc),
|
||||
Acquire.putType -> Cat(wmask, alloc),
|
||||
Acquire.putBlockType -> Cat(wmask, alloc),
|
||||
Acquire.putAtomicType -> Cat(addr_byte, operand_size, opcode, alloc),
|
||||
Acquire.prefetchType -> Cat(opcode, alloc)))
|
||||
}
|
||||
|
||||
def fullWriteMask(implicit p: Parameters) = SInt(-1, width = p(TLKey(p(TLId))).writeMaskBits).toUInt
|
||||
|
||||
// Most generic constructor
|
||||
@ -295,6 +372,7 @@ object Acquire {
|
||||
acq.union := union
|
||||
acq
|
||||
}
|
||||
|
||||
// Copy constructor
|
||||
def apply(a: Acquire): Acquire = {
|
||||
val acq = Wire(new Acquire()(a.p))
|
||||
@ -303,6 +381,30 @@ object Acquire {
|
||||
}
|
||||
}
|
||||
|
||||
object BuiltInAcquireBuilder {
|
||||
def apply(
|
||||
a_type: UInt,
|
||||
client_xact_id: UInt,
|
||||
addr_block: UInt,
|
||||
addr_beat: UInt = UInt(0),
|
||||
data: UInt = UInt(0),
|
||||
addr_byte: UInt = UInt(0),
|
||||
operand_size: UInt = MT_Q,
|
||||
opcode: UInt = UInt(0),
|
||||
wmask: UInt = UInt(0),
|
||||
alloc: Bool = Bool(true))
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
a_type = a_type,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
addr_beat = addr_beat,
|
||||
data = data,
|
||||
union = Acquire.makeUnion(a_type, addr_byte, operand_size, opcode, wmask, alloc))
|
||||
}
|
||||
}
|
||||
|
||||
/** Get a single beat of data from the outer memory hierarchy
|
||||
*
|
||||
* The client can hint whether he block containing this beat should be
|
||||
@ -322,13 +424,13 @@ object Get {
|
||||
addr_beat: UInt,
|
||||
alloc: Bool = Bool(true))
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.getType,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
addr_beat = addr_beat,
|
||||
union = Cat(MT_Q, M_XRD, alloc))
|
||||
opcode = M_XRD,
|
||||
alloc = alloc)
|
||||
}
|
||||
def apply(
|
||||
client_xact_id: UInt,
|
||||
@ -338,13 +440,15 @@ object Get {
|
||||
operand_size: UInt,
|
||||
alloc: Bool)
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.getType,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
addr_beat = addr_beat,
|
||||
union = Cat(addr_byte, operand_size, M_XRD, alloc))
|
||||
addr_byte = addr_byte,
|
||||
operand_size = operand_size,
|
||||
opcode = M_XRD,
|
||||
alloc = alloc)
|
||||
}
|
||||
}
|
||||
|
||||
@ -363,12 +467,12 @@ object GetBlock {
|
||||
addr_block: UInt,
|
||||
alloc: Bool = Bool(true))
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.getBlockType,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
union = Cat(MT_Q, M_XRD, alloc))
|
||||
opcode = M_XRD,
|
||||
alloc = alloc)
|
||||
}
|
||||
}
|
||||
|
||||
@ -383,13 +487,11 @@ object GetPrefetch {
|
||||
client_xact_id: UInt,
|
||||
addr_block: UInt)
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.prefetchType,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
addr_beat = UInt(0),
|
||||
union = Cat(MT_Q, M_XRD, Bool(true)))
|
||||
opcode = M_XRD)
|
||||
}
|
||||
}
|
||||
|
||||
@ -413,14 +515,14 @@ object Put {
|
||||
wmask: Option[UInt]= None,
|
||||
alloc: Bool = Bool(true))
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.putType,
|
||||
addr_block = addr_block,
|
||||
addr_beat = addr_beat,
|
||||
client_xact_id = client_xact_id,
|
||||
data = data,
|
||||
union = Cat(wmask.getOrElse(Acquire.fullWriteMask), alloc))
|
||||
wmask = wmask.getOrElse(Acquire.fullWriteMask),
|
||||
alloc = alloc)
|
||||
}
|
||||
}
|
||||
|
||||
@ -445,14 +547,14 @@ object PutBlock {
|
||||
data: UInt,
|
||||
wmask: UInt)
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.putBlockType,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
addr_beat = addr_beat,
|
||||
data = data,
|
||||
union = Cat(wmask, (wmask != Acquire.fullWriteMask)))
|
||||
wmask = wmask,
|
||||
alloc = (wmask != Acquire.fullWriteMask))
|
||||
}
|
||||
def apply(
|
||||
client_xact_id: UInt,
|
||||
@ -461,14 +563,14 @@ object PutBlock {
|
||||
data: UInt,
|
||||
alloc: Bool = Bool(true))
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.putBlockType,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
addr_beat = addr_beat,
|
||||
data = data,
|
||||
union = Cat(Acquire.fullWriteMask, alloc))
|
||||
wmask = Acquire.fullWriteMask,
|
||||
alloc = alloc)
|
||||
}
|
||||
}
|
||||
|
||||
@ -483,13 +585,11 @@ object PutPrefetch {
|
||||
client_xact_id: UInt,
|
||||
addr_block: UInt)
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.prefetchType,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
addr_beat = UInt(0),
|
||||
union = Cat(M_XWR, Bool(true)))
|
||||
opcode = M_XWR)
|
||||
}
|
||||
}
|
||||
|
||||
@ -513,14 +613,15 @@ object PutAtomic {
|
||||
operand_size: UInt,
|
||||
data: UInt)
|
||||
(implicit p: Parameters): Acquire = {
|
||||
Acquire(
|
||||
is_builtin_type = Bool(true),
|
||||
BuiltInAcquireBuilder(
|
||||
a_type = Acquire.putAtomicType,
|
||||
client_xact_id = client_xact_id,
|
||||
addr_block = addr_block,
|
||||
addr_beat = addr_beat,
|
||||
data = data,
|
||||
union = Cat(addr_byte, operand_size, atomic_opcode, Bool(true)))
|
||||
addr_byte = addr_byte,
|
||||
operand_size = operand_size,
|
||||
opcode = atomic_opcode)
|
||||
}
|
||||
}
|
||||
|
||||
@ -529,14 +630,9 @@ object PutAtomic {
|
||||
* The available types of Probes are customized by a particular
|
||||
* [[uncore.CoherencePolicy]].
|
||||
*/
|
||||
class Probe(implicit p: Parameters) extends ManagerToClientChannel()(p)
|
||||
with HasCacheBlockAddress {
|
||||
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)
|
||||
}
|
||||
class Probe(implicit p: Parameters) extends ManagerToClientChannel
|
||||
with HasCacheBlockAddress
|
||||
with HasProbeType
|
||||
|
||||
/** [[uncore.Probe]] with an extra field stating its destination id */
|
||||
class ProbeToDst(implicit p: Parameters) extends Probe()(p) with HasClientId
|
||||
@ -573,34 +669,29 @@ object Probe {
|
||||
* a particular [[uncore.CoherencePolicy]]. Releases may contain data or may be
|
||||
* simple acknowledgements. Voluntary Releases are acknowledged with [[uncore.Grant Grants]].
|
||||
*/
|
||||
class ReleaseMetadata(implicit p: Parameters) extends ClientToManagerChannel()(p)
|
||||
class ReleaseMetadata(implicit p: Parameters) extends ClientToManagerChannel
|
||||
with HasTileLinkBeatId
|
||||
with HasCacheBlockAddress
|
||||
with HasClientTransactionId {
|
||||
val r_type = UInt(width = tlCoh.releaseTypeWidth)
|
||||
val voluntary = Bool()
|
||||
|
||||
// Helper funcs
|
||||
def is(t: UInt) = r_type === t
|
||||
def hasData(dummy: Int = 0) = tlCoh.releaseTypesWithData.contains(r_type)
|
||||
//TODO: Assumes all releases write back full cache blocks:
|
||||
def hasMultibeatData(dummy: Int = 0) = Bool(tlDataBeats > 1) && tlCoh.releaseTypesWithData.contains(r_type)
|
||||
def isVoluntary(dummy: Int = 0) = voluntary
|
||||
def requiresAck(dummy: Int = 0) = !Bool(tlNetworkPreservesPointToPointOrdering)
|
||||
with HasClientTransactionId
|
||||
with HasReleaseType {
|
||||
def full_addr(dummy: Int = 0) = Cat(this.addr_block, this.addr_beat, UInt(0, width = tlByteAddrBits))
|
||||
}
|
||||
|
||||
/** [[uncore.ReleaseMetadata]] with an extra field containing the data beat */
|
||||
class Release(implicit p: Parameters) extends ReleaseMetadata()(p) with HasTileLinkData
|
||||
class Release(implicit p: Parameters) extends ReleaseMetadata
|
||||
with HasTileLinkData
|
||||
|
||||
/** [[uncore.ReleaseMetadata]] with an extra field containing the entire cache block */
|
||||
class BufferedRelease(implicit p: Parameters) extends ReleaseMetadata()(p) with HasTileLinkBlock
|
||||
class BufferedRelease(implicit p: Parameters) extends ReleaseMetadata
|
||||
with HasTileLinkBlock
|
||||
|
||||
/** [[uncore.Release]] with an extra field stating its source id */
|
||||
class ReleaseFromSrc(implicit p: Parameters) extends Release()(p) with HasClientId
|
||||
class ReleaseFromSrc(implicit p: Parameters) extends Release
|
||||
with HasClientId
|
||||
|
||||
/** [[uncore.BufferedRelease]] with an extra field stating its source id */
|
||||
class BufferedReleaseFromSrc(implicit p: Parameters) extends BufferedRelease()(p) with HasClientId
|
||||
class BufferedReleaseFromSrc(implicit p: Parameters) extends BufferedRelease
|
||||
with HasClientId
|
||||
|
||||
/** Contains a [[uncore.Release]] factory
|
||||
*
|
||||
@ -641,26 +732,11 @@ object Release {
|
||||
* coherence policies may also define custom Grant types. Grants may contain data
|
||||
* or may be simple acknowledgements. Grants are responded to with [[uncore.Finish]].
|
||||
*/
|
||||
class GrantMetadata(implicit p: Parameters) extends ManagerToClientChannel()(p)
|
||||
class GrantMetadata(implicit p: Parameters) extends ManagerToClientChannel
|
||||
with HasTileLinkBeatId
|
||||
with HasClientTransactionId
|
||||
with HasManagerTransactionId {
|
||||
val is_builtin_type = Bool()
|
||||
val g_type = UInt(width = tlGrantTypeBits)
|
||||
|
||||
// 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),
|
||||
tlCoh.grantTypesWithData.contains(g_type))
|
||||
def hasMultibeatData(dummy: Int = 0): Bool =
|
||||
Bool(tlDataBeats > 1) && Mux(isBuiltInType(),
|
||||
Grant.typesWithMultibeatData.contains(g_type),
|
||||
tlCoh.grantTypesWithData.contains(g_type))
|
||||
def isVoluntary(dummy: Int = 0): Bool = isBuiltInType() && (g_type === Grant.voluntaryAckType)
|
||||
def requiresAck(dummy: Int = 0): Bool = !Bool(tlNetworkPreservesPointToPointOrdering) && !isVoluntary()
|
||||
with HasManagerTransactionId
|
||||
with HasGrantType {
|
||||
def makeFinish(dummy: Int = 0): Finish = {
|
||||
val f = Wire(new Finish)
|
||||
f.manager_xact_id := this.manager_xact_id
|
||||
@ -669,16 +745,20 @@ class GrantMetadata(implicit p: Parameters) extends ManagerToClientChannel()(p)
|
||||
}
|
||||
|
||||
/** [[uncore.GrantMetadata]] with an extra field containing a single beat of data */
|
||||
class Grant(implicit p: Parameters) extends GrantMetadata()(p) with HasTileLinkData
|
||||
class Grant(implicit p: Parameters) extends GrantMetadata
|
||||
with HasTileLinkData
|
||||
|
||||
/** [[uncore.Grant]] with an extra field stating its destination */
|
||||
class GrantToDst(implicit p: Parameters) extends Grant()(p) with HasClientId
|
||||
class GrantToDst(implicit p: Parameters) extends Grant
|
||||
with HasClientId
|
||||
|
||||
/** [[uncore.GrantMetadata]] with an extra field containing an entire cache block */
|
||||
class BufferedGrant(implicit p: Parameters) extends GrantMetadata()(p) with HasTileLinkBlock
|
||||
class BufferedGrant(implicit p: Parameters) extends GrantMetadata
|
||||
with HasTileLinkBlock
|
||||
|
||||
/** [[uncore.BufferedGrant]] with an extra field stating its destination */
|
||||
class BufferedGrantToDst(implicit p: Parameters) extends BufferedGrant()(p) with HasClientId
|
||||
class BufferedGrantToDst(implicit p: Parameters) extends BufferedGrant
|
||||
with HasClientId
|
||||
|
||||
/** Contains definitions of the the built-in grant types and factories
|
||||
* for [[uncore.Grant]] and [[uncore.GrantToDst]]
|
||||
@ -854,14 +934,6 @@ class ClientTileLinkIOWrapper(implicit p: Parameters) extends TLModule()(p) {
|
||||
io.out.release.valid := Bool(false)
|
||||
}
|
||||
|
||||
/** Used to track metadata for transactions where multiple secondary misses have been merged
|
||||
* and handled by a single transaction tracker.
|
||||
*/
|
||||
class SecondaryMissInfo(implicit p: Parameters) extends TLBundle()(p)
|
||||
with HasTileLinkBeatId
|
||||
with HasClientTransactionId
|
||||
// TODO: add a_type to merge e.g. Get+GetBlocks, and/or HasClientId
|
||||
|
||||
/** A helper module that automatically issues [[uncore.Finish]] messages in repsonse
|
||||
* to [[uncore.Grant]] that it receives from a manager and forwards to a client
|
||||
*/
|
||||
@ -1264,8 +1336,10 @@ trait HasDataBeatCounters {
|
||||
}
|
||||
|
||||
/** Counter for beats on outgoing [[chisel.DecoupledIO]] */
|
||||
def connectOutgoingDataBeatCounter[T <: TileLinkChannel](in: DecoupledIO[T], beat: UInt = UInt(0)): (UInt, Bool) =
|
||||
connectDataBeatCounter(in.fire(), in.bits, beat)
|
||||
def connectOutgoingDataBeatCounter[T <: TileLinkChannel](
|
||||
out: DecoupledIO[T],
|
||||
beat: UInt = UInt(0)): (UInt, Bool) =
|
||||
connectDataBeatCounter(out.fire(), out.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.
|
||||
|
@ -132,11 +132,11 @@ abstract class XactTracker(implicit p: Parameters) extends CoherenceAgentModule(
|
||||
def dropPendingBitWhenBeat[T <: HasBeat](dec: Bool, in: T): UInt =
|
||||
~Fill(in.tlDataBeats, dec) | ~UIntToOH(in.addr_beat)
|
||||
|
||||
def addPendingBitWhenBeatHasData[T <: HasBeat](in: DecoupledIO[T]): UInt =
|
||||
addPendingBitWhenBeat(in.fire() && in.bits.hasData(), in.bits)
|
||||
def addPendingBitWhenBeatHasData[T <: HasBeat](in: DecoupledIO[T], inc: Bool = Bool(true)): UInt =
|
||||
addPendingBitWhenBeat(in.fire() && in.bits.hasData() && inc, in.bits)
|
||||
|
||||
def addPendingBitWhenBeatHasDataAndAllocs(in: DecoupledIO[AcquireFromSrc]): UInt =
|
||||
addPendingBitWhenBeat(in.fire() && in.bits.hasData() && in.bits.allocate(), in.bits)
|
||||
addPendingBitWhenBeatHasData(in, in.bits.allocate())
|
||||
|
||||
def addPendingBitWhenBeatIsGetOrAtomic(in: DecoupledIO[AcquireFromSrc]): UInt = {
|
||||
val a = in.bits
|
||||
@ -145,6 +145,9 @@ abstract class XactTracker(implicit p: Parameters) extends CoherenceAgentModule(
|
||||
addPendingBitWhenBeat(in.fire() && isGetOrAtomic, a)
|
||||
}
|
||||
|
||||
def addPendingBitsFromAcquire(a: SecondaryMissInfo): UInt =
|
||||
Mux(a.hasMultibeatData(), Fill(a.tlDataBeats, UInt(1, 1)), UIntToOH(a.addr_beat))
|
||||
|
||||
def dropPendingBitWhenBeatHasData[T <: HasBeat](in: DecoupledIO[T]): UInt =
|
||||
dropPendingBitWhenBeat(in.fire() && in.bits.hasData(), in.bits)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user