From 16bfbda3c9ff3c4a46ff2705eefb58d98f8597e8 Mon Sep 17 00:00:00 2001 From: Henry Cook Date: Mon, 4 Apr 2016 22:17:11 -0700 Subject: [PATCH] Refactor the TransactionTracker logic in all the L2 TileLink Managers. They now share common sub-transactions within traits, and use a common set of state transitions and scoreboarding logic. Tracker allocation logic has also been updated. No changes to external IOs or the TileLink protocol. A new bufferless Broadcast hub is also included, but does not yet pass fuzzing checks. --- uncore/src/main/scala/broadcast.scala | 512 ++++------- uncore/src/main/scala/bufferless.scala | 141 +++ uncore/src/main/scala/cache.scala | 1117 ++++++++++-------------- uncore/src/main/scala/converters.scala | 38 +- uncore/src/main/scala/metadata.scala | 3 +- uncore/src/main/scala/sdq.scala | 118 +++ uncore/src/main/scala/tilelink.scala | 19 +- uncore/src/main/scala/trackers.scala | 546 ++++++++++++ uncore/src/main/scala/uncore.scala | 106 +-- 9 files changed, 1486 insertions(+), 1114 deletions(-) create mode 100644 uncore/src/main/scala/bufferless.scala create mode 100644 uncore/src/main/scala/sdq.scala create mode 100644 uncore/src/main/scala/trackers.scala diff --git a/uncore/src/main/scala/broadcast.scala b/uncore/src/main/scala/broadcast.scala index 5ecaa7e2..1c51287f 100644 --- a/uncore/src/main/scala/broadcast.scala +++ b/uncore/src/main/scala/broadcast.scala @@ -4,390 +4,198 @@ package uncore import Chisel._ import cde.{Parameters, Field} -case object L2StoreDataQueueDepth extends Field[Int] +class L2BroadcastHub(implicit p: Parameters) extends HierarchicalCoherenceAgent()(p) { -trait HasBroadcastHubParameters extends HasCoherenceAgentParameters { - val sdqDepth = p(L2StoreDataQueueDepth)*innerDataBeats - val dqIdxBits = math.max(log2Up(nReleaseTransactors) + 1, log2Up(sdqDepth)) - val nDataQueueLocations = 3 //Stores, VoluntaryWBs, Releases -} - -class DataQueueLocation(implicit p: Parameters) extends CoherenceAgentBundle()(p) - with HasBroadcastHubParameters { - val idx = UInt(width = dqIdxBits) - val loc = UInt(width = log2Ceil(nDataQueueLocations)) -} - -object DataQueueLocation { - def apply(idx: UInt, loc: UInt)(implicit p: Parameters) = { - val d = Wire(new DataQueueLocation) - d.idx := idx - d.loc := loc - d - } -} - -class L2BroadcastHub(implicit p: Parameters) extends ManagerCoherenceAgent()(p) - with HasBroadcastHubParameters { - val internalDataBits = new DataQueueLocation().getWidth - val inStoreQueue :: inVolWBQueue :: inClientReleaseQueue :: Nil = Enum(UInt(), nDataQueueLocations) - - val usingStoreDataQueue = p.alterPartial({ - case TLKey(`innerTLId`) => innerTLParams.copy(overrideDataBitsPerBeat = Some(internalDataBits)) - case TLKey(`outerTLId`) => outerTLParams.copy(overrideDataBitsPerBeat = Some(internalDataBits)) - }) - - // Create SHRs for outstanding transactions - val trackerList = + // Create TSHRs for outstanding transactions + val irelTrackerList = (0 until nReleaseTransactors).map(id => - Module(new BroadcastVoluntaryReleaseTracker(id)(usingStoreDataQueue))) ++ + Module(new BufferedBroadcastVoluntaryReleaseTracker(id))) + val iacqTrackerList = (nReleaseTransactors until nTransactors).map(id => - Module(new BroadcastAcquireTracker(id)(usingStoreDataQueue))) + Module(new BufferedBroadcastAcquireTracker(id))) + val trackerList = irelTrackerList ++ iacqTrackerList // Propagate incoherence flags - trackerList.map(_.io.incoherent := io.incoherent) + trackerList.map(_.io.incoherent) foreach { _ := io.incoherent } - // Queue to store impending Put data - val sdq = Reg(Vec(sdqDepth, io.iacq().data)) - val sdq_val = Reg(init=Bits(0, sdqDepth)) - val sdq_alloc_id = PriorityEncoder(~sdq_val) - val sdq_rdy = !sdq_val.andR - val sdq_enq = trackerList.map( t => - (t.io.alloc.iacq || t.io.matches.iacq) && - t.io.inner.acquire.fire() && - t.io.iacq().hasData() - ).reduce(_||_) - when (sdq_enq) { sdq(sdq_alloc_id) := io.iacq().data } + // Create an arbiter for the one memory port + val outerList = trackerList.map(_.io.outer) + val outer_arb = Module(new ClientTileLinkIOArbiter(outerList.size) + (p.alterPartial({ case TLId => p(OuterTLId) }))) + outer_arb.io.in <> outerList + io.outer <> outer_arb.io.out // Handle acquire transaction initiation val irel_vs_iacq_conflict = - io.inner.acquire.valid && - io.inner.release.valid && - io.irel().conflicts(io.iacq()) - val sdqLoc = List.fill(nTransactors) { - DataQueueLocation(sdq_alloc_id, inStoreQueue).toBits - } - doInputRoutingWithAllocation( - io.inner.acquire, - trackerList.map(_.io.inner.acquire), - trackerList.map(_.io.matches.iacq), - trackerList.map(_.io.alloc.iacq), - Some(sdqLoc), - Some(sdq_rdy && !irel_vs_iacq_conflict), - Some(sdq_rdy)) + io.inner.acquire.valid && + io.inner.release.valid && + io.irel().conflicts(io.iacq()) - // Queue to store impending Voluntary Release data - val voluntary = io.irel().isVoluntary() - val vwbdq_enq = io.inner.release.fire() && voluntary && io.irel().hasData() - val (rel_data_cnt, rel_data_done) = Counter(vwbdq_enq, innerDataBeats) //TODO Zero width - val vwbdq = Reg(Vec(innerDataBeats, io.irel().data)) //TODO Assumes nReleaseTransactors == 1 - when(vwbdq_enq) { vwbdq(rel_data_cnt) := io.irel().data } + doInputRoutingWithAllocation( + in = io.inner.acquire, + outs = trackerList.map(_.io.inner.acquire), + allocs = trackerList.map(_.io.alloc_iacq), + allocOverride = !irel_vs_iacq_conflict) // Handle releases, which might be voluntary and might have data - val vwbqLoc = (0 until nTransactors).map(i => - (DataQueueLocation(rel_data_cnt, - (if(i < nReleaseTransactors) inVolWBQueue - else inClientReleaseQueue)).toBits)) doInputRoutingWithAllocation( - io.inner.release, - trackerList.map(_.io.inner.release), - trackerList.map(_.io.matches.irel), - trackerList.map(_.io.alloc.irel), - Some(vwbqLoc)) + in = io.inner.release, + outs = trackerList.map(_.io.inner.release), + allocs = trackerList.map(_.io.alloc_irel)) // Wire probe requests and grant reply to clients, finish acks from clients - // Note that we bypass the Grant data subbundles - doOutputArbitration(io.inner.grant, trackerList.map(_.io.inner.grant)) - io.inner.grant.bits.data := io.outer.grant.bits.data - io.inner.grant.bits.addr_beat := io.outer.grant.bits.addr_beat doOutputArbitration(io.inner.probe, trackerList.map(_.io.inner.probe)) + + doOutputArbitration(io.inner.grant, trackerList.map(_.io.inner.grant)) + doInputRouting(io.inner.finish, trackerList.map(_.io.inner.finish)) - - // Create an arbiter for the one memory port - val outer_arb = Module(new ClientUncachedTileLinkIOArbiter(trackerList.size) - (usingStoreDataQueue.alterPartial({ case TLId => p(OuterTLId) }))) - outer_arb.io.in <> trackerList.map(_.io.outer) - // Get the pending data out of the store data queue - val outer_data_ptr = new DataQueueLocation().fromBits(outer_arb.io.out.acquire.bits.data) - val is_in_sdq = outer_data_ptr.loc === inStoreQueue - val free_sdq = io.outer.acquire.fire() && - io.outer.acquire.bits.hasData() && - outer_data_ptr.loc === inStoreQueue - io.outer <> outer_arb.io.out - io.outer.acquire.bits.data := MuxLookup(outer_data_ptr.loc, io.irel().data, Array( - inStoreQueue -> sdq(outer_data_ptr.idx), - inVolWBQueue -> vwbdq(outer_data_ptr.idx))) - - // Update SDQ valid bits - when (io.outer.acquire.valid || sdq_enq) { - sdq_val := sdq_val & ~(UIntToOH(outer_data_ptr.idx) & Fill(sdqDepth, free_sdq)) | - PriorityEncoderOH(~sdq_val(sdqDepth-1,0)) & Fill(sdqDepth, sdq_enq) - } } class BroadcastXactTracker(implicit p: Parameters) extends XactTracker()(p) { - val io = new ManagerXactTrackerIO + val io = new HierarchicalXactTrackerIO pinAllReadyValidLow(io) } -class BroadcastVoluntaryReleaseTracker(trackerId: Int) - (implicit p: Parameters) extends BroadcastXactTracker()(p) { - val s_idle :: s_busy :: Nil = Enum(UInt(), 2) - val state = Reg(init=s_idle) +trait BroadcastsToAllClients extends HasCoherenceAgentParameters { + val coh = HierarchicalMetadata.onReset + val inner_coh = coh.inner + val outer_coh = coh.outer + def full_representation = ~UInt(0, width = innerNCachingClients) +} - val xact = Reg(new BufferedReleaseFromSrc()(p.alterPartial({ case TLId => innerTLId }))) - val coh = ManagerMetadata.onReset - - val pending_irels = Reg(init=Bits(0, width = io.inner.tlDataBeats)) - val pending_writes = Reg(init=Bits(0, width = io.outer.tlDataBeats)) - val pending_ignt = Reg(init=Bool(false)) - - val all_pending_done = !(pending_irels.orR || pending_writes.orR || pending_ignt) - - // Accept a voluntary Release (and any further beats of data) - pending_irels := (pending_irels & dropPendingBitWhenBeatHasData(io.inner.release)) - io.inner.release.ready := ((state === s_idle) && io.irel().isVoluntary()) || pending_irels.orR - when(io.inner.release.fire()) { xact.data_buffer(io.irel().addr_beat) := io.irel().data } - - // Write the voluntarily written back data to outer memory using an Acquire.PutBlock - //TODO: Use io.outer.release instead? - pending_writes := (pending_writes & dropPendingBitWhenBeatHasData(io.outer.acquire)) | - addPendingBitWhenBeatHasData(io.inner.release) - val curr_write_beat = PriorityEncoder(pending_writes) - io.outer.acquire.valid := state === s_busy && pending_writes.orR - io.outer.acquire.bits := PutBlock( - client_xact_id = UInt(trackerId), - addr_block = xact.addr_block, - addr_beat = curr_write_beat, - data = xact.data_buffer(curr_write_beat)) - (p.alterPartial({ case TLId => outerTLId })) - - // Send an acknowledgement - io.inner.grant.valid := state === s_busy && pending_ignt && !pending_irels && io.outer.grant.valid - io.inner.grant.bits := coh.makeGrant(xact) - when(io.inner.grant.fire()) { pending_ignt := Bool(false) } - io.outer.grant.ready := state === s_busy && io.inner.grant.ready - - // State machine updates and transaction handler metadata intialization - when(state === s_idle && io.inner.release.valid && io.alloc.irel) { - xact := io.irel() - when(io.irel().hasMultibeatData()) { - pending_irels := dropPendingBitWhenBeatHasData(io.inner.release) - }. otherwise { - pending_irels := UInt(0) - } - pending_writes := addPendingBitWhenBeatHasData(io.inner.release) - pending_ignt := io.irel().requiresAck() - state := s_busy - } - when(state === s_busy && all_pending_done) { state := s_idle } - - // These IOs are used for routing in the parent - io.matches.iacq := (state =/= s_idle) && xact.conflicts(io.iacq()) - io.matches.irel := (state =/= s_idle) && xact.conflicts(io.irel()) && io.irel().isVoluntary() - io.matches.oprb := Bool(false) +abstract class BroadcastVoluntaryReleaseTracker(trackerId: Int)(implicit p: Parameters) + extends VoluntaryReleaseTracker(trackerId)(p) + with EmitsVoluntaryReleases + with BroadcastsToAllClients { + val io = new HierarchicalXactTrackerIO + pinAllReadyValidLow(io) // Checks for illegal behavior - assert(!(state === s_idle && io.inner.release.fire() && !io.irel().isVoluntary()), + assert(!(state === s_idle && io.inner.release.fire() && io.alloc_irel.should && !io.irel().isVoluntary()), "VoluntaryReleaseTracker accepted Release that wasn't voluntary!") } -class BroadcastAcquireTracker(trackerId: Int) - (implicit p: Parameters) extends BroadcastXactTracker()(p) { - val s_idle :: s_probe :: s_mem_read :: s_mem_write :: s_make_grant :: s_mem_resp :: s_ack :: Nil = Enum(UInt(), 7) - val state = Reg(init=s_idle) +abstract class BroadcastAcquireTracker(trackerId: Int)(implicit p: Parameters) + extends AcquireTracker(trackerId)(p) + with EmitsVoluntaryReleases + with BroadcastsToAllClients { + val io = new HierarchicalXactTrackerIO + pinAllReadyValidLow(io) - val xact = Reg(new BufferedAcquireFromSrc()(p.alterPartial({ case TLId => innerTLId }))) - val coh = ManagerMetadata.onReset + val alwaysWriteFullBeat = false + val nSecondaryMisses = 1 + def iacq_can_merge = Bool(false) - assert(!(state =/= s_idle && xact.isBuiltInType() && - Vec(Acquire.putAtomicType, Acquire.getPrefetchType, Acquire.putPrefetchType).contains(xact.a_type)), - "Broadcast Hub does not support PutAtomics or prefetches") // TODO - - val release_count = Reg(init=UInt(0, width = log2Up(io.inner.tlNCachingClients+1))) - val pending_probes = Reg(init=Bits(0, width = io.inner.tlNCachingClients)) - val curr_p_id = PriorityEncoder(pending_probes) - val mask_self = SInt(-1, width = io.inner.tlNCachingClients) - .toUInt - .bitSet(io.inner.acquire.bits.client_id, io.inner.acquire.bits.requiresSelfProbe()) - val mask_incoherent = mask_self & ~io.incoherent.toBits - - val collect_iacq_data = Reg(init=Bool(false)) - val iacq_data_valid = Reg(init=Bits(0, width = innerDataBeats)) - val iacq_data_done = connectIncomingDataBeatCounter(io.inner.acquire) - val irel_data_done = connectIncomingDataBeatCounter(io.inner.release) - val (ignt_data_cnt, ignt_data_done) = connectOutgoingDataBeatCounter(io.inner.grant) - val (oacq_data_cnt, oacq_data_done) = connectOutgoingDataBeatCounter(io.outer.acquire) - val ognt_data_done = connectIncomingDataBeatCounter(io.outer.grant) - val pending_ognt_ack = Reg(init=Bool(false)) - val pending_outer_write = xact.hasData() - val pending_outer_write_ = io.iacq().hasData() - val pending_outer_read = io.ignt().hasData() - val pending_outer_read_ = coh.makeGrant(io.iacq(), UInt(trackerId)).hasData() - val subblock_type = xact.isSubBlockType() - - // These IOs are used for routing in the parent - io.matches.iacq := (state =/= s_idle) && xact.conflicts(io.iacq()) - io.matches.irel := (state =/= s_idle) && xact.conflicts(io.irel()) && !io.irel().isVoluntary() - io.matches.oprb := Bool(false) - - val outerParams = p.alterPartial({ case TLId => outerTLId }) - - val oacq_probe = PutBlock( - client_xact_id = UInt(trackerId), - addr_block = io.irel().addr_block, - addr_beat = io.irel().addr_beat, - data = io.irel().data)(outerParams) - - val oacq_write_beat = Put( - client_xact_id = UInt(trackerId), - addr_block = xact.addr_block, - addr_beat = xact.addr_beat, - data = xact.data_buffer(0), - wmask = xact.wmask())(outerParams) - - val oacq_write_block = PutBlock( - client_xact_id = UInt(trackerId), - addr_block = xact.addr_block, - addr_beat = oacq_data_cnt, - data = xact.data_buffer(oacq_data_cnt), - wmask = xact.wmask_buffer(oacq_data_cnt))(outerParams) - - val oacq_read_beat = Get( - client_xact_id = UInt(trackerId), - addr_block = xact.addr_block, - addr_beat = xact.addr_beat, - addr_byte = xact.addr_byte(), - operand_size = xact.op_size(), - alloc = Bool(false))(outerParams) - - val oacq_read_block = GetBlock( - client_xact_id = UInt(trackerId), - addr_block = xact.addr_block)(outerParams) - - io.outer.acquire.valid := Bool(false) - io.outer.acquire.bits := Mux(state === s_probe, oacq_probe, - Mux(state === s_mem_write, - Mux(subblock_type, oacq_write_beat, oacq_write_block), - Mux(subblock_type, oacq_read_beat, oacq_read_block))) - io.outer.grant.ready := Bool(false) - - io.inner.probe.valid := Bool(false) - io.inner.probe.bits := coh.makeProbe(curr_p_id, xact) - - io.inner.grant.valid := Bool(false) - io.inner.grant.bits := coh.makeGrant(xact, UInt(trackerId)) // Data bypassed in parent - - io.inner.acquire.ready := Bool(false) - io.inner.release.ready := Bool(false) - io.inner.finish.ready := Bool(false) - - assert(!(state =/= s_idle && collect_iacq_data && io.inner.acquire.fire() && - io.iacq().client_id =/= xact.client_id), - "AcquireTracker accepted data beat from different network source than initial request.") - - assert(!(state =/= s_idle && collect_iacq_data && io.inner.acquire.fire() && - io.iacq().client_xact_id =/= xact.client_xact_id), - "AcquireTracker accepted data beat from different client transaction than initial request.") - - assert(!(state === s_idle && io.inner.acquire.fire() && io.alloc.iacq && - io.iacq().hasMultibeatData() && io.iacq().addr_beat =/= UInt(0)), + // Checks for illegal behavior + // TODO: this could be allowed, but is a useful check against allocation gone wild + assert(!(state === s_idle && io.inner.acquire.fire() && io.alloc_iacq.should && + io.iacq().hasMultibeatData() && !io.iacq().first()), "AcquireTracker initialized with a tail data beat.") - when(collect_iacq_data) { - io.inner.acquire.ready := Bool(true) - when(io.inner.acquire.valid) { - xact.data_buffer(io.iacq().addr_beat) := io.iacq().data - xact.wmask_buffer(io.iacq().addr_beat) := io.iacq().wmask() - iacq_data_valid := iacq_data_valid.bitSet(io.iacq().addr_beat, Bool(true)) - } - when(iacq_data_done) { collect_iacq_data := Bool(false) } - } + assert(!(state =/= s_idle && pending_ignt && xact_iacq.isPrefetch()), + "Broadcast Hub does not support Prefetches.") + + assert(!(state =/= s_idle && pending_ignt && xact_iacq.isAtomic()), + "Broadcast Hub does not support PutAtomics.") +} + +class BufferedBroadcastVoluntaryReleaseTracker(trackerId: Int)(implicit p: Parameters) + extends BroadcastVoluntaryReleaseTracker(trackerId)(p) + with HasDataBuffer { + + // Tell the parent if any incoming messages conflict with the ongoing transaction + routeInParent() + io.alloc_iacq.can := Bool(false) + + // Start transaction by accepting inner release + innerRelease(block_vol_ignt = pending_orel || vol_ognt_counter.pending) + + // A release beat can be accepted if we are idle, if its a mergeable transaction, or if its a tail beat + io.inner.release.ready := state === s_idle || irel_can_merge || irel_same_xact + + when(irel_is_allocating) { pending_orel := io.irel().hasData() } + + when(io.inner.release.fire()) { data_buffer(io.irel().addr_beat) := io.irel().data } + + // Dispatch outer release + outerRelease( + coh = outer_coh.onHit(M_XWR), + data = data_buffer(vol_ognt_counter.up.idx)) + + quiesce() +} + +class BufferedBroadcastAcquireTracker(trackerId: Int)(implicit p: Parameters) + extends BroadcastAcquireTracker(trackerId)(p) + with HasByteWriteMaskBuffer { + + // Setup IOs used for routing in the parent + routeInParent() + io.alloc_irel.can := Bool(false) + + // First, take care of accpeting new acquires or secondary misses + // Handling of primary and secondary misses' data and write mask merging + innerAcquire( + can_alloc = Bool(false), + next = s_inner_probe) + + io.inner.acquire.ready := state === s_idle || iacq_can_merge || iacq_same_xact + + // Track which clients yet need to be probed and make Probe message + // If a writeback occurs, we can forward its data via the buffer, + // and skip having to go outwards + val skip_outer_acquire = pending_ignt_data.andR + + innerProbe( + inner_coh.makeProbe(curr_probe_dst, xact_iacq, xact_addr_block), + Mux(!skip_outer_acquire, s_outer_acquire, s_busy)) + + // Handle incoming releases from clients, which may reduce sharer counts + // and/or write back dirty data, and may be unexpected voluntary releases + def irel_can_merge = io.irel().conflicts(xact_addr_block) && + io.irel().isVoluntary() && + !Vec(s_idle, s_meta_write).contains(state) && + !all_pending_done && + !io.outer.grant.fire() && + !io.inner.grant.fire() && + !vol_ignt_counter.pending + + innerRelease(block_vol_ignt = vol_ognt_counter.pending) + + //TODO: accept vol irels when state === s_idle, operate like the VolRelTracker + io.inner.release.ready := irel_can_merge || irel_same_xact + + mergeDataInner(io.inner.release) + + // If there was a writeback, forward it outwards + outerRelease( + coh = outer_coh.onHit(M_XWR), + data = data_buffer(vol_ognt_counter.up.idx)) + + // Send outer request for miss + outerAcquire( + caching = !xact_iacq.isBuiltInType(), + coh = outer_coh, + data = data_buffer(ognt_counter.up.idx), + wmask = wmask_buffer(ognt_counter.up.idx), + next = s_busy) + + // Handle the response from outer memory + mergeDataOuter(io.outer.grant) + + // Acknowledge or respond with data + innerGrant( + data = data_buffer(ignt_data_idx), + external_pending = pending_orel || ognt_counter.pending || vol_ognt_counter.pending) + + when(iacq_is_allocating) { + wmask_buffer.foreach { w => w := UInt(0) } // This is the only reg that must be clear in s_idle + initializeProbes() + } + + initDataInner(io.inner.acquire) + + // Wait for everything to quiesce + quiesce() - when(pending_ognt_ack) { - io.outer.grant.ready := Bool(true) - when(io.outer.grant.valid) { pending_ognt_ack := Bool(false) } - //TODO add finish queue if this isnt the last level manager - } - - switch (state) { - is(s_idle) { - io.inner.acquire.ready := Bool(true) - when(io.inner.acquire.valid && io.alloc.iacq) { - xact := io.iacq() - xact.data_buffer(UInt(0)) := io.iacq().data - xact.wmask_buffer(UInt(0)) := io.iacq().wmask() - collect_iacq_data := io.iacq().hasMultibeatData() - iacq_data_valid := io.iacq().hasData() << io.iacq().addr_beat - val needs_probes = mask_incoherent.orR - when(needs_probes) { - pending_probes := mask_incoherent - release_count := PopCount(mask_incoherent) - } - state := Mux(needs_probes, s_probe, - Mux(pending_outer_write_, s_mem_write, - Mux(pending_outer_read_, s_mem_read, s_make_grant))) - } - } - is(s_probe) { - // Generate probes - io.inner.probe.valid := pending_probes.orR - when(io.inner.probe.ready) { - pending_probes := pending_probes & ~UIntToOH(curr_p_id) - } - - // Handle releases, which may have data to be written back - val matches = io.matches.irel - io.inner.release.ready := (!io.irel().hasData() || io.outer.acquire.ready) && matches - when(io.inner.release.valid && matches) { - when(io.irel().hasData()) { - io.outer.acquire.valid := Bool(true) - when(io.outer.acquire.ready) { - when(oacq_data_done) { - pending_ognt_ack := Bool(true) - release_count := release_count - UInt(1) - when(release_count === UInt(1)) { - state := Mux(pending_outer_write, s_mem_write, - Mux(pending_outer_read, s_mem_read, s_make_grant)) - } - } - } - } .otherwise { - release_count := release_count - UInt(1) - when(release_count === UInt(1)) { - state := Mux(pending_outer_write, s_mem_write, - Mux(pending_outer_read, s_mem_read, s_make_grant)) - } - } - } - } - is(s_mem_write) { // Write data to outer memory - io.outer.acquire.valid := !pending_ognt_ack && (!collect_iacq_data || iacq_data_valid(oacq_data_cnt)) - when(oacq_data_done) { - pending_ognt_ack := Bool(true) - state := Mux(pending_outer_read, s_mem_read, s_mem_resp) - } - } - is(s_mem_read) { // Read data from outer memory (possibly what was just written) - io.outer.acquire.valid := !pending_ognt_ack - when(io.outer.acquire.fire()) { state := s_mem_resp } - } - is(s_mem_resp) { // Wait to forward grants from outer memory - io.outer.grant.ready := io.inner.grant.ready - io.inner.grant.valid := io.outer.grant.valid - when(ignt_data_done) { - state := Mux(io.ignt().requiresAck(), s_ack, s_idle) - } - } - is(s_make_grant) { // Manufacture a local grant (some kind of permission upgrade) - io.inner.grant.valid := Bool(true) - when(io.inner.grant.ready) { - state := Mux(io.ignt().requiresAck(), s_ack, s_idle) - } - } - is(s_ack) { // Wait for transaction to complete - io.inner.finish.ready := Bool(true) - when(io.inner.finish.valid) { state := s_idle } - } - } } diff --git a/uncore/src/main/scala/bufferless.scala b/uncore/src/main/scala/bufferless.scala new file mode 100644 index 00000000..e5ddcb98 --- /dev/null +++ b/uncore/src/main/scala/bufferless.scala @@ -0,0 +1,141 @@ +// See LICENSE for license details. + +package uncore +import Chisel._ +import cde.{Parameters, Field} + + +class BufferlessBroadcastHub(implicit p: Parameters) extends HierarchicalCoherenceAgent()(p) { + + // Create TSHRs for outstanding transactions + val irelTrackerList = + (0 until nReleaseTransactors).map(id => + Module(new BufferlessBroadcastVoluntaryReleaseTracker(id))) + val iacqTrackerList = + (nReleaseTransactors until nTransactors).map(id => + Module(new BufferlessBroadcastAcquireTracker(id))) + val trackerList = irelTrackerList ++ iacqTrackerList + + // Propagate incoherence flags + trackerList.map(_.io.incoherent) foreach { _ := io.incoherent } + + // Create an arbiter for the one memory port + val outerList = trackerList.map(_.io.outer) + val outer_arb = Module(new ClientTileLinkIOArbiter(outerList.size) + (p.alterPartial({ case TLId => p(OuterTLId) }))) + outer_arb.io.in <> outerList + io.outer <> outer_arb.io.out + + // Handle acquire transaction initiation + val irel_vs_iacq_conflict = + io.inner.acquire.valid && + io.inner.release.valid && + io.irel().conflicts(io.iacq()) + + doInputRoutingWithAllocation( + in = io.inner.acquire, + outs = trackerList.map(_.io.inner.acquire), + allocs = trackerList.map(_.io.alloc_iacq), + allocOverride = !irel_vs_iacq_conflict) + io.outer.acquire.bits.data := io.inner.acquire.bits.data + io.outer.acquire.bits.addr_beat := io.inner.acquire.bits.addr_beat + + // Handle releases, which might be voluntary and might have data + doInputRoutingWithAllocation( + in = io.inner.release, + outs = trackerList.map(_.io.inner.release), + allocs = trackerList.map(_.io.alloc_irel)) + io.outer.release.bits.data := io.inner.release.bits.data + io.outer.release.bits.addr_beat := io.inner.release.bits.addr_beat + + // Wire probe requests and grant reply to clients, finish acks from clients + doOutputArbitration(io.inner.probe, trackerList.map(_.io.inner.probe)) + + doOutputArbitration(io.inner.grant, trackerList.map(_.io.inner.grant)) + io.inner.grant.bits.data := io.outer.grant.bits.data + io.inner.grant.bits.addr_beat := io.outer.grant.bits.addr_beat + + doInputRouting(io.inner.finish, trackerList.map(_.io.inner.finish)) +} + +class BufferlessBroadcastVoluntaryReleaseTracker(trackerId: Int)(implicit p: Parameters) + extends BroadcastVoluntaryReleaseTracker(trackerId)(p) { + + // Tell the parent if any incoming messages conflict with the ongoing transaction + routeInParent() + io.alloc_iacq.can := Bool(false) + + // Start transaction by accepting inner release + innerRelease(block_vol_ignt = pending_orel || vol_ognt_counter.pending) + + // A release beat can be accepted if we are idle, if its a mergeable transaction, or if its a tail beat + // and if the outer relase path is clear + val irel_could_accept = state === s_idle || irel_can_merge || irel_same_xact + io.inner.release.ready := irel_could_accept && + (!io.irel().hasData() || io.outer.release.ready) + + // Dispatch outer release + outerRelease(coh = outer_coh.onHit(M_XWR)) + io.outer.grant.ready := state === s_busy && io.inner.grant.ready // bypass data + + quiesce() +} + +class BufferlessBroadcastAcquireTracker(trackerId: Int)(implicit p: Parameters) + extends BroadcastAcquireTracker(trackerId)(p) { + + // Setup IOs used for routing in the parent + routeInParent() + io.alloc_irel.can := Bool(false) + + // First, take care of accpeting new acquires or secondary misses + // Handling of primary and secondary misses' data and write mask merging + innerAcquire( + can_alloc = Bool(false), + next = s_inner_probe) + + val iacq_could_accept = state === s_outer_acquire || iacq_can_merge || iacq_same_xact + io.inner.acquire.ready := iacq_could_accept && + (!io.iacq().hasData() || io.outer.acquire.fire()) + + // Track which clients yet need to be probed and make Probe message + innerProbe( + inner_coh.makeProbe(curr_probe_dst, xact_iacq, xact_addr_block), + s_outer_acquire) + + // Handle incoming releases from clients, which may reduce sharer counts + // and/or write back dirty data, and may be unexpected voluntary releases + def irel_can_merge = io.irel().conflicts(xact_addr_block) && + io.irel().isVoluntary() && + !vol_ignt_counter.pending && + (state =/= s_idle) + + innerRelease(block_vol_ignt = vol_ognt_counter.pending) + + val irel_could_accept = irel_can_merge || irel_same_xact + io.inner.release.ready := irel_could_accept && + (!io.irel().hasData() || io.outer.release.ready) + + // If there was a writeback, forward it outwards + outerRelease( + coh = outer_coh.onHit(M_XWR), + buffering = Bool(false)) + + // Send outer request for miss + outerAcquire( + caching = !xact_iacq.isBuiltInType(), + buffering = Bool(false), + coh = outer_coh, + next = s_busy) + + // Handle the response from outer memory + io.outer.grant.ready := state === s_busy && io.inner.grant.ready // bypass data + + // Acknowledge or respond with data + innerGrant(external_pending = pending_orel || ognt_counter.pending || vol_ognt_counter.pending) + + when(iacq_is_allocating) { initializeProbes() } + + // Wait for everything to quiesce + quiesce() +} diff --git a/uncore/src/main/scala/cache.scala b/uncore/src/main/scala/cache.scala index d5efde13..2ea72f0d 100644 --- a/uncore/src/main/scala/cache.scala +++ b/uncore/src/main/scala/cache.scala @@ -181,13 +181,17 @@ class MetadataArray[T <: Metadata](onReset: () => T)(implicit p: Parameters) ext case object L2DirectoryRepresentation extends Field[DirectoryRepresentation] -trait HasL2HellaCacheParameters extends HasCacheParameters with HasCoherenceAgentParameters { +trait HasOuterCacheParameters extends HasCacheParameters with HasCoherenceAgentParameters { val cacheId = p(CacheId) val idxLSB = cacheIdBits val idxMSB = idxLSB + idxBits - 1 val tagLSB = idxLSB + idxBits - def inSameSet(addr1: UInt, addr2: UInt): Bool = addr1(idxMSB,idxLSB) === addr2(idxMSB,idxLSB) - def haveSameTag(addr1: UInt, addr2: UInt): Bool = addr1 >> UInt(tagLSB) === addr2 >> UInt(tagLSB) + def inSameSet(block: HasCacheBlockAddress, addr: UInt): Bool = { + block.addr_block(idxMSB,idxLSB) === addr(idxMSB,idxLSB) + } + def haveSameTag(block: HasCacheBlockAddress, addr: UInt): Bool = { + block.addr_block >> UInt(tagLSB) === addr >> UInt(tagLSB) + } //val blockAddrBits = p(TLBlockAddrBits) val refillCyclesPerBeat = outerDataBits/rowBits val refillCycles = refillCyclesPerBeat*outerDataBeats @@ -198,11 +202,11 @@ trait HasL2HellaCacheParameters extends HasCacheParameters with HasCoherenceAgen require(rowBits == innerDataBits) // TODO: relax this by improving s_data_* states val nSecondaryMisses = p(NSecondaryMisses) val isLastLevelCache = true - val ignoresWriteMask = !p(ECCCode).isEmpty + val alwaysWriteFullBeat = !p(ECCCode).isEmpty } abstract class L2HellaCacheModule(implicit val p: Parameters) extends Module - with HasL2HellaCacheParameters { + with HasOuterCacheParameters { def doInternalOutputArbitration[T <: Data : ClassTag]( out: DecoupledIO[T], ins: Seq[DecoupledIO[T]]) { @@ -218,40 +222,51 @@ abstract class L2HellaCacheModule(implicit val p: Parameters) extends Module } abstract class L2HellaCacheBundle(implicit val p: Parameters) extends ParameterizedBundle()(p) - with HasL2HellaCacheParameters + with HasOuterCacheParameters trait HasL2Id extends HasCoherenceAgentParameters { val id = UInt(width = log2Up(nTransactors + 1)) } -trait HasL2InternalRequestState extends HasL2HellaCacheParameters { +trait HasL2InternalRequestState extends HasOuterCacheParameters { val tag_match = Bool() val meta = new L2Metadata val way_en = Bits(width = nWays) } -trait HasL2BeatAddr extends HasL2HellaCacheParameters { +trait HasL2BeatAddr extends HasOuterCacheParameters { val addr_beat = UInt(width = log2Up(refillCycles)) } -trait HasL2Data extends HasL2HellaCacheParameters +trait HasL2Data extends HasOuterCacheParameters with HasL2BeatAddr { val data = UInt(width = rowBits) def hasData(dummy: Int = 0) = Bool(true) def hasMultibeatData(dummy: Int = 0) = Bool(refillCycles > 1) } -class L2Metadata(implicit p: Parameters) extends Metadata()(p) with HasL2HellaCacheParameters { +class L2Metadata(implicit p: Parameters) extends Metadata()(p) with HasOuterCacheParameters { val coh = new HierarchicalMetadata } object L2Metadata { - def apply(tag: Bits, coh: HierarchicalMetadata)(implicit p: Parameters) = { + def apply(tag: Bits, coh: HierarchicalMetadata) + (implicit p: Parameters): L2Metadata = { val meta = Wire(new L2Metadata) meta.tag := tag meta.coh := coh meta } + + def apply( + tag: Bits, + inner: ManagerMetadata, + outer: ClientMetadata)(implicit p: Parameters): L2Metadata = { + val coh = Wire(new HierarchicalMetadata) + coh.inner := inner + coh.outer := outer + apply(tag, coh) + } } class L2MetaReadReq(implicit p: Parameters) extends MetaReadReq()(p) with HasL2Id { @@ -267,12 +282,12 @@ class L2MetaResp(implicit p: Parameters) extends L2HellaCacheBundle()(p) with HasL2Id with HasL2InternalRequestState -trait HasL2MetaReadIO extends HasL2HellaCacheParameters { +trait HasL2MetaReadIO extends HasOuterCacheParameters { val read = Decoupled(new L2MetaReadReq) val resp = Valid(new L2MetaResp).flip } -trait HasL2MetaWriteIO extends HasL2HellaCacheParameters { +trait HasL2MetaWriteIO extends HasOuterCacheParameters { val write = Decoupled(new L2MetaWriteReq) } @@ -280,6 +295,10 @@ class L2MetaRWIO(implicit p: Parameters) extends L2HellaCacheBundle()(p) with HasL2MetaReadIO with HasL2MetaWriteIO +trait HasL2MetaRWIO extends HasOuterCacheParameters { + val meta = new L2MetaRWIO +} + class L2MetadataArray(implicit p: Parameters) extends L2HellaCacheModule()(p) { val io = new L2MetaRWIO().flip @@ -330,21 +349,55 @@ class L2DataReadReq(implicit p: Parameters) extends L2HellaCacheBundle()(p) val way_en = Bits(width = nWays) } +object L2DataReadReq { + def apply( + id: UInt, + way_en: UInt, + addr_idx: UInt, + addr_beat: UInt)(implicit p: Parameters) = { + val req = Wire(new L2DataReadReq) + req.id := id + req.way_en := way_en + req.addr_idx := addr_idx + req.addr_beat := addr_beat + req + } +} + class L2DataWriteReq(implicit p: Parameters) extends L2DataReadReq()(p) with HasL2Data { val wmask = Bits(width = rowBits/8) } +object L2DataWriteReq { + def apply( + id: UInt, + way_en: UInt, + addr_idx: UInt, + addr_beat: UInt, + wmask: UInt, + data: UInt)(implicit p: Parameters) = { + val req = Wire(new L2DataWriteReq) + req.id := id + req.way_en := way_en + req.addr_idx := addr_idx + req.addr_beat := addr_beat + req.wmask := wmask + req.data := data + req + } +} + class L2DataResp(implicit p: Parameters) extends L2HellaCacheBundle()(p) with HasL2Id with HasL2Data -trait HasL2DataReadIO extends HasL2HellaCacheParameters { +trait HasL2DataReadIO extends HasOuterCacheParameters { val read = Decoupled(new L2DataReadReq) val resp = Valid(new L2DataResp).flip } -trait HasL2DataWriteIO extends HasL2HellaCacheParameters { +trait HasL2DataWriteIO extends HasOuterCacheParameters { val write = Decoupled(new L2DataWriteReq) } @@ -352,6 +405,10 @@ class L2DataRWIO(implicit p: Parameters) extends L2HellaCacheBundle()(p) with HasL2DataReadIO with HasL2DataWriteIO +trait HasL2DataRWIO extends HasOuterCacheParameters { + val data = new L2DataRWIO +} + class L2DataArray(delay: Int)(implicit p: Parameters) extends L2HellaCacheModule()(p) { val io = new L2DataRWIO().flip @@ -371,7 +428,7 @@ class L2DataArray(delay: Int)(implicit p: Parameters) extends L2HellaCacheModule } class L2HellaCacheBank(implicit p: Parameters) extends HierarchicalCoherenceAgent()(p) - with HasL2HellaCacheParameters { + with HasOuterCacheParameters { require(isPow2(nSets)) require(isPow2(nWays)) @@ -385,21 +442,22 @@ class L2HellaCacheBank(implicit p: Parameters) extends HierarchicalCoherenceAgen data.io <> tshrfile.io.data } -class TSHRFileIO(implicit p: Parameters) extends HierarchicalTLIO()(p) { - val meta = new L2MetaRWIO - val data = new L2DataRWIO -} +class TSHRFileIO(implicit p: Parameters) extends HierarchicalTLIO()(p) + with HasL2MetaRWIO + with HasL2DataRWIO class TSHRFile(implicit p: Parameters) extends L2HellaCacheModule()(p) with HasCoherenceAgentWiringHelpers { val io = new TSHRFileIO // Create TSHRs for outstanding transactions - val trackerList = + val irelTrackerList = (0 until nReleaseTransactors).map(id => - Module(new L2VoluntaryReleaseTracker(id))) ++ + Module(new CacheVoluntaryReleaseTracker(id))) + val iacqTrackerList = (nReleaseTransactors until nTransactors).map(id => - Module(new L2AcquireTracker(id))) + Module(new CacheAcquireTracker(id))) + val trackerList = irelTrackerList ++ iacqTrackerList // WritebackUnit evicts data from L2, including invalidating L1s val wb = Module(new L2WritebackUnit(nTransactors)) @@ -414,25 +472,23 @@ class TSHRFile(implicit p: Parameters) extends L2HellaCacheModule()(p) val irel_vs_iacq_conflict = io.inner.acquire.valid && io.inner.release.valid && - inSameSet(io.inner.acquire.bits.addr_block, io.inner.release.bits.addr_block) + inSameSet(io.inner.acquire.bits, io.inner.release.bits.addr_block) doInputRoutingWithAllocation( - io.inner.acquire, - trackerList.map(_.io.inner.acquire), - trackerList.map(_.io.matches.iacq), - trackerList.map(_.io.alloc.iacq), + in = io.inner.acquire, + outs = trackerList.map(_.io.inner.acquire), + allocs = trackerList.map(_.io.alloc_iacq), allocOverride = !irel_vs_iacq_conflict) - assert(PopCount(trackerList.map(_.io.alloc.iacq)) <= UInt(1), + assert(PopCount(trackerList.map(_.io.alloc_iacq.should)) <= UInt(1), "At most a single tracker should now be allocated for any given Acquire") // Wire releases from clients doInputRoutingWithAllocation( - io.inner.release, - trackerAndWbIOs.map(_.inner.release), - trackerAndWbIOs.map(_.matches.irel), - trackerAndWbIOs.map(_.alloc.irel)) + in = io.inner.release, + outs = trackerAndWbIOs.map(_.inner.release), + allocs = trackerAndWbIOs.map(_.alloc_irel)) - assert(PopCount(trackerAndWbIOs.map(_.alloc.irel)) <= UInt(1), + assert(PopCount(trackerAndWbIOs.map(_.alloc_irel.should)) <= UInt(1), "At most a single tracker should now be allocated for any given Release") // Wire probe requests and grant reply to clients, finish acks from clients @@ -457,21 +513,13 @@ class TSHRFile(implicit p: Parameters) extends L2HellaCacheModule()(p) } -class L2XactTrackerIO(implicit p: Parameters) extends HierarchicalXactTrackerIO()(p) { - val data = new L2DataRWIO - val meta = new L2MetaRWIO - val wb = new L2WritebackIO -} +class L2XactTrackerIO(implicit p: Parameters) extends HierarchicalXactTrackerIO()(p) + with HasL2DataRWIO + with HasL2MetaRWIO + with HasL2WritebackIO -abstract class L2XactTracker(implicit p: Parameters) extends XactTracker()(p) - with HasL2HellaCacheParameters { - class CacheBlockBuffer { // TODO - val buffer = Reg(Bits(width = p(CacheBlockBytes)*8)) - - def internal = Vec(internalDataBeats, Bits(width = rowBits)).fromBits(buffer) - def inner = Vec(innerDataBeats, Bits(width = innerDataBits)).fromBits(buffer) - def outer = Vec(outerDataBeats, Bits(width = outerDataBits)).fromBits(buffer) - } +trait HasRowBeatCounters extends HasOuterCacheParameters with HasPendingBitHelpers { + def mergeData(dataBits: Int)(beat: UInt, incoming: UInt): Unit def connectDataBeatCounter[S <: L2HellaCacheBundle](inc: Bool, data: S, beat: UInt, full_block: Bool) = { if(data.refillCycles > 1) { @@ -505,208 +553,115 @@ abstract class L2XactTracker(implicit p: Parameters) extends XactTracker()(p) def dropPendingBitInternal[T <: L2HellaCacheBundle with HasL2BeatAddr] (in: ValidIO[T]) = ~Fill(in.bits.refillCycles, in.valid) | ~UIntToOH(in.bits.addr_beat) - def addPendingBitWhenBeatHasPartialWritemask(in: DecoupledIO[AcquireFromSrc]): UInt = { - val a = in.bits - val isPartial = a.wmask() =/= Acquire.fullWriteMask - addPendingBitWhenBeat(in.fire() && (isPartial || Bool(ignoresWriteMask)), a) + // 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) } } } -class L2VoluntaryReleaseTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTracker()(p) { - val io = new L2XactTrackerIO - pinAllReadyValidLow(io) +trait ReadsFromOuterCacheDataArray extends HasCoherenceMetadataBuffer + with HasRowBeatCounters + with HasDataBuffer { + def io: HasL2DataRWIO - val s_idle :: s_meta_read :: s_meta_resp :: s_busy :: s_meta_write :: Nil = Enum(UInt(), 5) - val state = Reg(init=s_idle) + val pending_reads = Reg(init=Bits(0, width = innerDataBeats)) + val pending_resps = Reg(init=Bits(0, width = innerDataBeats)) + val curr_read_beat = PriorityEncoder(pending_reads) - val xact = Reg(new BufferedReleaseFromSrc()(p.alterPartial({case TLId => p(InnerTLId)}))) - val xact_way_en = Reg{ Bits(width = nWays) } - val xact_old_meta = Reg{ new L2Metadata } - val coh = xact_old_meta.coh + def readDataArray(drop_pending_bit: UInt, + add_pending_bit: UInt = UInt(0), + block_pending_read: Bool = Bool(false)) { + val port = io.data + pending_reads := (pending_reads & dropPendingBit(port.read) & drop_pending_bit) | add_pending_bit + port.read.valid := state === s_busy && pending_reads.orR && !block_pending_read + port.read.bits := L2DataReadReq( + id = UInt(trackerId), + way_en = xact_way_en, + addr_idx = xact_addr_idx, + addr_beat = curr_read_beat) - val pending_irel_beats = Reg(init=Bits(0, width = io.inner.tlDataBeats)) - val pending_writes = Reg(init=Bits(0, width = io.inner.tlDataBeats)) - val pending_ignt = Reg(init=Bool(false)) + pending_resps := (pending_resps & dropPendingBitInternal(port.resp)) | + addPendingBitInternal(port.read) - val all_pending_done = - !(pending_writes.orR || - pending_irel_beats.orR || - pending_ignt) + scoreboard += (pending_reads.orR, pending_resps.orR) - // These IOs are used for routing in the parent - io.matches.iacq := (state =/= s_idle) && inSameSet(io.iacq().addr_block, xact.addr_block) - io.matches.irel := (state =/= s_idle) && io.irel().conflicts(xact) - io.matches.oprb := (state =/= s_idle) && io.oprb().conflicts(xact) + mergeDataInternal(port.resp) + } +} - // Accept a voluntary Release (and any further beats of data) - pending_irel_beats := (pending_irel_beats & dropPendingBitWhenBeatHasData(io.inner.release)) - io.inner.release.ready := ((state === s_idle) && io.irel().isVoluntary()) || pending_irel_beats.orR - when(io.inner.release.fire()) { xact.data_buffer(io.irel().addr_beat) := io.irel().data } +trait WritesToOuterCacheDataArray extends HasCoherenceMetadataBuffer + with HasRowBeatCounters + with HasDataBuffer { + def io: HasL2DataRWIO - // 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(tagLSB) - - // Write the voluntarily written back data to this cache - pending_writes := (pending_writes & dropPendingBit(io.data.write)) | - addPendingBitWhenBeatHasData(io.inner.release) + val pending_writes = Reg(init=Bits(0, width = innerDataBeats)) val curr_write_beat = PriorityEncoder(pending_writes) - io.data.write.valid := state === s_busy && pending_writes.orR - 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 := ~UInt(0, io.data.write.bits.wmask.getWidth) - io.data.write.bits.data := xact.data_buffer(curr_write_beat) - // Send an acknowledgement - io.inner.grant.valid := state === s_busy && pending_ignt && !pending_irel_beats.orR - io.inner.grant.bits := coh.inner.makeGrant(xact) - when(io.inner.grant.fire()) { pending_ignt := Bool(false) } - - // 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.way_en := xact_way_en - io.meta.write.bits.data.tag := xact.addr_block >> UInt(tagLSB) - io.meta.write.bits.data.coh.inner := xact_old_meta.coh.inner.onRelease(xact) - io.meta.write.bits.data.coh.outer := Mux(xact.hasData(), - xact_old_meta.coh.outer.onHit(M_XWR), - xact_old_meta.coh.outer) - - // State machine updates and transaction handler metadata intialization - when(state === s_idle && io.inner.release.valid && io.alloc.irel) { - xact := io.irel() - pending_irel_beats := Mux(io.irel().hasMultibeatData(), - dropPendingBitWhenBeatHasData(io.inner.release), - UInt(0)) - pending_writes := addPendingBitWhenBeatHasData(io.inner.release) - pending_ignt := io.irel().requiresAck() - state := s_meta_read + def writeDataArray(add_pending_bit: UInt = UInt(0), + block_pending_write: Bool = Bool(false)) { + val port = io.data + pending_writes := (pending_writes & dropPendingBit(port.write)) | add_pending_bit + port.write.valid := state === s_busy && pending_writes.orR && !block_pending_write + port.write.bits := L2DataWriteReq( + id = UInt(trackerId), + way_en = xact_way_en, + addr_idx = xact_addr_idx, + addr_beat = curr_write_beat, + wmask = ~UInt(0, port.write.bits.wmask.getWidth), + data = data_buffer(curr_write_beat)) + + scoreboard += pending_writes.orR } - when(state === s_meta_read && io.meta.read.ready) { state := s_meta_resp } - when(state === s_meta_resp && io.meta.resp.valid) { - xact_old_meta := io.meta.resp.bits.meta - xact_way_en := io.meta.resp.bits.way_en - state := s_busy - } - when(state === s_busy && all_pending_done) { state := s_meta_write } - when(state === s_meta_write && io.meta.write.ready) { state := s_idle } - - // Checks for illegal behavior - assert(!(state === s_meta_resp && io.meta.resp.valid && !io.meta.resp.bits.tag_match), - "VoluntaryReleaseTracker accepted Release for a block not resident in this cache!") - assert(!(state === s_idle && io.inner.release.fire() && !io.irel().isVoluntary()), - "VoluntaryReleaseTracker accepted Release that wasn't voluntary!") } - -class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTracker()(p) { - val io = new L2XactTrackerIO - pinAllReadyValidLow(io) - - val s_idle :: s_meta_read :: s_meta_resp :: s_wb_req :: s_wb_resp :: s_inner_probe :: s_outer_acquire :: s_busy :: s_meta_write :: Nil = Enum(UInt(), 9) - val state = Reg(init=s_idle) - - // State holding transaction metadata - 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_bytes = Reg{ UInt() } - val xact_op_code = Reg{ UInt() } - val xact_addr_byte = Reg{ UInt() } - val xact_op_size = Reg{ UInt() } - val xact_vol_ir_r_type = Reg{ io.irel().r_type } - val xact_vol_ir_src = Reg{ io.irel().client_id } - val xact_vol_ir_client_xact_id = Reg{ io.irel().client_xact_id } - - // 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)) - - // 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(tagLSB) - val xact_vol_irel = Release( - src = xact_vol_ir_src, - voluntary = Bool(true), - r_type = xact_vol_ir_r_type, - client_xact_id = xact_vol_ir_client_xact_id, - addr_block = xact_addr_block) - (p.alterPartial({ case TLId => p(InnerTLId) })) - - // 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, - trackDown = (r: Release) => !r.isVoluntary())._1 - - val pending_vol_ignt = connectTwoWayBeatCounter( - max = 1, - up = io.inner.release, - down = io.inner.grant, - trackUp = (r: Release) => r.isVoluntary(), - trackDown = (g: Grant) => g.isVoluntary())._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( - 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, - trackUp = (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_irel_beats = 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 ignt_data_ready = Reg(init=Bits(0, width = io.inner.tlDataBeats)) - 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 || - pending_resps.orR || - pending_puts.orR || - pending_irel_beats.orR || - pending_ognt || - ignt_q.io.count > UInt(0) || - pending_vol_ignt || - //pending_meta_write || // Has own state: s_meta_write - pending_ifins) +trait HasAMOALU extends HasAcquireMetadataBuffer + with HasByteWriteMaskBuffer + with HasRowBeatCounters { + val io: L2XactTrackerIO // 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 := 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 := data_buffer.head // default, overwritten by calls to mergeData - val amo_result = Reg(init = UInt(0, xact.tlDataBits)) + val amo_result = Reg(init = UInt(0, innerDataBits)) + + def initializeAMOALUIOs() { + 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 := data_buffer.head // default, overwritten by calls to mergeData + } + + // Utility function for applying any buffered stored data to the cache line + // before storing it back into the data array + override def mergeData(dataBits: Int)(beat: UInt, incoming: UInt) { + val old_data = incoming // Refilled, written back, or de-cached data + val new_data = data_buffer(beat) // Newly Put data is already in the buffer + val amo_shift_bits = xact_amo_shift_bytes << 3 + amoalu.io.lhs := old_data >> amo_shift_bits + amoalu.io.rhs := new_data >> amo_shift_bits + val wmask = FillInterleaved(8, wmask_buffer(beat)) + data_buffer(beat) := ~wmask & old_data | + wmask & Mux(xact_iacq.isAtomic(), amoalu.io.out << amo_shift_bits, new_data) + when(xact_iacq.isAtomic() && xact_addr_beat === beat) { amo_result := old_data } + } +} + +trait HasCoherenceMetadataBuffer extends HasOuterCacheParameters + with HasBlockAddressBuffer + with HasXactTrackerStates { + def trackerId: Int + + val xact_way_en = Reg{ Bits(width = nWays) } + val xact_old_meta = Reg{ new L2Metadata } + val pending_coh = Reg{ xact_old_meta.coh } + val pending_meta_write = Reg{ Bool() } // pending_meta_write has own state (s_meta_write) + + val inner_coh = pending_coh.inner + val outer_coh = pending_coh.outer + + val xact_addr_idx = xact_addr_block(idxMSB,idxLSB) + val xact_addr_tag = xact_addr_block >> UInt(tagLSB) // Utility function for updating the metadata that will be kept in this cache def updatePendingCohWhen(flag: Bool, next: HierarchicalMetadata) { @@ -716,58 +671,140 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra } } - def addOtherBits(en: Bool, nBits: Int): UInt = - Mux(en, Cat(Fill(nBits - 1, UInt(1, 1)), UInt(0, 1)), UInt(0, nBits)) + def metaRead(port: HasL2MetaReadIO, next_state: UInt) { + port.read.valid := state === s_meta_read + port.read.bits.id := UInt(trackerId) + port.read.bits.idx := xact_addr_idx + port.read.bits.tag := xact_addr_tag - def addPendingBitsOnFirstBeat(in: DecoupledIO[Acquire]): UInt = - addOtherBits(in.fire() && - in.bits.hasMultibeatData() && - in.bits.addr_beat === UInt(0), - in.bits.tlDataBeats) + when(state === s_meta_read && port.read.ready) { state := s_meta_resp } - def dropPendingBitsOnFirstBeat(in: DecoupledIO[Acquire]): UInt = - ~addPendingBitsOnFirstBeat(in) + when(state === s_meta_resp && port.resp.valid) { + xact_old_meta := port.resp.bits.meta + xact_way_en := port.resp.bits.way_en + state := next_state + } + } + + def metaWrite(port: HasL2MetaWriteIO, to_write: L2Metadata, next_state: UInt) { + port.write.valid := state === s_meta_write + port.write.bits.id := UInt(trackerId) + port.write.bits.idx := xact_addr_idx + port.write.bits.way_en := xact_way_en + port.write.bits.data := to_write + + when(state === s_meta_write && port.write.ready) { state := next_state } + } +} + +trait TriggersWritebacks extends HasCoherenceMetadataBuffer { + def triggerWriteback(wb: L2WritebackIO, next_state: UInt) { + wb.req.valid := state === s_wb_req + wb.req.bits.id := UInt(trackerId) + wb.req.bits.idx := xact_addr_idx + wb.req.bits.tag := xact_old_meta.tag + wb.req.bits.coh := xact_old_meta.coh + wb.req.bits.way_en := xact_way_en + + when(state === s_wb_req && wb.req.ready) { state := s_wb_resp } + when(state === s_wb_resp && wb.resp.valid) { state := s_outer_acquire } + } +} + +class CacheVoluntaryReleaseTracker(trackerId: Int)(implicit p: Parameters) + extends VoluntaryReleaseTracker(trackerId)(p) + with HasDataBuffer + with WritesToOuterCacheDataArray { + val io = new L2XactTrackerIO + pinAllReadyValidLow(io) + + // Avoid metatdata races with writebacks + routeInParent(iacqMatches = inSameSet(_, xact_addr_block)) + io.alloc_iacq.can := Bool(false) + + // Initialize and accept pending Release beats + innerRelease( + block_vol_ignt = pending_writes.orR, + next = s_meta_read) + + io.inner.release.ready := state === s_idle || irel_can_merge || irel_same_xact + + // Begin a transaction by getting the current block metadata + metaRead(io.meta, s_busy) + + // Write the voluntarily written back data to this cache + writeDataArray(add_pending_bit = addPendingBitWhenBeatHasData(io.inner.release)) + + // End a transaction by updating the block metadata + metaWrite( + io.meta, + L2Metadata( + tag = xact_addr_tag, + inner = xact_old_meta.coh.inner.onRelease(xact_vol_irel), + outer = Mux(xact_vol_irel.hasData(), + xact_old_meta.coh.outer.onHit(M_XWR), + xact_old_meta.coh.outer)), + s_idle) + + when(io.inner.release.fire()) { data_buffer(io.irel().addr_beat) := io.irel().data } + + when(irel_is_allocating) { + pending_writes := addPendingBitWhenBeatHasData(io.inner.release) + } + + quiesce(s_meta_write) + + // Checks for illegal behavior + assert(!(state === s_meta_resp && io.meta.resp.valid && !io.meta.resp.bits.tag_match), + "VoluntaryReleaseTracker accepted Release for a block not resident in this cache!") +} + +class CacheAcquireTracker(trackerId: Int)(implicit p: Parameters) + extends AcquireTracker(trackerId)(p) + with HasByteWriteMaskBuffer + with HasAMOALU + with TriggersWritebacks + with ReadsFromOuterCacheDataArray + with WritesToOuterCacheDataArray { + val io = new L2XactTrackerIO + pinAllReadyValidLow(io) + initializeAMOALUIOs() + + + val pending_coh_on_ognt = HierarchicalMetadata( + ManagerMetadata.onReset, + pending_coh.outer.onGrant(io.outer.grant.bits, xact_op_code)) + + val pending_coh_on_ignt = HierarchicalMetadata( + pending_coh.inner.onGrant(io.ignt()), + Mux(ognt_counter.down.done, + pending_coh_on_ognt.outer, + pending_coh.outer)) + + val pending_coh_on_irel = HierarchicalMetadata( + pending_coh.inner.onRelease(io.irel()), // Drop sharer + Mux(io.irel().hasData(), // Dirty writeback + pending_coh.outer.onHit(M_XWR), + pending_coh.outer)) - // 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)) + 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 = data_buffer(beat) // Newly Put data is already in the buffer - amoalu.io.lhs := old_data >> (xact_amo_shift_bytes << 3) - amoalu.io.rhs := new_data >> (xact_amo_shift_bytes << 3) - val wmask = FillInterleaved(8, wmask_buffer(beat)) - data_buffer(beat) := ~wmask & old_data | - wmask & Mux(xact.isBuiltInType(Acquire.putAtomicType), - amoalu.io.out << (xact_amo_shift_bytes << 3), - new_data) - 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) } - } + // Setup IOs used for routing in the parent + val before_wb_alloc = Vec(s_meta_read, s_meta_resp, s_wb_req).contains(state) - 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) - } - } + routeInParent( + iacqMatches = inSameSet(_, xact_addr_block), + irelMatches = (irel: HasCacheBlockAddress) => + Mux(before_wb_alloc, inSameSet(irel, xact_addr_block), exactAddrMatch(irel))) + io.alloc_irel.can := Bool(false) - 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) - } - } - - // and Puts-under-Put, and either may also merge witha preceding prefetch + // TileLink allows for Gets-under-Get + // and Puts-under-Put, and either may also merge with a preceding prefetch // that requested the correct permissions (via op_code) def acquiresAreMergeable(sec: AcquireMetadata): Bool = { val allowedTypes = List((Acquire.getType, Acquire.getType), @@ -778,25 +815,15 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra (Acquire.getPrefetchType, Acquire.getType), (Acquire.putPrefetchType, Acquire.putType), (Acquire.putPrefetchType, Acquire.putBlockType)) - allowedTypes.map { case(a, b) => xact.isBuiltInType(a) && sec.isBuiltInType(b) }.reduce(_||_) && + allowedTypes.map { case(a, b) => xact_iacq.isBuiltInType(a) && sec.isBuiltInType(b) }.reduce(_||_) && xact_op_code === sec.op_code() && sec.conflicts(xact_addr_block) && xact_allocate } - // These IOs are used for routing in the parent - val iacq_in_same_set = inSameSet(xact_addr_block, io.iacq().addr_block) - val irel_in_same_set = inSameSet(xact_addr_block, io.irel().addr_block) - val before_wb_alloc = Vec(s_meta_read, s_meta_resp, s_wb_req).contains(state) - io.matches.iacq := (state =/= s_idle) && iacq_in_same_set - io.matches.irel := (state =/= s_idle) && - Mux(before_wb_alloc, irel_in_same_set, io.irel().conflicts(xact_addr_block)) - io.matches.oprb := Bool(false) //TODO - - // Actual transaction processing logic begins here: - // // First, take care of accpeting new acquires or secondary misses - val iacq_can_merge = acquiresAreMergeable(io.iacq()) && + // Handling of primary and secondary misses' data and write mask merging + def iacq_can_merge = acquiresAreMergeable(io.iacq()) && state =/= s_idle && state =/= s_meta_write && !all_pending_done && !io.inner.release.fire() && @@ -804,293 +831,147 @@ class L2AcquireTracker(trackerId: Int)(implicit p: Parameters) extends L2XactTra !io.data.resp.valid && ignt_q.io.enq.ready && ignt_q.io.deq.valid - val iacq_same_xact = xact.client_xact_id === io.iacq().client_xact_id && - xact.hasMultibeatData() && - ignt_q.io.deq.valid && // i.e. state =/= s_idle - pending_puts(io.iacq().addr_beat) - - val iacq_accepted = io.inner.acquire.fire() && - (io.alloc.iacq || iacq_can_merge || iacq_same_xact) + innerAcquire( + can_alloc = Bool(true), + next = s_meta_read) io.inner.acquire.ready := state === s_idle || iacq_can_merge || iacq_same_xact - // Handling of primary and secondary misses' data and write mask merging - when(iacq_accepted && 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 some metadata information that we'll use to make coherence updates with later - ignt_q.io.enq.valid := iacq_accepted && io.iacq().first() - ignt_q.io.enq.bits := io.iacq() - - // Track whether any beats are missing from a PutBlock - pending_puts := (pending_puts & - dropPendingBitWhenBeatHasData(io.inner.acquire)) | - addPendingBitsOnFirstBeat(io.inner.acquire) - // 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_idx - io.meta.read.bits.tag := xact_addr_tag + // Defined here because of Chisel default wire demands, used in s_meta_resp + 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) + else tag_match && coh.outer.isValid()) + val needs_writeback = !tag_match && + xact_allocate && + (coh.outer.requiresVoluntaryWriteback() || + coh.inner.requiresProbesOnVoluntaryWriteback()) + val needs_inner_probes = tag_match && coh.inner.requiresProbes(xact_iacq) + val should_update_meta = !tag_match && xact_allocate || + is_hit && pending_coh_on_hit =/= coh + def full_representation = io.meta.resp.bits.meta.coh.inner.full() + + metaRead( + io.meta, + Mux(needs_writeback, s_wb_req, + Mux(needs_inner_probes, s_inner_probe, + Mux(!is_hit, s_outer_acquire, s_busy)))) + + updatePendingCohWhen( + io.meta.resp.valid, + Mux(is_hit, pending_coh_on_hit, + Mux(tag_match, coh, pending_coh_on_miss))) // 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_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 + triggerWriteback(io.wb, s_outer_acquire) // Track which clients yet need to be probed and make Probe message - 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, xact_addr_block) + // If we're probing, we know the 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) + else xact_old_meta.coh.outer.isValid()) + + innerProbe( + inner_coh.makeProbe(curr_probe_dst, xact_iacq, xact_addr_block), + Mux(!skip_outer_acquire, s_outer_acquire, s_busy)) // Handle incoming releases from clients, which may reduce sharer counts // and/or write back dirty data, and may be unexpected voluntary releases - val irel_can_merge = io.irel().conflicts(xact_addr_block) && + + innerRelease() // Don't block on pending_writes because they won't happen until s_busy + + def irel_can_merge = io.irel().conflicts(xact_addr_block) && io.irel().isVoluntary() && !Vec(s_idle, s_meta_read, s_meta_resp, s_meta_write).contains(state) && !all_pending_done && !io.outer.grant.fire() && !io.inner.grant.fire() && - !pending_vol_ignt - - val irel_same_xact = io.irel().conflicts(xact_addr_block) && - !io.irel().isVoluntary() && - state === s_inner_probe - - val irel_accepted = io.inner.release.fire() && - (io.alloc.irel || irel_can_merge || irel_same_xact) + !vol_ignt_counter.pending io.inner.release.ready := irel_can_merge || irel_same_xact - val pending_coh_on_irel = HierarchicalMetadata( - pending_coh.inner.onRelease(io.irel()), // Drop sharer - Mux(io.irel().hasData(), // Dirty writeback - pending_coh.outer.onHit(M_XWR), - pending_coh.outer)) + updatePendingCohWhen(io.inner.release.fire(), pending_coh_on_irel) + mergeDataInner(io.inner.release) - when(io.inner.release.fire() && irel_can_merge) { - xact_vol_ir_r_type := io.irel().r_type - xact_vol_ir_src := io.irel().client_id - xact_vol_ir_client_xact_id := io.irel().client_xact_id - pending_irel_beats := Mux(io.irel().hasMultibeatData(), - dropPendingBitWhenBeatHasData(io.inner.release), - UInt(0)) - } - pending_irel_beats := (pending_irel_beats & dropPendingBitWhenBeatHasData(io.inner.release)) - - // 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_old_meta.coh.outer.makeAcquire( - op_code = xact_op_code, - client_xact_id = UInt(0), - addr_block = xact_addr_block), - BuiltInAcquireBuilder( - a_type = xact.a_type, - client_xact_id = UInt(0), - 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)}))) + // Send outer request + outerAcquire( + caching = xact_allocate, + coh = xact_old_meta.coh.outer, // TODO outer_coh? + data = data_buffer(ognt_counter.up.idx), + wmask = wmask_buffer(ognt_counter.up.idx), + next = s_busy) + // 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)) - updatePendingCohWhen(ognt_data_done, pending_coh_on_ognt) + updatePendingCohWhen(ognt_counter.down.done, pending_coh_on_ognt) mergeDataOuter(io.outer.grant) + // Send read request and get resp // 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)) | - (~ignt_data_ready & ( - 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) + readDataArray( + drop_pending_bit = (dropPendingBitWhenBeatHasData(io.inner.release) & + dropPendingBitWhenBeatHasData(io.outer.grant)), + add_pending_bit = addPendingBitWhenBeatNeedsRead(io.inner.acquire, Bool(alwaysWriteFullBeat)), + block_pending_read = ognt_counter.pending) + // Do write // 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) & - dropPendingBitsOnFirstBeat(io.inner.acquire)) | - 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_puts.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 := ~UInt(0, io.data.write.bits.wmask.getWidth) - io.data.write.bits.data := data_buffer(curr_write_beat) + writeDataArray( + add_pending_bit = (addPendingBitWhenBeatHasDataAndAllocs(io.inner.acquire) | + addPendingBitWhenBeatHasData(io.inner.release) | + addPendingBitWhenBeatHasData(io.outer.grant, xact_allocate)), + block_pending_write = (ognt_counter.pending || + pending_put_data.orR || + pending_reads(curr_write_beat) || + pending_resps(curr_write_beat))) - // soon as the data is released, granted, put, or read from the cache - 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 all data is - // received and committed to the data array or outer memory - val ignt_ack_ready = !(state === s_idle || - state === s_meta_read || - pending_puts.orR || - pending_writes.orR || - pending_ognt) + // Acknowledge or respond with data + innerGrant( + data = Mux(xact_iacq.isAtomic(), amo_result, data_buffer(ignt_data_idx)), + external_pending = pending_writes.orR || ognt_counter.pending, + add = addPendingBitInternal(io.data.resp)) - ignt_q.io.deq.ready := !pending_vol_ignt && ignt_data_done - io.inner.grant.valid := pending_vol_ignt || - (state === s_busy && - ignt_q.io.deq.valid && - Mux(io.ignt().hasData(), - ignt_data_ready(ignt_data_idx), - ignt_ack_ready)) - // Make the Grant message using the data stored in the secondary miss queue - val grant_from_acquire = pending_coh.inner.makeGrant( - sec = ignt_q.io.deq.bits, - manager_xact_id = UInt(trackerId), - data = Mux(xact.is(Acquire.putAtomicType), - amo_result, - data_buffer(ignt_data_idx))) - val grant_from_release = pending_coh.inner.makeGrant(xact_vol_irel) - io.inner.grant.bits := Mux(pending_vol_ignt, grant_from_release, grant_from_acquire) - io.inner.grant.bits.addr_beat := ignt_data_idx // override based on outgoing counter - - val pending_coh_on_ignt = HierarchicalMetadata( - pending_coh.inner.onGrant(io.ignt()), - Mux(ognt_data_done, - pending_coh_on_ognt.outer, - pending_coh.outer)) updatePendingCohWhen(io.inner.grant.fire() && io.ignt().last(), pending_coh_on_ignt) - // We must wait for as many Finishes as we sent Grants - io.inner.finish.ready := state === s_busy - // 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_idx - io.meta.write.bits.way_en := xact_way_en - io.meta.write.bits.data.tag := xact_addr_tag - io.meta.write.bits.data.coh := pending_coh - - // State machine updates and transaction handler metadata intialization - when(state === s_idle && io.inner.acquire.valid && io.alloc.iacq) { - xact_addr_block := io.iacq().addr_block - xact_allocate := io.iacq().allocate() - xact_amo_shift_bytes := io.iacq().amo_shift_bytes() - 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), - dropPendingBitWhenBeatHasData(io.inner.acquire), - UInt(0)) - pending_reads := Mux( // GetBlocks and custom types read all beats - io.iacq().isBuiltInType(Acquire.getBlockType) || !io.iacq().isBuiltInType(), - SInt(-1), - (addPendingBitWhenBeatIsGetOrAtomic(io.inner.acquire) | - addPendingBitWhenBeatHasPartialWritemask(io.inner.acquire)).toSInt).toUInt - pending_writes := addPendingBitWhenBeatHasDataAndAllocs(io.inner.acquire) - pending_resps := UInt(0) - ignt_data_ready := UInt(0) - pending_meta_write := Bool(false) - state := s_meta_read - } - when(state === s_meta_read && io.meta.read.ready) { state := s_meta_resp } + metaWrite(io.meta, L2Metadata(xact_addr_tag, pending_coh), s_idle) + + // Initialization of some scoreboard logic based on the original + // Acquire message on on the results of the metadata read: when(state === s_meta_resp && io.meta.resp.valid) { - xact_tag_match := io.meta.resp.bits.tag_match - xact_old_meta := io.meta.resp.bits.meta - 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) - else tag_match && coh.outer.isValid()) - val needs_writeback = !tag_match && - 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 || - is_hit && pending_coh_on_hit =/= coh - // Determine any changes to the coherence metadata - when (should_update_meta) { pending_meta_write := Bool(true) } - pending_coh := Mux(is_hit, pending_coh_on_hit, Mux(tag_match, coh, pending_coh_on_miss)) - // If we need to probe some clients, make a bitmask identifying them - when (needs_inner_probes) { - val full_sharers = coh.inner.full() - val mask_self = Mux( - xact.requiresSelfProbe(), - coh.inner.full() | UIntToOH(xact.client_id), - coh.inner.full() & ~UIntToOH(xact.client_id)) - val mask_incoherent = mask_self & ~io.incoherent.toBits - pending_iprbs := mask_incoherent - } // 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) + when(is_hit && !xact_allocate && xact_iacq.hasData()) { + pending_writes := addPendingBitsFromAcquire(xact_iacq) xact_allocate := Bool(true) } - // Next: request writeback, issue probes, query outer memory, or respond - state := Mux(needs_writeback, s_wb_req, - Mux(needs_inner_probes, s_inner_probe, - Mux(!is_hit, s_outer_acquire, s_busy))) + when (needs_inner_probes) { initializeProbes() } + pending_meta_write := should_update_meta //TODO what edge case was this covering? } - when(state === s_wb_req && io.wb.req.ready) { state := s_wb_resp } - when(state === s_wb_resp && io.wb.resp.valid) { state := s_outer_acquire } - 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) - else 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) { + + // Initialize more transaction metadata. Pla + when(iacq_is_allocating) { 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) + amo_result := UInt(0) + pending_meta_write := Bool(false) + pending_reads := Mux( // Pick out the specific beats of data that need to be read + io.iacq().isBuiltInType(Acquire.getBlockType) || !io.iacq().isBuiltInType(), + ~UInt(0, width = innerDataBeats), + addPendingBitWhenBeatNeedsRead(io.inner.acquire, Bool(alwaysWriteFullBeat))) + pending_writes := addPendingBitWhenBeatHasDataAndAllocs(io.inner.acquire) + pending_resps := UInt(0) } - when(state === s_meta_write && io.meta.write.ready) { state := s_idle } + + initDataInner(io.inner.acquire) + + // Wait for everything to quiesce + quiesce(Mux(pending_meta_write, s_meta_write, s_idle)) } class L2WritebackReq(implicit p: Parameters) extends L2Metadata()(p) with HasL2Id { @@ -1105,161 +986,89 @@ class L2WritebackIO(implicit p: Parameters) extends L2HellaCacheBundle()(p) { val resp = Valid(new L2WritebackResp).flip } -class L2WritebackUnitIO(implicit p: Parameters) extends HierarchicalXactTrackerIO()(p) { - val wb = new L2WritebackIO().flip - val data = new L2DataRWIO +trait HasL2WritebackIO extends HasOuterCacheParameters { + val wb = new L2WritebackIO() } -class L2WritebackUnit(trackerId: Int)(implicit p: Parameters) extends L2XactTracker()(p) { +class L2WritebackUnitIO(implicit p: Parameters) extends HierarchicalXactTrackerIO()(p) + with HasL2DataRWIO { + val wb = new L2WritebackIO().flip() +} + +class L2WritebackUnit(val trackerId: Int)(implicit p: Parameters) extends XactTracker()(p) + with AcceptsVoluntaryReleases + with EmitsVoluntaryReleases + with EmitsInnerProbes + with ReadsFromOuterCacheDataArray + with RoutesInParent { val io = new L2WritebackUnitIO pinAllReadyValidLow(io) - val s_idle :: s_inner_probe :: s_busy :: s_outer_grant :: s_wb_resp :: Nil = Enum(UInt(), 5) - val state = Reg(init=s_idle) + val xact_id = Reg{ io.wb.req.bits.id } - val xact = Reg(new L2WritebackReq) - val data_buffer = Reg(init=Vec.fill(innerDataBeats)(UInt(0, width = innerDataBits))) - val xact_vol_ir_r_type = Reg{ io.irel().r_type } - val xact_vol_ir_src = Reg{ io.irel().client_id } - val xact_vol_ir_client_xact_id = Reg{ io.irel().client_xact_id } + val pending_coh_on_irel = HierarchicalMetadata( + inner_coh.onRelease(io.irel()), // Drop sharer + Mux(io.irel().hasData(), // Dirty writeback + outer_coh.onHit(M_XWR), + outer_coh)) - val xact_addr_block = if (cacheIdBits == 0) - Cat(xact.tag, xact.idx) - else - Cat(xact.tag, xact.idx, UInt(cacheId, cacheIdBits)) - val xact_vol_irel = Release( - src = xact_vol_ir_src, - voluntary = Bool(true), - r_type = xact_vol_ir_r_type, - client_xact_id = xact_vol_ir_client_xact_id, - addr_block = xact_addr_block) - - val pending_irels = connectTwoWayBeatCounter( - max = io.inner.tlNCachingClients, - up = io.inner.probe, - down = io.inner.release, - trackDown = (r: Release) => !r.isVoluntary())._1 - - val pending_vol_ignt = connectTwoWayBeatCounter( - max = 1, - up = io.inner.release, - down = io.inner.grant, - trackUp = (r: Release) => r.isVoluntary(), - trackDown = (g: Grant) => g.isVoluntary())._1 - - val (pending_ognt, orel_data_idx, orel_data_done, ognt_data_idx, ognt_data_done) = - connectTwoWayBeatCounter( - max = 1, - up = io.outer.release, - down = io.outer.grant) - - val pending_iprbs = Reg(init = Bits(0, width = io.inner.tlNCachingClients)) - val pending_reads = Reg(init=Bits(0, width = io.inner.tlDataBeats)) - val pending_resps = Reg(init=Bits(0, width = io.inner.tlDataBeats)) - val pending_orel_data = Reg(init=Bits(0, width = io.inner.tlDataBeats)) - - // These IOs are used for routing in the parent - io.matches.iacq := (state =/= s_idle) && io.iacq().conflicts(xact_addr_block) - io.matches.irel := (state =/= s_idle) && io.irel().conflicts(xact_addr_block) - io.matches.oprb := (state =/= s_idle) && io.oprb().conflicts(xact_addr_block) + routeInParent() // Start the writeback sub-transaction io.wb.req.ready := state === s_idle // Track which clients yet need to be probed and make Probe message - 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 := xact.coh.inner.makeProbeForVoluntaryWriteback(curr_probe_dst, xact_addr_block) + innerProbe( + inner_coh.makeProbeForVoluntaryWriteback(curr_probe_dst, xact_addr_block), + s_busy) // Handle incoming releases from clients, which may reduce sharer counts // and/or write back dirty data - val irel_can_merge = io.irel().conflicts(xact_addr_block) && + innerRelease() + + def irel_can_merge = io.irel().conflicts(xact_addr_block) && io.irel().isVoluntary() && - state === s_inner_probe && - !pending_vol_ignt - - val irel_same_xact = io.irel().conflicts(xact_addr_block) && - !io.irel().isVoluntary() && - state === s_inner_probe - - val irel_accepted = io.inner.release.fire() && - (io.alloc.irel || irel_can_merge || irel_same_xact) + (state =/= s_idle) && + !(state === s_busy && all_pending_done) && + !vol_ignt_counter.pending io.inner.release.ready := irel_can_merge || irel_same_xact - val pending_coh_on_irel = HierarchicalMetadata( - xact.coh.inner.onRelease(io.irel()), // Drop sharer - Mux(io.irel().hasData(), // Dirty writeback - xact.coh.outer.onHit(M_XWR), - xact.coh.outer)) - when(io.inner.release.fire()) { - xact.coh := pending_coh_on_irel - when(io.irel().hasData()) { data_buffer(io.irel().addr_beat) := io.irel().data } - when(irel_can_merge) { - xact_vol_ir_r_type := io.irel().r_type - xact_vol_ir_src := io.irel().client_id - xact_vol_ir_client_xact_id := io.irel().client_xact_id - } - } + + updatePendingCohWhen(io.inner.release.fire(), pending_coh_on_irel) + + mergeDataInner(io.inner.release) // If a release didn't write back data, have to read it from data array - pending_reads := (pending_reads & - dropPendingBit(io.data.read) & - dropPendingBitWhenBeatHasData(io.inner.release)) - val curr_read_beat = PriorityEncoder(pending_reads) - io.data.read.valid := state === s_busy && pending_reads.orR - io.data.read.bits.id := UInt(trackerId) - io.data.read.bits.way_en := xact.way_en - io.data.read.bits.addr_idx := xact.idx - io.data.read.bits.addr_beat := curr_read_beat - io.data.write.valid := Bool(false) - - pending_resps := (pending_resps & dropPendingBitInternal(io.data.resp)) | - addPendingBitInternal(io.data.read) - when(io.data.resp.valid) { - data_buffer(io.data.resp.bits.addr_beat) := io.data.resp.bits.data - } + readDataArray(drop_pending_bit = dropPendingBitWhenBeatHasData(io.inner.release)) // Once the data is buffered we can write it back to outer memory - pending_orel_data := pending_orel_data | - addPendingBitWhenBeatHasData(io.inner.release) | - addPendingBitInternal(io.data.resp) - io.outer.release.valid := state === s_busy && - (!io.orel().hasData() || pending_orel_data(orel_data_idx)) - io.outer.release.bits := xact.coh.outer.makeVoluntaryWriteback( - client_xact_id = UInt(trackerId), - addr_block = xact_addr_block, - addr_beat = orel_data_idx, - data = data_buffer(orel_data_idx)) - - // Ack a voluntary release if we got one - io.inner.grant.valid := pending_vol_ignt - io.inner.grant.bits := xact.coh.inner.makeGrant(xact_vol_irel) - - // Wait for an acknowledgement - io.outer.grant.ready := state === s_outer_grant + outerRelease( + coh = outer_coh, + data = data_buffer(vol_ognt_counter.up.idx), + add_pending_bit = addPendingBitInternal(io.data.resp)) // Respond to the initiating transaction handler signalling completion of the writeback - io.wb.resp.valid := state === s_wb_resp - io.wb.resp.bits.id := xact.id + io.wb.resp.valid := state === s_busy && all_pending_done + io.wb.resp.bits.id := xact_id + quiesce() + + def full_representation = io.wb.req.bits.coh.inner.full() // State machine updates and transaction handler metadata intialization when(state === s_idle && io.wb.req.valid) { - xact := io.wb.req.bits + xact_id := io.wb.req.bits.id + xact_way_en := io.wb.req.bits.way_en + xact_addr_block := (if (cacheIdBits == 0) Cat(io.wb.req.bits.tag, io.wb.req.bits.idx) + else Cat(io.wb.req.bits.tag, io.wb.req.bits.idx, UInt(cacheId, cacheIdBits))) val coh = io.wb.req.bits.coh val needs_inner_probes = coh.inner.requiresProbesOnVoluntaryWriteback() - when(needs_inner_probes) { pending_iprbs := coh.inner.full() & ~io.incoherent.toBits } - pending_reads := ~UInt(0, width = innerDataBeats) + val needs_outer_release = coh.outer.requiresVoluntaryWriteback() + when(needs_inner_probes) { initializeProbes() } + pending_reads := Mux(needs_outer_release, ~UInt(0, width = innerDataBeats), UInt(0)) pending_resps := UInt(0) - pending_orel_data := UInt(0) + pending_orel := needs_outer_release + //pending_orel_data := UInt(0) + pending_coh := coh state := Mux(needs_inner_probes, s_inner_probe, s_busy) } - when(state === s_inner_probe && !(pending_iprbs.orR || pending_irels || pending_vol_ignt)) { - state := Mux(xact.coh.outer.requiresVoluntaryWriteback(), s_busy, s_wb_resp) - } - when(state === s_busy && orel_data_done) { - state := Mux(io.orel().requiresAck(), s_outer_grant, s_wb_resp) - } - when(state === s_outer_grant && ognt_data_done) { state := s_wb_resp } - when(state === s_wb_resp ) { state := s_idle } } diff --git a/uncore/src/main/scala/converters.scala b/uncore/src/main/scala/converters.scala index 569f4278..a76dc287 100644 --- a/uncore/src/main/scala/converters.scala +++ b/uncore/src/main/scala/converters.scala @@ -227,6 +227,17 @@ object ManagerTileLinkHeaderCreator { } } +class BeatCounterStatus extends Bundle { + val idx = UInt() + val done = Bool() +} + +class TwoWayBeatCounterStatus extends Bundle { + val pending = Bool() + val up = new BeatCounterStatus() + val down = new BeatCounterStatus() +} + /** Utility trait containing wiring functions to keep track of how many data beats have * been sent or recieved over a particular [[uncore.TileLinkChannel]] or pair of channels. * @@ -283,27 +294,31 @@ trait HasDataBeatCounters { /** Provides counters on two channels, as well a meta-counter that tracks how many * messages have been sent over the up channel but not yet responded to over the down channel * - * @param max max number of outstanding ups with no down + * @param status bundle of status of the counters * @param up outgoing channel * @param down incoming channel + * @param max max number of outstanding ups with no down * @param beat overrides cnts on single-beat messages * @param track whether up's message should be tracked * @return a tuple containing whether their are outstanding messages, up's count, * up's done, down's count, down's done */ - def connectTwoWayBeatCounter[T <: TileLinkChannel, S <: TileLinkChannel]( - max: Int, + def connectTwoWayBeatCounters[T <: TileLinkChannel, S <: TileLinkChannel]( + status: TwoWayBeatCounterStatus, up: DecoupledIO[T], down: DecoupledIO[S], + max: Int = 1, beat: UInt = UInt(0), trackUp: T => Bool = (t: T) => Bool(true), - trackDown: S => Bool = (s: S) => Bool(true)): (Bool, UInt, Bool, UInt, Bool) = { - val (up_idx, up_done) = connectDataBeatCounter(up.fire(), up.bits, beat) - val (down_idx, down_done) = connectDataBeatCounter(down.fire(), down.bits, beat) - val do_inc = up_done && trackUp(up.bits) - val do_dec = down_done && trackDown(down.bits) - val cnt = TwoWayCounter(do_inc, do_dec, max) - (cnt > UInt(0), up_idx, up_done, down_idx, down_done) + trackDown: S => Bool = (s: S) => Bool(true)) { + val (up_idx, up_done) = connectDataBeatCounter(up.fire() && trackUp(up.bits), up.bits, beat) + val (dn_idx, dn_done) = connectDataBeatCounter(down.fire() && trackDown(down.bits), down.bits, beat) + val cnt = TwoWayCounter(up_done, dn_done, max) + status.pending := cnt > UInt(0) + status.up.idx := up_idx + status.up.done := up_done + status.down.idx := dn_idx + status.down.done := dn_done } } @@ -391,9 +406,10 @@ class ClientTileLinkIOUnwrapper(implicit p: Parameters) extends TLModule()(p) { addr_beat = ognt.addr_beat, data = ognt.data) + assert(!io.in.release.valid || io.in.release.bits.isVoluntary(), "Unwrapper can only process voluntary releases.") val rel_grant = Grant( is_builtin_type = Bool(true), - g_type = Mux(gnt_voluntary, Grant.voluntaryAckType, ognt.g_type), + g_type = Grant.voluntaryAckType, // We should only every be working with voluntary releases client_xact_id = ognt.client_xact_id, manager_xact_id = ognt.manager_xact_id, addr_beat = ognt.addr_beat, diff --git a/uncore/src/main/scala/metadata.scala b/uncore/src/main/scala/metadata.scala index ede9e397..38c51428 100644 --- a/uncore/src/main/scala/metadata.scala +++ b/uncore/src/main/scala/metadata.scala @@ -47,13 +47,14 @@ class ClientMetadata(implicit p: Parameters) extends CoherenceMetadata()(p) { def makeAcquire( op_code: UInt, client_xact_id: UInt, - addr_block: UInt): Acquire = + addr_block: UInt): Acquire = { Acquire( is_builtin_type = Bool(false), a_type = co.getAcquireType(op_code, this), client_xact_id = client_xact_id, addr_block = addr_block, union = Cat(op_code, Bool(true)))(p) + } /** Constructs a Release message based on this metadata on cache control op * diff --git a/uncore/src/main/scala/sdq.scala b/uncore/src/main/scala/sdq.scala new file mode 100644 index 00000000..00790d56 --- /dev/null +++ b/uncore/src/main/scala/sdq.scala @@ -0,0 +1,118 @@ +// See LICENSE for license details. + +package uncore +import Chisel._ +import cde.{Parameters, Field} + +case object L2StoreDataQueueDepth extends Field[Int] + +trait HasStoreDataQueueParameters extends HasCoherenceAgentParameters { + val sdqDepth = p(L2StoreDataQueueDepth)*innerDataBeats + val dqIdxBits = math.max(log2Up(nReleaseTransactors) + 1, log2Up(sdqDepth)) + val nDataQueueLocations = 3 //Stores, VoluntaryWBs, Releases +} + +class DataQueueLocation(implicit p: Parameters) extends CoherenceAgentBundle()(p) + with HasStoreDataQueueParameters { + val idx = UInt(width = dqIdxBits) + val loc = UInt(width = log2Ceil(nDataQueueLocations)) +} + +object DataQueueLocation { + def apply(idx: UInt, loc: UInt)(implicit p: Parameters) = { + val d = Wire(new DataQueueLocation) + d.idx := idx + d.loc := loc + d + } +} + +trait HasStoreDataQueue extends HasStoreDataQueueParameters { + val io: HierarchicalTLIO + val trackerIOsList: Seq[HierarchicalXactTrackerIO] + + val internalDataBits = new DataQueueLocation().getWidth + val inStoreQueue :: inVolWBQueue :: inClientReleaseQueue :: Nil = Enum(UInt(), nDataQueueLocations) + + val usingStoreDataQueue = p.alterPartial({ + case TLKey(`innerTLId`) => innerTLParams.copy(overrideDataBitsPerBeat = Some(internalDataBits)) + case TLKey(`outerTLId`) => outerTLParams.copy(overrideDataBitsPerBeat = Some(internalDataBits)) + }) + + // Queue to store impending Put data + lazy val sdq = Reg(Vec(sdqDepth, io.iacq().data)) + lazy val sdq_val = Reg(init=Bits(0, sdqDepth)) + lazy val sdq_alloc_id = PriorityEncoder(~sdq_val) + lazy val sdq_rdy = !sdq_val.andR + lazy val sdq_enq = trackerIOsList.map( t => + (t.alloc_iacq.should || t.alloc_iacq.matches) && + t.inner.acquire.fire() && + t.iacq().hasData() + ).reduce(_||_) + + lazy val sdqLoc = List.fill(nTransactors) { + DataQueueLocation(sdq_alloc_id, inStoreQueue).toBits + } + + /* + doInputRoutingWithAllocation( + in = io.inner.acquire, + outs = trackerList.map(_.io.inner.acquire), + allocs = trackerList.map(_.io.alloc._iacq), + dataOverride = Some(sdqLoc), + allocOverride = Some(sdq_rdy && !irel_vs_iacq_conflict)) + */ + + // Queue to store impending Voluntary Release data + lazy val voluntary = io.irel().isVoluntary() + lazy val vwbdq_enq = io.inner.release.fire() && voluntary && io.irel().hasData() + lazy val (rel_data_cnt, rel_data_done) = Counter(vwbdq_enq, innerDataBeats) //TODO Zero width + lazy val vwbdq = Reg(Vec(innerDataBeats, io.irel().data)) //TODO Assumes nReleaseTransactors == 1 + + + lazy val vwbqLoc = (0 until nTransactors).map(i => + (DataQueueLocation(rel_data_cnt, + (if(i < nReleaseTransactors) inVolWBQueue + else inClientReleaseQueue)).toBits)) + /* + doInputRoutingWithAllocation( + io.inner.release, + trackerList.map(_.io.inner.release), + trackerList.map(_.io.matches.irel), + trackerList.map(_.io.alloc.irel), + Some(vwbqLoc)) + */ + + val outer_arb: ClientTileLinkIOArbiter + lazy val outer_data_ptr = new DataQueueLocation().fromBits(outer_arb.io.out.acquire.bits.data) + /* + val outer_arb = Module(new ClientTileLinkIOArbiter(trackerList.size) + (usingStoreDataQueue.alterPartial({ case TLId => p(OuterTLId) }))) + outer_arb.io.in <> trackerList + */ + // Get the pending data out of the store data queue + lazy val is_in_sdq = outer_data_ptr.loc === inStoreQueue + lazy val free_sdq = io.outer.acquire.fire() && + io.outer.acquire.bits.hasData() && + outer_data_ptr.loc === inStoreQueue + /* + io.outer <> outer_arb.io.out + io.outer.acquire.bits.data := MuxLookup(outer_data_ptr.loc, io.irel().data, Array( + inStoreQueue -> sdq(outer_data_ptr.idx), + inVolWBQueue -> vwbdq(outer_data_ptr.idx))) + */ + + // Enqueue SDQ data + def sdqEnqueue() { + when (sdq_enq) { sdq(sdq_alloc_id) := io.iacq().data } + when(vwbdq_enq) { vwbdq(rel_data_cnt) := io.irel().data } + } + + // Update SDQ valid bits + def sdqUpdate() { + when (io.outer.acquire.valid || sdq_enq) { + sdq_val := sdq_val & ~(UIntToOH(outer_data_ptr.idx) & Fill(sdqDepth, free_sdq)) | + PriorityEncoderOH(~sdq_val(sdqDepth-1,0)) & Fill(sdqDepth, sdq_enq) + } + } +} diff --git a/uncore/src/main/scala/tilelink.scala b/uncore/src/main/scala/tilelink.scala index 95e1ebf2..5599217b 100644 --- a/uncore/src/main/scala/tilelink.scala +++ b/uncore/src/main/scala/tilelink.scala @@ -132,8 +132,8 @@ trait HasTileLinkData extends HasTileLinkBeatId { def hasData(dummy: Int = 0): Bool def hasMultibeatData(dummy: Int = 0): Bool - def first(dummy: Int = 0): Bool = Mux(hasMultibeatData(), addr_beat === UInt(0), Bool(true)) - def last(dummy: Int = 0): Bool = Mux(hasMultibeatData(), addr_beat === UInt(tlDataBeats-1), Bool(true)) + def first(dummy: Int = 0): Bool = !hasMultibeatData() || addr_beat === UInt(0) + def last(dummy: Int = 0): Bool = !hasMultibeatData() || addr_beat === UInt(tlDataBeats-1) } /** An entire cache block of data */ @@ -186,6 +186,10 @@ trait HasAcquireUnion extends HasTileLinkParameters { } /** Full, beat-sized writemask */ def full_wmask(dummy: Int = 0) = FillInterleaved(8, wmask()) + + /** Is this message a built-in read message */ + def hasPartialWritemask(dummy: Int = 0): Bool = wmask() =/= Acquire.fullWriteMask + } trait HasAcquireType extends HasTileLinkParameters { @@ -207,6 +211,12 @@ trait HasAcquireType extends HasTileLinkParameters { def isPrefetch(dummy: Int = 0): Bool = isBuiltInType() && (is(Acquire.getPrefetchType) || is(Acquire.putPrefetchType)) + /** Is this message a built-in atomic message */ + def isAtomic(dummy: Int = 0): Bool = isBuiltInType() && is(Acquire.putAtomicType) + + /** Is this message a built-in read message */ + def isGet(dummy: Int = 0): Bool = isBuiltInType() && (is(Acquire.getType) || is(Acquire.getBlockType)) + /** Does this message contain data? Assumes that no custom message types have data. */ def hasData(dummy: Int = 0): Bool = isBuiltInType() && Acquire.typesWithData.contains(a_type) @@ -214,11 +224,6 @@ trait HasAcquireType extends HasTileLinkParameters { def hasMultibeatData(dummy: Int = 0): Bool = Bool(tlDataBeats > 1) && isBuiltInType() && Acquire.typesWithMultibeatData.contains(a_type) - /** Does this message require the manager to probe the client the very client that sent it? - * Needed if multiple caches are attached to the same port. - */ - def requiresSelfProbe(dummy: Int = 0) = Bool(false) - /** Mapping between each built-in Acquire type and a built-in Grant type. */ def getBuiltInGrantType(dummy: Int = 0): UInt = Acquire.getBuiltInGrantType(this.a_type) } diff --git a/uncore/src/main/scala/trackers.scala b/uncore/src/main/scala/trackers.scala new file mode 100644 index 00000000..87d7b0b9 --- /dev/null +++ b/uncore/src/main/scala/trackers.scala @@ -0,0 +1,546 @@ +// See LICENSE for license details. + +package uncore +import Chisel._ +import cde.{Parameters, Field} + +class TrackerAllocation extends Bundle { + val matches = Bool(OUTPUT) + val can = Bool(OUTPUT) + val should = Bool(INPUT) +} + +trait HasTrackerAllocationIO extends Bundle { + val alloc_iacq = new TrackerAllocation + val alloc_irel = new TrackerAllocation + val alloc_oprb = new TrackerAllocation +} + +class ManagerXactTrackerIO(implicit p: Parameters) extends ManagerTLIO()(p) + with HasTrackerAllocationIO + +class HierarchicalXactTrackerIO(implicit p: Parameters) extends HierarchicalTLIO()(p) + with HasTrackerAllocationIO + +abstract class XactTracker(implicit p: Parameters) extends CoherenceAgentModule()(p) + with HasXactTrackerStates + with HasPendingBitHelpers { + override val s_idle :: s_meta_read :: s_meta_resp :: s_wb_req :: s_wb_resp :: s_inner_probe :: s_outer_acquire :: s_busy :: s_meta_write :: Nil = Enum(UInt(), 9) + val state = Reg(init=s_idle) + + def quiesce(next: UInt = s_idle) { + all_pending_done := !scoreboard.foldLeft(Bool(false))(_||_) + when(state === s_busy && all_pending_done) { state := next } + } + + def pinAllReadyValidLow[T <: Data](b: Bundle) { + b.elements.foreach { + _._2 match { + case d: DecoupledIO[_] => + if(d.ready.dir == OUTPUT) d.ready := Bool(false) + else if(d.valid.dir == OUTPUT) d.valid := Bool(false) + case v: ValidIO[_] => if(v.valid.dir == OUTPUT) v.valid := Bool(false) + case b: Bundle => pinAllReadyValidLow(b) + case _ => + } + } + } +} + +trait HasXactTrackerStates { + def state: UInt + def s_idle: UInt = UInt(0) + def s_meta_read: UInt + def s_meta_resp: UInt + def s_wb_req: UInt + def s_wb_resp: UInt + def s_inner_probe: UInt + def s_outer_acquire: UInt + def s_busy: UInt + def s_meta_write: UInt +} + +trait HasPendingBitHelpers extends HasDataBeatCounters { + val scoreboard = scala.collection.mutable.ListBuffer.empty[Bool] + val all_pending_done = Wire(Bool()) + + def addPendingBitWhenBeat[T <: HasBeat](inc: Bool, in: T): UInt = + Fill(in.tlDataBeats, inc) & UIntToOH(in.addr_beat) + + def dropPendingBitWhenBeat[T <: HasBeat](dec: Bool, in: T): UInt = + ~Fill(in.tlDataBeats, dec) | ~UIntToOH(in.addr_beat) + + def addPendingBitWhenId[T <: HasClientId](inc: Bool, in: T): UInt = + Fill(in.tlNCachingClients, inc) & UIntToOH(in.client_id) + + def dropPendingBitWhenId[T <: HasClientId](dec: Bool, in: T): UInt = + ~Fill(in.tlNCachingClients, dec) | ~UIntToOH(in.client_id) + + 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 = + addPendingBitWhenBeatHasData(in, in.bits.allocate()) + + def addPendingBitWhenBeatNeedsRead(in: DecoupledIO[AcquireFromSrc], inc: Bool = Bool(true)): UInt = { + val a = in.bits + val needs_read = (a.isGet() || a.isAtomic() || a.hasPartialWritemask()) || inc + addPendingBitWhenBeat(in.fire() && needs_read, a) + } + + def addPendingBitWhenBeatHasPartialWritemask(in: DecoupledIO[AcquireFromSrc]): UInt = + addPendingBitWhenBeat(in.fire() && in.bits.hasPartialWritemask(), in.bits) + + 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) + + def dropPendingBitAtDest[T <: HasId](in: DecoupledIO[T]): UInt = + dropPendingBitWhenId(in.fire(), in.bits) + + def dropPendingBitAtDestWhenVoluntary[T <: HasId with MightBeVoluntary](in: DecoupledIO[T]): UInt = + dropPendingBitWhenId(in.fire() && in.bits.isVoluntary(), in.bits) + + def addPendingBitAtSrc[T <: HasId](in: DecoupledIO[T]): UInt = + addPendingBitWhenId(in.fire(), in.bits) + + def addPendingBitAtSrcWhenVoluntary[T <: HasId with MightBeVoluntary](in: DecoupledIO[T]): UInt = + addPendingBitWhenId(in.fire() && in.bits.isVoluntary(), in.bits) + + def addOtherBits(en: Bool, nBits: Int): UInt = + Mux(en, Cat(Fill(nBits - 1, UInt(1, 1)), UInt(0, 1)), UInt(0, nBits)) + + def addPendingBitsOnFirstBeat(in: DecoupledIO[Acquire]): UInt = + addOtherBits(in.fire() && + in.bits.hasMultibeatData() && + in.bits.addr_beat === UInt(0), + in.bits.tlDataBeats) + + def dropPendingBitsOnFirstBeat(in: DecoupledIO[Acquire]): UInt = + ~addPendingBitsOnFirstBeat(in) +} + +trait HasDataBuffer extends HasCoherenceAgentParameters { + val data_buffer = Reg(init=Vec.fill(innerDataBeats)(UInt(0, width = innerDataBits))) + + type TLDataBundle = TLBundle with HasTileLinkData with HasTileLinkBeatId + + def initDataInner[T <: Acquire](in: DecoupledIO[T]) { + when(in.fire() && in.bits.hasData()) { + data_buffer(in.bits.addr_beat) := in.bits.data + } + } + + // TODO: provide func for accessing when innerDataBeats =/= outerDataBeats or internalDataBeats + def mergeData(dataBits: Int)(beat: UInt, incoming: UInt) { + data_buffer(beat) := incoming + } + + def mergeDataInner[T <: TLDataBundle](in: DecoupledIO[T]) { + when(in.fire() && in.bits.hasData()) { + mergeData(innerDataBits)(in.bits.addr_beat, in.bits.data) + } + } + + def mergeDataOuter[T <: TLDataBundle](in: DecoupledIO[T]) { + when(in.fire() && in.bits.hasData()) { + mergeData(outerDataBits)(in.bits.addr_beat, in.bits.data) + } + } +} + +trait HasByteWriteMaskBuffer extends HasDataBuffer { + val wmask_buffer = Reg(init=Vec.fill(innerDataBeats)(UInt(0, width = innerWriteMaskBits))) + + override def initDataInner[T <: Acquire](in: DecoupledIO[T]) { + when(in.fire() && in.bits.hasData()) { + val beat = in.bits.addr_beat + val full = FillInterleaved(8, in.bits.wmask()) + data_buffer(beat) := (~full & data_buffer(beat)) | (full & in.bits.data) + wmask_buffer(beat) := in.bits.wmask() | wmask_buffer(beat) // assumes wmask_buffer is zeroed + } + } + + override def mergeData(dataBits: Int)(beat: UInt, incoming: UInt) { + val old_data = incoming // Refilled, written back, or de-cached data + val new_data = data_buffer(beat) // Newly Put data is already in the buffer + val wmask = FillInterleaved(8, wmask_buffer(beat)) + data_buffer(beat) := ~wmask & old_data | wmask & new_data + } +} + +trait HasBlockAddressBuffer extends HasCoherenceAgentParameters { + val xact_addr_block = Reg(init = UInt(0, width = blockAddrBits)) +} + + +trait HasAcquireMetadataBuffer extends HasBlockAddressBuffer { + val xact_allocate = Reg{ Bool() } + val xact_amo_shift_bytes = Reg{ UInt() } + val xact_op_code = Reg{ UInt() } + val xact_addr_byte = Reg{ UInt() } + val xact_op_size = Reg{ UInt() } + val xact_addr_beat = Wire(UInt()) + val xact_iacq = Wire(new SecondaryMissInfo) +} + +trait HasVoluntaryReleaseMetadataBuffer extends HasBlockAddressBuffer + with HasPendingBitHelpers + with HasXactTrackerStates { + def io: HierarchicalXactTrackerIO + + val xact_vol_ir_r_type = Reg{ UInt() } + val xact_vol_ir_src = Reg{ UInt() } + val xact_vol_ir_client_xact_id = Reg{ UInt() } + + def xact_vol_irel = Release( + src = xact_vol_ir_src, + voluntary = Bool(true), + r_type = xact_vol_ir_r_type, + client_xact_id = xact_vol_ir_client_xact_id, + addr_block = xact_addr_block) + (p.alterPartial({ case TLId => p(InnerTLId) })) +} + +trait AcceptsVoluntaryReleases extends HasVoluntaryReleaseMetadataBuffer { + def inner_coh: ManagerMetadata + + val pending_irel_data = Reg(init=Bits(0, width = innerDataBeats)) + val vol_ignt_counter = Wire(new TwoWayBeatCounterStatus) + + def irel_can_merge: Bool + def irel_same_xact: Bool + def irel_is_allocating: Bool = state === s_idle && io.alloc_irel.should && io.inner.release.valid + def irel_is_merging: Bool = (irel_can_merge || irel_same_xact) && io.inner.release.valid + + def innerRelease(block_vol_ignt: Bool = Bool(false), next: UInt = s_busy) { + connectTwoWayBeatCounters( + status = vol_ignt_counter, + up = io.inner.release, + down = io.inner.grant, + trackUp = (r: Release) => { + Mux(state === s_idle, io.alloc_irel.should, io.alloc_irel.matches) && r.isVoluntary() && r.requiresAck() + }, + trackDown = (g: Grant) => (state =/= s_idle) && g.isVoluntary()) + + pending_irel_data := (pending_irel_data & dropPendingBitWhenBeatHasData(io.inner.release)) + + when(irel_is_allocating) { + xact_addr_block := io.irel().addr_block + state := next + } + + when(io.inner.release.fire()) { + when(io.alloc_irel.should || (irel_can_merge && io.irel().first())) { + xact_vol_ir_r_type := io.irel().r_type + xact_vol_ir_src := io.irel().client_id + xact_vol_ir_client_xact_id := io.irel().client_xact_id + pending_irel_data := Mux(io.irel().hasMultibeatData(), + dropPendingBitWhenBeatHasData(io.inner.release), + UInt(0)) + } + } + + io.inner.grant.valid := Vec(s_wb_req, s_wb_resp, s_inner_probe, s_busy).contains(state) && + vol_ignt_counter.pending && + !(pending_irel_data.orR || block_vol_ignt) + + io.inner.grant.bits := inner_coh.makeGrant(xact_vol_irel) + + scoreboard += (pending_irel_data.orR, vol_ignt_counter.pending) + } + +} + +trait EmitsVoluntaryReleases extends HasVoluntaryReleaseMetadataBuffer { + val pending_orel = Reg(init=Bool(false)) + val pending_orel_data = Reg(init=Bits(0, width = innerDataBeats)) + val vol_ognt_counter = Wire(new TwoWayBeatCounterStatus) + + def outerRelease( + coh: ClientMetadata, + buffering: Bool = Bool(true), + data: UInt = io.irel().data, + add_pending_bit: UInt = UInt(0)) { + pending_orel_data := (pending_orel_data & dropPendingBitWhenBeatHasData(io.outer.release)) | + addPendingBitWhenBeatHasData(io.inner.release) | + add_pending_bit + + connectTwoWayBeatCounters( + status = vol_ognt_counter, + up = io.outer.release, + down = io.outer.grant, + trackUp = (r: Release) => r.isVoluntary() && r.requiresAck(), + trackDown = (g: Grant) => g.isVoluntary()) + + io.outer.release.valid := state === s_busy && + Mux(io.orel().hasData(), + Mux(buffering, + pending_orel_data(vol_ognt_counter.up.idx), + io.inner.release.valid), + pending_orel) + + + io.outer.release.bits := coh.makeVoluntaryWriteback( + client_xact_id = UInt(0), // TODO was tracker id, but not needed? + addr_block = xact_addr_block, + addr_beat = vol_ognt_counter.up.idx, + data = data) + + when(pending_orel_data.orR) { pending_orel := Bool(true) } + when(vol_ognt_counter.up.done) { pending_orel := Bool(false) } + + io.outer.grant.ready := state === s_busy + + scoreboard += (pending_orel, vol_ognt_counter.pending) + } +} + +trait EmitsInnerProbes extends HasBlockAddressBuffer + with HasXactTrackerStates + with HasPendingBitHelpers { + def io: HierarchicalXactTrackerIO + + val pending_iprbs = Reg(UInt(width = innerNCachingClients)) + val curr_probe_dst = PriorityEncoder(pending_iprbs) + val irel_counter = Wire(new TwoWayBeatCounterStatus) + + def full_representation: UInt + def initializeProbes() { pending_iprbs := full_representation & ~io.incoherent.toBits } + def irel_same_xact = io.irel().conflicts(xact_addr_block) && + !io.irel().isVoluntary() && + state === s_inner_probe + + def innerProbe(prb: Probe, next: UInt) { + pending_iprbs := pending_iprbs & dropPendingBitAtDest(io.inner.probe) + io.inner.probe.valid := state === s_inner_probe && pending_iprbs.orR + io.inner.probe.bits := prb + + connectTwoWayBeatCounters( + status = irel_counter, + up = io.inner.probe, + down = io.inner.release, + max = innerNCachingClients, + trackDown = (r: Release) => (state =/= s_idle) && !r.isVoluntary()) + + when(state === s_inner_probe && !(pending_iprbs.orR || irel_counter.pending)) { + state := next + } + + //N.B. no pending bits added to scoreboard because all handled in s_inner_probe + } +} + +trait RoutesInParent extends HasBlockAddressBuffer + with HasXactTrackerStates { + def io: HierarchicalXactTrackerIO + type AddrComparison = HasCacheBlockAddress => Bool + def exactAddrMatch(a: HasCacheBlockAddress): Bool = a.conflicts(xact_addr_block) + def routeInParent(iacqMatches: AddrComparison = exactAddrMatch, + irelMatches: AddrComparison = exactAddrMatch, + oprbMatches: AddrComparison = exactAddrMatch) { + io.alloc_iacq.matches := (state =/= s_idle) && iacqMatches(io.iacq()) + io.alloc_irel.matches := (state =/= s_idle) && irelMatches(io.irel()) + io.alloc_oprb.matches := (state =/= s_idle) && oprbMatches(io.oprb()) + io.alloc_iacq.can := state === s_idle + io.alloc_irel.can := state === s_idle + io.alloc_oprb.can := Bool(false) + } +} + +trait AcceptsInnerAcquires extends HasAcquireMetadataBuffer + with AcceptsVoluntaryReleases + with HasXactTrackerStates + with HasPendingBitHelpers { + def io: HierarchicalXactTrackerIO + def nSecondaryMisses: Int + def alwaysWriteFullBeat: Boolean + def inner_coh: ManagerMetadata + def trackerId: Int + + // Secondary miss queue holds transaction metadata used to make grants + lazy val ignt_q = Module(new Queue( + new SecondaryMissInfo()(p.alterPartial({ case TLId => p(InnerTLId) })), + 1 + nSecondaryMisses)) + + val pending_ignt = Wire(Bool()) + val ignt_data_idx = Wire(UInt()) + val ignt_data_done = Wire(Bool()) + val ifin_counter = Wire(new TwoWayBeatCounterStatus) + val pending_put_data = Reg(init=Bits(0, width = innerDataBeats)) + val pending_ignt_data = Reg(init=Bits(0, width = innerDataBeats)) + + def iacq_same_xact: Bool = { + (xact_iacq.client_xact_id === io.iacq().client_xact_id) && + xact_iacq.hasMultibeatData() && + pending_ignt && + pending_put_data(io.iacq().addr_beat) + } + def iacq_can_merge: Bool + def iacq_is_allocating: Bool = state === s_idle && io.alloc_iacq.should && io.inner.acquire.valid + def iacq_is_merging: Bool = (iacq_can_merge || iacq_same_xact) && io.inner.acquire.valid + + def innerAcquire(can_alloc: Bool, next: UInt) { + + // Enqueue some metadata information that we'll use to make coherence updates with later + ignt_q.io.enq.valid := iacq_is_allocating || (iacq_is_merging && !iacq_same_xact) + ignt_q.io.enq.bits := io.iacq() + + // Use the outputs of the queue to make further messages + xact_iacq := Mux(ignt_q.io.deq.valid, ignt_q.io.deq.bits, ignt_q.io.enq.bits) + xact_addr_beat := xact_iacq.addr_beat + pending_ignt := ignt_q.io.count > UInt(0) + + // Track whether any beats are missing from a PutBlock + pending_put_data := (pending_put_data & + dropPendingBitWhenBeatHasData(io.inner.acquire)) | + addPendingBitsOnFirstBeat(io.inner.acquire) + + // Intialize transaction metadata for accepted Acquire + when(iacq_is_allocating) { + xact_addr_block := io.iacq().addr_block + xact_allocate := io.iacq().allocate() && can_alloc + xact_amo_shift_bytes := io.iacq().amo_shift_bytes() + xact_op_code := io.iacq().op_code() + xact_addr_byte := io.iacq().addr_byte() + xact_op_size := io.iacq().op_size() + // Make sure to collect all data from a PutBlock + pending_put_data := Mux( + io.iacq().isBuiltInType(Acquire.putBlockType), + dropPendingBitWhenBeatHasData(io.inner.acquire), + UInt(0)) + pending_ignt_data := UInt(0) + state := next + } + + scoreboard += (pending_put_data.orR) + } + + def innerGrant( + data: UInt = io.ognt().data, + external_pending: Bool = Bool(false), + add: UInt = UInt(0)) { + // Track the number of outstanding inner.finishes + connectTwoWayBeatCounters( + status = ifin_counter, + up = io.inner.grant, + down = io.inner.finish, + max = nSecondaryMisses, + trackUp = (g: Grant) => g.requiresAck()) + + // Track which beats are ready for response + when(!iacq_is_allocating) { + pending_ignt_data := (pending_ignt_data & dropPendingBitWhenBeatHasData(io.inner.grant)) | + addPendingBitWhenBeatHasData(io.inner.release) | + addPendingBitWhenBeatHasData(io.outer.grant) | + add + } + + // We can issue a grant for a pending write once all data is + // received and committed to the data array or outer memory + val ignt_ack_ready = !(state === s_idle || + state === s_meta_read || + pending_put_data.orR) + + val ignt_from_iacq = inner_coh.makeGrant( + sec = ignt_q.io.deq.bits, + manager_xact_id = UInt(trackerId), + data = data) + + // Make the Grant message using the data stored in the secondary miss queue + val (cnt, done) = connectOutgoingDataBeatCounter(io.inner.grant, ignt_q.io.deq.bits.addr_beat) + ignt_data_idx := cnt + ignt_data_done := done + ignt_q.io.deq.ready := Bool(false) + when(!vol_ignt_counter.pending) { + ignt_q.io.deq.ready := ignt_data_done + io.inner.grant.bits := ignt_from_iacq + io.inner.grant.bits.addr_beat := ignt_data_idx // override based on outgoing counter + when (state === s_busy && pending_ignt) { + io.inner.grant.valid := !external_pending && + Mux(io.ignt().hasData(), pending_ignt_data(ignt_data_idx), ignt_ack_ready) + } + } + + // We must wait for as many Finishes as we sent Grants + io.inner.finish.ready := state === s_busy + + scoreboard += (pending_ignt, ifin_counter.pending) + } + +} + +trait EmitsOuterAcquires extends AcceptsInnerAcquires { + val ognt_counter = Wire(new TwoWayBeatCounterStatus) + + // 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 + def outerAcquire( + caching: Bool, + coh: ClientMetadata, + buffering: Bool = Bool(true), + data: UInt = io.iacq().data, + wmask: UInt = io.iacq().wmask(), + next: UInt = s_busy) { + + // Tracks outstanding Acquires, waiting for their matching Grant. + connectTwoWayBeatCounters( + status = ognt_counter, + up = io.outer.acquire, + down = io.outer.grant, + beat = xact_addr_beat, + trackDown = (g: Grant) => !g.isVoluntary()) + + io.outer.acquire.valid := state === s_outer_acquire && + (xact_allocate || + Mux(buffering, + !pending_put_data(ognt_counter.up.idx), + io.inner.acquire.valid)) + + io.outer.acquire.bits := + Mux(caching, + coh.makeAcquire( + op_code = xact_op_code, + client_xact_id = UInt(0), + addr_block = xact_addr_block), + BuiltInAcquireBuilder( + a_type = xact_iacq.a_type, + client_xact_id = UInt(0), + addr_block = xact_addr_block, + addr_beat = ognt_counter.up.idx, + data = data, + addr_byte = xact_addr_byte, + operand_size = xact_op_size, + opcode = xact_op_code, + wmask = wmask, + alloc = Bool(false)) + (p.alterPartial({ case TLId => p(OuterTLId)}))) + + when(state === s_outer_acquire && ognt_counter.up.done) { state := next } + + io.outer.grant.ready := state === s_busy + + scoreboard += ognt_counter.pending + } +} + +abstract class VoluntaryReleaseTracker(val trackerId: Int)(implicit p: Parameters) extends XactTracker()(p) + with AcceptsVoluntaryReleases + with RoutesInParent { + def irel_can_merge = Bool(false) + def irel_same_xact = io.irel().conflicts(xact_addr_block) && + io.irel().isVoluntary() && + pending_irel_data.orR +} + +abstract class AcquireTracker(val trackerId: Int)(implicit p: Parameters) extends XactTracker()(p) + with AcceptsInnerAcquires + with EmitsOuterAcquires + with EmitsInnerProbes + with RoutesInParent { +} diff --git a/uncore/src/main/scala/uncore.scala b/uncore/src/main/scala/uncore.scala index 8def2a1c..be20394d 100644 --- a/uncore/src/main/scala/uncore.scala +++ b/uncore/src/main/scala/uncore.scala @@ -3,6 +3,7 @@ package uncore import Chisel._ import cde.{Parameters, Field} +import junctions._ case object NReleaseTransactors extends Field[Int] case object NProbeTransactors extends Field[Int] @@ -18,6 +19,7 @@ trait HasCoherenceAgentParameters { val nReleaseTransactors = 1 val nAcquireTransactors = p(NAcquireTransactors) val nTransactors = nReleaseTransactors + nAcquireTransactors + val blockAddrBits = p(PAddrBits) - p(CacheBlockOffsetBits) val outerTLId = p(OuterTLId) val outerTLParams = p(TLKey(outerTLId)) val outerDataBeats = outerTLParams.dataBeats @@ -32,6 +34,7 @@ trait HasCoherenceAgentParameters { val innerWriteMaskBits = innerTLParams.writeMaskBits val innerBeatAddrBits = log2Up(innerDataBeats) val innerByteAddrBits = log2Up(innerDataBits/8) + val innerNCachingClients = innerTLParams.nCachingClients val maxManagerXacts = innerTLParams.maxManagerXacts require(outerDataBeats == innerDataBeats) //TODO: fix all xact_data Vecs to remove this requirement } @@ -66,31 +69,33 @@ trait HasCoherenceAgentWiringHelpers { * * When a match is reported, if ready is high the new transaction * is merged; when ready is low the transaction is being blocked. - * When no match is reported, any high readys are presumed to be + * When no match is reported, any high idles are presumed to be * from trackers that are available for allocation, and one is - * assigned via alloc based on priority; f no readys are high then - * all trackers are busy with other transactions. + * assigned via alloc based on priority; if no idles are high then + * all trackers are busy with other transactions. If idle is high + * but ready is low, the tracker will be allocated but does not + * have sufficient buffering for the data. */ def doInputRoutingWithAllocation[T <: TileLinkChannel with HasTileLinkData]( in: DecoupledIO[T], outs: Seq[DecoupledIO[T]], - matches: Seq[Bool], - allocs: Seq[Bool], + allocs: Seq[TrackerAllocation], dataOverrides: Option[Seq[UInt]] = None, allocOverride: Option[Bool] = None, matchOverride: Option[Bool] = None) { val ready_bits = Vec(outs.map(_.ready)).toBits - val alloc_bits = PriorityEncoderOH(ready_bits) - val match_bits = Vec(matches).toBits + val can_alloc_bits = Vec(allocs.map(_.can)).toBits + val should_alloc_bits = PriorityEncoderOH(can_alloc_bits) + val match_bits = Vec(allocs.map(_.matches)).toBits val no_matches = !match_bits.orR val alloc_ok = allocOverride.getOrElse(Bool(true)) val match_ok = matchOverride.getOrElse(Bool(true)) - in.ready := Mux(no_matches, ready_bits.orR, (match_bits & ready_bits).orR) && alloc_ok && match_ok - outs.zip(allocs).zipWithIndex.foreach { case((out, a), i) => + in.ready := (Mux(no_matches, can_alloc_bits, match_bits) & ready_bits).orR && alloc_ok && match_ok + outs.zip(allocs).zipWithIndex.foreach { case((out, alloc), i) => out.valid := in.valid && match_ok && alloc_ok out.bits := in.bits dataOverrides foreach { d => out.bits.data := d(i) } - a := alloc_bits(i) & no_matches & alloc_ok + alloc.should := should_alloc_bits(i) && no_matches && alloc_ok } } } @@ -141,87 +146,10 @@ class HierarchicalTLIO(implicit p: Parameters) extends CoherenceAgentBundle()(p) with HasInnerTLIO with HasCachedOuterTLIO -abstract class HierarchicalCoherenceAgent(implicit p: Parameters) extends CoherenceAgent()(p) { +abstract class HierarchicalCoherenceAgent(implicit p: Parameters) extends CoherenceAgent()(p) + with HasCoherenceAgentWiringHelpers { val io = new HierarchicalTLIO def innerTL = io.inner def outerTL = io.outer def incoherent = io.incoherent } - -trait HasTrackerAllocationIO extends Bundle { - val matches = new Bundle { - val iacq = Bool(OUTPUT) - val irel = Bool(OUTPUT) - val oprb = Bool(OUTPUT) - } - val alloc = new Bundle { - val iacq = Bool(INPUT) - val irel = Bool(INPUT) - val oprb = Bool(INPUT) - } -} - -class ManagerXactTrackerIO(implicit p: Parameters) extends ManagerTLIO()(p) - with HasTrackerAllocationIO - -class HierarchicalXactTrackerIO(implicit p: Parameters) extends HierarchicalTLIO()(p) - with HasTrackerAllocationIO - -abstract class XactTracker(implicit p: Parameters) extends CoherenceAgentModule()(p) - with HasDataBeatCounters { - def addPendingBitWhenBeat[T <: HasBeat](inc: Bool, in: T): UInt = - Fill(in.tlDataBeats, inc) & UIntToOH(in.addr_beat) - - def dropPendingBitWhenBeat[T <: HasBeat](dec: Bool, in: T): UInt = - ~Fill(in.tlDataBeats, dec) | ~UIntToOH(in.addr_beat) - - def addPendingBitWhenId[T <: HasClientId](inc: Bool, in: T): UInt = - Fill(in.tlNCachingClients, inc) & UIntToOH(in.client_id) - - def dropPendingBitWhenId[T <: HasClientId](dec: Bool, in: T): UInt = - ~Fill(in.tlNCachingClients, dec) | ~UIntToOH(in.client_id) - - 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 = - addPendingBitWhenBeatHasData(in, in.bits.allocate()) - - def addPendingBitWhenBeatIsGetOrAtomic(in: DecoupledIO[AcquireFromSrc]): UInt = { - val a = in.bits - val isGetOrAtomic = a.isBuiltInType() && - (Vec(Acquire.getType, Acquire.getBlockType, Acquire.putAtomicType).contains(a.a_type)) - 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) - - def dropPendingBitAtDest[T <: HasId](in: DecoupledIO[T]): UInt = - dropPendingBitWhenId(in.fire(), in.bits) - - def dropPendingBitAtDestWhenVoluntary[T <: HasId with MightBeVoluntary](in: DecoupledIO[T]): UInt = - dropPendingBitWhenId(in.fire() && in.bits.isVoluntary(), in.bits) - - def addPendingBitAtSrc[T <: HasId](in: DecoupledIO[T]): UInt = - addPendingBitWhenId(in.fire(), in.bits) - - def addPendingBitAtSrcWhenVoluntary[T <: HasId with MightBeVoluntary](in: DecoupledIO[T]): UInt = - addPendingBitWhenId(in.fire() && in.bits.isVoluntary(), in.bits) - - def pinAllReadyValidLow[T <: Data](b: Bundle) { - b.elements.foreach { - _._2 match { - case d: DecoupledIO[_] => - if(d.ready.dir == OUTPUT) d.ready := Bool(false) - else if(d.valid.dir == OUTPUT) d.valid := Bool(false) - case v: ValidIO[_] => if(v.valid.dir == OUTPUT) v.valid := Bool(false) - case b: Bundle => pinAllReadyValidLow(b) - case _ => - } - } - } -}