wip
This commit is contained in:
parent
d9f2e03753
commit
e00c0750eb
|
@ -122,7 +122,7 @@ object TestsWorkspace {
|
||||||
),
|
),
|
||||||
// new DivPlugin,
|
// new DivPlugin,
|
||||||
new CsrPlugin(CsrPluginConfig.all(0x80000020l).copy(deterministicInteruptionEntry = false)),
|
new CsrPlugin(CsrPluginConfig.all(0x80000020l).copy(deterministicInteruptionEntry = false)),
|
||||||
new DebugPlugin(ClockDomain.current.clone(reset = Bool().setName("debugReset"))),
|
// new DebugPlugin(ClockDomain.current.clone(reset = Bool().setName("debugReset"))),
|
||||||
new BranchPlugin(
|
new BranchPlugin(
|
||||||
earlyBranch = true,
|
earlyBranch = true,
|
||||||
catchAddressMisaligned = true,
|
catchAddressMisaligned = true,
|
||||||
|
|
|
@ -81,7 +81,7 @@ case class InstructionCacheCpuDecode(p : InstructionCacheConfig) extends Bundle
|
||||||
val data = ifGen(p.dataOnDecode) (Bits(p.cpuDataWidth bits))
|
val data = ifGen(p.dataOnDecode) (Bits(p.cpuDataWidth bits))
|
||||||
val error = Bool
|
val error = Bool
|
||||||
val mmuMiss = Bool
|
val mmuMiss = Bool
|
||||||
val illegalAccess =Bool
|
val illegalAccess = Bool
|
||||||
|
|
||||||
override def asMaster(): Unit = {
|
override def asMaster(): Unit = {
|
||||||
out(isValid, isUser, isStuck, pc)
|
out(isValid, isUser, isStuck, pc)
|
||||||
|
|
|
@ -0,0 +1,327 @@
|
||||||
|
package vexriscv.plugin
|
||||||
|
|
||||||
|
import vexriscv._
|
||||||
|
import spinal.core._
|
||||||
|
import spinal.lib._
|
||||||
|
import vexriscv.Riscv.IMM
|
||||||
|
import StreamVexPimper._
|
||||||
|
import scala.collection.mutable.ArrayBuffer
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
abstract class IBusFetcherImpl(catchAccessFault : Boolean,
|
||||||
|
pendingMax : Int = 7,
|
||||||
|
resetVector : BigInt,
|
||||||
|
keepPcPlus4 : Boolean,
|
||||||
|
decodePcGen : Boolean,
|
||||||
|
compressedGen : Boolean,
|
||||||
|
cmdToRspStageCount : Int,
|
||||||
|
rspStageGen : Boolean,
|
||||||
|
injectorReadyCutGen : Boolean,
|
||||||
|
relaxedPcCalculation : Boolean,
|
||||||
|
prediction : BranchPrediction,
|
||||||
|
catchAddressMisaligned : Boolean,
|
||||||
|
injectorStage : Boolean) extends Plugin[VexRiscv] with JumpService with IBusFetcher{
|
||||||
|
var prefetchExceptionPort : Flow[ExceptionCause] = null
|
||||||
|
|
||||||
|
var decodePrediction : DecodePredictionBus = null
|
||||||
|
assert(cmdToRspStageCount >= 1)
|
||||||
|
assert(!(compressedGen && !decodePcGen))
|
||||||
|
var fetcherHalt : Bool = null
|
||||||
|
lazy val decodeNextPcValid = Bool
|
||||||
|
lazy val decodeNextPc = UInt(32 bits)
|
||||||
|
def nextPc() = (decodeNextPcValid, decodeNextPc)
|
||||||
|
|
||||||
|
var predictionJumpInterface : Flow[UInt] = null
|
||||||
|
|
||||||
|
override def haltIt(): Unit = fetcherHalt := True
|
||||||
|
|
||||||
|
case class JumpInfo(interface : Flow[UInt], stage: Stage, priority : Int)
|
||||||
|
val jumpInfos = ArrayBuffer[JumpInfo]()
|
||||||
|
override def createJumpInterface(stage: Stage, priority : Int = 0): Flow[UInt] = {
|
||||||
|
val interface = Flow(UInt(32 bits))
|
||||||
|
jumpInfos += JumpInfo(interface,stage, priority)
|
||||||
|
interface
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
var decodeExceptionPort : Flow[ExceptionCause] = null
|
||||||
|
override def setup(pipeline: VexRiscv): Unit = {
|
||||||
|
fetcherHalt = False
|
||||||
|
if(catchAccessFault || catchAddressMisaligned) {
|
||||||
|
val exceptionService = pipeline.service(classOf[ExceptionService])
|
||||||
|
decodeExceptionPort = exceptionService.newExceptionPort(pipeline.decode,1).setName("iBusErrorExceptionnPort")
|
||||||
|
}
|
||||||
|
|
||||||
|
pipeline(RVC_GEN) = compressedGen
|
||||||
|
|
||||||
|
prediction match {
|
||||||
|
case NONE =>
|
||||||
|
case STATIC | DYNAMIC => {
|
||||||
|
predictionJumpInterface = createJumpInterface(pipeline.decode)
|
||||||
|
decodePrediction = pipeline.service(classOf[PredictionInterface]).askDecodePrediction()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
class FetchArea(pipeline : VexRiscv) extends Area {
|
||||||
|
import pipeline._
|
||||||
|
import pipeline.config._
|
||||||
|
|
||||||
|
//JumpService hardware implementation
|
||||||
|
val jump = new Area {
|
||||||
|
val sortedByStage = jumpInfos.sortWith((a, b) => {
|
||||||
|
(pipeline.indexOf(a.stage) > pipeline.indexOf(b.stage)) ||
|
||||||
|
(pipeline.indexOf(a.stage) == pipeline.indexOf(b.stage) && a.priority > b.priority)
|
||||||
|
})
|
||||||
|
val valids = sortedByStage.map(_.interface.valid)
|
||||||
|
val pcs = sortedByStage.map(_.interface.payload)
|
||||||
|
|
||||||
|
val pcLoad = Flow(UInt(32 bits))
|
||||||
|
pcLoad.valid := jumpInfos.map(_.interface.valid).orR
|
||||||
|
pcLoad.payload := MuxOH(OHMasking.first(valids.asBits), pcs)
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
val killLastStage = jump.pcLoad.valid || decode.arbitration.isRemoved
|
||||||
|
def flush = killLastStage
|
||||||
|
|
||||||
|
class PcFetch extends Area{
|
||||||
|
val output = Stream(UInt(32 bits))
|
||||||
|
}
|
||||||
|
|
||||||
|
val fetchPc = if(relaxedPcCalculation) new PcFetch {
|
||||||
|
//PC calculation without Jump
|
||||||
|
val pcReg = Reg(UInt(32 bits)) init (resetVector) addAttribute (Verilator.public)
|
||||||
|
val pcPlus4 = pcReg + 4
|
||||||
|
if (keepPcPlus4) KeepAttribute(pcPlus4)
|
||||||
|
when(output.fire) {
|
||||||
|
pcReg := pcPlus4
|
||||||
|
}
|
||||||
|
|
||||||
|
//Realign
|
||||||
|
if(compressedGen){
|
||||||
|
when(output.fire){
|
||||||
|
pcReg(1 downto 0) := 0
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
//application of the selected jump request
|
||||||
|
when(jump.pcLoad.valid) {
|
||||||
|
pcReg := jump.pcLoad.payload
|
||||||
|
}
|
||||||
|
|
||||||
|
output.valid := RegNext(True) init (False) // && !jump.pcLoad.valid
|
||||||
|
output.payload := pcReg
|
||||||
|
} else new PcFetch{
|
||||||
|
//PC calculation without Jump
|
||||||
|
val pcReg = Reg(UInt(32 bits)) init(resetVector) addAttribute(Verilator.public)
|
||||||
|
val inc = RegInit(False)
|
||||||
|
|
||||||
|
val pc = pcReg + (inc ## B"00").asUInt
|
||||||
|
val samplePcNext = False
|
||||||
|
|
||||||
|
when(jump.pcLoad.valid) {
|
||||||
|
inc := False
|
||||||
|
samplePcNext := True
|
||||||
|
pc := jump.pcLoad.payload
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
when(output.fire){
|
||||||
|
inc := True
|
||||||
|
samplePcNext := True
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
when(samplePcNext) {
|
||||||
|
pcReg := pc
|
||||||
|
}
|
||||||
|
|
||||||
|
if(compressedGen) {
|
||||||
|
when(output.fire) {
|
||||||
|
pcReg(1 downto 0) := 0
|
||||||
|
when(pc(1)){
|
||||||
|
inc := True
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
output.valid := RegNext(True) init (False)
|
||||||
|
output.payload := pc
|
||||||
|
}
|
||||||
|
|
||||||
|
val decodePc = ifGen(decodePcGen)(new Area {
|
||||||
|
//PC calculation without Jump
|
||||||
|
val pcReg = Reg(UInt(32 bits)) init (resetVector) addAttribute (Verilator.public)
|
||||||
|
val pcPlus = if(compressedGen)
|
||||||
|
pcReg + ((decode.input(IS_RVC)) ? U(2) | U(4))
|
||||||
|
else
|
||||||
|
pcReg + 4
|
||||||
|
|
||||||
|
if (keepPcPlus4) KeepAttribute(pcPlus)
|
||||||
|
when(decode.arbitration.isFiring) {
|
||||||
|
pcReg := pcPlus
|
||||||
|
}
|
||||||
|
|
||||||
|
//application of the selected jump request
|
||||||
|
when(jump.pcLoad.valid) {
|
||||||
|
pcReg := jump.pcLoad.payload
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
|
||||||
|
val iBusCmd = new Area {
|
||||||
|
def input = fetchPc.output
|
||||||
|
|
||||||
|
// ...
|
||||||
|
|
||||||
|
val output = Stream(UInt(32 bits))
|
||||||
|
}
|
||||||
|
|
||||||
|
case class FetchRsp() extends Bundle {
|
||||||
|
val pc = UInt(32 bits)
|
||||||
|
val rsp = IBusSimpleRsp()
|
||||||
|
val isRvc = Bool
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
val iBusRsp = new Area {
|
||||||
|
val inputStages = Vec(Stream(UInt(32 bits)), cmdToRspStageCount)
|
||||||
|
for(i <- 0 until cmdToRspStageCount) {
|
||||||
|
// val doFlush = if(i == cmdToRspStageCount- 1 && ???) killLastStage else flush
|
||||||
|
inputStages(i) << {i match {
|
||||||
|
case 0 => iBusCmd.output.m2sPipeWithFlush(flush, relaxedPcCalculation)
|
||||||
|
case _ => inputStages(i-1)/*.haltWhen(fetcherHalt)*/.m2sPipeWithFlush(flush)
|
||||||
|
}}
|
||||||
|
}
|
||||||
|
|
||||||
|
def input = inputStages.last
|
||||||
|
|
||||||
|
// ...
|
||||||
|
|
||||||
|
val join = Stream(FetchRsp())
|
||||||
|
val output = if(rspStageGen) join.m2sPipeWithFlush(flush) else join
|
||||||
|
}
|
||||||
|
|
||||||
|
val decompressor = ifGen(decodePcGen)(new Area{
|
||||||
|
def input = iBusRsp.output
|
||||||
|
val output = Stream(FetchRsp())
|
||||||
|
|
||||||
|
val bufferValid = RegInit(False)
|
||||||
|
val bufferError = Reg(Bool)
|
||||||
|
val bufferData = Reg(Bits(16 bits))
|
||||||
|
|
||||||
|
val raw = Mux(
|
||||||
|
sel = bufferValid,
|
||||||
|
whenTrue = input.rsp.inst(15 downto 0) ## bufferData,
|
||||||
|
whenFalse = input.rsp.inst(31 downto 16) ## (input.pc(1) ? input.rsp.inst(31 downto 16) | input.rsp.inst(15 downto 0))
|
||||||
|
)
|
||||||
|
val isRvc = raw(1 downto 0) =/= 3
|
||||||
|
val decompressed = RvcDecompressor(raw(15 downto 0))
|
||||||
|
output.valid := isRvc ? (bufferValid || input.valid) | (input.valid && (bufferValid || !input.pc(1)))
|
||||||
|
output.pc := input.pc
|
||||||
|
output.isRvc := isRvc
|
||||||
|
output.rsp.inst := isRvc ? decompressed | raw
|
||||||
|
output.rsp.error := (bufferValid && bufferError) || (input.valid && input.rsp.error && (!isRvc || (isRvc && !bufferValid)))
|
||||||
|
input.ready := (bufferValid ? (!isRvc && output.ready) | (input.pc(1) || output.ready))
|
||||||
|
|
||||||
|
|
||||||
|
bufferValid clearWhen(output.fire)
|
||||||
|
when(input.ready){
|
||||||
|
when(input.valid) {
|
||||||
|
bufferValid := !(!isRvc && !input.pc(1) && !bufferValid) && !(isRvc && input.pc(1))
|
||||||
|
bufferError := input.rsp.error
|
||||||
|
bufferData := input.rsp.inst(31 downto 16)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
bufferValid.clearWhen(flush)
|
||||||
|
})
|
||||||
|
|
||||||
|
def condApply[T](that : T, cond : Boolean)(func : (T) => T) = if(cond)func(that) else that
|
||||||
|
val injector = new Area {
|
||||||
|
val inputBeforeHalt = condApply(if(decodePcGen) decompressor.output else iBusRsp.output, injectorReadyCutGen)(_.s2mPipe(flush))
|
||||||
|
val decodeInput = if(injectorStage){
|
||||||
|
val inputBeforeStage = inputBeforeHalt.haltWhen(fetcherHalt)
|
||||||
|
val decodeInput = inputBeforeStage.m2sPipeWithFlush(killLastStage)
|
||||||
|
decode.insert(INSTRUCTION_ANTICIPATED) := Mux(decode.arbitration.isStuck, decode.input(INSTRUCTION), inputBeforeStage.rsp.inst)
|
||||||
|
decodeInput
|
||||||
|
} else {
|
||||||
|
inputBeforeHalt
|
||||||
|
}
|
||||||
|
|
||||||
|
if(decodePcGen){
|
||||||
|
decodeNextPcValid := True
|
||||||
|
decodeNextPc := decodePc.pcReg
|
||||||
|
}else {
|
||||||
|
val lastStageStream = if(injectorStage) inputBeforeHalt
|
||||||
|
else if(rspStageGen) iBusRsp.join
|
||||||
|
else if(cmdToRspStageCount > 1)iBusRsp.inputStages(cmdToRspStageCount-2)
|
||||||
|
else throw new Exception("Fetch should at least have two stages")
|
||||||
|
|
||||||
|
// when(fetcherHalt){
|
||||||
|
// lastStageStream.valid := False
|
||||||
|
// lastStageStream.ready := False
|
||||||
|
// }
|
||||||
|
decodeNextPcValid := RegNext(lastStageStream.isStall)
|
||||||
|
decodeNextPc := decode.input(PC)
|
||||||
|
}
|
||||||
|
|
||||||
|
decodeInput.ready := !decode.arbitration.isStuck
|
||||||
|
decode.arbitration.isValid := decodeInput.valid
|
||||||
|
decode.insert(PC) := (if(decodePcGen) decodePc.pcReg else decodeInput.pc)
|
||||||
|
decode.insert(INSTRUCTION) := decodeInput.rsp.inst
|
||||||
|
decode.insert(INSTRUCTION_READY) := True
|
||||||
|
if(compressedGen) decode.insert(IS_RVC) := decodeInput.isRvc
|
||||||
|
|
||||||
|
if(catchAccessFault){
|
||||||
|
decodeExceptionPort.valid := decode.arbitration.isValid && decodeInput.rsp.error
|
||||||
|
decodeExceptionPort.code := 1
|
||||||
|
decodeExceptionPort.badAddr := decode.input(PC)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
prediction match {
|
||||||
|
case NONE =>
|
||||||
|
case STATIC | DYNAMIC => {
|
||||||
|
def historyWidth = 2
|
||||||
|
def historyRamSizeLog2 = 10
|
||||||
|
// if(prediction == DYNAMIC) {
|
||||||
|
// case class BranchPredictorLine() extends Bundle{
|
||||||
|
// val history = SInt(historyWidth bits)
|
||||||
|
// }
|
||||||
|
//
|
||||||
|
// val historyCache = if(prediction == DYNAMIC) Mem(BranchPredictorLine(), 1 << historyRamSizeLog2) setName("branchCache") else null
|
||||||
|
// val historyCacheWrite = if(prediction == DYNAMIC) historyCache.writePort else null
|
||||||
|
//
|
||||||
|
//
|
||||||
|
// val readAddress = (2, historyRamSizeLog2 bits)
|
||||||
|
// fetch.insert(HISTORY_LINE) := historyCache.readSync(readAddress,!prefetch.arbitration.isStuckByOthers)
|
||||||
|
//
|
||||||
|
// }
|
||||||
|
|
||||||
|
|
||||||
|
val imm = IMM(decode.input(INSTRUCTION))
|
||||||
|
|
||||||
|
val conditionalBranchPrediction = (prediction match {
|
||||||
|
case STATIC => imm.b_sext.msb
|
||||||
|
// case DYNAMIC => decodeHistory.history.msb
|
||||||
|
})
|
||||||
|
decodePrediction.cmd.hadBranch := decode.input(BRANCH_CTRL) === BranchCtrlEnum.JAL || (decode.input(BRANCH_CTRL) === BranchCtrlEnum.B && conditionalBranchPrediction)
|
||||||
|
|
||||||
|
predictionJumpInterface.valid := decodePrediction.cmd.hadBranch && decode.arbitration.isFiring //TODO OH Doublon de priorité
|
||||||
|
predictionJumpInterface.payload := decode.input(PC) + ((decode.input(BRANCH_CTRL) === BranchCtrlEnum.JAL) ? imm.j_sext | imm.b_sext).asUInt
|
||||||
|
|
||||||
|
|
||||||
|
if(catchAddressMisaligned) {
|
||||||
|
???
|
||||||
|
// predictionExceptionPort.valid := input(INSTRUCTION_READY) && input(PREDICTION_HAD_BRANCHED) && arbitration.isValid && predictionJumpInterface.payload(1 downto 0) =/= 0
|
||||||
|
// predictionExceptionPort.code := 0
|
||||||
|
// predictionExceptionPort.badAddr := predictionJumpInterface.payload
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -5,9 +5,7 @@ import spinal.core._
|
||||||
import spinal.lib._
|
import spinal.lib._
|
||||||
import spinal.lib.bus.amba4.axi._
|
import spinal.lib.bus.amba4.axi._
|
||||||
import spinal.lib.bus.avalon.{AvalonMM, AvalonMMConfig}
|
import spinal.lib.bus.avalon.{AvalonMM, AvalonMMConfig}
|
||||||
import vexriscv.Riscv.IMM
|
|
||||||
|
|
||||||
import scala.collection.mutable.ArrayBuffer
|
|
||||||
|
|
||||||
|
|
||||||
case class IBusSimpleCmd() extends Bundle{
|
case class IBusSimpleCmd() extends Bundle{
|
||||||
|
@ -105,171 +103,50 @@ case class IBusSimpleBus(interfaceKeepData : Boolean) extends Bundle with IMaste
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
class IBusSimplePlugin(interfaceKeepData : Boolean, catchAccessFault : Boolean, pendingMax : Int = 7) extends Plugin[VexRiscv] with JumpService with IBusFetcher{
|
|
||||||
|
|
||||||
|
class IBusSimplePlugin(interfaceKeepData : Boolean, catchAccessFault : Boolean, pendingMax : Int = 7) extends IBusFetcherImpl(
|
||||||
|
catchAccessFault = catchAccessFault,
|
||||||
|
pendingMax = 7,
|
||||||
|
resetVector = BigInt(0x80000000l),
|
||||||
|
keepPcPlus4 = false,
|
||||||
|
decodePcGen = false,
|
||||||
|
compressedGen = false,
|
||||||
|
cmdToRspStageCount = 1,
|
||||||
|
rspStageGen = false,
|
||||||
|
injectorReadyCutGen = false,
|
||||||
|
relaxedPcCalculation = false,
|
||||||
|
prediction = NONE,
|
||||||
|
catchAddressMisaligned = false,
|
||||||
|
injectorStage = true){
|
||||||
var iBus : IBusSimpleBus = null
|
var iBus : IBusSimpleBus = null
|
||||||
var prefetchExceptionPort : Flow[ExceptionCause] = null
|
|
||||||
def resetVector = BigInt(0x80000000l)
|
def resetVector = BigInt(0x80000000l)
|
||||||
def keepPcPlus4 = false
|
def keepPcPlus4 = false
|
||||||
def decodePcGen = true
|
def decodePcGen = false
|
||||||
def compressedGen = true
|
def compressedGen = false
|
||||||
def cmdToRspStageCount = 1
|
def cmdToRspStageCount = 1
|
||||||
def rspStageGen = false
|
def rspStageGen = false
|
||||||
def injectorReadyCutGen = false
|
def injectorReadyCutGen = false
|
||||||
def relaxedPcCalculation = false
|
def relaxedPcCalculation = false
|
||||||
def prediction : BranchPrediction = STATIC
|
def prediction : BranchPrediction = NONE
|
||||||
def catchAddressMisaligned = true
|
def catchAddressMisaligned = false
|
||||||
var decodePrediction : DecodePredictionBus = null
|
def injectorStage = true
|
||||||
assert(cmdToRspStageCount >= 1)
|
|
||||||
assert(!(compressedGen && !decodePcGen))
|
|
||||||
lazy val fetcherHalt = False
|
|
||||||
lazy val decodeNextPcValid = Bool
|
|
||||||
lazy val decodeNextPc = UInt(32 bits)
|
|
||||||
def nextPc() = (decodeNextPcValid, decodeNextPc)
|
|
||||||
|
|
||||||
var predictionJumpInterface : Flow[UInt] = null
|
|
||||||
|
|
||||||
override def haltIt(): Unit = fetcherHalt := True
|
|
||||||
|
|
||||||
case class JumpInfo(interface : Flow[UInt], stage: Stage, priority : Int)
|
|
||||||
val jumpInfos = ArrayBuffer[JumpInfo]()
|
|
||||||
override def createJumpInterface(stage: Stage, priority : Int = 0): Flow[UInt] = {
|
|
||||||
val interface = Flow(UInt(32 bits))
|
|
||||||
jumpInfos += JumpInfo(interface,stage, priority)
|
|
||||||
interface
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
var decodeExceptionPort : Flow[ExceptionCause] = null
|
|
||||||
override def setup(pipeline: VexRiscv): Unit = {
|
override def setup(pipeline: VexRiscv): Unit = {
|
||||||
|
super.setup(pipeline)
|
||||||
iBus = master(IBusSimpleBus(interfaceKeepData)).setName("iBus")
|
iBus = master(IBusSimpleBus(interfaceKeepData)).setName("iBus")
|
||||||
if(catchAccessFault) {
|
|
||||||
val exceptionService = pipeline.service(classOf[ExceptionService])
|
|
||||||
decodeExceptionPort = exceptionService.newExceptionPort(pipeline.decode,1).setName("iBusErrorExceptionnPort")
|
|
||||||
}
|
|
||||||
|
|
||||||
pipeline(RVC_GEN) = compressedGen
|
|
||||||
|
|
||||||
prediction match {
|
|
||||||
case NONE =>
|
|
||||||
case STATIC | DYNAMIC => {
|
|
||||||
predictionJumpInterface = createJumpInterface(pipeline.decode)
|
|
||||||
decodePrediction = pipeline.service(classOf[PredictionInterface]).askDecodePrediction()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override def build(pipeline: VexRiscv): Unit = {
|
override def build(pipeline: VexRiscv): Unit = {
|
||||||
import pipeline._
|
import pipeline._
|
||||||
import pipeline.config._
|
import pipeline.config._
|
||||||
|
|
||||||
pipeline plug new Area {
|
pipeline plug new FetchArea(pipeline) {
|
||||||
|
|
||||||
//JumpService hardware implementation
|
val cmd = new Area {
|
||||||
val jump = new Area {
|
import iBusCmd._
|
||||||
val sortedByStage = jumpInfos.sortWith((a, b) => {
|
output << input.continueWhen(iBus.cmd.fire)
|
||||||
(pipeline.indexOf(a.stage) > pipeline.indexOf(b.stage)) ||
|
|
||||||
(pipeline.indexOf(a.stage) == pipeline.indexOf(b.stage) && a.priority > b.priority)
|
|
||||||
})
|
|
||||||
val valids = sortedByStage.map(_.interface.valid)
|
|
||||||
val pcs = sortedByStage.map(_.interface.payload)
|
|
||||||
|
|
||||||
val pcLoad = Flow(UInt(32 bits))
|
|
||||||
pcLoad.valid := jumpInfos.map(_.interface.valid).orR
|
|
||||||
pcLoad.payload := MuxOH(OHMasking.first(valids.asBits), pcs)
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def flush = jump.pcLoad.valid
|
|
||||||
|
|
||||||
class PcFetch extends Area{
|
|
||||||
val output = Stream(UInt(32 bits))
|
|
||||||
}
|
|
||||||
|
|
||||||
val fetchPc = if(relaxedPcCalculation) new PcFetch {
|
|
||||||
//PC calculation without Jump
|
|
||||||
val pcReg = Reg(UInt(32 bits)) init (resetVector) addAttribute (Verilator.public)
|
|
||||||
val pcPlus4 = pcReg + 4
|
|
||||||
if (keepPcPlus4) KeepAttribute(pcPlus4)
|
|
||||||
when(output.fire) {
|
|
||||||
pcReg := pcPlus4
|
|
||||||
}
|
|
||||||
|
|
||||||
//Realign
|
|
||||||
if(compressedGen){
|
|
||||||
when(output.fire){
|
|
||||||
pcReg(1 downto 0) := 0
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
//application of the selected jump request
|
|
||||||
when(jump.pcLoad.valid) {
|
|
||||||
pcReg := jump.pcLoad.payload
|
|
||||||
}
|
|
||||||
|
|
||||||
output.valid := RegNext(True) init (False) // && !jump.pcLoad.valid
|
|
||||||
output.payload := pcReg
|
|
||||||
} else new PcFetch{
|
|
||||||
//PC calculation without Jump
|
|
||||||
val pcReg = Reg(UInt(32 bits)) init(resetVector) addAttribute(Verilator.public)
|
|
||||||
val inc = RegInit(False)
|
|
||||||
|
|
||||||
val pc = pcReg + (inc ## B"00").asUInt
|
|
||||||
val samplePcNext = False
|
|
||||||
|
|
||||||
when(jump.pcLoad.valid) {
|
|
||||||
inc := False
|
|
||||||
samplePcNext := True
|
|
||||||
pc := jump.pcLoad.payload
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
when(output.fire){
|
|
||||||
inc := True
|
|
||||||
samplePcNext := True
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
when(samplePcNext) {
|
|
||||||
pcReg := pc
|
|
||||||
}
|
|
||||||
|
|
||||||
if(compressedGen) {
|
|
||||||
when(output.fire) {
|
|
||||||
pcReg(1 downto 0) := 0
|
|
||||||
when(pc(1)){
|
|
||||||
inc := True
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
output.valid := RegNext(True) init (False)
|
|
||||||
output.payload := pc
|
|
||||||
}
|
|
||||||
|
|
||||||
val decodePc = ifGen(decodePcGen)(new Area {
|
|
||||||
//PC calculation without Jump
|
|
||||||
val pcReg = Reg(UInt(32 bits)) init (resetVector) addAttribute (Verilator.public)
|
|
||||||
val pcPlus = if(compressedGen)
|
|
||||||
pcReg + ((decode.input(IS_RVC)) ? U(2) | U(4))
|
|
||||||
else
|
|
||||||
pcReg + 4
|
|
||||||
|
|
||||||
if (keepPcPlus4) KeepAttribute(pcPlus)
|
|
||||||
when(decode.arbitration.isFiring) {
|
|
||||||
pcReg := pcPlus
|
|
||||||
}
|
|
||||||
|
|
||||||
//application of the selected jump request
|
|
||||||
when(jump.pcLoad.valid) {
|
|
||||||
pcReg := jump.pcLoad.payload
|
|
||||||
}
|
|
||||||
})
|
|
||||||
|
|
||||||
|
|
||||||
val iBusCmd = new Area {
|
|
||||||
def input = fetchPc.output
|
|
||||||
|
|
||||||
val output = input.continueWhen(iBus.cmd.fire)
|
|
||||||
|
|
||||||
//Avoid sending to many iBus cmd
|
//Avoid sending to many iBus cmd
|
||||||
val pendingCmd = Reg(UInt(log2Up(pendingMax + 1) bits)) init (0)
|
val pendingCmd = Reg(UInt(log2Up(pendingMax + 1) bits)) init (0)
|
||||||
|
@ -280,26 +157,14 @@ class IBusSimplePlugin(interfaceKeepData : Boolean, catchAccessFault : Boolean,
|
||||||
iBus.cmd.pc := input.payload(31 downto 2) @@ "00"
|
iBus.cmd.pc := input.payload(31 downto 2) @@ "00"
|
||||||
}
|
}
|
||||||
|
|
||||||
case class FetchRsp() extends Bundle {
|
|
||||||
val pc = UInt(32 bits)
|
|
||||||
val rsp = IBusSimpleRsp()
|
|
||||||
val isRvc = Bool
|
|
||||||
}
|
|
||||||
|
|
||||||
def recursive[T](that : T,depth : Int, func : (T) => T) : T = depth match {
|
|
||||||
case 0 => that
|
|
||||||
case _ => recursive(func(that), depth -1, func)
|
|
||||||
}
|
|
||||||
|
|
||||||
val iBusRsp = new Area {
|
|
||||||
val inputFirstStage = if(relaxedPcCalculation) iBusCmd.output.m2sPipe(flush) else iBusCmd.output.m2sPipe().throwWhen(flush)
|
|
||||||
val input = recursive[Stream[UInt]](inputFirstStage, cmdToRspStageCount - 1, x => x.m2sPipe(flush))//iBusCmd.output.m2sPipe(flush)// ASYNC .throwWhen(flush)
|
|
||||||
|
|
||||||
|
val rsp = new Area {
|
||||||
|
import iBusRsp._
|
||||||
//Manage flush for iBus transactions in flight
|
//Manage flush for iBus transactions in flight
|
||||||
val discardCounter = Reg(UInt(log2Up(pendingMax + 1) bits)) init (0)
|
val discardCounter = Reg(UInt(log2Up(pendingMax + 1) bits)) init (0)
|
||||||
discardCounter := discardCounter - (iBus.rsp.fire && discardCounter =/= 0).asUInt
|
discardCounter := discardCounter - (iBus.rsp.fire && discardCounter =/= 0).asUInt
|
||||||
when(flush) {
|
when(flush) {
|
||||||
discardCounter := (if(relaxedPcCalculation) iBusCmd.pendingCmdNext else iBusCmd.pendingCmd - iBus.rsp.fire.asUInt)
|
discardCounter := (if(relaxedPcCalculation) cmd.pendingCmdNext else cmd.pendingCmd - iBus.rsp.fire.asUInt)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -314,94 +179,8 @@ class IBusSimplePlugin(interfaceKeepData : Boolean, catchAccessFault : Boolean,
|
||||||
fetchRsp.rsp.error.clearWhen(!rspBuffer.io.pop.valid) //Avoid interference with instruction injection from the debug plugin
|
fetchRsp.rsp.error.clearWhen(!rspBuffer.io.pop.valid) //Avoid interference with instruction injection from the debug plugin
|
||||||
|
|
||||||
|
|
||||||
def outputGen = StreamJoin(Seq(input, rspBuffer.io.pop), fetchRsp)
|
val join = StreamJoin(Seq(input, rspBuffer.io.pop), fetchRsp)
|
||||||
val output = if(rspStageGen) outputGen.m2sPipe(flush) else outputGen
|
output << (if(rspStageGen) join.m2sPipeWithFlush(flush) else join)
|
||||||
}
|
|
||||||
|
|
||||||
val decompressor = ifGen(decodePcGen)(new Area{
|
|
||||||
def input = iBusRsp.output
|
|
||||||
val output = Stream(FetchRsp())
|
|
||||||
|
|
||||||
val bufferValid = RegInit(False)
|
|
||||||
val bufferError = Reg(Bool)
|
|
||||||
val bufferData = Reg(Bits(16 bits))
|
|
||||||
|
|
||||||
val raw = Mux(
|
|
||||||
sel = bufferValid,
|
|
||||||
whenTrue = input.rsp.inst(15 downto 0) ## bufferData,
|
|
||||||
whenFalse = input.rsp.inst(31 downto 16) ## (input.pc(1) ? input.rsp.inst(31 downto 16) | input.rsp.inst(15 downto 0))
|
|
||||||
)
|
|
||||||
val isRvc = raw(1 downto 0) =/= 3
|
|
||||||
val decompressed = RvcDecompressor(raw(15 downto 0))
|
|
||||||
output.valid := isRvc ? (bufferValid || input.valid) | (input.valid && (bufferValid || !input.pc(1)))
|
|
||||||
output.pc := input.pc
|
|
||||||
output.isRvc := isRvc
|
|
||||||
output.rsp.inst := isRvc ? decompressed | raw
|
|
||||||
output.rsp.error := (bufferValid && bufferError) || (input.valid && input.rsp.error && (!isRvc || (isRvc && !bufferValid)))
|
|
||||||
input.ready := (bufferValid ? (!isRvc && output.ready) | (input.pc(1) || output.ready))
|
|
||||||
|
|
||||||
|
|
||||||
bufferValid clearWhen(output.fire)
|
|
||||||
when(input.ready){
|
|
||||||
when(input.valid) {
|
|
||||||
bufferValid := !(!isRvc && !input.pc(1) && !bufferValid) && !(isRvc && input.pc(1))
|
|
||||||
bufferError := input.rsp.error
|
|
||||||
bufferData := input.rsp.inst(31 downto 16)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
bufferValid.clearWhen(flush)
|
|
||||||
})
|
|
||||||
|
|
||||||
def condApply[T](that : T, cond : Boolean)(func : (T) => T) = if(cond)func(that) else that
|
|
||||||
val injector = new Area {
|
|
||||||
val inputBeforeHalt = condApply(if(decodePcGen) decompressor.output else iBusRsp.output, injectorReadyCutGen)(_.s2mPipe(flush))
|
|
||||||
val input = inputBeforeHalt.haltWhen(fetcherHalt)
|
|
||||||
val stage = input.m2sPipe(flush || decode.arbitration.isRemoved)
|
|
||||||
|
|
||||||
if(decodePcGen){
|
|
||||||
decodeNextPcValid := True
|
|
||||||
decodeNextPc := decodePc.pcReg
|
|
||||||
}else {
|
|
||||||
decodeNextPcValid := RegNext(inputBeforeHalt.isStall)
|
|
||||||
decodeNextPc := decode.input(PC)
|
|
||||||
}
|
|
||||||
|
|
||||||
stage.ready := !decode.arbitration.isStuck
|
|
||||||
decode.arbitration.isValid := stage.valid
|
|
||||||
decode.insert(PC) := (if(decodePcGen) decodePc.pcReg else stage.pc)
|
|
||||||
decode.insert(INSTRUCTION) := stage.rsp.inst
|
|
||||||
decode.insert(INSTRUCTION_ANTICIPATED) := Mux(decode.arbitration.isStuck, decode.input(INSTRUCTION), input.rsp.inst)
|
|
||||||
decode.insert(INSTRUCTION_READY) := True
|
|
||||||
if(compressedGen) decode.insert(IS_RVC) := stage.isRvc
|
|
||||||
|
|
||||||
if(catchAccessFault){
|
|
||||||
decodeExceptionPort.valid := decode.arbitration.isValid && stage.rsp.error
|
|
||||||
decodeExceptionPort.code := 1
|
|
||||||
decodeExceptionPort.badAddr := decode.input(PC)
|
|
||||||
}
|
|
||||||
|
|
||||||
prediction match {
|
|
||||||
case `NONE` =>
|
|
||||||
case `STATIC` => {
|
|
||||||
val imm = IMM(decode.input(INSTRUCTION))
|
|
||||||
|
|
||||||
val conditionalBranchPrediction = (prediction match {
|
|
||||||
case `STATIC` => imm.b_sext.msb
|
|
||||||
//case `DYNAMIC` => input(HISTORY_LINE).history.msb
|
|
||||||
})
|
|
||||||
decodePrediction.cmd.hadBranch := decode.input(BRANCH_CTRL) === BranchCtrlEnum.JAL || (decode.input(BRANCH_CTRL) === BranchCtrlEnum.B && conditionalBranchPrediction)
|
|
||||||
|
|
||||||
predictionJumpInterface.valid := decodePrediction.cmd.hadBranch && decode.arbitration.isFiring //TODO OH Doublon de priorité
|
|
||||||
predictionJumpInterface.payload := decode.input(PC) + ((decode.input(BRANCH_CTRL) === BranchCtrlEnum.JAL) ? imm.j_sext | imm.b_sext).asUInt
|
|
||||||
|
|
||||||
|
|
||||||
// if(catchAddressMisaligned) {
|
|
||||||
// predictionExceptionPort.valid := input(INSTRUCTION_READY) && input(PREDICTION_HAD_BRANCHED) && arbitration.isValid && predictionJumpInterface.payload(1 downto 0) =/= 0
|
|
||||||
// predictionExceptionPort.code := 0
|
|
||||||
// predictionExceptionPort.badAddr := predictionJumpInterface.payload
|
|
||||||
// }
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -95,11 +95,12 @@ object RvcDecompressor{
|
||||||
|
|
||||||
object StreamVexPimper{
|
object StreamVexPimper{
|
||||||
implicit class StreamFlushPimper[T <: Data](pimped : Stream[T]){
|
implicit class StreamFlushPimper[T <: Data](pimped : Stream[T]){
|
||||||
def m2sPipe(flush : Bool, collapsBubble : Boolean = true): Stream[T] = {
|
def m2sPipeWithFlush(flush : Bool, discardInput : Boolean = true, collapsBubble : Boolean = true): Stream[T] = {
|
||||||
val ret = cloneOf(pimped)
|
val ret = cloneOf(pimped)
|
||||||
|
|
||||||
val rValid = RegInit(False)
|
val rValid = RegInit(False)
|
||||||
val rData = Reg(pimped.dataType)
|
val rData = Reg(pimped.dataType)
|
||||||
|
if(!discardInput) rValid.clearWhen(flush)
|
||||||
|
|
||||||
pimped.ready := (Bool(collapsBubble) && !ret.valid) || ret.ready
|
pimped.ready := (Bool(collapsBubble) && !ret.valid) || ret.ready
|
||||||
|
|
||||||
|
@ -111,7 +112,7 @@ object StreamVexPimper{
|
||||||
ret.valid := rValid
|
ret.valid := rValid
|
||||||
ret.payload := rData
|
ret.payload := rData
|
||||||
|
|
||||||
rValid.clearWhen(flush)
|
if(discardInput) rValid.clearWhen(flush)
|
||||||
|
|
||||||
ret
|
ret
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue