Implement the initial AnalysisManagement infrastructure, with the introduction of the FunctionAnalysisManager and ModuleAnalysisManager classes. These classes provide analysis computation, caching, and invalidation for a specific IR unit. The invalidation is currently limited to either all or none, i.e. you cannot yet preserve specific analyses.

An analysis can be any class, but it must provide the following:
* A constructor for a given IR unit.

struct MyAnalysis {
  // Compute this analysis with the provided module.
  MyAnalysis(Module *module);
};

Analyses can be accessed from a Pass by calling either the 'getAnalysisResult<AnalysisT>' or 'getCachedAnalysisResult<AnalysisT>' methods. A FunctionPass may query for a cached analysis on the parent module with 'getCachedModuleAnalysisResult'. Similary, a ModulePass may query an analysis, it doesn't need to be cached, on a child function with 'getFunctionAnalysisResult'.

By default, when running a pass all cached analyses are set to be invalidated. If no transformation was performed, a pass can use the method 'markAllAnalysesPreserved' to preserve all analysis results. As noted above, preserving specific analyses is not yet supported.

PiperOrigin-RevId: 236505642
This commit is contained in:
River Riddle 2019-03-02 21:46:58 -08:00 committed by jpienaar
parent c1b02a17be
commit 485746f524
5 changed files with 360 additions and 42 deletions

View File

