diff options
-rw-r--r-- | include/klee/Internal/Module/KModule.h | 55 | ||||
-rw-r--r-- | include/klee/Internal/Support/ModuleUtil.h | 73 | ||||
-rw-r--r-- | include/klee/Interpreter.h | 14 | ||||
-rw-r--r-- | lib/Core/Context.h | 2 | ||||
-rw-r--r-- | lib/Core/Executor.cpp | 90 | ||||
-rw-r--r-- | lib/Core/Executor.h | 6 | ||||
-rw-r--r-- | lib/Core/SpecialFunctionHandler.cpp | 8 | ||||
-rw-r--r-- | lib/Core/SpecialFunctionHandler.h | 5 | ||||
-rw-r--r-- | lib/Core/StatsTracker.cpp | 13 | ||||
-rw-r--r-- | lib/Module/KModule.cpp | 128 | ||||
-rw-r--r-- | lib/Module/ModuleUtil.cpp | 706 | ||||
-rw-r--r-- | lib/Module/Optimize.cpp | 5 | ||||
-rw-r--r-- | runtime/CMakeLists.txt | 6 | ||||
-rw-r--r-- | test/Runtime/FreeStanding/freestanding_only.c | 33 | ||||
-rw-r--r-- | test/Runtime/POSIX/Replay.c | 1 | ||||
-rw-r--r-- | tools/klee/main.cpp | 309 |
16 files changed, 707 insertions, 747 deletions
diff --git a/include/klee/Internal/Module/KModule.h b/include/klee/Internal/Module/KModule.h index e8ded725..478e24d7 100644 --- a/include/klee/Internal/Module/KModule.h +++ b/include/klee/Internal/Module/KModule.h @@ -13,7 +13,10 @@ #include "klee/Config/Version.h" #include "klee/Interpreter.h" +#include "llvm/ADT/ArrayRef.h" + #include <map> +#include <memory> #include <set> #include <vector> @@ -50,12 +53,11 @@ namespace klee { /// "coverable" for statistics and search heuristics. bool trackCoverage; - private: - KFunction(const KFunction&); - KFunction &operator=(const KFunction&); - public: explicit KFunction(llvm::Function*, KModule *); + KFunction(const KFunction &) = delete; + KFunction &operator=(const KFunction &) = delete; + ~KFunction(); unsigned getArgRegister(unsigned index) { return index; } @@ -80,24 +82,24 @@ namespace klee { class KModule { public: - llvm::Module *module; - llvm::DataLayout *targetData; + std::unique_ptr<llvm::Module> module; + std::unique_ptr<llvm::DataLayout> targetData; // Our shadow versions of LLVM structures. - std::vector<KFunction*> functions; + std::vector<std::unique_ptr<KFunction>> functions; std::map<llvm::Function*, KFunction*> functionMap; // Functions which escape (may be called indirectly) // XXX change to KFunction std::set<llvm::Function*> escapingFunctions; - InstructionInfoTable *infos; + std::unique_ptr<InstructionInfoTable> infos; std::vector<llvm::Constant*> constants; - std::map<const llvm::Constant*, KConstant*> constantMap; + std::map<const llvm::Constant *, std::unique_ptr<KConstant>> constantMap; KConstant* getKConstant(const llvm::Constant *c); - Cell *constantTable; + std::unique_ptr<Cell[]> constantTable; // Functions which are part of KLEE runtime std::set<const llvm::Function*> internalFunctions; @@ -107,14 +109,37 @@ namespace klee { void addInternalFunction(const char* functionName); public: - KModule(llvm::Module *_module); - ~KModule(); + KModule() = default; - /// Initialize local data structures. + /// Optimise and prepare module such that KLEE can execute it // + void optimiseAndPrepare(const Interpreter::ModuleOptions &opts, + llvm::ArrayRef<const char *>); + + /// Manifest the generated module (e.g. assembly.ll, output.bc) and + /// prepares KModule + /// + /// @param ih + /// @param forceSourceOutput true if assembly.ll should be created + /// // FIXME: ihandler should not be here - void prepare(const Interpreter::ModuleOptions &opts, - InterpreterHandler *ihandler); + void manifest(InterpreterHandler *ih, bool forceSourceOutput); + + /// Link the provided modules together as one KLEE module. + /// + /// If the entry point is empty, all modules are linked together. + /// If the entry point is not empty, all modules are linked which resolve + /// the dependencies of the module containing entryPoint + /// + /// @param modules list of modules to be linked together + /// @param entryPoint name of the function which acts as the program's entry + /// point + /// @return true if at least one module has been linked in, false if nothing + /// changed + bool link(std::vector<std::unique_ptr<llvm::Module>> &modules, + const std::string &entryPoint); + + void instrument(const Interpreter::ModuleOptions &opts); /// Return an id for the given constant, creating a new one if necessary. unsigned getConstantID(llvm::Constant *c, KInstruction* ki); diff --git a/include/klee/Internal/Support/ModuleUtil.h b/include/klee/Internal/Support/ModuleUtil.h index 4c3243ce..8c819f40 100644 --- a/include/klee/Internal/Support/ModuleUtil.h +++ b/include/klee/Internal/Support/ModuleUtil.h @@ -13,8 +13,6 @@ #include "klee/Config/Version.h" #include "llvm/IR/Module.h" -#include "llvm/IR/Function.h" -#include "llvm/IR/LLVMContext.h" #if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) #include "llvm/IR/CallSite.h" @@ -22,33 +20,58 @@ #include "llvm/Support/CallSite.h" #endif +#include <memory> #include <string> +#include <vector> namespace klee { - /// Load llvm module from a bitcode archive file. - llvm::Module *loadModule(llvm::LLVMContext &ctx, - const std::string &path, - std::string &errorMsg); - - /// Link a module with a specified bitcode archive. - llvm::Module *linkWithLibrary(llvm::Module *module, - const std::string &libraryName); - - /// Return the Function* target of a Call or Invoke instruction, or - /// null if it cannot be determined (should be only for indirect - /// calls, although complicated constant expressions might be - /// another possibility). - /// - /// If `moduleIsFullyLinked` is set to true it will be assumed that the - // module containing the `llvm::CallSite` is fully linked. This assumption - // allows resolution of functions that are marked as overridable. - llvm::Function *getDirectCallTarget(llvm::CallSite, bool moduleIsFullyLinked); - - /// Return true iff the given Function value is used in something - /// other than a direct call (or a constant expression that - /// terminates in a direct call). - bool functionEscapes(const llvm::Function *f); +/// Links all the modules together into one and returns it. +/// +/// All the modules which are used for resolving entities are freed, +/// all the remaining ones are preserved. +/// +/// @param modules List of modules to link together: if resolveOnly true, +/// everything is linked against the first entry. +/// @param entryFunction if set, missing functions of the module containing the +/// entry function will be solved. +/// @return final module or null in this case errorMsg is set +std::unique_ptr<llvm::Module> +linkModules(std::vector<std::unique_ptr<llvm::Module>> &modules, + llvm::StringRef entryFunction, std::string &errorMsg); + +/// Return the Function* target of a Call or Invoke instruction, or +/// null if it cannot be determined (should be only for indirect +/// calls, although complicated constant expressions might be +/// another possibility). +/// +/// If `moduleIsFullyLinked` is set to true it will be assumed that the +/// module containing the `llvm::CallSite` is fully linked. This assumption +/// allows resolution of functions that are marked as overridable. +llvm::Function *getDirectCallTarget(llvm::CallSite, bool moduleIsFullyLinked); + +/// Return true iff the given Function value is used in something +/// other than a direct call (or a constant expression that +/// terminates in a direct call). +bool functionEscapes(const llvm::Function *f); + +/// Loads the file libraryName and reads all possible modules out of it. +/// +/// Different file types are possible: +/// * .bc binary file +/// * .ll IR file +/// * .a archive containing .bc and .ll files +/// +/// @param libraryName library to read +/// @param modules contains extracted modules +/// @param errorMsg contains the error description in case the file could not be +/// loaded +/// @return true if successful otherwise false +bool loadFile(const std::string &libraryName, llvm::LLVMContext &context, + std::vector<std::unique_ptr<llvm::Module>> &modules, + std::string &errorMsg); + +void checkModule(llvm::Module *m); } #endif diff --git a/include/klee/Interpreter.h b/include/klee/Interpreter.h index 40f59ff1..4d8a580c 100644 --- a/include/klee/Interpreter.h +++ b/include/klee/Interpreter.h @@ -9,10 +9,11 @@ #ifndef KLEE_INTERPRETER_H #define KLEE_INTERPRETER_H -#include <vector> -#include <string> #include <map> +#include <memory> #include <set> +#include <string> +#include <vector> struct KTest; @@ -98,12 +99,13 @@ public: const InterpreterOptions &_interpreterOpts, InterpreterHandler *ih); - /// Register the module to be executed. - /// + /// Register the module to be executed. + /// \param modules A list of modules that should form the final + /// module /// \return The final module after it has been optimized, checks /// inserted, and modified for interpretation. - virtual const llvm::Module * - setModule(llvm::Module *module, + virtual llvm::Module * + setModule(std::vector<std::unique_ptr<llvm::Module>> &modules, const ModuleOptions &opts) = 0; // supply a tree stream writer which the interpreter will use diff --git a/lib/Core/Context.h b/lib/Core/Context.h index 6c6eb936..e8b48363 100644 --- a/lib/Core/Context.h +++ b/lib/Core/Context.h @@ -28,7 +28,7 @@ namespace klee { public: Context() {} - + /// initialize - Construct the global Context instance. static void initialize(bool IsLittleEndian, Expr::Width PointerWidth); diff --git a/lib/Core/Executor.cpp b/lib/Core/Executor.cpp index 24fcea88..3d7eb21d 100644 --- a/lib/Core/Executor.cpp +++ b/lib/Core/Executor.cpp @@ -49,22 +49,22 @@ #include "klee/Internal/System/MemoryUsage.h" #include "klee/SolverStats.h" -#include "llvm/IR/Function.h" +#include "llvm/ADT/SmallPtrSet.h" +#include "llvm/ADT/StringExtras.h" #include "llvm/IR/Attributes.h" #include "llvm/IR/BasicBlock.h" #include "llvm/IR/Constants.h" +#include "llvm/IR/DataLayout.h" #include "llvm/IR/Function.h" #include "llvm/IR/Instructions.h" #include "llvm/IR/IntrinsicInst.h" #include "llvm/IR/LLVMContext.h" #include "llvm/IR/Module.h" -#include "llvm/IR/DataLayout.h" #include "llvm/IR/TypeBuilder.h" -#include "llvm/ADT/SmallPtrSet.h" -#include "llvm/ADT/StringExtras.h" #include "llvm/Support/CommandLine.h" #include "llvm/Support/ErrorHandling.h" #include "llvm/Support/FileSystem.h" +#include "llvm/Support/Path.h" #include "llvm/Support/Process.h" #include "llvm/Support/raw_ostream.h" @@ -324,8 +324,8 @@ const char *Executor::TerminateReasonNames[] = { }; Executor::Executor(LLVMContext &ctx, const InterpreterOptions &opts, - InterpreterHandler *ih) - : Interpreter(opts), kmodule(0), interpreterHandler(ih), searcher(0), + InterpreterHandler *ih) + : Interpreter(opts), interpreterHandler(ih), searcher(0), externalDispatcher(new ExternalDispatcher(ctx)), statsTracker(0), pathWriter(0), symPathWriter(0), specialFunctionHandler(0), processTree(0), replayKTest(0), replayPath(0), usingSeeds(0), @@ -393,22 +393,51 @@ Executor::Executor(LLVMContext &ctx, const InterpreterOptions &opts, } } +llvm::Module * +Executor::setModule(std::vector<std::unique_ptr<llvm::Module>> &modules, + const ModuleOptions &opts) { + assert(!kmodule && !modules.empty() && + "can only register one module"); // XXX gross -const Module *Executor::setModule(llvm::Module *module, - const ModuleOptions &opts) { - assert(!kmodule && module && "can only register one module"); // XXX gross - - kmodule = new KModule(module); + kmodule = std::unique_ptr<KModule>(new KModule()); - // Initialize the context. - DataLayout *TD = kmodule->targetData; - Context::initialize(TD->isLittleEndian(), - (Expr::Width) TD->getPointerSizeInBits()); + // Preparing the final module happens in multiple stages + + // Link with KLEE intrinsics library before running any optimizations + SmallString<128> LibPath(opts.LibraryDir); + llvm::sys::path::append(LibPath, "libkleeRuntimeIntrinsic.bca"); + std::string error; + if (!klee::loadFile(LibPath.str(), modules[0]->getContext(), modules, + error)) { + klee_error("Could not load KLEE intrinsic file %s", LibPath.c_str()); + } + // 1.) Link the modules together + while (kmodule->link(modules, opts.EntryPoint)) { + // 2.) Apply different instrumentation + kmodule->instrument(opts); + } + + // 3.) Optimise and prepare for KLEE + + // Create a list of functions that should be preserved if used + std::vector<const char *> preservedFunctions; specialFunctionHandler = new SpecialFunctionHandler(*this); + specialFunctionHandler->prepare(preservedFunctions); + + preservedFunctions.push_back(opts.EntryPoint.c_str()); + + // Preserve the free-standing library calls + preservedFunctions.push_back("memset"); + preservedFunctions.push_back("memcpy"); + preservedFunctions.push_back("memcmp"); + preservedFunctions.push_back("memmove"); + + kmodule->optimiseAndPrepare(opts, preservedFunctions); + + // 4.) Manifest the module + kmodule->manifest(interpreterHandler, StatsTracker::useStatistics()); - specialFunctionHandler->prepare(); - kmodule->prepare(opts, interpreterHandler); specialFunctionHandler->bind(); if (StatsTracker::useStatistics() || userSearcherRequiresMD2U()) { @@ -417,8 +446,13 @@ const Module *Executor::setModule(llvm::Module *module, interpreterHandler->getOutputFilename("assembly.ll"), userSearcherRequiresMD2U()); } - - return module; + + // Initialize the context. + DataLayout *TD = kmodule->targetData.get(); + Context::initialize(TD->isLittleEndian(), + (Expr::Width)TD->getPointerSizeInBits()); + + return kmodule->module.get(); } Executor::~Executor() { @@ -428,7 +462,6 @@ Executor::~Executor() { delete specialFunctionHandler; delete statsTracker; delete solver; - delete kmodule; while(!timers.empty()) { delete timers.back(); timers.pop_back(); @@ -441,7 +474,7 @@ Executor::~Executor() { void Executor::initializeGlobalObject(ExecutionState &state, ObjectState *os, const Constant *c, unsigned offset) { - DataLayout *targetData = kmodule->targetData; + const auto targetData = kmodule->targetData.get(); if (const ConstantVector *cp = dyn_cast<ConstantVector>(c)) { unsigned elementSize = targetData->getTypeStoreSize(cp->getType()->getElementType()); @@ -501,7 +534,7 @@ MemoryObject * Executor::addExternalObject(ExecutionState &state, extern void *__dso_handle __attribute__ ((__weak__)); void Executor::initializeGlobals(ExecutionState &state) { - Module *m = kmodule->module; + Module *m = kmodule->module.get(); if (m->getModuleInlineAsm() != "") klee_warning("executable has module level assembly (ignoring)"); @@ -1400,9 +1433,8 @@ Function* Executor::getTargetFunction(Value *calledVal, ExecutionState &state) { #endif std::string alias = state.getFnAlias(gv->getName()); if (alias != "") { - llvm::Module* currModule = kmodule->module; GlobalValue *old_gv = gv; - gv = currModule->getNamedValue(alias); + gv = kmodule->module->getNamedValue(alias); if (!gv) { klee_error("Function %s(), alias for %s not found!\n", alias.c_str(), old_gv->getName().str().c_str()); @@ -1738,7 +1770,7 @@ void Executor::executeInstruction(ExecutionState &state, KInstruction *ki) { Function *f = getTargetFunction(fp, state); // Skip debug intrinsics, we can't evaluate their metadata arguments. - if (f && isDebugIntrinsic(f, kmodule)) + if (f && isDebugIntrinsic(f, kmodule.get())) break; if (isa<InlineAsm>(fp)) { @@ -2595,14 +2627,14 @@ void Executor::bindInstructionConstants(KInstruction *KI) { } void Executor::bindModuleConstants() { - for (std::vector<KFunction*>::iterator it = kmodule->functions.begin(), - ie = kmodule->functions.end(); it != ie; ++it) { - KFunction *kf = *it; + for (auto &kfp : kmodule->functions) { + KFunction *kf = kfp.get(); for (unsigned i=0; i<kf->numInstructions; ++i) bindInstructionConstants(kf->instructions[i]); } - kmodule->constantTable = new Cell[kmodule->constants.size()]; + kmodule->constantTable = + std::unique_ptr<Cell[]>(new Cell[kmodule->constants.size()]); for (unsigned i=0; i<kmodule->constants.size(); ++i) { Cell &c = kmodule->constantTable[i]; c.value = evalConstant(kmodule->constants[i]); diff --git a/lib/Core/Executor.h b/lib/Core/Executor.h index a0174ab7..6ad5e987 100644 --- a/lib/Core/Executor.h +++ b/lib/Core/Executor.h @@ -120,7 +120,7 @@ private: class TimerInfo; - KModule *kmodule; + std::unique_ptr<KModule> kmodule; InterpreterHandler *interpreterHandler; Searcher *searcher; @@ -491,8 +491,8 @@ public: replayPosition = 0; } - virtual const llvm::Module * - setModule(llvm::Module *module, const ModuleOptions &opts); + llvm::Module *setModule(std::vector<std::unique_ptr<llvm::Module>> &modules, + const ModuleOptions &opts) override; virtual void useSeeds(const std::vector<struct KTest *> *seeds) { usingSeeds = seeds; diff --git a/lib/Core/SpecialFunctionHandler.cpp b/lib/Core/SpecialFunctionHandler.cpp index 11a73da3..62526c94 100644 --- a/lib/Core/SpecialFunctionHandler.cpp +++ b/lib/Core/SpecialFunctionHandler.cpp @@ -169,18 +169,18 @@ int SpecialFunctionHandler::size() { SpecialFunctionHandler::SpecialFunctionHandler(Executor &_executor) : executor(_executor) {} - -void SpecialFunctionHandler::prepare() { +void SpecialFunctionHandler::prepare( + std::vector<const char *> &preservedFunctions) { unsigned N = size(); for (unsigned i=0; i<N; ++i) { HandlerInfo &hi = handlerInfo[i]; Function *f = executor.kmodule->module->getFunction(hi.name); - + // No need to create if the function doesn't exist, since it cannot // be called in that case. - if (f && (!hi.doNotOverride || f->isDeclaration())) { + preservedFunctions.push_back(hi.name); // Make sure NoReturn attribute is set, for optimization and // coverage counting. if (hi.doesNotReturn) diff --git a/lib/Core/SpecialFunctionHandler.h b/lib/Core/SpecialFunctionHandler.h index b11a4974..f99a212f 100644 --- a/lib/Core/SpecialFunctionHandler.h +++ b/lib/Core/SpecialFunctionHandler.h @@ -77,7 +77,10 @@ namespace klee { /// prepared for execution. At the moment this involves deleting /// unused function bodies and marking intrinsics with appropriate /// flags for use in optimizations. - void prepare(); + /// + /// @param preservedFunctions contains all the function names which should + /// be preserved during optimization + void prepare(std::vector<const char *> &preservedFunctions); /// Initialize the internal handler map after the module has been /// prepared for execution. diff --git a/lib/Core/StatsTracker.cpp b/lib/Core/StatsTracker.cpp index f2a989e2..725cfd56 100644 --- a/lib/Core/StatsTracker.cpp +++ b/lib/Core/StatsTracker.cpp @@ -195,7 +195,7 @@ StatsTracker::StatsTracker(Executor &_executor, std::string _objectFilename, "--istats-write-after-instructions cannot be enabled at the same " "time."); - KModule *km = executor.kmodule; + KModule *km = executor.kmodule.get(); if (!sys::path::is_absolute(objectFilename)) { SmallString<128> current(objectFilename); @@ -219,9 +219,8 @@ StatsTracker::StatsTracker(Executor &_executor, std::string _objectFilename, if (OutputIStats) theStatisticManager->useIndexedStats(km->infos->getMaxID()); - for (std::vector<KFunction*>::iterator it = km->functions.begin(), - ie = km->functions.end(); it != ie; ++it) { - KFunction *kf = *it; + for (auto &kfp : km->functions) { + KFunction *kf = kfp.get(); kf->trackCoverage = 1; for (unsigned i=0; i<kf->numInstructions; ++i) { @@ -469,7 +468,7 @@ void StatsTracker::updateStateStatistics(uint64_t addend) { } void StatsTracker::writeIStats() { - Module *m = executor.kmodule->module; + const auto m = executor.kmodule->module.get(); uint64_t istatsMask = 0; llvm::raw_fd_ostream &of = *istatsFile; @@ -666,8 +665,8 @@ uint64_t klee::computeMinDistToUncovered(const KInstruction *ki, } void StatsTracker::computeReachableUncovered() { - KModule *km = executor.kmodule; - Module *m = km->module; + KModule *km = executor.kmodule.get(); + const auto m = km->module.get(); static bool init = true; const InstructionInfoTable &infos = *km->infos; StatisticManager &sm = *theStatisticManager; diff --git a/lib/Module/KModule.cpp b/lib/Module/KModule.cpp index 75e71c0a..d185f687 100644 --- a/lib/Module/KModule.cpp +++ b/lib/Module/KModule.cpp @@ -29,9 +29,12 @@ #include "llvm/IR/DataLayout.h" #if LLVM_VERSION_CODE < LLVM_VERSION(3, 5) +#include "llvm/Analysis/Verifier.h" +#include "llvm/Linker.h" #include "llvm/Support/CallSite.h" #else #include "llvm/IR/CallSite.h" +#include "llvm/Linker/Linker.h" #endif #include "klee/Internal/Module/LLVMPassManager.h" @@ -81,33 +84,10 @@ namespace { cl::desc("Print functions whose address is taken.")); } -KModule::KModule(Module *_module) - : module(_module), - targetData(new DataLayout(module)), - infos(0), - constantTable(0) { -} - -KModule::~KModule() { - delete[] constantTable; - delete infos; - - for (std::vector<KFunction*>::iterator it = functions.begin(), - ie = functions.end(); it != ie; ++it) - delete *it; - - for (std::map<const llvm::Constant*, KConstant*>::iterator it=constantMap.begin(), - itE=constantMap.end(); it!=itE;++it) - delete it->second; - - delete targetData; - delete module; -} - /***/ namespace llvm { -extern void Optimize(Module *, const std::string &EntryPoint); +extern void Optimize(Module *, llvm::ArrayRef<const char *> preservedFunctions); } // what a hack @@ -194,14 +174,31 @@ void KModule::addInternalFunction(const char* functionName){ internalFunctions.insert(internalFunction); } -void KModule::prepare(const Interpreter::ModuleOptions &opts, - InterpreterHandler *ih) { +bool KModule::link(std::vector<std::unique_ptr<llvm::Module>> &modules, + const std::string &entryPoint) { + auto numRemainingModules = modules.size(); + // Add the currently active module to the list of linkables + modules.push_back(std::move(module)); + std::string error; + module = std::unique_ptr<llvm::Module>( + klee::linkModules(modules, entryPoint, error)); + if (!module) + klee_error("Could not link KLEE files %s", error.c_str()); + + targetData = std::unique_ptr<llvm::DataLayout>(new DataLayout(module.get())); + + // Check if we linked anything + return modules.size() != numRemainingModules; +} + +void KModule::instrument(const Interpreter::ModuleOptions &opts) { // Inject checks prior to optimization... we also perform the // invariant transformations that we will end up doing later so that // optimize is seeing what is as close as possible to the final // module. LegacyLLVMPassManagerTy pm; pm.add(new RaiseAsmPass()); + // This pass will scalarize as much code as possible so that the Executor // does not need to handle operands of vector type for most instructions // other than InsertElementInst and ExtractElementInst. @@ -214,21 +211,13 @@ void KModule::prepare(const Interpreter::ModuleOptions &opts, pm.add(new IntrinsicCleanerPass(*targetData)); pm.run(*module); +} +void KModule::optimiseAndPrepare( + const Interpreter::ModuleOptions &opts, + llvm::ArrayRef<const char *> preservedFunctions) { if (opts.Optimize) - Optimize(module, opts.EntryPoint); - - // FIXME: Missing force import for various math functions. - - // FIXME: Find a way that we can test programs without requiring - // this to be linked in, it makes low level debugging much more - // annoying. - - SmallString<128> LibPath(opts.LibraryDir); - llvm::sys::path::append(LibPath, - "kleeRuntimeIntrinsic.bc" - ); - module = linkWithLibrary(module, LibPath.str()); + Optimize(module.get(), preservedFunctions); // Add internal functions which are not used to check if instructions // have been already visited @@ -237,10 +226,9 @@ void KModule::prepare(const Interpreter::ModuleOptions &opts, if (opts.CheckOvershift) addInternalFunction("klee_overshift_check"); - // Needs to happen after linking (since ctors/dtors can be modified) // and optimization (since global optimization can rewrite lists). - injectStaticConstructorsAndDestructors(module); + injectStaticConstructorsAndDestructors(module.get()); // Finally, run the passes that maintain invariants we expect during // interpretation. We run the intrinsic cleaner just in case we @@ -268,74 +256,69 @@ void KModule::prepare(const Interpreter::ModuleOptions &opts, if (!operandTypeCheckPass->checkPassed()) { klee_error("Unexpected instruction operand types detected"); } +} - if (OutputSource) { +void KModule::manifest(InterpreterHandler *ih, bool forceSourceOutput) { + if (OutputSource || forceSourceOutput) { std::unique_ptr<llvm::raw_fd_ostream> os(ih->openOutputFile("assembly.ll")); assert(os && !os->has_error() && "unable to open source output"); *os << *module; } if (OutputModule) { - llvm::raw_fd_ostream *f = ih->openOutputFile("final.bc"); - WriteBitcodeToFile(module, *f); - delete f; + std::unique_ptr<llvm::raw_fd_ostream> f(ih->openOutputFile("final.bc")); + WriteBitcodeToFile(module.get(), *f); } /* Build shadow structures */ - infos = new InstructionInfoTable(module); - - for (Module::iterator it = module->begin(), ie = module->end(); - it != ie; ++it) { - if (it->isDeclaration()) + infos = std::unique_ptr<InstructionInfoTable>( + new InstructionInfoTable(module.get())); + + for (auto &Function : *module) { + if (Function.isDeclaration()) continue; - Function *fn = &*it; - KFunction *kf = new KFunction(fn, this); - + auto kf = std::unique_ptr<KFunction>(new KFunction(&Function, this)); + for (unsigned i=0; i<kf->numInstructions; ++i) { KInstruction *ki = kf->instructions[i]; ki->info = &infos->getInfo(ki->inst); } - functions.push_back(kf); - functionMap.insert(std::make_pair(fn, kf)); + functionMap.insert(std::make_pair(&Function, kf.get())); + functions.push_back(std::move(kf)); } /* Compute various interesting properties */ - for (std::vector<KFunction*>::iterator it = functions.begin(), - ie = functions.end(); it != ie; ++it) { - KFunction *kf = *it; + for (auto &kf : functions) { if (functionEscapes(kf->function)) escapingFunctions.insert(kf->function); } if (DebugPrintEscapingFunctions && !escapingFunctions.empty()) { llvm::errs() << "KLEE: escaping functions: ["; - for (std::set<Function*>::iterator it = escapingFunctions.begin(), - ie = escapingFunctions.end(); it != ie; ++it) { - llvm::errs() << (*it)->getName() << ", "; - } + for (auto &Function : escapingFunctions) + llvm::errs() << Function->getName() << ", "; llvm::errs() << "]\n"; } } KConstant* KModule::getKConstant(const Constant *c) { - std::map<const llvm::Constant*, KConstant*>::iterator it = constantMap.find(c); + auto it = constantMap.find(c); if (it != constantMap.end()) - return it->second; + return it->second.get(); return NULL; } unsigned KModule::getConstantID(Constant *c, KInstruction* ki) { - KConstant *kc = getKConstant(c); - if (kc) + if (KConstant *kc = getKConstant(c)) return kc->id; unsigned id = constants.size(); - kc = new KConstant(c, id, ki); - constantMap.insert(std::make_pair(c, kc)); + auto kc = std::unique_ptr<KConstant>(new KConstant(c, id, ki)); + constantMap.insert(std::make_pair(c, std::move(kc))); constants.push_back(c); return id; } @@ -379,11 +362,10 @@ KFunction::KFunction(llvm::Function *_function, numArgs(function->arg_size()), numInstructions(0), trackCoverage(true) { - for (llvm::Function::iterator bbit = function->begin(), - bbie = function->end(); bbit != bbie; ++bbit) { - BasicBlock *bb = &*bbit; - basicBlockEntry[bb] = numInstructions; - numInstructions += bb->size(); + // Assign unique instruction IDs to each basic block + for (auto &BasicBlock : *function) { + basicBlockEntry[&BasicBlock] = numInstructions; + numInstructions += BasicBlock.size(); } instructions = new KInstruction*[numInstructions]; diff --git a/lib/Module/ModuleUtil.cpp b/lib/Module/ModuleUtil.cpp index b07d3d2f..4f2fdf19 100644 --- a/lib/Module/ModuleUtil.cpp +++ b/lib/Module/ModuleUtil.cpp @@ -14,26 +14,33 @@ #include "klee/Internal/Support/ErrorHandling.h" #include "../Core/SpecialFunctionHandler.h" -#include "llvm/IRReader/IRReader.h" #include "llvm/IR/Function.h" #include "llvm/IR/Instructions.h" #include "llvm/IR/IntrinsicInst.h" #include "llvm/IR/LLVMContext.h" #include "llvm/IR/Module.h" #include "llvm/IR/ValueSymbolTable.h" +#include "llvm/IRReader/IRReader.h" #include "llvm/Object/Archive.h" -#include "llvm/Object/ObjectFile.h" #include "llvm/Object/Error.h" +#include "llvm/Object/ObjectFile.h" +#include "llvm/Support/DataStream.h" #include "llvm/Support/FileSystem.h" #include "llvm/Support/SourceMgr.h" -#include "llvm/Support/DataStream.h" #if LLVM_VERSION_CODE < LLVM_VERSION(3, 5) -#include "llvm/Linker.h" +#include "llvm/Analysis/Verifier.h" #include "llvm/Assembly/AssemblyAnnotationWriter.h" +#include "llvm/Linker.h" #else -#include "llvm/Linker/Linker.h" #include "llvm/IR/AssemblyAnnotationWriter.h" +#include "llvm/IR/Verifier.h" +#include "llvm/Linker/Linker.h" +#endif + +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) +#include "llvm/IR/DiagnosticInfo.h" +#include "llvm/IR/DiagnosticPrinter.h" #endif #include "llvm/Analysis/ValueTracking.h" @@ -42,9 +49,12 @@ #include "llvm/Support/raw_ostream.h" #include "llvm/Support/Path.h" +#include "klee/Internal/Module/LLVMPassManager.h" + +#include <algorithm> +#include <fstream> #include <map> #include <set> -#include <fstream> #include <sstream> #include <string> @@ -71,23 +81,27 @@ GetAllUndefinedSymbols(Module *M, std::set<std::string> &UndefinedSymbols) { std::set<std::string> DefinedSymbols; UndefinedSymbols.clear(); KLEE_DEBUG_WITH_TYPE("klee_linker", - dbgs() << "*** Computing undefined symbols ***\n"); - - for (Module::iterator I = M->begin(), E = M->end(); I != E; ++I) - if (I->hasName()) { - if (I->isDeclaration()) - UndefinedSymbols.insert(I->getName()); - else if (!I->hasLocalLinkage()) { + dbgs() << "*** Computing undefined symbols for " + << M->getModuleIdentifier() << " ***\n"); + + for (auto const &Function : *M) { + if (Function.hasName()) { + if (Function.isDeclaration()) + UndefinedSymbols.insert(Function.getName()); + else if (!Function.hasLocalLinkage()) { #if LLVM_VERSION_CODE < LLVM_VERSION(3, 5) - assert(!I->hasDLLImportLinkage() && "Found dllimported non-external symbol!"); + assert(!Function.hasDLLImportLinkage() && + "Found dllimported non-external symbol!"); #else - assert(!I->hasDLLImportStorageClass() && "Found dllimported non-external symbol!"); + assert(!Function.hasDLLImportStorageClass() && + "Found dllimported non-external symbol!"); #endif - DefinedSymbols.insert(I->getName()); + DefinedSymbols.insert(Function.getName()); } } + } - for (Module::global_iterator I = M->global_begin(), E = M->global_end(); + for (Module::const_global_iterator I = M->global_begin(), E = M->global_end(); I != E; ++I) if (I->hasName()) { if (I->isDeclaration()) @@ -102,7 +116,7 @@ GetAllUndefinedSymbols(Module *M, std::set<std::string> &UndefinedSymbols) { } } - for (Module::alias_iterator I = M->alias_begin(), E = M->alias_end(); + for (Module::const_alias_iterator I = M->alias_begin(), E = M->alias_end(); I != E; ++I) if (I->hasName()) DefinedSymbols.insert(I->getName()); @@ -114,8 +128,7 @@ GetAllUndefinedSymbols(Module *M, std::set<std::string> &UndefinedSymbols) { for (std::set<std::string>::iterator I = UndefinedSymbols.begin(); I != UndefinedSymbols.end(); ++I ) { - if (DefinedSymbols.count(*I)) - { + if (DefinedSymbols.find(*I) != DefinedSymbols.end()) { SymbolsToRemove.push_back(*I); continue; } @@ -135,385 +148,126 @@ GetAllUndefinedSymbols(Module *M, std::set<std::string> &UndefinedSymbols) { dbgs() << "Symbol " << *I << " is undefined.\n"); } - // Remove KLEE intrinsics from set of undefined symbols - for (SpecialFunctionHandler::const_iterator sf = SpecialFunctionHandler::begin(), - se = SpecialFunctionHandler::end(); sf != se; ++sf) - { - if (UndefinedSymbols.find(sf->name) == UndefinedSymbols.end()) - continue; - - SymbolsToRemove.push_back(sf->name); - KLEE_DEBUG_WITH_TYPE("klee_linker", - dbgs() << "KLEE intrinsic " << sf->name << - " has will be removed from undefined symbols"<< "\n"); - } - // Now remove the symbols from undefined set. - for (size_t i = 0, j = SymbolsToRemove.size(); i < j; ++i ) - UndefinedSymbols.erase(SymbolsToRemove[i]); + for (auto const &symbol : SymbolsToRemove) + UndefinedSymbols.erase(symbol); KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "*** Finished computing undefined symbols ***\n"); } - -/*! A helper function for linkBCA() which cleans up - * memory allocated by that function. - */ -static void CleanUpLinkBCA(std::vector<Module*> &archiveModules) -{ - for (std::vector<Module*>::iterator I = archiveModules.begin(), E = archiveModules.end(); - I != E; ++I) - { - delete (*I); - } -} - -/*! A helper function for klee::linkWithLibrary() that links in an archive of bitcode - * modules into a composite bitcode module - * - * \param[in] archive Archive of bitcode modules - * \param[in,out] composite The bitcode module to link against the archive - * \param[out] errorMessage Set to an error message if linking fails - * - * \return True if linking succeeds otherwise false - */ -static bool linkBCA(object::Archive* archive, Module* composite, std::string& errorMessage) -{ - llvm::raw_string_ostream SS(errorMessage); - std::vector<Module*> archiveModules; - - // Is this efficient? Could we use StringRef instead? - std::set<std::string> undefinedSymbols; - GetAllUndefinedSymbols(composite, undefinedSymbols); - - if (undefinedSymbols.size() == 0) - { - // Nothing to do - KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "No undefined symbols. Not linking anything in!\n"); - return true; - } - - KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "Loading modules\n"); - // Load all bitcode files in to memory so we can examine their symbols -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - for (object::Archive::child_iterator AI = archive->child_begin(), - AE = archive->child_end(); AI != AE; ++AI) -#else - for (object::Archive::child_iterator AI = archive->begin_children(), - AE = archive->end_children(); AI != AE; ++AI) -#endif - { - - StringRef memberName; -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - ErrorOr<StringRef> memberNameErr = AI->getName(); - std::error_code ec = memberNameErr.getError(); - if (!ec) { - memberName = memberNameErr.get(); -#else - error_code ec = AI->getName(memberName); - - if ( ec == errc::success ) - { -#endif - KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "Loading archive member " << memberName << "\n"); - } - else - { - errorMessage="Archive member does not have a name!\n"; - return false; - } - -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - ErrorOr<std::unique_ptr<llvm::object::Binary> > child = AI->getAsBinary(); - ec = child.getError(); -#else - OwningPtr<object::Binary> child; - ec = AI->getAsBinary(child); -#endif - if (ec) { - // If we can't open as a binary object file its hopefully a bitcode file +static bool linkTwoModules(llvm::Module *Dest, + std::unique_ptr<llvm::Module> Src, + std::string &errorMsg) { #if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - ErrorOr<MemoryBufferRef> buff = AI->getMemoryBufferRef(); - ec = buff.getError(); -#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - ErrorOr<std::unique_ptr<MemoryBuffer> > buffErr = AI->getMemoryBuffer(); - std::unique_ptr<MemoryBuffer> buff = nullptr; - ec = buffErr.getError(); - if (!ec) - buff = std::move(buffErr.get()); + raw_string_ostream Stream(errorMsg); + DiagnosticPrinterRawOStream DP(Stream); + auto linkResult = Linker::LinkModules( + Dest, Src.release(), [&](const DiagnosticInfo &DI) { DI.print(DP); }); #else - OwningPtr<MemoryBuffer> buff; // Once this is destroyed will Module still be valid?? - ec = AI->getMemoryBuffer(buff); + auto linkResult = Linker::LinkModules(Dest, Src.release(), + Linker::DestroySource, &errorMsg); #endif - if (ec) { - SS << "Failed to get MemoryBuffer: " <<ec.message(); - SS.flush(); - return false; - } - - if (buff) - { - Module *Result = 0; - // FIXME: Maybe load bitcode file lazily? Then if we need to link, materialise the module -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 7) - ErrorOr<std::unique_ptr<Module> > resultErr = -#else - ErrorOr<Module *> resultErr = -#endif - parseBitcodeFile(buff.get(), composite->getContext()); - ec = resultErr.getError(); - if (ec) - errorMessage = ec.message(); - else -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 7) - Result = resultErr->release(); -#else - Result = resultErr.get(); -#endif -#else - Result = ParseBitcodeFile(buff.get(), composite->getContext(), - &errorMessage); -#endif - if(!Result) - { - SS << "Loading module failed : " << errorMessage << "\n"; - SS.flush(); - return false; - } - archiveModules.push_back(Result); - } - else - { - errorMessage="Buffer was NULL!"; - return false; - } - - } - else if (child.get()->isObject()) - { - SS << "Object file " << child.get()->getFileName().data() << - " in archive is not supported"; - SS.flush(); - return false; - } - else - { - SS << "Loading archive child with error "<< ec.message(); - SS.flush(); - return false; - } - - } - - KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "Loaded " << archiveModules.size() << " modules\n"); + return !linkResult; +} - std::set<std::string> previouslyUndefinedSymbols; +std::unique_ptr<llvm::Module> +klee::linkModules(std::vector<std::unique_ptr<llvm::Module>> &modules, + llvm::StringRef entryFunction, std::string &errorMsg) { + assert(!modules.empty() && "modules list should not be empty"); - // Walk through the modules looking for definitions of undefined symbols - // if we find a match we should link that module in. - unsigned int passCounter=0; - do - { - unsigned int modulesLoadedOnPass=0; - previouslyUndefinedSymbols = undefinedSymbols; + if (entryFunction.empty()) { + // If no entry function is provided, link all modules together into one + std::unique_ptr<llvm::Module> composite = std::move(modules.back()); + modules.pop_back(); - for (size_t i = 0, j = archiveModules.size(); i < j; ++i) - { - // skip empty archives - if (archiveModules[i] == 0) + // Just link all modules together + for (auto &module : modules) { + if (linkTwoModules(composite.get(), std::move(module), errorMsg)) continue; - Module * M = archiveModules[i]; - // Look for the undefined symbols in the composite module - for (std::set<std::string>::iterator S = undefinedSymbols.begin(), SE = undefinedSymbols.end(); - S != SE; ++S) - { - - // FIXME: We aren't handling weak symbols here! - // However the algorithm used in LLVM3.2 didn't seem to either - // so maybe it doesn't matter? - - if ( GlobalValue* GV = dyn_cast_or_null<GlobalValue>(M->getValueSymbolTable().lookup(*S))) - { - if (GV->isDeclaration()) continue; // Not a definition - - KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "Found " << GV->getName() << - " in " << M->getModuleIdentifier() << "\n"); - -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - if (Linker::LinkModules(composite, M)) -#else - if (Linker::LinkModules(composite, M, Linker::DestroySource, &errorMessage)) -#endif - { - // Linking failed - SS << "Linking archive module with composite failed:" << errorMessage; - SS.flush(); - CleanUpLinkBCA(archiveModules); - return false; - } - else - { - // Link succeed, now clean up - modulesLoadedOnPass++; - KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "Linking succeeded.\n"); - - delete M; - archiveModules[i] = 0; - - // We need to recompute the undefined symbols in the composite module - // after linking - GetAllUndefinedSymbols(composite, undefinedSymbols); - - break; // Look for symbols in next module - } - } - } + // Linking failed + errorMsg = "Linking archive module with composite failed:" + errorMsg; + return nullptr; } - passCounter++; - KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "Completed " << passCounter << - " linker passes.\n" << modulesLoadedOnPass << - " modules loaded on the last pass\n"); - } while (undefinedSymbols != previouslyUndefinedSymbols); // Iterate until we reach a fixed point - - - // What's left in archiveModules we don't want to link in so free it - CleanUpLinkBCA(archiveModules); - - return true; + // clean up every module as we already linked in every module + modules.clear(); + return composite; + } -} + // Starting from the module containing the entry function, resolve unresolved + // dependencies recursively -Module *klee::linkWithLibrary(Module *module, - const std::string &libraryName) { - KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() << "Linking file " << libraryName << "\n"); - if (!sys::fs::exists(libraryName)) { - klee_error("Link with library %s failed. No such file.", - libraryName.c_str()); + // search for the module containing the entry function + std::unique_ptr<llvm::Module> composite; + for (auto &module : modules) { + if (!module || !module->getNamedValue(entryFunction)) + continue; + if (composite) { + errorMsg = + "Function " + entryFunction.str() + + " defined in different modules (" + module->getModuleIdentifier() + + " already defined in: " + composite->getModuleIdentifier() + ")"; + return nullptr; + } + composite = std::move(module); } -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - ErrorOr<std::unique_ptr<MemoryBuffer> > bufferErr = - MemoryBuffer::getFile(libraryName); - std::error_code ec = bufferErr.getError(); -#else - OwningPtr<MemoryBuffer> Buffer; - error_code ec = MemoryBuffer::getFile(libraryName,Buffer); -#endif - if (ec) { - klee_error("Link with library %s failed: %s", libraryName.c_str(), - ec.message().c_str()); + // fail if not found + if (!composite) { + errorMsg = "'" + entryFunction.str() + "' function not found in module."; + return nullptr; } -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - MemoryBufferRef Buffer = bufferErr.get()->getMemBufferRef(); -#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - MemoryBuffer *Buffer = bufferErr->get(); -#endif - -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - sys::fs::file_magic magic = sys::fs::identify_magic(Buffer.getBuffer()); -#else - sys::fs::file_magic magic = sys::fs::identify_magic(Buffer->getBuffer()); -#endif - - LLVMContext &Context = module->getContext(); - std::string ErrorMessage; - - if (magic == sys::fs::file_magic::bitcode) { - Module *Result = 0; -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 7) - ErrorOr<std::unique_ptr<Module> > ResultErr = -#else - ErrorOr<Module *> ResultErr = -#endif - parseBitcodeFile(Buffer, Context); - if ((ec = ResultErr.getError())) { - ErrorMessage = ec.message(); -#else - Result = ParseBitcodeFile(Buffer.get(), Context, &ErrorMessage); - if (!Result) { -#endif - klee_error("Link with library %s failed: %s", libraryName.c_str(), - ErrorMessage.c_str()); - } + while (true) { + std::set<std::string> undefinedSymbols; + GetAllUndefinedSymbols(composite.get(), undefinedSymbols); -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 7) - Result = ResultErr->release(); -#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - Result = ResultErr.get(); -#endif + // Stop in nothing is undefined + if (undefinedSymbols.empty()) + break; -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - if (Linker::LinkModules(module, Result)) { - ErrorMessage = "linking error"; -#else - if (Linker::LinkModules(module, Result, Linker::DestroySource, &ErrorMessage)) { -#endif - klee_error("Link with library %s failed: %s", libraryName.c_str(), - ErrorMessage.c_str()); - } - -// unique_ptr owns the Module, we don't have to delete it -#if LLVM_VERSION_CODE < LLVM_VERSION(3, 7) - delete Result; -#endif + bool merged = false; + for (auto &module : modules) { + if (!module) + continue; - } else if (magic == sys::fs::file_magic::archive) { -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - ErrorOr<std::unique_ptr<object::Binary> > arch = - object::createBinary(Buffer, &Context); - ec = arch.getError(); -#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - ErrorOr<object::Binary *> arch = - object::createBinary(std::move(bufferErr.get()), &Context); - ec = arch.getError(); -#else - OwningPtr<object::Binary> arch; - ec = object::createBinary(Buffer.take(), arch); -#endif - if (ec) - klee_error("Link with library %s failed: %s", libraryName.c_str(), - ec.message().c_str()); + for (auto symbol : undefinedSymbols) { + GlobalValue *GV = + dyn_cast_or_null<GlobalValue>(module->getNamedValue(symbol)); + if (!GV || GV->isDeclaration()) + continue; -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - if (object::Archive *a = dyn_cast<object::Archive>(arch->get())) { -#else - if (object::Archive *a = dyn_cast<object::Archive>(arch.get())) { -#endif - // Handle in helper - if (!linkBCA(a, module, ErrorMessage)) - klee_error("Link with library %s failed: %s", libraryName.c_str(), - ErrorMessage.c_str()); - } - else { - klee_error("Link with library %s failed: Cast to archive failed", libraryName.c_str()); + // Found symbol, therefore merge in module + KLEE_DEBUG_WITH_TYPE("klee_linker", + dbgs() << "Found " << GV->getName() << " in " + << module->getModuleIdentifier() << "\n"); + if (linkTwoModules(composite.get(), std::move(module), errorMsg)) { + module = nullptr; + merged = true; + break; + } + // Linking failed + errorMsg = "Linking archive module with composite failed:" + errorMsg; + return nullptr; + } } + if (!merged) + break; + } - } else if (magic.is_object()) { -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) - std::unique_ptr<object::Binary> obj; -#else - OwningPtr<object::Binary> obj; -#endif - if (obj.get()->isObject()) { - klee_warning("Link with library: Object file %s in archive %s found. " - "Currently not supported.", - obj.get()->getFileName().data(), libraryName.c_str()); - } - } else { - klee_error("Link with library %s failed: Unrecognized file type.", - libraryName.c_str()); + // Condense the module array + std::vector<std::unique_ptr<llvm::Module>> LeftoverModules; + for (auto &module : modules) { + if (module) + LeftoverModules.emplace_back(std::move(module)); } - return module; + modules.swap(LeftoverModules); + return composite; } Function *klee::getDirectCallTarget(CallSite cs, bool moduleIsFullyLinked) { @@ -582,69 +336,197 @@ bool klee::functionEscapes(const Function *f) { return !valueIsOnlyCalled(f); } -#if LLVM_VERSION_CODE < LLVM_VERSION(3, 5) +bool klee::loadFile(const std::string &fileName, LLVMContext &context, + std::vector<std::unique_ptr<llvm::Module>> &modules, + std::string &errorMsg) { + KLEE_DEBUG_WITH_TYPE("klee_loader", dbgs() + << "Load file " << fileName << "\n"); -Module *klee::loadModule(LLVMContext &ctx, const std::string &path, std::string &errorMsg) { - OwningPtr<MemoryBuffer> bufferPtr; - error_code ec = MemoryBuffer::getFileOrSTDIN(path.c_str(), bufferPtr); - if (ec) { - errorMsg = ec.message(); - return 0; - } +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + ErrorOr<std::unique_ptr<MemoryBuffer>> bufferErr = + MemoryBuffer::getFileOrSTDIN(fileName); + std::error_code ec = bufferErr.getError(); +#else + OwningPtr<MemoryBuffer> Buffer; + error_code ec = MemoryBuffer::getFileOrSTDIN(fileName, Buffer); +#endif + if (ec) { + klee_error("Loading file %s failed: %s", fileName.c_str(), + ec.message().c_str()); + } + +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) + MemoryBufferRef Buffer = bufferErr.get()->getMemBufferRef(); +#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + MemoryBuffer *Buffer = bufferErr->get(); +#endif - Module *module = getLazyBitcodeModule(bufferPtr.get(), ctx, &errorMsg); +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) + sys::fs::file_magic magic = sys::fs::identify_magic(Buffer.getBuffer()); +#else + sys::fs::file_magic magic = sys::fs::identify_magic(Buffer->getBuffer()); +#endif + if (magic == sys::fs::file_magic::bitcode) { + SMDiagnostic Err; +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) + std::unique_ptr<llvm::Module> module(parseIR(Buffer, Err, context)); +#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + std::unique_ptr<llvm::Module> module(ParseIR(Buffer, Err, context)); +#else + std::unique_ptr<llvm::Module> module(ParseIR(Buffer.take(), Err, context)); +#endif if (!module) { - return 0; - } - if (module->MaterializeAllPermanently(&errorMsg)) { - delete module; - return 0; + klee_error("Loading file %s failed: %s", fileName.c_str(), + Err.getMessage().str().c_str()); } + modules.push_back(std::move(module)); + return true; + } - // In the case of success LLVM will take ownership of the module. - // Therefore we need to take ownership away from the `bufferPtr` otherwise the - // allocated memory will be deleted twice. - bufferPtr.take(); + if (magic == sys::fs::file_magic::archive) { +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) + ErrorOr<std::unique_ptr<object::Binary>> archOwner = + object::createBinary(Buffer, &context); + ec = archOwner.getError(); + llvm::object::Binary *arch = archOwner.get().get(); +#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + ErrorOr<object::Binary *> archOwner = + object::createBinary(std::move(bufferErr.get()), &context); + ec = archOwner.getError(); + llvm::object::Binary *arch = archOwner.get(); +#else + OwningPtr<object::Binary> archOwner; + ec = object::createBinary(Buffer.take(), archOwner); + llvm::object::Binary *arch = archOwner.get(); +#endif + if (ec) + klee_error("Loading file %s failed: %s", fileName.c_str(), + ec.message().c_str()); - errorMsg = ""; - return module; -} + if (auto archive = dyn_cast<object::Archive>(arch)) { +// Load all bitcode files into memory +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + for (object::Archive::child_iterator AI = archive->child_begin(), + AE = archive->child_end(); + AI != AE; ++AI) +#else + for (object::Archive::child_iterator AI = archive->begin_children(), + AE = archive->end_children(); + AI != AE; ++AI) +#endif + { + StringRef memberName; +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + ErrorOr<StringRef> memberNameErr = AI->getName(); + std::error_code ec = memberNameErr.getError(); + if (!ec) { + memberName = memberNameErr.get(); #else + error_code ec = AI->getName(memberName); -Module *klee::loadModule(LLVMContext &ctx, const std::string &path, std::string &errorMsg) { - auto buffer = MemoryBuffer::getFileOrSTDIN(path.c_str()); - if (!buffer) { - errorMsg = buffer.getError().message().c_str(); - return nullptr; - } + if (ec == errc::success) { +#endif + KLEE_DEBUG_WITH_TYPE("klee_linker", dbgs() + << "Loading archive member " + << memberName << "\n"); + } else { + errorMsg = "Archive member does not have a name!\n"; + return false; + } +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + ErrorOr<std::unique_ptr<llvm::object::Binary>> child = + AI->getAsBinary(); + ec = child.getError(); +#else + OwningPtr<object::Binary> child; + ec = AI->getAsBinary(child); +#endif + if (ec) { +// If we can't open as a binary object file its hopefully a bitcode file #if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - auto errorOrModule = getLazyBitcodeModule(std::move(buffer.get()), ctx); + ErrorOr<MemoryBufferRef> buff = AI->getMemoryBufferRef(); + ec = buff.getError(); +#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + ErrorOr<std::unique_ptr<MemoryBuffer>> buffErr = + AI->getMemoryBuffer(); + std::unique_ptr<MemoryBuffer> buff = nullptr; + ec = buffErr.getError(); + if (!ec) + buff = std::move(buffErr.get()); #else - auto errorOrModule = getLazyBitcodeModule(buffer->get(), ctx); + OwningPtr<MemoryBuffer> buff; + ec = AI->getMemoryBuffer(buff); #endif + if (ec) { + errorMsg = "Failed to get MemoryBuffer: " + ec.message(); + return false; + } - if (!errorOrModule) { - errorMsg = errorOrModule.getError().message().c_str(); - return nullptr; - } - // The module has taken ownership of the MemoryBuffer so release it - // from the std::unique_ptr - buffer->release(); -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 7) - auto module = errorOrModule->release(); + if (buff) { + // FIXME: Maybe load bitcode file lazily? Then if we need to link, + // materialise + // the module + SMDiagnostic Err; +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) + std::unique_ptr<llvm::Module> module = + parseIR(buff.get(), Err, context); +#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + std::unique_ptr<llvm::Module> module( + ParseIR(buff.get(), Err, context)); #else - auto module = *errorOrModule; + std::unique_ptr<llvm::Module> module( + ParseIR(buff.take(), Err, context)); #endif + if (!module) { + klee_error("Loading file %s failed: %s", fileName.c_str(), + Err.getMessage().str().c_str()); + } + + modules.push_back(std::move(module)); + } else { + errorMsg = "Buffer was NULL!"; + return false; + } - if (auto ec = module->materializeAllPermanently()) { - errorMsg = ec.message(); - return nullptr; + } else if (child.get()->isObject()) { + errorMsg = "Object file " + child.get()->getFileName().str() + + " in archive is not supported"; + return false; + } else { + errorMsg = "Loading archive child with error " + ec.message(); + return false; + } + } + } + return true; } + if (magic.is_object()) { + errorMsg = "Loading file " + fileName + + " Object file as input is currently not supported"; + return false; + } + // This might still be an assembly file. Let's try to parse it. + SMDiagnostic Err; +#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) + std::unique_ptr<llvm::Module> module(parseIR(Buffer, Err, context)); +#elif LLVM_VERSION_CODE >= LLVM_VERSION(3, 5) + std::unique_ptr<llvm::Module> module(ParseIR(Buffer, Err, context)); +#else +std::unique_ptr<llvm::Module> module(ParseIR(Buffer.take(), Err, context)); +#endif + if (!module) { + klee_error("Loading file %s failed: Unrecognized file type.", + fileName.c_str()); + } + modules.push_back(std::move(module)); + return true; +} - errorMsg = ""; - return module; +void klee::checkModule(llvm::Module *m) { + LegacyLLVMPassManagerTy pm; + pm.add(createVerifierPass()); + pm.run(*m); } -#endif diff --git a/lib/Module/Optimize.cpp b/lib/Module/Optimize.cpp index 64e4863f..18c56cc2 100644 --- a/lib/Module/Optimize.cpp +++ b/lib/Module/Optimize.cpp @@ -144,7 +144,7 @@ static void AddStandardCompilePasses(klee::LegacyLLVMPassManagerTy &PM) { /// Optimize - Perform link time optimizations. This will run the scalar /// optimizations, any loaded plugin-optimization modules, and then the /// inter-procedural optimizations if applicable. -void Optimize(Module *M, const std::string &EntryPoint) { +void Optimize(Module *M, llvm::ArrayRef<const char *> preservedFunctions) { // Instantiate the pass manager to organize the passes. klee::LegacyLLVMPassManagerTy Passes; @@ -174,8 +174,7 @@ void Optimize(Module *M, const std::string &EntryPoint) { // for a main function. If main is defined, mark all other functions // internal. if (!DisableInternalize) { - ModulePass *pass = createInternalizePass( - std::vector<const char *>(1, EntryPoint.c_str())); + ModulePass *pass = createInternalizePass(preservedFunctions); addPass(Passes, pass); } diff --git a/runtime/CMakeLists.txt b/runtime/CMakeLists.txt index 53a2f838..61693afc 100644 --- a/runtime/CMakeLists.txt +++ b/runtime/CMakeLists.txt @@ -120,9 +120,9 @@ add_dependencies(clean_all clean_runtime) set(RUNTIME_FILES_TO_INSTALL) list(APPEND RUNTIME_FILES_TO_INSTALL - "${KLEE_RUNTIME_DIRECTORY}/kleeRuntimeIntrinsic.bc" - "${KLEE_RUNTIME_DIRECTORY}/klee-libc.bc" - "${KLEE_RUNTIME_DIRECTORY}/kleeRuntimeFreeStanding.bc" + "${KLEE_RUNTIME_DIRECTORY}/libkleeRuntimeIntrinsic.bca" + "${KLEE_RUNTIME_DIRECTORY}/libklee-libc.bca" + "${KLEE_RUNTIME_DIRECTORY}/libkleeRuntimeFreeStanding.bca" ) if (ENABLE_POSIX_RUNTIME) diff --git a/test/Runtime/FreeStanding/freestanding_only.c b/test/Runtime/FreeStanding/freestanding_only.c new file mode 100644 index 00000000..7bdf6c97 --- /dev/null +++ b/test/Runtime/FreeStanding/freestanding_only.c @@ -0,0 +1,33 @@ +// RUN: %llvmgcc %s -emit-llvm -O0 -g -c -D_FORTIFY_SOURCE=0 -o %t2.bc +// RUN: rm -rf %t.klee-out +// RUN: %klee --output-dir=%t.klee-out %t2.bc 2> %t.log +// RUN: FileCheck %s --input-file=%t.log +// RUN: rm -rf %t.klee-out +// RUN: %klee --optimize --output-dir=%t.klee-out %t2.bc 2> %t.log +// RUN: FileCheck %s --input-file=%t.log + +#include <assert.h> +#include <stdlib.h> +#include <string.h> + +#define LENGTH 5 +int main(int argc, char **argv) { + char *src = (char *)malloc(LENGTH); + char *dst = (char *)malloc(LENGTH); + + memset(src, 42, LENGTH); + // CHECK-NOT: calling external: memset + + memcpy(dst, src, LENGTH); + // CHECK-NOT: calling external: memcpy + + memmove(dst, src, LENGTH); + // CHECK-NOT: calling external: memmove + + assert(memcmp(src, dst, LENGTH) == 0); + // CHECK-NOT: calling external: memcmp + + assert(*src == 42); + assert(*src == *dst); + return 0; +} diff --git a/test/Runtime/POSIX/Replay.c b/test/Runtime/POSIX/Replay.c index ba11f05f..2b166e1f 100644 --- a/test/Runtime/POSIX/Replay.c +++ b/test/Runtime/POSIX/Replay.c @@ -13,6 +13,7 @@ #ifdef KLEE_EXECUTION #define EXIT klee_silent_exit #else +#include <stdlib.h> #define EXIT exit #endif diff --git a/tools/klee/main.cpp b/tools/klee/main.cpp index 1178c9cf..b81131d9 100644 --- a/tools/klee/main.cpp +++ b/tools/klee/main.cpp @@ -120,20 +120,18 @@ namespace { OptExitOnError("exit-on-error", cl::desc("Exit if errors occur")); - - enum LibcType { - NoLibc, KleeLibc, UcLibc - }; - - cl::opt<LibcType> - Libc("libc", - cl::desc("Choose libc version (none by default)."), - cl::values(clEnumValN(NoLibc, "none", "Don't link in a libc"), - clEnumValN(KleeLibc, "klee", "Link in klee libc"), - clEnumValN(UcLibc, "uclibc", "Link in uclibc (adapted for klee)") - KLEE_LLVM_CL_VAL_END), - cl::init(NoLibc)); - + enum class LibcType { FreeStandingLibc, KleeLibc, UcLibc }; + + cl::opt<LibcType> Libc( + "libc", cl::desc("Choose libc version (none by default)."), + cl::values( + clEnumValN( + LibcType::FreeStandingLibc, "none", + "Don't link in a libc (only provide freestanding environment)"), + clEnumValN(LibcType::KleeLibc, "klee", "Link in klee libc"), + clEnumValN(LibcType::UcLibc, "uclibc", + "Link in uclibc (adapted for klee)") KLEE_LLVM_CL_VAL_END), + cl::init(LibcType::FreeStandingLibc)); cl::opt<bool> WithPOSIXRuntime("posix-runtime", @@ -622,7 +620,7 @@ static void parseArguments(int argc, char **argv) { cl::ParseCommandLineOptions(argc, argv, " klee\n"); } -static int initEnv(Module *mainModule) { +static void initEnv(Module *mainModule) { /* nArgcP = alloc oldArgc->getType() @@ -637,13 +635,11 @@ static int initEnv(Module *mainModule) { */ Function *mainFn = mainModule->getFunction(EntryPoint); - if (!mainFn) { + if (!mainFn) klee_error("'%s' function not found in module.", EntryPoint.c_str()); - } - if (mainFn->arg_size() < 2) { + if (mainFn->arg_size() < 2) klee_error("Cannot handle ""--posix-runtime"" when main() has less than two arguments.\n"); - } Instruction *firstInst = &*(mainFn->begin()->begin()); @@ -680,8 +676,6 @@ static int initEnv(Module *mainModule) { new StoreInst(oldArgc, argcPtr, initEnvCall); new StoreInst(oldArgv, argvPtr, initEnvCall); - - return 0; } @@ -828,7 +822,7 @@ static const char *unsafeExternals[] = { "kill", // mmmhmmm }; #define NELEMS(array) (sizeof(array)/sizeof(array[0])) -void externalsAndGlobalsCheck(const Module *m) { +void externalsAndGlobalsCheck(const llvm::Module *m) { std::map<std::string, bool> externals; std::set<std::string> modelled(modelledExternals, modelledExternals+NELEMS(modelledExternals)); @@ -838,14 +832,14 @@ void externalsAndGlobalsCheck(const Module *m) { unsafeExternals+NELEMS(unsafeExternals)); switch (Libc) { - case KleeLibc: + case LibcType::KleeLibc: dontCare.insert(dontCareKlee, dontCareKlee+NELEMS(dontCareKlee)); break; - case UcLibc: + case LibcType::UcLibc: dontCare.insert(dontCareUclibc, dontCareUclibc+NELEMS(dontCareUclibc)); break; - case NoLibc: /* silence compiler warning */ + case LibcType::FreeStandingLibc: /* silence compiler warning */ break; } @@ -976,148 +970,113 @@ static char *format_tdiff(char *buf, long seconds) } #ifndef SUPPORT_KLEE_UCLIBC -static llvm::Module *linkWithUclibc(llvm::Module *mainModule, StringRef libDir) { +static void +linkWithUclibc(StringRef libDir, + std::vector<std::unique_ptr<llvm::Module>> &modules) { klee_error("invalid libc, no uclibc support!\n"); } #else static void replaceOrRenameFunction(llvm::Module *module, const char *old_name, const char *new_name) { - Function *f, *f2; - f = module->getFunction(new_name); - f2 = module->getFunction(old_name); - if (f2) { - if (f) { - f2->replaceAllUsesWith(f); - f2->eraseFromParent(); + Function *new_function, *old_function; + new_function = module->getFunction(new_name); + old_function = module->getFunction(old_name); + if (old_function) { + if (new_function) { + old_function->replaceAllUsesWith(new_function); + old_function->eraseFromParent(); } else { - f2->setName(new_name); - assert(f2->getName() == new_name); + old_function->setName(new_name); + assert(old_function->getName() == new_name); } } } -static llvm::Module *linkWithUclibc(llvm::Module *mainModule, StringRef libDir) { - LLVMContext &ctx = mainModule->getContext(); - // Ensure that klee-uclibc exists - SmallString<128> uclibcBCA(libDir); - llvm::sys::path::append(uclibcBCA, KLEE_UCLIBC_BCA_NAME); - -#if LLVM_VERSION_CODE >= LLVM_VERSION(3, 6) - Twine uclibcBCA_twine(uclibcBCA.c_str()); - if (!llvm::sys::fs::exists(uclibcBCA_twine)) -#else - bool uclibcExists=false; - llvm::sys::fs::exists(uclibcBCA.c_str(), uclibcExists); - if (!uclibcExists) -#endif - klee_error("Cannot find klee-uclibc : %s", uclibcBCA.c_str()); - - Function *f; - // force import of __uClibc_main - mainModule->getOrInsertFunction( - "__uClibc_main", - FunctionType::get(Type::getVoidTy(ctx), std::vector<Type *>(), true)); - - // force various imports - if (WithPOSIXRuntime) { - llvm::Type *i8Ty = Type::getInt8Ty(ctx); - mainModule->getOrInsertFunction("realpath", - PointerType::getUnqual(i8Ty), - PointerType::getUnqual(i8Ty), - PointerType::getUnqual(i8Ty), - NULL); - mainModule->getOrInsertFunction("getutent", - PointerType::getUnqual(i8Ty), - NULL); - mainModule->getOrInsertFunction("__fgetc_unlocked", - Type::getInt32Ty(ctx), - PointerType::getUnqual(i8Ty), - NULL); - mainModule->getOrInsertFunction("__fputc_unlocked", - Type::getInt32Ty(ctx), - Type::getInt32Ty(ctx), - PointerType::getUnqual(i8Ty), - NULL); - } - - f = mainModule->getFunction("__ctype_get_mb_cur_max"); - if (f) f->setName("_stdlib_mb_cur_max"); - - // Strip of asm prefixes for 64 bit versions because they are not - // present in uclibc and we want to make sure stuff will get - // linked. In the off chance that both prefixed and unprefixed - // versions are present in the module, make sure we don't create a - // naming conflict. - for (Module::iterator fi = mainModule->begin(), fe = mainModule->end(); - fi != fe; ++fi) { - Function *f = &*fi; - const std::string &name = f->getName(); - if (name[0]=='\01') { - unsigned size = name.size(); - if (name[size-2]=='6' && name[size-1]=='4') { - std::string unprefixed = name.substr(1); - - // See if the unprefixed version exists. - if (Function *f2 = mainModule->getFunction(unprefixed)) { - f->replaceAllUsesWith(f2); - f->eraseFromParent(); - } else { - f->setName(unprefixed); - } - } - } - } - - mainModule = klee::linkWithLibrary(mainModule, uclibcBCA.c_str()); - assert(mainModule && "unable to link with uclibc"); - - - replaceOrRenameFunction(mainModule, "__libc_open", "open"); - replaceOrRenameFunction(mainModule, "__libc_fcntl", "fcntl"); +static void +createLibCWrapper(std::vector<std::unique_ptr<llvm::Module>> &modules, + llvm::StringRef intendedFunction, + llvm::StringRef libcMainFunction) { // XXX we need to rearchitect so this can also be used with - // programs externally linked with uclibc. + // programs externally linked with libc implementation. - // We now need to swap things so that __uClibc_main is the entry + // We now need to swap things so that libcMainFunction is the entry // point, in such a way that the arguments are passed to - // __uClibc_main correctly. We do this by renaming the user main - // and generating a stub function to call __uClibc_main. There is + // libcMainFunction correctly. We do this by renaming the user main + // and generating a stub function to call intendedFunction. There is // also an implicit cooperation in that runFunctionAsMain sets up - // the environment arguments to what uclibc expects (following + // the environment arguments to what a libc expects (following // argv), since it does not explicitly take an envp argument. - Function *userMainFn = mainModule->getFunction(EntryPoint); + auto &ctx = modules[0]->getContext(); + Function *userMainFn = modules[0]->getFunction(intendedFunction); assert(userMainFn && "unable to get user main"); - Function *uclibcMainFn = mainModule->getFunction("__uClibc_main"); - assert(uclibcMainFn && "unable to get uclibc main"); - userMainFn->setName("__user_main"); + // Rename entry point using a prefix + userMainFn->setName("__user_" + intendedFunction); + + // force import of libcMainFunction + llvm::Function *libcMainFn = nullptr; + for (auto &module : modules) { + if ((libcMainFn = module->getFunction(libcMainFunction))) + break; + } + if (!libcMainFn) + klee_error("Could not add %s wrapper", libcMainFunction.str().c_str()); + + auto inModuleRefernce = libcMainFn->getParent()->getOrInsertFunction( + userMainFn->getName(), userMainFn->getFunctionType()); - const FunctionType *ft = uclibcMainFn->getFunctionType(); - assert(ft->getNumParams() == 7); + const auto ft = libcMainFn->getFunctionType(); + + if (ft->getNumParams() != 7) + klee_error("Imported %s wrapper does not have the correct " + "number of arguments", + libcMainFunction.str().c_str()); std::vector<Type *> fArgs; fArgs.push_back(ft->getParamType(1)); // argc fArgs.push_back(ft->getParamType(2)); // argv - Function *stub = Function::Create(FunctionType::get(Type::getInt32Ty(ctx), fArgs, false), - GlobalVariable::ExternalLinkage, - EntryPoint, - mainModule); + Function *stub = + Function::Create(FunctionType::get(Type::getInt32Ty(ctx), fArgs, false), + GlobalVariable::ExternalLinkage, intendedFunction, + libcMainFn->getParent()); BasicBlock *bb = BasicBlock::Create(ctx, "entry", stub); std::vector<llvm::Value*> args; - args.push_back(llvm::ConstantExpr::getBitCast(userMainFn, - ft->getParamType(0))); + args.push_back( + llvm::ConstantExpr::getBitCast(inModuleRefernce, ft->getParamType(0))); args.push_back(&*(stub->arg_begin())); // argc args.push_back(&*(++stub->arg_begin())); // argv args.push_back(Constant::getNullValue(ft->getParamType(3))); // app_init args.push_back(Constant::getNullValue(ft->getParamType(4))); // app_fini args.push_back(Constant::getNullValue(ft->getParamType(5))); // rtld_fini args.push_back(Constant::getNullValue(ft->getParamType(6))); // stack_end - CallInst::Create(uclibcMainFn, args, "", bb); + CallInst::Create(libcMainFn, args, "", bb); new UnreachableInst(ctx, bb); +} +static void +linkWithUclibc(StringRef libDir, + std::vector<std::unique_ptr<llvm::Module>> &modules) { + LLVMContext &ctx = modules[0]->getContext(); + + size_t newModules = modules.size(); + + // Ensure that klee-uclibc exists + SmallString<128> uclibcBCA(libDir); + std::string errorMsg; + llvm::sys::path::append(uclibcBCA, KLEE_UCLIBC_BCA_NAME); + if (!klee::loadFile(uclibcBCA.c_str(), ctx, modules, errorMsg)) + klee_error("Cannot find klee-uclibc '%s': %s", uclibcBCA.c_str(), + errorMsg.c_str()); + + for (auto i = newModules, j = modules.size(); i < j; ++i) { + replaceOrRenameFunction(modules[i].get(), "__libc_open", "open"); + replaceOrRenameFunction(modules[i].get(), "__libc_fcntl", "fcntl"); + } + + createLibCWrapper(modules, EntryPoint, "__uClibc_main"); klee_message("NOTE: Using klee-uclibc : %s", uclibcBCA.c_str()); - return mainModule; } #endif @@ -1200,16 +1159,27 @@ int main(int argc, char **argv, char **envp) { // Load the bytecode... std::string errorMsg; LLVMContext ctx; - Module *mainModule = klee::loadModule(ctx, InputFile, errorMsg); - if (!mainModule) { + std::vector<std::unique_ptr<llvm::Module>> loadedModules; + if (!klee::loadFile(InputFile, ctx, loadedModules, errorMsg)) { + klee_error("error loading program '%s': %s", InputFile.c_str(), + errorMsg.c_str()); + } + // Load and link the whole files content. The assumption is that this is the + // application under test. + // Nothing gets removed in the first place. + std::unique_ptr<llvm::Module> M(klee::linkModules( + loadedModules, "" /* link all modules together */, errorMsg)); + if (!M) { klee_error("error loading program '%s': %s", InputFile.c_str(), errorMsg.c_str()); } + llvm::Module *mainModule = M.get(); + // Push the module as the first entry + loadedModules.emplace_back(std::move(M)); + if (WithPOSIXRuntime) { - int r = initEnv(mainModule); - if (r != 0) - return r; + initEnv(mainModule); } std::string LibraryDir = KleeHandler::getRunTimeLibraryPath(argv[0]); @@ -1219,44 +1189,45 @@ int main(int argc, char **argv, char **envp) { /*CheckOvershift=*/CheckOvershift); switch (Libc) { - case NoLibc: /* silence compiler warning */ - break; - - case KleeLibc: { + case LibcType::KleeLibc: { // FIXME: Find a reasonable solution for this. SmallString<128> Path(Opts.LibraryDir); - llvm::sys::path::append(Path, "klee-libc.bc"); - mainModule = klee::linkWithLibrary(mainModule, Path.c_str()); - assert(mainModule && "unable to link with klee-libc"); + llvm::sys::path::append(Path, "libklee-libc.bca"); + if (!klee::loadFile(Path.c_str(), mainModule->getContext(), loadedModules, + errorMsg)) + klee_error("error loading klee libc '%s': %s", Path.c_str(), + errorMsg.c_str()); + } + /* Falls through. */ + case LibcType::FreeStandingLibc: { + SmallString<128> Path(Opts.LibraryDir); + llvm::sys::path::append(Path, "libkleeRuntimeFreeStanding.bca"); + if (!klee::loadFile(Path.c_str(), mainModule->getContext(), loadedModules, + errorMsg)) + klee_error("error loading free standing support '%s': %s", Path.c_str(), + errorMsg.c_str()); break; } - - case UcLibc: - mainModule = linkWithUclibc(mainModule, LibraryDir); + case LibcType::UcLibc: + linkWithUclibc(LibraryDir, loadedModules); break; } if (WithPOSIXRuntime) { SmallString<128> Path(Opts.LibraryDir); llvm::sys::path::append(Path, "libkleeRuntimePOSIX.bca"); - klee_message("NOTE: Using model: %s", Path.c_str()); - mainModule = klee::linkWithLibrary(mainModule, Path.c_str()); - assert(mainModule && "unable to link with simple model"); + klee_message("NOTE: Using POSIX model: %s", Path.c_str()); + if (!klee::loadFile(Path.c_str(), mainModule->getContext(), loadedModules, + errorMsg)) + klee_error("error loading POSIX support '%s': %s", Path.c_str(), + errorMsg.c_str()); } - std::vector<std::string>::iterator libs_it; - std::vector<std::string>::iterator libs_ie; - for (libs_it = LinkLibraries.begin(), libs_ie = LinkLibraries.end(); - libs_it != libs_ie; ++libs_it) { - const char * libFilename = libs_it->c_str(); - klee_message("Linking in library: %s.\n", libFilename); - mainModule = klee::linkWithLibrary(mainModule, libFilename); - } - // Get the desired main function. klee_main initializes uClibc - // locale and other data and then calls main. - Function *mainFn = mainModule->getFunction(EntryPoint); - if (!mainFn) { - klee_error("'%s' function not found in module.", EntryPoint.c_str()); + for (const auto &library : LinkLibraries) { + if (!klee::loadFile(library, mainModule->getContext(), loadedModules, + errorMsg)) + klee_error("error loading free standing support '%s': %s", + library.c_str(), errorMsg.c_str()); } // FIXME: Change me to std types. @@ -1309,6 +1280,7 @@ int main(int argc, char **argv, char **envp) { KleeHandler *handler = new KleeHandler(pArgc, pArgv); Interpreter *interpreter = theInterpreter = Interpreter::create(ctx, IOpts, handler); + assert(interpreter); handler->setInterpreter(interpreter); for (int i=0; i<argc; i++) { @@ -1316,8 +1288,15 @@ int main(int argc, char **argv, char **envp) { } handler->getInfoStream() << "PID: " << getpid() << "\n"; - const Module *finalModule = - interpreter->setModule(mainModule, Opts); + // Get the desired main function. klee_main initializes uClibc + // locale and other data and then calls main. + + auto finalModule = interpreter->setModule(loadedModules, Opts); + Function *mainFn = finalModule->getFunction(EntryPoint); + if (!mainFn) { + klee_error("'%s' function not found in module.", EntryPoint.c_str()); + } + externalsAndGlobalsCheck(finalModule); if (ReplayPathFile != "") { |