[BOLT] Initial experimental instrumentation pass

Summary:
An instrumentation pass that modifies the input binary to
generate a profile after execution finishes. It modifies branches to
increment counters stored in the process memory and injects a new
function that dumps this data to an fdata file, readable by BOLT.

This instrumentation is experimental and currently uses a naive
approach where every branch is instrumented. This is not ideal for
runtime performance, but should be good enough for us to
evaluate/debug LBR profile quality against instrumentation.

Does not support instrumenting indirect calls yet, only direct
calls, direct branches and indirect local branches.

(cherry picked from FBD15998096)
This commit is contained in:
Rafael Auler 2019-06-19 20:10:49 -07:00 committed by Maksim Panchenko
parent db02a1a142
commit 0d23cbaa52
15 changed files with 984 additions and 25 deletions

View File

@ -255,7 +255,7 @@ void BinaryBasicBlock::replaceSuccessor(BinaryBasicBlock *Succ,
BinaryBasicBlock *NewSucc,
uint64_t Count,
uint64_t MispredictedCount) {
Succ->removePredecessor(this);
Succ->removePredecessor(this, /*Multiple=*/false);
auto I = succ_begin();
auto BI = BranchInfo.begin();
for (; I != succ_end(); ++I) {
@ -280,7 +280,7 @@ void BinaryBasicBlock::removeAllSuccessors() {
}
void BinaryBasicBlock::removeSuccessor(BinaryBasicBlock *Succ) {
Succ->removePredecessor(this);
Succ->removePredecessor(this, /*Multiple=*/false);
auto I = succ_begin();
auto BI = BranchInfo.begin();
for (; I != succ_end(); ++I) {
@ -299,13 +299,16 @@ void BinaryBasicBlock::addPredecessor(BinaryBasicBlock *Pred) {
Predecessors.push_back(Pred);
}
void BinaryBasicBlock::removePredecessor(BinaryBasicBlock *Pred) {
void BinaryBasicBlock::removePredecessor(BinaryBasicBlock *Pred,
bool Multiple) {
// Note: the predecessor could be listed multiple times.
bool Erased{false};
for (auto PredI = Predecessors.begin(); PredI != Predecessors.end(); ) {
if (*PredI == Pred) {
Erased = true;
PredI = Predecessors.erase(PredI);
if (!Multiple)
return;
} else {
++PredI;
}

View File

@ -364,13 +364,17 @@ public:
/// Find the fallthrough successor for a block, or nullptr if there is
/// none.
const BinaryBasicBlock* getFallthrough() const {
BinaryBasicBlock* getFallthrough() {
if (succ_size() == 2)
return getConditionalSuccessor(false);
else
return getSuccessor();
}
const BinaryBasicBlock *getFallthrough() const {
return const_cast<BinaryBasicBlock *>(this)->getFallthrough();
}
/// Return branch info corresponding to a taken branch.
const BinaryBranchInfo &getTakenBranchInfo() const {
assert(BranchInfo.size() == 2 &&
@ -753,6 +757,11 @@ public:
return Instructions.emplace(At, std::move(NewInst));
}
iterator insertInstruction(iterator At, MCInst &NewInst) {
adjustNumPseudos(NewInst, 1);
return Instructions.emplace(At, NewInst);
}
/// Helper to retrieve any terminators in \p BB before \p Pos. This is used
/// to skip CFI instructions and to retrieve the first terminator instruction
/// in basic blocks with two terminators (conditional jump and unconditional
@ -897,7 +906,10 @@ private:
/// Remove predecessor of the basic block. Don't use directly, instead
/// use removeSuccessor() function.
void removePredecessor(BinaryBasicBlock *Pred);
/// If \p Multiple is set to true, it will remove all predecessors that
/// are equal to \p Pred. Otherwise, the first instance of \p Pred found
/// will be removed. This only matters in awkward, redundant CFGs.
void removePredecessor(BinaryBasicBlock *Pred, bool Multiple=true);
/// Return offset of the basic block from the function start.
uint32_t getOffset() const {

View File

@ -57,6 +57,7 @@ extern bool shouldProcess(const BinaryFunction &);
extern cl::opt<bool> UpdateDebugSections;
extern cl::opt<unsigned> Verbosity;
extern cl::opt<bool> EnableBAT;
extern cl::opt<bool> Instrument;
cl::opt<bool>
AlignBlocks("align-blocks",
@ -1794,7 +1795,7 @@ void BinaryFunction::postProcessCFG() {
// translation table later. This has no cost, since annotations are allocated
// by a bumpptr allocator and won't be released anyway until late in the
// pipeline.
if (!opts::EnableBAT)
if (!opts::EnableBAT && !opts::Instrument)
for (auto *BB : layout())
for (auto &Inst : *BB)
BC.MIB->removeAnnotation(Inst, "Offset");

View File

@ -295,6 +295,13 @@ bool DataReader::expectAndConsumeFS() {
return true;
}
void DataReader::consumeAllRemainingFS() {
while (ParsingBuf[0] == FieldSeparator) {
ParsingBuf = ParsingBuf.drop_front(1);
Col += 1;
}
}
bool DataReader::checkAndConsumeNewLine() {
if (ParsingBuf[0] != '\n')
return false;
@ -389,12 +396,14 @@ ErrorOr<Location> DataReader::parseLocation(char EndChar,
if (!expectAndConsumeFS())
return make_error_code(llvm::errc::io_error);
consumeAllRemainingFS();
// Read the string containing the symbol or the DSO name
auto NameRes = parseString(FieldSeparator);
if (std::error_code EC = NameRes.getError())
return EC;
StringRef Name = NameRes.get();
consumeAllRemainingFS();
// Read the offset
auto Offset = parseHexField(EndChar, EndNl);
@ -410,21 +419,25 @@ ErrorOr<BranchInfo> DataReader::parseBranchInfo() {
return EC;
Location From = Res.get();
consumeAllRemainingFS();
Res = parseLocation(FieldSeparator);
if (std::error_code EC = Res.getError())
return EC;
Location To = Res.get();
consumeAllRemainingFS();
auto MRes = parseNumberField(FieldSeparator);
if (std::error_code EC = MRes.getError())
return EC;
int64_t NumMispreds = MRes.get();
consumeAllRemainingFS();
auto BRes = parseNumberField(FieldSeparator, /* EndNl = */ true);
if (std::error_code EC = BRes.getError())
return EC;
int64_t NumBranches = BRes.get();
consumeAllRemainingFS();
if (!checkAndConsumeNewLine()) {
reportError("expected end of line");
return make_error_code(llvm::errc::io_error);
@ -439,15 +452,18 @@ ErrorOr<MemInfo> DataReader::parseMemInfo() {
return EC;
Location Offset = Res.get();
consumeAllRemainingFS();
Res = parseMemLocation(FieldSeparator);
if (std::error_code EC = Res.getError())
return EC;
Location Addr = Res.get();
consumeAllRemainingFS();
auto CountRes = parseNumberField(FieldSeparator, true);
if (std::error_code EC = CountRes.getError())
return EC;
consumeAllRemainingFS();
if (!checkAndConsumeNewLine()) {
reportError("expected end of line");
return make_error_code(llvm::errc::io_error);
@ -462,11 +478,13 @@ ErrorOr<SampleInfo> DataReader::parseSampleInfo() {
return EC;
Location Address = Res.get();
consumeAllRemainingFS();
auto BRes = parseNumberField(FieldSeparator, /* EndNl = */ true);
if (std::error_code EC = BRes.getError())
return EC;
int64_t Occurrences = BRes.get();
consumeAllRemainingFS();
if (!checkAndConsumeNewLine()) {
reportError("expected end of line");
return make_error_code(llvm::errc::io_error);

View File

@ -425,6 +425,7 @@ protected:
void reportError(StringRef ErrorMsg);
bool expectAndConsumeFS();
void consumeAllRemainingFS();
bool checkAndConsumeNewLine();
ErrorOr<StringRef> parseString(char EndChar, bool EndNl=false);
ErrorOr<int64_t> parseNumberField(char EndChar, bool EndNl=false);

View File

@ -334,6 +334,11 @@ public:
return false;
}
/// Check whether we support inverting this branch
virtual bool isUnsupportedBranch(unsigned Opcode) const {
return false;
}
/// Return true of the instruction is of pseudo kind.
bool isPseudo(const MCInst &Inst) const {
return Info->get(Inst.getOpcode()).isPseudo();
@ -358,6 +363,14 @@ public:
llvm_unreachable("not implemented");
}
virtual void createPushFlags(MCInst &Inst, unsigned Size) const {
llvm_unreachable("not implemented");
}
virtual void createPopFlags(MCInst &Inst, unsigned Size) const {
llvm_unreachable("not implemented");
}
virtual bool createDirectCall(MCInst &Inst, const MCSymbol *Target,
MCContext *Ctx) {
llvm_unreachable("not implemented");
@ -1340,6 +1353,13 @@ public:
return false;
}
/// Create instruction to increment contents of target by 1
virtual bool createIncMemory(MCInst &Inst, const MCSymbol *Target,
MCContext *Ctx) const {
llvm_unreachable("not implemented");
return false;
}
/// Create a fragment of code (sequence of instructions) that load a 32-bit
/// address from memory, zero-extends it to 64 and jump to it (indirect jump).
virtual bool
@ -1631,8 +1651,13 @@ public:
/// empty vector of instructions. The label is meant to indicate the basic
/// block where all previous snippets are joined, i.e. the instructions that
/// would immediate follow the original call.
using ICPdata = std::vector<std::pair<MCSymbol*, std::vector<MCInst>>>;
virtual ICPdata indirectCallPromotion(
using BlocksVectorTy = std::vector<std::pair<MCSymbol*, std::vector<MCInst>>>;
struct MultiBlocksCode {
BlocksVectorTy Blocks;
std::vector<MCSymbol*> Successors;
};
virtual BlocksVectorTy indirectCallPromotion(
const MCInst &CallInst,
const std::vector<std::pair<MCSymbol *, uint64_t>> &Targets,
const std::vector<std::pair<MCSymbol *, uint64_t>> &VtableSyms,
@ -1641,19 +1666,40 @@ public:
MCContext *Ctx
) {
llvm_unreachable("not implemented");
return ICPdata();
return BlocksVectorTy();
}
virtual ICPdata jumpTablePromotion(
virtual BlocksVectorTy jumpTablePromotion(
const MCInst &IJmpInst,
const std::vector<std::pair<MCSymbol *,uint64_t>>& Targets,
const std::vector<MCInst *> &TargetFetchInsns,
MCContext *Ctx
) const {
llvm_unreachable("not implemented");
return ICPdata();
return BlocksVectorTy();
}
/// Part of the runtime library for instrumented code, this runs at the end
/// of the process and writes the current instrumentation counters to a file
/// compatible with BOLT profile. \p Locs identifies the region in memory
/// where the counters are (\p NumLocs counters), \p Descriptions, the region
/// encoding information about each counter, which is the source of the branch
/// and the destination, \p Strings, the string table with function names used
/// in descriptions, \p FilenameSym, the profile file name to write to,
/// \p Chars, a 0 to F string used for printing hex/decimal numbers.
virtual MultiBlocksCode createInstrumentedDataDumpCode(
MCSymbol *Locs,
MCSymbol *Descriptions,
MCSymbol *Strings,
MCSymbol *FilenameSym,
MCSymbol *Spaces,
MCSymbol *Chars,
size_t NumLocs,
MCContext *Ctx
) const {
llvm_unreachable("not implemented");
return MultiBlocksCode();
}
};
} // namespace bolt

View File

@ -1292,7 +1292,7 @@ void PrintProfileStats::runOnFunctions(BinaryContext &BC) {
double Mean = 0.0;
for (const auto &BB : Function) {
// Do not compute score for low frequency blocks, entry or exit blocks
if (IncomingMap[&BB] < 100 || OutgoingMap[&BB] == 0)
if (IncomingMap[&BB] < 100 || OutgoingMap[&BB] == 0 || BB.isEntryPoint())
continue;
++NumBlocks;
const double Difference = (double)OutgoingMap[&BB] - IncomingMap[&BB];

View File

@ -15,6 +15,7 @@ add_llvm_library(LLVMBOLTPasses
IdenticalCodeFolding.cpp
IndirectCallPromotion.cpp
Inliner.cpp
Instrumentation.cpp
JTFootprintReduction.cpp
LivenessAnalysis.cpp
LongJmp.cpp

View File

@ -812,7 +812,7 @@ IndirectCallPromotion::rewriteCall(
BinaryFunction &Function,
BinaryBasicBlock *IndCallBlock,
const MCInst &CallInst,
MCPlusBuilder::ICPdata &&ICPcode,
MCPlusBuilder::BlocksVectorTy &&ICPcode,
const std::vector<MCInst *> &MethodFetchInsns
) const {
// Create new basic blocks with correct code in each one first.

View File

@ -222,7 +222,7 @@ class IndirectCallPromotion : public BinaryFunctionPass {
BinaryFunction &Function,
BinaryBasicBlock *IndCallBlock,
const MCInst &CallInst,
MCPlusBuilder::ICPdata &&ICPcode,
MCPlusBuilder::BlocksVectorTy &&ICPcode,
const std::vector<MCInst *> &MethodFetchInsns) const;
BinaryBasicBlock *fixCFG(BinaryContext &BC,

View File

@ -0,0 +1,320 @@
//===--- Passes/Instrumentation.cpp ---------------------------------------===//
//
// The LLVM Compiler Infrastructure
//
// This file is distributed under the University of Illinois Open Source
// License. See LICENSE.TXT for details.
//
//===----------------------------------------------------------------------===//
//
//===----------------------------------------------------------------------===//
#include "Instrumentation.h"
#include "Passes/DataflowInfoManager.h"
#include "llvm/Support/Options.h"
#define DEBUG_TYPE "bolt-instrumentation"
using namespace llvm;
namespace opts {
extern cl::OptionCategory BoltCategory;
extern bool shouldProcess(const llvm::bolt::BinaryFunction &Function);
cl::opt<std::string> InstrumentationFilename(
"instrumentation-file",
cl::desc("file name where instrumented profile will be saved"),
cl::init("/tmp/prof.fdata"),
cl::Optional,
cl::cat(BoltCategory));
}
namespace llvm {
namespace bolt {
uint32_t Instrumentation::getFunctionNameIndex(const BinaryFunction &Function) {
auto Iter = FuncToStringIdx.find(&Function);
if (Iter != FuncToStringIdx.end())
return Iter->second;
auto Idx = StringTable.size();
FuncToStringIdx.emplace(std::make_pair(&Function, Idx));
StringTable.append(Function.getNames()[0]);
StringTable.append(1, '\0');
return Idx;
}
Instrumentation::CounterDescription Instrumentation::createDescription(
const BinaryFunction &FromFunction, uint32_t From,
const BinaryFunction &ToFunction, uint32_t To) {
CounterDescription Res;
Res.FromFuncStringIdx = getFunctionNameIndex(FromFunction);
Res.FromOffset = From;
Res.ToFuncStringIdx = getFunctionNameIndex(ToFunction);
Res.ToOffset = To;
return Res;
}
std::vector<MCInst> Instrumentation::createInstrumentationSnippet(
BinaryFunction &FromFunction, uint32_t FromOffset, BinaryFunction &ToFunc,
uint32_t ToOffset) {
Descriptions.emplace_back(
createDescription(FromFunction, FromOffset, ToFunc, ToOffset));
BinaryContext &BC = FromFunction.getBinaryContext();
MCSymbol *Label =
BC.Ctx->createTempSymbol("InstrEntry", true);
Labels.emplace_back(Label);
std::vector<MCInst> CounterInstrs(5);
// Don't clobber application red zone (ABI dependent)
BC.MIB->createStackPointerIncrement(CounterInstrs[0], 128,
/*NoFlagsClobber=*/true);
BC.MIB->createPushFlags(CounterInstrs[1], 2);
BC.MIB->createIncMemory(CounterInstrs[2], Label, &*BC.Ctx);
BC.MIB->createPopFlags(CounterInstrs[3], 2);
BC.MIB->createStackPointerDecrement(CounterInstrs[4], 128,
/*NoFlagsClobber=*/true);
return CounterInstrs;
}
bool Instrumentation::instrumentOneTarget(BinaryBasicBlock::iterator &Iter,
BinaryFunction &FromFunction,
BinaryBasicBlock &FromBB,
uint32_t From, BinaryFunction &ToFunc,
BinaryBasicBlock *TargetBB,
uint32_t ToOffset) {
std::vector<MCInst> CounterInstrs =
createInstrumentationSnippet(FromFunction, From, ToFunc, ToOffset);
BinaryContext &BC = FromFunction.getBinaryContext();
const MCInst &Inst = *Iter;
if (BC.MIB->isCall(Inst) && !TargetBB) {
for (auto &NewInst : CounterInstrs) {
Iter = FromBB.insertInstruction(Iter, NewInst);
++Iter;
}
return true;
}
if (!TargetBB)
return false;
// Indirect branch, conditional branches or fall-throughs
// Regular cond branch, put counter at start of target block
if (TargetBB->pred_size() == 1 && &FromBB != TargetBB &&
!TargetBB->isEntryPoint()) {
auto RemoteIter = TargetBB->begin();
for (auto &NewInst : CounterInstrs) {
RemoteIter = TargetBB->insertInstruction(RemoteIter, NewInst);
++RemoteIter;
}
return true;
}
if (FromBB.succ_size() == 1 && &FromBB != TargetBB) {
for (auto &NewInst : CounterInstrs) {
Iter = FromBB.insertInstruction(Iter, NewInst);
++Iter;
}
return true;
}
// Critical edge, create BB and put counter there
SplitWorklist.emplace_back(std::make_pair(&FromBB, TargetBB));
SplitInstrs.emplace_back(std::move(CounterInstrs));
return true;
}
void Instrumentation::runOnFunctions(BinaryContext &BC) {
if (!BC.isX86())
return;
const auto Flags = BinarySection::getFlags(/*IsReadOnly=*/false,
/*IsText=*/false,
/*IsAllocatable=*/true);
BC.registerOrUpdateSection(".bolt.instrumentation", ELF::SHT_PROGBITS, Flags,
nullptr, 0, 1,
/*local=*/true);
uint64_t InstrumentationSites{0ULL};
uint64_t InstrumentationSitesSavingFlags{0ULL};
for (auto &BFI : BC.getBinaryFunctions()) {
BinaryFunction &Function = BFI.second;
if (!Function.isSimple() || !opts::shouldProcess(Function))
continue;
SplitWorklist.clear();
SplitInstrs.clear();
for (auto BBI = Function.begin(); BBI != Function.end(); ++BBI) {
auto &BB{*BBI};
bool HasUnconditionalBranch{false};
bool HasJumpTable{false};
for (auto I = BB.begin(); I != BB.end(); ++I) {
const auto &Inst = *I;
if (!BC.MIB->hasAnnotation(Inst, "Offset"))
continue;
const bool IsJumpTable = Function.getJumpTable(Inst);
if (IsJumpTable)
HasJumpTable = true;
else if (BC.MIB->isUnconditionalBranch(Inst))
HasUnconditionalBranch = true;
else if ((!BC.MIB->isCall(Inst) &&
!BC.MIB->isConditionalBranch(Inst)) ||
BC.MIB->isUnsupportedBranch(Inst.getOpcode()))
continue;
uint32_t FromOffset = BC.MIB->getAnnotationAs<uint32_t>(Inst, "Offset");
const MCSymbol *Target = BC.MIB->getTargetSymbol(Inst);
BinaryBasicBlock *TargetBB = Function.getBasicBlockForLabel(Target);
uint32_t ToOffset = TargetBB ? TargetBB->getInputOffset() : 0;
BinaryFunction *TargetFunc =
TargetBB ? &Function : BC.getFunctionForSymbol(Target);
// Should be null for indirect branches/calls
if (TargetFunc) {
if (instrumentOneTarget(I, Function, BB, FromOffset, *TargetFunc,
TargetBB, ToOffset))
++InstrumentationSites;
continue;
}
if (IsJumpTable) {
for (auto &Succ : BB.successors()) {
if (instrumentOneTarget(I, Function, BB, FromOffset, Function,
&*Succ, Succ->getInputOffset()))
++InstrumentationSites;
}
continue;
}
// FIXME: handle indirect calls
} // End of instructions loop
// Instrument fallthroughs (when the direct jump instruction is missing)
if (!HasUnconditionalBranch && !HasJumpTable && BB.succ_size() > 0 &&
BB.size() > 0) {
auto *FTBB = BB.getFallthrough();
auto I = BB.begin();
auto LastInstr = BB.end();
--LastInstr;
while (LastInstr != I && BC.MIB->isPseudo(*LastInstr))
--LastInstr;
uint32_t FromOffset = 0;
// The last instruction in the BB should have an annotation, except
// if it was branching to the end of the function as a result of
// __builtin_unreachable(), in which case it was deleted by fixBranches.
// Ignore this case. FIXME: force fixBranches() to preserve the offset.
if (!BC.MIB->hasAnnotation(*LastInstr, "Offset"))
continue;
FromOffset = BC.MIB->getAnnotationAs<uint32_t>(*LastInstr, "Offset");
if (instrumentOneTarget(I, Function, BB, FromOffset, Function, FTBB,
FTBB->getInputOffset()))
++InstrumentationSites;
}
} // End of BBs loop
// Consume list of critical edges: split them and add instrumentation to the
// newly created BBs
auto Iter = SplitInstrs.begin();
for (auto &BBPair : SplitWorklist) {
auto *NewBB = Function.splitEdge(BBPair.first, BBPair.second);
NewBB->addInstructions(Iter->begin(), Iter->end());
++Iter;
}
}
outs() << "BOLT-INSTRUMENTER: Instrumented " << InstrumentationSites
<< " sites, " << InstrumentationSitesSavingFlags << " saving flags.\n";
createDumpFunction(BC);
DEBUG(DumpFunction->dump());
}
void Instrumentation::createDumpFunction(BinaryContext &BC) {
DumpFunction =
BC.createInjectedBinaryFunction("BOLTInstrumentationDataDump");
Locs = BC.Ctx->createTempSymbol("BOLTInstrumentationLocs", true);
DescriptionsSym =
BC.Ctx->createTempSymbol("BOLTInstrumentationDescriptions", true);
StringsSym = BC.Ctx->createTempSymbol("BOLTInstrumentationStrings", true);
FilenameSym = BC.Ctx->createTempSymbol("BOLTInstrumentationFilename", true);
Spaces = BC.Ctx->createTempSymbol("BOLTInstrumentationSpaces", true);
Chars = BC.Ctx->createTempSymbol("BOLTInstrumentationChars", true);
auto Code = BC.MIB->createInstrumentedDataDumpCode(
Locs, DescriptionsSym, StringsSym, FilenameSym, Spaces, Chars,
Labels.size(), &*BC.Ctx);
std::vector<std::unique_ptr<BinaryBasicBlock>> BBs;
for (auto &SymBlock : Code.Blocks) {
auto &Symbol = SymBlock.first;
auto &Block = SymBlock.second;
BBs.emplace_back(DumpFunction->createBasicBlock(
BinaryBasicBlock::INVALID_OFFSET, Symbol));
BBs.back()->addInstructions(Block.begin(), Block.end());
BBs.back()->setCFIState(0);
}
auto BBIter = BBs.begin();
for (auto &Succ : Code.Successors) {
if (Succ)
(*BBIter)->addSuccessor(DumpFunction->getBasicBlockForLabel(Succ), 0,
0);
auto NextBBIter = std::next(BBIter);
if (NextBBIter != BBs.end())
(*BBIter)->addSuccessor(NextBBIter->get(), 0, 0);
++BBIter;
}
DumpFunction->insertBasicBlocks(nullptr, std::move(BBs),
/*UpdateLayout=*/true,
/*UpdateCFIState=*/false);
DumpFunction->updateState(BinaryFunction::State::CFG_Finalized);
}
void Instrumentation::emitDescription(
const Instrumentation::CounterDescription &Desc, MCStreamer &Streamer) {
Streamer.EmitIntValue(Desc.FromFuncStringIdx, /*Size=*/4);
Streamer.EmitIntValue(Desc.FromOffset, /*Size=*/4);
Streamer.EmitIntValue(Desc.ToFuncStringIdx, /*Size=*/4);
Streamer.EmitIntValue(Desc.ToOffset, /*Size=*/4);
}
void Instrumentation::emit(BinaryContext &BC, MCStreamer &Streamer) {
const auto Flags = BinarySection::getFlags(/*IsReadOnly=*/false,
/*IsText=*/false,
/*IsAllocatable=*/true);
auto *Section = BC.Ctx->getELFSection(".bolt.instrumentation",
ELF::SHT_PROGBITS,
Flags);
Streamer.SwitchSection(Section);
Streamer.EmitLabel(Locs);
for (const auto &Label : Labels) {
Streamer.EmitLabel(Label);
Streamer.emitFill(8, 0);
}
Streamer.EmitLabel(DescriptionsSym);
for (const auto &Desc : Descriptions) {
emitDescription(Desc, Streamer);
}
Streamer.EmitLabel(StringsSym);
Streamer.EmitBytes(StringTable);
Streamer.EmitLabel(Spaces);
Streamer.EmitBytes(" ");
Streamer.EmitLabel(Chars);
Streamer.EmitBytes("0123456789abcdef");
Streamer.EmitLabel(FilenameSym);
Streamer.EmitBytes(opts::InstrumentationFilename);
Streamer.emitFill(1, 0);
outs() << "BOLT-INSTRUMENTER: Total size of string table emitted: "
<< StringTable.size() << " bytes\n";
outs() << "BOLT-INSTRUMENTER: Total size of counters: "
<< (Labels.size() * 8) << " bytes\n";
outs() << "BOLT-INSTRUMENTER: Total size of descriptors: "
<< (Labels.size() * 16) << " bytes\n";
outs() << "BOLT-INSTRUMENTER: Total data: "
<< (Labels.size() * 24 + StringTable.size()) << " bytes\n";
outs() << "BOLT-INSTRUMENTER: Profile will be saved to file "
<< opts::InstrumentationFilename << "\n";
}
}
}

View File

@ -0,0 +1,160 @@
//===--- Passes/Instrumentation.h -----------------------------------------===//
//
// The LLVM Compiler Infrastructure
//
// This file is distributed under the University of Illinois Open Source
// License. See LICENSE.TXT for details.
//
//===----------------------------------------------------------------------===//
//
//===----------------------------------------------------------------------===//
#ifndef LLVM_TOOLS_LLVM_BOLT_PASSES_INSTRUMENTATION_H
#define LLVM_TOOLS_LLVM_BOLT_PASSES_INSTRUMENTATION_H
#include "BinaryPasses.h"
#include "llvm/MC/MCSection.h"
#include "llvm/MC/MCStreamer.h"
#include "llvm/MC/MCSymbol.h"
namespace llvm {
namespace bolt {
/// This is an instrumentation pass that modifies the input binary to generate
/// a profile after execution finishes. It modifies branches to increment
/// counters stored in the process memory and inserts a new function that
/// dumps this data to an fdata file.
///
/// The runtime for instrumentation has a string table that holds function
/// names. It also must include two data structures: the counter values being
/// incremented after each instrumented branch and a description of these
/// counters to be written in a file during dump. The description references
/// string indices in the string table for function names, as well as function
/// offsets locating branch source and destination. The counter values will be
/// converted to decimal form when writing the dumped fdata.
///
/// OPPORTUNITIES ON PERFORMANCE
/// This instrumentation is experimental and currently uses a naive approach
/// where every branch is instrumented. This is not ideal for runtime
/// performance, but should be good enough for us to evaluate/debug LBR profile
/// quality against instrumentation. Hopefully we can make this more efficient
/// in the future, but most optimizations here can cost a lot in BOLT processing
/// time. Keep in mind the instrumentation pass runs on every single BB of the
/// entire input binary, thus it is very expensive to do analyses, such as FLAGS
/// liveness to avoid spilling flags on every branch, if the binary is large.
///
/// MISSING: instrumentation of indirect calls
class Instrumentation {
public:
Instrumentation() {}
/// Modifies all functions by inserting instrumentation code (first step)
void runOnFunctions(BinaryContext &BC);
/// Emit data structures that will be necessary during runtime (second step)
void emit(BinaryContext &BC, MCStreamer &Streamer);
/// Access the function injected by the instrumentation pass necessary to
/// write profile to a file. This is only valid after instrumentation
/// finished (step 1).
BinaryFunction *getDumpFunction() const {
return DumpFunction;
}
private:
// Instrumented branch location information
struct CounterDescription {
uint32_t FromFuncStringIdx;
uint32_t FromOffset;
uint32_t ToFuncStringIdx;
uint32_t ToOffset;
};
/// Create a new injected function that will be needed at runtime to write
/// profile
void createDumpFunction(BinaryContext &BC);
/// Retrieve the string table index for the name of \p Function. We encode
/// instrumented locations descriptions with the aid of a string table to
/// manage memory of the instrumentation runtime in a more efficient way.
/// If this function name is not represented in the string table yet, it will
/// be inserted and its index returned.
uint32_t getFunctionNameIndex(const BinaryFunction &Function);
/// Populate all information needed to identify an instrumented location:
/// branch source location in terms of function name plus offset, as well as
/// branch destination (also name + offset). This will be encoded in the
/// binary as static data and function name strings will reference a strtab.
CounterDescription createDescription(const BinaryFunction &FromFunction,
uint32_t From,
const BinaryFunction &ToFunction,
uint32_t To);
/// Create the sequence of instructions to instrument a branch happening
/// at \p FromFunction + \p FromOffset to \p ToFunc + \p ToOffset
std::vector<MCInst> createInstrumentationSnippet(BinaryFunction &FromFunction,
uint32_t FromOffset,
BinaryFunction &ToFunc,
uint32_t ToOffset);
/// Instrument the branch in \p Iter located at \p FromFunction + \p From,
/// basic block \p FromBB. The destination of the branch is \p ToFunc +
/// \p ToOffset. \p TargetBB should be non-null if this is a local branch
/// and null if it is a call. Return true on success.
bool instrumentOneTarget(BinaryBasicBlock::iterator &Iter,
BinaryFunction &FromFunction,
BinaryBasicBlock &FromBB, uint32_t From,
BinaryFunction &ToFunc, BinaryBasicBlock *TargetBB,
uint32_t ToOffset);
void emitDescription(const CounterDescription &Desc, MCStreamer &Streamer);
/// Critical edges worklist
/// This worklist keeps track of CFG edges <From-To> that needs to be split.
/// This task is deferred until we finish processing all BBs because we can't
/// modify the CFG while iterating over it. For each edge, \p SplitInstrs
/// stores the list of instrumentation instructions as a vector of MCInsts.
/// instrumentOneTarget() populates this, runOnFunctions() consumes.
std::vector<std::pair<BinaryBasicBlock *, BinaryBasicBlock *>> SplitWorklist;
std::vector<std::vector<MCInst>> SplitInstrs;
/// Stores function names, to be emitted to the runtime
std::string StringTable;
/// strtab indices in StringTable for each function name
std::unordered_map<const BinaryFunction *, uint32_t> FuncToStringIdx;
std::vector<CounterDescription> Descriptions;
/// Identify all counters used in runtime while instrumentation is running
std::vector<MCSymbol *> Labels;
/// Label marking start of the memory region containing instrumentation
/// counters, total vector size is Labels.size() 8-byte counters
MCSymbol *Locs;
/// Start of the vector with descriptions (one CounterDescription for each
/// counter), vector size is Labels.size() CounterDescription-sized elmts
MCSymbol *DescriptionsSym;
/// Label identifying where our string table was emitted to
MCSymbol *StringsSym;
/// File name where profile is going to written to after target binary
/// finishes a run
MCSymbol *FilenameSym;
/// Label for a string containing 8 spaces used by the algorithm that writes
/// profile during conversion of integer to string. \p Chars stores ASCII
/// representation of numbers from 0 to F.
MCSymbol *Spaces;
MCSymbol *Chars;
/// We keep a pointer to our injected function whose final address will be
/// needed later to patch the destructor routines in the binary to call us
/// upon end of execution
BinaryFunction *DumpFunction;
};
}
}
#endif

View File

@ -93,6 +93,12 @@ extern cl::opt<JumpTableSupportLevel> JumpTables;
extern cl::list<std::string> ReorderData;
extern cl::opt<bolt::ReorderFunctions::ReorderType> ReorderFunctions;
cl::opt<bool>
Instrument("instrument-experimental",
cl::desc("instrument code to generate accurate profile data (experimental)"),
cl::ZeroOrMore,
cl::cat(BoltOptCategory));
static cl::opt<bool>
ForceToDataRelocations("force-data-relocations",
cl::desc("force relocations to data sections to always be processed"),
@ -1896,6 +1902,11 @@ void RewriteInstance::adjustCommandLineOptions() {
"supported\n";
}
if (opts::Instrument && !BC->HasRelocations) {
errs() << "BOLT-ERROR: instrumentation requires relocations\n";
exit(1);
}
if (opts::AlignMacroOpFusion != MFT_NONE && !BC->isX86()) {
outs() << "BOLT-INFO: disabling -align-macro-fusion on non-x86 platform\n";
opts::AlignMacroOpFusion = MFT_NONE;
@ -2731,6 +2742,10 @@ void RewriteInstance::postProcessFunctions() {
void RewriteInstance::runOptimizationPasses() {
NamedRegionTimer T("runOptimizationPasses", "run optimization passes",
TimerGroupName, TimerGroupDesc, opts::TimeRewrite);
if (opts::Instrument) {
Instrumenter = llvm::make_unique<Instrumentation>();
Instrumenter->runOnFunctions(*BC);
}
BinaryFunctionPassManager::runAllPasses(*BC);
}
@ -2889,6 +2904,8 @@ void RewriteInstance::emitSections() {
BC->getTextSection()->setAlignment(BC->PageAlign);
emitFunctions(Streamer.get());
if (opts::Instrument)
Instrumenter->emit(*BC, *Streamer.get());
if (!BC->HasRelocations && opts::UpdateDebugSections)
DebugInfoRewriter->updateDebugLineInfoForNonSimpleFunctions();
@ -3269,9 +3286,9 @@ void RewriteInstance::mapDataSections(orc::VModuleKey Key) {
// Map special sections to their addresses in the output image.
// These are the sections that we generate via MCStreamer.
// The order is important.
std::vector<std::string> Sections = { ".eh_frame", ".eh_frame_old",
".gcc_except_table",
".rodata", ".rodata.cold" };
std::vector<std::string> Sections = {
".eh_frame", ".eh_frame_old", ".gcc_except_table",
".rodata", ".rodata.cold", ".bolt.instrumentation"};
for (auto &SectionName : Sections) {
auto Section = BC->getUniqueSectionByName(SectionName);
if (!Section || !Section->isAllocatable() || !Section->isFinalized())
@ -3613,6 +3630,10 @@ void RewriteInstance::patchELFPHDRTable() {
NewPhdr.p_filesz = NewTextSegmentSize;
NewPhdr.p_memsz = NewTextSegmentSize;
NewPhdr.p_flags = ELF::PF_X | ELF::PF_R;
// FIXME: Currently instrumentation is experimental and the runtime data
// is emitted with code, thus everything needs to be writable
if (opts::Instrument)
NewPhdr.p_flags |= ELF::PF_W;
NewPhdr.p_align = BC->PageAlign;
ModdedGnuStack = true;
} else if (!opts::UseGnuStack && Phdr.p_type == ELF::PT_DYNAMIC) {
@ -3624,7 +3645,10 @@ void RewriteInstance::patchELFPHDRTable() {
NewTextPhdr.p_paddr = PHDRTableAddress;
NewTextPhdr.p_filesz = NewTextSegmentSize;
NewTextPhdr.p_memsz = NewTextSegmentSize;
// FIXME: experimental instrumentation hack described above
NewTextPhdr.p_flags = ELF::PF_X | ELF::PF_R;
if (opts::Instrument)
NewTextPhdr.p_flags |= ELF::PF_W;
NewTextPhdr.p_align = BC->PageAlign;
OS.write(reinterpret_cast<const char *>(&NewTextPhdr),
sizeof(NewTextPhdr));
@ -4581,6 +4605,12 @@ void RewriteInstance::patchELFDynamic(ELFObjectFile<ELFT> *File) {
NewDE.d_un.d_ptr = NewAddress;
}
}
// FIXME: Put the old FINI pointer as a tail call in the generated
// dumper function
if (opts::Instrument && DE->getTag() == ELF::DT_FINI) {
NewDE.d_un.d_ptr =
Instrumenter->getDumpFunction()->getOutputAddress();
}
break;
case ELF::DT_FLAGS:
if (BC->RequiresZNow) {

View File

@ -16,6 +16,7 @@
#include "BinaryFunction.h"
#include "ExecutableFileMemoryManager.h"
#include "Passes/Instrumentation.h"
#include "llvm/ADT/ArrayRef.h"
#include "llvm/ExecutionEngine/Orc/RTDyldObjectLinkingLayer.h"
#include "llvm/MC/StringTableBuilder.h"
@ -367,6 +368,8 @@ private:
std::unique_ptr<BoltAddressTranslation> BAT;
std::unique_ptr<Instrumentation> Instrumenter;
/// Patchers used to apply simple changes to sections of the input binary.
/// Maps section name -> patcher.
SectionPatchersType SectionPatchers;

View File

@ -635,6 +635,19 @@ public:
return false;
}
bool isUnsupportedBranch(unsigned Opcode) const override {
switch (Opcode) {
default:
return false;
case X86::LOOP:
case X86::LOOPE:
case X86::LOOPNE:
case X86::JECXZ:
case X86::JRCXZ:
return true;
}
}
bool isLoad(const MCInst &Inst) const override {
if (isPop(Inst))
return true;
@ -2697,6 +2710,22 @@ public:
return true;
}
bool createIncMemory(MCInst &Inst, const MCSymbol *Target,
MCContext *Ctx) const override {
Inst.setOpcode(X86::INC64m);
Inst.clear();
Inst.addOperand(MCOperand::createReg(X86::RIP)); // BaseReg
Inst.addOperand(MCOperand::createImm(1)); // ScaleAmt
Inst.addOperand(MCOperand::createReg(X86::NoRegister)); // IndexReg
Inst.addOperand(MCOperand::createExpr(
MCSymbolRefExpr::create(Target, MCSymbolRefExpr::VK_None,
*Ctx))); // Displacement
Inst.addOperand(MCOperand::createReg(X86::NoRegister)); // AddrSegmentReg
return true;
}
bool createIJmp32Frag(SmallVectorImpl<MCInst> &Insts,
const MCOperand &BaseReg, const MCOperand &Scale,
const MCOperand &IndexReg, const MCOperand &Offset,
@ -3006,6 +3035,17 @@ public:
unsigned Size) const override {
Inst.clear();
unsigned NewOpcode = 0;
if (Reg == X86::EFLAGS) {
switch (Size) {
case 2: NewOpcode = X86::PUSHF16; break;
case 4: NewOpcode = X86::PUSHF32; break;
case 8: NewOpcode = X86::PUSHF64; break;
default:
assert(false);
}
Inst.setOpcode(NewOpcode);
return;
}
switch (Size) {
case 2: NewOpcode = X86::PUSH16r; break;
case 4: NewOpcode = X86::PUSH32r; break;
@ -3021,6 +3061,17 @@ public:
unsigned Size) const override {
Inst.clear();
unsigned NewOpcode = 0;
if (Reg == X86::EFLAGS) {
switch (Size) {
case 2: NewOpcode = X86::POPF16; break;
case 4: NewOpcode = X86::POPF32; break;
case 8: NewOpcode = X86::POPF64; break;
default:
assert(false);
}
Inst.setOpcode(NewOpcode);
return;
}
switch (Size) {
case 2: NewOpcode = X86::POP16r; break;
case 4: NewOpcode = X86::POP32r; break;
@ -3032,7 +3083,15 @@ public:
Inst.addOperand(MCOperand::createReg(Reg));
}
ICPdata indirectCallPromotion(
void createPushFlags(MCInst &Inst, unsigned Size) const override {
return createPushRegister(Inst, X86::EFLAGS, Size);
}
void createPopFlags(MCInst &Inst, unsigned Size) const override {
return createPopRegister(Inst, X86::EFLAGS, Size);
}
BlocksVectorTy indirectCallPromotion(
const MCInst &CallInst,
const std::vector<std::pair<MCSymbol *, uint64_t>> &Targets,
const std::vector<std::pair<MCSymbol *, uint64_t>> &VtableSyms,
@ -3042,7 +3101,7 @@ public:
) override {
const bool IsTailCall = isTailCall(CallInst);
const bool IsJumpTable = getJumpTable(CallInst) != 0;
ICPdata Results;
BlocksVectorTy Results;
// Label for the current code block.
MCSymbol* NextTarget = nullptr;
@ -3106,7 +3165,7 @@ public:
const auto Addr = Targets[i].second;
// Immediate address is out of sign extended 32 bit range.
if (int64_t(Addr) != int64_t(int32_t(Addr))) {
return ICPdata();
return BlocksVectorTy();
}
Target.addOperand(MCOperand::createImm(Addr));
}
@ -3172,7 +3231,7 @@ public:
const auto Addr = Targets[i].second;
// Immediate address is out of sign extended 32 bit range.
if (int64_t(Addr) != int64_t(int32_t(Addr))) {
return ICPdata();
return BlocksVectorTy();
}
Compare.addOperand(MCOperand::createImm(Addr));
}
@ -3274,7 +3333,7 @@ public:
return Results;
}
ICPdata jumpTablePromotion(
BlocksVectorTy jumpTablePromotion(
const MCInst &IJmpInst,
const std::vector<std::pair<MCSymbol *,uint64_t>> &Targets,
const std::vector<MCInst *> &TargetFetchInsns,
@ -3283,9 +3342,9 @@ public:
assert(getJumpTable(IJmpInst) != 0);
uint16_t IndexReg = getAnnotationAs<uint16_t>(IJmpInst, "JTIndexReg");
if (IndexReg == 0)
return ICPdata();
return BlocksVectorTy();
ICPdata Results;
BlocksVectorTy Results;
// Label for the current code block.
MCSymbol* NextTarget = nullptr;
@ -3304,7 +3363,7 @@ public:
const auto CaseIdx = Targets[i].second;
// Immediate address is out of sign extended 32 bit range.
if (int64_t(CaseIdx) != int64_t(int32_t(CaseIdx))) {
return ICPdata();
return BlocksVectorTy();
}
CompareInst.addOperand(MCOperand::createImm(CaseIdx));
shortenInstruction(CompareInst);
@ -3334,6 +3393,311 @@ public:
return Results;
}
MultiBlocksCode createInstrumentedDataDumpCode(
MCSymbol *Locs,
MCSymbol *Descriptions,
MCSymbol *Strings,
MCSymbol *FilenameSym,
MCSymbol *Spaces,
MCSymbol *Chars,
size_t NumLocs,
MCContext *Ctx
) const override {
std::vector<MCInst>* Code;
MultiBlocksCode Result;
#define INS(x, y) Code->emplace_back(MCInstBuilder(x).y)
#define INS_NOARGS(x) Code->emplace_back(MCInstBuilder(x))
#define REG(x) addReg(x)
#define IMM(x) addImm(x)
#define EXPR(x) \
addExpr(MCSymbolRefExpr::create(x, MCSymbolRefExpr::VK_None, *Ctx))
#define NOREG addReg(X86::NoRegister)
#define BEGIN_BLOCK(x) \
Result.Blocks.emplace_back(std::make_pair<>(x, std::vector<MCInst>())); \
Code = &Result.Blocks.back().second; \
Result.Successors.emplace_back(nullptr);
#define BEGIN_BLOCK_FALLTHROUGH \
Result.Blocks.emplace_back( \
std::make_pair<>(Ctx->createTempSymbol(), std::vector<MCInst>())); \
Code = &Result.Blocks.back().second; \
Result.Successors.emplace_back(nullptr);
#define SET_SUCC(x) Result.Successors.back() = x;
using namespace llvm::X86;
// I know, this got ridiculously large, we should have a better way to
// write our runtime library for instrumentation. At this point I think it
// is kind of nice to do not depend on loading an object on disk to link
// against the input binary, but in the future it may be inevitable.
// String buffer allocated on stack to store data sent to write()
const uint32_t COPY_BUFFER_SIZE = 0x1000;
// These constants depend on the target OS
const uint32_t OPEN_MODE = 0666;
const uint32_t OPEN_FLAGS = 0x241; // O_WRONLY|O_TRUNC|O_CREAT
const uint32_t SYSCALL_WRITE = 1;
const uint32_t SYSCALL_OPEN = 2;
const uint32_t SYSCALL_CLOSE = 3;
MCSymbol *LoopBody1 = Ctx->createTempSymbol(); // for each instrumented br
MCSymbol *LoopBody1_1 = Ctx->createTempSymbol(); // for each src/dst
MCSymbol *LoopBody1_1_1 = Ctx->createTempSymbol(); // write func name
MCSymbol *LoopBody1_1_2 = Ctx->createTempSymbol(); // write offset in hex
MCSymbol *LoopBody1_2 = Ctx->createTempSymbol(); // write branch frequency
MCSymbol *Loop1End = Ctx->createTempSymbol();
BEGIN_BLOCK_FALLTHROUGH; // Start our prologue
// pushq %rbx
INS(PUSH64r, REG(RBX));
// pushq %r12
INS(PUSH64r, REG(R12));
// pushq %r13
INS(PUSH64r, REG(R13));
// pushq %r14
INS(PUSH64r, REG(R14));
// pushq %r15
INS(PUSH64r, REG(R15));
// subq $0x1000, %rsp // Reserve space for write buffer
INS(SUB64ri32, REG(RSP).REG(RSP).IMM(COPY_BUFFER_SIZE));
// leaq filename(%rip), %rdi
INS(LEA64r, REG(RDI).REG(RIP).IMM(0x1).NOREG.EXPR(FilenameSym).NOREG);
// movq $0x241, %rsi // O_WRONLY|O_TRUNC|O_CREAT
INS(MOV64ri32, REG(RSI).IMM(OPEN_FLAGS));
// movq $0666, %rdx // mode
INS(MOV64ri32, REG(RDX).IMM(OPEN_MODE));
// mov $0x2, %rax
INS(MOV64ri32, REG(RAX).IMM(SYSCALL_OPEN));
// syscall
INS_NOARGS(SYSCALL); // open()
// movq %rax, %r13
INS(MOV64rr, REG(R13).REG(RAX));
// xorq %r14, %r14 // Induction variable for main loop
// // over all intrumentation counters
INS(XOR64rr, REG(R14).REG(R14).REG(R14));
// leaq label1(%rip), %r11 // Load start of descriptions vector
INS(LEA64r, REG(R11).REG(RIP).IMM(1).NOREG.EXPR(Descriptions).NOREG);
// loopbody1: // Main loop
BEGIN_BLOCK(LoopBody1);
// movq %rsp, %r15 // Reset write buffer pointer
INS(MOV64rr, REG(R15).REG(RSP));
// xorq %rbx, %rbx // Induction variable for our loop of 2
// // iterations to read From description and
// // then To description
INS(XOR64rr, REG(RBX).REG(RBX).REG(RBX));
// loopbody1_1:
BEGIN_BLOCK(LoopBody1_1);
// movb $0x31, (%r15) // Write '1' and a space before func name
INS(MOV8mi, REG(R15).IMM(1).NOREG.IMM(0).NOREG.IMM(0x31));
// incq %r15
INS(INC64r, REG(R15).REG(R15));
// movb $0x20, (%r15)
INS(MOV8mi, REG(R15).IMM(1).NOREG.IMM(0).NOREG.IMM(0x20));
// incq %r15
INS(INC64r, REG(R15).REG(R15));
// leaq strings(%rip), %r12 // Load string table base
INS(LEA64r, REG(R12).REG(RIP).IMM(0x1).NOREG.EXPR(Strings).NOREG);
// addl (%r11), %r12d // Add string index
INS(ADD32rm, REG(R12D).REG(R12D).REG(R11).IMM(1).NOREG.IMM(0).NOREG);
// addq $4, %r11
INS(ADD64ri8, REG(R11).REG(R11).IMM(4));
// loopbody1_1_1: // Loop over counter description string
// // copying it to our write buffer
BEGIN_BLOCK(LoopBody1_1_1);
// mov (%r12), %ax
INS(MOV8rm, REG(AX).REG(R12).IMM(1).NOREG.IMM(0).NOREG);
// mov %ax, (%r15)
INS(MOV8mr, REG(R15).IMM(1).NOREG.IMM(0).NOREG.REG(AX));
// incq %r15
INS(INC64r, REG(R15).REG(R15));
// incq %r12
INS(INC64r, REG(R12).REG(R12));
// test %ax, %ax
INS(TEST8rr, REG(AX).REG(AX));
// jnz loopbody1_1_1
INS(JNE_4, EXPR(LoopBody1_1_1));
SET_SUCC(LoopBody1_1_1);
BEGIN_BLOCK_FALLTHROUGH; // Copy " " to write buffer -- empty
// // spaces that will be overwritten with the
// // offset value in hex, right to left
// decq %r15
INS(DEC64r, REG(R15).REG(R15));
// movq spaces(%rip), %rdx
INS(MOV64rm, REG(RDX).REG(RIP).IMM(1).NOREG.EXPR(Spaces).NOREG);
// movq %rdx, (%r15)
INS(MOV64mr, REG(R15).IMM(1).NOREG.IMM(0).NOREG.REG(RDX));
// addq $8, %r15
INS(ADD64ri8, REG(R15).REG(R15).IMM(8));
// movq %rdx, (%r15)
INS(MOV64mr, REG(R15).IMM(1).NOREG.IMM(0).NOREG.REG(RDX));
// addq $8, %r15
INS(ADD64ri8, REG(R15).REG(R15).IMM(8));
// movq %r15, %r12
INS(MOV64rr, REG(R12).REG(R15));
// decq %r15
INS(DEC64r, REG(R15).REG(R15));
// xorq %rax, %rax
INS(XOR64rr, REG(RAX).REG(RAX).REG(RAX));
// movl (%r11), %eax
INS(MOV32rm, REG(EAX).REG(R11).IMM(1).NOREG.IMM(0).NOREG);
// addq $4, %r11
INS(ADD64ri8, REG(R11).REG(R11).IMM(4));
// loopbody1_1_2: // Loop to print address in hexadecimal
BEGIN_BLOCK(LoopBody1_1_2);
// decq %r15
INS(DEC64r, REG(R15).REG(R15));
// xorq %rdx, %rdx
INS(XOR64rr, REG(RDX).REG(RDX).REG(RDX));
// movq $0x10, %rsi
INS(MOV64ri32, REG(RSI).IMM(0x10));
// divq %rsi
INS(DIV64r, REG(RSI));
// leaq chars(%rip), %rdi
INS(LEA64r, REG(RDI).REG(RIP).IMM(1).NOREG.EXPR(Chars).NOREG);
// mov (%rdi, %rdx, 1), %cx
INS(MOV8rm, REG(CX).REG(RDI).IMM(1).REG(RDX).IMM(0).NOREG);
// mov %cx, (%r15)
INS(MOV8mr, REG(R15).IMM(1).NOREG.IMM(0).NOREG.REG(CX));
// testq %rax, %rax
INS(TEST64rr, REG(RAX).REG(RAX));
// jnz loopbody1_1_2
INS(JNE_4, EXPR(LoopBody1_1_2));
SET_SUCC(LoopBody1_1_2);
BEGIN_BLOCK_FALLTHROUGH;
// movq %r12, %r15 // Loop end (2 iteration loop for From/To)
INS(MOV64rr, REG(R15).REG(R12));
// incq %rbx
INS(INC64r, REG(RBX).REG(RBX));
// cmpq $2, %rbx
INS(CMP64ri8, REG(RBX).IMM(2));
// jne loopbody1_1
INS(JNE_4, EXPR(LoopBody1_1));
SET_SUCC(LoopBody1_1);
BEGIN_BLOCK_FALLTHROUGH; // Copy " " to write buffer -- empty
// // spaces that will be overwritten with the
// // counter value in decimal, right to left
// movb $0x30, (%r15) // Write '0' and a space before counter val
// // representing zero mispredictions
INS(MOV8mi, REG(R15).IMM(1).NOREG.IMM(0).NOREG.IMM(0x30));
// incq %r15
INS(INC64r, REG(R15).REG(R15));
// movb $0x20, (%r15)
INS(MOV8mi, REG(R15).IMM(1).NOREG.IMM(0).NOREG.IMM(0x20));
// incq %r15
INS(INC64r, REG(R15).REG(R15));
// movq spaces(%rip), %rdx
INS(MOV64rm, REG(RDX).REG(RIP).IMM(1).NOREG.EXPR(Spaces).NOREG);
// movq %rdx, (%r15)
INS(MOV64mr, REG(R15).IMM(1).NOREG.IMM(0).NOREG.REG(RDX));
// addq $8, %r15
INS(ADD64ri8, REG(R15).REG(R15).IMM(8));
// movq %rdx, (%r15)
INS(MOV64mr, REG(R15).IMM(1).NOREG.IMM(0).NOREG.REG(RDX));
// addq $8, %r15
INS(ADD64ri8, REG(R15).REG(R15).IMM(8));
// movq %r15, %r12
INS(MOV64rr, REG(R12).REG(R15));
// leaq count1(%rip), %rdx
INS(LEA64r, REG(RDX).REG(RIP).IMM(1).NOREG.EXPR(Locs).NOREG);
// movq (%rdx, %r14, 8), %rax // Load current instrumentation counter value
INS(MOV64rm, REG(RAX).REG(RDX).IMM(8).REG(R14).IMM(0).NOREG);
// testq $rax, %rax
INS(TEST64rr, REG(RAX).REG(RAX));
// je loop1end
INS(JE_4, EXPR(Loop1End));
SET_SUCC(Loop1End);
// loopbody1_2: // Loop to print counter value in decimal
BEGIN_BLOCK(LoopBody1_2);
// decq %r15
INS(DEC64r, REG(R15).REG(R15));
// xorq %rdx, %rdx
INS(XOR64rr, REG(RDX).REG(RDX).REG(RDX));
// movq $0xa, %rsi
INS(MOV64ri32, REG(RSI).IMM(0xa));
// divq %rsi
INS(DIV64r, REG(RSI));
// leaq chars(%rip), %rdi
INS(LEA64r, REG(RDI).REG(RIP).IMM(1).NOREG.EXPR(Chars).NOREG);
// mov (%rdi, %rdx, 1), %cx
INS(MOV8rm, REG(CX).REG(RDI).IMM(1).REG(RDX).IMM(0).NOREG);
// mov %cx, (%r15)
INS(MOV8mr, REG(R15).IMM(1).NOREG.IMM(0).NOREG.REG(CX));
// testq %rax, %rax
INS(TEST64rr, REG(RAX).REG(RAX));
// jnz loopbody1_2
INS(JNE_4, EXPR(LoopBody1_2));
SET_SUCC(LoopBody1_2);
BEGIN_BLOCK_FALLTHROUGH; // Flush write buffer to file
// movb $0xa, (%r12) // Put a '\n' at the end of write buffer
INS(MOV8mi, REG(R12).IMM(1).NOREG.IMM(0).NOREG.IMM(0xa));
// incq %r12
INS(INC64r, REG(R12).REG(R12));
// movq %r13, %rdi
INS(MOV64rr, REG(RDI).REG(R13));
// movq %rsp, %rsi
INS(MOV64rr, REG(RSI).REG(RSP));
// movq %r12, %rdx
INS(MOV64rr, REG(RDX).REG(R12));
// subq %rsp, %rdx
INS(SUB64rr, REG(RDX).REG(RDX).REG(RSP));
// movq $0x1, %rax
INS(MOV64ri32, REG(RAX).IMM(SYSCALL_WRITE));
// pushq %r11
INS(PUSH64r, REG(R11));
// syscall // write()
INS_NOARGS(SYSCALL);
// popq %r11
INS(POP64r, REG(R11));
// loop1end: // Main loop header
BEGIN_BLOCK(Loop1End);
// incq %r14
INS(INC64r, REG(R14).REG(R14));
// cmp $0xXXXXXX, %r14
INS(CMP64ri32, REG(R14).IMM(NumLocs));
// jnz loopbody1
INS(JNE_4, EXPR(LoopBody1));
SET_SUCC(LoopBody1);
BEGIN_BLOCK_FALLTHROUGH; // Finish by closing file and returning
// movq %r13, %rdi
INS(MOV64rr, REG(RDI).REG(R13));
// mov $0x3, %eax // close()
INS(MOV32ri, REG(EAX).IMM(SYSCALL_CLOSE));
// syscall
INS_NOARGS(SYSCALL);
// addq $0x1000, %rsp
INS(ADD64ri32, REG(RSP).REG(RSP).IMM(0x1000));
// popq %r15
INS(POP64r, REG(R15));
// popq %r14
INS(POP64r, REG(R14));
// popq %r13
INS(POP64r, REG(R13));
// popq %r12
INS(POP64r, REG(R12));
// popq %rbx
INS(POP64r, REG(RBX));
// ret
INS_NOARGS(RETQ);
#undef INS
#undef INS_NOARGS
#undef REG
#undef IMM
#undef EXPR
#undef NOREG
#undef BEGIN_BLOCK
#undef BEGIN_BLOCK_FALLTHROUGH
return Result;
}
};
}