@ -0,0 +1,220 @@
//===- AnalysisManager.h - Analysis Management Infrastructure ---*- C++ -*-===//
//
// Copyright 2019 The MLIR Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// =============================================================================
#ifndef MLIR_PASS_ANALYSISMANAGER_H
#define MLIR_PASS_ANALYSISMANAGER_H
#include "mlir/IR/Module.h"
#include "mlir/Support/LLVM.h"
#include "llvm/ADT/DenseMap.h"
#include "llvm/ADT/SmallPtrSet.h"
namespace mlir {
/// A special type used by analyses to provide an address that identifies a
/// particular analysis set or a concrete analysis type.
struct AnalysisID {
template <typename AnalysisT> static AnalysisID *getID() {
static AnalysisID id;
return &id;
}
};
//===----------------------------------------------------------------------===//
// Analysis Preservation and Result Modeling
//===----------------------------------------------------------------------===//
namespace detail {
/// A utility class to represent the analyses that are known to be preserved.
class PreservedAnalyses {
public:
/// Mark all analyses as preserved.
void preserveAll() { preservedIDs.insert(&allAnalysesID); }
/// Returns if all analyses were marked preserved.
bool isAll() const { return preservedIDs.count(&allAnalysesID); }
private:
/// An identifier used to represent all potential analyses.
constexpr static AnalysisID allAnalysesID = {};
/// The set of analyses that are known to be preserved.
SmallPtrSet<const void *, 2> preservedIDs;
};
/// The abstract polymorphic base class representing an analysis.
struct AnalysisConcept {
virtual ~AnalysisConcept() = default;
};
/// A derived analysis model used to hold a specific analysis object.
template <typename AnalysisT> struct AnalysisModel : public AnalysisConcept {
template <typename... Args>
explicit AnalysisModel(Args &&... args)
: analysis(std::forward<Args>(args)...) {}
AnalysisT analysis;
};
/// This class represents a cache of analysis results for a single IR unit. All
/// computation, caching, and invalidation of analyses takes place here.
template <typename IRUnitT> class AnalysisResultMap {
/// A mapping between an analysis id and an existing analysis instance.
using ResultMap =
DenseMap<const AnalysisID *, std::unique_ptr<AnalysisConcept>>;
public:
explicit AnalysisResultMap(IRUnitT *ir) : ir(ir) {}
/// Get an analysis for the current IR unit, computing it if necessary.
template <typename AnalysisT> AnalysisT &getResult() {
typename ResultMap::iterator it;
bool wasInserted;
std::tie(it, wasInserted) =
results.try_emplace(AnalysisID::getID<AnalysisT>());
// If we don't have a cached result for this function, compute it directly
// and add it to the cache.
if (wasInserted)
it->second = llvm::make_unique<AnalysisModel<AnalysisT>>(ir);
return static_cast<AnalysisModel<AnalysisT> &>(*it->second).analysis;
}
/// Get a cached analysis instance if one exists, otherwise return null.
template <typename AnalysisT>
llvm::Optional<std::reference_wrapper<AnalysisT>> getCachedResult() const {
auto res = results.find(AnalysisID::getID<AnalysisT>());
if (res == results.end())
return llvm::None;
return {static_cast<AnalysisModel<AnalysisT> &>(*res->second).analysis};
}
/// Returns the IR unit that this result map represents.
IRUnitT *getIRUnit() { return ir; }
const IRUnitT *getIRUnit() const { return ir; }
/// Clear any held analysis results.
void clear() { results.clear(); }
/// Invalidate any cached analyses based upon the given set of preserved
/// analyses.
void invalidate(const detail::PreservedAnalyses &pa) {
// If all analyses were preserved, then there is nothing to do here.
if (pa.isAll())
return;
// TODO: Fine grain invalidation of analyses.
clear();
}
private:
IRUnitT *ir;
ResultMap results;
};
} // namespace detail
//===----------------------------------------------------------------------===//
// Analysis Management
//===----------------------------------------------------------------------===//
/// An analysis manager for a specific function instance. This class can only be
/// constructed from a ModuleAnalysisManager instance.
class FunctionAnalysisManager {
public:
// Query for a cached analysis on the parent Module. The analysis may not
// exist and if it does it may be stale.
template <typename AnalysisT>
llvm::Optional<std::reference_wrapper<AnalysisT>>
getCachedModuleResult() const {
return parentImpl->getCachedResult<AnalysisT>();
}
// Query for the given analysis for the current function.
template <typename AnalysisT> AnalysisT &getResult() {
return impl->getResult<AnalysisT>();
}
// Query for a cached entry of the given analysis on the current function.
template <typename AnalysisT>
llvm::Optional<std::reference_wrapper<AnalysisT>> getCachedResult() const {
return impl->getCachedResult<AnalysisT>();
}
/// Invalidate any non preserved analyses,
void invalidate(const detail::PreservedAnalyses &pa) { impl->invalidate(pa); }
/// Clear any held analyses.
void clear() { impl->clear(); }
private:
FunctionAnalysisManager(const detail::AnalysisResultMap<Module> *parentImpl,
detail::AnalysisResultMap<Function> *impl)
: parentImpl(parentImpl), impl(impl) {}
/// A reference to the results map of the parent module within the owning
/// analysis manager.
const detail::AnalysisResultMap<Module> *parentImpl;
/// A reference to the results map within the owning analysis manager.
detail::AnalysisResultMap<Function> *impl;
/// Allow access to the constructor.
friend class ModuleAnalysisManager;
};
/// An analysis manager for a specific module instance.
class ModuleAnalysisManager {
public:
ModuleAnalysisManager(Module *module) : moduleAnalyses(module) {}
ModuleAnalysisManager(const ModuleAnalysisManager &) = delete;
ModuleAnalysisManager &operator=(const ModuleAnalysisManager &) = delete;
/// Query for the analysis of a function. The analysis is computed if it does
/// not exist.
template <typename AnalysisT>
AnalysisT &getFunctionResult(Function *function) {
return slice(function).getResult<AnalysisT>();
}
/// Query for the analysis of a module. The analysis is computed if it does
/// not exist.
template <typename AnalysisT> AnalysisT &getResult() {
return moduleAnalyses.getResult<AnalysisT>();
}
/// Query for a cached analysis for the module, or return nullptr.
template <typename AnalysisT>
llvm::Optional<std::reference_wrapper<AnalysisT>> getCachedResult() const {
return moduleAnalyses.getCachedResult<AnalysisT>();
}
/// Create an analysis slice for the given child function.
FunctionAnalysisManager slice(Function *function);
/// Invalidate any non preserved analyses.
void invalidate(const detail::PreservedAnalyses &pa);
private:
/// The cached analyses for functions within the current module.
DenseMap<Function *, detail::AnalysisResultMap<Function>> functionAnalyses;
/// The analyses for the owning module.
detail::AnalysisResultMap<Module> moduleAnalyses;
};
} // end namespace mlir
#endif // MLIR_PASS_ANALYSISMANAGER_H

View File

