未验证 提交 6786cfb7 编写于 作者: W William Wang 提交者: GitHub

dcache: repipeline ecc check logic for timing (#1582)

This commit re-pipelines ECC check logic in data cache and exception generate logic for better timing.
Now ecc error is checked 1 cycle after reading result from data sram. An extra cycle is added for load
writeback to ROB.

Future work: move the pipeline to https://github.com/OpenXiangShan/XiangShan/blob/master/src/main/scala/xiangshan/backend/CtrlBlock.scala#L266-L277, which add a regnext.

* dcache: repipeline ecc check logic for timing

* chore: fix normal loadAccessFault logic

* wbu: delay load unit wb for 1 cycle

* dcache: add 1 extra cycle for beu error report
上级 fa9d712c
......@@ -167,7 +167,7 @@ case class XSCoreParameters
EnableLdVioCheckAfterReset: Boolean = true,
EnableSoftPrefetchAfterReset: Boolean = true,
EnableCacheErrorAfterReset: Boolean = true,
RefillSize: Int = 512,
EnableAccurateLoadError: Boolean = true,
MMUAsidLen: Int = 16, // max is 16, 0 is not supported now
itlbParameters: TLBParameters = TLBParameters(
name = "itlb",
......@@ -387,7 +387,7 @@ trait HasXSParameter {
val EnableLdVioCheckAfterReset = coreParams.EnableLdVioCheckAfterReset
val EnableSoftPrefetchAfterReset = coreParams.EnableSoftPrefetchAfterReset
val EnableCacheErrorAfterReset = coreParams.EnableCacheErrorAfterReset
val RefillSize = coreParams.RefillSize
val EnableAccurateLoadError = coreParams.EnableAccurateLoadError
val asidLen = coreParams.MMUAsidLen
val BTLBWidth = coreParams.LoadPipelineWidth + coreParams.StorePipelineWidth
val refillBothTlb = coreParams.refillBothTlb
......
......@@ -272,6 +272,9 @@ class XSCoreImp(outer: XSCoreBase) extends LazyModuleImp(outer)
outer.wbArbiter.module.io.in <> allWriteback
val rfWriteback = outer.wbArbiter.module.io.out
// memblock error exception writeback, 1 cycle after normal writeback
wb2Ctrl.io.delayedLoadError <> memBlock.io.delayedLoadError
wb2Ctrl.io.redirect <> ctrlBlock.io.redirect
outer.wb2Ctrl.generateWritebackIO()
......
......@@ -70,6 +70,7 @@ class MemBlockImp(outer: MemBlock) extends LazyModuleImp(outer)
val stIssuePtr = Output(new SqPtr())
// out
val writeback = Vec(exuParameters.LsExuCnt + exuParameters.StuCnt, DecoupledIO(new ExuOutput))
val delayedLoadError = Vec(exuParameters.LduCnt, Output(Bool()))
val otherFastWakeup = Vec(exuParameters.LduCnt + 2 * exuParameters.StuCnt, ValidIO(new MicroOp))
// misc
val stIn = Vec(exuParameters.StuCnt, ValidIO(new ExuInput))
......@@ -79,7 +80,7 @@ class MemBlockImp(outer: MemBlock) extends LazyModuleImp(outer)
val tlbCsr = Input(new TlbCsrBundle)
val fenceToSbuffer = Flipped(new FenceToSbuffer)
val enqLsq = new LsqEnqIO
val memPredUpdate = Vec(exuParameters.StuCnt, Input(new MemPredUpdateReq))
// val memPredUpdate = Vec(exuParameters.StuCnt, Input(new MemPredUpdateReq))
val lsqio = new Bundle {
val exceptionAddr = new ExceptionAddrIO // to csr
val rob = Flipped(new RobLsqIO) // rob to lsq
......@@ -277,15 +278,21 @@ class MemBlockImp(outer: MemBlock) extends LazyModuleImp(outer)
// Lsq to load unit's rs
// passdown to lsq
// passdown to lsq (load s2)
lsq.io.loadIn(i) <> loadUnits(i).io.lsq.loadIn
lsq.io.ldout(i) <> loadUnits(i).io.lsq.ldout
lsq.io.loadDataForwarded(i) <> loadUnits(i).io.lsq.loadDataForwarded
lsq.io.trigger(i) <> loadUnits(i).io.lsq.trigger
// passdown to lsq (load s3)
lsq.io.dcacheRequireReplay(i) <> loadUnits(i).io.lsq.dcacheRequireReplay
lsq.io.delayedLoadError(i) <> loadUnits(i).io.delayedLoadError
// alter writeback exception info
io.delayedLoadError(i) := loadUnits(i).io.lsq.delayedLoadError
// update mem dependency predictor
io.memPredUpdate(i) := DontCare
lsq.io.dcacheRequireReplay(i) <> loadUnits(i).io.lsq.dcacheRequireReplay
// io.memPredUpdate(i) := DontCare
// Trigger Regs
// addr: 0-2 for store, 3-5 for load
......
......@@ -71,6 +71,7 @@ case class ExuConfig
val hasRedirect = fuConfigs.map(_.hasRedirect).reduce(_ || _)
val hasFastUopOut = fuConfigs.map(_.fastUopOut).reduce(_ || _)
val exceptionOut = fuConfigs.map(_.exceptionOut).reduce(_ ++ _).distinct.sorted
val hasLoadError = fuConfigs.map(_.hasLoadError).reduce(_ || _)
val flushPipe: Boolean = fuConfigs.map(_.flushPipe).reduce(_ ||_)
val replayInst: Boolean = fuConfigs.map(_.replayInst).reduce(_ || _)
val trigger: Boolean = fuConfigs.map(_.trigger).reduce(_ || _)
......
......@@ -23,6 +23,7 @@ import difftest.{DifftestFpWriteback, DifftestIntWriteback}
import freechips.rocketchip.diplomacy.{LazyModule, LazyModuleImp}
import utils._
import xiangshan._
import xiangshan.ExceptionNO._
import xiangshan.backend.HasExuWbHelper
class ExuWbArbiter(n: Int, hasFastUopOut: Boolean, fastVec: Seq[Boolean])(implicit p: Parameters) extends XSModule {
......@@ -346,22 +347,37 @@ class Wb2Ctrl(configs: Seq[ExuConfig])(implicit p: Parameters) extends LazyModul
module.io.in := sink._1.zip(sink._2).zip(sourceMod).flatMap(x => x._1._1.writebackSource1(x._2)(x._1._2))
}
lazy val module = new LazyModuleImp(this) with HasWritebackSourceImp {
lazy val module = new LazyModuleImp(this)
with HasWritebackSourceImp
with HasXSParameter
{
val io = IO(new Bundle {
val redirect = Flipped(ValidIO(new Redirect))
val in = Vec(configs.length, Input(Decoupled(new ExuOutput)))
val out = Vec(configs.length, ValidIO(new ExuOutput))
val delayedLoadError = Vec(LoadPipelineWidth, Input(Bool())) // Dirty fix of data ecc error timing
})
for (((out, in), config) <- io.out.zip(io.in).zip(configs)) {
out.valid := in.fire
out.bits := in.bits
if (config.hasFastUopOut) {
if (config.hasFastUopOut || config.hasLoadError) {
out.valid := RegNext(in.fire && !in.bits.uop.robIdx.needFlush(io.redirect))
out.bits.uop := RegNext(in.bits.uop)
}
}
if(EnableAccurateLoadError){
for ((((out, in), config), delayed_error) <- io.out.zip(io.in).zip(configs)
.filter(_._2.hasLoadError)
.zip(io.delayedLoadError)
){
// overwrite load exception writeback
out.bits.uop.cf.exceptionVec(loadAccessFault) := delayed_error ||
RegNext(in.bits.uop.cf.exceptionVec(loadAccessFault))
}
}
override def writebackSource: Option[Seq[Seq[ValidIO[ExuOutput]]]] = Some(Seq(io.out))
}
......
......@@ -53,6 +53,7 @@ case class FuConfig
fastImplemented: Boolean = false,
hasInputBuffer: Boolean = false,
exceptionOut: Seq[Int] = Seq(),
hasLoadError: Boolean = false,
flushPipe: Boolean = false,
replayInst: Boolean = false,
trigger: Boolean = false
......
......@@ -257,7 +257,7 @@ class DCacheWordReqWithVaddr(implicit p: Parameters) extends DCacheWordReq {
val wline = Bool()
}
class DCacheWordResp(implicit p: Parameters) extends DCacheBundle
class BaseDCacheWordResp(implicit p: Parameters) extends DCacheBundle
{
val data = UInt(DataBits.W)
val id = UInt(reqIdWidth.W)
......@@ -268,13 +268,23 @@ class DCacheWordResp(implicit p: Parameters) extends DCacheBundle
val replay = Bool()
// data has been corrupted
val tag_error = Bool() // tag error
val error = Bool() // all kinds of errors, include tag error
def dump() = {
XSDebug("DCacheWordResp: data: %x id: %d miss: %b replay: %b\n",
data, id, miss, replay)
}
}
class DCacheWordResp(implicit p: Parameters) extends BaseDCacheWordResp
{
// 1 cycle after data resp
val error_delayed = Bool() // all kinds of errors, include tag error
}
class DCacheWordRespWithError(implicit p: Parameters) extends BaseDCacheWordResp
{
val error = Bool() // all kinds of errors, include tag error
}
class DCacheLineResp(implicit p: Parameters) extends DCacheBundle
{
val data = UInt((cfg.blockBytes * 8).W)
......@@ -317,10 +327,16 @@ class DCacheWordIO(implicit p: Parameters) extends DCacheBundle
val resp = Flipped(DecoupledIO(new DCacheWordResp))
}
class DCacheWordIOWithVaddr(implicit p: Parameters) extends DCacheBundle
class UncacheWordIO(implicit p: Parameters) extends DCacheBundle
{
val req = DecoupledIO(new DCacheWordReq)
val resp = Flipped(DecoupledIO(new DCacheWordRespWithError))
}
class AtomicWordIO(implicit p: Parameters) extends DCacheBundle
{
val req = DecoupledIO(new DCacheWordReqWithVaddr)
val resp = Flipped(DecoupledIO(new DCacheWordResp))
val resp = Flipped(DecoupledIO(new DCacheWordRespWithError))
}
// used by load unit
......@@ -359,7 +375,7 @@ class DCacheToLsuIO(implicit p: Parameters) extends DCacheBundle {
val load = Vec(LoadPipelineWidth, Flipped(new DCacheLoadIO)) // for speculative load
val lsq = ValidIO(new Refill) // refill to load queue, wake up load misses
val store = new DCacheToSbufferIO // for sbuffer
val atomics = Flipped(new DCacheWordIOWithVaddr) // atomics reqs
val atomics = Flipped(new AtomicWordIO) // atomics reqs
val release = ValidIO(new Release) // cacheline release hint for ld-ld violation check
}
......@@ -430,7 +446,7 @@ class DCacheImp(outer: DCache) extends LazyModuleImp(outer) with HasDCacheParame
val errors = ldu.map(_.io.error) ++ // load error
Seq(mainPipe.io.error) // store / misc error
io.error <> RegNext(Mux1H(errors.map(e => e.valid -> e)))
io.error <> RegNext(Mux1H(errors.map(e => RegNext(e.valid) -> RegNext(e))))
//----------------------------------------
// meta array
......@@ -483,12 +499,12 @@ class DCacheImp(outer: DCache) extends LazyModuleImp(outer) with HasDCacheParame
bankedDataArray.io.readline <> mainPipe.io.data_read
bankedDataArray.io.readline_intend := mainPipe.io.data_read_intend
mainPipe.io.readline_error := bankedDataArray.io.readline_error
mainPipe.io.readline_error_delayed := bankedDataArray.io.readline_error_delayed
mainPipe.io.data_resp := bankedDataArray.io.resp
(0 until LoadPipelineWidth).map(i => {
bankedDataArray.io.read(i) <> ldu(i).io.banked_data_read
bankedDataArray.io.read_error(i) <> ldu(i).io.read_error
bankedDataArray.io.read_error_delayed(i) <> ldu(i).io.read_error_delayed
ldu(i).io.banked_data_resp := bankedDataArray.io.resp
......
......@@ -32,8 +32,8 @@ class MMIOEntry(edge: TLEdgeOut)(implicit p: Parameters) extends DCacheModule
val id = Input(UInt())
// client requests
val req = Flipped(DecoupledIO(new DCacheWordReq ))
val resp = DecoupledIO(new DCacheWordResp)
val req = Flipped(DecoupledIO(new DCacheWordReq))
val resp = DecoupledIO(new DCacheWordRespWithError)
val mem_acquire = DecoupledIO(new TLBundleA(edge.bundle))
val mem_grant = Flipped(DecoupledIO(new TLBundleD(edge.bundle)))
......@@ -128,7 +128,7 @@ class MMIOEntry(edge: TLEdgeOut)(implicit p: Parameters) extends DCacheModule
io.resp.bits.miss := false.B
io.resp.bits.replay := false.B
io.resp.bits.tag_error := false.B
io.resp.bits.error := false.B
io.resp.bits.error := false.B
when (io.resp.fire()) {
state := s_invalid
......@@ -137,7 +137,7 @@ class MMIOEntry(edge: TLEdgeOut)(implicit p: Parameters) extends DCacheModule
}
class UncacheIO(implicit p: Parameters) extends DCacheBundle {
val lsq = Flipped(new DCacheWordIO)
val lsq = Flipped(new UncacheWordIO)
}
// convert DCacheIO to TileLink
......@@ -165,7 +165,7 @@ class UncacheImp(outer: Uncache)
val (bus, edge) = outer.clientNode.out.head
val resp_arb = Module(new Arbiter(new DCacheWordResp, 1))
val resp_arb = Module(new Arbiter(new DCacheWordRespWithError, 1))
val req = io.lsq.req
val resp = io.lsq.resp
......
......@@ -49,7 +49,7 @@ class L1BankedDataReadResult(implicit p: Parameters) extends DCacheBundle
// you can choose which bank to read to save power
val ecc = Bits(eccBits.W)
val raw_data = Bits(DCacheSRAMRowBits.W)
val error = Bool() // slow to generate, use it with care
val error_delayed = Bool() // 1 cycle later than data resp
def asECCData() = {
Cat(ecc, raw_data)
......@@ -78,8 +78,8 @@ abstract class AbstractBankedDataArray(implicit p: Parameters) extends DCacheMod
val resp = Output(Vec(DCacheBanks, new L1BankedDataReadResult()))
// val nacks = Output(Vec(LoadPipelineWidth, Bool()))
// val errors = Output(Vec(LoadPipelineWidth + 1, new L1CacheErrorInfo)) // read ports + readline port
val read_error = Output(Vec(LoadPipelineWidth, Bool()))
val readline_error = Output(Bool())
val read_error_delayed = Output(Vec(LoadPipelineWidth, Bool()))
val readline_error_delayed = Output(Bool())
// when bank_conflict, read (1) port should be ignored
val bank_conflict_slow = Output(Vec(LoadPipelineWidth, Bool()))
val bank_conflict_fast = Output(Vec(LoadPipelineWidth, Bool()))
......@@ -255,8 +255,8 @@ class BankedDataArray(implicit p: Parameters) extends AbstractBankedDataArray {
// read each bank, get bank result
val bank_result = Wire(Vec(DCacheBanks, new L1BankedDataReadResult()))
dontTouch(bank_result)
val read_bank_error = Wire(Vec(DCacheBanks, Bool()))
dontTouch(read_bank_error)
val read_bank_error_delayed = Wire(Vec(DCacheBanks, Bool()))
dontTouch(read_bank_error_delayed)
val rr_bank_conflict = Seq.tabulate(LoadPipelineWidth)(x => Seq.tabulate(LoadPipelineWidth)(y =>
bank_addrs(x) === bank_addrs(y) && io.read(x).valid && io.read(y).valid
))
......@@ -327,23 +327,26 @@ class BankedDataArray(implicit p: Parameters) extends AbstractBankedDataArray {
PriorityMux(Seq.tabulate(LoadPipelineWidth)(i => bank_addr_matchs(i) -> set_addrs(i)))
)
val read_enable = bank_addr_matchs.asUInt.orR || readline_match
// read raw data
val data_bank = data_banks(bank_index)
data_bank.io.r.en := bank_addr_matchs.asUInt.orR || readline_match
data_bank.io.r.en := read_enable
data_bank.io.r.way_en := bank_way_en
data_bank.io.r.addr := bank_set_addr
bank_result(bank_index).raw_data := data_bank.io.r.data
// read ECC
val ecc_bank = ecc_banks(bank_index)
ecc_bank.io.r.req.valid := bank_addr_matchs.asUInt.orR || readline_match
ecc_bank.io.r.req.valid := read_enable
ecc_bank.io.r.req.bits.apply(setIdx = bank_set_addr)
bank_result(bank_index).ecc := Mux1H(RegNext(bank_way_en), ecc_bank.io.r.resp.data)
// use ECC to check error
val data = bank_result(bank_index).asECCData()
bank_result(bank_index).error := dcacheParameters.dataCode.decode(data).error
read_bank_error(bank_index) := bank_result(bank_index).error
val ecc_data = bank_result(bank_index).asECCData()
val ecc_data_delayed = RegEnable(ecc_data, RegNext(read_enable))
bank_result(bank_index).error_delayed := dcacheParameters.dataCode.decode(ecc_data_delayed).error
read_bank_error_delayed(bank_index) := bank_result(bank_index).error_delayed
}
// read result: expose banked read result
......@@ -352,13 +355,13 @@ class BankedDataArray(implicit p: Parameters) extends AbstractBankedDataArray {
// error detection
// normal read ports
(0 until LoadPipelineWidth).map(rport_index => {
io.read_error(rport_index) := RegNext(io.read(rport_index).fire()) &&
read_bank_error(RegNext(bank_addrs(rport_index))) &&
!io.bank_conflict_slow(rport_index)
io.read_error_delayed(rport_index) := RegNext(RegNext(io.read(rport_index).fire())) &&
read_bank_error_delayed(RegNext(RegNext(bank_addrs(rport_index)))) &&
!RegNext(io.bank_conflict_slow(rport_index))
})
// readline port
io.readline_error := RegNext(io.readline.fire()) &&
VecInit((0 until DCacheBanks).map(i => io.resp(i).error)).asUInt().orR
io.readline_error_delayed := RegNext(RegNext(io.readline.fire())) &&
VecInit((0 until DCacheBanks).map(i => io.resp(i).error_delayed)).asUInt().orR
// write data_banks & ecc_banks
val sram_waddr = addr_to_dcache_set(io.write.bits.addr)
......
......@@ -40,7 +40,7 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
val banked_data_read = DecoupledIO(new L1BankedDataReadReq)
val banked_data_resp = Input(Vec(DCacheBanks, new L1BankedDataReadResult()))
val read_error = Input(Bool())
val read_error_delayed = Input(Bool())
// banked data read conflict
val bank_conflict_slow = Input(Bool())
......@@ -81,7 +81,7 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
// Tag read for new requests
meta_read.idx := get_idx(io.lsu.req.bits.addr)
meta_read.way_en := ~0.U(nWays.W)
// meta_read.tag := DontCare
// meta_read.tag := DontCare
tag_read.idx := get_idx(io.lsu.req.bits.addr)
tag_read.way_en := ~0.U(nWays.W)
......@@ -89,6 +89,9 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
// Pipeline
// --------------------------------------------------------------------------------
// stage 0
// --------------------------------------------------------------------------------
// read tag
val s0_valid = io.lsu.req.fire()
val s0_req = io.lsu.req.bits
val s0_fire = s0_valid && s1_ready
......@@ -98,6 +101,9 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
// --------------------------------------------------------------------------------
// stage 1
// --------------------------------------------------------------------------------
// tag match, read data
val s1_valid = RegInit(false.B)
val s1_req = RegEnable(s0_req, s0_fire)
// in stage 1, load unit gets the physical address
......@@ -164,6 +170,9 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
// --------------------------------------------------------------------------------
// stage 2
// --------------------------------------------------------------------------------
// return data
// val s2_valid = RegEnable(next = s1_valid && !io.lsu.s1_kill, init = false.B, enable = s1_fire)
val s2_valid = RegInit(false.B)
val s2_req = RegEnable(s1_req, s1_fire)
......@@ -172,6 +181,8 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
val s2_bank_oh = RegEnable(s1_bank_oh, s1_fire)
s2_ready := true.B
val s2_fire = s2_valid
when (s1_fire) { s2_valid := !io.lsu.s1_kill }
.elsewhen(io.lsu.resp.fire()) { s2_valid := false.B }
......@@ -208,8 +219,6 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
val s2_tag_error = RegEnable(s1_tag_error, s1_fire)
val s2_flag_error = RegEnable(s1_flag_error, s1_fire)
val s2_data_error = io.read_error // banked_data_resp_word.error && !bank_conflict_slow
val s2_error = RegEnable(s1_error, s1_fire) || s2_data_error
val s2_hit = s2_tag_match && s2_has_permission && s2_hit_coh === s2_new_hit_coh
......@@ -250,8 +259,7 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
resp.bits.miss := real_miss || io.bank_conflict_slow
// load pipe need replay when there is a bank conflict
resp.bits.replay := resp.bits.miss && (!io.miss_req.fire() || s2_nack) || io.bank_conflict_slow
resp.bits.tag_error := s2_tag_error
resp.bits.error := s2_error && (s2_hit || s2_tag_error)
resp.bits.tag_error := s2_tag_error // report tag_error in load s2
XSPerfAccumulate("dcache_read_bank_conflict", io.bank_conflict_slow && s2_valid)
......@@ -268,24 +276,41 @@ class LoadPipe(id: Int)(implicit p: Parameters) extends DCacheModule with HasPer
io.lsu.s1_bank_conflict := io.bank_conflict_fast
assert(RegNext(s1_ready && s2_ready), "load pipeline should never be blocked")
// update plru, report error in s3
// --------------------------------------------------------------------------------
// stage 3
// --------------------------------------------------------------------------------
// report ecc error
io.replace_access.valid := RegNext(RegNext(RegNext(io.meta_read.fire()) && s1_valid) && !s2_nack_no_mshr)
io.replace_access.bits.set := RegNext(RegNext(get_idx(s1_req.addr)))
io.replace_access.bits.way := RegNext(RegNext(Mux(s1_tag_match, OHToUInt(s1_tag_match_way), io.replace_way.way)))
val s3_valid = RegNext(s2_valid)
val s3_addr = RegEnable(s2_addr, s2_fire)
val s3_hit = RegEnable(s2_hit, s2_fire)
io.error := 0.U.asTypeOf(new L1CacheErrorInfo())
val s3_data_error = io.read_error_delayed // banked_data_resp_word.error && !bank_conflict
val s3_tag_error = RegEnable(s2_tag_error, s2_fire)
val s3_flag_error = RegEnable(s2_flag_error, s2_fire)
val s3_error = s3_tag_error || s3_flag_error || s3_data_error
// error_delayed signal will be used to update uop.exception 1 cycle after load writeback
resp.bits.error_delayed := s3_error && (s3_hit || s3_tag_error)
// report tag / data / l2 error (with paddr) to bus error unit
io.error.report_to_beu := RegNext((s2_tag_error || s2_data_error) && s2_valid)
io.error.paddr := RegNext(s2_addr)
io.error.source.tag := RegNext(s2_tag_error)
io.error.source.data := RegNext(s2_data_error)
io.error.source.l2 := RegNext(s2_flag_error)
io.error := 0.U.asTypeOf(new L1CacheErrorInfo())
io.error.report_to_beu := (s3_tag_error || s3_data_error) && s3_valid
io.error.paddr := s3_addr
io.error.source.tag := s3_tag_error
io.error.source.data := s3_data_error
io.error.source.l2 := s3_flag_error
io.error.opType.load := true.B
// report tag error / l2 corrupted to CACHE_ERROR csr
io.error.valid := RegNext(s2_error && s2_valid)
io.error.valid := s3_error && s3_valid
// update plru, report error in s3
// -------
io.replace_access.valid := RegNext(RegNext(RegNext(io.meta_read.fire()) && s1_valid) && !s2_nack_no_mshr)
io.replace_access.bits.set := RegNext(RegNext(get_idx(s1_req.addr)))
io.replace_access.bits.way := RegNext(RegNext(Mux(s1_tag_match, OHToUInt(s1_tag_match_way), io.replace_way.way)))
// --------------------------------------------------------------------------------
// Debug logging functions
def dump_pipeline_reqs(pipeline_stage_name: String, valid: Bool,
req: DCacheWordReq ) = {
......
......@@ -36,7 +36,7 @@ class AtomicsResp(implicit p: Parameters) extends DCacheBundle {
class AtomicsReplayEntry(implicit p: Parameters) extends DCacheModule
{
val io = IO(new Bundle {
val lsu = Flipped(new DCacheWordIOWithVaddr)
val lsu = Flipped(new AtomicWordIO)
val pipe_req = Decoupled(new MainPipeReq)
val pipe_resp = Flipped(ValidIO(new AtomicsResp))
val block_lr = Input(Bool())
......
......@@ -113,7 +113,7 @@ class MainPipe(implicit p: Parameters) extends DCacheModule with HasPerfEvents {
val data_read_intend = Output(Bool())
val data_read = DecoupledIO(new L1BankedDataReadLineReq)
val data_resp = Input(Vec(DCacheBanks, new L1BankedDataReadResult()))
val readline_error = Input(Bool())
val readline_error_delayed = Input(Bool())
val data_write = DecoupledIO(new L1BankedDataWriteReq)
val meta_read = DecoupledIO(new MetaReadReq)
......@@ -328,9 +328,9 @@ class MainPipe(implicit p: Parameters) extends DCacheModule with HasPerfEvents {
val s2_flag_error = RegEnable(s1_flag_error, s1_fire)
val s2_tag_error = RegEnable(s1_tag_error, s1_fire)
val s2_l2_error = s2_req.error
// s2_data_error will be reported by data array
val s2_data_error = io.readline_error && s2_need_data && s2_coh.state =/= ClientStates.Nothing
val s2_error = s2_flag_error || s2_tag_error || s2_data_error || s2_l2_error
val s2_error = s2_flag_error || s2_tag_error || s2_l2_error // data_error not included
val s2_may_report_data_error = s2_need_data && s2_coh.state =/= ClientStates.Nothing
val s2_hit = s2_tag_match && s2_has_permission
val s2_amo_hit = s2_hit && !s2_req.probe && !s2_req.miss && s2_req.isAMO
......@@ -396,7 +396,15 @@ class MainPipe(implicit p: Parameters) extends DCacheModule with HasPerfEvents {
val s3_data_word = RegEnable(s2_data_word, s2_fire_to_s3)
val s3_data = RegEnable(s2_data, s2_fire_to_s3)
val s3_l2_error = s3_req.error
val s3_error = RegEnable(s2_error, s2_fire_to_s3)
// data_error will be reported by data array 1 cycle after data read resp
val s3_data_error = Wire(Bool())
s3_data_error := Mux(RegNext(RegNext(s1_fire)), // ecc check result is generated 2 cycle after read req
io.readline_error_delayed && RegNext(s2_may_report_data_error),
RegNext(s3_data_error) // do not update s3_data_error if !s1_fire
)
// error signal for amo inst
// s3_error = s3_flag_error || s3_tag_error || s3_l2_error || s3_data_error
val s3_error = RegEnable(s2_error, s2_fire_to_s3) || s3_data_error
val (probe_has_dirty_data, probe_shrink_param, probe_new_coh) = s3_coh.onProbe(s3_req.probe_param)
val s3_need_replacement = RegEnable(s2_need_replacement, s2_fire_to_s3)
......@@ -745,17 +753,21 @@ class MainPipe(implicit p: Parameters) extends DCacheModule with HasPerfEvents {
io.status.s3.bits.set := s3_idx
io.status.s3.bits.way_en := s3_way_en
// report error to beu and csr, 1 cycle after read data resp
io.error := 0.U.asTypeOf(new L1CacheErrorInfo())
io.error.report_to_beu := RegNext((s2_tag_error || s2_data_error) && s2_fire)
io.error.paddr := RegNext(s2_req.addr)
io.error.source.tag := RegNext(s2_tag_error)
io.error.source.data := RegNext(s2_data_error)
io.error.source.l2 := RegNext(s2_flag_error || s2_l2_error)
io.error.opType.store := RegNext(s2_req.isStore && !s2_req.probe)
io.error.opType.probe := RegNext(s2_req.probe)
io.error.opType.release := RegNext(s2_req.replace)
io.error.opType.atom := RegNext(s2_req.isAMO && !s2_req.probe)
io.error.valid := RegNext(s2_error && s2_fire)
// report error, update error csr
io.error.valid := s3_error && RegNext(s2_fire)
// only tag_error and data_error will be reported to beu
// l2_error should not be reported (l2 will report that)
io.error.report_to_beu := (RegEnable(s2_tag_error, s2_fire) || s3_data_error) && RegNext(s2_fire)
io.error.paddr := RegEnable(s2_req.addr, s2_fire)
io.error.source.tag := RegEnable(s2_tag_error, s2_fire)
io.error.source.data := s3_data_error
io.error.source.l2 := RegEnable(s2_flag_error || s2_l2_error, s2_fire)
io.error.opType.store := RegEnable(s2_req.isStore && !s2_req.probe, s2_fire)
io.error.opType.probe := RegEnable(s2_req.probe, s2_fire)
io.error.opType.release := RegEnable(s2_req.replace, s2_fire)
io.error.opType.atom := RegEnable(s2_req.isAMO && !s2_req.probe, s2_fire)
val perfEvents = Seq(
("dcache_mp_req ", s0_fire ),
......
......@@ -62,6 +62,7 @@ class LsqWrappper(implicit p: Parameters) extends XSModule with HasDCacheParamet
val storeInRe = Vec(StorePipelineWidth, Input(new LsPipelineBundle()))
val storeDataIn = Vec(StorePipelineWidth, Flipped(Valid(new ExuOutput))) // store data, send to sq from rs
val loadDataForwarded = Vec(LoadPipelineWidth, Input(Bool()))
val delayedLoadError = Vec(LoadPipelineWidth, Input(Bool()))
val dcacheRequireReplay = Vec(LoadPipelineWidth, Input(Bool()))
val sbuffer = Vec(EnsbufferWidth, Decoupled(new DCacheWordReqWithVaddr))
val ldout = Vec(LoadPipelineWidth, DecoupledIO(new ExuOutput)) // writeback int load
......@@ -72,7 +73,7 @@ class LsqWrappper(implicit p: Parameters) extends XSModule with HasDCacheParamet
val rollback = Output(Valid(new Redirect))
val refill = Flipped(ValidIO(new Refill))
val release = Flipped(ValidIO(new Release))
val uncache = new DCacheWordIO
val uncache = new UncacheWordIO
val exceptionAddr = new ExceptionAddrIO
val sqempty = Output(Bool())
val issuePtrExt = Output(new SqPtr)
......@@ -116,6 +117,7 @@ class LsqWrappper(implicit p: Parameters) extends XSModule with HasDCacheParamet
loadQueue.io.loadIn <> io.loadIn
loadQueue.io.storeIn <> io.storeIn
loadQueue.io.loadDataForwarded <> io.loadDataForwarded
loadQueue.io.delayedLoadError <> io.delayedLoadError
loadQueue.io.dcacheRequireReplay <> io.dcacheRequireReplay
loadQueue.io.ldout <> io.ldout
loadQueue.io.rob <> io.rob
......
......@@ -25,7 +25,7 @@ import xiangshan.backend.fu.fpu.FPU
import xiangshan.backend.rob.RobLsqIO
import xiangshan.cache._
import xiangshan.frontend.FtqPtr
import xiangshan.ExceptionNO._
class LqPtr(implicit p: Parameters) extends CircularQueuePtr[LqPtr](
p => p(XSCoreParamsKey).LoadQueueSize
......@@ -87,6 +87,7 @@ class LoadQueue(implicit p: Parameters) extends XSModule
val loadIn = Vec(LoadPipelineWidth, Flipped(Valid(new LsPipelineBundle)))
val storeIn = Vec(StorePipelineWidth, Flipped(Valid(new LsPipelineBundle)))
val loadDataForwarded = Vec(LoadPipelineWidth, Input(Bool()))
val delayedLoadError = Vec(LoadPipelineWidth, Input(Bool()))
val dcacheRequireReplay = Vec(LoadPipelineWidth, Input(Bool()))
val ldout = Vec(LoadPipelineWidth, DecoupledIO(new ExuOutput)) // writeback int load
val load_s1 = Vec(LoadPipelineWidth, Flipped(new PipeLoadForwardQueryIO)) // TODO: to be renamed
......@@ -95,7 +96,7 @@ class LoadQueue(implicit p: Parameters) extends XSModule
val rollback = Output(Valid(new Redirect)) // replay now starts from load instead of store
val refill = Flipped(ValidIO(new Refill))
val release = Flipped(ValidIO(new Release))
val uncache = new DCacheWordIO
val uncache = new UncacheWordIO
val exceptionAddr = new ExceptionAddrIO
val lqFull = Output(Bool())
val lqCancelCnt = Output(UInt(log2Up(LoadQueueSize + 1).W))
......@@ -292,6 +293,8 @@ class LoadQueue(implicit p: Parameters) extends XSModule
for (i <- 0 until LoadPipelineWidth) {
val loadWbIndex = io.loadIn(i).bits.uop.lqIdx.value
val lastCycleLoadWbIndex = RegNext(loadWbIndex)
// update miss state in load s3
if(!EnableFastForward){
// dcacheRequireReplay will be used to update lq flag 1 cycle after for better timing
//
......@@ -299,13 +302,18 @@ class LoadQueue(implicit p: Parameters) extends XSModule
when(dcacheRequireReplay(i) && !refill_addr_hit(RegNext(io.loadIn(i).bits.paddr), io.refill.bits.addr)) {
// do not writeback if that inst will be resend from rs
// rob writeback will not be triggered by a refill before inst replay
miss(RegNext(loadWbIndex)) := false.B // disable refill listening
datavalid(RegNext(loadWbIndex)) := false.B // disable refill listening
assert(!datavalid(RegNext(loadWbIndex)))
miss(lastCycleLoadWbIndex) := false.B // disable refill listening
datavalid(lastCycleLoadWbIndex) := false.B // disable refill listening
assert(!datavalid(lastCycleLoadWbIndex))
}
}
// update load error state in load s3
when(RegNext(io.loadIn(i).fire()) && io.delayedLoadError(i)){
uop(lastCycleLoadWbIndex).cf.exceptionVec(loadAccessFault) := true.B
}
}
// Writeback up to 2 missed load insts to CDB
//
// Pick 2 missed load (data refilled), write them back to cdb
......
......@@ -72,7 +72,7 @@ class StoreQueue(implicit p: Parameters) extends XSModule
val mmioStout = DecoupledIO(new ExuOutput) // writeback uncached store
val forward = Vec(LoadPipelineWidth, Flipped(new PipeLoadForwardQueryIO))
val rob = Flipped(new RobLsqIO)
val uncache = new DCacheWordIO
val uncache = new UncacheWordIO
// val refill = Flipped(Valid(new DCacheLineReq ))
val exceptionAddr = new ExceptionAddrIO
val sqempty = Output(Bool())
......
......@@ -21,7 +21,7 @@ import chisel3._
import chisel3.util._
import utils._
import xiangshan._
import xiangshan.cache.{DCacheWordIOWithVaddr, MemoryOpConstants}
import xiangshan.cache.{AtomicWordIO, MemoryOpConstants}
import xiangshan.cache.mmu.{TlbCmd, TlbRequestIO}
import difftest._
import xiangshan.ExceptionNO._
......@@ -33,7 +33,7 @@ class AtomicsUnit(implicit p: Parameters) extends XSModule with MemoryOpConstant
val in = Flipped(Decoupled(new ExuInput))
val storeDataIn = Flipped(Valid(new ExuOutput)) // src2 from rs
val out = Decoupled(new ExuOutput)
val dcache = new DCacheWordIOWithVaddr
val dcache = new AtomicWordIO
val dtlb = new TlbRequestIO
val pmpResp = Flipped(new PMPRespBundle())
val rsIdx = Input(UInt(log2Up(IssQueSize).W))
......
......@@ -30,6 +30,7 @@ class LoadToLsqIO(implicit p: Parameters) extends XSBundle {
val loadIn = ValidIO(new LsPipelineBundle)
val ldout = Flipped(DecoupledIO(new ExuOutput))
val loadDataForwarded = Output(Bool())
val delayedLoadError = Output(Bool())
val dcacheRequireReplay = Output(Bool())
val forward = new PipeLoadForwardQueryIO
val loadViolationQuery = new LoadViolationQueryIO
......@@ -283,6 +284,7 @@ class LoadUnit_S2(implicit p: Parameters) extends XSModule with HasLoadHelper {
val fullForward = Output(Bool())
val fastpath = Output(new LoadToLoadIO)
val dcache_kill = Output(Bool())
val delayedLoadError = Output(Bool())
val loadViolationQueryResp = Flipped(Valid(new LoadViolationQueryResp))
val csrCtrl = Flipped(new CustomCSRCtrlIO)
val sentFastUop = Input(Bool())
......@@ -311,18 +313,21 @@ class LoadUnit_S2(implicit p: Parameters) extends XSModule with HasLoadHelper {
}
val s2_exception = ExceptionNO.selectByFu(s2_exception_vec, lduCfg).asUInt.orR
// s2_exception_vec add exception caused by ecc error
// writeback access fault caused by ecc error / bus error
//
// ecc data error is slow to generate, so we will not use it until the last moment
// (s2_exception_with_error_vec is the final output: io.out.bits.uop.cf.exceptionVec)
val s2_exception_with_error_vec = WireInit(s2_exception_vec)
// * ecc data error is slow to generate, so we will not use it until load stage 3
// * in load stage 3, an extra signal io.load_error will be used to
// now cache ecc error will raise an access fault
// at the same time, error info (including error paddr) will be write to
// an customized CSR "CACHE_ERROR"
s2_exception_with_error_vec(loadAccessFault) := s2_exception_vec(loadAccessFault) ||
io.dcacheResp.bits.error &&
io.csrCtrl.cache_error_enable
val debug_s2_exception_with_error = ExceptionNO.selectByFu(s2_exception_with_error_vec, lduCfg).asUInt.orR
if (EnableAccurateLoadError) {
io.delayedLoadError := io.dcacheResp.bits.error_delayed &&
io.csrCtrl.cache_error_enable &&
RegNext(io.out.valid)
} else {
io.delayedLoadError := false.B
}
val actually_mmio = pmp.mmio
val s2_uop = io.in.bits.uop
......@@ -333,7 +338,6 @@ class LoadUnit_S2(implicit p: Parameters) extends XSModule with HasLoadHelper {
val s2_cache_miss = io.dcacheResp.bits.miss
val s2_cache_replay = io.dcacheResp.bits.replay
val s2_cache_tag_error = io.dcacheResp.bits.tag_error
val s2_cache_error = io.dcacheResp.bits.error
val s2_forward_fail = io.lsq.matchInvalid || io.sbuffer.matchInvalid
val s2_ldld_violation = io.loadViolationQueryResp.valid &&
io.loadViolationQueryResp.bits.have_violation &&
......@@ -412,7 +416,7 @@ class LoadUnit_S2(implicit p: Parameters) extends XSModule with HasLoadHelper {
io.out.bits.uop.ctrl.replayInst := s2_need_replay_from_fetch
io.out.bits.mmio := s2_mmio
io.out.bits.uop.ctrl.flushPipe := s2_mmio && io.sentFastUop
io.out.bits.uop.cf.exceptionVec := s2_exception_with_error_vec
io.out.bits.uop.cf.exceptionVec := s2_exception_vec // cache error not included
// For timing reasons, sometimes we can not let
// io.out.bits.miss := s2_cache_miss && !s2_exception && !fullForward
......@@ -525,6 +529,9 @@ class LoadUnit(implicit p: Parameters) extends XSModule
val fastpathIn = Input(Vec(LoadPipelineWidth, new LoadToLoadIO))
val loadFastMatch = Input(UInt(exuParameters.LduCnt.W))
val delayedLoadError = Output(Bool()) // load ecc error
// Note that io.delayedLoadError and io.lsq.delayedLoadError is different
val csrCtrl = Flipped(new CustomCSRCtrlIO)
})
......@@ -573,7 +580,10 @@ class LoadUnit(implicit p: Parameters) extends XSModule
load_s2.io.loadViolationQueryResp <> io.lsq.loadViolationQuery.resp
load_s2.io.csrCtrl <> io.csrCtrl
load_s2.io.sentFastUop := RegEnable(io.fastUop.valid, load_s1.io.out.fire()) // RegNext is also ok
// actually load s3
io.lsq.dcacheRequireReplay := load_s2.io.dcacheRequireReplay
io.lsq.delayedLoadError := load_s2.io.delayedLoadError
// feedback tlb miss / dcache miss queue full
io.feedbackSlow.valid := RegNext(load_s2.io.rsFeedback.valid && !load_s2.io.out.bits.uop.robIdx.needFlush(io.redirect))
......@@ -644,6 +654,8 @@ class LoadUnit(implicit p: Parameters) extends XSModule
io.ldout.bits := Mux(hitLoadOut.valid, hitLoadOut.bits, io.lsq.ldout.bits)
io.ldout.valid := hitLoadOut.valid || io.lsq.ldout.valid
io.delayedLoadError := hitLoadOut.valid && load_s2.io.delayedLoadError
io.lsq.ldout.ready := !hitLoadOut.valid
when(io.feedbackSlow.valid && !io.feedbackSlow.bits.hit){
......
......@@ -730,7 +730,8 @@ package object xiangshan {
latency = UncertainLatency(),
exceptionOut = Seq(loadAddrMisaligned, loadAccessFault, loadPageFault),
flushPipe = true,
replayInst = true
replayInst = true,
hasLoadError = true
)
val staCfg = FuConfig(
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册