From 6683274656276b8ed6a471ae5a7f1d5a51bf2da4 Mon Sep 17 00:00:00 2001 From: qinjun-li Date: Wed, 12 Jun 2024 18:26:02 +0800 Subject: [PATCH] [rtl] switch T1 TileLink to AXI4 --- configgen/src/Main.scala | 84 ----------- t1/src/T1.scala | 98 ++++++------- t1/src/lsu/Bundle.scala | 45 +++++- t1/src/lsu/LSU.scala | 223 ++++++++++++------------------ t1/src/lsu/LoadUnit.scala | 98 +++---------- t1/src/lsu/SimpleAccessUnit.scala | 102 +++++++------- t1/src/lsu/StoreUnit.scala | 79 +++-------- t1/src/lsu/StrideBase.scala | 1 - t1/src/package.scala | 1 - 9 files changed, 262 insertions(+), 469 deletions(-) diff --git a/configgen/src/Main.scala b/configgen/src/Main.scala index 2426a118d..c48760fef 100644 --- a/configgen/src/Main.scala +++ b/configgen/src/Main.scala @@ -9,7 +9,6 @@ import chisel3.util.experimental.BitSet import mainargs._ import org.chipsalliance.t1.rtl._ import org.chipsalliance.t1.rtl.decoder.T1CustomInstruction -import org.chipsalliance.t1.rtl.lsu.LSUInstantiateParameter import org.chipsalliance.t1.rtl.vrf.RamType import java.util.LinkedHashMap @@ -70,29 +69,6 @@ object Main { dLen, extensions = Seq("Zve32f"), t1customInstructions = Nil, - lsuBankParameters = - // scalar bank 0-1G - Seq( - BitSet(BitPat("b00??????????????????????????????")) - ).map(bs => LSUBankParameter("scalar", bs, 8, true)) ++ - // ddr bank 1G-3G 512M/bank - Seq( - BitSet(BitPat("b01???????????????????????00?????"), BitPat("b10???????????????????????00?????")), - BitSet(BitPat("b01???????????????????????01?????"), BitPat("b10???????????????????????01?????")), - BitSet(BitPat("b01???????????????????????10?????"), BitPat("b10???????????????????????10?????")), - BitSet(BitPat("b01???????????????????????11?????"), BitPat("b10???????????????????????11?????")) - ).zipWithIndex.map { case (bs: BitSet, idx: Int) => LSUBankParameter(s"ddrBank$idx", bs, 8, false) } ++ - // sRam bank 3G+ 256K/bank, 8banks - Seq( - BitSet(BitPat("b11000000000?????????????000?????")), - BitSet(BitPat("b11000000000?????????????001?????")), - BitSet(BitPat("b11000000000?????????????010?????")), - BitSet(BitPat("b11000000000?????????????011?????")), - BitSet(BitPat("b11000000000?????????????100?????")), - BitSet(BitPat("b11000000000?????????????101?????")), - BitSet(BitPat("b11000000000?????????????110?????")), - BitSet(BitPat("b11000000000?????????????111?????")) - ).zipWithIndex.map { case (bs: BitSet, idx: Int) => LSUBankParameter(s"sramBank$idx", bs, 8, false) }, vrfBankSize = 1, vrfRamType = RamType.p0rwp1rw, vfuInstantiateParameter = VFUInstantiateParameter( @@ -142,35 +118,6 @@ object Main { dLen, extensions = Seq("Zve32x"), t1customInstructions = Nil, - // banks=8 dLen=512 beatbyte16 - lsuBankParameters = - // scalar bank 0-1G - Seq( - BitSet(BitPat("b00??????????????????????????????")) - ).map(bs => LSUBankParameter("scalar", bs, 8, true)) ++ - // ddr bank 1G-3G 512M/bank - // bp '01???????????????????????00?????' - // base '01000000000000000000000000000000' - // cmask '11000000000000000000000001100000' - // cmaskinv '00111111111111111111111110011111' - // asmask '11000000000000000000000001100000' - Seq( - BitSet(BitPat("b01??????????????????????00??????"), BitPat("b10??????????????????????00??????")), - BitSet(BitPat("b01??????????????????????01??????"), BitPat("b10??????????????????????01??????")), - BitSet(BitPat("b01??????????????????????10??????"), BitPat("b10??????????????????????10??????")), - BitSet(BitPat("b01??????????????????????11??????"), BitPat("b10??????????????????????11??????")) - ).zipWithIndex.map { case (bs: BitSet, idx: Int) => LSUBankParameter(s"ddrBank$idx", bs, 8, false) } ++ - // sRam bank 3G+ 256K/bank, 8banks - Seq( - BitSet(BitPat("b11000000000????????????000??????")), - BitSet(BitPat("b11000000000????????????001??????")), - BitSet(BitPat("b11000000000????????????010??????")), - BitSet(BitPat("b11000000000????????????011??????")), - BitSet(BitPat("b11000000000????????????100??????")), - BitSet(BitPat("b11000000000????????????101??????")), - BitSet(BitPat("b11000000000????????????110??????")), - BitSet(BitPat("b11000000000????????????111??????")) - ).zipWithIndex.map { case (bs: BitSet, idx: Int) => LSUBankParameter(s"sramBank$idx", bs, 8, false) }, vrfBankSize = 2, vrfRamType = RamType.p0rp1w, vfuInstantiateParameter = VFUInstantiateParameter( @@ -220,37 +167,6 @@ object Main { dLen, extensions = Seq("Zve32x"), t1customInstructions = Nil, - lsuBankParameters = - // scalar bank 0-1G - Seq( - BitSet(BitPat("b00??????????????????????????????")) - ).map(bs => LSUBankParameter("scalar", bs, 8, true)) ++ - // ddr bank 1G-3G 512M/bank - Seq( - BitSet(BitPat("b01?????????????????????00???????"), BitPat("b10?????????????????????00???????")), - BitSet(BitPat("b01?????????????????????01???????"), BitPat("b10?????????????????????01???????")), - BitSet(BitPat("b01?????????????????????10???????"), BitPat("b10?????????????????????10???????")), - BitSet(BitPat("b01?????????????????????11???????"), BitPat("b10?????????????????????11???????")) - ).zipWithIndex.map { case (bs: BitSet, idx: Int) => LSUBankParameter(s"ddrBank$idx", bs, 8, false) } ++ - // sRam bank 3G+ 256K/bank, 16banks - Seq( - BitSet(BitPat("b1100000000?????????0000?????????")), - BitSet(BitPat("b1100000000?????????0001?????????")), - BitSet(BitPat("b1100000000?????????0010?????????")), - BitSet(BitPat("b1100000000?????????0011?????????")), - BitSet(BitPat("b1100000000?????????0100?????????")), - BitSet(BitPat("b1100000000?????????0101?????????")), - BitSet(BitPat("b1100000000?????????0110?????????")), - BitSet(BitPat("b1100000000?????????0111?????????")), - BitSet(BitPat("b1100000000?????????1000?????????")), - BitSet(BitPat("b1100000000?????????1001?????????")), - BitSet(BitPat("b1100000000?????????1010?????????")), - BitSet(BitPat("b1100000000?????????1011?????????")), - BitSet(BitPat("b1100000000?????????1100?????????")), - BitSet(BitPat("b1100000000?????????1101?????????")), - BitSet(BitPat("b1100000000?????????1110?????????")), - BitSet(BitPat("b1100000000?????????1111?????????")) - ).zipWithIndex.map { case (bs: BitSet, idx: Int) => LSUBankParameter(s"sramBank$idx", bs, 8, false) }, vrfBankSize = 4, vrfRamType = RamType.p0rw, vfuInstantiateParameter = VFUInstantiateParameter( diff --git a/t1/src/T1.scala b/t1/src/T1.scala index 322990b89..c23248a91 100644 --- a/t1/src/T1.scala +++ b/t1/src/T1.scala @@ -4,16 +4,16 @@ package org.chipsalliance.t1.rtl import chisel3._ -import chisel3.experimental.hierarchy.{Definition, Instance, Instantiate, instantiable, public} +import chisel3.experimental.hierarchy.{Instance, Instantiate, instantiable, public} import chisel3.experimental.{SerializableModule, SerializableModuleParameter} -import chisel3.util._ -import chisel3.util.experimental.decode._ -import tilelink.{TLBundle, TLBundleParameter, TLChannelAParameter, TLChannelDParameter} -import chisel3.probe.{Probe, ProbeValue, define, force} +import chisel3.probe.{Probe, ProbeValue, define} import chisel3.properties.{AnyClassType, Class, ClassType, Property} +import chisel3.util._ import chisel3.util.experimental.BitSet import org.chipsalliance.rvdecoderdb.Instruction import org.chipsalliance.t1.rtl.decoder.{Decoder, DecoderParam, T1CustomInstruction} +import chisel3.util.experimental.decode._ +import org.chipsalliance.amba.axi4.bundle.{AXI4BundleParameter, AXI4RWIrrevocable} import org.chipsalliance.t1.rtl.lsu.{LSU, LSUParameter, LSUProbe} import org.chipsalliance.t1.rtl.vrf.{RamType, VRFParam, VRFProbe} @@ -57,16 +57,6 @@ object T1Parameter { implicit def rwP: upickle.default.ReadWriter[T1Parameter] = upickle.default.macroRW } -object LSUBankParameter{ - implicit def bitSetP:upickle.default.ReadWriter[BitSet] = upickle.default.readwriter[String].bimap[BitSet]( - bs => bs.terms.map("b" + _.rawString).mkString("\n"), - str => BitSet.fromString(str) - ) - - implicit def rwP: upickle.default.ReadWriter[LSUBankParameter] = upickle.default.macroRW -} -case class LSUBankParameter(name: String, region: BitSet, beatbyte: Int, accessScalar: Boolean) - /** * @param xLen XLEN * @param vLen VLEN @@ -92,8 +82,6 @@ case class T1Parameter( dLen: Int, extensions: Seq[String], t1customInstructions: Seq[T1CustomInstruction], - // LSU - lsuBankParameters: Seq[LSUBankParameter], // Lane vrfBankSize: Int, vrfRamType: RamType, @@ -112,11 +100,6 @@ case class T1Parameter( case RamType.p0rp1w => "First Port Read, Second Port Write." case RamType.p0rwp1rw => "Dual Ports Read Write." }} - |LSU: - |${lsuBankParameters.zipWithIndex.map{case (lsuP, idx) => - s"""BANK${idx}W${lsuP.beatbyte * 8}b ${if(lsuP.accessScalar) "can" else "can't"} access scalar memory - | ${lsuP.region.terms.map(_.rawString).mkString("\n ")} - |""".stripMargin}} |""".stripMargin val allInstructions: Seq[Instruction] = { @@ -198,10 +181,6 @@ case class T1Parameter( /** the hardware width of [[groupNumberMax]]. */ val groupNumberMaxBits: Int = log2Ceil(groupNumberMax) - require(lsuBankParameters.map(_.beatbyte).toSet.size == 1, "The width is temporarily unified") - /** Used in memory bundle parameter. */ - val memoryDataWidthBytes: Int = lsuBankParameters.head.beatbyte - /** LSU MSHR Size, Contains a load unit, a store unit and an other unit. */ val lsuMSHRSize: Int = 3 @@ -218,9 +197,6 @@ case class T1Parameter( */ val sizeWidth: Int = log2Ceil(log2Ceil(lsuTransposeSize)) - /** for TileLink `mask` element. */ - val maskWidth: Int = lsuBankParameters.head.beatbyte - val vrfReadLatency = 2 // each element: Each lane will be connected to the other two lanes, @@ -229,13 +205,30 @@ case class T1Parameter( val decoderParam: DecoderParam = DecoderParam(fpuEnable, allInstructions) - /** parameter for TileLink. */ - val tlParam: TLBundleParameter = TLBundleParameter( - a = TLChannelAParameter(physicalAddressWidth, sourceWidth, memoryDataWidthBytes * 8, sizeWidth, maskWidth), - b = None, - c = None, - d = TLChannelDParameter(sourceWidth, sourceWidth, memoryDataWidthBytes * 8, sizeWidth), - e = None + /** paraemter for AXI4. */ + val axi4BundleParameter: AXI4BundleParameter = AXI4BundleParameter( + idWidth = sourceWidth, + dataWidth = dLen, + addrWidth = physicalAddressWidth, + userReqWidth = 0, + userDataWidth = 0, + userRespWidth = 0, + hasAW = true, + hasW = true, + hasB = true, + hasAR = true, + hasR = true, + supportId = true, + supportRegion = false, + supportLen = true, + supportSize = true, + supportBurst = true, + supportLock = false, + supportCache = false, + supportQos = false, + supportStrb = true, + supportResp = true, + supportProt = false, ) /** Parameter for [[Lane]] */ @@ -263,15 +256,13 @@ case class T1Parameter( sourceWidth = sourceWidth, sizeWidth = sizeWidth, // TODO: configurable for each LSU [[p.supportMask]] - maskWidth = maskWidth, - banks = lsuBankParameters, + maskWidth = dLen / 32, lsuMSHRSize = lsuMSHRSize, // TODO: make it configurable for each lane toVRFWriteQueueSize = 96, transferSize = lsuTransposeSize, vrfReadLatency = vrfReadLatency, - // TODO: configurable for each LSU - tlParam = tlParam, + axi4BundleParameter = axi4BundleParameter, name = "main" ) def vrfParam: VRFParam = VRFParam(vLen, laneNumber, datapathWidth, chainingSize, vrfBankSize, vrfRamType) @@ -303,23 +294,28 @@ class T1(val parameter: T1Parameter) extends Module with SerializableModule[T1Pa * and the `kill` logic in Vector processor is too high, * thus the request should come from commit stage to avoid any interrupt or excepiton. */ + @public val request: DecoupledIO[VRequest] = IO(Flipped(Decoupled(new VRequest(parameter.xLen)))) - /** response to CPU. */ + @public val response: ValidIO[VResponse] = IO(Valid(new VResponse(parameter.xLen))) - /** CSR interface from CPU. */ + @public val csrInterface: CSRInterface = IO(Input(new CSRInterface(parameter.laneParam.vlMaxBits))) - /** from CPU LSU, store buffer is cleared, memory can observe memory requests after this is asserted. */ + @public val storeBufferClear: Bool = IO(Input(Bool())) - - /** TileLink memory ports. */ - val memoryPorts: Vec[TLBundle] = IO(Vec(parameter.lsuBankParameters.size, parameter.tlParam.bundle())) - + // TODO: expose region name here. + @public + val highBandwidthLoadStorePort: AXI4RWIrrevocable = IO(new AXI4RWIrrevocable(parameter.axi4BundleParameter)) + @public + val indexedLoadStorePort: AXI4RWIrrevocable = IO(new AXI4RWIrrevocable(parameter.axi4BundleParameter.copy(dataWidth=32))) // TODO: this is an example of adding a new Probe + @public val lsuProbe = IO(Probe(new LSUProbe(parameter.lsuParameters))) + @public val laneProbes = Seq.tabulate(parameter.laneNumber)(laneIdx => IO(Probe(new LaneProbe(parameter.chainingSize))).suggestName(s"lane${laneIdx}Probe")) + @public val laneVrfProbes = Seq.tabulate(parameter.laneNumber)(laneIdx => IO(Probe(new VRFProbe( parameter.laneParam.vrfParam.regNumBits, parameter.laneParam.vrfOffsetBits, @@ -1602,12 +1598,8 @@ class T1(val parameter: T1Parameter) extends Module with SerializableModule[T1Pa } } - memoryPorts.zip(lsu.tlPort).foreach { - case (source, sink) => - val dBuffer = Queue(source.d, 1, flow = true) - sink <> source - sink.d <> dBuffer - } + highBandwidthLoadStorePort <> lsu.axi4Port + indexedLoadStorePort <> lsu.simpleAccessPorts // 暂时直接连lsu的写,后续需要处理scheduler的写 vrfWrite.zip(lsu.vrfWritePort).foreach { case (sink, source) => sink <> source } diff --git a/t1/src/lsu/Bundle.scala b/t1/src/lsu/Bundle.scala index e9c02fdd2..d5af70ae4 100644 --- a/t1/src/lsu/Bundle.scala +++ b/t1/src/lsu/Bundle.scala @@ -41,11 +41,6 @@ class SimpleAccessStatus(laneNumber: Int) extends LSUBaseStatus { val isIndexLS: Bool = Bool() } -class StoreStatus(bankSize: Int) extends LSUBaseStatus { - // cache line 的发送不能被打断 - val releasePort: Vec[Bool] = Vec(bankSize, Bool()) -} - class MSHRStage0Bundle(param: MSHRParam) extends Bundle { // 读的相关 val readVS: UInt = UInt(param.regNumBits.W) @@ -69,3 +64,43 @@ class SimpleAccessStage1(param: MSHRParam) extends Bundle { val address: UInt = UInt(param.paWidth.W) val readData: UInt = UInt(param.datapathWidth.W) } + +class MemRequest(param: MSHRParam) extends Bundle { + val src: UInt = UInt(param.cacheLineIndexBits.W) + val address: UInt = UInt(param.paWidth.W) +} + +class MemDataBundle(param: MSHRParam) extends Bundle { + // todo: DLEN? + val data: UInt = UInt((param.lsuTransposeSize * 8).W) + val index: UInt = UInt(param.cacheLineIndexBits.W) +} + +class MemWrite(param: MSHRParam) extends Bundle { + // todo: DLEN? + val data: UInt = UInt((param.lsuTransposeSize * 8).W) + val mask: UInt = UInt(param.lsuTransposeSize.W) + val index: UInt = UInt(param.cacheLineIndexBits.W) + val address: UInt = UInt(param.paWidth.W) +} + +class SimpleMemRequest(param: MSHRParam) extends Bundle { + val address: UInt = UInt(param.paWidth.W) + val size: UInt = UInt(2.W) + val source: UInt = UInt(8.W) +} + +class SimpleMemReadResponse(param: MSHRParam) extends Bundle { + // todo: DLEN? + val data: UInt = UInt((param.lsuTransposeSize * 8).W) + val source: UInt = UInt(8.W) +} + +class SimpleMemWrite(param: MSHRParam) extends Bundle { + // todo: DLEN? + val data: UInt = UInt((param.lsuTransposeSize * 8).W) + val mask: UInt = UInt(param.lsuTransposeSize.W) + val source: UInt = UInt(8.W) + val address: UInt = UInt(param.paWidth.W) + val size: UInt = UInt(2.W) +} diff --git a/t1/src/lsu/LSU.scala b/t1/src/lsu/LSU.scala index 0f133df8e..27a411b16 100644 --- a/t1/src/lsu/LSU.scala +++ b/t1/src/lsu/LSU.scala @@ -4,38 +4,11 @@ package org.chipsalliance.t1.rtl.lsu import chisel3._ -import chisel3.experimental.hierarchy.{Instance, Instantiate, instantiable, public} +import chisel3.experimental.hierarchy.{instantiable, public} import chisel3.probe.{Probe, ProbeValue, define} import chisel3.util._ -import chisel3.util.experimental.BitSet -import org.chipsalliance.t1.rtl.{CSRInterface, LSUBankParameter, LSURequest, LSUWriteQueueBundle, VRFReadRequest, VRFWriteRequest, firstlastHelper, indexToOH, instIndexL} -import tilelink.{TLBundle, TLBundleParameter, TLChannelA, TLChannelD} - -// TODO: need some idea from BankBinder -object LSUInstantiateParameter { - implicit def bitSetP:upickle.default.ReadWriter[BitSet] = upickle.default.readwriter[String].bimap[BitSet]( - _.toString, - BitSet.fromString - ) - - implicit def rwP: upickle.default.ReadWriter[LSUInstantiateParameter] = upickle.default.macroRW -} - -/** Public LSU parameter expose to upper level. */ -case class LSUInstantiateParameter(name: String, base: BigInt, size: BigInt, banks: Int) { - // TODO: uarch tuning for different LSUs to reduce segment overhead. - // these tweaks should only be applied to some special MMIO LSU, e.g. systolic array, etc - val supportStride: Boolean = true - val supportSegment: Set[Int] = Seq.tabulate(8)(_ + 1).toSet - val supportMask: Boolean = true - // TODO: support MMU for linux. - val supportMMU: Boolean = false - - // used for add latency from LSU to corresponding lanes, it should be managed by floorplan - val latencyToLanes: Seq[Int] = Seq(1) - // used for add queue for avoid dead lock on memory. - val maxLatencyToEndpoint: Int = 96 -} +import org.chipsalliance.amba.axi4.bundle.{AXI4BundleParameter, AXI4RWIrrevocable} +import org.chipsalliance.t1.rtl._ /** * @param datapathWidth ELEN @@ -53,23 +26,14 @@ case class LSUParameter( sourceWidth: Int, sizeWidth: Int, maskWidth: Int, - banks: Seq[LSUBankParameter], lsuMSHRSize: Int, toVRFWriteQueueSize: Int, transferSize: Int, // TODO: refactor to per lane parameter. vrfReadLatency: Int, - tlParam: TLBundleParameter, + axi4BundleParameter: AXI4BundleParameter, name: String ) { - val memoryBankSize: Int = banks.size - - banks.zipWithIndex.foreach { case (bs, i) => - Seq.tabulate(banks.size) { bankIndex => - require(i == bankIndex || !banks(bankIndex).region.overlap(bs.region)) - } - } - val sewMin: Int = 8 /** the maximum address offsets number can be accessed from lanes for one time. */ @@ -94,7 +58,7 @@ case class LSUParameter( val sourceQueueSize: Int = vLen * 8 / (transferSize * 8) def mshrParam: MSHRParam = - MSHRParam(chainingSize, datapathWidth, vLen, laneNumber, paWidth, transferSize, memoryBankSize, vrfReadLatency, banks, tlParam) + MSHRParam(chainingSize, datapathWidth, vLen, laneNumber, paWidth, transferSize, vrfReadLatency) /** see [[VRFParam.regNumBits]] */ val regNumBits: Int = log2Ceil(32) @@ -164,11 +128,11 @@ class LSU(param: LSUParameter) extends Module { @public val maskSelect: Vec[UInt] = IO(Output(Vec(param.lsuMSHRSize, UInt(param.maskGroupSizeBits.W)))) - /** TileLink Port to next level memory. - * TODO: rename to `tlPorts` - */ @public - val tlPort: Vec[TLBundle] = IO(Vec(param.memoryBankSize, param.tlParam.bundle())) + val axi4Port: AXI4RWIrrevocable = IO(new AXI4RWIrrevocable(param.axi4BundleParameter)) + + @public + val simpleAccessPorts: AXI4RWIrrevocable = IO(new AXI4RWIrrevocable(param.axi4BundleParameter.copy(dataWidth=32))) /** read channel to [[V]], which will redirect it to [[Lane.vrf]]. * [[vrfReadDataPorts.head.ready]] will be deasserted if there are VRF hazards. @@ -367,94 +331,89 @@ class LSU(param: LSUParameter) extends Module { } } - val accessPortA: Seq[DecoupledIO[TLChannelA]] = Seq(loadUnit.tlPortA, otherUnit.tlPort.a) - val mshrTryToUseTLAChannel: Vec[UInt] = - WireInit(VecInit(accessPortA.map( - p => - Mux( - p.valid, - VecInit(param.banks.map(bs => bs.region.matches(p.bits.address))).asUInt, - 0.U - ) - ))) - mshrTryToUseTLAChannel.foreach(select => assert(PopCount(select) <= 1.U, "address overlap")) - val sourceQueueVec: Seq[Queue[UInt]] = - tlPort.map(_ => Module(new Queue(UInt(param.mshrParam.sourceWidth.W), param.sourceQueueSize))) - // connect tile link a - val readyVec: Seq[Bool] = tlPort.zipWithIndex.map { case (tl, index) => - val port: DecoupledIO[TLChannelA] = tl.a - val storeRequest: DecoupledIO[TLChannelA] = storeUnit.tlPortA(index) - val sourceQueue: Queue[UInt] = sourceQueueVec(index) - val portFree: Bool = storeUnit.status.releasePort(index) - val Seq(loadTryToUse, otherTryToUse) = mshrTryToUseTLAChannel.map(_(index)) - val portReady = port.ready && sourceQueue.io.enq.ready - /** - * a 通道的优先级分两种情况: - * 1. store unit 声明占用时, 无条件给 store unit - * 1. 不声明占用时, load > store > other - * */ - val requestSelect = Seq( - // select load unit - portFree && loadTryToUse, - // select store unit - !portFree || (!loadTryToUse && storeRequest.valid), - // select otherUnit - portFree && !loadTryToUse && !storeRequest.valid - ) - val selectIndex: UInt = OHToUInt(requestSelect) - val (_, _, done, _) = param.mshrParam.fistLast( - // todo: use param - param.transferSize.U, - // other no burst - !port.bits.opcode(2) && !requestSelect(2), - port.fire - ) - - // 选出一个请求连到 a 通道上 - val selectBits = Mux1H(requestSelect, Seq(loadUnit.tlPortA.bits, storeRequest.bits, otherUnit.tlPort.a.bits)) - port.valid := (storeRequest.valid || ((loadTryToUse || otherTryToUse) && portFree)) && sourceQueue.io.enq.ready - port.bits := selectBits - port.bits.source := selectIndex - - // record source id by queue - sourceQueue.io.enq.valid := done - sourceQueue.io.enq.bits := selectBits.source - - // 反连 ready - storeRequest.ready := requestSelect(1) && portReady - Seq(requestSelect.head && portReady, requestSelect.last && portReady) - }.transpose.map(rv => VecInit(rv).asUInt.orR) - loadUnit.tlPortA.ready := readyVec.head - otherUnit.tlPort.a.ready := readyVec.last - - // connect tile link D - val tlDFireForOther: Vec[Bool] = Wire(Vec(param.memoryBankSize, Bool())) - tlPort.zipWithIndex.foldLeft(false.B) {case (o, (tl, index)) => - val port: DecoupledIO[TLChannelD] = tl.d - val isAccessAck = port.bits.opcode === 0.U - val (_, _, done, _) = param.mshrParam.fistLast( - // todo: use param - param.transferSize.U, - // other no burst - port.bits.opcode(0) && !port.bits.source(1), - port.fire - ) - val sourceQueue: Queue[UInt] = sourceQueueVec(index) - sourceQueue.io.deq.ready := done - // 0 -> load unit, 0b10 -> other unit - val responseForOther: Bool = port.bits.source(1) - loadUnit.tlPortD(index).valid := port.valid && !responseForOther && !isAccessAck - loadUnit.tlPortD(index).bits := port.bits - loadUnit.tlPortD(index).bits.source := sourceQueue.io.deq.bits - port.ready := isAccessAck || Mux(responseForOther, !o && otherUnit.tlPort.d.ready, loadUnit.tlPortD(index).ready) - tlDFireForOther(index) := !o && responseForOther && port.valid - o || responseForOther - } - otherUnit.tlPort.d.valid := tlDFireForOther.asUInt.orR - otherUnit.tlPort.d.bits := Mux1H(tlDFireForOther, tlPort.map(_.d.bits)) - otherUnit.tlPort.d.bits.source := Mux1H(tlDFireForOther, sourceQueueVec.map(_.io.deq.bits)) + val sourceQueue = Module(new Queue(UInt(param.mshrParam.sourceWidth.W), param.sourceQueueSize)) + // load unit connect + axi4Port.ar.valid := loadUnit.memRequest.valid && sourceQueue.io.enq.ready + axi4Port.ar.bits <> DontCare + axi4Port.ar.bits.addr := loadUnit.memRequest.bits.address + axi4Port.ar.bits.len := 0.U + axi4Port.ar.bits.size := param.mshrParam.cacheLineBits.U + axi4Port.ar.bits.burst := 1.U //INCR + loadUnit.memRequest.ready := sourceQueue.io.enq.ready && axi4Port.ar.ready + + loadUnit.memResponse.valid := axi4Port.r.valid + loadUnit.memResponse.bits.data := axi4Port.r.bits.data + loadUnit.memResponse.bits.index := sourceQueue.io.deq.bits + axi4Port.r.ready := loadUnit.memResponse.ready + + sourceQueue.io.enq.valid := loadUnit.memRequest.valid && axi4Port.ar.ready + sourceQueue.io.enq.bits := loadUnit.memRequest.bits.src + sourceQueue.io.deq.ready := axi4Port.r.fire + + // store unit <> axi + val dataQueue: Queue[MemWrite] = Module(new Queue(chiselTypeOf(storeUnit.memRequest.bits), 2)) + axi4Port.aw.valid := storeUnit.memRequest.valid && dataQueue.io.enq.ready + axi4Port.aw.bits <> DontCare + axi4Port.aw.bits.len := 0.U + axi4Port.aw.bits.burst := 1.U //INCR + axi4Port.aw.bits.size := param.mshrParam.cacheLineBits.U + axi4Port.aw.bits.addr := storeUnit.memRequest.bits.address + axi4Port.aw.bits.id := storeUnit.memRequest.bits.index + storeUnit.memRequest.ready := axi4Port.aw.ready && dataQueue.io.enq.ready + + dataQueue.io.enq.valid := storeUnit.memRequest.valid && axi4Port.aw.ready + dataQueue.io.enq.bits := storeUnit.memRequest.bits + + axi4Port.w.valid := dataQueue.io.deq.valid + axi4Port.w.bits <> DontCare + axi4Port.w.bits.data := dataQueue.io.deq.bits.data + axi4Port.w.bits.strb := dataQueue.io.deq.bits.mask + axi4Port.w.bits.last := true.B + dataQueue.io.deq.ready := axi4Port.w.ready + + // todo: add write token ? + axi4Port.b.ready := true.B + simpleAccessPorts.b.ready := true.B + + // other unit <> axi + val simpleSourceQueue: Queue[UInt] = Module(new Queue(UInt(param.mshrParam.sourceWidth.W), param.sourceQueueSize)) + simpleAccessPorts.ar.valid := otherUnit.memReadRequest.valid && simpleSourceQueue.io.enq.ready + simpleAccessPorts.ar.bits <> DontCare + simpleAccessPorts.ar.bits.addr := otherUnit.memReadRequest.bits.address + simpleAccessPorts.ar.bits.len := 0.U + simpleAccessPorts.ar.bits.size := otherUnit.memReadRequest.bits.size + simpleAccessPorts.ar.bits.burst := 1.U //INCR + otherUnit.memReadRequest.ready := simpleSourceQueue.io.enq.ready && simpleAccessPorts.ar.ready + + otherUnit.memReadResponse.valid := simpleAccessPorts.r.valid + otherUnit.memReadResponse.bits.data := simpleAccessPorts.r.bits.data + otherUnit.memReadResponse.bits.source := simpleSourceQueue.io.deq.bits + simpleAccessPorts.r.ready := otherUnit.memReadResponse.ready + + simpleSourceQueue.io.enq.valid := otherUnit.memReadResponse.valid && simpleAccessPorts.ar.ready + simpleSourceQueue.io.enq.bits := otherUnit.memReadResponse.bits.source + simpleSourceQueue.io.deq.ready := simpleAccessPorts.r.fire + + val simpleDataQueue: Queue[SimpleMemWrite] = Module(new Queue(chiselTypeOf(otherUnit.memWriteRequest.bits), 2)) + simpleAccessPorts.aw.valid := storeUnit.memRequest.valid && dataQueue.io.enq.ready + simpleAccessPorts.aw.bits <> DontCare + simpleAccessPorts.aw.bits.len := 0.U + simpleAccessPorts.aw.bits.burst := 1.U //INCR + simpleAccessPorts.aw.bits.size := otherUnit.memWriteRequest.bits.size + simpleAccessPorts.aw.bits.addr := otherUnit.memWriteRequest.bits.address + simpleAccessPorts.aw.bits.id := otherUnit.memWriteRequest.bits.source + otherUnit.memWriteRequest.ready := simpleAccessPorts.aw.ready && simpleDataQueue.io.enq.ready + + simpleDataQueue.io.enq.valid := otherUnit.memWriteRequest.valid && simpleAccessPorts.aw.ready + simpleDataQueue.io.enq.bits := otherUnit.memWriteRequest.bits + + simpleAccessPorts.w.valid := simpleDataQueue.io.deq.valid + simpleAccessPorts.w.bits <> DontCare + simpleAccessPorts.w.bits.data := simpleDataQueue.io.deq.bits.data + simpleAccessPorts.w.bits.strb := simpleDataQueue.io.deq.bits.mask + simpleAccessPorts.w.bits.last := true.B + simpleDataQueue.io.deq.ready := simpleAccessPorts.w.ready - // index offset connect otherUnit.offsetReadResult := offsetReadResult // gather last signal from all MSHR to notify LSU diff --git a/t1/src/lsu/LoadUnit.scala b/t1/src/lsu/LoadUnit.scala index c812c544f..7c28c4c0d 100644 --- a/t1/src/lsu/LoadUnit.scala +++ b/t1/src/lsu/LoadUnit.scala @@ -6,22 +6,16 @@ package org.chipsalliance.t1.rtl.lsu import chisel3._ import chisel3.experimental.hierarchy.{instantiable, public} import chisel3.util._ -import tilelink.{TLChannelA, TLChannelD} import chisel3.probe.{Probe, ProbeValue, define} -import org.chipsalliance.t1.rtl.{VRFWriteRequest, cutUInt, firstlastHelper, multiShifter} - -class cacheLineDequeueBundle(param: MSHRParam) extends Bundle { - val data: UInt = UInt((param.lsuTransposeSize * 8).W) - val index: UInt = UInt(param.cacheLineIndexBits.W) -} +import org.chipsalliance.t1.rtl.{VRFWriteRequest, cutUInt, multiShifter} @instantiable class LoadUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { /** TileLink Port which will be route to the [[LSU.tlPort]]. */ @public - val tlPortA: DecoupledIO[TLChannelA] = IO(param.tlParam.bundle().a) + val memRequest: DecoupledIO[MemRequest] = IO(Decoupled(new MemRequest(param))) @public - val tlPortD: Vec[DecoupledIO[TLChannelD]] = IO(Vec(param.memoryBankSize, param.tlParam.bundle().d)) + val memResponse: DecoupledIO[MemDataBundle] = IO(Flipped(Decoupled(new MemDataBundle(param)))) @public val status: LSUBaseStatus = IO(Output(new LSUBaseStatus)) @public @@ -38,87 +32,42 @@ class LoadUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { )) val nextCacheLineIndex = Wire(UInt(param.cacheLineIndexBits.W)) - val cacheLineIndex = RegEnable(Mux(lsuRequest.valid, 0.U, nextCacheLineIndex), tlPortA.fire || lsuRequest.valid) + val cacheLineIndex = RegEnable(Mux(lsuRequest.valid, 0.U, nextCacheLineIndex), memRequest.fire || lsuRequest.valid) nextCacheLineIndex := cacheLineIndex + 1.U val validInstruction = !invalidInstruction && lsuRequest.valid val lastRequest: Bool = cacheLineNumberReg === cacheLineIndex val sendRequest: Bool = - RegEnable(lsuRequest.valid && (csrInterface.vl > 0.U), false.B, validInstruction || (tlPortA.fire && lastRequest)) + RegEnable(lsuRequest.valid && (csrInterface.vl > 0.U), false.B, validInstruction || (memRequest.fire && lastRequest)) val requestAddress = ((lsuRequestReg.rs1Data >> param.cacheLineBits).asUInt + cacheLineIndex) ## 0.U(param.cacheLineBits.W) val writeReadyReg: Bool = RegEnable(writeReadyForLsu && !lsuRequest.valid, false.B, lsuRequest.valid || writeReadyForLsu) - tlPortA.bits.opcode := 4.U - tlPortA.bits.param := 0.U - tlPortA.bits.size := param.cacheLineBits.U - tlPortA.bits.source := cacheLineIndex - tlPortA.bits.address := requestAddress - tlPortA.bits.mask := -1.S(tlPortA.bits.mask.getWidth.W).asUInt - tlPortA.bits.data := 0.U - tlPortA.bits.corrupt := false.B - tlPortA.valid := sendRequest && !addressConflict - - val queue: Seq[DecoupledIO[TLChannelD]] = - Seq.tabulate(param.memoryBankSize)(index => Queue(tlPortD(index), burstSize)) - - val lastCacheLineAck: Vec[Bool] = Wire(Vec(param.memoryBankSize, Bool())) - val anyLastCacheLineAck = lastCacheLineAck.asUInt.orR - val cacheLineDequeue: Vec[DecoupledIO[cacheLineDequeueBundle]] = - Wire(Vec(param.memoryBankSize, Decoupled(new cacheLineDequeueBundle(param)))) - // 拼凑cache line - queue.zipWithIndex.foreach { case (port, index) => - // todo: size from channel d? - val (_, last, _, _) = param.fistLast(param.cacheLineBits.U, true.B, port.fire) - - val cacheLineValid = RegInit(false.B) - val dataShifterRegForPort = RegInit(0.U((param.lsuTransposeSize * 8).W)) - val cacheIndex = RegInit(0.U(param.cacheLineIndexBits.W)) - val dataUpdate = (port.bits.data ## dataShifterRegForPort) >> param.tlParam.d.dataWidth - when(port.fire) { - dataShifterRegForPort := dataUpdate - cacheIndex := port.bits.source - } - lastCacheLineAck(index) := port.fire && (port.bits.source === cacheLineNumberReg) - - port.ready := !cacheLineValid - cacheLineDequeue(index).valid := cacheLineValid || (port.fire & last) - cacheLineDequeue(index).bits.data := Mux( - cacheLineValid, - dataShifterRegForPort, - dataUpdate - ) - cacheLineDequeue(index).bits.index := cacheIndex + memRequest.bits.src := cacheLineIndex + memRequest.bits.address := requestAddress + memRequest.valid := sendRequest && !addressConflict - when((port.fire & last) ^ cacheLineDequeue(index).fire) { - cacheLineValid := port.fire - } - } + val anyLastCacheLineAck: Bool = memResponse.fire && (memResponse.bits.index === cacheLineNumberReg) // 接收拼凑出来的cache line // 对齐 - val alignedDequeue: DecoupledIO[cacheLineDequeueBundle] = Wire(Decoupled(new cacheLineDequeueBundle(param))) - val unalignedCacheLine: ValidIO[cacheLineDequeueBundle] = - RegInit(0.U.asTypeOf(Valid(new cacheLineDequeueBundle(param)))) + val alignedDequeue: DecoupledIO[MemDataBundle] = Wire(Decoupled(new MemDataBundle(param))) + val unalignedCacheLine: ValidIO[MemDataBundle] = RegInit(0.U.asTypeOf(Valid(new MemDataBundle(param)))) val unalignedEnqueueReady: Bool = alignedDequeue.ready || !unalignedCacheLine.valid + memResponse.ready := unalignedEnqueueReady val nextIndex: UInt = Mux(unalignedCacheLine.valid, unalignedCacheLine.bits.index + 1.U, 0.U) - val nextCacheLineMatch: Seq[Bool] = cacheLineDequeue.map(deq => deq.valid && (deq.bits.index === nextIndex)) - cacheLineDequeue.zip(nextCacheLineMatch).foreach { case (d, r) => - d.ready := r && unalignedEnqueueReady - } - val nextData: UInt = Mux1H(nextCacheLineMatch, cacheLineDequeue.map(_.bits.data)) - val dataValid: Bool = VecInit(cacheLineDequeue.zip(nextCacheLineMatch).map {case (d, r) => d.valid && r }).asUInt.orR - val unalignedEnqueueFire: Bool = dataValid && unalignedEnqueueReady + val dataValid: Bool = memResponse.valid + val unalignedEnqueueFire: Bool = memResponse.fire val alignedDequeueValid: Bool = unalignedCacheLine.valid && (dataValid || (unalignedCacheLine.bits.index === cacheLineNumberReg && lastCacheNeedPush)) // update unalignedCacheLine when(unalignedEnqueueFire) { - unalignedCacheLine.bits.data := nextData + unalignedCacheLine.bits.data := memResponse.bits.data unalignedCacheLine.bits.index := nextIndex } @@ -128,7 +77,7 @@ class LoadUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { alignedDequeue.valid := alignedDequeueValid alignedDequeue.bits.data := - multiShifter(right = true, multiSize = 8)(nextData ## unalignedCacheLine.bits.data, initOffset) + multiShifter(right = true, multiSize = 8)(memResponse.bits.data ## unalignedCacheLine.bits.data, initOffset) alignedDequeue.bits.index := unalignedCacheLine.bits.index val bufferFull: Bool = RegInit(false.B) @@ -241,15 +190,12 @@ class LoadUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { accessPtr := Mux(bufferDequeueFire, lsuRequestReg.instructionInformation.nf, accessPtr - 1.U) } - val lastCacheRequest: Bool = lastRequest && tlPortA.fire + val lastCacheRequest: Bool = lastRequest && memRequest.fire val lastCacheRequestReg: Bool = RegEnable(lastCacheRequest, true.B, lastCacheRequest || validInstruction) val lastCacheLineAckReg: Bool = RegEnable(anyLastCacheLineAck, true.B, anyLastCacheLineAck || validInstruction) val bufferClear: Bool = !( - // tile link port queue clear - queue.map(_.valid).reduce(_ || _) || - // 拼cache line 的空了 - cacheLineDequeue.map(_.valid).reduce(_ || _) || + memResponse.valid || // 对齐的空了 alignedDequeue.valid || bufferFull || @@ -284,17 +230,17 @@ class LoadUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { // valid: Load Unit try to send signal to channel A @public val tlPortAValidProbe = IO(Output(Probe(Bool()))) - define(tlPortAValidProbe, ProbeValue(tlPortA.valid)) + define(tlPortAValidProbe, ProbeValue(memRequest.valid)) @public val tlPortAReadyProbe = IO(Output(Probe(Bool()))) - define(tlPortAReadyProbe, ProbeValue(tlPortA.ready)) + define(tlPortAReadyProbe, ProbeValue(memRequest.ready)) // Fail to send signal to tilelink Channel A because of address conflict @public val addressConflictProbe = IO(Output(Probe(Bool()))) define(addressConflictProbe, ProbeValue(addressConflict)) - // Tilelink used for accepting signal from receive signal from Channel D + /*// Tilelink used for accepting signal from receive signal from Channel D @public val tlPortDValidProbe: Seq[Bool] = tlPortD.map(port => { val probe = IO(Output(Probe(Bool()))) @@ -335,7 +281,7 @@ class LoadUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { define(probe, ProbeValue(port.ready)) probe }).toSeq - +*/ // After receiving new cacheline from top, or current item is the last cacheline, // pop out data and transform it to an aligned cacheline, go through alignedDequeue to next level @public diff --git a/t1/src/lsu/SimpleAccessUnit.scala b/t1/src/lsu/SimpleAccessUnit.scala index 845a94d14..95f613d24 100644 --- a/t1/src/lsu/SimpleAccessUnit.scala +++ b/t1/src/lsu/SimpleAccessUnit.scala @@ -5,12 +5,9 @@ package org.chipsalliance.t1.rtl.lsu import chisel3._ import chisel3.experimental.hierarchy.{instantiable, public} -import chisel3.util._ import chisel3.probe._ -import chisel3.util.experimental.BitSet -import org.chipsalliance.t1.rtl.{CSRInterface, LSUBankParameter, LSURequest, VRFReadRequest, VRFWriteRequest, ffo, firstlastHelper} -import tilelink.{TLBundle, TLBundleParameter} -import org.chipsalliance.t1.rtl.lsu.MemoryWriteProbe +import chisel3.util._ +import org.chipsalliance.t1.rtl._ /** * @param datapathWidth ELEN @@ -51,10 +48,7 @@ case class MSHRParam( laneNumber: Int, paWidth: Int, lsuTransposeSize: Int, - memoryBankSize: Int, - vrfReadLatency: Int, - banks: Seq[LSUBankParameter], - outerTLParam: TLBundleParameter) { + vrfReadLatency: Int) { /** see [[LaneParameter.lmulMax]] */ val lmulMax: Int = 8 @@ -101,12 +95,6 @@ case class MSHRParam( /** The hardware length of [[maskGroupSize]] */ val maskGroupSizeBits: Int = log2Ceil(maskGroupSize) - /** override [[LSUParameter.tlParam]] to purge the MSHR source. */ - val tlParam: TLBundleParameter = outerTLParam.copy( - a = outerTLParam.a.copy(sourceWidth = sourceWidth), - d = outerTLParam.d.copy(sourceWidth = sourceWidth) - ) - /** see [[VRFParam.regNumBits]] */ val regNumBits: Int = log2Ceil(32) @@ -126,9 +114,6 @@ case class MSHRParam( * +1 Corresponding unaligned case * */ val cacheLineIndexBits: Int = log2Ceil(vLen/lsuTransposeSize + 1) - - val fistLast: (UInt, Bool, Bool) => (Bool, Bool, Bool, UInt) = - firstlastHelper(lsuTransposeSize, tlParam.a.dataWidth / 8) } /** Miss Status Handler Register @@ -179,9 +164,12 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { @public val maskSelect: ValidIO[UInt] = IO(Valid(UInt(param.maskGroupSizeBits.W))) - /** TileLink Port which will be route to the [[LSU.tlPort]]. */ @public - val tlPort: TLBundle = IO(param.tlParam.bundle()) + val memReadRequest: DecoupledIO[SimpleMemRequest] = IO(Decoupled(new SimpleMemRequest(param))) + @public + val memReadResponse: DecoupledIO[SimpleMemReadResponse] = IO(Flipped(Decoupled(new SimpleMemReadResponse(param)))) + @public + val memWriteRequest: DecoupledIO[SimpleMemWrite] = IO(Decoupled(new SimpleMemWrite(param))) /** write channel to [[V]], which will redirect it to [[Lane.vrf]]. * see [[LSU.vrfWritePort]] @@ -211,7 +199,8 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { val s0Fire: Bool = Wire(Bool()) val s1Fire: Bool = Wire(Bool()) - val s2Fire: Bool = Wire(Bool()) + val memRequestFire: Bool = memReadRequest.fire || memWriteRequest.fire + val s2Fire: Bool = memRequestFire /** request from LSU. */ val lsuRequestReg: LSURequest = RegEnable(lsuRequest.bits, 0.U.asTypeOf(lsuRequest.bits), lsuRequest.valid) @@ -621,7 +610,7 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { } /** onehot version of LSB of `tlPort.a.bits.source` */ - val memoryRequestSourceOH: UInt = UIntToOH(tlPort.a.bits.source(log2Ceil(param.maxOffsetPerLaneAccess) - 1, 0)) + val memoryRequestSourceOH: UInt = Wire(UInt(param.maxOffsetPerLaneAccess.W)) /** detect the case segment load store hazard. */ val sourceFree: Bool = !(memoryRequestSourceOH & outstandingTLDMessages).orR @@ -690,14 +679,14 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { // handle fault only first /** the current TileLink message in A Channel is the first transaction in this instruction. */ - val firstMemoryRequestOfInstruction: Bool = RegEnable(lsuRequest.valid, false.B, lsuRequest.valid || tlPort.a.fire) + val firstMemoryRequestOfInstruction: Bool = RegEnable(lsuRequest.valid, false.B, lsuRequest.valid || memReadRequest.fire) /** if assert, need to wait for memory response, this is used for fault only first instruction. */ val waitFirstMemoryResponseForFaultOnlyFirst: Bool = RegEnable( lsuRequest.valid && lsuRequest.bits.instructionInformation.fof, false.B, - lsuRequest.valid || tlPort.d.fire + lsuRequest.valid || memReadResponse.fire ) /** signal to check the the first memory request is responded for fault only first instruction. */ @@ -816,8 +805,9 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { /** request inside s1. */ val s1Reg: SimpleAccessStage1 = RegEnable(s1Wire, 0.U.asTypeOf(s1Wire), s1Fire) + val memRequestReady = Mux(lsuRequestReg.instructionInformation.isStore, memWriteRequest.ready, memReadRequest.ready) /** ready to enqueue to s2. */ - val s2EnqueueReady: Bool = tlPort.a.ready && sourceFree + val s2EnqueueReady: Bool = memRequestReady && sourceFree s1EnqueueReady := s2EnqueueReady || !s1Valid @@ -850,11 +840,11 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { s1EnqQueue.io.deq.ready := s1EnqueueReady && s1DataEnqValid s1EnqDataQueue.io.deq.ready := s1EnqueueReady - val addressInBeatByte: UInt = s1Reg.address(log2Ceil(param.tlParam.a.maskWidth) - 1, 0) + val addressInBeatByte: UInt = s1Reg.address(log2Ceil(param.laneNumber * 4) - 1, 0) // 1 -> 1 2 -> 3 4 -> 15 val baseMask: UInt = dataEEWOH(2) ## dataEEWOH(2) ## !dataEEWOH(0) ## true.B /** compute the mask for store transaction. */ - val storeMask: UInt = (baseMask << addressInBeatByte).asUInt(param.tlParam.a.maskWidth - 1, 0) + val storeMask: UInt = (baseMask << addressInBeatByte).asUInt(param.laneNumber * 4 - 1, 0) /** offset caused by element index(byte level) in the datapath. */ val storeOffsetByIndex: UInt = (s1Reg.indexInMaskGroup(1, 0) << dataEEW).asUInt(1, 0) @@ -865,9 +855,6 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { val storeData: UInt = ((s1Reg.readData << (addressInBeatByte ## 0.U(3.W))) >> (storeOffsetByIndex ## 0.U(3.W))).asUInt // only PutFull / Get for now - tlPort.a.bits.opcode := !lsuRequestReg.instructionInformation.isStore ## 0.U(2.W) - tlPort.a.bits.param := 0.U - tlPort.a.bits.size := dataEEW /** source for memory request * make volatile field LSB to reduce the source conflict. @@ -877,20 +864,25 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { s1Reg.indexInMaskGroup ## s1Reg.segmentIndex, s1Reg.indexInMaskGroup ) + memoryRequestSourceOH := UIntToOH(memoryRequestSource(log2Ceil(param.maxOffsetPerLaneAccess) - 1, 0)) // offset index + segment index // log(32)-> 5 log(8) -> 3 - tlPort.a.bits.source := memoryRequestSource - tlPort.a.bits.address := s1Reg.address - tlPort.a.bits.mask := storeMask - tlPort.a.bits.data := storeData - tlPort.a.bits.corrupt := false.B - tlPort.a.valid := s1Valid && sourceFree - s2Fire := tlPort.a.fire + memReadRequest.bits.source := memoryRequestSource + memReadRequest.bits.address := s1Reg.address + memReadRequest.bits.size := dataEEW + memReadRequest.valid := s1Valid && sourceFree && !lsuRequestReg.instructionInformation.isStore + + memWriteRequest.valid := s1Valid && sourceFree && lsuRequestReg.instructionInformation.isStore + memWriteRequest.bits.address := s1Reg.address + memWriteRequest.bits.size := dataEEW + memWriteRequest.bits.data := storeData + memWriteRequest.bits.mask := storeMask + memWriteRequest.bits.source := memoryRequestSource - val offsetRecord = RegInit(VecInit(Seq.fill(memoryRequestSourceOH.getWidth)(0.U(log2Ceil(param.tlParam.a.maskWidth).W)))) + val offsetRecord = RegInit(VecInit(Seq.fill(memoryRequestSourceOH.getWidth)(0.U(log2Ceil(param.laneNumber * 4).W)))) offsetRecord.zipWithIndex.foreach { case (d, i) => - when(tlPort.a.fire && memoryRequestSourceOH(i)) { + when(memReadRequest.fire && memoryRequestSourceOH(i)) { d := s1Reg.address } } @@ -900,23 +892,23 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { /** extract `indexInMaskGroup` from response. */ val indexInMaskGroupResponse: UInt = Mux( isSegmentLoadStore, - (tlPort.d.bits.source >> 3).asUInt, - tlPort.d.bits.source + (memReadResponse.bits.source >> 3).asUInt, + memReadResponse.bits.source )(param.maxOffsetPerLaneAccessBits - 1, 0) /** the LSB(maskGroupWidth) for response, MSHR only maintains maskGroupWidth of request. */ - val responseSourceLSBOH: UInt = UIntToOH(tlPort.d.bits.source(log2Ceil(param.maxOffsetPerLaneAccess) - 1, 0)) + val responseSourceLSBOH: UInt = UIntToOH(memReadResponse.bits.source(log2Ceil(param.maxOffsetPerLaneAccess) - 1, 0)) /** which byte to access in VRF for load instruction. * see [[storeBaseByteOffset]] */ val loadBaseByteOffset: UInt = ((groupIndex ## indexInMaskGroupResponse) << dataEEW) - vrfWritePort.valid := tlPort.d.valid && !lsuRequestReg.instructionInformation.isStore - val addressOffset = offsetRecord(tlPort.d.bits.source(log2Ceil(param.maxOffsetPerLaneAccess) - 1, 0)) ## 0.U(3.W) - tlPort.d.ready := vrfWritePort.ready + vrfWritePort.valid := memReadResponse.valid + val addressOffset = offsetRecord(memReadResponse.bits.source(log2Ceil(param.maxOffsetPerLaneAccess) - 1, 0)) ## 0.U(3.W) + memReadResponse.ready := vrfWritePort.ready // TODO: handle alignment for VRF and memory - vrfWritePort.bits.data := (((tlPort.d.bits.data >> addressOffset) << (loadBaseByteOffset(1, 0) ## 0.U(3.W)))).asUInt + vrfWritePort.bits.data := (((memReadResponse.bits.data >> addressOffset) << (loadBaseByteOffset(1, 0) ## 0.U(3.W)))).asUInt vrfWritePort.bits.last := last vrfWritePort.bits.instructionIndex := lsuRequestReg.instructionIndex vrfWritePort.bits.mask := Mux1H( @@ -934,7 +926,7 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { // vd offset for segment instructions Mux( isSegmentLoadStore, - tlPort.d.bits.source(2, 0) * segmentInstructionIndexInterval, + memReadResponse.bits.source(2, 0) * segmentInstructionIndexInterval, 0.U ) + // vd offset for element index @@ -955,18 +947,18 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { vrfWritePort.bits.offset := writeOffset // update [[outstandingTLDMessages]] - when((tlPort.d.fire || tlPort.a.fire) && !lsuRequestReg.instructionInformation.isStore) { + when((memReadResponse.fire || memReadRequest.fire) && !lsuRequestReg.instructionInformation.isStore) { // 同时进出得让相应被拉高 outstandingTLDMessages := (outstandingTLDMessages & // free outstanding source since got response from memory. ~Mux( - tlPort.d.fire, + memReadResponse.fire, responseSourceLSBOH, 0.U(param.maxOffsetPerLaneAccess.W) ): UInt) | // allocate outstanding source since corresponding memory request has been issued. Mux( - tlPort.a.fire, + memReadRequest.fire, memoryRequestSourceOH, 0.U(param.maxOffsetPerLaneAccess.W) ) @@ -982,7 +974,7 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { when( state === wResponse && noOutstandingMessages && pipelineClear // TODO: cosim bug for multiple response for same request!!! - && !tlPort.d.valid + && !memReadResponse.valid ) { // switch state to idle when(last) { @@ -1075,13 +1067,13 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { @public val s1FireProbe: Bool = IO(Output(Probe(chiselTypeOf(s1Fire)))) define(s1FireProbe, ProbeValue(s1Fire)) - +/* @public val tlPortAReadyProbe = IO(Output(Probe(Bool()))) define(tlPortAReadyProbe, ProbeValue(tlPort.a.ready)) @public val tlPortAValidProbe = IO(Output(Probe(Bool()))) - define(tlPortAValidProbe, ProbeValue(tlPort.a.valid)) + define(tlPortAValidProbe, ProbeValue(tlPort.a.valid))*/ @public val s1ValidProbe = IO(Output(Probe(Bool()))) define(s1ValidProbe, ProbeValue(s1Valid)) @@ -1092,13 +1084,13 @@ class SimpleAccessUnit(param: MSHRParam) extends Module with LSUPublic { @public val s2FireProbe: Bool = IO(Output(Probe(chiselTypeOf(s2Fire)))) define(s2FireProbe, ProbeValue(s2Fire)) - +/* @public val tlPortDReadyProbe = IO(Output(Probe(Bool()))) define(tlPortDReadyProbe, ProbeValue(tlPort.d.ready)) @public val tlPortDValidProbe = IO(Output(Probe(Bool()))) - define(tlPortDValidProbe, ProbeValue(tlPort.d.valid)) + define(tlPortDValidProbe, ProbeValue(tlPort.d.valid))*/ @public val stateValueProbe: UInt = IO(Output(Probe(chiselTypeOf(state)))) diff --git a/t1/src/lsu/StoreUnit.scala b/t1/src/lsu/StoreUnit.scala index fb3846d2d..1c9b32f86 100644 --- a/t1/src/lsu/StoreUnit.scala +++ b/t1/src/lsu/StoreUnit.scala @@ -8,7 +8,6 @@ import chisel3.experimental.hierarchy.{instantiable, public} import chisel3.util._ import chisel3.probe._ import org.chipsalliance.t1.rtl.{EmptyBundle, VRFReadRequest, cutUInt, multiShifter} -import tilelink.TLChannelA import org.chipsalliance.t1.rtl.lsu.MemoryWriteProbe class cacheLineEnqueueBundle(param: MSHRParam) extends Bundle { @@ -20,10 +19,10 @@ class cacheLineEnqueueBundle(param: MSHRParam) extends Bundle { @instantiable class StoreUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { @public - val tlPortA: Vec[DecoupledIO[TLChannelA]] = IO(Vec(param.memoryBankSize, param.tlParam.bundle().a)) + val memRequest: DecoupledIO[MemWrite] = IO(Decoupled(new MemWrite(param))) @public - val status: StoreStatus = IO(Output(new StoreStatus(param.memoryBankSize))) + val status: LSUBaseStatus = IO(Output(new LSUBaseStatus)) /** write channel to [[V]], which will redirect it to [[Lane.vrf]]. * see [[LSU.vrfWritePort]] @@ -160,15 +159,14 @@ class StoreUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { val maskForBufferData: Vec[UInt] = RegInit(VecInit(Seq.fill(8)(0.U(param.lsuTransposeSize.W)))) val maskForBufferDequeue: UInt = maskForBufferData(cacheLineIndexInBuffer) val tailLeft2: Bool = RegInit(false.B) - val alignedDequeue: DecoupledIO[cacheLineEnqueueBundle] = Wire(Decoupled(new cacheLineEnqueueBundle(param))) - val alignedDequeueFire: Bool = alignedDequeue.fire + val alignedDequeueFire: Bool = memRequest.fire // cache 不对齐的时候的上一条残留 val cacheLineTemp: UInt = RegEnable(dataBuffer.head, 0.U((param.lsuTransposeSize * 8).W), alignedDequeueFire) val maskTemp: UInt = RegInit(0.U(param.lsuTransposeSize.W)) val tailValid: Bool = RegInit(false.B) val isLastCacheLineInBuffer: Bool = cacheLineIndexInBuffer === lsuRequestReg.instructionInformation.nf val bufferWillClear: Bool = alignedDequeueFire && isLastCacheLineInBuffer - accessBufferDequeueReady := !bufferValid || (alignedDequeue.ready && isLastCacheLineInBuffer) + accessBufferDequeueReady := !bufferValid || (memRequest.ready && isLastCacheLineInBuffer) val bufferStageEnqueueData: Vec[UInt] = Mux(bufferFull, accessData, accessDataUpdate) // 处理mask, 对于 segment type 来说 一个mask 管 nf 个element val fillBySeg: UInt = Mux1H(UIntToOH(lsuRequestReg.instructionInformation.nf), Seq.tabulate(8) { segSize => @@ -243,68 +241,25 @@ class StoreUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { } // 连接 alignedDequeue - alignedDequeue.valid := bufferValid || tailValid + memRequest.valid := bufferValid || tailValid // aligned - alignedDequeue.bits.data := + memRequest.bits.data := multiShifter(right = false, multiSize = 8)(dataBuffer.head ## cacheLineTemp, initOffset) >> cacheLineTemp.getWidth val selectMaskForTail: UInt = Mux(bufferValid, maskForBufferDequeue, 0.U(maskTemp.getWidth.W)) - alignedDequeue.bits.mask := ((selectMaskForTail ## maskTemp) << initOffset) >> maskTemp.getWidth - alignedDequeue.bits.index := bufferBaseCacheLineIndex + memRequest.bits.mask := ((selectMaskForTail ## maskTemp) << initOffset) >> maskTemp.getWidth + memRequest.bits.index := bufferBaseCacheLineIndex // select by address set val alignedDequeueAddress: UInt = ((lsuRequestReg.rs1Data >> param.cacheLineBits).asUInt + bufferBaseCacheLineIndex) ## 0.U(param.cacheLineBits.W) - val selectOH: UInt = VecInit(param.banks.map(bs => bs.region.matches(alignedDequeueAddress))).asUInt - assert(PopCount(selectOH) === 1.U, "address overlap") - val currentAddress: Vec[UInt] = Wire(Vec(param.memoryBankSize, UInt(param.tlParam.a.addressWidth.W))) - val sendStageReady: Vec[Bool] = Wire(Vec(param.memoryBankSize, Bool())) - // tl 发送单元 - val readyVec = Seq.tabulate(param.memoryBankSize) { portIndex => - val dataToSend: ValidIO[cacheLineEnqueueBundle] = RegInit(0.U.asTypeOf(Valid(new cacheLineEnqueueBundle(param)))) - val addressReg: UInt = RegInit(0.U(param.paWidth.W)) - val port: DecoupledIO[TLChannelA] = tlPortA(portIndex) - val portFire: Bool = port.fire - val (first, last, done, _) = param.fistLast(param.cacheLineBits.U, true.B, portFire) - val enqueueReady: Bool = !dataToSend.valid || (port.ready && !addressConflict && last) - val enqueueFire: Bool = enqueueReady && alignedDequeue.valid && selectOH(portIndex) - val firstCacheLine = RegEnable(lsuRequest.valid, true.B, lsuRequest.valid || enqueueFire) - currentAddress(portIndex) := Mux(firstCacheLine, 0.U, addressReg) - status.releasePort(portIndex) := first - when(enqueueFire) { - dataToSend.bits := alignedDequeue.bits - addressReg := alignedDequeueAddress - }.elsewhen(portFire) { - dataToSend.bits.mask := dataToSend.bits.mask >> param.tlParam.a.maskWidth - dataToSend.bits.data := dataToSend.bits.data >> param.tlParam.a.dataWidth - } - - when(enqueueFire ^ done) { - dataToSend.valid := enqueueFire - } - - port.valid := dataToSend.valid && !addressConflict - port.bits.opcode := 0.U - port.bits.param := 0.U - port.bits.size := param.cacheLineBits.U - port.bits.source := dataToSend.bits.index - port.bits.address := addressReg - port.bits.mask := dataToSend.bits.mask(param.tlParam.a.maskWidth - 1, 0) - port.bits.data := dataToSend.bits.data(param.tlParam.a.dataWidth - 1, 0) - port.bits.corrupt := false.B - sendStageReady(portIndex) := enqueueReady - - !dataToSend.valid - } + memRequest.bits.address := alignedDequeueAddress - val sendStageClear: Bool = readyVec.reduce(_ && _) - alignedDequeue.ready := (sendStageReady.asUInt & selectOH).orR - - status.idle := sendStageClear && !bufferValid && !readStageValid && readQueueClear && !bufferFull + status.idle := !bufferValid && !readStageValid && readQueueClear && !bufferFull val idleNext: Bool = RegNext(status.idle, true.B) status.last := (!idleNext && status.idle) || invalidInstructionNext status.changeMaskGroup := maskSelect.valid && !lsuRequest.valid status.instructionIndex := lsuRequestReg.instructionIndex - status.startAddress := Mux1H(selectOH, currentAddress) + status.startAddress := alignedDequeueAddress status.endAddress := ((lsuRequestReg.rs1Data >> param.cacheLineBits).asUInt + cacheLineNumberReg) ## 0.U(param.cacheLineBits.W) dontTouch(status) @@ -314,8 +269,8 @@ class StoreUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { */ probeWire.valid := alignedDequeueFire probeWire.index := 1.U - probeWire.data := alignedDequeue.bits.data - probeWire.mask := alignedDequeue.bits.mask + probeWire.data := memRequest.bits.data + probeWire.mask := memRequest.bits.mask probeWire.address := alignedDequeueAddress // Store Unit is idle @@ -328,14 +283,14 @@ class StoreUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { val lsuRequestValidProbe = IO(Output(Probe(Bool()))) define(lsuRequestValidProbe, ProbeValue(lsuRequest.valid)) - @public +/* @public val tlPortAIsValidProbe = Seq.fill(param.memoryBankSize)(IO(Output(Probe(Bool())))) @public val tlPortAIsReadyProbe = Seq.fill(param.memoryBankSize)(IO(Output(Probe(Bool())))) tlPortA.zipWithIndex.foreach({ case(port, i) => define(tlPortAIsValidProbe(i), ProbeValue(port.valid)) define(tlPortAIsReadyProbe(i), ProbeValue(port.ready)) - }) + })*/ @public val addressConflictProbe = IO(Output(Probe(Bool()))) @@ -354,10 +309,10 @@ class StoreUnit(param: MSHRParam) extends StrideBase(param) with LSUPublic { val vrfReadyToStoreProbe = IO(Output(Probe(Bool()))) define(vrfReadyToStoreProbe, ProbeValue(vrfReadyToStore)) - @public +/* @public val alignedDequeueValidProbe = IO(Output(Probe(Bool()))) define(alignedDequeueValidProbe, ProbeValue(alignedDequeue.valid)) @public val alignedDequeueReadyProbe = IO(Output(Probe(Bool()))) - define(alignedDequeueReadyProbe, ProbeValue(alignedDequeue.ready)) + define(alignedDequeueReadyProbe, ProbeValue(alignedDequeue.ready))*/ } diff --git a/t1/src/lsu/StrideBase.scala b/t1/src/lsu/StrideBase.scala index e0c406d76..bc262de29 100644 --- a/t1/src/lsu/StrideBase.scala +++ b/t1/src/lsu/StrideBase.scala @@ -20,7 +20,6 @@ trait LSUPublic { abstract class StrideBase(param: MSHRParam) extends Module { // max nField = 8 val bufferSize: Int = 8 - val burstSize: Int = param.lsuTransposeSize * 8 / param.tlParam.d.dataWidth // 直接维护data group吧 // (vl * 8) / (datapath * laneNumber) diff --git a/t1/src/package.scala b/t1/src/package.scala index 1f2581530..c5ee29c6e 100644 --- a/t1/src/package.scala +++ b/t1/src/package.scala @@ -9,7 +9,6 @@ import chisel3.util._ import chisel3.util.experimental.decode.DecodeBundle import org.chipsalliance.t1.rtl.decoder.{Decoder, TableGenerator} import org.chipsalliance.t1.rtl.lane.Distributor -import tilelink.{TLBundleParameter, TLChannelD} package object rtl { def csa32(s: UInt, c: UInt, a: UInt): (UInt, UInt) = {