[BOLT-AArch64] Support relocation mode for bzip2

Summary:
As we deal with incomplete addresses in address-computing
sequences of code in AArch64, we found it is easier to handle them in
relocation mode in the presence of relocations.

Incomplete addresses may mislead BOLT into thinking there are
instructions referring to a basic block when, in fact, this may be the
base address of a data reference. If the relocation is present, we can
easily spot such cases.

This diff contains extensions in relocation mode to understand and
deal with AArch64 relocations. It also adds code to process data inside
functions as marked by AArch64 ABI (symbol table entries named "$d").
In our code, this is called constant islands handling. Last, it extends
bughunter with a "cross" mode, in which the host generates the binaries
and the user test them (uploading to the target), useful when debugging
in AArch64.

(cherry picked from FBD6024570)
This commit is contained in:
Rafael Auler
2017-09-20 10:43:01 -07:00
committed by Maksim Panchenko
parent 76d7740cc9
commit 624b2d984a
8 changed files with 488 additions and 66 deletions

View File

@@ -489,13 +489,71 @@ size_t Relocation::getSizeForType(uint64_t Type) {
case ELF::R_X86_64_TPOFF32:
case ELF::R_X86_64_GOTPCRELX:
case ELF::R_X86_64_REX_GOTPCRELX:
case ELF::R_AARCH64_CALL26:
case ELF::R_AARCH64_ADR_PREL_PG_HI21:
case ELF::R_AARCH64_LDST64_ABS_LO12_NC:
case ELF::R_AARCH64_ADD_ABS_LO12_NC:
case ELF::R_AARCH64_LDST32_ABS_LO12_NC:
case ELF::R_AARCH64_LDST8_ABS_LO12_NC:
case ELF::R_AARCH64_ADR_GOT_PAGE:
case ELF::R_AARCH64_LD64_GOT_LO12_NC:
case ELF::R_AARCH64_JUMP26:
return 4;
case ELF::R_X86_64_PC64:
case ELF::R_X86_64_64:
case ELF::R_AARCH64_ABS64:
return 8;
}
}
uint64_t Relocation::extractValue(uint64_t Type, uint64_t Contents) {
switch (Type) {
default:
llvm_unreachable("unsupported relocation type");
case ELF::R_AARCH64_ABS64:
return Contents;
case ELF::R_AARCH64_JUMP26:
case ELF::R_AARCH64_CALL26:
// Immediate goes in bits 25:0 of B and BL.
Contents &= ~0xfffffffffc000000ULL;
return SignExtend64<28>(Contents << 2);
case ELF::R_AARCH64_ADR_GOT_PAGE:
case ELF::R_AARCH64_ADR_PREL_PG_HI21: {
// Bits 32:12 of Symbol address goes in bits 30:29 + 23:5 of ADRP
// instruction
Contents &= ~0xffffffff9f00001fU;
auto LowBits = (Contents >> 29) & 0x3;
auto HighBits = (Contents >> 5) & 0x7ffff;
Contents = LowBits | (HighBits << 2);
return SignExtend64<32>(Contents << 12);
}
case ELF::R_AARCH64_LD64_GOT_LO12_NC:
case ELF::R_AARCH64_LDST64_ABS_LO12_NC: {
// Immediate goes in bits 21:10 of LD/ST instruction, taken
// from bits 11:3 of Symbol address
Contents &= ~0xffffffffffc003ffU;
return Contents >> (10 - 3);
}
case ELF::R_AARCH64_ADD_ABS_LO12_NC: {
// Immediate goes in bits 21:10 of ADD instruction
Contents &= ~0xffffffffffc003ffU;
return Contents >> (10 - 0);
}
case ELF::R_AARCH64_LDST32_ABS_LO12_NC: {
// Immediate goes in bits 21:10 of ADD instruction, taken
// from bits 11:2 of Symbol address
Contents &= ~0xffffffffffc003ffU;
return Contents >> (10 - 2);
}
case ELF::R_AARCH64_LDST8_ABS_LO12_NC: {
// Immediate goes in bits 21:10 of ADD instruction, taken
// from bits 11:0 of Symbol address
Contents &= ~0xffffffffffc003ffU;
return Contents >> (10 - 0);
}
}
}
bool Relocation::isPCRelative(uint64_t Type) {
switch (Type) {
default:
@@ -505,6 +563,12 @@ bool Relocation::isPCRelative(uint64_t Type) {
case ELF::R_X86_64_32:
case ELF::R_X86_64_32S:
case ELF::R_X86_64_TPOFF32:
case ELF::R_AARCH64_ABS64:
case ELF::R_AARCH64_LDST64_ABS_LO12_NC:
case ELF::R_AARCH64_ADD_ABS_LO12_NC:
case ELF::R_AARCH64_LDST32_ABS_LO12_NC:
case ELF::R_AARCH64_LDST8_ABS_LO12_NC:
case ELF::R_AARCH64_LD64_GOT_LO12_NC:
return false;
case ELF::R_X86_64_PC8:
@@ -514,6 +578,10 @@ bool Relocation::isPCRelative(uint64_t Type) {
case ELF::R_X86_64_GOTTPOFF:
case ELF::R_X86_64_GOTPCRELX:
case ELF::R_X86_64_REX_GOTPCRELX:
case ELF::R_AARCH64_CALL26:
case ELF::R_AARCH64_ADR_PREL_PG_HI21:
case ELF::R_AARCH64_ADR_GOT_PAGE:
case ELF::R_AARCH64_JUMP26:
return true;
}
}

View File

@@ -64,6 +64,11 @@ struct Relocation {
/// Return size of the given relocation \p Type.
static size_t getSizeForType(uint64_t Type);
/// Extract current relocated value from binary contents. This is used for
/// RISC architectures where values are encoded in specific bits depending
/// on the relocation value.
static uint64_t extractValue(uint64_t Type, uint64_t Contents);
/// Return true if relocation type is PC-relative. Return false otherwise.
static bool isPCRelative(uint64_t Type);
@@ -154,6 +159,11 @@ public:
/// final addresses functions will have.
uint64_t LayoutStartAddress{0};
/// Old .text info.
uint64_t OldTextSectionAddress{0};
uint64_t OldTextSectionOffset{0};
uint64_t OldTextSectionSize{0};
/// True if the binary requires immediate relocation processing.
bool RequiresZNow{false};

View File

@@ -795,10 +795,21 @@ void BinaryFunction::disassemble(ArrayRef<uint8_t> FunctionData) {
}
}
if (BC.TheTriple->getArch() == llvm::Triple::aarch64 &&
isInConstantIsland(TargetAddress)) {
TargetSymbol = BC.getOrCreateGlobalSymbol(TargetAddress, "ISLANDat");
IslandSymbols[TargetAddress - getAddress()] = TargetSymbol;
}
// Note that the address does not necessarily have to reside inside
// a section, it could be an absolute address too.
auto Section = BC.getSectionForAddress(TargetAddress);
if (Section && Section->isText()) {
// Assume AArch64's ADRP never references code - it does, but this is fixed
// after reading relocations. ADRP contents now are not really meaningful
// without its supporting relocation.
if (!TargetSymbol && Section && Section->isText() &&
(BC.TheTriple->getArch() != llvm::Triple::aarch64 ||
!BC.MIA->isADRP(Instruction))) {
if (containsAddress(TargetAddress)) {
if (TargetAddress != getAddress()) {
// The address could potentially escape. Mark it as another entry
@@ -829,6 +840,16 @@ void BinaryFunction::disassemble(ArrayRef<uint8_t> FunctionData) {
MCInst Instruction;
const uint64_t AbsoluteInstrAddr = getAddress() + Offset;
// Check for data inside code and ignore it
if (DataOffsets.find(Offset) != DataOffsets.end()) {
auto Iter = CodeOffsets.upper_bound(Offset);
if (Iter != CodeOffsets.end()) {
Size = *Iter - Offset;
continue;
}
break;
}
if (!BC.DisAsm->getInstruction(Instruction,
Size,
FunctionData.slice(Offset),
@@ -985,10 +1006,16 @@ void BinaryFunction::disassemble(ArrayRef<uint8_t> FunctionData) {
// code without re-assembly.
size_t RelSize = (Size < 5) ? 1 : 4;
auto RelOffset = Offset + Size - RelSize;
if (BC.TheTriple->getArch() == llvm::Triple::aarch64) {
RelSize = 0;
RelOffset = Offset;
}
auto RI = MoveRelocations.find(RelOffset);
if (RI == MoveRelocations.end()) {
uint64_t RelType = (RelSize == 1) ? ELF::R_X86_64_PC8
: ELF::R_X86_64_PC32;
if (BC.TheTriple->getArch() == llvm::Triple::aarch64)
RelType = ELF::R_AARCH64_CALL26;
DEBUG(dbgs() << "BOLT-DEBUG: creating relocation for static"
<< " function call to " << TargetSymbol->getName()
<< " at offset 0x"
@@ -2485,6 +2512,9 @@ void BinaryFunction::emitBody(MCStreamer &Streamer, bool EmitColdPart) {
LastIsPrefix = BC.MIA->isPrefix(Instr);
}
}
if (!EmitColdPart)
emitConstantIslands(Streamer);
}
void BinaryFunction::emitBodyRaw(MCStreamer *Streamer) {
@@ -2545,6 +2575,70 @@ void BinaryFunction::emitBodyRaw(MCStreamer *Streamer) {
}
}
void BinaryFunction::emitConstantIslands(MCStreamer &Streamer) {
if (DataOffsets.empty())
return;
Streamer.EmitLabel(getFunctionConstantIslandLabel());
// Raw contents of the function.
StringRef SectionContents;
Section.getContents(SectionContents);
// Raw contents of the function.
StringRef FunctionContents =
SectionContents.substr(getAddress() - Section.getAddress(),
getMaxSize());
if (opts::Verbosity)
outs() << "BOLT-INFO: emitting constant island for function " << *this
<< "\n";
auto IS = IslandSymbols.begin();
// We split the island into smaller blocks and output labels between them.
for (auto DataIter = DataOffsets.begin(); DataIter != DataOffsets.end();
++DataIter) {
uint64_t FunctionOffset = *DataIter;
uint64_t EndOffset = 0ULL;
// Determine size of this data chunk
auto NextData = std::next(DataIter);
auto CodeIter = CodeOffsets.lower_bound(*DataIter);
if (CodeIter == CodeOffsets.end() && NextData == DataOffsets.end()) {
EndOffset = getMaxSize();
} else if (CodeIter == CodeOffsets.end()) {
EndOffset = *NextData;
} else if (NextData == DataOffsets.end()) {
EndOffset = *CodeIter;
} else {
EndOffset = (*CodeIter > *NextData) ? *NextData : *CodeIter;
}
if (FunctionOffset == EndOffset)
continue; // Size is zero, nothing to emit
// Emit labels and data
while (IS != IslandSymbols.end() && IS->first < EndOffset) {
auto NextStop = IS->first;
assert(NextStop <= EndOffset && "internal overflow error");
if (FunctionOffset < NextStop) {
Streamer.EmitBytes(FunctionContents.slice(FunctionOffset, NextStop));
FunctionOffset = NextStop;
}
DEBUG(dbgs() << "BOLT-DEBUG: emitted label " << IS->second->getName()
<< " at offset 0x" << Twine::utohexstr(IS->first) << '\n');
Streamer.EmitLabel(IS->second);
++IS;
}
assert(FunctionOffset <= EndOffset && "overflow error");
if (FunctionOffset < EndOffset) {
Streamer.EmitBytes(FunctionContents.slice(FunctionOffset, EndOffset));
}
}
assert(IS == IslandSymbols.end() && "some symbols were not emitted!");
}
namespace {
#ifndef MAX_PATH
@@ -3334,10 +3428,37 @@ BinaryBasicBlock *BinaryFunction::splitEdge(BinaryBasicBlock *From,
return NewBBPtr;
}
bool BinaryFunction::isDataMarker(const SymbolRef &Symbol,
uint64_t SymbolSize) const {
// For aarch64, the ABI defines mapping symbols so we identify data in the
// code section (see IHI0056B). $d identifies a symbol starting data contents.
if (BC.TheTriple->getArch() == llvm::Triple::aarch64 &&
Symbol.getType() == SymbolRef::ST_Unknown &&
SymbolSize == 0 &&
(!Symbol.getName().getError() && *Symbol.getName() == "$d"))
return true;
return false;
}
bool BinaryFunction::isCodeMarker(const SymbolRef &Symbol,
uint64_t SymbolSize) const {
// For aarch64, the ABI defines mapping symbols so we identify data in the
// code section (see IHI0056B). $x identifies a symbol starting code or the
// end of a data chunk inside code.
if (BC.TheTriple->getArch() == llvm::Triple::aarch64 &&
Symbol.getType() == SymbolRef::ST_Unknown &&
SymbolSize == 0 &&
(!Symbol.getName().getError() && *Symbol.getName() == "$x"))
return true;
return false;
}
bool BinaryFunction::isSymbolValidInScope(const SymbolRef &Symbol,
uint64_t SymbolSize) const {
// Some symbols are tolerated inside function bodies, others are not.
// The real function boundaries may not be known at this point.
if (isDataMarker(Symbol, SymbolSize) || isCodeMarker(Symbol, SymbolSize))
return true;
// It's okay to have a zero-sized symbol in the middle of non-zero-sized
// function.

View File

@@ -415,6 +415,18 @@ private:
/// Temporary holder of offsets that are potentially entry points.
std::unordered_set<uint64_t> EntryOffsets;
/// Temporary holder of offsets that are data markers (used in AArch)
/// It is possible to have data in code sections. To ease the identification
/// of data in code sections, the ABI requires the symbol table to have
/// symbols named "$d" identifying the start of data inside code and "$x"
/// identifying the end of a chunk of data inside code. DataOffsets contain
/// all offsets of $d symbols and CodeOffsets all offsets of $x symbols.
std::set<uint64_t> DataOffsets;
std::set<uint64_t> CodeOffsets;
/// The address offset where we emitted the constant island, that is, the
/// chunk of data in the function code area (AArch only)
int64_t OutputDataOffset;
/// Map labels to corresponding basic blocks.
std::unordered_map<const MCSymbol *, BinaryBasicBlock *> LabelToBB;
@@ -621,6 +633,10 @@ private:
/// Offsets in function that should have PC-relative relocation.
std::set<uint64_t> PCRelativeRelocationOffsets;
/// Offsets in function that are data values in a constant island identified
/// after disassembling
std::map<uint64_t, MCSymbol *> IslandSymbols;
// Blocks are kept sorted in the layout order. If we need to change the
// layout (if BasicBlocksLayout stores a different order than BasicBlocks),
// the terminating instructions need to be modified.
@@ -657,6 +673,8 @@ private:
/// Symbol at the end of the cold part of split function.
mutable MCSymbol *FunctionColdEndLabel{nullptr};
mutable MCSymbol *FunctionConstantIslandLabel{nullptr};
/// Unique number associated with the function.
uint64_t FunctionNumber;
@@ -690,6 +708,16 @@ private:
/// of the function.
MCSymbol *getOrCreateLocalLabel(uint64_t Address, bool CreatePastEnd = false);
/// Register an entry point at a given \p Offset into the function.
void markDataAtOffset(uint64_t Offset) {
DataOffsets.emplace(Offset);
}
/// Register an entry point at a given \p Offset into the function.
void markCodeAtOffset(uint64_t Offset) {
CodeOffsets.emplace(Offset);
}
/// Register an entry point at a given \p Offset into the function.
MCSymbol *addEntryPointAtOffset(uint64_t Offset) {
EntryOffsets.emplace(Offset);
@@ -1097,6 +1125,17 @@ public:
return FunctionColdEndLabel;
}
/// Return a label used to identify where the constant island was emitted
/// (AArch only). This is used to update the symbol table accordingly,
/// emitting data marker symbols as required by the ABI.
MCSymbol *getFunctionConstantIslandLabel() const {
if (!FunctionConstantIslandLabel) {
FunctionConstantIslandLabel =
BC.Ctx->createTempSymbol("func_const_island", true);
}
return FunctionConstantIslandLabel;
}
/// Return true if this is a function representing a PLT entry.
bool isPLTFunction() const {
return PLTSymbol != nullptr;
@@ -1126,6 +1165,13 @@ public:
case ELF::R_X86_64_32:
case ELF::R_X86_64_32S:
case ELF::R_X86_64_64:
case ELF::R_AARCH64_ABS64:
case ELF::R_AARCH64_LDST64_ABS_LO12_NC:
case ELF::R_AARCH64_ADD_ABS_LO12_NC:
case ELF::R_AARCH64_LDST32_ABS_LO12_NC:
case ELF::R_AARCH64_LDST8_ABS_LO12_NC:
case ELF::R_AARCH64_CALL26:
case ELF::R_AARCH64_ADR_PREL_PG_HI21:
Relocations.emplace(Offset,
Relocation{Offset, Symbol, RelType, Addend, Value});
break;
@@ -1614,6 +1660,71 @@ public:
return ColdLSDASymbol;
}
/// True if the symbol is a mapping symbol used in AArch64 to delimit
/// data inside code section.
bool isDataMarker(const SymbolRef &Symbol, uint64_t SymbolSize) const;
bool isCodeMarker(const SymbolRef &Symbol, uint64_t SymbolSize) const;
void setOutputDataAddress(uint64_t Address) {
OutputDataOffset = Address;
}
uint64_t getOutputDataAddress() const {
return OutputDataOffset;
}
/// Detects whether \p Address is inside a data region in this function
/// (constant islands).
bool isInConstantIsland(uint64_t Address) const {
if (Address <= getAddress())
return false;
auto Offset = Address - getAddress();
if (Offset >= getMaxSize())
return false;
auto DataIter = DataOffsets.upper_bound(Offset);
if (DataIter == DataOffsets.begin())
return false;
DataIter = std::prev(DataIter);
auto CodeIter = CodeOffsets.upper_bound(Offset);
if (CodeIter == CodeOffsets.begin())
return true;
return *std::prev(CodeIter) <= *DataIter;
}
uint64_t estimateConstantIslandSize() const {
uint64_t Size = 0;
for (auto DataIter = DataOffsets.begin(); DataIter != DataOffsets.end();
++DataIter) {
auto NextData = std::next(DataIter);
auto CodeIter = CodeOffsets.lower_bound(*DataIter);
if (CodeIter == CodeOffsets.end() &&
NextData == DataOffsets.end()) {
Size += getMaxSize() - *DataIter;
continue;
}
uint64_t NextMarker;
if (CodeIter == CodeOffsets.end())
NextMarker = *NextData;
else if (NextData == DataOffsets.end())
NextMarker = *CodeIter;
else
NextMarker = (*CodeIter > *NextData) ? *NextData : *CodeIter;
Size += NextMarker - *DataIter;
}
return Size;
}
bool hasConstantIsland() const {
return !DataOffsets.empty();
}
/// Return true iff the symbol could be seen inside this function otherwise
/// it is probably another function.
bool isSymbolValidInScope(const SymbolRef &Symbol, uint64_t SymbolSize) const;
@@ -1787,6 +1898,9 @@ public:
/// Emit function as a blob with relocations and labels for relocations.
void emitBodyRaw(MCStreamer *Streamer);
/// Helper for emitBody to write data inside a function (used for AArch64)
void emitConstantIslands(MCStreamer &Streamer);
/// Merge profile data of this function into those of the given
/// function. The functions should have been proven identical with
/// isIdenticalWith.

View File

@@ -17,6 +17,9 @@ using namespace llvm;
namespace opts {
extern cl::opt<bool> Relocs;
extern cl::opt<bool> UseOldText;
extern cl::opt<unsigned> AlignFunctions;
extern cl::opt<unsigned> AlignFunctionsMaxBytes;
}
namespace llvm {
@@ -24,13 +27,13 @@ namespace bolt {
namespace {
constexpr unsigned ColdFragAlign = 16;
constexpr unsigned PageAlign = 0x200000;
std::pair<std::unique_ptr<BinaryBasicBlock>, MCSymbol *>
createNewStub(const BinaryContext &BC, BinaryFunction &Func,
const MCSymbol *TgtSym) {
auto *StubSym = BC.Ctx->createTempSymbol("Stub", true);
auto StubBB = Func.createBasicBlock(0, StubSym);
std::vector<MCInst> Seq;
BC.MIA->createLongJmp(Seq, TgtSym, BC.Ctx.get());
StubBB->addInstructions(Seq.begin(), Seq.end());
@@ -186,13 +189,36 @@ void LongJmpPass::tentativeBBLayout(const BinaryContext &BC,
}
}
void LongJmpPass::tentativeLayout(
const BinaryContext &BC,
std::vector<BinaryFunction *> &SortedFunctions) {
assert(!opts::Relocs && "Pass is incompatible with relocs");
uint64_t DotAddress = BC.LayoutStartAddress;
uint64_t LongJmpPass::tentativeLayoutRelocMode(
const BinaryContext &BC, std::vector<BinaryFunction *> &SortedFunctions,
uint64_t DotAddress) {
// Hot
for (auto Func : SortedFunctions) {
DotAddress = RoundUpToAlignment(DotAddress, BinaryFunction::MinAlign);
auto Pad = OffsetToAlignment(DotAddress, opts::AlignFunctions);
if (Pad <= opts::AlignFunctionsMaxBytes)
DotAddress += Pad;
HotAddresses[Func] = DotAddress;
DEBUG(dbgs() << Func->getPrintName()
<< " tentative: " << Twine::utohexstr(DotAddress) << "\n");
if (!Func->isSimple()) {
DotAddress += Func->getMaxSize();
} else {
if (!Func->isSplit()) {
DotAddress += Func->estimateSize();
} else {
DotAddress += Func->estimateHotSize();
DotAddress += Func->estimateConstantIslandSize();
}
}
}
// Cold
for (auto Func : SortedFunctions) {
DotAddress = RoundUpToAlignment(DotAddress, BinaryFunction::MinAlign);
auto Pad = OffsetToAlignment(DotAddress, opts::AlignFunctions);
if (Pad <= opts::AlignFunctionsMaxBytes)
DotAddress += Pad;
HotAddresses[Func] = Func->getAddress();
DotAddress = RoundUpToAlignment(DotAddress, ColdFragAlign);
ColdAddresses[Func] = DotAddress;
@@ -200,6 +226,42 @@ void LongJmpPass::tentativeLayout(
DotAddress += Func->estimateColdSize();
tentativeBBLayout(BC, *Func);
}
return DotAddress;
}
void LongJmpPass::tentativeLayout(
const BinaryContext &BC,
std::vector<BinaryFunction *> &SortedFunctions) {
uint64_t DotAddress = BC.LayoutStartAddress;
if (!opts::Relocs) {
for (auto Func : SortedFunctions) {
HotAddresses[Func] = Func->getAddress();
DotAddress = RoundUpToAlignment(DotAddress, 16);
ColdAddresses[Func] = DotAddress;
if (Func->isSplit())
DotAddress += Func->estimateColdSize();
tentativeBBLayout(BC, *Func);
}
return;
}
// Relocation mode
auto EstimatedTextSize = tentativeLayoutRelocMode(BC, SortedFunctions, 0);
// Initial padding
if (opts::UseOldText && EstimatedTextSize <= BC.OldTextSectionSize) {
DotAddress = BC.OldTextSectionAddress;
auto Pad = OffsetToAlignment(DotAddress, PageAlign);
if (Pad + EstimatedTextSize <= BC.OldTextSectionSize) {
DotAddress += Pad;
}
} else {
DotAddress = RoundUpToAlignment(BC.LayoutStartAddress, PageAlign);
}
tentativeLayoutRelocMode(BC, SortedFunctions, DotAddress);
}
void LongJmpPass::removeStubRef(const BinaryContext &BC,

View File

@@ -79,6 +79,10 @@ class LongJmpPass : public BinaryFunctionPass {
/// by RewriteInstance::mapFileSections()
void tentativeLayout(const BinaryContext &BC,
std::vector<BinaryFunction *> &SortedFunctions);
uint64_t
tentativeLayoutRelocMode(const BinaryContext &BC,
std::vector<BinaryFunction *> &SortedFunctions,
uint64_t DotAddress);
void tentativeBBLayout(const BinaryContext &BC, const BinaryFunction &Func);
/// Helper to identify whether \p Inst is branching to a stub

View File

@@ -90,14 +90,14 @@ OutputFilename("o",
cl::Required,
cl::cat(BoltOutputCategory));
static cl::opt<unsigned>
cl::opt<unsigned>
AlignFunctions("align-functions",
cl::desc("align functions at a given value (relocation mode)"),
cl::init(64),
cl::ZeroOrMore,
cl::cat(BoltOptCategory));
static cl::opt<unsigned>
cl::opt<unsigned>
AlignFunctionsMaxBytes("align-functions-max-bytes",
cl::desc("maximum number of bytes to use to align functions"),
cl::init(32),
@@ -406,7 +406,6 @@ size_t padFunction(const BinaryFunction &Function) {
} // namespace opts
constexpr const char *RewriteInstance::SectionsToOverwrite[];
constexpr const char *RewriteInstance::SectionsToOverwriteRelocMode[];
const std::string RewriteInstance::OrgSecPrefix = ".bolt.org";
@@ -673,11 +672,12 @@ void RewriteInstance::aggregateData() {
}
void RewriteInstance::discoverStorage() {
// Tell EE that we guarantee we don't need stubs for x86, but not for aarch64
// Stubs are harmful because RuntimeDyld may try to increase the size of
// sections accounting for stubs when we need those sections to match the
// same size seen in the input binary, in case this section is a copy
// of the original one seen in the binary.
EFMM.reset(new ExecutableFileMemoryManager(
/*AllowStubs*/ (BC->TheTriple->getArch() == llvm::Triple::aarch64 &&
opts::Relocs)));
/*AllowStubs*/ false));
auto ELF64LEFile = dyn_cast<ELF64LEObjectFile>(InputFile);
if (!ELF64LEFile) {
@@ -715,9 +715,9 @@ void RewriteInstance::discoverStorage() {
StringRef SectionContents;
Section.getContents(SectionContents);
if (SectionName == ".text") {
OldTextSectionAddress = Section.getAddress();
OldTextSectionSize = Section.getSize();
OldTextSectionOffset =
BC->OldTextSectionAddress = Section.getAddress();
BC->OldTextSectionSize = Section.getSize();
BC->OldTextSectionOffset =
SectionContents.data() - InputFile->getData().data();
}
@@ -869,6 +869,22 @@ void RewriteInstance::run() {
return;
}
// Flip unsupported flags in AArch64 mode
if (BC->TheTriple->getArch() == llvm::Triple::aarch64) {
if (opts::BoostMacroops) {
opts::BoostMacroops = false;
outs() << "BOLT-INFO: disabling -boost-macroops for AArch64\n";
}
if (opts::Relocs && opts::UseOldText) {
opts::UseOldText = false;
outs() << "BOLT-INFO: disabling -use-old-text for AArch64\n";
}
if (!opts::Relocs) {
outs() << "BOLT-WARNING: non-relocation mode for AArch64 is not fully "
"supported\n";
}
}
auto executeRewritePass = [&](const std::set<uint64_t> &NonSimpleFunctions) {
discoverStorage();
readSpecialSections();
@@ -1020,13 +1036,32 @@ void RewriteInstance::discoverFileObjects() {
continue;
}
FileSymRefs[Address] = Symbol;
// In aarch, make $x symbols be replaceable by a more meaningful one
// whenever possible
if (BC->TheTriple->getArch() != llvm::Triple::aarch64 ||
FileSymRefs.find(Address) == FileSymRefs.end()) {
FileSymRefs[Address] = Symbol;
} else {
if (FileSymRefs[Address].getType() == SymbolRef::ST_Unknown &&
*FileSymRefs[Address].getName() == "$x")
FileSymRefs[Address] = Symbol;
else if (Symbol.getType() != SymbolRef::ST_Unknown ||
*NameOrError != "$x")
FileSymRefs[Address] = Symbol;
}
// There's nothing horribly wrong with anonymous symbols, but let's
// ignore them for now.
if (NameOrError->empty())
continue;
// For aarch64, the ABI defines mapping symbols so we identify data in the
// code section (see IHI0056B). $d identifies data contents.
if (BC->TheTriple->getArch() == llvm::Triple::aarch64 &&
Symbol.getType() == SymbolRef::ST_Unknown &&
(*NameOrError == "$d" || *NameOrError == "$x"))
continue;
/// It is possible we are seeing a globalized local. LLVM might treat it as
/// a local if it has a "private global" prefix, e.g. ".L". Thus we have to
/// change the prefix to enforce global scope of the symbol.
@@ -1376,16 +1411,21 @@ void RewriteInstance::adjustFunctionBoundaries() {
// This is potentially another entry point into the function.
auto EntryOffset = NextSymRefI->first - Function.getAddress();
DEBUG(dbgs() << "BOLT-DEBUG: adding entry point to function "
<< Function << " at offset 0x"
<< Twine::utohexstr(EntryOffset) << '\n');
Function.addEntryPointAtOffset(EntryOffset);
// In non-relocation mode there's potentially an external undetectable
// reference to the entry point and hence we cannot move this entry point.
// Optimizing without moving could be difficult.
if (!opts::Relocs)
Function.setSimple(false);
if (Function.isDataMarker(Symbol, SymbolSize)) {
Function.markDataAtOffset(EntryOffset);
} else if (Function.isCodeMarker(Symbol, SymbolSize)) {
Function.markCodeAtOffset(EntryOffset);
} else {
DEBUG(dbgs() << "BOLT-DEBUG: adding entry point to function "
<< Function << " at offset 0x"
<< Twine::utohexstr(EntryOffset) << '\n');
Function.addEntryPointAtOffset(EntryOffset);
// In non-relocation mode there's potentially an external undetectable
// reference to the entry point and hence we cannot move this entry
// point. Optimizing without moving could be difficult.
if (!opts::Relocs)
Function.setSimple(false);
}
++NextSymRefI;
}
@@ -1662,6 +1702,9 @@ void RewriteInstance::readRelocations(const SectionRef &Section) {
DE.getSigned(&RelocationOffset,
Relocation::getSizeForType(Rel.getType())));
if (BC->TheTriple->getArch() == llvm::Triple::aarch64)
ExtractedValue = Relocation::extractValue(Rel.getType(), ExtractedValue);
bool IsPCRelative = Relocation::isPCRelative(Rel.getType());
auto Addend = getRelocationAddend(InputFile, Rel);
uint64_t Address = 0;
@@ -2420,13 +2463,13 @@ void RewriteInstance::mapFileSections(
auto &SI = SMII->second;
uint64_t NewTextSectionOffset = 0;
if (opts::UseOldText && SI.Size <= OldTextSectionSize) {
if (opts::UseOldText && SI.Size <= BC->OldTextSectionSize) {
outs() << "BOLT-INFO: using original .text for new code\n";
// Utilize the original .text for storage.
NewTextSectionStartAddress = OldTextSectionAddress;
NewTextSectionOffset = OldTextSectionOffset;
NewTextSectionStartAddress = BC->OldTextSectionAddress;
NewTextSectionOffset = BC->OldTextSectionOffset;
auto Padding = OffsetToAlignment(NewTextSectionStartAddress, PageAlign);
if (Padding + SI.Size <= OldTextSectionSize) {
if (Padding + SI.Size <= BC->OldTextSectionSize) {
outs() << "BOLT-INFO: using 0x200000 alignment\n";
NewTextSectionStartAddress += Padding;
NewTextSectionOffset += Padding;
@@ -2434,7 +2477,7 @@ void RewriteInstance::mapFileSections(
} else {
if (opts::UseOldText) {
errs() << "BOLT-ERROR: original .text too small to fit the new code. "
<< SI.Size << " bytes needed, have " << OldTextSectionSize
<< SI.Size << " bytes needed, have " << BC->OldTextSectionSize
<< " bytes available.\n";
}
auto Padding = OffsetToAlignment(NewTextSectionStartAddress, PageAlign);
@@ -2621,6 +2664,11 @@ void RewriteInstance::updateOutputValues(const MCAsmLayout &Layout) {
const auto StartOffset = Layout.getSymbolOffset(*Function.getSymbol());
const auto EndOffset =
Layout.getSymbolOffset(*Function.getFunctionEndLabel());
if (Function.hasConstantIsland()) {
const auto DataOffset =
Layout.getSymbolOffset(*Function.getFunctionConstantIslandLabel());
Function.setOutputDataAddress(BaseAddress + DataOffset);
}
Function.setOutputAddress(BaseAddress + StartOffset);
Function.setOutputSize(EndOffset - StartOffset);
if (Function.isSplit()) {
@@ -3314,9 +3362,6 @@ void RewriteInstance::patchELFSectionHeaderTable(ELFObjectFile<ELFT> *File) {
template <typename ELFT>
void RewriteInstance::patchELFSymTabs(ELFObjectFile<ELFT> *File) {
if (!opts::Relocs)
return;
auto *Obj = File->getELFFile();
// Set pointer at the end of the output file, so we can pwrite old symbol
// tables if we need to.
@@ -3350,7 +3395,10 @@ void RewriteInstance::patchELFSymTabs(ELFObjectFile<ELFT> *File) {
NewSymbol.getType() != ELF::STT_SECTION) {
NewSymbol.st_value = Function->getOutputAddress();
NewSymbol.st_size = Function->getOutputSize();
NewSymbol.st_shndx = NewTextSectionIndex;
if (opts::Relocs)
NewSymbol.st_shndx = NewTextSectionIndex;
else
NewSymbol.st_shndx = NewSectionIndex[NewSymbol.st_shndx];
if (!PatchExisting && Function->isSplit()) {
auto NewColdSym = NewSymbol;
SmallVector<char, 256> Buf;
@@ -3362,6 +3410,24 @@ void RewriteInstance::patchELFSymTabs(ELFObjectFile<ELFT> *File) {
Write(0, reinterpret_cast<const char *>(&NewColdSym),
sizeof(NewColdSym));
}
if (!PatchExisting && Function->hasConstantIsland()) {
auto DataMark = Function->getOutputDataAddress();
auto CISize = Function->estimateConstantIslandSize();
auto CodeMark = DataMark + CISize;
auto DataMarkSym = NewSymbol;
DataMarkSym.st_name = AddToStrTab("$d");
DataMarkSym.st_value = DataMark;
DataMarkSym.st_size = 0;
DataMarkSym.setType(ELF::STT_NOTYPE);
DataMarkSym.setBinding(ELF::STB_LOCAL);
auto CodeMarkSym = DataMarkSym;
CodeMarkSym.st_name = AddToStrTab("$x");
CodeMarkSym.st_value = CodeMark;
Write(0, reinterpret_cast<const char *>(&DataMarkSym),
sizeof(DataMarkSym));
Write(0, reinterpret_cast<const char *>(&CodeMarkSym),
sizeof(CodeMarkSym));
}
} else {
if (NewSymbol.st_shndx < ELF::SHN_LORESERVE) {
NewSymbol.st_shndx = NewSectionIndex[NewSymbol.st_shndx];
@@ -3369,7 +3435,7 @@ void RewriteInstance::patchELFSymTabs(ELFObjectFile<ELFT> *File) {
// Detect local syms in the text section that we didn't update
// and were preserved by the linker to support relocations against
// .text (t15274167). Remove then from the symtab.
if (opts::Relocs && NewSymbol.getType() == ELF::STT_NOTYPE &&
if (NewSymbol.getType() == ELF::STT_NOTYPE &&
NewSymbol.getBinding() == ELF::STB_LOCAL &&
NewSymbol.st_size == 0) {
if (auto SecOrErr =
@@ -3804,10 +3870,8 @@ void RewriteInstance::rewriteFile() {
// Finalize memory image of section string table.
finalizeSectionStringTable();
if (opts::Relocs) {
// Update symbol tables.
patchELFSymTabs();
}
// Update symbol tables.
patchELFSymTabs();
// Copy non-allocatable sections once allocatable part is finished.
rewriteNoteSections();
@@ -3927,16 +3991,9 @@ uint64_t RewriteInstance::getFileOffsetForAddress(uint64_t Address) const {
}
bool RewriteInstance::willOverwriteSection(StringRef SectionName) {
if (opts::Relocs) {
for (auto &OverwriteName : SectionsToOverwriteRelocMode) {
if (SectionName == OverwriteName)
return true;
}
} else {
for (auto &OverwriteName : SectionsToOverwrite) {
if (SectionName == OverwriteName)
return true;
}
for (auto &OverwriteName : SectionsToOverwrite) {
if (SectionName == OverwriteName)
return true;
}
auto SMII = EFMM->SectionMapInfo.find(SectionName);

View File

@@ -404,15 +404,6 @@ private:
/// When updating debug info, these are the sections we overwrite.
static constexpr const char *SectionsToOverwrite[] = {
".shstrtab",
".debug_aranges",
".debug_line",
".debug_loc",
".debug_ranges",
".gdb_index",
};
static constexpr const char *SectionsToOverwriteRelocMode[] = {
".shstrtab",
".symtab",
".strtab",
@@ -458,11 +449,6 @@ private:
uint64_t PHDRTableOffset{0};
unsigned Phnum{0};
/// Old .text info.
uint64_t OldTextSectionAddress{0};
uint64_t OldTextSectionOffset{0};
uint64_t OldTextSectionSize{0};
/// New code segment info.
uint64_t NewTextSegmentAddress{0};
uint64_t NewTextSegmentOffset{0};