[BOLT] Support full instrumentation

Summary:
Add full instrumentation support (branches, direct and
indirect calls). Add output statistics to show how many hot bytes
were split from cold ones in functions. Add -cold-threshold option
to allow splitting warm code (non-zero count). Add option in
bolt-diff to report missing functions in profile 2.

In instrumentation, fini hooks are fixed to run proper finalization
code after program finishes. Hooks for startup are added to setup
the runtime structures that needs initilization, such as indirect call
hash tables.

Add support for automatically dumping profile data every N seconds by
forking a watcher process during runtime.

(cherry picked from FBD17644396)
This commit is contained in:
Rafael Auler 2019-12-13 17:27:03 -08:00 committed by Maksim Panchenko
parent e46d52de5b
commit 16a497c627
18 changed files with 1959 additions and 532 deletions

View File

@ -18,7 +18,7 @@ add_library(bolt_rt STATIC
)
# Don't let the compiler think it can create calls to standard libs
target_compile_options(bolt_rt PRIVATE -ffreestanding)
target_compile_options(bolt_rt PRIVATE -ffreestanding -fno-exceptions -fno-rtti)
target_include_directories(bolt_rt PRIVATE ${CMAKE_CURRENT_BINARY_DIR})
install(TARGETS bolt_rt DESTINATION lib)

File diff suppressed because it is too large Load Diff

View File