@ -18,14 +18,11 @@
#ifndef MLIR_PASS_PASS_H
#define MLIR_PASS_PASS_H
#include "mlir/IR/Module.h"
#include "mlir/Pass/AnalysisManager.h"
#include "mlir/Pass/PassRegistry.h"
#include "llvm/ADT/PointerIntPair.h"
namespace mlir {
class Function;
class Module;
/// The abstract base pass class. This class contains information describing the
/// derived pass object, e.g its kind and abstract PassInfo.
class Pass {
@ -67,11 +64,20 @@ class ModulePassExecutor;
/// The state for a single execution of a pass. This provides a unified
/// interface for accessing and initializing necessary state for pass execution.
template <typename IRUnitT> struct PassExecutionState {
explicit PassExecutionState(IRUnitT *ir) : irAndPassFailed(ir, false) {}
template <typename IRUnitT, typename AnalysisManagerT>
struct PassExecutionState {
PassExecutionState(IRUnitT *ir, AnalysisManagerT &analysisManager)
: irAndPassFailed(ir, false), analysisManager(analysisManager) {}
/// The current IR unit being transformed.
/// The current IR unit being transformed and a bool for if the pass signaled
/// a failure.
llvm::PointerIntPair<IRUnitT *, 1, bool> irAndPassFailed;
/// The analysis manager for the IR unit.
AnalysisManagerT &analysisManager;
/// The set of preserved analyses for the current execution.
detail::PreservedAnalyses preservedAnalyses;
};
} // namespace detail
@ -79,6 +85,9 @@ template <typename IRUnitT> struct PassExecutionState {
/// not inherit from this class directly, and instead should use the CRTP
/// FunctionPass class.
class FunctionPassBase : public Pass {
using PassStateT =
detail::PassExecutionState<Function, FunctionAnalysisManager>;
public:
static bool classof(const Pass *pass) {
return pass->getKind() == Kind::FunctionPass;
@ -96,19 +105,24 @@ protected:
}
/// Returns the current pass state.
detail::PassExecutionState<Function> &getPassState() {
PassStateT &getPassState() {
assert(passState && "pass state was never initialized");
return *passState;
}
/// Returns the current analysis manager.
FunctionAnalysisManager &getAnalysisManager() {
return getPassState().analysisManager;
}
private:
/// Forwarding function to execute this pass. Returns false if the pass
/// execution failed, true otherwise.
LLVM_NODISCARD
bool run(Function *fn);
bool run(Function *fn, FunctionAnalysisManager &fam);
/// The current execution state for the pass.
llvm::Optional<detail::PassExecutionState<Function>> passState;
llvm::Optional<PassStateT> passState;
/// Allow access to 'run'.
friend detail::FunctionPassExecutor;
@ -117,6 +131,8 @@ private:
/// Pass to transform a module. Derived passes should not inherit from this
/// class directly, and instead should use the CRTP ModulePass class.
class ModulePassBase : public Pass {
using PassStateT = detail::PassExecutionState<Module, ModuleAnalysisManager>;
public:
static bool classof(const Pass *pass) {
return pass->getKind() == Kind::ModulePass;
@ -132,19 +148,24 @@ protected:
Module &getModule() { return *getPassState().irAndPassFailed.getPointer(); }
/// Returns the current pass state.
detail::PassExecutionState<Module> &getPassState() {
PassStateT &getPassState() {
assert(passState && "pass state was never initialized");
return *passState;
}
/// Returns the current analysis manager.
ModuleAnalysisManager &getAnalysisManager() {
return getPassState().analysisManager;
}
private:
/// Forwarding function to execute this pass. Returns false if the pass
/// execution failed, true otherwise.
LLVM_NODISCARD
bool run(Module *module);
bool run(Module *module, ModuleAnalysisManager &mam);
/// The current execution state for the pass.
llvm::Optional<detail::PassExecutionState<Module>> passState;
llvm::Optional<PassStateT> passState;
/// Allow access to 'run'.
friend detail::ModulePassExecutor;
@ -162,13 +183,30 @@ protected:
PassModel() : BasePassT(PassID::getID<PassT>()) {}
/// TODO(riverriddle) Provide additional utilities for cloning, getting the
/// derived class name, etc..
/// derived class name, etc.
/// Signal that some invariant was broken when running. The IR is allowed to
/// be in an invalid state.
void signalPassFailure() {
this->getPassState().irAndPassFailed.setInt(true);
}
/// Query the result of an analysis for the current ir unit.
template <typename AnalysisT> AnalysisT &getAnalysisResult() {
return this->getAnalysisManager().template getResult<AnalysisT>();
}
/// Query the cached result of an analysis for the current ir unit if one
/// exists.
template <typename AnalysisT>
llvm::Optional<std::reference_wrapper<AnalysisT>> getCachedAnalysisResult() {
return this->getAnalysisManager().template getCachedResult<AnalysisT>();
}
/// Mark all analyses as preserved.
void markAllAnalysesPreserved() {
this->getPassState().preservedAnalyses.preserveAll();
}
};
} // end namespace detail
@ -183,14 +221,28 @@ protected:
/// Derived function passes are expected to provide the following:
/// - A 'void runOnFunction()' method.
template <typename T>
using FunctionPass = detail::PassModel<Function, T, FunctionPassBase>;
struct FunctionPass : public detail::PassModel<Function, T, FunctionPassBase> {
/// Returns the analysis result for the parent module if it exists.
template <typename AnalysisT>
llvm::Optional<std::reference_wrapper<AnalysisT>>
getCachedModuleAnalysisResult() {
return this->getAnalysisManager()
.template getCachedModuleResult<AnalysisT>();
}
};
/// A model for providing module pass specific utilities.
///
/// Derived module passes are expected to provide the following:
/// - A 'void runOnModule()' method.
template <typename T>
using ModulePass = detail::PassModel<Module, T, ModulePassBase>;
struct ModulePass : public detail::PassModel<Module, T, ModulePassBase> {
/// Returns the analysis result for a child function.
template <typename AnalysisT>
AnalysisT &getFunctionAnalysisResult(Function *f) {
return this->getAnalysisManager().template getFunctionResult<AnalysisT>(f);
}
};
} // end namespace mlir
#endif // MLIR_PASS_PASS_H

View File

@ -34,27 +34,34 @@ using namespace mlir::detail;
/// single .o file.
void Pass::anchor() {}
/// Forwarding function to execute this pass.
bool FunctionPassBase::run(Function *fn) {
/// Initialize the pass state.
passState.emplace(fn);
/// Forwarding function to execute this pass. Returns false if the pass
/// execution failed, true otherwise.
bool FunctionPassBase::run(Function *fn, FunctionAnalysisManager &fam) {
// Initialize the pass state.
passState.emplace(fn, fam);
/// Invoke the virtual runOnFunction function.
// Invoke the virtual runOnFunction function.
runOnFunction();
// Invalidate any non preserved analyses.
fam.invalidate(passState->preservedAnalyses);
// Return false if the pass signaled a failure.
return !passState->irAndPassFailed.getInt();
}
/// Forwarding function to execute this pass. Returns false if the pass
/// execution failed, true otherwise.
bool ModulePassBase::run(Module *module) {
/// Initialize the pass state.
passState.emplace(module);
bool ModulePassBase::run(Module *module, ModuleAnalysisManager &mam) {
// Initialize the pass state.
passState.emplace(module, mam);
/// Invoke the virtual runOnModule function.
// Invoke the virtual runOnModule function.
runOnModule();
// Invalidate any non preserved analyses.
mam.invalidate(passState->preservedAnalyses);
// Return false if the pass signaled a failure.
return !passState->irAndPassFailed.getInt();
}
@ -91,8 +98,7 @@ public:
/// Run the executor on the given function. Returns false if the pass
/// execution failed, true otherwise.
LLVM_NODISCARD
bool run(Function *function);
bool run(Function *function, FunctionAnalysisManager &fam);
/// Add a pass to the current executor. This takes ownership over the provided
/// pass pointer.
@ -118,8 +124,7 @@ public:
/// Run the executor on the given module. Returns false if the pass
/// execution failed, true otherwise.
LLVM_NODISCARD
bool run(Module *module);
bool run(Module *module, ModuleAnalysisManager &mam);
/// Add a pass to the current executor. This takes ownership over the provided
/// pass pointer.
@ -137,10 +142,11 @@ private:
} // end namespace mlir
/// Run all of the passes in this manager over the current function.
bool detail::FunctionPassExecutor::run(Function *function) {
bool detail::FunctionPassExecutor::run(Function *function,
FunctionAnalysisManager &fam) {
for (auto &pass : passes) {
/// Create an execution state for this pass.
if (!pass->run(function))
if (!pass->run(function, fam))
return false;
// TODO: This should be opt-out and handled separately.
if (function->verify())
@ -150,9 +156,10 @@ bool detail::FunctionPassExecutor::run(Function *function) {
}
/// Run all of the passes in this manager over the current module.
bool detail::ModulePassExecutor::run(Module *module) {
bool detail::ModulePassExecutor::run(Module *module,
ModuleAnalysisManager &mam) {
for (auto &pass : passes) {
if (!pass->run(module))
if (!pass->run(module, mam))
return false;
// TODO: This should be opt-out and handled separately.
if (module->verify())
@ -194,14 +201,22 @@ private:
/// Execute the held function pass over all non-external functions within the
/// module.
void ModuleToFunctionPassAdaptor::runOnModule() {
ModuleAnalysisManager &mam = getAnalysisManager();
for (auto &func : getModule()) {
// Skip external functions.
if (func.isExternal())
continue;
// Run the held function pipeline over the current function.
if (!fpe.run(&func))
auto fam = mam.slice(&func);
if (!fpe.run(&func, fam))
return signalPassFailure();
// Clear out any computed function analyses. These analyses won't be used
// any more in this pipeline, and this helps reduce the current working set
// of memory. If preserving these analyses becomes important in the future
// we can re-evalutate this.
fam.clear();
}
}
@ -253,4 +268,29 @@ void PassManager::addPass(FunctionPassBase *pass) {
}
/// Run the passes within this manager on the provided module.
bool PassManager::run(Module *module) { return mpe->run(module); }
bool PassManager::run(Module *module) {
ModuleAnalysisManager mam(module);
return mpe->run(module, mam);
}
//===----------------------------------------------------------------------===//
// AnalysisManager
//===----------------------------------------------------------------------===//
/// Create an analysis slice for the given child function.
FunctionAnalysisManager ModuleAnalysisManager::slice(Function *function) {
assert(function->getModule() == moduleAnalyses.getIRUnit() &&
"function has a different parent module");
auto it = functionAnalyses.try_emplace(function, function);
return {&moduleAnalyses, &it.first->second};
}
/// Invalidate any non preserved analyses.
void ModuleAnalysisManager::invalidate(const detail::PreservedAnalyses &pa) {
if (pa.isAll())
return;
// TODO: Fine grain invalidation of analyses.
moduleAnalyses.clear();
functionAnalyses.clear();
}

View File

@ -217,8 +217,14 @@ void CSE::simplifyBlockList(DominanceInfo &domInfo, BlockList &blockList) {
}
void CSE::runOnFunction() {
DominanceInfo domInfo(&getFunction());
simplifyBlockList(domInfo, getFunction().getBlockList());
simplifyBlockList(getAnalysisResult<DominanceInfo>(),
getFunction().getBlockList());
// If no operations were erased, then we mark all analyses as preserved.
if (opsToErase.empty()) {
markAllAnalysesPreserved();
return;
}
/// Erase any operations that were marked as dead during simplification.
for (auto *op : opsToErase)

View File

@ -212,13 +212,13 @@ void MemRefDataFlowOpt::forwardStoreToLoad(OpPointer<LoadOp> loadOp) {
void MemRefDataFlowOpt::runOnFunction() {
// Only supports single block functions at the moment.
Function &f = getFunction();
if (f.getBlocks().size() != 1)
if (f.getBlocks().size() != 1) {
markAllAnalysesPreserved();
return;
}
DominanceInfo theDomInfo(&f);
domInfo = &theDomInfo;
PostDominanceInfo thePostDomInfo(&f);
postDomInfo = &thePostDomInfo;
domInfo = &getAnalysisResult<DominanceInfo>();
postDomInfo = &getAnalysisResult<PostDominanceInfo>();
loadOpsToErase.clear();
memrefsToErase.clear();