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:
@ -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 {
|
||||
|
Reference in New Issue
Block a user