@ -2013,6 +2013,7 @@ void BinaryFunction::addEntryPoint(uint64_t Address) {
addEntryPointAtOffset(Offset);
Labels.emplace(Offset, EntrySymbol);
BC.setSymbolToFunctionMap(EntrySymbol, this);
if (OldSym != nullptr && EntrySymbol != OldSym) {
updateReferences(OldSym, EntrySymbol);
}
@ -3753,6 +3754,46 @@ BinaryBasicBlock *BinaryFunction::splitEdge(BinaryBasicBlock *From,
return NewBBPtr;
}
void BinaryFunction::deleteConservativeEdges() {
// Our goal is to aggressively remove edges from the CFG that we believe are
// wrong. This is used for instrumentation, where it is safe to remove
// fallthrough edges because we won't reorder blocks.
for (auto I = BasicBlocks.begin(), E = BasicBlocks.end(); I != E; ++I) {
auto BB = *I;
if (BB->succ_size() != 1 || BB->size() == 0)
continue;
auto NextBB = std::next(I);
MCInst* Last = BB->getLastNonPseudoInstr();
// Fallthrough is a landing pad? Delete this edge (as long as we don't
// have a direct jump to it)
if ((*BB->succ_begin())->isLandingPad() && NextBB != E &&
*BB->succ_begin() == *NextBB && Last && !BC.MIB->isBranch(*Last)) {
BB->removeAllSuccessors();
continue;
}
// Look for suspicious calls at the end of BB where gcc may optimize it and
// remove the jump to the epilogue when it knows the call won't return.
if (!Last || !BC.MIB->isCall(*Last))
continue;
auto *CalleeSymbol = BC.MIB->getTargetSymbol(*Last);
if (!CalleeSymbol)
continue;
StringRef CalleeName = CalleeSymbol->getName();
if (CalleeName != "__cxa_throw@PLT" &&
CalleeName != "_Unwind_Resume@PLT" &&
CalleeName != "__cxa_rethrow@PLT" &&
CalleeName != "exit@PLT" &&
CalleeName != "abort@PLT" )
continue;
BB->removeAllSuccessors();
}
}
bool BinaryFunction::isDataMarker(const SymbolRef &Symbol,
uint64_t SymbolSize) const {
// For aarch64, the ABI defines mapping symbols so we identify data in the

View File

@ -571,7 +571,9 @@ private:
MCSymbol *addEntryPointAtOffset(uint64_t Offset) {
EntryOffsets.emplace(Offset);
IsMultiEntry = (Offset == 0 ? IsMultiEntry : true);
return getOrCreateLocalLabel(getAddress() + Offset);
MCSymbol *Sym = getOrCreateLocalLabel(getAddress() + Offset);
BC.setSymbolToFunctionMap(Sym, this);
return Sym;
}
/// Register an internal offset in a function referenced from outside.
@ -1489,6 +1491,11 @@ public:
/// correct CFG edges.
BinaryBasicBlock *splitEdge(BinaryBasicBlock *From, BinaryBasicBlock *To);
/// We may have built an overly conservative CFG for functions with calls
/// to functions that the compiler knows will never return. In this case,
/// clear all successors from these blocks.
void deleteConservativeEdges();
/// Determine direction of the branch based on the current layout.
/// Callee is responsible of updating basic block indices prior to using
/// this function (e.g. by calling BinaryFunction::updateLayoutIndices()).

View File

@ -43,6 +43,14 @@ PrintUnmapped("print-unmapped",
cl::ZeroOrMore,
cl::cat(BoltDiffCategory));
static cl::opt<bool>
PrintProfiledUnmapped("print-profiled-unmapped",
cl::desc("print functions that have profile in binary 1 but do not "
"in binary 2"),
cl::init(false),
cl::ZeroOrMore,
cl::cat(BoltDiffCategory));
static cl::opt<bool>
PrintDiffCFG("print-diff-cfg",
cl::desc("print the CFG of important functions that changed in "
@ -244,6 +252,8 @@ class RewriteInstanceDiff {
/// Match functions in binary 2 with functions in binary 1
void matchFunctions() {
outs() << "BOLT-DIFF: Mapping functions in Binary2 to Binary1\n";
uint64_t BothHaveProfile = 0ull;
std::set<const BinaryFunction *> Bin1ProfiledMapped;
for (const auto &BFI2 : RI2.BC->getBinaryFunctions()) {
const auto &Function2 = BFI2.second;
@ -258,6 +268,10 @@ class RewriteInstanceDiff {
FuncMap.insert(std::make_pair<>(&Function2, Iter->second));
Bin1MappedFuncs.insert(Iter->second);
Bin2MappedFuncs.insert(&Function2);
if (Function2.hasValidProfile() && Iter->second->hasValidProfile()) {
++BothHaveProfile;
Bin1ProfiledMapped.insert(Iter->second);
}
Match = true;
break;
}
@ -268,6 +282,10 @@ class RewriteInstanceDiff {
FuncMap.insert(std::make_pair<>(&Function2, Iter->second));
Bin1MappedFuncs.insert(Iter->second);
Bin2MappedFuncs.insert(&Function2);
if (Function2.hasValidProfile() && Iter->second->hasValidProfile()) {
++BothHaveProfile;
Bin1ProfiledMapped.insert(Iter->second);
}
continue;
}
if (LTOName.empty())
@ -277,8 +295,39 @@ class RewriteInstanceDiff {
FuncMap.insert(std::make_pair<>(&Function2, LTOIter->second));
Bin1MappedFuncs.insert(LTOIter->second);
Bin2MappedFuncs.insert(&Function2);
if (Function2.hasValidProfile() && LTOIter->second->hasValidProfile()) {
++BothHaveProfile;
Bin1ProfiledMapped.insert(LTOIter->second);
}
}
}
PrintProgramStats PPS(opts::NeverPrint);
outs() << "* BOLT-DIFF: Starting print program stats pass for binary 1\n";
PPS.runOnFunctions(*RI1.BC);
outs() << "* BOLT-DIFF: Starting print program stats pass for binary 2\n";
PPS.runOnFunctions(*RI2.BC);
outs() << "=====\n";
outs() << "Inputs share " << BothHaveProfile
<< " functions with valid profile.\n";
if (opts::PrintProfiledUnmapped) {
outs() << "\nFunctions in profile 1 that are missing in the profile 2:\n";
std::vector<const BinaryFunction *> Unmapped;
for (const auto &BFI : RI1.BC->getBinaryFunctions()) {
const auto &Function = BFI.second;
if (!Function.hasValidProfile() || Bin1ProfiledMapped.count(&Function))
continue;
Unmapped.emplace_back(&Function);
}
std::sort(Unmapped.begin(), Unmapped.end(),
[&](const BinaryFunction *A, const BinaryFunction *B) {
return A->getFunctionScore() > B->getFunctionScore();
});
for (auto Function : Unmapped) {
outs() << Function->getPrintName() << " : ";
outs() << Function->getFunctionScore() << "\n";
}
outs() << "=====\n";
}
}
/// Check if opcodes in BB1 match those in BB2

View File

@ -77,38 +77,23 @@ void DWARFRewriter::updateDebugInfo() {
ARangesSectionWriter = llvm::make_unique<DebugARangesSectionWriter>();
RangesSectionWriter = llvm::make_unique<DebugRangesSectionWriter>(&BC);
LocationListWriter = llvm::make_unique<DebugLocWriter>(&BC);
size_t NumCUs = BC.DwCtx->getNumCompileUnits();
if (opts::NoThreads || opts::DeterministicDebugInfo) {
// Use single entry for efficiency when running single-threaded
NumCUs = 1;
}
LocListWritersByCU.resize(NumCUs);
for (size_t CUIndex = 0; CUIndex < NumCUs; ++CUIndex) {
LocListWritersByCU[CUIndex] = llvm::make_unique<DebugLocWriter>(&BC);
}
auto processUnitDIE = [&](size_t CUIndex, const DWARFDie DIE) {
auto processUnitDIE = [&](const DWARFDie DIE) {
const BinaryFunction *CachedFunction = nullptr;
std::map<DebugAddressRangesVector, uint64_t> CachedRanges{};
updateUnitDebugInfo(CUIndex, DIE, std::vector<const BinaryFunction *>{},
updateUnitDebugInfo(DIE, std::vector<const BinaryFunction *>{},
CachedFunction, CachedRanges);
};
if (opts::NoThreads || opts::DeterministicDebugInfo) {
for (auto &CU : BC.DwCtx->compile_units()) {
processUnitDIE(0, CU->getUnitDIE(false));
}
for (auto &CU : BC.DwCtx->compile_units())
processUnitDIE(CU->getUnitDIE(false));
} else {
// Update unit debug info in parallel
auto &ThreadPool = ParallelUtilities::getThreadPool();
size_t CUIndex = 0;
for (auto &CU : BC.DwCtx->compile_units()) {
ThreadPool.async(processUnitDIE, CUIndex, CU->getUnitDIE(false));
CUIndex++;
}
for (auto &CU : BC.DwCtx->compile_units())
ThreadPool.async(processUnitDIE, CU->getUnitDIE(false));
ThreadPool.wait();
}
@ -121,7 +106,6 @@ void DWARFRewriter::updateDebugInfo() {
}
void DWARFRewriter::updateUnitDebugInfo(
size_t CUIndex,
const DWARFDie DIE, std::vector<const BinaryFunction *> FunctionStack,
const BinaryFunction *&CachedFunction,
std::map<DebugAddressRangesVector, uint64_t> &CachedRanges) {
@ -235,7 +219,7 @@ void DWARFRewriter::updateUnitDebugInfo(
Value = *V;
if (Value.isFormClass(DWARFFormValue::FC_Constant) ||
Value.isFormClass(DWARFFormValue::FC_SectionOffset)) {
auto LocListOffset = DebugLocWriter::EmptyListTag;
auto LocListSectionOffset = LocationListWriter->getEmptyListOffset();
if (Function) {
// Limit parsing to a single list to save memory.
DWARFDebugLoc::LocationList LL;
@ -263,19 +247,12 @@ void DWARFRewriter::updateUnitDebugInfo(
<< Twine::utohexstr(DIE.getDwarfUnit()->getOffset())
<< '\n';
});
LocListOffset = LocListWritersByCU[CUIndex]->addList(OutputLL);
LocListSectionOffset = LocationListWriter->addList(OutputLL);
}
}
if (LocListOffset != DebugLocWriter::EmptyListTag) {
std::lock_guard<std::mutex> Lock(LocListDebugInfoPatchesMutex);
LocListDebugInfoPatches.push_back(
{AttrOffset, CUIndex, LocListOffset});
} else {
std::lock_guard<std::mutex> Lock(DebugInfoPatcherMutex);
DebugInfoPatcher->addLE32Patch(AttrOffset,
DebugLocWriter::EmptyListOffset);
}
std::lock_guard<std::mutex> Lock(DebugInfoPatcherMutex);
DebugInfoPatcher->addLE32Patch(AttrOffset, LocListSectionOffset);
} else {
assert((Value.isFormClass(DWARFFormValue::FC_Exprloc) ||
Value.isFormClass(DWARFFormValue::FC_Block)) &&
@ -307,8 +284,7 @@ void DWARFRewriter::updateUnitDebugInfo(
// Recursively update each child.
for (auto Child = DIE.getFirstChild(); Child; Child = Child.getSibling()) {
updateUnitDebugInfo(CUIndex, Child, FunctionStack, CachedFunction,
CachedRanges);
updateUnitDebugInfo(Child, FunctionStack, CachedFunction, CachedRanges);
}
if (IsFunctionDef)
@ -518,7 +494,7 @@ void DWARFRewriter::finalizeDebugSections() {
copyByteArray(*RangesSectionContents),
RangesSectionContents->size());
auto LocationListSectionContents = makeFinalLocListsSection();
auto LocationListSectionContents = LocationListWriter->finalize();
BC.registerOrUpdateNoteSection(".debug_loc",
copyByteArray(*LocationListSectionContents),
LocationListSectionContents->size());
@ -687,39 +663,6 @@ void DWARFRewriter::convertPending(const DWARFAbbreviationDeclaration *Abbrev) {
ConvertedRangesAbbrevs.emplace(Abbrev);
}
std::unique_ptr<LocBufferVector> DWARFRewriter::makeFinalLocListsSection() {
auto LocBuffer = llvm::make_unique<LocBufferVector>();
auto LocStream = llvm::make_unique<raw_svector_ostream>(*LocBuffer);
auto Writer =
std::unique_ptr<MCObjectWriter>(BC.createObjectWriter(*LocStream));
uint32_t SectionOffset = 0;
// Add an empty list as the first entry;
Writer->writeLE64(0);
Writer->writeLE64(0);
SectionOffset += 2 * 8;
std::vector<uint32_t> SectionOffsetByCU(LocListWritersByCU.size());
for (size_t CUIndex = 0; CUIndex < LocListWritersByCU.size(); ++CUIndex) {
SectionOffsetByCU[CUIndex] = SectionOffset;
auto CurrCULocationLists = LocListWritersByCU[CUIndex]->finalize();
Writer->writeBytes(*CurrCULocationLists);
SectionOffset += CurrCULocationLists->size();
}
for (auto &Patch : LocListDebugInfoPatches) {
DebugInfoPatcher->addLE32Patch(
Patch.DebugInfoOffset,
SectionOffsetByCU[Patch.CUIndex]
+ Patch.CUWriterOffset
);
}
return std::move(LocBuffer);
}
void DWARFRewriter::flushPendingRanges() {
for (auto &I : PendingRanges) {
for (auto &RangePair : I.second) {

View File

@ -16,7 +16,6 @@
#include "RewriteInstance.h"
#include <map>
#include <mutex>
#include <vector>
namespace llvm {
@ -49,26 +48,12 @@ class DWARFRewriter {
/// .debug_aranges DWARF section.
std::unique_ptr<DebugARangesSectionWriter> ARangesSectionWriter;
/// Use a separate location list writer for each compilation unit
std::vector<std::unique_ptr<DebugLocWriter>> LocListWritersByCU;
struct LocListDebugInfoPatchType {
uint32_t DebugInfoOffset;
size_t CUIndex;
uint64_t CUWriterOffset;
};
/// The list of debug info patches to be made once individual
/// location list writers have been filled
std::vector<LocListDebugInfoPatchType> LocListDebugInfoPatches;
std::mutex LocListDebugInfoPatchesMutex;
std::unique_ptr<DebugLocWriter> LocationListWriter;
/// Recursively update debug info for all DIEs in \p Unit.
/// If \p Function is not empty, it points to a function corresponding
/// to a parent DW_TAG_subprogram node of the current \p DIE.
void updateUnitDebugInfo(
size_t CUIndex,
const DWARFDie DIE, std::vector<const BinaryFunction *> FunctionStack,
const BinaryFunction *&CachedFunction,
std::map<DebugAddressRangesVector, uint64_t> &CachedRanges);
@ -83,8 +68,6 @@ class DWARFRewriter {
void updateDWARFObjectAddressRanges(const DWARFDie DIE,
uint64_t DebugRangesOffset);
std::unique_ptr<LocBufferVector> makeFinalLocListsSection();
/// Generate new contents for .debug_ranges and .debug_aranges section.
void finalizeDebugSections();

View File

@ -153,15 +153,21 @@ DebugLocWriter::DebugLocWriter(BinaryContext *BC) {
LocStream = llvm::make_unique<raw_svector_ostream>(*LocBuffer);
Writer =
std::unique_ptr<MCObjectWriter>(BC->createObjectWriter(*LocStream));
// Add an empty list as the first entry;
Writer->writeLE64(0);
Writer->writeLE64(0);
SectionOffset += 2 * 8;
}
// DWARF 4: 2.6.2
uint64_t DebugLocWriter::addList(const DWARFDebugLoc::LocationList &LocList) {
if (LocList.Entries.empty())
return EmptyListTag;
return getEmptyListOffset();
// Since there is a separate DebugLocWriter for each thread,
// we don't need a lock to read the SectionOffset and update it.
// Reading the SectionOffset and updating it should be atomic to guarantee
// unique and correct offsets in patches.
std::lock_guard<std::mutex> Lock(WriterMutex);
const auto EntryOffset = SectionOffset;
for (const auto &Entry : LocList.Entries) {

View File

@ -168,25 +168,19 @@ private:
using LocBufferVector = SmallVector<char, 16>;
/// Serializes part of a .debug_loc DWARF section with LocationLists.
/// Serializes the .debug_loc DWARF section with LocationLists.
class DebugLocWriter {
public:
DebugLocWriter(BinaryContext *BC);
uint64_t addList(const DWARFDebugLoc::LocationList &LocList);
uint64_t getEmptyListOffset() const { return EmptyListOffset; }
std::unique_ptr<LocBufferVector> finalize() {
return std::move(LocBuffer);
}
/// Offset of an empty location list.
static constexpr uint32_t EmptyListOffset = 0;
/// Value returned by addList if list is empty
/// Use 64 bits here so that a max 32 bit value can still
/// be stored while we use max 64 bit value as empty tag
static constexpr uint64_t EmptyListTag = -1;
private:
std::unique_ptr<LocBufferVector> LocBuffer;
@ -194,10 +188,13 @@ private:
std::unique_ptr<MCObjectWriter> Writer;
std::mutex WriterMutex;
/// Offset of an empty location list.
static uint64_t const EmptyListOffset = 0;
/// Current offset in the section (updated as new entries are written).
/// Starts with 0 here since this only writes part of a full location lists
/// section. In the final section, the first 16 bytes are reserved for an
/// empty list.
/// Starts with 16 since the first 16 bytes are reserved for an empty range.
uint32_t SectionOffset{0};
};

View File

@ -1229,6 +1229,11 @@ public:
return false;
}
/// Morph an indirect call into a load where \p Reg holds the call target.
virtual void convertIndirectCallToLoad(MCInst &Inst, MCPhysReg Reg) const {
llvm_unreachable("not implemented");
}
/// Replace instruction with a shorter version that could be relaxed later
/// if needed.
virtual bool shortenInstruction(MCInst &Inst) const {
@ -1360,6 +1365,12 @@ public:
return false;
}
virtual bool createIndirectCall(MCInst &Inst, const MCSymbol *TargetLocation,
MCContext *Ctx, bool IsTailCall) const {
llvm_unreachable("not implemented");
return false;
}
/// Creates a new tail call instruction in Inst and sets its operand to
/// Target.
///
@ -1410,6 +1421,11 @@ public:
return false;
}
virtual void createLoadImmediate(MCInst &Inst, const MCPhysReg Dest,
uint32_t Imm) const {
llvm_unreachable("not implemented");
}
/// Create instruction to increment contents of target by 1
virtual bool createIncMemory(MCInst &Inst, const MCSymbol *Target,
MCContext *Ctx) const {
@ -1693,6 +1709,24 @@ public:
/// Remove meta-data, but don't destroy it.
void stripAnnotations(MCInst &Inst);
virtual std::vector<MCInst>
createInstrumentedIndirectCall(const MCInst &CallInst, bool TailCall,
MCSymbol *HandlerFuncAddr, int CallSiteID,
MCContext *Ctx) const {
llvm_unreachable("not implemented");
return std::vector<MCInst>();
}
virtual std::vector<MCInst> createInstrumentedNoopIndCallHandler() const {
llvm_unreachable("not implemented");
return std::vector<MCInst>();
}
virtual std::vector<MCInst> createInstrumentedNoopIndTailCallHandler() const {
llvm_unreachable("not implemented");
return std::vector<MCInst>();
}
/// This method takes an indirect call instruction and splits it up into an
/// equivalent set of instructions that use direct calls for target
/// symbols/addresses that are contained in the Targets vector. This is done

View File

@ -30,6 +30,27 @@ cl::opt<std::string> InstrumentationFilename(
cl::Optional,
cl::cat(BoltCategory));
cl::opt<bool> InstrumentationFileAppendPID(
"instrumentation-file-append-pid",
cl::desc("append PID to saved profile file name (default: false)"),
cl::init(false),
cl::Optional,
cl::cat(BoltCategory));
cl::opt<bool> ConservativeInstrumentation(
"conservative-instrumentation",
cl::desc(
"don't trust our CFG and disable spanning trees and any counter "
"inference, put a counter everywhere (for debugging, default: false)"),
cl::init(false), cl::Optional, cl::cat(BoltCategory));
cl::opt<uint32_t>
InstrumentationSleepTime("instrumentation-sleep-time",
cl::desc("interval between profile writes, "
"default: 0 = write only at program end"),
cl::init(0), cl::Optional,
cl::cat(BoltCategory));
cl::opt<bool> InstrumentHotOnly(
"instrument-hot-only",
cl::desc("only insert instrumentation on hot functions (need profile)"),
@ -40,7 +61,7 @@ cl::opt<bool> InstrumentHotOnly(
cl::opt<bool> InstrumentCalls(
"instrument-calls",
cl::desc("record profile for inter-function control flow activity"),
cl::init(false),
cl::init(true),
cl::Optional,
cl::cat(BoltCategory));
}
@ -59,16 +80,45 @@ uint32_t Instrumentation::getFunctionNameIndex(const BinaryFunction &Function) {
return Idx;
}
void Instrumentation::createCallDescription(
const BinaryFunction &FromFunction, uint32_t From,
const BinaryFunction &ToFunction, uint32_t To) {
bool Instrumentation::createCallDescription(FunctionDescription &FuncDesc,
const BinaryFunction &FromFunction,
uint32_t From, uint32_t FromNodeID,
const BinaryFunction &ToFunction,
uint32_t To, bool IsInvoke) {
CallDescription CD;
// Ordinarily, we don't augment direct calls with an explicit counter, except
// when forced to do so or when we know this callee could be throwing
// exceptions, in which case there is no other way to accurately record its
// frequency.
bool ForceInstrumentation = opts::ConservativeInstrumentation || IsInvoke;
CD.FromLoc.FuncString = getFunctionNameIndex(FromFunction);
CD.FromLoc.Offset = From;
CD.FromNode = FromNodeID;
CD.Target = &ToFunction;
CD.ToLoc.FuncString = getFunctionNameIndex(ToFunction);
CD.ToLoc.Offset = To;
CD.Counter = Counters.size();
CallDescriptions.emplace_back(CD);
CD.Counter = ForceInstrumentation ? Counters.size() : 0xffffffff;
if (ForceInstrumentation)
++DirectCallCounters;
FuncDesc.Calls.emplace_back(CD);
return ForceInstrumentation;
}
void Instrumentation::createIndCallDescription(
const BinaryFunction &FromFunction, uint32_t From) {
IndCallDescription ICD;
ICD.FromLoc.FuncString = getFunctionNameIndex(FromFunction);
ICD.FromLoc.Offset = From;
IndCallDescriptions.emplace_back(ICD);
}
void Instrumentation::createIndCallTargetDescription(
const BinaryFunction &ToFunction, uint32_t To) {
IndCallTargetDescription ICD;
ICD.ToLoc.FuncString = getFunctionNameIndex(ToFunction);
ICD.ToLoc.Offset = To;
ICD.Target = &ToFunction;
IndCallTargetDescriptions.emplace_back(ICD);
}
bool Instrumentation::createEdgeDescription(
@ -90,16 +140,19 @@ bool Instrumentation::createEdgeDescription(
ED.ToLoc.Offset = To;
ED.ToNode = ToNodeID;
ED.Counter = Instrumented ? Counters.size() : 0xffffffff;
if (Instrumented)
++BranchCounters;
FuncDesc.Edges.emplace_back(ED);
return true;
return Instrumented;
}
void Instrumentation::createExitNodeDescription(FunctionDescription &FuncDesc,
void Instrumentation::createLeafNodeDescription(FunctionDescription &FuncDesc,
uint32_t Node) {
InstrumentedNode IN;
IN.Node = Node;
IN.Counter = Counters.size();
FuncDesc.ExitNodes.emplace_back(IN);
++LeafNodeCounters;
FuncDesc.LeafNodes.emplace_back(IN);
}
std::vector<MCInst>
@ -122,13 +175,13 @@ Instrumentation::createInstrumentationSnippet(BinaryContext &BC, bool IsLeaf) {
return CounterInstrs;
}
void Instrumentation::instrumentExitNode(BinaryContext &BC,
void Instrumentation::instrumentLeafNode(BinaryContext &BC,
BinaryBasicBlock &BB,
BinaryBasicBlock::iterator Iter,
bool IsLeaf,
FunctionDescription &FuncDesc,
uint32_t Node) {
createExitNodeDescription(FuncDesc, Node);
createLeafNodeDescription(FuncDesc, Node);
std::vector<MCInst> CounterInstrs = createInstrumentationSnippet(BC, IsLeaf);
for (auto &NewInst : CounterInstrs) {
@ -137,17 +190,41 @@ void Instrumentation::instrumentExitNode(BinaryContext &BC,
}
}
void Instrumentation::instrumentIndirectTarget(BinaryBasicBlock &BB,
BinaryBasicBlock::iterator &Iter,
BinaryFunction &FromFunction,
uint32_t From) {
auto L = FromFunction.getBinaryContext().scopeLock();
const auto IndCallSiteID = IndCallDescriptions.size();
createIndCallDescription(FromFunction, From);
BinaryContext &BC = FromFunction.getBinaryContext();
bool IsTailCall = BC.MIB->isTailCall(*Iter);
std::vector<MCInst> CounterInstrs = BC.MIB->createInstrumentedIndirectCall(
*Iter, IsTailCall,
IsTailCall ? IndTailCallHandlerFunc : IndCallHandlerFunc, IndCallSiteID,
&*BC.Ctx);
Iter = BB.eraseInstruction(Iter);
for (auto &NewInst : CounterInstrs) {
Iter = BB.insertInstruction(Iter, NewInst);
++Iter;
}
--Iter;
}
bool Instrumentation::instrumentOneTarget(
SplitWorklistTy &SplitWorklist, SplitInstrsTy &SplitInstrs,
BinaryBasicBlock::iterator &Iter, BinaryFunction &FromFunction,
BinaryBasicBlock &FromBB, uint32_t From, BinaryFunction &ToFunc,
BinaryBasicBlock *TargetBB, uint32_t ToOffset, bool IsLeaf,
BinaryBasicBlock *TargetBB, uint32_t ToOffset, bool IsLeaf, bool IsInvoke,
FunctionDescription *FuncDesc, uint32_t FromNodeID, uint32_t ToNodeID) {
{
auto L = FromFunction.getBinaryContext().scopeLock();
bool Created{true};
if (!TargetBB)
createCallDescription(FromFunction, From, ToFunc, ToOffset);
Created = createCallDescription(*FuncDesc, FromFunction, From, FromNodeID,
ToFunc, ToOffset, IsInvoke);
else
Created = createEdgeDescription(*FuncDesc, FromFunction, From, FromNodeID,
ToFunc, ToOffset, ToNodeID,
@ -209,7 +286,9 @@ void Instrumentation::instrumentFunction(BinaryContext &BC,
FuncDesc = &FunctionDescriptions.back();
}
FuncDesc->Function = &Function;
Function.disambiguateJumpTables(AllocId);
Function.deleteConservativeEdges();
std::unordered_map<const BinaryBasicBlock *, uint32_t> BBToID;
uint32_t Id = 0;
@ -228,48 +307,57 @@ void Instrumentation::instrumentFunction(BinaryContext &BC,
STOutSet;
for (auto BBI = Function.layout_rbegin(); BBI != Function.layout_rend();
++BBI) {
if ((*BBI)->isEntryPoint())
if ((*BBI)->isEntryPoint() || (*BBI)->isLandingPad()) {
Stack.push(std::make_pair(nullptr, *BBI));
if (opts::InstrumentCalls && (*BBI)->isEntryPoint()) {
EntryNode E;
E.Node = BBToID[&**BBI];
E.Address = (*BBI)->getInputOffset();
FuncDesc->EntryNodes.emplace_back(E);
createIndCallTargetDescription(Function, (*BBI)->getInputOffset());
}
}
}
// Modified version of BinaryFunction::dfs() to build a spanning tree
while (!Stack.empty()) {
BinaryBasicBlock *BB;
const BinaryBasicBlock *Pred;
std::tie(Pred, BB) = Stack.top();
Stack.pop();
if (VisitedSet.find(BB) != VisitedSet.end())
continue;
if (!opts::ConservativeInstrumentation) {
while (!Stack.empty()) {
BinaryBasicBlock *BB;
const BinaryBasicBlock *Pred;
std::tie(Pred, BB) = Stack.top();
Stack.pop();
if (VisitedSet.find(BB) != VisitedSet.end())
continue;
VisitedSet.insert(BB);
if (Pred)
STOutSet[Pred].insert(BB);
VisitedSet.insert(BB);
if (Pred)
STOutSet[Pred].insert(BB);
for (auto *SuccBB : BB->landing_pads())
Stack.push(std::make_pair(BB, SuccBB));
for (auto *SuccBB : BB->successors())
Stack.push(std::make_pair(BB, SuccBB));
for (auto *SuccBB : BB->successors())
Stack.push(std::make_pair(BB, SuccBB));
}
}
// Determine whether this is a leaf function, which needs special
// instructions to protect the red zone
bool IsLeafFunction{true};
DenseSet<const BinaryBasicBlock *> InvokeBlocks;
for (auto BBI = Function.begin(), BBE = Function.end(); BBI != BBE; ++BBI) {
for (auto I = BBI->begin(), E = BBI->end(); I != E; ++I) {
if (BC.MIB->isCall(*I)) {
if (BC.MIB->isInvoke(*I)) {
InvokeBlocks.insert(&*BBI);
}
IsLeafFunction = false;
break;
}
}
if (!IsLeafFunction)
break;
}
for (auto BBI = Function.begin(), BBE = Function.end(); BBI != BBE; ++BBI) {
auto &BB{*BBI};
bool HasUnconditionalBranch{false};
bool HasJumpTable{false};
bool IsInvokeBlock = InvokeBlocks.count(&BB) > 0;
for (auto I = BB.begin(); I != BB.end(); ++I) {
const auto &Inst = *I;
@ -293,10 +381,15 @@ void Instrumentation::instrumentFunction(BinaryContext &BC,
TargetBB ? &Function : BC.getFunctionForSymbol(Target);
// Should be null for indirect branches/calls
if (TargetFunc && !TargetBB) {
if (opts::InstrumentCalls)
if (opts::InstrumentCalls) {
const auto *ForeignBB = TargetFunc->getBasicBlockForLabel(Target);
if (ForeignBB)
ToOffset = ForeignBB->getInputOffset();
instrumentOneTarget(SplitWorklist, SplitInstrs, I, Function, BB,
FromOffset, *TargetFunc, TargetBB, ToOffset,
IsLeafFunction);
IsLeafFunction, IsInvokeBlock, FuncDesc,
BBToID[&BB]);
}
continue;
}
if (TargetFunc) {
@ -310,8 +403,8 @@ void Instrumentation::instrumentFunction(BinaryContext &BC,
}
instrumentOneTarget(SplitWorklist, SplitInstrs, I, Function, BB,
FromOffset, *TargetFunc, TargetBB, ToOffset,
IsLeafFunction, FuncDesc, BBToID[&BB],
BBToID[TargetBB]);
IsLeafFunction, IsInvokeBlock, FuncDesc,
BBToID[&BB], BBToID[TargetBB]);
continue;
}
@ -325,14 +418,20 @@ void Instrumentation::instrumentFunction(BinaryContext &BC,
BBToID[&*Succ], /*Instrumented=*/false);
continue;
}
instrumentOneTarget(SplitWorklist, SplitInstrs, I, Function, BB,
FromOffset, Function, &*Succ,
Succ->getInputOffset(), IsLeafFunction, FuncDesc,
BBToID[&BB], BBToID[&*Succ]);
instrumentOneTarget(
SplitWorklist, SplitInstrs, I, Function, BB, FromOffset, Function,
&*Succ, Succ->getInputOffset(), IsLeafFunction, IsInvokeBlock,
FuncDesc, BBToID[&BB], BBToID[&*Succ]);
}
continue;
}
// FIXME: handle indirect calls
// Handle indirect calls -- could be direct calls with unknown targets
// or secondary entry points of known functions, so check it is indirect
// to be sure.
if (opts::InstrumentCalls && BC.MIB->isIndirectCall(*I))
instrumentIndirectTarget(BB, I, Function, FromOffset);
} // End of instructions loop
// Instrument fallthroughs (when the direct jump instruction is missing)
@ -364,16 +463,19 @@ void Instrumentation::instrumentFunction(BinaryContext &BC,
}
instrumentOneTarget(SplitWorklist, SplitInstrs, I, Function, BB,
FromOffset, Function, FTBB, FTBB->getInputOffset(),
IsLeafFunction, FuncDesc, BBToID[&BB], BBToID[FTBB]);
IsLeafFunction, IsInvokeBlock, FuncDesc, BBToID[&BB],
BBToID[FTBB]);
}
} // End of BBs loop
// Instrument spanning tree leaves
for (auto BBI = Function.begin(), BBE = Function.end(); BBI != BBE; ++BBI) {
auto &BB{*BBI};
if (STOutSet[&BB].size() == 0 && BB.size() > 0)
instrumentExitNode(BC, BB, BB.begin(), IsLeafFunction, *FuncDesc,
BBToID[&BB]);
if (!opts::ConservativeInstrumentation) {
for (auto BBI = Function.begin(), BBE = Function.end(); BBI != BBE; ++BBI) {
auto &BB{*BBI};
if (STOutSet[&BB].size() == 0)
instrumentLeafNode(BC, BB, BB.begin(), IsLeafFunction, *FuncDesc,
BBToID[&BB]);
}
}
// Consume list of critical edges: split them and add instrumentation to the
@ -405,6 +507,10 @@ void Instrumentation::runOnFunctions(BinaryContext &BC) {
/*Alignment=*/1,
/*IsReadOnly=*/true, ELF::SHT_NOTE);
IndCallHandlerFunc = BC.Ctx->getOrCreateSymbol("__bolt_trampoline_ind_call");
IndTailCallHandlerFunc =
BC.Ctx->getOrCreateSymbol("__bolt_trampoline_ind_tailcall");
ParallelUtilities::PredicateTy SkipPredicate = [&](const BinaryFunction &BF) {
return (!BF.isSimple() || !opts::shouldProcess(BF) ||
(opts::InstrumentHotOnly && !BF.getKnownExecutionCount()));
@ -418,13 +524,43 @@ void Instrumentation::runOnFunctions(BinaryContext &BC) {
ParallelUtilities::runOnEachFunctionWithUniqueAllocId(
BC, ParallelUtilities::SchedulingPolicy::SP_INST_QUADRATIC, WorkFun,
SkipPredicate, "instrumentation", /* ForceSequential=*/true);
createAuxiliaryFunctions(BC);
}
void Instrumentation::createAuxiliaryFunctions(BinaryContext &BC) {
auto createSimpleFunction =
[&](StringRef Title, std::vector<MCInst> Instrs) -> BinaryFunction * {
BinaryFunction *Func = BC.createInjectedBinaryFunction(Title);
std::vector<std::unique_ptr<BinaryBasicBlock>> BBs;
BBs.emplace_back(
Func->createBasicBlock(BinaryBasicBlock::INVALID_OFFSET, nullptr));
BBs.back()->addInstructions(Instrs.begin(), Instrs.end());
BBs.back()->setCFIState(0);
Func->insertBasicBlocks(nullptr, std::move(BBs),
/*UpdateLayout=*/true,
/*UpdateCFIState=*/false);
Func->updateState(BinaryFunction::State::CFG_Finalized);
return Func;
};
InitialIndCallHandlerFunction =
createSimpleFunction("__bolt_instr_default_ind_call_handler",
BC.MIB->createInstrumentedNoopIndCallHandler());
InitialIndTailCallHandlerFunction =
createSimpleFunction("__bolt_instr_default_ind_tailcall_handler",
BC.MIB->createInstrumentedNoopIndTailCallHandler());
}
uint32_t Instrumentation::getFDSize() const {
uint32_t FuncDescSize = 0;
for (const auto &Func : FunctionDescriptions) {
FuncDescSize += 8 + Func.Edges.size() * sizeof(EdgeDescription) +
Func.ExitNodes.size() * sizeof(InstrumentedNode);
FuncDescSize += 16 + Func.Edges.size() * sizeof(EdgeDescription) +
Func.LeafNodes.size() * sizeof(InstrumentedNode) +
Func.Calls.size() * sizeof(CallDescription) +
Func.EntryNodes.size() * sizeof(EntryNode);
}
return FuncDescSize;
}
@ -432,26 +568,49 @@ uint32_t Instrumentation::getFDSize() const {
void Instrumentation::emitTablesAsELFNote(BinaryContext &BC) {
std::string TablesStr;
raw_string_ostream OS(TablesStr);
// This is sync'ed with runtime/instr.cpp:readDescriptions()
auto getOutputAddress = [](const BinaryFunction &Func,
uint64_t Offset) -> uint64_t {
return Offset == 0
? Func.getOutputAddress()
: Func.translateInputToOutputAddress(Func.getAddress() + Offset);
};
// Indirect targets need to be sorted for fast lookup during runtime
std::sort(IndCallTargetDescriptions.begin(), IndCallTargetDescriptions.end(),
[&](const IndCallTargetDescription &A,
const IndCallTargetDescription &B) {
return getOutputAddress(*A.Target, A.ToLoc.Offset) <
getOutputAddress(*B.Target, B.ToLoc.Offset);
});
// Start of the vector with descriptions (one CounterDescription for each
// counter), vector size is Counters.size() CounterDescription-sized elmts
const auto CDSize = CallDescriptions.size() * sizeof(CallDescription);
OS.write(reinterpret_cast<const char *>(&CDSize), 4);
for (const auto &Desc : CallDescriptions) {
const auto IDSize = IndCallDescriptions.size() * sizeof(IndCallDescription);
OS.write(reinterpret_cast<const char *>(&IDSize), 4);
for (const auto &Desc : IndCallDescriptions) {
OS.write(reinterpret_cast<const char *>(&Desc.FromLoc.FuncString), 4);
OS.write(reinterpret_cast<const char *>(&Desc.FromLoc.Offset), 4);
}
const auto ITDSize =
IndCallTargetDescriptions.size() * sizeof(IndCallTargetDescription);
OS.write(reinterpret_cast<const char *>(&ITDSize), 4);
for (const auto &Desc : IndCallTargetDescriptions) {
OS.write(reinterpret_cast<const char *>(&Desc.ToLoc.FuncString), 4);
OS.write(reinterpret_cast<const char *>(&Desc.ToLoc.Offset), 4);
OS.write(reinterpret_cast<const char *>(&Desc.Counter), 4);
uint64_t TargetFuncAddress =
getOutputAddress(*Desc.Target, Desc.ToLoc.Offset);
OS.write(reinterpret_cast<const char *>(&TargetFuncAddress), 8);
}
const auto FDSize = getFDSize();
OS.write(reinterpret_cast<const char *>(&FDSize), 4);
for (const auto &Desc : FunctionDescriptions) {
const auto ExitsNum = Desc.ExitNodes.size();
OS.write(reinterpret_cast<const char *>(&ExitsNum), 4);
for (const auto &ExitNode : Desc.ExitNodes) {
OS.write(reinterpret_cast<const char *>(&ExitNode.Node), 4);
OS.write(reinterpret_cast<const char *>(&ExitNode.Counter), 4);
const auto LeafNum = Desc.LeafNodes.size();
OS.write(reinterpret_cast<const char *>(&LeafNum), 4);
for (const auto &LeafNode : Desc.LeafNodes) {
OS.write(reinterpret_cast<const char *>(&LeafNode.Node), 4);
OS.write(reinterpret_cast<const char *>(&LeafNode.Counter), 4);
}
const auto EdgesNum = Desc.Edges.size();
OS.write(reinterpret_cast<const char *>(&EdgesNum), 4);
@ -464,6 +623,27 @@ void Instrumentation::emitTablesAsELFNote(BinaryContext &BC) {
OS.write(reinterpret_cast<const char *>(&Edge.ToNode), 4);
OS.write(reinterpret_cast<const char *>(&Edge.Counter), 4);
}
const auto CallsNum = Desc.Calls.size();
OS.write(reinterpret_cast<const char *>(&CallsNum), 4);
for (const auto &Call : Desc.Calls) {
OS.write(reinterpret_cast<const char *>(&Call.FromLoc.FuncString), 4);
OS.write(reinterpret_cast<const char *>(&Call.FromLoc.Offset), 4);
OS.write(reinterpret_cast<const char *>(&Call.FromNode), 4);
OS.write(reinterpret_cast<const char *>(&Call.ToLoc.FuncString), 4);
OS.write(reinterpret_cast<const char *>(&Call.ToLoc.Offset), 4);
OS.write(reinterpret_cast<const char *>(&Call.Counter), 4);
uint64_t TargetFuncAddress =
getOutputAddress(*Call.Target, Call.ToLoc.Offset);
OS.write(reinterpret_cast<const char *>(&TargetFuncAddress), 8);
}
const auto EntryNum = Desc.EntryNodes.size();
OS.write(reinterpret_cast<const char *>(&EntryNum), 4);
for (const auto &EntryNode : Desc.EntryNodes) {
OS.write(reinterpret_cast<const char *>(&EntryNode.Node), 8);
uint64_t TargetFuncAddress =
getOutputAddress(*Desc.Function, EntryNode.Address);
OS.write(reinterpret_cast<const char *>(&TargetFuncAddress), 8);
}
}
// Our string table lives immediately after descriptions vector
OS << StringTable;
@ -476,9 +656,9 @@ void Instrumentation::emitTablesAsELFNote(BinaryContext &BC) {
/*IsReadOnly=*/true, ELF::SHT_NOTE);
}
void Instrumentation::emit(BinaryContext &BC, MCStreamer &Streamer) {
emitTablesAsELFNote(BC);
void Instrumentation::emit(BinaryContext &BC, MCStreamer &Streamer,
const BinaryFunction &InitFunction,
const BinaryFunction &FiniFunction) {
const auto Flags = BinarySection::getFlags(/*IsReadOnly=*/false,
/*IsText=*/false,
/*IsAllocatable=*/true);
@ -491,12 +671,21 @@ void Instrumentation::emit(BinaryContext &BC, MCStreamer &Streamer) {
// Label marking start of the memory region containing instrumentation
// counters, total vector size is Counters.size() 8-byte counters
MCSymbol *Locs = BC.Ctx->getOrCreateSymbol("__bolt_instr_locations");
MCSymbol *NumCalls = BC.Ctx->getOrCreateSymbol("__bolt_instr_num_calls");
MCSymbol *NumLocs = BC.Ctx->getOrCreateSymbol("__bolt_num_counters");
MCSymbol *NumIndCalls =
BC.Ctx->getOrCreateSymbol("__bolt_instr_num_ind_calls");
MCSymbol *NumIndCallTargets =
BC.Ctx->getOrCreateSymbol("__bolt_instr_num_ind_targets");
MCSymbol *NumFuncs = BC.Ctx->getOrCreateSymbol("__bolt_instr_num_funcs");
/// File name where profile is going to written to after target binary
/// finishes a run
MCSymbol *FilenameSym = BC.Ctx->getOrCreateSymbol("__bolt_instr_filename");
MCSymbol *UsePIDSym = BC.Ctx->getOrCreateSymbol("__bolt_instr_use_pid");
MCSymbol *InitPtr = BC.Ctx->getOrCreateSymbol("__bolt_instr_init_ptr");
MCSymbol *FiniPtr = BC.Ctx->getOrCreateSymbol("__bolt_instr_fini_ptr");
MCSymbol *SleepSym = BC.Ctx->getOrCreateSymbol("__bolt_instr_sleep_time");
Section->setAlignment(BC.RegularPageSize);
Streamer.SwitchSection(Section);
Streamer.EmitLabel(Locs);
Streamer.EmitSymbolAttribute(Locs,
@ -505,10 +694,39 @@ void Instrumentation::emit(BinaryContext &BC, MCStreamer &Streamer) {
Streamer.EmitLabel(Label);
Streamer.emitFill(8, 0);
}
Streamer.EmitLabel(NumCalls);
Streamer.EmitSymbolAttribute(NumCalls,
const uint64_t Padding =
alignTo(8 * Counters.size(), BC.RegularPageSize) - 8 * Counters.size();
if (Padding)
Streamer.emitFill(Padding, 0);
Streamer.EmitLabel(SleepSym);
Streamer.EmitSymbolAttribute(SleepSym,
MCSymbolAttr::MCSA_Global);
Streamer.EmitIntValue(CallDescriptions.size(), /*Size=*/4);
Streamer.EmitIntValue(opts::InstrumentationSleepTime, /*Size=*/4);
Streamer.EmitLabel(NumLocs);
Streamer.EmitSymbolAttribute(NumLocs,
MCSymbolAttr::MCSA_Global);
Streamer.EmitIntValue(Counters.size(), /*Size=*/4);
Streamer.EmitLabel(IndCallHandlerFunc);
Streamer.EmitSymbolAttribute(IndCallHandlerFunc,
MCSymbolAttr::MCSA_Global);
Streamer.EmitValue(MCSymbolRefExpr::create(
InitialIndCallHandlerFunction->getSymbol(), *BC.Ctx),
/*Size=*/8);
Streamer.EmitLabel(IndTailCallHandlerFunc);
Streamer.EmitSymbolAttribute(IndTailCallHandlerFunc,
MCSymbolAttr::MCSA_Global);
Streamer.EmitValue(
MCSymbolRefExpr::create(InitialIndTailCallHandlerFunction->getSymbol(),
*BC.Ctx),
/*Size=*/8);
Streamer.EmitLabel(NumIndCalls);
Streamer.EmitSymbolAttribute(NumIndCalls,
MCSymbolAttr::MCSA_Global);
Streamer.EmitIntValue(IndCallDescriptions.size(), /*Size=*/4);
Streamer.EmitLabel(NumIndCallTargets);
Streamer.EmitSymbolAttribute(NumIndCallTargets,
MCSymbolAttr::MCSA_Global);
Streamer.EmitIntValue(IndCallTargetDescriptions.size(), /*Size=*/4);
Streamer.EmitLabel(NumFuncs);
Streamer.EmitSymbolAttribute(NumFuncs,
MCSymbolAttr::MCSA_Global);
@ -516,20 +734,43 @@ void Instrumentation::emit(BinaryContext &BC, MCStreamer &Streamer) {
Streamer.EmitLabel(FilenameSym);
Streamer.EmitBytes(opts::InstrumentationFilename);
Streamer.emitFill(1, 0);
Streamer.EmitLabel(UsePIDSym);
Streamer.EmitIntValue(opts::InstrumentationFileAppendPID ? 1 : 0, /*Size=*/1);
Streamer.EmitLabel(InitPtr);
Streamer.EmitSymbolAttribute(InitPtr,
MCSymbolAttr::MCSA_Global);
Streamer.EmitValue(MCSymbolRefExpr::create(InitFunction.getSymbol(), *BC.Ctx),
/*Size=*/8);
Streamer.EmitLabel(FiniPtr);
Streamer.EmitSymbolAttribute(FiniPtr, MCSymbolAttr::MCSA_Global);
Streamer.EmitValue(MCSymbolRefExpr::create(FiniFunction.getSymbol(), *BC.Ctx),
/*Size=*/8);
uint32_t FuncDescSize = getFDSize();
outs() << "BOLT-INSTRUMENTER: Number of call descriptors: "
<< CallDescriptions.size() << "\n";
outs() << "BOLT-INSTRUMENTER: Number of indirect call site descriptors: "
<< IndCallDescriptions.size() << "\n";
outs() << "BOLT-INSTRUMENTER: Number of indirect call target descriptors: "
<< IndCallTargetDescriptions.size() << "\n";
outs() << "BOLT-INSTRUMENTER: Number of function descriptors: "
<< FunctionDescriptions.size() << "\n";
outs() << "BOLT-INSTRUMENTER: Number of counters: " << Counters.size()
outs() << "BOLT-INSTRUMENTER: Number of branch counters: " << BranchCounters
<< "\n";
outs() << "BOLT-INSTRUMENTER: Number of ST leaf node counters: "
<< LeafNodeCounters << "\n";
outs() << "BOLT-INSTRUMENTER: Number of direct call counters: "
<< DirectCallCounters << "\n";
outs() << "BOLT-INSTRUMENTER: Total number of counters: " << Counters.size()
<< "\n";
outs() << "BOLT-INSTRUMENTER: Total size of counters: "
<< (Counters.size() * 8) << " bytes (static alloc memory)\n";
outs() << "BOLT-INSTRUMENTER: Total size of string table emitted: "
<< StringTable.size() << " bytes in file\n";
outs() << "BOLT-INSTRUMENTER: Total size of descriptors: "
<< (FuncDescSize + CallDescriptions.size() * sizeof(CallDescription))
<< (FuncDescSize +
IndCallDescriptions.size() * sizeof(IndCallDescription) +
IndCallTargetDescriptions.size() *
sizeof(IndCallTargetDescription))
<< " bytes in file\n";
outs() << "BOLT-INSTRUMENTER: Profile will be saved to file "
<< opts::InstrumentationFilename << "\n";

View File

@ -7,6 +7,12 @@
//
//===----------------------------------------------------------------------===//
//
// This is an instrumentation pass that modifies the input binary to generate
// a profile after execution finishes. It can modify branches and calls to
// increment counters stored in the process memory. A runtime library is linked
// into the final binary to handle writing these counters to an fdata file. See
// runtime/instr.cpp
//
//===----------------------------------------------------------------------===//
#ifndef LLVM_TOOLS_LLVM_BOLT_PASSES_INSTRUMENTATION_H
#define LLVM_TOOLS_LLVM_BOLT_PASSES_INSTRUMENTATION_H
@ -19,30 +25,6 @@
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() {}
@ -51,10 +33,28 @@ public:
void runOnFunctions(BinaryContext &BC);
/// Emit data structures that will be necessary during runtime (second step)
void emit(BinaryContext &BC, MCStreamer &Streamer);
void emit(BinaryContext &BC, MCStreamer &Streamer,
const BinaryFunction &InitFunction,
const BinaryFunction &FiniFunction);
/// Create a non-allocatable ELF section with read-only tables necessary for
/// writing the instrumented data profile during program finish. The runtime
/// library needs to open the program executable file and read this data from
/// disk, this is not loaded by the system.
void emitTablesAsELFNote(BinaryContext &BC);
private:
// Location information -- this is a location in the program binary
// All structs here are part of the program metadata serialization format and
// consist of POD types or array of POD types that are trivially mapped from
// disk to memory. This provides the runtime library with a basic
// understanding of the program structure, so it can build a CFG for each
// function and deduce execution counts for edges that don't require explicit
// counters. It also provides function names and offsets used when writing the
// fdata file.
// Location information -- analoguous to the concept of the same name in fdata
// writing/reading. The difference is that the name is stored as an index to a
// string table written separately.
struct LocDescription {
uint32_t FuncString;
uint32_t Offset;
@ -63,8 +63,22 @@ private:
// Inter-function control flow transfer instrumentation
struct CallDescription {
LocDescription FromLoc;
uint32_t FromNode; // Node refers to the CFG node index of the call site
LocDescription ToLoc;
uint32_t Counter;
const BinaryFunction *Target;
};
// Spans multiple counters during runtime - this is an indirect call site
struct IndCallDescription {
LocDescription FromLoc;
};
// This is an indirect call target (any entry point from any function). This
// is stored sorted in the binary for fast lookups during data writing.
struct IndCallTargetDescription {
LocDescription ToLoc;
const BinaryFunction *Target;
};
// Intra-function control flow transfer instrumentation
@ -76,15 +90,29 @@ private:
uint32_t Counter;
};
// Basic block frequency (CFG node) instrumentation - only used for spanning
// tree leaf nodes.
struct InstrumentedNode {
uint32_t Node;
uint32_t Counter;
};
// Entry basic blocks for a function. We record their output addresses to
// check frequency of this address (via node number) against all tracked calls
// to this address and discover traffic coming from uninstrumented code.
struct EntryNode {
uint64_t Node;
uint64_t Address;
};
// Base struct organizing all metadata pertaining to a single function
struct FunctionDescription {
std::vector<InstrumentedNode> ExitNodes;
const BinaryFunction *Function;
std::vector<InstrumentedNode> LeafNodes;
std::vector<EdgeDescription> Edges;
DenseSet<std::pair<uint32_t, uint32_t>> EdgesSet;
std::vector<CallDescription> Calls;
std::vector<EntryNode> EntryNodes;
};
void instrumentFunction(BinaryContext &BC, BinaryFunction &Function,
@ -97,21 +125,24 @@ private:
/// 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.
void createCallDescription(const BinaryFunction &FromFunction, uint32_t From,
const BinaryFunction &ToFunction, uint32_t To);
/// Metadata creation methods
void createIndCallDescription(const BinaryFunction &FromFunction,
uint32_t From);
void createIndCallTargetDescription(const BinaryFunction &ToFunction,
uint32_t To);
bool createCallDescription(FunctionDescription &FuncDesc,
const BinaryFunction &FromFunction, uint32_t From,
uint32_t FromNodeID,
const BinaryFunction &ToFunction, uint32_t To,
bool IsInvoke);
bool createEdgeDescription(FunctionDescription &FuncDesc,
const BinaryFunction &FromFunction, uint32_t From,
uint32_t FromNodeID,
const BinaryFunction &ToFunction, uint32_t To,
uint32_t ToNodeID, bool Instrumented);
void createExitNodeDescription(FunctionDescription &FuncDesc, uint32_t Node);
void createLeafNodeDescription(FunctionDescription &FuncDesc, uint32_t Node);
/// Create the sequence of instructions to instrument a branch happening
/// at \p FromFunction + \p FromOffset to \p ToFunc + \p ToOffset
/// Create the sequence of instructions to increment a counter
std::vector<MCInst> createInstrumentationSnippet(BinaryContext &BC,
bool IsLeaf);
@ -120,50 +151,71 @@ private:
// 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 SplitInstrsTy
// stores the list of instrumentation instructions as a vector of MCInsts.
// instrumentOneTarget() populates this, runOnFunctions() consumes.
// instrumentOneTarget() populates this, instrumentFunction() consumes.
using SplitWorklistTy =
std::vector<std::pair<BinaryBasicBlock *, BinaryBasicBlock *>>;
using SplitInstrsTy = std::vector<std::vector<MCInst>>;
/// 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.
/// Instrument the branch or call in \p Iter. \p TargetBB should be non-null
/// if this is a local branch and null if it is a call. Return true if the
/// location was instrumented with an explicit counter or false if it just
/// created the description, but no explicit counters were necessary.
bool instrumentOneTarget(SplitWorklistTy &SplitWorklist,
SplitInstrsTy &SplitInstrs,
BinaryBasicBlock::iterator &Iter,
BinaryFunction &FromFunction,
BinaryBasicBlock &FromBB, uint32_t From,
BinaryFunction &ToFunc, BinaryBasicBlock *TargetBB,
uint32_t ToOffset, bool IsLeaf,
FunctionDescription *FuncDesc = nullptr,
uint32_t FromNodeID = 0, uint32_t ToNodeID = 0);
uint32_t ToOffset, bool IsLeaf, bool IsInvoke,
FunctionDescription *FuncDesc,
uint32_t FromNodeID, uint32_t ToNodeID = 0);
void instrumentExitNode(BinaryContext &BC, BinaryBasicBlock &BB,
void instrumentLeafNode(BinaryContext &BC, BinaryBasicBlock &BB,
BinaryBasicBlock::iterator Iter, bool IsLeaf,
FunctionDescription &FuncDesc, uint32_t Node);
void instrumentIndirectTarget(BinaryBasicBlock &BB,
BinaryBasicBlock::iterator &Iter,
BinaryFunction &FromFunction, uint32_t From);
void createAuxiliaryFunctions(BinaryContext &BC);
uint32_t getFDSize() const;
/// Create a non-allocatable ELF section with read-only tables necessary for
/// writing the instrumented data profile during program finish. The runtime
/// library needs to open the program executable file and read this data from
/// disk, this is not loaded by the system.
void emitTablesAsELFNote(BinaryContext &BC);
/// 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;
/// Intra-function control flow
/// Intra-function control flow and direct calls
std::vector<FunctionDescription> FunctionDescriptions;
mutable std::shared_timed_mutex FDMutex;
/// Inter-function control flow
std::vector<CallDescription> CallDescriptions;
/// Inter-function control flow via indirect calls
std::vector<IndCallDescription> IndCallDescriptions;
std::vector<IndCallTargetDescription> IndCallTargetDescriptions;
/// Identify all counters used in runtime while instrumentation is running
std::vector<MCSymbol *> Counters;
/// Our runtime indirect call instrumenter function
MCSymbol *IndCallHandlerFunc;
MCSymbol *IndTailCallHandlerFunc;
/// Our generated initial indirect call handler function that does nothing
/// except calling the indirect call target. The target program starts
/// using this no-op instrumentation function until our runtime library
/// setup runs and installs the correct handler. We need something before
/// our setup runs in case dyld starts running init code for other libs when
/// we did not have time to set up our indirect call counters yet.
BinaryFunction *InitialIndCallHandlerFunction;
BinaryFunction *InitialIndTailCallHandlerFunction;
/// Statistics on counters
uint32_t DirectCallCounters{0};
uint32_t BranchCounters{0};
uint32_t LeafNodeCounters{0};
};
}

View File

@ -29,6 +29,14 @@ namespace opts {
extern cl::OptionCategory BoltOptCategory;
extern cl::opt<bool> NoThreads;
static cl::opt<unsigned> ColdThreshold(
"cold-threshold",
cl::desc("tenths of percents of main entry frequency to use as a "
"threshold when evaluating whether a basic block is cold "
"(0 means it is only considered cold if the block has zero "
"samples). Default: 0 "),
cl::init(0), cl::ZeroOrMore, cl::Hidden, cl::cat(BoltOptCategory));
static cl::opt<bool>
PrintClusters("print-clusters",
cl::desc("print clusters"),
@ -629,6 +637,9 @@ void OptimizeCacheReorderAlgorithm::reorderBasicBlocks(
if (BF.layout_empty())
return;
const uint64_t ColdThreshold =
opts::ColdThreshold * (*BF.layout_begin())->getExecutionCount() / 1000;
// Cluster basic blocks.
CAlgo->clusterBasicBlocks(BF);
std::vector<ClusterAlgorithm::ClusterTy> &Clusters = CAlgo->Clusters;
@ -668,6 +679,13 @@ void OptimizeCacheReorderAlgorithm::reorderBasicBlocks(
for (uint32_t ClusterIndex : ClusterOrder) {
ClusterAlgorithm::ClusterTy &Cluster = Clusters[ClusterIndex];
Order.insert(Order.end(), Cluster.begin(), Cluster.end());
// Force zero execution count on clusters that do not meet the cut off
// specified by --cold-threshold.
if (AvgFreq[ClusterIndex] < static_cast<double>(ColdThreshold)) {
for (auto BBPtr : Cluster) {
BBPtr->setExecutionCount(0);
}
}
}
}

View File

@ -98,9 +98,16 @@ void SplitFunctions::runOnFunctions(BinaryContext &BC) {
ParallelUtilities::runOnEachFunction(
BC, ParallelUtilities::SchedulingPolicy::SP_BB_LINEAR, WorkFun, SkipFunc,
"SplitFunctions");
if (SplitBytesHot + SplitBytesCold > 0) {
outs() << "BOLT-INFO: splitting separates " << SplitBytesHot
<< " hot bytes from " << SplitBytesCold << " cold bytes "
<< format("(%.2lf%% of split functions is hot).\n",
100.0 * SplitBytesHot / (SplitBytesHot + SplitBytesCold));
}
}
void SplitFunctions::splitFunction(BinaryFunction &BF) const {
void SplitFunctions::splitFunction(BinaryFunction &BF) {
if (!BF.size())
return;
@ -222,6 +229,9 @@ void SplitFunctions::splitFunction(BinaryFunction &BF) const {
for (auto &BB : BF) {
BB.setIsCold(false);
}
} else {
SplitBytesHot += HotSize;
SplitBytesCold += ColdSize;
}
}
}

View File

@ -16,6 +16,7 @@
#include "BinaryFunction.h"
#include "Passes/BinaryPasses.h"
#include "llvm/Support/CommandLine.h"
#include <atomic>
namespace llvm {
namespace bolt {
@ -33,7 +34,10 @@ public:
private:
/// Split function body into fragments.
void splitFunction(BinaryFunction &Function) const;
void splitFunction(BinaryFunction &Function);
std::atomic<uint64_t> SplitBytesHot{0ull};
std::atomic<uint64_t> SplitBytesCold{0ull};
public:
explicit SplitFunctions(const cl::opt<bool> &PrintPass)

View File

@ -98,8 +98,8 @@ extern cl::opt<bolt::ReorderFunctions::ReorderType> ReorderFunctions;
extern cl::opt<bool> TimeBuild;
cl::opt<bool>
Instrument("instrument-experimental",
cl::desc("instrument code to generate accurate profile data (experimental)"),
Instrument("instrument",
cl::desc("instrument code to generate accurate profile data"),
cl::ZeroOrMore,
cl::cat(BoltOptCategory));
@ -2925,8 +2925,12 @@ void RewriteInstance::emitAndLink() {
BC->getTextSection()->setAlignment(BC->PageAlign);
emitFunctions(Streamer.get());
if (opts::Instrument)
Instrumenter->emit(*BC, *Streamer.get());
if (opts::Instrument) {
readELFDynamic();
assert(StartFunction && FiniFunction &&
"_start and DT_FINI functions must be set");
Instrumenter->emit(*BC, *Streamer.get(), *StartFunction, *FiniFunction);
}
if (!BC->HasRelocations && opts::UpdateDebugSections)
DebugInfoRewriter->updateDebugLineInfoForNonSimpleFunctions();
@ -3035,8 +3039,10 @@ void RewriteInstance::emitAndLink() {
cantFail(OLT->emitAndFinalize(K));
// Link instrumentation runtime library
if (opts::Instrument)
if (opts::Instrument) {
linkRuntime();
Instrumenter->emitTablesAsELFNote(*BC);
}
// Once the code is emitted, we can rename function sections to actual
// output sections and de-register sections used for emission.
@ -3102,17 +3108,30 @@ void RewriteInstance::linkRuntime() {
<< LibPath << "\n";
exit(1);
}
InstrumentationRuntimeFiniAddress =
cantFail(OLT->findSymbol("__bolt_instr_fini", false).getAddress());
if (!InstrumentationRuntimeFiniAddress) {
errs() << "BOLT-ERROR: instrumentation library does not define "
"__bolt_instr_fini: "
<< LibPath << "\n";
exit(1);
}
InstrumentationRuntimeStartAddress =
cantFail(OLT->findSymbol("__bolt_instr_data_dump", false).getAddress());
cantFail(OLT->findSymbol("__bolt_instr_start", false).getAddress());
if (!InstrumentationRuntimeStartAddress) {
errs() << "BOLT-ERROR: instrumentation library does not define "
"__bolt_instr_data_dump: "
"__bolt_instr_start: "
<< LibPath << "\n";
exit(1);
}
outs() << "BOLT-INFO: output linked against instrumentation runtime "
"library, lib entry point is 0x"
<< Twine::utohexstr(InstrumentationRuntimeStartAddress) << "\n";
<< Twine::utohexstr(InstrumentationRuntimeFiniAddress) << "\n";
outs() << "BOLT-INFO: clear procedure is 0x"
<< Twine::utohexstr(
cantFail(OLT->findSymbol("__bolt_instr_clear_counters", false)
.getAddress()))
<< "\n";
}
void RewriteInstance::updateMetadata() {
@ -4056,7 +4075,8 @@ void RewriteInstance::patchELFSectionHeaderTable(ELFObjectFile<ELFT> *File) {
auto NewEhdr = *Obj->getHeader();
if (BC->HasRelocations) {
NewEhdr.e_entry = getNewFunctionAddress(NewEhdr.e_entry);
NewEhdr.e_entry = opts::Instrument ? InstrumentationRuntimeStartAddress
: getNewFunctionAddress(NewEhdr.e_entry);
assert(NewEhdr.e_entry && "cannot find new address for entry point");
}
NewEhdr.e_phoff = PHDRTableOffset;
@ -4515,7 +4535,6 @@ void RewriteInstance::patchELFDynamic(ELFObjectFile<ELFT> *File) {
"error accessing dynamic table");
const Elf_Dyn *DTE = cantFail(Obj->dynamic_table_end(DynamicPhdr),
"error accessing dynamic table");
bool FiniFound = false;
for (auto *DE = DTB; DE != DTE; ++DE) {
auto NewDE = *DE;
bool ShouldPatch = true;
@ -4532,12 +4551,8 @@ 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 = InstrumentationRuntimeStartAddress;
FiniFound = true;
}
if (opts::Instrument && DE->getTag() == ELF::DT_FINI)
NewDE.d_un.d_ptr = InstrumentationRuntimeFiniAddress;
break;
case ELF::DT_FLAGS:
if (BC->RequiresZNow) {
@ -4558,13 +4573,6 @@ void RewriteInstance::patchELFDynamic(ELFObjectFile<ELFT> *File) {
}
}
if (opts::Instrument && !FiniFound) {
errs() << "BOLT-ERROR: input binary lacks DT_FINI entry in the dynamic "
"section but instrumentation currently relies on patching "
"DT_FINI to write the profile.\n";
exit(1);
}
if (BC->RequiresZNow && !ZNowSet) {
errs() << "BOLT-ERROR: output binary requires immediate relocation "
"processing which depends on DT_FLAGS or DT_FLAGS_1 presence in "
@ -4573,6 +4581,65 @@ void RewriteInstance::patchELFDynamic(ELFObjectFile<ELFT> *File) {
}
}
template <typename ELFT>
void RewriteInstance::readELFDynamic(ELFObjectFile<ELFT> *File) {
auto *Obj = File->getELFFile();
using Elf_Phdr = typename ELFFile<ELFT>::Elf_Phdr;
using Elf_Dyn = typename ELFFile<ELFT>::Elf_Dyn;
if (!opts::Instrument || !BC->HasRelocations)
return;
// Locate DYNAMIC by looking through program headers.
const Elf_Phdr *DynamicPhdr = 0;
for (auto &Phdr : cantFail(Obj->program_headers())) {
if (Phdr.p_type == ELF::PT_DYNAMIC) {
DynamicPhdr = &Phdr;
assert(Phdr.p_memsz == Phdr.p_filesz && "dynamic sizes should match");
break;
}
}
assert(DynamicPhdr && "missing dynamic in ELF binary");
// Go through all dynamic entries and patch functions addresses with
// new ones.
const Elf_Dyn *DTB = cantFail(Obj->dynamic_table_begin(DynamicPhdr),
"error accessing dynamic table");
const Elf_Dyn *DTE = cantFail(Obj->dynamic_table_end(DynamicPhdr),
"error accessing dynamic table");
bool FiniFound = false;
for (auto *DE = DTB; DE != DTE; ++DE) {
if (DE->getTag() != ELF::DT_FINI)
continue;
const auto *Function = BC->getBinaryFunctionAtAddress(DE->getPtr());
if (!Function) {
errs() << "BOLT-ERROR: failed to locate fini function.\n";
exit(1);
}
FiniFunction = Function;
FiniFound = true;
}
if (!FiniFound) {
errs()
<< "BOLT-ERROR: input binary lacks DT_INIT/FINI entry in the dynamic "
"section but instrumentation currently relies on patching "
"DT_FINI to write the profile.\n";
exit(1);
}
// Read start function
auto Ehdr = *Obj->getHeader();
const auto *Function = BC->getBinaryFunctionAtAddress(Ehdr.e_entry);
if (!Function) {
errs() << "BOLT-ERROR: failed to locate _start function.\n";
exit(1);
}
StartFunction = Function;
}
uint64_t RewriteInstance::getNewFunctionAddress(uint64_t OldAddress) {
const auto *Function = BC->getBinaryFunctionAtAddress(OldAddress,
/*Shallow=*/true);

View File

@ -215,6 +215,10 @@ private:
/// Create the regular symbol table and patch dyn symbol tables.
ELF_FUNCTION(patchELFSymTabs);
/// Read dynamic section/segment of ELF to allow us to link a runtime lib
/// later.
ELF_FUNCTION(readELFDynamic);
/// Patch dynamic section/segment of ELF.
ELF_FUNCTION(patchELFDynamic);
@ -367,7 +371,10 @@ private:
uint64_t NewTextSegmentSize{0};
/// Extra linking
uint64_t InstrumentationRuntimeFiniAddress{0};
uint64_t InstrumentationRuntimeStartAddress{0};
const BinaryFunction *StartFunction{nullptr};
const BinaryFunction *FiniFunction{nullptr};
/// Track next available address for new allocatable sections.
uint64_t NextAvailableAddress{0};

View File

@ -2138,6 +2138,23 @@ public:
return true;
}
void convertIndirectCallToLoad(MCInst &Inst, MCPhysReg Reg) const override {
if (Inst.getOpcode() == X86::CALL64m ||
Inst.getOpcode() == X86::TAILJMPm) {
Inst.setOpcode(X86::MOV64rm);
Inst.insert(Inst.begin(), MCOperand::createReg(Reg));
return;
}
if (Inst.getOpcode() == X86::CALL64r ||
Inst.getOpcode() == X86::TAILJMPr) {
Inst.setOpcode(X86::MOV64rr);
Inst.insert(Inst.begin(), MCOperand::createReg(Reg));
return;
}
Inst.dump();
llvm_unreachable("not implemented");
}
bool shortenInstruction(MCInst &Inst) const override {
unsigned OldOpcode = Inst.getOpcode();
unsigned NewOpcode = OldOpcode;
@ -2741,6 +2758,14 @@ public:
return true;
}
void createLoadImmediate(MCInst &Inst, const MCPhysReg Dest,
uint32_t Imm) const {
Inst.setOpcode(X86::MOV64ri32);
Inst.clear();
Inst.addOperand(MCOperand::createReg(Dest));
Inst.addOperand(MCOperand::createImm(Imm));
}
bool createIncMemory(MCInst &Inst, const MCSymbol *Target,
MCContext *Ctx) const override {
@ -2899,6 +2924,19 @@ public:
return true;
}
bool createIndirectCall(MCInst &Inst, const MCSymbol *TargetLocation,
MCContext *Ctx, bool IsTailCall) const override {
Inst.setOpcode(IsTailCall ? X86::TAILJMPm : X86::CALL64m);
Inst.addOperand(MCOperand::createReg(X86::RIP)); // BaseReg
Inst.addOperand(MCOperand::createImm(1)); // ScaleAmt
Inst.addOperand(MCOperand::createReg(X86::NoRegister)); // IndexReg
Inst.addOperand(MCOperand::createExpr( // Displacement
MCSymbolRefExpr::create(TargetLocation, MCSymbolRefExpr::VK_None,
*Ctx)));
Inst.addOperand(MCOperand::createReg(X86::NoRegister)); // AddrSegmentReg
return true;
}
bool createTailCall(MCInst &Inst, const MCSymbol *Target,
MCContext *Ctx) override {
Inst.setOpcode(X86::TAILJMPd);
@ -3122,6 +3160,90 @@ public:
return createPopRegister(Inst, X86::EFLAGS, Size);
}
void createSwap(MCInst &Inst, MCPhysReg Source, MCPhysReg MemBaseReg,
int64_t Disp) const {
Inst.setOpcode(X86::XCHG64rm);
Inst.addOperand(MCOperand::createReg(Source));
Inst.addOperand(MCOperand::createReg(Source));
Inst.addOperand(MCOperand::createReg(MemBaseReg)); // BaseReg
Inst.addOperand(MCOperand::createImm(1)); // ScaleAmt
Inst.addOperand(MCOperand::createReg(X86::NoRegister)); // IndexReg
Inst.addOperand(MCOperand::createImm(Disp)); // Displacement
Inst.addOperand(MCOperand::createReg(X86::NoRegister));//AddrSegmentReg
}
void createIndirectBranch(MCInst &Inst, MCPhysReg MemBaseReg,
int64_t Disp) const {
Inst.setOpcode(X86::JMP64m);
Inst.addOperand(MCOperand::createReg(MemBaseReg)); // BaseReg
Inst.addOperand(MCOperand::createImm(1)); // ScaleAmt
Inst.addOperand(MCOperand::createReg(X86::NoRegister)); // IndexReg
Inst.addOperand(MCOperand::createImm(Disp)); // Displacement
Inst.addOperand(MCOperand::createReg(X86::NoRegister)); // AddrSegmentReg
}
std::vector<MCInst>
createInstrumentedIndirectCall(const MCInst &CallInst, bool TailCall,
MCSymbol *HandlerFuncAddr, int CallSiteID,
MCContext *Ctx) const override {
std::vector<MCInst> Insts(6);
MCPhysReg TempReg = getIntArgRegister(0);
// Code sequence used to enter indirect call instrumentation helper:
// push %rdi
// movq target, %rdi ;; via convertIndirectCallTargetToLoad
// push %rdi
// movq $CallSiteID, %rdi
// push %rdi
// callq/jmp *HandlerFuncAddr
createPushRegister(Insts[0], TempReg, 8);
Insts[1] = CallInst;
convertIndirectCallToLoad(Insts[1], TempReg);
createPushRegister(Insts[2], TempReg, 8);
createLoadImmediate(Insts[3], TempReg, CallSiteID);
createPushRegister(Insts[4], TempReg, 8);
createIndirectCall(Insts[5], HandlerFuncAddr, Ctx,
/*TailCall=*/TailCall);
// Carry over metadata
for (int I = MCPlus::getNumPrimeOperands(CallInst),
E = CallInst.getNumOperands();
I != E; ++I) {
Insts[5].addOperand(CallInst.getOperand(I));
}
return Insts;
}
std::vector<MCInst>
createInstrumentedNoopIndCallHandler() const override {
const MCPhysReg TempReg = getIntArgRegister(0);
// For the default indirect call handler that is supposed to be a no-op,
// we just need to undo the sequence created for every ind call in
// instrumentIndirectTarget(), which can be accomplished minimally with:
// pop %rdi
// add $16, %rsp
// xchg (%rsp), %rdi
// jmp *-8(%rsp)
std::vector<MCInst> Insts(4);
createPopRegister(Insts[0], TempReg, 8);
createStackPointerDecrement(Insts[1], 16, /*NoFlagsClobber=*/false);
createSwap(Insts[2], TempReg, X86::RSP, 0);
createIndirectBranch(Insts[3], X86::RSP, -8);
return Insts;
}
std::vector<MCInst>
createInstrumentedNoopIndTailCallHandler() const override {
const MCPhysReg TempReg = getIntArgRegister(0);
// Same thing as above, but for tail calls
// add $16, %rsp
// pop %rdi
// jmp *-16(%rsp)
std::vector<MCInst> Insts(3);
createStackPointerDecrement(Insts[0], 16, /*NoFlagsClobber=*/false);
createPopRegister(Insts[1], TempReg, 8);
createIndirectBranch(Insts[2], X86::RSP, -16);
return Insts;
}
BlocksVectorTy indirectCallPromotion(
const MCInst &CallInst,
const std::vector<std::pair<MCSymbol *, uint64_t>> &Targets,