xref: /XiangShan/src/main/scala/xiangshan/frontend/icache/ICache.scala (revision 96e3f6c1467adc591a1cc2ea2aa16a200bb2313f)
1/***************************************************************************************
2* Copyright (c) 2024 Beijing Institute of Open Source Chip (BOSC)
3* Copyright (c) 2020-2024 Institute of Computing Technology, Chinese Academy of Sciences
4* Copyright (c) 2020-2021 Peng Cheng Laboratory
5*
6* XiangShan is licensed under Mulan PSL v2.
7* You can use this software according to the terms and conditions of the Mulan PSL v2.
8* You may obtain a copy of Mulan PSL v2 at:
9*          http://license.coscl.org.cn/MulanPSL2
10*
11* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
12* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
13* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
14*
15* See the Mulan PSL v2 for more details.
16*
17*
18* Acknowledgement
19*
20* This implementation is inspired by several key papers:
21* [1] Glenn Reinman, Brad Calder, and Todd Austin. "[Fetch directed instruction prefetching.]
22* (https://doi.org/10.1109/MICRO.1999.809439)" 32nd Annual ACM/IEEE International Symposium on Microarchitecture
23* (MICRO). 1999.
24***************************************************************************************/
25
26package xiangshan.frontend.icache
27
28import chisel3._
29import chisel3.util._
30import freechips.rocketchip.diplomacy.IdRange
31import freechips.rocketchip.diplomacy.LazyModule
32import freechips.rocketchip.diplomacy.LazyModuleImp
33import freechips.rocketchip.tilelink._
34import freechips.rocketchip.util.BundleFieldBase
35import huancun.AliasField
36import huancun.PrefetchField
37import org.chipsalliance.cde.config.Parameters
38import utility._
39import utils._
40import xiangshan._
41import xiangshan.cache._
42import xiangshan.cache.mmu.TlbRequestIO
43import xiangshan.frontend._
44
45case class ICacheParameters(
46    nSets:               Int = 256,
47    nWays:               Int = 4,
48    rowBits:             Int = 64,
49    nTLBEntries:         Int = 32,
50    tagECC:              Option[String] = None,
51    dataECC:             Option[String] = None,
52    replacer:            Option[String] = Some("random"),
53    PortNumber:          Int = 2,
54    nFetchMshr:          Int = 4,
55    nPrefetchMshr:       Int = 10,
56    nWayLookupSize:      Int = 32,
57    DataCodeUnit:        Int = 64,
58    ICacheDataBanks:     Int = 8,
59    ICacheDataSRAMWidth: Int = 66,
60    // TODO: hard code, need delete
61    partWayNum: Int = 4,
62    nMMIOs:     Int = 1,
63    blockBytes: Int = 64
64) extends L1CacheParameters {
65
66  val setBytes     = nSets * blockBytes
67  val aliasBitsOpt = if (setBytes > pageSize) Some(log2Ceil(setBytes / pageSize)) else None
68  val reqFields: Seq[BundleFieldBase] = Seq(
69    PrefetchField(),
70    ReqSourceField()
71  ) ++ aliasBitsOpt.map(AliasField)
72  val echoFields: Seq[BundleFieldBase] = Nil
73  def tagCode:    Code                 = Code.fromString(tagECC)
74  def dataCode:   Code                 = Code.fromString(dataECC)
75  def replacement = ReplacementPolicy.fromString(replacer, nWays, nSets)
76}
77
78trait HasICacheParameters extends HasL1CacheParameters with HasInstrMMIOConst with HasIFUConst {
79  val cacheParams = icacheParameters
80
81  def ICacheSets          = cacheParams.nSets
82  def ICacheWays          = cacheParams.nWays
83  def PortNumber          = cacheParams.PortNumber
84  def nFetchMshr          = cacheParams.nFetchMshr
85  def nPrefetchMshr       = cacheParams.nPrefetchMshr
86  def nWayLookupSize      = cacheParams.nWayLookupSize
87  def DataCodeUnit        = cacheParams.DataCodeUnit
88  def ICacheDataBanks     = cacheParams.ICacheDataBanks
89  def ICacheDataSRAMWidth = cacheParams.ICacheDataSRAMWidth
90  def partWayNum          = cacheParams.partWayNum
91
92  def ICacheMetaBits      = tagBits // FIXME: unportable: maybe use somemethod to get width
93  def ICacheMetaCodeBits  = 1       // FIXME: unportable: maybe use cacheParams.tagCode.somemethod to get width
94  def ICacheMetaEntryBits = ICacheMetaBits + ICacheMetaCodeBits
95
96  def ICacheDataBits     = blockBits / ICacheDataBanks
97  def ICacheDataCodeSegs = math.ceil(ICacheDataBits / DataCodeUnit).toInt // split data to segments for ECC checking
98  def ICacheDataCodeBits =
99    ICacheDataCodeSegs * 1 // FIXME: unportable: maybe use cacheParams.dataCode.somemethod to get width
100  def ICacheDataEntryBits = ICacheDataBits + ICacheDataCodeBits
101  def ICacheBankVisitNum  = 32 * 8 / ICacheDataBits + 1
102  def highestIdxBit       = log2Ceil(nSets) - 1
103
104  require((ICacheDataBanks >= 2) && isPow2(ICacheDataBanks))
105  require(ICacheDataSRAMWidth >= ICacheDataEntryBits)
106  require(isPow2(ICacheSets), s"nSets($ICacheSets) must be pow2")
107  require(isPow2(ICacheWays), s"nWays($ICacheWays) must be pow2")
108
109  def getBits(num: Int) = log2Ceil(num).W
110
111  def generatePipeControl(lastFire: Bool, thisFire: Bool, thisFlush: Bool, lastFlush: Bool): Bool = {
112    val valid = RegInit(false.B)
113    when(thisFlush)(valid := false.B)
114      .elsewhen(lastFire && !lastFlush)(valid := true.B)
115      .elsewhen(thisFire)(valid := false.B)
116    valid
117  }
118
119  def ResultHoldBypass[T <: Data](data: T, valid: Bool): T =
120    Mux(valid, data, RegEnable(data, valid))
121
122  def ResultHoldBypass[T <: Data](data: T, init: T, valid: Bool): T =
123    Mux(valid, data, RegEnable(data, init, valid))
124
125  def holdReleaseLatch(valid: Bool, release: Bool, flush: Bool): Bool = {
126    val bit = RegInit(false.B)
127    when(flush)(bit := false.B)
128      .elsewhen(valid && !release)(bit := true.B)
129      .elsewhen(release)(bit := false.B)
130    bit || valid
131  }
132
133  def blockCounter(block: Bool, flush: Bool, threshold: Int): Bool = {
134    val counter = RegInit(0.U(log2Up(threshold + 1).W))
135    when(block)(counter := counter + 1.U)
136    when(flush)(counter := 0.U)
137    counter > threshold.U
138  }
139
140  def InitQueue[T <: Data](entry: T, size: Int): Vec[T] =
141    return RegInit(VecInit(Seq.fill(size)(0.U.asTypeOf(entry.cloneType))))
142
143  def encodeMetaECC(meta: UInt): UInt = {
144    require(meta.getWidth == ICacheMetaBits)
145    val code = cacheParams.tagCode.encode(meta) >> ICacheMetaBits
146    code.asTypeOf(UInt(ICacheMetaCodeBits.W))
147  }
148
149  def encodeDataECC(data: UInt): UInt = {
150    require(data.getWidth == ICacheDataBits)
151    val datas = data.asTypeOf(Vec(ICacheDataCodeSegs, UInt((ICacheDataBits / ICacheDataCodeSegs).W)))
152    val codes = VecInit(datas.map(cacheParams.dataCode.encode(_) >> (ICacheDataBits / ICacheDataCodeSegs)))
153    codes.asTypeOf(UInt(ICacheDataCodeBits.W))
154  }
155
156  def getBankSel(blkOffset: UInt, valid: Bool = true.B): Vec[UInt] = {
157    val bankIdxLow  = Cat(0.U(1.W), blkOffset) >> log2Ceil(blockBytes / ICacheDataBanks)
158    val bankIdxHigh = (Cat(0.U(1.W), blkOffset) + 32.U) >> log2Ceil(blockBytes / ICacheDataBanks)
159    val bankSel     = VecInit((0 until ICacheDataBanks * 2).map(i => (i.U >= bankIdxLow) && (i.U <= bankIdxHigh)))
160    assert(
161      !valid || PopCount(bankSel) === ICacheBankVisitNum.U,
162      "The number of bank visits must be %d, but bankSel=0x%x",
163      ICacheBankVisitNum.U,
164      bankSel.asUInt
165    )
166    bankSel.asTypeOf(UInt((ICacheDataBanks * 2).W)).asTypeOf(Vec(2, UInt(ICacheDataBanks.W)))
167  }
168
169  def getLineSel(blkOffset: UInt)(implicit p: Parameters): Vec[Bool] = {
170    val bankIdxLow = blkOffset >> log2Ceil(blockBytes / ICacheDataBanks)
171    val lineSel    = VecInit((0 until ICacheDataBanks).map(i => i.U < bankIdxLow))
172    lineSel
173  }
174
175  def getBlkAddr(addr:           UInt) = addr >> blockOffBits
176  def getPhyTagFromBlk(addr:     UInt): UInt = addr >> (pgUntagBits - blockOffBits)
177  def getIdxFromBlk(addr:        UInt) = addr(idxBits - 1, 0)
178  def get_paddr_from_ptag(vaddr: UInt, ptag: UInt) = Cat(ptag, vaddr(pgUntagBits - 1, 0))
179}
180
181abstract class ICacheBundle(implicit p: Parameters) extends XSBundle
182    with HasICacheParameters
183
184abstract class ICacheModule(implicit p: Parameters) extends XSModule
185    with HasICacheParameters
186
187abstract class ICacheArray(implicit p: Parameters) extends XSModule
188    with HasICacheParameters
189
190class ICacheMetadata(implicit p: Parameters) extends ICacheBundle {
191  val tag = UInt(tagBits.W)
192}
193
194object ICacheMetadata {
195  def apply(tag: Bits)(implicit p: Parameters) = {
196    val meta = Wire(new ICacheMetadata)
197    meta.tag := tag
198    meta
199  }
200}
201
202class ICacheMetaArray()(implicit p: Parameters) extends ICacheArray {
203  class ICacheMetaEntry(implicit p: Parameters) extends ICacheBundle {
204    val meta: ICacheMetadata = new ICacheMetadata
205    val code: UInt           = UInt(ICacheMetaCodeBits.W)
206  }
207
208  private object ICacheMetaEntry {
209    def apply(meta: ICacheMetadata)(implicit p: Parameters): ICacheMetaEntry = {
210      val entry = Wire(new ICacheMetaEntry)
211      entry.meta := meta
212      entry.code := encodeMetaECC(meta.asUInt)
213      entry
214    }
215  }
216
217  // sanity check
218  require(ICacheMetaEntryBits == (new ICacheMetaEntry).getWidth)
219
220  val io = IO(new Bundle {
221    val write    = Flipped(DecoupledIO(new ICacheMetaWriteBundle))
222    val read     = Flipped(DecoupledIO(new ICacheReadBundle))
223    val readResp = Output(new ICacheMetaRespBundle)
224    val flush    = Vec(PortNumber, Flipped(ValidIO(new ICacheMetaFlushBundle)))
225    val flushAll = Input(Bool())
226  })
227
228  val port_0_read_0 = io.read.valid && !io.read.bits.vSetIdx(0)(0)
229  val port_0_read_1 = io.read.valid && io.read.bits.vSetIdx(0)(0)
230  val port_1_read_1 = io.read.valid && io.read.bits.vSetIdx(1)(0) && io.read.bits.isDoubleLine
231  val port_1_read_0 = io.read.valid && !io.read.bits.vSetIdx(1)(0) && io.read.bits.isDoubleLine
232
233  val port_0_read_0_reg = RegEnable(port_0_read_0, 0.U.asTypeOf(port_0_read_0), io.read.fire)
234  val port_0_read_1_reg = RegEnable(port_0_read_1, 0.U.asTypeOf(port_0_read_1), io.read.fire)
235  val port_1_read_1_reg = RegEnable(port_1_read_1, 0.U.asTypeOf(port_1_read_1), io.read.fire)
236  val port_1_read_0_reg = RegEnable(port_1_read_0, 0.U.asTypeOf(port_1_read_0), io.read.fire)
237
238  val bank_0_idx = Mux(port_0_read_0, io.read.bits.vSetIdx(0), io.read.bits.vSetIdx(1))
239  val bank_1_idx = Mux(port_0_read_1, io.read.bits.vSetIdx(0), io.read.bits.vSetIdx(1))
240  val bank_idx   = Seq(bank_0_idx, bank_1_idx)
241
242  val write_bank_0 = io.write.valid && !io.write.bits.bankIdx
243  val write_bank_1 = io.write.valid && io.write.bits.bankIdx
244
245  val write_meta_bits = ICacheMetaEntry(meta =
246    ICacheMetadata(
247      tag = io.write.bits.phyTag
248    )
249  )
250
251  val tagArrays = (0 until 2) map { bank =>
252    val tagArray = Module(new SRAMTemplate(
253      new ICacheMetaEntry(),
254      set = nSets / 2,
255      way = nWays,
256      shouldReset = true,
257      holdRead = true,
258      singlePort = true,
259      withClockGate = true
260    ))
261
262    // meta connection
263    if (bank == 0) {
264      tagArray.io.r.req.valid := port_0_read_0 || port_1_read_0
265      tagArray.io.r.req.bits.apply(setIdx = bank_0_idx(highestIdxBit, 1))
266      tagArray.io.w.req.valid := write_bank_0
267      tagArray.io.w.req.bits.apply(
268        data = write_meta_bits,
269        setIdx = io.write.bits.virIdx(highestIdxBit, 1),
270        waymask = io.write.bits.waymask
271      )
272    } else {
273      tagArray.io.r.req.valid := port_0_read_1 || port_1_read_1
274      tagArray.io.r.req.bits.apply(setIdx = bank_1_idx(highestIdxBit, 1))
275      tagArray.io.w.req.valid := write_bank_1
276      tagArray.io.w.req.bits.apply(
277        data = write_meta_bits,
278        setIdx = io.write.bits.virIdx(highestIdxBit, 1),
279        waymask = io.write.bits.waymask
280      )
281    }
282
283    tagArray
284  }
285
286  val read_set_idx_next = RegEnable(io.read.bits.vSetIdx, 0.U.asTypeOf(io.read.bits.vSetIdx), io.read.fire)
287  val valid_array       = RegInit(VecInit(Seq.fill(nWays)(0.U(nSets.W))))
288  val valid_metas       = Wire(Vec(PortNumber, Vec(nWays, Bool())))
289  // valid read
290  (0 until PortNumber).foreach(i =>
291    (0 until nWays).foreach(way =>
292      valid_metas(i)(way) := valid_array(way)(read_set_idx_next(i))
293    )
294  )
295  io.readResp.entryValid := valid_metas
296
297  io.read.ready := !io.write.valid && !io.flush.map(_.valid).reduce(_ || _) && !io.flushAll &&
298    tagArrays.map(_.io.r.req.ready).reduce(_ && _)
299
300  // valid write
301  val way_num = OHToUInt(io.write.bits.waymask)
302  when(io.write.valid) {
303    valid_array(way_num) := valid_array(way_num).bitSet(io.write.bits.virIdx, true.B)
304  }
305
306  XSPerfAccumulate("meta_refill_num", io.write.valid)
307
308  io.readResp.metas <> DontCare
309  io.readResp.codes <> DontCare
310  val readMetaEntries = tagArrays.map(port => port.io.r.resp.asTypeOf(Vec(nWays, new ICacheMetaEntry())))
311  val readMetas       = readMetaEntries.map(_.map(_.meta))
312  val readCodes       = readMetaEntries.map(_.map(_.code))
313
314  // TEST: force ECC to fail by setting readCodes to 0
315  if (ICacheForceMetaECCError) {
316    readCodes.foreach(_.foreach(_ := 0.U))
317  }
318
319  when(port_0_read_0_reg) {
320    io.readResp.metas(0) := readMetas(0)
321    io.readResp.codes(0) := readCodes(0)
322  }.elsewhen(port_0_read_1_reg) {
323    io.readResp.metas(0) := readMetas(1)
324    io.readResp.codes(0) := readCodes(1)
325  }
326
327  when(port_1_read_0_reg) {
328    io.readResp.metas(1) := readMetas(0)
329    io.readResp.codes(1) := readCodes(0)
330  }.elsewhen(port_1_read_1_reg) {
331    io.readResp.metas(1) := readMetas(1)
332    io.readResp.codes(1) := readCodes(1)
333  }
334
335  io.write.ready := true.B // TODO : has bug ? should be !io.cacheOp.req.valid
336
337  /*
338   * flush logic
339   */
340  // flush standalone set (e.g. flushed by mainPipe before doing re-fetch)
341  when(io.flush.map(_.valid).reduce(_ || _)) {
342    (0 until nWays).foreach { w =>
343      valid_array(w) := (0 until PortNumber).map { i =>
344        Mux(
345          // check if set `virIdx` in way `w` is requested to be flushed by port `i`
346          io.flush(i).valid && io.flush(i).bits.waymask(w),
347          valid_array(w).bitSet(io.flush(i).bits.virIdx, false.B),
348          valid_array(w)
349        )
350      }.reduce(_ & _)
351    }
352  }
353
354  // flush all (e.g. fence.i)
355  when(io.flushAll) {
356    (0 until nWays).foreach(w => valid_array(w) := 0.U)
357  }
358
359  // PERF: flush counter
360  XSPerfAccumulate("flush", io.flush.map(_.valid).reduce(_ || _))
361  XSPerfAccumulate("flush_all", io.flushAll)
362}
363
364class ICacheDataArray(implicit p: Parameters) extends ICacheArray {
365  class ICacheDataEntry(implicit p: Parameters) extends ICacheBundle {
366    val data = UInt(ICacheDataBits.W)
367    val code = UInt(ICacheDataCodeBits.W)
368  }
369
370  object ICacheDataEntry {
371    def apply(data: UInt)(implicit p: Parameters) = {
372      val entry = Wire(new ICacheDataEntry)
373      entry.data := data
374      entry.code := encodeDataECC(data)
375      entry
376    }
377  }
378
379  val io = IO {
380    new Bundle {
381      val write = Flipped(DecoupledIO(new ICacheDataWriteBundle))
382      // TODO: fix hard code
383      val read     = Flipped(Vec(4, DecoupledIO(new ICacheReadBundle)))
384      val readResp = Output(new ICacheDataRespBundle)
385    }
386  }
387
388  /**
389    ******************************************************************************
390    * data array
391    ******************************************************************************
392    */
393  val writeDatas   = io.write.bits.data.asTypeOf(Vec(ICacheDataBanks, UInt(ICacheDataBits.W)))
394  val writeEntries = writeDatas.map(ICacheDataEntry(_).asUInt)
395
396  val bankSel  = getBankSel(io.read(0).bits.blkOffset, io.read(0).valid)
397  val lineSel  = getLineSel(io.read(0).bits.blkOffset)
398  val waymasks = io.read(0).bits.wayMask
399  val masks    = Wire(Vec(nWays, Vec(ICacheDataBanks, Bool())))
400  (0 until nWays).foreach { way =>
401    (0 until ICacheDataBanks).foreach { bank =>
402      masks(way)(bank) := Mux(
403        lineSel(bank),
404        waymasks(1)(way) && bankSel(1)(bank).asBool,
405        waymasks(0)(way) && bankSel(0)(bank).asBool
406      )
407    }
408  }
409
410  val dataArrays = (0 until nWays).map { way =>
411    (0 until ICacheDataBanks).map { bank =>
412      val sramBank = Module(new SRAMTemplateWithFixedWidth(
413        UInt(ICacheDataEntryBits.W),
414        set = nSets,
415        width = ICacheDataSRAMWidth,
416        shouldReset = true,
417        holdRead = true,
418        singlePort = true,
419        withClockGate = true
420      ))
421
422      // read
423      sramBank.io.r.req.valid := io.read(bank % 4).valid && masks(way)(bank)
424      sramBank.io.r.req.bits.apply(setIdx =
425        Mux(lineSel(bank), io.read(bank % 4).bits.vSetIdx(1), io.read(bank % 4).bits.vSetIdx(0))
426      )
427      // write
428      sramBank.io.w.req.valid := io.write.valid && io.write.bits.waymask(way).asBool
429      sramBank.io.w.req.bits.apply(
430        data = writeEntries(bank),
431        setIdx = io.write.bits.virIdx,
432        // waymask is invalid when way of SRAMTemplate <= 1
433        waymask = 0.U
434      )
435      sramBank
436    }
437  }
438
439  /**
440    ******************************************************************************
441    * read logic
442    ******************************************************************************
443    */
444  val masksReg = RegEnable(masks, 0.U.asTypeOf(masks), io.read(0).valid)
445  val readDataWithCode = (0 until ICacheDataBanks).map(bank =>
446    Mux1H(VecInit(masksReg.map(_(bank))).asTypeOf(UInt(nWays.W)), dataArrays.map(_(bank).io.r.resp.asUInt))
447  )
448  val readEntries = readDataWithCode.map(_.asTypeOf(new ICacheDataEntry()))
449  val readDatas   = VecInit(readEntries.map(_.data))
450  val readCodes   = VecInit(readEntries.map(_.code))
451
452  // TEST: force ECC to fail by setting readCodes to 0
453  if (ICacheForceDataECCError) {
454    readCodes.foreach(_ := 0.U)
455  }
456
457  /**
458    ******************************************************************************
459    * IO
460    ******************************************************************************
461    */
462  io.readResp.datas := readDatas
463  io.readResp.codes := readCodes
464  io.write.ready    := true.B
465  io.read.foreach(_.ready := !io.write.valid)
466}
467
468class ICacheReplacer(implicit p: Parameters) extends ICacheModule {
469  val io = IO(new Bundle {
470    val touch  = Vec(PortNumber, Flipped(ValidIO(new ReplacerTouch)))
471    val victim = Flipped(new ReplacerVictim)
472  })
473
474  val replacers = Seq.fill(PortNumber)(ReplacementPolicy.fromString(cacheParams.replacer, nWays, nSets / PortNumber))
475
476  // touch
477  val touch_sets = Seq.fill(PortNumber)(Wire(Vec(2, UInt(log2Ceil(nSets / 2).W))))
478  val touch_ways = Seq.fill(PortNumber)(Wire(Vec(2, Valid(UInt(log2Ceil(nWays).W)))))
479  (0 until PortNumber).foreach { i =>
480    touch_sets(i)(0) := Mux(
481      io.touch(i).bits.vSetIdx(0),
482      io.touch(1).bits.vSetIdx(highestIdxBit, 1),
483      io.touch(0).bits.vSetIdx(highestIdxBit, 1)
484    )
485    touch_ways(i)(0).bits  := Mux(io.touch(i).bits.vSetIdx(0), io.touch(1).bits.way, io.touch(0).bits.way)
486    touch_ways(i)(0).valid := Mux(io.touch(i).bits.vSetIdx(0), io.touch(1).valid, io.touch(0).valid)
487  }
488
489  // victim
490  io.victim.way := Mux(
491    io.victim.vSetIdx.bits(0),
492    replacers(1).way(io.victim.vSetIdx.bits(highestIdxBit, 1)),
493    replacers(0).way(io.victim.vSetIdx.bits(highestIdxBit, 1))
494  )
495
496  // touch the victim in next cycle
497  val victim_vSetIdx_reg =
498    RegEnable(io.victim.vSetIdx.bits, 0.U.asTypeOf(io.victim.vSetIdx.bits), io.victim.vSetIdx.valid)
499  val victim_way_reg = RegEnable(io.victim.way, 0.U.asTypeOf(io.victim.way), io.victim.vSetIdx.valid)
500  (0 until PortNumber).foreach { i =>
501    touch_sets(i)(1)       := victim_vSetIdx_reg(highestIdxBit, 1)
502    touch_ways(i)(1).bits  := victim_way_reg
503    touch_ways(i)(1).valid := RegNext(io.victim.vSetIdx.valid) && (victim_vSetIdx_reg(0) === i.U)
504  }
505
506  ((replacers zip touch_sets) zip touch_ways).map { case ((r, s), w) => r.access(s, w) }
507}
508
509class ICacheIO(implicit p: Parameters) extends ICacheBundle {
510  val hartId       = Input(UInt(hartIdLen.W))
511  val ftqPrefetch  = Flipped(new FtqToPrefetchIO)
512  val softPrefetch = Vec(backendParams.LduCnt, Flipped(Valid(new SoftIfetchPrefetchBundle)))
513  val stop         = Input(Bool())
514  val fetch        = new ICacheMainPipeBundle
515  val toIFU        = Output(Bool())
516  val pmp          = Vec(2 * PortNumber, new ICachePMPBundle)
517  val itlb         = Vec(PortNumber, new TlbRequestIO)
518  val perfInfo     = Output(new ICachePerfInfo)
519  val error        = ValidIO(new L1CacheErrorInfo)
520  /* CSR control signal */
521  val csr_pf_enable     = Input(Bool())
522  val csr_parity_enable = Input(Bool())
523  val fencei            = Input(Bool())
524  val flush             = Input(Bool())
525}
526
527class ICache()(implicit p: Parameters) extends LazyModule with HasICacheParameters {
528  override def shouldBeInlined: Boolean = false
529
530  val clientParameters = TLMasterPortParameters.v1(
531    Seq(TLMasterParameters.v1(
532      name = "icache",
533      sourceId = IdRange(0, cacheParams.nFetchMshr + cacheParams.nPrefetchMshr + 1)
534    )),
535    requestFields = cacheParams.reqFields,
536    echoFields = cacheParams.echoFields
537  )
538
539  val clientNode = TLClientNode(Seq(clientParameters))
540
541  lazy val module = new ICacheImp(this)
542}
543
544class ICacheImp(outer: ICache) extends LazyModuleImp(outer) with HasICacheParameters with HasPerfEvents {
545  val io = IO(new ICacheIO)
546
547  println("ICache:")
548  println("  TagECC: " + cacheParams.tagECC)
549  println("  DataECC: " + cacheParams.dataECC)
550  println("  ICacheSets: " + cacheParams.nSets)
551  println("  ICacheWays: " + cacheParams.nWays)
552  println("  PortNumber: " + cacheParams.PortNumber)
553  println("  nFetchMshr: " + cacheParams.nFetchMshr)
554  println("  nPrefetchMshr: " + cacheParams.nPrefetchMshr)
555  println("  nWayLookupSize: " + cacheParams.nWayLookupSize)
556  println("  DataCodeUnit: " + cacheParams.DataCodeUnit)
557  println("  ICacheDataBanks: " + cacheParams.ICacheDataBanks)
558  println("  ICacheDataSRAMWidth: " + cacheParams.ICacheDataSRAMWidth)
559
560  val (bus, edge) = outer.clientNode.out.head
561
562  val metaArray  = Module(new ICacheMetaArray)
563  val dataArray  = Module(new ICacheDataArray)
564  val mainPipe   = Module(new ICacheMainPipe)
565  val missUnit   = Module(new ICacheMissUnit(edge))
566  val replacer   = Module(new ICacheReplacer)
567  val prefetcher = Module(new IPrefetchPipe)
568  val wayLookup  = Module(new WayLookup)
569
570  dataArray.io.write <> missUnit.io.data_write
571  dataArray.io.read <> mainPipe.io.dataArray.toIData
572  dataArray.io.readResp <> mainPipe.io.dataArray.fromIData
573
574  metaArray.io.flushAll := io.fencei
575  metaArray.io.flush <> mainPipe.io.metaArrayFlush
576  metaArray.io.write <> missUnit.io.meta_write
577  metaArray.io.read <> prefetcher.io.metaRead.toIMeta
578  metaArray.io.readResp <> prefetcher.io.metaRead.fromIMeta
579
580  prefetcher.io.flush             := io.flush
581  prefetcher.io.csr_pf_enable     := io.csr_pf_enable
582  prefetcher.io.csr_parity_enable := io.csr_parity_enable
583  prefetcher.io.MSHRResp          := missUnit.io.fetch_resp
584  prefetcher.io.flushFromBpu      := io.ftqPrefetch.flushFromBpu
585  // cache softPrefetch
586  private val softPrefetchValid = RegInit(false.B)
587  private val softPrefetch      = RegInit(0.U.asTypeOf(new IPrefetchReq))
588  /* FIXME:
589   * If there is already a pending softPrefetch request, it will be overwritten.
590   * Also, if there are multiple softPrefetch requests in the same cycle, only the first one will be accepted.
591   * We should implement a softPrefetchQueue (like ibuffer, multi-in, single-out) to solve this.
592   * However, the impact on performance still needs to be assessed.
593   * Considering that the frequency of prefetch.i may not be high, let's start with a temporary dummy solution.
594   */
595  when(io.softPrefetch.map(_.valid).reduce(_ || _)) {
596    softPrefetchValid := true.B
597    softPrefetch.fromSoftPrefetch(MuxCase(
598      0.U.asTypeOf(new SoftIfetchPrefetchBundle),
599      io.softPrefetch.map(req => req.valid -> req.bits)
600    ))
601  }.elsewhen(prefetcher.io.req.fire) {
602    softPrefetchValid := false.B
603  }
604  // pass ftqPrefetch
605  private val ftqPrefetch = WireInit(0.U.asTypeOf(new IPrefetchReq))
606  ftqPrefetch.fromFtqICacheInfo(io.ftqPrefetch.req.bits)
607  // software prefetch has higher priority
608  prefetcher.io.req.valid                 := softPrefetchValid || io.ftqPrefetch.req.valid
609  prefetcher.io.req.bits                  := Mux(softPrefetchValid, softPrefetch, ftqPrefetch)
610  prefetcher.io.req.bits.backendException := io.ftqPrefetch.backendException
611  io.ftqPrefetch.req.ready                := prefetcher.io.req.ready && !softPrefetchValid
612
613  missUnit.io.hartId := io.hartId
614  missUnit.io.fencei := io.fencei
615  missUnit.io.flush  := io.flush
616  missUnit.io.fetch_req <> mainPipe.io.mshr.req
617  missUnit.io.prefetch_req <> prefetcher.io.MSHRReq
618  missUnit.io.mem_grant.valid := false.B
619  missUnit.io.mem_grant.bits  := DontCare
620  missUnit.io.mem_grant <> bus.d
621
622  mainPipe.io.flush             := io.flush
623  mainPipe.io.respStall         := io.stop
624  mainPipe.io.csr_parity_enable := io.csr_parity_enable
625  mainPipe.io.hartId            := io.hartId
626  mainPipe.io.mshr.resp         := missUnit.io.fetch_resp
627  mainPipe.io.fetch.req <> io.fetch.req
628  mainPipe.io.wayLookupRead <> wayLookup.io.read
629
630  wayLookup.io.flush := io.flush
631  wayLookup.io.write <> prefetcher.io.wayLookupWrite
632  wayLookup.io.update := missUnit.io.fetch_resp
633
634  replacer.io.touch <> mainPipe.io.touch
635  replacer.io.victim <> missUnit.io.victim
636
637  io.pmp(0) <> mainPipe.io.pmp(0)
638  io.pmp(1) <> mainPipe.io.pmp(1)
639  io.pmp(2) <> prefetcher.io.pmp(0)
640  io.pmp(3) <> prefetcher.io.pmp(1)
641
642  io.itlb(0) <> prefetcher.io.itlb(0)
643  io.itlb(1) <> prefetcher.io.itlb(1)
644
645  // notify IFU that Icache pipeline is available
646  io.toIFU    := mainPipe.io.fetch.req.ready
647  io.perfInfo := mainPipe.io.perfInfo
648
649  io.fetch.resp <> mainPipe.io.fetch.resp
650  io.fetch.topdownIcacheMiss := mainPipe.io.fetch.topdownIcacheMiss
651  io.fetch.topdownItlbMiss   := mainPipe.io.fetch.topdownItlbMiss
652
653  bus.b.ready := false.B
654  bus.c.valid := false.B
655  bus.c.bits  := DontCare
656  bus.e.valid := false.B
657  bus.e.bits  := DontCare
658
659  bus.a <> missUnit.io.mem_acquire
660
661  // Parity error port
662  val errors       = mainPipe.io.errors
663  val errors_valid = errors.map(e => e.valid).reduce(_ | _)
664  io.error.bits <> RegEnable(
665    PriorityMux(errors.map(e => e.valid -> e.bits)),
666    0.U.asTypeOf(errors(0).bits),
667    errors_valid
668  )
669  io.error.valid := RegNext(errors_valid, false.B)
670
671  XSPerfAccumulate(
672    "softPrefetch_drop_not_ready",
673    io.softPrefetch.map(_.valid).reduce(_ || _) && softPrefetchValid && !prefetcher.io.req.fire
674  )
675  XSPerfAccumulate("softPrefetch_drop_multi_req", PopCount(io.softPrefetch.map(_.valid)) > 1.U)
676  XSPerfAccumulate("softPrefetch_block_ftq", softPrefetchValid && io.ftqPrefetch.req.valid)
677
678  val perfEvents = Seq(
679    ("icache_miss_cnt  ", false.B),
680    ("icache_miss_penalty", BoolStopWatch(start = false.B, stop = false.B || false.B, startHighPriority = true))
681  )
682  generatePerfEvent()
683}
684
685class ICachePartWayReadBundle[T <: Data](gen: T, pWay: Int)(implicit p: Parameters)
686    extends ICacheBundle {
687  val req = Flipped(Vec(
688    PortNumber,
689    Decoupled(new Bundle {
690      val ridx = UInt((log2Ceil(nSets) - 1).W)
691    })
692  ))
693  val resp = Output(new Bundle {
694    val rdata = Vec(PortNumber, Vec(pWay, gen))
695  })
696}
697
698class ICacheWriteBundle[T <: Data](gen: T, pWay: Int)(implicit p: Parameters)
699    extends ICacheBundle {
700  val wdata    = gen
701  val widx     = UInt((log2Ceil(nSets) - 1).W)
702  val wbankidx = Bool()
703  val wmask    = Vec(pWay, Bool())
704}
705
706class ICachePartWayArray[T <: Data](gen: T, pWay: Int)(implicit p: Parameters) extends ICacheArray {
707
708  // including part way data
709  val io = IO {
710    new Bundle {
711      val read  = new ICachePartWayReadBundle(gen, pWay)
712      val write = Flipped(ValidIO(new ICacheWriteBundle(gen, pWay)))
713    }
714  }
715
716  io.read.req.map(_.ready := !io.write.valid)
717
718  val srams = (0 until PortNumber) map { bank =>
719    val sramBank = Module(new SRAMTemplate(
720      gen,
721      set = nSets / 2,
722      way = pWay,
723      shouldReset = true,
724      holdRead = true,
725      singlePort = true,
726      withClockGate = true
727    ))
728
729    sramBank.io.r.req.valid := io.read.req(bank).valid
730    sramBank.io.r.req.bits.apply(setIdx = io.read.req(bank).bits.ridx)
731
732    if (bank == 0) sramBank.io.w.req.valid := io.write.valid && !io.write.bits.wbankidx
733    else sramBank.io.w.req.valid           := io.write.valid && io.write.bits.wbankidx
734    sramBank.io.w.req.bits.apply(
735      data = io.write.bits.wdata,
736      setIdx = io.write.bits.widx,
737      waymask = io.write.bits.wmask.asUInt
738    )
739
740    sramBank
741  }
742
743  io.read.req.map(_.ready := !io.write.valid && srams.map(_.io.r.req.ready).reduce(_ && _))
744
745  io.read.resp.rdata := VecInit(srams.map(bank => bank.io.r.resp.asTypeOf(Vec(pWay, gen))))
746
747}
748
749// Automatically partition the SRAM based on the width of the data and the desired width.
750// final SRAM width = width * way
751class SRAMTemplateWithFixedWidth[T <: Data](
752    gen:           T,
753    set:           Int,
754    width:         Int,
755    way:           Int = 1,
756    shouldReset:   Boolean = false,
757    holdRead:      Boolean = false,
758    singlePort:    Boolean = false,
759    bypassWrite:   Boolean = false,
760    withClockGate: Boolean = false
761) extends Module {
762
763  val dataBits  = gen.getWidth
764  val bankNum   = math.ceil(dataBits.toDouble / width.toDouble).toInt
765  val totalBits = bankNum * width
766
767  val io = IO(new Bundle {
768    val r = Flipped(new SRAMReadBus(gen, set, way))
769    val w = Flipped(new SRAMWriteBus(gen, set, way))
770  })
771
772  val wordType = UInt(width.W)
773  val writeDatas = (0 until bankNum).map(bank =>
774    VecInit((0 until way).map(i =>
775      io.w.req.bits.data(i).asTypeOf(UInt(totalBits.W)).asTypeOf(Vec(bankNum, wordType))(bank)
776    ))
777  )
778
779  val srams = (0 until bankNum) map { bank =>
780    val sramBank = Module(new SRAMTemplate(
781      wordType,
782      set = set,
783      way = way,
784      shouldReset = shouldReset,
785      holdRead = holdRead,
786      singlePort = singlePort,
787      bypassWrite = bypassWrite,
788      withClockGate = withClockGate
789    ))
790    // read req
791    sramBank.io.r.req.valid       := io.r.req.valid
792    sramBank.io.r.req.bits.setIdx := io.r.req.bits.setIdx
793
794    // write req
795    sramBank.io.w.req.valid       := io.w.req.valid
796    sramBank.io.w.req.bits.setIdx := io.w.req.bits.setIdx
797    sramBank.io.w.req.bits.data   := writeDatas(bank)
798    sramBank.io.w.req.bits.waymask.map(_ := io.w.req.bits.waymask.get)
799
800    sramBank
801  }
802
803  io.r.req.ready := !io.w.req.valid
804  (0 until way).foreach { i =>
805    io.r.resp.data(i) := VecInit((0 until bankNum).map(bank =>
806      srams(bank).io.r.resp.data(i)
807    )).asTypeOf(UInt(totalBits.W))(dataBits - 1, 0).asTypeOf(gen.cloneType)
808  }
809
810  io.r.req.ready := srams.head.io.r.req.ready
811  io.w.req.ready := srams.head.io.w.req.ready
812}
813