VPlan.cpp
VPlanHCFGBuilder.cpp
VPlanHCFGTransforms.cpp
+ VPlanPredicator.cpp
VPlanSLP.cpp
VPlanVerifier.cpp
#define LV_NAME "loop-vectorize"
#define DEBUG_TYPE LV_NAME
+extern cl::opt<bool> EnableVPlanPredication;
+
static cl::opt<bool>
EnableIfConversion("enable-if-conversion", cl::init(true), cl::Hidden,
cl::desc("Enable if-conversion during vectorization."));
// Check whether the BranchInst is a supported one. Only unconditional
// branches, conditional branches with an outer loop invariant condition or
// backedges are supported.
- if (Br && Br->isConditional() &&
+ // FIXME: We skip these checks when VPlan predication is enabled as we
+ // want to allow divergent branches. This whole check will be removed
+ // once VPlan predication is on by default.
+ if (!EnableVPlanPredication && Br && Br->isConditional() &&
!TheLoop->isLoopInvariant(Br->getCondition()) &&
!LI->isLoopHeader(Br->getSuccessor(0)) &&
!LI->isLoopHeader(Br->getSuccessor(1))) {
#include "VPRecipeBuilder.h"
#include "VPlanHCFGBuilder.h"
#include "VPlanHCFGTransforms.h"
+#include "VPlanPredicator.h"
#include "llvm/ADT/APInt.h"
#include "llvm/ADT/ArrayRef.h"
#include "llvm/ADT/DenseMap.h"
cl::desc("Enable VPlan-native vectorization path with "
"support for outer loop vectorization."));
+// FIXME: Remove this switch once we have divergence analysis. Currently we
+// assume divergent non-backedge branches when this switch is true.
+cl::opt<bool> EnableVPlanPredication(
+ "enable-vplan-predication", cl::init(false), cl::Hidden,
+ cl::desc("Enable VPlan-native vectorization path predicator with "
+ "support for outer loop vectorization."));
+
// This flag enables the stress testing of the VPlan H-CFG construction in the
// VPlan-native vectorization path. It must be used in conjuction with
// -enable-vplan-native-path. -vplan-verify-hcfg can also be used to enable the
VPlanHCFGBuilder HCFGBuilder(OrigLoop, LI, *Plan);
HCFGBuilder.buildHierarchicalCFG();
+ for (unsigned VF = Range.Start; VF < Range.End; VF *= 2)
+ Plan->addVF(VF);
+
+ if (EnableVPlanPredication) {
+ VPlanPredicator VPP(*Plan);
+ VPP.predicate();
+
+ // Avoid running transformation to recipes until masked code generation in
+ // VPlan-native path is in place.
+ return Plan;
+ }
+
SmallPtrSet<Instruction *, 1> DeadInstructions;
VPlanHCFGTransforms::VPInstructionsToVPRecipes(
Plan, Legal->getInductionVars(), DeadInstructions);
- for (unsigned VF = Range.Start; VF < Range.End; VF *= 2)
- Plan->addVF(VF);
-
return Plan;
}
VectorizationFactor VF = LVP.planInVPlanNativePath(OptForSize, UserVF);
// If we are stress testing VPlan builds, do not attempt to generate vector
- // code.
- if (VPlanBuildStressTest)
+ // code. Masked vector code generation support will follow soon.
+ if (VPlanBuildStressTest || EnableVPlanPredication)
return false;
LVP.setBestPlan(VF.Width, 1);
bumpIndent(1);
OS << Indent << "\"" << DOT::EscapeString(BasicBlock->getName()) << ":\\n\"";
bumpIndent(1);
+
+ // Dump the block predicate.
+ const VPValue *Pred = BasicBlock->getPredicate();
+ if (Pred) {
+ OS << " +\n" << Indent << " \"BlockPredicate: ";
+ if (const VPInstruction *PredI = dyn_cast<VPInstruction>(Pred)) {
+ PredI->printAsOperand(OS);
+ OS << " (" << DOT::EscapeString(PredI->getParent()->getName())
+ << ")\\l\"";
+ } else
+ Pred->printAsOperand(OS);
+ }
+
for (const VPRecipeBase &Recipe : *BasicBlock)
Recipe.print(OS, Indent);
/// Successor selector, null for zero or single successor blocks.
VPValue *CondBit = nullptr;
+ /// Current block predicate - null if the block does not need a predicate.
+ VPValue *Predicate = nullptr;
+
/// Add \p Successor as the last successor to this block.
void appendSuccessor(VPBlockBase *Successor) {
assert(Successor && "Cannot add nullptr successor!");
void setCondBit(VPValue *CV) { CondBit = CV; }
+ VPValue *getPredicate() { return Predicate; }
+
+ const VPValue *getPredicate() const { return Predicate; }
+
+ void setPredicate(VPValue *Pred) { Predicate = Pred; }
+
/// Set a given VPBlockBase \p Successor as the single successor of this
/// VPBlockBase. This VPBlockBase is not added as predecessor of \p Successor.
/// This VPBlockBase must have no successors.
appendPredecessor(Pred);
}
+ /// Remove all the predecessor of this block.
+ void clearPredecessors() { Predecessors.clear(); }
+
+ /// Remove all the successors of this block and set to null its condition bit
+ void clearSuccessors() {
+ Successors.clear();
+ CondBit = nullptr;
+ }
+
/// The method which generates the output IR that correspond to this
/// VPBlockBase, thereby "executing" the VPlan.
virtual void execute(struct VPTransformState *State) = 0;
From->removeSuccessor(To);
To->removePredecessor(From);
}
+
+ /// Returns true if the edge \p FromBlock -> \p ToBlock is a back-edge.
+ static bool isBackEdge(const VPBlockBase *FromBlock,
+ const VPBlockBase *ToBlock, const VPLoopInfo *VPLI) {
+ assert(FromBlock->getParent() == ToBlock->getParent() &&
+ FromBlock->getParent() && "Must be in same region");
+ const VPLoop *FromLoop = VPLI->getLoopFor(FromBlock);
+ const VPLoop *ToLoop = VPLI->getLoopFor(ToBlock);
+ if (!FromLoop || !ToLoop || FromLoop != ToLoop)
+ return false;
+
+ // A back-edge is a branch from the loop latch to its header.
+ return ToLoop->isLoopLatch(FromBlock) && ToBlock == ToLoop->getHeader();
+ }
+
+ /// Returns true if \p Block is a loop latch
+ static bool blockIsLoopLatch(const VPBlockBase *Block,
+ const VPLoopInfo *VPLInfo) {
+ if (const VPLoop *ParentVPL = VPLInfo->getLoopFor(Block))
+ return ParentVPL->isLoopLatch(Block);
+
+ return false;
+ }
+
+ /// Count and return the number of succesors of \p PredBlock excluding any
+ /// backedges.
+ static unsigned countSuccessorsNoBE(VPBlockBase *PredBlock,
+ VPLoopInfo *VPLI) {
+ unsigned Count = 0;
+ for (VPBlockBase *SuccBlock : PredBlock->getSuccessors()) {
+ if (!VPBlockUtils::isBackEdge(PredBlock, SuccBlock, VPLI))
+ Count++;
+ }
+ return Count;
+ }
};
class VPInterleavedAccessInfo {
--- /dev/null
+//===-- VPlanPredicator.cpp -------------------------------------*- C++ -*-===//
+//
+// The LLVM Compiler Infrastructure
+//
+// This file is distributed under the University of Illinois Open Source
+// License. See LICENSE.TXT for details.
+//
+//===----------------------------------------------------------------------===//
+///
+/// \file
+/// This file implements the VPlanPredicator class which contains the public
+/// interfaces to predicate and linearize the VPlan region.
+///
+//===----------------------------------------------------------------------===//
+
+#include "VPlanPredicator.h"
+#include "VPlan.h"
+#include "llvm/ADT/DepthFirstIterator.h"
+#include "llvm/ADT/GraphTraits.h"
+#include "llvm/ADT/PostOrderIterator.h"
+#include "llvm/Support/Debug.h"
+#include "llvm/Support/raw_ostream.h"
+
+#define DEBUG_TYPE "VPlanPredicator"
+
+using namespace llvm;
+
+// Generate VPInstructions at the beginning of CurrBB that calculate the
+// predicate being propagated from PredBB to CurrBB depending on the edge type
+// between them. For example if:
+// i. PredBB is controlled by predicate %BP, and
+// ii. The edge PredBB->CurrBB is the false edge, controlled by the condition
+// bit value %CBV then this function will generate the following two
+// VPInstructions at the start of CurrBB:
+// %IntermediateVal = not %CBV
+// %FinalVal = and %BP %IntermediateVal
+// It returns %FinalVal.
+VPValue *VPlanPredicator::getOrCreateNotPredicate(VPBasicBlock *PredBB,
+ VPBasicBlock *CurrBB) {
+ VPValue *CBV = PredBB->getCondBit();
+
+ // Set the intermediate value - this is either 'CBV', or 'not CBV'
+ // depending on the edge type.
+ EdgeType ET = getEdgeTypeBetween(PredBB, CurrBB);
+ VPValue *IntermediateVal = nullptr;
+ switch (ET) {
+ case EdgeType::TRUE_EDGE:
+ // CurrBB is the true successor of PredBB - nothing to do here.
+ IntermediateVal = CBV;
+ break;
+
+ case EdgeType::FALSE_EDGE:
+ // CurrBB is the False successor of PredBB - compute not of CBV.
+ IntermediateVal = Builder.createNot(CBV);
+ break;
+ }
+
+ // Now AND intermediate value with PredBB's block predicate if it has one.
+ VPValue *BP = PredBB->getPredicate();
+ if (BP)
+ return Builder.createAnd(BP, IntermediateVal);
+ else
+ return IntermediateVal;
+}
+
+// Generate a tree of ORs for all IncomingPredicates in WorkList.
+// Note: This function destroys the original Worklist.
+//
+// P1 P2 P3 P4 P5
+// \ / \ / /
+// OR1 OR2 /
+// \ | /
+// \ +/-+
+// \ / |
+// OR3 |
+// \ |
+// OR4 <- Returns this
+// |
+//
+// The algorithm uses a worklist of predicates as its main data structure.
+// We pop a pair of values from the front (e.g. P1 and P2), generate an OR
+// (in this example OR1), and push it back. In this example the worklist
+// contains {P3, P4, P5, OR1}.
+// The process iterates until we have only one element in the Worklist (OR4).
+// The last element is the root predicate which is returned.
+VPValue *VPlanPredicator::genPredicateTree(std::list<VPValue *> &Worklist) {
+ if (Worklist.empty())
+ return nullptr;
+
+ // The worklist initially contains all the leaf nodes. Initialize the tree
+ // using them.
+ while (Worklist.size() >= 2) {
+ // Pop a pair of values from the front.
+ VPValue *LHS = Worklist.front();
+ Worklist.pop_front();
+ VPValue *RHS = Worklist.front();
+ Worklist.pop_front();
+
+ // Create an OR of these values.
+ VPValue *Or = Builder.createOr(LHS, RHS);
+
+ // Push OR to the back of the worklist.
+ Worklist.push_back(Or);
+ }
+
+ assert(Worklist.size() == 1 && "Expected 1 item in worklist");
+
+ // The root is the last node in the worklist.
+ VPValue *Root = Worklist.front();
+
+ // This root needs to replace the existing block predicate. This is done in
+ // the caller function.
+ return Root;
+}
+
+// Return whether the edge FromBlock -> ToBlock is a TRUE_EDGE or FALSE_EDGE
+VPlanPredicator::EdgeType
+VPlanPredicator::getEdgeTypeBetween(VPBlockBase *FromBlock,
+ VPBlockBase *ToBlock) {
+ unsigned Count = 0;
+ for (VPBlockBase *SuccBlock : FromBlock->getSuccessors()) {
+ if (SuccBlock == ToBlock) {
+ assert(Count < 2 && "Switch not supported currently");
+ return (Count == 0) ? EdgeType::TRUE_EDGE : EdgeType::FALSE_EDGE;
+ }
+ Count++;
+ }
+
+ llvm_unreachable("Broken getEdgeTypeBetween");
+}
+
+// Generate all predicates needed for CurrBlock by going through its immediate
+// predecessor blocks.
+void VPlanPredicator::createOrPropagatePredicates(VPBlockBase *CurrBlock,
+ VPRegionBlock *Region) {
+ // Blocks that dominate region exit inherit the predicate from the region.
+ // Return after setting the predicate.
+ if (VPDomTree.dominates(CurrBlock, Region->getExit())) {
+ VPValue *RegionBP = Region->getPredicate();
+ CurrBlock->setPredicate(RegionBP);
+ return;
+ }
+
+ // Collect all incoming predicates in a worklist.
+ std::list<VPValue *> IncomingPredicates;
+
+ // Set the builder's insertion point to the top of the current BB
+ VPBasicBlock *CurrBB = cast<VPBasicBlock>(CurrBlock->getEntryBasicBlock());
+ Builder.setInsertPoint(CurrBB, CurrBB->begin());
+
+ // For each predecessor, generate the VPInstructions required for
+ // computing 'BP AND (not) CBV" at the top of CurrBB.
+ // Collect the outcome of this calculation for all predecessors
+ // into IncomingPredicates.
+ for (VPBlockBase *PredBlock : CurrBlock->getPredecessors()) {
+ // Skip back-edges
+ if (VPBlockUtils::isBackEdge(PredBlock, CurrBlock, VPLI))
+ continue;
+
+ VPValue *IncomingPredicate = nullptr;
+ unsigned NumPredSuccsNoBE =
+ VPBlockUtils::countSuccessorsNoBE(PredBlock, VPLI);
+
+ // If there is an unconditional branch to the currBB, then we don't create
+ // edge predicates. We use the predecessor's block predicate instead.
+ if (NumPredSuccsNoBE == 1)
+ IncomingPredicate = PredBlock->getPredicate();
+ else if (NumPredSuccsNoBE == 2) {
+ // Emit recipes into CurrBlock if required
+ assert(isa<VPBasicBlock>(PredBlock) && "Only BBs have multiple exits");
+ IncomingPredicate =
+ getOrCreateNotPredicate(cast<VPBasicBlock>(PredBlock), CurrBB);
+ } else
+ llvm_unreachable("FIXME: switch statement ?");
+
+ if (IncomingPredicate)
+ IncomingPredicates.push_back(IncomingPredicate);
+ }
+
+ // Logically OR all incoming predicates by building the Predicate Tree.
+ VPValue *Predicate = genPredicateTree(IncomingPredicates);
+
+ // Now update the block's predicate with the new one.
+ CurrBlock->setPredicate(Predicate);
+}
+
+// Generate all predicates needed for Region.
+void VPlanPredicator::predicateRegionRec(VPRegionBlock *Region) {
+ VPBasicBlock *EntryBlock = cast<VPBasicBlock>(Region->getEntry());
+ ReversePostOrderTraversal<VPBlockBase *> RPOT(EntryBlock);
+
+ // Generate edge predicates and append them to the block predicate. RPO is
+ // necessary since the predecessor blocks' block predicate needs to be set
+ // before the current block's block predicate can be computed.
+ for (VPBlockBase *Block : make_range(RPOT.begin(), RPOT.end())) {
+ // TODO: Handle nested regions once we start generating the same.
+ assert(!isa<VPRegionBlock>(Block) && "Nested region not expected");
+ createOrPropagatePredicates(Block, Region);
+ }
+}
+
+// Linearize the CFG within Region.
+// TODO: Predication and linearization need RPOT for every region.
+// This traversal is expensive. Since predication is not adding new
+// blocks, we should be able to compute RPOT once in predication and
+// reuse it here. This becomes even more important once we have nested
+// regions.
+void VPlanPredicator::linearizeRegionRec(VPRegionBlock *Region) {
+ ReversePostOrderTraversal<VPBlockBase *> RPOT(Region->getEntry());
+ VPBlockBase *PrevBlock = nullptr;
+
+ for (VPBlockBase *CurrBlock : make_range(RPOT.begin(), RPOT.end())) {
+ // TODO: Handle nested regions once we start generating the same.
+ assert(!isa<VPRegionBlock>(CurrBlock) && "Nested region not expected");
+
+ // Linearize control flow by adding an unconditional edge between PrevBlock
+ // and CurrBlock skipping loop headers and latches to keep intact loop
+ // header predecessors and loop latch successors.
+ if (PrevBlock && !VPLI->isLoopHeader(CurrBlock) &&
+ !VPBlockUtils::blockIsLoopLatch(PrevBlock, VPLI)) {
+
+ LLVM_DEBUG(dbgs() << "Linearizing: " << PrevBlock->getName() << "->"
+ << CurrBlock->getName() << "\n");
+
+ PrevBlock->clearSuccessors();
+ CurrBlock->clearPredecessors();
+ VPBlockUtils::connectBlocks(PrevBlock, CurrBlock);
+ }
+
+ PrevBlock = CurrBlock;
+ }
+}
+
+// Entry point. The driver function for the predicator.
+void VPlanPredicator::predicate(void) {
+ // Predicate the blocks within Region.
+ predicateRegionRec(cast<VPRegionBlock>(Plan.getEntry()));
+
+ // Linearlize the blocks with Region.
+ linearizeRegionRec(cast<VPRegionBlock>(Plan.getEntry()));
+}
+
+VPlanPredicator::VPlanPredicator(VPlan &Plan)
+ : Plan(Plan), VPLI(&(Plan.getVPLoopInfo())) {
+ // FIXME: Predicator is currently computing the dominator information for the
+ // top region. Once we start storing dominator information in a VPRegionBlock,
+ // we can avoid this recalculation.
+ VPDomTree.recalculate(*(cast<VPRegionBlock>(Plan.getEntry())));
+}
--- /dev/null
+//===-- VPlanPredicator.h ---------------------------------------*- C++ -*-===//
+//
+// The LLVM Compiler Infrastructure
+//
+// This file is distributed under the University of Illinois Open Source
+// License. See LICENSE.TXT for details.
+//
+//===----------------------------------------------------------------------===//
+///
+/// \file
+/// This file defines the VPlanPredicator class which contains the public
+/// interfaces to predicate and linearize the VPlan region.
+///
+//===----------------------------------------------------------------------===//
+
+#ifndef LLVM_TRANSFORMS_VECTORIZE_VPLAN_PREDICATOR_H
+#define LLVM_TRANSFORMS_VECTORIZE_VPLAN_PREDICATOR_H
+
+#include "LoopVectorizationPlanner.h"
+#include "VPlan.h"
+#include "VPlanDominatorTree.h"
+
+namespace llvm {
+
+class VPlanPredicator {
+private:
+ enum class EdgeType {
+ TRUE_EDGE,
+ FALSE_EDGE,
+ };
+
+ // VPlan being predicated.
+ VPlan &Plan;
+
+ // VPLoopInfo for Plan's HCFG.
+ VPLoopInfo *VPLI;
+
+ // Dominator tree for Plan's HCFG.
+ VPDominatorTree VPDomTree;
+
+ // VPlan builder used to generate VPInstructions for block predicates.
+ VPBuilder Builder;
+
+ /// Get the type of edge from \p FromBlock to \p ToBlock. Returns TRUE_EDGE if
+ /// \p ToBlock is either the unconditional successor or the conditional true
+ /// successor of \p FromBlock and FALSE_EDGE otherwise.
+ EdgeType getEdgeTypeBetween(VPBlockBase *FromBlock, VPBlockBase *ToBlock);
+
+ /// Create and return VPValue corresponding to the predicate for the edge from
+ /// \p PredBB to \p CurrentBlock.
+ VPValue *getOrCreateNotPredicate(VPBasicBlock *PredBB, VPBasicBlock *CurrBB);
+
+ /// Generate and return the result of ORing all the predicate VPValues in \p
+ /// Worklist.
+ VPValue *genPredicateTree(std::list<VPValue *> &Worklist);
+
+ /// Create or propagate predicate for \p CurrBlock in region \p Region using
+ /// predicate(s) of its predecessor(s)
+ void createOrPropagatePredicates(VPBlockBase *CurrBlock,
+ VPRegionBlock *Region);
+
+ /// Predicate the CFG within \p Region.
+ void predicateRegionRec(VPRegionBlock *Region);
+
+ /// Linearize the CFG within \p Region.
+ void linearizeRegionRec(VPRegionBlock *Region);
+
+public:
+ VPlanPredicator(VPlan &Plan);
+
+ /// Predicate Plan's HCFG.
+ void predicate(void);
+};
+} // end namespace llvm
+#endif // LLVM_TRANSFORMS_VECTORIZE_VPLAN_PREDICATOR_H
add_llvm_unittest(VectorizeTests
VPlanDominatorTreeTest.cpp
VPlanLoopInfoTest.cpp
+ VPlanPredicatorTest.cpp
VPlanTest.cpp
VPlanHCFGTest.cpp
VPlanSlpTest.cpp
--- /dev/null
+//===- llvm/unittests/Transforms/Vectorize/VPlanPredicatorTest.cpp -----===//
+//
+// The LLVM Compiler Infrastructure
+//
+// This file is distributed under the University of Illinois Open Source
+// License. See LICENSE.TXT for details.
+//
+//===----------------------------------------------------------------------===//
+
+#include "../lib/Transforms/Vectorize/VPlanPredicator.h"
+#include "VPlanTestBase.h"
+#include "gtest/gtest.h"
+
+namespace llvm {
+namespace {
+
+class VPlanPredicatorTest : public VPlanTestBase {};
+
+TEST_F(VPlanPredicatorTest, BasicPredicatorTest) {
+ const char *ModuleString =
+ "@arr = common global [8 x [8 x i64]] "
+ "zeroinitializer, align 16\n"
+ "@arr2 = common global [8 x [8 x i64]] "
+ "zeroinitializer, align 16\n"
+ "@arr3 = common global [8 x [8 x i64]] "
+ "zeroinitializer, align 16\n"
+ "define void @f(i64 %n1) {\n"
+ "entry:\n"
+ " br label %for.cond1.preheader\n"
+ "for.cond1.preheader: \n"
+ " %i1.029 = phi i64 [ 0, %entry ], [ %inc14, %for.inc13 ]\n"
+ " br label %for.body3\n"
+ "for.body3: \n"
+ " %i2.028 = phi i64 [ 0, %for.cond1.preheader ], [ %inc, %for.inc ]\n"
+ " %arrayidx4 = getelementptr inbounds [8 x [8 x i64]], [8 x [8 x i64]]* "
+ "@arr, i64 0, i64 %i2.028, i64 %i1.029\n"
+ " %0 = load i64, i64* %arrayidx4, align 8\n"
+ " %cmp5 = icmp ugt i64 %0, 10\n"
+ " br i1 %cmp5, label %if.then, label %for.inc\n"
+ "if.then: \n"
+ " %arrayidx7 = getelementptr inbounds [8 x [8 x i64]], [8 x [8 x i64]]* "
+ "@arr2, i64 0, i64 %i2.028, i64 %i1.029\n"
+ " %1 = load i64, i64* %arrayidx7, align 8\n"
+ " %cmp8 = icmp ugt i64 %1, 100\n"
+ " br i1 %cmp8, label %if.then9, label %for.inc\n"
+ "if.then9: \n"
+ " %add = add nuw nsw i64 %i2.028, %i1.029\n"
+ " %arrayidx11 = getelementptr inbounds [8 x [8 x i64]], [8 x [8 x "
+ "i64]]* @arr3, i64 0, i64 %i2.028, i64 %i1.029\n"
+ " store i64 %add, i64* %arrayidx11, align 8\n"
+ " br label %for.inc\n"
+ "for.inc: \n"
+ " %inc = add nuw nsw i64 %i2.028, 1\n"
+ " %exitcond = icmp eq i64 %inc, 8\n"
+ " br i1 %exitcond, label %for.inc13, label %for.body3\n"
+ "for.inc13: \n"
+ " %inc14 = add nuw nsw i64 %i1.029, 1\n"
+ " %exitcond30 = icmp eq i64 %inc14, 8\n"
+ " br i1 %exitcond30, label %for.end15, label %for.cond1.preheader\n"
+ "for.end15: \n"
+ " ret void\n"
+ "}\n";
+
+ Module &M = parseModule(ModuleString);
+
+ Function *F = M.getFunction("f");
+ BasicBlock *LoopHeader = F->getEntryBlock().getSingleSuccessor();
+ auto Plan = buildHCFG(LoopHeader);
+
+ VPRegionBlock *TopRegion = cast<VPRegionBlock>(Plan->getEntry());
+ VPBlockBase *PH = TopRegion->getEntry();
+ VPBlockBase *H = PH->getSingleSuccessor();
+ VPBlockBase *InnerLoopH = H->getSingleSuccessor();
+ VPBlockBase *OuterIf = InnerLoopH->getSuccessors()[0];
+ VPBlockBase *InnerLoopLatch = InnerLoopH->getSuccessors()[1];
+ VPBlockBase *InnerIf = OuterIf->getSuccessors()[0];
+ VPValue *CBV1 = InnerLoopH->getCondBit();
+ VPValue *CBV2 = OuterIf->getCondBit();
+
+ // Apply predication.
+ VPlanPredicator VPP(*Plan);
+ VPP.predicate();
+
+ VPBlockBase *InnerLoopLinSucc = InnerLoopH->getSingleSuccessor();
+ VPBlockBase *OuterIfLinSucc = OuterIf->getSingleSuccessor();
+ VPBlockBase *InnerIfLinSucc = InnerIf->getSingleSuccessor();
+ VPValue *OuterIfPred = OuterIf->getPredicate();
+ VPInstruction *InnerAnd =
+ cast<VPInstruction>(InnerIf->getEntryBasicBlock()->begin());
+ VPValue *InnerIfPred = InnerIf->getPredicate();
+
+ // Test block predicates
+ EXPECT_NE(nullptr, CBV1);
+ EXPECT_NE(nullptr, CBV2);
+ EXPECT_NE(nullptr, InnerAnd);
+ EXPECT_EQ(CBV1, OuterIfPred);
+ EXPECT_EQ(InnerAnd->getOpcode(), Instruction::And);
+ EXPECT_EQ(InnerAnd->getOperand(0), CBV1);
+ EXPECT_EQ(InnerAnd->getOperand(1), CBV2);
+ EXPECT_EQ(InnerIfPred, InnerAnd);
+
+ // Test Linearization
+ EXPECT_EQ(InnerLoopLinSucc, OuterIf);
+ EXPECT_EQ(OuterIfLinSucc, InnerIf);
+ EXPECT_EQ(InnerIfLinSucc, InnerLoopLatch);
+}
+
+// Test generation of Not and Or during predication.
+TEST_F(VPlanPredicatorTest, PredicatorNegOrTest) {
+ const char *ModuleString =
+ "@arr = common global [100 x [100 x i32]] zeroinitializer, align 16\n"
+ "@arr2 = common global [100 x [100 x i32]] zeroinitializer, align 16\n"
+ "@arr3 = common global [100 x [100 x i32]] zeroinitializer, align 16\n"
+ "define void @foo() {\n"
+ "entry:\n"
+ " br label %for.cond1.preheader\n"
+ "for.cond1.preheader: \n"
+ " %indvars.iv42 = phi i64 [ 0, %entry ], [ %indvars.iv.next43, "
+ "%for.inc22 ]\n"
+ " br label %for.body3\n"
+ "for.body3: \n"
+ " %indvars.iv = phi i64 [ 0, %for.cond1.preheader ], [ "
+ "%indvars.iv.next, %if.end21 ]\n"
+ " %arrayidx5 = getelementptr inbounds [100 x [100 x i32]], [100 x [100 "
+ "x i32]]* @arr, i64 0, i64 %indvars.iv, i64 %indvars.iv42\n"
+ " %0 = load i32, i32* %arrayidx5, align 4\n"
+ " %cmp6 = icmp slt i32 %0, 100\n"
+ " br i1 %cmp6, label %if.then, label %if.end21\n"
+ "if.then: \n"
+ " %cmp7 = icmp sgt i32 %0, 10\n"
+ " br i1 %cmp7, label %if.then8, label %if.else\n"
+ "if.then8: \n"
+ " %add = add nsw i32 %0, 10\n"
+ " %arrayidx12 = getelementptr inbounds [100 x [100 x i32]], [100 x [100 "
+ "x i32]]* @arr2, i64 0, i64 %indvars.iv, i64 %indvars.iv42\n"
+ " store i32 %add, i32* %arrayidx12, align 4\n"
+ " br label %if.end\n"
+ "if.else: \n"
+ " %sub = add nsw i32 %0, -10\n"
+ " %arrayidx16 = getelementptr inbounds [100 x [100 x i32]], [100 x [100 "
+ "x i32]]* @arr3, i64 0, i64 %indvars.iv, i64 %indvars.iv42\n"
+ " store i32 %sub, i32* %arrayidx16, align 4\n"
+ " br label %if.end\n"
+ "if.end: \n"
+ " store i32 222, i32* %arrayidx5, align 4\n"
+ " br label %if.end21\n"
+ "if.end21: \n"
+ " %indvars.iv.next = add nuw nsw i64 %indvars.iv, 1\n"
+ " %exitcond = icmp eq i64 %indvars.iv.next, 100\n"
+ " br i1 %exitcond, label %for.inc22, label %for.body3\n"
+ "for.inc22: \n"
+ " %indvars.iv.next43 = add nuw nsw i64 %indvars.iv42, 1\n"
+ " %exitcond44 = icmp eq i64 %indvars.iv.next43, 100\n"
+ " br i1 %exitcond44, label %for.end24, label %for.cond1.preheader\n"
+ "for.end24: \n"
+ " ret void\n"
+ "}\n";
+
+ Module &M = parseModule(ModuleString);
+ Function *F = M.getFunction("foo");
+ BasicBlock *LoopHeader = F->getEntryBlock().getSingleSuccessor();
+ auto Plan = buildHCFG(LoopHeader);
+
+ VPRegionBlock *TopRegion = cast<VPRegionBlock>(Plan->getEntry());
+ VPBlockBase *PH = TopRegion->getEntry();
+ VPBlockBase *H = PH->getSingleSuccessor();
+ VPBlockBase *OuterIfCmpBlk = H->getSingleSuccessor();
+ VPBlockBase *InnerIfCmpBlk = OuterIfCmpBlk->getSuccessors()[0];
+ VPBlockBase *InnerIfTSucc = InnerIfCmpBlk->getSuccessors()[0];
+ VPBlockBase *InnerIfFSucc = InnerIfCmpBlk->getSuccessors()[1];
+ VPBlockBase *TSuccSucc = InnerIfTSucc->getSingleSuccessor();
+ VPBlockBase *FSuccSucc = InnerIfFSucc->getSingleSuccessor();
+
+ VPValue *OuterCBV = OuterIfCmpBlk->getCondBit();
+ VPValue *InnerCBV = InnerIfCmpBlk->getCondBit();
+
+ // Apply predication.
+ VPlanPredicator VPP(*Plan);
+ VPP.predicate();
+
+ VPInstruction *And =
+ cast<VPInstruction>(InnerIfTSucc->getEntryBasicBlock()->begin());
+ VPInstruction *Not =
+ cast<VPInstruction>(InnerIfFSucc->getEntryBasicBlock()->begin());
+ VPInstruction *NotAnd = cast<VPInstruction>(
+ &*std::next(InnerIfFSucc->getEntryBasicBlock()->begin(), 1));
+ VPInstruction *Or =
+ cast<VPInstruction>(TSuccSucc->getEntryBasicBlock()->begin());
+
+ // Test block predicates
+ EXPECT_NE(nullptr, OuterCBV);
+ EXPECT_NE(nullptr, InnerCBV);
+ EXPECT_NE(nullptr, And);
+ EXPECT_NE(nullptr, Not);
+ EXPECT_NE(nullptr, NotAnd);
+
+ EXPECT_EQ(And->getOpcode(), Instruction::And);
+ EXPECT_EQ(NotAnd->getOpcode(), Instruction::And);
+ EXPECT_EQ(Not->getOpcode(), VPInstruction::Not);
+
+ EXPECT_EQ(And->getOperand(0), OuterCBV);
+ EXPECT_EQ(And->getOperand(1), InnerCBV);
+
+ EXPECT_EQ(Not->getOperand(0), InnerCBV);
+
+ EXPECT_EQ(NotAnd->getOperand(0), OuterCBV);
+ EXPECT_EQ(NotAnd->getOperand(1), Not);
+
+ EXPECT_EQ(InnerIfTSucc->getPredicate(), And);
+ EXPECT_EQ(InnerIfFSucc->getPredicate(), NotAnd);
+
+ EXPECT_EQ(TSuccSucc, FSuccSucc);
+ EXPECT_EQ(Or->getOpcode(), Instruction::Or);
+ EXPECT_EQ(TSuccSucc->getPredicate(), Or);
+
+ // Test operands of the Or - account for differences in predecessor block
+ // ordering.
+ VPInstruction *OrOp0Inst = cast<VPInstruction>(Or->getOperand(0));
+ VPInstruction *OrOp1Inst = cast<VPInstruction>(Or->getOperand(1));
+
+ bool ValidOrOperands = false;
+ if (((OrOp0Inst == And) && (OrOp1Inst == NotAnd)) ||
+ ((OrOp0Inst == NotAnd) && (OrOp1Inst == And)))
+ ValidOrOperands = true;
+
+ EXPECT_TRUE(ValidOrOperands);
+}
+
+} // namespace
+} // namespace llvm