1
0

Refactor package hierarchy and remove legacy bus protocol implementations (#845)

* Refactors package hierarchy.

Additionally:
  - Removes legacy ground tests and configs
  - Removes legacy bus protocol implementations
  - Removes NTiles
  - Adds devices package
  - Adds more functions to util package
This commit is contained in:
Henry Cook
2017-07-07 10:48:16 -07:00
committed by GitHub
parent c28c23150d
commit 4c595d175c
238 changed files with 1347 additions and 10978 deletions

View File

@ -1,12 +1,11 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import config._
import tile._
import Instructions._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.tile.CoreModule
object ALU
{
@ -44,7 +43,9 @@ object ALU
def cmpInverted(cmd: UInt) = cmd(0)
def cmpEq(cmd: UInt) = !cmd(3)
}
import ALU._
import Instructions._
class ALU(implicit p: Parameters) extends CoreModule()(p) {
val io = new Bundle {

View File

@ -0,0 +1,106 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package freechips.rocketchip.rocket
import Chisel._
import freechips.rocketchip.config.Parameters
class StoreGen(typ: UInt, addr: UInt, dat: UInt, maxSize: Int) {
val size = typ(log2Up(log2Up(maxSize)+1)-1,0)
def misaligned =
(addr & ((UInt(1) << size) - UInt(1))(log2Up(maxSize)-1,0)).orR
def mask = {
var res = UInt(1)
for (i <- 0 until log2Up(maxSize)) {
val upper = Mux(addr(i), res, UInt(0)) | Mux(size >= UInt(i+1), UInt((BigInt(1) << (1 << i))-1), UInt(0))
val lower = Mux(addr(i), UInt(0), res)
res = Cat(upper, lower)
}
res
}
protected def genData(i: Int): UInt =
if (i >= log2Up(maxSize)) dat
else Mux(size === UInt(i), Fill(1 << (log2Up(maxSize)-i), dat((8 << i)-1,0)), genData(i+1))
def data = genData(0)
def wordData = genData(2)
}
class LoadGen(typ: UInt, signed: Bool, addr: UInt, dat: UInt, zero: Bool, maxSize: Int) {
private val size = new StoreGen(typ, addr, dat, maxSize).size
private def genData(logMinSize: Int): UInt = {
var res = dat
for (i <- log2Up(maxSize)-1 to logMinSize by -1) {
val pos = 8 << i
val shifted = Mux(addr(i), res(2*pos-1,pos), res(pos-1,0))
val doZero = Bool(i == 0) && zero
val zeroed = Mux(doZero, UInt(0), shifted)
res = Cat(Mux(size === UInt(i) || doZero, Fill(8*maxSize-pos, signed && zeroed(pos-1)), res(8*maxSize-1,pos)), zeroed)
}
res
}
def wordData = genData(2)
def data = genData(0)
}
class AMOALU(operandBits: Int)(implicit p: Parameters) extends Module {
require(operandBits == 32 || operandBits == 64)
val io = new Bundle {
val mask = UInt(INPUT, operandBits/8)
val cmd = Bits(INPUT, M_SZ)
val lhs = Bits(INPUT, operandBits)
val rhs = Bits(INPUT, operandBits)
val out = Bits(OUTPUT, operandBits)
}
val max = io.cmd === M_XA_MAX || io.cmd === M_XA_MAXU
val min = io.cmd === M_XA_MIN || io.cmd === M_XA_MINU
val add = io.cmd === M_XA_ADD
val logic_and = io.cmd === M_XA_OR || io.cmd === M_XA_AND
val logic_xor = io.cmd === M_XA_XOR || io.cmd === M_XA_OR
val adder_out =
if (operandBits == 32) io.lhs + io.rhs
else {
val mask = ~UInt(0,64) ^ (!io.mask(3) << 31)
(io.lhs & mask) + (io.rhs & mask)
}
val less = {
val sgned = {
val mask = M_XA_MIN ^ M_XA_MINU
(io.cmd & mask) === (M_XA_MIN & mask)
}
if (operandBits == 32) {
Mux(io.lhs(31) === io.rhs(31), io.lhs < io.rhs, Mux(sgned, io.lhs(31), io.rhs(31)))
} else {
val cmp_lhs = Mux(!io.mask(4), io.lhs(31), io.lhs(63))
val cmp_rhs = Mux(!io.mask(4), io.rhs(31), io.rhs(63))
val lt_lo = io.lhs(31,0) < io.rhs(31,0)
val lt_hi = io.lhs(63,32) < io.rhs(63,32)
val eq_hi = io.lhs(63,32) === io.rhs(63,32)
val lt =
Mux(io.mask(4) && io.mask(3), lt_hi || eq_hi && lt_lo,
Mux(io.mask(4), lt_hi, lt_lo))
Mux(cmp_lhs === cmp_rhs, lt, Mux(sgned, cmp_lhs, cmp_rhs))
}
}
val minmax = Mux(Mux(less, min, max), io.lhs, io.rhs)
val logic =
Mux(logic_and, io.lhs & io.rhs, 0.U) |
Mux(logic_xor, io.lhs ^ io.rhs, 0.U)
val out =
Mux(add, adder_out,
Mux(logic_and || logic_xor, logic,
minmax))
val wmask = FillInterleaved(8, io.mask)
io.out := wmask & out | ~wmask & io.lhs
}

View File

@ -1,13 +1,14 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import tile.HasCoreParameters
import util._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.coreplex.CacheBlockBytes
import freechips.rocketchip.tile.HasCoreParameters
import freechips.rocketchip.util._
case class BTBParams(
nEntries: Int = 40,
@ -18,7 +19,7 @@ case class BTBParams(
trait HasBtbParameters extends HasCoreParameters {
val btbParams = tileParams.btb.getOrElse(BTBParams(nEntries = 0))
val matchBits = btbParams.nMatchBits max log2Ceil(p(coreplex.CacheBlockBytes) * tileParams.icache.get.nSets)
val matchBits = btbParams.nMatchBits max log2Ceil(p(CacheBlockBytes) * tileParams.icache.get.nSets)
val entries = btbParams.nEntries
val nRAS = btbParams.nRAS
val updatesOutOfOrder = btbParams.updatesOutOfOrder

View File

@ -1,12 +1,12 @@
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import tile._
import util._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.tile.{CoreModule, CoreBundle}
import freechips.rocketchip.util._
class BPControl(implicit p: Parameters) extends CoreBundle()(p) {
val ttype = UInt(width = 4)

View File

@ -1,17 +1,15 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package rocket
import collection.mutable.LinkedHashMap
package freechips.rocketchip.rocket
import Chisel._
import Instructions._
import config._
import tile._
import uncore.devices._
import util._
import Chisel.ImplicitConversions._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.tile._
import freechips.rocketchip.util._
import scala.collection.mutable.LinkedHashMap
import Instructions._
class MStatus extends Bundle {
// not truly part of mstatus, but convenient

View File

@ -1,9 +1,9 @@
// See LICENSE.Berkeley for license details.
package rocket
package constants
package freechips.rocketchip.rocket.constants
import Chisel._
import freechips.rocketchip.util._
import scala.math._
trait ScalarOpConstants {
@ -59,3 +59,38 @@ trait ScalarOpConstants {
def DW_64 = Bool(true)
def DW_XPR = DW_64
}
trait MemoryOpConstants {
val NUM_XA_OPS = 9
val M_SZ = 5
def M_X = BitPat("b?????");
def M_XRD = UInt("b00000"); // int load
def M_XWR = UInt("b00001"); // int store
def M_PFR = UInt("b00010"); // prefetch with intent to read
def M_PFW = UInt("b00011"); // prefetch with intent to write
def M_XA_SWAP = UInt("b00100");
def M_FLUSH_ALL = UInt("b00101") // flush all lines
def M_XLR = UInt("b00110");
def M_XSC = UInt("b00111");
def M_XA_ADD = UInt("b01000");
def M_XA_XOR = UInt("b01001");
def M_XA_OR = UInt("b01010");
def M_XA_AND = UInt("b01011");
def M_XA_MIN = UInt("b01100");
def M_XA_MAX = UInt("b01101");
def M_XA_MINU = UInt("b01110");
def M_XA_MAXU = UInt("b01111");
def M_FLUSH = UInt("b10000") // write back dirty data and cede R/W permissions
def M_PWR = UInt("b10001") // partial (masked) store
def M_PRODUCE = UInt("b10010") // write back dirty data and cede W permissions
def M_CLEAN = UInt("b10011") // write back dirty data and retain R/W permissions
def M_SFENCE = UInt("b10100") // flush TLB
def isAMOLogical(cmd: UInt) = cmd.isOneOf(M_XA_SWAP, M_XA_XOR, M_XA_OR, M_XA_AND)
def isAMOArithmetic(cmd: UInt) = cmd.isOneOf(M_XA_ADD, M_XA_MIN, M_XA_MAX, M_XA_MINU, M_XA_MAXU)
def isAMO(cmd: UInt) = isAMOLogical(cmd) || isAMOArithmetic(cmd)
def isPrefetch(cmd: UInt) = cmd === M_PFR || cmd === M_PFW
def isRead(cmd: UInt) = cmd === M_XRD || cmd === M_XLR || cmd === M_XSC || isAMO(cmd)
def isWrite(cmd: UInt) = cmd === M_XWR || cmd === M_PWR || cmd === M_XSC || isAMO(cmd)
def isWriteIntent(cmd: UInt) = isWrite(cmd) || cmd === M_PFW || cmd === M_XLR
}

View File

@ -1,15 +1,14 @@
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import diplomacy._
import uncore.constants._
import uncore.tilelink2._
import uncore.util._
import util._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.coreplex.{RationalCrossing, RocketCrossing, RocketTilesKey}
import freechips.rocketchip.diplomacy.AddressSet
import freechips.rocketchip.tilelink._
import freechips.rocketchip.util._
import TLMessages._
class DCacheDataReq(implicit p: Parameters) extends L1HellaCacheBundle()(p) {
@ -83,8 +82,8 @@ class DCacheModule(outer: DCache) extends HellaCacheModule(outer) {
dataArb.io.out.ready := true
metaArb.io.out.ready := true
val rational = p(coreplex.RocketCrossing) match {
case coreplex.RationalCrossing(_) => true
val rational = p(RocketCrossing) match {
case RationalCrossing(_) => true
case _ => false
}
@ -160,7 +159,7 @@ class DCacheModule(outer: DCache) extends HellaCacheModule(outer) {
val s1_victim_way = Wire(init = replacer.way)
val (s1_hit_way, s1_hit_state, s1_meta, s1_victim_meta) =
if (usingDataScratchpad) {
val baseAddr = GetPropertyByHartId(p(coreplex.RocketTilesKey), _.dcache.flatMap(_.scratch.map(_.U)), io.hartid)
val baseAddr = GetPropertyByHartId(p(RocketTilesKey), _.dcache.flatMap(_.scratch.map(_.U)), io.hartid)
val inScratchpad = s1_paddr >= baseAddr && s1_paddr < baseAddr + nSets * cacheBlockBytes
val hitState = Mux(inScratchpad, ClientMetadata.maximum, ClientMetadata.onReset)
val dummyMeta = L1Metadata(UInt(0), ClientMetadata.onReset)

View File

@ -1,20 +1,21 @@
// See LICENSE.Berkeley for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import scala.collection.mutable.{ArrayBuffer, Map, Set}
object DecodeLogic
{
def term(lit: BitPat) =
new Term(lit.value, BigInt(2).pow(lit.getWidth)-(lit.mask+1))
def logic(addr: UInt, addrWidth: Int, cache: scala.collection.mutable.Map[Term,Bool], terms: Seq[Term]) = {
def logic(addr: UInt, addrWidth: Int, cache: Map[Term,Bool], terms: Seq[Term]) = {
terms.map { t =>
cache.getOrElseUpdate(t, (if (t.mask == 0) addr else addr & Bits(BigInt(2).pow(addrWidth)-(t.mask+1), addrWidth)) === Bits(t.value, addrWidth))
}.foldLeft(Bool(false))(_||_)
}
def apply(addr: UInt, default: BitPat, mapping: Iterable[(BitPat, BitPat)]): UInt = {
val cache = caches.getOrElseUpdate(addr, collection.mutable.Map[Term,Bool]())
val cache = caches.getOrElseUpdate(addr, Map[Term,Bool]())
val dterm = term(default)
val (keys, values) = mapping.unzip
val addrWidth = keys.map(_.getWidth).max
@ -41,7 +42,7 @@ object DecodeLogic
}).reverse)
}
def apply(addr: UInt, default: Seq[BitPat], mappingIn: Iterable[(BitPat, Seq[BitPat])]): Seq[UInt] = {
val mapping = collection.mutable.ArrayBuffer.fill(default.size)(collection.mutable.ArrayBuffer[(BitPat, BitPat)]())
val mapping = ArrayBuffer.fill(default.size)(ArrayBuffer[(BitPat, BitPat)]())
for ((key, values) <- mappingIn)
for ((value, i) <- values zipWithIndex)
mapping(i) += key -> value
@ -52,7 +53,7 @@ object DecodeLogic
apply(addr, default, mappingIn.map(m => (BitPat(m._1), m._2)).asInstanceOf[Iterable[(BitPat, Seq[BitPat])]])
def apply(addr: UInt, trues: Iterable[UInt], falses: Iterable[UInt]): Bool =
apply(addr, BitPat.dontCare(1), trues.map(BitPat(_) -> BitPat("b1")) ++ falses.map(BitPat(_) -> BitPat("b0"))).toBool
private val caches = collection.mutable.Map[UInt,collection.mutable.Map[Term,Bool]]()
private val caches = Map[UInt,Map[Term,Bool]]()
}
class Term(val value: BigInt, val mask: BigInt = 0)
@ -87,7 +88,7 @@ object Simplify
var prime = List[Term]()
implicants.foreach(_.prime = true)
val cols = (0 to bits).map(b => implicants.filter(b == _.mask.bitCount))
val table = cols.map(c => (0 to bits).map(b => collection.mutable.Set(c.filter(b == _.value.bitCount):_*)))
val table = cols.map(c => (0 to bits).map(b => Set(c.filter(b == _.value.bitCount):_*)))
for (i <- 0 to bits) {
for (j <- 0 until bits-i)
table(i)(j).foreach(a => table(i+1)(j) ++= table(i)(j+1).filter(_.similar(a)).map(_.merge(a)))
@ -127,7 +128,7 @@ object Simplify
}
def getCover(implicants: Seq[Term], minterms: Seq[Term], bits: Int) = {
if (minterms.nonEmpty) {
val cover = minterms.map(m => implicants.filter(_.covers(m)).map(i => collection.mutable.Set(i)))
val cover = minterms.map(m => implicants.filter(_.covers(m)).map(i => Set(i)))
val all = cover.reduceLeft((c0, c1) => c0.map(a => c1.map(_ ++ a)).reduceLeft(_++_))
all.map(_.toList).reduceLeft((a, b) => if (cheaper(a, b, bits)) a else b)
} else
@ -164,7 +165,7 @@ object SimplifyDC
minterms.foreach(_.prime = true)
var mint = minterms.map(t => new Term(t.value, t.mask))
val cols = (0 to bits).map(b => mint.filter(b == _.mask.bitCount))
val table = cols.map(c => (0 to bits).map(b => collection.mutable.Set(c.filter(b == _.value.bitCount):_*)))
val table = cols.map(c => (0 to bits).map(b => Set(c.filter(b == _.value.bitCount):_*)))
for (i <- 0 to bits) {
for (j <- 0 until bits-i) {

View File

@ -1,10 +1,10 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import util._
import Chisel._
import freechips.rocketchip.util._
class EventSet(gate: (UInt, UInt) => Bool, events: Seq[(String, () => Bool)]) {
def size = events.size

View File

@ -1,17 +1,17 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import chisel3.core.withReset
import config._
import coreplex._
import diplomacy._
import uncore.tilelink2._
import tile._
import util._
import freechips.rocketchip.config._
import freechips.rocketchip.coreplex._
import freechips.rocketchip.diplomacy.{LazyModule, LazyModuleImp}
import freechips.rocketchip.tilelink._
import freechips.rocketchip.tile._
import freechips.rocketchip.util._
class FrontendReq(implicit p: Parameters) extends CoreBundle()(p) {
val pc = UInt(width = vaddrBitsExtended)

View File

@ -1,17 +1,15 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import config.{Parameters, Field}
import coreplex._
import diplomacy._
import tile._
import uncore.constants._
import uncore.tilelink2._
import uncore.util.{Code, IdentityCode}
import util.{ParameterizedBundle, RandomReplacement}
import freechips.rocketchip.config.{Parameters, Field}
import freechips.rocketchip.coreplex._
import freechips.rocketchip.diplomacy._
import freechips.rocketchip.tile._
import freechips.rocketchip.tilelink._
import freechips.rocketchip.util._
import scala.collection.mutable.ListBuffer
import scala.math.max

View File

@ -1,11 +1,10 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import config._
import util.{ParameterizedBundle, DecoupledHelper}
import freechips.rocketchip.config.Parameters
class HellaCacheArbiter(n: Int)(implicit p: Parameters) extends Module
{
@ -65,51 +64,3 @@ class HellaCacheArbiter(n: Int)(implicit p: Parameters) extends Module
}
}
}
class InOrderArbiter[T <: Data, U <: Data](reqTyp: T, respTyp: U, n: Int)
(implicit p: Parameters) extends Module {
val io = new Bundle {
val in_req = Vec(n, Decoupled(reqTyp)).flip
val in_resp = Vec(n, Decoupled(respTyp))
val out_req = Decoupled(reqTyp)
val out_resp = Decoupled(respTyp).flip
}
if (n > 1) {
val route_q = Module(new Queue(UInt(width = log2Up(n)), 2))
val req_arb = Module(new RRArbiter(reqTyp, n))
req_arb.io.in <> io.in_req
val req_helper = DecoupledHelper(
req_arb.io.out.valid,
route_q.io.enq.ready,
io.out_req.ready)
io.out_req.bits := req_arb.io.out.bits
io.out_req.valid := req_helper.fire(io.out_req.ready)
route_q.io.enq.bits := req_arb.io.chosen
route_q.io.enq.valid := req_helper.fire(route_q.io.enq.ready)
req_arb.io.out.ready := req_helper.fire(req_arb.io.out.valid)
val resp_sel = route_q.io.deq.bits
val resp_ready = io.in_resp(resp_sel).ready
val resp_helper = DecoupledHelper(
resp_ready,
route_q.io.deq.valid,
io.out_resp.valid)
val resp_valid = resp_helper.fire(resp_ready)
for (i <- 0 until n) {
io.in_resp(i).bits := io.out_resp.bits
io.in_resp(i).valid := resp_valid && resp_sel === UInt(i)
}
route_q.io.deq.ready := resp_helper.fire(route_q.io.deq.valid)
io.out_resp.ready := resp_helper.fire(io.out_resp.valid)
} else {
io.out_req <> io.in_req.head
io.in_resp.head <> io.out_resp
}
}

View File

@ -1,12 +1,12 @@
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import tile._
import util._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.tile._
import freechips.rocketchip.util._
class Instruction(implicit val p: Parameters) extends ParameterizedBundle with HasCoreParameters {
val xcpt0 = new FrontendExceptions // exceptions on first half of instruction

View File

@ -1,16 +1,16 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import config._
import diplomacy._
import tile._
import uncore.tilelink2._
import uncore.util._
import util._
import Chisel.ImplicitConversions._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.coreplex.RocketTilesKey
import freechips.rocketchip.diplomacy._
import freechips.rocketchip.tile._
import freechips.rocketchip.tilelink._
import freechips.rocketchip.util._
case class ICacheParams(
nSets: Int = 64,
@ -106,7 +106,7 @@ class ICacheModule(outer: ICache) extends LazyModuleImp(outer)
val scratchpadMax = tl_in.map(tl => Reg(UInt(width = log2Ceil(nSets * (nWays - 1)))))
def lineInScratchpad(line: UInt) = scratchpadMax.map(scratchpadOn && line <= _).getOrElse(false.B)
def addrMaybeInScratchpad(addr: UInt) = if (outer.icacheParams.itimAddr.isEmpty) false.B else {
val base = GetPropertyByHartId(p(coreplex.RocketTilesKey), _.icache.flatMap(_.itimAddr.map(_.U)), io.hartid)
val base = GetPropertyByHartId(p(RocketTilesKey), _.icache.flatMap(_.itimAddr.map(_.U)), io.hartid)
addr >= base && addr < base + outer.size
}
def addrInScratchpad(addr: UInt) = addrMaybeInScratchpad(addr) && lineInScratchpad(addr(untagBits+log2Ceil(nWays)-1, blockOffBits))
@ -224,8 +224,8 @@ class ICacheModule(outer: ICache) extends LazyModuleImp(outer)
// output signals
outer.icacheParams.latency match {
case 1 =>
require(tECC.isInstanceOf[uncore.util.IdentityCode])
require(dECC.isInstanceOf[uncore.util.IdentityCode])
require(tECC.isInstanceOf[IdentityCode])
require(dECC.isInstanceOf[IdentityCode])
require(outer.icacheParams.itimAddr.isEmpty)
io.resp.bits.data := Mux1H(s1_tag_hit, s1_dout)
io.resp.bits.ae := s1_tl_error.asUInt.orR

View File

@ -1,16 +1,15 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.tile.HasCoreParameters
import freechips.rocketchip.util._
import Instructions._
import uncore.constants.MemoryOpConstants._
import ALU._
import config._
import tile.HasCoreParameters
import util._
abstract trait DecodeConstants extends HasCoreParameters
{

View File

@ -1,9 +1,10 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import scala.collection.mutable.ArrayBuffer
/* Automatically generated by parse-opcodes */
object Instructions {
@ -231,7 +232,7 @@ object Causes {
val load_page_fault = 0xd
val store_page_fault = 0xf
val all = {
val res = collection.mutable.ArrayBuffer[Int]()
val res = ArrayBuffer[Int]()
res += misaligned_fetch
res += fetch_access
res += illegal_instruction
@ -463,7 +464,7 @@ object CSRs {
val mhpmcounter30h = 0xb9e
val mhpmcounter31h = 0xb9f
val all = {
val res = collection.mutable.ArrayBuffer[Int]()
val res = ArrayBuffer[Int]()
res += fflags
res += frm
res += fcsr
@ -615,7 +616,7 @@ object CSRs {
res.toArray
}
val all32 = {
val res = collection.mutable.ArrayBuffer(all:_*)
val res = ArrayBuffer(all:_*)
res += cycleh
res += timeh
res += instreth

View File

@ -1,12 +1,12 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import ALU._
import util._
import Chisel.ImplicitConversions._
import freechips.rocketchip.util._
import ALU._
class MultiplierReq(dataBits: Int, tagBits: Int) extends Bundle {
val fn = Bits(width = SZ_ALU_FN)

View File

@ -1,16 +1,14 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import diplomacy._
import uncore.constants._
import uncore.tilelink2._
import uncore.util._
import util._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.diplomacy._
import freechips.rocketchip.tilelink._
import freechips.rocketchip.util._
trait HasMissInfo extends HasL1HellaCacheParameters {
val tag_match = Bool()

View File

@ -1,12 +1,12 @@
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import tile._
import util._
import freechips.rocketchip.config._
import freechips.rocketchip.tile._
import freechips.rocketchip.util._
class PMPConfig extends Bundle {
val l = Bool()

View File

@ -1,18 +1,15 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import tile._
import coreplex.CacheBlockBytes
import uncore.constants._
import uncore.tilelink2._
import util._
import uncore.util.ParityCode
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.coreplex.CacheBlockBytes
import freechips.rocketchip.tile._
import freechips.rocketchip.tilelink._
import freechips.rocketchip.util._
import scala.collection.mutable.ListBuffer
class PTWReq(implicit p: Parameters) extends CoreBundle()(p) {

View File

@ -1,12 +1,12 @@
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import tile._
import util._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.tile._
import freechips.rocketchip.util._
class ExpandedInstruction extends Bundle {
val bits = UInt(width = 32)

View File

@ -1,17 +1,16 @@
// See LICENSE.Berkeley for license details.
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import chisel3.core.withReset
import config._
import tile._
import uncore.constants._
import diplomacy._
import util._
import Chisel.ImplicitConversions._
import collection.immutable.ListMap
import chisel3.core.withReset
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.tile._
import freechips.rocketchip.util._
import scala.collection.immutable.ListMap
import scala.collection.mutable.ArrayBuffer
case class RocketCoreParams(
bootFreqHz: BigInt = 0,
@ -401,7 +400,7 @@ class Rocket(implicit p: Parameters) extends CoreModule()(p)
mem_reg_wdata := alu.io.out
when (ex_ctrl.rxs2 && (ex_ctrl.mem || ex_ctrl.rocc || ex_sfence)) {
val typ = Mux(ex_ctrl.rocc, log2Ceil(xLen/8).U, ex_ctrl.mem_type)
mem_reg_rs2 := new uncore.util.StoreGen(typ, 0.U, ex_rs(1), coreDataBytes).data
mem_reg_rs2 := new StoreGen(typ, 0.U, ex_rs(1), coreDataBytes).data
}
}
@ -724,7 +723,7 @@ class Rocket(implicit p: Parameters) extends CoreModule()(p)
class RegFile(n: Int, w: Int, zero: Boolean = false) {
private val rf = Mem(n, UInt(width = w))
private def access(addr: UInt) = rf(~addr(log2Up(n)-1,0))
private val reads = collection.mutable.ArrayBuffer[(UInt,UInt)]()
private val reads = ArrayBuffer[(UInt,UInt)]()
private var canRead = true
def read(addr: UInt) = {
require(canRead)

View File

@ -1,16 +1,16 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import config._
import coreplex._
import diplomacy._
import tile._
import uncore.devices._
import uncore.tilelink2._
import util._
import freechips.rocketchip.config._
import freechips.rocketchip.coreplex._
import freechips.rocketchip.diplomacy._
import freechips.rocketchip.tile._
import freechips.rocketchip.tilelink._
import freechips.rocketchip.util._
case class RocketTileParams(
core: RocketCoreParams = RocketCoreParams(),
@ -24,6 +24,7 @@ case class RocketTileParams(
}
class RocketTile(val rocketParams: RocketTileParams, val hartid: Int)(implicit p: Parameters) extends BaseTile(rocketParams)(p)
with HasExternalInterrupts
with HasLazyRoCC // implies CanHaveSharedFPU with CanHavePTW with HasHellaCache
with CanHaveScratchpad { // implies CanHavePTW with HasHellaCache with HasICacheFrontend
@ -124,9 +125,11 @@ class RocketTile(val rocketParams: RocketTileParams, val hartid: Int)(implicit p
}
class RocketTileBundle(outer: RocketTile) extends BaseTileBundle(outer)
with HasExternalInterruptsBundle
with CanHaveScratchpadBundle
class RocketTileModule(outer: RocketTile) extends BaseTileModule(outer, () => new RocketTileBundle(outer))
with HasExternalInterruptsModule
with HasLazyRoCCModule
with CanHaveScratchpadModule {
@ -257,7 +260,7 @@ class RationalRocketTile(rtp: RocketTileParams, hartid: Int)(implicit p: Paramet
masterNode :=* source.node
val slaveNode = new TLRationalInputNode() { override def reverse = true }
val sink = LazyModule(new TLRationalCrossingSink(util.SlowToFast))
val sink = LazyModule(new TLRationalCrossingSink(SlowToFast))
rocket.slaveNode :*= sink.node
sink.node :*= slaveNode

View File

@ -1,17 +1,16 @@
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import coreplex.CacheBlockBytes
import diplomacy._
import tile._
import uncore.constants._
import uncore.tilelink2._
import uncore.util._
import util._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.coreplex.CacheBlockBytes
import freechips.rocketchip.diplomacy._
import freechips.rocketchip.tile._
import freechips.rocketchip.tilelink._
import freechips.rocketchip.util._
class ScratchpadSlavePort(address: AddressSet)(implicit p: Parameters) extends LazyModule
with HasCoreParameters {

View File

@ -1,12 +1,13 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import util._
import freechips.rocketchip.config.Parameters
import freechips.rocketchip.util._
/**
* This module buffers requests made by the SimpleHellaCacheIF in case they

View File

@ -1,17 +1,17 @@
// See LICENSE.SiFive for license details.
// See LICENSE.Berkeley for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import Chisel.ImplicitConversions._
import config._
import diplomacy._
import coreplex.CacheBlockBytes
import tile.{XLen, CoreModule, CoreBundle}
import uncore.tilelink2._
import uncore.constants._
import util._
import freechips.rocketchip.config.{Field, Parameters}
import freechips.rocketchip.coreplex.CacheBlockBytes
import freechips.rocketchip.diplomacy.RegionType
import freechips.rocketchip.tile.{XLen, CoreModule, CoreBundle}
import freechips.rocketchip.tilelink._
import freechips.rocketchip.util._
case object PAddrBits extends Field[Int]
case object PgLevels extends Field[Int]

View File

@ -1,10 +1,11 @@
// See LICENSE.SiFive for license details.
package rocket
package freechips.rocketchip.rocket
import Chisel._
import diplomacy._
import uncore.tilelink2._
import freechips.rocketchip.diplomacy._
import freechips.rocketchip.tilelink._
case class TLBPermissions(
homogeneous: Bool, // if false, the below are undefined

View File

@ -1,3 +1,5 @@
// See LICENSE.Berkeley for license details.
package object rocket extends rocket.constants.ScalarOpConstants
package freechips.rocketchip
package object rocket extends rocket.constants.ScalarOpConstants with rocket.constants.MemoryOpConstants