summaryrefslogtreecommitdiff
path: root/src/cpu/testers
diff options
context:
space:
mode:
Diffstat (limited to 'src/cpu/testers')
-rw-r--r--src/cpu/testers/traffic_gen/SConscript1
-rw-r--r--src/cpu/testers/traffic_gen/generators.cc307
-rw-r--r--src/cpu/testers/traffic_gen/generators.hh450
-rw-r--r--src/cpu/testers/traffic_gen/traffic_gen.cc265
-rw-r--r--src/cpu/testers/traffic_gen/traffic_gen.hh399
5 files changed, 762 insertions, 660 deletions
diff --git a/src/cpu/testers/traffic_gen/SConscript b/src/cpu/testers/traffic_gen/SConscript
index 09a06fe04..00a3a84bc 100644
--- a/src/cpu/testers/traffic_gen/SConscript
+++ b/src/cpu/testers/traffic_gen/SConscript
@@ -44,6 +44,7 @@ Import('*')
if env['HAVE_PROTOBUF']:
SimObject('TrafficGen.py')
+ Source('generators.cc')
Source('traffic_gen.cc')
DebugFlag('TrafficGen')
diff --git a/src/cpu/testers/traffic_gen/generators.cc b/src/cpu/testers/traffic_gen/generators.cc
new file mode 100644
index 000000000..5cf40b2e6
--- /dev/null
+++ b/src/cpu/testers/traffic_gen/generators.cc
@@ -0,0 +1,307 @@
+/*
+ * Copyright (c) 2012-2013 ARM Limited
+ * All rights reserved
+ *
+ * The license below extends only to copyright in the software and shall
+ * not be construed as granting a license to any other intellectual
+ * property including but not limited to intellectual property relating
+ * to a hardware implementation of the functionality of the software
+ * licensed hereunder. You may use the software subject to the license
+ * terms below provided that you ensure that this notice is replicated
+ * unmodified and in its entirety in all distributions of the software,
+ * modified or unmodified, in source code or in binary form.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met: redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer;
+ * redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution;
+ * neither the name of the copyright holders nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ * Authors: Thomas Grass
+ * Andreas Hansson
+ * Sascha Bischoff
+ */
+
+#include "base/random.hh"
+#include "cpu/testers/traffic_gen/generators.hh"
+#include "debug/TrafficGen.hh"
+#include "proto/packet.pb.h"
+
+BaseGen::BaseGen(QueuedMasterPort& _port, MasterID master_id, Tick _duration)
+ : port(_port), masterID(master_id), duration(_duration)
+{
+}
+
+void
+BaseGen::send(Addr addr, unsigned size, const MemCmd& cmd)
+{
+ // Create new request
+ Request::Flags flags;
+ Request *req = new Request(addr, size, flags, masterID);
+
+ // Embed it in a packet
+ PacketPtr pkt = new Packet(req, cmd);
+
+ uint8_t* pkt_data = new uint8_t[req->getSize()];
+ pkt->dataDynamicArray(pkt_data);
+
+ if (cmd.isWrite()) {
+ memset(pkt_data, 0xA, req->getSize());
+ }
+
+ port.schedTimingReq(pkt, curTick());
+}
+
+void
+LinearGen::enter()
+{
+ // reset the address and the data counter
+ nextAddr = startAddr;
+ dataManipulated = 0;
+
+ // this test only needs to happen once, but cannot be performed
+ // before init() is called and the ports are connected
+ if (port.deviceBlockSize() && blocksize > port.deviceBlockSize())
+ fatal("TrafficGen %s block size (%d) is larger than port"
+ " block size (%d)\n", blocksize, port.deviceBlockSize());
+}
+
+void
+LinearGen::execute()
+{
+ // choose if we generate a read or a write here
+ bool isRead = readPercent != 0 &&
+ (readPercent == 100 || random_mt.random<uint8_t>(0, 100) < readPercent);
+
+ assert((readPercent == 0 && !isRead) || (readPercent == 100 && isRead) ||
+ readPercent != 100);
+
+ DPRINTF(TrafficGen, "LinearGen::execute: %c to addr %x, size %d\n",
+ isRead ? 'r' : 'w', nextAddr, blocksize);
+
+ send(nextAddr, blocksize, isRead ? MemCmd::ReadReq : MemCmd::WriteReq);
+
+ // increment the address
+ nextAddr += blocksize;
+
+ // Add the amount of data manipulated to the total
+ dataManipulated += blocksize;
+}
+
+Tick
+LinearGen::nextExecuteTick()
+{
+ // If we have reached the end of the address space, reset the
+ // address to the start of the range
+ if (nextAddr + blocksize > endAddr) {
+ DPRINTF(TrafficGen, "Wrapping address to the start of "
+ "the range\n");
+ nextAddr = startAddr;
+ }
+
+ // Check to see if we have reached the data limit. If dataLimit is
+ // zero we do not have a data limit and therefore we will keep
+ // generating requests for the entire residency in this state.
+ if (dataLimit && dataManipulated >= dataLimit) {
+ DPRINTF(TrafficGen, "Data limit for LinearGen reached.\n");
+ // there are no more requests, therefore return MaxTick
+ return MaxTick;
+ } else {
+ // return the time when the next request should take place
+ return curTick() + random_mt.random<Tick>(minPeriod, maxPeriod);
+ }
+}
+
+void
+RandomGen::enter()
+{
+ // reset the counter to zero
+ dataManipulated = 0;
+
+ // this test only needs to happen once, but cannot be performed
+ // before init() is called and the ports are connected
+ if (port.deviceBlockSize() && blocksize > port.deviceBlockSize())
+ fatal("TrafficGen %s block size (%d) is larger than port"
+ " block size (%d)\n", blocksize, port.deviceBlockSize());
+}
+
+void
+RandomGen::execute()
+{
+ // choose if we generate a read or a write here
+ bool isRead = readPercent != 0 &&
+ (readPercent == 100 || random_mt.random<uint8_t>(0, 100) < readPercent);
+
+ assert((readPercent == 0 && !isRead) || (readPercent == 100 && isRead) ||
+ readPercent != 100);
+
+ // address of the request
+ Addr addr = random_mt.random<Addr>(startAddr, endAddr - 1);
+
+ // round down to start address of block
+ addr -= addr % blocksize;
+
+ DPRINTF(TrafficGen, "RandomGen::execute: %c to addr %x, size %d\n",
+ isRead ? 'r' : 'w', addr, blocksize);
+
+ // send a new request packet
+ send(addr, blocksize, isRead ? MemCmd::ReadReq : MemCmd::WriteReq);
+
+ // Add the amount of data manipulated to the total
+ dataManipulated += blocksize;
+}
+
+Tick
+RandomGen::nextExecuteTick()
+{
+ // Check to see if we have reached the data limit. If dataLimit is
+ // zero we do not have a data limit and therefore we will keep
+ // generating requests for the entire residency in this state.
+ if (dataLimit && dataManipulated >= dataLimit)
+ {
+ DPRINTF(TrafficGen, "Data limit for RandomGen reached.\n");
+ // No more requests. Return MaxTick.
+ return MaxTick;
+ } else {
+ // Return the time when the next request should take place.
+ return curTick() + random_mt.random<Tick>(minPeriod, maxPeriod);
+ }
+}
+
+TraceGen::InputStream::InputStream(const std::string& filename)
+ : trace(filename)
+{
+ // Create a protobuf message for the header and read it from the stream
+ Message::PacketHeader header_msg;
+ if (!trace.read(header_msg)) {
+ panic("Failed to read packet header from %s\n", filename);
+
+ if (header_msg.tick_freq() != SimClock::Frequency) {
+ panic("Trace %s was recorded with a different tick frequency %d\n",
+ header_msg.tick_freq());
+ }
+ }
+}
+
+void
+TraceGen::InputStream::reset()
+{
+ trace.reset();
+}
+
+bool
+TraceGen::InputStream::read(TraceElement& element)
+{
+ Message::Packet pkt_msg;
+ if (trace.read(pkt_msg)) {
+ element.cmd = pkt_msg.cmd();
+ element.addr = pkt_msg.addr();
+ element.blocksize = pkt_msg.size();
+ element.tick = pkt_msg.tick();
+ return true;
+ }
+
+ // We have reached the end of the file
+ return false;
+}
+
+Tick
+TraceGen::nextExecuteTick() {
+ if (traceComplete)
+ // We are at the end of the file, thus we have no more data in
+ // the trace Return MaxTick to signal that there will be no
+ // more transactions in this active period for the state.
+ return MaxTick;
+
+
+ //Reset the nextElement to the default values
+ currElement = nextElement;
+ nextElement.clear();
+
+ // We need to look at the next line to calculate the next time an
+ // event occurs, or potentially return MaxTick to signal that
+ // nothing has to be done.
+ if (!trace.read(nextElement)) {
+ traceComplete = true;
+ return MaxTick;
+ }
+
+ DPRINTF(TrafficGen, "currElement: %c addr %d size %d tick %d (%d)\n",
+ currElement.cmd.isRead() ? 'r' : 'w',
+ currElement.addr,
+ currElement.blocksize,
+ currElement.tick + tickOffset,
+ currElement.tick);
+
+ DPRINTF(TrafficGen, "nextElement: %c addr %d size %d tick %d (%d)\n",
+ nextElement.cmd.isRead() ? 'r' : 'w',
+ nextElement.addr,
+ nextElement.blocksize,
+ nextElement.tick + tickOffset,
+ nextElement.tick);
+
+ return tickOffset + nextElement.tick;
+}
+
+void
+TraceGen::enter()
+{
+ // update the trace offset to the time where the state was entered.
+ tickOffset = curTick();
+
+ // clear everything
+ nextElement.clear();
+ currElement.clear();
+
+ traceComplete = false;
+}
+
+void
+TraceGen::execute()
+{
+ // it is the responsibility of nextExecuteTick to prevent the
+ // state graph from executing the state if it should not
+ assert(currElement.isValid());
+
+ DPRINTF(TrafficGen, "TraceGen::execute: %c %d %d %d\n",
+ currElement.cmd.isRead() ? 'r' : 'w',
+ currElement.addr,
+ currElement.blocksize,
+ currElement.tick);
+
+ send(currElement.addr + addrOffset, currElement.blocksize,
+ currElement.cmd);
+}
+
+void
+TraceGen::exit()
+{
+ // Check if we reached the end of the trace file. If we did not
+ // then we want to generate a warning stating that not the entire
+ // trace was played.
+ if (!traceComplete) {
+ warn("Trace player %s was unable to replay the entire trace!\n",
+ name());
+ }
+
+ // Clear any flags and start over again from the beginning of the
+ // file
+ trace.reset();
+}
diff --git a/src/cpu/testers/traffic_gen/generators.hh b/src/cpu/testers/traffic_gen/generators.hh
new file mode 100644
index 000000000..899a3c078
--- /dev/null
+++ b/src/cpu/testers/traffic_gen/generators.hh
@@ -0,0 +1,450 @@
+/*
+ * Copyright (c) 2012-2013 ARM Limited
+ * All rights reserved
+ *
+ * The license below extends only to copyright in the software and shall
+ * not be construed as granting a license to any other intellectual
+ * property including but not limited to intellectual property relating
+ * to a hardware implementation of the functionality of the software
+ * licensed hereunder. You may use the software subject to the license
+ * terms below provided that you ensure that this notice is replicated
+ * unmodified and in its entirety in all distributions of the software,
+ * modified or unmodified, in source code or in binary form.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met: redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer;
+ * redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution;
+ * neither the name of the copyright holders nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ * Authors: Thomas Grass
+ * Andreas Hansson
+ * Sascha Bischoff
+ */
+
+/**
+ * @file
+ * Declaration of a set of generator behaviours that are used by the
+ * stand-alone traffic generator, but can also be instantiated
+ * elsewhere.
+ */
+
+#ifndef __CPU_TRAFFIC_GEN_GENERATORS_HH__
+#define __CPU_TRAFFIC_GEN_GENERATORS_HH__
+
+#include "mem/qport.hh"
+#include "proto/protoio.hh"
+
+/**
+ * Base class for all generators, with the shared functionality and
+ * virtual functions for entering, executing and leaving the
+ * generator.
+ */
+class BaseGen
+{
+
+ protected:
+
+ /** Port used to send requests */
+ QueuedMasterPort& port;
+
+ /** The MasterID used for generating requests */
+ const MasterID masterID;
+
+ /**
+ * Create a new request and associated packet and schedule
+ * it to be sent in the current tick.
+ *
+ * @param addr Physical address to use
+ * @param size Size of the request
+ * @param cmd Memory command to send
+ */
+ void send(Addr addr, unsigned size, const MemCmd& cmd);
+
+ public:
+
+ /** Time to spend in this state */
+ const Tick duration;
+
+ /**
+ * Create a base generator.
+ *
+ * @param _port port used to send requests
+ * @param master_id MasterID set on each request
+ * @param _duration duration of this state before transitioning
+ */
+ BaseGen(QueuedMasterPort& _port, MasterID master_id,
+ Tick _duration);
+
+ virtual ~BaseGen() { }
+
+ /**
+ * Get the name, useful for DPRINTFs.
+ *
+ * @return the port name
+ */
+ std::string name() const { return port.name(); }
+
+ /**
+ * Enter this generator state.
+ */
+ virtual void enter() = 0;
+
+ /**
+ * Execute this generator state.
+ */
+ virtual void execute() = 0;
+
+ /**
+ * Exit this generator state. By default do nothing.
+ */
+ virtual void exit() { };
+
+ /**
+ * Determine the next execute tick. MaxTick means that
+ * there will not be any further event in the current
+ * activation cycle of the state.
+ *
+ * @return next tick when the state should be executed
+ */
+ virtual Tick nextExecuteTick() = 0;
+
+};
+
+/**
+ * The idle generator does nothing.
+ */
+class IdleGen : public BaseGen
+{
+
+ public:
+
+ IdleGen(QueuedMasterPort& _port, MasterID master_id,
+ Tick _duration)
+ : BaseGen(_port, master_id, _duration)
+ { }
+
+ void enter() { }
+
+ void execute() { }
+
+ Tick nextExecuteTick() { return MaxTick; }
+};
+
+/**
+ * The linear generator generates sequential requests from a
+ * start to an end address, with a fixed block size. A
+ * fraction of the requests are reads, as determined by the
+ * read percent. There is an optional data limit for when to
+ * stop generating new requests.
+ */
+class LinearGen : public BaseGen
+{
+
+ public:
+
+ /**
+ * Create a linear address sequence generator. Set
+ * min_period == max_period for a fixed inter-transaction
+ * time.
+ *
+ * @param _port port used to send requests
+ * @param master_id MasterID set on each request
+ * @param _duration duration of this state before transitioning
+ * @param start_addr Start address
+ * @param end_addr End address
+ * @param _blocksize Size used for transactions injected
+ * @param min_period Lower limit of random inter-transaction time
+ * @param max_period Upper limit of random inter-transaction time
+ * @param read_percent Percent of transactions that are reads
+ * @param data_limit Upper limit on how much data to read/write
+ */
+ LinearGen(QueuedMasterPort& _port, MasterID master_id,
+ Tick _duration, Addr start_addr, Addr end_addr,
+ Addr _blocksize, Tick min_period, Tick max_period,
+ uint8_t read_percent, Addr data_limit)
+ : BaseGen(_port, master_id, _duration),
+ startAddr(start_addr), endAddr(end_addr),
+ blocksize(_blocksize), minPeriod(min_period),
+ maxPeriod(max_period), readPercent(read_percent),
+ dataLimit(data_limit)
+ { }
+
+ void enter();
+
+ void execute();
+
+ Tick nextExecuteTick();
+
+ private:
+
+ /** Start of address range */
+ const Addr startAddr;
+
+ /** End of address range */
+ const Addr endAddr;
+
+ /** Blocksize and address increment */
+ const Addr blocksize;
+
+ /** Request generation period */
+ const Tick minPeriod;
+ const Tick maxPeriod;
+
+ /**
+ * Percent of generated transactions that should be reads
+ */
+ const uint8_t readPercent;
+
+ /** Maximum amount of data to manipulate */
+ const Addr dataLimit;
+
+ /** Address of next request */
+ Addr nextAddr;
+
+ /**
+ * Counter to determine the amount of data
+ * manipulated. Used to determine if we should continue
+ * generating requests.
+ */
+ Addr dataManipulated;
+};
+
+/**
+ * The random generator is similar to the linear one, but does
+ * not generate sequential addresses. Instead it randomly
+ * picks an address in the range, aligned to the block size.
+ */
+class RandomGen : public BaseGen
+{
+
+ public:
+
+ /**
+ * Create a random address sequence generator. Set
+ * min_period == max_period for a fixed inter-transaction
+ * time.
+ *
+ * @param _port port used to send requests
+ * @param master_id MasterID set on each request
+ * @param _duration duration of this state before transitioning
+ * @param start_addr Start address
+ * @param end_addr End address
+ * @param _blocksize Size used for transactions injected
+ * @param min_period Lower limit of random inter-transaction time
+ * @param max_period Upper limit of random inter-transaction time
+ * @param read_percent Percent of transactions that are reads
+ * @param data_limit Upper limit on how much data to read/write
+ */
+ RandomGen(QueuedMasterPort& _port, MasterID master_id,
+ Tick _duration, Addr start_addr, Addr end_addr,
+ Addr _blocksize, Tick min_period, Tick max_period,
+ uint8_t read_percent, Addr data_limit)
+ : BaseGen(_port, master_id, _duration),
+ startAddr(start_addr), endAddr(end_addr),
+ blocksize(_blocksize), minPeriod(min_period),
+ maxPeriod(max_period), readPercent(read_percent),
+ dataLimit(data_limit)
+ { }
+
+ void enter();
+
+ void execute();
+
+ Tick nextExecuteTick();
+
+ private:
+
+ /** Start of address range */
+ const Addr startAddr;
+
+ /** End of address range */
+ const Addr endAddr;
+
+ /** Block size */
+ const Addr blocksize;
+
+ /** Request generation period */
+ const Tick minPeriod;
+ const Tick maxPeriod;
+
+ /**
+ * Percent of generated transactions that should be reads
+ */
+ const uint8_t readPercent;
+
+ /** Maximum amount of data to manipulate */
+ const Addr dataLimit;
+
+ /**
+ * Counter to determine the amount of data
+ * manipulated. Used to determine if we should continue
+ * generating requests.
+ */
+ Addr dataManipulated;
+};
+
+/**
+ * The trace replay generator reads a trace file and plays
+ * back the transactions. The trace is offset with respect to
+ * the time when the state was entered.
+ */
+class TraceGen : public BaseGen
+{
+
+ private:
+
+ /**
+ * This struct stores a line in the trace file.
+ */
+ struct TraceElement {
+
+ /** Specifies if the request is to be a read or a write */
+ MemCmd cmd;
+
+ /** The address for the request */
+ Addr addr;
+
+ /** The size of the access for the request */
+ Addr blocksize;
+
+ /** The time at which the request should be sent */
+ Tick tick;
+
+ /**
+ * Check validity of this element.
+ *
+ * @return if this element is valid
+ */
+ bool isValid() const {
+ return cmd != MemCmd::InvalidCmd;
+ }
+
+ /**
+ * Make this element invalid.
+ */
+ void clear() {
+ cmd = MemCmd::InvalidCmd;
+ }
+ };
+
+ /**
+ * The InputStream encapsulates a trace file and the
+ * internal buffers and populates TraceElements based on
+ * the input.
+ */
+ class InputStream
+ {
+
+ private:
+
+ /// Input file stream for the protobuf trace
+ ProtoInputStream trace;
+
+ public:
+
+ /**
+ * Create a trace input stream for a given file name.
+ *
+ * @param filename Path to the file to read from
+ */
+ InputStream(const std::string& filename);
+
+ /**
+ * Reset the stream such that it can be played once
+ * again.
+ */
+ void reset();
+
+ /**
+ * Attempt to read a trace element from the stream,
+ * and also notify the caller if the end of the file
+ * was reached.
+ *
+ * @param element Trace element to populate
+ * @return True if an element could be read successfully
+ */
+ bool read(TraceElement& element);
+ };
+
+ public:
+
+ /**
+ * Create a trace generator.
+ *
+ * @param _port port used to send requests
+ * @param master_id MasterID set on each request
+ * @param _duration duration of this state before transitioning
+ * @param trace_file File to read the transactions from
+ * @param addr_offset Positive offset to add to trace address
+ */
+ TraceGen(QueuedMasterPort& _port, MasterID master_id,
+ Tick _duration, const std::string& trace_file,
+ Addr addr_offset)
+ : BaseGen(_port, master_id, _duration),
+ trace(trace_file),
+ addrOffset(addr_offset),
+ traceComplete(false)
+ {
+ }
+
+ void enter();
+
+ void execute();
+
+ void exit();
+
+ /**
+ * Read a line of the trace file. Returns the raw tick
+ * when the next request should be generated. If the end
+ * of the file has been reached, it returns MaxTick to
+ * indicate that there will be no more requests.
+ */
+ Tick nextExecuteTick();
+
+ private:
+
+ /** Input stream used for reading the input trace file */
+ InputStream trace;
+
+ /** Store the current and next element in the trace */
+ TraceElement currElement;
+ TraceElement nextElement;
+
+ /**
+ * Stores the time when the state was entered. This is to add an
+ * offset to the times stored in the trace file.
+ */
+ Tick tickOffset;
+
+ /**
+ * Offset for memory requests. Used to shift the trace
+ * away from the CPU address space.
+ */
+ Addr addrOffset;
+
+ /**
+ * Set to true when the trace replay for one instance of
+ * state is complete.
+ */
+ bool traceComplete;
+};
+
+#endif
diff --git a/src/cpu/testers/traffic_gen/traffic_gen.cc b/src/cpu/testers/traffic_gen/traffic_gen.cc
index 292fe54e0..0aef7e061 100644
--- a/src/cpu/testers/traffic_gen/traffic_gen.cc
+++ b/src/cpu/testers/traffic_gen/traffic_gen.cc
@@ -45,7 +45,6 @@
#include "cpu/testers/traffic_gen/traffic_gen.hh"
#include "debug/Checkpoint.hh"
#include "debug/TrafficGen.hh"
-#include "proto/packet.pb.h"
#include "sim/stats.hh"
#include "sim/system.hh"
@@ -344,270 +343,6 @@ TrafficGen::StateGraph::enterState(uint32_t newState)
states[currState]->enter();
}
-TrafficGen::StateGraph::BaseGen::BaseGen(QueuedMasterPort& _port,
- MasterID master_id,
- Tick _duration)
- : port(_port), masterID(master_id), duration(_duration)
-{
-}
-
-void
-TrafficGen::StateGraph::BaseGen::send(Addr addr, unsigned size,
- const MemCmd& cmd)
-{
- // Create new request
- Request::Flags flags;
- Request *req = new Request(addr, size, flags, masterID);
-
- // Embed it in a packet
- PacketPtr pkt = new Packet(req, cmd);
-
- uint8_t* pkt_data = new uint8_t[req->getSize()];
- pkt->dataDynamicArray(pkt_data);
-
- if (cmd.isWrite()) {
- memset(pkt_data, 0xA, req->getSize());
- }
-
- port.schedTimingReq(pkt, curTick());
-}
-
-void
-TrafficGen::StateGraph::LinearGen::enter()
-{
- // reset the address and the data counter
- nextAddr = startAddr;
- dataManipulated = 0;
-
- // this test only needs to happen once, but cannot be performed
- // before init() is called and the ports are connected
- if (port.deviceBlockSize() && blocksize > port.deviceBlockSize())
- fatal("TrafficGen %s block size (%d) is larger than port"
- " block size (%d)\n", blocksize, port.deviceBlockSize());
-
-}
-
-void
-TrafficGen::StateGraph::LinearGen::execute()
-{
- // choose if we generate a read or a write here
- bool isRead = readPercent != 0 &&
- (readPercent == 100 || random_mt.random<uint8_t>(0, 100) < readPercent);
-
- assert((readPercent == 0 && !isRead) || (readPercent == 100 && isRead) ||
- readPercent != 100);
-
- DPRINTF(TrafficGen, "LinearGen::execute: %c to addr %x, size %d\n",
- isRead ? 'r' : 'w', nextAddr, blocksize);
-
- send(nextAddr, blocksize, isRead ? MemCmd::ReadReq : MemCmd::WriteReq);
-
- // increment the address
- nextAddr += blocksize;
-
- // Add the amount of data manipulated to the total
- dataManipulated += blocksize;
-}
-
-Tick
-TrafficGen::StateGraph::LinearGen::nextExecuteTick()
-{
- // If we have reached the end of the address space, reset the
- // address to the start of the range
- if (nextAddr + blocksize > endAddr) {
- DPRINTF(TrafficGen, "Wrapping address to the start of "
- "the range\n");
- nextAddr = startAddr;
- }
-
- // Check to see if we have reached the data limit. If dataLimit is
- // zero we do not have a data limit and therefore we will keep
- // generating requests for the entire residency in this state.
- if (dataLimit && dataManipulated >= dataLimit) {
- DPRINTF(TrafficGen, "Data limit for LinearGen reached.\n");
- // there are no more requests, therefore return MaxTick
- return MaxTick;
- } else {
- // return the time when the next request should take place
- return curTick() + random_mt.random<Tick>(minPeriod, maxPeriod);
- }
-}
-
-void
-TrafficGen::StateGraph::RandomGen::enter()
-{
- // reset the counter to zero
- dataManipulated = 0;
-
- // this test only needs to happen once, but cannot be performed
- // before init() is called and the ports are connected
- if (port.deviceBlockSize() && blocksize > port.deviceBlockSize())
- fatal("TrafficGen %s block size (%d) is larger than port"
- " block size (%d)\n", name(), blocksize, port.deviceBlockSize());
-}
-
-void
-TrafficGen::StateGraph::RandomGen::execute()
-{
- // choose if we generate a read or a write here
- bool isRead = readPercent != 0 &&
- (readPercent == 100 || random_mt.random<uint8_t>(0, 100) < readPercent);
-
- assert((readPercent == 0 && !isRead) || (readPercent == 100 && isRead) ||
- readPercent != 100);
-
- // address of the request
- Addr addr = random_mt.random<Addr>(startAddr, endAddr - 1);
-
- // round down to start address of block
- addr -= addr % blocksize;
-
- DPRINTF(TrafficGen, "RandomGen::execute: %c to addr %x, size %d\n",
- isRead ? 'r' : 'w', addr, blocksize);
-
- // send a new request packet
- send(addr, blocksize, isRead ? MemCmd::ReadReq : MemCmd::WriteReq);
-
- // Add the amount of data manipulated to the total
- dataManipulated += blocksize;
-}
-
-Tick
-TrafficGen::StateGraph::RandomGen::nextExecuteTick()
-{
- // Check to see if we have reached the data limit. If dataLimit is
- // zero we do not have a data limit and therefore we will keep
- // generating requests for the entire residency in this state.
- if (dataLimit && dataManipulated >= dataLimit)
- {
- DPRINTF(TrafficGen, "Data limit for RandomGen reached.\n");
- // No more requests. Return MaxTick.
- return MaxTick;
- } else {
- // Return the time when the next request should take place.
- return curTick() + random_mt.random<Tick>(minPeriod, maxPeriod);
- }
-}
-
-TrafficGen::StateGraph::TraceGen::InputStream::InputStream(const string&
- filename)
- : trace(filename)
-{
- // Create a protobuf message for the header and read it from the stream
- Message::PacketHeader header_msg;
- if (!trace.read(header_msg)) {
- panic("Failed to read packet header from %s\n", filename);
-
- if (header_msg.tick_freq() != SimClock::Frequency) {
- panic("Trace %s was recorded with a different tick frequency %d\n",
- header_msg.tick_freq());
- }
- }
-}
-
-void
-TrafficGen::StateGraph::TraceGen::InputStream::reset()
-{
- trace.reset();
-}
-
-bool
-TrafficGen::StateGraph::TraceGen::InputStream::read(TraceElement& element)
-{
- Message::Packet pkt_msg;
- if (trace.read(pkt_msg)) {
- element.cmd = pkt_msg.cmd();
- element.addr = pkt_msg.addr();
- element.blocksize = pkt_msg.size();
- element.tick = pkt_msg.tick();
- return true;
- }
-
- // We have reached the end of the file
- return false;
-}
-
-Tick
-TrafficGen::StateGraph::TraceGen::nextExecuteTick() {
- if (traceComplete)
- // We are at the end of the file, thus we have no more data in
- // the trace Return MaxTick to signal that there will be no
- // more transactions in this active period for the state.
- return MaxTick;
-
-
- //Reset the nextElement to the default values
- currElement = nextElement;
- nextElement.clear();
-
- // We need to look at the next line to calculate the next time an
- // event occurs, or potentially return MaxTick to signal that
- // nothing has to be done.
- if (!trace.read(nextElement)) {
- traceComplete = true;
- return MaxTick;
- }
-
- DPRINTF(TrafficGen, "currElement: %c addr %d size %d tick %d (%d)\n",
- currElement.cmd.isRead() ? 'r' : 'w',
- currElement.addr,
- currElement.blocksize,
- currElement.tick + tickOffset,
- currElement.tick);
-
- DPRINTF(TrafficGen, "nextElement: %c addr %d size %d tick %d (%d)\n",
- nextElement.cmd.isRead() ? 'r' : 'w',
- nextElement.addr,
- nextElement.blocksize,
- nextElement.tick + tickOffset,
- nextElement.tick);
-
- return tickOffset + nextElement.tick;
-}
-
-void
-TrafficGen::StateGraph::TraceGen::enter() {
- // update the trace offset to the time where the state was entered.
- tickOffset = curTick();
-
- // clear everything
- nextElement.clear();
- currElement.clear();
-
- traceComplete = false;
-}
-
-void
-TrafficGen::StateGraph::TraceGen::execute() {
- // it is the responsibility of nextExecuteTick to prevent the
- // state graph from executing the state if it should not
- assert(currElement.isValid());
-
- DPRINTF(TrafficGen, "TraceGen::execute: %c %d %d %d\n",
- currElement.cmd.isRead() ? 'r' : 'w',
- currElement.addr,
- currElement.blocksize,
- currElement.tick);
-
- send(currElement.addr + addrOffset, currElement.blocksize,
- currElement.cmd);
-}
-
-void
-TrafficGen::StateGraph::TraceGen::exit() {
- // Check if we reached the end of the trace file. If we did not
- // then we want to generate a warning stating that not the entire
- // trace was played.
- if (!traceComplete) {
- warn("Trace player %s was unable to replay the entire trace!\n",
- name());
- }
-
- // Clear any flags and start over again from the beginning of the
- // file
- trace.reset();
-}
-
bool
TrafficGen::TrafficGenPort::recvTimingResp(PacketPtr pkt)
{
diff --git a/src/cpu/testers/traffic_gen/traffic_gen.hh b/src/cpu/testers/traffic_gen/traffic_gen.hh
index fa08f4461..6793e24b6 100644
--- a/src/cpu/testers/traffic_gen/traffic_gen.hh
+++ b/src/cpu/testers/traffic_gen/traffic_gen.hh
@@ -38,14 +38,14 @@
* Andreas Hansson
* Sascha Bischoff
*/
-#ifndef __MEM_TRAFFIC_GEN_HH__
-#define __MEM_TRAFFIC_GEN_HH__
+#ifndef __CPU_TRAFFIC_GEN_TRAFFIC_GEN_HH__
+#define __CPU_TRAFFIC_GEN_TRAFFIC_GEN_HH__
#include "base/hashmap.hh"
+#include "cpu/testers/traffic_gen/generators.hh"
#include "mem/mem_object.hh"
#include "mem/qport.hh"
#include "params/TrafficGen.hh"
-#include "proto/protoio.hh"
/**
* The traffic generator is a master module that generates stimuli for
@@ -159,397 +159,6 @@ class TrafficGen : public MemObject
double p;
};
- /** Base class for all generator states */
- class BaseGen
- {
-
- protected:
-
- /** Port used to send requests */
- QueuedMasterPort& port;
-
- /** The MasterID used for generating requests */
- const MasterID masterID;
-
- /**
- * Create a new request and associated packet and schedule
- * it to be sent in the current tick.
- *
- * @param addr Physical address to use
- * @param size Size of the request
- * @param cmd Memory command to send
- */
- void send(Addr addr, unsigned size, const MemCmd& cmd);
-
- public:
-
- /** Time to spend in this state */
- const Tick duration;
-
- /**
- * Create a base generator.
- *
- * @param _port port used to send requests
- * @param master_id MasterID set on each request
- * @param _duration duration of this state before transitioning
- */
- BaseGen(QueuedMasterPort& _port, MasterID master_id,
- Tick _duration);
-
- virtual ~BaseGen() { }
-
- /**
- * Get the name, useful for DPRINTFs.
- *
- * @return the port name
- */
- std::string name() const { return port.name(); }
-
- /**
- * Enter this generator state.
- */
- virtual void enter() = 0;
-
- /**
- * Execute this generator state.
- */
- virtual void execute() = 0;
-
- /**
- * Exit this generator state. By default do nothing.
- */
- virtual void exit() { };
-
- /**
- * Determine the next execute tick. MaxTick means that
- * there will not be any further event in the current
- * activation cycle of the state.
- *
- * @return next tick when the state should be executed
- */
- virtual Tick nextExecuteTick() = 0;
-
- };
-
- /**
- * The idle generator does nothing.
- */
- class IdleGen : public BaseGen
- {
-
- public:
-
- IdleGen(QueuedMasterPort& _port, MasterID master_id,
- Tick _duration)
- : BaseGen(_port, master_id, _duration)
- { }
-
- void enter() { }
-
- void execute() { }
-
- Tick nextExecuteTick() { return MaxTick; }
- };
-
- /**
- * The linear generator generates sequential requests from a
- * start to an end address, with a fixed block size. A
- * fraction of the requests are reads, as determined by the
- * read percent. There is an optional data limit for when to
- * stop generating new requests.
- */
- class LinearGen : public BaseGen
- {
-
- public:
-
- /**
- * Create a linear address sequence generator. Set
- * min_period == max_period for a fixed inter-transaction
- * time.
- *
- * @param _port port used to send requests
- * @param master_id MasterID set on each request
- * @param _duration duration of this state before transitioning
- * @param start_addr Start address
- * @param end_addr End address
- * @param _blocksize Size used for transactions injected
- * @param min_period Lower limit of random inter-transaction time
- * @param max_period Upper limit of random inter-transaction time
- * @param read_percent Percent of transactions that are reads
- * @param data_limit Upper limit on how much data to read/write
- */
- LinearGen(QueuedMasterPort& _port, MasterID master_id,
- Tick _duration, Addr start_addr, Addr end_addr,
- Addr _blocksize, Tick min_period, Tick max_period,
- uint8_t read_percent, Addr data_limit)
- : BaseGen(_port, master_id, _duration),
- startAddr(start_addr), endAddr(end_addr),
- blocksize(_blocksize), minPeriod(min_period),
- maxPeriod(max_period), readPercent(read_percent),
- dataLimit(data_limit)
- { }
-
- void enter();
-
- void execute();
-
- Tick nextExecuteTick();
-
- private:
-
- /** Start of address range */
- const Addr startAddr;
-
- /** End of address range */
- const Addr endAddr;
-
- /** Blocksize and address increment */
- const Addr blocksize;
-
- /** Request generation period */
- const Tick minPeriod;
- const Tick maxPeriod;
-
- /**
- * Percent of generated transactions that should be reads
- */
- const uint8_t readPercent;
-
- /** Maximum amount of data to manipulate */
- const Addr dataLimit;
-
- /** Address of next request */
- Addr nextAddr;
-
- /**
- * Counter to determine the amount of data
- * manipulated. Used to determine if we should continue
- * generating requests.
- */
- Addr dataManipulated;
- };
-
- /**
- * The random generator is similar to the linear one, but does
- * not generate sequential addresses. Instead it randomly
- * picks an address in the range, aligned to the block size.
- */
- class RandomGen : public BaseGen
- {
-
- public:
-
- /**
- * Create a random address sequence generator. Set
- * min_period == max_period for a fixed inter-transaction
- * time.
- *
- * @param _port port used to send requests
- * @param master_id MasterID set on each request
- * @param _duration duration of this state before transitioning
- * @param start_addr Start address
- * @param end_addr End address
- * @param _blocksize Size used for transactions injected
- * @param min_period Lower limit of random inter-transaction time
- * @param max_period Upper limit of random inter-transaction time
- * @param read_percent Percent of transactions that are reads
- * @param data_limit Upper limit on how much data to read/write
- */
- RandomGen(QueuedMasterPort& _port, MasterID master_id,
- Tick _duration, Addr start_addr, Addr end_addr,
- Addr _blocksize, Tick min_period, Tick max_period,
- uint8_t read_percent, Addr data_limit)
- : BaseGen(_port, master_id, _duration),
- startAddr(start_addr), endAddr(end_addr),
- blocksize(_blocksize), minPeriod(min_period),
- maxPeriod(max_period), readPercent(read_percent),
- dataLimit(data_limit)
- { }
-
- void enter();
-
- void execute();
-
- Tick nextExecuteTick();
-
- private:
-
- /** Start of address range */
- const Addr startAddr;
-
- /** End of address range */
- const Addr endAddr;
-
- /** Block size */
- const Addr blocksize;
-
- /** Request generation period */
- const Tick minPeriod;
- const Tick maxPeriod;
-
- /**
- * Percent of generated transactions that should be reads
- */
- const uint8_t readPercent;
-
- /** Maximum amount of data to manipulate */
- const Addr dataLimit;
-
- /**
- * Counter to determine the amount of data
- * manipulated. Used to determine if we should continue
- * generating requests.
- */
- Addr dataManipulated;
- };
-
- /**
- * The trace replay generator reads a trace file and plays
- * back the transactions. The trace is offset with respect to
- * the time when the state was entered.
- */
- class TraceGen : public BaseGen
- {
-
- private:
-
- /**
- * This struct stores a line in the trace file.
- */
- struct TraceElement {
-
- /** Specifies if the request is to be a read or a write */
- MemCmd cmd;
-
- /** The address for the request */
- Addr addr;
-
- /** The size of the access for the request */
- Addr blocksize;
-
- /** The time at which the request should be sent */
- Tick tick;
-
- /**
- * Check validity of this element.
- *
- * @return if this element is valid
- */
- bool isValid() const {
- return cmd != MemCmd::InvalidCmd;
- }
-
- /**
- * Make this element invalid.
- */
- void clear() {
- cmd = MemCmd::InvalidCmd;
- }
- };
-
- /**
- * The InputStream encapsulates a trace file and the
- * internal buffers and populates TraceElements based on
- * the input.
- */
- class InputStream
- {
-
- private:
-
- /// Input file stream for the protobuf trace
- ProtoInputStream trace;
-
- public:
-
- /**
- * Create a trace input stream for a given file name.
- *
- * @param filename Path to the file to read from
- */
- InputStream(const std::string& filename);
-
- /**
- * Reset the stream such that it can be played once
- * again.
- */
- void reset();
-
- /**
- * Attempt to read a trace element from the stream,
- * and also notify the caller if the end of the file
- * was reached.
- *
- * @param element Trace element to populate
- * @return True if an element could be read successfully
- */
- bool read(TraceElement& element);
- };
-
- public:
-
- /**
- * Create a trace generator.
- *
- * @param _port port used to send requests
- * @param master_id MasterID set on each request
- * @param _duration duration of this state before transitioning
- * @param trace_file File to read the transactions from
- * @param addr_offset Positive offset to add to trace address
- */
- TraceGen(QueuedMasterPort& _port, MasterID master_id,
- Tick _duration, const std::string& trace_file,
- Addr addr_offset)
- : BaseGen(_port, master_id, _duration),
- trace(trace_file),
- addrOffset(addr_offset),
- traceComplete(false)
- {
- }
-
- void enter();
-
- void execute();
-
- void exit();
-
- /**
- * Read a line of the trace file. Returns the raw tick
- * when the next request should be generated. If the end
- * of the file has been reached, it returns MaxTick to
- * indicate that there will be no more requests.
- */
- Tick nextExecuteTick();
-
- private:
-
- /** Input stream used for reading the input trace file */
- InputStream trace;
-
- /** Store the current and next element in the trace */
- TraceElement currElement;
- TraceElement nextElement;
-
- /**
- * Stores the time when the state was entered. This is to add an
- * offset to the times stored in the trace file.
- */
- Tick tickOffset;
-
- /**
- * Offset for memory requests. Used to shift the trace
- * away from the CPU address space.
- */
- Addr addrOffset;
-
- /**
- * Set to true when the trace replay for one instance of
- * state is complete.
- */
- bool traceComplete;
- };
-
/** Pointer to owner of request handler */
TrafficGen& owner;
@@ -625,4 +234,4 @@ class TrafficGen : public MemObject
};
-#endif //__MEM_TRAFFIC_GEN_HH__
+#endif //__CPU_TRAFFIC_GEN_TRAFFIC_GEN_HH__