[Orc] Remove extraneous semicolon that found its way into r250712.
[oota-llvm.git] / include / llvm / ExecutionEngine / Orc / CompileOnDemandLayer.h
index b10a275b12e72031c9cd94da0f7ce287b1cae976..926c1a76b5a541c7c78bca910505fd33d5f01b25 100644 (file)
 #define LLVM_EXECUTIONENGINE_ORC_COMPILEONDEMANDLAYER_H
 
 #include "IndirectionUtils.h"
-#include "LookasideRTDyldMM.h"
+#include "LambdaResolver.h"
+#include "LogicalDylib.h"
 #include "llvm/ADT/STLExtras.h"
 #include "llvm/ExecutionEngine/SectionMemoryManager.h"
+#include "llvm/Transforms/Utils/Cloning.h"
 #include <list>
+#include <set>
+
+#include "llvm/Support/Debug.h"
 
 namespace llvm {
 namespace orc {
 
 /// @brief Compile-on-demand layer.
 ///
-///   Modules added to this layer have their calls indirected, and are then
-/// broken up into a set of single-function modules, each of which is added
-/// to the layer below in a singleton set. The lower layer can be any layer that
-/// accepts IR module sets.
-///
-/// It is expected that this layer will frequently be used on top of a
-/// LazyEmittingLayer. The combination of the two ensures that each function is
-/// compiled only when it is first called.
-template <typename BaseLayerT, typename CompileCallbackMgrT>
+///   When a module is added to this layer a stub is created for each of its
+/// function definitions. The stubs and other global values are immediately
+/// added to the layer below. When a stub is called it triggers the extraction
+/// of the function body from the original module. The extracted body is then
+/// compiled and executed.
+template <typename BaseLayerT,
+          typename CompileCallbackMgrT = JITCompileCallbackManagerBase,
+          typename IndirectStubsMgrT = IndirectStubsManagerBase>
 class CompileOnDemandLayer {
-public:
-  /// @brief Lookup helper that provides compatibility with the classic
-  ///        static-compilation symbol resolution process.
-  ///
-  ///   The CompileOnDemand (COD) layer splits modules up into multiple
-  /// sub-modules, each held in its own llvm::Module instance, in order to
-  /// support lazy compilation. When a module that contains private symbols is
-  /// broken up symbol linkage changes may be required to enable access to
-  /// "private" data that now resides in a different llvm::Module instance. To
-  /// retain expected symbol resolution behavior for clients of the COD layer,
-  /// the CODScopedLookup class uses a two-tiered lookup system to resolve
-  /// symbols. Lookup first scans sibling modules that were split from the same
-  /// original module (logical-module scoped lookup), then scans all other
-  /// modules that have been added to the lookup scope (logical-dylib scoped
-  /// lookup).
-  class CODScopedLookup {
-  private:
-    typedef typename BaseLayerT::ModuleSetHandleT BaseLayerModuleSetHandleT;
-    typedef std::vector<BaseLayerModuleSetHandleT> SiblingHandlesList;
-    typedef std::list<SiblingHandlesList> PseudoDylibModuleSetHandlesList;
+private:
 
+  template <typename MaterializerFtor>
+  class LambdaMaterializer : public ValueMaterializer {
   public:
-    /// @brief Handle for a logical module.
-    typedef typename PseudoDylibModuleSetHandlesList::iterator LMHandle;
-
-    /// @brief Construct a scoped lookup.
-    CODScopedLookup(BaseLayerT &BaseLayer) : BaseLayer(BaseLayer) {}
-
-    /// @brief Start a new context for a single logical module.
-    LMHandle createLogicalModule() {
-      Handles.push_back(SiblingHandlesList());
-      return std::prev(Handles.end());
+    LambdaMaterializer(MaterializerFtor M) : M(std::move(M)) {}
+    Value* materializeValueFor(Value *V) final {
+      return M(V);
     }
-
-    /// @brief Add a concrete Module's handle to the given logical Module's
-    ///        lookup scope.
-    void addToLogicalModule(LMHandle LMH, BaseLayerModuleSetHandleT H) {
-      LMH->push_back(H);
-    }
-
-    /// @brief Remove a logical Module from the CODScopedLookup entirely.
-    void removeLogicalModule(LMHandle LMH) { Handles.erase(LMH); }
-
-    /// @brief Look up a symbol in this context.
-    JITSymbol findSymbol(LMHandle LMH, const std::string &Name) {
-      if (auto Symbol = findSymbolIn(LMH, Name))
-        return Symbol;
-
-      for (auto I = Handles.begin(), E = Handles.end(); I != E; ++I)
-        if (I != LMH)
-          if (auto Symbol = findSymbolIn(I, Name))
-            return Symbol;
-
-      return nullptr;
-    }
-
   private:
-
-    JITSymbol findSymbolIn(LMHandle LMH, const std::string &Name) {
-      for (auto H : *LMH)
-        if (auto Symbol = BaseLayer.findSymbolIn(H, Name, false))
-          return Symbol;
-      return nullptr;
-    }
-
-    BaseLayerT &BaseLayer;
-    PseudoDylibModuleSetHandlesList Handles;
+    MaterializerFtor M;
   };
 
-private:
-  typedef typename BaseLayerT::ModuleSetHandleT BaseLayerModuleSetHandleT;
-  typedef std::vector<BaseLayerModuleSetHandleT> BaseLayerModuleSetHandleListT;
+  template <typename MaterializerFtor>
+  LambdaMaterializer<MaterializerFtor>
+  createLambdaMaterializer(MaterializerFtor M) {
+    return LambdaMaterializer<MaterializerFtor>(std::move(M));
+  }
 
-  struct ModuleSetInfo {
-    // Symbol lookup - just one for the whole module set.
-    std::shared_ptr<CODScopedLookup> Lookup;
+  typedef typename BaseLayerT::ModuleSetHandleT BaseLayerModuleSetHandleT;
 
-    // Logical module handles.
-    std::vector<typename CODScopedLookup::LMHandle> LMHandles;
+  struct LogicalModuleResources {
+    std::shared_ptr<Module> SourceModule;
+    std::set<const Function*> StubsToClone;
+    std::unique_ptr<IndirectStubsMgrT> StubsMgr;
 
-    // List of vectors of module set handles:
-    // One vector per logical module - each vector holds the handles for the
-    // exploded modules for that logical module in the base layer.
-    BaseLayerModuleSetHandleListT BaseLayerModuleSetHandles;
+    JITSymbol findSymbol(StringRef Name, bool ExportedSymbolsOnly) {
+      if (Name.endswith("$stub_ptr") && !ExportedSymbolsOnly) {
+        assert(!ExportedSymbolsOnly && "Stubs are never exported");
+        return StubsMgr->findPointer(Name.drop_back(9));
+      }
+      return StubsMgr->findStub(Name, ExportedSymbolsOnly);
+    }
 
-    ModuleSetInfo(std::shared_ptr<CODScopedLookup> Lookup)
-        : Lookup(std::move(Lookup)) {}
+  };
 
-    void releaseResources(BaseLayerT &BaseLayer) {
-      for (auto LMH : LMHandles)
-        Lookup->removeLogicalModule(LMH);
-      for (auto H : BaseLayerModuleSetHandles)
-        BaseLayer.removeModuleSet(H);
-    }
+  struct LogicalDylibResources {
+    typedef std::function<RuntimeDyld::SymbolInfo(const std::string&)>
+      SymbolResolverFtor;
+    SymbolResolverFtor ExternalSymbolResolver;
   };
 
-  typedef std::list<ModuleSetInfo> ModuleSetInfoListT;
+  typedef LogicalDylib<BaseLayerT, LogicalModuleResources,
+                       LogicalDylibResources> CODLogicalDylib;
+
+  typedef typename CODLogicalDylib::LogicalModuleHandle LogicalModuleHandle;
+  typedef std::list<CODLogicalDylib> LogicalDylibList;
 
 public:
+
   /// @brief Handle to a set of loaded modules.
-  typedef typename ModuleSetInfoListT::iterator ModuleSetHandleT;
+  typedef typename LogicalDylibList::iterator ModuleSetHandleT;
 
-  // @brief Fallback lookup functor.
-  typedef std::function<uint64_t(const std::string &)> LookupFtor;
+  /// @brief Module partitioning functor.
+  typedef std::function<std::set<Function*>(Function&)> PartitioningFtor;
+
+  /// @brief Builder for IndirectStubsManagers.
+  typedef std::function<std::unique_ptr<IndirectStubsMgrT>()>
+    IndirectStubsManagerBuilderT;
 
   /// @brief Construct a compile-on-demand layer instance.
-  CompileOnDemandLayer(BaseLayerT &BaseLayer, LLVMContext &Context)
-    : BaseLayer(BaseLayer),
-      CompileCallbackMgr(BaseLayer, Context, 0, 64) {}
+  CompileOnDemandLayer(BaseLayerT &BaseLayer, PartitioningFtor Partition,
+                       CompileCallbackMgrT &CallbackMgr,
+                       IndirectStubsManagerBuilderT CreateIndirectStubsManager,
+                       bool CloneStubsIntoPartitions = true)
+      : BaseLayer(BaseLayer),  Partition(Partition),
+        CompileCallbackMgr(CallbackMgr),
+        CreateIndirectStubsManager(std::move(CreateIndirectStubsManager)),
+        CloneStubsIntoPartitions(CloneStubsIntoPartitions) {}
 
   /// @brief Add a module to the compile-on-demand layer.
-  template <typename ModuleSetT>
+  template <typename ModuleSetT, typename MemoryManagerPtrT,
+            typename SymbolResolverPtrT>
   ModuleSetHandleT addModuleSet(ModuleSetT Ms,
-                                LookupFtor FallbackLookup = nullptr) {
-
-    // If the user didn't supply a fallback lookup then just use
-    // getSymbolAddress.
-    if (!FallbackLookup)
-      FallbackLookup = [=](const std::string &Name) {
-                         return findSymbol(Name, true).getAddress();
-                       };
-
-    // Create a lookup context and ModuleSetInfo for this module set.
-    // For the purposes of symbol resolution the set Ms will be treated as if
-    // the modules it contained had been linked together as a dylib.
-    auto DylibLookup = std::make_shared<CODScopedLookup>(BaseLayer);
-    ModuleSetHandleT H =
-        ModuleSetInfos.insert(ModuleSetInfos.end(), ModuleSetInfo(DylibLookup));
-    ModuleSetInfo &MSI = ModuleSetInfos.back();
+                                MemoryManagerPtrT MemMgr,
+                                SymbolResolverPtrT Resolver) {
+
+    assert(MemMgr == nullptr &&
+           "User supplied memory managers not supported with COD yet.");
+
+    LogicalDylibs.push_back(CODLogicalDylib(BaseLayer));
+    auto &LDResources = LogicalDylibs.back().getDylibResources();
+
+    LDResources.ExternalSymbolResolver =
+      [Resolver](const std::string &Name) {
+        return Resolver->findSymbol(Name);
+      };
 
     // Process each of the modules in this module set.
     for (auto &M : Ms)
-      partitionAndAdd(*M, MSI, FallbackLookup);
+      addLogicalModule(LogicalDylibs.back(),
+                       std::shared_ptr<Module>(std::move(M)));
 
-    return H;
+    return std::prev(LogicalDylibs.end());
   }
 
   /// @brief Remove the module represented by the given handle.
@@ -178,8 +141,7 @@ public:
   ///   This will remove all modules in the layers below that were derived from
   /// the module represented by H.
   void removeModuleSet(ModuleSetHandleT H) {
-    H->releaseResources(BaseLayer);
-    ModuleSetInfos.erase(H);
+    LogicalDylibs.erase(H);
   }
 
   /// @brief Search for the given named symbol.
@@ -187,166 +149,288 @@ public:
   /// @param ExportedSymbolsOnly If true, search only for exported symbols.
   /// @return A handle for the given named symbol, if it exists.
   JITSymbol findSymbol(StringRef Name, bool ExportedSymbolsOnly) {
-    return BaseLayer.findSymbol(Name, ExportedSymbolsOnly);
+    for (auto LDI = LogicalDylibs.begin(), LDE = LogicalDylibs.end();
+         LDI != LDE; ++LDI)
+      if (auto Symbol = findSymbolIn(LDI, Name, ExportedSymbolsOnly))
+        return Symbol;
+    return nullptr;
   }
 
   /// @brief Get the address of a symbol provided by this layer, or some layer
   ///        below this one.
   JITSymbol findSymbolIn(ModuleSetHandleT H, const std::string &Name,
                          bool ExportedSymbolsOnly) {
-    BaseLayerModuleSetHandleListT &BaseLayerHandles = H->second;
-    for (auto &BH : BaseLayerHandles) {
-      if (auto Symbol = BaseLayer.findSymbolIn(BH, Name, ExportedSymbolsOnly))
-        return Symbol;
-    }
-    return nullptr;
+    return H->findSymbol(Name, ExportedSymbolsOnly);
   }
 
 private:
 
-  void partitionAndAdd(Module &M, ModuleSetInfo &MSI,
-                       LookupFtor FallbackLookup) {
-    const char *AddrSuffix = "$orc_addr";
-    const char *BodySuffix = "$orc_body";
-
-    // We're going to break M up into a bunch of sub-modules, but we want
-    // internal linkage symbols to still resolve sensibly. CODScopedLookup
-    // provides the "logical module" concept to make this work, so create a
-    // new logical module for M.
-    auto DylibLookup = MSI.Lookup;
-    auto LogicalModule = DylibLookup->createLogicalModule();
-    MSI.LMHandles.push_back(LogicalModule);
-
-    // Partition M into a "globals and stubs" module, a "common symbols" module,
-    // and a list of single-function modules.
-    auto PartitionedModule = fullyPartition(M);
-    auto StubsModule = std::move(PartitionedModule.GlobalVars);
-    auto CommonsModule = std::move(PartitionedModule.Commons);
-    auto FunctionModules = std::move(PartitionedModule.Functions);
-
-    // Emit the commons stright away.
-    auto CommonHandle = addModule(std::move(CommonsModule), MSI, LogicalModule,
-                                  FallbackLookup);
-    BaseLayer.emitAndFinalize(CommonHandle);
-
-    // Map of definition names to callback-info data structures. We'll use
-    // this to build the compile actions for the stubs below.
-    typedef std::map<std::string,
-                     typename CompileCallbackMgrT::CompileCallbackInfo>
-      StubInfoMap;
-    StubInfoMap StubInfos;
-
-    // Now we need to take each of the extracted Modules and add them to
-    // base layer. Each Module will be added individually to make sure they
-    // can be compiled separately, and each will get its own lookaside
-    // memory manager that will resolve within this logical module first.
-    for (auto &SubM : FunctionModules) {
-
-      // Keep track of the stubs we create for this module so that we can set
-      // their compile actions.
-      std::vector<typename StubInfoMap::iterator> NewStubInfos;
-
-      // Search for function definitions and insert stubs into the stubs
-      // module.
-      for (auto &F : *SubM) {
-        if (F.isDeclaration())
-          continue;
+  void addLogicalModule(CODLogicalDylib &LD, std::shared_ptr<Module> SrcM) {
 
-        std::string Name = F.getName();
-        Function *Proto = StubsModule->getFunction(Name);
-        assert(Proto && "Failed to clone function decl into stubs module.");
-        auto CallbackInfo =
-          CompileCallbackMgr.getCompileCallback(*Proto->getFunctionType());
-        GlobalVariable *FunctionBodyPointer =
-          createImplPointer(*Proto, Name + AddrSuffix,
-                            CallbackInfo.getAddress());
-        makeStub(*Proto, *FunctionBodyPointer);
-
-        F.setName(Name + BodySuffix);
-        F.setVisibility(GlobalValue::HiddenVisibility);
-
-        auto KV = std::make_pair(std::move(Name), std::move(CallbackInfo));
-        NewStubInfos.push_back(StubInfos.insert(StubInfos.begin(), KV));
-      }
+    // Bump the linkage and rename any anonymous/privote members in SrcM to
+    // ensure that everything will resolve properly after we partition SrcM.
+    makeAllSymbolsExternallyAccessible(*SrcM);
+
+    // Create a logical module handle for SrcM within the logical dylib.
+    auto LMH = LD.createLogicalModule();
+    auto &LMResources =  LD.getLogicalModuleResources(LMH);
 
-      auto H = addModule(std::move(SubM), MSI, LogicalModule, FallbackLookup);
+    LMResources.SourceModule = SrcM;
 
-      // Set the compile actions for this module:
-      for (auto &KVPair : NewStubInfos) {
-        std::string BodyName = Mangle(KVPair->first + BodySuffix,
-                                      M.getDataLayout());
-        auto &CCInfo = KVPair->second;
+    // Create the GlobalValues module.
+    const DataLayout &DL = SrcM->getDataLayout();
+    auto GVsM = llvm::make_unique<Module>((SrcM->getName() + ".globals").str(),
+                                          SrcM->getContext());
+    GVsM->setDataLayout(DL);
+
+    // Create function stubs.
+    ValueToValueMapTy VMap;
+    {
+      typename IndirectStubsMgrT::StubInitsMap StubInits;
+      for (auto &F : *SrcM) {
+        // Skip declarations.
+        if (F.isDeclaration())
+          continue;
+
+        // Record all functions defined by this module.
+        if (CloneStubsIntoPartitions)
+          LMResources.StubsToClone.insert(&F);
+
+        // Create a callback, associate it with the stub for the function,
+        // and set the compile action to compile the partition containing the
+        // function.
+        auto CCInfo = CompileCallbackMgr.getCompileCallback(SrcM->getContext());
+        StubInits[mangle(F.getName(), DL)] =
+          std::make_pair(CCInfo.getAddress(),
+                         JITSymbolBase::flagsFromGlobalValue(F));
         CCInfo.setCompileAction(
-          [=](){
-            return BaseLayer.findSymbolIn(H, BodyName, false).getAddress();
+          [this, &LD, LMH, &F]() {
+            return this->extractAndCompile(LD, LMH, F);
           });
       }
 
+      LMResources.StubsMgr = CreateIndirectStubsManager();
+      auto EC = LMResources.StubsMgr->init(StubInits);
+      (void)EC;
+      // FIXME: This should be propagated back to the user. Stub creation may
+      //        fail for remote JITs.
+      assert(!EC && "Error generating stubs");
     }
 
-    // Ok - we've processed all the partitioned modules. Now add the
-    // stubs/globals module and set the update actions.
-    auto StubsH =
-      addModule(std::move(StubsModule), MSI, LogicalModule, FallbackLookup);
-
-    for (auto &KVPair : StubInfos) {
-      std::string AddrName = Mangle(KVPair.first + AddrSuffix,
-                                    M.getDataLayout());
-      auto &CCInfo = KVPair.second;
-      CCInfo.setUpdateAction(
-        CompileCallbackMgr.getLocalFPUpdater(StubsH, AddrName));
+    // Clone global variable decls.
+    for (auto &GV : SrcM->globals())
+      if (!GV.isDeclaration() && !VMap.count(&GV))
+        cloneGlobalVariableDecl(*GVsM, GV, &VMap);
+
+    // And the aliases.
+    for (auto &A : SrcM->aliases())
+      if (!VMap.count(&A))
+        cloneGlobalAliasDecl(*GVsM, A, VMap);
+
+    // Now we need to clone the GV and alias initializers.
+
+    // Initializers may refer to functions declared (but not defined) in this
+    // module. Build a materializer to clone decls on demand.
+    auto Materializer = createLambdaMaterializer(
+      [&GVsM, &LMResources](Value *V) -> Value* {
+        if (auto *F = dyn_cast<Function>(V)) {
+          // Decls in the original module just get cloned.
+          if (F->isDeclaration())
+            return cloneFunctionDecl(*GVsM, *F);
+
+          // Definitions in the original module (which we have emitted stubs
+          // for at this point) get turned into a constant alias to the stub
+          // instead.
+          const DataLayout &DL = GVsM->getDataLayout();
+          std::string FName = mangle(F->getName(), DL);
+          auto StubSym = LMResources.StubsMgr->findStub(FName, false);
+          unsigned PtrBitWidth = DL.getPointerTypeSizeInBits(F->getType());
+          ConstantInt *StubAddr =
+            ConstantInt::get(GVsM->getContext(),
+                             APInt(PtrBitWidth, StubSym.getAddress()));
+          Constant *Init = ConstantExpr::getCast(Instruction::IntToPtr,
+                                                 StubAddr, F->getType());
+          return GlobalAlias::create(F->getFunctionType(),
+                                     F->getType()->getAddressSpace(),
+                                     F->getLinkage(), F->getName(),
+                                     Init, GVsM.get());
+        }
+        // else....
+        return nullptr;
+      });
+
+    // Clone the global variable initializers.
+    for (auto &GV : SrcM->globals())
+      if (!GV.isDeclaration())
+        moveGlobalVariableInitializer(GV, VMap, &Materializer);
+
+    // Clone the global alias initializers.
+    for (auto &A : SrcM->aliases()) {
+      auto *NewA = cast<GlobalAlias>(VMap[&A]);
+      assert(NewA && "Alias not cloned?");
+      Value *Init = MapValue(A.getAliasee(), VMap, RF_None, nullptr,
+                             &Materializer);
+      NewA->setAliasee(cast<Constant>(Init));
     }
-  }
 
-  // Add the given Module to the base layer using a memory manager that will
-  // perform the appropriate scoped lookup (i.e. will look first with in the
-  // module from which it was extracted, then into the set to which that module
-  // belonged, and finally externally).
-  BaseLayerModuleSetHandleT addModule(
-                               std::unique_ptr<Module> M,
-                               ModuleSetInfo &MSI,
-                               typename CODScopedLookup::LMHandle LogicalModule,
-                               LookupFtor FallbackLookup) {
-
-    // Add this module to the JIT with a memory manager that uses the
-    // DylibLookup to resolve symbols.
-    std::vector<std::unique_ptr<Module>> MSet;
-    MSet.push_back(std::move(M));
-
-    auto DylibLookup = MSI.Lookup;
-    auto MM =
-      createLookasideRTDyldMM<SectionMemoryManager>(
-        [=](const std::string &Name) {
-          if (auto Symbol = DylibLookup->findSymbol(LogicalModule, Name))
-            return Symbol.getAddress();
-          return FallbackLookup(Name);
+    // Build a resolver for the globals module and add it to the base layer.
+    auto GVsResolver = createLambdaResolver(
+        [&LD, LMH](const std::string &Name) {
+          auto &LMResources = LD.getLogicalModuleResources(LMH);
+          if (auto Sym = LMResources.StubsMgr->findStub(Name, false))
+            return RuntimeDyld::SymbolInfo(Sym.getAddress(), Sym.getFlags());
+          return LD.getDylibResources().ExternalSymbolResolver(Name);
         },
-        [=](const std::string &Name) {
-          return DylibLookup->findSymbol(LogicalModule, Name).getAddress();
+        [](const std::string &Name) {
+          return RuntimeDyld::SymbolInfo(nullptr);
         });
 
-    BaseLayerModuleSetHandleT H =
-      BaseLayer.addModuleSet(std::move(MSet), std::move(MM));
-    // Add this module to the logical module lookup.
-    DylibLookup->addToLogicalModule(LogicalModule, H);
-    MSI.BaseLayerModuleSetHandles.push_back(H);
-
-    return H;
+    std::vector<std::unique_ptr<Module>> GVsMSet;
+    GVsMSet.push_back(std::move(GVsM));
+    auto GVsH =
+      BaseLayer.addModuleSet(std::move(GVsMSet),
+                             llvm::make_unique<SectionMemoryManager>(),
+                             std::move(GVsResolver));
+    LD.addToLogicalModule(LMH, GVsH);
   }
 
-  static std::string Mangle(StringRef Name, const DataLayout &DL) {
-    Mangler M(&DL);
+  static std::string mangle(StringRef Name, const DataLayout &DL) {
     std::string MangledName;
     {
       raw_string_ostream MangledNameStream(MangledName);
-      M.getNameWithPrefix(MangledNameStream, Name);
+      Mangler::getNameWithPrefix(MangledNameStream, Name, DL);
     }
     return MangledName;
   }
 
+  TargetAddress extractAndCompile(CODLogicalDylib &LD,
+                                  LogicalModuleHandle LMH,
+                                  Function &F) {
+    auto &LMResources = LD.getLogicalModuleResources(LMH);
+    Module &SrcM = *LMResources.SourceModule;
+
+    // If F is a declaration we must already have compiled it.
+    if (F.isDeclaration())
+      return 0;
+
+    // Grab the name of the function being called here.
+    std::string CalledFnName = mangle(F.getName(), SrcM.getDataLayout());
+
+    auto Part = Partition(F);
+    auto PartH = emitPartition(LD, LMH, Part);
+
+    TargetAddress CalledAddr = 0;
+    for (auto *SubF : Part) {
+      std::string FnName = mangle(SubF->getName(), SrcM.getDataLayout());
+      auto FnBodySym = BaseLayer.findSymbolIn(PartH, FnName, false);
+      assert(FnBodySym && "Couldn't find function body.");
+
+      TargetAddress FnBodyAddr = FnBodySym.getAddress();
+
+      // If this is the function we're calling record the address so we can
+      // return it from this function.
+      if (SubF == &F)
+        CalledAddr = FnBodyAddr;
+
+      // Update the function body pointer for the stub.
+      if (auto EC = LMResources.StubsMgr->updatePointer(FnName, FnBodyAddr))
+        return 0;
+    }
+
+    return CalledAddr;
+  }
+
+  template <typename PartitionT>
+  BaseLayerModuleSetHandleT emitPartition(CODLogicalDylib &LD,
+                                          LogicalModuleHandle LMH,
+                                          const PartitionT &Part) {
+    auto &LMResources = LD.getLogicalModuleResources(LMH);
+    Module &SrcM = *LMResources.SourceModule;
+
+    // Create the module.
+    std::string NewName = SrcM.getName();
+    for (auto *F : Part) {
+      NewName += ".";
+      NewName += F->getName();
+    }
+
+    auto M = llvm::make_unique<Module>(NewName, SrcM.getContext());
+    M->setDataLayout(SrcM.getDataLayout());
+    ValueToValueMapTy VMap;
+
+    auto Materializer = createLambdaMaterializer(
+      [this, &LMResources, &M, &VMap](Value *V) -> Value* {
+        if (auto *GV = dyn_cast<GlobalVariable>(V)) {
+          return cloneGlobalVariableDecl(*M, *GV);
+        } else if (auto *F = dyn_cast<Function>(V)) {
+          // Check whether we want to clone an available_externally definition.
+          if (LMResources.StubsToClone.count(F)) {
+            // Ok - we want an inlinable stub. For that to work we need a decl
+            // for the stub pointer.
+            auto *StubPtr = createImplPointer(*F->getType(), *M,
+                                              F->getName() + "$stub_ptr",
+                                              nullptr);
+            auto *ClonedF = cloneFunctionDecl(*M, *F);
+            makeStub(*ClonedF, *StubPtr);
+            ClonedF->setLinkage(GlobalValue::AvailableExternallyLinkage);
+            ClonedF->addFnAttr(Attribute::AlwaysInline);
+            return ClonedF;
+          }
+
+          return cloneFunctionDecl(*M, *F);
+        } else if (auto *A = dyn_cast<GlobalAlias>(V)) {
+          auto *PTy = cast<PointerType>(A->getType());
+          if (PTy->getElementType()->isFunctionTy())
+            return Function::Create(cast<FunctionType>(PTy->getElementType()),
+                                    GlobalValue::ExternalLinkage,
+                                    A->getName(), M.get());
+          // else
+          return new GlobalVariable(*M, PTy->getElementType(), false,
+                                    GlobalValue::ExternalLinkage,
+                                    nullptr, A->getName(), nullptr,
+                                    GlobalValue::NotThreadLocal,
+                                    PTy->getAddressSpace());
+        }
+        // Else.
+        return nullptr;
+      });
+
+    // Create decls in the new module.
+    for (auto *F : Part)
+      cloneFunctionDecl(*M, *F, &VMap);
+
+    // Move the function bodies.
+    for (auto *F : Part)
+      moveFunctionBody(*F, VMap, &Materializer);
+
+    // Create memory manager and symbol resolver.
+    auto MemMgr = llvm::make_unique<SectionMemoryManager>();
+    auto Resolver = createLambdaResolver(
+        [this, &LD, LMH](const std::string &Name) {
+          if (auto Symbol = LD.findSymbolInternally(LMH, Name))
+            return RuntimeDyld::SymbolInfo(Symbol.getAddress(),
+                                           Symbol.getFlags());
+          return LD.getDylibResources().ExternalSymbolResolver(Name);
+        },
+        [this, &LD, LMH](const std::string &Name) {
+          if (auto Symbol = LD.findSymbolInternally(LMH, Name))
+            return RuntimeDyld::SymbolInfo(Symbol.getAddress(),
+                                           Symbol.getFlags());
+          return RuntimeDyld::SymbolInfo(nullptr);
+        });
+    std::vector<std::unique_ptr<Module>> PartMSet;
+    PartMSet.push_back(std::move(M));
+    return BaseLayer.addModuleSet(std::move(PartMSet), std::move(MemMgr),
+                                  std::move(Resolver));
+  }
+
   BaseLayerT &BaseLayer;
-  CompileCallbackMgrT CompileCallbackMgr;
-  ModuleSetInfoListT ModuleSetInfos;
+  PartitioningFtor Partition;
+  CompileCallbackMgrT &CompileCallbackMgr;
+  IndirectStubsManagerBuilderT CreateIndirectStubsManager;
+
+  LogicalDylibList LogicalDylibs;
+  bool CloneStubsIntoPartitions;
 };
 
 } // End namespace orc.